Skip to content

Commit

Permalink
sql: properly reject nested generators in ROWS FROM
Browse files Browse the repository at this point in the history
Prior to this patch, invalid uses of SRFs as arguments to other
functions in ROWS FROM were not properly rejected, and were only
caught at evaluation time (i.e. much too late).

This patch fixes it by rejecting these uses early.

Release note (bug fix): invalid uses of set-generating functions in
FROM clauses are now reported with the same error code as PostgreSQL.
  • Loading branch information
knz committed Jul 11, 2018
1 parent 7a263ea commit e870a82
Show file tree
Hide file tree
Showing 5 changed files with 57 additions and 6 deletions.
4 changes: 2 additions & 2 deletions pkg/ccl/logictestccl/testdata/logic_test/partitioning
Original file line number Diff line number Diff line change
Expand Up @@ -360,12 +360,12 @@ CREATE TABLE t (a INT, b INT, c INT, PRIMARY KEY (a, b)) PARTITION BY LIST (a) (
PARTITION p1 VALUES IN ((SELECT 1))
)

statement error PARTITION p1: impure functions are not allowed in partition
statement error PARTITION p1: now\(\): impure functions are not allowed in partition
CREATE TABLE t (a TIMESTAMP PRIMARY KEY) PARTITION BY LIST (a) (
PARTITION p1 VALUES IN (now())
)

statement error PARTITION p1: impure functions are not allowed in partition
statement error PARTITION p1: uuid_v4\(\): impure functions are not allowed in partition
CREATE TABLE t (a TIMESTAMP PRIMARY KEY) PARTITION BY LIST (a) (
PARTITION p1 VALUES IN (uuid_v4() || 'foo')
)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/logictest/testdata/logic_test/create_as
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ CREATE TABLE foo (x, y, z) AS SELECT catalog_name, schema_name, sql_path FROM in
statement error pq: unexpected type coltypes.TTuple
CREATE TABLE foo2 (x) AS (VALUES(ROW()))

statement error pq: generator functions are not allowed in VALUES
statement error generator functions are not allowed in VALUES
CREATE TABLE foo2 (x) AS (VALUES(generate_series(1,3)))

statement error pq: value type unknown cannot be used for table columns
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/rows_from
Original file line number Diff line number Diff line change
Expand Up @@ -67,3 +67,8 @@ x n generate_series
NULL NULL 13
NULL NULL 14
NULL NULL 15

# Regression test for #27389.

statement error pg_get_keywords\(\): set-returning functions must appear at the top level of FROM
SELECT * FROM ROWS FROM(generate_series(length((pg_get_keywords()).word),10));
3 changes: 2 additions & 1 deletion pkg/sql/project_set.go
Original file line number Diff line number Diff line change
Expand Up @@ -115,7 +115,8 @@ func (p *planner) ProjectSet(
defer p.semaCtx.Properties.Restore(p.semaCtx.Properties)

// Ensure there are no aggregate or window functions in the clause.
p.semaCtx.Properties.Require("FROM", tree.RejectAggregates|tree.RejectWindowApplications)
p.semaCtx.Properties.Require("FROM",
tree.RejectAggregates|tree.RejectWindowApplications|tree.RejectNestedGenerators)

// Analyze the provided expressions.
n.ivarHelper = tree.MakeIndexedVarHelper(n, len(srcCols))
Expand Down
49 changes: 47 additions & 2 deletions pkg/sql/sem/tree/type_check.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,12 +104,30 @@ type SemaRejectFlags int

// Valid values for SemaRejectFlags.
const (
AllowAll SemaRejectFlags = 0
AllowAll SemaRejectFlags = 0

// RejectAggregates rejects min(), max(), etc.
RejectAggregates SemaRejectFlags = 1 << iota

// RejectWindowApplications rejects "x() over y", etc.
RejectWindowApplications

// RejectGenerators rejects any use of SRFs, e.g "generate_series()".
RejectGenerators

// RejectNestedGenerators rejects any use of SRFs inside the
// argument list of another function call, which can itself be a SRF
// (RejectGenerators notwithstanding).
// This is used e.g. when processing the calls inside ROWS FROM.
RejectNestedGenerators

// RejectImpureFunctions rejects any non-const functions like now().
RejectImpureFunctions

// RejectSubqueries rejects subqueries in scalar contexts.
RejectSubqueries

// RejectSpecial is used in common places like the LIMIT clause.
RejectSpecial SemaRejectFlags = RejectAggregates | RejectGenerators | RejectWindowApplications
)

Expand All @@ -133,6 +151,11 @@ type ScalarProperties struct {
// SeenImpureFunctions is set to true if the expression originally
// contained an impure function.
SeenImpure bool

// inFuncExpr is temporarily set to true while type checking the
// parameters of a function. Used to process RejectNestedGenerators
// properly.
inFuncExpr bool
}

// Clear resets the scalar properties to defaults.
Expand Down Expand Up @@ -609,6 +632,12 @@ var (
errInsufficientPriv = pgerror.NewError(pgerror.CodeInsufficientPrivilegeError, "insufficient privilege")
)

// NewInvalidNestedSRFError creates a rejection for a nested SRF.
func NewInvalidNestedSRFError(context string) error {
return pgerror.NewErrorf(pgerror.CodeFeatureNotSupportedError,
"set-returning functions must appear at the top level of %s", context)
}

// NewInvalidFunctionUsageError creates a rejection for a special function.
func NewInvalidFunctionUsageError(class FunctionClass, context string) error {
var cat string
Expand Down Expand Up @@ -660,6 +689,10 @@ func (sc *SemaContext) checkFunctionUsage(expr *FuncExpr, def *FunctionDefinitio
}
}
if def.Class == GeneratorClass {
if sc.Properties.Derived.inFuncExpr &&
sc.Properties.required.rejectFlags&RejectNestedGenerators != 0 {
return NewInvalidNestedSRFError(sc.Properties.required.context)
}
if sc.Properties.required.rejectFlags&RejectGenerators != 0 {
return NewInvalidFunctionUsageError(GeneratorClass, sc.Properties.required.context)
}
Expand Down Expand Up @@ -690,7 +723,19 @@ func (expr *FuncExpr) TypeCheck(ctx *SemaContext, desired types.T) (TypedExpr, e
}

if err := ctx.checkFunctionUsage(expr, def); err != nil {
return nil, err
return nil, errors.Wrapf(err, "%s()", def.Name)
}
if ctx != 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(ctx *SemaContext, prev bool) {
ctx.Properties.Derived.inFuncExpr = prev
}(ctx, ctx.Properties.Derived.inFuncExpr)
ctx.Properties.Derived.inFuncExpr = true
}

typedSubExprs, fns, err := typeCheckOverloadedExprs(ctx, desired, def.Definition, false, expr.Exprs...)
Expand Down

0 comments on commit e870a82

Please sign in to comment.