Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
60827: sql: propagate result types to inputs during set ops with nulls and tuples r=yuzefovich a=yuzefovich

**sql: move MergeResultTypes from physicalplan package and unexport**

Release note: None

**sql: propagate result types to inputs in set ops with nulls and tuples**

The physical planning for set operations is special because it allows
for the result types from its inputs to be of different type family in
some cases. Namely, it is ok if one input has `types.Unknown` whereas
the other input is of some "known" type. In order to handle such case
the execbuilder plans casts; however, currently it is not possible to
plan casts to `Tuple` type. As a result, the vectorized engine is not
able to execute the query because it expects the data coming from both
inputs to be of the same type.

This commit modifies the logic of reconciling the result types from both
inputs to additionally update unknown types on the inputs if the other
input is of a Tuple type. Such behavior is acceptable because in the
original plan we only expected to have NULL values, and the Tuple type
is able to handle such case too.

The problem can also be reproduced in a logic test with the row-by-row
engine in the `fakedist` setting. This bug was introduced in
a76ee31, and before that change we had
the following behavior: overwrite `plan.ResultTypes` to the merged types
and then plan a stage of distinct processors if we have UNION. This
resulted in the input spec for the input stream to distinct having
correctly set types. After the change, that was no longer the case.

Before that change the vectorized engine couldn't handle such queries
due to type mismatch during `SupportsVectorized` check, so we fell back
to the row-by-row engine. However, this commit makes it so that the
vectorized engine is able to execute such queries.

Fixes: #59611.

Release note (bug fix): CockroachDB previously could encounter an
internal error when performing UNION operation when the first input
resulted only in NULL values and consequent inputs produce tuples, and
this is now fixed. Only 21.1 alpha versions are affected.

60853: sql: do not allow reference FK partial unique constraints r=mgartner a=mgartner

This commit prevents users from creating a FK with a reference column
that has a partial unique constraint. These constraints do not guarantee
uniqueness in the entire table so they cannot be used.

There is no release note because these constraints are gated behind the
experimental_enable_unique_without_index_constraints session variable.

Release note: None

60944: bulkio: Avoid using AssertionFailed error when planning replication. r=miretskiy a=miretskiy

AssertionFailed error message is not appropriate.
Return "unsupported error" instead.

Release Notes: None

60945: sql: add tests with unique constraints and virtual columns r=RaduBerinde a=RaduBerinde

Logic tests for unique indexes and constraints involving virtual
columns.

Informs #57608.

Release note: None

Co-authored-by: Yahor Yuzefovich <[email protected]>
Co-authored-by: Marcus Gartner <[email protected]>
Co-authored-by: Yevgeniy Miretskiy <[email protected]>
Co-authored-by: Radu Berinde <[email protected]>
  • Loading branch information
5 people committed Feb 23, 2021
5 parents 1043629 + 4208a25 + 0a5de20 + dd83317 + 74c2a60 commit da99e33
Show file tree
Hide file tree
Showing 12 changed files with 380 additions and 86 deletions.
1 change: 0 additions & 1 deletion pkg/ccl/streamingccl/streamproducer/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ go_library(
"//pkg/sql/sem/tree",
"//pkg/sql/types",
"//pkg/util/hlc",
"@com_github_cockroachdb_errors//:errors",
],
)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/errors"
)

