Skip to content

Commit

Permalink
Merge #73276
Browse files Browse the repository at this point in the history
73276: sql/opt: handle error when casting to anonymous tuple r=mgartner a=rafiss

v22.1 will include more support for tuple casts, so it's no longer
correct to skip this cast in the optimizer. This fixes a bug when casting to
an anonymous record type.

To fully support this, the type-checking logic for same-typed tuple
expressions had to be made a little smarter.

No release note since the bug was never released.

refs ab420f0

Release note: None

Co-authored-by: Rafi Shamim <[email protected]>
  • Loading branch information
craig[bot] and rafiss committed Dec 9, 2021
2 parents 80fba5a + 5c891be commit e89328d
Show file tree
Hide file tree
Showing 7 changed files with 125 additions and 52 deletions.
8 changes: 0 additions & 8 deletions pkg/sql/logictest/testdata/logic_test/builtin_function
Original file line number Diff line number Diff line change
Expand Up @@ -2572,14 +2572,6 @@ bool
query error pq: more than one function named 'unnest'
SELECT pg_get_function_result('unnest'::regproc);

# Regression test for #40297.
statement ok
CREATE TABLE t40297 AS SELECT g FROM generate_series(NULL, NULL) AS g

query I
SELECT COALESCE((SELECT ()), NULL) FROM t40297
----

query T
SELECT CASE WHEN true THEN (1, 2) ELSE NULL END
----
Expand Down
17 changes: 17 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/record
Original file line number Diff line number Diff line change
Expand Up @@ -180,3 +180,20 @@ SELECT s, s::a FROM strings ORDER BY 1
----
(1,2) (1,2)
(5,6) (5,6)

query T
SELECT '(1 , 2)'::a
----
(1," 2")

statement error pgcode 22P02 malformed record literal
SELECT '()'::a

statement error pgcode 0A000 cannot parse anonymous record type
SELECT s, s::record FROM strings ORDER BY 1

statement error pgcode 0A000 cannot parse anonymous record type
SELECT '()'::record

statement error pgcode 0A000 cannot parse anonymous record type
SELECT '(1,4)'::record
51 changes: 51 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/tuple
Original file line number Diff line number Diff line change
Expand Up @@ -980,6 +980,57 @@ SELECT (1::INT, NULL)
----
(1,)

# Regression test for #40297: make sure tuples and nulls type-check when
# used together in a "same-typed expression" such as function arguments.
subtest regression_40297

statement ok
CREATE TABLE t(x INT);
CREATE TABLE t40297 AS SELECT g FROM generate_series(NULL, NULL) AS g

query I
SELECT COALESCE((SELECT ()), NULL) FROM t40297
----

# Adding a cast shouldn't affect the type-checking.
query I
SELECT COALESCE((SELECT ())::record, NULL) FROM t40297
----

# Tuples must be the same length.
statement error incompatible COALESCE expressions: expected \(SELECT \(1,\)\) to be of type tuple, found type tuple{int}
SELECT COALESCE((SELECT ()), (SELECT ROW (1))) FROM t40297

# Adding a cast here should still work too.
statement error incompatible COALESCE expressions: expected \(SELECT \(1,\)\)::T to be of type tuple, found type tuple{int AS x}
SELECT COALESCE((SELECT ()), (SELECT ROW (1))::t) FROM t40297

# If a NULL is casted, then it no longer can be coerced to the right type.
statement error incompatible COALESCE expressions: expected NULL::T to be of type tuple, found type tuple{int AS x}
SELECT COALESCE((SELECT ()), NULL::t) FROM t40297

# Type-checking should still work for non-empty tuples.
query I
SELECT COALESCE((SELECT '(1)'::t), NULL::t) FROM t40297
----

# (SELECT (1)) is interpreted as an INT.
statement error incompatible COALESCE expressions: expected NULL::T to be of type int, found type tuple{int AS x}
SELECT COALESCE((SELECT (1)), NULL::t) FROM t40297

# Adding ROW allows it to be interpreted as a tuple.
query I
SELECT COALESCE((SELECT ROW (1)), NULL::t) FROM t40297
----

# An anonymous tuple type only works if it is used with a tuple of the correct length.
statement error incompatible COALESCE expressions: expected NULL::T to be of type tuple{int, int}, found type tuple{int AS x}
SELECT COALESCE((SELECT ROW (1, 2)), NULL::t) FROM t40297

# It should still type-check correctly without using a subquery.
statement error incompatible COALESCE expressions: expected NULL::T to be of type tuple{int, int}, found type tuple{int AS x}
SELECT COALESCE(ROW (1, 2), NULL::t) FROM t40297

# Regression test for #70767. Make sure we avoid encoding arrays where the
# array content type is AnyTuple.
subtest regression_70767
Expand Down
13 changes: 0 additions & 13 deletions pkg/sql/opt/exec/execbuilder/scalar.go
Original file line number Diff line number Diff line change
Expand Up @@ -112,10 +112,6 @@ func (b *Builder) buildTypedExpr(
}

