Skip to content

Commit

Permalink
Merge #86005 #86424
Browse files Browse the repository at this point in the history
86005: sql: allow anonymous args and numeric arg references in UDFs r=mgartner a=mgartner

It is now possible to create UDFs with anonymous arguments that can be
referenced like a placeholder in a prepared statement. For example:

    CREATE FUNCTION add(INT, INT) RETURNS INT LANGUAGE SQL AS $$
      SELECT $1 + $2;
    $$

Named arguments can also be referenced by numeric placeholders. For
example:

    CREATE FUNCTION add(x INT, y INT) RETURNS INT LANGUAGE SQL AS $$
      SELECT x + $2;
    $$

Release note: None

Release justification: Adds critical functionality to a new feature.


86424: colflow: handle error during draining correctly r=yuzefovich a=yuzefovich

This commit fixes the way we handle internal errors that occur during
the draining in the `FlowCoordinator`. Previously, with all such errors
we would call `MoveToDraining`, but if the processor is already
draining, then it would panic. This could occur, for example, when
accounting for the metadata footprint and exceeding the limit (which was
added several weeks ago). This is now fixed.

Found when looking at #86372.

Release justification: bug fix.

Release note: None

Co-authored-by: Marcus Gartner <[email protected]>
Co-authored-by: Yahor Yuzefovich <[email protected]>
  • Loading branch information
