Skip to content

Commit

Permalink
sql: check unsupported types during schema changes and type/function …
Browse files Browse the repository at this point in the history
…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 cockroachdb#111560

Release note: None
  • Loading branch information
DrewKimball committed Oct 4, 2023
1 parent 969d1c1 commit 3cba25c
Show file tree
Hide file tree
Showing 12 changed files with 256 additions and 16 deletions.
2 changes: 2 additions & 0 deletions pkg/sql/backfill/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down Expand Up @@ -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,
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/create_type.go
Original file line number Diff line number Diff line change
Expand Up @@ -423,6 +423,10 @@ func CreateCompositeTypeDesc(
if err != nil {
return nil, err
}
err = tree.CheckUnsupportedType(params.ctx, &params.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")
Expand Down
147 changes: 141 additions & 6 deletions pkg/sql/logictest/testdata/logic_test/pg_lsn_mixed
Original file line number Diff line number Diff line change
@@ -1,8 +1,143 @@
# 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.
# ----------------------------------------------------------------------

# 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;

# 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');

# UDT that references PG_LSN.
statement error pgcode 0A000 unimplemented: this syntax
CREATE TYPE typ AS (x INT, y PG_LSN);

# 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;

# ----------------------------------------------------------------------
# Verify that PG_LSN is not allowed after upgrading the gateway.
# ----------------------------------------------------------------------

upgrade 0

user root nodeidx=0

# 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;

# 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');

# 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);

# 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;
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@ go_test(
"//pkg/sql/logictest:testdata", # keep
],
exec_properties = {"Pool": "large"},
shard_count = 11,
shard_count = 12,
tags = [
"cpu:2",
],
Expand Down

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

21 changes: 19 additions & 2 deletions pkg/sql/opt/optbuilder/create_function.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@
package optbuilder

import (
"context"
"fmt"

"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
Expand Down Expand Up @@ -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,
Expand Down Expand Up @@ -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)
}
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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)
}
}
2 changes: 2 additions & 0 deletions pkg/sql/planner.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down Expand Up @@ -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

Expand Down
1 change: 1 addition & 0 deletions pkg/sql/schemachanger/scbuild/tree_context_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}

Expand Down
1 change: 1 addition & 0 deletions pkg/sql/sem/eval/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ go_library(
"testing_knobs.go",
"tuple.go",
"unary_op.go",
"unsupported_types.go",
"window_funcs.go",
"window_funcs_util.go",
],
Expand Down
47 changes: 47 additions & 0 deletions pkg/sql/sem/eval/unsupported_types.go
Original file line number Diff line number Diff line change
@@ -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
}
31 changes: 31 additions & 0 deletions pkg/sql/sem/tree/type_check.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -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,
Expand Down Expand Up @@ -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)
}

0 comments on commit 3cba25c

Please sign in to comment.