Skip to content

Commit

Permalink
Merge #108188
Browse files Browse the repository at this point in the history
108188: sql: refactor semantic analysis and fix some bugs r=mgartner a=mgartner

#### sql/sem/tree: simplify SemaCtx reject flag checks

Release note: None

#### sql/sem/tree: split derived SemaContext properties from contextual info

Properties derived about expressions during semantic analysis are
communicated to callers via ScalarProperties. Prior to this commit, this
type was also used to provide contextual information while traversing
sub-expressions during semantic analysis. For example, it would indicate
whether the current expression is a descendent of a window function
expression.

These two types of information, derived and contextual, are
fundamentally different. Derived properties bubble up from the bottom of
the tree to the top, while context propagates downward into
sub-expressions. This difference made it difficult to maintaining them
correctly in a single type and difficult to reason about. This commit
introduces the ScalarScene type which is used for providing internal
contextual information during semantic analysis.

Release note: None

#### sql/sem/tree: do not Restore SemaRejectFlags during semantic analysis

This commit fixes a bug introduced in #105582 that caused
SemaRejectFlags to be restored during semantic analysis, preventing the
analysis from detecting some forms of invalid expressions.

Fixes #108166

There is no release note because the related bug does not exist in any
releases.

Release note: None

#### sql: do not allow subqueries to be cast to enums in views and UDFs

This commit is a follow-up to #106868 after additional reproductions of
the original bug were found. For now, we disallow any CAST expressions
that contain a subquery in the input and the target type is an ENUM.
I've created #108184 to track this limitation.

Fixes #107654

There is no release note because the release note from #106868 should be
sufficient.

Release note: None

#### sql/randgen: fix typo in comment

Release note: None


Co-authored-by: Marcus Gartner <marcus@cockroachlabs.com>
  • Loading branch information
