Skip to content

Commit 0a04ffd

Browse files
committed
sql: Add latency recordings to udf sub statement calls
Adds RoutineStatementLatencyRecorder struct that implements sqlstats.StatementLatencyRecorder. Now statement latencies are recorded Epic: CRDB-55081 Release note: None
1 parent 2fb5fac commit 0a04ffd

File tree

5 files changed

+86
-11
lines changed

5 files changed

+86
-11
lines changed

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

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -57,6 +57,7 @@ go_library(
5757
"//pkg/util/metamorphic",
5858
"//pkg/util/timeutil",
5959
"//pkg/util/treeprinter",
60+
"@com_github_cockroachdb_crlib//crtime",
6061
"@com_github_cockroachdb_errors//:errors",
6162
"@com_github_cockroachdb_redact//:redact",
6263
],

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

Lines changed: 11 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -28,6 +28,7 @@ import (
2828
"github.com/cockroachdb/cockroach/pkg/sql/types"
2929
"github.com/cockroachdb/cockroach/pkg/util/errorutil"
3030
"github.com/cockroachdb/cockroach/pkg/util/log"
31+
"github.com/cockroachdb/crlib/crtime"
3132
"github.com/cockroachdb/errors"
3233
"github.com/cockroachdb/redact"
3334
)
@@ -883,7 +884,9 @@ func (b *Builder) buildSubquery(
883884
if err != nil {
884885
return err
885886
}
886-
err = fn(plan, tree.RoutineBodyStmt{} /* stmtForDistSQLDiagram */, true /* isFinalPlan */)
887+
recorder := tree.NewStatementLatencyRecorder()
888+
recorder.RecordPhase(tree.RoutineStatementStarted, crtime.NowMono())
889+
err = fn(plan, tree.RoutineBodyStmt{} /* stmt */, recorder, true /* isFinalPlan */)
887890
if err != nil {
888891
return err
889892
}
@@ -1175,6 +1178,9 @@ func (b *Builder) buildRoutinePlanGenerator(
11751178

11761179
format := tree.FmtHideConstants | tree.FmtFlags(tree.QueryFormattingForFingerprintsMask.Get(&b.evalCtx.Settings.SV))
11771180
for i := range stmts {
1181+
latencyRecorder := tree.NewStatementLatencyRecorder()
1182+
latencyRecorder.RecordPhase(tree.RoutineStatementStarted, crtime.NowMono())
1183+
latencyRecorder.RecordPhase(tree.RoutineStatementStartParsing, crtime.NowMono())
11781184
var bodyStmt tree.RoutineBodyStmt
11791185
stmt := stmts[i]
11801186
props := stmtProps[i]
@@ -1208,7 +1214,8 @@ func (b *Builder) buildRoutinePlanGenerator(
12081214
AppName: appName,
12091215
FingerprintId: uint64(fpId),
12101216
}
1211-
1217+
latencyRecorder.RecordPhase(tree.RoutineStatementEndParsing, crtime.NowMono())
1218+
latencyRecorder.RecordPhase(tree.RoutineStatementStartPlanning, crtime.NowMono())
12121219
o.Init(ctx, b.evalCtx, b.catalog)
12131220
f := o.Factory()
12141221

@@ -1312,8 +1319,9 @@ func (b *Builder) buildRoutinePlanGenerator(
13121319
return expectedLazyRoutineError("subquery")
13131320
}
13141321
}
1322+
latencyRecorder.RecordPhase(tree.RoutineStatementEndPlanning, crtime.NowMono())
13151323
incrementRoutineStmtCounter(b.evalCtx.StartedRoutineStatementCounters, dbName, appName, tag)
1316-
err = fn(plan, bodyStmt, isFinalPlan)
1324+
err = fn(plan, bodyStmt, latencyRecorder, isFinalPlan)
13171325
if err != nil {
13181326
return err
13191327
}

pkg/sql/routine.go

Lines changed: 12 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,7 @@ import (
3030
"github.com/cockroachdb/cockroach/pkg/util/metamorphic"
3131
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
3232
"github.com/cockroachdb/cockroach/pkg/util/tracing"
33+
"github.com/cockroachdb/crlib/crtime"
3334
"github.com/cockroachdb/errors"
3435
)
3536

@@ -329,7 +330,7 @@ func (g *routineGenerator) startInternal(ctx context.Context, txn *kv.Txn) (err
329330
rrw := NewRowResultWriter(&g.rch)
330331
var cursorHelper *plpgsqlCursorHelper
331332
err = g.expr.ForEachPlan(ctx, ef, rrw, g.args,
332-
func(plan tree.RoutinePlan, stmt tree.RoutineBodyStmt, isFinalPlan bool) error {
333+
func(plan tree.RoutinePlan, stmt tree.RoutineBodyStmt, latencyRecorder *tree.RoutineStatementLatencyRecorder, isFinalPlan bool) error {
333334
stmtIdx++
334335
opName := "routine-stmt-" + g.expr.Name + "-" + strconv.Itoa(stmtIdx)
335336
ctx, sp := tracing.ChildSpan(ctx, opName)
@@ -372,26 +373,30 @@ func (g *routineGenerator) startInternal(ctx context.Context, txn *kv.Txn) (err
372373

373374
// Run the plan.
374375
params := runParams{ctx, g.p.ExtendedEvalContext(), g.p}
375-
var builder sqlstats.RecordedStatementStatsBuilder[sqlstats.StatementLatencyRecorder, topLevelQueryStats, PlanInfo]
376+
var builder sqlstats.RecordedStatementStatsBuilder[*tree.RoutineStatementLatencyRecorder, topLevelQueryStats, PlanInfo]
376377
if stmt.FingerprintId != 0 {
377-
builder = sqlstats.NewRecordedStatementStatsBuilder[sqlstats.StatementLatencyRecorder, topLevelQueryStats, PlanInfo](
378+
builder = sqlstats.NewRecordedStatementStatsBuilder[*tree.RoutineStatementLatencyRecorder, topLevelQueryStats, PlanInfo](
378379
appstatspb.StmtFingerprintID(stmt.FingerprintId),
379380
g.p.extendedEvalCtx.SessionID,
380381
stmt.DbName,
381382
stmt.FingerprintStr,
382383
stmt.SummaryStr,
383384
clusterunique.ID{},
384-
stmt.StmtType, stmt.AppName,
385-
PlanInfo{
385+
stmt.StmtType,
386+
stmt.AppName, PlanInfo{
386387
planFlags: g.p.curPlan.flags, // TODO: Is this right?
387388
planGist: explain.PlanGist{},
388389
})
389390
defer func() {
390-
g.p.ExtendedEvalContext().sqlStatsIngester.RecordStatement(builder.Build())
391+
latencyRecorder.RecordPhase(tree.RoutineStatementEnd, crtime.NowMono())
392+
// TODO: only record if statement stats recording is enabled. We can also avoid building the
393+
// tree.RoutineBodyStmt struct passed into this function if stats recording is disabled.
394+
g.p.ExtendedEvalContext().sqlStatsIngester.RecordStatement(builder.LatencyRecorder(latencyRecorder).Build())
391395
}()
392396
}
397+
latencyRecorder.RecordPhase(tree.RoutineStatementStartExec, crtime.NowMono())
393398
queryStats, err := runPlanInsidePlan(ctx, params, plan.(*planComponents), w, g, stmt.StmtString)
394-
399+
latencyRecorder.RecordPhase(tree.RoutineStatementEndExec, crtime.NowMono())
395400
if err != nil {
396401
builder = builder.StatementError(err)
397402
return err

pkg/sql/sem/tree/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -188,6 +188,7 @@ go_library(
188188
"//pkg/util/uuid",
189189
"//pkg/util/vector",
190190
"@com_github_cockroachdb_apd_v3//:apd",
191+
"@com_github_cockroachdb_crlib//crtime",
191192
"@com_github_cockroachdb_errors//:errors",
192193
"@com_github_cockroachdb_redact//:redact",
193194
"@com_github_google_go_cmp//cmp",

pkg/sql/sem/tree/routine.go

Lines changed: 61 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -12,6 +12,7 @@ import (
1212
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
1313
"github.com/cockroachdb/cockroach/pkg/sql/types"
1414
"github.com/cockroachdb/cockroach/pkg/util/buildutil"
15+
"github.com/cockroachdb/crlib/crtime"
1516
"github.com/cockroachdb/errors"
1617
)
1718

@@ -45,7 +46,7 @@ type RoutineBodyStmt struct {
4546
// to specify the SQL stmt corresponding to the plan.
4647
// - isFinalPlan is true if no more plans will be generated after the current
4748
// plan.
48-
type RoutinePlanGeneratedFunc func(plan RoutinePlan, stmt RoutineBodyStmt, isFinalPlan bool) error
49+
type RoutinePlanGeneratedFunc func(plan RoutinePlan, stmt RoutineBodyStmt, latencyRecorder *RoutineStatementLatencyRecorder, isFinalPlan bool) error
4950

5051
// RoutinePlan represents a plan for a statement in a routine. It currently maps
5152
// to exec.Plan. We use the empty interface here rather than exec.Plan to avoid
@@ -412,3 +413,62 @@ func (node *TxnControlExpr) Walk(v Visitor) Expr {
412413
// Cannot walk into a TxnOp, so this is a no-op.
413414
return node
414415
}
416+
417+
type RoutineStatementPhase int
418+
419+
const (
420+
RoutineStatementStarted RoutineStatementPhase = iota
421+
RoutineStatementStartParsing
422+
RoutineStatementEndParsing
423+
RoutineStatementStartPlanning
424+
RoutineStatementEndPlanning
425+
RoutineStatementStartExec
426+
RoutineStatementEndExec
427+
RoutineStatementEnd
428+
RoutineStatementNumPhases
429+
)
430+
431+
type RoutineStatementLatencyRecorder struct {
432+
times [RoutineStatementNumPhases]crtime.Mono
433+
}
434+
435+
func NewStatementLatencyRecorder() *RoutineStatementLatencyRecorder {
436+
return &RoutineStatementLatencyRecorder{}
437+
}
438+
439+
func (r *RoutineStatementLatencyRecorder) RecordPhase(
440+
phase RoutineStatementPhase, time crtime.Mono,
441+
) {
442+
r.times[phase] = time
443+
}
444+
445+
func (r *RoutineStatementLatencyRecorder) RunLatency() time.Duration {
446+
return r.times[RoutineStatementEndExec].Sub(r.times[RoutineStatementStartExec])
447+
}
448+
449+
// IdleLatency There is no idle latency for routine statements
450+
func (r *RoutineStatementLatencyRecorder) IdleLatency() time.Duration {
451+
return 0
452+
}
453+
454+
func (r *RoutineStatementLatencyRecorder) ServiceLatency() time.Duration {
455+
return r.times[RoutineStatementEndExec].Sub(r.times[RoutineStatementStarted])
456+
}
457+
func (r *RoutineStatementLatencyRecorder) ParsingLatency() time.Duration {
458+
return r.times[RoutineStatementEndParsing].Sub(r.times[RoutineStatementStartParsing])
459+
}
460+
func (r *RoutineStatementLatencyRecorder) PlanningLatency() time.Duration {
461+
return r.times[RoutineStatementEndPlanning].Sub(r.times[RoutineStatementStartPlanning])
462+
}
463+
func (r *RoutineStatementLatencyRecorder) ProcessingLatency() time.Duration {
464+
return r.ServiceLatency() // TODO: is there a difference between processing latency and service latency?
465+
}
466+
func (r *RoutineStatementLatencyRecorder) ExecOverheadLatency() time.Duration {
467+
return r.ServiceLatency() - r.ProcessingLatency() // Will be 0 since processing and service latency are the same
468+
}
469+
func (r *RoutineStatementLatencyRecorder) StartTime() time.Time {
470+
return r.times[RoutineStatementStarted].ToUTC()
471+
}
472+
func (r *RoutineStatementLatencyRecorder) EndTime() time.Time {
473+
return r.times[RoutineStatementEnd].ToUTC()
474+
}

0 commit comments

Comments
 (0)