Skip to content

Commit

Permalink
Merge pull request #112789 from cockroachdb/blathers/backport-release…
Browse files Browse the repository at this point in the history
…-23.2-112690

release-23.2: sql: disallow invocation of procedures outside of CALL
  • Loading branch information
mgartner authored Oct 20, 2023
2 parents 4538f11 + f559fec commit c309d64
Show file tree
Hide file tree
Showing 6 changed files with 115 additions and 7 deletions.
73 changes: 73 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/procedure
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,39 @@ SELECT currval('s')
statement error pgcode 42809 p\(\) is a procedure\nHINT: To call a procedure, use CALL.
SELECT p()

statement error pgcode 42809 p\(\) is a procedure\nHINT: To call a procedure, use CALL.
CALL p(p())

statement error pgcode 42809 p\(\) is a procedure\nHINT: To call a procedure, use CALL.
SELECT * FROM (VALUES (1), (2), (3)) LIMIT p()

statement error pgcode 42809 p\(\) is a procedure\nHINT: To call a procedure, use CALL.
SELECT * FROM (VALUES (1), (2), (3)) ORDER BY p()

statement error pgcode 42809 p\(\) is a procedure\nHINT: To call a procedure, use CALL.
SELECT * FROM (VALUES (1), (2), (3)) v(i) WHERE i = p()

statement error pgcode 42809 p\(\) is a procedure\nHINT: To call a procedure, use CALL.
SELECT * FROM (VALUES (1), (2), (3)) v(i) GROUP BY i HAVING i > p()

statement error pgcode 42809 p\(\) is a procedure\nHINT: To call a procedure, use CALL.
SELECT * FROM (VALUES (1), (2)) v(i) JOIN (VALUES (2), (3)) w(j) ON i = p()

statement error pgcode 42809 p\(\) is a procedure\nHINT: To call a procedure, use CALL.
SELECT * FROM generate_series(1, p())

statement error pgcode 42809 p\(\) is a procedure\nHINT: To call a procedure, use CALL.
SELECT * FROM generate_series(1, p())

statement error pgcode 42809 p\(\) is a procedure\nHINT: To call a procedure, use CALL.
SELECT abs(p())

statement error pgcode 42809 p\(\) is a procedure\nHINT: To call a procedure, use CALL.
SELECT nth_value(1, p()) OVER () FROM (VALUES (1), (2))

statement error pgcode 42809 p\(\) is a procedure\nHINT: To call a procedure, use CALL.
SELECT nth_value(1, i) OVER (ORDER BY p()) FROM (VALUES (1), (2)) v(i)

statement ok
CREATE OR REPLACE PROCEDURE p() LANGUAGE SQL AS ''

Expand Down Expand Up @@ -85,6 +118,27 @@ CREATE OR REPLACE PROCEDURE p2() LANGUAGE SQL AS $$
CALL p();
$$

statement error pgcode 42883 unknown function: p\(\)
CREATE FUNCTION err(i INT) RETURNS VOID LANGUAGE SQL AS 'SELECT p()'

statement error pgcode 0A000 unimplemented: CALL usage inside a function definition
CREATE FUNCTION err(i INT) RETURNS VOID LANGUAGE SQL AS 'CALL p()'

statement error pgcode 42809 p\(\) is a procedure\nHINT: To call a procedure, use CALL.
CREATE TABLE err (i INT DEFAULT p())

statement error pgcode 42809 p\(\) is a procedure\nHINT: To call a procedure, use CALL.
CREATE TABLE err (i INT AS (p()) STORED)

statement error pgcode 42809 p\(\) is a procedure\nHINT: To call a procedure, use CALL.
CREATE TABLE err (i INT, INDEX (p()))

statement error pgcode 42809 p\(\) is a procedure\nHINT: To call a procedure, use CALL.
CREATE TABLE err (a INT, b INT, INDEX (a, (b + p())))

statement error pgcode 42809 p\(\) is a procedure\nHINT: To call a procedure, use CALL.
CREATE TABLE err (a INT, INDEX (a) WHERE p() = 1)