craig[bot] and mgartner committed Aug 7, 2023
2 parents cdf6d15 + 25c344c commit d6f6de2
Show file tree
Hide file tree
Showing 14 changed files with 165 additions and 138 deletions.
2 changes: 1 addition & 1 deletion pkg/sql/create_function.go
Expand Up @@ -443,7 +443,7 @@ func setFuncOptions(
return err
}
typeReplacedFuncBody, err := serializeUserDefinedTypes(
params.ctx, params.p.SemaCtx(), seqReplacedFuncBody, true, /* multiStmt */
params.ctx, params.p.SemaCtx(), seqReplacedFuncBody, true /* multiStmt */, "UDFs",
)
if err != nil {
return err
Expand Down
30 changes: 13 additions & 17 deletions pkg/sql/create_view.go
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,7 +598,8 @@ func (p *planner) replaceViewDesc(
toReplace.ViewQuery = updatedQuery
}

typeReplacedQuery, err := serializeUserDefinedTypes(ctx, p.SemaCtx(), toReplace.ViewQuery, false /* multiStmt */)
typeReplacedQuery, err := serializeUserDefinedTypes(ctx, p.SemaCtx(), toReplace.ViewQuery,
false /* multiStmt */, "view queries")
if err != nil {
return nil, err
}
Expand Down
12 changes: 12 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/delete
Expand Up @@ -576,3 +576,15 @@ statement error pgcode 42803 sum\(\): aggregate functions are not allowed in ORD
DELETE FROM t107634 ORDER BY sum(a) LIMIT 1;

subtest end

# Regression test for #108166. Do not allow aggregate functions in ORDER BY when
# the function is wrapped by a conditional expression.
subtest regression_108166

statement ok
CREATE TABLE t108166 (a INT)

statement error pgcode 42803 sum\(\): aggregate functions are not allowed in ORDER BY in DELETE
DELETE FROM t108166 ORDER BY COALESCE(sum(a), 1) LIMIT 1;

subtest end
12 changes: 6 additions & 6 deletions pkg/sql/logictest/testdata/logic_test/srfs
Expand Up @@ -1322,16 +1322,16 @@ subtest generator-syntax

# Regression test for #97119 and #94890 - return syntax error when CASE or
# COALESCE is used with a set-generating function as argument.
statement error pq: set-returning functions are not allowed in CASE
statement error pq: set-returning functions are not allowed in conditional expressions
SELECT CASE generate_series(1, 3) WHEN 3 THEN 0 ELSE 1 END;

statement error pq: set-returning functions are not allowed in CASE
statement error pq: set-returning functions are not allowed in conditional expressions
SELECT CASE WHEN true THEN generate_series(1, 3) ELSE 1 END;

statement error pq: set-returning functions are not allowed in CASE
statement error pq: set-returning functions are not allowed in conditional expressions
SELECT CASE WHEN false THEN 1 ELSE generate_series(1, 3) END;

statement error pq: set-returning functions are not allowed in COALESCE
statement error pq: set-returning functions are not allowed in conditional expressions
SELECT COALESCE(generate_series(1, 10));

# A subquery with a generator function is allowed within CASE and COALESCE.
Expand Down Expand Up @@ -1376,12 +1376,12 @@ SELECT COALESCE(sum(x) OVER ()) FROM xy;
15

# IF does not allow generator functions.
statement error pq: set-returning functions are not allowed in IF
statement error pq: set-returning functions are not allowed in conditional expressions
SELECT IF(x > y, generate_series(1, 3), 0) FROM xy;

# IFNULL does not allow generator functions. Note that the error mentions
# COALESCE because IFNULL is parsed directly as a COALESCE expression.
statement error pq: set-returning functions are not allowed in COALESCE
statement error pq: set-returning functions are not allowed in conditional expressions
SELECT IFNULL(1, generate_series(1, 2));

# NULLIF allows generator functions.
Expand Down
24 changes: 10 additions & 14 deletions pkg/sql/logictest/testdata/logic_test/udf_regressions
Expand Up @@ -554,29 +554,25 @@ $$
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
1 change: 0 additions & 1 deletion pkg/sql/logictest/testdata/logic_test/udf_unsupported
Expand Up @@ -183,4 +183,3 @@ statement error pgcode 0A000 unimplemented: cannot create UDFs under a temporary
CREATE FUNCTION $temp_schema_102964.f_102964 () RETURNS INT AS 'SELECT 1' LANGUAGE sql;

subtest end

12 changes: 12 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/update
Expand Up @@ -672,3 +672,15 @@ statement error pgcode 42803 sum\(\): aggregate functions are not allowed in ORD
UPDATE t107634 SET a = 1 ORDER BY sum(a) LIMIT 1;

subtest end

# Regression test for #108166. Do not allow aggregate functions in ORDER BY when
# the function is wrapped by a conditional expression.
subtest regression_108166

statement ok
CREATE TABLE t108166 (a INT)

statement error pgcode 42803 sum\(\): aggregate functions are not allowed in ORDER BY in UPDATE
UPDATE t108166 SET a = 1 ORDER BY COALESCE(sum(a), 1) LIMIT 1;

subtest end
26 changes: 10 additions & 16 deletions pkg/sql/logictest/testdata/logic_test/views
Expand Up @@ -1900,28 +1900,22 @@ SELECT * FROM v104927

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

Expand Down
13 changes: 4 additions & 9 deletions pkg/sql/opt/optbuilder/scope.go
Expand Up @@ -1358,15 +1358,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
3 changes: 3 additions & 0 deletions pkg/sql/opt/optbuilder/srfs.go
Expand Up @@ -50,6 +50,9 @@ func (s *srf) TypeCheck(
// invalid usage here.
return nil, tree.NewInvalidFunctionUsageError(tree.GeneratorClass, ctx.TypeCheckContext())
}
if ctx.Properties.Ancestors.Has(tree.ConditionalAncestor) {
return nil, tree.NewInvalidFunctionUsageError(tree.GeneratorClass, "conditional expressions")
}
if ctx.Properties.Derived.SeenGenerator {
// This error happens if this srf struct is nested inside a raw srf that
// has not yet been replaced. This is possible since scope.replaceSRF first
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/optbuilder/testdata/aggregate
Expand Up @@ -2616,7 +2616,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
4 changes: 2 additions & 2 deletions pkg/sql/randgen/expr.go
Expand Up @@ -161,8 +161,8 @@ func randExpr(
}
}
if len(cols) > 1 {
// If any of the columns are nullable, set the computed column to be
// nullable.
// If any of the columns are nullable, the resulting expression
// could be null.
for _, x := range cols {
if x.Nullable.Nullability != tree.NotNull {
nullability = x.Nullable.Nullability
Expand Down
21 changes: 7 additions & 14 deletions pkg/sql/schemachanger/scbuild/builder_state.go
Expand Up @@ -1591,20 +1591,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 @@ -1613,6 +1599,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

0 comments on commit d6f6de2

Please sign in to comment.