From 92e404a3dbe5720960da838f9bb3c033a7564d5c Mon Sep 17 00:00:00 2001 From: Drew Kimball Date: Wed, 3 Jul 2024 15:21:51 -0600 Subject: [PATCH 1/3] sql: allow trigger functions to be created This commit adds support for trigger function creation. Since trigger functions don't yet known the type of the table they'll be associated with, SQL expressions are not evaluated at creation time. The lazy evaluation is implemented by replacing expressions with NULL, and relations with a single no-column row. This allows CRDB to check that PL/pgSQL usage is correct, and defer checking the SQL usage until the trigger function is associated with a trigger. Fixes #126356 Fixes #126357 Release note (sql change): It is now possible to create PL/pgSQL trigger functions, which can be executed by a trigger in response to table mutation events. Note that this patch does not add support for triggers; only trigger functions. --- .../logictestccl/testdata/logic_test/triggers | 333 +++++++++++++++++- pkg/sql/create_function.go | 28 +- pkg/sql/opt/optbuilder/create_function.go | 41 ++- pkg/sql/opt/optbuilder/plpgsql.go | 61 +++- pkg/sql/opt/optbuilder/routine.go | 19 +- .../schemachanger/scbuild/builder_state.go | 15 +- .../internal/scbuildstmt/create_function.go | 2 +- .../internal/scbuildstmt/dependencies.go | 3 +- 8 files changed, 451 insertions(+), 51 deletions(-) diff --git a/pkg/ccl/logictestccl/testdata/logic_test/triggers b/pkg/ccl/logictestccl/testdata/logic_test/triggers index 301543afe916..02f00405bb70 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/triggers +++ b/pkg/ccl/logictestccl/testdata/logic_test/triggers @@ -1,28 +1,23 @@ -# TODO(#126356): remove this case when trigger functions are supported. -statement error pgcode 0A000 pq: unimplemented: trigger functions are not yet supported -CREATE FUNCTION f() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ BEGIN RETURN NULL; END $$; - # ============================================================================== # Trigger functions cannot be directly invoked. # ============================================================================== subtest direct_invocation -# TODO(#126356): uncomment these cases when trigger functions are supported. -#statement ok -#CREATE FUNCTION f() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ BEGIN RETURN NULL; END $$; -# -#statement error pgcode 0A000 pq: trigger functions can only be called as triggers -#SELECT f(); -# -#statement error pgcode 0A000 pq: trigger functions can only be called as triggers -#CREATE FUNCTION foo() RETURNS INT LANGUAGE SQL AS $$ SELECT f(); SELECT 1; $$; -# -#statement error pgcode 0A000 pq: trigger functions can only be called as triggers -#CREATE FUNCTION foo() RETURNS INT LANGUAGE PLpgSQL AS $$ BEGIN SELECT f(); RETURN 1; END $$; -# -#statement ok -#DROP FUNCTION f; +statement ok +CREATE FUNCTION f() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ BEGIN RETURN NULL; END $$; + +statement error pgcode 0A000 pq: trigger functions can only be called as triggers +SELECT f(); + +statement error pgcode 0A000 pq: trigger functions can only be called as triggers +CREATE FUNCTION foo() RETURNS INT LANGUAGE SQL AS $$ SELECT f(); SELECT 1; $$; + +statement error pgcode 0A000 pq: trigger functions can only be called as triggers +CREATE FUNCTION foo() RETURNS INT LANGUAGE PLpgSQL AS $$ BEGIN SELECT f(); RETURN 1; END $$; + +statement ok +DROP FUNCTION f; # ============================================================================== # Test invalid usage of parameters in trigger functions. @@ -157,4 +152,304 @@ CREATE TYPE udt AS (x INT, y TRIGGER, z TEXT); statement error pgcode 42601 pq: at or near "\[": syntax error CREATE TYPE udt AS (x INT, y TRIGGER[], z TEXT); +# ============================================================================== +# Trigger functions support basic PL/pgSQL statements. +# ============================================================================== + +subtest basic_plpgsql + +# RETURN statement. +statement ok +CREATE FUNCTION f() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ BEGIN RETURN ROW(1, 2); END $$; + +statement ok +DROP FUNCTION f; + +# Variable declaration and assignment. +statement ok +CREATE FUNCTION f() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ + DECLARE + x INT := 1; + y INT; + BEGIN + y := 2; + RETURN NULL; + END +$$; + +statement ok +DROP FUNCTION f; + +# RAISE statement. +statement ok +CREATE FUNCTION f() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ BEGIN RAISE NOTICE 'hello'; RETURN NULL; END $$; + +statement ok +DROP FUNCTION f; + +# IF statement. +statement ok +CREATE FUNCTION f() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ + BEGIN + IF now() > '2021-07-12 09:02:10-08:00'::TIMESTAMPTZ THEN + RETURN NULL; + ELSE + RETURN ROW(1, 2, 3); + END IF; + END +$$; + +statement ok +DROP FUNCTION f; + +# WHILE statement. +statement ok +CREATE FUNCTION f() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ + DECLARE + x INT := 0; + BEGIN + WHILE x < 10 LOOP + x := x + 1; + END LOOP; + RETURN ROW(x); + END +$$; + +statement ok +DROP FUNCTION f; + +# OPEN and FETCH statements. +statement ok +CREATE FUNCTION f() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ + DECLARE + c CURSOR FOR SELECT 1; + x INT; + BEGIN + OPEN c; + FETCH c INTO x; + CLOSE c; + RETURN ROW(x); + END +$$; + +statement ok +DROP FUNCTION f; + +# Combination of statements. +statement ok +CREATE FUNCTION f() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ + DECLARE + x INT := 1; + y INT := 2; + BEGIN + RAISE NOTICE 'x: %, y: %', x, y; + IF x = 1 THEN + RETURN ROW(1, 2); + ELSE + RETURN ROW(3, 4); + END IF; + END +$$; + +statement ok +DROP FUNCTION f; + +# ============================================================================== +# Correct usage of PL/pgSQL statements is enforced at function creation. +# ============================================================================== + +subtest invalid_plpgsql + +# RETURN statement must return a row. +statement error pgcode 42601 pq: missing expression at or near "RETURN;" +CREATE FUNCTION f() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ BEGIN RETURN; END $$; + +# Assigning to a nonexistent variable is not allowed. +statement error pgcode 42601 pq: "nonexistent" is not a known variable +CREATE FUNCTION f() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ + BEGIN + nonexistent := 'foo'; + RAISE NOTICE '%', nonexistent; + RETURN NULL; + END +$$; + +# Cannot assign to a constant variable. +statement error pgcode 22005 pq: variable "x" is declared CONSTANT +CREATE FUNCTION f() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ + DECLARE + x CONSTANT INT := 1; + BEGIN + x := 2; + RETURN NULL; + END +$$; + +# Cursor cannot be opened with an INSERT statement. +statement error pgcode 42P11 pq: cannot open INSERT query as cursor +CREATE FUNCTION f() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ + DECLARE + c CURSOR FOR INSERT INTO t VALUES (1); + BEGIN + OPEN c; + RETURN NULL; + END +$$; + +# Transaction control statements are not allowed. +statement error pgcode 2D000 pq: invalid transaction termination +CREATE FUNCTION f() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ BEGIN COMMIT; RETURN NULL; END $$; + +# ============================================================================== +# Trigger functions have a set of implicitly-defined variables. +# ============================================================================== + +# It is possible to assign to the implicit variables, including OLD and NEW. +# TODO(#126727) The tg_op assignment is lower-cased because the INTO clause is +# currently case-sensitive. +statement ok +CREATE FUNCTION f() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ + BEGIN + TG_NAME := 'foo'; + SELECT t INTO tg_op FROM ops_table; + OLD := ROW(1, 2, 3); + NEW := (SELECT * FROM xyz LIMIT 1); + RETURN NEW; + END +$$; + +statement ok +DROP FUNCTION f; + +# Shadowing the implicit variables is not allowed (tracked in #117508). +statement error pgcode 0A000 pq: unimplemented: variable shadowing is not yet implemented +CREATE FUNCTION f() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ + DECLARE + tg_op TEXT := 'foo'; + BEGIN + RETURN NEW; + END +$$; + +# ============================================================================== +# SQL expressions are not analyzed during function creation. +# ============================================================================== + +subtest lazy_analysis + +# Arbitrary variables/columns (and fields of those variables) may be referenced +# in an unbound PL/pgSQL trigger function, even if they do not exist. +statement ok +CREATE FUNCTION f() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ + DECLARE + foo INT := NEW.x; + BEGIN + RAISE NOTICE '%', NEW.this_field_may_not_exist; + RAISE NOTICE '%', OLD.we_do_not_now_until_trigger_creation; + RETURN OLD.y + foo; + END +$$; + +statement ok +DROP FUNCTION f; + +# Arbitrary relations may be referenced in an unbound PL/pgSQL trigger function, +# even if they do not exist. +statement ok +CREATE FUNCTION f() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ + DECLARE + foo INT := (SELECT x FROM new_rows LIMIT 1); + BEGIN + RAISE NOTICE 'bar: %', (SELECT one, two FROM non_existent_table); + RETURN (SELECT y FROM old_rows LIMIT 1) + foo; + END +$$; + +statement ok +DROP FUNCTION f; + +# SQL statements must still have correct syntax. +statement error pgcode 42601 pq: at or near ";": at or near "sel": syntax error +CREATE FUNCTION f() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ + BEGIN + SEL y FROM old_rows LIMIT 1; + RETURN foo; + END +$$; + +# ============================================================================== +# Test CREATE OR REPLACE behavior for trigger functions. +# ============================================================================== + +subtest create_or_replace + +statement ok +CREATE FUNCTION f() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ BEGIN RETURN NULL; END $$; + +statement ok +CREATE OR REPLACE FUNCTION f() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ BEGIN RETURN NULL; END $$; + +# The first function should have been replaced. +query T +SELECT create_statement FROM [SHOW CREATE FUNCTION f]; +---- +CREATE FUNCTION public.f() + RETURNS TRIGGER + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE plpgsql + SECURITY INVOKER + AS $$ + BEGIN + RETURN NULL; + END; +$$ + +statement ok +CREATE OR REPLACE FUNCTION f() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ BEGIN RETURN ROW(1, 2); END $$; + +# The replacement function should have a different body. +query T +SELECT create_statement FROM [SHOW CREATE FUNCTION f]; +---- +CREATE FUNCTION public.f() + RETURNS TRIGGER + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE plpgsql + SECURITY INVOKER + AS $$ + BEGIN + RETURN (1, 2); + END; +$$ + +statement ok +DROP FUNCTION f; + +# CREATE OR REPLACE should succeed when there is no existing function. +statement ok +CREATE OR REPLACE FUNCTION f() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ BEGIN RETURN NULL; END $$; + +query T +SELECT create_statement FROM [SHOW CREATE FUNCTION f]; +---- +CREATE FUNCTION public.f() + RETURNS TRIGGER + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE plpgsql + SECURITY INVOKER + AS $$ + BEGIN + RETURN NULL; + END; +$$ + +statement ok +DROP FUNCTION f; + subtest end diff --git a/pkg/sql/create_function.go b/pkg/sql/create_function.go index 033f0d843f55..00b1a75cd167 100644 --- a/pkg/sql/create_function.go +++ b/pkg/sql/create_function.go @@ -587,17 +587,25 @@ func setFuncOptions( } if lang != catpb.Function_UNKNOWN_LANGUAGE && body != "" { - // Replace any sequence names in the function body with IDs. - seqReplacedFuncBody, err := replaceSeqNamesWithIDsLang(params.ctx, params.p, body, true, lang) - if err != nil { - return err - } - typeReplacedFuncBody, err := serializeUserDefinedTypesLang( - params.ctx, params.p.SemaCtx(), seqReplacedFuncBody, true /* multiStmt */, "UDFs", lang) - if err != nil { - return err + // Trigger functions do not analyze SQL statements beyond parsing, so type + // and sequence names should not be replaced during trigger-function + // creation. + returnType := udfDesc.ReturnType.Type + lazilyEvalSQL := returnType != nil && returnType.Identical(types.Trigger) + if !lazilyEvalSQL { + // Replace any sequence names in the function body with IDs. + body, err = replaceSeqNamesWithIDsLang(params.ctx, params.p, body, true, lang) + if err != nil { + return err + } + // Replace any UDT names in the function body with IDs. + body, err = serializeUserDefinedTypesLang( + params.ctx, params.p.SemaCtx(), body, true /* multiStmt */, "UDFs", lang) + if err != nil { + return err + } } - udfDesc.SetFuncBody(typeReplacedFuncBody) + udfDesc.SetFuncBody(body) } return nil } diff --git a/pkg/sql/opt/optbuilder/create_function.go b/pkg/sql/opt/optbuilder/create_function.go index 94b042e1ecae..30f4b5bfcef7 100644 --- a/pkg/sql/opt/optbuilder/create_function.go +++ b/pkg/sql/opt/optbuilder/create_function.go @@ -417,9 +417,6 @@ func (b *Builder) buildCreateFunction(cf *tree.CreateRoutine, inScope *scope) (o "set-returning PL/pgSQL functions are not yet supported", )) } - if funcReturnType.Identical(types.Trigger) { - panic(unimplemented.NewWithIssue(126356, "trigger functions are not yet supported")) - } // Parse the function body. stmt, err := plpgsqlparser.Parse(funcBodyStr) @@ -439,13 +436,38 @@ func (b *Builder) buildCreateFunction(cf *tree.CreateRoutine, inScope *scope) (o } } + // Special handling for trigger functions. + buildSQL := true + if funcReturnType.Identical(types.Trigger) { + // Trigger functions cannot have user-defined parameters. However, they do + // have a set of implicitly defined parameters. + for i := range createTriggerFuncParams { + param := &createTriggerFuncParams[i] + paramColName := funcParamColName(param.name, i) + col := b.synthesizeColumn( + bodyScope, paramColName, param.typ, nil /* expr */, nil, /* scalar */ + ) + col.setParamOrd(i) + } + routineParams = createTriggerFuncParams + + // The actual return type for a trigger function is not known until it is + // bound to a trigger. Therefore, during function creation we use NULL as a + // placeholder type. + funcReturnType = types.Unknown + + // Analysis of SQL expressions for trigger functions must be deferred + // until the function is bound to a trigger. + buildSQL = false + } + // We need to disable stable function folding because we want to catch the // volatility of stable functions. If folded, we only get a scalar and lose // the volatility. b.factory.FoldingControl().TemporarilyDisallowStableFolds(func() { plBuilder := newPLpgSQLBuilder( - b, cf.Name.Object(), stmt.AST.Label, nil, /* colRefs */ - routineParams, funcReturnType, cf.IsProcedure, nil, /* outScope */ + b, cf.Name.Object(), stmt.AST.Label, nil /* colRefs */, routineParams, + funcReturnType, cf.IsProcedure, buildSQL, nil, /* outScope */ ) stmtScope = plBuilder.buildRootBlock(stmt.AST, bodyScope, routineParams) }) @@ -500,6 +522,15 @@ func (b *Builder) buildCreateFunction(cf *tree.CreateRoutine, inScope *scope) (o return outScope } +// createTriggerFuncParams is the set of implicitly-defined parameters for a +// PL/pgSQL trigger function. createTriggerFuncParams is used during trigger +// function creation, when the type of the NEW and OLD variables is not yet +// known. +var createTriggerFuncParams = append([]routineParam{ + {name: "new", typ: types.Unknown, class: tree.RoutineParamIn}, + {name: "old", typ: types.Unknown, class: tree.RoutineParamIn}, +}, triggerFuncStaticParams...) + func formatFuncBodyStmt( fmtCtx *tree.FmtCtx, ast tree.NodeFormatter, lang tree.RoutineLanguage, newLine bool, ) { diff --git a/pkg/sql/opt/optbuilder/plpgsql.go b/pkg/sql/opt/optbuilder/plpgsql.go index 948fc6516104..0aa095fce980 100644 --- a/pkg/sql/opt/optbuilder/plpgsql.go +++ b/pkg/sql/opt/optbuilder/plpgsql.go @@ -130,6 +130,23 @@ import ( // effects, such as pushing a volatile expression into a join or union. // See addBarrierIfVolatile for more information. // +// +---------------------+ +// | Lazy SQL Evaluation | +// +---------------------+ +// +// Trigger functions are created before they are associated with a particular +// table by a CREATE TRIGGER statement. This means that column references within +// SQL statements and expressions cannot be resolved when the trigger function +// is created. However, it is still possible (and desirable) to validate the +// PL/pgSQL code at this time. +// +// In order to validate the PL/pgSQL during the creation of a trigger function +// without analyzing SQL statements, we replace: +// - SQL expressions with typed NULL values, and +// - SQL statements by a single-row VALUES operator with no columns. +// +// See also the buildSQLExpr and buildSQLStatement methods. +// // +-----------------+ // | Further Reading | // +-----------------+ @@ -176,6 +193,7 @@ type plpgsqlBuilder struct { routineName string isProcedure bool + buildSQL bool identCounter int } @@ -193,6 +211,7 @@ func newPLpgSQLBuilder( routineParams []routineParam, returnType *types.T, isProcedure bool, + buildSQL bool, outScope *scope, ) *plpgsqlBuilder { const initialBlocksCap = 2 @@ -203,6 +222,7 @@ func newPLpgSQLBuilder( blocks: make([]plBlock, 0, initialBlocksCap), routineName: routineName, isProcedure: isProcedure, + buildSQL: buildSQL, outScope: outScope, } // Build the initial block for the routine parameters, which are considered @@ -469,7 +489,7 @@ func (b *plpgsqlBuilder) buildPLpgSQLStatements(stmts []ast.Statement, s *scope) } // RETURN is handled by projecting a single column with the expression // that is being returned. - returnScalar := b.buildPLpgSQLExpr(expr, b.returnType, s) + returnScalar := b.buildSQLExpr(expr, b.returnType, s) b.addBarrierIfVolatile(s, returnScalar) returnColName := scopeColName("").WithMetadataName(b.makeIdentifier("stmt_return")) returnScope := s.push() @@ -536,12 +556,12 @@ func (b *plpgsqlBuilder) buildPLpgSQLStatements(stmts []ast.Statement, s *scope) // Build a scalar CASE statement that conditionally executes each branch // of the IF statement as a subquery. - cond := b.buildPLpgSQLExpr(t.Condition, types.Bool, s) + cond := b.buildSQLExpr(t.Condition, types.Bool, s) thenScalar := b.ob.factory.ConstructSubquery(thenScope.expr, &memo.SubqueryPrivate{}) whens := make(memo.ScalarListExpr, 0, len(t.ElseIfList)+1) whens = append(whens, b.ob.factory.ConstructWhen(cond, thenScalar)) for j := range t.ElseIfList { - elsifCond := b.buildPLpgSQLExpr(t.ElseIfList[j].Condition, types.Bool, s) + elsifCond := b.buildSQLExpr(t.ElseIfList[j].Condition, types.Bool, s) elsifScalar := b.ob.factory.ConstructSubquery(elsifScopes[j].expr, &memo.SubqueryPrivate{}) whens = append(whens, b.ob.factory.ConstructWhen(elsifCond, elsifScalar)) } @@ -702,7 +722,7 @@ func (b *plpgsqlBuilder) buildPLpgSQLStatements(stmts []ast.Statement, s *scope) // Create a new continuation routine to handle executing a SQL statement. execCon := b.makeContinuation("_stmt_exec") - stmtScope := b.ob.buildStmtAtRootWithScope(t.SqlStmt, nil /* desiredTypes */, execCon.s) + stmtScope := b.buildSQLStatement(t.SqlStmt, execCon.s) if len(t.Target) == 0 { // When there is no INTO target, build the SQL statement into a body // statement that is only executed for its side effects. @@ -799,7 +819,7 @@ func (b *plpgsqlBuilder) buildPLpgSQLStatements(stmts []ast.Statement, s *scope) Scroll: t.Scroll, CursorSQL: fmtCtx.CloseAndGetString(), } - openScope := b.ob.buildStmtAtRootWithScope(query, nil /* desiredTypes */, openCon.s) + openScope := b.buildSQLStatement(query, openCon.s) if openScope.expr.Relational().CanMutate { // Cursors with mutations are invalid. panic(cursorMutationErr) @@ -1120,7 +1140,7 @@ func (b *plpgsqlBuilder) addPLpgSQLAssign(inScope *scope, ident ast.Variable, va // volatile, add barriers before and after the projection to prevent optimizer // rules from reordering or removing its side effects. colName := scopeColName(ident) - scalar := b.buildPLpgSQLExpr(val, typ, inScope) + scalar := b.buildSQLExpr(val, typ, inScope) b.addBarrierIfVolatile(inScope, scalar) b.ob.synthesizeColumn(assignScope, colName, typ, nil, scalar) b.ob.constructProjectForScope(inScope, assignScope) @@ -1253,7 +1273,7 @@ func (b *plpgsqlBuilder) getRaiseArgs(s *scope, raise *ast.Raise) memo.ScalarLis if isDup { panic(pgerror.Newf(pgcode.Syntax, "RAISE option already specified: %s", name)) } - return b.buildPLpgSQLExpr(expr, types.String, s) + return b.buildSQLExpr(expr, types.String, s) } for _, option := range raise.Options { optName := strings.ToUpper(option.OptType) @@ -1345,7 +1365,7 @@ func (b *plpgsqlBuilder) makeRaiseFormatMessage( } // If the argument is NULL, postgres prints "". expr := &tree.CastExpr{Expr: args[argIdx], Type: types.String} - arg := b.buildPLpgSQLExpr(expr, types.String, s) + arg := b.buildSQLExpr(expr, types.String, s) arg = b.ob.factory.ConstructCoalesce(memo.ScalarListExpr{arg, makeConstStr("")}) addToResult(arg) argIdx++ @@ -1495,7 +1515,7 @@ func (b *plpgsqlBuilder) handleEndOfFunction(inScope *scope) *scope { } returnScope := inScope.push() colName := scopeColName("_implicit_return") - returnScalar := b.buildPLpgSQLExpr(returnExpr, b.returnType, inScope) + returnScalar := b.buildSQLExpr(returnExpr, b.returnType, inScope) b.ob.synthesizeColumn(returnScope, colName, b.returnType, nil /* expr */, returnScalar) b.ob.constructProjectForScope(inScope, returnScope) return returnScope @@ -1872,9 +1892,13 @@ func (b *plpgsqlBuilder) addBarrier(s *scope) { s.expr = b.ob.factory.ConstructBarrier(s.expr) } -// buildPLpgSQLExpr parses and builds the given SQL expression into a ScalarExpr -// within the given scope. -func (b *plpgsqlBuilder) buildPLpgSQLExpr(expr ast.Expr, typ *types.T, s *scope) opt.ScalarExpr { +// buildSQLExpr type-checks and builds the given SQL expression into a +// ScalarExpr within the given scope. +func (b *plpgsqlBuilder) buildSQLExpr(expr ast.Expr, typ *types.T, s *scope) opt.ScalarExpr { + if !b.buildSQL { + // For lazy SQL evaluation, replace all expressions with NULL. + return memo.NullSingleton + } expr, _ = tree.WalkExpr(s, expr) typedExpr, err := expr.TypeCheck(b.ob.ctx, b.ob.semaCtx, typ) if err != nil { @@ -1884,6 +1908,19 @@ func (b *plpgsqlBuilder) buildPLpgSQLExpr(expr ast.Expr, typ *types.T, s *scope) return b.coerceType(scalar, typ) } +// buildSQLStatement type-checks and builds the given SQL statement into a +// RelExpr within the given scope. +func (b *plpgsqlBuilder) buildSQLStatement(stmt tree.Statement, inScope *scope) (outScope *scope) { + if !b.buildSQL { + // For lazy SQL evaluation, replace all statements with a single row without + // any columns. + outScope = inScope.push() + outScope.expr = b.ob.factory.ConstructNoColsRow() + return outScope + } + return b.ob.buildStmtAtRootWithScope(stmt, nil /* desiredTypes */, inScope) +} + // coerceType implements PLpgSQL type-coercion behavior. func (b *plpgsqlBuilder) coerceType(scalar opt.ScalarExpr, typ *types.T) opt.ScalarExpr { resolved := scalar.DataType() diff --git a/pkg/sql/opt/optbuilder/routine.go b/pkg/sql/opt/optbuilder/routine.go index bb661e970ed4..5f89ff5ff083 100644 --- a/pkg/sql/opt/optbuilder/routine.go +++ b/pkg/sql/opt/optbuilder/routine.go @@ -431,7 +431,8 @@ func (b *Builder) buildRoutine( var expr memo.RelExpr var physProps *physical.Required plBuilder := newPLpgSQLBuilder( - b, def.Name, stmt.AST.Label, colRefs, routineParams, f.ResolvedType(), isProc, outScope, + b, def.Name, stmt.AST.Label, colRefs, routineParams, f.ResolvedType(), + isProc, true /* buildSQL */, outScope, ) stmtScope := plBuilder.buildRootBlock(stmt.AST, bodyScope, routineParams) expr, physProps = b.finishBuildLastStmt( @@ -789,3 +790,19 @@ func (b *Builder) withinNestedPLpgSQLCall(fn func()) { b.insideNestedPLpgSQLCall = true fn() } + +// triggerFuncStaticParams is the set of implicitly-defined parameters for a +// PL/pgSQL trigger function, excluding the NEW and OLD parameters which are +// determined by the table when a trigger is created. +var triggerFuncStaticParams = []routineParam{ + {name: "tg_name", typ: types.Name, class: tree.RoutineParamIn}, + {name: "tg_when", typ: types.String, class: tree.RoutineParamIn}, + {name: "tg_level", typ: types.String, class: tree.RoutineParamIn}, + {name: "tg_op", typ: types.String, class: tree.RoutineParamIn}, + {name: "tg_relid", typ: types.Oid, class: tree.RoutineParamIn}, + {name: "tg_relname", typ: types.Name, class: tree.RoutineParamIn}, + {name: "tg_table_name", typ: types.Name, class: tree.RoutineParamIn}, + {name: "tg_table_schema", typ: types.Name, class: tree.RoutineParamIn}, + {name: "tg_nargs", typ: types.Int, class: tree.RoutineParamIn}, + {name: "tg_argv", typ: types.StringArray, class: tree.RoutineParamIn}, +} diff --git a/pkg/sql/schemachanger/scbuild/builder_state.go b/pkg/sql/schemachanger/scbuild/builder_state.go index 269367b66e22..21a79ba56225 100644 --- a/pkg/sql/schemachanger/scbuild/builder_state.go +++ b/pkg/sql/schemachanger/scbuild/builder_state.go @@ -1587,10 +1587,21 @@ func (b *builderState) WrapFunctionBody( fnID descpb.ID, bodyStr string, lang catpb.Function_Language, + returnType tree.ResolvableTypeReference, refProvider scbuildstmt.ReferenceProvider, ) *scpb.FunctionBody { - bodyStr = b.replaceSeqNamesWithIDs(bodyStr, lang) - bodyStr = b.serializeUserDefinedTypes(bodyStr, lang) + // Trigger functions do not analyze SQL statements beyond parsing, so type and + // sequence names should not be replaced during trigger-function creation. + var lazilyEvalSQL bool + if returnType != nil { + if typ, ok := returnType.(*types.T); ok && typ.Identical(types.Trigger) { + lazilyEvalSQL = true + } + } + if !lazilyEvalSQL { + bodyStr = b.replaceSeqNamesWithIDs(bodyStr, lang) + bodyStr = b.serializeUserDefinedTypes(bodyStr, lang) + } fnBody := &scpb.FunctionBody{ FunctionID: fnID, Body: bodyStr, diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_function.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_function.go index b95ec0d072da..ea9fefd14ee9 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_function.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_function.go @@ -199,7 +199,7 @@ func CreateFunction(b BuildCtx, n *tree.CreateRoutine) { validateTypeReferences(b, refProvider, db.DatabaseID) validateFunctionRelationReferences(b, refProvider, db.DatabaseID) validateFunctionToFunctionReferences(b, refProvider, db.DatabaseID) - b.Add(b.WrapFunctionBody(fnID, fnBodyStr, lang, refProvider)) + b.Add(b.WrapFunctionBody(fnID, fnBodyStr, lang, typ, refProvider)) b.LogEventForExistingTarget(&fn) } diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/dependencies.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/dependencies.go index 575ef9a1bd1c..324e39ee9d70 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/dependencies.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/dependencies.go @@ -319,7 +319,8 @@ type TableHelpers interface { type FunctionHelpers interface { BuildReferenceProvider(stmt tree.Statement) ReferenceProvider - WrapFunctionBody(fnID descpb.ID, bodyStr string, lang catpb.Function_Language, provider ReferenceProvider) *scpb.FunctionBody + WrapFunctionBody(fnID descpb.ID, bodyStr string, lang catpb.Function_Language, + returnType tree.ResolvableTypeReference, provider ReferenceProvider) *scpb.FunctionBody } type SchemaHelpers interface { From 1599165b94aeb3779226d0060dfcb2208bbe85c2 Mon Sep 17 00:00:00 2001 From: Jackson Owens Date: Wed, 25 Sep 2024 11:33:35 -0400 Subject: [PATCH 2/3] storage: remove WAL failover enterprise license check The license check may be performed before the node has fully initialized and knows whether a valid enterprise license is available. A warning may be logged erroneously, which may confuse. Remove the license check altogether given the direction CockroachDB enterprise licensing is moving in general. Epic: none Informs #129240. Release note: none --- pkg/storage/open.go | 9 +-------- pkg/storage/open_test.go | 5 ----- pkg/storage/testdata/wal_failover_config | 8 -------- 3 files changed, 1 insertion(+), 21 deletions(-) diff --git a/pkg/storage/open.go b/pkg/storage/open.go index 7a4b2fbc015f..1e14cddd4c9b 100644 --- a/pkg/storage/open.go +++ b/pkg/storage/open.go @@ -22,7 +22,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage/disk" "github.com/cockroachdb/cockroach/pkg/storage/fs" - "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" "github.com/cockroachdb/pebble" "github.com/cockroachdb/pebble/vfs" @@ -447,7 +446,6 @@ func WALFailover( func makePebbleWALFailoverOptsForDir( settings *cluster.Settings, dir wal.Dir, ) *pebble.WALFailoverOptions { - cclWALFailoverLogEvery := log.Every(10 * time.Minute) return &pebble.WALFailoverOptions{ Secondary: dir, FailoverOptions: wal.FailoverOptions{ @@ -455,12 +453,7 @@ func makePebbleWALFailoverOptsForDir( // UnhealthyOperationLatencyThreshold should be pulled from the // cluster setting. UnhealthyOperationLatencyThreshold: func() (time.Duration, bool) { - // WAL failover is a licensed feature. - licenseOK := base.CCLDistributionAndEnterpriseEnabled(settings) - if !licenseOK && cclWALFailoverLogEvery.ShouldLog() { - log.Warningf(context.Background(), "Ignoring WAL failover configuration because it requires an enterprise license.") - } - return walFailoverUnhealthyOpThreshold.Get(&settings.SV), licenseOK + return walFailoverUnhealthyOpThreshold.Get(&settings.SV), true }, }, } diff --git a/pkg/storage/open_test.go b/pkg/storage/open_test.go index 920f86ba1cf5..5854bceda133 100644 --- a/pkg/storage/open_test.go +++ b/pkg/storage/open_test.go @@ -128,11 +128,6 @@ func TestWALFailover(t *testing.T) { } settings := cluster.MakeTestingClusterSettingsWithVersions(version, version, true /* initializeVersion */) - // Mock an enterpise license, or not if disable-enterprise is specified. - enterpriseEnabledFunc := base.CCLDistributionAndEnterpriseEnabled - base.CCLDistributionAndEnterpriseEnabled = func(st *cluster.Settings) bool { return !td.HasArg("disable-enterprise") } - defer func() { base.CCLDistributionAndEnterpriseEnabled = enterpriseEnabledFunc }() - engine, err := Open(context.Background(), openEnv, settings, WALFailover(cfg, envs, defaultFS, nil)) if err != nil { openEnv.Close() diff --git a/pkg/storage/testdata/wal_failover_config b/pkg/storage/testdata/wal_failover_config index 5a75523e9dfe..31918e7e9cab 100644 --- a/pkg/storage/testdata/wal_failover_config +++ b/pkg/storage/testdata/wal_failover_config @@ -90,14 +90,6 @@ open flag=disabled envs=(foo,bar) open=bar OK WALRecoveryDir: foo/auxiliary/wals-among-stores -# Ensure that WAL failover refuses to failover if there's no enterprise -# license configured. -open flag=among-stores envs=(foo,bar) open=foo disable-enterprise ----- -OK -secondary = bar/auxiliary/wals-among-stores -UnhealthyOperationLatencyThreshold() = (100ms,false) - open flag=disabled envs=(foo,bar) open=foo ---- OK From 8a3158fe7b1293022dd3044beb7d7c00efb736be Mon Sep 17 00:00:00 2001 From: Bergin Dedej Date: Wed, 25 Sep 2024 12:47:27 -0400 Subject: [PATCH 3/3] pkg/ccl/testccl/sqlccl/sqlccl_test: TestShowTransferState skip flake These test has been flaking a couple of times. Adding skip.WithIssue(t, 128125) to skip this test, until it is properly fixed. Informs: #128125 Release note: None --- pkg/ccl/testccl/sqlccl/show_transfer_state_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pkg/ccl/testccl/sqlccl/show_transfer_state_test.go b/pkg/ccl/testccl/sqlccl/show_transfer_state_test.go index 385e392bf277..7ce456723a41 100644 --- a/pkg/ccl/testccl/sqlccl/show_transfer_state_test.go +++ b/pkg/ccl/testccl/sqlccl/show_transfer_state_test.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -28,6 +29,7 @@ import ( func TestShowTransferState(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + skip.WithIssue(t, 128125) ctx := context.Background() s, mainDB, _ := serverutils.StartServer(t, base.TestServerArgs{