Skip to content

Commit 2fb5fac

Browse files
committed
sql,*: record UDF stats
Updates the UDF routine generators to pass down the necessary information in order to generate and record RecordedStmtStats structs Epic: CRDB-55081 Release note: None
1 parent a26646b commit 2fb5fac

File tree

12 files changed

+123
-50
lines changed

12 files changed

+123
-50
lines changed

pkg/sql/conn_executor.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3849,6 +3849,7 @@ func (ex *connExecutor) initEvalCtx(ctx context.Context, evalCtx *extendedEvalCo
38493849
validateDbZoneConfig: &ex.extraTxnState.validateDbZoneConfig,
38503850
persistedSQLStats: ex.server.persistedSQLStats,
38513851
localSQLStats: ex.server.localSqlStats,
3852+
sqlStatsIngester: ex.server.sqlStatsIngester,
38523853
indexUsageStats: ex.indexUsageStats,
38533854
statementPreparer: ex,
38543855
}

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

Lines changed: 1 addition & 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",

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: 35 additions & 46 deletions
Original file line numberDiff line numberDiff line change
@@ -8,6 +8,7 @@ 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"
1314
"github.com/cockroachdb/cockroach/pkg/sql/opt/memo"
@@ -689,16 +690,7 @@ func (b *Builder) buildExistsSubquery(
689690

690691
// Create a plan generator that can plan the single statement
691692
// representing the subquery, and wrap the routine in a COALESCE.
692-
planGen := b.buildRoutinePlanGenerator(
693-
params,
694-
stmts,
695-
stmtProps,
696-
nil, /* stmtStr */
697-
make([]string, len(stmts)),
698-
true, /* allowOuterWithRefs */
699-
wrapRootExpr,
700-
0, /* resultBufferID */
701-
)
693+
planGen := b.buildRoutinePlanGenerator(params, stmts, stmtProps, nil, make([]string, len(stmts)), nil, true, wrapRootExpr, 0)
702694
return tree.NewTypedCoalesceExpr(tree.TypedExprs{
703695
tree.NewTypedRoutineExpr(
704696
"exists",
@@ -816,16 +808,7 @@ func (b *Builder) buildSubquery(
816808

817809
// Create a tree.RoutinePlanFn that can plan the single statement
818810
// representing the subquery.
819-
planGen := b.buildRoutinePlanGenerator(
820-
params,
821-
stmts,
822-
stmtProps,
823-
nil, /* stmtStr */
824-
make([]string, len(stmts)),
825-
true, /* allowOuterWithRefs */
826-
nil, /* wrapRootExpr */
827-
0, /* resultBufferID */
828-
)
811+
planGen := b.buildRoutinePlanGenerator(params, stmts, stmtProps, nil, make([]string, len(stmts)), nil, true, nil, 0)
829812
_, tailCall := b.tailCalls[subquery]
830813
return tree.NewTypedRoutineExpr(
831814
"subquery",
@@ -900,7 +883,7 @@ func (b *Builder) buildSubquery(
900883
if err != nil {
901884
return err
902885
}
903-
err = fn(plan, "" /* stmtForDistSQLDiagram */, true /* isFinalPlan */)
886+
err = fn(plan, tree.RoutineBodyStmt{} /* stmtForDistSQLDiagram */, true /* isFinalPlan */)
904887
if err != nil {
905888
return err
906889
}
@@ -1011,16 +994,7 @@ func (b *Builder) buildUDF(ctx *buildScalarCtx, scalar opt.ScalarExpr) (tree.Typ
1011994
}
1012995

1013996
// Create a tree.RoutinePlanFn that can plan the statements in the UDF body.
1014-
planGen := b.buildRoutinePlanGenerator(
1015-
udf.Def.Params,
1016-
udf.Def.Body,
1017-
udf.Def.BodyProps,
1018-
udf.Def.BodyStmts,
1019-
udf.Def.BodyTags,
1020-
false, /* allowOuterWithRefs */
1021-
nil, /* wrapRootExpr */
1022-
udf.Def.ResultBufferID,
1023-
)
997+
planGen := b.buildRoutinePlanGenerator(udf.Def.Params, udf.Def.Body, udf.Def.BodyProps, udf.Def.BodyStmts, udf.Def.BodyTags, udf.Def.BodyASTs, false, nil, udf.Def.ResultBufferID)
1024998

1025999
// Enable stepping for volatile functions so that statements within the UDF
10261000
// see mutations made by the invoking statement and by previously executed
@@ -1085,16 +1059,7 @@ func (b *Builder) initRoutineExceptionHandler(
10851059
Actions: make([]*tree.RoutineExpr, len(exceptionBlock.Actions)),
10861060
}
10871061
for i, action := range exceptionBlock.Actions {
1088-
actionPlanGen := b.buildRoutinePlanGenerator(
1089-
action.Params,
1090-
action.Body,
1091-
action.BodyProps,
1092-
action.BodyStmts,
1093-
action.BodyTags,
1094-
false, /* allowOuterWithRefs */
1095-
nil, /* wrapRootExpr */
1096-
0, /* resultBufferID */
1097-
)
1062+
actionPlanGen := b.buildRoutinePlanGenerator(action.Params, action.Body, action.BodyProps, action.BodyStmts, action.BodyTags, nil, false, nil, 0)
10981063
// Build a routine with no arguments for the exception handler. The actual
10991064
// arguments will be supplied when (if) the handler is invoked.
11001065
exceptionHandler.Actions[i] = tree.NewTypedRoutineExpr(
@@ -1141,6 +1106,7 @@ func (b *Builder) buildRoutinePlanGenerator(
11411106
stmtProps []*physical.Required,
11421107
stmtStr []string,
11431108
stmtTags []string,
1109+
stmtASTs []tree.Statement,
11441110
allowOuterWithRefs bool,
11451111
wrapRootExpr wrapRootExprFn,
11461112
resultBufferID memo.RoutineResultBufferID,
@@ -1207,15 +1173,42 @@ func (b *Builder) buildRoutinePlanGenerator(
12071173
dbName := b.evalCtx.SessionData().Database
12081174
appName := b.evalCtx.SessionData().ApplicationName
12091175

1176+
format := tree.FmtHideConstants | tree.FmtFlags(tree.QueryFormattingForFingerprintsMask.Get(&b.evalCtx.Settings.SV))
12101177
for i := range stmts {
1178+
var bodyStmt tree.RoutineBodyStmt
12111179
stmt := stmts[i]
12121180
props := stmtProps[i]
12131181
var tag string
1182+
var fingerprint string
1183+
var summary string
1184+
var fpId appstatspb.StmtFingerprintID
1185+
var stmtString string
1186+
var stmtType tree.StatementType
12141187
// Theoretically, stmts and stmtTags should have the same length,
12151188
// but just to avoid an out-of-bounds panic, we have this check.
12161189
if i < len(stmtTags) {
12171190
tag = stmtTags[i]
12181191
}
1192+
if i < len(stmtStr) {
1193+
stmtString = stmtStr[i]
1194+
}
1195+
if i < len(stmtASTs) {
1196+
fingerprint = tree.FormatStatementHideConstants(stmtASTs[i], format)
1197+
summary = tree.FormatStatementSummary(stmtASTs[i], format)
1198+
fpId = appstatspb.ConstructStatementFingerprintID(fingerprint, b.evalCtx.TxnImplicit, dbName)
1199+
stmtType = stmtASTs[i].StatementType()
1200+
}
1201+
1202+
bodyStmt = tree.RoutineBodyStmt{
1203+
FingerprintStr: fingerprint,
1204+
SummaryStr: summary,
1205+
StmtType: stmtType,
1206+
StmtString: stmtString,
1207+
DbName: dbName,
1208+
AppName: appName,
1209+
FingerprintId: uint64(fpId),
1210+
}
1211+
12191212
o.Init(ctx, b.evalCtx, b.catalog)
12201213
f := o.Factory()
12211214

@@ -1319,12 +1312,8 @@ func (b *Builder) buildRoutinePlanGenerator(
13191312
return expectedLazyRoutineError("subquery")
13201313
}
13211314
}
1322-
var stmtForDistSQLDiagram string
1323-
if i < len(stmtStr) {
1324-
stmtForDistSQLDiagram = stmtStr[i]
1325-
}
13261315
incrementRoutineStmtCounter(b.evalCtx.StartedRoutineStatementCounters, dbName, appName, tag)
1327-
err = fn(plan, stmtForDistSQLDiagram, isFinalPlan)
1316+
err = fn(plan, bodyStmt, isFinalPlan)
13281317
if err != nil {
13291318
return err
13301319
}

pkg/sql/opt/memo/expr.go

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

745+
BodyASTs []tree.Statement
746+
745747
// BodyStmts, if set, is the string representation of each statement in
746748
// Body. It is only populated when verbose tracing is enabled.
747749
BodyStmts []string

pkg/sql/opt/optbuilder/routine.go

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -420,10 +420,15 @@ func (b *Builder) buildRoutine(
420420
var bodyProps []*physical.Required
421421
var bodyStmts []string
422422
var bodyTags []string
423+
var bodyASTs []tree.Statement
423424
switch o.Language {
424425
case tree.RoutineLangSQL:
425426
// Parse the function body.
426427
stmts, err := parser.Parse(o.Body)
428+
bodyASTs = make([]tree.Statement, len(stmts))
429+
for i := range stmts {
430+
bodyASTs[i] = stmts[i].AST
431+
}
427432
if err != nil {
428433
panic(err)
429434
}
@@ -539,6 +544,7 @@ func (b *Builder) buildRoutine(
539544
BodyProps: bodyProps,
540545
BodyStmts: bodyStmts,
541546
BodyTags: bodyTags,
547+
BodyASTs: bodyASTs,
542548
Params: params,
543549
ResultBufferID: resultBufferID,
544550
},

pkg/sql/planner.go

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -109,6 +109,8 @@ type extendedEvalContext struct {
109109

110110
localSQLStats *sslocal.SQLStats
111111

112+
sqlStatsIngester *sslocal.SQLStatsIngester
113+
112114
indexUsageStats *idxusage.LocalIndexUsageStats
113115

114116
SchemaChangerState *SchemaChangerState
@@ -536,12 +538,14 @@ func internalExtendedEvalCtx(
536538
var indexUsageStatsController eval.IndexUsageStatsController
537539
var sqlStatsProvider *persistedsqlstats.PersistedSQLStats
538540
var localSqlStatsProvider *sslocal.SQLStats
541+
var sqlStatsIngester *sslocal.SQLStatsIngester
539542
if ief := execCfg.InternalDB; ief != nil {
540543
if ief.server != nil {
541544
indexUsageStats = ief.server.indexUsageStats
542545
schemaTelemetryController = ief.server.schemaTelemetryController
543546
sqlStatsProvider = ief.server.persistedSQLStats
544547
localSqlStatsProvider = ief.server.localSqlStats
548+
sqlStatsIngester = ief.server.sqlStatsIngester
545549
} else {
546550
// If the indexUsageStats is nil from the sql.Server, we create a dummy
547551
// index usage stats collector. The sql.Server in the ExecutorConfig
@@ -553,6 +557,7 @@ func internalExtendedEvalCtx(
553557
indexUsageStatsController = &idxusage.Controller{}
554558
sqlStatsProvider = &persistedsqlstats.PersistedSQLStats{}
555559
localSqlStatsProvider = &sslocal.SQLStats{}
560+
sqlStatsIngester = &sslocal.SQLStatsIngester{}
556561
}
557562
}
558563
ret := extendedEvalContext{
@@ -578,6 +583,7 @@ func internalExtendedEvalCtx(
578583
indexUsageStats: indexUsageStats,
579584
persistedSQLStats: sqlStatsProvider,
580585
localSQLStats: localSqlStatsProvider,
586+
sqlStatsIngester: sqlStatsIngester,
581587
jobs: newTxnJobsCollection(),
582588
}
583589
ret.copyFromExecCfg(execCfg)

pkg/sql/routine.go

Lines changed: 27 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -11,14 +11,18 @@ import (
1111

1212
"github.com/cockroachdb/cockroach/pkg/kv"
1313
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
14+
"github.com/cockroachdb/cockroach/pkg/sql/appstatspb"
1415
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
16+
"github.com/cockroachdb/cockroach/pkg/sql/clusterunique"
1517
"github.com/cockroachdb/cockroach/pkg/sql/isql"
18+
"github.com/cockroachdb/cockroach/pkg/sql/opt/exec/explain"
1619
"github.com/cockroachdb/cockroach/pkg/sql/opt/memo"
1720
"github.com/cockroachdb/cockroach/pkg/sql/parser/statements"
1821
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
1922
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
2023
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
2124
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
25+
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats"
2226
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
2327
"github.com/cockroachdb/cockroach/pkg/sql/types"
2428
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
@@ -325,7 +329,7 @@ func (g *routineGenerator) startInternal(ctx context.Context, txn *kv.Txn) (err
325329
rrw := NewRowResultWriter(&g.rch)
326330
var cursorHelper *plpgsqlCursorHelper
327331
err = g.expr.ForEachPlan(ctx, ef, rrw, g.args,
328-
func(plan tree.RoutinePlan, stmtForDistSQLDiagram string, isFinalPlan bool) error {
332+
func(plan tree.RoutinePlan, stmt tree.RoutineBodyStmt, isFinalPlan bool) error {
329333
stmtIdx++
330334
opName := "routine-stmt-" + g.expr.Name + "-" + strconv.Itoa(stmtIdx)
331335
ctx, sp := tracing.ChildSpan(ctx, opName)
@@ -368,10 +372,31 @@ func (g *routineGenerator) startInternal(ctx context.Context, txn *kv.Txn) (err
368372

369373
// Run the plan.
370374
params := runParams{ctx, g.p.ExtendedEvalContext(), g.p}
371-
queryStats, err := runPlanInsidePlan(ctx, params, plan.(*planComponents), w, g, stmtForDistSQLDiagram)
375+
var builder sqlstats.RecordedStatementStatsBuilder[sqlstats.StatementLatencyRecorder, topLevelQueryStats, PlanInfo]
376+
if stmt.FingerprintId != 0 {
377+
builder = sqlstats.NewRecordedStatementStatsBuilder[sqlstats.StatementLatencyRecorder, topLevelQueryStats, PlanInfo](
378+
appstatspb.StmtFingerprintID(stmt.FingerprintId),
379+
g.p.extendedEvalCtx.SessionID,
380+
stmt.DbName,
381+
stmt.FingerprintStr,
382+
stmt.SummaryStr,
383+
clusterunique.ID{},
384+
stmt.StmtType, stmt.AppName,
385+
PlanInfo{
386+
planFlags: g.p.curPlan.flags, // TODO: Is this right?
387+
planGist: explain.PlanGist{},
388+
})
389+
defer func() {
390+
g.p.ExtendedEvalContext().sqlStatsIngester.RecordStatement(builder.Build())
391+
}()
392+
}
393+
queryStats, err := runPlanInsidePlan(ctx, params, plan.(*planComponents), w, g, stmt.StmtString)
394+
372395
if err != nil {
396+
builder = builder.StatementError(err)
373397
return err
374398
}
399+
builder = builder.QueryLevelStats(queryStats)
375400
forwardInnerQueryStats(g.p.routineMetadataForwarder, queryStats)
376401
if openCursor {
377402
return cursorHelper.createCursor(g.p)

pkg/sql/sem/tree/routine.go

Lines changed: 11 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -29,13 +29,23 @@ type RoutinePlanGenerator func(
2929
fn RoutinePlanGeneratedFunc,
3030
) error
3131

32+
type RoutineBodyStmt struct {
33+
FingerprintStr string
34+
SummaryStr string
35+
StmtType StatementType
36+
StmtString string
37+
DbName string
38+
AppName string
39+
FingerprintId uint64
40+
}
41+
3242
// RoutinePlanGeneratedFunc is the function type that is called for each plan
3343
// enumerated by a RoutinePlanGenerator.
3444
// - stmtForDistSQLDiagram, if set, will be used when generating DistSQL diagram
3545
// to specify the SQL stmt corresponding to the plan.
3646
// - isFinalPlan is true if no more plans will be generated after the current
3747
// plan.
38-
type RoutinePlanGeneratedFunc func(plan RoutinePlan, stmtForDistSQLDiagram string, isFinalPlan bool) error
48+
type RoutinePlanGeneratedFunc func(plan RoutinePlan, stmt RoutineBodyStmt, isFinalPlan bool) error
3949

4050
// RoutinePlan represents a plan for a statement in a routine. It currently maps
4151
// to exec.Plan. We use the empty interface here rather than exec.Plan to avoid

pkg/sql/sqlstats/insights/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -28,6 +28,7 @@ go_library(
2828
"//pkg/util/metric",
2929
"//pkg/util/quantile",
3030
"//pkg/util/syncutil",
31+
"//pkg/util/uint128",
3132
"//pkg/util/uuid",
3233
"@com_github_cockroachdb_redact//:redact",
3334
"@com_github_prometheus_client_model//go",

0 commit comments

Comments
 (0)