From efc62671b41bb1f5f845b015669dfd23554cb61e Mon Sep 17 00:00:00 2001 From: Drew Kimball Date: Mon, 2 Oct 2023 02:07:01 -0600 Subject: [PATCH] sql: check unsupported types during schema changes and type/function creation It is necessary to check the cluster version when building an expression that references a newly-added type, in order to ensure that all nodes support that type. Previously, these checks were omitted for casts in the vectorized engine, expressions for partial index predicates and check constraints, function parameters and return types, and user-defined composite types. This patch adds version-checking for function/procedure parameters and return types, as well as for user-defined types. It also augments the type-checking logic for casts and type annotations with a version check; this handles the remaining cases. The execution-time checks for cast expressions are left untouched, just in case the new type-checking logic misses important cases. For now, these changes only apply to the `PG_LSN` type, which will be new in 23.2. A future commit will add support for `REFCURSOR`, and will need to use the same checks. Informs #111560 Release note: None --- pkg/sql/backfill/backfill.go | 2 + pkg/sql/create_type.go | 4 + .../testdata/logic_test/pg_lsn_mixed | 167 +++++++++++++++++- .../BUILD.bazel | 2 +- .../generated_test.go | 7 + .../local-mixed-22.2-23.1/generated_test.go | 7 - pkg/sql/opt/optbuilder/create_function.go | 21 ++- pkg/sql/planner.go | 2 + .../scbuild/tree_context_builder.go | 1 + pkg/sql/sem/eval/BUILD.bazel | 1 + pkg/sql/sem/eval/unsupported_types.go | 47 +++++ pkg/sql/sem/tree/type_check.go | 31 ++++ 12 files changed, 276 insertions(+), 16 deletions(-) create mode 100644 pkg/sql/sem/eval/unsupported_types.go diff --git a/pkg/sql/backfill/backfill.go b/pkg/sql/backfill/backfill.go index c1db5289f33d..fdc4c255133a 100644 --- a/pkg/sql/backfill/backfill.go +++ b/pkg/sql/backfill/backfill.go @@ -236,6 +236,7 @@ func (cb *ColumnBackfiller) InitForDistributedUse( // Set up a SemaContext to type check the default and computed expressions. semaCtx := tree.MakeSemaContext() semaCtx.TypeResolver = &resolver + semaCtx.UnsupportedTypeChecker = eval.NewUnsupportedTypeChecker(evalCtx.Settings.Version) var err error defaultExprs, err = schemaexpr.MakeDefaultExprs( ctx, cb.added, &transform.ExprTransformContext{}, evalCtx, &semaCtx, @@ -670,6 +671,7 @@ func (ib *IndexBackfiller) InitForDistributedUse( // Set up a SemaContext to type check the default and computed expressions. semaCtx := tree.MakeSemaContext() semaCtx.TypeResolver = &resolver + semaCtx.UnsupportedTypeChecker = eval.NewUnsupportedTypeChecker(evalCtx.Settings.Version) // Convert any partial index predicate strings into expressions. predicates, colExprs, referencedColumns, err = constructExprs( ctx, desc, ib.added, ib.cols, ib.addedCols, ib.computedCols, evalCtx, &semaCtx, diff --git a/pkg/sql/create_type.go b/pkg/sql/create_type.go index 83bd3910171b..06abc86f62f7 100644 --- a/pkg/sql/create_type.go +++ b/pkg/sql/create_type.go @@ -423,6 +423,10 @@ func CreateCompositeTypeDesc( if err != nil { return nil, err } + err = tree.CheckUnsupportedType(params.ctx, ¶ms.p.semaCtx, typ) + if err != nil { + return nil, err + } if typ.UserDefined() { return nil, unimplemented.NewWithIssue(91779, "composite types that reference user-defined types not yet supported") diff --git a/pkg/sql/logictest/testdata/logic_test/pg_lsn_mixed b/pkg/sql/logictest/testdata/logic_test/pg_lsn_mixed index 41e80a673aa8..a1ce07d46a2b 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_lsn_mixed +++ b/pkg/sql/logictest/testdata/logic_test/pg_lsn_mixed @@ -1,8 +1,163 @@ -# LogicTest: local-mixed-22.2-23.1 -# TODO(otan): add tests for mixed 23.1-23.2. +# LogicTest: cockroach-go-testserver-upgrade-to-master -statement error must be finalized to use pg_lsn -SELECT '1010F/AAAA'::text::pg_lsn +statement ok +CREATE TABLE xy (x INT, y INT); -statement error pg_lsn not supported until version 23.2 -CREATE TABLE pg_lsn_table(id pg_lsn, val pg_lsn) +# ---------------------------------------------------------------------- +# Test PG_LSN references with all nodes running old binaries. +# ---------------------------------------------------------------------- + +subtest all_old_cast + +# Cast to PG_LSN. +statement error pgcode 0A000 unimplemented: this syntax +SELECT 'A01F0/1AAA'::PG_LSN; + +# Cast to PG_LSN using the vectorized engine. +statement error pgcode 0A000 unimplemented: this syntax +SELECT 'a01f0/1aaa'::PG_LSN FROM generate_series(1, 100) LIMIT 1; + +subtest all_old_table + +# Table that references PG_LSN. +statement error pgcode 0A000 unimplemented: this syntax +CREATE TABLE t (x PG_LSN); + +# Add a PG_LSN column. +statement error pgcode 0A000 unimplemented: this syntax +ALTER TABLE xy ADD COLUMN curs PG_LSN; + +# Alter a column type to PG_LSN. +statement ok +SET enable_experimental_alter_column_type_general=true; + +statement error pgcode 0A000 unimplemented: this syntax +ALTER TABLE xy ALTER COLUMN y TYPE PG_LSN; + +# Create a partial index that uses the PG_LSN type. +statement error pgcode 0A000 unimplemented: this syntax +CREATE INDEX part ON xy (x) WHERE y::PG_LSN < 'a01f0/1aaa'; + +# Add a check constraint that uses the PG_LSN type. +statement error pgcode 0A000 unimplemented: this syntax +ALTER TABLE xy ADD CONSTRAINT bar CHECK (y::PG_LSN < 'fffff100/100'); + +subtest all_old_type + +# UDT that references PG_LSN. +statement error pgcode 0A000 unimplemented: this syntax +CREATE TYPE typ AS (x INT, y PG_LSN); + +subtest all_old_function + +# Function that returns PG_LSN. +statement error pgcode 0A000 unimplemented: this syntax +CREATE OR REPLACE FUNCTION f() RETURNS PG_LSN AS $$ + SELECT 'a01f0/1aaa'; +$$ LANGUAGE SQL; + +# Function that takes PG_LSN argument. +statement error pgcode 0A000 unimplemented: this syntax +CREATE OR REPLACE FUNCTION f(curs PG_LSN) RETURNS STRING AS $$ + SELECT curs; +$$ LANGUAGE SQL; + +# Function that references PG_LSN internally. +statement error pgcode 0A000 unimplemented: this syntax +CREATE OR REPLACE FUNCTION f() RETURNS INT AS $$ + SELECT 'a01f0/1aaa'::PG_LSN; + SELECT 0; +$$ LANGUAGE SQL; + +# Function that returns a composite type with PG_LSN component. +statement error pgcode 0A000 unimplemented: this syntax +CREATE FUNCTION f() RETURNS RECORD AS $$ + SELECT (1, 'a01f0/1aaa'::PG_LSN, true); +$$ LANGUAGE SQL; + +subtest end + +# ---------------------------------------------------------------------- +# Verify that PG_LSN is not allowed after upgrading the gateway. +# ---------------------------------------------------------------------- + +upgrade 0 + +user root nodeidx=0 + +subtest upgrade_one_cast + +# Cast to PG_LSN. +statement error pgcode 0A000 pq: pg_lsn not supported until version 23.2 +SELECT 'a01f0/1aaa'::PG_LSN; + +# Cast to PG_LSN using the vectorized engine. +statement error pgcode 0A000 pq: pg_lsn not supported until version 23.2 +SELECT 'a01f0/1aaa'::PG_LSN FROM generate_series(1, 100) LIMIT 1; + +subtest upgrade_one_table + +# Table that references PG_LSN. +statement error pgcode 0A000 pq: pg_lsn not supported until version 23.2 +CREATE TABLE t (x PG_LSN); + +# Add a PG_LSN column. +statement error pgcode 0A000 pq: pg_lsn not supported until version 23.2 +ALTER TABLE xy ADD COLUMN curs PG_LSN; + +# Alter a column type to PG_LSN. +statement ok +SET enable_experimental_alter_column_type_general=true; + +statement error pgcode 0A000 pq: pg_lsn not supported until version 23.2 +ALTER TABLE xy ALTER COLUMN y TYPE PG_LSN; + +# Create a partial index that uses the PG_LSN type. +statement error pgcode 0A000 pq: pg_lsn not supported until version 23.2 +CREATE INDEX part ON xy (x) WHERE y::PG_LSN < 'a01f0/1aaa'; + +# Add a check constraint that uses the PG_LSN type. +statement error pgcode 0A000 pq: pg_lsn not supported until version 23.2 +ALTER TABLE xy ADD CONSTRAINT bar CHECK (y::PG_LSN < 'fffff100/100'); + +subtest upgrade_one_type + +# UDT that references PG_LSN. +statement error pgcode 0A000 pq: pg_lsn not supported until version 23.2 +CREATE TYPE typ AS (x INT, y PG_LSN); + +subtest upgrade_one_function + +# Function that returns PG_LSN. +statement error pgcode 0A000 pq: pg_lsn not supported until version 23.2 +CREATE OR REPLACE FUNCTION f() RETURNS PG_LSN AS $$ + SELECT 'a01f0/1aaa'; +$$ LANGUAGE SQL; + +# Function that takes PG_LSN argument. +statement error pgcode 0A000 pq: pg_lsn not supported until version 23.2 +CREATE OR REPLACE FUNCTION f(curs PG_LSN) RETURNS INT AS $$ + SELECT 0; +$$ LANGUAGE SQL; + +# Function that references PG_LSN internally. +statement error pgcode 0A000 pq: pg_lsn not supported until version 23.2 +CREATE OR REPLACE FUNCTION f() RETURNS INT AS $$ + SELECT 'a01f0/1aaa'::PG_LSN; + SELECT 0; +$$ LANGUAGE SQL; + +# Function that returns a composite type with PG_LSN component. +statement error pgcode 0A000 pq: pg_lsn not supported until version 23.2 +CREATE FUNCTION f() RETURNS RECORD AS $$ + SELECT (1, 'a01f0/1aaa'::PG_LSN, true); +$$ LANGUAGE SQL; + +statement error pgcode 0A000 pq: pg_lsn not supported until version 23.2 +CREATE FUNCTION f() RETURNS RECORD AS $$ + BEGIN + RETURN (1, 'a01f0/1aaa'::PG_LSN, true); + END +$$ LANGUAGE PLpgSQL; + +subtest end diff --git a/pkg/sql/logictest/tests/cockroach-go-testserver-upgrade-to-master/BUILD.bazel b/pkg/sql/logictest/tests/cockroach-go-testserver-upgrade-to-master/BUILD.bazel index 9e1a32cc3035..cee2384240f3 100644 --- a/pkg/sql/logictest/tests/cockroach-go-testserver-upgrade-to-master/BUILD.bazel +++ b/pkg/sql/logictest/tests/cockroach-go-testserver-upgrade-to-master/BUILD.bazel @@ -15,7 +15,7 @@ go_test( "//pkg/sql/logictest:testdata", # keep ], exec_properties = {"Pool": "large"}, - shard_count = 11, + shard_count = 12, tags = [ "cpu:2", ], diff --git a/pkg/sql/logictest/tests/cockroach-go-testserver-upgrade-to-master/generated_test.go b/pkg/sql/logictest/tests/cockroach-go-testserver-upgrade-to-master/generated_test.go index 8e13fd4eca21..2f79468f9058 100644 --- a/pkg/sql/logictest/tests/cockroach-go-testserver-upgrade-to-master/generated_test.go +++ b/pkg/sql/logictest/tests/cockroach-go-testserver-upgrade-to-master/generated_test.go @@ -154,3 +154,10 @@ func TestLogic_mixed_version_upgrade_repair_descriptors( defer leaktest.AfterTest(t)() runLogicTest(t, "mixed_version_upgrade_repair_descriptors") } + +func TestLogic_pg_lsn_mixed( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "pg_lsn_mixed") +} diff --git a/pkg/sql/logictest/tests/local-mixed-22.2-23.1/generated_test.go b/pkg/sql/logictest/tests/local-mixed-22.2-23.1/generated_test.go index f8e02f2fceed..316850d0bbc7 100644 --- a/pkg/sql/logictest/tests/local-mixed-22.2-23.1/generated_test.go +++ b/pkg/sql/logictest/tests/local-mixed-22.2-23.1/generated_test.go @@ -1317,13 +1317,6 @@ func TestLogic_pg_extension( runLogicTest(t, "pg_extension") } -func TestLogic_pg_lsn_mixed( - t *testing.T, -) { - defer leaktest.AfterTest(t)() - runLogicTest(t, "pg_lsn_mixed") -} - func TestLogic_pgcrypto_builtins( t *testing.T, ) { diff --git a/pkg/sql/opt/optbuilder/create_function.go b/pkg/sql/opt/optbuilder/create_function.go index de80d5cc743f..5238330f8078 100644 --- a/pkg/sql/opt/optbuilder/create_function.go +++ b/pkg/sql/opt/optbuilder/create_function.go @@ -11,6 +11,7 @@ package optbuilder import ( + "context" "fmt" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -146,6 +147,8 @@ func (b *Builder) buildCreateFunction(cf *tree.CreateRoutine, inScope *scope) (o if err != nil { panic(err) } + // The parameter type must be supported by the current cluster version. + checkUnsupportedType(b.ctx, b.semaCtx, typ) if types.IsRecordType(typ) { if language == tree.RoutineLangSQL { panic(pgerror.Newf(pgcode.InvalidFunctionDefinition, @@ -258,7 +261,7 @@ func (b *Builder) buildCreateFunction(cf *tree.CreateRoutine, inScope *scope) (o // TODO(mgartner): stmtScope.cols does not describe the result // columns of the statement. We should use physical.Presentation // instead. - err = validateReturnType(funcReturnType, stmtScope.cols) + err = validateReturnType(b.ctx, b.semaCtx, funcReturnType, stmtScope.cols) if err != nil { panic(err) } @@ -301,7 +304,15 @@ func formatFuncBodyStmt(fmtCtx *tree.FmtCtx, ast tree.NodeFormatter, newLine boo fmtCtx.WriteString(";") } -func validateReturnType(expected *types.T, cols []scopeColumn) error { +func validateReturnType( + ctx context.Context, semaCtx *tree.SemaContext, expected *types.T, cols []scopeColumn, +) error { + // The return type must be supported by the current cluster version. + checkUnsupportedType(ctx, semaCtx, expected) + for i := range cols { + checkUnsupportedType(ctx, semaCtx, cols[i].typ) + } + // If return type is void, any column types are valid. if expected.Equivalent(types.Void) { return nil @@ -406,3 +417,9 @@ func checkStmtVolatility( } } } + +func checkUnsupportedType(ctx context.Context, semaCtx *tree.SemaContext, typ *types.T) { + if err := tree.CheckUnsupportedType(ctx, semaCtx, typ); err != nil { + panic(err) + } +} diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index 6bd0b345192c..ec60d24963d3 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -404,6 +404,7 @@ func newInternalPlanner( p.semaCtx.NameResolver = p p.semaCtx.DateStyle = sd.GetDateStyle() p.semaCtx.IntervalStyle = sd.GetIntervalStyle() + p.semaCtx.UnsupportedTypeChecker = eval.NewUnsupportedTypeChecker(execCfg.Settings.Version) plannerMon := mon.NewMonitor(redact.Sprintf("internal-planner.%s.%s", user, opName), mon.MemoryResource, @@ -903,6 +904,7 @@ func (p *planner) resetPlanner( p.semaCtx.NameResolver = p p.semaCtx.DateStyle = sd.GetDateStyle() p.semaCtx.IntervalStyle = sd.GetIntervalStyle() + p.semaCtx.UnsupportedTypeChecker = eval.NewUnsupportedTypeChecker(p.execCfg.Settings.Version) p.autoCommit = false diff --git a/pkg/sql/schemachanger/scbuild/tree_context_builder.go b/pkg/sql/schemachanger/scbuild/tree_context_builder.go index e5ee28360a0c..921552fc381a 100644 --- a/pkg/sql/schemachanger/scbuild/tree_context_builder.go +++ b/pkg/sql/schemachanger/scbuild/tree_context_builder.go @@ -36,6 +36,7 @@ func newSemaCtx(d Dependencies) *tree.SemaContext { semaCtx.NameResolver = d.CatalogReader() semaCtx.DateStyle = d.SessionData().GetDateStyle() semaCtx.IntervalStyle = d.SessionData().GetIntervalStyle() + semaCtx.UnsupportedTypeChecker = eval.NewUnsupportedTypeChecker(d.ClusterSettings().Version) return &semaCtx } diff --git a/pkg/sql/sem/eval/BUILD.bazel b/pkg/sql/sem/eval/BUILD.bazel index 0f597f8de4b4..67b3d2883676 100644 --- a/pkg/sql/sem/eval/BUILD.bazel +++ b/pkg/sql/sem/eval/BUILD.bazel @@ -29,6 +29,7 @@ go_library( "testing_knobs.go", "tuple.go", "unary_op.go", + "unsupported_types.go", "window_funcs.go", "window_funcs_util.go", ], diff --git a/pkg/sql/sem/eval/unsupported_types.go b/pkg/sql/sem/eval/unsupported_types.go new file mode 100644 index 000000000000..a6c9f98f2b05 --- /dev/null +++ b/pkg/sql/sem/eval/unsupported_types.go @@ -0,0 +1,47 @@ +// Copyright 2023 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 eval + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" +) + +type unsupportedTypeChecker struct { + version clusterversion.Handle +} + +// NewUnsupportedTypeChecker returns a new tree.UnsupportedTypeChecker that can +// be used to check whether a type is allowed by the current cluster version. +func NewUnsupportedTypeChecker(handle clusterversion.Handle) tree.UnsupportedTypeChecker { + return &unsupportedTypeChecker{version: handle} +} + +var _ tree.UnsupportedTypeChecker = &unsupportedTypeChecker{} + +// CheckType implements the tree.UnsupportedTypeChecker interface. +func (tc *unsupportedTypeChecker) CheckType(ctx context.Context, typ *types.T) error { + var errorTypeString string + if typ.Family() == types.PGLSNFamily { + errorTypeString = "pg_lsn" + } + if errorTypeString != "" && !tc.version.IsActive(ctx, clusterversion.V23_2) { + return pgerror.Newf(pgcode.FeatureNotSupported, + "%s not supported until version 23.2", errorTypeString, + ) + } + return nil +} diff --git a/pkg/sql/sem/tree/type_check.go b/pkg/sql/sem/tree/type_check.go index 0a01e31c40f2..850b26a564a5 100644 --- a/pkg/sql/sem/tree/type_check.go +++ b/pkg/sql/sem/tree/type_check.go @@ -75,6 +75,10 @@ type SemaContext struct { DateStyle pgdate.DateStyle // IntervalStyle refers to the IntervalStyle to parse as. IntervalStyle duration.IntervalStyle + + // UnsupportedTypeChecker is used to determine whether a builtin data type is + // supported by the current cluster version. It may be unset. + UnsupportedTypeChecker UnsupportedTypeChecker } // SemaProperties is a holder for required and derived properties @@ -599,6 +603,9 @@ func (expr *CastExpr) TypeCheck( if err != nil { return nil, err } + if err = CheckUnsupportedType(ctx, semaCtx, exprType); err != nil { + return nil, err + } expr.Type = exprType canElideCast := true switch { @@ -739,6 +746,9 @@ func (expr *AnnotateTypeExpr) TypeCheck( if err != nil { return nil, err } + if err = CheckUnsupportedType(ctx, semaCtx, annotateType); err != nil { + return nil, err + } expr.Type = annotateType subExpr, err := typeCheckAndRequire( ctx, @@ -3411,3 +3421,24 @@ func getMostSignificantOverload( } return ret, nil } + +// UnsupportedTypeChecker is used to check that a type is supported by the +// current cluster version. It is an interface because some packages cannot +// import the clusterversion package. +type UnsupportedTypeChecker interface { + // CheckType returns an error if the given type is not supported by the + // current cluster version. + CheckType(ctx context.Context, typ *types.T) error +} + +// CheckUnsupportedType returns an error if the given type is not supported by +// the current cluster version. If the given SemaContext is nil or +// uninitialized, CheckUnsupportedType returns nil. +func CheckUnsupportedType(ctx context.Context, semaCtx *SemaContext, typ *types.T) error { + if semaCtx == nil || semaCtx.UnsupportedTypeChecker == nil { + // Sometimes TypeCheck() is called with a nil SemaContext for tests, and + // some (non-test) locations don't initialize all fields of the SemaContext. + return nil + } + return semaCtx.UnsupportedTypeChecker.CheckType(ctx, typ) +}