Skip to content

Commit ab654a6

Browse files
ui, sql: Record KV CPU Time in statement and transaction statistics
This commit starts recording the newly added kvCPUTime field from topLevelQueryStats and exposes it in the UI. Fixes: https://cockroachlabs.atlassian.net/browse/CRDB-57265 Release note (ui change): KV CPU Time is now recorded to statement_statistics and transaction_statistics and is displayed in the SQL Activity page.
1 parent 7c9dd4b commit ab654a6

File tree

20 files changed

+136
-7
lines changed

20 files changed

+136
-7
lines changed

pkg/sql/appstatspb/app_stats.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -131,6 +131,7 @@ func (t *TransactionStatistics) Add(other *TransactionStatistics) {
131131
t.RowsRead.Add(other.RowsRead, t.Count, other.Count)
132132
t.BytesRead.Add(other.BytesRead, t.Count, other.Count)
133133
t.RowsWritten.Add(other.RowsWritten, t.Count, other.Count)
134+
t.KVCPUTime.Add(other.KVCPUTime, t.Count, other.Count)
134135

135136
t.ExecStats.Add(other.ExecStats)
136137

@@ -181,6 +182,7 @@ func (s *StatementStatistics) Add(other *StatementStatistics) {
181182
s.BytesRead.Add(other.BytesRead, s.Count, other.Count)
182183
s.RowsRead.Add(other.RowsRead, s.Count, other.Count)
183184
s.RowsWritten.Add(other.RowsWritten, s.Count, other.Count)
185+
s.KVCPUTime.Add(other.KVCPUTime, s.Count, other.Count)
184186
s.Nodes = util.CombineUnique(s.Nodes, other.Nodes)
185187
s.KVNodeIDs = util.CombineUnique(s.KVNodeIDs, other.KVNodeIDs)
186188
s.Regions = util.CombineUnique(s.Regions, other.Regions)

pkg/sql/appstatspb/app_stats.proto

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -142,6 +142,11 @@ message StatementStatistics {
142142
// system.statement_hints table.
143143
optional int64 stmt_hints_count = 37 [(gogoproto.nullable) = false];
144144

145+
// kv_cpu_time is the CPU time in nanoseconds spent by KV. This is
146+
// representative of the "synchronous" portion of work performed by KV
147+
// (ex/ not replication related work).
148+
optional NumericStat kv_cpu_time = 38 [(gogoproto.nullable) = false, (gogoproto.customname) = "KVCPUTime"];
149+
145150
// Note: be sure to update `sql/app_stats.go` when adding/removing fields here!
146151

147152
reserved 13, 14, 17, 18, 19, 20;
@@ -188,6 +193,11 @@ message TransactionStatistics {
188193

189194
// RowsWritten collects the number of rows written to disk.
190195
optional NumericStat rows_written = 10 [(gogoproto.nullable) = false];
196+
197+
// kv_cpu_time is the CPU time in nanoseconds spent by KV. This is
198+
// representative of the "synchronous" portion of work performed by KV
199+
// (ex/ not replication related work).
200+
optional NumericStat kv_cpu_time = 12 [(gogoproto.nullable) = false, (gogoproto.customname) = "KVCPUTime"];
191201
}
192202

193203

pkg/sql/conn_executor.go

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -1757,15 +1757,14 @@ type connExecutor struct {
17571757
// client to send statements while holding the transaction open.
17581758
idleLatency time.Duration
17591759

1760-
// rowsRead and bytesRead are separate from QueryLevelStats because they are
1761-
// accumulated independently since they are always collected, as opposed to
1762-
// QueryLevelStats which are sampled.
1760+
// rowsRead, bytesRead, kvCPUTime, and rowsWritten are separate from accumulatedStats
1761+
// since they are always collected as opposed to QueryLevelStats which are sampled.
17631762
rowsRead int64
17641763
bytesRead int64
1765-
17661764
// rowsWritten tracks the number of rows written (modified) by all
17671765
// statements in this txn so far.
17681766
rowsWritten int64
1767+
kvCPUTime time.Duration
17691768

17701769
// rowsWrittenLogged and rowsReadLogged indicates whether we have
17711770
// already logged an event about reaching written/read rows setting,

pkg/sql/conn_executor_exec.go

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2951,6 +2951,7 @@ func (ex *connExecutor) dispatchToExecutionEngine(
29512951
ex.extraTxnState.rowsRead += stats.rowsRead
29522952
ex.extraTxnState.bytesRead += stats.bytesRead
29532953
ex.extraTxnState.rowsWritten += stats.rowsWritten
2954+
ex.extraTxnState.kvCPUTime += stats.kvCPUTime
29542955

29552956
if ppInfo := getPausablePortalInfo(planner); ppInfo != nil && !ppInfo.dispatchToExecutionEngine.cleanup.isComplete {
29562957
// We need to ensure that we're using the planner bound to the first-time
@@ -4199,6 +4200,7 @@ func (ex *connExecutor) onTxnRestart(ctx context.Context) {
41994200
ex.extraTxnState.rowsRead = 0
42004201
ex.extraTxnState.bytesRead = 0
42014202
ex.extraTxnState.rowsWritten = 0
4203+
ex.extraTxnState.kvCPUTime = 0
42024204

42034205
if ex.server.cfg.TestingKnobs.BeforeRestart != nil {
42044206
ex.server.cfg.TestingKnobs.BeforeRestart(ctx, ex.state.mu.autoRetryReason)
@@ -4230,6 +4232,7 @@ func (ex *connExecutor) recordTransactionStart(txnID uuid.UUID) {
42304232
ex.extraTxnState.accumulatedStats = execstats.QueryLevelStats{}
42314233
ex.extraTxnState.idleLatency = 0
42324234
ex.extraTxnState.rowsRead = 0
4235+
ex.extraTxnState.kvCPUTime = 0
42334236
ex.extraTxnState.bytesRead = 0
42344237
ex.extraTxnState.rowsWritten = 0
42354238
ex.extraTxnState.rowsWrittenLogged = false
@@ -4342,6 +4345,7 @@ func (ex *connExecutor) recordTransactionFinish(
43424345
RowsRead: ex.extraTxnState.rowsRead,
43434346
RowsWritten: ex.extraTxnState.rowsWritten,
43444347
BytesRead: ex.extraTxnState.bytesRead,
4348+
KVCPUTime: ex.extraTxnState.kvCPUTime,
43454349
Priority: ex.state.mu.priority,
43464350
// TODO(107318): add isolation level
43474351
// TODO(107318): add qos

pkg/sql/executor_statement_metrics.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -208,7 +208,7 @@ func (ex *connExecutor) recordStatementSummary(
208208
).
209209
PlanGist(planner.instrumentation.planGist.String(), planner.instrumentation.planGist.Hash()).
210210
LatencyRecorder(ex.statsCollector).
211-
QueryLevelStats(stats.bytesRead, stats.rowsRead, stats.rowsWritten).
211+
QueryLevelStats(stats.bytesRead, stats.rowsRead, stats.rowsWritten, stats.kvCPUTime.Nanoseconds()).
212212
ExecStats(queryLevelStats).
213213
// TODO(mgartner): Use a slice of struct{uint64, uint64} instead of
214214
// converting to strings.

pkg/sql/routine.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -415,7 +415,7 @@ func (g *routineGenerator) startInternal(ctx context.Context, txn *kv.Txn) (err
415415
statsBuilder.StatementError(err)
416416
return err
417417
}
418-
statsBuilder.QueryLevelStats(queryStats.bytesRead, queryStats.rowsRead, queryStats.rowsWritten)
418+
statsBuilder.QueryLevelStats(queryStats.bytesRead, queryStats.rowsRead, queryStats.rowsWritten, queryStats.kvCPUTime.Nanoseconds())
419419
forwardInnerQueryStats(g.p.routineMetadataForwarder, queryStats)
420420
if openCursor {
421421
return cursorHelper.createCursor(g.p)

pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_encoding_test.go

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -97,6 +97,10 @@ func TestSQLStatsJsonEncoding(t *testing.T) {
9797
"mean": {{.Float}},
9898
"sqDiff": {{.Float}}
9999
},
100+
"kvCPUTime": {
101+
"mean": {{.Float}},
102+
"sqDiff": {{.Float}}
103+
},
100104
"nodes": [{{joinInts .IntArray}}],
101105
"kvNodeIds": [{{joinInt32s .Int32Array}}],
102106
"regions": [{{joinStrings .StringArray}}],
@@ -470,6 +474,10 @@ func TestSQLStatsJsonEncoding(t *testing.T) {
470474
"rowsWritten": {
471475
"mean": {{.Float}},
472476
"sqDiff": {{.Float}}
477+
},
478+
"kvCPUTime": {
479+
"mean": {{.Float}},
480+
"sqDiff": {{.Float}}
473481
}
474482
},
475483
"execution_statistics": {

pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_impl.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -306,6 +306,7 @@ func (t *innerTxnStats) jsonFields() jsonFields {
306306
{"bytesRead", (*numericStats)(&t.BytesRead)},
307307
{"rowsRead", (*numericStats)(&t.RowsRead)},
308308
{"rowsWritten", (*numericStats)(&t.RowsWritten)},
309+
{"kvCPUTime", (*numericStats)(&t.KVCPUTime)},
309310
}
310311
}
311312

@@ -335,6 +336,7 @@ func (s *innerStmtStats) jsonFields() jsonFields {
335336
{"bytesRead", (*numericStats)(&s.BytesRead)},
336337
{"rowsRead", (*numericStats)(&s.RowsRead)},
337338
{"rowsWritten", (*numericStats)(&s.RowsWritten)},
339+
{"kvCPUTime", (*numericStats)(&s.KVCPUTime)},
338340
{"nodes", (*int64Array)(&s.Nodes)},
339341
{"kvNodeIds", (*int32Array)(&s.KVNodeIDs)},
340342
{"regions", (*stringArray)(&s.Regions)},

pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -100,6 +100,7 @@ func (s *Container) RecordStatement(ctx context.Context, value *sqlstats.Recorde
100100
stats.mu.data.BytesRead.Record(stats.mu.data.Count, float64(value.BytesRead))
101101
stats.mu.data.RowsRead.Record(stats.mu.data.Count, float64(value.RowsRead))
102102
stats.mu.data.RowsWritten.Record(stats.mu.data.Count, float64(value.RowsWritten))
103+
stats.mu.data.KVCPUTime.Record(stats.mu.data.Count, float64(value.KVCPUTime))
103104
stats.mu.data.LastExecTimestamp = s.getTimeNow()
104105
stats.mu.data.Nodes = util.CombineUnique(stats.mu.data.Nodes, value.Nodes)
105106
stats.mu.data.KVNodeIDs = util.CombineUnique(stats.mu.data.KVNodeIDs, value.KVNodeIDs)
@@ -239,6 +240,7 @@ func (s *Container) RecordTransaction(ctx context.Context, value *sqlstats.Recor
239240
stats.mu.data.RowsRead.Record(stats.mu.data.Count, float64(value.RowsRead))
240241
stats.mu.data.RowsWritten.Record(stats.mu.data.Count, float64(value.RowsWritten))
241242
stats.mu.data.BytesRead.Record(stats.mu.data.Count, float64(value.BytesRead))
243+
stats.mu.data.KVCPUTime.Record(stats.mu.data.Count, float64(value.KVCPUTime.Nanoseconds()))
242244

243245
if value.CollectedExecStats {
244246
stats.mu.data.ExecStats.Count++

pkg/sql/sqlstats/ssmemstorage/ss_mem_writer_test.go

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -117,6 +117,7 @@ func TestContainer_Add(t *testing.T) {
117117
BytesRead: 100,
118118
RowsRead: 20,
119119
RowsWritten: 5,
120+
KVCPUTime: 30,
120121
Failed: true,
121122
Generic: true,
122123
AppliedStmtHints: true,
@@ -137,6 +138,7 @@ func TestContainer_Add(t *testing.T) {
137138
RowsRead: 20,
138139
RowsWritten: 5,
139140
BytesRead: 100,
141+
KVCPUTime: 30 * time.Nanosecond,
140142
}
141143
require.NoError(t, src.RecordTransaction(ctx, txnStats))
142144

@@ -160,6 +162,7 @@ func TestContainer_Add(t *testing.T) {
160162
BytesRead: 60,
161163
RowsRead: 70,
162164
RowsWritten: 80,
165+
KVCPUTime: 90,
163166
Failed: true,
164167
Generic: true,
165168
AppliedStmtHints: true,
@@ -177,6 +180,7 @@ func TestContainer_Add(t *testing.T) {
177180
RowsRead: 20,
178181
RowsWritten: 5,
179182
BytesRead: 100,
183+
KVCPUTime: 90 * time.Nanosecond,
180184
}
181185
require.NoError(t, dest.RecordStatement(ctx, reducedStmtStats))
182186
require.NoError(t, dest.RecordTransaction(ctx, reducedTxnStats))
@@ -221,6 +225,7 @@ func verifyStmtStatsMultiple(
221225
require.InEpsilon(t, float64(stmtStats.BytesRead), destStmtStats.mu.data.BytesRead.Mean, epsilon)
222226
require.InEpsilon(t, float64(stmtStats.RowsRead), destStmtStats.mu.data.RowsRead.Mean, epsilon)
223227
require.InEpsilon(t, float64(stmtStats.RowsWritten), destStmtStats.mu.data.RowsWritten.Mean, epsilon)
228+
require.InEpsilon(t, float64(stmtStats.KVCPUTime), destStmtStats.mu.data.KVCPUTime.Mean, epsilon)
224229
}
225230

226231
// verifyStmtStatsReduced verifies that statement statistics have been properly
@@ -241,6 +246,7 @@ func verifyStmtStatsReduced(
241246
require.InEpsilon(t, float64(stmtStats.BytesRead)/cnt, destStmtStats.mu.data.BytesRead.Mean, epsilon)
242247
require.InEpsilon(t, float64(stmtStats.RowsRead)/cnt, destStmtStats.mu.data.RowsRead.Mean, epsilon)
243248
require.InEpsilon(t, float64(stmtStats.RowsWritten)/cnt, destStmtStats.mu.data.RowsWritten.Mean, epsilon)
249+
require.InEpsilon(t, float64(stmtStats.KVCPUTime)/cnt, destStmtStats.mu.data.KVCPUTime.Mean, epsilon)
244250
}
245251

246252
// verifyTxnStatsMultiple verifies that transaction statistics have been recorded
@@ -259,6 +265,7 @@ func verifyTxnStatsMultiple(
259265
require.InEpsilon(t, float64(txnStats.RowsRead), destTxnStats.mu.data.RowsRead.Mean, epsilon)
260266
require.InEpsilon(t, float64(txnStats.RowsWritten), destTxnStats.mu.data.RowsWritten.Mean, epsilon)
261267
require.InEpsilon(t, float64(txnStats.BytesRead), destTxnStats.mu.data.BytesRead.Mean, epsilon)
268+
require.InEpsilon(t, float64(txnStats.KVCPUTime.Nanoseconds()), destTxnStats.mu.data.KVCPUTime.Mean, epsilon)
262269
}
263270

264271
// verifyTxnStatsReduced verifies that transaction statistics have been properly
@@ -277,6 +284,7 @@ func verifyTxnStatsReduced(
277284
require.InEpsilon(t, float64(txnStats.RowsRead)/cnt, destTxnStats.mu.data.RowsRead.Mean, epsilon)
278285
require.InEpsilon(t, float64(txnStats.RowsWritten)/cnt, destTxnStats.mu.data.RowsWritten.Mean, epsilon)
279286
require.InEpsilon(t, float64(txnStats.BytesRead)/cnt, destTxnStats.mu.data.BytesRead.Mean, epsilon)
287+
require.InEpsilon(t, float64(txnStats.KVCPUTime.Nanoseconds())/cnt, destTxnStats.mu.data.KVCPUTime.Mean, epsilon)
280288
}
281289

282290
func testMonitor(

0 commit comments

Comments
 (0)