statement ok
CREATE TABLE t (
k INT PRIMARY KEY,
Expand All @@ -108,6 +162,25 @@ SELECT * FROM t
----
1 11

statement ok
CREATE FUNCTION one() RETURNS INT LANGUAGE SQL AS 'SELECT 1'

statement ok
CALL t_update(one(), 12)

query II
SELECT * FROM t
----
1 12

statement ok
CALL t_update(one(), one()+12)

query II
SELECT * FROM t
----
1 13

statement ok
CREATE FUNCTION t_update() RETURNS INT LANGUAGE SQL AS 'SELECT 1'

Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/opt/optbuilder/groupby.go
Original file line number Diff line number Diff line change
Expand Up @@ -459,7 +459,8 @@ func (b *Builder) analyzeHaving(having *tree.Where, fromScope *scope) tree.Typed
// in case we are recursively called within a subquery context.
defer b.semaCtx.Properties.Restore(b.semaCtx.Properties)
b.semaCtx.Properties.Require(
exprKindHaving.String(), tree.RejectWindowApplications|tree.RejectGenerators,
exprKindHaving.String(),
tree.RejectWindowApplications|tree.RejectGenerators|tree.RejectProcedures,
)
fromScope.context = exprKindHaving
return fromScope.resolveAndRequireType(having.Expr, types.Bool)
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/opt/optbuilder/join.go
Original file line number Diff line number Diff line change
Expand Up @@ -117,7 +117,8 @@ func (b *Builder) buildJoin(
if on, ok := cond.(*tree.OnJoinCond); ok {
// Do not allow special functions in the ON clause.
b.semaCtx.Properties.Require(
exprKindOn.String(), tree.RejectGenerators|tree.RejectWindowApplications,
exprKindOn.String(),
tree.RejectGenerators|tree.RejectWindowApplications|tree.RejectProcedures,
)
outScope.context = exprKindOn
filter := b.buildScalar(
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/optbuilder/select.go
Original file line number Diff line number Diff line change
Expand Up @@ -1332,7 +1332,7 @@ func (b *Builder) buildWhere(where *tree.Where, inScope *scope) {
where.Expr,
types.Bool,
exprKindWhere,
tree.RejectGenerators|tree.RejectWindowApplications,
tree.RejectGenerators|tree.RejectWindowApplications|tree.RejectProcedures,
inScope,
)

Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/opt/optbuilder/srfs.go
Original file line number Diff line number Diff line change
Expand Up @@ -89,7 +89,8 @@ func (b *Builder) buildZip(exprs tree.Exprs, inScope *scope) (outScope *scope) {
// context.
defer b.semaCtx.Properties.Restore(b.semaCtx.Properties)
b.semaCtx.Properties.Require(exprKindFrom.String(),
tree.RejectAggregates|tree.RejectWindowApplications|tree.RejectNestedGenerators)
tree.RejectAggregates|tree.RejectWindowApplications|
tree.RejectNestedGenerators|tree.RejectProcedures)
inScope.context = exprKindFrom

// Build each of the provided expressions.
Expand Down
38 changes: 35 additions & 3 deletions pkg/sql/sem/tree/type_check.go
Original file line number Diff line number Diff line change
Expand Up @@ -120,6 +120,11 @@ func (s *SemaProperties) Require(context string, rejectFlags SemaRejectFlags) {
s.Ancestors.clear()
}

// Reject adds the given flags to the set of required constraints of s.
func (s *SemaProperties) Reject(rejectFlags SemaRejectFlags) {
s.required.rejectFlags |= rejectFlags
}

// IsSet checks if the given rejectFlag is set as a required property.
func (s *SemaProperties) IsSet(rejectFlags SemaRejectFlags) bool {
return s.required.rejectFlags&rejectFlags != 0
Expand Down Expand Up @@ -171,8 +176,14 @@ const (
// RejectSubqueries rejects subqueries in scalar contexts.
RejectSubqueries

// RejectProcedures rejects procedures in scalar contexts.
RejectProcedures

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

// ScalarProperties contains the properties of the current scalar
Expand Down Expand Up @@ -1156,8 +1167,19 @@ func (expr *FuncExpr) TypeCheck(
(*qualifiedOverloads)(&def.Overloads), expr.Exprs...,
)
defer s.release()
if err := s.typeCheckOverloadedExprs(ctx, semaCtx, desired, false); err != nil {
return nil, pgerror.Wrapf(err, pgcode.InvalidParameterValue, "%s()", def.Name)

if err := func() error {
// Disallow procedures in function arguments.
if semaCtx != nil {
defer semaCtx.Properties.Restore(semaCtx.Properties)
semaCtx.Properties.Reject(RejectProcedures)
}
if err := s.typeCheckOverloadedExprs(ctx, semaCtx, desired, false); err != nil {
return pgerror.Wrapf(err, pgcode.InvalidParameterValue, "%s()", def.Name)
}
return nil
}(); err != nil {
return nil, err
}

var hasUDFOverload bool
Expand Down Expand Up @@ -1267,6 +1289,16 @@ func (expr *FuncExpr) TypeCheck(
}
}

if overloadImpl.Type == ProcedureRoutine && semaCtx.Properties.IsSet(RejectProcedures) {
return nil, errors.WithHint(
pgerror.Newf(
pgcode.WrongObjectType,
"%s(%s) is a procedure", def.Name, overloadImpl.Types.String(),
),
"To call a procedure, use CALL.",
)
}

if expr.IsWindowFunctionApplication() {
// Make sure the window function application is of either a built-in window
// function or of a builtin aggregate function.
Expand Down

0 comments on commit c309d64

Please sign in to comment.