3 people committed Aug 19, 2022
3 parents 9a1c3a9 + 6e191d5 + 4ec6d67 commit 513fae1
Show file tree
Hide file tree
Showing 9 changed files with 526 additions and 29 deletions.
12 changes: 11 additions & 1 deletion pkg/sql/colflow/flow_coordinator.go
Original file line number Diff line number Diff line change
Expand Up @@ -150,7 +150,17 @@ func (f *FlowCoordinator) nextAdapter() {
// Next is part of the execinfra.RowSource interface.
func (f *FlowCoordinator) Next() (rowenc.EncDatumRow, *execinfrapb.ProducerMetadata) {
if err := colexecerror.CatchVectorizedRuntimeError(f.nextAdapter); err != nil {
f.MoveToDraining(err)
if f.State == execinfra.StateRunning {
f.MoveToDraining(err)
} else {
// We have encountered an error during draining, so we will just
// return the error as metadata directly. This could occur, for
// example, when accounting for the metadata footprint and exceeding
// the limit.
meta := execinfrapb.GetProducerMeta()
meta.Err = err
return nil, meta
}
return nil, f.DrainHelper()
}
return f.row, f.meta
Expand Down
14 changes: 8 additions & 6 deletions pkg/sql/create_function.go
Original file line number Diff line number Diff line change
Expand Up @@ -205,13 +205,15 @@ func (n *createFunctionNode) getMutableFuncDesc(
pbArgs := make([]descpb.FunctionDescriptor_Argument, len(n.cf.Args))
argNameSeen := make(map[tree.Name]struct{})
for i, arg := range n.cf.Args {
if _, ok := argNameSeen[arg.Name]; ok {
// Argument names cannot be used more than once.
return nil, false, pgerror.Newf(
pgcode.InvalidFunctionDefinition, "parameter name %q used more than once", arg.Name,
)
if arg.Name != "" {
if _, ok := argNameSeen[arg.Name]; ok {
// Argument names cannot be used more than once.
return nil, false, pgerror.Newf(
pgcode.InvalidFunctionDefinition, "parameter name %q used more than once", arg.Name,
)
}
argNameSeen[arg.Name] = struct{}{}
}
argNameSeen[arg.Name] = struct{}{}
pbArg, err := makeFunctionArg(params.ctx, arg, params.p)
if err != nil {
return nil, false, err
Expand Down
123 changes: 123 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/udf
Original file line number Diff line number Diff line change
Expand Up @@ -1446,6 +1446,129 @@ SELECT fetch_b(99999999)
----
NULL


subtest args

# TODO(mgartner): Technically $3 is a parameter, and the error message should be
# more similar to Postgres's "there is no parameter $3".
statement error no value provided for placeholder: \$3
CREATE FUNCTION err(x INT, y INT) RETURNS INT LANGUAGE SQL AS 'SELECT x + y + $1 + $2 + $3'

statement error no value provided for placeholder: \$3
CREATE FUNCTION err(INT, INT) RETURNS INT LANGUAGE SQL AS 'SELECT $1 + $2 + $3'

statement error placeholder index must be between 1 and 65536
CREATE FUNCTION err(INT) RETURNS INT LANGUAGE SQL AS 'SELECT 1 + $0'

statement ok
CREATE FUNCTION add(x INT, y INT, z INT) RETURNS INT LANGUAGE SQL AS $$
SELECT (x - $1 + x) + ($2 - y + $2) + (z - $3 + z);
$$

statement ok
CREATE FUNCTION mult(x INT, y INT, z INT) RETURNS INT LANGUAGE SQL AS $$
SELECT $1 * y * (z - $3 + z);
$$

query II rowsort
SELECT a + b + a, add(a, b, a) FROM ab
----
3 3
6 6
9 9
9 9
11 11

query I
SELECT a FROM ab WHERE (a + b + b) != add(a, b, b)
----

query II rowsort
SELECT
(a + b + a) * (a + 3 + 7) * (b + 11 + 17),
mult(add(a, b, a), add(a, 3, 7), add(b, 11, 17))
FROM ab
----
957 957
2160 2160
3627 3627
3654 3654
4785 4785

statement ok
PREPARE do_math(INT, INT, INT, INT) AS
SELECT
(a + b + a) * (a + $1 + $2) * (b + $3 + $4),
mult(add(a, b, a), add(a, $1, $2), add(b, $3, $4))
FROM ab

query II rowsort
EXECUTE do_math(3, 7, 11, 17)
----
957 957
2160 2160
3627 3627
3654 3654
4785 4785

statement error pgcode 54023 functions cannot have more than 100 arguments
CREATE FUNCTION err(
INT, INT, INT, INT, INT, INT, INT, INT, INT, INT,
INT, INT, INT, INT, INT, INT, INT, INT, INT, INT,
INT, INT, INT, INT, INT, INT, INT, INT, INT, INT,
INT, INT, INT, INT, INT, INT, INT, INT, INT, INT,
INT, INT, INT, INT, INT, INT, INT, INT, INT, INT,
INT, INT, INT, INT, INT, INT, INT, INT, INT, INT,
INT, INT, INT, INT, INT, INT, INT, INT, INT, INT,
INT, INT, INT, INT, INT, INT, INT, INT, INT, INT,
INT, INT, INT, INT, INT, INT, INT, INT, INT, INT,
INT, INT, INT, INT, INT, INT, INT, INT, INT, INT,
INT
) RETURNS INT LANGUAGE SQL AS 'SELECT $1';

# Up to 100 arguments are allowed.
statement ok
CREATE FUNCTION add(
INT, INT, INT, INT, INT, INT, INT, INT, INT, INT,
INT, INT, INT, INT, INT, INT, INT, INT, INT, INT,
INT, INT, INT, INT, INT, INT, INT, INT, INT, INT,
INT, INT, INT, INT, INT, INT, INT, INT, INT, INT,
INT, INT, INT, INT, INT, INT, INT, INT, INT, INT,
INT, INT, INT, INT, INT, INT, INT, INT, INT, INT,
INT, INT, INT, INT, INT, INT, INT, INT, INT, INT,
INT, INT, INT, INT, INT, INT, INT, INT, INT, INT,
INT, INT, INT, INT, INT, INT, INT, INT, INT, INT,
INT, INT, INT, INT, INT, INT, INT, INT, INT, INT
) RETURNS INT LANGUAGE SQL AS $$
SELECT $1 + $2 + $3 + $4 + $5 + $6 + $7 + $8 + $9 + $10 +
$11 + $12 + $13 + $14 + $15 + $16 + $17 + $18 + $19 + $20 +
$21 + $22 + $23 + $24 + $25 + $26 + $27 + $28 + $29 + $30 +
$31 + $32 + $33 + $34 + $35 + $36 + $37 + $38 + $39 + $40 +
$41 + $42 + $43 + $44 + $45 + $46 + $47 + $48 + $49 + $50 +
$51 + $52 + $53 + $54 + $55 + $56 + $57 + $58 + $59 + $60 +
$61 + $62 + $63 + $64 + $65 + $66 + $67 + $68 + $69 + $70 +
$71 + $72 + $73 + $74 + $75 + $76 + $77 + $78 + $79 + $80 +
$81 + $82 + $83 + $84 + $85 + $86 + $87 + $88 + $89 + $90 +
$91 + $92 + $93 + $94 + $95 + $96 + $97 + $98 + $99 + $100;
$$;

query TI
SELECT sum(i),
add(1, 2, 3, 4, 5, 6, 7, 8, 9, 10,
11, 12, 13, 14, 15, 16, 17, 18, 19, 20,
21, 22, 23, 24, 25, 26, 27, 28, 29, 30,
31, 32, 33, 34, 35, 36, 37, 38, 39, 40,
41, 42, 43, 44, 45, 46, 47, 48, 49, 50,
51, 52, 53, 54, 55, 56, 57, 58, 59, 60,
61, 62, 63, 64, 65, 66, 67, 68, 69, 70,
71, 72, 73, 74, 75, 76, 77, 78, 79, 80,
81, 82, 83, 84, 85, 86, 87, 88, 89, 90,
91, 92, 93, 94, 95, 96, 97, 98, 99, 100)
FROM generate_series(1, 100) AS g(i)
----
5050 5050


subtest volatility

statement ok
Expand Down
11 changes: 3 additions & 8 deletions pkg/sql/opt/optbuilder/create_function.go
Original file line number Diff line number Diff line change
Expand Up @@ -93,8 +93,6 @@ func (b *Builder) buildCreateFunction(cf *tree.CreateFunction, inScope *scope) (
// bodyScope is the base scope for each statement in the body. We add the
// named arguments to the scope so that references to them in the body can
// be resolved.
// TODO(mgartner): Support numeric argument references, like $1. We should
// error if there is a reference $n and less than n arguments.
bodyScope := b.allocScope()
for i := range cf.Args {
arg := &cf.Args[i]
Expand All @@ -104,12 +102,9 @@ func (b *Builder) buildCreateFunction(cf *tree.CreateFunction, inScope *scope) (
}

// Add the argument to the base scope of the body.
id := b.factory.Metadata().AddColumn(string(arg.Name), typ)
bodyScope.appendColumn(&scopeColumn{
name: scopeColName(arg.Name),
typ: typ,
id: id,
})
argColName := funcArgColName(arg.Name, i)
col := b.synthesizeColumn(bodyScope, argColName, typ, nil /* expr */, nil /* scalar */)
col.setArgOrd(i)

// Collect the user defined type dependencies.
typeIDs, err := typedesc.GetTypeDescriptorClosure(typ)
Expand Down
15 changes: 4 additions & 11 deletions pkg/sql/opt/optbuilder/scalar.go
Original file line number Diff line number Diff line change
Expand Up @@ -625,12 +625,8 @@ func (b *Builder) buildUDF(
// arguments, we add them as columns to the scope so that references to them
// can be resolved.
//
// TODO(mgartner): Support anonymous arguments and placeholder-like syntax
// for referencing arguments, e.g., $1.
//
// TODO(mgartner): We may need to set bodyScope.atRoot=true to prevent
// CTEs that mutate and are not at the top-level.
//
bodyScope := b.allocScope()
var argCols opt.ColList
if o.Types.Length() > 0 {
Expand All @@ -643,13 +639,10 @@ func (b *Builder) buildUDF(
argCols = make(opt.ColList, len(args))
for i := range args {
arg := &args[i]
id := b.factory.Metadata().AddColumn(arg.Name, arg.Typ)
argCols[i] = id
bodyScope.appendColumn(&scopeColumn{
name: scopeColName(tree.Name(arg.Name)),
typ: arg.Typ,
id: id,
})
argColName := funcArgColName(tree.Name(arg.Name), i)
col := b.synthesizeColumn(bodyScope, argColName, arg.Typ, nil /* expr */, nil /* scalar */)
col.setArgOrd(i)
argCols[i] = col.id
}
}

Expand Down
32 changes: 30 additions & 2 deletions pkg/sql/opt/optbuilder/scope.go
Original file line number Diff line number Diff line change
Expand Up @@ -411,8 +411,9 @@ func (s *scope) makePresentationWithHiddenCols() physical.Presentation {
}

// walkExprTree walks the given expression and performs name resolution,
// replaces unresolved column names with columnProps, and replaces subqueries
// with typed subquery structs.
// replaces unresolved column names with columnProps, replaces subqueries with
// typed subquery structs, and replaces placeholders with scope columns when
// they are references to function arguments.
func (s *scope) walkExprTree(expr tree.Expr) tree.Expr {
// TODO(peter): The caller should specify the desired number of columns. This
// is needed when a subquery is used by an UPDATE statement.
Expand Down Expand Up @@ -630,6 +631,22 @@ func (s *scope) findExistingCol(expr tree.TypedExpr, allowSideEffects bool) *sco
return col
}

// findFuncArgCol returns the column that represents a function argument and has
// an ordinal matching the given placeholder index. If such a column is not
// found in the current scope, ancestor scopes are successively searched. If no
// matching function argument column is found, nil is returned.
func (s *scope) findFuncArgCol(idx tree.PlaceholderIdx) *scopeColumn {
for ; s != nil; s = s.parent {
for i := range s.cols {
col := &s.cols[i]
if col.funcArgReferencedBy(idx) {
return col
}
}
}
return nil
}

// startAggFunc is called when the builder starts building an aggregate
// function. It is used to disallow nested aggregates and ensure that a
// grouping error is not called on the aggregate arguments. For example:
Expand Down Expand Up @@ -1014,6 +1031,17 @@ func (s *scope) VisitPre(expr tree.Expr) (recurse bool, newExpr tree.Expr) {
}
return false, colI.(*scopeColumn)

case *tree.Placeholder:
// Replace placeholders that are references to function arguments with
// scope columns that represent those arguments.
//
// NOTE: This likely won't work if we want to allow PREPARE statements
// within user-defined function bodies. We'll need to avoid replacing
// placeholders that are prepared statement parameters.
if col := s.findFuncArgCol(t.Idx); col != nil {
return false, col
}

case *tree.FuncExpr:
semaCtx := s.builder.semaCtx
// TODO(mgartner): At this point the the function has not been type checked
Expand Down
48 changes: 48 additions & 0 deletions pkg/sql/opt/optbuilder/scope_column.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,9 +12,12 @@ package optbuilder

import (
"context"
"fmt"

"github.com/cockroachdb/cockroach/pkg/sql/opt"
"github.com/cockroachdb/cockroach/pkg/sql/opt/cat"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -54,6 +57,13 @@ type scopeColumn struct {
// This field is only used for ordering columns.
descending bool

// funcArgOrd is the 1-based ordinal of the argument of the function that
// the column corresponds to. It is used to resolve placeholders (e.g., $1)
// in function bodies that are references to function arguments. If the
// column does not represent a function argument, then argOrd is the zero
// value.
argOrd funcArgOrd

// scalar is the scalar expression associated with this column. If it is nil,
// then the column is a passthrough from an inner scope or a table column.
scalar opt.ScalarExpr
Expand Down Expand Up @@ -106,6 +116,31 @@ func (cv columnVisibility) String() string {
}
}

// maxFuncArgs is the maximum number of arguments allowed in a function.
const maxFuncArgs = 100

// funcArgOrd is a 1-based ordinal of a function argument.
type funcArgOrd int8

// setArgOrd sets the column's 1-based function argument ordinal to the given
// 0-based ordinal. Panics if the given ordinal is not in the range
// [0, maxFuncArgs).
func (c *scopeColumn) setArgOrd(ord int) {
if ord < 0 {
panic(errors.AssertionFailedf("expected non-negative argument ordinal"))
}
if ord >= maxFuncArgs {
panic(pgerror.New(pgcode.TooManyArguments, "functions cannot have more than 100 arguments"))
}
c.argOrd = funcArgOrd(ord + 1)
}

// funcArgReferencedBy returns true if the scopeColumn is a function argument
// column that can be referenced by the given placeholder.
func (c *scopeColumn) funcArgReferencedBy(idx tree.PlaceholderIdx) bool {
return c.argOrd > 0 && tree.PlaceholderIdx(c.argOrd-1) == idx
}

// clearName sets the empty table and column name. This is used to make the
// column anonymous so that it cannot be referenced, but will still be
// projected.
Expand Down Expand Up @@ -241,6 +276,19 @@ func scopeColName(name tree.Name) scopeColumnName {
}
}

// funcArgColName creates a scopeColumnName that can be referenced by the given
// name and will be added to the metadata with the given name, if the given name
// is not empty. If the given name is empty, the returned scopeColumnName
// represents an anonymous function argument that cannot be referenced, and it
// will be added to the metadata with the descriptive name "arg<ord>".
func funcArgColName(name tree.Name, ord int) scopeColumnName {
alias := string(name)
if alias == "" {
alias = fmt.Sprintf("arg%d", ord+1)
}
return scopeColName(name).WithMetadataName(alias)
}

// WithMetadataName returns a copy of s with the metadata name set to the given
// name. This only affects the name of the column in the metadata. It does not
// change the name by which the column can be referenced.
Expand Down
Loading

0 comments on commit 513fae1

Please sign in to comment.