func (b *Builder) buildNull(ctx *buildScalarCtx, scalar opt.ScalarExpr) (tree.TypedExpr, error) {
if scalar.DataType().Family() == types.TupleFamily && !scalar.DataType().UserDefined() {
// See comment in buildCast.
return tree.DNull, nil
}
return tree.ReType(tree.DNull, scalar.DataType()), nil
}

Expand Down Expand Up @@ -337,15 +333,6 @@ func (b *Builder) buildCast(ctx *buildScalarCtx, scalar opt.ScalarExpr) (tree.Ty
if err != nil {
return nil, err
}
if cast.Typ.Family() == types.TupleFamily && !cast.Typ.UserDefined() {
// TODO(radu): casts to Tuple are not supported (they can't be serialized
// for distsql) unless they are user-defined, in which case they have an
// OID and can be serialized with the ::@<id> syntax. This should only
// happen when the input is always NULL so the expression should still be
// valid without the cast (though there could be cornercases where the type
// does matter).
return input, nil
}
return tree.NewTypedCastExpr(input, cast.Typ), nil
}

Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/sem/tree/cast.go
Original file line number Diff line number Diff line change
Expand Up @@ -2309,6 +2309,10 @@ func performCastWithoutPrecisionTruncation(
case types.TupleFamily:
switch v := d.(type) {
case *DTuple:
if t == types.AnyTuple {
// If AnyTuple is the target type, we can just use the input tuple.
return v, nil
}
// To cast a Tuple to a Tuple, the lengths must be the same on both sides.
// Then, each element is casted to the other element type. The labels of
// the target type are kept.
Expand Down
51 changes: 35 additions & 16 deletions pkg/sql/sem/tree/type_check.go
Original file line number Diff line number Diff line change
Expand Up @@ -462,7 +462,11 @@ func resolveCast(
case toFamily == types.TupleFamily && fromFamily == types.TupleFamily:
// Casts from tuple to tuple type succeed if the lengths of the tuples are
// the same, and if there are casts resolvable across all of the elements
// pointwise.
// pointwise. Casts to AnyTuple are always allowed since they are
// implemented as a no-op.
if castTo == types.AnyTuple {
return nil
}
fromTuple := castFrom.TupleContents()
toTuple := castTo.TupleContents()
if len(fromTuple) != len(toTuple) {
Expand Down Expand Up @@ -2465,10 +2469,6 @@ func typeCheckTupleComparison(
func typeCheckSameTypedTupleExprs(
ctx context.Context, semaCtx *SemaContext, desired *types.T, exprs ...Expr,
) ([]TypedExpr, *types.T, error) {
// Hold the resolved type expressions of the provided exprs, in order.
// TODO(nvanbenschoten): Look into reducing allocations here.
typedExprs := make([]TypedExpr, len(exprs))

// All other exprs must be tuples.
first := exprs[0].(*Tuple)
if err := checkAllExprsAreTuplesOrNulls(ctx, semaCtx, exprs[1:]); err != nil {
Expand All @@ -2491,7 +2491,9 @@ func typeCheckSameTypedTupleExprs(
sameTypeExprs = sameTypeExprs[:0]
sameTypeExprsIndices = sameTypeExprsIndices[:0]
for exprIdx, expr := range exprs {
if expr == DNull {
// Skip expressions that are not Tuple expressions (e.g. NULLs or CastExpr).
// They are checked at the end of this function.
if _, isTuple := expr.(*Tuple); !isTuple {
continue
}
sameTypeExprs = append(sameTypeExprs, expr.(*Tuple).Exprs[elemIdx])
Expand All @@ -2511,11 +2513,24 @@ func typeCheckSameTypedTupleExprs(
}
resTypes.TupleContents()[elemIdx] = resType
}
// Hold the resolved type expressions of the provided exprs, in order.
// TODO(nvanbenschoten): Look into reducing allocations here.
typedExprs := make([]TypedExpr, len(exprs))
for tupleIdx, expr := range exprs {
if expr != DNull {
expr.(*Tuple).typ = resTypes
if t, isTuple := expr.(*Tuple); isTuple {
// For Tuple exprs we can update the type with what we've inferred.
t.typ = resTypes
typedExprs[tupleIdx] = t
} else {
typedExpr, err := expr.TypeCheck(ctx, semaCtx, resTypes)
if err != nil {
return nil, nil, err
}
if !typedExpr.ResolvedType().EquivalentOrNull(resTypes, true /* allowNullTupleEquivalence */) {
return nil, nil, unexpectedTypeError(expr, resTypes, typedExpr.ResolvedType())
}
typedExprs[tupleIdx] = typedExpr
}
typedExprs[tupleIdx] = expr.(TypedExpr)
}
return typedExprs, resTypes, nil
}
Expand All @@ -2527,25 +2542,29 @@ func checkAllExprsAreTuplesOrNulls(ctx context.Context, semaCtx *SemaContext, ex
_, isTuple := expr.(*Tuple)
isNull := expr == DNull
if !(isTuple || isNull) {
// We avoid calling TypeCheck on Tuple exprs since that causes the
// types to be resolved, which we only want to do later in type-checking.
typedExpr, err := expr.TypeCheck(ctx, semaCtx, types.Any)
if err != nil {
return err
}
return unexpectedTypeError(expr, types.AnyTuple, typedExpr.ResolvedType())
if typedExpr.ResolvedType().Family() != types.TupleFamily {
return unexpectedTypeError(expr, types.AnyTuple, typedExpr.ResolvedType())
}
}
}
return nil
}

// checkAllTuplesHaveLength checks that all tuples in exprs have the expected
// length. Note that all nulls are skipped in this check.
// length. We only need to check Tuple exprs, since other expressions like
// CastExpr are handled later in type-checking
func checkAllTuplesHaveLength(exprs []Expr, expectedLen int) error {
for _, expr := range exprs {
if expr == DNull {
continue
}
if err := checkTupleHasLength(expr.(*Tuple), expectedLen); err != nil {
return err
if t, isTuple := expr.(*Tuple); isTuple {
if err := checkTupleHasLength(t, expectedLen); err != nil {
return err
}
}
}
return nil
Expand Down
33 changes: 18 additions & 15 deletions pkg/sql/sem/tree/type_check_internal_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -260,7 +260,7 @@ func TestTypeCheckSameTypedExprs(t *testing.T) {
{ptypesNone, types.Decimal, exprs(intConst("1"), placeholder(0)), types.Decimal, ptypesDecimal},
{ptypesNone, types.Decimal, exprs(decConst("1.1"), placeholder(0)), types.Decimal, ptypesDecimal},
} {
t.Run(fmt.Sprintf("%d", i), func(t *testing.T) {
t.Run(fmt.Sprintf("test_%d", i), func(t *testing.T) {
attemptTypeCheckSameTypedExprs(t, i, d)
})
}
Expand Down Expand Up @@ -298,7 +298,9 @@ func TestTypeCheckSameTypedTupleExprs(t *testing.T) {
// Verify desired type when possible with unresolved constants.
{ptypesNone, ttuple(types.Int, types.Decimal), exprs(tuple(placeholder(0), intConst("1")), tuple(intConst("1"), placeholder(1))), ttuple(types.Int, types.Decimal), ptypesIntAndDecimal},
} {
attemptTypeCheckSameTypedExprs(t, i, d)
t.Run(fmt.Sprintf("test_%d", i), func(t *testing.T) {
attemptTypeCheckSameTypedExprs(t, i, d)
})
}
}

Expand Down Expand Up @@ -331,19 +333,20 @@ func TestTypeCheckSameTypedExprsError(t *testing.T) {
}
ctx := context.Background()
for i, d := range testData {
semaCtx := tree.MakeSemaContext()
if err := semaCtx.Placeholders.Init(len(d.ptypes), d.ptypes); err != nil {
t.Error(err)
continue
}
desired := types.Any
if d.desired != nil {
desired = d.desired
}
forEachPerm(d.exprs, 0, func(exprs []copyableExpr) {
if _, _, err := tree.TypeCheckSameTypedExprs(ctx, &semaCtx, desired, buildExprs(exprs)...); !testutils.IsError(err, d.expectedErr) {
t.Errorf("%d: expected %s, but found %v", i, d.expectedErr, err)
t.Run(fmt.Sprintf("test_%d", i), func(t *testing.T) {
semaCtx := tree.MakeSemaContext()
if err := semaCtx.Placeholders.Init(len(d.ptypes), d.ptypes); err != nil {
t.Error(err)
}
desired := types.Any
if d.desired != nil {
desired = d.desired
}
forEachPerm(d.exprs, 0, func(exprs []copyableExpr) {
if _, _, err := tree.TypeCheckSameTypedExprs(ctx, &semaCtx, desired, buildExprs(exprs)...); !testutils.IsError(err, d.expectedErr) {
t.Errorf("%d: expected %s, but found %v", i, d.expectedErr, err)
}
})
})
}
}
Expand Down Expand Up @@ -520,7 +523,7 @@ func TestProcessPlaceholderAnnotations(t *testing.T) {
},
}
for i, d := range testData {
t.Run(fmt.Sprintf("%d", i), func(t *testing.T) {
t.Run(fmt.Sprintf("test_%d", i), func(t *testing.T) {
args := d.initArgs
stmt := &tree.ValuesClause{Rows: []tree.Exprs{d.stmtExprs}}
if err := tree.ProcessPlaceholderAnnotations(&semaCtx, stmt, args); err != nil {
Expand Down

0 comments on commit e89328d

Please sign in to comment.