Skip to content

Commit 90c89be

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 9e09035 commit 90c89be

File tree

5 files changed

+84
-7
lines changed

5 files changed

+84
-7
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: 10 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@ import (
2929
"github.com/cockroachdb/cockroach/pkg/util/metamorphic"
3030
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
3131
"github.com/cockroachdb/cockroach/pkg/util/tracing"
32+
"github.com/cockroachdb/crlib/crtime"
3233
"github.com/cockroachdb/errors"
3334
)
3435

@@ -328,7 +329,7 @@ func (g *routineGenerator) startInternal(ctx context.Context, txn *kv.Txn) (err
328329
rrw := NewRowResultWriter(&g.rch)
329330
var cursorHelper *plpgsqlCursorHelper
330331
err = g.expr.ForEachPlan(ctx, ef, rrw, g.args,
331-
func(plan tree.RoutinePlan, stmt tree.RoutineBodyStmt, isFinalPlan bool) error {
332+
func(plan tree.RoutinePlan, stmt tree.RoutineBodyStmt, latencyRecorder *tree.RoutineStatementLatencyRecorder, isFinalPlan bool) error {
332333
stmtIdx++
333334
opName := "routine-stmt-" + g.expr.Name + "-" + strconv.Itoa(stmtIdx)
334335
ctx, sp := tracing.ChildSpan(ctx, opName)
@@ -373,21 +374,27 @@ func (g *routineGenerator) startInternal(ctx context.Context, txn *kv.Txn) (err
373374
params := runParams{ctx, g.p.ExtendedEvalContext(), g.p}
374375
var builder *sqlstats.RecordedStatementStatsBuilder
375376
if stmt.FingerprintId != 0 {
377+
// TODO: What other builder options should we set here?
376378
builder = sqlstats.NewRecordedStatementStatsBuilder(appstatspb.StmtFingerprintID(stmt.FingerprintId), g.p.extendedEvalCtx.SessionID, stmt.DbName, stmt, stmt.AppName).
377379
PlanFlags(PlanInfo{
378380
planFlags: g.p.curPlan.flags, // TODO: Is this right?
379381
planGist: explain.PlanGist{},
380382
})
381383
defer func() {
382-
g.p.ExtendedEvalContext().sqlStatsIngester.RecordStatement(builder.Build())
384+
latencyRecorder.RecordPhase(tree.RoutineStatementEndExec, crtime.NowMono())
385+
// TODO: only record if statement stats recording is enabled. We can also avoid building the
386+
// tree.RoutineBodyStmt struct passed into this function if stats recording is disabled.
387+
g.p.ExtendedEvalContext().sqlStatsIngester.RecordStatement(builder.LatencyRecorder(latencyRecorder).Build())
383388
}()
384389
}
390+
latencyRecorder.RecordPhase(tree.RoutineStatementStartExec, crtime.NowMono())
385391
queryStats, err := runPlanInsidePlan(ctx, params, plan.(*planComponents), w, g, stmt.StmtString)
386-
392+
latencyRecorder.RecordPhase(tree.RoutineStatementEndExec, crtime.NowMono())
387393
if err != nil {
388394
builder.StatementError(err)
389395
return err
390396
}
397+
// TODO: What other stats can we record here?
391398
builder.QueryLevelStats(queryStats)
392399
forwardInnerQueryStats(g.p.routineMetadataForwarder, queryStats)
393400
if openCursor {

pkg/sql/sem/tree/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -189,6 +189,7 @@ go_library(
189189
"//pkg/util/uuid",
190190
"//pkg/util/vector",
191191
"@com_github_cockroachdb_apd_v3//:apd",
192+
"@com_github_cockroachdb_crlib//crtime",
192193
"@com_github_cockroachdb_errors//:errors",
193194
"@com_github_cockroachdb_redact//:redact",
194195
"@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
@@ -13,6 +13,7 @@ import (
1313
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
1414
"github.com/cockroachdb/cockroach/pkg/sql/types"
1515
"github.com/cockroachdb/cockroach/pkg/util/buildutil"
16+
"github.com/cockroachdb/crlib/crtime"
1617
"github.com/cockroachdb/errors"
1718
)
1819

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

5152
// RoutinePlan represents a plan for a statement in a routine. It currently maps
5253
// to exec.Plan. We use the empty interface here rather than exec.Plan to avoid
@@ -433,3 +434,62 @@ func (s RoutineBodyStmt) GetQueryID() clusterunique.ID {
433434
func (s RoutineBodyStmt) StatementType() StatementType {
434435
return s.StmtType
435436
}
437+
438+
type RoutineStatementPhase int
439+
440+
const (
441+
RoutineStatementStarted RoutineStatementPhase = iota
442+
RoutineStatementStartParsing
443+
RoutineStatementEndParsing
444+
RoutineStatementStartPlanning
445+
RoutineStatementEndPlanning
446+
RoutineStatementStartExec
447+
RoutineStatementEndExec
448+
RoutineStatementEnd
449+
RoutineStatementNumPhases
450+
)
451+
452+
type RoutineStatementLatencyRecorder struct {
453+
times [RoutineStatementNumPhases]crtime.Mono
454+
}
455+
456+
func NewStatementLatencyRecorder() *RoutineStatementLatencyRecorder {
457+
return &RoutineStatementLatencyRecorder{}
458+
}
459+
460+
func (r *RoutineStatementLatencyRecorder) RecordPhase(
461+
phase RoutineStatementPhase, time crtime.Mono,
462+
) {
463+
r.times[phase] = time
464+
}
465+
466+
func (r *RoutineStatementLatencyRecorder) RunLatency() time.Duration {
467+
return r.times[RoutineStatementEndExec].Sub(r.times[RoutineStatementStartExec])
468+
}
469+
470+
// IdleLatency There is no idle latency for routine statements
471+
func (r *RoutineStatementLatencyRecorder) IdleLatency() time.Duration {
472+
return 0
473+
}
474+
475+
func (r *RoutineStatementLatencyRecorder) ServiceLatency() time.Duration {
476+
return r.times[RoutineStatementEndExec].Sub(r.times[RoutineStatementStarted])
477+
}
478+
func (r *RoutineStatementLatencyRecorder) ParsingLatency() time.Duration {
479+
return r.times[RoutineStatementEndParsing].Sub(r.times[RoutineStatementStartParsing])
480+
}
481+
func (r *RoutineStatementLatencyRecorder) PlanningLatency() time.Duration {
482+
return r.times[RoutineStatementEndPlanning].Sub(r.times[RoutineStatementStartPlanning])
483+
}
484+
func (r *RoutineStatementLatencyRecorder) ProcessingLatency() time.Duration {
485+
return r.ServiceLatency() // TODO: is there a difference between processing latency and service latency?
486+
}
487+
func (r *RoutineStatementLatencyRecorder) ExecOverheadLatency() time.Duration {
488+
return r.ServiceLatency() - r.ProcessingLatency() // Will be 0 since processing and service latency are the same
489+
}
490+
func (r *RoutineStatementLatencyRecorder) StartTime() time.Time {
491+
return r.times[RoutineStatementStarted].ToUTC()
492+
}
493+
func (r *RoutineStatementLatencyRecorder) EndTime() time.Time {
494+
return r.times[RoutineStatementEnd].ToUTC()
495+
}

0 commit comments

Comments
 (0)