// replicationStreamEval is a representation of tree.ReplicationStream, prepared
Expand Down Expand Up @@ -128,7 +127,7 @@ func doCreateReplicationStream(

if sinkURI != "" {
// TODO(yevgeniy): Support replication stream sinks.
return errors.AssertionFailedf("replication streaming into sink not supported")
return pgerror.New(pgcode.FeatureNotSupported, "replication streaming into sink not supported")
}

var scanStart hlc.Timestamp
Expand All @@ -141,7 +140,7 @@ func doCreateReplicationStream(
var spans []roachpb.Span
if eval.Targets.Tenant == (roachpb.TenantID{}) {
// TODO(yevgeniy): Only tenant streaming supported now; Support granular streaming.
return errors.AssertionFailedf("granular replication streaming not supported")
return pgerror.New(pgcode.FeatureNotSupported, "granular replication streaming not supported")
}

telemetry.Count(`replication.create.tenant`)
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,7 @@ go_library(
"distsql_plan_ctas.go",
"distsql_plan_join.go",
"distsql_plan_scrub_physical.go",
"distsql_plan_set_op.go",
"distsql_plan_stats.go",
"distsql_plan_window.go",
"distsql_running.go",
Expand Down Expand Up @@ -426,6 +427,7 @@ go_test(
"descriptor_mutation_test.go",
"distsql_physical_planner_test.go",
"distsql_plan_backfill_test.go",
"distsql_plan_set_op_test.go",
"distsql_running_test.go",
"drop_helpers_test.go",
"drop_test.go",
Expand Down
7 changes: 7 additions & 0 deletions pkg/sql/catalog/tabledesc/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -362,6 +362,13 @@ func FindFKReferencedUniqueConstraint(
uniqueWithoutIndexConstraints := referencedTable.GetUniqueWithoutIndexConstraints()
for i := range uniqueWithoutIndexConstraints {
c := &uniqueWithoutIndexConstraints[i]

// A partial unique constraint cannot be a reference constraint for a
// FK.
if c.IsPartial() {
continue
}

// TODO(rytaft): We should allow out-of-order unique constraints, as long
// as they have the same columns.
if descpb.ColumnIDs(c.ColumnIDs).Equals(referencedColIDs) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/distsql_physical_planner.go
Original file line number Diff line number Diff line change
Expand Up @@ -3268,7 +3268,7 @@ func (dsp *DistSQLPlanner) createPlanForSetOp(
p.PlanToStreamColMap = planToStreamColMap

// Merge the plans' result types and merge ordering.
resultTypes, err := physicalplan.MergeResultTypes(leftPlan.GetResultTypes(), rightPlan.GetResultTypes())
resultTypes, err := mergeResultTypesForSetOp(leftPlan, rightPlan)
if err != nil {
return nil, err
}
Expand Down
66 changes: 66 additions & 0 deletions pkg/sql/distsql_plan_set_op.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,66 @@
// Copyright 2021 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package sql

import (
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/errors"
)

// mergeResultTypesForSetOp reconciles the ResultTypes between two plans. It
// enforces that each pair of ColumnTypes must either match or be null, in which
// case the non-null type is used. This logic is necessary for cases like SELECT
// NULL UNION SELECT 1.
//
// This method is intended to be used only for planning of set operations.
func mergeResultTypesForSetOp(leftPlan, rightPlan *PhysicalPlan) ([]*types.T, error) {
left, right := leftPlan.GetResultTypes(), rightPlan.GetResultTypes()
if len(left) != len(right) {
return nil, errors.Errorf("ResultTypes length mismatch: %d and %d", len(left), len(right))
}
merged := make([]*types.T, len(left))
for i := range left {
leftType, rightType := left[i], right[i]
if rightType.Family() == types.UnknownFamily {
merged[i] = leftType
} else if leftType.Family() == types.UnknownFamily {
merged[i] = rightType
} else if leftType.Equivalent(rightType) {
// The types are equivalent for the purpose of UNION. Precision,
// Width, Oid, etc. do not affect the merging of values.
merged[i] = leftType
} else {
return nil, errors.Errorf(
"conflicting ColumnTypes: %s and %s", leftType.DebugString(), rightType.DebugString())
}
}
updateUnknownTypesForSetOp(leftPlan, merged)
updateUnknownTypesForSetOp(rightPlan, merged)
return merged, nil
}

// updateUnknownTypesForSetOp modifies plan's output types of the
// types.UnknownFamily type family to be of the corresponding Tuple type coming
// from the merged types. This is needed because at the moment the execbuilder
// is not able to plan casts to tuples.
//
// This method is intended to be used only for planning of set operations.
// TODO(yuzefovich): remove this once the execbuilder plans casts to tuples.
func updateUnknownTypesForSetOp(plan *PhysicalPlan, merged []*types.T) {
currentTypes := plan.GetResultTypes()
for i := range merged {
if merged[i].Family() == types.TupleFamily && currentTypes[i].Family() == types.UnknownFamily {
for _, procIdx := range plan.ResultRouters {
plan.Processors[procIdx].Spec.ResultTypes[i] = merged[i]
}
}
}
}
90 changes: 90 additions & 0 deletions pkg/sql/distsql_plan_set_op_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,90 @@
// Copyright 2021 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package sql

import (
"reflect"
"testing"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/physicalplan"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
)

func TestMergeResultTypesForSetOp(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

empty := []*types.T{}
null := []*types.T{types.Unknown}
typeInt := []*types.T{types.Int}
typeTuple := []*types.T{types.MakeTuple(typeInt)}

testData := []struct {
name string
left []*types.T
right []*types.T
expected *[]*types.T
err bool
}{
{"both empty", empty, empty, &empty, false},
{"left empty", empty, typeInt, nil, true},
{"right empty", typeInt, empty, nil, true},
{"both null", null, null, &null, false},
{"left null", null, typeInt, &typeInt, false},
{"right null", typeInt, null, &typeInt, false},
{"both int", typeInt, typeInt, &typeInt, false},
{"left null, right tuple", null, typeTuple, &typeTuple, false},
{"right null, left tuple", typeTuple, null, &typeTuple, false},
}
checkUnknownTypesUpdate := func(plan PhysicalPlan, orig, merged []*types.T) {
for i, typ := range plan.GetResultTypes() {
if orig[i].Family() == types.UnknownFamily {
if typ.Family() == types.UnknownFamily && merged[i].Family() == types.TupleFamily {
t.Fatal("should have updated types NULL to tuple type on the original plan")
}
if typ.Family() != types.UnknownFamily && merged[i].Family() != types.TupleFamily {
t.Fatal("should have NOT updated types NULL to tuple type on the original plan")
}
}
}
}
infra := physicalplan.MakePhysicalInfrastructure(uuid.FastMakeV4(), roachpb.NodeID(1))
var leftPlan, rightPlan PhysicalPlan
leftPlan.PhysicalInfrastructure = &infra
rightPlan.PhysicalInfrastructure = &infra
leftPlan.ResultRouters = []physicalplan.ProcessorIdx{infra.AddProcessor(physicalplan.Processor{})}
rightPlan.ResultRouters = []physicalplan.ProcessorIdx{infra.AddProcessor(physicalplan.Processor{})}
for _, td := range testData {
t.Run(td.name, func(t *testing.T) {
leftPlan.Processors[0].Spec.ResultTypes = td.left
rightPlan.Processors[1].Spec.ResultTypes = td.right
result, err := mergeResultTypesForSetOp(&leftPlan, &rightPlan)
if td.err {
if err == nil {
t.Fatalf("expected error, got %+v", result)
}
return
}
if err != nil {
t.Fatalf("unexpected error: %s", err)
}
if !reflect.DeepEqual(*td.expected, result) {
t.Fatalf("expected %+v, got %+v", *td.expected, result)
}
checkUnknownTypesUpdate(leftPlan, td.left, result)
checkUnknownTypesUpdate(rightPlan, td.right, result)
})
}
}
13 changes: 10 additions & 3 deletions pkg/sql/logictest/testdata/logic_test/fk
Original file line number Diff line number Diff line change
Expand Up @@ -3634,11 +3634,18 @@ statement ok
CREATE TABLE child2 (c INT PRIMARY KEY, p INT REFERENCES db1.public.parent(p))

# Test that foreign keys cannot reference columns that are indexed by a partial
# unique index. Partial unique indexes do not guarantee uniqueness in the entire
# table.
# unique index or a partial unique constraint. Partial unique indexes and
# constraints do not guarantee uniqueness in the entire table.

statement ok
CREATE TABLE partial_parent (p INT, UNIQUE INDEX (p) WHERE p > 100)
SET experimental_enable_unique_without_index_constraints = true

statement ok
CREATE TABLE partial_parent (
p INT,
UNIQUE INDEX (p) WHERE p > 100,
UNIQUE WITHOUT INDEX (p) WHERE p > 0
)

statement error there is no unique constraint matching given keys for referenced table partial_parent
CREATE TABLE partial_child (p INT REFERENCES partial_parent (p))
Expand Down
24 changes: 24 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/union
Original file line number Diff line number Diff line change
Expand Up @@ -366,3 +366,27 @@ SELECT b FROM ab UNION SELECT a FROM ab
----
1
2

# Regression test for the vectorized engine not being able to handle a UNION
# between NULL and a tuple (#59611).
statement ok
CREATE TABLE t59611 (a INT);
INSERT INTO t59611 VALUES (1)

query T
WITH
cte (cte_col)
AS (
SELECT
*
FROM
(VALUES ((SELECT NULL FROM t59611 LIMIT 1:::INT8)))
UNION SELECT * FROM (VALUES ((1, 2)))
)
SELECT
NULL
FROM
cte
----
NULL
NULL
Loading

0 comments on commit da99e33

Please sign in to comment.