diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md index fd56756a8447..6621189e85e9 100644 --- a/docs/generated/sql/functions.md +++ b/docs/generated/sql/functions.md @@ -2829,6 +2829,8 @@ may increase either contention or retry errors, or both.

crdb_internal.approximate_timestamp(timestamp: decimal) → timestamp

Converts the crdb_internal_mvcc_timestamp column into an approximate timestamp.

+crdb_internal.assignment_cast(val: anyelement, type: anyelement) → anyelement

This function is used internally to perform assignment casts during mutations.

+
crdb_internal.check_consistency(stats_only: bool, start_key: bytes, end_key: bytes) → tuple{int AS range_id, bytes AS start_key, string AS start_key_pretty, string AS status, string AS detail}

Runs a consistency check on ranges touching the specified key range. an empty start or end key is treated as the minimum and maximum possible, respectively. stats_only should only be set to false when targeting a small number of ranges to avoid overloading the cluster. Each returned row contains the range ID, the status (a roachpb.CheckConsistencyResponse_Status), and verbose detail.

Example usage: SELECT * FROM crdb_internal.check_consistency(true, ‘\x02’, ‘\x04’)

diff --git a/pkg/sql/colexec/builtin_funcs.go b/pkg/sql/colexec/builtin_funcs.go index 35458d45dfbc..a4818ea6867f 100644 --- a/pkg/sql/colexec/builtin_funcs.go +++ b/pkg/sql/colexec/builtin_funcs.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) type defaultBuiltinFuncOperator struct { @@ -118,7 +119,11 @@ func NewBuiltinFunctionOperator( outputIdx int, input colexecop.Operator, ) (colexecop.Operator, error) { - switch funcExpr.ResolvedOverload().SpecializedVecBuiltin { + overload := funcExpr.ResolvedOverload() + if overload.FnWithExprs != nil { + return nil, errors.New("builtins with FnWithExprs are not supported in the vectorized engine") + } + switch overload.SpecializedVecBuiltin { case tree.SubstringStringIntInt: input = colexecutils.NewVectorTypeEnforcer(allocator, input, types.String, outputIdx) return newSubstringOperator( diff --git a/pkg/sql/insert.go b/pkg/sql/insert.go index f4c1ce39ad30..670e8c454b6a 100644 --- a/pkg/sql/insert.go +++ b/pkg/sql/insert.go @@ -123,7 +123,7 @@ func (r *insertRun) initRowContainer(params runParams, columns colinfo.ResultCol // processSourceRow processes one row from the source for insertion and, if // result rows are needed, saves it in the result row container. func (r *insertRun) processSourceRow(params runParams, rowVals tree.Datums) error { - if err := enforceLocalColumnConstraints(rowVals, r.insertCols); err != nil { + if err := enforceLocalColumnConstraints(rowVals, r.insertCols, false /* isUpdate */); err != nil { return err } diff --git a/pkg/sql/logictest/testdata/logic_test/alter_column_type b/pkg/sql/logictest/testdata/logic_test/alter_column_type index 44d0ae7c1173..cfad19db8249 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_column_type +++ b/pkg/sql/logictest/testdata/logic_test/alter_column_type @@ -200,9 +200,6 @@ rowid INT8 false unique_rowid() · {primary} true statement ok INSERT INTO t2 VALUES ('5') -statement error pq: value type int doesn't match type string of column "id" -INSERT INTO t2 VALUES (6) - # Verify ALTER COLUMN TYPE from INT to STRING works correctly. # Column order should stay the same. statement ok diff --git a/pkg/sql/logictest/testdata/logic_test/array b/pkg/sql/logictest/testdata/logic_test/array index b6cbe109cac7..0e2a3a948834 100644 --- a/pkg/sql/logictest/testdata/logic_test/array +++ b/pkg/sql/logictest/testdata/logic_test/array @@ -810,7 +810,7 @@ SELECT * FROM a {hello} {goodbye} -statement error value type collatedstring{fr}\[\] doesn't match type collatedstring{en}\[\] of column "b" +statement ok INSERT INTO a VALUES (ARRAY['hello' COLLATE fr]) statement ok @@ -841,7 +841,7 @@ CREATE TABLE a (b STRING[]) statement ok INSERT INTO a VALUES (ARRAY['foo']) -statement error value type collatedstring{en}\[\] doesn't match type string\[\] of column "b" +statement ok INSERT INTO a VALUES (ARRAY['foo' COLLATE en]) statement ok diff --git a/pkg/sql/logictest/testdata/logic_test/cast b/pkg/sql/logictest/testdata/logic_test/cast index b3724a73a510..c9f2e9e00d8a 100644 --- a/pkg/sql/logictest/testdata/logic_test/cast +++ b/pkg/sql/logictest/testdata/logic_test/cast @@ -1,3 +1,183 @@ +# Tests for assignment casts. +subtest assignment_casts + +statement ok +CREATE TABLE assn_cast ( + c CHAR, + vc VARCHAR(1), + qc "char", + b BIT, + i INT, + t timestamp, + d DECIMAL(10, 0), + s STRING +) + +statement ok +INSERT INTO assn_cast(c) VALUES ('a') + +statement ok +INSERT INTO assn_cast(c) VALUES (null), ('b') + +statement error value too long for type CHAR +INSERT INTO assn_cast(c) VALUES ('abc') + +query T +INSERT INTO assn_cast(c) VALUES (1) RETURNING c +---- +1 + +statement error value too long for type CHAR +INSERT INTO assn_cast(c) VALUES (123) + +statement ok +PREPARE insert_c AS INSERT INTO assn_cast(c) VALUES ($1) + +statement error value too long for type CHAR +EXECUTE insert_c('foo') + +statement error value too long for type CHAR +EXECUTE insert_c('foo'::STRING) + +statement ok +DELETE FROM assn_cast + +statement +EXECUTE insert_c(' ') + +statement +EXECUTE insert_c(' '::STRING) + +query T +SELECT concat('"', c, '"') FROM assn_cast +---- +"" +"" + +statement ok +INSERT INTO assn_cast(vc) VALUES ('a') + +statement ok +INSERT INTO assn_cast(vc) VALUES (null), ('b') + +statement error value too long for type VARCHAR\(1\) +INSERT INTO assn_cast(vc) VALUES ('abc') + +query T +INSERT INTO assn_cast(vc) VALUES (1) RETURNING vc +---- +1 + +statement error value too long for type VARCHAR\(1\) +INSERT INTO assn_cast(vc) VALUES (123) + +statement ok +INSERT INTO assn_cast(qc) VALUES ('a') + +statement ok +INSERT INTO assn_cast(qc) VALUES (null), ('b') + +query T +INSERT INTO assn_cast(qc) VALUES ('abc') RETURNING qc +---- +a + +# Note: This statement fails in Postgres because the default integer type is an +# INT4, and the INT4 -> "char" cast is explicit. Our default integer type +# is an INT8 and INT8 -> "char" is an assignment cast. +query T +INSERT INTO assn_cast(qc) VALUES (123) RETURNING qc +---- +{ + +statement error \"char\" out of range +INSERT INTO assn_cast(qc) VALUES (1234) + +statement ok +PREPARE insert_qc AS INSERT INTO assn_cast(qc) VALUES ($1) + +statement ok +DELETE FROM assn_cast + +statement ok +EXECUTE insert_qc('foo') + +statement ok +EXECUTE insert_qc('foo'::STRING) + +query T +SELECT qc FROM assn_cast +---- +f +f + +statement ok +INSERT into assn_cast(b) VALUES ('1') + +statement ok +INSERT INTO assn_cast(b) VALUES (null), ('1') + +# TODO(mgartner): To match Postgres behavior, this statement should fail with +# the message "value too long for type BIT". +statement ok +INSERT into assn_cast(b) VALUES ('01') + +statement error value type int doesn't match type bit of column \"b\" +INSERT into assn_cast(b) VALUES (1) + +statement ok +INSERT INTO assn_cast(i) VALUES ('1') + +statement ok +INSERT INTO assn_cast(i) VALUES (null), ('1') + +statement ok +PREPARE insert_i AS INSERT INTO assn_cast(i) VALUES ($1) + +statement ok +EXECUTE insert_i('1') + +statement error value type string doesn't match type int of column \"i\" +INSERT INTO assn_cast(i) VALUES ('1'::STRING) + +statement ok +INSERT INTO assn_cast(t) VALUES ('1970-01-01'::timestamptz) + +statement ok +INSERT INTO assn_cast(d) VALUES (11.22), (88.99) + +statement ok +PREPARE insert_d AS INSERT INTO assn_cast(d) VALUES ($1) + +statement ok +EXECUTE insert_d(123.45) + +statement ok +PREPARE insert_d2 AS INSERT INTO assn_cast(d) SELECT * FROM (VALUES ($1::DECIMAL(10, 2))) + +statement ok +EXECUTE insert_d2(67.89) + +query F rowsort +SELECT d FROM assn_cast WHERE d IS NOT NULL +---- +11 +89 +123 +68 + +statement ok +INSERT INTO assn_cast(s) VALUES (1) + +statement ok +PREPARE insert_s AS INSERT INTO assn_cast(s) VALUES ($1) + +# TODO(mgartner): This should succeed to match the behavior of Postgres. +statement error expected EXECUTE parameter expression to have type string, but \'1\' has type int +EXECUTE insert_s(1) + +subtest regressions + statement ok CREATE TABLE t45837 AS SELECT 1.25::decimal AS d diff --git a/pkg/sql/logictest/testdata/logic_test/collatedstring b/pkg/sql/logictest/testdata/logic_test/collatedstring index 2d7c5c94c7b9..79aa830dba7b 100644 --- a/pkg/sql/logictest/testdata/logic_test/collatedstring +++ b/pkg/sql/logictest/testdata/logic_test/collatedstring @@ -231,8 +231,10 @@ INSERT INTO t VALUES ('x' COLLATE en), ('ü' COLLATE en) -statement error value type collatedstring{de} doesn't match type collatedstring{en} of column "a" -INSERT INTO t VALUES ('X' COLLATE de) +statement ok +INSERT INTO t VALUES + ('X' COLLATE de), + ('y' COLLATE de) query T SELECT a FROM t ORDER BY t.a @@ -243,6 +245,8 @@ b B ü x +X +y query T SELECT a FROM t ORDER BY t.a COLLATE da @@ -252,6 +256,8 @@ A b B x +X +y ü query T diff --git a/pkg/sql/logictest/testdata/logic_test/computed b/pkg/sql/logictest/testdata/logic_test/computed index 3162c9edeb70..8df3f2f6a214 100644 --- a/pkg/sql/logictest/testdata/logic_test/computed +++ b/pkg/sql/logictest/testdata/logic_test/computed @@ -883,9 +883,17 @@ CREATE TABLE x ( b INT AS (a+1) STORED ) -query error value type decimal doesn't match type int of column "a" +statement ok INSERT INTO x VALUES(1.4) +query II +SELECT * FROM x +---- +1 2 + +query error value type date doesn't match type int of column "a" +INSERT INTO x VALUES('1970-01-01'::date) + # Regression test for #34901: verify that builtins can be used in computed # column expressions without a "memory budget exceeded" error while backfilling statement ok @@ -1025,3 +1033,33 @@ SET experimental_computed_column_rewrites = "bad" statement error invalid column rewrites expression SET CLUSTER SETTING sql.defaults.experimental_computed_column_rewrites = "bad" + +# Regression test for #69327. Computed columns should be evaluated after +# assignment casts have been performed. +statement ok +CREATE TABLE t69327 ( + c "char", + v STRING AS (c) STORED +); +INSERT INTO t69327 VALUES ('foo'::STRING) + +# Both columns should have a value of "f". +query TT +SELECT * FROM t69327 +---- +f f + +# Regression test for #69665.Computed columns should be evaluated after +# assignment casts have been performed. +statement ok +CREATE TABLE t69665 ( + c CHAR, + v STRING AS (c) STORED +); +INSERT INTO t69665 VALUES (' '::STRING) + +# Both columns should be empty values. +query II +SELECT length(c), length(v) FROM t69665 +---- +0 0 diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 829f32690fc4..764afd00e561 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -2367,7 +2367,7 @@ char_len dc 1 4 char_len ec 12 48 char_len dv NULL NULL char_len ev 12 48 -char_len dq NULL NULL +char_len dq 1 4 char_len f NULL NULL char_len g 1 NULL char_len h 12 NULL diff --git a/pkg/sql/logictest/testdata/logic_test/insert b/pkg/sql/logictest/testdata/logic_test/insert index 71fe04178406..72cf0b24293a 100644 --- a/pkg/sql/logictest/testdata/logic_test/insert +++ b/pkg/sql/logictest/testdata/logic_test/insert @@ -129,17 +129,20 @@ INSERT INTO kv4 (int, bool) VALUES (3, 'a') statement ok INSERT INTO kv4 (int, bool) VALUES (3, true) -statement error value type int doesn't match type char of column "char" +statement error value too long for type CHAR +INSERT INTO kv4 (int, char) VALUES (4, 11) + +statement ok INSERT INTO kv4 (int, char) VALUES (4, 1) statement ok -INSERT INTO kv4 (int, char) VALUES (4, 'a') +INSERT INTO kv4 (int, char) VALUES (5, 'a') -statement error value type int doesn't match type float of column "float" -INSERT INTO kv4 (int, float) VALUES (5, 1::INT) +statement ok +INSERT INTO kv4 (int, float) VALUES (6, 1::INT) statement ok -INSERT INTO kv4 (int, float) VALUES (5, 2.3) +INSERT INTO kv4 (int, float) VALUES (7, 2.3) query ITBTR rowsort SELECT * from kv4 @@ -147,8 +150,10 @@ SELECT * from kv4 1 NULL NULL NULL NULL 2 1 NULL NULL NULL 3 NULL true NULL NULL -4 NULL NULL a NULL -5 NULL NULL NULL 2.3 +4 NULL NULL 1 NULL +5 NULL NULL a NULL +6 NULL NULL NULL 1 +7 NULL NULL NULL 2.3 statement ok CREATE TABLE kv5 ( @@ -446,7 +451,7 @@ INSERT INTO string_t VALUES ('str') query error value type string doesn't match type bytes of column "b" INSERT INTO bytes_t SELECT * FROM string_t -query error value type bytes doesn't match type string of column "s" +statement ok INSERT INTO string_t SELECT * FROM bytes_t subtest string_width_check diff --git a/pkg/sql/logictest/testdata/logic_test/typing b/pkg/sql/logictest/testdata/logic_test/typing index 33a11d293578..1d0abda00b79 100644 --- a/pkg/sql/logictest/testdata/logic_test/typing +++ b/pkg/sql/logictest/testdata/logic_test/typing @@ -13,8 +13,8 @@ SELECT * FROM f statement ok CREATE TABLE i (x INT) -statement error value type decimal doesn't match type int of column "x" -INSERT INTO i(x) VALUES (4.5) +statement error value type timestamptz doesn't match type int of column "x" +INSERT INTO i(x) VALUES ('1970-01-01'::timestamptz) statement ok INSERT INTO i(x) VALUES (2.0) diff --git a/pkg/sql/opt/exec/execbuilder/format.go b/pkg/sql/opt/exec/execbuilder/format.go index 2d209e37dbdc..3bff35888aa7 100644 --- a/pkg/sql/opt/exec/execbuilder/format.go +++ b/pkg/sql/opt/exec/execbuilder/format.go @@ -25,12 +25,17 @@ func init() { // if an expression tree contains only scalar expressions; if so, it tries to // execbuild them and print the SQL expressions. func fmtInterceptor(f *memo.ExprFmtCtx, scalar opt.ScalarExpr) string { - if !onlyScalars(scalar) { + // An AssignmentCastExpr is built as a crdb_internal.assignment_cast + // function call by execbuilder. Formatting it as such would be confusing in + // an opt tree, because it would look like a FunctionExpr. So we print the + // full nodes instead. + if !onlyScalarsWithoutAssignmentCasts(scalar) { return "" } - // Let the filters node show up; we will apply the code on each filter. - if scalar.Op() == opt.FiltersOp { + switch scalar.Op() { + case opt.FiltersOp: + // Let the filters node show up; we will apply the code on each filter. return "" } @@ -59,12 +64,12 @@ func fmtInterceptor(f *memo.ExprFmtCtx, scalar opt.ScalarExpr) string { return fmtCtx.String() } -func onlyScalars(expr opt.Expr) bool { - if !opt.IsScalarOp(expr) { +func onlyScalarsWithoutAssignmentCasts(expr opt.Expr) bool { + if !opt.IsScalarOp(expr) || expr.Op() == opt.AssignmentCastOp { return false } for i, n := 0, expr.ChildCount(); i < n; i++ { - if !onlyScalars(expr.Child(i)) { + if !onlyScalarsWithoutAssignmentCasts(expr.Child(i)) { return false } } diff --git a/pkg/sql/opt/exec/execbuilder/scalar.go b/pkg/sql/opt/exec/execbuilder/scalar.go index ad606e9ebeda..790c893cb782 100644 --- a/pkg/sql/opt/exec/execbuilder/scalar.go +++ b/pkg/sql/opt/exec/execbuilder/scalar.go @@ -14,6 +14,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/opt/exec" "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" + "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -37,23 +38,24 @@ func init() { // the functions depend on scalarBuildFuncMap which in turn depends on the // functions). scalarBuildFuncMap = [opt.NumOperators]buildFunc{ - opt.VariableOp: (*Builder).buildVariable, - opt.ConstOp: (*Builder).buildTypedExpr, - opt.NullOp: (*Builder).buildNull, - opt.PlaceholderOp: (*Builder).buildTypedExpr, - opt.TupleOp: (*Builder).buildTuple, - opt.FunctionOp: (*Builder).buildFunction, - opt.CaseOp: (*Builder).buildCase, - opt.CastOp: (*Builder).buildCast, - opt.CoalesceOp: (*Builder).buildCoalesce, - opt.ColumnAccessOp: (*Builder).buildColumnAccess, - opt.ArrayOp: (*Builder).buildArray, - opt.AnyOp: (*Builder).buildAny, - opt.AnyScalarOp: (*Builder).buildAnyScalar, - opt.IndirectionOp: (*Builder).buildIndirection, - opt.CollateOp: (*Builder).buildCollate, - opt.ArrayFlattenOp: (*Builder).buildArrayFlatten, - opt.IfErrOp: (*Builder).buildIfErr, + opt.VariableOp: (*Builder).buildVariable, + opt.ConstOp: (*Builder).buildTypedExpr, + opt.NullOp: (*Builder).buildNull, + opt.PlaceholderOp: (*Builder).buildTypedExpr, + opt.TupleOp: (*Builder).buildTuple, + opt.FunctionOp: (*Builder).buildFunction, + opt.CaseOp: (*Builder).buildCase, + opt.CastOp: (*Builder).buildCast, + opt.AssignmentCastOp: (*Builder).buildAssignmentCast, + opt.CoalesceOp: (*Builder).buildCoalesce, + opt.ColumnAccessOp: (*Builder).buildColumnAccess, + opt.ArrayOp: (*Builder).buildArray, + opt.AnyOp: (*Builder).buildAny, + opt.AnyScalarOp: (*Builder).buildAnyScalar, + opt.IndirectionOp: (*Builder).buildIndirection, + opt.CollateOp: (*Builder).buildCollate, + opt.ArrayFlattenOp: (*Builder).buildArrayFlatten, + opt.IfErrOp: (*Builder).buildIfErr, // Item operators. opt.ProjectionsItemOp: (*Builder).buildItem, @@ -345,6 +347,38 @@ func (b *Builder) buildCast(ctx *buildScalarCtx, scalar opt.ScalarExpr) (tree.Ty return tree.NewTypedCastExpr(input, cast.Typ), nil } +// buildAssignmentCast builds an AssignmentCastExpr with input i and type T into +// a built-in function call crdb_internal.assignment_cast(i, NULL::T). +func (b *Builder) buildAssignmentCast( + ctx *buildScalarCtx, scalar opt.ScalarExpr, +) (tree.TypedExpr, error) { + cast := scalar.(*memo.AssignmentCastExpr) + input, err := b.buildScalar(ctx, cast.Input) + if err != nil { + return nil, err + } + if cast.Typ.Family() == types.TupleFamily { + // TODO(radu): casts to Tuple are not supported (they can't be + // serialized for distsql). This should only happen when the input is + // always NULL so the expression should still be valid without the cast + // (though there could be cornercases where the type does matter). + return input, nil + } + const fnName = "crdb_internal.assignment_cast" + funcRef := tree.WrapFunction(fnName) + props, overloads := builtins.GetBuiltinProperties(fnName) + return tree.NewTypedFuncExpr( + funcRef, + 0, /* aggQualifier */ + tree.TypedExprs{input, tree.NewTypedCastExpr(tree.DNull, cast.Typ)}, + nil, /* filter */ + nil, /* windowDef */ + cast.Typ, + props, + &overloads[0], + ), nil +} + func (b *Builder) buildCoalesce( ctx *buildScalarCtx, scalar opt.ScalarExpr, ) (tree.TypedExpr, error) { diff --git a/pkg/sql/opt/memo/expr_format.go b/pkg/sql/opt/memo/expr_format.go index da9a892d5486..fa4b45c16ddd 100644 --- a/pkg/sql/opt/memo/expr_format.go +++ b/pkg/sql/opt/memo/expr_format.go @@ -1077,8 +1077,9 @@ func (f *ExprFmtCtx) formatScalarPrivate(scalar opt.ScalarExpr) { // We don't want to show the OriginalExpr. private = nil - case *CastExpr: - private = t.Typ.SQLString() + case *CastExpr, *AssignmentCastExpr: + typ := scalar.Private().(*types.T) + private = typ.SQLString() case *KVOptionsItem: fmt.Fprintf(f.Buffer, " %s", t.Key) diff --git a/pkg/sql/opt/memo/logical_props_builder.go b/pkg/sql/opt/memo/logical_props_builder.go index 13be2b781d3a..1367102adac8 100644 --- a/pkg/sql/opt/memo/logical_props_builder.go +++ b/pkg/sql/opt/memo/logical_props_builder.go @@ -1594,8 +1594,9 @@ func BuildSharedProps(e opt.Expr, shared *props.Shared, evalCtx *tree.EvalContex case *FunctionExpr: shared.VolatilitySet.Add(t.Overload.Volatility) - case *CastExpr: - from, to := t.Input.DataType(), t.Typ + case *CastExpr, *AssignmentCastExpr: + from := e.Child(0).(opt.ScalarExpr).DataType() + to := e.Private().(*types.T) volatility, ok := tree.LookupCastVolatility(from, to, evalCtx.SessionData()) if !ok { panic(errors.AssertionFailedf("no volatility for cast %s::%s", from, to)) diff --git a/pkg/sql/opt/ops/scalar.opt b/pkg/sql/opt/ops/scalar.opt index 6319484c3653..dcf85083cc5e 100644 --- a/pkg/sql/opt/ops/scalar.opt +++ b/pkg/sql/opt/ops/scalar.opt @@ -267,7 +267,7 @@ define ZipItem { Cols ColList } -# And is the boolean conjunction operator that evalutes to true only if both of +# And is the boolean conjunction operator that evaluates to true only if both of # its conditions evaluate to true. [Scalar, Bool] define And { @@ -630,8 +630,8 @@ define UnaryCbrt { Input ScalarExpr } -# Cast converts the input expression into an expression of the target type. -# Note that the conversion may cause trunction based on the target types' width, +# Cast converts the input expression into an expression of the target type. Note +# that the conversion may cause truncation based on the target types' width, # such as in this example: # # 'hello'::VARCHAR(2) @@ -645,6 +645,23 @@ define Cast { Typ Type } +# AssignmentCast is similar to CastExpr, but is performed in the context of an +# INSERT, UPDATE, or UPSERT to match the type of a mutation value to the type of +# the target column. An expression separate from CastExpr is required because it +# behaves slightly differently than an explicit cast. For example, while an +# explicit cast will truncate a value to fit the width of a type, an assignment +# cast will error instead if the value does not fit the type. See +# tree.CastContext for more details. +# +# An assignment cast is represented as a distinct expression within the +# optimizer, but is built into a crdb_internal.assignment_cast function call in +# execbuilder. +[Scalar] +define AssignmentCast { + Input ScalarExpr + Typ Type +} + # IfErr is roughly a runtime try-catch operator. It has different semantics # depending on which of its fields are set. # diff --git a/pkg/sql/opt/optbuilder/insert.go b/pkg/sql/opt/optbuilder/insert.go index 3a47a9ecd7e5..c38765b13a12 100644 --- a/pkg/sql/opt/optbuilder/insert.go +++ b/pkg/sql/opt/optbuilder/insert.go @@ -255,6 +255,7 @@ func (b *Builder) buildInsert(ins *tree.Insert, inScope *scope) (outScope *scope // // INSERT INTO DEFAULT VALUES // + isUpsert := ins.OnConflict != nil && !ins.OnConflict.DoNothing if !ins.DefaultValues() { // Replace any DEFAULT expressions in the VALUES clause, if a VALUES clause // exists: @@ -263,15 +264,15 @@ func (b *Builder) buildInsert(ins *tree.Insert, inScope *scope) (outScope *scope // rows := mb.replaceDefaultExprs(ins.Rows) - mb.buildInputForInsert(inScope, rows) + mb.buildInputForInsert(inScope, rows, isUpsert) } else { - mb.buildInputForInsert(inScope, nil /* rows */) + mb.buildInputForInsert(inScope, nil /* rows */, isUpsert) } // Add default columns that were not explicitly specified by name or // implicitly targeted by input columns. Also add any computed columns. In // both cases, include columns undergoing mutations in the write-only state. - mb.addSynthesizedColsForInsert() + mb.addSynthesizedColsForInsert(isUpsert) var returning tree.ReturningExprs if resultsNeeded(ins.Returning) { @@ -553,7 +554,9 @@ func (mb *mutationBuilder) addTargetTableColsForInsert(maxCols int) { // buildInputForInsert constructs the memo group for the input expression and // constructs a new output scope containing that expression's output columns. -func (mb *mutationBuilder) buildInputForInsert(inScope *scope, inputRows *tree.Select) { +func (mb *mutationBuilder) buildInputForInsert( + inScope *scope, inputRows *tree.Select, isUpsert bool, +) { // Handle DEFAULT VALUES case by creating a single empty row as input. if inputRows == nil { mb.outScope = inScope.push() @@ -603,16 +606,23 @@ func (mb *mutationBuilder) buildInputForInsert(inScope *scope, inputRows *tree.S mb.addTargetTableColsForInsert(len(mb.outScope.cols)) } + if !isUpsert { + mb.outScope = mb.addAssignmentCasts(mb.outScope, desiredTypes) + } + // Loop over input columns and: // 1. Type check each column + // 2. Check if the INSERT violates a GENERATED ALWAYS AS IDENTITY column. // 2. Assign name to each column // 3. Add column ID to the insertColIDs list. for i := range mb.outScope.cols { inCol := &mb.outScope.cols[i] ord := mb.tabID.ColumnOrdinal(mb.targetColList[i]) - // Type check the input column against the corresponding table column. - checkDatumTypeFitsColumnType(mb.tab.Column(ord), inCol.typ) + if isUpsert { + // Type check the input column against the corresponding table column. + checkDatumTypeFitsColumnType(mb.tab.Column(ord), inCol.typ) + } // Check if the input column is created with `GENERATED ALWAYS AS IDENTITY` // syntax. If yes, and user does not specify the `OVERRIDING SYSTEM VALUE` @@ -634,7 +644,7 @@ func (mb *mutationBuilder) buildInputForInsert(inScope *scope, inputRows *tree.S // columns that are not yet part of the target column list. This includes all // write-only mutation columns, since they must always have default or computed // values. -func (mb *mutationBuilder) addSynthesizedColsForInsert() { +func (mb *mutationBuilder) addSynthesizedColsForInsert(isUpsert bool) { // Start by adding non-computed columns that have not already been explicitly // specified in the query. Do this before adding computed columns, since those // may depend on non-computed columns. @@ -646,13 +656,17 @@ func (mb *mutationBuilder) addSynthesizedColsForInsert() { // Possibly round DECIMAL-related columns containing insertion values (whether // synthesized or not). - mb.roundDecimalValues(mb.insertColIDs, false /* roundComputedCols */) + if isUpsert { + mb.roundDecimalValues(mb.insertColIDs, false /* roundComputedCols */) + } // Now add all computed columns. mb.addSynthesizedComputedCols(mb.insertColIDs, false /* restrict */) // Possibly round DECIMAL-related computed columns. - mb.roundDecimalValues(mb.insertColIDs, true /* roundComputedCols */) + if isUpsert { + mb.roundDecimalValues(mb.insertColIDs, true /* roundComputedCols */) + } } // buildInsert constructs an Insert operator, possibly wrapped by a Project diff --git a/pkg/sql/opt/optbuilder/mutation_builder.go b/pkg/sql/opt/optbuilder/mutation_builder.go index addf91e903de..b9d2f5d1741f 100644 --- a/pkg/sql/opt/optbuilder/mutation_builder.go +++ b/pkg/sql/opt/optbuilder/mutation_builder.go @@ -1370,6 +1370,7 @@ func resultsNeeded(r tree.ReturningClause) bool { // be different (eg. TEXT and VARCHAR will fit the same scalar type String). // // This is used by the UPDATE, INSERT and UPSERT code. +// TODO(mgartner): Remove this once assignment casts are fully supported. func checkDatumTypeFitsColumnType(col *cat.Column, typ *types.T) { if typ.Equivalent(col.DatumType()) { return @@ -1383,6 +1384,62 @@ func checkDatumTypeFitsColumnType(col *cat.Column, typ *types.T) { panic(err) } +// addAssignmentCasts builds a projection that wraps mutation values with +// assignment casts when possible so that the resulting columns have types +// identical to those in outTypes. If all the columns in inScope already have +// identical types, then no projection is built. If there is no valid assignment +// cast from a column type in inScope to the corresponding target column type, +// then this function will error. +func (mb *mutationBuilder) addAssignmentCasts(inScope *scope, outTypes []*types.T) *scope { + expr := inScope.expr.(memo.RelExpr) + + // Do a quick check to see if any casts are needed. + castRequired := false + for i := 0; i < len(inScope.cols); i++ { + if !inScope.cols[i].typ.Identical(outTypes[i]) { + castRequired = true + break + } + } + if !castRequired { + // No mutation casts are needed. + return inScope + } + + projectionScope := inScope.push() + projectionScope.cols = make([]scopeColumn, 0, len(inScope.cols)) + for i := 0; i < len(inScope.cols); i++ { + srcType := inScope.cols[i].typ + targetType := outTypes[i] + if !srcType.Identical(targetType) { + // Check if an assignment cast is available from the inScope column + // type to the out type. + if !tree.ValidCast(srcType, targetType, tree.CastContextAssignment) { + ord := mb.tabID.ColumnOrdinal(mb.targetColList[i]) + colName := string(mb.tab.Column(ord).ColName()) + err := pgerror.Newf(pgcode.DatatypeMismatch, + "value type %s doesn't match type %s of column %q", + srcType, targetType, tree.ErrNameString(colName)) + err = errors.WithHint(err, "you will need to rewrite or cast the expression") + panic(err) + } + + // Create a new column which casts the input column to the correct + // type. + variable := mb.b.factory.ConstructVariable(inScope.cols[i].id) + cast := mb.b.factory.ConstructAssignmentCast(variable, outTypes[i]) + mb.b.synthesizeColumn(projectionScope, inScope.cols[i].name, outTypes[i], nil /* expr */, cast) + } else { + // The column is already the correct type, so add it as a + // passthrough column. + projectionScope.appendColumn(&inScope.cols[i]) + } + } + + projectionScope.expr = mb.b.constructProject(expr, projectionScope.cols) + return projectionScope +} + // checkColumnIsNotGeneratedAlwaysAsIdentity verifies that if current column // is not created as an IDENTITY column with the // `GENERATED ALWAYS AS IDENTITY` syntax. diff --git a/pkg/sql/opt/optbuilder/testdata/insert b/pkg/sql/opt/optbuilder/testdata/insert index d4874a080430..d6a2b64beffc 100644 --- a/pkg/sql/opt/optbuilder/testdata/insert +++ b/pkg/sql/opt/optbuilder/testdata/insert @@ -53,6 +53,15 @@ CREATE TABLE decimals ( ) ---- +exec-ddl +CREATE TABLE assn_cast ( + c CHAR, + qc "char", + i INT, + s STRING +) +---- + exec-ddl CREATE TABLE on_update_bare ( a INT NOT NULL, @@ -747,7 +756,25 @@ insert abcde build INSERT INTO xyz (x) VALUES (10) ---- -error (42804): value type int doesn't match type string of column "x" +insert xyz + ├── columns: + ├── insert-mapping: + │ ├── column1:7 => x:1 + │ ├── y_default:8 => y:2 + │ └── z_default:9 => z:3 + └── project + ├── columns: y_default:8 z_default:9 column1:7 + ├── project + │ ├── columns: column1:7 + │ ├── values + │ │ ├── columns: column1:6!null + │ │ └── (10,) + │ └── projections + │ └── assignment-cast: STRING [as=column1:7] + │ └── column1:6 + └── projections + ├── NULL::INT8 [as=y_default:8] + └── NULL::FLOAT8 [as=z_default:9] # Try to insert into computed column. build @@ -1154,7 +1181,7 @@ insert checks └── abcde.a:7 > 0 [as=check2:17] # ------------------------------------------------------------------------------ -# Test decimal column rounding. +# Test assignment casts. # ------------------------------------------------------------------------------ build @@ -1163,37 +1190,107 @@ INSERT INTO decimals (a, b) VALUES (1.1, ARRAY[0.95, NULL, 15]) insert decimals ├── columns: ├── insert-mapping: - │ ├── a:10 => decimals.a:1 - │ ├── b:11 => decimals.b:2 - │ ├── c_default:12 => c:3 - │ └── d_comp:14 => d:4 - ├── check columns: check1:15 check2:16 + │ ├── column1:9 => a:1 + │ ├── column2:8 => b:2 + │ ├── c_default:10 => c:3 + │ └── d_comp:11 => d:4 + ├── check columns: check1:12 check2:13 └── project - ├── columns: check1:15 check2:16 a:10 b:11 c_default:12 d_comp:14 + ├── columns: check1:12 check2:13 column2:8 column1:9 c_default:10!null d_comp:11 ├── project - │ ├── columns: d_comp:14 a:10 b:11 c_default:12 + │ ├── columns: d_comp:11 column2:8 column1:9 c_default:10!null │ ├── project - │ │ ├── columns: d_comp:13 a:10 b:11 c_default:12 + │ │ ├── columns: c_default:10!null column2:8 column1:9 │ │ ├── project - │ │ │ ├── columns: a:10 b:11 c_default:12 - │ │ │ ├── project - │ │ │ │ ├── columns: c_default:9!null column1:7!null column2:8 - │ │ │ │ ├── values - │ │ │ │ │ ├── columns: column1:7!null column2:8 - │ │ │ │ │ └── (1.1, ARRAY[0.95,NULL,15]) - │ │ │ │ └── projections - │ │ │ │ └── 1.23::DECIMAL(10,1) [as=c_default:9] + │ │ │ ├── columns: column1:9 column2:8 + │ │ │ ├── values + │ │ │ │ ├── columns: column1:7!null column2:8 + │ │ │ │ └── (1.1, ARRAY[0.95,NULL,15]) │ │ │ └── projections - │ │ │ ├── crdb_internal.round_decimal_values(column1:7, 0) [as=a:10] - │ │ │ ├── crdb_internal.round_decimal_values(column2:8, 1) [as=b:11] - │ │ │ └── crdb_internal.round_decimal_values(c_default:9, 1) [as=c_default:12] + │ │ │ └── assignment-cast: DECIMAL(10) [as=column1:9] + │ │ │ └── column1:7 │ │ └── projections - │ │ └── (a:10 + c_default:12::DECIMAL)::DECIMAL(10,1) [as=d_comp:13] + │ │ └── 1.23::DECIMAL(10,1) [as=c_default:10] + │ └── projections + │ └── (column1:9::DECIMAL + c_default:10::DECIMAL)::DECIMAL(10,1) [as=d_comp:11] + └── projections + ├── round(column1:9) = column1:9 [as=check1:12] + └── column2:8[0] > 1 [as=check2:13] + +assign-placeholders-norm query-args=(1.1, (ARRAY[0.95, NULL, 15])) +INSERT INTO decimals (a, b) VALUES ($1, $2) +---- +insert decimals + ├── columns: + ├── insert-mapping: + │ ├── column1:9 => a:1 + │ ├── column2:8 => b:2 + │ ├── c_default:10 => c:3 + │ └── d_comp:11 => d:4 + ├── check columns: check1:12 check2:13 + └── project + ├── columns: check1:12 check2:13 d_comp:11 column2:8!null column1:9 c_default:10!null + ├── values + │ ├── columns: column2:8!null c_default:10!null column1:9 + │ └── tuple + │ ├── ARRAY[0.95,NULL,15] + │ ├── 1.2 + │ └── assignment-cast: DECIMAL(10) + │ └── 1.1 + └── projections + ├── column1:9 = round(column1:9) [as=check1:12] + ├── CAST(NULL AS BOOL) [as=check2:13] + └── (column1:9::DECIMAL + 1.2)::DECIMAL(10,1) [as=d_comp:11] + +build +INSERT INTO assn_cast (c, qc, i, s) VALUES (' ', 'foo', '1', 2) +---- +insert assn_cast + ├── columns: + ├── insert-mapping: + │ ├── column1:12 => c:1 + │ ├── column2:13 => qc:2 + │ ├── column3:10 => i:3 + │ ├── column4:14 => s:4 + │ └── rowid_default:15 => rowid:5 + └── project + ├── columns: rowid_default:15 column3:10!null column1:12 column2:13 column4:14 + ├── project + │ ├── columns: column1:12 column2:13 column4:14 column3:10!null + │ ├── values + │ │ ├── columns: column1:8!null column2:9!null column3:10!null column4:11!null + │ │ └── ('', 'f', 1, 2) │ └── projections - │ └── crdb_internal.round_decimal_values(d_comp:13, 1) [as=d_comp:14] + │ ├── assignment-cast: CHAR [as=column1:12] + │ │ └── column1:8 + │ ├── assignment-cast: "char" [as=column2:13] + │ │ └── column2:9 + │ └── assignment-cast: STRING [as=column4:14] + │ └── column4:11 └── projections - ├── round(a:10) = a:10 [as=check1:15] - └── b:11[0] > 1 [as=check2:16] + └── unique_rowid() [as=rowid_default:15] + +assign-placeholders-norm query-args=(' ', 'foo', '1') +INSERT INTO assn_cast (c, qc, i) VALUES ($1, $2, $3) +---- +insert assn_cast + ├── columns: + ├── insert-mapping: + │ ├── column1:11 => c:1 + │ ├── column2:12 => qc:2 + │ ├── column3:10 => i:3 + │ ├── s_default:13 => s:4 + │ └── rowid_default:14 => rowid:5 + └── values + ├── columns: column3:10!null s_default:13 rowid_default:14 column1:11 column2:12 + └── tuple + ├── 1 + ├── CAST(NULL AS STRING) + ├── unique_rowid() + ├── assignment-cast: CHAR + │ └── ' ' + └── assignment-cast: "char" + └── 'foo' # Regression test for #38293; the default values should be separate projections. exec-ddl diff --git a/pkg/sql/opt/optbuilder/update.go b/pkg/sql/opt/optbuilder/update.go index 5d132b1ccd82..01db003562b3 100644 --- a/pkg/sql/opt/optbuilder/update.go +++ b/pkg/sql/opt/optbuilder/update.go @@ -230,6 +230,7 @@ func (mb *mutationBuilder) addUpdateCols(exprs tree.UpdateExprs) { } // Add new column to the projections scope. + // TODO(mgartner): Perform an assignment cast if necessary. targetColMeta := mb.md.ColumnMeta(targetColID) desiredType := targetColMeta.Type texpr := inScope.resolveType(expr, desiredType) diff --git a/pkg/sql/opt/xform/testdata/external/tpcc b/pkg/sql/opt/xform/testdata/external/tpcc index f72e41ab254e..3b0c8dd9f2de 100644 --- a/pkg/sql/opt/xform/testdata/external/tpcc +++ b/pkg/sql/opt/xform/testdata/external/tpcc @@ -364,18 +364,18 @@ insert order_line │ ├── column4:16 => ol_number:4 │ ├── column5:17 => order_line.ol_i_id:5 │ ├── column6:18 => order_line.ol_supply_w_id:6 - │ ├── ol_delivery_d_default:22 => ol_delivery_d:7 + │ ├── ol_delivery_d_default:24 => ol_delivery_d:7 │ ├── column7:19 => ol_quantity:8 - │ ├── ol_amount:23 => order_line.ol_amount:9 - │ └── column9:21 => ol_dist_info:10 + │ ├── column8:22 => ol_amount:9 + │ └── column9:23 => ol_dist_info:10 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── project - │ ├── columns: ol_amount:23 ol_delivery_d_default:22 column1:13!null column2:14!null column3:15!null column4:16!null column5:17!null column6:18!null column7:19!null column9:21!null + │ ├── columns: ol_delivery_d_default:24 column8:22 column9:23 column1:13!null column2:14!null column3:15!null column4:16!null column5:17!null column6:18!null column7:19!null │ ├── cardinality: [6 - 6] │ ├── immutable - │ ├── fd: ()-->(22) + │ ├── fd: ()-->(24) │ ├── values │ │ ├── columns: column1:13!null column2:14!null column3:15!null column4:16!null column5:17!null column6:18!null column7:19!null column8:20!null column9:21!null │ │ ├── cardinality: [6 - 6] @@ -386,34 +386,37 @@ insert order_line │ │ ├── (3045, 2, 10, 4, 56624, 0, 6, 273.360000, 'RsaCXoEzmssaF9m9cdLXe0Yh') │ │ └── (3045, 2, 10, 6, 92966, 0, 4, 366.760000, 'saCXoEzmssaF9m9cdLXe0Yhg') │ └── projections - │ ├── crdb_internal.round_decimal_values(column8:20, 2) [as=ol_amount:23, outer=(20), immutable] - │ └── CAST(NULL AS TIMESTAMP) [as=ol_delivery_d_default:22] + │ ├── CAST(NULL AS TIMESTAMP) [as=ol_delivery_d_default:24] + │ ├── assignment-cast: DECIMAL(6,2) [as=column8:22, outer=(20), immutable] + │ │ └── column8:20 + │ └── assignment-cast: CHAR(24) [as=column9:23, outer=(21), immutable] + │ └── column9:21 └── f-k-checks ├── f-k-checks-item: order_line(ol_w_id,ol_d_id,ol_o_id) -> order(o_w_id,o_d_id,o_id) │ └── anti-join (lookup order) - │ ├── columns: ol_w_id:24!null ol_d_id:25!null ol_o_id:26!null - │ ├── key columns: [24 25 26] = [29 28 27] + │ ├── columns: ol_w_id:25!null ol_d_id:26!null ol_o_id:27!null + │ ├── key columns: [25 26 27] = [30 29 28] │ ├── lookup columns are key │ ├── cardinality: [0 - 6] │ ├── with-scan &1 - │ │ ├── columns: ol_w_id:24!null ol_d_id:25!null ol_o_id:26!null + │ │ ├── columns: ol_w_id:25!null ol_d_id:26!null ol_o_id:27!null │ │ ├── mapping: - │ │ │ ├── column3:15 => ol_w_id:24 - │ │ │ ├── column2:14 => ol_d_id:25 - │ │ │ └── column1:13 => ol_o_id:26 + │ │ │ ├── column3:15 => ol_w_id:25 + │ │ │ ├── column2:14 => ol_d_id:26 + │ │ │ └── column1:13 => ol_o_id:27 │ │ └── cardinality: [6 - 6] │ └── filters (true) └── f-k-checks-item: order_line(ol_supply_w_id,ol_i_id) -> stock(s_w_id,s_i_id) └── anti-join (lookup stock) - ├── columns: ol_supply_w_id:37!null ol_i_id:38!null - ├── key columns: [37 38] = [40 39] + ├── columns: ol_supply_w_id:38!null ol_i_id:39!null + ├── key columns: [38 39] = [41 40] ├── lookup columns are key ├── cardinality: [0 - 6] ├── with-scan &1 - │ ├── columns: ol_supply_w_id:37!null ol_i_id:38!null + │ ├── columns: ol_supply_w_id:38!null ol_i_id:39!null │ ├── mapping: - │ │ ├── column6:18 => ol_supply_w_id:37 - │ │ └── column5:17 => ol_i_id:38 + │ │ ├── column6:18 => ol_supply_w_id:38 + │ │ └── column5:17 => ol_i_id:39 │ └── cardinality: [6 - 6] └── filters (true) @@ -600,60 +603,71 @@ VALUES insert history ├── columns: ├── insert-mapping: - │ ├── rowid_default:20 => rowid:1 + │ ├── rowid_default:22 => rowid:1 │ ├── column1:12 => history.h_c_id:2 │ ├── column2:13 => history.h_c_d_id:3 │ ├── column3:14 => history.h_c_w_id:4 │ ├── column4:15 => history.h_d_id:5 │ ├── column5:16 => history.h_w_id:6 │ ├── column7:18 => h_date:7 - │ ├── h_amount:21 => history.h_amount:8 - │ └── column8:19 => h_data:9 + │ ├── column6:20 => h_amount:8 + │ └── column8:21 => h_data:9 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:12!null column2:13!null column3:14!null column4:15!null column5:16!null column7:18!null column8:19!null rowid_default:20 h_amount:21!null + │ ├── columns: column1:12!null column2:13!null column3:14!null column4:15!null column5:16!null column7:18!null column6:20 column8:21 rowid_default:22 │ ├── cardinality: [1 - 1] │ ├── volatile │ ├── key: () - │ ├── fd: ()-->(12-16,18-21) - │ └── (1343, 5, 10, 5, 10, '2019-08-26 16:50:41', '8 Kdcgphy3', gen_random_uuid(), 3860.61) + │ ├── fd: ()-->(12-16,18,20-22) + │ └── tuple + │ ├── 1343 + │ ├── 5 + │ ├── 10 + │ ├── 5 + │ ├── 10 + │ ├── '2019-08-26 16:50:41' + │ ├── assignment-cast: DECIMAL(6,2) + │ │ └── 3860.61 + │ ├── assignment-cast: VARCHAR(24) + │ │ └── '8 Kdcgphy3' + │ └── gen_random_uuid() └── f-k-checks ├── f-k-checks-item: history(h_c_w_id,h_c_d_id,h_c_id) -> customer(c_w_id,c_d_id,c_id) │ └── anti-join (lookup customer) - │ ├── columns: h_c_w_id:22!null h_c_d_id:23!null h_c_id:24!null - │ ├── key columns: [22 23 24] = [27 26 25] + │ ├── columns: h_c_w_id:23!null h_c_d_id:24!null h_c_id:25!null + │ ├── key columns: [23 24 25] = [28 27 26] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(22-24) + │ ├── fd: ()-->(23-25) │ ├── with-scan &1 - │ │ ├── columns: h_c_w_id:22!null h_c_d_id:23!null h_c_id:24!null + │ │ ├── columns: h_c_w_id:23!null h_c_d_id:24!null h_c_id:25!null │ │ ├── mapping: - │ │ │ ├── column3:14 => h_c_w_id:22 - │ │ │ ├── column2:13 => h_c_d_id:23 - │ │ │ └── column1:12 => h_c_id:24 + │ │ │ ├── column3:14 => h_c_w_id:23 + │ │ │ ├── column2:13 => h_c_d_id:24 + │ │ │ └── column1:12 => h_c_id:25 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(22-24) + │ │ └── fd: ()-->(23-25) │ └── filters (true) └── f-k-checks-item: history(h_w_id,h_d_id) -> district(d_w_id,d_id) └── anti-join (lookup district) - ├── columns: h_w_id:48!null h_d_id:49!null - ├── key columns: [48 49] = [51 50] + ├── columns: h_w_id:49!null h_d_id:50!null + ├── key columns: [49 50] = [52 51] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(48,49) + ├── fd: ()-->(49,50) ├── with-scan &1 - │ ├── columns: h_w_id:48!null h_d_id:49!null + │ ├── columns: h_w_id:49!null h_d_id:50!null │ ├── mapping: - │ │ ├── column5:16 => h_w_id:48 - │ │ └── column4:15 => h_d_id:49 + │ │ ├── column5:16 => h_w_id:49 + │ │ └── column4:15 => h_d_id:50 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(48,49) + │ └── fd: ()-->(49,50) └── filters (true) # -------------------------------------------------- diff --git a/pkg/sql/opt/xform/testdata/external/tpcc-later-stats b/pkg/sql/opt/xform/testdata/external/tpcc-later-stats index 305a3e63ded6..67aea57d269a 100644 --- a/pkg/sql/opt/xform/testdata/external/tpcc-later-stats +++ b/pkg/sql/opt/xform/testdata/external/tpcc-later-stats @@ -367,18 +367,18 @@ insert order_line │ ├── column4:16 => ol_number:4 │ ├── column5:17 => order_line.ol_i_id:5 │ ├── column6:18 => order_line.ol_supply_w_id:6 - │ ├── ol_delivery_d_default:22 => ol_delivery_d:7 + │ ├── ol_delivery_d_default:24 => ol_delivery_d:7 │ ├── column7:19 => ol_quantity:8 - │ ├── ol_amount:23 => order_line.ol_amount:9 - │ └── column9:21 => ol_dist_info:10 + │ ├── column8:22 => ol_amount:9 + │ └── column9:23 => ol_dist_info:10 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── project - │ ├── columns: ol_amount:23 ol_delivery_d_default:22 column1:13!null column2:14!null column3:15!null column4:16!null column5:17!null column6:18!null column7:19!null column9:21!null + │ ├── columns: ol_delivery_d_default:24 column8:22 column9:23 column1:13!null column2:14!null column3:15!null column4:16!null column5:17!null column6:18!null column7:19!null │ ├── cardinality: [6 - 6] │ ├── immutable - │ ├── fd: ()-->(22) + │ ├── fd: ()-->(24) │ ├── values │ │ ├── columns: column1:13!null column2:14!null column3:15!null column4:16!null column5:17!null column6:18!null column7:19!null column8:20!null column9:21!null │ │ ├── cardinality: [6 - 6] @@ -389,34 +389,37 @@ insert order_line │ │ ├── (3045, 2, 10, 4, 56624, 0, 6, 273.360000, 'RsaCXoEzmssaF9m9cdLXe0Yh') │ │ └── (3045, 2, 10, 6, 92966, 0, 4, 366.760000, 'saCXoEzmssaF9m9cdLXe0Yhg') │ └── projections - │ ├── crdb_internal.round_decimal_values(column8:20, 2) [as=ol_amount:23, outer=(20), immutable] - │ └── CAST(NULL AS TIMESTAMP) [as=ol_delivery_d_default:22] + │ ├── CAST(NULL AS TIMESTAMP) [as=ol_delivery_d_default:24] + │ ├── assignment-cast: DECIMAL(6,2) [as=column8:22, outer=(20), immutable] + │ │ └── column8:20 + │ └── assignment-cast: CHAR(24) [as=column9:23, outer=(21), immutable] + │ └── column9:21 └── f-k-checks ├── f-k-checks-item: order_line(ol_w_id,ol_d_id,ol_o_id) -> order(o_w_id,o_d_id,o_id) │ └── anti-join (lookup order) - │ ├── columns: ol_w_id:24!null ol_d_id:25!null ol_o_id:26!null - │ ├── key columns: [24 25 26] = [29 28 27] + │ ├── columns: ol_w_id:25!null ol_d_id:26!null ol_o_id:27!null + │ ├── key columns: [25 26 27] = [30 29 28] │ ├── lookup columns are key │ ├── cardinality: [0 - 6] │ ├── with-scan &1 - │ │ ├── columns: ol_w_id:24!null ol_d_id:25!null ol_o_id:26!null + │ │ ├── columns: ol_w_id:25!null ol_d_id:26!null ol_o_id:27!null │ │ ├── mapping: - │ │ │ ├── column3:15 => ol_w_id:24 - │ │ │ ├── column2:14 => ol_d_id:25 - │ │ │ └── column1:13 => ol_o_id:26 + │ │ │ ├── column3:15 => ol_w_id:25 + │ │ │ ├── column2:14 => ol_d_id:26 + │ │ │ └── column1:13 => ol_o_id:27 │ │ └── cardinality: [6 - 6] │ └── filters (true) └── f-k-checks-item: order_line(ol_supply_w_id,ol_i_id) -> stock(s_w_id,s_i_id) └── anti-join (lookup stock) - ├── columns: ol_supply_w_id:37!null ol_i_id:38!null - ├── key columns: [37 38] = [40 39] + ├── columns: ol_supply_w_id:38!null ol_i_id:39!null + ├── key columns: [38 39] = [41 40] ├── lookup columns are key ├── cardinality: [0 - 6] ├── with-scan &1 - │ ├── columns: ol_supply_w_id:37!null ol_i_id:38!null + │ ├── columns: ol_supply_w_id:38!null ol_i_id:39!null │ ├── mapping: - │ │ ├── column6:18 => ol_supply_w_id:37 - │ │ └── column5:17 => ol_i_id:38 + │ │ ├── column6:18 => ol_supply_w_id:38 + │ │ └── column5:17 => ol_i_id:39 │ └── cardinality: [6 - 6] └── filters (true) @@ -603,60 +606,71 @@ VALUES insert history ├── columns: ├── insert-mapping: - │ ├── rowid_default:20 => rowid:1 + │ ├── rowid_default:22 => rowid:1 │ ├── column1:12 => history.h_c_id:2 │ ├── column2:13 => history.h_c_d_id:3 │ ├── column3:14 => history.h_c_w_id:4 │ ├── column4:15 => history.h_d_id:5 │ ├── column5:16 => history.h_w_id:6 │ ├── column7:18 => h_date:7 - │ ├── h_amount:21 => history.h_amount:8 - │ └── column8:19 => h_data:9 + │ ├── column6:20 => h_amount:8 + │ └── column8:21 => h_data:9 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:12!null column2:13!null column3:14!null column4:15!null column5:16!null column7:18!null column8:19!null rowid_default:20 h_amount:21!null + │ ├── columns: column1:12!null column2:13!null column3:14!null column4:15!null column5:16!null column7:18!null column6:20 column8:21 rowid_default:22 │ ├── cardinality: [1 - 1] │ ├── volatile │ ├── key: () - │ ├── fd: ()-->(12-16,18-21) - │ └── (1343, 5, 10, 5, 10, '2019-08-26 16:50:41', '8 Kdcgphy3', gen_random_uuid(), 3860.61) + │ ├── fd: ()-->(12-16,18,20-22) + │ └── tuple + │ ├── 1343 + │ ├── 5 + │ ├── 10 + │ ├── 5 + │ ├── 10 + │ ├── '2019-08-26 16:50:41' + │ ├── assignment-cast: DECIMAL(6,2) + │ │ └── 3860.61 + │ ├── assignment-cast: VARCHAR(24) + │ │ └── '8 Kdcgphy3' + │ └── gen_random_uuid() └── f-k-checks ├── f-k-checks-item: history(h_c_w_id,h_c_d_id,h_c_id) -> customer(c_w_id,c_d_id,c_id) │ └── anti-join (lookup customer) - │ ├── columns: h_c_w_id:22!null h_c_d_id:23!null h_c_id:24!null - │ ├── key columns: [22 23 24] = [27 26 25] + │ ├── columns: h_c_w_id:23!null h_c_d_id:24!null h_c_id:25!null + │ ├── key columns: [23 24 25] = [28 27 26] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(22-24) + │ ├── fd: ()-->(23-25) │ ├── with-scan &1 - │ │ ├── columns: h_c_w_id:22!null h_c_d_id:23!null h_c_id:24!null + │ │ ├── columns: h_c_w_id:23!null h_c_d_id:24!null h_c_id:25!null │ │ ├── mapping: - │ │ │ ├── column3:14 => h_c_w_id:22 - │ │ │ ├── column2:13 => h_c_d_id:23 - │ │ │ └── column1:12 => h_c_id:24 + │ │ │ ├── column3:14 => h_c_w_id:23 + │ │ │ ├── column2:13 => h_c_d_id:24 + │ │ │ └── column1:12 => h_c_id:25 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(22-24) + │ │ └── fd: ()-->(23-25) │ └── filters (true) └── f-k-checks-item: history(h_w_id,h_d_id) -> district(d_w_id,d_id) └── anti-join (lookup district) - ├── columns: h_w_id:48!null h_d_id:49!null - ├── key columns: [48 49] = [51 50] + ├── columns: h_w_id:49!null h_d_id:50!null + ├── key columns: [49 50] = [52 51] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(48,49) + ├── fd: ()-->(49,50) ├── with-scan &1 - │ ├── columns: h_w_id:48!null h_d_id:49!null + │ ├── columns: h_w_id:49!null h_d_id:50!null │ ├── mapping: - │ │ ├── column5:16 => h_w_id:48 - │ │ └── column4:15 => h_d_id:49 + │ │ ├── column5:16 => h_w_id:49 + │ │ └── column4:15 => h_d_id:50 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(48,49) + │ └── fd: ()-->(49,50) └── filters (true) # -------------------------------------------------- diff --git a/pkg/sql/opt/xform/testdata/external/tpcc-no-stats b/pkg/sql/opt/xform/testdata/external/tpcc-no-stats index 0fb3c349396e..911431bade11 100644 --- a/pkg/sql/opt/xform/testdata/external/tpcc-no-stats +++ b/pkg/sql/opt/xform/testdata/external/tpcc-no-stats @@ -361,18 +361,18 @@ insert order_line │ ├── column4:16 => ol_number:4 │ ├── column5:17 => order_line.ol_i_id:5 │ ├── column6:18 => order_line.ol_supply_w_id:6 - │ ├── ol_delivery_d_default:22 => ol_delivery_d:7 + │ ├── ol_delivery_d_default:24 => ol_delivery_d:7 │ ├── column7:19 => ol_quantity:8 - │ ├── ol_amount:23 => order_line.ol_amount:9 - │ └── column9:21 => ol_dist_info:10 + │ ├── column8:22 => ol_amount:9 + │ └── column9:23 => ol_dist_info:10 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── project - │ ├── columns: ol_amount:23 ol_delivery_d_default:22 column1:13!null column2:14!null column3:15!null column4:16!null column5:17!null column6:18!null column7:19!null column9:21!null + │ ├── columns: ol_delivery_d_default:24 column8:22 column9:23 column1:13!null column2:14!null column3:15!null column4:16!null column5:17!null column6:18!null column7:19!null │ ├── cardinality: [6 - 6] │ ├── immutable - │ ├── fd: ()-->(22) + │ ├── fd: ()-->(24) │ ├── values │ │ ├── columns: column1:13!null column2:14!null column3:15!null column4:16!null column5:17!null column6:18!null column7:19!null column8:20!null column9:21!null │ │ ├── cardinality: [6 - 6] @@ -383,34 +383,37 @@ insert order_line │ │ ├── (3045, 2, 10, 4, 56624, 0, 6, 273.360000, 'RsaCXoEzmssaF9m9cdLXe0Yh') │ │ └── (3045, 2, 10, 6, 92966, 0, 4, 366.760000, 'saCXoEzmssaF9m9cdLXe0Yhg') │ └── projections - │ ├── crdb_internal.round_decimal_values(column8:20, 2) [as=ol_amount:23, outer=(20), immutable] - │ └── CAST(NULL AS TIMESTAMP) [as=ol_delivery_d_default:22] + │ ├── CAST(NULL AS TIMESTAMP) [as=ol_delivery_d_default:24] + │ ├── assignment-cast: DECIMAL(6,2) [as=column8:22, outer=(20), immutable] + │ │ └── column8:20 + │ └── assignment-cast: CHAR(24) [as=column9:23, outer=(21), immutable] + │ └── column9:21 └── f-k-checks ├── f-k-checks-item: order_line(ol_w_id,ol_d_id,ol_o_id) -> order(o_w_id,o_d_id,o_id) │ └── anti-join (lookup order) - │ ├── columns: ol_w_id:24!null ol_d_id:25!null ol_o_id:26!null - │ ├── key columns: [24 25 26] = [29 28 27] + │ ├── columns: ol_w_id:25!null ol_d_id:26!null ol_o_id:27!null + │ ├── key columns: [25 26 27] = [30 29 28] │ ├── lookup columns are key │ ├── cardinality: [0 - 6] │ ├── with-scan &1 - │ │ ├── columns: ol_w_id:24!null ol_d_id:25!null ol_o_id:26!null + │ │ ├── columns: ol_w_id:25!null ol_d_id:26!null ol_o_id:27!null │ │ ├── mapping: - │ │ │ ├── column3:15 => ol_w_id:24 - │ │ │ ├── column2:14 => ol_d_id:25 - │ │ │ └── column1:13 => ol_o_id:26 + │ │ │ ├── column3:15 => ol_w_id:25 + │ │ │ ├── column2:14 => ol_d_id:26 + │ │ │ └── column1:13 => ol_o_id:27 │ │ └── cardinality: [6 - 6] │ └── filters (true) └── f-k-checks-item: order_line(ol_supply_w_id,ol_i_id) -> stock(s_w_id,s_i_id) └── anti-join (lookup stock) - ├── columns: ol_supply_w_id:37!null ol_i_id:38!null - ├── key columns: [37 38] = [40 39] + ├── columns: ol_supply_w_id:38!null ol_i_id:39!null + ├── key columns: [38 39] = [41 40] ├── lookup columns are key ├── cardinality: [0 - 6] ├── with-scan &1 - │ ├── columns: ol_supply_w_id:37!null ol_i_id:38!null + │ ├── columns: ol_supply_w_id:38!null ol_i_id:39!null │ ├── mapping: - │ │ ├── column6:18 => ol_supply_w_id:37 - │ │ └── column5:17 => ol_i_id:38 + │ │ ├── column6:18 => ol_supply_w_id:38 + │ │ └── column5:17 => ol_i_id:39 │ └── cardinality: [6 - 6] └── filters (true) @@ -597,60 +600,71 @@ VALUES insert history ├── columns: ├── insert-mapping: - │ ├── rowid_default:20 => rowid:1 + │ ├── rowid_default:22 => rowid:1 │ ├── column1:12 => history.h_c_id:2 │ ├── column2:13 => history.h_c_d_id:3 │ ├── column3:14 => history.h_c_w_id:4 │ ├── column4:15 => history.h_d_id:5 │ ├── column5:16 => history.h_w_id:6 │ ├── column7:18 => h_date:7 - │ ├── h_amount:21 => history.h_amount:8 - │ └── column8:19 => h_data:9 + │ ├── column6:20 => h_amount:8 + │ └── column8:21 => h_data:9 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:12!null column2:13!null column3:14!null column4:15!null column5:16!null column7:18!null column8:19!null rowid_default:20 h_amount:21!null + │ ├── columns: column1:12!null column2:13!null column3:14!null column4:15!null column5:16!null column7:18!null column6:20 column8:21 rowid_default:22 │ ├── cardinality: [1 - 1] │ ├── volatile │ ├── key: () - │ ├── fd: ()-->(12-16,18-21) - │ └── (1343, 5, 10, 5, 10, '2019-08-26 16:50:41', '8 Kdcgphy3', gen_random_uuid(), 3860.61) + │ ├── fd: ()-->(12-16,18,20-22) + │ └── tuple + │ ├── 1343 + │ ├── 5 + │ ├── 10 + │ ├── 5 + │ ├── 10 + │ ├── '2019-08-26 16:50:41' + │ ├── assignment-cast: DECIMAL(6,2) + │ │ └── 3860.61 + │ ├── assignment-cast: VARCHAR(24) + │ │ └── '8 Kdcgphy3' + │ └── gen_random_uuid() └── f-k-checks ├── f-k-checks-item: history(h_c_w_id,h_c_d_id,h_c_id) -> customer(c_w_id,c_d_id,c_id) │ └── anti-join (lookup customer) - │ ├── columns: h_c_w_id:22!null h_c_d_id:23!null h_c_id:24!null - │ ├── key columns: [22 23 24] = [27 26 25] + │ ├── columns: h_c_w_id:23!null h_c_d_id:24!null h_c_id:25!null + │ ├── key columns: [23 24 25] = [28 27 26] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(22-24) + │ ├── fd: ()-->(23-25) │ ├── with-scan &1 - │ │ ├── columns: h_c_w_id:22!null h_c_d_id:23!null h_c_id:24!null + │ │ ├── columns: h_c_w_id:23!null h_c_d_id:24!null h_c_id:25!null │ │ ├── mapping: - │ │ │ ├── column3:14 => h_c_w_id:22 - │ │ │ ├── column2:13 => h_c_d_id:23 - │ │ │ └── column1:12 => h_c_id:24 + │ │ │ ├── column3:14 => h_c_w_id:23 + │ │ │ ├── column2:13 => h_c_d_id:24 + │ │ │ └── column1:12 => h_c_id:25 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(22-24) + │ │ └── fd: ()-->(23-25) │ └── filters (true) └── f-k-checks-item: history(h_w_id,h_d_id) -> district(d_w_id,d_id) └── anti-join (lookup district) - ├── columns: h_w_id:48!null h_d_id:49!null - ├── key columns: [48 49] = [51 50] + ├── columns: h_w_id:49!null h_d_id:50!null + ├── key columns: [49 50] = [52 51] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(48,49) + ├── fd: ()-->(49,50) ├── with-scan &1 - │ ├── columns: h_w_id:48!null h_d_id:49!null + │ ├── columns: h_w_id:49!null h_d_id:50!null │ ├── mapping: - │ │ ├── column5:16 => h_w_id:48 - │ │ └── column4:15 => h_d_id:49 + │ │ ├── column5:16 => h_w_id:49 + │ │ └── column4:15 => h_d_id:50 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(48,49) + │ └── fd: ()-->(49,50) └── filters (true) # -------------------------------------------------- diff --git a/pkg/sql/opt/xform/testdata/external/tpce b/pkg/sql/opt/xform/testdata/external/tpce index ca9625df40fd..0a727186301e 100644 --- a/pkg/sql/opt/xform/testdata/external/tpce +++ b/pkg/sql/opt/xform/testdata/external/tpce @@ -552,10 +552,10 @@ update_trade_submitted AS ( SELECT * FROM request_list; ---- with &2 (update_last_trade) - ├── columns: tr_t_id:137!null tr_bid_price:138!null tr_tt_id:139!null tr_qty:140!null + ├── columns: tr_t_id:138!null tr_bid_price:139!null tr_tt_id:140!null tr_qty:141!null ├── volatile, mutations - ├── key: (137) - ├── fd: (137)-->(138-140) + ├── key: (138) + ├── fd: (138)-->(139-141) ├── project │ ├── columns: "?column?":19 │ ├── cardinality: [0 - 1] @@ -592,10 +592,10 @@ with &2 (update_last_trade) │ └── projections │ └── NULL [as="?column?":19] └── with &3 (request_list) - ├── columns: tr_t_id:137!null tr_bid_price:138!null tr_tt_id:139!null tr_qty:140!null + ├── columns: tr_t_id:138!null tr_bid_price:139!null tr_tt_id:140!null tr_qty:141!null ├── volatile, mutations - ├── key: (137) - ├── fd: (137)-->(138-140) + ├── key: (138) + ├── fd: (138)-->(139-141) ├── project │ ├── columns: tr_bid_price:28!null trade_request.tr_t_id:20!null trade_request.tr_tt_id:21!null trade_request.tr_qty:23!null │ ├── immutable @@ -622,10 +622,10 @@ with &2 (update_last_trade) │ └── projections │ └── trade_request.tr_bid_price:24::FLOAT8 [as=tr_bid_price:28, outer=(24), immutable] └── with &4 (delete_trade_request) - ├── columns: tr_t_id:137!null tr_bid_price:138!null tr_tt_id:139!null tr_qty:140!null + ├── columns: tr_t_id:138!null tr_bid_price:139!null tr_tt_id:140!null tr_qty:141!null ├── volatile, mutations - ├── key: (137) - ├── fd: (137)-->(138-140) + ├── key: (138) + ├── fd: (138)-->(139-141) ├── project │ ├── columns: "?column?":50 │ ├── volatile, mutations @@ -661,128 +661,130 @@ with &2 (update_last_trade) │ └── projections │ └── NULL [as="?column?":50] └── with &6 (insert_trade_history) - ├── columns: tr_t_id:137!null tr_bid_price:138!null tr_tt_id:139!null tr_qty:140!null + ├── columns: tr_t_id:138!null tr_bid_price:139!null tr_tt_id:140!null tr_qty:141!null ├── volatile, mutations - ├── key: (137) - ├── fd: (137)-->(138-140) + ├── key: (138) + ├── fd: (138)-->(139-141) ├── project - │ ├── columns: "?column?":85 + │ ├── columns: "?column?":86 │ ├── volatile, mutations - │ ├── fd: ()-->(85) + │ ├── fd: ()-->(86) │ ├── insert trade_history │ │ ├── columns: trade_history.th_t_id:51!null trade_history.th_st_id:53!null │ │ ├── insert-mapping: │ │ │ ├── tr_t_id:56 => trade_history.th_t_id:51 │ │ │ ├── timestamp:61 => th_dts:52 - │ │ │ └── "?column?":60 => trade_history.th_st_id:53 + │ │ │ └── "?column?":62 => trade_history.th_st_id:53 │ │ ├── input binding: &5 │ │ ├── volatile, mutations │ │ ├── key: (51) │ │ ├── fd: ()-->(53) │ │ ├── project - │ │ │ ├── columns: "?column?":60!null timestamp:61!null tr_t_id:56!null + │ │ │ ├── columns: "?column?":62 timestamp:61!null tr_t_id:56!null + │ │ │ ├── immutable │ │ │ ├── key: (56) - │ │ │ ├── fd: ()-->(60,61) + │ │ │ ├── fd: ()-->(61,62) │ │ │ ├── with-scan &3 (request_list) │ │ │ │ ├── columns: tr_t_id:56!null │ │ │ │ ├── mapping: │ │ │ │ │ └── trade_request.tr_t_id:20 => tr_t_id:56 │ │ │ │ └── key: (56) │ │ │ └── projections - │ │ │ ├── 'SBMT' [as="?column?":60] + │ │ │ ├── assignment-cast: VARCHAR(4) [as="?column?":62, immutable] + │ │ │ │ └── 'SBMT' │ │ │ └── '2020-06-15 22:27:42.148484' [as=timestamp:61] │ │ └── f-k-checks │ │ ├── f-k-checks-item: trade_history(th_t_id) -> trade(t_id) │ │ │ └── anti-join (lookup trade) - │ │ │ ├── columns: th_t_id:62!null - │ │ │ ├── key columns: [62] = [63] + │ │ │ ├── columns: th_t_id:63!null + │ │ │ ├── key columns: [63] = [64] │ │ │ ├── lookup columns are key - │ │ │ ├── key: (62) + │ │ │ ├── key: (63) │ │ │ ├── with-scan &5 - │ │ │ │ ├── columns: th_t_id:62!null + │ │ │ │ ├── columns: th_t_id:63!null │ │ │ │ ├── mapping: - │ │ │ │ │ └── tr_t_id:56 => th_t_id:62 - │ │ │ │ └── key: (62) + │ │ │ │ │ └── tr_t_id:56 => th_t_id:63 + │ │ │ │ └── key: (63) │ │ │ └── filters (true) │ │ └── f-k-checks-item: trade_history(th_st_id) -> status_type(st_id) │ │ └── anti-join (lookup status_type) - │ │ ├── columns: th_st_id:80!null - │ │ ├── key columns: [80] = [81] + │ │ ├── columns: th_st_id:81 + │ │ ├── key columns: [81] = [82] │ │ ├── lookup columns are key - │ │ ├── fd: ()-->(80) + │ │ ├── fd: ()-->(81) │ │ ├── with-scan &5 - │ │ │ ├── columns: th_st_id:80!null + │ │ │ ├── columns: th_st_id:81 │ │ │ ├── mapping: - │ │ │ │ └── "?column?":60 => th_st_id:80 - │ │ │ └── fd: ()-->(80) + │ │ │ │ └── "?column?":62 => th_st_id:81 + │ │ │ └── fd: ()-->(81) │ │ └── filters (true) │ └── projections - │ └── NULL [as="?column?":85] + │ └── NULL [as="?column?":86] └── with &8 (update_trade_submitted) - ├── columns: tr_t_id:137!null tr_bid_price:138!null tr_tt_id:139!null tr_qty:140!null + ├── columns: tr_t_id:138!null tr_bid_price:139!null tr_tt_id:140!null tr_qty:141!null ├── volatile, mutations - ├── key: (137) - ├── fd: (137)-->(138-140) + ├── key: (138) + ├── fd: (138)-->(139-141) ├── project - │ ├── columns: "?column?":136 + │ ├── columns: "?column?":137 │ ├── volatile, mutations - │ ├── fd: ()-->(136) + │ ├── fd: ()-->(137) │ ├── update trade - │ │ ├── columns: t_id:86!null - │ │ ├── fetch columns: t_id:103 t_dts:104 trade.t_st_id:105 t_tt_id:106 t_is_cash:107 t_s_symb:108 t_qty:109 t_bid_price:110 t_ca_id:111 t_exec_name:112 t_trade_price:113 t_chrg:114 t_comm:115 t_lifo:117 + │ │ ├── columns: t_id:87!null + │ │ ├── fetch columns: t_id:104 t_dts:105 trade.t_st_id:106 t_tt_id:107 t_is_cash:108 t_s_symb:109 t_qty:110 t_bid_price:111 t_ca_id:112 t_exec_name:113 t_trade_price:114 t_chrg:115 t_comm:116 t_lifo:118 │ │ ├── update-mapping: - │ │ │ ├── t_dts_new:125 => t_dts:87 - │ │ │ └── t_st_id_new:124 => trade.t_st_id:88 + │ │ │ ├── t_dts_new:126 => t_dts:88 + │ │ │ └── t_st_id_new:125 => trade.t_st_id:89 │ │ ├── input binding: &7 │ │ ├── volatile, mutations - │ │ ├── key: (86) + │ │ ├── key: (87) │ │ ├── project - │ │ │ ├── columns: t_st_id_new:124!null t_dts_new:125!null t_id:103!null t_dts:104!null trade.t_st_id:105!null t_tt_id:106!null t_is_cash:107!null t_s_symb:108!null t_qty:109!null t_bid_price:110!null t_ca_id:111!null t_exec_name:112!null t_trade_price:113 t_chrg:114!null t_comm:115!null t_lifo:117!null - │ │ │ ├── key: (103) - │ │ │ ├── fd: ()-->(124,125), (103)-->(104-115,117) + │ │ │ ├── columns: t_st_id_new:125!null t_dts_new:126!null t_id:104!null t_dts:105!null trade.t_st_id:106!null t_tt_id:107!null t_is_cash:108!null t_s_symb:109!null t_qty:110!null t_bid_price:111!null t_ca_id:112!null t_exec_name:113!null t_trade_price:114 t_chrg:115!null t_comm:116!null t_lifo:118!null + │ │ │ ├── key: (104) + │ │ │ ├── fd: ()-->(125,126), (104)-->(105-116,118) │ │ │ ├── project - │ │ │ │ ├── columns: t_id:103!null t_dts:104!null trade.t_st_id:105!null t_tt_id:106!null t_is_cash:107!null t_s_symb:108!null t_qty:109!null t_bid_price:110!null t_ca_id:111!null t_exec_name:112!null t_trade_price:113 t_chrg:114!null t_comm:115!null t_lifo:117!null - │ │ │ │ ├── key: (103) - │ │ │ │ ├── fd: (103)-->(104-115,117) + │ │ │ │ ├── columns: t_id:104!null t_dts:105!null trade.t_st_id:106!null t_tt_id:107!null t_is_cash:108!null t_s_symb:109!null t_qty:110!null t_bid_price:111!null t_ca_id:112!null t_exec_name:113!null t_trade_price:114 t_chrg:115!null t_comm:116!null t_lifo:118!null + │ │ │ │ ├── key: (104) + │ │ │ │ ├── fd: (104)-->(105-116,118) │ │ │ │ └── inner-join (lookup trade) - │ │ │ │ ├── columns: t_id:103!null t_dts:104!null trade.t_st_id:105!null t_tt_id:106!null t_is_cash:107!null t_s_symb:108!null t_qty:109!null t_bid_price:110!null t_ca_id:111!null t_exec_name:112!null t_trade_price:113 t_chrg:114!null t_comm:115!null t_lifo:117!null tr_t_id:120!null - │ │ │ │ ├── key columns: [120] = [103] + │ │ │ │ ├── columns: t_id:104!null t_dts:105!null trade.t_st_id:106!null t_tt_id:107!null t_is_cash:108!null t_s_symb:109!null t_qty:110!null t_bid_price:111!null t_ca_id:112!null t_exec_name:113!null t_trade_price:114 t_chrg:115!null t_comm:116!null t_lifo:118!null tr_t_id:121!null + │ │ │ │ ├── key columns: [121] = [104] │ │ │ │ ├── lookup columns are key - │ │ │ │ ├── key: (120) - │ │ │ │ ├── fd: (103)-->(104-115,117), (103)==(120), (120)==(103) + │ │ │ │ ├── key: (121) + │ │ │ │ ├── fd: (104)-->(105-116,118), (104)==(121), (121)==(104) │ │ │ │ ├── with-scan &3 (request_list) - │ │ │ │ │ ├── columns: tr_t_id:120!null + │ │ │ │ │ ├── columns: tr_t_id:121!null │ │ │ │ │ ├── mapping: - │ │ │ │ │ │ └── trade_request.tr_t_id:20 => tr_t_id:120 - │ │ │ │ │ └── key: (120) + │ │ │ │ │ │ └── trade_request.tr_t_id:20 => tr_t_id:121 + │ │ │ │ │ └── key: (121) │ │ │ │ └── filters (true) │ │ │ └── projections - │ │ │ ├── 'SBMT' [as=t_st_id_new:124] - │ │ │ └── '2020-06-15 22:27:42.148484' [as=t_dts_new:125] + │ │ │ ├── 'SBMT' [as=t_st_id_new:125] + │ │ │ └── '2020-06-15 22:27:42.148484' [as=t_dts_new:126] │ │ └── f-k-checks │ │ └── f-k-checks-item: trade(t_st_id) -> status_type(st_id) │ │ └── anti-join (lookup status_type) - │ │ ├── columns: t_st_id:131!null - │ │ ├── key columns: [131] = [132] + │ │ ├── columns: t_st_id:132!null + │ │ ├── key columns: [132] = [133] │ │ ├── lookup columns are key - │ │ ├── fd: ()-->(131) + │ │ ├── fd: ()-->(132) │ │ ├── with-scan &7 - │ │ │ ├── columns: t_st_id:131!null + │ │ │ ├── columns: t_st_id:132!null │ │ │ ├── mapping: - │ │ │ │ └── t_st_id_new:124 => t_st_id:131 - │ │ │ └── fd: ()-->(131) + │ │ │ │ └── t_st_id_new:125 => t_st_id:132 + │ │ │ └── fd: ()-->(132) │ │ └── filters (true) │ └── projections - │ └── NULL [as="?column?":136] + │ └── NULL [as="?column?":137] └── with-scan &3 (request_list) - ├── columns: tr_t_id:137!null tr_bid_price:138!null tr_tt_id:139!null tr_qty:140!null + ├── columns: tr_t_id:138!null tr_bid_price:139!null tr_tt_id:140!null tr_qty:141!null ├── mapping: - │ ├── trade_request.tr_t_id:20 => tr_t_id:137 - │ ├── tr_bid_price:28 => tr_bid_price:138 - │ ├── trade_request.tr_tt_id:21 => tr_tt_id:139 - │ └── trade_request.tr_qty:23 => tr_qty:140 - ├── key: (137) - └── fd: (137)-->(138-140) + │ ├── trade_request.tr_t_id:20 => tr_t_id:138 + │ ├── tr_bid_price:28 => tr_bid_price:139 + │ ├── trade_request.tr_tt_id:21 => tr_tt_id:140 + │ └── trade_request.tr_qty:23 => tr_qty:141 + ├── key: (138) + └── fd: (138)-->(139-141) # -------------------------------------------------- # T4 @@ -2921,183 +2923,225 @@ insert_trade_history AS ( SELECT 1; ---- with &2 (insert_trade) - ├── columns: "?column?":116!null + ├── columns: "?column?":121!null ├── cardinality: [1 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(116) + ├── fd: ()-->(121) ├── project - │ ├── columns: "?column?":83!null + │ ├── columns: "?column?":87!null │ ├── cardinality: [1 - 1] │ ├── volatile, mutations │ ├── key: () - │ ├── fd: ()-->(83) + │ ├── fd: ()-->(87) │ ├── insert trade │ │ ├── columns: t_id:1!null │ │ ├── insert-mapping: │ │ │ ├── column1:18 => t_id:1 │ │ │ ├── column2:19 => t_dts:2 - │ │ │ ├── column3:20 => trade.t_st_id:3 - │ │ │ ├── column4:21 => trade.t_tt_id:4 + │ │ │ ├── column3:33 => trade.t_st_id:3 + │ │ │ ├── column4:34 => trade.t_tt_id:4 │ │ │ ├── column5:22 => t_is_cash:5 - │ │ │ ├── column6:23 => trade.t_s_symb:6 - │ │ │ ├── column7:24 => t_qty:7 - │ │ │ ├── t_bid_price:33 => trade.t_bid_price:8 + │ │ │ ├── column6:35 => trade.t_s_symb:6 + │ │ │ ├── column7:36 => t_qty:7 + │ │ │ ├── column8:37 => t_bid_price:8 │ │ │ ├── column9:26 => trade.t_ca_id:9 - │ │ │ ├── column10:27 => t_exec_name:10 - │ │ │ ├── t_trade_price:34 => trade.t_trade_price:11 - │ │ │ ├── t_chrg:35 => trade.t_chrg:12 - │ │ │ ├── t_comm:36 => trade.t_comm:13 - │ │ │ ├── t_tax:37 => trade.t_tax:14 + │ │ │ ├── column10:38 => t_exec_name:10 + │ │ │ ├── column11:28 => t_trade_price:11 + │ │ │ ├── column12:39 => t_chrg:12 + │ │ │ ├── column13:40 => t_comm:13 + │ │ │ ├── column14:41 => t_tax:14 │ │ │ └── column15:32 => t_lifo:15 - │ │ ├── check columns: check1:38 check2:39 check3:40 check4:41 check5:42 + │ │ ├── check columns: check1:42 check2:43 check3:44 check4:45 check5:46 │ │ ├── input binding: &1 │ │ ├── cardinality: [1 - 1] │ │ ├── volatile, mutations │ │ ├── key: () │ │ ├── fd: ()-->(1) - │ │ ├── values - │ │ │ ├── columns: column1:18!null column2:19!null column3:20!null column4:21!null column5:22!null column6:23!null column7:24!null column9:26!null column10:27!null column15:32!null t_bid_price:33!null t_trade_price:34 t_chrg:35!null t_comm:36!null t_tax:37!null check1:38!null check2:39!null check3:40!null check4:41!null check5:42!null + │ │ ├── project + │ │ │ ├── columns: check1:42 check2:43 check3:44 check4:45 check5:46 column1:18!null column2:19!null column5:22!null column9:26!null column11:28 column15:32!null column3:33 column4:34 column6:35 column7:36 column8:37 column10:38 column12:39 column13:40 column14:41 │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── immutable │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(18-24,26,27,32-42) - │ │ │ └── (0, '2020-06-17 22:27:42.148484', 'SBMT', 'TMB', true, 'SYMB', 10, 0, 'Name', true, 1E+2, NULL, 1, 0, 0, true, true, true, true, true) + │ │ │ ├── fd: ()-->(18,19,22,26,28,32-46) + │ │ │ ├── values + │ │ │ │ ├── columns: column1:18!null column2:19!null column5:22!null column9:26!null column11:28 column15:32!null column3:33 column4:34 column6:35 column7:36 column8:37 column10:38 column12:39 column13:40 column14:41 + │ │ │ │ ├── cardinality: [1 - 1] + │ │ │ │ ├── immutable + │ │ │ │ ├── key: () + │ │ │ │ ├── fd: ()-->(18,19,22,26,28,32-41) + │ │ │ │ └── tuple + │ │ │ │ ├── 0 + │ │ │ │ ├── '2020-06-17 22:27:42.148484' + │ │ │ │ ├── true + │ │ │ │ ├── 0 + │ │ │ │ ├── CAST(NULL AS DECIMAL(8,2)) + │ │ │ │ ├── true + │ │ │ │ ├── assignment-cast: VARCHAR(4) + │ │ │ │ │ └── 'SBMT' + │ │ │ │ ├── assignment-cast: VARCHAR(3) + │ │ │ │ │ └── 'TMB' + │ │ │ │ ├── assignment-cast: VARCHAR(15) + │ │ │ │ │ └── 'SYMB' + │ │ │ │ ├── assignment-cast: INT4 + │ │ │ │ │ └── 10 + │ │ │ │ ├── assignment-cast: DECIMAL(8,2) + │ │ │ │ │ └── 1E+2 + │ │ │ │ ├── assignment-cast: VARCHAR(49) + │ │ │ │ │ └── 'Name' + │ │ │ │ ├── assignment-cast: DECIMAL(10,2) + │ │ │ │ │ └── 1 + │ │ │ │ ├── assignment-cast: DECIMAL(10,2) + │ │ │ │ │ └── 0 + │ │ │ │ └── assignment-cast: DECIMAL(10,2) + │ │ │ │ └── 0 + │ │ │ └── projections + │ │ │ ├── column7:36 > 0 [as=check1:42, outer=(36)] + │ │ │ ├── column8:37 > 0 [as=check2:43, outer=(37), immutable] + │ │ │ ├── column12:39 >= 0 [as=check3:44, outer=(39), immutable] + │ │ │ ├── column13:40 >= 0 [as=check4:45, outer=(40), immutable] + │ │ │ └── column14:41 >= 0 [as=check5:46, outer=(41), immutable] │ │ └── f-k-checks │ │ ├── f-k-checks-item: trade(t_st_id) -> status_type(st_id) │ │ │ └── anti-join (lookup status_type) - │ │ │ ├── columns: t_st_id:43!null - │ │ │ ├── key columns: [43] = [44] + │ │ │ ├── columns: t_st_id:47 + │ │ │ ├── key columns: [47] = [48] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(43) + │ │ │ ├── fd: ()-->(47) │ │ │ ├── with-scan &1 - │ │ │ │ ├── columns: t_st_id:43!null + │ │ │ │ ├── columns: t_st_id:47 │ │ │ │ ├── mapping: - │ │ │ │ │ └── column3:20 => t_st_id:43 + │ │ │ │ │ └── column3:33 => t_st_id:47 │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(43) + │ │ │ │ └── fd: ()-->(47) │ │ │ └── filters (true) │ │ ├── f-k-checks-item: trade(t_tt_id) -> trade_type(tt_id) │ │ │ └── anti-join (lookup trade_type) - │ │ │ ├── columns: t_tt_id:48!null - │ │ │ ├── key columns: [48] = [49] + │ │ │ ├── columns: t_tt_id:52 + │ │ │ ├── key columns: [52] = [53] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(48) + │ │ │ ├── fd: ()-->(52) │ │ │ ├── with-scan &1 - │ │ │ │ ├── columns: t_tt_id:48!null + │ │ │ │ ├── columns: t_tt_id:52 │ │ │ │ ├── mapping: - │ │ │ │ │ └── column4:21 => t_tt_id:48 + │ │ │ │ │ └── column4:34 => t_tt_id:52 │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(48) + │ │ │ │ └── fd: ()-->(52) │ │ │ └── filters (true) │ │ ├── f-k-checks-item: trade(t_s_symb) -> security(s_symb) │ │ │ └── anti-join (lookup security) - │ │ │ ├── columns: t_s_symb:55!null - │ │ │ ├── key columns: [55] = [56] + │ │ │ ├── columns: t_s_symb:59 + │ │ │ ├── key columns: [59] = [60] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(55) + │ │ │ ├── fd: ()-->(59) │ │ │ ├── with-scan &1 - │ │ │ │ ├── columns: t_s_symb:55!null + │ │ │ │ ├── columns: t_s_symb:59 │ │ │ │ ├── mapping: - │ │ │ │ │ └── column6:23 => t_s_symb:55 + │ │ │ │ │ └── column6:35 => t_s_symb:59 │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(55) + │ │ │ │ └── fd: ()-->(59) │ │ │ └── filters (true) │ │ └── f-k-checks-item: trade(t_ca_id) -> customer_account(ca_id) │ │ └── anti-join (lookup customer_account) - │ │ ├── columns: t_ca_id:74!null - │ │ ├── key columns: [74] = [75] + │ │ ├── columns: t_ca_id:78!null + │ │ ├── key columns: [78] = [79] │ │ ├── lookup columns are key │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(74) + │ │ ├── fd: ()-->(78) │ │ ├── with-scan &1 - │ │ │ ├── columns: t_ca_id:74!null + │ │ │ ├── columns: t_ca_id:78!null │ │ │ ├── mapping: - │ │ │ │ └── column9:26 => t_ca_id:74 + │ │ │ │ └── column9:26 => t_ca_id:78 │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ └── fd: ()-->(74) + │ │ │ └── fd: ()-->(78) │ │ └── filters (true) │ └── projections - │ └── 1 [as="?column?":83] + │ └── 1 [as="?column?":87] └── with &4 (insert_trade_history) - ├── columns: "?column?":116!null + ├── columns: "?column?":121!null ├── cardinality: [1 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(116) + ├── fd: ()-->(121) ├── project - │ ├── columns: "?column?":115!null + │ ├── columns: "?column?":120!null │ ├── cardinality: [1 - 1] │ ├── volatile, mutations │ ├── key: () - │ ├── fd: ()-->(115) + │ ├── fd: ()-->(120) │ ├── insert trade_history - │ │ ├── columns: trade_history.th_t_id:84!null trade_history.th_st_id:86!null + │ │ ├── columns: trade_history.th_t_id:88!null trade_history.th_st_id:90!null │ │ ├── insert-mapping: - │ │ │ ├── column1:89 => trade_history.th_t_id:84 - │ │ │ ├── column2:90 => th_dts:85 - │ │ │ └── column3:91 => trade_history.th_st_id:86 + │ │ │ ├── column1:93 => trade_history.th_t_id:88 + │ │ │ ├── column2:94 => th_dts:89 + │ │ │ └── column3:96 => trade_history.th_st_id:90 │ │ ├── input binding: &3 │ │ ├── cardinality: [1 - 1] │ │ ├── volatile, mutations │ │ ├── key: () - │ │ ├── fd: ()-->(84,86) + │ │ ├── fd: ()-->(88,90) │ │ ├── values - │ │ │ ├── columns: column1:89!null column2:90!null column3:91!null + │ │ │ ├── columns: column1:93!null column2:94!null column3:96 │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── immutable │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(89-91) - │ │ │ └── (0, '2020-06-15 22:27:42.148484', 'SBMT') + │ │ │ ├── fd: ()-->(93,94,96) + │ │ │ └── tuple + │ │ │ ├── 0 + │ │ │ ├── '2020-06-15 22:27:42.148484' + │ │ │ └── assignment-cast: VARCHAR(4) + │ │ │ └── 'SBMT' │ │ └── f-k-checks │ │ ├── f-k-checks-item: trade_history(th_t_id) -> trade(t_id) │ │ │ └── anti-join (lookup trade) - │ │ │ ├── columns: th_t_id:92!null - │ │ │ ├── key columns: [92] = [93] + │ │ │ ├── columns: th_t_id:97!null + │ │ │ ├── key columns: [97] = [98] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(92) + │ │ │ ├── fd: ()-->(97) │ │ │ ├── with-scan &3 - │ │ │ │ ├── columns: th_t_id:92!null + │ │ │ │ ├── columns: th_t_id:97!null │ │ │ │ ├── mapping: - │ │ │ │ │ └── column1:89 => th_t_id:92 + │ │ │ │ │ └── column1:93 => th_t_id:97 │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(92) + │ │ │ │ └── fd: ()-->(97) │ │ │ └── filters (true) │ │ └── f-k-checks-item: trade_history(th_st_id) -> status_type(st_id) │ │ └── anti-join (lookup status_type) - │ │ ├── columns: th_st_id:110!null - │ │ ├── key columns: [110] = [111] + │ │ ├── columns: th_st_id:115 + │ │ ├── key columns: [115] = [116] │ │ ├── lookup columns are key │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(110) + │ │ ├── fd: ()-->(115) │ │ ├── with-scan &3 - │ │ │ ├── columns: th_st_id:110!null + │ │ │ ├── columns: th_st_id:115 │ │ │ ├── mapping: - │ │ │ │ └── column3:91 => th_st_id:110 + │ │ │ │ └── column3:96 => th_st_id:115 │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ └── fd: ()-->(110) + │ │ │ └── fd: ()-->(115) │ │ └── filters (true) │ └── projections - │ └── 1 [as="?column?":115] + │ └── 1 [as="?column?":120] └── values - ├── columns: "?column?":116!null + ├── columns: "?column?":121!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(116) + ├── fd: ()-->(121) └── (1,) # Q12 @@ -3171,284 +3215,347 @@ insert_trade_request AS ( SELECT 1; ---- with &2 (insert_trade) - ├── columns: "?column?":187!null + ├── columns: "?column?":195!null ├── cardinality: [1 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(187) + ├── fd: ()-->(195) ├── project - │ ├── columns: "?column?":83!null + │ ├── columns: "?column?":87!null │ ├── cardinality: [1 - 1] │ ├── volatile, mutations │ ├── key: () - │ ├── fd: ()-->(83) + │ ├── fd: ()-->(87) │ ├── insert trade │ │ ├── columns: t_id:1!null │ │ ├── insert-mapping: │ │ │ ├── column1:18 => t_id:1 │ │ │ ├── column2:19 => t_dts:2 - │ │ │ ├── column3:20 => trade.t_st_id:3 - │ │ │ ├── column4:21 => trade.t_tt_id:4 + │ │ │ ├── column3:33 => trade.t_st_id:3 + │ │ │ ├── column4:34 => trade.t_tt_id:4 │ │ │ ├── column5:22 => t_is_cash:5 - │ │ │ ├── column6:23 => trade.t_s_symb:6 - │ │ │ ├── column7:24 => t_qty:7 - │ │ │ ├── t_bid_price:33 => trade.t_bid_price:8 + │ │ │ ├── column6:35 => trade.t_s_symb:6 + │ │ │ ├── column7:36 => t_qty:7 + │ │ │ ├── column8:37 => t_bid_price:8 │ │ │ ├── column9:26 => trade.t_ca_id:9 - │ │ │ ├── column10:27 => t_exec_name:10 - │ │ │ ├── t_trade_price:34 => trade.t_trade_price:11 - │ │ │ ├── t_chrg:35 => trade.t_chrg:12 - │ │ │ ├── t_comm:36 => trade.t_comm:13 - │ │ │ ├── t_tax:37 => trade.t_tax:14 + │ │ │ ├── column10:38 => t_exec_name:10 + │ │ │ ├── column11:28 => t_trade_price:11 + │ │ │ ├── column12:39 => t_chrg:12 + │ │ │ ├── column13:40 => t_comm:13 + │ │ │ ├── column14:41 => t_tax:14 │ │ │ └── column15:32 => t_lifo:15 - │ │ ├── check columns: check1:38 check2:39 check3:40 check4:41 check5:42 + │ │ ├── check columns: check1:42 check2:43 check3:44 check4:45 check5:46 │ │ ├── input binding: &1 │ │ ├── cardinality: [1 - 1] │ │ ├── volatile, mutations │ │ ├── key: () │ │ ├── fd: ()-->(1) - │ │ ├── values - │ │ │ ├── columns: column1:18!null column2:19!null column3:20!null column4:21!null column5:22!null column6:23!null column7:24!null column9:26!null column10:27!null column15:32!null t_bid_price:33!null t_trade_price:34 t_chrg:35!null t_comm:36!null t_tax:37!null check1:38!null check2:39!null check3:40!null check4:41!null check5:42!null + │ │ ├── project + │ │ │ ├── columns: check1:42 check2:43 check3:44 check4:45 check5:46 column1:18!null column2:19!null column5:22!null column9:26!null column11:28 column15:32!null column3:33 column4:34 column6:35 column7:36 column8:37 column10:38 column12:39 column13:40 column14:41 │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── immutable │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(18-24,26,27,32-42) - │ │ │ └── (0, '2020-06-17 22:27:42.148484', 'SBMT', 'TMB', true, 'SYMB', 10, 0, 'Name', true, 1E+2, NULL, 1, 0, 0, true, true, true, true, true) + │ │ │ ├── fd: ()-->(18,19,22,26,28,32-46) + │ │ │ ├── values + │ │ │ │ ├── columns: column1:18!null column2:19!null column5:22!null column9:26!null column11:28 column15:32!null column3:33 column4:34 column6:35 column7:36 column8:37 column10:38 column12:39 column13:40 column14:41 + │ │ │ │ ├── cardinality: [1 - 1] + │ │ │ │ ├── immutable + │ │ │ │ ├── key: () + │ │ │ │ ├── fd: ()-->(18,19,22,26,28,32-41) + │ │ │ │ └── tuple + │ │ │ │ ├── 0 + │ │ │ │ ├── '2020-06-17 22:27:42.148484' + │ │ │ │ ├── true + │ │ │ │ ├── 0 + │ │ │ │ ├── CAST(NULL AS DECIMAL(8,2)) + │ │ │ │ ├── true + │ │ │ │ ├── assignment-cast: VARCHAR(4) + │ │ │ │ │ └── 'SBMT' + │ │ │ │ ├── assignment-cast: VARCHAR(3) + │ │ │ │ │ └── 'TMB' + │ │ │ │ ├── assignment-cast: VARCHAR(15) + │ │ │ │ │ └── 'SYMB' + │ │ │ │ ├── assignment-cast: INT4 + │ │ │ │ │ └── 10 + │ │ │ │ ├── assignment-cast: DECIMAL(8,2) + │ │ │ │ │ └── 1E+2 + │ │ │ │ ├── assignment-cast: VARCHAR(49) + │ │ │ │ │ └── 'Name' + │ │ │ │ ├── assignment-cast: DECIMAL(10,2) + │ │ │ │ │ └── 1 + │ │ │ │ ├── assignment-cast: DECIMAL(10,2) + │ │ │ │ │ └── 0 + │ │ │ │ └── assignment-cast: DECIMAL(10,2) + │ │ │ │ └── 0 + │ │ │ └── projections + │ │ │ ├── column7:36 > 0 [as=check1:42, outer=(36)] + │ │ │ ├── column8:37 > 0 [as=check2:43, outer=(37), immutable] + │ │ │ ├── column12:39 >= 0 [as=check3:44, outer=(39), immutable] + │ │ │ ├── column13:40 >= 0 [as=check4:45, outer=(40), immutable] + │ │ │ └── column14:41 >= 0 [as=check5:46, outer=(41), immutable] │ │ └── f-k-checks │ │ ├── f-k-checks-item: trade(t_st_id) -> status_type(st_id) │ │ │ └── anti-join (lookup status_type) - │ │ │ ├── columns: t_st_id:43!null - │ │ │ ├── key columns: [43] = [44] + │ │ │ ├── columns: t_st_id:47 + │ │ │ ├── key columns: [47] = [48] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(43) + │ │ │ ├── fd: ()-->(47) │ │ │ ├── with-scan &1 - │ │ │ │ ├── columns: t_st_id:43!null + │ │ │ │ ├── columns: t_st_id:47 │ │ │ │ ├── mapping: - │ │ │ │ │ └── column3:20 => t_st_id:43 + │ │ │ │ │ └── column3:33 => t_st_id:47 │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(43) + │ │ │ │ └── fd: ()-->(47) │ │ │ └── filters (true) │ │ ├── f-k-checks-item: trade(t_tt_id) -> trade_type(tt_id) │ │ │ └── anti-join (lookup trade_type) - │ │ │ ├── columns: t_tt_id:48!null - │ │ │ ├── key columns: [48] = [49] + │ │ │ ├── columns: t_tt_id:52 + │ │ │ ├── key columns: [52] = [53] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(48) + │ │ │ ├── fd: ()-->(52) │ │ │ ├── with-scan &1 - │ │ │ │ ├── columns: t_tt_id:48!null + │ │ │ │ ├── columns: t_tt_id:52 │ │ │ │ ├── mapping: - │ │ │ │ │ └── column4:21 => t_tt_id:48 + │ │ │ │ │ └── column4:34 => t_tt_id:52 │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(48) + │ │ │ │ └── fd: ()-->(52) │ │ │ └── filters (true) │ │ ├── f-k-checks-item: trade(t_s_symb) -> security(s_symb) │ │ │ └── anti-join (lookup security) - │ │ │ ├── columns: t_s_symb:55!null - │ │ │ ├── key columns: [55] = [56] + │ │ │ ├── columns: t_s_symb:59 + │ │ │ ├── key columns: [59] = [60] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(55) + │ │ │ ├── fd: ()-->(59) │ │ │ ├── with-scan &1 - │ │ │ │ ├── columns: t_s_symb:55!null + │ │ │ │ ├── columns: t_s_symb:59 │ │ │ │ ├── mapping: - │ │ │ │ │ └── column6:23 => t_s_symb:55 + │ │ │ │ │ └── column6:35 => t_s_symb:59 │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(55) + │ │ │ │ └── fd: ()-->(59) │ │ │ └── filters (true) │ │ └── f-k-checks-item: trade(t_ca_id) -> customer_account(ca_id) │ │ └── anti-join (lookup customer_account) - │ │ ├── columns: t_ca_id:74!null - │ │ ├── key columns: [74] = [75] + │ │ ├── columns: t_ca_id:78!null + │ │ ├── key columns: [78] = [79] │ │ ├── lookup columns are key │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(74) + │ │ ├── fd: ()-->(78) │ │ ├── with-scan &1 - │ │ │ ├── columns: t_ca_id:74!null + │ │ │ ├── columns: t_ca_id:78!null │ │ │ ├── mapping: - │ │ │ │ └── column9:26 => t_ca_id:74 + │ │ │ │ └── column9:26 => t_ca_id:78 │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ └── fd: ()-->(74) + │ │ │ └── fd: ()-->(78) │ │ └── filters (true) │ └── projections - │ └── 1 [as="?column?":83] + │ └── 1 [as="?column?":87] └── with &4 (insert_trade_history) - ├── columns: "?column?":187!null + ├── columns: "?column?":195!null ├── cardinality: [1 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(187) + ├── fd: ()-->(195) ├── project - │ ├── columns: "?column?":115!null + │ ├── columns: "?column?":120!null │ ├── cardinality: [1 - 1] │ ├── volatile, mutations │ ├── key: () - │ ├── fd: ()-->(115) + │ ├── fd: ()-->(120) │ ├── insert trade_history - │ │ ├── columns: trade_history.th_t_id:84!null trade_history.th_st_id:86!null + │ │ ├── columns: trade_history.th_t_id:88!null trade_history.th_st_id:90!null │ │ ├── insert-mapping: - │ │ │ ├── column1:89 => trade_history.th_t_id:84 - │ │ │ ├── column2:90 => th_dts:85 - │ │ │ └── column3:91 => trade_history.th_st_id:86 + │ │ │ ├── column1:93 => trade_history.th_t_id:88 + │ │ │ ├── column2:94 => th_dts:89 + │ │ │ └── column3:96 => trade_history.th_st_id:90 │ │ ├── input binding: &3 │ │ ├── cardinality: [1 - 1] │ │ ├── volatile, mutations │ │ ├── key: () - │ │ ├── fd: ()-->(84,86) + │ │ ├── fd: ()-->(88,90) │ │ ├── values - │ │ │ ├── columns: column1:89!null column2:90!null column3:91!null + │ │ │ ├── columns: column1:93!null column2:94!null column3:96 │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── immutable │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(89-91) - │ │ │ └── (0, '2020-06-15 22:27:42.148484', 'SBMT') + │ │ │ ├── fd: ()-->(93,94,96) + │ │ │ └── tuple + │ │ │ ├── 0 + │ │ │ ├── '2020-06-15 22:27:42.148484' + │ │ │ └── assignment-cast: VARCHAR(4) + │ │ │ └── 'SBMT' │ │ └── f-k-checks │ │ ├── f-k-checks-item: trade_history(th_t_id) -> trade(t_id) │ │ │ └── anti-join (lookup trade) - │ │ │ ├── columns: th_t_id:92!null - │ │ │ ├── key columns: [92] = [93] + │ │ │ ├── columns: th_t_id:97!null + │ │ │ ├── key columns: [97] = [98] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(92) + │ │ │ ├── fd: ()-->(97) │ │ │ ├── with-scan &3 - │ │ │ │ ├── columns: th_t_id:92!null + │ │ │ │ ├── columns: th_t_id:97!null │ │ │ │ ├── mapping: - │ │ │ │ │ └── column1:89 => th_t_id:92 + │ │ │ │ │ └── column1:93 => th_t_id:97 │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(92) + │ │ │ │ └── fd: ()-->(97) │ │ │ └── filters (true) │ │ └── f-k-checks-item: trade_history(th_st_id) -> status_type(st_id) │ │ └── anti-join (lookup status_type) - │ │ ├── columns: th_st_id:110!null - │ │ ├── key columns: [110] = [111] + │ │ ├── columns: th_st_id:115 + │ │ ├── key columns: [115] = [116] │ │ ├── lookup columns are key │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(110) + │ │ ├── fd: ()-->(115) │ │ ├── with-scan &3 - │ │ │ ├── columns: th_st_id:110!null + │ │ │ ├── columns: th_st_id:115 │ │ │ ├── mapping: - │ │ │ │ └── column3:91 => th_st_id:110 + │ │ │ │ └── column3:96 => th_st_id:115 │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ └── fd: ()-->(110) + │ │ │ └── fd: ()-->(115) │ │ └── filters (true) │ └── projections - │ └── 1 [as="?column?":115] + │ └── 1 [as="?column?":120] └── with &6 (insert_trade_request) - ├── columns: "?column?":187!null + ├── columns: "?column?":195!null ├── cardinality: [1 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(187) + ├── fd: ()-->(195) ├── project - │ ├── columns: "?column?":186!null + │ ├── columns: "?column?":194!null │ ├── cardinality: [1 - 1] │ ├── volatile, mutations │ ├── key: () - │ ├── fd: ()-->(186) + │ ├── fd: ()-->(194) │ ├── insert trade_request - │ │ ├── columns: trade_request.tr_t_id:116!null + │ │ ├── columns: trade_request.tr_t_id:121!null │ │ ├── insert-mapping: - │ │ │ ├── column1:124 => trade_request.tr_t_id:116 - │ │ │ ├── column2:125 => trade_request.tr_tt_id:117 - │ │ │ ├── column3:126 => trade_request.tr_s_symb:118 - │ │ │ ├── column4:127 => tr_qty:119 - │ │ │ ├── tr_bid_price:130 => trade_request.tr_bid_price:120 - │ │ │ └── column6:129 => trade_request.tr_b_id:121 - │ │ ├── check columns: check1:131 check2:132 - │ │ ├── partial index put columns: partial_index_put1:133 + │ │ │ ├── column1:129 => trade_request.tr_t_id:121 + │ │ │ ├── column2:135 => trade_request.tr_tt_id:122 + │ │ │ ├── column3:136 => trade_request.tr_s_symb:123 + │ │ │ ├── column4:137 => tr_qty:124 + │ │ │ ├── column5:138 => tr_bid_price:125 + │ │ │ └── column6:134 => trade_request.tr_b_id:126 + │ │ ├── check columns: check1:139 check2:140 + │ │ ├── partial index put columns: partial_index_put1:141 │ │ ├── input binding: &5 │ │ ├── cardinality: [1 - 1] │ │ ├── volatile, mutations │ │ ├── key: () - │ │ ├── fd: ()-->(116) - │ │ ├── values - │ │ │ ├── columns: column1:124!null column2:125!null column3:126!null column4:127!null column6:129!null tr_bid_price:130!null check1:131!null check2:132!null partial_index_put1:133!null + │ │ ├── fd: ()-->(121) + │ │ ├── project + │ │ │ ├── columns: partial_index_put1:141 check1:139 check2:140 column1:129!null column6:134!null column2:135 column3:136 column4:137 column5:138 │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── immutable │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(124-127,129-133) - │ │ │ └── (0, 'TMB', 'SYMB', 10, 0, 1E+2, true, true, false) + │ │ │ ├── fd: ()-->(129,134-141) + │ │ │ ├── values + │ │ │ │ ├── columns: column1:129!null column6:134!null column2:135 column3:136 column4:137 column5:138 + │ │ │ │ ├── cardinality: [1 - 1] + │ │ │ │ ├── immutable + │ │ │ │ ├── key: () + │ │ │ │ ├── fd: ()-->(129,134-138) + │ │ │ │ └── tuple + │ │ │ │ ├── 0 + │ │ │ │ ├── 0 + │ │ │ │ ├── assignment-cast: VARCHAR(3) + │ │ │ │ │ └── 'TMB' + │ │ │ │ ├── assignment-cast: VARCHAR(15) + │ │ │ │ │ └── 'SYMB' + │ │ │ │ ├── assignment-cast: INT4 + │ │ │ │ │ └── 10 + │ │ │ │ └── assignment-cast: DECIMAL(8,2) + │ │ │ │ └── 1E+2 + │ │ │ └── projections + │ │ │ ├── column2:135 IN ('TLB', 'TLS', 'TSL') [as=partial_index_put1:141, outer=(135)] + │ │ │ ├── column4:137 > 0 [as=check1:139, outer=(137)] + │ │ │ └── column5:138 > 0 [as=check2:140, outer=(138), immutable] │ │ └── f-k-checks │ │ ├── f-k-checks-item: trade_request(tr_t_id) -> trade(t_id) │ │ │ └── anti-join (lookup trade) - │ │ │ ├── columns: tr_t_id:134!null - │ │ │ ├── key columns: [134] = [135] + │ │ │ ├── columns: tr_t_id:142!null + │ │ │ ├── key columns: [142] = [143] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(134) + │ │ │ ├── fd: ()-->(142) │ │ │ ├── with-scan &5 - │ │ │ │ ├── columns: tr_t_id:134!null + │ │ │ │ ├── columns: tr_t_id:142!null │ │ │ │ ├── mapping: - │ │ │ │ │ └── column1:124 => tr_t_id:134 + │ │ │ │ │ └── column1:129 => tr_t_id:142 │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(134) + │ │ │ │ └── fd: ()-->(142) │ │ │ └── filters (true) │ │ ├── f-k-checks-item: trade_request(tr_tt_id) -> trade_type(tt_id) │ │ │ └── anti-join (lookup trade_type) - │ │ │ ├── columns: tr_tt_id:152!null - │ │ │ ├── key columns: [152] = [153] + │ │ │ ├── columns: tr_tt_id:160 + │ │ │ ├── key columns: [160] = [161] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(152) + │ │ │ ├── fd: ()-->(160) │ │ │ ├── with-scan &5 - │ │ │ │ ├── columns: tr_tt_id:152!null + │ │ │ │ ├── columns: tr_tt_id:160 │ │ │ │ ├── mapping: - │ │ │ │ │ └── column2:125 => tr_tt_id:152 + │ │ │ │ │ └── column2:135 => tr_tt_id:160 │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(152) + │ │ │ │ └── fd: ()-->(160) │ │ │ └── filters (true) │ │ ├── f-k-checks-item: trade_request(tr_s_symb) -> security(s_symb) │ │ │ └── anti-join (lookup security) - │ │ │ ├── columns: tr_s_symb:159!null - │ │ │ ├── key columns: [159] = [160] + │ │ │ ├── columns: tr_s_symb:167 + │ │ │ ├── key columns: [167] = [168] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(159) + │ │ │ ├── fd: ()-->(167) │ │ │ ├── with-scan &5 - │ │ │ │ ├── columns: tr_s_symb:159!null + │ │ │ │ ├── columns: tr_s_symb:167 │ │ │ │ ├── mapping: - │ │ │ │ │ └── column3:126 => tr_s_symb:159 + │ │ │ │ │ └── column3:136 => tr_s_symb:167 │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(159) + │ │ │ │ └── fd: ()-->(167) │ │ │ └── filters (true) │ │ └── f-k-checks-item: trade_request(tr_b_id) -> broker(b_id) │ │ └── anti-join (lookup broker) - │ │ ├── columns: tr_b_id:178!null - │ │ ├── key columns: [178] = [179] + │ │ ├── columns: tr_b_id:186!null + │ │ ├── key columns: [186] = [187] │ │ ├── lookup columns are key │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(178) + │ │ ├── fd: ()-->(186) │ │ ├── with-scan &5 - │ │ │ ├── columns: tr_b_id:178!null + │ │ │ ├── columns: tr_b_id:186!null │ │ │ ├── mapping: - │ │ │ │ └── column6:129 => tr_b_id:178 + │ │ │ │ └── column6:134 => tr_b_id:186 │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ └── fd: ()-->(178) + │ │ │ └── fd: ()-->(186) │ │ └── filters (true) │ └── projections - │ └── 1 [as="?column?":186] + │ └── 1 [as="?column?":194] └── values - ├── columns: "?column?":187!null + ├── columns: "?column?":195!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(187) + ├── fd: ()-->(195) └── (1,) # -------------------------------------------------- @@ -3599,49 +3706,55 @@ insert holding_summary ├── columns: ├── insert-mapping: │ ├── column1:6 => holding_summary.hs_ca_id:1 - │ ├── column2:7 => holding_summary.hs_s_symb:2 - │ └── column3:8 => hs_qty:3 + │ ├── column2:9 => holding_summary.hs_s_symb:2 + │ └── column3:10 => hs_qty:3 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:6!null column2:7!null column3:8!null + │ ├── columns: column1:6!null column2:9 column3:10 │ ├── cardinality: [1 - 1] + │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(6-8) - │ └── (0, 'ROACH', 100) + │ ├── fd: ()-->(6,9,10) + │ └── tuple + │ ├── 0 + │ ├── assignment-cast: VARCHAR(15) + │ │ └── 'ROACH' + │ └── assignment-cast: INT4 + │ └── 100 └── f-k-checks ├── f-k-checks-item: holding_summary(hs_ca_id) -> customer_account(ca_id) │ └── anti-join (lookup customer_account) - │ ├── columns: hs_ca_id:9!null - │ ├── key columns: [9] = [10] + │ ├── columns: hs_ca_id:11!null + │ ├── key columns: [11] = [12] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(9) + │ ├── fd: ()-->(11) │ ├── with-scan &1 - │ │ ├── columns: hs_ca_id:9!null + │ │ ├── columns: hs_ca_id:11!null │ │ ├── mapping: - │ │ │ └── column1:6 => hs_ca_id:9 + │ │ │ └── column1:6 => hs_ca_id:11 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(9) + │ │ └── fd: ()-->(11) │ └── filters (true) └── f-k-checks-item: holding_summary(hs_s_symb) -> security(s_symb) └── anti-join (lookup security) - ├── columns: hs_s_symb:18!null - ├── key columns: [18] = [19] + ├── columns: hs_s_symb:20 + ├── key columns: [20] = [21] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(18) + ├── fd: ()-->(20) ├── with-scan &1 - │ ├── columns: hs_s_symb:18!null + │ ├── columns: hs_s_symb:20 │ ├── mapping: - │ │ └── column2:7 => hs_s_symb:18 + │ │ └── column2:9 => hs_s_symb:20 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(18) + │ └── fd: ()-->(20) └── filters (true) # Q4 @@ -3837,53 +3950,71 @@ insert holding ├── insert-mapping: │ ├── column1:9 => holding.h_t_id:1 │ ├── column2:10 => holding.h_ca_id:2 - │ ├── column3:11 => holding.h_s_symb:3 + │ ├── column3:15 => holding.h_s_symb:3 │ ├── column4:12 => h_dts:4 - │ ├── h_price:15 => holding.h_price:5 - │ └── column6:14 => h_qty:6 - ├── check columns: check1:16 + │ ├── column5:16 => h_price:5 + │ └── column6:17 => h_qty:6 + ├── check columns: check1:18 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations - ├── values - │ ├── columns: column1:9!null column2:10!null column3:11!null column4:12!null column6:14!null h_price:15!null check1:16!null + ├── project + │ ├── columns: check1:18 column1:9!null column2:10!null column4:12!null column3:15 column5:16 column6:17 │ ├── cardinality: [1 - 1] + │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(9-12,14-16) - │ └── (0, 0, 'ROACH', '2020-06-15 22:27:42.148484', 10, 1E+2, true) + │ ├── fd: ()-->(9,10,12,15-18) + │ ├── values + │ │ ├── columns: column1:9!null column2:10!null column4:12!null column3:15 column5:16 column6:17 + │ │ ├── cardinality: [1 - 1] + │ │ ├── immutable + │ │ ├── key: () + │ │ ├── fd: ()-->(9,10,12,15-17) + │ │ └── tuple + │ │ ├── 0 + │ │ ├── 0 + │ │ ├── '2020-06-15 22:27:42.148484' + │ │ ├── assignment-cast: VARCHAR(15) + │ │ │ └── 'ROACH' + │ │ ├── assignment-cast: DECIMAL(8,2) + │ │ │ └── 1E+2 + │ │ └── assignment-cast: INT4 + │ │ └── 10 + │ └── projections + │ └── column5:16 > 0 [as=check1:18, outer=(16), immutable] └── f-k-checks ├── f-k-checks-item: holding(h_t_id) -> trade(t_id) │ └── anti-join (lookup trade) - │ ├── columns: h_t_id:17!null - │ ├── key columns: [17] = [18] + │ ├── columns: h_t_id:19!null + │ ├── key columns: [19] = [20] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(17) + │ ├── fd: ()-->(19) │ ├── with-scan &1 - │ │ ├── columns: h_t_id:17!null + │ │ ├── columns: h_t_id:19!null │ │ ├── mapping: - │ │ │ └── column1:9 => h_t_id:17 + │ │ │ └── column1:9 => h_t_id:19 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(17) + │ │ └── fd: ()-->(19) │ └── filters (true) └── f-k-checks-item: holding(h_ca_id,h_s_symb) -> holding_summary(hs_ca_id,hs_s_symb) └── anti-join (lookup holding_summary) - ├── columns: h_ca_id:35!null h_s_symb:36!null - ├── key columns: [35 36] = [37 38] + ├── columns: h_ca_id:37!null h_s_symb:38 + ├── key columns: [37 38] = [39 40] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(35,36) + ├── fd: ()-->(37,38) ├── with-scan &1 - │ ├── columns: h_ca_id:35!null h_s_symb:36!null + │ ├── columns: h_ca_id:37!null h_s_symb:38 │ ├── mapping: - │ │ ├── column2:10 => h_ca_id:35 - │ │ └── column3:11 => h_s_symb:36 + │ │ ├── column2:10 => h_ca_id:37 + │ │ └── column3:15 => h_s_symb:38 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(35,36) + │ └── fd: ()-->(37,38) └── filters (true) # Q11 @@ -4157,11 +4288,11 @@ insert_trade_history AS ( SELECT 1 ---- with &2 (update_trade_commission) - ├── columns: "?column?":102!null + ├── columns: "?column?":103!null ├── cardinality: [1 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(102) + ├── fd: ()-->(103) ├── project │ ├── columns: "?column?":51!null │ ├── cardinality: [0 - 1] @@ -4219,11 +4350,11 @@ with &2 (update_trade_commission) │ └── projections │ └── 1 [as="?column?":51] └── with &4 (update_broker_commission) - ├── columns: "?column?":102!null + ├── columns: "?column?":103!null ├── cardinality: [1 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(102) + ├── fd: ()-->(103) ├── project │ ├── columns: "?column?":69!null │ ├── cardinality: [0 - 1] @@ -4258,74 +4389,79 @@ with &2 (update_trade_commission) │ └── projections │ └── 1 [as="?column?":69] └── with &6 (insert_trade_history) - ├── columns: "?column?":102!null + ├── columns: "?column?":103!null ├── cardinality: [1 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(102) + ├── fd: ()-->(103) ├── project - │ ├── columns: "?column?":101!null + │ ├── columns: "?column?":102!null │ ├── cardinality: [1 - 1] │ ├── volatile, mutations │ ├── key: () - │ ├── fd: ()-->(101) + │ ├── fd: ()-->(102) │ ├── insert trade_history │ │ ├── columns: trade_history.th_t_id:70!null trade_history.th_st_id:72!null │ │ ├── insert-mapping: │ │ │ ├── column1:75 => trade_history.th_t_id:70 │ │ │ ├── column2:76 => th_dts:71 - │ │ │ └── column3:77 => trade_history.th_st_id:72 + │ │ │ └── column3:78 => trade_history.th_st_id:72 │ │ ├── input binding: &5 │ │ ├── cardinality: [1 - 1] │ │ ├── volatile, mutations │ │ ├── key: () │ │ ├── fd: ()-->(70,72) │ │ ├── values - │ │ │ ├── columns: column1:75!null column2:76!null column3:77!null + │ │ │ ├── columns: column1:75!null column2:76!null column3:78 │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── immutable │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(75-77) - │ │ │ └── (0, '2020-06-15 22:27:42.148484', 'ACTV') + │ │ │ ├── fd: ()-->(75,76,78) + │ │ │ └── tuple + │ │ │ ├── 0 + │ │ │ ├── '2020-06-15 22:27:42.148484' + │ │ │ └── assignment-cast: VARCHAR(4) + │ │ │ └── 'ACTV' │ │ └── f-k-checks │ │ ├── f-k-checks-item: trade_history(th_t_id) -> trade(t_id) │ │ │ └── anti-join (lookup trade) - │ │ │ ├── columns: th_t_id:78!null - │ │ │ ├── key columns: [78] = [79] + │ │ │ ├── columns: th_t_id:79!null + │ │ │ ├── key columns: [79] = [80] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(78) + │ │ │ ├── fd: ()-->(79) │ │ │ ├── with-scan &5 - │ │ │ │ ├── columns: th_t_id:78!null + │ │ │ │ ├── columns: th_t_id:79!null │ │ │ │ ├── mapping: - │ │ │ │ │ └── column1:75 => th_t_id:78 + │ │ │ │ │ └── column1:75 => th_t_id:79 │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(78) + │ │ │ │ └── fd: ()-->(79) │ │ │ └── filters (true) │ │ └── f-k-checks-item: trade_history(th_st_id) -> status_type(st_id) │ │ └── anti-join (lookup status_type) - │ │ ├── columns: th_st_id:96!null - │ │ ├── key columns: [96] = [97] + │ │ ├── columns: th_st_id:97 + │ │ ├── key columns: [97] = [98] │ │ ├── lookup columns are key │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(96) + │ │ ├── fd: ()-->(97) │ │ ├── with-scan &5 - │ │ │ ├── columns: th_st_id:96!null + │ │ │ ├── columns: th_st_id:97 │ │ │ ├── mapping: - │ │ │ │ └── column3:77 => th_st_id:96 + │ │ │ │ └── column3:78 => th_st_id:97 │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ └── fd: ()-->(96) + │ │ │ └── fd: ()-->(97) │ │ └── filters (true) │ └── projections - │ └── 1 [as="?column?":101] + │ └── 1 [as="?column?":102] └── values - ├── columns: "?column?":102!null + ├── columns: "?column?":103!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(102) + ├── fd: ()-->(103) └── (1,) @@ -4355,134 +4491,148 @@ WHERE ca_id = 0 RETURNING ca_bal::FLOAT8; ---- with &2 (insert_settlement) - ├── columns: ca_bal:80!null + ├── columns: ca_bal:82!null ├── cardinality: [0 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(80) + ├── fd: ()-->(82) ├── project - │ ├── columns: "?column?":30!null + │ ├── columns: "?column?":31!null │ ├── cardinality: [1 - 1] │ ├── volatile, mutations │ ├── key: () - │ ├── fd: ()-->(30) + │ ├── fd: ()-->(31) │ ├── insert settlement │ │ ├── columns: settlement.se_t_id:1!null │ │ ├── insert-mapping: │ │ │ ├── column1:7 => settlement.se_t_id:1 - │ │ │ ├── column2:8 => se_cash_type:2 + │ │ │ ├── column2:11 => se_cash_type:2 │ │ │ ├── column3:9 => se_cash_due_date:3 - │ │ │ └── se_amt:11 => settlement.se_amt:4 + │ │ │ └── column4:12 => se_amt:4 │ │ ├── input binding: &1 │ │ ├── cardinality: [1 - 1] │ │ ├── volatile, mutations │ │ ├── key: () │ │ ├── fd: ()-->(1) │ │ ├── values - │ │ │ ├── columns: column1:7!null column2:8!null column3:9!null se_amt:11!null + │ │ │ ├── columns: column1:7!null column3:9!null column2:11 column4:12 │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── immutable │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(7-9,11) - │ │ │ └── (0, 'Margin', '2020-06-15', 1E+2) + │ │ │ ├── fd: ()-->(7,9,11,12) + │ │ │ └── tuple + │ │ │ ├── 0 + │ │ │ ├── '2020-06-15' + │ │ │ ├── assignment-cast: VARCHAR(40) + │ │ │ │ └── 'Margin' + │ │ │ └── assignment-cast: DECIMAL(10,2) + │ │ │ └── 1E+2 │ │ └── f-k-checks │ │ └── f-k-checks-item: settlement(se_t_id) -> trade(t_id) │ │ └── anti-join (lookup trade) - │ │ ├── columns: se_t_id:12!null - │ │ ├── key columns: [12] = [13] + │ │ ├── columns: se_t_id:13!null + │ │ ├── key columns: [13] = [14] │ │ ├── lookup columns are key │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(12) + │ │ ├── fd: ()-->(13) │ │ ├── with-scan &1 - │ │ │ ├── columns: se_t_id:12!null + │ │ │ ├── columns: se_t_id:13!null │ │ │ ├── mapping: - │ │ │ │ └── column1:7 => se_t_id:12 + │ │ │ │ └── column1:7 => se_t_id:13 │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ └── fd: ()-->(12) + │ │ │ └── fd: ()-->(13) │ │ └── filters (true) │ └── projections - │ └── 1 [as="?column?":30] + │ └── 1 [as="?column?":31] └── with &4 (insert_cash_transaction) - ├── columns: ca_bal:80!null + ├── columns: ca_bal:82!null ├── cardinality: [0 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(80) + ├── fd: ()-->(82) ├── project - │ ├── columns: "?column?":60!null + │ ├── columns: "?column?":62!null │ ├── cardinality: [1 - 1] │ ├── volatile, mutations │ ├── key: () - │ ├── fd: ()-->(60) + │ ├── fd: ()-->(62) │ ├── insert cash_transaction - │ │ ├── columns: cash_transaction.ct_t_id:31!null + │ │ ├── columns: cash_transaction.ct_t_id:32!null │ │ ├── insert-mapping: - │ │ │ ├── column1:37 => cash_transaction.ct_t_id:31 - │ │ │ ├── column2:38 => ct_dts:32 - │ │ │ ├── ct_amt:41 => cash_transaction.ct_amt:33 - │ │ │ └── column4:40 => ct_name:34 + │ │ │ ├── column1:38 => cash_transaction.ct_t_id:32 + │ │ │ ├── column2:39 => ct_dts:33 + │ │ │ ├── column3:42 => ct_amt:34 + │ │ │ └── column4:43 => ct_name:35 │ │ ├── input binding: &3 │ │ ├── cardinality: [1 - 1] │ │ ├── volatile, mutations │ │ ├── key: () - │ │ ├── fd: ()-->(31) + │ │ ├── fd: ()-->(32) │ │ ├── values - │ │ │ ├── columns: column1:37!null column2:38!null column4:40!null ct_amt:41!null + │ │ │ ├── columns: column1:38!null column2:39!null column3:42 column4:43 │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── immutable │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(37,38,40,41) - │ │ │ └── (0, '2020-06-10 22:27:42.148484', 'Buy 2 shares of ROACH', 1E+2) + │ │ │ ├── fd: ()-->(38,39,42,43) + │ │ │ └── tuple + │ │ │ ├── 0 + │ │ │ ├── '2020-06-10 22:27:42.148484' + │ │ │ ├── assignment-cast: DECIMAL(10,2) + │ │ │ │ └── 1E+2 + │ │ │ └── assignment-cast: VARCHAR(100) + │ │ │ └── 'Buy 2 shares of ROACH' │ │ └── f-k-checks │ │ └── f-k-checks-item: cash_transaction(ct_t_id) -> trade(t_id) │ │ └── anti-join (lookup trade) - │ │ ├── columns: ct_t_id:42!null - │ │ ├── key columns: [42] = [43] + │ │ ├── columns: ct_t_id:44!null + │ │ ├── key columns: [44] = [45] │ │ ├── lookup columns are key │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(42) + │ │ ├── fd: ()-->(44) │ │ ├── with-scan &3 - │ │ │ ├── columns: ct_t_id:42!null + │ │ │ ├── columns: ct_t_id:44!null │ │ │ ├── mapping: - │ │ │ │ └── column1:37 => ct_t_id:42 + │ │ │ │ └── column1:38 => ct_t_id:44 │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ └── fd: ()-->(42) + │ │ │ └── fd: ()-->(44) │ │ └── filters (true) │ └── projections - │ └── 1 [as="?column?":60] + │ └── 1 [as="?column?":62] └── project - ├── columns: ca_bal:80!null + ├── columns: ca_bal:82!null ├── cardinality: [0 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(80) + ├── fd: ()-->(82) ├── update customer_account - │ ├── columns: ca_id:61!null customer_account.ca_bal:66!null - │ ├── fetch columns: ca_id:69 ca_b_id:70 ca_c_id:71 ca_name:72 ca_tax_st:73 customer_account.ca_bal:74 + │ ├── columns: ca_id:63!null customer_account.ca_bal:68!null + │ ├── fetch columns: ca_id:71 ca_b_id:72 ca_c_id:73 ca_name:74 ca_tax_st:75 customer_account.ca_bal:76 │ ├── update-mapping: - │ │ └── ca_bal_new:78 => customer_account.ca_bal:66 + │ │ └── ca_bal_new:80 => customer_account.ca_bal:68 │ ├── cardinality: [0 - 1] │ ├── volatile, mutations │ ├── key: () - │ ├── fd: ()-->(61,66) + │ ├── fd: ()-->(63,68) │ └── project - │ ├── columns: ca_bal_new:78 ca_id:69!null ca_b_id:70!null ca_c_id:71!null ca_name:72 ca_tax_st:73!null customer_account.ca_bal:74!null + │ ├── columns: ca_bal_new:80 ca_id:71!null ca_b_id:72!null ca_c_id:73!null ca_name:74 ca_tax_st:75!null customer_account.ca_bal:76!null │ ├── cardinality: [0 - 1] │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(69-74,78) + │ ├── fd: ()-->(71-76,80) │ ├── scan customer_account - │ │ ├── columns: ca_id:69!null ca_b_id:70!null ca_c_id:71!null ca_name:72 ca_tax_st:73!null customer_account.ca_bal:74!null - │ │ ├── constraint: /69: [/0 - /0] + │ │ ├── columns: ca_id:71!null ca_b_id:72!null ca_c_id:73!null ca_name:74 ca_tax_st:75!null customer_account.ca_bal:76!null + │ │ ├── constraint: /71: [/0 - /0] │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(69-74) + │ │ └── fd: ()-->(71-76) │ └── projections - │ └── crdb_internal.round_decimal_values(customer_account.ca_bal:74::DECIMAL + 1E+2, 2) [as=ca_bal_new:78, outer=(74), immutable] + │ └── crdb_internal.round_decimal_values(customer_account.ca_bal:76::DECIMAL + 1E+2, 2) [as=ca_bal_new:80, outer=(76), immutable] └── projections - └── customer_account.ca_bal:66::FLOAT8 [as=ca_bal:80, outer=(66), immutable] + └── customer_account.ca_bal:68::FLOAT8 [as=ca_bal:82, outer=(68), immutable] # Q16 opt @@ -4496,68 +4646,75 @@ insert_settlement AS ( SELECT ca_bal::FLOAT8 FROM customer_account WHERE ca_id = 0; ---- with &2 (insert_settlement) - ├── columns: ca_bal:39!null + ├── columns: ca_bal:40!null ├── cardinality: [0 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(39) + ├── fd: ()-->(40) ├── project - │ ├── columns: "?column?":30!null + │ ├── columns: "?column?":31!null │ ├── cardinality: [1 - 1] │ ├── volatile, mutations │ ├── key: () - │ ├── fd: ()-->(30) + │ ├── fd: ()-->(31) │ ├── insert settlement │ │ ├── columns: settlement.se_t_id:1!null │ │ ├── insert-mapping: │ │ │ ├── column1:7 => settlement.se_t_id:1 - │ │ │ ├── column2:8 => se_cash_type:2 + │ │ │ ├── column2:11 => se_cash_type:2 │ │ │ ├── column3:9 => se_cash_due_date:3 - │ │ │ └── se_amt:11 => settlement.se_amt:4 + │ │ │ └── column4:12 => se_amt:4 │ │ ├── input binding: &1 │ │ ├── cardinality: [1 - 1] │ │ ├── volatile, mutations │ │ ├── key: () │ │ ├── fd: ()-->(1) │ │ ├── values - │ │ │ ├── columns: column1:7!null column2:8!null column3:9!null se_amt:11!null + │ │ │ ├── columns: column1:7!null column3:9!null column2:11 column4:12 │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── immutable │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(7-9,11) - │ │ │ └── (0, 'Margin', '2020-06-15', 1E+2) + │ │ │ ├── fd: ()-->(7,9,11,12) + │ │ │ └── tuple + │ │ │ ├── 0 + │ │ │ ├── '2020-06-15' + │ │ │ ├── assignment-cast: VARCHAR(40) + │ │ │ │ └── 'Margin' + │ │ │ └── assignment-cast: DECIMAL(10,2) + │ │ │ └── 1E+2 │ │ └── f-k-checks │ │ └── f-k-checks-item: settlement(se_t_id) -> trade(t_id) │ │ └── anti-join (lookup trade) - │ │ ├── columns: se_t_id:12!null - │ │ ├── key columns: [12] = [13] + │ │ ├── columns: se_t_id:13!null + │ │ ├── key columns: [13] = [14] │ │ ├── lookup columns are key │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(12) + │ │ ├── fd: ()-->(13) │ │ ├── with-scan &1 - │ │ │ ├── columns: se_t_id:12!null + │ │ │ ├── columns: se_t_id:13!null │ │ │ ├── mapping: - │ │ │ │ └── column1:7 => se_t_id:12 + │ │ │ │ └── column1:7 => se_t_id:13 │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ └── fd: ()-->(12) + │ │ │ └── fd: ()-->(13) │ │ └── filters (true) │ └── projections - │ └── 1 [as="?column?":30] + │ └── 1 [as="?column?":31] └── project - ├── columns: ca_bal:39!null + ├── columns: ca_bal:40!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(39) + ├── fd: ()-->(40) ├── scan customer_account - │ ├── columns: ca_id:31!null customer_account.ca_bal:36!null - │ ├── constraint: /31: [/0 - /0] + │ ├── columns: ca_id:32!null customer_account.ca_bal:37!null + │ ├── constraint: /32: [/0 - /0] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(31,36) + │ └── fd: ()-->(32,37) └── projections - └── customer_account.ca_bal:36::FLOAT8 [as=ca_bal:39, outer=(36), immutable] + └── customer_account.ca_bal:37::FLOAT8 [as=ca_bal:40, outer=(37), immutable] # -------------------------------------------------- # T9 diff --git a/pkg/sql/opt/xform/testdata/external/tpce-no-stats b/pkg/sql/opt/xform/testdata/external/tpce-no-stats index 8d25e672b3eb..e626a42c677a 100644 --- a/pkg/sql/opt/xform/testdata/external/tpce-no-stats +++ b/pkg/sql/opt/xform/testdata/external/tpce-no-stats @@ -570,10 +570,10 @@ update_trade_submitted AS ( SELECT * FROM request_list; ---- with &2 (update_last_trade) - ├── columns: tr_t_id:137!null tr_bid_price:138!null tr_tt_id:139!null tr_qty:140!null + ├── columns: tr_t_id:138!null tr_bid_price:139!null tr_tt_id:140!null tr_qty:141!null ├── volatile, mutations - ├── key: (137) - ├── fd: (137)-->(138-140) + ├── key: (138) + ├── fd: (138)-->(139-141) ├── project │ ├── columns: "?column?":19 │ ├── cardinality: [0 - 1] @@ -610,10 +610,10 @@ with &2 (update_last_trade) │ └── projections │ └── NULL [as="?column?":19] └── with &3 (request_list) - ├── columns: tr_t_id:137!null tr_bid_price:138!null tr_tt_id:139!null tr_qty:140!null + ├── columns: tr_t_id:138!null tr_bid_price:139!null tr_tt_id:140!null tr_qty:141!null ├── volatile, mutations - ├── key: (137) - ├── fd: (137)-->(138-140) + ├── key: (138) + ├── fd: (138)-->(139-141) ├── project │ ├── columns: tr_bid_price:28!null trade_request.tr_t_id:20!null trade_request.tr_tt_id:21!null trade_request.tr_qty:23!null │ ├── immutable @@ -640,10 +640,10 @@ with &2 (update_last_trade) │ └── projections │ └── trade_request.tr_bid_price:24::FLOAT8 [as=tr_bid_price:28, outer=(24), immutable] └── with &4 (delete_trade_request) - ├── columns: tr_t_id:137!null tr_bid_price:138!null tr_tt_id:139!null tr_qty:140!null + ├── columns: tr_t_id:138!null tr_bid_price:139!null tr_tt_id:140!null tr_qty:141!null ├── volatile, mutations - ├── key: (137) - ├── fd: (137)-->(138-140) + ├── key: (138) + ├── fd: (138)-->(139-141) ├── project │ ├── columns: "?column?":50 │ ├── volatile, mutations @@ -679,128 +679,130 @@ with &2 (update_last_trade) │ └── projections │ └── NULL [as="?column?":50] └── with &6 (insert_trade_history) - ├── columns: tr_t_id:137!null tr_bid_price:138!null tr_tt_id:139!null tr_qty:140!null + ├── columns: tr_t_id:138!null tr_bid_price:139!null tr_tt_id:140!null tr_qty:141!null ├── volatile, mutations - ├── key: (137) - ├── fd: (137)-->(138-140) + ├── key: (138) + ├── fd: (138)-->(139-141) ├── project - │ ├── columns: "?column?":85 + │ ├── columns: "?column?":86 │ ├── volatile, mutations - │ ├── fd: ()-->(85) + │ ├── fd: ()-->(86) │ ├── insert trade_history │ │ ├── columns: trade_history.th_t_id:51!null trade_history.th_st_id:53!null │ │ ├── insert-mapping: │ │ │ ├── tr_t_id:56 => trade_history.th_t_id:51 │ │ │ ├── timestamp:61 => th_dts:52 - │ │ │ └── "?column?":60 => trade_history.th_st_id:53 + │ │ │ └── "?column?":62 => trade_history.th_st_id:53 │ │ ├── input binding: &5 │ │ ├── volatile, mutations │ │ ├── key: (51) │ │ ├── fd: ()-->(53) │ │ ├── project - │ │ │ ├── columns: "?column?":60!null timestamp:61!null tr_t_id:56!null + │ │ │ ├── columns: "?column?":62 timestamp:61!null tr_t_id:56!null + │ │ │ ├── immutable │ │ │ ├── key: (56) - │ │ │ ├── fd: ()-->(60,61) + │ │ │ ├── fd: ()-->(61,62) │ │ │ ├── with-scan &3 (request_list) │ │ │ │ ├── columns: tr_t_id:56!null │ │ │ │ ├── mapping: │ │ │ │ │ └── trade_request.tr_t_id:20 => tr_t_id:56 │ │ │ │ └── key: (56) │ │ │ └── projections - │ │ │ ├── 'SBMT' [as="?column?":60] + │ │ │ ├── assignment-cast: VARCHAR(4) [as="?column?":62, immutable] + │ │ │ │ └── 'SBMT' │ │ │ └── '2020-06-15 22:27:42.148484' [as=timestamp:61] │ │ └── f-k-checks │ │ ├── f-k-checks-item: trade_history(th_t_id) -> trade(t_id) │ │ │ └── anti-join (lookup trade) - │ │ │ ├── columns: th_t_id:62!null - │ │ │ ├── key columns: [62] = [63] + │ │ │ ├── columns: th_t_id:63!null + │ │ │ ├── key columns: [63] = [64] │ │ │ ├── lookup columns are key - │ │ │ ├── key: (62) + │ │ │ ├── key: (63) │ │ │ ├── with-scan &5 - │ │ │ │ ├── columns: th_t_id:62!null + │ │ │ │ ├── columns: th_t_id:63!null │ │ │ │ ├── mapping: - │ │ │ │ │ └── tr_t_id:56 => th_t_id:62 - │ │ │ │ └── key: (62) + │ │ │ │ │ └── tr_t_id:56 => th_t_id:63 + │ │ │ │ └── key: (63) │ │ │ └── filters (true) │ │ └── f-k-checks-item: trade_history(th_st_id) -> status_type(st_id) │ │ └── anti-join (lookup status_type) - │ │ ├── columns: th_st_id:80!null - │ │ ├── key columns: [80] = [81] + │ │ ├── columns: th_st_id:81 + │ │ ├── key columns: [81] = [82] │ │ ├── lookup columns are key - │ │ ├── fd: ()-->(80) + │ │ ├── fd: ()-->(81) │ │ ├── with-scan &5 - │ │ │ ├── columns: th_st_id:80!null + │ │ │ ├── columns: th_st_id:81 │ │ │ ├── mapping: - │ │ │ │ └── "?column?":60 => th_st_id:80 - │ │ │ └── fd: ()-->(80) + │ │ │ │ └── "?column?":62 => th_st_id:81 + │ │ │ └── fd: ()-->(81) │ │ └── filters (true) │ └── projections - │ └── NULL [as="?column?":85] + │ └── NULL [as="?column?":86] └── with &8 (update_trade_submitted) - ├── columns: tr_t_id:137!null tr_bid_price:138!null tr_tt_id:139!null tr_qty:140!null + ├── columns: tr_t_id:138!null tr_bid_price:139!null tr_tt_id:140!null tr_qty:141!null ├── volatile, mutations - ├── key: (137) - ├── fd: (137)-->(138-140) + ├── key: (138) + ├── fd: (138)-->(139-141) ├── project - │ ├── columns: "?column?":136 + │ ├── columns: "?column?":137 │ ├── volatile, mutations - │ ├── fd: ()-->(136) + │ ├── fd: ()-->(137) │ ├── update trade - │ │ ├── columns: t_id:86!null - │ │ ├── fetch columns: t_id:103 t_dts:104 trade.t_st_id:105 t_tt_id:106 t_is_cash:107 t_s_symb:108 t_qty:109 t_bid_price:110 t_ca_id:111 t_exec_name:112 t_trade_price:113 t_chrg:114 t_comm:115 t_lifo:117 + │ │ ├── columns: t_id:87!null + │ │ ├── fetch columns: t_id:104 t_dts:105 trade.t_st_id:106 t_tt_id:107 t_is_cash:108 t_s_symb:109 t_qty:110 t_bid_price:111 t_ca_id:112 t_exec_name:113 t_trade_price:114 t_chrg:115 t_comm:116 t_lifo:118 │ │ ├── update-mapping: - │ │ │ ├── t_dts_new:125 => t_dts:87 - │ │ │ └── t_st_id_new:124 => trade.t_st_id:88 + │ │ │ ├── t_dts_new:126 => t_dts:88 + │ │ │ └── t_st_id_new:125 => trade.t_st_id:89 │ │ ├── input binding: &7 │ │ ├── volatile, mutations - │ │ ├── key: (86) + │ │ ├── key: (87) │ │ ├── project - │ │ │ ├── columns: t_st_id_new:124!null t_dts_new:125!null t_id:103!null t_dts:104!null trade.t_st_id:105!null t_tt_id:106!null t_is_cash:107!null t_s_symb:108!null t_qty:109!null t_bid_price:110!null t_ca_id:111!null t_exec_name:112!null t_trade_price:113 t_chrg:114!null t_comm:115!null t_lifo:117!null - │ │ │ ├── key: (103) - │ │ │ ├── fd: ()-->(124,125), (103)-->(104-115,117) + │ │ │ ├── columns: t_st_id_new:125!null t_dts_new:126!null t_id:104!null t_dts:105!null trade.t_st_id:106!null t_tt_id:107!null t_is_cash:108!null t_s_symb:109!null t_qty:110!null t_bid_price:111!null t_ca_id:112!null t_exec_name:113!null t_trade_price:114 t_chrg:115!null t_comm:116!null t_lifo:118!null + │ │ │ ├── key: (104) + │ │ │ ├── fd: ()-->(125,126), (104)-->(105-116,118) │ │ │ ├── project - │ │ │ │ ├── columns: t_id:103!null t_dts:104!null trade.t_st_id:105!null t_tt_id:106!null t_is_cash:107!null t_s_symb:108!null t_qty:109!null t_bid_price:110!null t_ca_id:111!null t_exec_name:112!null t_trade_price:113 t_chrg:114!null t_comm:115!null t_lifo:117!null - │ │ │ │ ├── key: (103) - │ │ │ │ ├── fd: (103)-->(104-115,117) + │ │ │ │ ├── columns: t_id:104!null t_dts:105!null trade.t_st_id:106!null t_tt_id:107!null t_is_cash:108!null t_s_symb:109!null t_qty:110!null t_bid_price:111!null t_ca_id:112!null t_exec_name:113!null t_trade_price:114 t_chrg:115!null t_comm:116!null t_lifo:118!null + │ │ │ │ ├── key: (104) + │ │ │ │ ├── fd: (104)-->(105-116,118) │ │ │ │ └── inner-join (lookup trade) - │ │ │ │ ├── columns: t_id:103!null t_dts:104!null trade.t_st_id:105!null t_tt_id:106!null t_is_cash:107!null t_s_symb:108!null t_qty:109!null t_bid_price:110!null t_ca_id:111!null t_exec_name:112!null t_trade_price:113 t_chrg:114!null t_comm:115!null t_lifo:117!null tr_t_id:120!null - │ │ │ │ ├── key columns: [120] = [103] + │ │ │ │ ├── columns: t_id:104!null t_dts:105!null trade.t_st_id:106!null t_tt_id:107!null t_is_cash:108!null t_s_symb:109!null t_qty:110!null t_bid_price:111!null t_ca_id:112!null t_exec_name:113!null t_trade_price:114 t_chrg:115!null t_comm:116!null t_lifo:118!null tr_t_id:121!null + │ │ │ │ ├── key columns: [121] = [104] │ │ │ │ ├── lookup columns are key - │ │ │ │ ├── key: (120) - │ │ │ │ ├── fd: (103)-->(104-115,117), (103)==(120), (120)==(103) + │ │ │ │ ├── key: (121) + │ │ │ │ ├── fd: (104)-->(105-116,118), (104)==(121), (121)==(104) │ │ │ │ ├── with-scan &3 (request_list) - │ │ │ │ │ ├── columns: tr_t_id:120!null + │ │ │ │ │ ├── columns: tr_t_id:121!null │ │ │ │ │ ├── mapping: - │ │ │ │ │ │ └── trade_request.tr_t_id:20 => tr_t_id:120 - │ │ │ │ │ └── key: (120) + │ │ │ │ │ │ └── trade_request.tr_t_id:20 => tr_t_id:121 + │ │ │ │ │ └── key: (121) │ │ │ │ └── filters (true) │ │ │ └── projections - │ │ │ ├── 'SBMT' [as=t_st_id_new:124] - │ │ │ └── '2020-06-15 22:27:42.148484' [as=t_dts_new:125] + │ │ │ ├── 'SBMT' [as=t_st_id_new:125] + │ │ │ └── '2020-06-15 22:27:42.148484' [as=t_dts_new:126] │ │ └── f-k-checks │ │ └── f-k-checks-item: trade(t_st_id) -> status_type(st_id) │ │ └── anti-join (lookup status_type) - │ │ ├── columns: t_st_id:131!null - │ │ ├── key columns: [131] = [132] + │ │ ├── columns: t_st_id:132!null + │ │ ├── key columns: [132] = [133] │ │ ├── lookup columns are key - │ │ ├── fd: ()-->(131) + │ │ ├── fd: ()-->(132) │ │ ├── with-scan &7 - │ │ │ ├── columns: t_st_id:131!null + │ │ │ ├── columns: t_st_id:132!null │ │ │ ├── mapping: - │ │ │ │ └── t_st_id_new:124 => t_st_id:131 - │ │ │ └── fd: ()-->(131) + │ │ │ │ └── t_st_id_new:125 => t_st_id:132 + │ │ │ └── fd: ()-->(132) │ │ └── filters (true) │ └── projections - │ └── NULL [as="?column?":136] + │ └── NULL [as="?column?":137] └── with-scan &3 (request_list) - ├── columns: tr_t_id:137!null tr_bid_price:138!null tr_tt_id:139!null tr_qty:140!null + ├── columns: tr_t_id:138!null tr_bid_price:139!null tr_tt_id:140!null tr_qty:141!null ├── mapping: - │ ├── trade_request.tr_t_id:20 => tr_t_id:137 - │ ├── tr_bid_price:28 => tr_bid_price:138 - │ ├── trade_request.tr_tt_id:21 => tr_tt_id:139 - │ └── trade_request.tr_qty:23 => tr_qty:140 - ├── key: (137) - └── fd: (137)-->(138-140) + │ ├── trade_request.tr_t_id:20 => tr_t_id:138 + │ ├── tr_bid_price:28 => tr_bid_price:139 + │ ├── trade_request.tr_tt_id:21 => tr_tt_id:140 + │ └── trade_request.tr_qty:23 => tr_qty:141 + ├── key: (138) + └── fd: (138)-->(139-141) # -------------------------------------------------- # T4 @@ -2952,183 +2954,225 @@ insert_trade_history AS ( SELECT 1; ---- with &2 (insert_trade) - ├── columns: "?column?":116!null + ├── columns: "?column?":121!null ├── cardinality: [1 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(116) + ├── fd: ()-->(121) ├── project - │ ├── columns: "?column?":83!null + │ ├── columns: "?column?":87!null │ ├── cardinality: [1 - 1] │ ├── volatile, mutations │ ├── key: () - │ ├── fd: ()-->(83) + │ ├── fd: ()-->(87) │ ├── insert trade │ │ ├── columns: t_id:1!null │ │ ├── insert-mapping: │ │ │ ├── column1:18 => t_id:1 │ │ │ ├── column2:19 => t_dts:2 - │ │ │ ├── column3:20 => trade.t_st_id:3 - │ │ │ ├── column4:21 => trade.t_tt_id:4 + │ │ │ ├── column3:33 => trade.t_st_id:3 + │ │ │ ├── column4:34 => trade.t_tt_id:4 │ │ │ ├── column5:22 => t_is_cash:5 - │ │ │ ├── column6:23 => trade.t_s_symb:6 - │ │ │ ├── column7:24 => t_qty:7 - │ │ │ ├── t_bid_price:33 => trade.t_bid_price:8 + │ │ │ ├── column6:35 => trade.t_s_symb:6 + │ │ │ ├── column7:36 => t_qty:7 + │ │ │ ├── column8:37 => t_bid_price:8 │ │ │ ├── column9:26 => trade.t_ca_id:9 - │ │ │ ├── column10:27 => t_exec_name:10 - │ │ │ ├── t_trade_price:34 => trade.t_trade_price:11 - │ │ │ ├── t_chrg:35 => trade.t_chrg:12 - │ │ │ ├── t_comm:36 => trade.t_comm:13 - │ │ │ ├── t_tax:37 => trade.t_tax:14 + │ │ │ ├── column10:38 => t_exec_name:10 + │ │ │ ├── column11:28 => t_trade_price:11 + │ │ │ ├── column12:39 => t_chrg:12 + │ │ │ ├── column13:40 => t_comm:13 + │ │ │ ├── column14:41 => t_tax:14 │ │ │ └── column15:32 => t_lifo:15 - │ │ ├── check columns: check1:38 check2:39 check3:40 check4:41 check5:42 + │ │ ├── check columns: check1:42 check2:43 check3:44 check4:45 check5:46 │ │ ├── input binding: &1 │ │ ├── cardinality: [1 - 1] │ │ ├── volatile, mutations │ │ ├── key: () │ │ ├── fd: ()-->(1) - │ │ ├── values - │ │ │ ├── columns: column1:18!null column2:19!null column3:20!null column4:21!null column5:22!null column6:23!null column7:24!null column9:26!null column10:27!null column15:32!null t_bid_price:33!null t_trade_price:34 t_chrg:35!null t_comm:36!null t_tax:37!null check1:38!null check2:39!null check3:40!null check4:41!null check5:42!null + │ │ ├── project + │ │ │ ├── columns: check1:42 check2:43 check3:44 check4:45 check5:46 column1:18!null column2:19!null column5:22!null column9:26!null column11:28 column15:32!null column3:33 column4:34 column6:35 column7:36 column8:37 column10:38 column12:39 column13:40 column14:41 │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── immutable │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(18-24,26,27,32-42) - │ │ │ └── (0, '2020-06-17 22:27:42.148484', 'SBMT', 'TMB', true, 'SYMB', 10, 0, 'Name', true, 1E+2, NULL, 1, 0, 0, true, true, true, true, true) + │ │ │ ├── fd: ()-->(18,19,22,26,28,32-46) + │ │ │ ├── values + │ │ │ │ ├── columns: column1:18!null column2:19!null column5:22!null column9:26!null column11:28 column15:32!null column3:33 column4:34 column6:35 column7:36 column8:37 column10:38 column12:39 column13:40 column14:41 + │ │ │ │ ├── cardinality: [1 - 1] + │ │ │ │ ├── immutable + │ │ │ │ ├── key: () + │ │ │ │ ├── fd: ()-->(18,19,22,26,28,32-41) + │ │ │ │ └── tuple + │ │ │ │ ├── 0 + │ │ │ │ ├── '2020-06-17 22:27:42.148484' + │ │ │ │ ├── true + │ │ │ │ ├── 0 + │ │ │ │ ├── CAST(NULL AS DECIMAL(8,2)) + │ │ │ │ ├── true + │ │ │ │ ├── assignment-cast: VARCHAR(4) + │ │ │ │ │ └── 'SBMT' + │ │ │ │ ├── assignment-cast: VARCHAR(3) + │ │ │ │ │ └── 'TMB' + │ │ │ │ ├── assignment-cast: VARCHAR(15) + │ │ │ │ │ └── 'SYMB' + │ │ │ │ ├── assignment-cast: INT4 + │ │ │ │ │ └── 10 + │ │ │ │ ├── assignment-cast: DECIMAL(8,2) + │ │ │ │ │ └── 1E+2 + │ │ │ │ ├── assignment-cast: VARCHAR(49) + │ │ │ │ │ └── 'Name' + │ │ │ │ ├── assignment-cast: DECIMAL(10,2) + │ │ │ │ │ └── 1 + │ │ │ │ ├── assignment-cast: DECIMAL(10,2) + │ │ │ │ │ └── 0 + │ │ │ │ └── assignment-cast: DECIMAL(10,2) + │ │ │ │ └── 0 + │ │ │ └── projections + │ │ │ ├── column7:36 > 0 [as=check1:42, outer=(36)] + │ │ │ ├── column8:37 > 0 [as=check2:43, outer=(37), immutable] + │ │ │ ├── column12:39 >= 0 [as=check3:44, outer=(39), immutable] + │ │ │ ├── column13:40 >= 0 [as=check4:45, outer=(40), immutable] + │ │ │ └── column14:41 >= 0 [as=check5:46, outer=(41), immutable] │ │ └── f-k-checks │ │ ├── f-k-checks-item: trade(t_st_id) -> status_type(st_id) │ │ │ └── anti-join (lookup status_type) - │ │ │ ├── columns: t_st_id:43!null - │ │ │ ├── key columns: [43] = [44] + │ │ │ ├── columns: t_st_id:47 + │ │ │ ├── key columns: [47] = [48] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(43) + │ │ │ ├── fd: ()-->(47) │ │ │ ├── with-scan &1 - │ │ │ │ ├── columns: t_st_id:43!null + │ │ │ │ ├── columns: t_st_id:47 │ │ │ │ ├── mapping: - │ │ │ │ │ └── column3:20 => t_st_id:43 + │ │ │ │ │ └── column3:33 => t_st_id:47 │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(43) + │ │ │ │ └── fd: ()-->(47) │ │ │ └── filters (true) │ │ ├── f-k-checks-item: trade(t_tt_id) -> trade_type(tt_id) │ │ │ └── anti-join (lookup trade_type) - │ │ │ ├── columns: t_tt_id:48!null - │ │ │ ├── key columns: [48] = [49] + │ │ │ ├── columns: t_tt_id:52 + │ │ │ ├── key columns: [52] = [53] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(48) + │ │ │ ├── fd: ()-->(52) │ │ │ ├── with-scan &1 - │ │ │ │ ├── columns: t_tt_id:48!null + │ │ │ │ ├── columns: t_tt_id:52 │ │ │ │ ├── mapping: - │ │ │ │ │ └── column4:21 => t_tt_id:48 + │ │ │ │ │ └── column4:34 => t_tt_id:52 │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(48) + │ │ │ │ └── fd: ()-->(52) │ │ │ └── filters (true) │ │ ├── f-k-checks-item: trade(t_s_symb) -> security(s_symb) │ │ │ └── anti-join (lookup security) - │ │ │ ├── columns: t_s_symb:55!null - │ │ │ ├── key columns: [55] = [56] + │ │ │ ├── columns: t_s_symb:59 + │ │ │ ├── key columns: [59] = [60] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(55) + │ │ │ ├── fd: ()-->(59) │ │ │ ├── with-scan &1 - │ │ │ │ ├── columns: t_s_symb:55!null + │ │ │ │ ├── columns: t_s_symb:59 │ │ │ │ ├── mapping: - │ │ │ │ │ └── column6:23 => t_s_symb:55 + │ │ │ │ │ └── column6:35 => t_s_symb:59 │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(55) + │ │ │ │ └── fd: ()-->(59) │ │ │ └── filters (true) │ │ └── f-k-checks-item: trade(t_ca_id) -> customer_account(ca_id) │ │ └── anti-join (lookup customer_account) - │ │ ├── columns: t_ca_id:74!null - │ │ ├── key columns: [74] = [75] + │ │ ├── columns: t_ca_id:78!null + │ │ ├── key columns: [78] = [79] │ │ ├── lookup columns are key │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(74) + │ │ ├── fd: ()-->(78) │ │ ├── with-scan &1 - │ │ │ ├── columns: t_ca_id:74!null + │ │ │ ├── columns: t_ca_id:78!null │ │ │ ├── mapping: - │ │ │ │ └── column9:26 => t_ca_id:74 + │ │ │ │ └── column9:26 => t_ca_id:78 │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ └── fd: ()-->(74) + │ │ │ └── fd: ()-->(78) │ │ └── filters (true) │ └── projections - │ └── 1 [as="?column?":83] + │ └── 1 [as="?column?":87] └── with &4 (insert_trade_history) - ├── columns: "?column?":116!null + ├── columns: "?column?":121!null ├── cardinality: [1 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(116) + ├── fd: ()-->(121) ├── project - │ ├── columns: "?column?":115!null + │ ├── columns: "?column?":120!null │ ├── cardinality: [1 - 1] │ ├── volatile, mutations │ ├── key: () - │ ├── fd: ()-->(115) + │ ├── fd: ()-->(120) │ ├── insert trade_history - │ │ ├── columns: trade_history.th_t_id:84!null trade_history.th_st_id:86!null + │ │ ├── columns: trade_history.th_t_id:88!null trade_history.th_st_id:90!null │ │ ├── insert-mapping: - │ │ │ ├── column1:89 => trade_history.th_t_id:84 - │ │ │ ├── column2:90 => th_dts:85 - │ │ │ └── column3:91 => trade_history.th_st_id:86 + │ │ │ ├── column1:93 => trade_history.th_t_id:88 + │ │ │ ├── column2:94 => th_dts:89 + │ │ │ └── column3:96 => trade_history.th_st_id:90 │ │ ├── input binding: &3 │ │ ├── cardinality: [1 - 1] │ │ ├── volatile, mutations │ │ ├── key: () - │ │ ├── fd: ()-->(84,86) + │ │ ├── fd: ()-->(88,90) │ │ ├── values - │ │ │ ├── columns: column1:89!null column2:90!null column3:91!null + │ │ │ ├── columns: column1:93!null column2:94!null column3:96 │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── immutable │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(89-91) - │ │ │ └── (0, '2020-06-15 22:27:42.148484', 'SBMT') + │ │ │ ├── fd: ()-->(93,94,96) + │ │ │ └── tuple + │ │ │ ├── 0 + │ │ │ ├── '2020-06-15 22:27:42.148484' + │ │ │ └── assignment-cast: VARCHAR(4) + │ │ │ └── 'SBMT' │ │ └── f-k-checks │ │ ├── f-k-checks-item: trade_history(th_t_id) -> trade(t_id) │ │ │ └── anti-join (lookup trade) - │ │ │ ├── columns: th_t_id:92!null - │ │ │ ├── key columns: [92] = [93] + │ │ │ ├── columns: th_t_id:97!null + │ │ │ ├── key columns: [97] = [98] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(92) + │ │ │ ├── fd: ()-->(97) │ │ │ ├── with-scan &3 - │ │ │ │ ├── columns: th_t_id:92!null + │ │ │ │ ├── columns: th_t_id:97!null │ │ │ │ ├── mapping: - │ │ │ │ │ └── column1:89 => th_t_id:92 + │ │ │ │ │ └── column1:93 => th_t_id:97 │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(92) + │ │ │ │ └── fd: ()-->(97) │ │ │ └── filters (true) │ │ └── f-k-checks-item: trade_history(th_st_id) -> status_type(st_id) │ │ └── anti-join (lookup status_type) - │ │ ├── columns: th_st_id:110!null - │ │ ├── key columns: [110] = [111] + │ │ ├── columns: th_st_id:115 + │ │ ├── key columns: [115] = [116] │ │ ├── lookup columns are key │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(110) + │ │ ├── fd: ()-->(115) │ │ ├── with-scan &3 - │ │ │ ├── columns: th_st_id:110!null + │ │ │ ├── columns: th_st_id:115 │ │ │ ├── mapping: - │ │ │ │ └── column3:91 => th_st_id:110 + │ │ │ │ └── column3:96 => th_st_id:115 │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ └── fd: ()-->(110) + │ │ │ └── fd: ()-->(115) │ │ └── filters (true) │ └── projections - │ └── 1 [as="?column?":115] + │ └── 1 [as="?column?":120] └── values - ├── columns: "?column?":116!null + ├── columns: "?column?":121!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(116) + ├── fd: ()-->(121) └── (1,) # Q12 @@ -3202,284 +3246,347 @@ insert_trade_request AS ( SELECT 1; ---- with &2 (insert_trade) - ├── columns: "?column?":187!null + ├── columns: "?column?":195!null ├── cardinality: [1 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(187) + ├── fd: ()-->(195) ├── project - │ ├── columns: "?column?":83!null + │ ├── columns: "?column?":87!null │ ├── cardinality: [1 - 1] │ ├── volatile, mutations │ ├── key: () - │ ├── fd: ()-->(83) + │ ├── fd: ()-->(87) │ ├── insert trade │ │ ├── columns: t_id:1!null │ │ ├── insert-mapping: │ │ │ ├── column1:18 => t_id:1 │ │ │ ├── column2:19 => t_dts:2 - │ │ │ ├── column3:20 => trade.t_st_id:3 - │ │ │ ├── column4:21 => trade.t_tt_id:4 + │ │ │ ├── column3:33 => trade.t_st_id:3 + │ │ │ ├── column4:34 => trade.t_tt_id:4 │ │ │ ├── column5:22 => t_is_cash:5 - │ │ │ ├── column6:23 => trade.t_s_symb:6 - │ │ │ ├── column7:24 => t_qty:7 - │ │ │ ├── t_bid_price:33 => trade.t_bid_price:8 + │ │ │ ├── column6:35 => trade.t_s_symb:6 + │ │ │ ├── column7:36 => t_qty:7 + │ │ │ ├── column8:37 => t_bid_price:8 │ │ │ ├── column9:26 => trade.t_ca_id:9 - │ │ │ ├── column10:27 => t_exec_name:10 - │ │ │ ├── t_trade_price:34 => trade.t_trade_price:11 - │ │ │ ├── t_chrg:35 => trade.t_chrg:12 - │ │ │ ├── t_comm:36 => trade.t_comm:13 - │ │ │ ├── t_tax:37 => trade.t_tax:14 + │ │ │ ├── column10:38 => t_exec_name:10 + │ │ │ ├── column11:28 => t_trade_price:11 + │ │ │ ├── column12:39 => t_chrg:12 + │ │ │ ├── column13:40 => t_comm:13 + │ │ │ ├── column14:41 => t_tax:14 │ │ │ └── column15:32 => t_lifo:15 - │ │ ├── check columns: check1:38 check2:39 check3:40 check4:41 check5:42 + │ │ ├── check columns: check1:42 check2:43 check3:44 check4:45 check5:46 │ │ ├── input binding: &1 │ │ ├── cardinality: [1 - 1] │ │ ├── volatile, mutations │ │ ├── key: () │ │ ├── fd: ()-->(1) - │ │ ├── values - │ │ │ ├── columns: column1:18!null column2:19!null column3:20!null column4:21!null column5:22!null column6:23!null column7:24!null column9:26!null column10:27!null column15:32!null t_bid_price:33!null t_trade_price:34 t_chrg:35!null t_comm:36!null t_tax:37!null check1:38!null check2:39!null check3:40!null check4:41!null check5:42!null + │ │ ├── project + │ │ │ ├── columns: check1:42 check2:43 check3:44 check4:45 check5:46 column1:18!null column2:19!null column5:22!null column9:26!null column11:28 column15:32!null column3:33 column4:34 column6:35 column7:36 column8:37 column10:38 column12:39 column13:40 column14:41 │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── immutable │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(18-24,26,27,32-42) - │ │ │ └── (0, '2020-06-17 22:27:42.148484', 'SBMT', 'TMB', true, 'SYMB', 10, 0, 'Name', true, 1E+2, NULL, 1, 0, 0, true, true, true, true, true) + │ │ │ ├── fd: ()-->(18,19,22,26,28,32-46) + │ │ │ ├── values + │ │ │ │ ├── columns: column1:18!null column2:19!null column5:22!null column9:26!null column11:28 column15:32!null column3:33 column4:34 column6:35 column7:36 column8:37 column10:38 column12:39 column13:40 column14:41 + │ │ │ │ ├── cardinality: [1 - 1] + │ │ │ │ ├── immutable + │ │ │ │ ├── key: () + │ │ │ │ ├── fd: ()-->(18,19,22,26,28,32-41) + │ │ │ │ └── tuple + │ │ │ │ ├── 0 + │ │ │ │ ├── '2020-06-17 22:27:42.148484' + │ │ │ │ ├── true + │ │ │ │ ├── 0 + │ │ │ │ ├── CAST(NULL AS DECIMAL(8,2)) + │ │ │ │ ├── true + │ │ │ │ ├── assignment-cast: VARCHAR(4) + │ │ │ │ │ └── 'SBMT' + │ │ │ │ ├── assignment-cast: VARCHAR(3) + │ │ │ │ │ └── 'TMB' + │ │ │ │ ├── assignment-cast: VARCHAR(15) + │ │ │ │ │ └── 'SYMB' + │ │ │ │ ├── assignment-cast: INT4 + │ │ │ │ │ └── 10 + │ │ │ │ ├── assignment-cast: DECIMAL(8,2) + │ │ │ │ │ └── 1E+2 + │ │ │ │ ├── assignment-cast: VARCHAR(49) + │ │ │ │ │ └── 'Name' + │ │ │ │ ├── assignment-cast: DECIMAL(10,2) + │ │ │ │ │ └── 1 + │ │ │ │ ├── assignment-cast: DECIMAL(10,2) + │ │ │ │ │ └── 0 + │ │ │ │ └── assignment-cast: DECIMAL(10,2) + │ │ │ │ └── 0 + │ │ │ └── projections + │ │ │ ├── column7:36 > 0 [as=check1:42, outer=(36)] + │ │ │ ├── column8:37 > 0 [as=check2:43, outer=(37), immutable] + │ │ │ ├── column12:39 >= 0 [as=check3:44, outer=(39), immutable] + │ │ │ ├── column13:40 >= 0 [as=check4:45, outer=(40), immutable] + │ │ │ └── column14:41 >= 0 [as=check5:46, outer=(41), immutable] │ │ └── f-k-checks │ │ ├── f-k-checks-item: trade(t_st_id) -> status_type(st_id) │ │ │ └── anti-join (lookup status_type) - │ │ │ ├── columns: t_st_id:43!null - │ │ │ ├── key columns: [43] = [44] + │ │ │ ├── columns: t_st_id:47 + │ │ │ ├── key columns: [47] = [48] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(43) + │ │ │ ├── fd: ()-->(47) │ │ │ ├── with-scan &1 - │ │ │ │ ├── columns: t_st_id:43!null + │ │ │ │ ├── columns: t_st_id:47 │ │ │ │ ├── mapping: - │ │ │ │ │ └── column3:20 => t_st_id:43 + │ │ │ │ │ └── column3:33 => t_st_id:47 │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(43) + │ │ │ │ └── fd: ()-->(47) │ │ │ └── filters (true) │ │ ├── f-k-checks-item: trade(t_tt_id) -> trade_type(tt_id) │ │ │ └── anti-join (lookup trade_type) - │ │ │ ├── columns: t_tt_id:48!null - │ │ │ ├── key columns: [48] = [49] + │ │ │ ├── columns: t_tt_id:52 + │ │ │ ├── key columns: [52] = [53] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(48) + │ │ │ ├── fd: ()-->(52) │ │ │ ├── with-scan &1 - │ │ │ │ ├── columns: t_tt_id:48!null + │ │ │ │ ├── columns: t_tt_id:52 │ │ │ │ ├── mapping: - │ │ │ │ │ └── column4:21 => t_tt_id:48 + │ │ │ │ │ └── column4:34 => t_tt_id:52 │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(48) + │ │ │ │ └── fd: ()-->(52) │ │ │ └── filters (true) │ │ ├── f-k-checks-item: trade(t_s_symb) -> security(s_symb) │ │ │ └── anti-join (lookup security) - │ │ │ ├── columns: t_s_symb:55!null - │ │ │ ├── key columns: [55] = [56] + │ │ │ ├── columns: t_s_symb:59 + │ │ │ ├── key columns: [59] = [60] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(55) + │ │ │ ├── fd: ()-->(59) │ │ │ ├── with-scan &1 - │ │ │ │ ├── columns: t_s_symb:55!null + │ │ │ │ ├── columns: t_s_symb:59 │ │ │ │ ├── mapping: - │ │ │ │ │ └── column6:23 => t_s_symb:55 + │ │ │ │ │ └── column6:35 => t_s_symb:59 │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(55) + │ │ │ │ └── fd: ()-->(59) │ │ │ └── filters (true) │ │ └── f-k-checks-item: trade(t_ca_id) -> customer_account(ca_id) │ │ └── anti-join (lookup customer_account) - │ │ ├── columns: t_ca_id:74!null - │ │ ├── key columns: [74] = [75] + │ │ ├── columns: t_ca_id:78!null + │ │ ├── key columns: [78] = [79] │ │ ├── lookup columns are key │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(74) + │ │ ├── fd: ()-->(78) │ │ ├── with-scan &1 - │ │ │ ├── columns: t_ca_id:74!null + │ │ │ ├── columns: t_ca_id:78!null │ │ │ ├── mapping: - │ │ │ │ └── column9:26 => t_ca_id:74 + │ │ │ │ └── column9:26 => t_ca_id:78 │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ └── fd: ()-->(74) + │ │ │ └── fd: ()-->(78) │ │ └── filters (true) │ └── projections - │ └── 1 [as="?column?":83] + │ └── 1 [as="?column?":87] └── with &4 (insert_trade_history) - ├── columns: "?column?":187!null + ├── columns: "?column?":195!null ├── cardinality: [1 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(187) + ├── fd: ()-->(195) ├── project - │ ├── columns: "?column?":115!null + │ ├── columns: "?column?":120!null │ ├── cardinality: [1 - 1] │ ├── volatile, mutations │ ├── key: () - │ ├── fd: ()-->(115) + │ ├── fd: ()-->(120) │ ├── insert trade_history - │ │ ├── columns: trade_history.th_t_id:84!null trade_history.th_st_id:86!null + │ │ ├── columns: trade_history.th_t_id:88!null trade_history.th_st_id:90!null │ │ ├── insert-mapping: - │ │ │ ├── column1:89 => trade_history.th_t_id:84 - │ │ │ ├── column2:90 => th_dts:85 - │ │ │ └── column3:91 => trade_history.th_st_id:86 + │ │ │ ├── column1:93 => trade_history.th_t_id:88 + │ │ │ ├── column2:94 => th_dts:89 + │ │ │ └── column3:96 => trade_history.th_st_id:90 │ │ ├── input binding: &3 │ │ ├── cardinality: [1 - 1] │ │ ├── volatile, mutations │ │ ├── key: () - │ │ ├── fd: ()-->(84,86) + │ │ ├── fd: ()-->(88,90) │ │ ├── values - │ │ │ ├── columns: column1:89!null column2:90!null column3:91!null + │ │ │ ├── columns: column1:93!null column2:94!null column3:96 │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── immutable │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(89-91) - │ │ │ └── (0, '2020-06-15 22:27:42.148484', 'SBMT') + │ │ │ ├── fd: ()-->(93,94,96) + │ │ │ └── tuple + │ │ │ ├── 0 + │ │ │ ├── '2020-06-15 22:27:42.148484' + │ │ │ └── assignment-cast: VARCHAR(4) + │ │ │ └── 'SBMT' │ │ └── f-k-checks │ │ ├── f-k-checks-item: trade_history(th_t_id) -> trade(t_id) │ │ │ └── anti-join (lookup trade) - │ │ │ ├── columns: th_t_id:92!null - │ │ │ ├── key columns: [92] = [93] + │ │ │ ├── columns: th_t_id:97!null + │ │ │ ├── key columns: [97] = [98] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(92) + │ │ │ ├── fd: ()-->(97) │ │ │ ├── with-scan &3 - │ │ │ │ ├── columns: th_t_id:92!null + │ │ │ │ ├── columns: th_t_id:97!null │ │ │ │ ├── mapping: - │ │ │ │ │ └── column1:89 => th_t_id:92 + │ │ │ │ │ └── column1:93 => th_t_id:97 │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(92) + │ │ │ │ └── fd: ()-->(97) │ │ │ └── filters (true) │ │ └── f-k-checks-item: trade_history(th_st_id) -> status_type(st_id) │ │ └── anti-join (lookup status_type) - │ │ ├── columns: th_st_id:110!null - │ │ ├── key columns: [110] = [111] + │ │ ├── columns: th_st_id:115 + │ │ ├── key columns: [115] = [116] │ │ ├── lookup columns are key │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(110) + │ │ ├── fd: ()-->(115) │ │ ├── with-scan &3 - │ │ │ ├── columns: th_st_id:110!null + │ │ │ ├── columns: th_st_id:115 │ │ │ ├── mapping: - │ │ │ │ └── column3:91 => th_st_id:110 + │ │ │ │ └── column3:96 => th_st_id:115 │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ └── fd: ()-->(110) + │ │ │ └── fd: ()-->(115) │ │ └── filters (true) │ └── projections - │ └── 1 [as="?column?":115] + │ └── 1 [as="?column?":120] └── with &6 (insert_trade_request) - ├── columns: "?column?":187!null + ├── columns: "?column?":195!null ├── cardinality: [1 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(187) + ├── fd: ()-->(195) ├── project - │ ├── columns: "?column?":186!null + │ ├── columns: "?column?":194!null │ ├── cardinality: [1 - 1] │ ├── volatile, mutations │ ├── key: () - │ ├── fd: ()-->(186) + │ ├── fd: ()-->(194) │ ├── insert trade_request - │ │ ├── columns: trade_request.tr_t_id:116!null + │ │ ├── columns: trade_request.tr_t_id:121!null │ │ ├── insert-mapping: - │ │ │ ├── column1:124 => trade_request.tr_t_id:116 - │ │ │ ├── column2:125 => trade_request.tr_tt_id:117 - │ │ │ ├── column3:126 => trade_request.tr_s_symb:118 - │ │ │ ├── column4:127 => tr_qty:119 - │ │ │ ├── tr_bid_price:130 => trade_request.tr_bid_price:120 - │ │ │ └── column6:129 => trade_request.tr_b_id:121 - │ │ ├── check columns: check1:131 check2:132 - │ │ ├── partial index put columns: partial_index_put1:133 + │ │ │ ├── column1:129 => trade_request.tr_t_id:121 + │ │ │ ├── column2:135 => trade_request.tr_tt_id:122 + │ │ │ ├── column3:136 => trade_request.tr_s_symb:123 + │ │ │ ├── column4:137 => tr_qty:124 + │ │ │ ├── column5:138 => tr_bid_price:125 + │ │ │ └── column6:134 => trade_request.tr_b_id:126 + │ │ ├── check columns: check1:139 check2:140 + │ │ ├── partial index put columns: partial_index_put1:141 │ │ ├── input binding: &5 │ │ ├── cardinality: [1 - 1] │ │ ├── volatile, mutations │ │ ├── key: () - │ │ ├── fd: ()-->(116) - │ │ ├── values - │ │ │ ├── columns: column1:124!null column2:125!null column3:126!null column4:127!null column6:129!null tr_bid_price:130!null check1:131!null check2:132!null partial_index_put1:133!null + │ │ ├── fd: ()-->(121) + │ │ ├── project + │ │ │ ├── columns: partial_index_put1:141 check1:139 check2:140 column1:129!null column6:134!null column2:135 column3:136 column4:137 column5:138 │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── immutable │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(124-127,129-133) - │ │ │ └── (0, 'TMB', 'SYMB', 10, 0, 1E+2, true, true, false) + │ │ │ ├── fd: ()-->(129,134-141) + │ │ │ ├── values + │ │ │ │ ├── columns: column1:129!null column6:134!null column2:135 column3:136 column4:137 column5:138 + │ │ │ │ ├── cardinality: [1 - 1] + │ │ │ │ ├── immutable + │ │ │ │ ├── key: () + │ │ │ │ ├── fd: ()-->(129,134-138) + │ │ │ │ └── tuple + │ │ │ │ ├── 0 + │ │ │ │ ├── 0 + │ │ │ │ ├── assignment-cast: VARCHAR(3) + │ │ │ │ │ └── 'TMB' + │ │ │ │ ├── assignment-cast: VARCHAR(15) + │ │ │ │ │ └── 'SYMB' + │ │ │ │ ├── assignment-cast: INT4 + │ │ │ │ │ └── 10 + │ │ │ │ └── assignment-cast: DECIMAL(8,2) + │ │ │ │ └── 1E+2 + │ │ │ └── projections + │ │ │ ├── column2:135 IN ('TLB', 'TLS', 'TSL') [as=partial_index_put1:141, outer=(135)] + │ │ │ ├── column4:137 > 0 [as=check1:139, outer=(137)] + │ │ │ └── column5:138 > 0 [as=check2:140, outer=(138), immutable] │ │ └── f-k-checks │ │ ├── f-k-checks-item: trade_request(tr_t_id) -> trade(t_id) │ │ │ └── anti-join (lookup trade) - │ │ │ ├── columns: tr_t_id:134!null - │ │ │ ├── key columns: [134] = [135] + │ │ │ ├── columns: tr_t_id:142!null + │ │ │ ├── key columns: [142] = [143] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(134) + │ │ │ ├── fd: ()-->(142) │ │ │ ├── with-scan &5 - │ │ │ │ ├── columns: tr_t_id:134!null + │ │ │ │ ├── columns: tr_t_id:142!null │ │ │ │ ├── mapping: - │ │ │ │ │ └── column1:124 => tr_t_id:134 + │ │ │ │ │ └── column1:129 => tr_t_id:142 │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(134) + │ │ │ │ └── fd: ()-->(142) │ │ │ └── filters (true) │ │ ├── f-k-checks-item: trade_request(tr_tt_id) -> trade_type(tt_id) │ │ │ └── anti-join (lookup trade_type) - │ │ │ ├── columns: tr_tt_id:152!null - │ │ │ ├── key columns: [152] = [153] + │ │ │ ├── columns: tr_tt_id:160 + │ │ │ ├── key columns: [160] = [161] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(152) + │ │ │ ├── fd: ()-->(160) │ │ │ ├── with-scan &5 - │ │ │ │ ├── columns: tr_tt_id:152!null + │ │ │ │ ├── columns: tr_tt_id:160 │ │ │ │ ├── mapping: - │ │ │ │ │ └── column2:125 => tr_tt_id:152 + │ │ │ │ │ └── column2:135 => tr_tt_id:160 │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(152) + │ │ │ │ └── fd: ()-->(160) │ │ │ └── filters (true) │ │ ├── f-k-checks-item: trade_request(tr_s_symb) -> security(s_symb) │ │ │ └── anti-join (lookup security) - │ │ │ ├── columns: tr_s_symb:159!null - │ │ │ ├── key columns: [159] = [160] + │ │ │ ├── columns: tr_s_symb:167 + │ │ │ ├── key columns: [167] = [168] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(159) + │ │ │ ├── fd: ()-->(167) │ │ │ ├── with-scan &5 - │ │ │ │ ├── columns: tr_s_symb:159!null + │ │ │ │ ├── columns: tr_s_symb:167 │ │ │ │ ├── mapping: - │ │ │ │ │ └── column3:126 => tr_s_symb:159 + │ │ │ │ │ └── column3:136 => tr_s_symb:167 │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(159) + │ │ │ │ └── fd: ()-->(167) │ │ │ └── filters (true) │ │ └── f-k-checks-item: trade_request(tr_b_id) -> broker(b_id) │ │ └── anti-join (lookup broker) - │ │ ├── columns: tr_b_id:178!null - │ │ ├── key columns: [178] = [179] + │ │ ├── columns: tr_b_id:186!null + │ │ ├── key columns: [186] = [187] │ │ ├── lookup columns are key │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(178) + │ │ ├── fd: ()-->(186) │ │ ├── with-scan &5 - │ │ │ ├── columns: tr_b_id:178!null + │ │ │ ├── columns: tr_b_id:186!null │ │ │ ├── mapping: - │ │ │ │ └── column6:129 => tr_b_id:178 + │ │ │ │ └── column6:134 => tr_b_id:186 │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ └── fd: ()-->(178) + │ │ │ └── fd: ()-->(186) │ │ └── filters (true) │ └── projections - │ └── 1 [as="?column?":186] + │ └── 1 [as="?column?":194] └── values - ├── columns: "?column?":187!null + ├── columns: "?column?":195!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(187) + ├── fd: ()-->(195) └── (1,) # -------------------------------------------------- @@ -3630,49 +3737,55 @@ insert holding_summary ├── columns: ├── insert-mapping: │ ├── column1:6 => holding_summary.hs_ca_id:1 - │ ├── column2:7 => holding_summary.hs_s_symb:2 - │ └── column3:8 => hs_qty:3 + │ ├── column2:9 => holding_summary.hs_s_symb:2 + │ └── column3:10 => hs_qty:3 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:6!null column2:7!null column3:8!null + │ ├── columns: column1:6!null column2:9 column3:10 │ ├── cardinality: [1 - 1] + │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(6-8) - │ └── (0, 'ROACH', 100) + │ ├── fd: ()-->(6,9,10) + │ └── tuple + │ ├── 0 + │ ├── assignment-cast: VARCHAR(15) + │ │ └── 'ROACH' + │ └── assignment-cast: INT4 + │ └── 100 └── f-k-checks ├── f-k-checks-item: holding_summary(hs_ca_id) -> customer_account(ca_id) │ └── anti-join (lookup customer_account) - │ ├── columns: hs_ca_id:9!null - │ ├── key columns: [9] = [10] + │ ├── columns: hs_ca_id:11!null + │ ├── key columns: [11] = [12] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(9) + │ ├── fd: ()-->(11) │ ├── with-scan &1 - │ │ ├── columns: hs_ca_id:9!null + │ │ ├── columns: hs_ca_id:11!null │ │ ├── mapping: - │ │ │ └── column1:6 => hs_ca_id:9 + │ │ │ └── column1:6 => hs_ca_id:11 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(9) + │ │ └── fd: ()-->(11) │ └── filters (true) └── f-k-checks-item: holding_summary(hs_s_symb) -> security(s_symb) └── anti-join (lookup security) - ├── columns: hs_s_symb:18!null - ├── key columns: [18] = [19] + ├── columns: hs_s_symb:20 + ├── key columns: [20] = [21] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(18) + ├── fd: ()-->(20) ├── with-scan &1 - │ ├── columns: hs_s_symb:18!null + │ ├── columns: hs_s_symb:20 │ ├── mapping: - │ │ └── column2:7 => hs_s_symb:18 + │ │ └── column2:9 => hs_s_symb:20 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(18) + │ └── fd: ()-->(20) └── filters (true) # Q4 @@ -3868,53 +3981,71 @@ insert holding ├── insert-mapping: │ ├── column1:9 => holding.h_t_id:1 │ ├── column2:10 => holding.h_ca_id:2 - │ ├── column3:11 => holding.h_s_symb:3 + │ ├── column3:15 => holding.h_s_symb:3 │ ├── column4:12 => h_dts:4 - │ ├── h_price:15 => holding.h_price:5 - │ └── column6:14 => h_qty:6 - ├── check columns: check1:16 + │ ├── column5:16 => h_price:5 + │ └── column6:17 => h_qty:6 + ├── check columns: check1:18 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations - ├── values - │ ├── columns: column1:9!null column2:10!null column3:11!null column4:12!null column6:14!null h_price:15!null check1:16!null + ├── project + │ ├── columns: check1:18 column1:9!null column2:10!null column4:12!null column3:15 column5:16 column6:17 │ ├── cardinality: [1 - 1] + │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(9-12,14-16) - │ └── (0, 0, 'ROACH', '2020-06-15 22:27:42.148484', 10, 1E+2, true) + │ ├── fd: ()-->(9,10,12,15-18) + │ ├── values + │ │ ├── columns: column1:9!null column2:10!null column4:12!null column3:15 column5:16 column6:17 + │ │ ├── cardinality: [1 - 1] + │ │ ├── immutable + │ │ ├── key: () + │ │ ├── fd: ()-->(9,10,12,15-17) + │ │ └── tuple + │ │ ├── 0 + │ │ ├── 0 + │ │ ├── '2020-06-15 22:27:42.148484' + │ │ ├── assignment-cast: VARCHAR(15) + │ │ │ └── 'ROACH' + │ │ ├── assignment-cast: DECIMAL(8,2) + │ │ │ └── 1E+2 + │ │ └── assignment-cast: INT4 + │ │ └── 10 + │ └── projections + │ └── column5:16 > 0 [as=check1:18, outer=(16), immutable] └── f-k-checks ├── f-k-checks-item: holding(h_t_id) -> trade(t_id) │ └── anti-join (lookup trade) - │ ├── columns: h_t_id:17!null - │ ├── key columns: [17] = [18] + │ ├── columns: h_t_id:19!null + │ ├── key columns: [19] = [20] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(17) + │ ├── fd: ()-->(19) │ ├── with-scan &1 - │ │ ├── columns: h_t_id:17!null + │ │ ├── columns: h_t_id:19!null │ │ ├── mapping: - │ │ │ └── column1:9 => h_t_id:17 + │ │ │ └── column1:9 => h_t_id:19 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(17) + │ │ └── fd: ()-->(19) │ └── filters (true) └── f-k-checks-item: holding(h_ca_id,h_s_symb) -> holding_summary(hs_ca_id,hs_s_symb) └── anti-join (lookup holding_summary) - ├── columns: h_ca_id:35!null h_s_symb:36!null - ├── key columns: [35 36] = [37 38] + ├── columns: h_ca_id:37!null h_s_symb:38 + ├── key columns: [37 38] = [39 40] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(35,36) + ├── fd: ()-->(37,38) ├── with-scan &1 - │ ├── columns: h_ca_id:35!null h_s_symb:36!null + │ ├── columns: h_ca_id:37!null h_s_symb:38 │ ├── mapping: - │ │ ├── column2:10 => h_ca_id:35 - │ │ └── column3:11 => h_s_symb:36 + │ │ ├── column2:10 => h_ca_id:37 + │ │ └── column3:15 => h_s_symb:38 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(35,36) + │ └── fd: ()-->(37,38) └── filters (true) # Q11 @@ -4186,11 +4317,11 @@ insert_trade_history AS ( SELECT 1 ---- with &2 (update_trade_commission) - ├── columns: "?column?":102!null + ├── columns: "?column?":103!null ├── cardinality: [1 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(102) + ├── fd: ()-->(103) ├── project │ ├── columns: "?column?":51!null │ ├── cardinality: [0 - 1] @@ -4248,11 +4379,11 @@ with &2 (update_trade_commission) │ └── projections │ └── 1 [as="?column?":51] └── with &4 (update_broker_commission) - ├── columns: "?column?":102!null + ├── columns: "?column?":103!null ├── cardinality: [1 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(102) + ├── fd: ()-->(103) ├── project │ ├── columns: "?column?":69!null │ ├── cardinality: [0 - 1] @@ -4287,74 +4418,79 @@ with &2 (update_trade_commission) │ └── projections │ └── 1 [as="?column?":69] └── with &6 (insert_trade_history) - ├── columns: "?column?":102!null + ├── columns: "?column?":103!null ├── cardinality: [1 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(102) + ├── fd: ()-->(103) ├── project - │ ├── columns: "?column?":101!null + │ ├── columns: "?column?":102!null │ ├── cardinality: [1 - 1] │ ├── volatile, mutations │ ├── key: () - │ ├── fd: ()-->(101) + │ ├── fd: ()-->(102) │ ├── insert trade_history │ │ ├── columns: trade_history.th_t_id:70!null trade_history.th_st_id:72!null │ │ ├── insert-mapping: │ │ │ ├── column1:75 => trade_history.th_t_id:70 │ │ │ ├── column2:76 => th_dts:71 - │ │ │ └── column3:77 => trade_history.th_st_id:72 + │ │ │ └── column3:78 => trade_history.th_st_id:72 │ │ ├── input binding: &5 │ │ ├── cardinality: [1 - 1] │ │ ├── volatile, mutations │ │ ├── key: () │ │ ├── fd: ()-->(70,72) │ │ ├── values - │ │ │ ├── columns: column1:75!null column2:76!null column3:77!null + │ │ │ ├── columns: column1:75!null column2:76!null column3:78 │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── immutable │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(75-77) - │ │ │ └── (0, '2020-06-15 22:27:42.148484', 'ACTV') + │ │ │ ├── fd: ()-->(75,76,78) + │ │ │ └── tuple + │ │ │ ├── 0 + │ │ │ ├── '2020-06-15 22:27:42.148484' + │ │ │ └── assignment-cast: VARCHAR(4) + │ │ │ └── 'ACTV' │ │ └── f-k-checks │ │ ├── f-k-checks-item: trade_history(th_t_id) -> trade(t_id) │ │ │ └── anti-join (lookup trade) - │ │ │ ├── columns: th_t_id:78!null - │ │ │ ├── key columns: [78] = [79] + │ │ │ ├── columns: th_t_id:79!null + │ │ │ ├── key columns: [79] = [80] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(78) + │ │ │ ├── fd: ()-->(79) │ │ │ ├── with-scan &5 - │ │ │ │ ├── columns: th_t_id:78!null + │ │ │ │ ├── columns: th_t_id:79!null │ │ │ │ ├── mapping: - │ │ │ │ │ └── column1:75 => th_t_id:78 + │ │ │ │ │ └── column1:75 => th_t_id:79 │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(78) + │ │ │ │ └── fd: ()-->(79) │ │ │ └── filters (true) │ │ └── f-k-checks-item: trade_history(th_st_id) -> status_type(st_id) │ │ └── anti-join (lookup status_type) - │ │ ├── columns: th_st_id:96!null - │ │ ├── key columns: [96] = [97] + │ │ ├── columns: th_st_id:97 + │ │ ├── key columns: [97] = [98] │ │ ├── lookup columns are key │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(96) + │ │ ├── fd: ()-->(97) │ │ ├── with-scan &5 - │ │ │ ├── columns: th_st_id:96!null + │ │ │ ├── columns: th_st_id:97 │ │ │ ├── mapping: - │ │ │ │ └── column3:77 => th_st_id:96 + │ │ │ │ └── column3:78 => th_st_id:97 │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ └── fd: ()-->(96) + │ │ │ └── fd: ()-->(97) │ │ └── filters (true) │ └── projections - │ └── 1 [as="?column?":101] + │ └── 1 [as="?column?":102] └── values - ├── columns: "?column?":102!null + ├── columns: "?column?":103!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(102) + ├── fd: ()-->(103) └── (1,) @@ -4384,134 +4520,148 @@ WHERE ca_id = 0 RETURNING ca_bal::FLOAT8; ---- with &2 (insert_settlement) - ├── columns: ca_bal:80!null + ├── columns: ca_bal:82!null ├── cardinality: [0 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(80) + ├── fd: ()-->(82) ├── project - │ ├── columns: "?column?":30!null + │ ├── columns: "?column?":31!null │ ├── cardinality: [1 - 1] │ ├── volatile, mutations │ ├── key: () - │ ├── fd: ()-->(30) + │ ├── fd: ()-->(31) │ ├── insert settlement │ │ ├── columns: settlement.se_t_id:1!null │ │ ├── insert-mapping: │ │ │ ├── column1:7 => settlement.se_t_id:1 - │ │ │ ├── column2:8 => se_cash_type:2 + │ │ │ ├── column2:11 => se_cash_type:2 │ │ │ ├── column3:9 => se_cash_due_date:3 - │ │ │ └── se_amt:11 => settlement.se_amt:4 + │ │ │ └── column4:12 => se_amt:4 │ │ ├── input binding: &1 │ │ ├── cardinality: [1 - 1] │ │ ├── volatile, mutations │ │ ├── key: () │ │ ├── fd: ()-->(1) │ │ ├── values - │ │ │ ├── columns: column1:7!null column2:8!null column3:9!null se_amt:11!null + │ │ │ ├── columns: column1:7!null column3:9!null column2:11 column4:12 │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── immutable │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(7-9,11) - │ │ │ └── (0, 'Margin', '2020-06-15', 1E+2) + │ │ │ ├── fd: ()-->(7,9,11,12) + │ │ │ └── tuple + │ │ │ ├── 0 + │ │ │ ├── '2020-06-15' + │ │ │ ├── assignment-cast: VARCHAR(40) + │ │ │ │ └── 'Margin' + │ │ │ └── assignment-cast: DECIMAL(10,2) + │ │ │ └── 1E+2 │ │ └── f-k-checks │ │ └── f-k-checks-item: settlement(se_t_id) -> trade(t_id) │ │ └── anti-join (lookup trade) - │ │ ├── columns: se_t_id:12!null - │ │ ├── key columns: [12] = [13] + │ │ ├── columns: se_t_id:13!null + │ │ ├── key columns: [13] = [14] │ │ ├── lookup columns are key │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(12) + │ │ ├── fd: ()-->(13) │ │ ├── with-scan &1 - │ │ │ ├── columns: se_t_id:12!null + │ │ │ ├── columns: se_t_id:13!null │ │ │ ├── mapping: - │ │ │ │ └── column1:7 => se_t_id:12 + │ │ │ │ └── column1:7 => se_t_id:13 │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ └── fd: ()-->(12) + │ │ │ └── fd: ()-->(13) │ │ └── filters (true) │ └── projections - │ └── 1 [as="?column?":30] + │ └── 1 [as="?column?":31] └── with &4 (insert_cash_transaction) - ├── columns: ca_bal:80!null + ├── columns: ca_bal:82!null ├── cardinality: [0 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(80) + ├── fd: ()-->(82) ├── project - │ ├── columns: "?column?":60!null + │ ├── columns: "?column?":62!null │ ├── cardinality: [1 - 1] │ ├── volatile, mutations │ ├── key: () - │ ├── fd: ()-->(60) + │ ├── fd: ()-->(62) │ ├── insert cash_transaction - │ │ ├── columns: cash_transaction.ct_t_id:31!null + │ │ ├── columns: cash_transaction.ct_t_id:32!null │ │ ├── insert-mapping: - │ │ │ ├── column1:37 => cash_transaction.ct_t_id:31 - │ │ │ ├── column2:38 => ct_dts:32 - │ │ │ ├── ct_amt:41 => cash_transaction.ct_amt:33 - │ │ │ └── column4:40 => ct_name:34 + │ │ │ ├── column1:38 => cash_transaction.ct_t_id:32 + │ │ │ ├── column2:39 => ct_dts:33 + │ │ │ ├── column3:42 => ct_amt:34 + │ │ │ └── column4:43 => ct_name:35 │ │ ├── input binding: &3 │ │ ├── cardinality: [1 - 1] │ │ ├── volatile, mutations │ │ ├── key: () - │ │ ├── fd: ()-->(31) + │ │ ├── fd: ()-->(32) │ │ ├── values - │ │ │ ├── columns: column1:37!null column2:38!null column4:40!null ct_amt:41!null + │ │ │ ├── columns: column1:38!null column2:39!null column3:42 column4:43 │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── immutable │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(37,38,40,41) - │ │ │ └── (0, '2020-06-10 22:27:42.148484', 'Buy 2 shares of ROACH', 1E+2) + │ │ │ ├── fd: ()-->(38,39,42,43) + │ │ │ └── tuple + │ │ │ ├── 0 + │ │ │ ├── '2020-06-10 22:27:42.148484' + │ │ │ ├── assignment-cast: DECIMAL(10,2) + │ │ │ │ └── 1E+2 + │ │ │ └── assignment-cast: VARCHAR(100) + │ │ │ └── 'Buy 2 shares of ROACH' │ │ └── f-k-checks │ │ └── f-k-checks-item: cash_transaction(ct_t_id) -> trade(t_id) │ │ └── anti-join (lookup trade) - │ │ ├── columns: ct_t_id:42!null - │ │ ├── key columns: [42] = [43] + │ │ ├── columns: ct_t_id:44!null + │ │ ├── key columns: [44] = [45] │ │ ├── lookup columns are key │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(42) + │ │ ├── fd: ()-->(44) │ │ ├── with-scan &3 - │ │ │ ├── columns: ct_t_id:42!null + │ │ │ ├── columns: ct_t_id:44!null │ │ │ ├── mapping: - │ │ │ │ └── column1:37 => ct_t_id:42 + │ │ │ │ └── column1:38 => ct_t_id:44 │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ └── fd: ()-->(42) + │ │ │ └── fd: ()-->(44) │ │ └── filters (true) │ └── projections - │ └── 1 [as="?column?":60] + │ └── 1 [as="?column?":62] └── project - ├── columns: ca_bal:80!null + ├── columns: ca_bal:82!null ├── cardinality: [0 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(80) + ├── fd: ()-->(82) ├── update customer_account - │ ├── columns: ca_id:61!null customer_account.ca_bal:66!null - │ ├── fetch columns: ca_id:69 ca_b_id:70 ca_c_id:71 ca_name:72 ca_tax_st:73 customer_account.ca_bal:74 + │ ├── columns: ca_id:63!null customer_account.ca_bal:68!null + │ ├── fetch columns: ca_id:71 ca_b_id:72 ca_c_id:73 ca_name:74 ca_tax_st:75 customer_account.ca_bal:76 │ ├── update-mapping: - │ │ └── ca_bal_new:78 => customer_account.ca_bal:66 + │ │ └── ca_bal_new:80 => customer_account.ca_bal:68 │ ├── cardinality: [0 - 1] │ ├── volatile, mutations │ ├── key: () - │ ├── fd: ()-->(61,66) + │ ├── fd: ()-->(63,68) │ └── project - │ ├── columns: ca_bal_new:78 ca_id:69!null ca_b_id:70!null ca_c_id:71!null ca_name:72 ca_tax_st:73!null customer_account.ca_bal:74!null + │ ├── columns: ca_bal_new:80 ca_id:71!null ca_b_id:72!null ca_c_id:73!null ca_name:74 ca_tax_st:75!null customer_account.ca_bal:76!null │ ├── cardinality: [0 - 1] │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(69-74,78) + │ ├── fd: ()-->(71-76,80) │ ├── scan customer_account - │ │ ├── columns: ca_id:69!null ca_b_id:70!null ca_c_id:71!null ca_name:72 ca_tax_st:73!null customer_account.ca_bal:74!null - │ │ ├── constraint: /69: [/0 - /0] + │ │ ├── columns: ca_id:71!null ca_b_id:72!null ca_c_id:73!null ca_name:74 ca_tax_st:75!null customer_account.ca_bal:76!null + │ │ ├── constraint: /71: [/0 - /0] │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(69-74) + │ │ └── fd: ()-->(71-76) │ └── projections - │ └── crdb_internal.round_decimal_values(customer_account.ca_bal:74::DECIMAL + 1E+2, 2) [as=ca_bal_new:78, outer=(74), immutable] + │ └── crdb_internal.round_decimal_values(customer_account.ca_bal:76::DECIMAL + 1E+2, 2) [as=ca_bal_new:80, outer=(76), immutable] └── projections - └── customer_account.ca_bal:66::FLOAT8 [as=ca_bal:80, outer=(66), immutable] + └── customer_account.ca_bal:68::FLOAT8 [as=ca_bal:82, outer=(68), immutable] # Q16 opt @@ -4525,68 +4675,75 @@ insert_settlement AS ( SELECT ca_bal::FLOAT8 FROM customer_account WHERE ca_id = 0; ---- with &2 (insert_settlement) - ├── columns: ca_bal:39!null + ├── columns: ca_bal:40!null ├── cardinality: [0 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(39) + ├── fd: ()-->(40) ├── project - │ ├── columns: "?column?":30!null + │ ├── columns: "?column?":31!null │ ├── cardinality: [1 - 1] │ ├── volatile, mutations │ ├── key: () - │ ├── fd: ()-->(30) + │ ├── fd: ()-->(31) │ ├── insert settlement │ │ ├── columns: settlement.se_t_id:1!null │ │ ├── insert-mapping: │ │ │ ├── column1:7 => settlement.se_t_id:1 - │ │ │ ├── column2:8 => se_cash_type:2 + │ │ │ ├── column2:11 => se_cash_type:2 │ │ │ ├── column3:9 => se_cash_due_date:3 - │ │ │ └── se_amt:11 => settlement.se_amt:4 + │ │ │ └── column4:12 => se_amt:4 │ │ ├── input binding: &1 │ │ ├── cardinality: [1 - 1] │ │ ├── volatile, mutations │ │ ├── key: () │ │ ├── fd: ()-->(1) │ │ ├── values - │ │ │ ├── columns: column1:7!null column2:8!null column3:9!null se_amt:11!null + │ │ │ ├── columns: column1:7!null column3:9!null column2:11 column4:12 │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── immutable │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(7-9,11) - │ │ │ └── (0, 'Margin', '2020-06-15', 1E+2) + │ │ │ ├── fd: ()-->(7,9,11,12) + │ │ │ └── tuple + │ │ │ ├── 0 + │ │ │ ├── '2020-06-15' + │ │ │ ├── assignment-cast: VARCHAR(40) + │ │ │ │ └── 'Margin' + │ │ │ └── assignment-cast: DECIMAL(10,2) + │ │ │ └── 1E+2 │ │ └── f-k-checks │ │ └── f-k-checks-item: settlement(se_t_id) -> trade(t_id) │ │ └── anti-join (lookup trade) - │ │ ├── columns: se_t_id:12!null - │ │ ├── key columns: [12] = [13] + │ │ ├── columns: se_t_id:13!null + │ │ ├── key columns: [13] = [14] │ │ ├── lookup columns are key │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(12) + │ │ ├── fd: ()-->(13) │ │ ├── with-scan &1 - │ │ │ ├── columns: se_t_id:12!null + │ │ │ ├── columns: se_t_id:13!null │ │ │ ├── mapping: - │ │ │ │ └── column1:7 => se_t_id:12 + │ │ │ │ └── column1:7 => se_t_id:13 │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ └── fd: ()-->(12) + │ │ │ └── fd: ()-->(13) │ │ └── filters (true) │ └── projections - │ └── 1 [as="?column?":30] + │ └── 1 [as="?column?":31] └── project - ├── columns: ca_bal:39!null + ├── columns: ca_bal:40!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(39) + ├── fd: ()-->(40) ├── scan customer_account - │ ├── columns: ca_id:31!null customer_account.ca_bal:36!null - │ ├── constraint: /31: [/0 - /0] + │ ├── columns: ca_id:32!null customer_account.ca_bal:37!null + │ ├── constraint: /32: [/0 - /0] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(31,36) + │ └── fd: ()-->(32,37) └── projections - └── customer_account.ca_bal:36::FLOAT8 [as=ca_bal:39, outer=(36), immutable] + └── customer_account.ca_bal:37::FLOAT8 [as=ca_bal:40, outer=(37), immutable] # -------------------------------------------------- # T9 diff --git a/pkg/sql/opt/xform/testdata/external/trading b/pkg/sql/opt/xform/testdata/external/trading index 311e5e4b8d4c..9697a12680e1 100644 --- a/pkg/sql/opt/xform/testdata/external/trading +++ b/pkg/sql/opt/xform/testdata/external/trading @@ -1245,19 +1245,28 @@ insert transactions │ ├── column1:10 => dealerid:1 │ ├── column2:11 => isbuy:2 │ ├── column3:12 => date:3 - │ ├── column4:13 => accountname:4 - │ ├── column5:14 => customername:5 + │ ├── column4:16 => accountname:4 + │ ├── column5:17 => customername:5 │ ├── column6:15 => operationid:6 - │ └── version_default:16 => version:7 + │ └── version_default:18 => version:7 ├── cardinality: [0 - 0] ├── volatile, mutations └── values - ├── columns: column1:10!null column2:11!null column3:12!null column4:13!null column5:14!null column6:15!null version_default:16 + ├── columns: column1:10!null column2:11!null column3:12!null column6:15!null column4:16 column5:17 version_default:18 ├── cardinality: [1 - 1] ├── volatile ├── key: () - ├── fd: ()-->(10-16) - └── (1, false, '2020-03-01 00:00:00+00:00', 'the-account', 'the-customer', '70f03eb1-4f58-4c26-b72d-c524a9d537dd', cluster_logical_timestamp()) + ├── fd: ()-->(10-12,15-18) + └── tuple + ├── 1 + ├── false + ├── '2020-03-01 00:00:00+00:00' + ├── '70f03eb1-4f58-4c26-b72d-c524a9d537dd' + ├── assignment-cast: VARCHAR(128) + │ └── 'the-account' + ├── assignment-cast: VARCHAR(128) + │ └── 'the-customer' + └── cluster_logical_timestamp() # Upsert buy or sell transaction. opt diff --git a/pkg/sql/opt/xform/testdata/external/trading-mutation b/pkg/sql/opt/xform/testdata/external/trading-mutation index 29acf18214e5..d2e875fb987a 100644 --- a/pkg/sql/opt/xform/testdata/external/trading-mutation +++ b/pkg/sql/opt/xform/testdata/external/trading-mutation @@ -1249,20 +1249,30 @@ insert transactions │ ├── column1:12 => dealerid:1 │ ├── column2:13 => isbuy:2 │ ├── column3:14 => date:3 - │ ├── column4:15 => accountname:4 - │ ├── column5:16 => customername:5 + │ ├── column4:18 => accountname:4 + │ ├── column5:19 => customername:5 │ ├── column6:17 => operationid:6 - │ ├── version_default:18 => version:7 - │ └── olddate_default:19 => olddate:8 + │ ├── version_default:20 => version:7 + │ └── olddate_default:21 => olddate:8 ├── cardinality: [0 - 0] ├── volatile, mutations └── values - ├── columns: column1:12!null column2:13!null column3:14!null column4:15!null column5:16!null column6:17!null version_default:18 olddate_default:19!null + ├── columns: column1:12!null column2:13!null column3:14!null column6:17!null column4:18 column5:19 version_default:20 olddate_default:21!null ├── cardinality: [1 - 1] ├── volatile ├── key: () - ├── fd: ()-->(12-19) - └── (1, false, '2020-03-01 00:00:00+00:00', 'the-account', 'the-customer', '70f03eb1-4f58-4c26-b72d-c524a9d537dd', cluster_logical_timestamp(), '0001-01-01 00:00:00') + ├── fd: ()-->(12-14,17-21) + └── tuple + ├── 1 + ├── false + ├── '2020-03-01 00:00:00+00:00' + ├── '70f03eb1-4f58-4c26-b72d-c524a9d537dd' + ├── assignment-cast: VARCHAR(128) + │ └── 'the-account' + ├── assignment-cast: VARCHAR(128) + │ └── 'the-customer' + ├── cluster_logical_timestamp() + └── '0001-01-01 00:00:00' # Upsert buy or sell transaction. opt diff --git a/pkg/sql/opt/xform/testdata/external/ycsb b/pkg/sql/opt/xform/testdata/external/ycsb index 8d458fbbb53e..33ac1816a25b 100644 --- a/pkg/sql/opt/xform/testdata/external/ycsb +++ b/pkg/sql/opt/xform/testdata/external/ycsb @@ -93,7 +93,7 @@ INSERT INTO usertable VALUES ( insert usertable ├── columns: ├── insert-mapping: - │ ├── column1:14 => ycsb_key:1 + │ ├── column1:25 => ycsb_key:1 │ ├── column2:15 => field0:2 │ ├── column3:16 => field1:3 │ ├── column4:17 => field2:4 @@ -107,11 +107,24 @@ insert usertable ├── cardinality: [0 - 0] ├── volatile, mutations └── values - ├── columns: column1:14!null column2:15!null column3:16!null column4:17!null column5:18!null column6:19!null column7:20!null column8:21!null column9:22!null column10:23!null column11:24!null + ├── columns: column2:15!null column3:16!null column4:17!null column5:18!null column6:19!null column7:20!null column8:21!null column9:22!null column10:23!null column11:24!null column1:25 ├── cardinality: [1 - 1] + ├── immutable ├── key: () - ├── fd: ()-->(14-24) - └── ('user123', 'field0data', 'field1data', 'field2data', 'field3data', 'field4data', 'field5data', 'field6data', 'field7data', 'field8data', 'field9data') + ├── fd: ()-->(15-25) + └── tuple + ├── 'field0data' + ├── 'field1data' + ├── 'field2data' + ├── 'field3data' + ├── 'field4data' + ├── 'field5data' + ├── 'field6data' + ├── 'field7data' + ├── 'field8data' + ├── 'field9data' + └── assignment-cast: VARCHAR(255) + └── 'user123' # -------------------------------------------------- # Workload E: Short ranges diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index 7578526c43bc..34dd85a1116c 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -10351,7 +10351,7 @@ simple_typename: // Eventually this clause will be used to parse user-defined types as well, // since their names can be quoted. if $1 == "char" { - $$.val = types.MakeQChar(0) + $$.val = types.QChar } else if $1 == "serial" { switch sqllex.(*lexer).nakedIntType.Width() { case 32: @@ -11627,7 +11627,7 @@ typed_literal: // Eventually this clause will be used to parse user-defined types as well, // since their names can be quoted. if typName == "char" { - $$.val = &tree.CastExpr{Expr: tree.NewStrVal($2), Type: types.MakeQChar(0), SyntaxMode: tree.CastPrepend} + $$.val = &tree.CastExpr{Expr: tree.NewStrVal($2), Type: types.QChar, SyntaxMode: tree.CastPrepend} } else if typName == "serial" { switch sqllex.(*lexer).nakedIntType.Width() { case 32: diff --git a/pkg/sql/pgwire/testdata/pgtest/char b/pkg/sql/pgwire/testdata/pgtest/char index d0f0f873f672..70f4ba232855 100644 --- a/pkg/sql/pgwire/testdata/pgtest/char +++ b/pkg/sql/pgwire/testdata/pgtest/char @@ -62,7 +62,7 @@ until ReadyForQuery ---- {"Type":"ParseComplete"} -{"Type":"ParameterDescription","ParameterOIDs":[23,18]} +{"Type":"ParameterDescription","ParameterOIDs":[20,25]} {"Type":"NoData"} {"Type":"BindComplete"} {"Type":"CommandComplete","CommandTag":"INSERT 0 1"} @@ -110,7 +110,7 @@ ReadyForQuery # Use the binary format for the "char" parameter. # ParameterFormatCodes = [1] for binary format send -Bind {"PreparedStatement": "s1", "ParameterFormatCodes": [1,1], "ResultFormatCodes": [0], "Parameters":[{"binary":"00000004"}, {"binary":"46"}]} +Bind {"PreparedStatement": "s1", "ParameterFormatCodes": [1,1], "ResultFormatCodes": [0], "Parameters":[{"binary":"0000000000000004"}, {"binary":"46"}]} Execute Sync ---- @@ -164,7 +164,7 @@ ReadyForQuery # Pass in a null byte. # ParameterFormatCodes = [1] for binary format send -Bind {"PreparedStatement": "s1", "ParameterFormatCodes": [1,1], "ResultFormatCodes": [0], "Parameters":[{"binary":"00000006"}, {"binary":"00"}]} +Bind {"PreparedStatement": "s1", "ParameterFormatCodes": [1,1], "ResultFormatCodes": [0], "Parameters":[{"binary":"0000000000000006"}, {"binary":"00"}]} Execute Sync ---- @@ -184,7 +184,7 @@ until ignore_table_oids ReadyForQuery ---- {"Type":"RowDescription","Fields":[{"Name":"a","TableOID":0,"TableAttributeNumber":1,"DataTypeOID":23,"DataTypeSize":4,"TypeModifier":-1,"Format":0},{"Name":"b","TableOID":0,"TableAttributeNumber":2,"DataTypeOID":18,"DataTypeSize":1,"TypeModifier":-1,"Format":0}]} -{"Type":"DataRow","Values":[{"text":"6"},null]} +{"Type":"DataRow","Values":[{"text":"6"},{"binary":"00"}]} {"Type":"CommandComplete","CommandTag":"SELECT 1"} {"Type":"ReadyForQuery","TxStatus":"I"} diff --git a/pkg/sql/sem/builtins/all_builtins.go b/pkg/sql/sem/builtins/all_builtins.go index 909e61fba065..cd52b180c18f 100644 --- a/pkg/sql/sem/builtins/all_builtins.go +++ b/pkg/sql/sem/builtins/all_builtins.go @@ -60,17 +60,25 @@ func init() { if overload.Fn != nil { fnCount++ } + if overload.FnWithExprs != nil { + fnCount++ + } if overload.Generator != nil { overload.Fn = unsuitableUseOfGeneratorFn + overload.FnWithExprs = unsuitableUseOfGeneratorFnWithExprs fnCount++ } if overload.GeneratorWithExprs != nil { overload.Fn = unsuitableUseOfGeneratorFn + overload.FnWithExprs = unsuitableUseOfGeneratorFnWithExprs fnCount++ } if fnCount > 1 { - panic(fmt.Sprintf("builtin %s: at most 1 of Fn, Generator, and GeneratorWithExprs must be set on overloads; (found %d)", - name, fnCount)) + panic(fmt.Sprintf( + "builtin %s: at most 1 of Fn, FnWithExprs, Generator, and GeneratorWithExprs"+ + "must be set on overloads; (found %d)", + name, fnCount, + )) } } } diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index aca446b88b0b..78f2ad261e6c 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -5276,6 +5276,34 @@ value if you rely on the HLC for accuracy.`, }, ), + "crdb_internal.assignment_cast": makeBuiltin( + tree.FunctionProperties{ + Category: categorySystemInfo, + // The idiomatic usage of this function is to "pass" a target type T + // by passing NULL::T, so we must allow NULL arguments. + NullableArgs: true, + }, + tree.Overload{ + Types: tree.ArgTypes{ + {"val", types.Any}, + {"type", types.Any}, + }, + ReturnType: tree.IdentityReturnType(1), + FnWithExprs: func(evalCtx *tree.EvalContext, args tree.Exprs) (tree.Datum, error) { + targetType := args[1].(tree.TypedExpr).ResolvedType() + val, err := args[0].(tree.TypedExpr).Eval(evalCtx) + if err != nil { + return nil, err + } + return tree.PerformAssignmentCast(evalCtx, val, targetType) + }, + Info: "This function is used internally to perform assignment casts during mutations.", + // The volatility of an assignment cast depends on the argument + // types, so we set it to the maximum volatility of all casts. + Volatility: tree.VolatilityStable, + }, + ), + "crdb_internal.round_decimal_values": makeBuiltin( tree.FunctionProperties{ Category: categorySystemInfo, diff --git a/pkg/sql/sem/builtins/generator_builtins.go b/pkg/sql/sem/builtins/generator_builtins.go index 38310e42fa49..5c253d557cf7 100644 --- a/pkg/sql/sem/builtins/generator_builtins.go +++ b/pkg/sql/sem/builtins/generator_builtins.go @@ -434,6 +434,10 @@ var unsuitableUseOfGeneratorFn = func(_ *tree.EvalContext, _ tree.Datums) (tree. return nil, errors.AssertionFailedf("generator functions cannot be evaluated as scalars") } +var unsuitableUseOfGeneratorFnWithExprs = func(_ *tree.EvalContext, _ tree.Exprs) (tree.Datum, error) { + return nil, errors.AssertionFailedf("generator functions cannot be evaluated as scalars") +} + func makeGeneratorOverloadWithReturnType( in tree.TypeList, retType tree.ReturnTyper, diff --git a/pkg/sql/sem/tree/BUILD.bazel b/pkg/sql/sem/tree/BUILD.bazel index 3faa8953380d..225f309643f1 100644 --- a/pkg/sql/sem/tree/BUILD.bazel +++ b/pkg/sql/sem/tree/BUILD.bazel @@ -23,7 +23,7 @@ go_library( "annotation.go", "as_of.go", "backup.go", - "casts.go", + "cast.go", "changefeed.go", "col_name.go", "collatedstring.go", @@ -129,6 +129,7 @@ go_library( "//pkg/sql/catalog/catconstants", "//pkg/sql/lex", "//pkg/sql/lexbase", + "//pkg/sql/oidext", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", "//pkg/sql/pgwire/pgnotice", @@ -176,7 +177,8 @@ go_test( name = "tree_test", size = "small", srcs = [ - "casts_test.go", + "cast_map_test.go", + "cast_test.go", "col_types_test.go", "collatedstring_test.go", "compare_test.go", @@ -221,11 +223,14 @@ go_test( "//pkg/security/securitytest", "//pkg/settings/cluster", "//pkg/sql/catalog/typedesc", + "//pkg/sql/faketreeeval", "//pkg/sql/oidext", "//pkg/sql/opt/exec/execbuilder", "//pkg/sql/opt/optbuilder", "//pkg/sql/opt/xform", "//pkg/sql/parser", + "//pkg/sql/pgwire/pgcode", + "//pkg/sql/pgwire/pgerror", "//pkg/sql/randgen", "//pkg/sql/sem/builtins", "//pkg/sql/sessiondata", diff --git a/pkg/sql/sem/tree/cast.go b/pkg/sql/sem/tree/cast.go new file mode 100644 index 000000000000..aafa04487f22 --- /dev/null +++ b/pkg/sql/sem/tree/cast.go @@ -0,0 +1,2499 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package tree + +import ( + "fmt" + "math" + "math/big" + "strconv" + "strings" + "time" + "unicode/utf8" + + "github.com/cockroachdb/apd/v2" + "github.com/cockroachdb/cockroach/pkg/geo" + "github.com/cockroachdb/cockroach/pkg/geo/geopb" + "github.com/cockroachdb/cockroach/pkg/server/telemetry" + "github.com/cockroachdb/cockroach/pkg/sql/lex" + "github.com/cockroachdb/cockroach/pkg/sql/oidext" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/bitarray" + "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/cockroach/pkg/util/json" + "github.com/cockroachdb/cockroach/pkg/util/timeofday" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate" + "github.com/cockroachdb/errors" + "github.com/lib/pq/oid" +) + +// CastContext represents the contexts in which a cast can be performed. There +// are three types of cast contexts: explicit, assignment, and implicit. Not all +// casts can be performed in all contexts. See the description of each context +// below for more details. +// +// The concept of cast contexts is taken directly from Postgres's cast behavior. +// More information can be found in the Postgres documentation on type +// conversion: https://www.postgresql.org/docs/current/typeconv.html +type CastContext uint8 + +const ( + _ CastContext = iota + // CastContextExplicit is a cast performed explicitly with the syntax + // CAST(x AS T) or x::T. + CastContextExplicit + // CastContextAssignment is a cast implicitly performed during an INSERT, + // UPSERT, or UPDATE statement. + CastContextAssignment + // CastContextImplicit is a cast performed implicitly. For example, the DATE + // below is implicitly cast to a TIMESTAMPTZ so that the values can be + // compared. + // + // SELECT '2021-01-10'::DATE < now() + // + CastContextImplicit +) + +// contextOrigin indicates the source of information for a cast's maximum +// context (see cast.maxContext below). It is only used to annotate entries in +// castMap and to perform assertions on cast entries in the init function. It +// has no effect on the behavior of a cast. +type contextOrigin uint8 + +const ( + _ contextOrigin = iota + // contextOriginPgCast specifies that a cast's maximum context is based on + // information in Postgres's pg_cast table. + contextOriginPgCast + // contextOriginSameType specifies that a cast's maximum context is not + // included in Postgres's pg_cast table. This is only used for casts where + // the source and target are the same. Such casts have a maximum context of + // implicit. + contextOriginSameType + // contextOriginAutomaticIOConversion specifies that a cast's maximum + // context is not included in Postgres's pg_cast table. In Postgres's + // internals, these casts are evaluated by each data type's input and output + // functions. + // + // Automatic casts can only convert to or from string types [1]. Conversions + // to string types are assignment casts and conversions from string types + // are explicit casts [2]. These rules are asserted in the init function. + // + // [1] https://www.postgresql.org/docs/13/catalog-pg-cast.html#CATALOG-PG-CAST + // [2] https://www.postgresql.org/docs/13/sql-createcast.html#SQL-CREATECAST-NOTES + contextOriginAutomaticIOConversion + // contextOriginNullConversion specifies that a cast's maximum context is + // not included in Postgres's pg_cast table. This is only used for casts + // where the source is the unknown type. The unknown type is only used for + // expressions that statically evaluate to NULL. NULL can be implicitly + // converted to any type. + contextOriginNullConversion +) + +// cast includes details about a cast from one OID to another. +// TODO(mgartner, otan): Move PerformCast logic to this struct. +// TODO(mgartner, otan): Move volatility to this struct. +type cast struct { + // maxContext is the maximum context in which the cast is allowed. A cast + // can only be performed in a context that is at or below the specified + // maximum context. + // + // CastContextExplicit casts can only be performed in an explicit context. + // + // CastContextAssignment casts can be performed in an explicit context or in + // an assignment context in an INSERT, UPSERT, or UPDATE statement. + // + // CastContextImplicit casts can be performed in any context. + maxContext CastContext + // origin is the source of truth for the cast's context. It is used to + // annotate entries in castMap and to perform assertions on cast entries in + // the init function. It has no effect on the behavior of a cast. + origin contextOrigin +} + +// castMap defines all possible casts. It maps from a source OID to a target OID +// to a cast struct that contains information about the cast. +// +// Validation is performed on the map in init(). +// +// Entries with a contextOriginPgCast origin were automatically generated by the +// cast_map_gen.sh script. The script outputs some types that we do not support. +// Those types were manually deleted. Entries with +// contextOriginAutomaticIOConversion origin were manually added. +var castMap = map[oid.Oid]map[oid.Oid]cast{ + oid.T_bit: { + oid.T_bit: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_int4: {maxContext: CastContextExplicit, origin: contextOriginPgCast}, + oid.T_int8: {maxContext: CastContextExplicit, origin: contextOriginPgCast}, + oid.T_varbit: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_bool: { + oid.T_bool: {maxContext: CastContextImplicit, origin: contextOriginSameType}, + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_int4: {maxContext: CastContextExplicit, origin: contextOriginPgCast}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + // Automatic I/O conversions to string types. + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oidext.T_box2d: { + oidext.T_box2d: {maxContext: CastContextImplicit, origin: contextOriginSameType}, + oidext.T_geometry: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_bpchar: { + oid.T_bpchar: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_name: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_text: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_varchar: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + // Automatic I/O conversions from bpchar to other types. + oid.T_bit: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_bool: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oidext.T_box2d: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_bytea: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_date: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_float4: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_float8: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oidext.T_geography: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oidext.T_geometry: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_inet: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_int2: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_int4: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_int8: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_interval: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_jsonb: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_numeric: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_oid: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regclass: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regnamespace: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regproc: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regprocedure: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regrole: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regtype: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_time: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_timestamp: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_timestamptz: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_timetz: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_varbit: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_bytea: { + oid.T_bytea: {maxContext: CastContextImplicit, origin: contextOriginSameType}, + oidext.T_geography: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oidext.T_geometry: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_char: { + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_char: {maxContext: CastContextImplicit, origin: contextOriginSameType}, + oid.T_int4: {maxContext: CastContextExplicit, origin: contextOriginPgCast}, + oid.T_text: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + // Automatic I/O conversions to string types. + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + // Automatic I/O conversions from "char" to other types. + oid.T_bit: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_bool: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oidext.T_box2d: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_bytea: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_date: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_float4: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_float8: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oidext.T_geography: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oidext.T_geometry: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_inet: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_int2: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_int8: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_interval: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_jsonb: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_numeric: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_oid: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regclass: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regnamespace: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regproc: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regprocedure: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regrole: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regtype: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_time: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_timestamp: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_timestamptz: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_timetz: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_varbit: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_date: { + oid.T_date: {maxContext: CastContextImplicit, origin: contextOriginSameType}, + oid.T_timestamp: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_timestamptz: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_float4: { + oid.T_float4: {maxContext: CastContextImplicit, origin: contextOriginSameType}, + oid.T_float8: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_int2: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_int4: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_int8: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_numeric: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_float8: { + oid.T_float4: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_float8: {maxContext: CastContextImplicit, origin: contextOriginSameType}, + oid.T_int2: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_int4: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_int8: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_numeric: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oidext.T_geography: { + oid.T_bytea: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oidext.T_geography: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oidext.T_geometry: {maxContext: CastContextExplicit, origin: contextOriginPgCast}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oidext.T_geometry: { + oidext.T_box2d: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_bytea: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oidext.T_geography: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oidext.T_geometry: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_jsonb: {maxContext: CastContextExplicit, origin: contextOriginPgCast}, + oid.T_text: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_inet: { + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_inet: {maxContext: CastContextImplicit, origin: contextOriginSameType}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + // Automatic I/O conversions to string types. + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_int2: { + oid.T_float4: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_float8: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_int2: {maxContext: CastContextImplicit, origin: contextOriginSameType}, + oid.T_int4: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_int8: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_numeric: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_oid: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regclass: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regnamespace: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regproc: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regprocedure: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regrole: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regtype: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_int4: { + oid.T_bit: {maxContext: CastContextExplicit, origin: contextOriginPgCast}, + oid.T_bool: {maxContext: CastContextExplicit, origin: contextOriginPgCast}, + oid.T_char: {maxContext: CastContextExplicit, origin: contextOriginPgCast}, + oid.T_float4: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_float8: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_int2: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_int4: {maxContext: CastContextImplicit, origin: contextOriginSameType}, + oid.T_int8: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_numeric: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_oid: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regclass: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regnamespace: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regproc: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regprocedure: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regrole: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regtype: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_int8: { + oid.T_bit: {maxContext: CastContextExplicit, origin: contextOriginPgCast}, + oid.T_float4: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_float8: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_int2: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_int4: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_int8: {maxContext: CastContextImplicit, origin: contextOriginSameType}, + oid.T_numeric: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_oid: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regclass: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regnamespace: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regproc: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regprocedure: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regrole: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regtype: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_interval: { + oid.T_interval: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_time: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_jsonb: { + oid.T_bool: {maxContext: CastContextExplicit, origin: contextOriginPgCast}, + oid.T_float4: {maxContext: CastContextExplicit, origin: contextOriginPgCast}, + oid.T_float8: {maxContext: CastContextExplicit, origin: contextOriginPgCast}, + oid.T_int2: {maxContext: CastContextExplicit, origin: contextOriginPgCast}, + oid.T_int4: {maxContext: CastContextExplicit, origin: contextOriginPgCast}, + oid.T_int8: {maxContext: CastContextExplicit, origin: contextOriginPgCast}, + oid.T_jsonb: {maxContext: CastContextImplicit, origin: contextOriginSameType}, + oid.T_numeric: {maxContext: CastContextExplicit, origin: contextOriginPgCast}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_name: { + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_name: {maxContext: CastContextImplicit, origin: contextOriginSameType}, + oid.T_text: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + // Automatic I/O conversions to string types. + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + // Automatic I/O conversions from NAME to other types. + oid.T_bit: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_bool: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oidext.T_box2d: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_bytea: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_date: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_float4: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_float8: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oidext.T_geography: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oidext.T_geometry: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_inet: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_int2: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_int4: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_int8: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_interval: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_jsonb: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_numeric: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_oid: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regclass: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regnamespace: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regproc: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regprocedure: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regrole: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regtype: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_time: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_timestamp: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_timestamptz: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_timetz: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_varbit: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_numeric: { + oid.T_float4: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_float8: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_int2: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_int4: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_int8: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_numeric: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_oid: { + oid.T_int4: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_int8: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_oid: {maxContext: CastContextImplicit, origin: contextOriginSameType}, + oid.T_regclass: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regnamespace: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regproc: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regprocedure: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regrole: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regtype: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_regclass: { + oid.T_int4: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_int8: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_oid: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regclass: {maxContext: CastContextImplicit, origin: contextOriginSameType}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_regnamespace: { + oid.T_int4: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_int8: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_oid: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regnamespace: {maxContext: CastContextImplicit, origin: contextOriginSameType}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_regproc: { + oid.T_int4: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_int8: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_oid: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regproc: {maxContext: CastContextImplicit, origin: contextOriginSameType}, + oid.T_regprocedure: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_regprocedure: { + oid.T_int4: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_int8: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_oid: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regproc: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regprocedure: {maxContext: CastContextImplicit, origin: contextOriginSameType}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_regrole: { + oid.T_int4: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_int8: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_oid: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regrole: {maxContext: CastContextImplicit, origin: contextOriginSameType}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_regtype: { + oid.T_int4: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_int8: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_oid: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regtype: {maxContext: CastContextImplicit, origin: contextOriginSameType}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_text: { + oid.T_bpchar: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oidext.T_geometry: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_name: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regclass: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_varchar: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_text: {maxContext: CastContextImplicit, origin: contextOriginSameType}, + // Automatic I/O conversions from TEXT to other types. + oid.T_bit: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_bool: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oidext.T_box2d: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_bytea: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_date: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_float4: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_float8: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oidext.T_geography: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_inet: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_int2: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_int4: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_int8: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_interval: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_jsonb: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_numeric: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_oid: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regnamespace: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regproc: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regprocedure: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regrole: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regtype: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_time: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_timestamp: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_timestamptz: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_timetz: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_varbit: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_time: { + oid.T_interval: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_time: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_timetz: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_timestamp: { + oid.T_date: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_time: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_timestamp: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_timestamptz: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_timestamptz: { + oid.T_date: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_time: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_timestamp: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_timestamptz: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_timetz: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_timetz: { + oid.T_time: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_timetz: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_unknown: { + // Unknown is the type of an expression that statically evaluates to + // NULL. NULL can be implicitly cast to any type. + oid.T_bit: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_bool: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_bpchar: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oidext.T_box2d: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_bytea: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_char: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_date: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_float4: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_float8: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oidext.T_geography: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oidext.T_geometry: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_inet: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_int2: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_int4: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_int8: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_interval: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_jsonb: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_name: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_numeric: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_oid: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_regclass: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_regnamespace: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_regproc: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_regprocedure: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_regrole: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_regtype: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_text: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_time: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_timestamp: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_timestamptz: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_timetz: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_varbit: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + oid.T_varchar: {maxContext: CastContextImplicit, origin: contextOriginNullConversion}, + }, + oid.T_varbit: { + oid.T_bit: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_varbit: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + // Automatic I/O conversions to string types. + oid.T_bpchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_name: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_text: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + oid.T_varchar: {maxContext: CastContextAssignment, origin: contextOriginAutomaticIOConversion}, + }, + oid.T_varchar: { + oid.T_bpchar: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_char: {maxContext: CastContextAssignment, origin: contextOriginPgCast}, + oid.T_name: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_regclass: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_text: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + oid.T_varchar: {maxContext: CastContextImplicit, origin: contextOriginPgCast}, + // Automatic I/O conversions from VARCHAR to other types. + oid.T_bit: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_bool: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oidext.T_box2d: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_bytea: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_date: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_float4: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_float8: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oidext.T_geography: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oidext.T_geometry: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_inet: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_int2: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_int4: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_int8: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_interval: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_jsonb: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_numeric: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_oid: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regnamespace: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regproc: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regprocedure: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regrole: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_regtype: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_time: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_timestamp: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_timestamptz: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_timetz: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + oid.T_varbit: {maxContext: CastContextExplicit, origin: contextOriginAutomaticIOConversion}, + }, +} + +// init performs sanity checks on castMap. +func init() { + var stringTypes = [...]oid.Oid{ + oid.T_bpchar, + oid.T_name, + oid.T_char, + oid.T_varchar, + oid.T_text, + } + isStringType := func(o oid.Oid) bool { + for _, strOid := range stringTypes { + if o == strOid { + return true + } + } + return false + } + + // Assert that there is a cast to and from every string type. + for _, strType := range stringTypes { + for otherType := range castMap { + strTypeName := oid.TypeName[strType] + otherTypeName := oid.TypeName[otherType] + if _, from := castMap[strType][otherType]; !from && otherType != oid.T_unknown { + panic(fmt.Sprintf("there must be a cast from %s to %s", strTypeName, otherTypeName)) + } + if _, to := castMap[otherType][strType]; !to { + panic(fmt.Sprintf("there must be a cast from %s to %s", otherTypeName, strTypeName)) + } + } + } + + // Assert that each cast is valid. + for src, tgts := range castMap { + for tgt := range tgts { + ent := castMap[src][tgt] + srcStr := oid.TypeName[src] + tgtStr := oid.TypeName[tgt] + + // Assert that maxContext, method, and origin have been set. + if ent.maxContext == CastContext(0) { + panic(fmt.Sprintf("cast from %s to %s has no maxContext set", srcStr, tgtStr)) + } + if ent.origin == contextOrigin(0) { + panic(fmt.Sprintf("cast from %s to %s has no origin set", srcStr, tgtStr)) + } + + // Casts from a type to the same type should be implicit. + if src == tgt { + if ent.maxContext != CastContextImplicit { + panic(fmt.Sprintf( + "cast from %s to %s must be an implicit cast", + srcStr, tgtStr, + )) + } + } + + // Automatic I/O conversions to string types are assignment casts. + if isStringType(tgt) && ent.origin == contextOriginAutomaticIOConversion && + ent.maxContext != CastContextAssignment { + panic(fmt.Sprintf( + "automatic conversion from %s to %s must be an assignment cast", + srcStr, tgtStr, + )) + } + + // Automatic I/O conversions from string types are explicit casts. + if isStringType(src) && !isStringType(tgt) && ent.origin == contextOriginAutomaticIOConversion && + ent.maxContext != CastContextExplicit { + panic(fmt.Sprintf( + "automatic conversion from %s to %s must be an explicit cast", + srcStr, tgtStr, + )) + } + + // Casts from NULL are implicit. + if src == oid.T_unknown { + if ent.maxContext != CastContextImplicit { + panic(fmt.Sprintf( + "cast from %s to %s must be an implicit cast", + srcStr, tgtStr, + )) + } + if tgt != oid.T_unknown && ent.origin != contextOriginNullConversion { + panic(fmt.Sprintf( + "cast from T_unknown to %s must have an origin of contextOriginNullConversion", + tgtStr, + )) + } + } + + // Only casts from NULL should use contextOriginNullConversion. + if ent.origin == contextOriginNullConversion && src != oid.T_unknown { + panic("contextOriginNullConversion should only be set for casts from unknown") + } + } + } +} + +// ForEachCast calls fn for every valid cast from a source type to a target +// type. +func ForEachCast(fn func(src, tgt oid.Oid)) { + for src, tgts := range castMap { + for tgt := range tgts { + fn(src, tgt) + } + } +} + +// ValidCast returns true if a valid cast exists from src to tgt in the given +// context. +func ValidCast(src, tgt *types.T, ctx CastContext) bool { + srcFamily := src.Family() + tgtFamily := tgt.Family() + + // If src and tgt are array types, check for a valid cast between their + // content types. + if srcFamily == types.ArrayFamily && tgtFamily == types.ArrayFamily { + return ValidCast(src.ArrayContents(), tgt.ArrayContents(), ctx) + } + + // If src and tgt are tuple types, check for a valid cast between each + // corresponding tuple element. + if srcFamily == types.TupleFamily && tgtFamily == types.TupleFamily { + srcTypes := src.TupleContents() + tgtTypes := tgt.TupleContents() + // The tuple types must have the same number of elements. + if len(srcTypes) != len(tgtTypes) { + return false + } + for i := range srcTypes { + if ok := ValidCast(srcTypes[i], tgtTypes[i], ctx); !ok { + return false + } + } + return true + } + + // If src and tgt are not array or tuple types, check castMap for a valid + // cast. + if c, ok := lookupCast(src.Oid(), tgt.Oid()); ok { + return c.maxContext >= ctx + } + + return false +} + +// lookupCast returns a cast that describes the cast from src to tgt if +// it exists. If it does not exist, ok=false is returned. +func lookupCast(src, tgt oid.Oid) (cast, bool) { + if tgts, ok := castMap[src]; ok { + if c, ok := tgts[tgt]; ok { + return c, true + } + } + return cast{}, false +} + +type castInfo struct { + from types.Family + to types.Family + volatility Volatility + + // volatilityHint is an optional string for VolatilityStable casts. When set, + // it is used as an error hint suggesting a possible workaround when stable + // casts are not allowed. + volatilityHint string + + // Telemetry counter; set by init(). + counter telemetry.Counter + + // If set, the volatility of this cast is not cross-checked against postgres. + // Use this with caution. + ignoreVolatilityCheck bool +} + +// validCasts lists all valid explicit casts. +// +// This list must be kept in sync with the capabilities of PerformCast. +// +// Each cast defines a volatility: +// +// - immutable casts yield the same result on the same arguments in whatever +// context they are evaluated. +// +// - stable casts can yield a different result depending on the evaluation context: +// - session settings (e.g. bytes encoding format) +// - current timezone +// - current time (e.g. 'now'::string). +// +// TODO(#55094): move the PerformCast code for each cast into functions defined +// within each cast. +// +var validCasts = []castInfo{ + // Casts to BitFamily. + {from: types.UnknownFamily, to: types.BitFamily, volatility: VolatilityImmutable}, + {from: types.BitFamily, to: types.BitFamily, volatility: VolatilityImmutable}, + {from: types.IntFamily, to: types.BitFamily, volatility: VolatilityImmutable}, + {from: types.StringFamily, to: types.BitFamily, volatility: VolatilityImmutable}, + {from: types.CollatedStringFamily, to: types.BitFamily, volatility: VolatilityImmutable}, + + // Casts to BoolFamily. + {from: types.UnknownFamily, to: types.BoolFamily, volatility: VolatilityImmutable}, + {from: types.BoolFamily, to: types.BoolFamily, volatility: VolatilityImmutable}, + {from: types.IntFamily, to: types.BoolFamily, volatility: VolatilityImmutable}, + {from: types.FloatFamily, to: types.BoolFamily, volatility: VolatilityImmutable}, + {from: types.DecimalFamily, to: types.BoolFamily, volatility: VolatilityImmutable}, + {from: types.StringFamily, to: types.BoolFamily, volatility: VolatilityImmutable}, + {from: types.CollatedStringFamily, to: types.BoolFamily, volatility: VolatilityImmutable}, + {from: types.JsonFamily, to: types.BoolFamily, volatility: VolatilityImmutable}, + + // Casts to IntFamily. + {from: types.UnknownFamily, to: types.IntFamily, volatility: VolatilityImmutable}, + {from: types.BoolFamily, to: types.IntFamily, volatility: VolatilityImmutable}, + {from: types.IntFamily, to: types.IntFamily, volatility: VolatilityImmutable}, + {from: types.FloatFamily, to: types.IntFamily, volatility: VolatilityImmutable}, + {from: types.DecimalFamily, to: types.IntFamily, volatility: VolatilityImmutable}, + {from: types.StringFamily, to: types.IntFamily, volatility: VolatilityImmutable}, + {from: types.CollatedStringFamily, to: types.IntFamily, volatility: VolatilityImmutable}, + {from: types.TimestampFamily, to: types.IntFamily, volatility: VolatilityImmutable}, + {from: types.TimestampTZFamily, to: types.IntFamily, volatility: VolatilityImmutable}, + {from: types.DateFamily, to: types.IntFamily, volatility: VolatilityImmutable}, + {from: types.IntervalFamily, to: types.IntFamily, volatility: VolatilityImmutable}, + {from: types.OidFamily, to: types.IntFamily, volatility: VolatilityImmutable}, + {from: types.BitFamily, to: types.IntFamily, volatility: VolatilityImmutable}, + {from: types.JsonFamily, to: types.IntFamily, volatility: VolatilityImmutable}, + + // Casts to FloatFamily. + {from: types.UnknownFamily, to: types.FloatFamily, volatility: VolatilityImmutable}, + {from: types.BoolFamily, to: types.FloatFamily, volatility: VolatilityImmutable}, + {from: types.IntFamily, to: types.FloatFamily, volatility: VolatilityImmutable}, + {from: types.FloatFamily, to: types.FloatFamily, volatility: VolatilityImmutable}, + {from: types.DecimalFamily, to: types.FloatFamily, volatility: VolatilityImmutable}, + {from: types.StringFamily, to: types.FloatFamily, volatility: VolatilityImmutable}, + {from: types.CollatedStringFamily, to: types.FloatFamily, volatility: VolatilityImmutable}, + {from: types.TimestampFamily, to: types.FloatFamily, volatility: VolatilityImmutable}, + {from: types.TimestampTZFamily, to: types.FloatFamily, volatility: VolatilityImmutable}, + {from: types.DateFamily, to: types.FloatFamily, volatility: VolatilityImmutable}, + {from: types.IntervalFamily, to: types.FloatFamily, volatility: VolatilityImmutable}, + {from: types.JsonFamily, to: types.FloatFamily, volatility: VolatilityImmutable}, + + // Casts to Box2D Family. + {from: types.UnknownFamily, to: types.Box2DFamily, volatility: VolatilityImmutable}, + {from: types.StringFamily, to: types.Box2DFamily, volatility: VolatilityImmutable}, + {from: types.CollatedStringFamily, to: types.Box2DFamily, volatility: VolatilityImmutable}, + {from: types.GeometryFamily, to: types.Box2DFamily, volatility: VolatilityImmutable}, + {from: types.Box2DFamily, to: types.Box2DFamily, volatility: VolatilityImmutable}, + + // Casts to GeographyFamily. + {from: types.UnknownFamily, to: types.GeographyFamily, volatility: VolatilityImmutable}, + {from: types.BytesFamily, to: types.GeographyFamily, volatility: VolatilityImmutable}, + {from: types.JsonFamily, to: types.GeographyFamily, volatility: VolatilityImmutable}, + {from: types.StringFamily, to: types.GeographyFamily, volatility: VolatilityImmutable}, + {from: types.CollatedStringFamily, to: types.GeographyFamily, volatility: VolatilityImmutable}, + {from: types.GeographyFamily, to: types.GeographyFamily, volatility: VolatilityImmutable}, + {from: types.GeometryFamily, to: types.GeographyFamily, volatility: VolatilityImmutable}, + + // Casts to GeometryFamily. + {from: types.UnknownFamily, to: types.GeometryFamily, volatility: VolatilityImmutable}, + {from: types.Box2DFamily, to: types.GeometryFamily, volatility: VolatilityImmutable}, + {from: types.BytesFamily, to: types.GeometryFamily, volatility: VolatilityImmutable}, + {from: types.JsonFamily, to: types.GeometryFamily, volatility: VolatilityImmutable}, + {from: types.StringFamily, to: types.GeometryFamily, volatility: VolatilityImmutable}, + {from: types.CollatedStringFamily, to: types.GeometryFamily, volatility: VolatilityImmutable}, + {from: types.GeographyFamily, to: types.GeometryFamily, volatility: VolatilityImmutable}, + {from: types.GeometryFamily, to: types.GeometryFamily, volatility: VolatilityImmutable}, + + // Casts to DecimalFamily. + {from: types.UnknownFamily, to: types.DecimalFamily, volatility: VolatilityImmutable}, + {from: types.BoolFamily, to: types.DecimalFamily, volatility: VolatilityImmutable}, + {from: types.IntFamily, to: types.DecimalFamily, volatility: VolatilityImmutable}, + {from: types.FloatFamily, to: types.DecimalFamily, volatility: VolatilityImmutable}, + {from: types.DecimalFamily, to: types.DecimalFamily, volatility: VolatilityImmutable}, + {from: types.StringFamily, to: types.DecimalFamily, volatility: VolatilityImmutable}, + {from: types.CollatedStringFamily, to: types.DecimalFamily, volatility: VolatilityImmutable}, + {from: types.TimestampFamily, to: types.DecimalFamily, volatility: VolatilityImmutable}, + {from: types.TimestampTZFamily, to: types.DecimalFamily, volatility: VolatilityImmutable}, + {from: types.DateFamily, to: types.DecimalFamily, volatility: VolatilityImmutable}, + {from: types.IntervalFamily, to: types.DecimalFamily, volatility: VolatilityImmutable}, + {from: types.JsonFamily, to: types.DecimalFamily, volatility: VolatilityImmutable}, + + // Casts to StringFamily. + {from: types.UnknownFamily, to: types.StringFamily, volatility: VolatilityImmutable}, + {from: types.BoolFamily, to: types.StringFamily, volatility: VolatilityImmutable}, + {from: types.IntFamily, to: types.StringFamily, volatility: VolatilityImmutable}, + {from: types.FloatFamily, to: types.StringFamily, volatility: VolatilityStable}, + {from: types.DecimalFamily, to: types.StringFamily, volatility: VolatilityImmutable}, + {from: types.StringFamily, to: types.StringFamily, volatility: VolatilityImmutable}, + {from: types.CollatedStringFamily, to: types.StringFamily, volatility: VolatilityImmutable}, + {from: types.BitFamily, to: types.StringFamily, volatility: VolatilityImmutable}, + {from: types.ArrayFamily, to: types.StringFamily, volatility: VolatilityStable}, + {from: types.TupleFamily, to: types.StringFamily, volatility: VolatilityImmutable}, + {from: types.GeometryFamily, to: types.StringFamily, volatility: VolatilityImmutable}, + {from: types.Box2DFamily, to: types.StringFamily, volatility: VolatilityImmutable}, + {from: types.GeographyFamily, to: types.StringFamily, volatility: VolatilityImmutable}, + {from: types.BytesFamily, to: types.StringFamily, volatility: VolatilityStable}, + { + from: types.TimestampFamily, + to: types.StringFamily, + volatility: VolatilityImmutable, + volatilityHint: "TIMESTAMP to STRING casts are dependent on DateStyle; consider " + + "using to_char(timestamp) instead.", + }, + { + from: types.TimestampTZFamily, + to: types.StringFamily, + volatility: VolatilityStable, + volatilityHint: "TIMESTAMPTZ to STRING casts depend on the current timezone; consider " + + "using to_char(t AT TIME ZONE 'UTC') instead.", + }, + { + from: types.IntervalFamily, + to: types.StringFamily, + volatility: VolatilityImmutable, + volatilityHint: "INTERVAL to STRING casts depends on IntervalStyle; consider using to_char(interval)", + }, + {from: types.UuidFamily, to: types.StringFamily, volatility: VolatilityImmutable}, + { + from: types.DateFamily, + to: types.StringFamily, + volatility: VolatilityImmutable, + volatilityHint: "DATE to STRING casts are dependent on DateStyle; consider " + + "using to_char(date) instead.", + }, + {from: types.TimeFamily, to: types.StringFamily, volatility: VolatilityImmutable}, + {from: types.TimeTZFamily, to: types.StringFamily, volatility: VolatilityImmutable}, + {from: types.OidFamily, to: types.StringFamily, volatility: VolatilityImmutable}, + {from: types.INetFamily, to: types.StringFamily, volatility: VolatilityImmutable}, + {from: types.JsonFamily, to: types.StringFamily, volatility: VolatilityImmutable}, + {from: types.EnumFamily, to: types.StringFamily, volatility: VolatilityImmutable}, + + // Casts to CollatedStringFamily. + {from: types.UnknownFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, + {from: types.BoolFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, + {from: types.IntFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, + {from: types.FloatFamily, to: types.CollatedStringFamily, volatility: VolatilityStable}, + {from: types.DecimalFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, + {from: types.StringFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, + {from: types.CollatedStringFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, + {from: types.BitFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, + {from: types.ArrayFamily, to: types.CollatedStringFamily, volatility: VolatilityStable}, + {from: types.TupleFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, + {from: types.Box2DFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, + {from: types.GeometryFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, + {from: types.GeographyFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, + {from: types.BytesFamily, to: types.CollatedStringFamily, volatility: VolatilityStable}, + {from: types.TimestampFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, + {from: types.TimestampTZFamily, to: types.CollatedStringFamily, volatility: VolatilityStable}, + {from: types.IntervalFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, + {from: types.UuidFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, + {from: types.DateFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, + {from: types.TimeFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, + {from: types.TimeTZFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, + {from: types.OidFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, + {from: types.INetFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, + {from: types.JsonFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, + {from: types.EnumFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, + + // Casts to BytesFamily. + {from: types.UnknownFamily, to: types.BytesFamily, volatility: VolatilityImmutable}, + {from: types.StringFamily, to: types.BytesFamily, volatility: VolatilityImmutable}, + {from: types.CollatedStringFamily, to: types.BytesFamily, volatility: VolatilityImmutable}, + {from: types.BytesFamily, to: types.BytesFamily, volatility: VolatilityImmutable}, + {from: types.UuidFamily, to: types.BytesFamily, volatility: VolatilityImmutable}, + {from: types.GeometryFamily, to: types.BytesFamily, volatility: VolatilityImmutable}, + {from: types.GeographyFamily, to: types.BytesFamily, volatility: VolatilityImmutable}, + + // Casts to DateFamily. + {from: types.UnknownFamily, to: types.DateFamily, volatility: VolatilityImmutable}, + { + from: types.StringFamily, + to: types.DateFamily, + volatility: VolatilityStable, + volatilityHint: "STRING to DATE casts depend on session DateStyle; use parse_date(string) instead", + }, + {from: types.CollatedStringFamily, to: types.DateFamily, volatility: VolatilityStable}, + {from: types.DateFamily, to: types.DateFamily, volatility: VolatilityImmutable}, + {from: types.TimestampFamily, to: types.DateFamily, volatility: VolatilityImmutable}, + {from: types.TimestampTZFamily, to: types.DateFamily, volatility: VolatilityStable}, + {from: types.IntFamily, to: types.DateFamily, volatility: VolatilityImmutable}, + + // Casts to TimeFamily. + {from: types.UnknownFamily, to: types.TimeFamily, volatility: VolatilityImmutable}, + { + from: types.StringFamily, + to: types.TimeFamily, + volatility: VolatilityStable, + volatilityHint: "STRING to TIME casts depend on session DateStyle; use parse_time(string) instead", + }, + {from: types.CollatedStringFamily, to: types.TimeFamily, volatility: VolatilityStable}, + {from: types.TimeFamily, to: types.TimeFamily, volatility: VolatilityImmutable}, + {from: types.TimeTZFamily, to: types.TimeFamily, volatility: VolatilityImmutable}, + {from: types.TimestampFamily, to: types.TimeFamily, volatility: VolatilityImmutable}, + {from: types.TimestampTZFamily, to: types.TimeFamily, volatility: VolatilityStable}, + {from: types.IntervalFamily, to: types.TimeFamily, volatility: VolatilityImmutable}, + + // Casts to TimeTZFamily. + {from: types.UnknownFamily, to: types.TimeTZFamily, volatility: VolatilityImmutable}, + { + from: types.StringFamily, + to: types.TimeTZFamily, + volatility: VolatilityStable, + volatilityHint: "STRING to TIMETZ casts depend on session DateStyle; use parse_timetz(string) instead", + }, + {from: types.CollatedStringFamily, to: types.TimeTZFamily, volatility: VolatilityStable}, + {from: types.TimeFamily, to: types.TimeTZFamily, volatility: VolatilityStable}, + {from: types.TimeTZFamily, to: types.TimeTZFamily, volatility: VolatilityImmutable}, + {from: types.TimestampTZFamily, to: types.TimeTZFamily, volatility: VolatilityStable}, + + // Casts to TimestampFamily. + {from: types.UnknownFamily, to: types.TimestampFamily, volatility: VolatilityImmutable}, + { + from: types.StringFamily, to: types.TimestampFamily, volatility: VolatilityStable, + volatilityHint: "STRING to TIMESTAMP casts are context-dependent because of relative timestamp strings " + + "like 'now' and session settings such as DateStyle; use parse_timestamp(string) instead.", + }, + {from: types.CollatedStringFamily, to: types.TimestampFamily, volatility: VolatilityStable}, + {from: types.DateFamily, to: types.TimestampFamily, volatility: VolatilityImmutable}, + {from: types.TimestampFamily, to: types.TimestampFamily, volatility: VolatilityImmutable}, + { + from: types.TimestampTZFamily, to: types.TimestampFamily, volatility: VolatilityStable, + volatilityHint: "TIMESTAMPTZ to TIMESTAMP casts depend on the current timezone; consider using AT TIME ZONE 'UTC' instead", + }, + {from: types.IntFamily, to: types.TimestampFamily, volatility: VolatilityImmutable}, + + // Casts to TimestampTZFamily. + {from: types.UnknownFamily, to: types.TimestampTZFamily, volatility: VolatilityImmutable}, + {from: types.StringFamily, to: types.TimestampTZFamily, volatility: VolatilityStable}, + {from: types.CollatedStringFamily, to: types.TimestampTZFamily, volatility: VolatilityStable}, + {from: types.DateFamily, to: types.TimestampTZFamily, volatility: VolatilityStable}, + {from: types.TimestampFamily, to: types.TimestampTZFamily, volatility: VolatilityStable}, + {from: types.TimestampTZFamily, to: types.TimestampTZFamily, volatility: VolatilityImmutable}, + {from: types.IntFamily, to: types.TimestampTZFamily, volatility: VolatilityImmutable}, + + // Casts to IntervalFamily. + {from: types.UnknownFamily, to: types.IntervalFamily, volatility: VolatilityImmutable}, + { + from: types.StringFamily, + to: types.IntervalFamily, + volatility: VolatilityImmutable, + volatilityHint: "STRING to INTERVAL casts depend on session IntervalStyle; use parse_interval(string) instead", + }, + {from: types.CollatedStringFamily, to: types.IntervalFamily, volatility: VolatilityImmutable}, + {from: types.IntFamily, to: types.IntervalFamily, volatility: VolatilityImmutable}, + {from: types.TimeFamily, to: types.IntervalFamily, volatility: VolatilityImmutable}, + {from: types.IntervalFamily, to: types.IntervalFamily, volatility: VolatilityImmutable}, + {from: types.FloatFamily, to: types.IntervalFamily, volatility: VolatilityImmutable}, + {from: types.DecimalFamily, to: types.IntervalFamily, volatility: VolatilityImmutable}, + + // Casts to OidFamily. + {from: types.UnknownFamily, to: types.OidFamily, volatility: VolatilityImmutable}, + {from: types.StringFamily, to: types.OidFamily, volatility: VolatilityStable}, + {from: types.CollatedStringFamily, to: types.OidFamily, volatility: VolatilityStable}, + {from: types.IntFamily, to: types.OidFamily, volatility: VolatilityStable, ignoreVolatilityCheck: true}, + {from: types.OidFamily, to: types.OidFamily, volatility: VolatilityStable}, + + // Casts to UnknownFamily. + {from: types.UnknownFamily, to: types.UnknownFamily, volatility: VolatilityImmutable}, + + // Casts to UuidFamily. + {from: types.UnknownFamily, to: types.UuidFamily, volatility: VolatilityImmutable}, + {from: types.StringFamily, to: types.UuidFamily, volatility: VolatilityImmutable}, + {from: types.CollatedStringFamily, to: types.UuidFamily, volatility: VolatilityImmutable}, + {from: types.BytesFamily, to: types.UuidFamily, volatility: VolatilityImmutable}, + {from: types.UuidFamily, to: types.UuidFamily, volatility: VolatilityImmutable}, + + // Casts to INetFamily. + {from: types.UnknownFamily, to: types.INetFamily, volatility: VolatilityImmutable}, + {from: types.StringFamily, to: types.INetFamily, volatility: VolatilityImmutable}, + {from: types.CollatedStringFamily, to: types.INetFamily, volatility: VolatilityImmutable}, + {from: types.INetFamily, to: types.INetFamily, volatility: VolatilityImmutable}, + + // Casts to ArrayFamily. + {from: types.UnknownFamily, to: types.ArrayFamily, volatility: VolatilityImmutable}, + {from: types.StringFamily, to: types.ArrayFamily, volatility: VolatilityStable}, + + // Casts to JsonFamily. + {from: types.UnknownFamily, to: types.JsonFamily, volatility: VolatilityImmutable}, + {from: types.StringFamily, to: types.JsonFamily, volatility: VolatilityImmutable}, + {from: types.JsonFamily, to: types.JsonFamily, volatility: VolatilityImmutable}, + {from: types.GeometryFamily, to: types.JsonFamily, volatility: VolatilityImmutable}, + {from: types.GeographyFamily, to: types.JsonFamily, volatility: VolatilityImmutable}, + + // Casts to EnumFamily. + {from: types.UnknownFamily, to: types.EnumFamily, volatility: VolatilityImmutable}, + {from: types.StringFamily, to: types.EnumFamily, volatility: VolatilityImmutable}, + {from: types.EnumFamily, to: types.EnumFamily, volatility: VolatilityImmutable}, + {from: types.BytesFamily, to: types.EnumFamily, volatility: VolatilityImmutable}, + + // Casts to TupleFamily. + {from: types.UnknownFamily, to: types.TupleFamily, volatility: VolatilityImmutable}, +} + +type castsMapKey struct { + from, to types.Family +} + +var castsMap map[castsMapKey]*castInfo + +// styleCastsMap contains castInfos for casts affected by a style parameter. +var styleCastsMap map[castsMapKey]*castInfo + +func init() { + castsMap = make(map[castsMapKey]*castInfo, len(validCasts)) + styleCastsMap = make(map[castsMapKey]*castInfo) + for i := range validCasts { + c := &validCasts[i] + + // Initialize counter. + c.counter = sqltelemetry.CastOpCounter(c.from.Name(), c.to.Name()) + + key := castsMapKey{from: c.from, to: c.to} + castsMap[key] = c + + if isDateStyleCastAffected(c.from, c.to) || isIntervalStyleCastAffected(c.from, c.to) { + cCopy := *c + cCopy.volatility = VolatilityStable + styleCastsMap[key] = &cCopy + } + } +} + +func isIntervalStyleCastAffected(from, to types.Family) bool { + switch from { + case types.StringFamily, types.CollatedStringFamily: + switch to { + case types.IntervalFamily: + return true + } + case types.IntervalFamily: + switch to { + case types.StringFamily, types.CollatedStringFamily: + return true + } + } + return false +} + +func isDateStyleCastAffected(from, to types.Family) bool { + switch from { + case types.StringFamily, types.CollatedStringFamily: + switch to { + case types.TimeFamily, + types.TimeTZFamily, + types.DateFamily, + types.TimestampFamily: + return true + } + case types.DateFamily, + types.TimestampFamily: + switch to { + case types.StringFamily, types.CollatedStringFamily: + return true + } + } + return false +} + +// lookupCastInfo returns the information for a valid cast. +// Returns nil if this is not a valid cast. +// Does not handle array and tuple casts. +func lookupCastInfo( + from, to types.Family, intervalStyleEnabled bool, dateStyleEnabled bool, +) *castInfo { + k := castsMapKey{from: from, to: to} + if (intervalStyleEnabled && isIntervalStyleCastAffected(from, to)) || + (dateStyleEnabled && isDateStyleCastAffected(from, to)) { + if r, ok := styleCastsMap[k]; ok { + return r + } + } + return castsMap[k] +} + +// LookupCastVolatility returns the volatility of a valid cast. +func LookupCastVolatility(from, to *types.T, sd *sessiondata.SessionData) (_ Volatility, ok bool) { + fromFamily := from.Family() + toFamily := to.Family() + // Special case for casting between arrays. + if fromFamily == types.ArrayFamily && toFamily == types.ArrayFamily { + return LookupCastVolatility(from.ArrayContents(), to.ArrayContents(), sd) + } + // Special case for casting between tuples. + if fromFamily == types.TupleFamily && toFamily == types.TupleFamily { + fromTypes := from.TupleContents() + toTypes := to.TupleContents() + // Handle case where an overload makes a tuple get casted to tuple{}. + if len(toTypes) == 1 && toTypes[0].Family() == types.AnyFamily { + return VolatilityStable, true + } + if len(fromTypes) != len(toTypes) { + return 0, false + } + maxVolatility := VolatilityLeakProof + for i := range fromTypes { + v, ok := LookupCastVolatility(fromTypes[i], toTypes[i], sd) + if !ok { + return 0, false + } + if v > maxVolatility { + maxVolatility = v + } + } + return maxVolatility, true + } + + cast := lookupCastInfo( + fromFamily, + toFamily, + sd != nil && sd.IntervalStyleEnabled, + sd != nil && sd.DateStyleEnabled, + ) + if cast == nil { + return 0, false + } + return cast.volatility, true +} + +// PerformCast performs a cast from the provided Datum to the specified +// types.T. +func PerformCast(ctx *EvalContext, d Datum, t *types.T) (Datum, error) { + ret, err := performCastWithoutPrecisionTruncation(ctx, d, t, true /* truncateWidth */) + if err != nil { + return nil, err + } + return AdjustValueToType(t, ret) +} + +// PerformAssignmentCast performs an assignment cast from the provided Datum to +// the specified type. It is similar to PerformCast, but differs because it +// errors if the datum's width is too wide for the given type rather than +// silently truncating. The one exception is casts to the special "char" type +// which are truncated. +func PerformAssignmentCast(ctx *EvalContext, d Datum, t *types.T) (Datum, error) { + if !ValidCast(d.ResolvedType(), t, CastContextAssignment) { + return nil, pgerror.Newf( + pgcode.CannotCoerce, + "invalid assignment cast: %s -> %s", d.ResolvedType(), t, + ) + } + d, err := performCastWithoutPrecisionTruncation(ctx, d, t, false /* truncateWidth */) + if err != nil { + return nil, err + } + return AdjustValueToType(t, d) +} + +// AdjustValueToType checks that the width (for strings, byte arrays, and bit +// strings) and scale (decimal). and, shape/srid (for geospatial types) fits the +// specified column type. +// +// Additionally, some precision truncation may occur for the specified column type. +// +// In case of decimals, it can truncate fractional digits in the input +// value in order to fit the target column. If the input value fits the target +// column, it is returned unchanged. If the input value can be truncated to fit, +// then a truncated copy is returned. Otherwise, an error is returned. +// +// In the case of time, it can truncate fractional digits of time datums +// to its relevant rounding for the given type definition. +// +// In the case of geospatial types, it will check whether the SRID and Shape in the +// datum matches the type definition. +// +// This method is used by casts and parsing. It is important to note that this +// function will error if the given value is too wide for the given type. For +// explicit casts and parsing, inVal should be truncated before this function is +// called so that an error is not returned. For assignment casts, inVal should +// not be truncated before this function is called, so that an error is +// returned. The one exception for assignment casts is for the special "char" +// type. An assignment cast to "char" does not error and truncates a value if +// the width of the value is wider than a single character. For this exception, +// AdjustValueToType performs the truncation itself. +func AdjustValueToType(typ *types.T, inVal Datum) (outVal Datum, err error) { + switch typ.Family() { + case types.StringFamily, types.CollatedStringFamily: + var sv string + if v, ok := AsDString(inVal); ok { + sv = string(v) + } else if v, ok := inVal.(*DCollatedString); ok { + sv = v.Contents + } + sv = adjustStringValueToType(typ, sv) + if typ.Width() > 0 && utf8.RuneCountInString(sv) > int(typ.Width()) { + return nil, pgerror.Newf(pgcode.StringDataRightTruncation, + "value too long for type %s", + typ.SQLString()) + } + + if typ.Oid() == oid.T_bpchar || typ.Oid() == oid.T_char { + if _, ok := AsDString(inVal); ok { + return NewDString(sv), nil + } else if _, ok := inVal.(*DCollatedString); ok { + return NewDCollatedString(sv, typ.Locale(), &CollationEnvironment{}) + } + } + case types.IntFamily: + if v, ok := AsDInt(inVal); ok { + if typ.Width() == 32 || typ.Width() == 16 { + // Width is defined in bits. + width := uint(typ.Width() - 1) + + // We're performing range checks in line with Go's + // implementation of math.(Max|Min)(16|32) numbers that store + // the boundaries of the allowed range. + // NOTE: when updating the code below, make sure to update + // execgen/cast_gen_util.go as well. + shifted := v >> width + if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) { + if typ.Width() == 16 { + return nil, ErrInt2OutOfRange + } + return nil, ErrInt4OutOfRange + } + } + } + case types.BitFamily: + if v, ok := AsDBitArray(inVal); ok { + if typ.Width() > 0 { + bitLen := v.BitLen() + switch typ.Oid() { + case oid.T_varbit: + if bitLen > uint(typ.Width()) { + return nil, pgerror.Newf(pgcode.StringDataRightTruncation, + "bit string length %d too large for type %s", bitLen, typ.SQLString()) + } + default: + if bitLen != uint(typ.Width()) { + return nil, pgerror.Newf(pgcode.StringDataLengthMismatch, + "bit string length %d does not match type %s", bitLen, typ.SQLString()) + } + } + } + } + case types.DecimalFamily: + if inDec, ok := inVal.(*DDecimal); ok { + if inDec.Form != apd.Finite || typ.Precision() == 0 { + // Non-finite form or unlimited target precision, so no need to limit. + break + } + if int64(typ.Precision()) >= inDec.NumDigits() && typ.Scale() == inDec.Exponent { + // Precision and scale of target column are sufficient. + break + } + + var outDec DDecimal + outDec.Set(&inDec.Decimal) + err := LimitDecimalWidth(&outDec.Decimal, int(typ.Precision()), int(typ.Scale())) + if err != nil { + return nil, errors.Wrapf(err, "type %s", typ.SQLString()) + } + return &outDec, nil + } + case types.ArrayFamily: + if inArr, ok := inVal.(*DArray); ok { + var outArr *DArray + elementType := typ.ArrayContents() + for i, inElem := range inArr.Array { + outElem, err := AdjustValueToType(elementType, inElem) + if err != nil { + return nil, err + } + if outElem != inElem { + if outArr == nil { + outArr = &DArray{} + *outArr = *inArr + outArr.Array = make(Datums, len(inArr.Array)) + copy(outArr.Array, inArr.Array[:i]) + } + } + if outArr != nil { + outArr.Array[i] = inElem + } + } + if outArr != nil { + return outArr, nil + } + } + case types.TimeFamily: + if in, ok := inVal.(*DTime); ok { + return in.Round(TimeFamilyPrecisionToRoundDuration(typ.Precision())), nil + } + case types.TimestampFamily: + if in, ok := inVal.(*DTimestamp); ok { + return in.Round(TimeFamilyPrecisionToRoundDuration(typ.Precision())) + } + case types.TimestampTZFamily: + if in, ok := inVal.(*DTimestampTZ); ok { + return in.Round(TimeFamilyPrecisionToRoundDuration(typ.Precision())) + } + case types.TimeTZFamily: + if in, ok := inVal.(*DTimeTZ); ok { + return in.Round(TimeFamilyPrecisionToRoundDuration(typ.Precision())), nil + } + case types.IntervalFamily: + if in, ok := inVal.(*DInterval); ok { + itm, err := typ.IntervalTypeMetadata() + if err != nil { + return nil, err + } + return NewDInterval(in.Duration, itm), nil + } + case types.GeometryFamily: + if in, ok := inVal.(*DGeometry); ok { + if err := geo.SpatialObjectFitsColumnMetadata( + in.Geometry.SpatialObject(), + typ.InternalType.GeoMetadata.SRID, + typ.InternalType.GeoMetadata.ShapeType, + ); err != nil { + return nil, err + } + } + case types.GeographyFamily: + if in, ok := inVal.(*DGeography); ok { + if err := geo.SpatialObjectFitsColumnMetadata( + in.Geography.SpatialObject(), + typ.InternalType.GeoMetadata.SRID, + typ.InternalType.GeoMetadata.ShapeType, + ); err != nil { + return nil, err + } + } + } + return inVal, nil +} + +// adjustStringToType checks that the width for strings fits the +// specified column type. +func adjustStringValueToType(typ *types.T, sv string) string { + switch typ.Oid() { + case oid.T_char: + // "char" is supposed to truncate long values + return util.TruncateString(sv, 1) + case oid.T_bpchar: + // bpchar types truncate trailing whitespace. + return strings.TrimRight(sv, " ") + } + return sv +} + +// formatBitArrayToType formats bit arrays such that they fill the total width +// if too short, or truncate if too long. +func formatBitArrayToType(d *DBitArray, t *types.T) *DBitArray { + if t.Width() == 0 || d.BitLen() == uint(t.Width()) { + return d + } + a := d.BitArray.Clone() + switch t.Oid() { + case oid.T_varbit: + // VARBITs do not have padding attached, so only truncate. + if uint(t.Width()) < a.BitLen() { + a = a.ToWidth(uint(t.Width())) + } + default: + a = a.ToWidth(uint(t.Width())) + } + return &DBitArray{a} +} + +// performCastWithoutPrecisionTruncation performs the cast, but does not perform +// precision truncation. For example, if d is of type DECIMAL(6, 2) and t is +// DECIMAL(4, 2), d is not truncated to fit into t. However, if truncateWidth is +// true, widths are truncated to match the target type t for some types, +// including the bit and string types. If truncateWidth is false, the input +// datum is not truncated. +// +// In an ideal state, components of AdjustValueToType should be embedded into +// this function, but the code base needs a general refactor of parsing +// and casting logic before this can happen. +// See also: #55094. +func performCastWithoutPrecisionTruncation( + ctx *EvalContext, d Datum, t *types.T, truncateWidth bool, +) (Datum, error) { + // No conversion is needed if d is NULL. + if d == DNull { + return d, nil + } + + // If we're casting a DOidWrapper, then we want to cast the wrapped datum. + // It is also reasonable to lose the old Oid value too. + // Note that we pass in nil as the first argument since we're not interested + // in evaluating the placeholders. + d = UnwrapDatum(nil /* evalCtx */, d) + switch t.Family() { + case types.BitFamily: + var ba *DBitArray + switch v := d.(type) { + case *DBitArray: + ba = v + case *DInt: + var err error + ba, err = NewDBitArrayFromInt(int64(*v), uint(t.Width())) + if err != nil { + return nil, err + } + case *DString: + res, err := bitarray.Parse(string(*v)) + if err != nil { + return nil, err + } + ba = &DBitArray{res} + case *DCollatedString: + res, err := bitarray.Parse(v.Contents) + if err != nil { + return nil, err + } + ba = &DBitArray{res} + } + if truncateWidth { + ba = formatBitArrayToType(ba, t) + } + return ba, nil + + case types.BoolFamily: + switch v := d.(type) { + case *DBool: + return d, nil + case *DInt: + return MakeDBool(*v != 0), nil + case *DFloat: + return MakeDBool(*v != 0), nil + case *DDecimal: + return MakeDBool(v.Sign() != 0), nil + case *DString: + return ParseDBool(string(*v)) + case *DCollatedString: + return ParseDBool(v.Contents) + case *DJSON: + b, ok := v.AsBool() + if !ok { + return nil, failedCastFromJSON(v, t) + } + return MakeDBool(DBool(b)), nil + } + + case types.IntFamily: + var res *DInt + switch v := d.(type) { + case *DBitArray: + res = v.AsDInt(uint(t.Width())) + case *DBool: + if *v { + res = NewDInt(1) + } else { + res = DZero + } + case *DInt: + // TODO(knz): enforce the coltype width here. + res = v + case *DFloat: + f := float64(*v) + // Use `<=` and `>=` here instead of just `<` and `>` because when + // math.MaxInt64 and math.MinInt64 are converted to float64s, they are + // rounded to numbers with larger absolute values. Note that the first + // next FP value after and strictly greater than float64(math.MinInt64) + // is -9223372036854774784 (= float64(math.MinInt64)+513) and the first + // previous value and strictly smaller than float64(math.MaxInt64) + // is 9223372036854774784 (= float64(math.MaxInt64)-513), and both are + // convertible to int without overflow. + if math.IsNaN(f) || f <= float64(math.MinInt64) || f >= float64(math.MaxInt64) { + return nil, ErrIntOutOfRange + } + res = NewDInt(DInt(f)) + case *DDecimal: + i, err := roundDecimalToInt(ctx, &v.Decimal) + if err != nil { + return nil, err + } + res = NewDInt(DInt(i)) + case *DString: + var err error + if res, err = ParseDInt(string(*v)); err != nil { + return nil, err + } + case *DCollatedString: + var err error + if res, err = ParseDInt(v.Contents); err != nil { + return nil, err + } + case *DTimestamp: + res = NewDInt(DInt(v.Unix())) + case *DTimestampTZ: + res = NewDInt(DInt(v.Unix())) + case *DDate: + // TODO(mjibson): This cast is unsupported by postgres. Should we remove ours? + if !v.IsFinite() { + return nil, ErrIntOutOfRange + } + res = NewDInt(DInt(v.UnixEpochDays())) + case *DInterval: + iv, ok := v.AsInt64() + if !ok { + return nil, ErrIntOutOfRange + } + res = NewDInt(DInt(iv)) + case *DOid: + res = &v.DInt + case *DJSON: + dec, ok := v.AsDecimal() + if !ok { + return nil, failedCastFromJSON(v, t) + } + i, err := dec.Int64() + if err != nil { + // Attempt to round the number to an integer. + i, err = roundDecimalToInt(ctx, dec) + if err != nil { + return nil, err + } + } + res = NewDInt(DInt(i)) + } + if res != nil { + return res, nil + } + + case types.EnumFamily: + switch v := d.(type) { + case *DString: + return MakeDEnumFromLogicalRepresentation(t, string(*v)) + case *DBytes: + return MakeDEnumFromPhysicalRepresentation(t, []byte(*v)) + case *DEnum: + return d, nil + } + + case types.FloatFamily: + switch v := d.(type) { + case *DBool: + if *v { + return NewDFloat(1), nil + } + return NewDFloat(0), nil + case *DInt: + return NewDFloat(DFloat(*v)), nil + case *DFloat: + return d, nil + case *DDecimal: + f, err := v.Float64() + if err != nil { + return nil, ErrFloatOutOfRange + } + return NewDFloat(DFloat(f)), nil + case *DString: + return ParseDFloat(string(*v)) + case *DCollatedString: + return ParseDFloat(v.Contents) + case *DTimestamp: + micros := float64(v.Nanosecond() / int(time.Microsecond)) + return NewDFloat(DFloat(float64(v.Unix()) + micros*1e-6)), nil + case *DTimestampTZ: + micros := float64(v.Nanosecond() / int(time.Microsecond)) + return NewDFloat(DFloat(float64(v.Unix()) + micros*1e-6)), nil + case *DDate: + // TODO(mjibson): This cast is unsupported by postgres. Should we remove ours? + if !v.IsFinite() { + return nil, ErrFloatOutOfRange + } + return NewDFloat(DFloat(float64(v.UnixEpochDays()))), nil + case *DInterval: + return NewDFloat(DFloat(v.AsFloat64())), nil + case *DJSON: + dec, ok := v.AsDecimal() + if !ok { + return nil, failedCastFromJSON(v, t) + } + fl, err := dec.Float64() + if err != nil { + return nil, ErrFloatOutOfRange + } + return NewDFloat(DFloat(fl)), nil + } + + case types.DecimalFamily: + var dd DDecimal + var err error + unset := false + switch v := d.(type) { + case *DBool: + if *v { + dd.SetInt64(1) + } + case *DInt: + dd.SetInt64(int64(*v)) + case *DDate: + // TODO(mjibson): This cast is unsupported by postgres. Should we remove ours? + if !v.IsFinite() { + return nil, errDecOutOfRange + } + dd.SetInt64(v.UnixEpochDays()) + case *DFloat: + _, err = dd.SetFloat64(float64(*v)) + case *DDecimal: + // Small optimization to avoid copying into dd in normal case. + if t.Precision() == 0 { + return d, nil + } + dd.Set(&v.Decimal) + case *DString: + err = dd.SetString(string(*v)) + case *DCollatedString: + err = dd.SetString(v.Contents) + case *DTimestamp: + val := &dd.Coeff + val.SetInt64(v.Unix()) + val.Mul(val, big10E6) + micros := v.Nanosecond() / int(time.Microsecond) + val.Add(val, big.NewInt(int64(micros))) + dd.Exponent = -6 + case *DTimestampTZ: + val := &dd.Coeff + val.SetInt64(v.Unix()) + val.Mul(val, big10E6) + micros := v.Nanosecond() / int(time.Microsecond) + val.Add(val, big.NewInt(int64(micros))) + dd.Exponent = -6 + case *DInterval: + v.AsBigInt(&dd.Coeff) + dd.Exponent = -9 + case *DJSON: + dec, ok := v.AsDecimal() + if !ok { + return nil, failedCastFromJSON(v, t) + } + dd.Set(dec) + default: + unset = true + } + if err != nil { + return nil, err + } + if !unset { + // dd.Coeff must be positive. If it was set to a negative value + // above, transfer the sign to dd.Negative. + if dd.Coeff.Sign() < 0 { + dd.Negative = true + dd.Coeff.Abs(&dd.Coeff) + } + err = LimitDecimalWidth(&dd.Decimal, int(t.Precision()), int(t.Scale())) + if err != nil { + return nil, errors.Wrapf(err, "type %s", t.SQLString()) + } + return &dd, nil + } + + case types.StringFamily, types.CollatedStringFamily: + var s string + typ := t + switch t := d.(type) { + case *DBitArray: + s = t.BitArray.String() + case *DFloat: + s = strconv.FormatFloat(float64(*t), 'g', + ctx.SessionData().DataConversionConfig.GetFloatPrec(), 64) + case *DInt: + if typ.Oid() == oid.T_char { + // int to "char" casts just return the correspondong ASCII byte. + if *t > math.MaxInt8 || *t < math.MinInt8 { + return nil, errCharOutOfRange + } else if *t == 0 { + s = "" + } else { + s = string([]byte{byte(*t)}) + } + } else { + s = d.String() + } + case *DBool, *DDecimal: + s = d.String() + case *DTimestamp, *DDate, *DTime, *DTimeTZ, *DGeography, *DGeometry, *DBox2D: + s = AsStringWithFlags(d, FmtBareStrings) + case *DTimestampTZ: + // Convert to context timezone for correct display. + ts, err := MakeDTimestampTZ(t.In(ctx.GetLocation()), time.Microsecond) + if err != nil { + return nil, err + } + s = AsStringWithFlags( + ts, + FmtBareStrings, + ) + case *DTuple: + s = AsStringWithFlags( + d, + FmtPgwireText, + FmtDataConversionConfig(ctx.SessionData().DataConversionConfig), + ) + case *DArray: + s = AsStringWithFlags( + d, + FmtPgwireText, + FmtDataConversionConfig(ctx.SessionData().DataConversionConfig), + ) + case *DInterval: + // When converting an interval to string, we need a string representation + // of the duration (e.g. "5s") and not of the interval itself (e.g. + // "INTERVAL '5s'"). + s = AsStringWithFlags( + d, + FmtPgwireText, + FmtDataConversionConfig(ctx.SessionData().DataConversionConfig), + ) + case *DUuid: + s = t.UUID.String() + case *DIPAddr: + s = AsStringWithFlags(d, FmtBareStrings) + case *DString: + s = string(*t) + case *DCollatedString: + s = t.Contents + case *DBytes: + s = lex.EncodeByteArrayToRawBytes( + string(*t), + ctx.SessionData().DataConversionConfig.BytesEncodeFormat, + false, /* skipHexPrefix */ + ) + case *DOid: + s = t.String() + case *DJSON: + s = t.JSON.String() + case *DEnum: + s = t.LogicalRep + } + switch t.Family() { + case types.StringFamily: + if t.Oid() == oid.T_name { + return NewDName(s), nil + } + + // bpchar types truncate trailing whitespace. + if t.Oid() == oid.T_bpchar { + s = strings.TrimRight(s, " ") + } + + // If the string type specifies a limit we truncate to that limit: + // 'hello'::CHAR(2) -> 'he' + // This is true of all the string type variants. + if truncateWidth && t.Width() > 0 { + s = util.TruncateString(s, int(t.Width())) + } + return NewDString(s), nil + case types.CollatedStringFamily: + // bpchar types truncate trailing whitespace. + if t.Oid() == oid.T_bpchar { + s = strings.TrimRight(s, " ") + } + // Ditto truncation like for TString. + if truncateWidth && t.Width() > 0 { + s = util.TruncateString(s, int(t.Width())) + } + return NewDCollatedString(s, t.Locale(), &ctx.CollationEnv) + } + + case types.BytesFamily: + switch t := d.(type) { + case *DString: + return ParseDByte(string(*t)) + case *DCollatedString: + return NewDBytes(DBytes(t.Contents)), nil + case *DUuid: + return NewDBytes(DBytes(t.GetBytes())), nil + case *DBytes: + return d, nil + case *DGeography: + return NewDBytes(DBytes(t.Geography.EWKB())), nil + case *DGeometry: + return NewDBytes(DBytes(t.Geometry.EWKB())), nil + } + + case types.UuidFamily: + switch t := d.(type) { + case *DString: + return ParseDUuidFromString(string(*t)) + case *DCollatedString: + return ParseDUuidFromString(t.Contents) + case *DBytes: + return ParseDUuidFromBytes([]byte(*t)) + case *DUuid: + return d, nil + } + + case types.INetFamily: + switch t := d.(type) { + case *DString: + return ParseDIPAddrFromINetString(string(*t)) + case *DCollatedString: + return ParseDIPAddrFromINetString(t.Contents) + case *DIPAddr: + return d, nil + } + + case types.Box2DFamily: + switch d := d.(type) { + case *DString: + return ParseDBox2D(string(*d)) + case *DCollatedString: + return ParseDBox2D(d.Contents) + case *DBox2D: + return d, nil + case *DGeometry: + bbox := d.CartesianBoundingBox() + if bbox == nil { + return DNull, nil + } + return NewDBox2D(*bbox), nil + } + + case types.GeographyFamily: + switch d := d.(type) { + case *DString: + return ParseDGeography(string(*d)) + case *DCollatedString: + return ParseDGeography(d.Contents) + case *DGeography: + if err := geo.SpatialObjectFitsColumnMetadata( + d.Geography.SpatialObject(), + t.InternalType.GeoMetadata.SRID, + t.InternalType.GeoMetadata.ShapeType, + ); err != nil { + return nil, err + } + return d, nil + case *DGeometry: + g, err := d.AsGeography() + if err != nil { + return nil, err + } + if err := geo.SpatialObjectFitsColumnMetadata( + g.SpatialObject(), + t.InternalType.GeoMetadata.SRID, + t.InternalType.GeoMetadata.ShapeType, + ); err != nil { + return nil, err + } + return &DGeography{g}, nil + case *DJSON: + t, err := d.AsText() + if err != nil { + return nil, err + } + if t == nil { + return DNull, nil + } + g, err := geo.ParseGeographyFromGeoJSON([]byte(*t)) + if err != nil { + return nil, err + } + return &DGeography{g}, nil + case *DBytes: + g, err := geo.ParseGeographyFromEWKB(geopb.EWKB(*d)) + if err != nil { + return nil, err + } + return &DGeography{g}, nil + } + case types.GeometryFamily: + switch d := d.(type) { + case *DString: + return ParseDGeometry(string(*d)) + case *DCollatedString: + return ParseDGeometry(d.Contents) + case *DGeometry: + if err := geo.SpatialObjectFitsColumnMetadata( + d.Geometry.SpatialObject(), + t.InternalType.GeoMetadata.SRID, + t.InternalType.GeoMetadata.ShapeType, + ); err != nil { + return nil, err + } + return d, nil + case *DGeography: + if err := geo.SpatialObjectFitsColumnMetadata( + d.Geography.SpatialObject(), + t.InternalType.GeoMetadata.SRID, + t.InternalType.GeoMetadata.ShapeType, + ); err != nil { + return nil, err + } + g, err := d.AsGeometry() + if err != nil { + return nil, err + } + return &DGeometry{g}, nil + case *DJSON: + t, err := d.AsText() + if err != nil { + return nil, err + } + if t == nil { + return DNull, nil + } + g, err := geo.ParseGeometryFromGeoJSON([]byte(*t)) + if err != nil { + return nil, err + } + return &DGeometry{g}, nil + case *DBox2D: + g, err := geo.MakeGeometryFromGeomT(d.ToGeomT(geopb.DefaultGeometrySRID)) + if err != nil { + return nil, err + } + return &DGeometry{g}, nil + case *DBytes: + g, err := geo.ParseGeometryFromEWKB(geopb.EWKB(*d)) + if err != nil { + return nil, err + } + return &DGeometry{g}, nil + } + + case types.DateFamily: + switch d := d.(type) { + case *DString: + res, _, err := ParseDDate(ctx, string(*d)) + return res, err + case *DCollatedString: + res, _, err := ParseDDate(ctx, d.Contents) + return res, err + case *DDate: + return d, nil + case *DInt: + // TODO(mjibson): This cast is unsupported by postgres. Should we remove ours? + t, err := pgdate.MakeDateFromUnixEpoch(int64(*d)) + return NewDDate(t), err + case *DTimestampTZ: + return NewDDateFromTime(d.Time.In(ctx.GetLocation())) + case *DTimestamp: + return NewDDateFromTime(d.Time) + } + + case types.TimeFamily: + roundTo := TimeFamilyPrecisionToRoundDuration(t.Precision()) + switch d := d.(type) { + case *DString: + res, _, err := ParseDTime(ctx, string(*d), roundTo) + return res, err + case *DCollatedString: + res, _, err := ParseDTime(ctx, d.Contents, roundTo) + return res, err + case *DTime: + return d.Round(roundTo), nil + case *DTimeTZ: + return MakeDTime(d.TimeOfDay.Round(roundTo)), nil + case *DTimestamp: + return MakeDTime(timeofday.FromTime(d.Time).Round(roundTo)), nil + case *DTimestampTZ: + // Strip time zone. Times don't carry their location. + stripped, err := d.stripTimeZone(ctx) + if err != nil { + return nil, err + } + return MakeDTime(timeofday.FromTime(stripped.Time).Round(roundTo)), nil + case *DInterval: + return MakeDTime(timeofday.Min.Add(d.Duration).Round(roundTo)), nil + } + + case types.TimeTZFamily: + roundTo := TimeFamilyPrecisionToRoundDuration(t.Precision()) + switch d := d.(type) { + case *DString: + res, _, err := ParseDTimeTZ(ctx, string(*d), roundTo) + return res, err + case *DCollatedString: + res, _, err := ParseDTimeTZ(ctx, d.Contents, roundTo) + return res, err + case *DTime: + return NewDTimeTZFromLocation(timeofday.TimeOfDay(*d).Round(roundTo), ctx.GetLocation()), nil + case *DTimeTZ: + return d.Round(roundTo), nil + case *DTimestampTZ: + return NewDTimeTZFromTime(d.Time.In(ctx.GetLocation()).Round(roundTo)), nil + } + + case types.TimestampFamily: + roundTo := TimeFamilyPrecisionToRoundDuration(t.Precision()) + // TODO(knz): Timestamp from float, decimal. + switch d := d.(type) { + case *DString: + res, _, err := ParseDTimestamp(ctx, string(*d), roundTo) + return res, err + case *DCollatedString: + res, _, err := ParseDTimestamp(ctx, d.Contents, roundTo) + return res, err + case *DDate: + t, err := d.ToTime() + if err != nil { + return nil, err + } + return MakeDTimestamp(t, roundTo) + case *DInt: + return MakeDTimestamp(timeutil.Unix(int64(*d), 0), roundTo) + case *DTimestamp: + return d.Round(roundTo) + case *DTimestampTZ: + // Strip time zone. Timestamps don't carry their location. + stripped, err := d.stripTimeZone(ctx) + if err != nil { + return nil, err + } + return stripped.Round(roundTo) + } + + case types.TimestampTZFamily: + roundTo := TimeFamilyPrecisionToRoundDuration(t.Precision()) + // TODO(knz): TimestampTZ from float, decimal. + switch d := d.(type) { + case *DString: + res, _, err := ParseDTimestampTZ(ctx, string(*d), roundTo) + return res, err + case *DCollatedString: + res, _, err := ParseDTimestampTZ(ctx, d.Contents, roundTo) + return res, err + case *DDate: + t, err := d.ToTime() + if err != nil { + return nil, err + } + _, before := t.Zone() + _, after := t.In(ctx.GetLocation()).Zone() + return MakeDTimestampTZ(t.Add(time.Duration(before-after)*time.Second), roundTo) + case *DTimestamp: + _, before := d.Time.Zone() + _, after := d.Time.In(ctx.GetLocation()).Zone() + return MakeDTimestampTZ(d.Time.Add(time.Duration(before-after)*time.Second), roundTo) + case *DInt: + return MakeDTimestampTZ(timeutil.Unix(int64(*d), 0), roundTo) + case *DTimestampTZ: + return d.Round(roundTo) + } + + case types.IntervalFamily: + itm, err := t.IntervalTypeMetadata() + if err != nil { + return nil, err + } + switch v := d.(type) { + case *DString: + return ParseDIntervalWithTypeMetadata(ctx.GetIntervalStyle(), string(*v), itm) + case *DCollatedString: + return ParseDIntervalWithTypeMetadata(ctx.GetIntervalStyle(), v.Contents, itm) + case *DInt: + return NewDInterval(duration.FromInt64(int64(*v)), itm), nil + case *DFloat: + return NewDInterval(duration.FromFloat64(float64(*v)), itm), nil + case *DTime: + return NewDInterval(duration.MakeDuration(int64(*v)*1000, 0, 0), itm), nil + case *DDecimal: + d := ctx.getTmpDec() + dnanos := v.Decimal + dnanos.Exponent += 9 + // We need HighPrecisionCtx because duration values can contain + // upward of 35 decimal digits and DecimalCtx only provides 25. + _, err := HighPrecisionCtx.Quantize(d, &dnanos, 0) + if err != nil { + return nil, err + } + if dnanos.Negative { + d.Coeff.Neg(&d.Coeff) + } + dv, ok := duration.FromBigInt(&d.Coeff) + if !ok { + return nil, errDecOutOfRange + } + return NewDInterval(dv, itm), nil + case *DInterval: + return NewDInterval(v.Duration, itm), nil + } + case types.JsonFamily: + switch v := d.(type) { + case *DString: + return ParseDJSON(string(*v)) + case *DJSON: + return v, nil + case *DGeography: + j, err := geo.SpatialObjectToGeoJSON(v.Geography.SpatialObject(), -1, geo.SpatialObjectToGeoJSONFlagZero) + if err != nil { + return nil, err + } + return ParseDJSON(string(j)) + case *DGeometry: + j, err := geo.SpatialObjectToGeoJSON(v.Geometry.SpatialObject(), -1, geo.SpatialObjectToGeoJSONFlagZero) + if err != nil { + return nil, err + } + return ParseDJSON(string(j)) + } + case types.ArrayFamily: + switch v := d.(type) { + case *DString: + res, _, err := ParseDArrayFromString(ctx, string(*v), t.ArrayContents()) + return res, err + case *DArray: + dcast := NewDArray(t.ArrayContents()) + for _, e := range v.Array { + ecast := DNull + if e != DNull { + var err error + ecast, err = PerformCast(ctx, e, t.ArrayContents()) + if err != nil { + return nil, err + } + } + + if err := dcast.Append(ecast); err != nil { + return nil, err + } + } + return dcast, nil + } + case types.OidFamily: + switch v := d.(type) { + case *DOid: + return performIntToOidCast(ctx, t, v.DInt) + case *DInt: + // OIDs are always unsigned 32-bit integers. Some languages, like Java, + // store OIDs as signed 32-bit integers, so we implement the cast + // by converting to a uint32 first. This matches Postgres behavior. + i := DInt(uint32(*v)) + return performIntToOidCast(ctx, t, i) + case *DString: + return ParseDOid(ctx, string(*v), t) + } + } + + return nil, pgerror.Newf( + pgcode.CannotCoerce, "invalid cast: %s -> %s", d.ResolvedType(), t) +} + +// performIntToOidCast casts the input integer to the OID type given by the +// input types.T. +func performIntToOidCast(ctx *EvalContext, t *types.T, v DInt) (Datum, error) { + switch t.Oid() { + case oid.T_oid: + return &DOid{semanticType: t, DInt: v}, nil + case oid.T_regtype: + // Mapping an oid to a regtype is easy: we have a hardcoded map. + ret := &DOid{semanticType: t, DInt: v} + if typ, ok := types.OidToType[oid.Oid(v)]; ok { + ret.name = typ.PGName() + } else if types.IsOIDUserDefinedType(oid.Oid(v)) { + typ, err := ctx.Planner.ResolveTypeByOID(ctx.Context, oid.Oid(v)) + if err != nil { + return nil, err + } + ret.name = typ.PGName() + } + return ret, nil + + case oid.T_regproc, oid.T_regprocedure: + // Mapping an oid to a regproc is easy: we have a hardcoded map. + name, ok := OidToBuiltinName[oid.Oid(v)] + ret := &DOid{semanticType: t, DInt: v} + if !ok { + return ret, nil + } + ret.name = name + return ret, nil + + default: + oid, err := ctx.Planner.ResolveOIDFromOID(ctx.Ctx(), t, NewDOid(v)) + if err != nil { + oid = NewDOid(v) + oid.semanticType = t + } + return oid, nil + } +} + +func roundDecimalToInt(ctx *EvalContext, d *apd.Decimal) (int64, error) { + newD := ctx.getTmpDec() + if _, err := DecimalCtx.RoundToIntegralValue(newD, d); err != nil { + return 0, err + } + i, err := newD.Int64() + if err != nil { + return 0, ErrIntOutOfRange + } + return i, nil +} + +func failedCastFromJSON(j *DJSON, t *types.T) error { + return pgerror.Newf( + pgcode.InvalidParameterValue, + "cannot cast jsonb %s to type %s", + j.Type(), t, + ) +} + +// PopulateRecordWithJSON is used for the json to record function family, like +// json_populate_record. Given a JSON object, a desired tuple type, and a tuple +// of the same length as the desired type, this function will populate the tuple +// by setting each named field in the tuple to the value of the key with the +// same name in the input JSON object. Fields in the tuple that are not present +// in the JSON will not be modified, and JSON keys that are not in the tuple +// will be ignored. +// Each field will be set by a best-effort coercion to its type from the JSON +// field. The logic is more permissive than casts. +func PopulateRecordWithJSON( + ctx *EvalContext, j json.JSON, desiredType *types.T, tup *DTuple, +) error { + if j.Type() != json.ObjectJSONType { + return pgerror.Newf(pgcode.InvalidParameterValue, "expected JSON object") + } + tupleTypes := desiredType.TupleContents() + labels := desiredType.TupleLabels() + if labels == nil { + return pgerror.Newf( + pgcode.InvalidParameterValue, + "anonymous records cannot be used with json{b}_populate_record{set}", + ) + } + for i := range tupleTypes { + val, err := j.FetchValKey(labels[i]) + if err != nil || val == nil { + // No value? Use the value that was already in the tuple. + continue + } + tup.D[i], err = PopulateDatumWithJSON(ctx, val, tupleTypes[i]) + if err != nil { + return err + } + } + return nil +} + +// PopulateDatumWithJSON is used for the json to record function family, like +// json_populate_record. It's less restrictive than the casting system, which +// is why it's implemented separately. +func PopulateDatumWithJSON(ctx *EvalContext, j json.JSON, desiredType *types.T) (Datum, error) { + if j == json.NullJSONValue { + return DNull, nil + } + switch desiredType.Family() { + case types.ArrayFamily: + if j.Type() != json.ArrayJSONType { + return nil, pgerror.Newf(pgcode.InvalidTextRepresentation, "expected JSON array") + } + n := j.Len() + elementTyp := desiredType.ArrayContents() + d := NewDArray(elementTyp) + d.Array = make(Datums, n) + for i := 0; i < n; i++ { + elt, err := j.FetchValIdx(i) + if err != nil { + return nil, err + } + d.Array[i], err = PopulateDatumWithJSON(ctx, elt, elementTyp) + if err != nil { + return nil, err + } + } + return d, nil + case types.TupleFamily: + tup := NewDTupleWithLen(desiredType, len(desiredType.TupleContents())) + for i := range tup.D { + tup.D[i] = DNull + } + err := PopulateRecordWithJSON(ctx, j, desiredType, tup) + return tup, err + } + var s string + switch j.Type() { + case json.StringJSONType: + t, err := j.AsText() + if err != nil { + return nil, err + } + s = *t + default: + s = j.String() + } + return PerformCast(ctx, NewDString(s), desiredType) +} diff --git a/pkg/sql/sem/tree/cast_map_gen.sh b/pkg/sql/sem/tree/cast_map_gen.sh new file mode 100755 index 000000000000..99597d170e0d --- /dev/null +++ b/pkg/sql/sem/tree/cast_map_gen.sh @@ -0,0 +1,80 @@ +#!/bin/bash + +# casts_gen.sh generates castMap entries by reading from Postgres's pg_cast +# table. To use this script, Postgres must be installed with the PostGIS +# extension and already running. +# +# By default the script connects to the "postgres" database. To use a different +# database, supply its name as the first argument to the script. + +DATABASE="${1-postgres}" + +psql $DATABASE -Xqtc "SELECT postgis_full_version()" &> /dev/null +if [ $? -ne 0 ]; then + echo "error: postgis must be installed in database $DATABASE"; + echo "hint: you can specify another database as the first argument"; + exit 1; +fi + +PG_CAST_QUERY=" + SELECT + ( + CASE castsource::REGTYPE::TEXT + WHEN 'bigint' THEN 'int8' + WHEN 'bit varying' THEN 'varbit' + WHEN 'boolean' THEN 'bool' + WHEN '\"char\"' THEN 'char' + WHEN 'character' THEN 'bpchar' + WHEN 'character varying' THEN 'varchar' + WHEN 'double precision' THEN 'float8' + WHEN 'integer' THEN 'int4' + WHEN 'real' THEN 'float4' + WHEN 'smallint' THEN 'int2' + WHEN 'timestamp with time zone' THEN 'timestamptz' + WHEN 'timestamp without time zone' THEN 'timestamp' + WHEN 'time with time zone' THEN 'timetz' + WHEN 'time without time zone' THEN 'time' + ELSE castsource::REGTYPE::TEXT + END + ), + ( + CASE casttarget::REGTYPE::TEXT + WHEN 'bigint' THEN 'int8' + WHEN 'bit varying' THEN 'varbit' + WHEN 'boolean' THEN 'bool' + WHEN '\"char\"' THEN 'char' + WHEN 'character' THEN 'bpchar' + WHEN 'character varying' THEN 'varchar' + WHEN 'double precision' THEN 'float8' + WHEN 'integer' THEN 'int4' + WHEN 'real' THEN 'float4' + WHEN 'smallint' THEN 'int2' + WHEN 'timestamp with time zone' THEN 'timestamptz' + WHEN 'timestamp without time zone' THEN 'timestamp' + WHEN 'time with time zone' THEN 'timetz' + WHEN 'time without time zone' THEN 'time' + ELSE casttarget::REGTYPE::TEXT + END + ), + ( + CASE castcontext + WHEN 'e' THEN 'CastContextExplicit' + WHEN 'a' THEN 'CastContextAssignment' + WHEN 'i' THEN 'CastContextImplicit' + END + ) + FROM pg_cast + ORDER BY 1, 2" + +psql $DATABASE --csv -Xqt -c "$PG_CAST_QUERY" | + awk -F, ' + { + if ($1 != src) + { + src = $1; + if (NR > 1) print "},"; + print "oid.T_" $1 ": {"; + } + print "\toid.T_" $2 ": {maxContext: " $3 ", origin: pgCast},"; + } + END { print "}"; }' diff --git a/pkg/sql/sem/tree/cast_map_test.go b/pkg/sql/sem/tree/cast_map_test.go new file mode 100644 index 000000000000..0b0feb453f89 --- /dev/null +++ b/pkg/sql/sem/tree/cast_map_test.go @@ -0,0 +1,63 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package tree_test + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/faketreeeval" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/randgen" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/randutil" + "github.com/lib/pq/oid" +) + +// TestCastMap tests that every cast in tree.castMap can be performed by +// PerformCast. +func TestCastMap(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + evalCtx := tree.MakeTestingEvalContext(cluster.MakeTestingClusterSettings()) + rng, _ := randutil.NewTestPseudoRand() + evalCtx.Planner = &faketreeeval.DummyEvalPlanner{} + + tree.ForEachCast(func(src, tgt oid.Oid) { + srcType := types.OidToType[src] + tgtType := types.OidToType[tgt] + srcDatum := randgen.RandDatum(rng, srcType, false /* nullOk */) + + // TODO(mgartner): We do not allow casting a negative integer to bit + // types with unbounded widths. Until we add support for this, we + // ensure that the srcDatum is positive. + if srcType.Family() == types.IntFamily && tgtType.Family() == types.BitFamily { + srcVal := *srcDatum.(*tree.DInt) + if srcVal < 0 { + srcDatum = tree.NewDInt(-srcVal) + } + } + + _, err := tree.PerformCast(&evalCtx, srcDatum, tgtType) + // If the error is a CannotCoerce error, then PerformCast does not + // support casting from src to tgt. The one exception is negative + // integers to bit types which return the same error code (see the TODO + // above). + if err != nil && pgerror.HasCandidateCode(err) && pgerror.GetPGCode(err) == pgcode.CannotCoerce { + t.Errorf("cast from %s to %s failed: %s", srcType, tgtType, err) + } + }) +} diff --git a/pkg/sql/sem/tree/casts_test.go b/pkg/sql/sem/tree/cast_test.go similarity index 97% rename from pkg/sql/sem/tree/casts_test.go rename to pkg/sql/sem/tree/cast_test.go index 40426e46f412..c849c6b5e47e 100644 --- a/pkg/sql/sem/tree/casts_test.go +++ b/pkg/sql/sem/tree/cast_test.go @@ -145,7 +145,7 @@ func TestCastsFromUnknown(t *testing.T) { // These type families are exceptions. default: - cast := lookupCast(types.UnknownFamily, fam, false /* intervalStyleEnabled */, false /* dateStyleEnabled */) + cast := lookupCastInfo(types.UnknownFamily, fam, false /* intervalStyleEnabled */, false /* dateStyleEnabled */) if cast == nil { t.Errorf("cast from Unknown to %s does not exist", fam) } diff --git a/pkg/sql/sem/tree/casts.go b/pkg/sql/sem/tree/casts.go deleted file mode 100644 index 92e70da84170..000000000000 --- a/pkg/sql/sem/tree/casts.go +++ /dev/null @@ -1,1610 +0,0 @@ -// Copyright 2020 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package tree - -import ( - "math" - "math/big" - "strconv" - "strings" - "time" - "unicode/utf8" - - "github.com/cockroachdb/apd/v2" - "github.com/cockroachdb/cockroach/pkg/geo" - "github.com/cockroachdb/cockroach/pkg/geo/geopb" - "github.com/cockroachdb/cockroach/pkg/server/telemetry" - "github.com/cockroachdb/cockroach/pkg/sql/lex" - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" - "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" - "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/cockroach/pkg/util" - "github.com/cockroachdb/cockroach/pkg/util/bitarray" - "github.com/cockroachdb/cockroach/pkg/util/duration" - "github.com/cockroachdb/cockroach/pkg/util/json" - "github.com/cockroachdb/cockroach/pkg/util/timeofday" - "github.com/cockroachdb/cockroach/pkg/util/timeutil" - "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate" - "github.com/cockroachdb/errors" - "github.com/lib/pq/oid" -) - -type castInfo struct { - from types.Family - to types.Family - volatility Volatility - - // volatilityHint is an optional string for VolatilityStable casts. When set, - // it is used as an error hint suggesting a possible workaround when stable - // casts are not allowed. - volatilityHint string - - // Telemetry counter; set by init(). - counter telemetry.Counter - - // If set, the volatility of this cast is not cross-checked against postgres. - // Use this with caution. - ignoreVolatilityCheck bool -} - -// validCasts lists all valid explicit casts. -// -// This list must be kept in sync with the capabilities of PerformCast. -// -// Each cast defines a volatility: -// -// - immutable casts yield the same result on the same arguments in whatever -// context they are evaluated. -// -// - stable casts can yield a different result depending on the evaluation context: -// - session settings (e.g. bytes encoding format) -// - current timezone -// - current time (e.g. 'now'::string). -// -// TODO(#55094): move the PerformCast code for each cast into functions defined -// within each cast. -// -var validCasts = []castInfo{ - // Casts to BitFamily. - {from: types.UnknownFamily, to: types.BitFamily, volatility: VolatilityImmutable}, - {from: types.BitFamily, to: types.BitFamily, volatility: VolatilityImmutable}, - {from: types.IntFamily, to: types.BitFamily, volatility: VolatilityImmutable}, - {from: types.StringFamily, to: types.BitFamily, volatility: VolatilityImmutable}, - {from: types.CollatedStringFamily, to: types.BitFamily, volatility: VolatilityImmutable}, - - // Casts to BoolFamily. - {from: types.UnknownFamily, to: types.BoolFamily, volatility: VolatilityImmutable}, - {from: types.BoolFamily, to: types.BoolFamily, volatility: VolatilityImmutable}, - {from: types.IntFamily, to: types.BoolFamily, volatility: VolatilityImmutable}, - {from: types.FloatFamily, to: types.BoolFamily, volatility: VolatilityImmutable}, - {from: types.DecimalFamily, to: types.BoolFamily, volatility: VolatilityImmutable}, - {from: types.StringFamily, to: types.BoolFamily, volatility: VolatilityImmutable}, - {from: types.CollatedStringFamily, to: types.BoolFamily, volatility: VolatilityImmutable}, - {from: types.JsonFamily, to: types.BoolFamily, volatility: VolatilityImmutable}, - - // Casts to IntFamily. - {from: types.UnknownFamily, to: types.IntFamily, volatility: VolatilityImmutable}, - {from: types.BoolFamily, to: types.IntFamily, volatility: VolatilityImmutable}, - {from: types.IntFamily, to: types.IntFamily, volatility: VolatilityImmutable}, - {from: types.FloatFamily, to: types.IntFamily, volatility: VolatilityImmutable}, - {from: types.DecimalFamily, to: types.IntFamily, volatility: VolatilityImmutable}, - {from: types.StringFamily, to: types.IntFamily, volatility: VolatilityImmutable}, - {from: types.CollatedStringFamily, to: types.IntFamily, volatility: VolatilityImmutable}, - {from: types.TimestampFamily, to: types.IntFamily, volatility: VolatilityImmutable}, - {from: types.TimestampTZFamily, to: types.IntFamily, volatility: VolatilityImmutable}, - {from: types.DateFamily, to: types.IntFamily, volatility: VolatilityImmutable}, - {from: types.IntervalFamily, to: types.IntFamily, volatility: VolatilityImmutable}, - {from: types.OidFamily, to: types.IntFamily, volatility: VolatilityImmutable}, - {from: types.BitFamily, to: types.IntFamily, volatility: VolatilityImmutable}, - {from: types.JsonFamily, to: types.IntFamily, volatility: VolatilityImmutable}, - - // Casts to FloatFamily. - {from: types.UnknownFamily, to: types.FloatFamily, volatility: VolatilityImmutable}, - {from: types.BoolFamily, to: types.FloatFamily, volatility: VolatilityImmutable}, - {from: types.IntFamily, to: types.FloatFamily, volatility: VolatilityImmutable}, - {from: types.FloatFamily, to: types.FloatFamily, volatility: VolatilityImmutable}, - {from: types.DecimalFamily, to: types.FloatFamily, volatility: VolatilityImmutable}, - {from: types.StringFamily, to: types.FloatFamily, volatility: VolatilityImmutable}, - {from: types.CollatedStringFamily, to: types.FloatFamily, volatility: VolatilityImmutable}, - {from: types.TimestampFamily, to: types.FloatFamily, volatility: VolatilityImmutable}, - {from: types.TimestampTZFamily, to: types.FloatFamily, volatility: VolatilityImmutable}, - {from: types.DateFamily, to: types.FloatFamily, volatility: VolatilityImmutable}, - {from: types.IntervalFamily, to: types.FloatFamily, volatility: VolatilityImmutable}, - {from: types.JsonFamily, to: types.FloatFamily, volatility: VolatilityImmutable}, - - // Casts to Box2D Family. - {from: types.UnknownFamily, to: types.Box2DFamily, volatility: VolatilityImmutable}, - {from: types.StringFamily, to: types.Box2DFamily, volatility: VolatilityImmutable}, - {from: types.CollatedStringFamily, to: types.Box2DFamily, volatility: VolatilityImmutable}, - {from: types.GeometryFamily, to: types.Box2DFamily, volatility: VolatilityImmutable}, - {from: types.Box2DFamily, to: types.Box2DFamily, volatility: VolatilityImmutable}, - - // Casts to GeographyFamily. - {from: types.UnknownFamily, to: types.GeographyFamily, volatility: VolatilityImmutable}, - {from: types.BytesFamily, to: types.GeographyFamily, volatility: VolatilityImmutable}, - {from: types.JsonFamily, to: types.GeographyFamily, volatility: VolatilityImmutable}, - {from: types.StringFamily, to: types.GeographyFamily, volatility: VolatilityImmutable}, - {from: types.CollatedStringFamily, to: types.GeographyFamily, volatility: VolatilityImmutable}, - {from: types.GeographyFamily, to: types.GeographyFamily, volatility: VolatilityImmutable}, - {from: types.GeometryFamily, to: types.GeographyFamily, volatility: VolatilityImmutable}, - - // Casts to GeometryFamily. - {from: types.UnknownFamily, to: types.GeometryFamily, volatility: VolatilityImmutable}, - {from: types.Box2DFamily, to: types.GeometryFamily, volatility: VolatilityImmutable}, - {from: types.BytesFamily, to: types.GeometryFamily, volatility: VolatilityImmutable}, - {from: types.JsonFamily, to: types.GeometryFamily, volatility: VolatilityImmutable}, - {from: types.StringFamily, to: types.GeometryFamily, volatility: VolatilityImmutable}, - {from: types.CollatedStringFamily, to: types.GeometryFamily, volatility: VolatilityImmutable}, - {from: types.GeographyFamily, to: types.GeometryFamily, volatility: VolatilityImmutable}, - {from: types.GeometryFamily, to: types.GeometryFamily, volatility: VolatilityImmutable}, - - // Casts to DecimalFamily. - {from: types.UnknownFamily, to: types.DecimalFamily, volatility: VolatilityImmutable}, - {from: types.BoolFamily, to: types.DecimalFamily, volatility: VolatilityImmutable}, - {from: types.IntFamily, to: types.DecimalFamily, volatility: VolatilityImmutable}, - {from: types.FloatFamily, to: types.DecimalFamily, volatility: VolatilityImmutable}, - {from: types.DecimalFamily, to: types.DecimalFamily, volatility: VolatilityImmutable}, - {from: types.StringFamily, to: types.DecimalFamily, volatility: VolatilityImmutable}, - {from: types.CollatedStringFamily, to: types.DecimalFamily, volatility: VolatilityImmutable}, - {from: types.TimestampFamily, to: types.DecimalFamily, volatility: VolatilityImmutable}, - {from: types.TimestampTZFamily, to: types.DecimalFamily, volatility: VolatilityImmutable}, - {from: types.DateFamily, to: types.DecimalFamily, volatility: VolatilityImmutable}, - {from: types.IntervalFamily, to: types.DecimalFamily, volatility: VolatilityImmutable}, - {from: types.JsonFamily, to: types.DecimalFamily, volatility: VolatilityImmutable}, - - // Casts to StringFamily. - {from: types.UnknownFamily, to: types.StringFamily, volatility: VolatilityImmutable}, - {from: types.BoolFamily, to: types.StringFamily, volatility: VolatilityImmutable}, - {from: types.IntFamily, to: types.StringFamily, volatility: VolatilityImmutable}, - {from: types.FloatFamily, to: types.StringFamily, volatility: VolatilityStable}, - {from: types.DecimalFamily, to: types.StringFamily, volatility: VolatilityImmutable}, - {from: types.StringFamily, to: types.StringFamily, volatility: VolatilityImmutable}, - {from: types.CollatedStringFamily, to: types.StringFamily, volatility: VolatilityImmutable}, - {from: types.BitFamily, to: types.StringFamily, volatility: VolatilityImmutable}, - {from: types.ArrayFamily, to: types.StringFamily, volatility: VolatilityStable}, - {from: types.TupleFamily, to: types.StringFamily, volatility: VolatilityImmutable}, - {from: types.GeometryFamily, to: types.StringFamily, volatility: VolatilityImmutable}, - {from: types.Box2DFamily, to: types.StringFamily, volatility: VolatilityImmutable}, - {from: types.GeographyFamily, to: types.StringFamily, volatility: VolatilityImmutable}, - {from: types.BytesFamily, to: types.StringFamily, volatility: VolatilityStable}, - { - from: types.TimestampFamily, - to: types.StringFamily, - volatility: VolatilityImmutable, - volatilityHint: "TIMESTAMP to STRING casts are dependent on DateStyle; consider " + - "using to_char(timestamp) instead.", - }, - { - from: types.TimestampTZFamily, - to: types.StringFamily, - volatility: VolatilityStable, - volatilityHint: "TIMESTAMPTZ to STRING casts depend on the current timezone; consider " + - "using to_char(t AT TIME ZONE 'UTC') instead.", - }, - { - from: types.IntervalFamily, - to: types.StringFamily, - volatility: VolatilityImmutable, - volatilityHint: "INTERVAL to STRING casts depends on IntervalStyle; consider using to_char(interval)", - }, - {from: types.UuidFamily, to: types.StringFamily, volatility: VolatilityImmutable}, - { - from: types.DateFamily, - to: types.StringFamily, - volatility: VolatilityImmutable, - volatilityHint: "DATE to STRING casts are dependent on DateStyle; consider " + - "using to_char(date) instead.", - }, - {from: types.TimeFamily, to: types.StringFamily, volatility: VolatilityImmutable}, - {from: types.TimeTZFamily, to: types.StringFamily, volatility: VolatilityImmutable}, - {from: types.OidFamily, to: types.StringFamily, volatility: VolatilityImmutable}, - {from: types.INetFamily, to: types.StringFamily, volatility: VolatilityImmutable}, - {from: types.JsonFamily, to: types.StringFamily, volatility: VolatilityImmutable}, - {from: types.EnumFamily, to: types.StringFamily, volatility: VolatilityImmutable}, - - // Casts to CollatedStringFamily. - {from: types.UnknownFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, - {from: types.BoolFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, - {from: types.IntFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, - {from: types.FloatFamily, to: types.CollatedStringFamily, volatility: VolatilityStable}, - {from: types.DecimalFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, - {from: types.StringFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, - {from: types.CollatedStringFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, - {from: types.BitFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, - {from: types.ArrayFamily, to: types.CollatedStringFamily, volatility: VolatilityStable}, - {from: types.TupleFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, - {from: types.Box2DFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, - {from: types.GeometryFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, - {from: types.GeographyFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, - {from: types.BytesFamily, to: types.CollatedStringFamily, volatility: VolatilityStable}, - {from: types.TimestampFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, - {from: types.TimestampTZFamily, to: types.CollatedStringFamily, volatility: VolatilityStable}, - {from: types.IntervalFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, - {from: types.UuidFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, - {from: types.DateFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, - {from: types.TimeFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, - {from: types.TimeTZFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, - {from: types.OidFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, - {from: types.INetFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, - {from: types.JsonFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, - {from: types.EnumFamily, to: types.CollatedStringFamily, volatility: VolatilityImmutable}, - - // Casts to BytesFamily. - {from: types.UnknownFamily, to: types.BytesFamily, volatility: VolatilityImmutable}, - {from: types.StringFamily, to: types.BytesFamily, volatility: VolatilityImmutable}, - {from: types.CollatedStringFamily, to: types.BytesFamily, volatility: VolatilityImmutable}, - {from: types.BytesFamily, to: types.BytesFamily, volatility: VolatilityImmutable}, - {from: types.UuidFamily, to: types.BytesFamily, volatility: VolatilityImmutable}, - {from: types.GeometryFamily, to: types.BytesFamily, volatility: VolatilityImmutable}, - {from: types.GeographyFamily, to: types.BytesFamily, volatility: VolatilityImmutable}, - - // Casts to DateFamily. - {from: types.UnknownFamily, to: types.DateFamily, volatility: VolatilityImmutable}, - { - from: types.StringFamily, - to: types.DateFamily, - volatility: VolatilityStable, - volatilityHint: "STRING to DATE casts depend on session DateStyle; use parse_date(string) instead", - }, - {from: types.CollatedStringFamily, to: types.DateFamily, volatility: VolatilityStable}, - {from: types.DateFamily, to: types.DateFamily, volatility: VolatilityImmutable}, - {from: types.TimestampFamily, to: types.DateFamily, volatility: VolatilityImmutable}, - {from: types.TimestampTZFamily, to: types.DateFamily, volatility: VolatilityStable}, - {from: types.IntFamily, to: types.DateFamily, volatility: VolatilityImmutable}, - - // Casts to TimeFamily. - {from: types.UnknownFamily, to: types.TimeFamily, volatility: VolatilityImmutable}, - { - from: types.StringFamily, - to: types.TimeFamily, - volatility: VolatilityStable, - volatilityHint: "STRING to TIME casts depend on session DateStyle; use parse_time(string) instead", - }, - {from: types.CollatedStringFamily, to: types.TimeFamily, volatility: VolatilityStable}, - {from: types.TimeFamily, to: types.TimeFamily, volatility: VolatilityImmutable}, - {from: types.TimeTZFamily, to: types.TimeFamily, volatility: VolatilityImmutable}, - {from: types.TimestampFamily, to: types.TimeFamily, volatility: VolatilityImmutable}, - {from: types.TimestampTZFamily, to: types.TimeFamily, volatility: VolatilityStable}, - {from: types.IntervalFamily, to: types.TimeFamily, volatility: VolatilityImmutable}, - - // Casts to TimeTZFamily. - {from: types.UnknownFamily, to: types.TimeTZFamily, volatility: VolatilityImmutable}, - { - from: types.StringFamily, - to: types.TimeTZFamily, - volatility: VolatilityStable, - volatilityHint: "STRING to TIMETZ casts depend on session DateStyle; use parse_timetz(string) instead", - }, - {from: types.CollatedStringFamily, to: types.TimeTZFamily, volatility: VolatilityStable}, - {from: types.TimeFamily, to: types.TimeTZFamily, volatility: VolatilityStable}, - {from: types.TimeTZFamily, to: types.TimeTZFamily, volatility: VolatilityImmutable}, - {from: types.TimestampTZFamily, to: types.TimeTZFamily, volatility: VolatilityStable}, - - // Casts to TimestampFamily. - {from: types.UnknownFamily, to: types.TimestampFamily, volatility: VolatilityImmutable}, - { - from: types.StringFamily, to: types.TimestampFamily, volatility: VolatilityStable, - volatilityHint: "STRING to TIMESTAMP casts are context-dependent because of relative timestamp strings " + - "like 'now' and session settings such as DateStyle; use parse_timestamp(string) instead.", - }, - {from: types.CollatedStringFamily, to: types.TimestampFamily, volatility: VolatilityStable}, - {from: types.DateFamily, to: types.TimestampFamily, volatility: VolatilityImmutable}, - {from: types.TimestampFamily, to: types.TimestampFamily, volatility: VolatilityImmutable}, - { - from: types.TimestampTZFamily, to: types.TimestampFamily, volatility: VolatilityStable, - volatilityHint: "TIMESTAMPTZ to TIMESTAMP casts depend on the current timezone; consider using AT TIME ZONE 'UTC' instead", - }, - {from: types.IntFamily, to: types.TimestampFamily, volatility: VolatilityImmutable}, - - // Casts to TimestampTZFamily. - {from: types.UnknownFamily, to: types.TimestampTZFamily, volatility: VolatilityImmutable}, - {from: types.StringFamily, to: types.TimestampTZFamily, volatility: VolatilityStable}, - {from: types.CollatedStringFamily, to: types.TimestampTZFamily, volatility: VolatilityStable}, - {from: types.DateFamily, to: types.TimestampTZFamily, volatility: VolatilityStable}, - {from: types.TimestampFamily, to: types.TimestampTZFamily, volatility: VolatilityStable}, - {from: types.TimestampTZFamily, to: types.TimestampTZFamily, volatility: VolatilityImmutable}, - {from: types.IntFamily, to: types.TimestampTZFamily, volatility: VolatilityImmutable}, - - // Casts to IntervalFamily. - {from: types.UnknownFamily, to: types.IntervalFamily, volatility: VolatilityImmutable}, - { - from: types.StringFamily, - to: types.IntervalFamily, - volatility: VolatilityImmutable, - volatilityHint: "STRING to INTERVAL casts depend on session IntervalStyle; use parse_interval(string) instead", - }, - {from: types.CollatedStringFamily, to: types.IntervalFamily, volatility: VolatilityImmutable}, - {from: types.IntFamily, to: types.IntervalFamily, volatility: VolatilityImmutable}, - {from: types.TimeFamily, to: types.IntervalFamily, volatility: VolatilityImmutable}, - {from: types.IntervalFamily, to: types.IntervalFamily, volatility: VolatilityImmutable}, - {from: types.FloatFamily, to: types.IntervalFamily, volatility: VolatilityImmutable}, - {from: types.DecimalFamily, to: types.IntervalFamily, volatility: VolatilityImmutable}, - - // Casts to OidFamily. - {from: types.UnknownFamily, to: types.OidFamily, volatility: VolatilityImmutable}, - {from: types.StringFamily, to: types.OidFamily, volatility: VolatilityStable}, - {from: types.CollatedStringFamily, to: types.OidFamily, volatility: VolatilityStable}, - {from: types.IntFamily, to: types.OidFamily, volatility: VolatilityStable, ignoreVolatilityCheck: true}, - {from: types.OidFamily, to: types.OidFamily, volatility: VolatilityStable}, - - // Casts to UnknownFamily. - {from: types.UnknownFamily, to: types.UnknownFamily, volatility: VolatilityImmutable}, - - // Casts to UuidFamily. - {from: types.UnknownFamily, to: types.UuidFamily, volatility: VolatilityImmutable}, - {from: types.StringFamily, to: types.UuidFamily, volatility: VolatilityImmutable}, - {from: types.CollatedStringFamily, to: types.UuidFamily, volatility: VolatilityImmutable}, - {from: types.BytesFamily, to: types.UuidFamily, volatility: VolatilityImmutable}, - {from: types.UuidFamily, to: types.UuidFamily, volatility: VolatilityImmutable}, - - // Casts to INetFamily. - {from: types.UnknownFamily, to: types.INetFamily, volatility: VolatilityImmutable}, - {from: types.StringFamily, to: types.INetFamily, volatility: VolatilityImmutable}, - {from: types.CollatedStringFamily, to: types.INetFamily, volatility: VolatilityImmutable}, - {from: types.INetFamily, to: types.INetFamily, volatility: VolatilityImmutable}, - - // Casts to ArrayFamily. - {from: types.UnknownFamily, to: types.ArrayFamily, volatility: VolatilityImmutable}, - {from: types.StringFamily, to: types.ArrayFamily, volatility: VolatilityStable}, - - // Casts to JsonFamily. - {from: types.UnknownFamily, to: types.JsonFamily, volatility: VolatilityImmutable}, - {from: types.StringFamily, to: types.JsonFamily, volatility: VolatilityImmutable}, - {from: types.JsonFamily, to: types.JsonFamily, volatility: VolatilityImmutable}, - {from: types.GeometryFamily, to: types.JsonFamily, volatility: VolatilityImmutable}, - {from: types.GeographyFamily, to: types.JsonFamily, volatility: VolatilityImmutable}, - - // Casts to EnumFamily. - {from: types.UnknownFamily, to: types.EnumFamily, volatility: VolatilityImmutable}, - {from: types.StringFamily, to: types.EnumFamily, volatility: VolatilityImmutable}, - {from: types.EnumFamily, to: types.EnumFamily, volatility: VolatilityImmutable}, - {from: types.BytesFamily, to: types.EnumFamily, volatility: VolatilityImmutable}, - - // Casts to TupleFamily. - {from: types.UnknownFamily, to: types.TupleFamily, volatility: VolatilityImmutable}, -} - -type castsMapKey struct { - from, to types.Family -} - -var castsMap map[castsMapKey]*castInfo - -// styleCastsMap contains castInfos for casts affected by a style parameter. -var styleCastsMap map[castsMapKey]*castInfo - -func init() { - castsMap = make(map[castsMapKey]*castInfo, len(validCasts)) - styleCastsMap = make(map[castsMapKey]*castInfo) - for i := range validCasts { - c := &validCasts[i] - - // Initialize counter. - c.counter = sqltelemetry.CastOpCounter(c.from.Name(), c.to.Name()) - - key := castsMapKey{from: c.from, to: c.to} - castsMap[key] = c - - if isDateStyleCastAffected(c.from, c.to) || isIntervalStyleCastAffected(c.from, c.to) { - cCopy := *c - cCopy.volatility = VolatilityStable - styleCastsMap[key] = &cCopy - } - } -} - -func isIntervalStyleCastAffected(from, to types.Family) bool { - switch from { - case types.StringFamily, types.CollatedStringFamily: - switch to { - case types.IntervalFamily: - return true - } - case types.IntervalFamily: - switch to { - case types.StringFamily, types.CollatedStringFamily: - return true - } - } - return false -} - -func isDateStyleCastAffected(from, to types.Family) bool { - switch from { - case types.StringFamily, types.CollatedStringFamily: - switch to { - case types.TimeFamily, - types.TimeTZFamily, - types.DateFamily, - types.TimestampFamily: - return true - } - case types.DateFamily, - types.TimestampFamily: - switch to { - case types.StringFamily, types.CollatedStringFamily: - return true - } - } - return false -} - -// lookupCast returns the information for a valid cast. -// Returns nil if this is not a valid cast. -// Does not handle array and tuple casts. -func lookupCast(from, to types.Family, intervalStyleEnabled bool, dateStyleEnabled bool) *castInfo { - k := castsMapKey{from: from, to: to} - if (intervalStyleEnabled && isIntervalStyleCastAffected(from, to)) || - (dateStyleEnabled && isDateStyleCastAffected(from, to)) { - if r, ok := styleCastsMap[k]; ok { - return r - } - } - return castsMap[k] -} - -// LookupCastVolatility returns the volatility of a valid cast. -func LookupCastVolatility(from, to *types.T, sd *sessiondata.SessionData) (_ Volatility, ok bool) { - fromFamily := from.Family() - toFamily := to.Family() - // Special case for casting between arrays. - if fromFamily == types.ArrayFamily && toFamily == types.ArrayFamily { - return LookupCastVolatility(from.ArrayContents(), to.ArrayContents(), sd) - } - // Special case for casting between tuples. - if fromFamily == types.TupleFamily && toFamily == types.TupleFamily { - fromTypes := from.TupleContents() - toTypes := to.TupleContents() - // Handle case where an overload makes a tuple get casted to tuple{}. - if len(toTypes) == 1 && toTypes[0].Family() == types.AnyFamily { - return VolatilityStable, true - } - if len(fromTypes) != len(toTypes) { - return 0, false - } - maxVolatility := VolatilityLeakProof - for i := range fromTypes { - v, ok := LookupCastVolatility(fromTypes[i], toTypes[i], sd) - if !ok { - return 0, false - } - if v > maxVolatility { - maxVolatility = v - } - } - return maxVolatility, true - } - - cast := lookupCast( - fromFamily, - toFamily, - sd != nil && sd.IntervalStyleEnabled, - sd != nil && sd.DateStyleEnabled, - ) - if cast == nil { - return 0, false - } - return cast.volatility, true -} - -// PerformCast performs a cast from the provided Datum to the specified -// types.T. -func PerformCast(ctx *EvalContext, d Datum, t *types.T) (Datum, error) { - ret, err := performCastWithoutPrecisionTruncation(ctx, d, t) - if err != nil { - return nil, err - } - return AdjustValueToType(t, ret) -} - -// AdjustValueToType checks that the width (for strings, byte arrays, and bit -// strings) and scale (decimal). and, shape/srid (for geospatial types) fits the -// specified column type. -// -// Additionally, some precision truncation may occur for the specified column type. -// -// In case of decimals, it can truncate fractional digits in the input -// value in order to fit the target column. If the input value fits the target -// column, it is returned unchanged. If the input value can be truncated to fit, -// then a truncated copy is returned. Otherwise, an error is returned. -// -// In the case of time, it can truncate fractional digits of time datums -// to its relevant rounding for the given type definition. -// -// In the case of geospatial types, it will check whether the SRID and Shape in the -// datum matches the type definition. -// -// This method is used by casts, parsing, INSERT and UPDATE. It is important to note -// that width must be altered *before* this function, as width truncations should -// only occur during casting and parsing but not update/inserts (see -// enforceLocalColumnConstraints). -func AdjustValueToType(typ *types.T, inVal Datum) (outVal Datum, err error) { - switch typ.Family() { - case types.StringFamily, types.CollatedStringFamily: - var sv string - if v, ok := AsDString(inVal); ok { - sv = string(v) - } else if v, ok := inVal.(*DCollatedString); ok { - sv = v.Contents - } - sv = adjustStringValueToType(typ, sv) - if typ.Width() > 0 && utf8.RuneCountInString(sv) > int(typ.Width()) { - return nil, pgerror.Newf(pgcode.StringDataRightTruncation, - "value too long for type %s", - typ.SQLString()) - } - - if typ.Oid() == oid.T_bpchar || typ.Oid() == oid.T_char { - if _, ok := AsDString(inVal); ok { - return NewDString(sv), nil - } else if _, ok := inVal.(*DCollatedString); ok { - return NewDCollatedString(sv, typ.Locale(), &CollationEnvironment{}) - } - } - case types.IntFamily: - if v, ok := AsDInt(inVal); ok { - if typ.Width() == 32 || typ.Width() == 16 { - // Width is defined in bits. - width := uint(typ.Width() - 1) - - // We're performing range checks in line with Go's - // implementation of math.(Max|Min)(16|32) numbers that store - // the boundaries of the allowed range. - // NOTE: when updating the code below, make sure to update - // execgen/cast_gen_util.go as well. - shifted := v >> width - if (v >= 0 && shifted > 0) || (v < 0 && shifted < -1) { - if typ.Width() == 16 { - return nil, ErrInt2OutOfRange - } - return nil, ErrInt4OutOfRange - } - } - } - case types.BitFamily: - if v, ok := AsDBitArray(inVal); ok { - if typ.Width() > 0 { - bitLen := v.BitLen() - switch typ.Oid() { - case oid.T_varbit: - if bitLen > uint(typ.Width()) { - return nil, pgerror.Newf(pgcode.StringDataRightTruncation, - "bit string length %d too large for type %s", bitLen, typ.SQLString()) - } - default: - if bitLen != uint(typ.Width()) { - return nil, pgerror.Newf(pgcode.StringDataLengthMismatch, - "bit string length %d does not match type %s", bitLen, typ.SQLString()) - } - } - } - } - case types.DecimalFamily: - if inDec, ok := inVal.(*DDecimal); ok { - if inDec.Form != apd.Finite || typ.Precision() == 0 { - // Non-finite form or unlimited target precision, so no need to limit. - break - } - if int64(typ.Precision()) >= inDec.NumDigits() && typ.Scale() == inDec.Exponent { - // Precision and scale of target column are sufficient. - break - } - - var outDec DDecimal - outDec.Set(&inDec.Decimal) - err := LimitDecimalWidth(&outDec.Decimal, int(typ.Precision()), int(typ.Scale())) - if err != nil { - return nil, errors.Wrapf(err, "type %s", typ.SQLString()) - } - return &outDec, nil - } - case types.ArrayFamily: - if inArr, ok := inVal.(*DArray); ok { - var outArr *DArray - elementType := typ.ArrayContents() - for i, inElem := range inArr.Array { - outElem, err := AdjustValueToType(elementType, inElem) - if err != nil { - return nil, err - } - if outElem != inElem { - if outArr == nil { - outArr = &DArray{} - *outArr = *inArr - outArr.Array = make(Datums, len(inArr.Array)) - copy(outArr.Array, inArr.Array[:i]) - } - } - if outArr != nil { - outArr.Array[i] = inElem - } - } - if outArr != nil { - return outArr, nil - } - } - case types.TimeFamily: - if in, ok := inVal.(*DTime); ok { - return in.Round(TimeFamilyPrecisionToRoundDuration(typ.Precision())), nil - } - case types.TimestampFamily: - if in, ok := inVal.(*DTimestamp); ok { - return in.Round(TimeFamilyPrecisionToRoundDuration(typ.Precision())) - } - case types.TimestampTZFamily: - if in, ok := inVal.(*DTimestampTZ); ok { - return in.Round(TimeFamilyPrecisionToRoundDuration(typ.Precision())) - } - case types.TimeTZFamily: - if in, ok := inVal.(*DTimeTZ); ok { - return in.Round(TimeFamilyPrecisionToRoundDuration(typ.Precision())), nil - } - case types.IntervalFamily: - if in, ok := inVal.(*DInterval); ok { - itm, err := typ.IntervalTypeMetadata() - if err != nil { - return nil, err - } - return NewDInterval(in.Duration, itm), nil - } - case types.GeometryFamily: - if in, ok := inVal.(*DGeometry); ok { - if err := geo.SpatialObjectFitsColumnMetadata( - in.Geometry.SpatialObject(), - typ.InternalType.GeoMetadata.SRID, - typ.InternalType.GeoMetadata.ShapeType, - ); err != nil { - return nil, err - } - } - case types.GeographyFamily: - if in, ok := inVal.(*DGeography); ok { - if err := geo.SpatialObjectFitsColumnMetadata( - in.Geography.SpatialObject(), - typ.InternalType.GeoMetadata.SRID, - typ.InternalType.GeoMetadata.ShapeType, - ); err != nil { - return nil, err - } - } - } - return inVal, nil -} - -// adjustStringToType checks that the width for strings fits the -// specified column type. -func adjustStringValueToType(typ *types.T, sv string) string { - switch typ.Oid() { - case oid.T_char: - // "char" is supposed to truncate long values - return util.TruncateString(sv, 1) - case oid.T_bpchar: - // bpchar types truncate trailing whitespace. - return strings.TrimRight(sv, " ") - } - return sv -} - -// formatBitArrayToType formats bit arrays such that they fill the total width -// if too short, or truncate if too long. -func formatBitArrayToType(d *DBitArray, t *types.T) *DBitArray { - if t.Width() == 0 || d.BitLen() == uint(t.Width()) { - return d - } - a := d.BitArray.Clone() - switch t.Oid() { - case oid.T_varbit: - // VARBITs do not have padding attached, so only truncate. - if uint(t.Width()) < a.BitLen() { - a = a.ToWidth(uint(t.Width())) - } - default: - a = a.ToWidth(uint(t.Width())) - } - return &DBitArray{a} -} - -// performCastWithoutPrecisionTruncation performs the cast, but does not do a -// check on whether the datum fits the type. -// In an ideal state, components of AdjustValueToType should be embedded into -// this function, but the code base needs a general refactor of parsing -// and casting logic before this can happen. -// See also: #55094. -func performCastWithoutPrecisionTruncation(ctx *EvalContext, d Datum, t *types.T) (Datum, error) { - // If we're casting a DOidWrapper, then we want to cast the wrapped datum. - // It is also reasonable to lose the old Oid value too. - // Note that we pass in nil as the first argument since we're not interested - // in evaluating the placeholders. - d = UnwrapDatum(nil /* evalCtx */, d) - switch t.Family() { - case types.BitFamily: - switch v := d.(type) { - case *DBitArray: - return formatBitArrayToType(v, t), nil - case *DInt: - r, err := NewDBitArrayFromInt(int64(*v), uint(t.Width())) - if err != nil { - return nil, err - } - return formatBitArrayToType(r, t), nil - case *DString: - res, err := bitarray.Parse(string(*v)) - if err != nil { - return nil, err - } - return formatBitArrayToType(&DBitArray{res}, t), nil - case *DCollatedString: - res, err := bitarray.Parse(v.Contents) - if err != nil { - return nil, err - } - return formatBitArrayToType(&DBitArray{res}, t), nil - } - - case types.BoolFamily: - switch v := d.(type) { - case *DBool: - return d, nil - case *DInt: - return MakeDBool(*v != 0), nil - case *DFloat: - return MakeDBool(*v != 0), nil - case *DDecimal: - return MakeDBool(v.Sign() != 0), nil - case *DString: - return ParseDBool(string(*v)) - case *DCollatedString: - return ParseDBool(v.Contents) - case *DJSON: - b, ok := v.AsBool() - if !ok { - return nil, failedCastFromJSON(v, t) - } - return MakeDBool(DBool(b)), nil - } - - case types.IntFamily: - var res *DInt - switch v := d.(type) { - case *DBitArray: - res = v.AsDInt(uint(t.Width())) - case *DBool: - if *v { - res = NewDInt(1) - } else { - res = DZero - } - case *DInt: - // TODO(knz): enforce the coltype width here. - res = v - case *DFloat: - f := float64(*v) - // Use `<=` and `>=` here instead of just `<` and `>` because when - // math.MaxInt64 and math.MinInt64 are converted to float64s, they are - // rounded to numbers with larger absolute values. Note that the first - // next FP value after and strictly greater than float64(math.MinInt64) - // is -9223372036854774784 (= float64(math.MinInt64)+513) and the first - // previous value and strictly smaller than float64(math.MaxInt64) - // is 9223372036854774784 (= float64(math.MaxInt64)-513), and both are - // convertible to int without overflow. - if math.IsNaN(f) || f <= float64(math.MinInt64) || f >= float64(math.MaxInt64) { - return nil, ErrIntOutOfRange - } - res = NewDInt(DInt(f)) - case *DDecimal: - i, err := roundDecimalToInt(ctx, &v.Decimal) - if err != nil { - return nil, err - } - res = NewDInt(DInt(i)) - case *DString: - var err error - if res, err = ParseDInt(string(*v)); err != nil { - return nil, err - } - case *DCollatedString: - var err error - if res, err = ParseDInt(v.Contents); err != nil { - return nil, err - } - case *DTimestamp: - res = NewDInt(DInt(v.Unix())) - case *DTimestampTZ: - res = NewDInt(DInt(v.Unix())) - case *DDate: - // TODO(mjibson): This cast is unsupported by postgres. Should we remove ours? - if !v.IsFinite() { - return nil, ErrIntOutOfRange - } - res = NewDInt(DInt(v.UnixEpochDays())) - case *DInterval: - iv, ok := v.AsInt64() - if !ok { - return nil, ErrIntOutOfRange - } - res = NewDInt(DInt(iv)) - case *DOid: - res = &v.DInt - case *DJSON: - dec, ok := v.AsDecimal() - if !ok { - return nil, failedCastFromJSON(v, t) - } - i, err := dec.Int64() - if err != nil { - // Attempt to round the number to an integer. - i, err = roundDecimalToInt(ctx, dec) - if err != nil { - return nil, err - } - } - res = NewDInt(DInt(i)) - } - if res != nil { - return res, nil - } - - case types.EnumFamily: - switch v := d.(type) { - case *DString: - return MakeDEnumFromLogicalRepresentation(t, string(*v)) - case *DBytes: - return MakeDEnumFromPhysicalRepresentation(t, []byte(*v)) - case *DEnum: - return d, nil - } - - case types.FloatFamily: - switch v := d.(type) { - case *DBool: - if *v { - return NewDFloat(1), nil - } - return NewDFloat(0), nil - case *DInt: - return NewDFloat(DFloat(*v)), nil - case *DFloat: - return d, nil - case *DDecimal: - f, err := v.Float64() - if err != nil { - return nil, ErrFloatOutOfRange - } - return NewDFloat(DFloat(f)), nil - case *DString: - return ParseDFloat(string(*v)) - case *DCollatedString: - return ParseDFloat(v.Contents) - case *DTimestamp: - micros := float64(v.Nanosecond() / int(time.Microsecond)) - return NewDFloat(DFloat(float64(v.Unix()) + micros*1e-6)), nil - case *DTimestampTZ: - micros := float64(v.Nanosecond() / int(time.Microsecond)) - return NewDFloat(DFloat(float64(v.Unix()) + micros*1e-6)), nil - case *DDate: - // TODO(mjibson): This cast is unsupported by postgres. Should we remove ours? - if !v.IsFinite() { - return nil, ErrFloatOutOfRange - } - return NewDFloat(DFloat(float64(v.UnixEpochDays()))), nil - case *DInterval: - return NewDFloat(DFloat(v.AsFloat64())), nil - case *DJSON: - dec, ok := v.AsDecimal() - if !ok { - return nil, failedCastFromJSON(v, t) - } - fl, err := dec.Float64() - if err != nil { - return nil, ErrFloatOutOfRange - } - return NewDFloat(DFloat(fl)), nil - } - - case types.DecimalFamily: - var dd DDecimal - var err error - unset := false - switch v := d.(type) { - case *DBool: - if *v { - dd.SetInt64(1) - } - case *DInt: - dd.SetInt64(int64(*v)) - case *DDate: - // TODO(mjibson): This cast is unsupported by postgres. Should we remove ours? - if !v.IsFinite() { - return nil, errDecOutOfRange - } - dd.SetInt64(v.UnixEpochDays()) - case *DFloat: - _, err = dd.SetFloat64(float64(*v)) - case *DDecimal: - // Small optimization to avoid copying into dd in normal case. - if t.Precision() == 0 { - return d, nil - } - dd.Set(&v.Decimal) - case *DString: - err = dd.SetString(string(*v)) - case *DCollatedString: - err = dd.SetString(v.Contents) - case *DTimestamp: - val := &dd.Coeff - val.SetInt64(v.Unix()) - val.Mul(val, big10E6) - micros := v.Nanosecond() / int(time.Microsecond) - val.Add(val, big.NewInt(int64(micros))) - dd.Exponent = -6 - case *DTimestampTZ: - val := &dd.Coeff - val.SetInt64(v.Unix()) - val.Mul(val, big10E6) - micros := v.Nanosecond() / int(time.Microsecond) - val.Add(val, big.NewInt(int64(micros))) - dd.Exponent = -6 - case *DInterval: - v.AsBigInt(&dd.Coeff) - dd.Exponent = -9 - case *DJSON: - dec, ok := v.AsDecimal() - if !ok { - return nil, failedCastFromJSON(v, t) - } - dd.Set(dec) - default: - unset = true - } - if err != nil { - return nil, err - } - if !unset { - // dd.Coeff must be positive. If it was set to a negative value - // above, transfer the sign to dd.Negative. - if dd.Coeff.Sign() < 0 { - dd.Negative = true - dd.Coeff.Abs(&dd.Coeff) - } - err = LimitDecimalWidth(&dd.Decimal, int(t.Precision()), int(t.Scale())) - return &dd, err - } - - case types.StringFamily, types.CollatedStringFamily: - var s string - typ := t - switch t := d.(type) { - case *DBitArray: - s = t.BitArray.String() - case *DFloat: - s = strconv.FormatFloat(float64(*t), 'g', - ctx.SessionData().DataConversionConfig.GetFloatPrec(), 64) - case *DInt: - if typ.Oid() == oid.T_char { - // int to "char" casts just return the correspondong ASCII byte. - if *t > math.MaxInt8 || *t < math.MinInt8 { - return nil, errCharOutOfRange - } else if *t == 0 { - s = "" - } else { - s = string([]byte{byte(*t)}) - } - } else { - s = d.String() - } - case *DBool, *DDecimal: - s = d.String() - case *DTimestamp, *DDate, *DTime, *DTimeTZ, *DGeography, *DGeometry, *DBox2D: - s = AsStringWithFlags(d, FmtBareStrings) - case *DTimestampTZ: - // Convert to context timezone for correct display. - ts, err := MakeDTimestampTZ(t.In(ctx.GetLocation()), time.Microsecond) - if err != nil { - return nil, err - } - s = AsStringWithFlags( - ts, - FmtBareStrings, - ) - case *DTuple: - s = AsStringWithFlags( - d, - FmtPgwireText, - FmtDataConversionConfig(ctx.SessionData().DataConversionConfig), - ) - case *DArray: - s = AsStringWithFlags( - d, - FmtPgwireText, - FmtDataConversionConfig(ctx.SessionData().DataConversionConfig), - ) - case *DInterval: - // When converting an interval to string, we need a string representation - // of the duration (e.g. "5s") and not of the interval itself (e.g. - // "INTERVAL '5s'"). - s = AsStringWithFlags( - d, - FmtPgwireText, - FmtDataConversionConfig(ctx.SessionData().DataConversionConfig), - ) - case *DUuid: - s = t.UUID.String() - case *DIPAddr: - s = AsStringWithFlags(d, FmtBareStrings) - case *DString: - s = string(*t) - case *DCollatedString: - s = t.Contents - case *DBytes: - s = lex.EncodeByteArrayToRawBytes( - string(*t), - ctx.SessionData().DataConversionConfig.BytesEncodeFormat, - false, /* skipHexPrefix */ - ) - case *DOid: - s = t.String() - case *DJSON: - s = t.JSON.String() - case *DEnum: - s = t.LogicalRep - } - switch t.Family() { - case types.StringFamily: - if t.Oid() == oid.T_name { - return NewDName(s), nil - } - - // bpchar types truncate trailing whitespace. - if t.Oid() == oid.T_bpchar { - s = strings.TrimRight(s, " ") - } - - // If the string type specifies a limit we truncate to that limit: - // 'hello'::CHAR(2) -> 'he' - // This is true of all the string type variants. - if t.Width() > 0 { - s = util.TruncateString(s, int(t.Width())) - } - return NewDString(s), nil - case types.CollatedStringFamily: - // bpchar types truncate trailing whitespace. - if t.Oid() == oid.T_bpchar { - s = strings.TrimRight(s, " ") - } - // Ditto truncation like for TString. - if t.Width() > 0 { - s = util.TruncateString(s, int(t.Width())) - } - return NewDCollatedString(s, t.Locale(), &ctx.CollationEnv) - } - - case types.BytesFamily: - switch t := d.(type) { - case *DString: - return ParseDByte(string(*t)) - case *DCollatedString: - return NewDBytes(DBytes(t.Contents)), nil - case *DUuid: - return NewDBytes(DBytes(t.GetBytes())), nil - case *DBytes: - return d, nil - case *DGeography: - return NewDBytes(DBytes(t.Geography.EWKB())), nil - case *DGeometry: - return NewDBytes(DBytes(t.Geometry.EWKB())), nil - } - - case types.UuidFamily: - switch t := d.(type) { - case *DString: - return ParseDUuidFromString(string(*t)) - case *DCollatedString: - return ParseDUuidFromString(t.Contents) - case *DBytes: - return ParseDUuidFromBytes([]byte(*t)) - case *DUuid: - return d, nil - } - - case types.INetFamily: - switch t := d.(type) { - case *DString: - return ParseDIPAddrFromINetString(string(*t)) - case *DCollatedString: - return ParseDIPAddrFromINetString(t.Contents) - case *DIPAddr: - return d, nil - } - - case types.Box2DFamily: - switch d := d.(type) { - case *DString: - return ParseDBox2D(string(*d)) - case *DCollatedString: - return ParseDBox2D(d.Contents) - case *DBox2D: - return d, nil - case *DGeometry: - bbox := d.CartesianBoundingBox() - if bbox == nil { - return DNull, nil - } - return NewDBox2D(*bbox), nil - } - - case types.GeographyFamily: - switch d := d.(type) { - case *DString: - return ParseDGeography(string(*d)) - case *DCollatedString: - return ParseDGeography(d.Contents) - case *DGeography: - if err := geo.SpatialObjectFitsColumnMetadata( - d.Geography.SpatialObject(), - t.InternalType.GeoMetadata.SRID, - t.InternalType.GeoMetadata.ShapeType, - ); err != nil { - return nil, err - } - return d, nil - case *DGeometry: - g, err := d.AsGeography() - if err != nil { - return nil, err - } - if err := geo.SpatialObjectFitsColumnMetadata( - g.SpatialObject(), - t.InternalType.GeoMetadata.SRID, - t.InternalType.GeoMetadata.ShapeType, - ); err != nil { - return nil, err - } - return &DGeography{g}, nil - case *DJSON: - t, err := d.AsText() - if err != nil { - return nil, err - } - if t == nil { - return DNull, nil - } - g, err := geo.ParseGeographyFromGeoJSON([]byte(*t)) - if err != nil { - return nil, err - } - return &DGeography{g}, nil - case *DBytes: - g, err := geo.ParseGeographyFromEWKB(geopb.EWKB(*d)) - if err != nil { - return nil, err - } - return &DGeography{g}, nil - } - case types.GeometryFamily: - switch d := d.(type) { - case *DString: - return ParseDGeometry(string(*d)) - case *DCollatedString: - return ParseDGeometry(d.Contents) - case *DGeometry: - if err := geo.SpatialObjectFitsColumnMetadata( - d.Geometry.SpatialObject(), - t.InternalType.GeoMetadata.SRID, - t.InternalType.GeoMetadata.ShapeType, - ); err != nil { - return nil, err - } - return d, nil - case *DGeography: - if err := geo.SpatialObjectFitsColumnMetadata( - d.Geography.SpatialObject(), - t.InternalType.GeoMetadata.SRID, - t.InternalType.GeoMetadata.ShapeType, - ); err != nil { - return nil, err - } - g, err := d.AsGeometry() - if err != nil { - return nil, err - } - return &DGeometry{g}, nil - case *DJSON: - t, err := d.AsText() - if err != nil { - return nil, err - } - if t == nil { - return DNull, nil - } - g, err := geo.ParseGeometryFromGeoJSON([]byte(*t)) - if err != nil { - return nil, err - } - return &DGeometry{g}, nil - case *DBox2D: - g, err := geo.MakeGeometryFromGeomT(d.ToGeomT(geopb.DefaultGeometrySRID)) - if err != nil { - return nil, err - } - return &DGeometry{g}, nil - case *DBytes: - g, err := geo.ParseGeometryFromEWKB(geopb.EWKB(*d)) - if err != nil { - return nil, err - } - return &DGeometry{g}, nil - } - - case types.DateFamily: - switch d := d.(type) { - case *DString: - res, _, err := ParseDDate(ctx, string(*d)) - return res, err - case *DCollatedString: - res, _, err := ParseDDate(ctx, d.Contents) - return res, err - case *DDate: - return d, nil - case *DInt: - // TODO(mjibson): This cast is unsupported by postgres. Should we remove ours? - t, err := pgdate.MakeDateFromUnixEpoch(int64(*d)) - return NewDDate(t), err - case *DTimestampTZ: - return NewDDateFromTime(d.Time.In(ctx.GetLocation())) - case *DTimestamp: - return NewDDateFromTime(d.Time) - } - - case types.TimeFamily: - roundTo := TimeFamilyPrecisionToRoundDuration(t.Precision()) - switch d := d.(type) { - case *DString: - res, _, err := ParseDTime(ctx, string(*d), roundTo) - return res, err - case *DCollatedString: - res, _, err := ParseDTime(ctx, d.Contents, roundTo) - return res, err - case *DTime: - return d.Round(roundTo), nil - case *DTimeTZ: - return MakeDTime(d.TimeOfDay.Round(roundTo)), nil - case *DTimestamp: - return MakeDTime(timeofday.FromTime(d.Time).Round(roundTo)), nil - case *DTimestampTZ: - // Strip time zone. Times don't carry their location. - stripped, err := d.stripTimeZone(ctx) - if err != nil { - return nil, err - } - return MakeDTime(timeofday.FromTime(stripped.Time).Round(roundTo)), nil - case *DInterval: - return MakeDTime(timeofday.Min.Add(d.Duration).Round(roundTo)), nil - } - - case types.TimeTZFamily: - roundTo := TimeFamilyPrecisionToRoundDuration(t.Precision()) - switch d := d.(type) { - case *DString: - res, _, err := ParseDTimeTZ(ctx, string(*d), roundTo) - return res, err - case *DCollatedString: - res, _, err := ParseDTimeTZ(ctx, d.Contents, roundTo) - return res, err - case *DTime: - return NewDTimeTZFromLocation(timeofday.TimeOfDay(*d).Round(roundTo), ctx.GetLocation()), nil - case *DTimeTZ: - return d.Round(roundTo), nil - case *DTimestampTZ: - return NewDTimeTZFromTime(d.Time.In(ctx.GetLocation()).Round(roundTo)), nil - } - - case types.TimestampFamily: - roundTo := TimeFamilyPrecisionToRoundDuration(t.Precision()) - // TODO(knz): Timestamp from float, decimal. - switch d := d.(type) { - case *DString: - res, _, err := ParseDTimestamp(ctx, string(*d), roundTo) - return res, err - case *DCollatedString: - res, _, err := ParseDTimestamp(ctx, d.Contents, roundTo) - return res, err - case *DDate: - t, err := d.ToTime() - if err != nil { - return nil, err - } - return MakeDTimestamp(t, roundTo) - case *DInt: - return MakeDTimestamp(timeutil.Unix(int64(*d), 0), roundTo) - case *DTimestamp: - return d.Round(roundTo) - case *DTimestampTZ: - // Strip time zone. Timestamps don't carry their location. - stripped, err := d.stripTimeZone(ctx) - if err != nil { - return nil, err - } - return stripped.Round(roundTo) - } - - case types.TimestampTZFamily: - roundTo := TimeFamilyPrecisionToRoundDuration(t.Precision()) - // TODO(knz): TimestampTZ from float, decimal. - switch d := d.(type) { - case *DString: - res, _, err := ParseDTimestampTZ(ctx, string(*d), roundTo) - return res, err - case *DCollatedString: - res, _, err := ParseDTimestampTZ(ctx, d.Contents, roundTo) - return res, err - case *DDate: - t, err := d.ToTime() - if err != nil { - return nil, err - } - _, before := t.Zone() - _, after := t.In(ctx.GetLocation()).Zone() - return MakeDTimestampTZ(t.Add(time.Duration(before-after)*time.Second), roundTo) - case *DTimestamp: - _, before := d.Time.Zone() - _, after := d.Time.In(ctx.GetLocation()).Zone() - return MakeDTimestampTZ(d.Time.Add(time.Duration(before-after)*time.Second), roundTo) - case *DInt: - return MakeDTimestampTZ(timeutil.Unix(int64(*d), 0), roundTo) - case *DTimestampTZ: - return d.Round(roundTo) - } - - case types.IntervalFamily: - itm, err := t.IntervalTypeMetadata() - if err != nil { - return nil, err - } - switch v := d.(type) { - case *DString: - return ParseDIntervalWithTypeMetadata(ctx.GetIntervalStyle(), string(*v), itm) - case *DCollatedString: - return ParseDIntervalWithTypeMetadata(ctx.GetIntervalStyle(), v.Contents, itm) - case *DInt: - return NewDInterval(duration.FromInt64(int64(*v)), itm), nil - case *DFloat: - return NewDInterval(duration.FromFloat64(float64(*v)), itm), nil - case *DTime: - return NewDInterval(duration.MakeDuration(int64(*v)*1000, 0, 0), itm), nil - case *DDecimal: - d := ctx.getTmpDec() - dnanos := v.Decimal - dnanos.Exponent += 9 - // We need HighPrecisionCtx because duration values can contain - // upward of 35 decimal digits and DecimalCtx only provides 25. - _, err := HighPrecisionCtx.Quantize(d, &dnanos, 0) - if err != nil { - return nil, err - } - if dnanos.Negative { - d.Coeff.Neg(&d.Coeff) - } - dv, ok := duration.FromBigInt(&d.Coeff) - if !ok { - return nil, errDecOutOfRange - } - return NewDInterval(dv, itm), nil - case *DInterval: - return NewDInterval(v.Duration, itm), nil - } - case types.JsonFamily: - switch v := d.(type) { - case *DString: - return ParseDJSON(string(*v)) - case *DJSON: - return v, nil - case *DGeography: - j, err := geo.SpatialObjectToGeoJSON(v.Geography.SpatialObject(), -1, geo.SpatialObjectToGeoJSONFlagZero) - if err != nil { - return nil, err - } - return ParseDJSON(string(j)) - case *DGeometry: - j, err := geo.SpatialObjectToGeoJSON(v.Geometry.SpatialObject(), -1, geo.SpatialObjectToGeoJSONFlagZero) - if err != nil { - return nil, err - } - return ParseDJSON(string(j)) - } - case types.ArrayFamily: - switch v := d.(type) { - case *DString: - res, _, err := ParseDArrayFromString(ctx, string(*v), t.ArrayContents()) - return res, err - case *DArray: - dcast := NewDArray(t.ArrayContents()) - for _, e := range v.Array { - ecast := DNull - if e != DNull { - var err error - ecast, err = PerformCast(ctx, e, t.ArrayContents()) - if err != nil { - return nil, err - } - } - - if err := dcast.Append(ecast); err != nil { - return nil, err - } - } - return dcast, nil - } - case types.OidFamily: - switch v := d.(type) { - case *DOid: - return performIntToOidCast(ctx, t, v.DInt) - case *DInt: - // OIDs are always unsigned 32-bit integers. Some languages, like Java, - // store OIDs as signed 32-bit integers, so we implement the cast - // by converting to a uint32 first. This matches Postgres behavior. - i := DInt(uint32(*v)) - return performIntToOidCast(ctx, t, i) - case *DString: - return ParseDOid(ctx, string(*v), t) - } - } - - return nil, pgerror.Newf( - pgcode.CannotCoerce, "invalid cast: %s -> %s", d.ResolvedType(), t) -} - -// performIntToOidCast casts the input integer to the OID type given by the -// input types.T. -func performIntToOidCast(ctx *EvalContext, t *types.T, v DInt) (Datum, error) { - switch t.Oid() { - case oid.T_oid: - return &DOid{semanticType: t, DInt: v}, nil - case oid.T_regtype: - // Mapping an oid to a regtype is easy: we have a hardcoded map. - ret := &DOid{semanticType: t, DInt: v} - if typ, ok := types.OidToType[oid.Oid(v)]; ok { - ret.name = typ.PGName() - } else if types.IsOIDUserDefinedType(oid.Oid(v)) { - typ, err := ctx.Planner.ResolveTypeByOID(ctx.Context, oid.Oid(v)) - if err != nil { - return nil, err - } - ret.name = typ.PGName() - } - return ret, nil - - case oid.T_regproc, oid.T_regprocedure: - // Mapping an oid to a regproc is easy: we have a hardcoded map. - name, ok := OidToBuiltinName[oid.Oid(v)] - ret := &DOid{semanticType: t, DInt: v} - if !ok { - return ret, nil - } - ret.name = name - return ret, nil - - default: - oid, err := ctx.Planner.ResolveOIDFromOID(ctx.Ctx(), t, NewDOid(v)) - if err != nil { - oid = NewDOid(v) - oid.semanticType = t - } - return oid, nil - } -} - -func roundDecimalToInt(ctx *EvalContext, d *apd.Decimal) (int64, error) { - newD := ctx.getTmpDec() - if _, err := DecimalCtx.RoundToIntegralValue(newD, d); err != nil { - return 0, err - } - i, err := newD.Int64() - if err != nil { - return 0, ErrIntOutOfRange - } - return i, nil -} - -func failedCastFromJSON(j *DJSON, t *types.T) error { - return pgerror.Newf( - pgcode.InvalidParameterValue, - "cannot cast jsonb %s to type %s", - j.Type(), t, - ) -} - -// PopulateRecordWithJSON is used for the json to record function family, like -// json_populate_record. Given a JSON object, a desired tuple type, and a tuple -// of the same length as the desired type, this function will populate the tuple -// by setting each named field in the tuple to the value of the key with the -// same name in the input JSON object. Fields in the tuple that are not present -// in the JSON will not be modified, and JSON keys that are not in the tuple -// will be ignored. -// Each field will be set by a best-effort coercion to its type from the JSON -// field. The logic is more permissive than casts. -func PopulateRecordWithJSON( - ctx *EvalContext, j json.JSON, desiredType *types.T, tup *DTuple, -) error { - if j.Type() != json.ObjectJSONType { - return pgerror.Newf(pgcode.InvalidParameterValue, "expected JSON object") - } - tupleTypes := desiredType.TupleContents() - labels := desiredType.TupleLabels() - if labels == nil { - return pgerror.Newf( - pgcode.InvalidParameterValue, - "anonymous records cannot be used with json{b}_populate_record{set}", - ) - } - for i := range tupleTypes { - val, err := j.FetchValKey(labels[i]) - if err != nil || val == nil { - // No value? Use the value that was already in the tuple. - continue - } - tup.D[i], err = PopulateDatumWithJSON(ctx, val, tupleTypes[i]) - if err != nil { - return err - } - } - return nil -} - -// PopulateDatumWithJSON is used for the json to record function family, like -// json_populate_record. It's less restrictive than the casting system, which -// is why it's implemented separately. -func PopulateDatumWithJSON(ctx *EvalContext, j json.JSON, desiredType *types.T) (Datum, error) { - if j == json.NullJSONValue { - return DNull, nil - } - switch desiredType.Family() { - case types.ArrayFamily: - if j.Type() != json.ArrayJSONType { - return nil, pgerror.Newf(pgcode.InvalidTextRepresentation, "expected JSON array") - } - n := j.Len() - elementTyp := desiredType.ArrayContents() - d := NewDArray(elementTyp) - d.Array = make(Datums, n) - for i := 0; i < n; i++ { - elt, err := j.FetchValIdx(i) - if err != nil { - return nil, err - } - d.Array[i], err = PopulateDatumWithJSON(ctx, elt, elementTyp) - if err != nil { - return nil, err - } - } - return d, nil - case types.TupleFamily: - tup := NewDTupleWithLen(desiredType, len(desiredType.TupleContents())) - for i := range tup.D { - tup.D[i] = DNull - } - err := PopulateRecordWithJSON(ctx, j, desiredType, tup) - return tup, err - } - var s string - switch j.Type() { - case json.StringJSONType: - t, err := j.AsText() - if err != nil { - return nil, err - } - s = *t - default: - s = j.String() - } - return PerformCast(ctx, NewDString(s), desiredType) -} diff --git a/pkg/sql/sem/tree/col_types_test.go b/pkg/sql/sem/tree/col_types_test.go index fe7b3fa627f0..c93df3178eca 100644 --- a/pkg/sql/sem/tree/col_types_test.go +++ b/pkg/sql/sem/tree/col_types_test.go @@ -61,7 +61,7 @@ func TestParseColumnType(t *testing.T) { {"CHAR(11)", types.MakeChar(11)}, {"VARCHAR", types.VarChar}, {"VARCHAR(2)", types.MakeVarChar(2)}, - {`"char"`, types.MakeQChar(0)}, + {`"char"`, types.QChar}, {"BYTES", types.Bytes}, {"STRING COLLATE da", types.MakeCollatedString(types.String, "da")}, {"CHAR COLLATE de", types.MakeCollatedString(types.MakeChar(1), "de")}, diff --git a/pkg/sql/sem/tree/eval.go b/pkg/sql/sem/tree/eval.go index f1bfbcdfa279..ffc116e3aeb7 100644 --- a/pkg/sql/sem/tree/eval.go +++ b/pkg/sql/sem/tree/eval.go @@ -4194,6 +4194,10 @@ func (expr *FuncExpr) MaybeWrapError(err error) error { // Eval implements the TypedExpr interface. func (expr *FuncExpr) Eval(ctx *EvalContext) (Datum, error) { + if expr.fn.FnWithExprs != nil { + return expr.fn.FnWithExprs(ctx, expr.Exprs) + } + nullResult, args, err := expr.evalArgs(ctx) if err != nil { return nil, err @@ -4679,7 +4683,8 @@ func (t *Placeholder) Eval(ctx *EvalContext) (Datum, error) { // checking, since the placeholder's type hint didn't match the desired // type for the placeholder. In this case, we cast the expression to // the desired type. - // TODO(jordan): introduce a restriction on what casts are allowed here. + // TODO(jordan,mgartner): Introduce a restriction on what casts are + // allowed here. Most likely, only implicit casts should be allowed. cast := NewTypedCastExpr(e, typ) return cast.Eval(ctx) } diff --git a/pkg/sql/sem/tree/overload.go b/pkg/sql/sem/tree/overload.go index 3f87420ed352..3e17e4d20f37 100644 --- a/pkg/sql/sem/tree/overload.go +++ b/pkg/sql/sem/tree/overload.go @@ -71,10 +71,14 @@ type Overload struct { // Only one of the following three attributes can be set. // Fn is the normal builtin implementation function. It's for functions that - // take in datums and return a datum. + // take in Datums and return a Datum. Fn func(*EvalContext, Datums) (Datum, error) - // Generator is for SRFs. SRFs take datums and return multiple rows of datums. + // FnWithExprs is for builtins that need access to their arguments as Exprs + // and not pre-evaluated Datums, but is otherwise identical to Fn. + FnWithExprs func(*EvalContext, Exprs) (Datum, error) + + // Generator is for SRFs. SRFs take Datums and return multiple rows of Datums. Generator GeneratorFactory // GeneratorWithExprs is for SRFs that need access to their arguments as Exprs diff --git a/pkg/sql/sem/tree/testdata/eval/cast b/pkg/sql/sem/tree/testdata/eval/cast index f325ea3a8be2..4e293cd97713 100644 --- a/pkg/sql/sem/tree/testdata/eval/cast +++ b/pkg/sql/sem/tree/testdata/eval/cast @@ -382,6 +382,16 @@ eval ---- e'a\U0001F41Bb' +eval +'hello'::varchar(2) +---- +'he' + +eval +'a🐛b🏠c'::varchar(3) +---- +e'a\U0001F41Bb' + eval 'hello'::bytes ---- @@ -1336,3 +1346,33 @@ eval 0::char ---- '0' + +eval +1::BIT +---- +B'1' + +eval +2::BIT +---- +B'0' + +eval +123::BIT +---- +B'1' + +eval +'1'::BIT +---- +B'1' + +eval +'01'::BIT +---- +B'0' + +eval +'2'::BIT +---- +could not parse string as bit array: "2" is not a valid binary digit diff --git a/pkg/sql/sem/tree/type_check.go b/pkg/sql/sem/tree/type_check.go index f0d3bea0dbc7..086682543cc8 100644 --- a/pkg/sql/sem/tree/type_check.go +++ b/pkg/sql/sem/tree/type_check.go @@ -456,7 +456,8 @@ func resolveCast( return nil default: - cast := lookupCast(fromFamily, toFamily, intervalStyleEnabled, dateStyleEnabled) + // TODO(mgartner): Use OID cast map. + cast := lookupCastInfo(fromFamily, toFamily, intervalStyleEnabled, dateStyleEnabled) if cast == nil { return pgerror.Newf(pgcode.CannotCoerce, "invalid cast: %s -> %s", castFrom, castTo) } @@ -1550,6 +1551,12 @@ func (expr *ArrayFlatten) TypeCheck( func (expr *Placeholder) TypeCheck( ctx context.Context, semaCtx *SemaContext, desired *types.T, ) (TypedExpr, error) { + // When we populate placeholder values from pgwire, there is no special + // handling of type details like widths. Therefore, we infer the types of + // placeholders as only canonical types. This is safe to do because a value + // can always be losslessly converted to its canonical type. + canonicalDesired := desired.CanonicalType() + // Perform placeholder typing. This function is only called during Prepare, // when there are no available values for the placeholders yet, because // during Execute all placeholders are replaced from the AST before type @@ -1567,7 +1574,7 @@ func (expr *Placeholder) TypeCheck( // the type system expects. Then, when the value is actually sent to us // later, we cast the input value (whose type is the expected type) to the // desired type here. - typ = desired + typ = canonicalDesired } // We call SetType regardless of the above condition to inform the // placeholder struct that this placeholder is locked to its type and cannot @@ -1581,10 +1588,10 @@ func (expr *Placeholder) TypeCheck( if desired.IsAmbiguous() { return nil, placeholderTypeAmbiguityError(expr.Idx) } - if err := semaCtx.Placeholders.SetType(expr.Idx, desired); err != nil { + if err := semaCtx.Placeholders.SetType(expr.Idx, canonicalDesired); err != nil { return nil, err } - expr.typ = desired + expr.typ = canonicalDesired return expr, nil } diff --git a/pkg/sql/sem/tree/type_check_test.go b/pkg/sql/sem/tree/type_check_test.go index 228cf9045a81..e30c0b7abceb 100644 --- a/pkg/sql/sem/tree/type_check_test.go +++ b/pkg/sql/sem/tree/type_check_test.go @@ -123,6 +123,8 @@ func TestTypeCheck(t *testing.T) { {`1 + $1`, `1:::INT8 + $1:::INT8`}, {`1:::DECIMAL + $1`, `1:::DECIMAL + $1:::DECIMAL`}, {`$1:::INT8`, `$1:::INT8`}, + {`2::DECIMAL(10,2) + $1`, `2:::DECIMAL::DECIMAL(10,2) + $1:::DECIMAL`}, + {`2::DECIMAL(10,0) + $1`, `2:::DECIMAL::DECIMAL(10) + $1:::DECIMAL`}, // Tuples with labels {`(ROW (1) AS a)`, `((1:::INT8,) AS a)`}, diff --git a/pkg/sql/table_test.go b/pkg/sql/table_test.go index 04d0ecaeb1b8..18939cc86ed5 100644 --- a/pkg/sql/table_test.go +++ b/pkg/sql/table_test.go @@ -160,7 +160,7 @@ func TestMakeTableDescColumns(t *testing.T) { }, { `"char"`, - types.MakeQChar(0), + types.QChar, true, }, { diff --git a/pkg/sql/tablewriter_upsert_opt.go b/pkg/sql/tablewriter_upsert_opt.go index e48eb9fcf0e2..c8a5048cd94d 100644 --- a/pkg/sql/tablewriter_upsert_opt.go +++ b/pkg/sql/tablewriter_upsert_opt.go @@ -268,7 +268,11 @@ func (tu *optTableUpserter) updateConflictingRow( // via GenerateInsertRow(). // - for the fetched part, we assume that the data in the table is // correct already. - if err := enforceLocalColumnConstraints(updateValues, tu.updateCols); err != nil { + if err := enforceLocalColumnConstraints( + updateValues, + tu.updateCols, + true, /* isUpdate */ + ); err != nil { return err } diff --git a/pkg/sql/types/oid.go b/pkg/sql/types/oid.go index ffc2442e91e1..2e567e4c42fe 100644 --- a/pkg/sql/types/oid.go +++ b/pkg/sql/types/oid.go @@ -64,7 +64,7 @@ var OidToType = map[oid.Oid]*T{ oid.T_bool: Bool, oid.T_bpchar: typeBpChar, oid.T_bytea: Bytes, - oid.T_char: typeQChar, + oid.T_char: QChar, oid.T_date: Date, oid.T_float4: Float4, oid.T_float8: Float, diff --git a/pkg/sql/types/types.go b/pkg/sql/types/types.go index 7656f0051903..988034950085 100644 --- a/pkg/sql/types/types.go +++ b/pkg/sql/types/types.go @@ -312,6 +312,15 @@ var ( VarChar = &T{InternalType: InternalType{ Family: StringFamily, Oid: oid.T_varchar, Locale: &emptyLocale}} + // QChar is the special "char" type that is a single-character column type. + // It's used by system tables. It is reported as "char" (with double quotes + // included) in SHOW CREATE and "char" in introspection for compatibility + // with PostgreSQL. + // + // See https://www.postgresql.org/docs/9.1/static/datatype-character.html + QChar = &T{InternalType: InternalType{ + Family: StringFamily, Oid: oid.T_char, Width: 1, Locale: &emptyLocale}} + // Name is a type-alias for String with a different OID (T_name). It is // reported as NAME in SHOW CREATE and "name" in introspection for // compatibility with PostgreSQL. @@ -583,7 +592,7 @@ var ( // typeBpChar is the "standard SQL" string type of fixed length, where "bp" // stands for "blank padded". It is not exported to avoid confusion with - // typeQChar, as well as confusion over its default width. + // QChar, as well as confusion over its default width. // // It is reported as CHAR in SHOW CREATE and "character" in introspection for // compatibility with PostgreSQL. @@ -591,16 +600,6 @@ var ( // Its default maximum with is 1. It always has a maximum width. typeBpChar = &T{InternalType: InternalType{ Family: StringFamily, Oid: oid.T_bpchar, Locale: &emptyLocale}} - - // typeQChar is a special PostgreSQL-only type supported for compatibility. - // It behaves like VARCHAR, its maximum width cannot be modified, and has a - // peculiar name in the syntax and introspection. It is not exported to avoid - // confusion with typeBpChar, as well as confusion over its default width. - // - // It is reported as "char" (with double quotes included) in SHOW CREATE and - // "char" in introspection for compatibility with PostgreSQL. - typeQChar = &T{InternalType: InternalType{ - Family: StringFamily, Oid: oid.T_char, Locale: &emptyLocale}} ) const ( @@ -798,28 +797,15 @@ func MakeVarChar(width int32) *T { } // MakeChar constructs a new instance of the CHAR type (oid = T_bpchar) having -// the given max # characters (0 = unspecified number). +// the given max number of characters. func MakeChar(width int32) *T { - if width == 0 { - return typeBpChar - } - if width < 0 { - panic(errors.AssertionFailedf("width %d cannot be negative", width)) + if width <= 0 { + panic(errors.AssertionFailedf("width for type char must be at least 1")) } return &T{InternalType: InternalType{ Family: StringFamily, Oid: oid.T_bpchar, Width: width, Locale: &emptyLocale}} } -// MakeQChar constructs a new instance of the "char" type (oid = T_char) having -// the given max # characters (0 = unspecified number). -func MakeQChar(width int32) *T { - if width == 0 { - return typeQChar - } - return &T{InternalType: InternalType{ - Family: StringFamily, Oid: oid.T_char, Width: width, Locale: &emptyLocale}} -} - // MakeCollatedString constructs a new instance of a CollatedStringFamily type // that is collated according to the given locale. The new type is based upon // the given string type, having the same oid and width values. For example: @@ -1190,27 +1176,30 @@ func (t *T) Precision() int32 { // Array types have the same type modifier as the contents of the array. // The value will be -1 for types that do not need atttypmod. func (t *T) TypeModifier() int32 { - typeModifier := int32(-1) if t.Family() == ArrayFamily { return t.ArrayContents().TypeModifier() } + // The type modifier for "char" is always -1. + if t.Oid() == oid.T_char { + return int32(-1) + } if width := t.Width(); width != 0 { switch t.Family() { case StringFamily, CollatedStringFamily: // Postgres adds 4 to the attypmod for bounded string types, the // var header size. - typeModifier = width + 4 + return width + 4 case BitFamily: - typeModifier = width + return width case DecimalFamily: // attTypMod is calculated by putting the precision in the upper // bits and the scale in the lower bits of a 32-bit int, and adding // 4 (the var header size). We mock this for clients' sake. See // numeric.c. - typeModifier = ((t.Precision() << 16) | width) + 4 + return ((t.Precision() << 16) | width) + 4 } } - return typeModifier + return int32(-1) } // Scale is an alias method for Width, used for clarity for types in @@ -1316,6 +1305,71 @@ func (f Family) Name() string { return ret } +// CanonicalType returns the canonical type of the given type's family. If the +// family does not have a canonical type, the original type is returned. +func (t *T) CanonicalType() *T { + switch t.Family() { + case BoolFamily: + return Bool + case IntFamily: + return Int + case FloatFamily: + return Float + case DecimalFamily: + return Decimal + case DateFamily: + return Date + case TimestampFamily: + return Timestamp + case IntervalFamily: + return Interval + case StringFamily: + return String + case BytesFamily: + return Bytes + case TimestampTZFamily: + return TimestampTZ + case CollatedStringFamily: + // CollatedStringFamily has no canonical type. + return t + case OidFamily: + return Oid + case UnknownFamily: + return Unknown + case UuidFamily: + return Uuid + case ArrayFamily: + // ArrayFamily has no canonical type. + return t + case INetFamily: + return INet + case TimeFamily: + return Time + case JsonFamily: + return Jsonb + case TimeTZFamily: + return TimeTZ + case TupleFamily: + // TupleFamily has no canonical type. + return t + case BitFamily: + return VarBit + case GeometryFamily: + return Geometry + case GeographyFamily: + return Geography + case EnumFamily: + // EnumFamily has no canonical type. + return t + case Box2DFamily: + return Box2D + case AnyFamily: + return Any + default: + panic(errors.AssertionFailedf("unexpected type family: %v", errors.Safe(t.Family()))) + } +} + // Name returns a single word description of the type that describes it // succinctly, but without all the details, such as width, locale, etc. The name // is sometimes the same as the name returned by SQLStandardName, but is more diff --git a/pkg/sql/types/types_test.go b/pkg/sql/types/types_test.go index 2fd5a284e42c..37dc54fe70ed 100644 --- a/pkg/sql/types/types_test.go +++ b/pkg/sql/types/types_test.go @@ -119,12 +119,10 @@ func TestTypes(t *testing.T) { {MakeCollatedString(MakeChar(20), enCollate), MakeScalar(CollatedStringFamily, oid.T_bpchar, 0, 20, enCollate)}, - {MakeCollatedString(typeQChar, enCollate), &T{InternalType: InternalType{ - Family: CollatedStringFamily, Oid: oid.T_char, Locale: &enCollate}}}, - {MakeCollatedString(MakeQChar(20), enCollate), &T{InternalType: InternalType{ - Family: CollatedStringFamily, Oid: oid.T_char, Width: 20, Locale: &enCollate}}}, - {MakeCollatedString(MakeQChar(20), enCollate), - MakeScalar(CollatedStringFamily, oid.T_char, 0, 20, enCollate)}, + {MakeCollatedString(QChar, enCollate), &T{InternalType: InternalType{ + Family: CollatedStringFamily, Oid: oid.T_char, Width: 1, Locale: &enCollate}}}, + {MakeCollatedString(QChar, enCollate), + MakeScalar(CollatedStringFamily, oid.T_char, 0, 1, enCollate)}, {MakeCollatedString(Name, enCollate), &T{InternalType: InternalType{ Family: CollatedStringFamily, Oid: oid.T_name, Locale: &enCollate}}}, @@ -381,19 +379,15 @@ func TestTypes(t *testing.T) { Family: StringFamily, Oid: oid.T_varchar, Width: 20, Locale: &emptyLocale}}}, {MakeVarChar(20), MakeScalar(StringFamily, oid.T_varchar, 0, 20, emptyLocale)}, - {MakeChar(0), typeBpChar}, - {MakeChar(0), &T{InternalType: InternalType{ - Family: StringFamily, Oid: oid.T_bpchar, Locale: &emptyLocale}}}, + {MakeChar(1), &T{InternalType: InternalType{ + Family: StringFamily, Oid: oid.T_bpchar, Width: 1, Locale: &emptyLocale}}}, {MakeChar(20), &T{InternalType: InternalType{ Family: StringFamily, Oid: oid.T_bpchar, Width: 20, Locale: &emptyLocale}}}, {MakeChar(20), MakeScalar(StringFamily, oid.T_bpchar, 0, 20, emptyLocale)}, - {MakeQChar(0), typeQChar}, - {MakeQChar(0), &T{InternalType: InternalType{ - Family: StringFamily, Oid: oid.T_char, Locale: &emptyLocale}}}, - {MakeQChar(20), &T{InternalType: InternalType{ - Family: StringFamily, Oid: oid.T_char, Width: 20, Locale: &emptyLocale}}}, - {MakeQChar(20), MakeScalar(StringFamily, oid.T_char, 0, 20, emptyLocale)}, + {QChar, &T{InternalType: InternalType{ + Family: StringFamily, Oid: oid.T_char, Width: 1, Locale: &emptyLocale}}}, + {QChar, MakeScalar(StringFamily, oid.T_char, 0, 1, emptyLocale)}, {Name, &T{InternalType: InternalType{ Family: StringFamily, Oid: oid.T_name, Locale: &emptyLocale}}}, @@ -691,7 +685,7 @@ func TestMarshalCompat(t *testing.T) { {MakeString(10), InternalType{Family: StringFamily, Oid: oid.T_text, Width: 10}}, {VarChar, InternalType{Family: StringFamily, Oid: oid.T_varchar, VisibleType: visibleVARCHAR}}, {MakeChar(10), InternalType{Family: StringFamily, Oid: oid.T_bpchar, Width: 10, VisibleType: visibleCHAR}}, - {MakeQChar(1), InternalType{Family: StringFamily, Oid: oid.T_char, Width: 1, VisibleType: visibleQCHAR}}, + {QChar, InternalType{Family: StringFamily, Oid: oid.T_char, Width: 1, VisibleType: visibleQCHAR}}, {Name, InternalType{Family: name, Oid: oid.T_name}}, } @@ -755,7 +749,7 @@ func TestUnmarshalCompat(t *testing.T) { {InternalType{Family: StringFamily, VisibleType: visibleVARCHAR}, VarChar}, {InternalType{Family: StringFamily, VisibleType: visibleVARCHAR, Width: 20}, MakeVarChar(20)}, {InternalType{Family: StringFamily, VisibleType: visibleCHAR}, typeBpChar}, - {InternalType{Family: StringFamily, VisibleType: visibleQCHAR}, typeQChar}, + {InternalType{Family: StringFamily, VisibleType: visibleQCHAR, Width: 1}, QChar}, } for _, tc := range testCases { diff --git a/pkg/sql/update.go b/pkg/sql/update.go index 029c635e2acf..4651ffb73cbd 100644 --- a/pkg/sql/update.go +++ b/pkg/sql/update.go @@ -278,7 +278,11 @@ func (u *updateNode) processSourceRow(params runParams, sourceVals tree.Datums) // Verify the schema constraints. For consistency with INSERT/UPSERT // and compatibility with PostgreSQL, we must do this before // processing the CHECK constraints. - if err := enforceLocalColumnConstraints(u.run.updateValues, u.run.tu.ru.UpdateCols); err != nil { + if err := enforceLocalColumnConstraints( + u.run.updateValues, + u.run.tu.ru.UpdateCols, + true, /* isUpdate */ + ); err != nil { return err } @@ -410,28 +414,23 @@ func (ss scalarSlot) checkColumnTypes(row []tree.TypedExpr) error { return colinfo.CheckDatumTypeFitsColumnType(ss.column, typ) } -// enforceLocalColumnConstraints asserts the column constraints that -// do not require data validation from other sources than the row data -// itself. This includes: -// - rejecting null values in non-nullable columns; -// - checking width constraints from the column type; -// - truncating results to the requested precision (not width). -// Note: the second point is what distinguishes this operation -// from a regular SQL cast -- here widths are checked, not -// used to truncate the value silently. -// -// The row buffer is modified in-place with the result of the -// checks. -func enforceLocalColumnConstraints(row tree.Datums, cols []catalog.Column) error { +// enforceLocalColumnConstraints asserts the column constraints that do not +// require data validation from other sources than the row data itself. This +// currently only includes checking for null values in non-nullable columns. +func enforceLocalColumnConstraints(row tree.Datums, cols []catalog.Column, isUpdate bool) error { for i, col := range cols { if !col.IsNullable() && row[i] == tree.DNull { return sqlerrors.NewNonNullViolationError(col.GetName()) } - outVal, err := tree.AdjustValueToType(col.GetType(), row[i]) - if err != nil { - return err + if isUpdate { + // TODO(mgartner): Remove this once assignment casts are supported + // for UPSERTs and UPDATEs. + outVal, err := tree.AdjustValueToType(col.GetType(), row[i]) + if err != nil { + return err + } + row[i] = outVal } - row[i] = outVal } return nil } diff --git a/pkg/sql/upsert.go b/pkg/sql/upsert.go index c6ca3b5df73b..8a8600adf3ab 100644 --- a/pkg/sql/upsert.go +++ b/pkg/sql/upsert.go @@ -137,7 +137,11 @@ func (n *upsertNode) BatchedNext(params runParams) (bool, error) { // processSourceRow processes one row from the source for upsertion. // The table writer is in charge of accumulating the result rows. func (n *upsertNode) processSourceRow(params runParams, rowVals tree.Datums) error { - if err := enforceLocalColumnConstraints(rowVals, n.run.insertCols); err != nil { + if err := enforceLocalColumnConstraints( + rowVals, + n.run.insertCols, + true, /* isUpdate */ + ); err != nil { return err } diff --git a/pkg/workload/schemachange/type_resolver.go b/pkg/workload/schemachange/type_resolver.go index cc08b2202377..8e8be208c35e 100644 --- a/pkg/workload/schemachange/type_resolver.go +++ b/pkg/workload/schemachange/type_resolver.go @@ -104,7 +104,7 @@ ORDER BY enumsortorder`, name.Object(), name.Schema()) return nil, pgerror.Newf(pgcode.UndefinedObject, "type %s with oid %s does not exist", name.Object(), objectID) } // Special case CHAR to have the right width. - if objectID == oid.T_char || objectID == oid.T_bpchar { + if objectID == oid.T_bpchar { t := *types.OidToType[objectID] t.InternalType.Width = 1 return &t, nil