From 41fb2c95fd0de7def265054c09c8377a2c28f80c Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Fri, 1 May 2020 14:48:16 -0700 Subject: [PATCH] sql: fix tuple IS NULL logic Previously, we treated all cases of `x IS NULL` as `x IS NOT DISTINCT FROM NULL`, and all cases of `x IS NOT NULL` as `x IS DISTINCT FROM NULL`. However, these transformations are not equivalent when `x` is a tuple. If all elements of `x` are `NULL`, then `x IS NULL` should evaluate to true, but `x IS DISTINCT FROM NULL` should evaluate to false. If one element of `x` is `NULL` and one is not null, then `x IS NOT NULL` should evaluate to false, but `x IS DISTINCT FROM NULL` should evaluate to true. Therefore, they are not equivalent. Below is a table of the correct semantics for tuple expressions. | Tuple | IS NOT DISTINCT FROM NULL | IS NULL | IS DISTINCT FROM NULL | IS NOT NULL | | ------------ | ------------------------- | --------- | --------------------- | ----------- | | (1, 1) | false | false | true | true | | (1, NULL) | false | **false** | true | **false** | | (NULL, NULL) | false | true | true | false | Notice that `IS NOT DISTINCT FROM NULL` is always the inverse of `IS DISTINCT FROM NULL`. However, `IS NULL` and `IS NOT NULL` are not inverses given the tuple `(1, NULL)`. This commit introduces new tree expressions for `IS NULL` and `IS NOT NULL`. These operators have evaluation logic that is different from `IS NOT DISTINCT FROM NULL` and `IS DISTINCT FROM NULL`, respectively. While an expression such as `x IS NOT DISTINCT FROM NULL` is parsed as a `tree.ComparisonExpr` with a `tree.IsNotDisinctFrom` operator, execbuiler will output the simpler `tree.IsNullExpr` when the two expressions are equivalent - when x is not a tuple. This commit also introduces new optimizer expression types, `IsTupleNull` and `IsTupleNotNull`. Normalization rules have been added for folding these expressions into boolean values when possible. Release note (bug fix): Fixes incorrect logic for `IS NULL` and `IS NOT NULL` operators with tuples, correctly differentiating them from `IS NOT DISTINCT FROM NULL` and `IS DISTINCT FROM NULL`, respectively. --- pkg/sql/colexec/execplan.go | 102 ++++--- pkg/sql/colexec/is_null_ops_test.go | 112 ++++++-- .../testdata/logic_test/check_constraints | 16 ++ pkg/sql/opt/exec/execbuilder/scalar.go | 27 ++ pkg/sql/opt/exec/execbuilder/testdata/scalar | 44 ++++ pkg/sql/opt/memo/expr.go | 2 +- pkg/sql/opt/norm/fold_constants.go | 67 +++++ pkg/sql/opt/norm/rules/comp.opt | 28 ++ pkg/sql/opt/norm/testdata/rules/comp | 210 ++++++++++++++- pkg/sql/opt/ops/scalar.opt | 19 ++ pkg/sql/opt/optbuilder/scalar.go | 16 ++ pkg/sql/opt/optbuilder/testdata/scalar | 16 ++ pkg/sql/parser/parse_test.go | 8 +- pkg/sql/parser/sql.y | 8 +- pkg/sql/sem/tree/eval.go | 42 +++ pkg/sql/sem/tree/expr.go | 69 ++++- pkg/sql/sem/tree/normalize_test.go | 8 +- pkg/sql/sem/tree/pretty.go | 21 +- pkg/sql/sem/tree/testdata/eval/is | 248 ++++++++++++++++++ pkg/sql/sem/tree/type_check.go | 22 ++ pkg/sql/sem/tree/walk.go | 22 ++ pkg/sql/sqlbase/structured.go | 6 +- 22 files changed, 1035 insertions(+), 78 deletions(-) diff --git a/pkg/sql/colexec/execplan.go b/pkg/sql/colexec/execplan.go index 0f31fa591416..9e9c2cb63806 100644 --- a/pkg/sql/colexec/execplan.go +++ b/pkg/sql/colexec/execplan.go @@ -779,9 +779,7 @@ func NewColOperator( outputIdx := len(spec.Input[0].ColumnTypes) result.Op = NewOrdinalityOp(streamingAllocator, inputs[0], outputIdx) result.IsStreaming = true - result.ColumnTypes = make([]*types.T, outputIdx+1) - copy(result.ColumnTypes, spec.Input[0].ColumnTypes) - result.ColumnTypes[outputIdx] = types.Int + result.ColumnTypes = appendOneType(spec.Input[0].ColumnTypes, types.Int) case core.HashJoiner != nil: if err := checkNumIn(inputs, 2); err != nil { @@ -1090,10 +1088,7 @@ func NewColOperator( if err != nil { return result, err } - oldColumnTypes := result.ColumnTypes - result.ColumnTypes = make([]*types.T, len(oldColumnTypes)+1) - copy(result.ColumnTypes, oldColumnTypes) - result.ColumnTypes[len(oldColumnTypes)] = returnType + result.ColumnTypes = appendOneType(result.ColumnTypes, returnType) input = result.Op } @@ -1120,6 +1115,13 @@ func NewColOperator( ColumnTypes: result.ColumnTypes, } err = ppr.planPostProcessSpec(ctx, flowCtx, post, streamingMemAccount) + // TODO(yuzefovich): update unit tests to remove panic-catcher when fallback + // to rowexec is not allowed. + if err != nil && processorConstructor == nil { + // Do not attempt to wrap as a row source if there is no + // processorConstructor because it would fail. + return result, err + } if err != nil { log.VEventf( ctx, 2, @@ -1483,6 +1485,18 @@ func planSelectionOperators( ) op = NewBoolVecToSelOp(op, resultIdx) return op, resultIdx, typs, internalMemUsed, err + case *tree.IsNullExpr: + op, resultIdx, typs, internalMemUsed, err = planProjectionOperators( + ctx, evalCtx, t.TypedInnerExpr(), columnTypes, input, acc, + ) + op = newIsNullSelOp(op, resultIdx, false) + return op, resultIdx, typs, internalMemUsed, err + case *tree.IsNotNullExpr: + op, resultIdx, typs, internalMemUsed, err = planProjectionOperators( + ctx, evalCtx, t.TypedInnerExpr(), columnTypes, input, acc, + ) + op = newIsNullSelOp(op, resultIdx, true) + return op, resultIdx, typs, internalMemUsed, err case *tree.ComparisonExpr: cmpOp := t.Operator leftOp, leftIdx, ct, internalMemUsedLeft, err := planProjectionOperators( @@ -1514,8 +1528,10 @@ func planSelectionOperators( err = errors.Errorf("IS DISTINCT FROM and IS NOT DISTINCT FROM are supported only with NULL argument") return nil, resultIdx, ct, internalMemUsed, err } - // IS NULL is replaced with IS NOT DISTINCT FROM NULL, so we want to - // negate when IS DISTINCT FROM is used. + // IS NOT DISTINCT FROM NULL is synonymous with IS NULL and IS + // DISTINCT FROM NULL is synonymous with IS NOT NULL (except for + // tuples). Therefore, negate when the operator is IS DISTINCT + // FROM NULL. negate := t.Operator == tree.IsDistinctFrom op = newIsNullSelOp(leftOp, leftIdx, negate) return op, resultIdx, ct, internalMemUsedLeft, err @@ -1551,9 +1567,7 @@ func planTypedMaybeNullProjectionOperators( if expr == tree.DNull { resultIdx = len(columnTypes) op = NewConstNullOp(colmem.NewAllocator(ctx, acc), input, resultIdx, exprTyp) - typs = make([]*types.T, len(columnTypes)+1) - copy(typs, columnTypes) - typs[len(columnTypes)] = exprTyp + typs = appendOneType(columnTypes, exprTyp) return op, resultIdx, typs, internalMemUsed, nil } return planProjectionOperators(ctx, evalCtx, expr, columnTypes, input, acc) @@ -1592,9 +1606,7 @@ func planCastOperator( } outputIdx := len(columnTypes) op, err = GetCastOperator(colmem.NewAllocator(ctx, acc), input, inputIdx, outputIdx, fromType, toType) - typs = make([]*types.T, len(columnTypes)+1) - copy(typs, columnTypes) - typs[len(columnTypes)] = toType + typs = appendOneType(columnTypes, toType) return op, outputIdx, typs, err } @@ -1618,6 +1630,11 @@ func planProjectionOperators( return planProjectionExpr(ctx, evalCtx, t.Operator, t.ResolvedType(), t.TypedLeft(), t.TypedRight(), columnTypes, input, acc) case *tree.BinaryExpr: return planProjectionExpr(ctx, evalCtx, t.Operator, t.ResolvedType(), t.TypedLeft(), t.TypedRight(), columnTypes, input, acc) + case *tree.IsNullExpr: + t.TypedInnerExpr() + return planIsNullProjectionOp(ctx, evalCtx, t.ResolvedType(), t.TypedInnerExpr(), columnTypes, input, acc, false) + case *tree.IsNotNullExpr: + return planIsNullProjectionOp(ctx, evalCtx, t.ResolvedType(), t.TypedInnerExpr(), columnTypes, input, acc, true) case *tree.CastExpr: expr := t.Expr.(tree.TypedExpr) // If the expression is NULL, we use planTypedMaybeNullProjectionOperators instead of planProjectionOperators @@ -1656,22 +1673,16 @@ func planProjectionOperators( inputCols = append(inputCols, resultIdx) internalMemUsed += projectionInternalMem } - funcOutputType := t.ResolvedType() resultIdx = len(typs) - oldTyps := typs - typs = make([]*types.T, len(oldTyps)+1) - copy(typs, oldTyps) - typs[len(oldTyps)] = funcOutputType op, err = NewBuiltinFunctionOperator( colmem.NewAllocator(ctx, acc), evalCtx, t, typs, inputCols, resultIdx, op, ) + typs = appendOneType(typs, t.ResolvedType()) return op, resultIdx, typs, internalMemUsed, err case tree.Datum: datumType := t.ResolvedType() - typs = make([]*types.T, len(columnTypes)+1) - copy(typs, columnTypes) resultIdx = len(columnTypes) - typs[resultIdx] = datumType + typs = appendOneType(columnTypes, datumType) if datumType.Family() == types.UnknownFamily { return nil, resultIdx, typs, internalMemUsed, errors.New("cannot plan null type unknown") } @@ -1709,9 +1720,7 @@ func planProjectionOperators( "unsupported type %s", caseOutputType) } caseOutputIdx := len(columnTypes) - typs = make([]*types.T, len(columnTypes)+1) - copy(typs, columnTypes) - typs[caseOutputIdx] = caseOutputType + typs = appendOneType(columnTypes, caseOutputType) thenIdxs := make([]int, len(t.Whens)+1) for i, when := range t.Whens { // The case operator is assembled from n WHEN arms, n THEN arms, and an @@ -1966,10 +1975,7 @@ func planProjectionExpr( if sMem, ok := op.(InternalMemoryOperator); ok { internalMemUsed += sMem.InternalMemoryUsage() } - oldTyps := typs - typs = make([]*types.T, len(oldTyps)+1) - copy(typs, oldTyps) - typs[len(oldTyps)] = actualOutputType + typs = appendOneType(typs, actualOutputType) if !outputType.Identical(actualOutputType) { // The projection operator outputs a column of a different type than // the expected logical type. In order to "synchronize" the reality and @@ -2001,9 +2007,7 @@ func planLogicalProjectionOp( ) (op colexecbase.Operator, resultIdx int, typs []*types.T, internalMemUsed int, err error) { // Add a new boolean column that will store the result of the projection. resultIdx = len(columnTypes) - typs = make([]*types.T, resultIdx+1) - copy(typs, columnTypes) - typs[resultIdx] = types.Bool + typs = appendOneType(columnTypes, types.Bool) var ( typedLeft, typedRight tree.TypedExpr leftProjOpChain, rightProjOpChain, outputOp colexecbase.Operator @@ -2053,3 +2057,35 @@ func planLogicalProjectionOp( } return outputOp, resultIdx, typs, internalMemUsedLeft + internalMemUsedRight, nil } + +// planIsNullProjectionOp plans the operator for IS NULL and IS NOT NULL +// expressions (tree.IsNullExpr and tree.IsNotNullExpr, respectively). +func planIsNullProjectionOp( + ctx context.Context, + evalCtx *tree.EvalContext, + outputType *types.T, + expr tree.TypedExpr, + columnTypes []*types.T, + input colexecbase.Operator, + acc *mon.BoundAccount, + negate bool, +) (op colexecbase.Operator, resultIdx int, typs []*types.T, internalMemUsed int, err error) { + op, resultIdx, typs, internalMemUsed, err = planProjectionOperators( + ctx, evalCtx, expr, columnTypes, input, acc, + ) + outputIdx := len(typs) + op = newIsNullProjOp(colmem.NewAllocator(ctx, acc), op, resultIdx, outputIdx, negate) + typs = appendOneType(typs, outputType) + return op, outputIdx, typs, internalMemUsed, err +} + +// appendOneType appends a *types.T to then end of a []*types.T. The size of the +// underlying array of the resulting slice is 1 greater than the input slice. +// This differs from the built-in append function, which can double the capacity +// of the slice if its length is less than 1024, or increase by 25% otherwise. +func appendOneType(typs []*types.T, t *types.T) []*types.T { + newTyps := make([]*types.T, len(typs)+1) + copy(newTyps, typs) + newTyps[len(newTyps)-1] = t + return newTyps +} diff --git a/pkg/sql/colexec/is_null_ops_test.go b/pkg/sql/colexec/is_null_ops_test.go index 1ffbaf0d5f8d..925849ba8d12 100644 --- a/pkg/sql/colexec/is_null_ops_test.go +++ b/pkg/sql/colexec/is_null_ops_test.go @@ -41,56 +41,88 @@ func TestIsNullProjOp(t *testing.T) { desc string inputTuples tuples outputTuples tuples - negate bool + projExpr string }{ { desc: "SELECT c, c IS NULL FROM t -- both", inputTuples: tuples{{0}, {nil}, {1}, {2}, {nil}}, outputTuples: tuples{{0, false}, {nil, true}, {1, false}, {2, false}, {nil, true}}, - negate: false, + projExpr: "IS NULL", }, { desc: "SELECT c, c IS NULL FROM t -- no NULLs", inputTuples: tuples{{0}, {1}, {2}}, outputTuples: tuples{{0, false}, {1, false}, {2, false}}, - negate: false, + projExpr: "IS NULL", }, { desc: "SELECT c, c IS NULL FROM t -- only NULLs", inputTuples: tuples{{nil}, {nil}}, outputTuples: tuples{{nil, true}, {nil, true}}, - negate: false, + projExpr: "IS NULL", }, { desc: "SELECT c, c IS NOT NULL FROM t -- both", inputTuples: tuples{{0}, {nil}, {1}, {2}, {nil}}, outputTuples: tuples{{0, true}, {nil, false}, {1, true}, {2, true}, {nil, false}}, - negate: true, + projExpr: "IS NOT NULL", }, { desc: "SELECT c, c IS NOT NULL FROM t -- no NULLs", inputTuples: tuples{{0}, {1}, {2}}, outputTuples: tuples{{0, true}, {1, true}, {2, true}}, - negate: true, + projExpr: "IS NOT NULL", }, { desc: "SELECT c, c IS NOT NULL FROM t -- only NULLs", inputTuples: tuples{{nil}, {nil}}, outputTuples: tuples{{nil, false}, {nil, false}}, - negate: true, + projExpr: "IS NOT NULL", + }, + { + desc: "SELECT c, c IS NOT DISTINCT FROM NULL FROM t -- both", + inputTuples: tuples{{0}, {nil}, {1}, {2}, {nil}}, + outputTuples: tuples{{0, false}, {nil, true}, {1, false}, {2, false}, {nil, true}}, + projExpr: "IS NOT DISTINCT FROM NULL", + }, + { + desc: "SELECT c, c IS NOT DISTINCT FROM NULL FROM t -- no NULLs", + inputTuples: tuples{{0}, {1}, {2}}, + outputTuples: tuples{{0, false}, {1, false}, {2, false}}, + projExpr: "IS NOT DISTINCT FROM NULL", + }, + { + desc: "SELECT c, c IS NOT DISTINCT FROM NULL FROM t -- only NULLs", + inputTuples: tuples{{nil}, {nil}}, + outputTuples: tuples{{nil, true}, {nil, true}}, + projExpr: "IS NOT DISTINCT FROM NULL", + }, + { + desc: "SELECT c, c IS DISTINCT FROM NULL FROM t -- both", + inputTuples: tuples{{0}, {nil}, {1}, {2}, {nil}}, + outputTuples: tuples{{0, true}, {nil, false}, {1, true}, {2, true}, {nil, false}}, + projExpr: "IS DISTINCT FROM NULL", + }, + { + desc: "SELECT c, c IS DISTINCT FROM NULL FROM t -- no NULLs", + inputTuples: tuples{{0}, {1}, {2}}, + outputTuples: tuples{{0, true}, {1, true}, {2, true}}, + projExpr: "IS DISTINCT FROM NULL", + }, + { + desc: "SELECT c, c IS DISTINCT FROM NULL FROM t -- only NULLs", + inputTuples: tuples{{nil}, {nil}}, + outputTuples: tuples{{nil, false}, {nil, false}}, + projExpr: "IS DISTINCT FROM NULL", }, } for _, c := range testCases { t.Run(c.desc, func(t *testing.T) { opConstructor := func(input []colexecbase.Operator) (colexecbase.Operator, error) { - projExpr := "IS NULL" - if c.negate { - projExpr = "IS NOT NULL" - } return createTestProjectingOperator( ctx, flowCtx, input[0], []*types.T{types.Int}, - fmt.Sprintf("@1 %s", projExpr), false, /* canFallbackToRowexec */ + fmt.Sprintf("@1 %s", c.projExpr), false, /* canFallbackToRowexec */ ) } runTests(t, []tuples{c.inputTuples}, c.outputTuples, orderedVerifier, opConstructor) @@ -115,60 +147,92 @@ func TestIsNullSelOp(t *testing.T) { desc string inputTuples tuples outputTuples tuples - negate bool + selExpr string }{ { desc: "SELECT c FROM t WHERE c IS NULL -- both", inputTuples: tuples{{0}, {nil}, {1}, {2}, {nil}}, outputTuples: tuples{{nil}, {nil}}, - negate: false, + selExpr: "IS NULL", }, { desc: "SELECT c FROM t WHERE c IS NULL -- no NULLs", inputTuples: tuples{{0}, {1}, {2}}, outputTuples: tuples{}, - negate: false, + selExpr: "IS NULL", }, { desc: "SELECT c FROM t WHERE c IS NULL -- only NULLs", inputTuples: tuples{{nil}, {nil}}, outputTuples: tuples{{nil}, {nil}}, - negate: false, + selExpr: "IS NULL", }, { desc: "SELECT c FROM t WHERE c IS NOT NULL -- both", inputTuples: tuples{{0}, {nil}, {1}, {2}, {nil}}, outputTuples: tuples{{0}, {1}, {2}}, - negate: true, + selExpr: "IS NOT NULL", }, { desc: "SELECT c FROM t WHERE c IS NOT NULL -- no NULLs", inputTuples: tuples{{0}, {1}, {2}}, outputTuples: tuples{{0}, {1}, {2}}, - negate: true, + selExpr: "IS NOT NULL", }, { desc: "SELECT c FROM t WHERE c IS NOT NULL -- only NULLs", inputTuples: tuples{{nil}, {nil}}, outputTuples: tuples{}, - negate: true, + selExpr: "IS NOT NULL", + }, + { + desc: "SELECT c FROM t WHERE c IS NOT DISTINCT FROM NULL -- both", + inputTuples: tuples{{0}, {nil}, {1}, {2}, {nil}}, + outputTuples: tuples{{nil}, {nil}}, + selExpr: "IS NOT DISTINCT FROM NULL", + }, + { + desc: "SELECT c FROM t WHERE c IS NOT DISTINCT FROM NULL -- no NULLs", + inputTuples: tuples{{0}, {1}, {2}}, + outputTuples: tuples{}, + selExpr: "IS NOT DISTINCT FROM NULL", + }, + { + desc: "SELECT c FROM t WHERE c IS NOT DISTINCT FROM NULL -- only NULLs", + inputTuples: tuples{{nil}, {nil}}, + outputTuples: tuples{{nil}, {nil}}, + selExpr: "IS NOT DISTINCT FROM NULL", + }, + { + desc: "SELECT c FROM t WHERE c IS DISTINCT FROM NULL -- both", + inputTuples: tuples{{0}, {nil}, {1}, {2}, {nil}}, + outputTuples: tuples{{0}, {1}, {2}}, + selExpr: "IS DISTINCT FROM NULL", + }, + { + desc: "SELECT c FROM t WHERE c IS DISTINCT FROM NULL -- no NULLs", + inputTuples: tuples{{0}, {1}, {2}}, + outputTuples: tuples{{0}, {1}, {2}}, + selExpr: "IS DISTINCT FROM NULL", + }, + { + desc: "SELECT c FROM t WHERE c IS DISTINCT FROM NULL -- only NULLs", + inputTuples: tuples{{nil}, {nil}}, + outputTuples: tuples{}, + selExpr: "IS DISTINCT FROM NULL", }, } for _, c := range testCases { t.Run(c.desc, func(t *testing.T) { opConstructor := func(input []colexecbase.Operator) (colexecbase.Operator, error) { - selExpr := "IS NULL" - if c.negate { - selExpr = "IS NOT NULL" - } spec := &execinfrapb.ProcessorSpec{ Input: []execinfrapb.InputSyncSpec{{ColumnTypes: []*types.T{types.Int}}}, Core: execinfrapb.ProcessorCoreUnion{ Noop: &execinfrapb.NoopCoreSpec{}, }, Post: execinfrapb.PostProcessSpec{ - Filter: execinfrapb.Expression{Expr: fmt.Sprintf("@1 %s", selExpr)}, + Filter: execinfrapb.Expression{Expr: fmt.Sprintf("@1 %s", c.selExpr)}, }, } args := NewColOperatorArgs{ diff --git a/pkg/sql/logictest/testdata/logic_test/check_constraints b/pkg/sql/logictest/testdata/logic_test/check_constraints index 305fb771bda5..6faa5964a676 100644 --- a/pkg/sql/logictest/testdata/logic_test/check_constraints +++ b/pkg/sql/logictest/testdata/logic_test/check_constraints @@ -321,3 +321,19 @@ ALTER TABLE t36293 ELSE false END ) + +# Regression tests for #46675. +statement ok +CREATE TABLE t46675isnull (k int, a int, CHECK ((k, a) IS NULL)) + +# IS NULL is true when the operand is a tuple with all NULL values. +statement ok +INSERT INTO t46675isnull VALUES (NULL, NULL) + +statement ok +CREATE TABLE t46675isnotnull (k int, a int, CHECK ((k, a) IS NOT NULL)) + +# IS NOT NULL is false when the operand is a tuple with at least one NULL +# value. +statement error pgcode 23514 pq: failed to satisfy CHECK constraint \(\(k, a\) IS NOT NULL\) +INSERT INTO t46675isnotnull VALUES (1, NULL) diff --git a/pkg/sql/opt/exec/execbuilder/scalar.go b/pkg/sql/opt/exec/execbuilder/scalar.go index 982864eb723e..8383f0a440a4 100644 --- a/pkg/sql/opt/exec/execbuilder/scalar.go +++ b/pkg/sql/opt/exec/execbuilder/scalar.go @@ -205,6 +205,20 @@ func (b *Builder) buildBoolean(ctx *buildScalarCtx, scalar opt.ScalarExpr) (tree case opt.RangeOp: return b.buildScalar(ctx, scalar.Child(0).(opt.ScalarExpr)) + case opt.IsTupleNullOp: + expr, err := b.buildScalar(ctx, scalar.Child(0).(opt.ScalarExpr)) + if err != nil { + return nil, err + } + return tree.NewTypedIsNullExpr(expr), nil + + case opt.IsTupleNotNullOp: + expr, err := b.buildScalar(ctx, scalar.Child(0).(opt.ScalarExpr)) + if err != nil { + return nil, err + } + return tree.NewTypedIsNotNullExpr(expr), nil + default: panic(errors.AssertionFailedf("invalid op %s", log.Safe(scalar.Op()))) } @@ -221,6 +235,19 @@ func (b *Builder) buildComparison( if err != nil { return nil, err } + + // When the operator is an IsOp, the right is NULL, and the left is not a + // tuple, return the unary tree.IsNullExpr. + if scalar.Op() == opt.IsOp && right == tree.DNull && left.ResolvedType().Family() != types.TupleFamily { + return tree.NewTypedIsNullExpr(left), nil + } + + // When the operator is an IsNotOp, the right is NULL, and the left is not a + // tuple, return the unary tree.IsNotNullExpr. + if scalar.Op() == opt.IsNotOp && right == tree.DNull && left.ResolvedType().Family() != types.TupleFamily { + return tree.NewTypedIsNotNullExpr(left), nil + } + operator := opt.ComparisonOpReverseMap[scalar.Op()] return tree.NewTypedComparisonExpr(operator, left, right), nil } diff --git a/pkg/sql/opt/exec/execbuilder/testdata/scalar b/pkg/sql/opt/exec/execbuilder/testdata/scalar index 5dd8e1acabd4..76513a98d5ef 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/scalar +++ b/pkg/sql/opt/exec/execbuilder/testdata/scalar @@ -228,6 +228,28 @@ render · · (r) · · table t@primary · · · spans FULL SCAN · · +query TTTTT +EXPLAIN (VERBOSE) SELECT (a, b) IS NULL AS r FROM t +---- +· distributed false · · +· vectorized true · · +render · · (r) · + │ render 0 (a, b) IS NULL · · + └── scan · · (a, b) · +· table t@primary · · +· spans FULL SCAN · · + +query TTTTT +EXPLAIN (VERBOSE) SELECT (a, b) IS NOT DISTINCT FROM NULL AS r FROM t +---- +· distributed false · · +· vectorized true · · +render · · (r) · + │ render 0 (a, b) IS NOT DISTINCT FROM NULL · · + └── scan · · (a, b) · +· table t@primary · · +· spans FULL SCAN · · + query TTTTT EXPLAIN (VERBOSE) SELECT a IS NOT DISTINCT FROM b AS r FROM t ---- @@ -261,6 +283,28 @@ render · · (r) · · table t@primary · · · spans FULL SCAN · · +query TTTTT +EXPLAIN (VERBOSE) SELECT (a, b) IS NOT NULL AS r FROM t +---- +· distributed false · · +· vectorized true · · +render · · (r) · + │ render 0 (a, b) IS NOT NULL · · + └── scan · · (a, b) · +· table t@primary · · +· spans FULL SCAN · · + +query TTTTT +EXPLAIN (VERBOSE) SELECT (a, b) IS DISTINCT FROM NULL AS r FROM t +---- +· distributed false · · +· vectorized true · · +render · · (r) · + │ render 0 (a, b) IS DISTINCT FROM NULL · · + └── scan · · (a, b) · +· table t@primary · · +· spans FULL SCAN · · + query TTTTT EXPLAIN (VERBOSE) SELECT a IS DISTINCT FROM b AS r FROM t ---- diff --git a/pkg/sql/opt/memo/expr.go b/pkg/sql/opt/memo/expr.go index e122efb72798..5d8af467559a 100644 --- a/pkg/sql/opt/memo/expr.go +++ b/pkg/sql/opt/memo/expr.go @@ -623,7 +623,7 @@ func ExprIsNeverNull(e opt.ScalarExpr, notNullCols opt.ColSet) bool { case *VariableExpr: return notNullCols.Contains(t.Col) - case *TrueExpr, *FalseExpr, *ConstExpr, *IsExpr, *IsNotExpr: + case *TrueExpr, *FalseExpr, *ConstExpr, *IsExpr, *IsNotExpr, *IsTupleNullExpr, *IsTupleNotNullExpr: return true case *NullExpr: diff --git a/pkg/sql/opt/norm/fold_constants.go b/pkg/sql/opt/norm/fold_constants.go index 273c15f22625..82921c857411 100644 --- a/pkg/sql/opt/norm/fold_constants.go +++ b/pkg/sql/opt/norm/fold_constants.go @@ -66,6 +66,73 @@ func (c *CustomFuncs) IsConstValueOrTuple(input opt.ScalarExpr) bool { return memo.CanExtractConstDatum(input) } +// HasNullElement returns true if the input tuple has at least one constant, +// null element. Note that it only returns true if one element is known to be +// null. For example, given the tuple (1, x), it will return false because x is +// not guaranteed to be null. +func (c *CustomFuncs) HasNullElement(input opt.ScalarExpr) bool { + tup := input.(*memo.TupleExpr) + for _, e := range tup.Elems { + if e.Op() == opt.NullOp { + return true + } + } + return false +} + +// HasAllNullElements returns true if the input tuple has only constant, null +// elements. Note that it only returns true if all elements are known to be +// null. For example, given the tuple (NULL, x), it will return false because x +// is not guaranteed to be null. +func (c *CustomFuncs) HasAllNullElements(input opt.ScalarExpr) bool { + tup := input.(*memo.TupleExpr) + for _, e := range tup.Elems { + if e.Op() != opt.NullOp { + return false + } + } + return true +} + +// HasNonNullElement returns true if the input tuple has at least one constant, +// non-null element. Note that it only returns true if one element is known to +// be non-null. For example, given the tuple (NULL, x), it will return false +// because x is not guaranteed to be non-null. +func (c *CustomFuncs) HasNonNullElement(input opt.ScalarExpr) bool { + tup := input.(*memo.TupleExpr) + for _, e := range tup.Elems { + // It is guaranteed that the input has at least one non-null element if + // e is not null and it is either a constant value, array, or tuple. + // Note that it doesn't matter whether a nested tuple has non-null + // elements or not. For example, (NULL, (NULL, NULL)) IS NULL evaluates + // to false because one first-level element is not null - the second is + // a tuple. + if e.Op() != opt.NullOp && (opt.IsConstValueOp(e) || e.Op() == opt.TupleOp || e.Op() == opt.ArrayOp) { + return true + } + } + return false +} + +// HasAllNonNullElements returns true if the input tuple has all constant, +// non-null elements. Note that it only returns true if all elements are known +// to be non-null. For example, given the tuple (1, x), it will return false +// because x is not guaranteed to be non-null. +func (c *CustomFuncs) HasAllNonNullElements(input opt.ScalarExpr) bool { + tup := input.(*memo.TupleExpr) + for _, e := range tup.Elems { + // It is not guaranteed that the input has all non-null elements if e + // is null or it is neither a constant value, array, nor tuple. Note + // that it doesn't matter whether a nested tuple has non-null elements + // or not. For example, (1, (NULL, NULL)) IS NOT NULL evaluates to true + // because all first-level elements are not null. + if e.Op() == opt.NullOp || !(opt.IsConstValueOp(e) || e.Op() == opt.TupleOp || e.Op() == opt.ArrayOp) { + return false + } + } + return true +} + // FoldBinary evaluates a binary expression with constant inputs. It returns // a constant expression as long as it finds an appropriate overload function // for the given operator and input types, and the evaluation causes no error. diff --git a/pkg/sql/opt/norm/rules/comp.opt b/pkg/sql/opt/norm/rules/comp.opt index d711acc95a2d..0fd038782247 100644 --- a/pkg/sql/opt/norm/rules/comp.opt +++ b/pkg/sql/opt/norm/rules/comp.opt @@ -131,6 +131,20 @@ => (False) +# FoldNullTupleIsTupleNull replaces x IS NULL with True if x is a tuple with +# only constant, null elements. +[FoldNullTupleIsTupleNull, Normalize] +(IsTupleNull $input:(Tuple) & (HasAllNullElements $input)) +=> +(True) + +# FoldNonNullTupleIsTupleNull replaces x IS NULL with False if x is a tuple +# with at least one constant, non-null element. +[FoldNonNullTupleIsTupleNull, Normalize] +(IsTupleNull $input:(Tuple) & (HasNonNullElement $input)) +=> +(False) + # FoldIsNotNull replaces NULL IS NOT NULL with False. [FoldIsNotNull, Normalize] (IsNot (Null) (Null)) @@ -143,6 +157,20 @@ => (True) +# FoldNonNullTupleIsTupleNotNull replaces x IS NOT NULL with True if x is a +# tuple with only constant, non-null elements. +[FoldNonNullTupleIsTupleNotNull, Normalize] +(IsTupleNotNull $input:(Tuple) & (HasAllNonNullElements $input)) +=> +(True) + +# FoldNullTupleIsTupleNotNull replaces x IS NOT NULL with False if x is a tuple +# with at least one constant, null element. +[FoldNullTupleIsTupleNotNull, Normalize] +(IsTupleNotNull $input:(Tuple) & (HasNullElement $input)) +=> +(False) + # CommuteNullIs moves a NULL onto the right side of an IS/IS NOT comparison. [CommuteNullIs, Normalize] (Is | IsNot $left:(Null) $right:^(Null)) diff --git a/pkg/sql/opt/norm/testdata/rules/comp b/pkg/sql/opt/norm/testdata/rules/comp index c65eeee724ee..22c1d0f2ca5a 100644 --- a/pkg/sql/opt/norm/testdata/rules/comp +++ b/pkg/sql/opt/norm/testdata/rules/comp @@ -321,7 +321,7 @@ values └── (false,) norm expect=FoldNonNullIsNull -SELECT (1, 2, 3) IS NULL AS r +SELECT (1, 2, 3) IS NOT DISTINCT FROM NULL AS r ---- values ├── columns: r:1!null @@ -330,6 +330,114 @@ values ├── fd: ()-->(1) └── (false,) +norm expect=FoldNonNullIsNull +SELECT (1, NULL) IS NOT DISTINCT FROM NULL AS r +---- +values + ├── columns: r:1!null + ├── cardinality: [1 - 1] + ├── key: () + ├── fd: ()-->(1) + └── (false,) + +norm expect=FoldNonNullIsNull +SELECT (NULL, NULL) IS NOT DISTINCT FROM NULL AS r +---- +values + ├── columns: r:1!null + ├── cardinality: [1 - 1] + ├── key: () + ├── fd: ()-->(1) + └── (false,) + +# -------------------------------------------------- +# FoldNullTupleIsTupleNull +# -------------------------------------------------- +norm expect=FoldNullTupleIsTupleNull +SELECT (NULL, NULL) IS NULL AS r +---- +values + ├── columns: r:1!null + ├── cardinality: [1 - 1] + ├── key: () + ├── fd: ()-->(1) + └── (true,) + +norm expect-not=FoldNullTupleIsTupleNull +SELECT (k, NULL) IS NULL FROM a +---- +project + ├── columns: "?column?":7!null + ├── scan a + │ ├── columns: k:1!null + │ └── key: (1) + └── projections + └── (k:1, NULL) IS NULL [as="?column?":7, outer=(1)] + +# -------------------------------------------------- +# FoldNonNullTupleIsTupleNull +# -------------------------------------------------- +norm expect=FoldNonNullTupleIsTupleNull +SELECT (1, 2) IS NULL AS r +---- +values + ├── columns: r:1!null + ├── cardinality: [1 - 1] + ├── key: () + ├── fd: ()-->(1) + └── (false,) + +norm expect=FoldNonNullTupleIsTupleNull +SELECT (1, NULL) IS NULL AS r +---- +values + ├── columns: r:1!null + ├── cardinality: [1 - 1] + ├── key: () + ├── fd: ()-->(1) + └── (false,) + +norm expect=FoldNonNullTupleIsTupleNull +SELECT (1, k) IS NULL FROM a +---- +project + ├── columns: "?column?":7!null + ├── fd: ()-->(7) + ├── scan a + └── projections + └── false [as="?column?":7] + +norm expect=FoldNonNullTupleIsTupleNull +SELECT ((NULL, NULL), NULL) IS NULL AS r +---- +values + ├── columns: r:1!null + ├── cardinality: [1 - 1] + ├── key: () + ├── fd: ()-->(1) + └── (false,) + +norm expect=FoldNonNullTupleIsTupleNull +SELECT (ARRAY[NULL, NULL], NULL) IS NULL AS r +---- +values + ├── columns: r:1!null + ├── cardinality: [1 - 1] + ├── key: () + ├── fd: ()-->(1) + └── (false,) + +norm expect-not=FoldNonNullTupleIsTupleNull +SELECT (k, NULL) IS NULL FROM a +---- +project + ├── columns: "?column?":7!null + ├── scan a + │ ├── columns: k:1!null + │ └── key: (1) + └── projections + └── (k:1, NULL) IS NULL [as="?column?":7, outer=(1)] + # -------------------------------------------------- # FoldIsNotNull # -------------------------------------------------- @@ -365,7 +473,50 @@ project └── i:2 IS NOT NULL [as=t:9, outer=(2)] norm expect=FoldNonNullIsNotNull -SELECT (1, 2, 3) IS NOT NULL AS r +SELECT (1, 2, 3) IS DISTINCT FROM NULL AS r +---- +values + ├── columns: r:1!null + ├── cardinality: [1 - 1] + ├── key: () + ├── fd: ()-->(1) + └── (true,) + +norm expect=FoldNonNullIsNotNull +SELECT (1, NULL) IS DISTINCT FROM NULL AS r +---- +values + ├── columns: r:1!null + ├── cardinality: [1 - 1] + ├── key: () + ├── fd: ()-->(1) + └── (true,) + +norm expect=FoldNonNullIsNotNull +SELECT (1, NULL) IS DISTINCT FROM NULL AS r +---- +values + ├── columns: r:1!null + ├── cardinality: [1 - 1] + ├── key: () + ├── fd: ()-->(1) + └── (true,) + +# -------------------------------------------------- +# FoldNonNullTupleIsTupleNotNull +# -------------------------------------------------- +norm expect=FoldNonNullTupleIsTupleNotNull +SELECT (1, 1) IS NOT NULL AS r +---- +values + ├── columns: r:1!null + ├── cardinality: [1 - 1] + ├── key: () + ├── fd: ()-->(1) + └── (true,) + +norm expect=FoldNonNullTupleIsTupleNotNull +SELECT (1, (NULL, NULL)) IS NOT NULL AS r ---- values ├── columns: r:1!null @@ -374,6 +525,61 @@ values ├── fd: ()-->(1) └── (true,) +norm expect=FoldNonNullTupleIsTupleNotNull +SELECT (1, ARRAY[NULL, NULL]) IS NOT NULL AS r +---- +values + ├── columns: r:1!null + ├── cardinality: [1 - 1] + ├── key: () + ├── fd: ()-->(1) + └── (true,) + +norm expect-not=FoldNonNullTupleIsTupleNotNull +SELECT (1, k) IS NOT NULL FROM a +---- +project + ├── columns: "?column?":7!null + ├── scan a + │ ├── columns: k:1!null + │ └── key: (1) + └── projections + └── (1, k:1) IS NOT NULL [as="?column?":7, outer=(1)] + +# -------------------------------------------------- +# FoldNullTupleIsTupleNotNull +# -------------------------------------------------- +norm expect=FoldNullTupleIsTupleNotNull +SELECT (1, NULL) IS NOT NULL AS r +---- +values + ├── columns: r:1!null + ├── cardinality: [1 - 1] + ├── key: () + ├── fd: ()-->(1) + └── (false,) + +norm expect=FoldNullTupleIsTupleNotNull +SELECT (k, NULL) IS NOT NULL FROM a +---- +project + ├── columns: "?column?":7!null + ├── fd: ()-->(7) + ├── scan a + └── projections + └── false [as="?column?":7] + +norm expect-not=FoldNonNullTupleIsTupleNotNull +SELECT (1, k) IS NOT NULL FROM a +---- +project + ├── columns: "?column?":7!null + ├── scan a + │ ├── columns: k:1!null + │ └── key: (1) + └── projections + └── (1, k:1) IS NOT NULL [as="?column?":7, outer=(1)] + # -------------------------------------------------- # CommuteNullIs # -------------------------------------------------- diff --git a/pkg/sql/opt/ops/scalar.opt b/pkg/sql/opt/ops/scalar.opt index 4b4f80e2df5c..c42261e2e2f0 100644 --- a/pkg/sql/opt/ops/scalar.opt +++ b/pkg/sql/opt/ops/scalar.opt @@ -302,6 +302,20 @@ define Not { Input ScalarExpr } +# IsTupleNull is the boolean expression with a tuple input that evaluates to +# true if the tuple is null or all elements in the tuple are null. +[Scalar, Bool] +define IsTupleNull { + Input ScalarExpr +} + +# IsTupleNotNull is the boolean expression with a tuple input that evaluates to +# true if the tuple is not null and all elements in the tuple are not null. +[Scalar, Bool] +define IsTupleNotNull { + Input ScalarExpr +} + [Scalar, Bool, Comparison] define Eq { Left ScalarExpr @@ -410,12 +424,17 @@ define NotRegIMatch { Right ScalarExpr } +# Is maps to the IS NOT DISTINCT FROM operator which is equivalent to IS for +# non-tuples. See IsTupleNull for the tuple-specific IS NULL operator. [Scalar, Bool, Comparison] define Is { Left ScalarExpr Right ScalarExpr } +# IsNot is the inverse of Is. It maps to the IS DISTINCT FROM operator which is +# equivalent to IS NOT for non-tuples. See IsTupleNotNull for the +# tuple-specific IS NOT NULL operator. [Scalar, Bool, Comparison] define IsNot { Left ScalarExpr diff --git a/pkg/sql/opt/optbuilder/scalar.go b/pkg/sql/opt/optbuilder/scalar.go index 535efd9d0912..bf723eaff716 100644 --- a/pkg/sql/opt/optbuilder/scalar.go +++ b/pkg/sql/opt/optbuilder/scalar.go @@ -308,6 +308,22 @@ func (b *Builder) buildScalar( input := b.buildScalar(t.TypedInnerExpr(), inScope, nil, nil, colRefs) out = b.factory.ConstructNot(input) + case *tree.IsNullExpr: + input := b.buildScalar(t.TypedInnerExpr(), inScope, nil, nil, colRefs) + if t.TypedInnerExpr().ResolvedType().Family() == types.TupleFamily { + out = b.factory.ConstructIsTupleNull(input) + } else { + out = b.factory.ConstructIs(input, memo.NullSingleton) + } + + case *tree.IsNotNullExpr: + input := b.buildScalar(t.TypedInnerExpr(), inScope, nil, nil, colRefs) + if t.TypedInnerExpr().ResolvedType().Family() == types.TupleFamily { + out = b.factory.ConstructIsTupleNotNull(input) + } else { + out = b.factory.ConstructIsNot(input, memo.NullSingleton) + } + case *tree.NullIfExpr: // Ensure that the type of the first expression matches the resolved type // of the NULLIF expression so that type inference will be correct in the diff --git a/pkg/sql/opt/optbuilder/testdata/scalar b/pkg/sql/opt/optbuilder/testdata/scalar index 4cfb2180d03f..bc4216396637 100644 --- a/pkg/sql/opt/optbuilder/testdata/scalar +++ b/pkg/sql/opt/optbuilder/testdata/scalar @@ -179,6 +179,22 @@ is-not [type=bool] ├── variable: "@1":1 [type=int] └── variable: "@2":2 [type=int] +build-scalar vars=(int, int) +(@1, @2) IS NULL +---- +is-tuple-null [type=bool] + └── tuple [type=tuple{int, int}] + ├── variable: "@1":1 [type=int] + └── variable: "@2":2 [type=int] + +build-scalar vars=(int, int) +(@1, @2) IS NOT NULL +---- +is-tuple-not-null [type=bool] + └── tuple [type=tuple{int, int}] + ├── variable: "@1":1 [type=int] + └── variable: "@2":2 [type=int] + build-scalar vars=(int, int) + @1 + (- @2) ---- diff --git a/pkg/sql/parser/parse_test.go b/pkg/sql/parser/parse_test.go index 5cb2dd13b4ee..f143b8300149 100644 --- a/pkg/sql/parser/parse_test.go +++ b/pkg/sql/parser/parse_test.go @@ -891,6 +891,8 @@ func TestParse(t *testing.T) { {`SELECT a FROM t WHERE a NOT BETWEEN SYMMETRIC b AND c`}, {`SELECT a FROM t WHERE a IS NULL`}, {`SELECT a FROM t WHERE a IS NOT NULL`}, + {`SELECT a FROM t WHERE (a, b) IS NULL`}, + {`SELECT a FROM t WHERE (a, b) IS NOT NULL`}, {`SELECT a FROM t WHERE a IS true`}, {`SELECT a FROM t WHERE a IS NOT true`}, {`SELECT a FROM t WHERE a IS false`}, @@ -899,6 +901,8 @@ func TestParse(t *testing.T) { {`SELECT a FROM t WHERE a IS NOT OF (FLOAT8, STRING)`}, {`SELECT a FROM t WHERE a IS DISTINCT FROM b`}, {`SELECT a FROM t WHERE a IS NOT DISTINCT FROM b`}, + {`SELECT a FROM t WHERE (a, b) IS NOT DISTINCT FROM NULL`}, + {`SELECT a FROM t WHERE (a, b) IS DISTINCT FROM NULL`}, {`SELECT a FROM t WHERE a < b`}, {`SELECT a FROM t WHERE a <= b`}, {`SELECT a FROM t WHERE a >= b`}, @@ -1593,8 +1597,8 @@ func TestParse2(t *testing.T) { {`SELECT a FROM t WHERE a ISNULL`, `SELECT a FROM t WHERE a IS NULL`}, {`SELECT a FROM t WHERE a NOTNULL`, `SELECT a FROM t WHERE a IS NOT NULL`}, - {`SELECT a FROM t WHERE a IS UNKNOWN`, `SELECT a FROM t WHERE a IS NULL`}, - {`SELECT a FROM t WHERE a IS NOT UNKNOWN`, `SELECT a FROM t WHERE a IS NOT NULL`}, + {`SELECT a FROM t WHERE a IS UNKNOWN`, `SELECT a FROM t WHERE a IS NOT DISTINCT FROM NULL`}, + {`SELECT a FROM t WHERE a IS NOT UNKNOWN`, `SELECT a FROM t WHERE a IS DISTINCT FROM NULL`}, {`SELECT +1`, `SELECT 1`}, {`SELECT - - 5`, `SELECT 5`}, diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index 276229932d0e..21bc61f521bf 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -8360,19 +8360,19 @@ a_expr: } | a_expr IS NULL %prec IS { - $$.val = &tree.ComparisonExpr{Operator: tree.IsNotDistinctFrom, Left: $1.expr(), Right: tree.DNull} + $$.val = &tree.IsNullExpr{Expr: $1.expr()} } | a_expr ISNULL %prec IS { - $$.val = &tree.ComparisonExpr{Operator: tree.IsNotDistinctFrom, Left: $1.expr(), Right: tree.DNull} + $$.val = &tree.IsNullExpr{Expr: $1.expr()} } | a_expr IS NOT NULL %prec IS { - $$.val = &tree.ComparisonExpr{Operator: tree.IsDistinctFrom, Left: $1.expr(), Right: tree.DNull} + $$.val = &tree.IsNotNullExpr{Expr: $1.expr()} } | a_expr NOTNULL %prec IS { - $$.val = &tree.ComparisonExpr{Operator: tree.IsDistinctFrom, Left: $1.expr(), Right: tree.DNull} + $$.val = &tree.IsNotNullExpr{Expr: $1.expr()} } | row OVERLAPS row { return unimplemented(sqllex, "overlaps") } | a_expr IS TRUE %prec IS diff --git a/pkg/sql/sem/tree/eval.go b/pkg/sql/sem/tree/eval.go index efb140404e0f..18db886cf007 100644 --- a/pkg/sql/sem/tree/eval.go +++ b/pkg/sql/sem/tree/eval.go @@ -4293,6 +4293,48 @@ func (expr *NotExpr) Eval(ctx *EvalContext) (Datum, error) { return MakeDBool(!v), nil } +// Eval implements the TypedExpr interface. +func (expr *IsNullExpr) Eval(ctx *EvalContext) (Datum, error) { + d, err := expr.Expr.(TypedExpr).Eval(ctx) + if err != nil { + return nil, err + } + if d == DNull { + return MakeDBool(true), nil + } + if t, ok := d.(*DTuple); ok { + // A tuple IS NULL if all elements are NULL. + for _, tupleDatum := range t.D { + if tupleDatum != DNull { + return MakeDBool(false), nil + } + } + return MakeDBool(true), nil + } + return MakeDBool(false), nil +} + +// Eval implements the TypedExpr interface. +func (expr *IsNotNullExpr) Eval(ctx *EvalContext) (Datum, error) { + d, err := expr.Expr.(TypedExpr).Eval(ctx) + if err != nil { + return nil, err + } + if d == DNull { + return MakeDBool(false), nil + } + if t, ok := d.(*DTuple); ok { + // A tuple IS NOT NULL if all elements are not NULL. + for _, tupleDatum := range t.D { + if tupleDatum == DNull { + return MakeDBool(false), nil + } + } + return MakeDBool(true), nil + } + return MakeDBool(true), nil +} + // Eval implements the TypedExpr interface. func (expr *NullIfExpr) Eval(ctx *EvalContext) (Datum, error) { expr1, err := expr.Expr1.(TypedExpr).Eval(ctx) diff --git a/pkg/sql/sem/tree/expr.go b/pkg/sql/sem/tree/expr.go index 0a8106a432a4..8ca0f92c1fdc 100644 --- a/pkg/sql/sem/tree/expr.go +++ b/pkg/sql/sem/tree/expr.go @@ -90,6 +90,8 @@ type operatorExpr interface { var _ operatorExpr = &AndExpr{} var _ operatorExpr = &OrExpr{} var _ operatorExpr = &NotExpr{} +var _ operatorExpr = &IsNullExpr{} +var _ operatorExpr = &IsNotNullExpr{} var _ operatorExpr = &BinaryExpr{} var _ operatorExpr = &UnaryExpr{} var _ operatorExpr = &ComparisonExpr{} @@ -261,6 +263,64 @@ func (node *NotExpr) TypedInnerExpr() TypedExpr { return node.Expr.(TypedExpr) } +// IsNullExpr represents an IS NULL expression. This is equivalent to IS NOT +// DISTINCT FROM NULL, except when the input is a tuple. +type IsNullExpr struct { + Expr Expr + + typeAnnotation +} + +func (*IsNullExpr) operatorExpr() {} + +// Format implements the NodeFormatter interface. +func (node *IsNullExpr) Format(ctx *FmtCtx) { + exprFmtWithParen(ctx, node.Expr) + ctx.WriteString(" IS NULL") +} + +// NewTypedIsNullExpr returns a new IsNullExpr that is verified to be +// well-typed. +func NewTypedIsNullExpr(expr TypedExpr) *IsNullExpr { + node := &IsNullExpr{Expr: expr} + node.typ = types.Bool + return node +} + +// TypedInnerExpr returns the IsNullExpr's inner expression as a TypedExpr. +func (node *IsNullExpr) TypedInnerExpr() TypedExpr { + return node.Expr.(TypedExpr) +} + +// IsNotNullExpr represents an IS NOT NULL expression. This is equivalent to IS +// DISTINCT FROM NULL, except when the input is a tuple. +type IsNotNullExpr struct { + Expr Expr + + typeAnnotation +} + +func (*IsNotNullExpr) operatorExpr() {} + +// Format implements the NodeFormatter interface. +func (node *IsNotNullExpr) Format(ctx *FmtCtx) { + exprFmtWithParen(ctx, node.Expr) + ctx.WriteString(" IS NOT NULL") +} + +// NewTypedIsNotNullExpr returns a new IsNotNullExpr that is verified to be +// well-typed. +func NewTypedIsNotNullExpr(expr TypedExpr) *IsNotNullExpr { + node := &IsNotNullExpr{Expr: expr} + node.typ = types.Bool + return node +} + +// TypedInnerExpr returns the IsNotNullExpr's inner expression as a TypedExpr. +func (node *IsNotNullExpr) TypedInnerExpr() TypedExpr { + return node.Expr.(TypedExpr) +} + // ParenExpr represents a parenthesized expression. type ParenExpr struct { Expr Expr @@ -421,9 +481,12 @@ func (*ComparisonExpr) operatorExpr() {} // Format implements the NodeFormatter interface. func (node *ComparisonExpr) Format(ctx *FmtCtx) { opStr := node.Operator.String() - if node.Operator == IsDistinctFrom && (node.Right == DNull || node.Right == DBoolTrue || node.Right == DBoolFalse) { + // IS and IS NOT are equivalent to IS NOT DISTINCT FROM and IS DISTINCT + // FROM, respectively, when the RHS is true or false. We prefer the less + // verbose IS and IS NOT in those cases. + if node.Operator == IsDistinctFrom && (node.Right == DBoolTrue || node.Right == DBoolFalse) { opStr = "IS NOT" - } else if node.Operator == IsNotDistinctFrom && (node.Right == DNull || node.Right == DBoolTrue || node.Right == DBoolFalse) { + } else if node.Operator == IsNotDistinctFrom && (node.Right == DBoolTrue || node.Right == DBoolFalse) { opStr = "IS" } if node.Operator.hasSubOperator() { @@ -1774,6 +1837,8 @@ func (node *IsOfTypeExpr) String() string { return AsString(node) } func (node *Name) String() string { return AsString(node) } func (node *UnrestrictedName) String() string { return AsString(node) } func (node *NotExpr) String() string { return AsString(node) } +func (node *IsNullExpr) String() string { return AsString(node) } +func (node *IsNotNullExpr) String() string { return AsString(node) } func (node *NullIfExpr) String() string { return AsString(node) } func (node *NumVal) String() string { return AsString(node) } func (node *OrExpr) String() string { return AsString(node) } diff --git a/pkg/sql/sem/tree/normalize_test.go b/pkg/sql/sem/tree/normalize_test.go index 9cc1c7c26341..60807477bdb6 100644 --- a/pkg/sql/sem/tree/normalize_test.go +++ b/pkg/sql/sem/tree/normalize_test.go @@ -231,8 +231,8 @@ func TestNormalizeExpr(t *testing.T) { {`NULL IS DISTINCT FROM NULL`, `false`}, {`1 IS NOT DISTINCT FROM NULL`, `false`}, {`1 IS DISTINCT FROM NULL`, `true`}, - {`d IS NOT DISTINCT FROM NULL`, `d IS NULL`}, - {`d IS DISTINCT FROM NULL`, `d IS NOT NULL`}, + {`d IS NOT DISTINCT FROM NULL`, `d IS NOT DISTINCT FROM NULL`}, + {`d IS DISTINCT FROM NULL`, `d IS DISTINCT FROM NULL`}, {`NULL IS NOT DISTINCT FROM TRUE`, `false`}, {`NULL IS DISTINCT FROM TRUE`, `true`}, {`false IS NOT DISTINCT FROM TRUE`, `false`}, @@ -243,8 +243,8 @@ func TestNormalizeExpr(t *testing.T) { {`false IS DISTINCT FROM FALSE`, `false`}, {`NULL IS NOT DISTINCT FROM 1`, `false`}, {`NULL IS DISTINCT FROM 1`, `true`}, - {`NULL IS NOT DISTINCT FROM d`, `d IS NULL`}, - {`NULL IS DISTINCT FROM d`, `d IS NOT NULL`}, + {`NULL IS NOT DISTINCT FROM d`, `d IS NOT DISTINCT FROM NULL`}, + {`NULL IS DISTINCT FROM d`, `d IS DISTINCT FROM NULL`}, // #15454: ensure that operators are pretty-printed correctly after normalization. {`(random() + 1.0)::INT8`, `(random() + 1.0)::INT8`}, {`('a' || left('b', random()::INT8)) COLLATE en`, `('a' || left('b', random()::INT8)) COLLATE en`}, diff --git a/pkg/sql/sem/tree/pretty.go b/pkg/sql/sem/tree/pretty.go index 536b93c09417..4dfb674d1d10 100644 --- a/pkg/sql/sem/tree/pretty.go +++ b/pkg/sql/sem/tree/pretty.go @@ -854,9 +854,12 @@ func (p *PrettyCfg) peelCompOperand(e Expr) Expr { func (node *ComparisonExpr) doc(p *PrettyCfg) pretty.Doc { opStr := node.Operator.String() - if node.Operator == IsDistinctFrom && (node.Right == DNull || node.Right == DBoolTrue || node.Right == DBoolFalse) { + // IS and IS NOT are equivalent to IS NOT DISTINCT FROM and IS DISTINCT + // FROM, respectively, when the RHS is true or false. We prefer the less + // verbose IS and IS NOT in those cases. + if node.Operator == IsDistinctFrom && (node.Right == DBoolTrue || node.Right == DBoolFalse) { opStr = "IS NOT" - } else if node.Operator == IsNotDistinctFrom && (node.Right == DNull || node.Right == DBoolTrue || node.Right == DBoolFalse) { + } else if node.Operator == IsNotDistinctFrom && (node.Right == DBoolTrue || node.Right == DBoolFalse) { opStr = "IS" } opDoc := pretty.Keyword(opStr) @@ -2080,6 +2083,20 @@ func (node *NotExpr) doc(p *PrettyCfg) pretty.Doc { ) } +func (node *IsNullExpr) doc(p *PrettyCfg) pretty.Doc { + return pretty.ConcatSpace( + p.exprDocWithParen(node.Expr), + pretty.Keyword("IS NULL"), + ) +} + +func (node *IsNotNullExpr) doc(p *PrettyCfg) pretty.Doc { + return pretty.ConcatSpace( + p.exprDocWithParen(node.Expr), + pretty.Keyword("IS NOT NULL"), + ) +} + func (node *CoalesceExpr) doc(p *PrettyCfg) pretty.Doc { return p.bracketKeyword( node.Name, "(", diff --git a/pkg/sql/sem/tree/testdata/eval/is b/pkg/sql/sem/tree/testdata/eval/is index fc5cc962873e..93343c072250 100644 --- a/pkg/sql/sem/tree/testdata/eval/is +++ b/pkg/sql/sem/tree/testdata/eval/is @@ -90,6 +90,88 @@ NULL IS NOT FALSE ---- true +# IS (NOT) DISTINCT FROM expressions. + +eval +0 IS NOT DISTINCT FROM NULL +---- +false + +eval +0 IS DISTINCT FROM NULL +---- +true + +eval +NULL IS NOT DISTINCT FROM NULL +---- +true + +eval +NULL IS DISTINCT FROM NULL +---- +false + +eval +TRUE IS NOT DISTINCT FROM TRUE +---- +true + +eval +TRUE IS DISTINCT FROM TRUE +---- +false + +eval +FALSE IS NOT DISTINCT FROM TRUE +---- +false + +eval +FALSE IS DISTINCT FROM TRUE +---- +true + +eval +NULL IS NOT DISTINCT FROM TRUE +---- +false + +eval +NULL IS DISTINCT FROM TRUE +---- +true + +eval +TRUE IS NOT DISTINCT FROM FALSE +---- +false + +eval +TRUE IS DISTINCT FROM FALSE +---- +true + +eval +FALSE IS NOT DISTINCT FROM FALSE +---- +true + +eval +FALSE IS DISTINCT FROM FALSE +---- +false + +eval +NULL IS NOT DISTINCT FROM FALSE +---- +false + +eval +NULL IS DISTINCT FROM FALSE +---- +true + # IS OF expressions. eval @@ -276,3 +358,169 @@ eval 1 IS NOT OF (BOOL, INT) ---- false + +# Tuple with no NULLS. + +eval +(1, 1) IS NULL +---- +false + +eval +(1, 1) IS NOT NULL +---- +true + +eval +(1, (NULL, NULL)) IS NULL +---- +false + +eval +(1, (NULL, NULL)) IS NOT NULL +---- +true + +eval +(1, 1) IS NOT DISTINCT FROM NULL +---- +false + +eval +(1, 1) IS DISTINCT FROM NULL +---- +true + +eval +(1, (NULL, NULL)) IS NOT DISTINCT FROM NULL +---- +false + +eval +(1, (NULL, NULL)) IS DISTINCT FROM NULL +---- +true + +eval +NOT ((1, 1) IS NULL) +---- +true + +eval +NOT ((1, 1) IS NOT NULL) +---- +false + +eval +NOT ((1, 1) IS NOT DISTINCT FROM NULL) +---- +true + +eval +NOT ((1, 1) IS DISTINCT FROM NULL) +---- +false + +# Tuple with some NULLS. + +eval +(1, NULL) IS NULL +---- +false + +eval +(1, NULL) IS NOT NULL +---- +false + +eval +((NULL, NULL), NULL) IS NULL +---- +false + +eval +((NULL, NULL), NULL) IS NOT NULL +---- +false + +eval +(1, NULL) IS NOT DISTINCT FROM NULL +---- +false + +eval +(1, NULL) IS DISTINCT FROM NULL +---- +true + +eval +((NULL, NULL), NULL) IS NOT DISTINCT FROM NULL +---- +false + +eval +((NULL, NULL), NULL) IS DISTINcT FROM NULL +---- +true + +eval +NOT ((1, NULL) IS NULL) +---- +true + +eval +NOT ((1, NULL) IS NOT NULL) +---- +true + +eval +NOT ((1, NULL) IS NOT DISTINCT FROM NULL) +---- +true + +eval +NOT ((1, NULL) IS DISTINCT FROM NULL) +---- +false + +# Tuple with all NULLS. + +eval +(NULL, NULL) IS NULL +---- +true + +eval +(NULL, NULL) IS NOT NULL +---- +false + +eval +(NULL, NULL) IS NOT DISTINCT FROM NULL +---- +false + +eval +(NULL, NULL) IS DISTINCT FROM NULL +---- +true + +eval +NOT ((NULL, NULL) IS NULL) +---- +false + +eval +NOT ((NULL, NULL) IS NOT NULL) +---- +true + +eval +NOT ((NULL, NULL) IS NOT DISTINCT FROM NULL) +---- +true + +eval +NOT ((NULL, NULL) IS DISTINCT FROM NULL) +---- +false diff --git a/pkg/sql/sem/tree/type_check.go b/pkg/sql/sem/tree/type_check.go index 410cb4c73e66..8040f94a3f70 100644 --- a/pkg/sql/sem/tree/type_check.go +++ b/pkg/sql/sem/tree/type_check.go @@ -1095,6 +1095,28 @@ func (expr *NotExpr) TypeCheck(ctx *SemaContext, desired *types.T) (TypedExpr, e return expr, nil } +// TypeCheck implements the Expr interface. +func (expr *IsNullExpr) TypeCheck(ctx *SemaContext, desired *types.T) (TypedExpr, error) { + exprTyped, err := expr.Expr.TypeCheck(ctx, types.Any) + if err != nil { + return nil, err + } + expr.Expr = exprTyped + expr.typ = types.Bool + return expr, nil +} + +// TypeCheck implements the Expr interface. +func (expr *IsNotNullExpr) TypeCheck(ctx *SemaContext, desired *types.T) (TypedExpr, error) { + exprTyped, err := expr.Expr.TypeCheck(ctx, types.Any) + if err != nil { + return nil, err + } + expr.Expr = exprTyped + expr.typ = types.Bool + return expr, nil +} + // TypeCheck implements the Expr interface. func (expr *NullIfExpr) TypeCheck(ctx *SemaContext, desired *types.T) (TypedExpr, error) { typedSubExprs, retType, err := TypeCheckSameTypedExprs(ctx, desired, expr.Expr1, expr.Expr2) diff --git a/pkg/sql/sem/tree/walk.go b/pkg/sql/sem/tree/walk.go index c3a73cd7df6b..81235171199b 100644 --- a/pkg/sql/sem/tree/walk.go +++ b/pkg/sql/sem/tree/walk.go @@ -432,6 +432,28 @@ func (expr *NotExpr) Walk(v Visitor) Expr { return expr } +// Walk implements the Expr interface. +func (expr *IsNullExpr) Walk(v Visitor) Expr { + e, changed := WalkExpr(v, expr.Expr) + if changed { + exprCopy := *expr + exprCopy.Expr = e + return &exprCopy + } + return expr +} + +// Walk implements the Expr interface. +func (expr *IsNotNullExpr) Walk(v Visitor) Expr { + e, changed := WalkExpr(v, expr.Expr) + if changed { + exprCopy := *expr + exprCopy.Expr = e + return &exprCopy + } + return expr +} + // Walk implements the Expr interface. func (expr *NullIfExpr) Walk(v Visitor) Expr { e1, changed1 := WalkExpr(v, expr.Expr1) diff --git a/pkg/sql/sqlbase/structured.go b/pkg/sql/sqlbase/structured.go index 2cdac597ad10..906779b756f5 100644 --- a/pkg/sql/sqlbase/structured.go +++ b/pkg/sql/sqlbase/structured.go @@ -3262,10 +3262,8 @@ func MakeNotNullCheckConstraint( inuseNames[name] = struct{}{} } - expr := &tree.ComparisonExpr{ - Operator: tree.IsDistinctFrom, - Left: &tree.ColumnItem{ColumnName: tree.Name(colName)}, - Right: tree.DNull, + expr := &tree.IsNotNullExpr{ + Expr: &tree.ColumnItem{ColumnName: tree.Name(colName)}, } return &TableDescriptor_CheckConstraint{