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.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
|