Skip to content

Commit

Permalink
exec: Fix NULL expression handling in CASE and AND operators
Browse files Browse the repository at this point in the history
This change fixes the CASE and AND operators when nulls were used as
expressions, which caused different errors than nulls existing as data
within the batches. Additionally, this change increases the safety of
the vectorized engine by disallowing the engine to plan
constNullOperators without knowing the "type" of the null to avoid
having the same panic arise. To get around this, when the "type" of the
null is known, a constNullOperator with a known type can be planned.
This is a temporary fix for the release. To be properly fixed, datum
nulls need to be aware of their type post typechecking, instead of being
"unknown".

Release justification: This PR fixes known panics within the vectorized engine.

Fixes #40526.

Release note: None
  • Loading branch information
Rohan Yadav committed Sep 14, 2019
1 parent 6853dc0 commit d3fc0c2
Show file tree
Hide file tree
Showing 4 changed files with 89 additions and 13 deletions.
2 changes: 1 addition & 1 deletion pkg/sql/colexec/const_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
})
}
}
14 changes: 12 additions & 2 deletions pkg/sql/colexec/const_tmpl.go
Original file line number Diff line number Diff line change
Expand Up @@ -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() {
Expand All @@ -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
}
Expand Down
52 changes: 42 additions & 10 deletions pkg/sql/colflow/execplan.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down Expand Up @@ -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
Expand All @@ -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
}
Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -1051,17 +1083,17 @@ 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
}
// Transform the booleans to a selection vector.
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
}
Expand All @@ -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
}
Expand All @@ -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
}
Expand Down
34 changes: 34 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/vectorize
Original file line number Diff line number Diff line change
Expand Up @@ -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
----

0 comments on commit d3fc0c2

Please sign in to comment.