Skip to content

Commit

Permalink
sql: Ensure inserts can be performed on columns with
Browse files Browse the repository at this point in the history
default/on update expr with differing types

Release note: None
  • Loading branch information
e-mbrown committed Jun 1, 2022
1 parent 0dc29f4 commit 257f561
Show file tree
Hide file tree
Showing 17 changed files with 101 additions and 54 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/avro_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -116,7 +116,7 @@ func parseValues(tableDesc catalog.TableDescriptor, values string) ([]rowenc.Enc
for colIdx, expr := range rowTuple {
col := tableDesc.PublicColumns()[colIdx]
typedExpr, err := schemaexpr.SanitizeVarFreeExpr(
ctx, expr, col.GetType(), "avro", &semaCtx, volatility.Stable)
ctx, expr, col.GetType(), "avro", &semaCtx, volatility.Stable, false /*allowAssignmentCast*/)
if err != nil {
return nil, err
}
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/partitionccl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -105,6 +105,7 @@ func valueEncodePartitionTuple(
typedExpr, err := schemaexpr.SanitizeVarFreeExpr(evalCtx.Context, expr, cols[i].GetType(), "partition",
&semaCtx,
volatility.Immutable,
false, /*allowAssignmentCast*/
)
if err != nil {
return nil, err
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -1136,7 +1136,7 @@ func sanitizeColumnExpression(
) (tree.TypedExpr, string, error) {
colDatumType := col.GetType()
typedExpr, err := schemaexpr.SanitizeVarFreeExpr(
p.ctx, expr, colDatumType, opName, &p.p.semaCtx, volatility.Volatile,
p.ctx, expr, colDatumType, opName, &p.p.semaCtx, volatility.Volatile, false, /*allowAssignmentCast*/
)
if err != nil {
return nil, "", pgerror.WithCandidateCode(err, pgcode.DatatypeMismatch)
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/alter_table_locality.go
Original file line number Diff line number Diff line change
Expand Up @@ -373,6 +373,7 @@ func (n *alterTableSetLocalityNode) alterTableLocalityToRegionalByRow(
"REGIONAL BY ROW DEFAULT",
params.p.SemaCtx(),
volatility.Volatile,
false, /*allowAssignmentCast*/
)
if err != nil {
return err
Expand Down
3 changes: 1 addition & 2 deletions pkg/sql/analyze_expr.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,8 +50,7 @@ func (p *planner) analyzeExpr(
var err error
p.semaCtx.IVarContainer = iVarHelper.Container()
if requireType {
typedExpr, err = tree.TypeCheckAndRequire(ctx, resolved, &p.semaCtx,
expectedType, typingContext)
typedExpr, err = tree.TypeCheckAndRequire(ctx, resolved, &p.semaCtx, expectedType, typingContext, false)
} else {
typedExpr, err = tree.TypeCheck(ctx, resolved, &p.semaCtx, expectedType)
}
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/catalog/schemaexpr/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ go_library(
"//pkg/sql/parser",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
"//pkg/sql/sem/cast",
"//pkg/sql/sem/catid",
"//pkg/sql/sem/eval",
"//pkg/sql/sem/transform",
Expand Down
12 changes: 10 additions & 2 deletions pkg/sql/catalog/schemaexpr/expr.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/cast"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sem/volatility"
Expand Down Expand Up @@ -105,6 +106,7 @@ func DequalifyAndValidateExpr(
context,
semaCtx,
maxVolatility,
false, /*allowAssignmentCast*/
)

if err != nil {
Expand Down Expand Up @@ -295,7 +297,7 @@ func deserializeExprForFormatting(
// typedExpr.
if fmtFlags == tree.FmtPGCatalog {
sanitizedExpr, err := SanitizeVarFreeExpr(ctx, expr, typedExpr.ResolvedType(), "FORMAT", semaCtx,
volatility.Immutable)
volatility.Immutable, false /*allowAssignmentCast*/)
// If the expr has no variables and has Immutable, we can evaluate
// it and turn it into a constant.
if err == nil {
Expand Down Expand Up @@ -398,6 +400,7 @@ func SanitizeVarFreeExpr(
context string,
semaCtx *tree.SemaContext,
maxVolatility volatility.V,
allowAssignmentCast bool,
) (tree.TypedExpr, error) {
if tree.ContainsVars(expr) {
return nil, pgerror.Newf(pgcode.Syntax,
Expand Down Expand Up @@ -436,7 +439,12 @@ func SanitizeVarFreeExpr(
actualType := typedExpr.ResolvedType()
if !expectedType.Equivalent(actualType) && typedExpr != tree.DNull {
// The expression must match the column type exactly unless it is a constant
// NULL value.
// NULL value or assignment casts are allowed.
if allowAssignmentCast {
if ok := cast.ValidCast(actualType, expectedType, cast.ContextAssignment); ok {
return typedExpr, nil
}
}
return nil, fmt.Errorf("expected %s expression to have type %s, but '%s' has type %s",
context, expectedType, expr, actualType)
}
Expand Down
28 changes: 4 additions & 24 deletions pkg/sql/catalog/tabledesc/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/cast"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sem/volatility"
Expand Down Expand Up @@ -159,22 +158,13 @@ func MakeColumnDefDescs(
col.Type = resType

if d.HasDefaultExpr() {
var innerErr error
// Verify the default expression type is compatible with the column type
// and does not contain invalid functions.
ret.DefaultExpr, err = schemaexpr.SanitizeVarFreeExpr(
ctx, d.DefaultExpr.Expr, resType, "DEFAULT", semaCtx, volatility.Volatile,
ctx, d.DefaultExpr.Expr, resType, "DEFAULT", semaCtx, volatility.Volatile, true, /*allowAssignmentCast*/
)
if err != nil {
// Check if the default expression type can be assignment-cast into the
// column type. If it can allow the default and column type to differ.
ret.DefaultExpr, innerErr = d.DefaultExpr.Expr.TypeCheck(ctx, semaCtx, types.Any)
if innerErr != nil {
return nil, err
}
if ok := cast.ValidCast(ret.DefaultExpr.ResolvedType(), resType, cast.ContextAssignment); !ok {
return nil, err
}
return nil, err
}

// Keep the type checked expression so that the type annotation gets
Expand All @@ -190,20 +180,10 @@ func MakeColumnDefDescs(
if d.HasOnUpdateExpr() {
// Verify the on update expression type is compatible with the column type
// and does not contain invalid functions.
var innerErr error
ret.OnUpdateExpr, err = schemaexpr.SanitizeVarFreeExpr(
ctx, d.OnUpdateExpr.Expr, resType, "ON UPDATE", semaCtx, volatility.Volatile,
ctx, d.OnUpdateExpr.Expr, resType, "ON UPDATE", semaCtx, volatility.Volatile, true, /*allowAssignmentCast*/
)
if err != nil {
// Check if the on update expression type can be assignment-cast into the
// column type. If it can allow the on update expr and column type to differ.
ret.OnUpdateExpr, innerErr = d.OnUpdateExpr.Expr.TypeCheck(ctx, semaCtx, types.Any)
if innerErr != nil {
return nil, err
}
if ok := cast.ValidCast(ret.OnUpdateExpr.ResolvedType(), resType, cast.ContextAssignment); !ok {
return nil, err
}
return nil, err
}

Expand Down Expand Up @@ -286,7 +266,7 @@ func EvalShardBucketCount(
shardBuckets = paramVal
}
typedExpr, err := schemaexpr.SanitizeVarFreeExpr(
ctx, shardBuckets, types.Int, "BUCKET_COUNT", semaCtx, volatility.Volatile,
ctx, shardBuckets, types.Int, "BUCKET_COUNT", semaCtx, volatility.Volatile, false, /*allowAssignmentCast*/
)
if err != nil {
return 0, err
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/execute.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ func (p *planner) fillInPlaceholders(
}
}
typedExpr, err := schemaexpr.SanitizeVarFreeExpr(
ctx, e, typ, "EXECUTE parameter" /* context */, &semaCtx, volatility.Volatile,
ctx, e, typ, "EXECUTE parameter" /* context */, &semaCtx, volatility.Volatile, false, /*allowAssignmentCast*/
)
if err != nil {
return nil, pgerror.WithCandidateCode(err, pgcode.WrongObjectType)
Expand Down
69 changes: 56 additions & 13 deletions pkg/sql/logictest/testdata/logic_test/cast
Original file line number Diff line number Diff line change
Expand Up @@ -1418,28 +1418,71 @@ query TT
SELECT (-1/3.0)::float4::text, (1/3.0)::float8::text
----
-0.3 0.3
=======

# Test that default/on update expression differing from column type
# Test that default/on update expression is allowed to differ from column type
statement ok
CREATE TABLE def_assn_cast (
a INT4 DEFAULT 1.0::FLOAT4,
b VARCHAR DEFAULT 'true'::BOOL,
c NAME DEFAULT 'foo'::BPCHAR,
d JSONB DEFAULT 'null'::CHAR,
id INT4,
a INT4 DEFAULT 1.0::FLOAT4,
b VARCHAR DEFAULT 'true'::BOOL,
c NAME DEFAULT 'foo'::BPCHAR
)

statement error pq: could not parse . as type .: invalid . value
statement ok
CREATE TABLE upd_assn_cast (
id INT4,
a TEXT ON UPDATE 'POINT(2.0 2.0)'::GEOMETRY,
b FLOAT8 ON UPDATE 6::INT4,
c VARCHAR ON UPDATE '{ "customer": "John Doe"}'::JSONB
)

# Ensure insertions are allowed
statement ok
INSERT INTO def_assn_cast(id) VALUES (1)

statement ok
INSERT INTO upd_assn_cast(id) VALUES (1)

statement ok
UPDATE upd_assn_cast SET id = 2

query IITT
SELECT * from def_assn_cast
----
1 1 true f

query ITFT
SELECT * from upd_assn_cast
----
2 010100000000000000000000400000000000000040 6 {"customer": "John Doe"}


statement error pq: could not parse .* as type .*: invalid .* value
CREATE TABLE fail_assn_cast (
a BOOL DEFAULT 'foo'
a BOOL DEFAULT 'foo'
)

statement error pq: could not parse . as type .: invalid . value
statement error pq: expected DEFAULT expression to have type .*, but .* has type .*
CREATE TABLE fail_assn_cast (
b JSONB DEFAULT 'null'::CHAR
a DATE DEFAULT 1.0::FLOAT4
)

statement error pq: expected DEFAULT expression to have type ., but . has type .
CREATE TABLE def_assn_cast (
a INT4 DEFAULT 1.0::BOOL
statement error pq: expected DEFAULT expression to have type .*, but .* has type .*
CREATE TABLE fail_assn_cast (
b JSONB DEFAULT 'null'::CHAR
)

statement error pq: expected ON UPDATE expression to have type .*, but .* has type .*
CREATE TABLE fail_assn_cast (
a INT4 ON UPDATE 1.0::BOOL
)

statement error pq: expected ON UPDATE expression to have type .*, but .* has type .*
CREATE TABLE fail_assn_cast (
a FLOAT4 ON UPDATE '01/02/03'::DATE
)

statement error pq: expected ON UPDATE expression to have type .*, but .* has type .*
CREATE TABLE fail_assn_cast (
a NUMERIC ON UPDATE '1-2'::INTERVAL
)
1 change: 1 addition & 0 deletions pkg/sql/opt/bench/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -429,6 +429,7 @@ func newHarness(tb testing.TB, query benchQuery) *harness {
"", /* context */
&h.semaCtx,
volatility.Volatile,
false, /*allowAssignmentCast*/
)
if err != nil {
tb.Fatalf("%v", err)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/optbuilder/scope.go
Original file line number Diff line number Diff line change
Expand Up @@ -478,7 +478,7 @@ func (s *scope) resolveType(expr tree.Expr, desired *types.T) tree.TypedExpr {
// desired type.
func (s *scope) resolveAndRequireType(expr tree.Expr, desired *types.T) tree.TypedExpr {
expr = s.walkExprTree(expr)
texpr, err := tree.TypeCheckAndRequire(s.builder.ctx, expr, s.builder.semaCtx, desired, s.context.String())
texpr, err := tree.TypeCheckAndRequire(s.builder.ctx, expr, s.builder.semaCtx, desired, s.context.String(), true)
if err != nil {
panic(err)
}
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/opt/testutils/opttester/opt_tester.go
Original file line number Diff line number Diff line change
Expand Up @@ -1219,6 +1219,7 @@ func (ot *OptTester) AssignPlaceholders(
"", /* context */
&ot.semaCtx,
volatility.Volatile,
false, /*allowAssignmentCast*/
)
if err != nil {
return nil, err
Expand Down
4 changes: 1 addition & 3 deletions pkg/sql/opt/testutils/testcat/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,9 +60,7 @@ func injectTableStats(tt *Table, statsExpr tree.Expr) {
ctx := context.Background()
semaCtx := tree.MakeSemaContext()
evalCtx := eval.MakeTestingEvalContext(cluster.MakeTestingClusterSettings())
typedExpr, err := tree.TypeCheckAndRequire(
ctx, statsExpr, &semaCtx, types.Jsonb, "INJECT STATISTICS",
)
typedExpr, err := tree.TypeCheckAndRequire(ctx, statsExpr, &semaCtx, types.Jsonb, "INJECT STATISTICS", false)
if err != nil {
panic(err)
}
Expand Down
8 changes: 4 additions & 4 deletions pkg/sql/planner.go
Original file line number Diff line number Diff line change
Expand Up @@ -614,7 +614,7 @@ func (p *planner) LookupTableByID(
func (p *planner) TypeAsString(
ctx context.Context, e tree.Expr, op string,
) (func() (string, error), error) {
typedE, err := tree.TypeCheckAndRequire(ctx, e, &p.semaCtx, types.String, op)
typedE, err := tree.TypeCheckAndRequire(ctx, e, &p.semaCtx, types.String, op, false)
if err != nil {
return nil, err
}
Expand All @@ -635,7 +635,7 @@ func (p *planner) TypeAsString(
func (p *planner) TypeAsStringOrNull(
ctx context.Context, e tree.Expr, op string,
) (func() (bool, string, error), error) {
typedE, err := tree.TypeCheckAndRequire(ctx, e, &p.semaCtx, types.String, op)
typedE, err := tree.TypeCheckAndRequire(ctx, e, &p.semaCtx, types.String, op, false)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -729,7 +729,7 @@ func (p *planner) TypeAsStringOpts(
if validate == KVStringOptRequireNoValue {
return nil, errors.Errorf("option %q does not take a value", k)
}
r, err := tree.TypeCheckAndRequire(ctx, opt.Value, &p.semaCtx, types.String, k)
r, err := tree.TypeCheckAndRequire(ctx, opt.Value, &p.semaCtx, types.String, k, false)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -765,7 +765,7 @@ func (p *planner) TypeAsStringArray(
) (func() ([]string, error), error) {
typedExprs := make([]tree.TypedExpr, len(exprs))
for i := range exprs {
typedE, err := tree.TypeCheckAndRequire(ctx, exprs[i], &p.semaCtx, types.String, op)
typedE, err := tree.TypeCheckAndRequire(ctx, exprs[i], &p.semaCtx, types.String, op, false)
if err != nil {
return nil, err
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/row/expr_walker.go
Original file line number Diff line number Diff line change
Expand Up @@ -784,7 +784,7 @@ func sanitizeExprsForImport(

// If we have immutable expressions, then we can just return it right away.
typedExpr, err := schemaexpr.SanitizeVarFreeExpr(
ctx, expr, targetType, "import_default", &semaCtx, volatility.Immutable)
ctx, expr, targetType, "import_default", &semaCtx, volatility.Immutable, false /*allowAssignmentCast*/)
if err == nil {
return typedExpr, overrideImmutable, nil
}
Expand Down
16 changes: 15 additions & 1 deletion pkg/sql/sem/tree/type_check.go
Original file line number Diff line number Diff line change
Expand Up @@ -274,14 +274,28 @@ func TypeCheck(
// an identical manner to TypeCheck. It then asserts that the resulting TypedExpr
// has the provided return type, returning both the typed expression and an error
// if it does not.
// Default/OnUpdate expr are allowed to take a different type than the
// column if it can be assignment cast. If allowAssignmentCast is true
// and mutSuffix is not nil we check if the cast is valid before
// returning the error.
func TypeCheckAndRequire(
ctx context.Context, expr Expr, semaCtx *SemaContext, required *types.T, op string,
ctx context.Context,
expr Expr,
semaCtx *SemaContext,
required *types.T,
op string,
allowAssignmentCast bool,
) (TypedExpr, error) {
typedExpr, err := TypeCheck(ctx, expr, semaCtx, required)
if err != nil {
return nil, err
}
if typ := typedExpr.ResolvedType(); !(typ.Equivalent(required) || typ.Family() == types.UnknownFamily) {
if allowAssignmentCast {
if ok := cast.ValidCast(typ, required, cast.ContextAssignment); ok {
return typedExpr, nil
}
}
return typedExpr, pgerror.Newf(
pgcode.DatatypeMismatch, "argument of %s must be type %s, not type %s", op, required, typ)
}
Expand Down

0 comments on commit 257f561

Please sign in to comment.