diff --git a/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go b/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go index eed192f6d298..fa92cd05e1a8 100644 --- a/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go +++ b/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go @@ -2019,6 +2019,13 @@ func TestTenantLogic_udf( runLogicTest(t, "udf") } +func TestTenantLogic_udf_star( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "udf_star") +} + func TestTenantLogic_union( t *testing.T, ) { diff --git a/pkg/sql/logictest/testdata/logic_test/udf b/pkg/sql/logictest/testdata/logic_test/udf index 7685e386689a..45056df03089 100644 --- a/pkg/sql/logictest/testdata/logic_test/udf +++ b/pkg/sql/logictest/testdata/logic_test/udf @@ -52,24 +52,6 @@ CREATE FUNCTION err(i INT) RETURNS INT LANGUAGE SQL AS 'SELECT j' statement error pgcode 42703 column \"j\" does not exist CREATE FUNCTION err(i INT) RETURNS INT LANGUAGE SQL AS 'SELECT a FROM ab WHERE a = j' -statement error pgcode 0A000 functions do not currently support \* expressions -CREATE FUNCTION err(i INT) RETURNS ab LANGUAGE SQL AS 'SELECT * FROM ab' - -statement error pgcode 0A000 functions do not currently support \* expressions -CREATE FUNCTION err(i INT) RETURNS ab LANGUAGE SQL AS 'SELECT ab.* FROM ab' - -statement error pgcode 0A000 functions do not currently support \* expressions -CREATE FUNCTION err(i INT) RETURNS ab LANGUAGE SQL AS $$ - SELECT 1; - SELECT * FROM ab; -$$ - -statement error pgcode 0A000 functions do not currently support \* expressions -CREATE FUNCTION err(i INT) RETURNS INT LANGUAGE SQL AS $$ - SELECT * FROM ab; - SELECT 1; -$$ - statement ok CREATE FUNCTION d(i INT2) RETURNS INT4 LANGUAGE SQL AS 'SELECT i' diff --git a/pkg/sql/logictest/testdata/logic_test/udf_star b/pkg/sql/logictest/testdata/logic_test/udf_star new file mode 100644 index 000000000000..0df172119f7f --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/udf_star @@ -0,0 +1,194 @@ +statement ok +CREATE TABLE t_onecol (a INT); +INSERT INTO t_onecol VALUES (1) + +statement ok +CREATE TABLE t_twocol (a INT, b INT); +INSERT INTO t_twocol VALUES (1,2) + +statement ok +CREATE FUNCTION f_unqualified_onecol() RETURNS INT AS +$$ + SELECT * FROM t_onecol; +$$ LANGUAGE SQL; + +statement ok +CREATE FUNCTION f_subquery() RETURNS INT AS +$$ + SELECT * FROM (SELECT a FROM (SELECT * FROM t_onecol) AS foo) AS bar; +$$ LANGUAGE SQL; + +statement ok +CREATE FUNCTION f_unqualified_twocol() RETURNS t_twocol AS +$$ + SELECT * FROM t_twocol; +$$ LANGUAGE SQL; + +statement ok +CREATE FUNCTION f_allcolsel() RETURNS t_twocol AS +$$ + SELECT t_twocol.* FROM t_twocol; +$$ LANGUAGE SQL; + +statement ok +CREATE FUNCTION f_allcolsel_alias() RETURNS t_twocol AS +$$ + SELECT t1.* FROM t_twocol AS t1, t_twocol AS t2 WHERE t1.a = t2.a; +$$ LANGUAGE SQL; + +statement ok +CREATE FUNCTION f_tuplestar() RETURNS t_twocol AS +$$ + SELECT (t_twocol.*).* FROM t_twocol; +$$ LANGUAGE SQL; + +statement ok +CREATE FUNCTION f_unqualified_multicol() RETURNS INT AS +$$ + SELECT *, a FROM t_onecol; + SELECT a FROM t_onecol; +$$ LANGUAGE SQL; + +statement ok +CREATE FUNCTION f_unqualified_doublestar() RETURNS INT AS +$$ + SELECT *, * FROM t_onecol; + SELECT a FROM t_onecol; +$$ LANGUAGE SQL; + +statement ok +CREATE FUNCTION f_exprstar() RETURNS STRING AS +$$ + SELECT word FROM (SELECT (pg_get_keywords()).* ORDER BY word LIMIT 1); +$$ LANGUAGE SQL; + +query TTT +SELECT oid, proname, prosrc +FROM pg_catalog.pg_proc WHERE proname LIKE 'f\_%' ORDER BY oid; +---- +100108 f_unqualified_onecol SELECT t_onecol.a FROM test.public.t_onecol; +100109 f_subquery SELECT bar.a FROM (SELECT a FROM (SELECT t_onecol.a FROM test.public.t_onecol) AS foo) AS bar; +100110 f_unqualified_twocol SELECT t_twocol.a, t_twocol.b FROM test.public.t_twocol; +100111 f_allcolsel SELECT t_twocol.a, t_twocol.b FROM test.public.t_twocol; +100112 f_allcolsel_alias SELECT t1.a, t1.b FROM test.public.t_twocol AS t1, test.public.t_twocol AS t2 WHERE t1.a = t2.a; +100113 f_tuplestar SELECT t_twocol.a, t_twocol.b FROM test.public.t_twocol; +100114 f_unqualified_multicol SELECT t_onecol.a, a FROM test.public.t_onecol; + SELECT a FROM test.public.t_onecol; +100115 f_unqualified_doublestar SELECT t_onecol.a, t_onecol.a FROM test.public.t_onecol; + SELECT a FROM test.public.t_onecol; +100116 f_exprstar SELECT word FROM (SELECT (pg_get_keywords()).word, (pg_get_keywords()).catcode, (pg_get_keywords()).catdesc ORDER BY word LIMIT 1); + +query TT +SHOW CREATE FUNCTION f_subquery +---- +f_subquery CREATE FUNCTION public.f_subquery() + RETURNS INT8 + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE SQL + AS $$ + SELECT bar.a FROM (SELECT a FROM (SELECT t_onecol.a FROM test.public.t_onecol) AS foo) AS bar; + $$ + +query TT +SHOW CREATE FUNCTION f_allcolsel_alias +---- +f_allcolsel_alias CREATE FUNCTION public.f_allcolsel_alias() + RETURNS T_TWOCOL + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE SQL + AS $$ + SELECT t1.a, t1.b FROM test.public.t_twocol AS t1, test.public.t_twocol AS t2 WHERE t1.a = t2.a; + $$ + +query I +SELECT f_unqualified_onecol() +---- +1 + +query I +SELECT f_subquery() +---- +1 + +query T +SELECT f_exprstar() +---- +abort + +# Adding a column to a table should not change the UDFs that reference it. +statement ok +ALTER TABLE t_onecol ADD COLUMN b INT DEFAULT 5; + +query I +SELECT f_unqualified_onecol() +---- +1 + +query I +SELECT f_subquery() +---- +1 + +# It's ok to drop a column that was not used by the original UDF. +statement ok +ALTER TABLE t_onecol DROP COLUMN b; + +query T +SELECT f_unqualified_twocol() +---- +(1,2) + +query T +SELECT f_allcolsel() +---- +(1,2) + +query T +SELECT f_allcolsel_alias() +---- +(1,2) + +statement ok +ALTER TABLE t_twocol ADD COLUMN c INT DEFAULT 5; + +# TODO(#95558): With early binding, postgres returns an error after adding a +# column when the table is used as the return type. Note that this behavior is +# ok for late binding. +query T +SELECT f_unqualified_twocol() +---- +(1,2) + +# Altering a column type is not allowed in postgres or CRDB. +statement error pq: cannot alter type of column "b" because function "f_unqualified_twocol" depends on it +ALTER TABLE t_twocol ALTER b TYPE FLOAT; + +# TODO(harding): Postgres allows column renaming when only referenced by UDFs. +statement error pq: cannot rename column "a" because function "f_unqualified_twocol" depends on it +ALTER TABLE t_twocol RENAME COLUMN a TO d; + +# TODO(harding): Postgres allows table renaming when only referenced by UDFs. +statement error pq: cannot rename relation "t_twocol" because function "f_unqualified_twocol" depends on it +ALTER TABLE t_twocol RENAME TO t_twocol_prime; + +# Dropping a column a UDF depends on is not allowed. +#statement error pq: cannot drop column "b" because function "f_unqualified_twocol" depends on it +#ALTER TABLE t_twocol DROP COLUMN b; + +# Dropping a column using CASCADE is ok. +statement ok +ALTER TABLE t_twocol DROP COLUMN b CASCADE; + +statement ok +DROP TABLE t_onecol CASCADE; + +# The only remaining function should not reference the tables. +query TTT +SELECT oid, proname, prosrc +FROM pg_catalog.pg_proc WHERE proname LIKE 'f\_%' ORDER BY oid; +---- +100116 f_exprstar SELECT word FROM (SELECT (pg_get_keywords()).word, (pg_get_keywords()).catcode, (pg_get_keywords()).catdesc ORDER BY word LIMIT 1); diff --git a/pkg/sql/logictest/tests/fakedist-disk/generated_test.go b/pkg/sql/logictest/tests/fakedist-disk/generated_test.go index f063f7c1018e..1d7d2263459a 100644 --- a/pkg/sql/logictest/tests/fakedist-disk/generated_test.go +++ b/pkg/sql/logictest/tests/fakedist-disk/generated_test.go @@ -1990,6 +1990,13 @@ func TestLogic_udf( runLogicTest(t, "udf") } +func TestLogic_udf_star( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "udf_star") +} + func TestLogic_union( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/fakedist-vec-off/generated_test.go b/pkg/sql/logictest/tests/fakedist-vec-off/generated_test.go index 1ed95c3e9141..bb58d21dbe4e 100644 --- a/pkg/sql/logictest/tests/fakedist-vec-off/generated_test.go +++ b/pkg/sql/logictest/tests/fakedist-vec-off/generated_test.go @@ -1997,6 +1997,13 @@ func TestLogic_udf( runLogicTest(t, "udf") } +func TestLogic_udf_star( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "udf_star") +} + func TestLogic_union( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/fakedist/generated_test.go b/pkg/sql/logictest/tests/fakedist/generated_test.go index aabbee91bed6..0580b4334d2a 100644 --- a/pkg/sql/logictest/tests/fakedist/generated_test.go +++ b/pkg/sql/logictest/tests/fakedist/generated_test.go @@ -2011,6 +2011,13 @@ func TestLogic_udf( runLogicTest(t, "udf") } +func TestLogic_udf_star( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "udf_star") +} + func TestLogic_union( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/local-legacy-schema-changer/generated_test.go b/pkg/sql/logictest/tests/local-legacy-schema-changer/generated_test.go index aa2ce22a5fb9..f6e62fb6e38f 100644 --- a/pkg/sql/logictest/tests/local-legacy-schema-changer/generated_test.go +++ b/pkg/sql/logictest/tests/local-legacy-schema-changer/generated_test.go @@ -1976,6 +1976,13 @@ func TestLogic_udf( runLogicTest(t, "udf") } +func TestLogic_udf_star( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "udf_star") +} + func TestLogic_union( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/local-vec-off/generated_test.go b/pkg/sql/logictest/tests/local-vec-off/generated_test.go index 0da606fd62c9..f554fe62764b 100644 --- a/pkg/sql/logictest/tests/local-vec-off/generated_test.go +++ b/pkg/sql/logictest/tests/local-vec-off/generated_test.go @@ -2011,6 +2011,13 @@ func TestLogic_udf( runLogicTest(t, "udf") } +func TestLogic_udf_star( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "udf_star") +} + func TestLogic_union( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/local/generated_test.go b/pkg/sql/logictest/tests/local/generated_test.go index 8569be3004de..87de51ba4005 100644 --- a/pkg/sql/logictest/tests/local/generated_test.go +++ b/pkg/sql/logictest/tests/local/generated_test.go @@ -2200,6 +2200,13 @@ func TestLogic_udf( runLogicTest(t, "udf") } +func TestLogic_udf_star( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "udf_star") +} + func TestLogic_union( t *testing.T, ) { diff --git a/pkg/sql/opt/optbuilder/delete.go b/pkg/sql/opt/optbuilder/delete.go index 834eee850a8c..bcac82ceb73d 100644 --- a/pkg/sql/opt/optbuilder/delete.go +++ b/pkg/sql/opt/optbuilder/delete.go @@ -66,7 +66,7 @@ func (b *Builder) buildDelete(del *tree.Delete, inScope *scope) (outScope *scope // Build the final delete statement, including any returned expressions. if resultsNeeded(del.Returning) { - mb.buildDelete(*del.Returning.(*tree.ReturningExprs)) + mb.buildDelete(del.Returning.(*tree.ReturningExprs)) } else { mb.buildDelete(nil /* returning */) } @@ -76,7 +76,7 @@ func (b *Builder) buildDelete(del *tree.Delete, inScope *scope) (outScope *scope // buildDelete constructs a Delete operator, possibly wrapped by a Project // operator that corresponds to the given RETURNING clause. -func (mb *mutationBuilder) buildDelete(returning tree.ReturningExprs) { +func (mb *mutationBuilder) buildDelete(returning *tree.ReturningExprs) { mb.buildFKChecksAndCascadesForDelete() // Project partial index DEL boolean columns. diff --git a/pkg/sql/opt/optbuilder/insert.go b/pkg/sql/opt/optbuilder/insert.go index 5951ab72482b..99cffb0a42e6 100644 --- a/pkg/sql/opt/optbuilder/insert.go +++ b/pkg/sql/opt/optbuilder/insert.go @@ -281,9 +281,9 @@ func (b *Builder) buildInsert(ins *tree.Insert, inScope *scope) (outScope *scope // See mutationBuilder.buildCheckInputScan. mb.insertExpr = mb.outScope.expr - var returning tree.ReturningExprs + var returning *tree.ReturningExprs if resultsNeeded(ins.Returning) { - returning = *ins.Returning.(*tree.ReturningExprs) + returning = ins.Returning.(*tree.ReturningExprs) } switch { @@ -669,7 +669,7 @@ func (mb *mutationBuilder) addSynthesizedColsForInsert() { // buildInsert constructs an Insert operator, possibly wrapped by a Project // operator that corresponds to the given RETURNING clause. -func (mb *mutationBuilder) buildInsert(returning tree.ReturningExprs) { +func (mb *mutationBuilder) buildInsert(returning *tree.ReturningExprs) { // Disambiguate names so that references in any expressions, such as a // check constraint, refer to the correct columns. mb.disambiguateColumns() @@ -874,7 +874,7 @@ func (mb *mutationBuilder) setUpsertCols(insertCols tree.NameList) { // buildUpsert constructs an Upsert operator, possibly wrapped by a Project // operator that corresponds to the given RETURNING clause. -func (mb *mutationBuilder) buildUpsert(returning tree.ReturningExprs) { +func (mb *mutationBuilder) buildUpsert(returning *tree.ReturningExprs) { // Merge input insert and update columns using CASE expressions. mb.projectUpsertColumns() diff --git a/pkg/sql/opt/optbuilder/mutation_builder.go b/pkg/sql/opt/optbuilder/mutation_builder.go index 5b152eed1da8..101164006ae2 100644 --- a/pkg/sql/opt/optbuilder/mutation_builder.go +++ b/pkg/sql/opt/optbuilder/mutation_builder.go @@ -1019,7 +1019,7 @@ func (mb *mutationBuilder) mapToReturnColID(tabOrd int) opt.ColumnID { // buildReturning wraps the input expression with a Project operator that // projects the given RETURNING expressions. -func (mb *mutationBuilder) buildReturning(returning tree.ReturningExprs) { +func (mb *mutationBuilder) buildReturning(returning *tree.ReturningExprs) { // Handle case of no RETURNING clause. if returning == nil { expr := mb.outScope.expr diff --git a/pkg/sql/opt/optbuilder/project.go b/pkg/sql/opt/optbuilder/project.go index a9c9e7622a9c..8f9615619b26 100644 --- a/pkg/sql/opt/optbuilder/project.go +++ b/pkg/sql/opt/optbuilder/project.go @@ -79,7 +79,7 @@ func (b *Builder) dropOrderingAndExtraCols(s *scope) { // and adds the resulting aliases and typed expressions to outScope. See the // header comment for analyzeSelectList. func (b *Builder) analyzeProjectionList( - selects tree.SelectExprs, desiredTypes []*types.T, inScope, outScope *scope, + selects *tree.SelectExprs, desiredTypes []*types.T, inScope, outScope *scope, ) { // We need to save and restore the previous values of the replaceSRFs field // and the field in semaCtx in case we are recursively called within a @@ -98,7 +98,7 @@ func (b *Builder) analyzeProjectionList( // and adds the resulting aliases and typed expressions to outScope. See the // header comment for analyzeSelectList. func (b *Builder) analyzeReturningList( - returning tree.ReturningExprs, desiredTypes []*types.T, inScope, outScope *scope, + returning *tree.ReturningExprs, desiredTypes []*types.T, inScope, outScope *scope, ) { // We need to save and restore the previous value of the field in // semaCtx in case we are recursively called within a subquery @@ -109,7 +109,7 @@ func (b *Builder) analyzeReturningList( b.semaCtx.Properties.Require(exprKindReturning.String(), tree.RejectSpecial) inScope.context = exprKindReturning - b.analyzeSelectList(tree.SelectExprs(returning), desiredTypes, inScope, outScope) + b.analyzeSelectList((*tree.SelectExprs)(returning), desiredTypes, inScope, outScope) } // analyzeSelectList is a helper function used by analyzeProjectionList and @@ -119,10 +119,15 @@ func (b *Builder) analyzeReturningList( // // As a side-effect, the appropriate scopes are updated with aggregations // (scope.groupby.aggs) +// +// If we are building a function, the `selects` expressions will be overwritten +// with expressions that replace any `*` expressions with their columns. func (b *Builder) analyzeSelectList( - selects tree.SelectExprs, desiredTypes []*types.T, inScope, outScope *scope, + selects *tree.SelectExprs, desiredTypes []*types.T, inScope, outScope *scope, ) { - for i, e := range selects { + var expansions tree.SelectExprs + for i, e := range *selects { + expanded := false // Start with fast path, looking for simple column reference. texpr := b.resolveColRef(e.Expr, inScope) if texpr == nil { @@ -142,8 +147,19 @@ func (b *Builder) analyzeSelectList( } aliases, exprs := b.expandStar(e.Expr, inScope) + if b.insideFuncDef { + expanded = true + for _, expr := range exprs { + switch col := expr.(type) { + case *scopeColumn: + expansions = append(expansions, tree.SelectExpr{Expr: tree.NewColumnItem(&col.table, col.name.ReferenceName())}) + case *tree.ColumnAccessExpr: + expansions = append(expansions, tree.SelectExpr{Expr: col}) + } + } + } if outScope.cols == nil { - outScope.cols = make([]scopeColumn, 0, len(selects)+len(exprs)-1) + outScope.cols = make([]scopeColumn, 0, len(*selects)+len(exprs)-1) } for j, e := range exprs { outScope.addColumn(scopeColName(tree.Name(aliases[j])), e) @@ -164,10 +180,16 @@ func (b *Builder) analyzeSelectList( // have to determine the output column name before we perform type // checking. if outScope.cols == nil { - outScope.cols = make([]scopeColumn, 0, len(selects)) + outScope.cols = make([]scopeColumn, 0, len(*selects)) } alias := b.getColName(e) outScope.addColumn(scopeColName(tree.Name(alias)), texpr) + if b.insideFuncDef && !expanded { + expansions = append(expansions, e) + } + } + if b.insideFuncDef { + *selects = expansions } } diff --git a/pkg/sql/opt/optbuilder/select.go b/pkg/sql/opt/optbuilder/select.go index 9b68a2d9fea3..ff5abf5903a9 100644 --- a/pkg/sql/opt/optbuilder/select.go +++ b/pkg/sql/opt/optbuilder/select.go @@ -51,7 +51,7 @@ func (b *Builder) buildDataSource( }(inScope.atRoot) inScope.atRoot = false // NB: The case statements are sorted lexicographically. - switch source := texpr.(type) { + switch source := (texpr).(type) { case *tree.AliasedTableExpr: if source.IndexFlags != nil { telemetry.Inc(sqltelemetry.IndexHintUseCounter) @@ -110,7 +110,6 @@ func (b *Builder) buildDataSource( } ds, depName, resName := b.resolveDataSource(tn, privilege.SELECT) - locking = locking.filter(tn.ObjectName) if locking.isSet() { // SELECT ... FOR [KEY] UPDATE/SHARE also requires UPDATE privileges. @@ -1052,7 +1051,7 @@ func (b *Builder) buildSelectClause( // function that refers to variables in fromScope or an ancestor scope, // buildAggregateFunction is called which adds columns to the appropriate // aggInScope and aggOutScope. - b.analyzeProjectionList(sel.Exprs, desiredTypes, fromScope, projectionsScope) + b.analyzeProjectionList(&sel.Exprs, desiredTypes, fromScope, projectionsScope) // Any aggregates in the HAVING, ORDER BY and DISTINCT ON clauses (if they // exist) will be added here. diff --git a/pkg/sql/opt/optbuilder/testdata/create_function b/pkg/sql/opt/optbuilder/testdata/create_function index 5ee1e03191e2..6d17e4731778 100644 --- a/pkg/sql/opt/optbuilder/testdata/create_function +++ b/pkg/sql/opt/optbuilder/testdata/create_function @@ -108,9 +108,15 @@ create-function └── s build -CREATE FUNCTION f() RETURNS INT LANGUAGE SQL AS $$ SELECT * FROM ab $$ +CREATE FUNCTION f() RETURNS ab LANGUAGE SQL AS $$ SELECT * FROM ab $$ ---- -error (0A000): unimplemented: functions do not currently support * expressions +create-function + ├── CREATE FUNCTION f() + │ RETURNS ab + │ LANGUAGE SQL + │ AS $$SELECT ab.a, ab.b FROM t.public.ab;$$ + └── dependencies + └── ab [columns: a b] build CREATE FUNCTION f() RETURNS INT LANGUAGE SQL BEGIN ATOMIC SELECT 1; END; diff --git a/pkg/sql/opt/optbuilder/testdata/udf b/pkg/sql/opt/optbuilder/testdata/udf index 4f4bc04dfb1a..97c72c00a056 100644 --- a/pkg/sql/opt/optbuilder/testdata/udf +++ b/pkg/sql/opt/optbuilder/testdata/udf @@ -1166,3 +1166,82 @@ project │ └── projections │ └── variable: i:10 [as=i:13] └── const: 1 + +# -------------------------------------------------- +# UDFs with * expressions. +# -------------------------------------------------- + +exec-ddl +CREATE TABLE tstar ( + a INT +) +---- + +exec-ddl +CREATE FUNCTION fn_star() RETURNS INT LANGUAGE SQL AS 'SELECT * FROM tstar' +---- + +build format=show-scalars +SELECT fn_star() +---- +project + ├── columns: fn_star:5 + ├── values + │ └── tuple + └── projections + └── udf: fn_star [as=fn_star:5] + └── body + └── limit + ├── columns: a:1 + ├── project + │ ├── columns: a:1 + │ └── scan tstar + │ └── columns: a:1 rowid:2!null crdb_internal_mvcc_timestamp:3 tableoid:4 + └── const: 1 + +exec-ddl +CREATE TABLE tstar2 ( + a INT, + b INT +) +---- + +exec-ddl +CREATE FUNCTION fn_star2() RETURNS INT LANGUAGE SQL AS 'SELECT * FROM tstar, tstar2 WHERE tstar.a = tstar2.b' +---- + +build format=show-scalars +SELECT fn_star2() +---- +project + ├── columns: fn_star2:11 + ├── values + │ └── tuple + └── projections + └── udf: fn_star2 [as=fn_star2:11] + └── body + └── project + ├── columns: column10:10 + ├── limit + │ ├── columns: tstar.a:1!null tstar2.a:5 b:6!null + │ ├── project + │ │ ├── columns: tstar.a:1!null tstar2.a:5 b:6!null + │ │ └── select + │ │ ├── columns: tstar.a:1!null tstar.rowid:2!null tstar.crdb_internal_mvcc_timestamp:3 tstar.tableoid:4 tstar2.a:5 b:6!null tstar2.rowid:7!null tstar2.crdb_internal_mvcc_timestamp:8 tstar2.tableoid:9 + │ │ ├── inner-join (cross) + │ │ │ ├── columns: tstar.a:1 tstar.rowid:2!null tstar.crdb_internal_mvcc_timestamp:3 tstar.tableoid:4 tstar2.a:5 b:6 tstar2.rowid:7!null tstar2.crdb_internal_mvcc_timestamp:8 tstar2.tableoid:9 + │ │ │ ├── scan tstar + │ │ │ │ └── columns: tstar.a:1 tstar.rowid:2!null tstar.crdb_internal_mvcc_timestamp:3 tstar.tableoid:4 + │ │ │ ├── scan tstar2 + │ │ │ │ └── columns: tstar2.a:5 b:6 tstar2.rowid:7!null tstar2.crdb_internal_mvcc_timestamp:8 tstar2.tableoid:9 + │ │ │ └── filters (true) + │ │ └── filters + │ │ └── eq + │ │ ├── variable: tstar.a:1 + │ │ └── variable: b:6 + │ └── const: 1 + └── projections + └── tuple [as=column10:10] + ├── variable: tstar.a:1 + ├── variable: tstar2.a:5 + └── variable: b:6 diff --git a/pkg/sql/opt/optbuilder/update.go b/pkg/sql/opt/optbuilder/update.go index 3ec93d7551f7..502e414de34b 100644 --- a/pkg/sql/opt/optbuilder/update.go +++ b/pkg/sql/opt/optbuilder/update.go @@ -108,7 +108,7 @@ func (b *Builder) buildUpdate(upd *tree.Update, inScope *scope) (outScope *scope // Build the final update statement, including any returned expressions. if resultsNeeded(upd.Returning) { - mb.buildUpdate(*upd.Returning.(*tree.ReturningExprs)) + mb.buildUpdate(upd.Returning.(*tree.ReturningExprs)) } else { mb.buildUpdate(nil /* returning */) } @@ -326,7 +326,7 @@ func (mb *mutationBuilder) addSynthesizedColsForUpdate() { // buildUpdate constructs an Update operator, possibly wrapped by a Project // operator that corresponds to the given RETURNING clause. -func (mb *mutationBuilder) buildUpdate(returning tree.ReturningExprs) { +func (mb *mutationBuilder) buildUpdate(returning *tree.ReturningExprs) { // Disambiguate names so that references in any expressions, such as a // check constraint, refer to the correct columns. mb.disambiguateColumns() diff --git a/pkg/sql/opt/optbuilder/util.go b/pkg/sql/opt/optbuilder/util.go index c3f7f68781a0..c210674d376c 100644 --- a/pkg/sql/opt/optbuilder/util.go +++ b/pkg/sql/opt/optbuilder/util.go @@ -64,9 +64,6 @@ func (b *Builder) expandStar( if b.insideViewDef { panic(unimplemented.NewWithIssue(10028, "views do not currently support * expressions")) } - if b.insideFuncDef { - panic(unimplemented.NewWithIssue(90080, "functions do not currently support * expressions")) - } switch t := expr.(type) { case *tree.TupleStar: texpr := inScope.resolveType(t.Expr, types.Any)