diff --git a/pkg/sql/colexec/const_test.go b/pkg/sql/colexec/const_test.go index b62e4bfd24e5..b2a870e315ca 100644 --- a/pkg/sql/colexec/const_test.go +++ b/pkg/sql/colexec/const_test.go @@ -55,7 +55,7 @@ func TestConstNull(t *testing.T) { for _, tc := range tcs { runTestsWithTyps(t, []tuples{tc.tuples}, []coltypes.T{coltypes.Int64}, tc.expected, orderedVerifier, []int{0, 1}, func(input []Operator) (Operator, error) { - return NewConstNullOp(input[0], 1), nil + return NewConstNullOp(input[0], 1, coltypes.Int64), nil }) } } diff --git a/pkg/sql/colexec/const_tmpl.go b/pkg/sql/colexec/const_tmpl.go index 585f3c06a86c..5093819b3bb7 100644 --- a/pkg/sql/colexec/const_tmpl.go +++ b/pkg/sql/colexec/const_tmpl.go @@ -111,16 +111,24 @@ func (c const_TYPEOp) Next(ctx context.Context) coldata.Batch { // NewConstNullOp creates a new operator that produces a constant (untyped) NULL // value at index outputIdx. -func NewConstNullOp(input Operator, outputIdx int) Operator { +func NewConstNullOp(input Operator, outputIdx int, typ coltypes.T) Operator { return &constNullOp{ OneInputNode: NewOneInputNode(input), outputIdx: outputIdx, + typ: typ, } } type constNullOp struct { OneInputNode outputIdx int + typ coltypes.T +} + +var _ StaticMemoryOperator = &constNullOp{} + +func (c constNullOp) EstimateStaticMemoryUsage() int { + return EstimateBatchSizeBytes([]coltypes.T{c.typ}, coldata.BatchSize) } func (c constNullOp) Init() { @@ -130,9 +138,11 @@ func (c constNullOp) Init() { func (c constNullOp) Next(ctx context.Context) coldata.Batch { batch := c.input.Next(ctx) n := batch.Length() + if batch.Width() == c.outputIdx { - batch.AppendCol(coltypes.Int8) + batch.AppendCol(c.typ) } + if n == 0 { return batch } diff --git a/pkg/sql/colflow/execplan.go b/pkg/sql/colflow/execplan.go index eaab0bed43b6..2eb182f639ee 100644 --- a/pkg/sql/colflow/execplan.go +++ b/pkg/sql/colflow/execplan.go @@ -917,6 +917,10 @@ func planSelectionOperators( op, resultIdx, ct, memUsed, err = planProjectionOperators(ctx, expr, columnTypes, input) op = colexec.NewBoolVecToSelOp(op, resultIdx) return op, resultIdx, ct, memUsed, err + case *tree.CaseExpr: + op, resultIdx, ct, memUsed, err = planProjectionOperators(ctx, expr, columnTypes, input) + op = colexec.NewBoolVecToSelOp(op, resultIdx) + return op, resultIdx, ct, memUsed, err case *tree.ComparisonExpr: cmpOp := t.Operator leftOp, leftIdx, ct, memUsageLeft, err := planProjectionOperators(ctx, t.TypedLeft(), columnTypes, input) @@ -955,6 +959,26 @@ func planSelectionOperators( } } +// planTypedMaybeNullProjectionOperators is used to plan projection operators, but is able to +// plan constNullOperators in the case that we know the "type" of the null. It is currently +// unsafe to plan a constNullOperator when we don't know the type of the null. +func planTypedMaybeNullProjectionOperators( + ctx *tree.EvalContext, + expr tree.TypedExpr, + exprTyp *types.T, + columnTypes []types.T, + input colexec.Operator, +) (op colexec.Operator, resultIdx int, ct []types.T, memUsed int, err error) { + if expr == tree.DNull { + resultIdx = len(columnTypes) + op = colexec.NewConstNullOp(input, resultIdx, typeconv.FromColumnType(exprTyp)) + ct = append(columnTypes, *exprTyp) + memUsed = op.(colexec.StaticMemoryOperator).EstimateStaticMemoryUsage() + return op, resultIdx, ct, memUsed, nil + } + return planProjectionOperators(ctx, expr, columnTypes, input) +} + // planProjectionOperators plans a chain of operators to execute the provided // expression. It returns the tail of the chain, as well as the column index // of the expression's result (if any, otherwise -1) and the column types of the @@ -972,7 +996,15 @@ func planProjectionOperators( return planProjectionExpr(ctx, t.Operator, t.ResolvedType(), t.TypedLeft(), t.TypedRight(), columnTypes, input) case *tree.CastExpr: expr := t.Expr.(tree.TypedExpr) - op, resultIdx, ct, memUsed, err = planProjectionOperators(ctx, expr, columnTypes, input) + // If the expression is NULL, we use planTypedMaybeNullProjectionOperators instead of planProjectionOperators + // because we can say that the type of the NULL is the type that we are casting to, rather than unknown. + // We can't use planProjectionOperators because it will reject planning a constNullOp without knowing + // the post typechecking "type" of the NULL. + if expr.ResolvedType() == types.Unknown { + op, resultIdx, ct, memUsed, err = planTypedMaybeNullProjectionOperators(ctx, expr, t.Type, columnTypes, input) + } else { + op, resultIdx, ct, memUsed, err = planProjectionOperators(ctx, expr, columnTypes, input) + } if err != nil { return nil, 0, nil, 0, err } @@ -1013,7 +1045,7 @@ func planProjectionOperators( resultIdx = len(ct) ct = append(ct, *datumType) if datumType.Family() == types.UnknownFamily { - return colexec.NewConstNullOp(input, resultIdx), resultIdx, ct, memUsed, nil + return nil, resultIdx, ct, memUsed, errors.New("cannot plan null type unknown") } typ := typeconv.FromColumnType(datumType) constVal, err := typeconv.GetDatumToPhysicalFn(datumType)(t) @@ -1051,8 +1083,8 @@ func planProjectionOperators( // results of the WHEN into a single output vector, assembling the final // result of the case projection. var whenMemUsed, thenMemUsed int - caseOps[i], resultIdx, ct, whenMemUsed, err = planProjectionOperators( - ctx, when.Cond.(tree.TypedExpr), ct, buffer) + caseOps[i], resultIdx, ct, whenMemUsed, err = planTypedMaybeNullProjectionOperators( + ctx, when.Cond.(tree.TypedExpr), t.ResolvedType(), ct, buffer) if err != nil { return nil, resultIdx, ct, 0, err } @@ -1060,8 +1092,8 @@ func planProjectionOperators( caseOps[i] = colexec.NewBoolVecToSelOp(caseOps[i], resultIdx) // Run the "then" clause on those tuples that were selected. - caseOps[i], thenIdxs[i], ct, thenMemUsed, err = planProjectionOperators(ctx, when.Val.(tree.TypedExpr), ct, - caseOps[i]) + caseOps[i], thenIdxs[i], ct, thenMemUsed, err = planTypedMaybeNullProjectionOperators( + ctx, when.Val.(tree.TypedExpr), t.ResolvedType(), ct, caseOps[i]) if err != nil { return nil, resultIdx, ct, 0, err } @@ -1075,8 +1107,8 @@ func planProjectionOperators( // If there's no ELSE arm, we write NULLs. elseExpr = tree.DNull } - elseOp, thenIdxs[len(t.Whens)], ct, elseMem, err = planProjectionOperators( - ctx, elseExpr.(tree.TypedExpr), ct, buffer) + elseOp, thenIdxs[len(t.Whens)], ct, elseMem, err = planTypedMaybeNullProjectionOperators( + ctx, elseExpr.(tree.TypedExpr), t.ResolvedType(), ct, buffer) if err != nil { return nil, resultIdx, ct, 0, err } @@ -1088,11 +1120,11 @@ func planProjectionOperators( case *tree.AndExpr: var leftOp, rightOp colexec.Operator var leftIdx, rightIdx, lMemUsed, rMemUsed int - leftOp, leftIdx, ct, lMemUsed, err = planProjectionOperators(ctx, t.TypedLeft(), columnTypes, input) + leftOp, leftIdx, ct, lMemUsed, err = planTypedMaybeNullProjectionOperators(ctx, t.TypedLeft(), types.Bool, columnTypes, input) if err != nil { return nil, resultIdx, ct, 0, err } - rightOp, rightIdx, ct, rMemUsed, err = planProjectionOperators(ctx, t.TypedRight(), ct, leftOp) + rightOp, rightIdx, ct, rMemUsed, err = planTypedMaybeNullProjectionOperators(ctx, t.TypedRight(), types.Bool, ct, leftOp) if err != nil { return nil, resultIdx, ct, 0, err } diff --git a/pkg/sql/logictest/testdata/logic_test/vectorize b/pkg/sql/logictest/testdata/logic_test/vectorize index 32141a2b5aec..26c58b67e2ce 100644 --- a/pkg/sql/logictest/testdata/logic_test/vectorize +++ b/pkg/sql/logictest/testdata/logic_test/vectorize @@ -862,3 +862,37 @@ SELECT CASE WHEN a <= b THEN 1 ELSE 2 END FROM t1 WHERE (a > b - 2 AND a < b + 2) OR (c > a AND c < b) ---- + +# Regression tests for NULL expression handling. +statement ok +CREATE TABLE t_case_null (x INT) + +statement ok +INSERT INTO t_case_null VALUES (0) + +query I +SELECT CASE WHEN x = 0 THEN 0 ELSE NULL END FROM t_case_null +---- +0 + +query I +SELECT CASE WHEN x = 0 THEN NULL ELSE 0 END FROM t_case_null +---- +NULL + +query I +SELECT CASE WHEN x = 1 THEN 1 ELSE NULL END FROM t_case_null +---- +NULL + +query I +SELECT * FROM t_case_null WHERE NULL AND NULL +---- + +query I +SELECT * FROM t_case_null WHERE NULL AND x = 0 +---- + +query I +SELECT * FROM t_case_null WHERE x = 0 AND NULL +----