Skip to content

Commit

Permalink
Merge pull request #42990 from yuzefovich/backport19.2-42968
Browse files Browse the repository at this point in the history
release-19.2: colexec: check for unsupported output type of a builtin
  • Loading branch information
yuzefovich authored Dec 5, 2019
2 parents 2b89a8b + 2813a3d commit 814ee59
Show file tree
Hide file tree
Showing 4 changed files with 23 additions and 8 deletions.
16 changes: 12 additions & 4 deletions pkg/sql/colexec/builtin_funcs.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/errors"
)

type defaultBuiltinFuncOperator struct {
Expand Down Expand Up @@ -180,28 +181,35 @@ func NewBuiltinFunctionOperator(
argumentCols []int,
outputIdx int,
input Operator,
) Operator {
) (Operator, error) {

switch funcExpr.ResolvedOverload().SpecializedVecBuiltin {
case tree.SubstringStringIntInt:
return &substringFunctionOperator{
OneInputNode: NewOneInputNode(input),
argumentCols: argumentCols,
outputIdx: outputIdx,
}
}, nil
default:
outputType := funcExpr.ResolvedType()
outputPhysType := typeconv.FromColumnType(outputType)
if outputPhysType == coltypes.Unhandled {
return nil, errors.Errorf(
"unsupported output type %q of %s",
outputType.String(), funcExpr.String(),
)
}
return &defaultBuiltinFuncOperator{
OneInputNode: NewOneInputNode(input),
evalCtx: evalCtx,
funcExpr: funcExpr,
outputIdx: outputIdx,
columnTypes: columnTypes,
outputType: outputType,
outputPhysType: typeconv.FromColumnType(outputType),
outputPhysType: outputPhysType,
converter: typeconv.GetDatumToPhysicalFn(outputType),
row: make(tree.Datums, len(argumentCols)),
argumentCols: argumentCols,
}
}, nil
}
}
6 changes: 4 additions & 2 deletions pkg/sql/colexec/builtin_funcs_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/stretchr/testify/require"
)

// Mock typing context for the typechecker.
Expand Down Expand Up @@ -96,7 +97,7 @@ func TestBasicBuiltinFunctions(t *testing.T) {
t.Fatal(err)
}

return NewBuiltinFunctionOperator(tctx, typedExpr.(*tree.FuncExpr), tc.outputTypes, tc.inputCols, 1, input[0]), nil
return NewBuiltinFunctionOperator(tctx, typedExpr.(*tree.FuncExpr), tc.outputTypes, tc.inputCols, 1, input[0])
})
})
}
Expand Down Expand Up @@ -147,7 +148,8 @@ func benchmarkBuiltinFunctions(b *testing.B, useSelectionVector bool, hasNulls b
if err != nil {
b.Fatal(err)
}
op := NewBuiltinFunctionOperator(tctx, typedExpr.(*tree.FuncExpr), []types.T{*types.Int}, []int{0}, 1, source)
op, err := NewBuiltinFunctionOperator(tctx, typedExpr.(*tree.FuncExpr), []types.T{*types.Int}, []int{0}, 1, source)
require.NoError(b, err)

b.SetBytes(int64(8 * coldata.BatchSize()))
b.ResetTimer()
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/colexec/execplan.go
Original file line number Diff line number Diff line change
Expand Up @@ -1167,8 +1167,8 @@ func planProjectionOperators(
funcOutputType := t.ResolvedType()
resultIdx = len(ct)
ct = append(ct, *funcOutputType)
op = NewBuiltinFunctionOperator(ctx, t, ct, inputCols, resultIdx, op)
return op, resultIdx, ct, memUsed, nil
op, err = NewBuiltinFunctionOperator(ctx, t, ct, inputCols, resultIdx, op)
return op, resultIdx, ct, memUsed, err
case tree.Datum:
datumType := t.ResolvedType()
ct = columnTypes
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/vectorize_types
Original file line number Diff line number Diff line change
Expand Up @@ -54,3 +54,8 @@ SELECT * FROM all_types ORDER BY 1
----
NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL
false 123 2019-10-22 00:00:00 +0000 +0000 1.23 123 123 123 123 1.23 123 63616665-6630-3064-6465-616462656562 2001-01-18 01:00:00.001 +0000 +0000

# This query uses a builtin that returns currently unsupported type
# (TimestampTZ). We're only interested in not getting an error. See #42871.
statement ok
SELECT experimental_strptime(_string, _string) IS NULL FROM all_types

0 comments on commit 814ee59

Please sign in to comment.