Skip to content

Commit

Permalink
Merge pull request #109237 from mgartner/backport23.1-108188
Browse files Browse the repository at this point in the history
release-23.1: sql: refactor semantic analysis and fix some bugs
  • Loading branch information
mgartner committed Aug 23, 2023
2 parents c6a5ddc + 3062dc8 commit da4f7d6
Show file tree
Hide file tree
Showing 8 changed files with 113 additions and 103 deletions.
4 changes: 3 additions & 1 deletion pkg/sql/create_function.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down
34 changes: 18 additions & 16 deletions pkg/sql/create_view.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down Expand Up @@ -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
Expand All @@ -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
Expand All @@ -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
Expand Down Expand Up @@ -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
Expand Down
24 changes: 10 additions & 14 deletions pkg/sql/logictest/testdata/logic_test/udf
Original file line number Diff line number Diff line change
Expand Up @@ -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
26 changes: 10 additions & 16 deletions pkg/sql/logictest/testdata/logic_test/views
Original file line number Diff line number Diff line change
Expand Up @@ -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
13 changes: 4 additions & 9 deletions pkg/sql/opt/optbuilder/scope.go
Original file line number Diff line number Diff line change
Expand Up @@ -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))
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/optbuilder/testdata/aggregate
Original file line number Diff line number Diff line change
Expand Up @@ -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
----
Expand Down
21 changes: 7 additions & 14 deletions pkg/sql/schemachanger/scbuild/builder_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand All @@ -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
Expand Down
92 changes: 60 additions & 32 deletions pkg/sql/sem/tree/type_check.go
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand All @@ -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.
Expand Down Expand Up @@ -180,23 +187,52 @@ 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.
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.
Expand Down Expand Up @@ -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")
}
Expand All @@ -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()
}
Expand All @@ -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)
}
Expand Down Expand Up @@ -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)
}
}

Expand Down

0 comments on commit da4f7d6

Please sign in to comment.