Skip to content

Commit 1be4e48

Browse files
committed
sql,*: record UDF/SP stats
Updates UDF and SP execution to support recording statement statistics and insights. Now, each sub statement executed in a UDF and SP will have latencies and execution stats stored in the sql stats subsystem. Additionally, these statements will be appear in the transaction statistics details page in table containing "statement fingerprints for this transaction". Fixes: CRDB-57025 Epic: CRDB-55081 Release note (sql change): SQL Statements executed in stored procedures and user defined functions will now have sql stats recorded and should appear in the sql activity and insights pages in db console. Limitations: statements executed in a UDF / SP won't be collected for active statement diagnostics requests. Statement diagnostics continue to only work for top level statement executions
1 parent 29d8485 commit 1be4e48

File tree

14 files changed

+230
-31
lines changed

14 files changed

+230
-31
lines changed

pkg/sql/conn_executor.go

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -1311,8 +1311,6 @@ func (s *Server) newConnExecutor(
13111311

13121312
ex.applicationName.Store(ex.sessionData().ApplicationName)
13131313
ex.applicationStats = applicationStats
1314-
// We ignore statements and transactions run by the internal executor by
1315-
// passing a nil writer.
13161314
ex.statsCollector = sslocal.NewStatsCollector(
13171315
s.cfg.Settings,
13181316
applicationStats,
@@ -4020,7 +4018,7 @@ func (ex *connExecutor) initPlanner(ctx context.Context, p *planner) {
40204018
p.cancelChecker.Reset(ctx)
40214019

40224020
ex.initEvalCtx(ctx, &p.extendedEvalCtx, p)
4023-
4021+
p.statsCollector = ex.statsCollector
40244022
p.sessionDataMutatorIterator = ex.dataMutatorIterator
40254023
p.noticeSender = nil
40264024
p.preparedStatements = ex.getPrepStmtsAccessor()

pkg/sql/opt/exec/execbuilder/BUILD.bazel

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@ go_library(
1717
deps = [
1818
"//pkg/kv/kvserver/concurrency/isolation",
1919
"//pkg/server/telemetry",
20+
"//pkg/sql/appstatspb",
2021
"//pkg/sql/catalog/colinfo",
2122
"//pkg/sql/catalog/descpb",
2223
"//pkg/sql/lexbase",
@@ -45,6 +46,7 @@ go_library(
4546
"//pkg/sql/sem/tree/treewindow",
4647
"//pkg/sql/sem/volatility",
4748
"//pkg/sql/sqlerrors",
49+
"//pkg/sql/sqlstats",
4850
"//pkg/sql/sqltelemetry",
4951
"//pkg/sql/types",
5052
"//pkg/util/buildutil",

pkg/sql/opt/exec/execbuilder/relational.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3731,6 +3731,7 @@ func (b *Builder) buildCall(c *memo.CallExpr) (_ execPlan, outputCols colOrdMap,
37313731
udf.Def.BodyProps,
37323732
udf.Def.BodyStmts,
37333733
udf.Def.BodyTags,
3734+
udf.Def.BodyASTs,
37343735
false, /* allowOuterWithRefs */
37353736
nil, /* wrapRootExpr */
37363737
0, /* resultBufferID */

pkg/sql/opt/exec/execbuilder/scalar.go

Lines changed: 46 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -8,8 +8,10 @@ package execbuilder
88
import (
99
"context"
1010

11+
"github.com/cockroachdb/cockroach/pkg/sql/appstatspb"
1112
"github.com/cockroachdb/cockroach/pkg/sql/opt"
1213
"github.com/cockroachdb/cockroach/pkg/sql/opt/exec"
14+
"github.com/cockroachdb/cockroach/pkg/sql/opt/exec/explain"
1315
"github.com/cockroachdb/cockroach/pkg/sql/opt/memo"
1416
"github.com/cockroachdb/cockroach/pkg/sql/opt/norm"
1517
"github.com/cockroachdb/cockroach/pkg/sql/opt/props"
@@ -24,6 +26,7 @@ import (
2426
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree/treecmp"
2527
"github.com/cockroachdb/cockroach/pkg/sql/sem/volatility"
2628
"github.com/cockroachdb/cockroach/pkg/sql/sqlerrors"
29+
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats"
2730
"github.com/cockroachdb/cockroach/pkg/sql/types"
2831
"github.com/cockroachdb/cockroach/pkg/util/errorutil"
2932
"github.com/cockroachdb/cockroach/pkg/util/log"
@@ -695,6 +698,7 @@ func (b *Builder) buildExistsSubquery(
695698
stmtProps,
696699
nil, /* stmtStr */
697700
make([]string, len(stmts)),
701+
nil, /* stmtASTs */
698702
true, /* allowOuterWithRefs */
699703
wrapRootExpr,
700704
0, /* resultBufferID */
@@ -822,6 +826,7 @@ func (b *Builder) buildSubquery(
822826
stmtProps,
823827
nil, /* stmtStr */
824828
make([]string, len(stmts)),
829+
nil, /* stmtASTs */
825830
true, /* allowOuterWithRefs */
826831
nil, /* wrapRootExpr */
827832
0, /* resultBufferID */
@@ -900,7 +905,7 @@ func (b *Builder) buildSubquery(
900905
if err != nil {
901906
return err
902907
}
903-
err = fn(plan, "" /* stmtForDistSQLDiagram */, true /* isFinalPlan */)
908+
err = fn(plan, nil /* routineStatsBuilder */, "" /* stmtForDistSQLDiagram */, true /* isFinalPlan */)
904909
if err != nil {
905910
return err
906911
}
@@ -1017,6 +1022,7 @@ func (b *Builder) buildUDF(ctx *buildScalarCtx, scalar opt.ScalarExpr) (tree.Typ
10171022
udf.Def.BodyProps,
10181023
udf.Def.BodyStmts,
10191024
udf.Def.BodyTags,
1025+
udf.Def.BodyASTs,
10201026
false, /* allowOuterWithRefs */
10211027
nil, /* wrapRootExpr */
10221028
udf.Def.ResultBufferID,
@@ -1091,6 +1097,7 @@ func (b *Builder) initRoutineExceptionHandler(
10911097
action.BodyProps,
10921098
action.BodyStmts,
10931099
action.BodyTags,
1100+
nil, /* stmtASTs */
10941101
false, /* allowOuterWithRefs */
10951102
nil, /* wrapRootExpr */
10961103
0, /* resultBufferID */
@@ -1141,6 +1148,7 @@ func (b *Builder) buildRoutinePlanGenerator(
11411148
stmtProps []*physical.Required,
11421149
stmtStr []string,
11431150
stmtTags []string,
1151+
stmtASTs []tree.Statement,
11441152
allowOuterWithRefs bool,
11451153
wrapRootExpr wrapRootExprFn,
11461154
resultBufferID memo.RoutineResultBufferID,
@@ -1174,6 +1182,8 @@ func (b *Builder) buildRoutinePlanGenerator(
11741182
//
11751183
// Note: we put o outside of the function so we allocate it only once.
11761184
var o xform.Optimizer
1185+
var gistFactory explain.PlanGistFactory
1186+
var latencyRecorder = sqlstats.NewStatementLatencyRecorder()
11771187
originalMemo := b.mem
11781188
planGen := func(
11791189
ctx context.Context,
@@ -1206,8 +1216,14 @@ func (b *Builder) buildRoutinePlanGenerator(
12061216

12071217
dbName := b.evalCtx.SessionData().Database
12081218
appName := b.evalCtx.SessionData().ApplicationName
1209-
1219+
// TODO(yuzefovich): look into computing fingerprintFormat lazily.
1220+
fingerprintFormat := tree.FmtHideConstants | tree.FmtFlags(tree.QueryFormattingForFingerprintsMask.Get(&b.evalCtx.Settings.SV))
12101221
for i := range stmts {
1222+
latencyRecorder.Reset()
1223+
var builder *sqlstats.RecordedStatementStatsBuilder
1224+
var statsBuilderWithLatencies tree.RoutineStatsBuilder
1225+
sqlstats.RecordStatementPhase(latencyRecorder, sqlstats.StatementStarted)
1226+
sqlstats.RecordStatementPhase(latencyRecorder, sqlstats.StatementStartParsing)
12111227
stmt := stmts[i]
12121228
props := stmtProps[i]
12131229
var tag string
@@ -1216,6 +1232,22 @@ func (b *Builder) buildRoutinePlanGenerator(
12161232
if i < len(stmtTags) {
12171233
tag = stmtTags[i]
12181234
}
1235+
if i < len(stmtASTs) && stmtASTs[i] != nil {
1236+
fingerprint := tree.FormatStatementHideConstants(stmtASTs[i], fingerprintFormat)
1237+
fpId := appstatspb.ConstructStatementFingerprintID(fingerprint, b.evalCtx.TxnImplicit, dbName)
1238+
summary := tree.FormatStatementSummary(stmtASTs[i], fingerprintFormat)
1239+
stmtType := stmtASTs[i].StatementType()
1240+
builder = sqlstats.NewRecordedStatementStatsBuilder(
1241+
fpId, dbName, fingerprint, summary, stmtType, appName,
1242+
)
1243+
1244+
statsBuilderWithLatencies = &sqlstats.StatsBuilderWithLatencyRecorder{
1245+
StatsBuilder: builder,
1246+
LatencyRecorder: latencyRecorder,
1247+
}
1248+
}
1249+
sqlstats.RecordStatementPhase(latencyRecorder, sqlstats.StatementEndParsing)
1250+
sqlstats.RecordStatementPhase(latencyRecorder, sqlstats.StatementStartPlanning)
12191251
o.Init(ctx, b.evalCtx, b.catalog)
12201252
f := o.Factory()
12211253

@@ -1287,9 +1319,14 @@ func (b *Builder) buildRoutinePlanGenerator(
12871319
tailCalls = make(map[opt.ScalarExpr]struct{})
12881320
memo.ExtractTailCalls(optimizedExpr, tailCalls)
12891321
}
1290-
12911322
// Build the memo into a plan.
12921323
ef := ref.(exec.Factory)
1324+
if builder != nil && !b.evalCtx.SessionData().DisablePlanGists {
1325+
gistFactory.Reset()
1326+
gistFactory.Init(ef)
1327+
ef = &gistFactory
1328+
}
1329+
12931330
eb := New(ctx, ef, &o, f.Memo(), b.catalog, optimizedExpr, b.semaCtx, b.evalCtx, false /* allowAutoCommit */, b.IsANSIDML)
12941331
eb.withExprs = withExprs
12951332
eb.disableTelemetry = true
@@ -1303,6 +1340,10 @@ func (b *Builder) buildRoutinePlanGenerator(
13031340
eb.addRoutineResultBuffer(resultBufferID, resultWriter)
13041341
}
13051342
plan, err := eb.Build()
1343+
if gistFactory.Initialized() {
1344+
planGist := gistFactory.PlanGist()
1345+
builder.PlanGist(planGist.String(), planGist.Hash())
1346+
}
13061347
if err != nil {
13071348
if errors.IsAssertionFailure(err) {
13081349
// Enhance the error with the EXPLAIN (OPT, VERBOSE) of the
@@ -1324,7 +1365,8 @@ func (b *Builder) buildRoutinePlanGenerator(
13241365
stmtForDistSQLDiagram = stmtStr[i]
13251366
}
13261367
incrementRoutineStmtCounter(b.evalCtx.StartedRoutineStatementCounters, dbName, appName, tag)
1327-
err = fn(plan, stmtForDistSQLDiagram, isFinalPlan)
1368+
sqlstats.RecordStatementPhase(latencyRecorder, sqlstats.StatementEndPlanning)
1369+
err = fn(plan, statsBuilderWithLatencies, stmtForDistSQLDiagram, isFinalPlan)
13281370
if err != nil {
13291371
return err
13301372
}

pkg/sql/opt/memo/expr.go

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -742,6 +742,11 @@ type UDFDefinition struct {
742742
// at the same position in Body.
743743
BodyProps []*physical.Required
744744

745+
// BodyASTs contains the AST representation of each statement in Body. The
746+
// size of this slice matches that of Body, but it may contain nil entries
747+
// for non tree.RoutineLangSQL types.
748+
BodyASTs []tree.Statement
749+
745750
// BodyStmts, if set, is the string representation of each statement in
746751
// Body. It is only populated when verbose tracing is enabled.
747752
BodyStmts []string

pkg/sql/opt/optbuilder/plpgsql.go

Lines changed: 24 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -678,7 +678,7 @@ func (b *plpgsqlBuilder) buildPLpgSQLStatements(stmts []ast.Statement, s *scope)
678678
if b.options.insideDataSource && b.setReturnType.Family() == types.TupleFamily {
679679
retNextScope = b.ob.expandRoutineTupleIntoCols(retNextScope)
680680
}
681-
b.appendBodyStmtFromScope(&retCon, retNextScope, "" /* stmtTag */)
681+
b.appendBodyStmtFromScope(&retCon, retNextScope, nil /* stmt */)
682682
b.appendPlpgSQLStmts(&retCon, stmts[i+1:])
683683
return b.callContinuation(&retCon, s)
684684

@@ -720,7 +720,7 @@ func (b *plpgsqlBuilder) buildPLpgSQLStatements(stmts []ast.Statement, s *scope)
720720
if !b.options.insideDataSource && b.setReturnType.Family() == types.TupleFamily {
721721
retQueryScope = b.ob.combineRoutineColsIntoTuple(retQueryScope)
722722
}
723-
b.appendBodyStmtFromScope(&retCon, retQueryScope, t.SqlStmt.StatementTag())
723+
b.appendBodyStmtFromScope(&retCon, retQueryScope, t.SqlStmt)
724724
b.appendPlpgSQLStmts(&retCon, stmts[i+1:])
725725
return b.callContinuation(&retCon, s)
726726

@@ -949,7 +949,7 @@ func (b *plpgsqlBuilder) buildPLpgSQLStatements(stmts []ast.Statement, s *scope)
949949
// crdb_internal.plpgsql_raise builtin function.
950950
con := b.makeContinuation("_stmt_raise")
951951
con.def.Volatility = volatility.Volatile
952-
b.appendBodyStmtFromScope(&con, b.buildPLpgSQLRaise(con.s, b.getRaiseArgs(con.s, t)), "" /* stmtTag */)
952+
b.appendBodyStmtFromScope(&con, b.buildPLpgSQLRaise(con.s, b.getRaiseArgs(con.s, t)), nil /* stmt */)
953953
b.appendPlpgSQLStmts(&con, stmts[i+1:])
954954
return b.callContinuation(&con, s)
955955

@@ -970,7 +970,7 @@ func (b *plpgsqlBuilder) buildPLpgSQLStatements(stmts []ast.Statement, s *scope)
970970
if len(t.Target) == 0 {
971971
// When there is no INTO target, build the SQL statement into a body
972972
// statement that is only executed for its side effects.
973-
b.appendBodyStmtFromScope(&execCon, stmtScope, t.SqlStmt.StatementTag())
973+
b.appendBodyStmtFromScope(&execCon, stmtScope, t.SqlStmt)
974974
b.appendPlpgSQLStmts(&execCon, stmts[i+1:])
975975
return b.callContinuation(&execCon, s)
976976
}
@@ -1028,7 +1028,7 @@ func (b *plpgsqlBuilder) buildPLpgSQLStatements(stmts []ast.Statement, s *scope)
10281028
intoScope = b.callContinuation(&retCon, intoScope)
10291029

10301030
// Step 3: call the INTO continuation from the parent scope.
1031-
b.appendBodyStmtFromScope(&execCon, intoScope, t.SqlStmt.StatementTag())
1031+
b.appendBodyStmtFromScope(&execCon, intoScope, t.SqlStmt)
10321032
return b.callContinuation(&execCon, s)
10331033

10341034
case *ast.Open:
@@ -1068,7 +1068,7 @@ func (b *plpgsqlBuilder) buildPLpgSQLStatements(stmts []ast.Statement, s *scope)
10681068
// Cursors with mutations are invalid.
10691069
panic(cursorMutationErr)
10701070
}
1071-
b.appendBodyStmtFromScope(&openCon, openScope, query.StatementTag())
1071+
b.appendBodyStmtFromScope(&openCon, openScope, query)
10721072
b.appendPlpgSQLStmts(&openCon, stmts[i+1:])
10731073

10741074
// Build a statement to generate a unique name for the cursor if one
@@ -1078,7 +1078,7 @@ func (b *plpgsqlBuilder) buildPLpgSQLStatements(stmts []ast.Statement, s *scope)
10781078
nameCon := b.makeContinuation("_gen_cursor_name")
10791079
nameCon.def.Volatility = volatility.Volatile
10801080
nameScope := b.buildCursorNameGen(&nameCon, t.CurVar)
1081-
b.appendBodyStmtFromScope(&nameCon, b.callContinuation(&openCon, nameScope), "" /* stmtTag */)
1081+
b.appendBodyStmtFromScope(&nameCon, b.callContinuation(&openCon, nameScope), nil /* stmt */)
10821082
return b.callContinuation(&nameCon, s)
10831083

10841084
case *ast.Close:
@@ -1118,7 +1118,7 @@ func (b *plpgsqlBuilder) buildPLpgSQLStatements(stmts []ast.Statement, s *scope)
11181118
closeScope := closeCon.s.push()
11191119
b.ob.synthesizeColumn(closeScope, closeColName, types.Int, nil /* expr */, closeCall)
11201120
b.ob.constructProjectForScope(closeCon.s, closeScope)
1121-
b.appendBodyStmtFromScope(&closeCon, closeScope, "" /* stmtTag */)
1121+
b.appendBodyStmtFromScope(&closeCon, closeScope, nil /* stmt */)
11221122
b.appendPlpgSQLStmts(&closeCon, stmts[i+1:])
11231123
return b.callContinuation(&closeCon, s)
11241124

@@ -1142,7 +1142,7 @@ func (b *plpgsqlBuilder) buildPLpgSQLStatements(stmts []ast.Statement, s *scope)
11421142
fetchCon.def.Volatility = volatility.Volatile
11431143
fetchScope := b.buildFetch(fetchCon.s, t)
11441144
if t.IsMove {
1145-
b.appendBodyStmtFromScope(&fetchCon, fetchScope, "" /* stmtTag */)
1145+
b.appendBodyStmtFromScope(&fetchCon, fetchScope, nil /* stmt */)
11461146
b.appendPlpgSQLStmts(&fetchCon, stmts[i+1:])
11471147
return b.callContinuation(&fetchCon, s)
11481148
}
@@ -1173,7 +1173,7 @@ func (b *plpgsqlBuilder) buildPLpgSQLStatements(stmts []ast.Statement, s *scope)
11731173
intoScope = b.callContinuation(&retCon, intoScope)
11741174

11751175
// Add the built statement to the FETCH continuation.
1176-
b.appendBodyStmtFromScope(&fetchCon, intoScope, "" /* stmtTag */)
1176+
b.appendBodyStmtFromScope(&fetchCon, intoScope, nil /* stmt */)
11771177
return b.callContinuation(&fetchCon, s)
11781178

11791179
case *ast.Null:
@@ -1275,7 +1275,7 @@ func (b *plpgsqlBuilder) buildPLpgSQLStatements(stmts []ast.Statement, s *scope)
12751275
if len(target) == 0 {
12761276
// When there is no INTO target, build the nested procedure call into a
12771277
// body statement that is only executed for its side effects.
1278-
b.appendBodyStmtFromScope(&callCon, callScope, "" /* stmtTag */)
1278+
b.appendBodyStmtFromScope(&callCon, callScope, nil /* stmt */)
12791279
b.appendPlpgSQLStmts(&callCon, stmts[i+1:])
12801280
return b.callContinuation(&callCon, s)
12811281
}
@@ -1290,7 +1290,7 @@ func (b *plpgsqlBuilder) buildPLpgSQLStatements(stmts []ast.Statement, s *scope)
12901290
intoScope = b.callContinuation(&retCon, intoScope)
12911291

12921292
// Add the built statement to the CALL continuation.
1293-
b.appendBodyStmtFromScope(&callCon, intoScope, "" /* stmtTag */)
1293+
b.appendBodyStmtFromScope(&callCon, intoScope, nil /* stmt */)
12941294
return b.callContinuation(&callCon, s)
12951295

12961296
case *ast.DoBlock:
@@ -1306,7 +1306,7 @@ func (b *plpgsqlBuilder) buildPLpgSQLStatements(stmts []ast.Statement, s *scope)
13061306
doCon := b.makeContinuation("_stmt_do")
13071307
doCon.def.Volatility = volatility.Volatile
13081308
bodyScope := b.ob.buildPLpgSQLDoBody(t)
1309-
b.appendBodyStmtFromScope(&doCon, bodyScope, "" /* stmtTag */)
1309+
b.appendBodyStmtFromScope(&doCon, bodyScope, nil /* stmt */)
13101310
b.appendPlpgSQLStmts(&doCon, stmts[i+1:])
13111311
return b.callContinuation(&doCon, s)
13121312

@@ -1456,7 +1456,7 @@ func (b *plpgsqlBuilder) handleIntForLoop(
14561456
)
14571457
// Call recursively into the loop body continuation.
14581458
incScope = b.callContinuation(&loopCon, incScope)
1459-
b.appendBodyStmtFromScope(&incrementCon, incScope, "" /* stmtTag */)
1459+
b.appendBodyStmtFromScope(&incrementCon, incScope, nil /* stmt */)
14601460

14611461
// Notably, we call the loop body continuation here, rather than the
14621462
// increment continuation, because the counter should not be incremented
@@ -2020,7 +2020,7 @@ func (b *plpgsqlBuilder) buildEndOfFunctionRaise(con *continuation) {
20202020
pgcode.RoutineExceptionFunctionExecutedNoReturnStatement.String(), /* code */
20212021
)
20222022
con.def.Volatility = volatility.Volatile
2023-
b.appendBodyStmtFromScope(con, b.buildPLpgSQLRaise(con.s, args), "" /* stmtTag */)
2023+
b.appendBodyStmtFromScope(con, b.buildPLpgSQLRaise(con.s, args), nil /* stmt */)
20242024

20252025
// Build a dummy statement that returns NULL. It won't be executed, but
20262026
// ensures that the continuation routine's return type is correct.
@@ -2029,7 +2029,7 @@ func (b *plpgsqlBuilder) buildEndOfFunctionRaise(con *continuation) {
20292029
typedNull := b.ob.factory.ConstructNull(b.returnType)
20302030
b.ob.synthesizeColumn(eofScope, eofColName, b.returnType, nil /* expr */, typedNull)
20312031
b.ob.constructProjectForScope(con.s, eofScope)
2032-
b.appendBodyStmtFromScope(con, eofScope, "" /* stmtTag */)
2032+
b.appendBodyStmtFromScope(con, eofScope, nil /* stmt */)
20332033
}
20342034

20352035
// addOneRowCheck handles INTO STRICT, where a SQL statement is required to
@@ -2279,8 +2279,10 @@ func (b *plpgsqlBuilder) makeContinuationWithTyp(
22792279
// appendBodyStmtFromScope is separate from makeContinuation to allow recursive
22802280
// routine definitions, which need to push the continuation before it is
22812281
// finished. The separation also allows for appending multiple body statements.
2282+
// If the continuation does not map to an explicit statement in the body, stmt
2283+
// may be nil.
22822284
func (b *plpgsqlBuilder) appendBodyStmtFromScope(
2283-
con *continuation, bodyScope *scope, stmtTag string,
2285+
con *continuation, bodyScope *scope, stmt tree.Statement,
22842286
) {
22852287
// Set the volatility of the continuation routine to the least restrictive
22862288
// volatility level in the Relational properties of the body statements.
@@ -2290,8 +2292,11 @@ func (b *plpgsqlBuilder) appendBodyStmtFromScope(
22902292
con.def.Volatility = vol
22912293
}
22922294
con.def.Body = append(con.def.Body, bodyExpr)
2293-
con.def.BodyTags = append(con.def.BodyTags, stmtTag)
22942295
con.def.BodyProps = append(con.def.BodyProps, bodyScope.makePhysicalProps())
2296+
if stmt != nil {
2297+
con.def.BodyTags = append(con.def.BodyTags, stmt.StatementTag())
2298+
con.def.BodyASTs = append(con.def.BodyASTs, stmt)
2299+
}
22952300
}
22962301

22972302
// appendPlpgSQLStmts builds the given PLpgSQL statements into a relational
@@ -2301,7 +2306,7 @@ func (b *plpgsqlBuilder) appendPlpgSQLStmts(con *continuation, stmts []ast.State
23012306
// Make sure to push s before constructing the continuation scope to ensure
23022307
// that the parameter columns are not projected.
23032308
continuationScope := b.buildPLpgSQLStatements(stmts, con.s.push())
2304-
b.appendBodyStmtFromScope(con, continuationScope, "" /* stmtTag */)
2309+
b.appendBodyStmtFromScope(con, continuationScope, nil /* stmt */)
23052310
}
23062311

23072312
// callContinuation adds a column that projects the result of calling the

0 commit comments

Comments
 (0)