diff --git a/pkg/sql/create_function.go b/pkg/sql/create_function.go index a80b194db6aa..9670325f0d91 100644 --- a/pkg/sql/create_function.go +++ b/pkg/sql/create_function.go @@ -432,7 +432,9 @@ func setFuncOption(params runParams, udfDesc *funcdesc.Mutable, option tree.Func if err != nil { return err } - typeReplacedFuncBody, err := serializeUserDefinedTypes(params.ctx, params.p.SemaCtx(), seqReplacedFuncBody, true /* multiStmt */) + typeReplacedFuncBody, err := serializeUserDefinedTypes( + params.ctx, params.p.SemaCtx(), seqReplacedFuncBody, true /* multiStmt */, "UDFs", + ) if err != nil { return err } diff --git a/pkg/sql/create_view.go b/pkg/sql/create_view.go index 5973a98a68af..c0f39816b3bc 100644 --- a/pkg/sql/create_view.go +++ b/pkg/sql/create_view.go @@ -411,7 +411,8 @@ func makeViewTableDesc( desc.ViewQuery = sequenceReplacedQuery } - typeReplacedQuery, err := serializeUserDefinedTypes(ctx, semaCtx, desc.ViewQuery, false /* multiStmt */) + typeReplacedQuery, err := serializeUserDefinedTypes(ctx, semaCtx, desc.ViewQuery, + false /* multiStmt */, "view queries") if err != nil { return tabledesc.Mutable{}, err } @@ -490,7 +491,7 @@ func replaceSeqNamesWithIDs( // and serialize any user defined types, so that renaming the type // does not corrupt the view. func serializeUserDefinedTypes( - ctx context.Context, semaCtx *tree.SemaContext, queries string, multiStmt bool, + ctx context.Context, semaCtx *tree.SemaContext, queries string, multiStmt bool, parentType string, ) (string, error) { replaceFunc := func(expr tree.Expr) (recurse bool, newExpr tree.Expr, err error) { var innerExpr tree.Expr @@ -505,20 +506,6 @@ func serializeUserDefinedTypes( default: return true, expr, nil } - // We cannot type-check subqueries without using optbuilder, and there - // is no need to because we only need to rewrite string values that are - // directly cast to enums. For example, we must rewrite the 'foo' in: - // - // SELECT 'foo'::myenum - // - // We don't need to rewrite the 'foo' in the query below, which can be - // corrupted by renaming the 'foo' value in the myenum type. - // - // SELECT (SELECT 'foo')::myenum - // - if _, ok := innerExpr.(*tree.Subquery); ok { - return true, expr, nil - } // semaCtx may be nil if this is a virtual view being created at // init time. var typeResolver tree.TypeReferenceResolver @@ -533,6 +520,14 @@ func serializeUserDefinedTypes( if !typ.UserDefined() { return true, expr, nil } + { + // We cannot type-check subqueries without using optbuilder, so we + // currently do not support casting expressions with subqueries to + // UDTs. + context := "casts to enums within " + parentType + defer semaCtx.Properties.Restore(semaCtx.Properties) + semaCtx.Properties.Require(context, tree.RejectSubqueries) + } texpr, err := innerExpr.TypeCheck(ctx, semaCtx, typ) if err != nil { return false, expr, err @@ -603,6 +598,13 @@ func (p *planner) replaceViewDesc( toReplace.ViewQuery = updatedQuery } + typeReplacedQuery, err := serializeUserDefinedTypes(ctx, p.SemaCtx(), toReplace.ViewQuery, + false /* multiStmt */, "view queries") + if err != nil { + return nil, err + } + toReplace.ViewQuery = typeReplacedQuery + // Reset the columns to add the new result columns onto. toReplace.Columns = make([]descpb.ColumnDescriptor, 0, len(n.columns)) toReplace.NextColumnID = 0 diff --git a/pkg/sql/logictest/testdata/logic_test/udf b/pkg/sql/logictest/testdata/logic_test/udf index 99f6ce527bba..78a6e6496acd 100644 --- a/pkg/sql/logictest/testdata/logic_test/udf +++ b/pkg/sql/logictest/testdata/logic_test/udf @@ -3575,29 +3575,25 @@ SELECT public."LOWERCASE_HINT_ERROR_EXPLICIT_SCHEMA_FN"(); subtest end -# Regression test for #105259. Do not type-check subqueries in UDFs outside -# optbuilder. Doing so can cause internal errors. +# Regression tests for #105259 and #107654. Do not type-check subqueries in UDFs +# outside optbuilder. Doing so can cause internal errors. subtest regression_105259 statement ok CREATE TYPE e105259 AS ENUM ('foo'); -statement ok +statement error pgcode 0A000 subqueries are not allowed in casts to enums within UDFs CREATE FUNCTION f() RETURNS VOID LANGUAGE SQL AS $$ SELECT (SELECT 'foo')::e105259; SELECT NULL; $$ -query T -SELECT f() ----- -NULL - -statement ok -ALTER TYPE e105259 RENAME VALUE 'foo' TO 'bar' - -# Renaming the enum value corrupts the UDF. This is expected behavior. -statement error pgcode 22P02 invalid input value for enum e105259: "foo" -SELECT f() +statement error pgcode 0A000 subqueries are not allowed in casts to enums within UDFs +CREATE FUNCTION f() RETURNS VOID LANGUAGE SQL AS $$ + SELECT ( + CASE WHEN true THEN (SELECT 'foo') ELSE NULL END + )::e105259; + SELECT NULL; +$$ subtest end diff --git a/pkg/sql/logictest/testdata/logic_test/views b/pkg/sql/logictest/testdata/logic_test/views index 9f8e531fd5db..3ff91b8b9d6c 100644 --- a/pkg/sql/logictest/testdata/logic_test/views +++ b/pkg/sql/logictest/testdata/logic_test/views @@ -1850,27 +1850,21 @@ DROP VIEW cd_v1 CASCADE; subtest end -# Regression test for #105259. Do not type-check subqueries in views outside -# optbuilder. Doing so can cause internal errors. -subtest regression_105259 +# Regression tests for #105259 and #107654. Do not type-check subqueries in +# views outside optbuilder. Doing so can cause internal errors. +subtest regression_105259_107654 statement ok CREATE TYPE e105259 AS ENUM ('foo'); -statement ok -CREATE VIEW v105259 AS +statement error pgcode 0A000 subqueries are not allowed in casts to enums within view queries +CREATE VIEW v AS SELECT (SELECT 'foo')::e105259 -query T -SELECT * FROM v105259 ----- -foo - -statement ok -ALTER TYPE e105259 RENAME VALUE 'foo' TO 'bar' - -# Renaming the enum value corrupts the view. This is expected behavior. -statement error pgcode 22P02 invalid input value for enum e105259: "foo" -SELECT * FROM v105259 +statement error pgcode 0A000 subqueries are not allowed in casts to enums within view queries +CREATE VIEW v AS +SELECT ( + CASE WHEN true THEN (SELECT 'foo') ELSE NULL END +)::e105259 subtest end diff --git a/pkg/sql/opt/optbuilder/scope.go b/pkg/sql/opt/optbuilder/scope.go index f31453995a23..4583e5e18cde 100644 --- a/pkg/sql/opt/optbuilder/scope.go +++ b/pkg/sql/opt/optbuilder/scope.go @@ -1357,15 +1357,10 @@ func (s *scope) replaceWindowFn(f *tree.FuncExpr, def *tree.ResolvedFunctionDefi // We will be performing type checking on expressions from PARTITION BY and // ORDER BY clauses below, and we need the semantic context to know that we // are in a window function. InWindowFunc is updated when type checking - // FuncExpr above, but it is reset upon returning from that, so we need to do - // this update manually. - defer func(ctx *tree.SemaContext, prevWindow bool) { - ctx.Properties.Derived.InWindowFunc = prevWindow - }( - s.builder.semaCtx, - s.builder.semaCtx.Properties.Derived.InWindowFunc, - ) - s.builder.semaCtx.Properties.Derived.InWindowFunc = true + // FuncExpr above, but it is reset upon returning from that, so we need to + // do this update manually. + defer s.builder.semaCtx.Properties.Ancestors.PopTo(s.builder.semaCtx.Properties.Ancestors) + s.builder.semaCtx.Properties.Ancestors.Push(tree.WindowFuncAncestor) oldPartitions := f.WindowDef.Partitions f.WindowDef.Partitions = make(tree.Exprs, len(oldPartitions)) diff --git a/pkg/sql/opt/optbuilder/testdata/aggregate b/pkg/sql/opt/optbuilder/testdata/aggregate index bf27a55a7789..23acc1373251 100644 --- a/pkg/sql/opt/optbuilder/testdata/aggregate +++ b/pkg/sql/opt/optbuilder/testdata/aggregate @@ -2615,7 +2615,7 @@ sort # Grouping columns cannot be reused inside an aggregate input expression # because the aggregate input expressions and grouping expressions are -# built as part of the same projection. +# built as part of the same projection. build SELECT max((k+v)/(k-v)) AS r, (k+v)*(k-v) AS s FROM kv GROUP BY k+v, k-v ---- diff --git a/pkg/sql/schemachanger/scbuild/builder_state.go b/pkg/sql/schemachanger/scbuild/builder_state.go index 5f1ab117a885..2da9d5d8d67b 100644 --- a/pkg/sql/schemachanger/scbuild/builder_state.go +++ b/pkg/sql/schemachanger/scbuild/builder_state.go @@ -1485,20 +1485,6 @@ func (b *builderState) serializeUserDefinedTypes(queryStr string) string { default: return true, expr, nil } - // We cannot type-check subqueries without using optbuilder, and there - // is no need to because we only need to rewrite string values that are - // directly cast to enums. For example, we must rewrite the 'foo' in: - // - // SELECT 'foo'::myenum - // - // We don't need to rewrite the 'foo' in the query below, which can be - // corrupted by renaming the 'foo' value in the myenum type. - // - // SELECT (SELECT 'foo')::myenum - // - if _, ok := innerExpr.(*tree.Subquery); ok { - return true, expr, nil - } var typ *types.T typ, err = tree.ResolveType(b.ctx, typRef, b.semaCtx.TypeResolver) if err != nil { @@ -1507,6 +1493,13 @@ func (b *builderState) serializeUserDefinedTypes(queryStr string) string { if !typ.UserDefined() { return true, expr, nil } + { + // We cannot type-check subqueries without using optbuilder, so we + // currently do not support casting expressions with subqueries to + // UDTs. + defer b.semaCtx.Properties.Restore(b.semaCtx.Properties) + b.semaCtx.Properties.Require("casts to enums within UDFs", tree.RejectSubqueries) + } texpr, err := innerExpr.TypeCheck(b.ctx, b.semaCtx, typ) if err != nil { return false, expr, err diff --git a/pkg/sql/sem/tree/type_check.go b/pkg/sql/sem/tree/type_check.go index 03a10e1c2e1b..aa46e6f5d1aa 100644 --- a/pkg/sql/sem/tree/type_check.go +++ b/pkg/sql/sem/tree/type_check.go @@ -82,17 +82,21 @@ type SemaContext struct { // Restore() method, see below. type SemaProperties struct { // required constraints type checking to only accept certain kinds - // of expressions. See SetConstraint + // of expressions. See Require. required semaRequirements // Derived is populated during semantic analysis with properties // from the expression being analyzed. The caller is responsible // for re-initializing this when needed. Derived ScalarProperties + + // Ancestors is mutated during semantic analysis to provide contextual + // information for each descendent during traversal of sub-expressions. + Ancestors ScalarAncestors } type semaRequirements struct { - // context is the name of the semantic anlysis context, for use in + // context is the name of the semantic analysis context, for use in // error messages. context string @@ -102,11 +106,14 @@ type semaRequirements struct { rejectFlags SemaRejectFlags } -// Require resets the derived properties and sets required constraints. +// Require resets the derived properties and the scalar ancestors, and sets +// required constraints. It must only be called before starting semantic +// analysis and during traversal by semantic analysis itself. func (s *SemaProperties) Require(context string, rejectFlags SemaRejectFlags) { s.required.context = context s.required.rejectFlags = rejectFlags s.Derived.Clear() + s.Ancestors.clear() } // IsSet checks if the given rejectFlag is set as a required property. @@ -180,16 +187,6 @@ type ScalarProperties struct { // SeenGenerator is set to true if the expression originally // contained a SRF. SeenGenerator bool - - // inFuncExpr is temporarily set to true while type checking the - // parameters of a function. Used to process RejectNestedGenerators - // properly. - inFuncExpr bool - - // InWindowFunc is temporarily set to true while type checking the - // parameters of a window function in order to reject nested window - // functions. - InWindowFunc bool } // Clear resets the scalar properties to defaults. @@ -197,6 +194,45 @@ func (sp *ScalarProperties) Clear() { *sp = ScalarProperties{} } +// ScalarAncestors provides context for the current scalar expression during +// semantic analysis. Ancestors are temporarily modified by expressions so that +// their descendent expressions can be analyzed with respect to their ancestors. +type ScalarAncestors byte + +const ( + // FuncExprAncestor is temporarily added to ScalarAncestors while type + // checking the parameters of a function. Used to process + // RejectNestedGenerators properly. + FuncExprAncestor ScalarAncestors = 1 << iota + + // WindowFuncAncestor is temporarily added to ScalarAncestors while type + // checking the parameters of a window function in order to reject nested + // window functions. + WindowFuncAncestor +) + +// Push adds the given ancestor to s. +func (s *ScalarAncestors) Push(other ScalarAncestors) { + *s = *s | other +} + +// Has returns true if s has the given ancestor. +func (s ScalarAncestors) Has(other ScalarAncestors) bool { + return s&other != 0 +} + +// PopTo returns s to the given set of ancestors. Use with: +// +// defer semaCtx.Properties.Ancestors.PopTo(semaCtx.Properties.Ancestors) +func (s *ScalarAncestors) PopTo(orig ScalarAncestors) { + *s = orig +} + +// clear resets s to the default set of ancestors. +func (s *ScalarAncestors) clear() { + *s = 0 +} + // MakeSemaContext initializes a simple SemaContext suitable // for "lightweight" type checking such as the one performed for default // expressions. @@ -946,7 +982,7 @@ func (sc *SemaContext) checkFunctionUsage(expr *FuncExpr, def *ResolvedFunctionD return NewInvalidFunctionUsageError(WindowClass, sc.Properties.required.context) } - if sc.Properties.Derived.InWindowFunc && + if sc.Properties.Ancestors.Has(WindowFuncAncestor) && sc.Properties.required.rejectFlags&RejectNestedWindowFunctions != 0 { return pgerror.Newf(pgcode.Windowing, "window function calls cannot be nested") } @@ -955,7 +991,7 @@ func (sc *SemaContext) checkFunctionUsage(expr *FuncExpr, def *ResolvedFunctionD // If it is an aggregate function *not used OVER a window*, then // we have an aggregation. if fnCls == AggregateClass { - if sc.Properties.Derived.inFuncExpr && + if sc.Properties.Ancestors.Has(FuncExprAncestor) && sc.Properties.required.rejectFlags&RejectNestedAggregates != 0 { return NewAggInAggError() } @@ -966,7 +1002,7 @@ func (sc *SemaContext) checkFunctionUsage(expr *FuncExpr, def *ResolvedFunctionD } } if fnCls == GeneratorClass { - if sc.Properties.Derived.inFuncExpr && + if sc.Properties.Ancestors.Has(FuncExprAncestor) && sc.Properties.required.rejectFlags&RejectNestedGenerators != 0 { return NewInvalidNestedSRFError(sc.Properties.required.context) } @@ -1054,23 +1090,15 @@ func (expr *FuncExpr) TypeCheck( } if semaCtx != nil { - // We'll need to remember we are in a function application to - // generate suitable errors in checkFunctionUsage(). We cannot - // set ctx.inFuncExpr earlier (in particular not before the call - // to checkFunctionUsage() above) because the top-level FuncExpr - // must be acceptable even if it is a SRF and - // RejectNestedGenerators is set. - defer func(semaCtx *SemaContext, prevFunc bool, prevWindow bool) { - semaCtx.Properties.Derived.inFuncExpr = prevFunc - semaCtx.Properties.Derived.InWindowFunc = prevWindow - }( - semaCtx, - semaCtx.Properties.Derived.inFuncExpr, - semaCtx.Properties.Derived.InWindowFunc, - ) - semaCtx.Properties.Derived.inFuncExpr = true + // We'll need to remember we are in a function application to generate + // suitable errors in checkFunctionUsage(). We cannot enter + // FuncExprAncestor earlier (in particular not before the call to + // checkFunctionUsage() above) because the top-level FuncExpr must be + // acceptable even if it is a SRF and RejectNestedGenerators is set. + defer semaCtx.Properties.Ancestors.PopTo(semaCtx.Properties.Ancestors) + semaCtx.Properties.Ancestors.Push(FuncExprAncestor) if expr.WindowDef != nil { - semaCtx.Properties.Derived.InWindowFunc = true + semaCtx.Properties.Ancestors.Push(WindowFuncAncestor) } }