From 9a840826e20c22fd0d447216d6dfc31d38efba73 Mon Sep 17 00:00:00 2001 From: alyshanjahani-crl Date: Wed, 26 Nov 2025 10:52:48 -0500 Subject: [PATCH] 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. --- pkg/sql/appstatspb/app_stats.go | 2 ++ pkg/sql/appstatspb/app_stats.proto | 10 ++++++++ pkg/sql/conn_executor.go | 9 ++++---- pkg/sql/conn_executor_exec.go | 11 ++++++--- pkg/sql/distsql_running.go | 4 ++-- pkg/sql/executor_statement_metrics.go | 2 +- pkg/sql/routine.go | 2 +- .../sqlstatsutil/json_encoding_test.go | 8 +++++++ .../sqlstatsutil/json_impl.go | 2 ++ .../sqlstats/ssmemstorage/ss_mem_writer.go | 2 ++ .../ssmemstorage/ss_mem_writer_test.go | 8 +++++++ pkg/sql/sqlstats/ssprovider.go | 5 +++- .../cluster-ui/src/api/statementsApi.ts | 2 ++ .../cluster-ui/src/barCharts/barCharts.tsx | 16 +++++++++++++ .../src/statementDetails/statementDetails.tsx | 7 ++++++ .../src/statementsTable/statementsTable.tsx | 12 ++++++++++ .../src/statsTableUtil/statsTableUtil.tsx | 23 +++++++++++++++++++ .../transactionDetailsUtils.tsx | 6 +++++ .../transactionsBarCharts.ts | 15 ++++++++++++ .../transactionsTable/transactionsTable.tsx | 13 +++++++++++ .../cluster-ui/src/util/appStats/appStats.ts | 6 +++++ 21 files changed, 152 insertions(+), 13 deletions(-) diff --git a/pkg/sql/appstatspb/app_stats.go b/pkg/sql/appstatspb/app_stats.go index 61668b01d34e..955f3664c55b 100644 --- a/pkg/sql/appstatspb/app_stats.go +++ b/pkg/sql/appstatspb/app_stats.go @@ -131,6 +131,7 @@ func (t *TransactionStatistics) Add(other *TransactionStatistics) { t.RowsRead.Add(other.RowsRead, t.Count, other.Count) t.BytesRead.Add(other.BytesRead, t.Count, other.Count) t.RowsWritten.Add(other.RowsWritten, t.Count, other.Count) + t.KVCPUTimeNanos.Add(other.KVCPUTimeNanos, t.Count, other.Count) t.ExecStats.Add(other.ExecStats) @@ -181,6 +182,7 @@ func (s *StatementStatistics) Add(other *StatementStatistics) { s.BytesRead.Add(other.BytesRead, s.Count, other.Count) s.RowsRead.Add(other.RowsRead, s.Count, other.Count) s.RowsWritten.Add(other.RowsWritten, s.Count, other.Count) + s.KVCPUTimeNanos.Add(other.KVCPUTimeNanos, s.Count, other.Count) s.Nodes = util.CombineUnique(s.Nodes, other.Nodes) s.KVNodeIDs = util.CombineUnique(s.KVNodeIDs, other.KVNodeIDs) s.Regions = util.CombineUnique(s.Regions, other.Regions) diff --git a/pkg/sql/appstatspb/app_stats.proto b/pkg/sql/appstatspb/app_stats.proto index d18300a08b7c..4255308395c9 100644 --- a/pkg/sql/appstatspb/app_stats.proto +++ b/pkg/sql/appstatspb/app_stats.proto @@ -142,6 +142,11 @@ message StatementStatistics { // system.statement_hints table. optional int64 stmt_hints_count = 37 [(gogoproto.nullable) = false]; + // kv_cpu_time_nanos is the CPU time in nanoseconds spent by KV. This is + // representative of the "synchronous" portion of work performed by KV + // (ex/ not replication related work). + optional NumericStat kv_cpu_time_nanos = 38 [(gogoproto.nullable) = false, (gogoproto.customname) = "KVCPUTimeNanos"]; + // Note: be sure to update `sql/app_stats.go` when adding/removing fields here! reserved 13, 14, 17, 18, 19, 20; @@ -188,6 +193,11 @@ message TransactionStatistics { // RowsWritten collects the number of rows written to disk. optional NumericStat rows_written = 10 [(gogoproto.nullable) = false]; + + // kv_cpu_time_nanos is the CPU time in nanoseconds spent by KV. This is + // representative of the "synchronous" portion of work performed by KV + // (ex/ not replication related work). + optional NumericStat kv_cpu_time_nanos = 12 [(gogoproto.nullable) = false, (gogoproto.customname) = "KVCPUTimeNanos"]; } diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index 359f71a7cb75..299afbf9e8f9 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -1757,15 +1757,14 @@ type connExecutor struct { // client to send statements while holding the transaction open. idleLatency time.Duration - // rowsRead and bytesRead are separate from QueryLevelStats because they are - // accumulated independently since they are always collected, as opposed to - // QueryLevelStats which are sampled. + // rowsRead, bytesRead, kvCPUTimeNanos, and rowsWritten are separate from accumulatedStats + // since they are always collected as opposed to QueryLevelStats which are sampled. rowsRead int64 bytesRead int64 - // rowsWritten tracks the number of rows written (modified) by all // statements in this txn so far. - rowsWritten int64 + rowsWritten int64 + kvCPUTimeNanos time.Duration // rowsWrittenLogged and rowsReadLogged indicates whether we have // already logged an event about reaching written/read rows setting, diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index ef61ac36c4d9..d64880b4414b 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -59,6 +59,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/cockroach/pkg/util/fsm" "github.com/cockroachdb/cockroach/pkg/util/hlc" + // This import is needed here to properly inject tree.ValidateJSONPath from // pkg/util/jsonpath/parser/parse.go. _ "github.com/cockroachdb/cockroach/pkg/util/jsonpath/parser" @@ -2951,6 +2952,7 @@ func (ex *connExecutor) dispatchToExecutionEngine( ex.extraTxnState.rowsRead += stats.rowsRead ex.extraTxnState.bytesRead += stats.bytesRead ex.extraTxnState.rowsWritten += stats.rowsWritten + ex.extraTxnState.kvCPUTimeNanos += stats.kvCPUTimeNanos if ppInfo := getPausablePortalInfo(planner); ppInfo != nil && !ppInfo.dispatchToExecutionEngine.cleanup.isComplete { // We need to ensure that we're using the planner bound to the first-time @@ -3339,8 +3341,8 @@ type topLevelQueryStats struct { // client receiving the PGWire protocol messages (as well as construcing // those messages). clientTime time.Duration - // kvCPUTime is the CPU time consumed by KV operations during query execution. - kvCPUTime time.Duration + // kvCPUTimeNanos is the CPU time consumed by KV operations during query execution. + kvCPUTimeNanos time.Duration // NB: when adding another field here, consider whether // forwardInnerQueryStats method needs an adjustment. } @@ -3353,7 +3355,7 @@ func (s *topLevelQueryStats) add(other *topLevelQueryStats) { s.indexRowsWritten += other.indexRowsWritten s.networkEgressEstimate += other.networkEgressEstimate s.clientTime += other.clientTime - s.kvCPUTime += other.kvCPUTime + s.kvCPUTimeNanos += other.kvCPUTimeNanos } // execWithDistSQLEngine converts a plan to a distributed SQL physical plan and @@ -4199,6 +4201,7 @@ func (ex *connExecutor) onTxnRestart(ctx context.Context) { ex.extraTxnState.rowsRead = 0 ex.extraTxnState.bytesRead = 0 ex.extraTxnState.rowsWritten = 0 + ex.extraTxnState.kvCPUTimeNanos = 0 if ex.server.cfg.TestingKnobs.BeforeRestart != nil { ex.server.cfg.TestingKnobs.BeforeRestart(ctx, ex.state.mu.autoRetryReason) @@ -4230,6 +4233,7 @@ func (ex *connExecutor) recordTransactionStart(txnID uuid.UUID) { ex.extraTxnState.accumulatedStats = execstats.QueryLevelStats{} ex.extraTxnState.idleLatency = 0 ex.extraTxnState.rowsRead = 0 + ex.extraTxnState.kvCPUTimeNanos = 0 ex.extraTxnState.bytesRead = 0 ex.extraTxnState.rowsWritten = 0 ex.extraTxnState.rowsWrittenLogged = false @@ -4342,6 +4346,7 @@ func (ex *connExecutor) recordTransactionFinish( RowsRead: ex.extraTxnState.rowsRead, RowsWritten: ex.extraTxnState.rowsWritten, BytesRead: ex.extraTxnState.bytesRead, + KVCPUTimeNanos: ex.extraTxnState.kvCPUTimeNanos, Priority: ex.state.mu.priority, // TODO(107318): add isolation level // TODO(107318): add qos diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index 5e1c7b9af667..c7d88dd35de5 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -1625,7 +1625,7 @@ func forwardInnerQueryStats(f metadataForwarder, stats topLevelQueryStats) { meta.Metrics.RowsWritten = stats.rowsWritten meta.Metrics.IndexRowsWritten = stats.indexRowsWritten meta.Metrics.IndexBytesWritten = stats.indexBytesWritten - meta.Metrics.KVCPUTime = int64(stats.kvCPUTime) + meta.Metrics.KVCPUTime = int64(stats.kvCPUTimeNanos) // stats.networkEgressEstimate and stats.clientTime are ignored since they // only matter at the "true" top-level query (and actually should be zero // here anyway). @@ -1676,7 +1676,7 @@ func (r *DistSQLReceiver) pushMeta(meta *execinfrapb.ProducerMetadata) execinfra r.stats.rowsWritten += meta.Metrics.RowsWritten r.stats.indexRowsWritten += meta.Metrics.IndexRowsWritten r.stats.indexBytesWritten += meta.Metrics.IndexBytesWritten - r.stats.kvCPUTime += time.Duration(meta.Metrics.KVCPUTime) + r.stats.kvCPUTimeNanos += time.Duration(meta.Metrics.KVCPUTime) if sm, ok := r.scanStageEstimateMap[meta.Metrics.StageID]; ok { sm.rowsRead += uint64(meta.Metrics.RowsRead) diff --git a/pkg/sql/executor_statement_metrics.go b/pkg/sql/executor_statement_metrics.go index bdfbcb74d382..da2e759a3958 100644 --- a/pkg/sql/executor_statement_metrics.go +++ b/pkg/sql/executor_statement_metrics.go @@ -208,7 +208,7 @@ func (ex *connExecutor) recordStatementSummary( ). PlanGist(planner.instrumentation.planGist.String(), planner.instrumentation.planGist.Hash()). LatencyRecorder(ex.statsCollector). - QueryLevelStats(stats.bytesRead, stats.rowsRead, stats.rowsWritten). + QueryLevelStats(stats.bytesRead, stats.rowsRead, stats.rowsWritten, stats.kvCPUTimeNanos.Nanoseconds()). ExecStats(queryLevelStats). // TODO(mgartner): Use a slice of struct{uint64, uint64} instead of // converting to strings. diff --git a/pkg/sql/routine.go b/pkg/sql/routine.go index 153507e543ea..4a7d8a34c6e4 100644 --- a/pkg/sql/routine.go +++ b/pkg/sql/routine.go @@ -415,7 +415,7 @@ func (g *routineGenerator) startInternal(ctx context.Context, txn *kv.Txn) (err statsBuilder.StatementError(err) return err } - statsBuilder.QueryLevelStats(queryStats.bytesRead, queryStats.rowsRead, queryStats.rowsWritten) + statsBuilder.QueryLevelStats(queryStats.bytesRead, queryStats.rowsRead, queryStats.rowsWritten, queryStats.kvCPUTimeNanos.Nanoseconds()) forwardInnerQueryStats(g.p.routineMetadataForwarder, queryStats) if openCursor { return cursorHelper.createCursor(g.p) diff --git a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_encoding_test.go b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_encoding_test.go index ea6851def635..740a0f87da98 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_encoding_test.go +++ b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_encoding_test.go @@ -97,6 +97,10 @@ func TestSQLStatsJsonEncoding(t *testing.T) { "mean": {{.Float}}, "sqDiff": {{.Float}} }, + "kvCPUTimeNanos": { + "mean": {{.Float}}, + "sqDiff": {{.Float}} + }, "nodes": [{{joinInts .IntArray}}], "kvNodeIds": [{{joinInt32s .Int32Array}}], "regions": [{{joinStrings .StringArray}}], @@ -470,6 +474,10 @@ func TestSQLStatsJsonEncoding(t *testing.T) { "rowsWritten": { "mean": {{.Float}}, "sqDiff": {{.Float}} + }, + "kvCPUTimeNanos": { + "mean": {{.Float}}, + "sqDiff": {{.Float}} } }, "execution_statistics": { diff --git a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_impl.go b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_impl.go index 3d76eac536f8..a1d427cb1541 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_impl.go +++ b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_impl.go @@ -306,6 +306,7 @@ func (t *innerTxnStats) jsonFields() jsonFields { {"bytesRead", (*numericStats)(&t.BytesRead)}, {"rowsRead", (*numericStats)(&t.RowsRead)}, {"rowsWritten", (*numericStats)(&t.RowsWritten)}, + {"kvCPUTimeNanos", (*numericStats)(&t.KVCPUTimeNanos)}, } } @@ -335,6 +336,7 @@ func (s *innerStmtStats) jsonFields() jsonFields { {"bytesRead", (*numericStats)(&s.BytesRead)}, {"rowsRead", (*numericStats)(&s.RowsRead)}, {"rowsWritten", (*numericStats)(&s.RowsWritten)}, + {"kvCPUTimeNanos", (*numericStats)(&s.KVCPUTimeNanos)}, {"nodes", (*int64Array)(&s.Nodes)}, {"kvNodeIds", (*int32Array)(&s.KVNodeIDs)}, {"regions", (*stringArray)(&s.Regions)}, diff --git a/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go b/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go index 6867261232e4..a7b7bbf4fdfb 100644 --- a/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go +++ b/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go @@ -100,6 +100,7 @@ func (s *Container) RecordStatement(ctx context.Context, value *sqlstats.Recorde stats.mu.data.BytesRead.Record(stats.mu.data.Count, float64(value.BytesRead)) stats.mu.data.RowsRead.Record(stats.mu.data.Count, float64(value.RowsRead)) stats.mu.data.RowsWritten.Record(stats.mu.data.Count, float64(value.RowsWritten)) + stats.mu.data.KVCPUTimeNanos.Record(stats.mu.data.Count, float64(value.KVCPUTimeNanos)) stats.mu.data.LastExecTimestamp = s.getTimeNow() stats.mu.data.Nodes = util.CombineUnique(stats.mu.data.Nodes, value.Nodes) 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 stats.mu.data.RowsRead.Record(stats.mu.data.Count, float64(value.RowsRead)) stats.mu.data.RowsWritten.Record(stats.mu.data.Count, float64(value.RowsWritten)) stats.mu.data.BytesRead.Record(stats.mu.data.Count, float64(value.BytesRead)) + stats.mu.data.KVCPUTimeNanos.Record(stats.mu.data.Count, float64(value.KVCPUTimeNanos.Nanoseconds())) if value.CollectedExecStats { stats.mu.data.ExecStats.Count++ diff --git a/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer_test.go b/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer_test.go index 020db798ec4b..30b75ce8c2a0 100644 --- a/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer_test.go +++ b/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer_test.go @@ -117,6 +117,7 @@ func TestContainer_Add(t *testing.T) { BytesRead: 100, RowsRead: 20, RowsWritten: 5, + KVCPUTimeNanos: 30, Failed: true, Generic: true, AppliedStmtHints: true, @@ -137,6 +138,7 @@ func TestContainer_Add(t *testing.T) { RowsRead: 20, RowsWritten: 5, BytesRead: 100, + KVCPUTimeNanos: 30 * time.Nanosecond, } require.NoError(t, src.RecordTransaction(ctx, txnStats)) @@ -160,6 +162,7 @@ func TestContainer_Add(t *testing.T) { BytesRead: 60, RowsRead: 70, RowsWritten: 80, + KVCPUTimeNanos: 90, Failed: true, Generic: true, AppliedStmtHints: true, @@ -177,6 +180,7 @@ func TestContainer_Add(t *testing.T) { RowsRead: 20, RowsWritten: 5, BytesRead: 100, + KVCPUTimeNanos: 90 * time.Nanosecond, } require.NoError(t, dest.RecordStatement(ctx, reducedStmtStats)) require.NoError(t, dest.RecordTransaction(ctx, reducedTxnStats)) @@ -221,6 +225,7 @@ func verifyStmtStatsMultiple( require.InEpsilon(t, float64(stmtStats.BytesRead), destStmtStats.mu.data.BytesRead.Mean, epsilon) require.InEpsilon(t, float64(stmtStats.RowsRead), destStmtStats.mu.data.RowsRead.Mean, epsilon) require.InEpsilon(t, float64(stmtStats.RowsWritten), destStmtStats.mu.data.RowsWritten.Mean, epsilon) + require.InEpsilon(t, float64(stmtStats.KVCPUTimeNanos), destStmtStats.mu.data.KVCPUTimeNanos.Mean, epsilon) } // verifyStmtStatsReduced verifies that statement statistics have been properly @@ -241,6 +246,7 @@ func verifyStmtStatsReduced( require.InEpsilon(t, float64(stmtStats.BytesRead)/cnt, destStmtStats.mu.data.BytesRead.Mean, epsilon) require.InEpsilon(t, float64(stmtStats.RowsRead)/cnt, destStmtStats.mu.data.RowsRead.Mean, epsilon) require.InEpsilon(t, float64(stmtStats.RowsWritten)/cnt, destStmtStats.mu.data.RowsWritten.Mean, epsilon) + require.InEpsilon(t, float64(stmtStats.KVCPUTimeNanos)/cnt, destStmtStats.mu.data.KVCPUTimeNanos.Mean, epsilon) } // verifyTxnStatsMultiple verifies that transaction statistics have been recorded @@ -259,6 +265,7 @@ func verifyTxnStatsMultiple( require.InEpsilon(t, float64(txnStats.RowsRead), destTxnStats.mu.data.RowsRead.Mean, epsilon) require.InEpsilon(t, float64(txnStats.RowsWritten), destTxnStats.mu.data.RowsWritten.Mean, epsilon) require.InEpsilon(t, float64(txnStats.BytesRead), destTxnStats.mu.data.BytesRead.Mean, epsilon) + require.InEpsilon(t, float64(txnStats.KVCPUTimeNanos.Nanoseconds()), destTxnStats.mu.data.KVCPUTimeNanos.Mean, epsilon) } // verifyTxnStatsReduced verifies that transaction statistics have been properly @@ -277,6 +284,7 @@ func verifyTxnStatsReduced( require.InEpsilon(t, float64(txnStats.RowsRead)/cnt, destTxnStats.mu.data.RowsRead.Mean, epsilon) require.InEpsilon(t, float64(txnStats.RowsWritten)/cnt, destTxnStats.mu.data.RowsWritten.Mean, epsilon) require.InEpsilon(t, float64(txnStats.BytesRead)/cnt, destTxnStats.mu.data.BytesRead.Mean, epsilon) + require.InEpsilon(t, float64(txnStats.KVCPUTimeNanos.Nanoseconds())/cnt, destTxnStats.mu.data.KVCPUTimeNanos.Mean, epsilon) } func testMonitor( diff --git a/pkg/sql/sqlstats/ssprovider.go b/pkg/sql/sqlstats/ssprovider.go index 80dfa486de21..4bad4928a9ca 100644 --- a/pkg/sql/sqlstats/ssprovider.go +++ b/pkg/sql/sqlstats/ssprovider.go @@ -85,6 +85,7 @@ type RecordedStmtStats struct { BytesRead int64 RowsRead int64 RowsWritten int64 + KVCPUTimeNanos int64 Nodes []int64 KVNodeIDs []int32 StatementType tree.StatementType @@ -123,6 +124,7 @@ type RecordedTxnStats struct { RowsRead int64 RowsWritten int64 BytesRead int64 + KVCPUTimeNanos time.Duration Priority roachpb.UserPriority TxnErr error Application string @@ -233,7 +235,7 @@ func (b *RecordedStatementStatsBuilder) LatencyRecorder( } func (b *RecordedStatementStatsBuilder) QueryLevelStats( - bytesRead int64, rowsRead int64, rowsWritten int64, + bytesRead int64, rowsRead int64, rowsWritten int64, kvCPUTime int64, ) *RecordedStatementStatsBuilder { if b == nil { return b @@ -241,6 +243,7 @@ func (b *RecordedStatementStatsBuilder) QueryLevelStats( b.stmtStats.BytesRead = bytesRead b.stmtStats.RowsRead = rowsRead b.stmtStats.RowsWritten = rowsWritten + b.stmtStats.KVCPUTimeNanos = kvCPUTime return b } diff --git a/pkg/ui/workspaces/cluster-ui/src/api/statementsApi.ts b/pkg/ui/workspaces/cluster-ui/src/api/statementsApi.ts index 4dba4d8bdc49..df0491050088 100644 --- a/pkg/ui/workspaces/cluster-ui/src/api/statementsApi.ts +++ b/pkg/ui/workspaces/cluster-ui/src/api/statementsApi.ts @@ -170,6 +170,7 @@ type Statistics = { runLat: NumericStat; svcLat: NumericStat; regions: string[]; + kvCPUTimeNanos?: NumericStat; }; type ExecStats = { @@ -253,6 +254,7 @@ export function convertStatementRawFormatToAggregatedStatistics( service_lat: s.statistics.statistics.svcLat, sql_type: s.metadata.stmtType, regions: s.statistics.statistics.regions, + kv_cpu_time_nanos: s.statistics.statistics.kvCPUTimeNanos, }, }; } diff --git a/pkg/ui/workspaces/cluster-ui/src/barCharts/barCharts.tsx b/pkg/ui/workspaces/cluster-ui/src/barCharts/barCharts.tsx index f8f6f1a79c53..dfa5f2b7d007 100644 --- a/pkg/ui/workspaces/cluster-ui/src/barCharts/barCharts.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/barCharts/barCharts.tsx @@ -52,6 +52,13 @@ const cpuBars = [ ), ]; +const kvCPUTimeBars = [ + bar( + "kv-cpu-time", + (d: StatementStatistics) => d.stats.kv_cpu_time_nanos?.mean, + ), +]; + const maxMemUsageBars = [ bar( "max-mem-usage", @@ -87,6 +94,9 @@ const contentionStdDev = bar(cx("contention-dev"), (d: StatementStatistics) => const cpuStdDev = bar(cx("cpu-dev"), (d: StatementStatistics) => stdDevLong(d.stats.exec_stats.cpu_sql_nanos, d.stats.exec_stats.count), ); +const kvCPUTimeStdDev = bar(cx("kv-cpu-time-dev"), (d: StatementStatistics) => + stdDevLong(d.stats.kv_cpu_time_nanos, d.stats.count), +); const maxMemUsageStdDev = bar( cx("max-mem-usage-dev"), (d: StatementStatistics) => @@ -123,6 +133,12 @@ export const cpuBarChart = barChartFactory( v => Duration(v), cpuStdDev, ); +export const kvCPUTimeBarChart = barChartFactory( + "grey", + kvCPUTimeBars, + v => Duration(v), + kvCPUTimeStdDev, +); export const maxMemUsageBarChart = barChartFactory( "grey", maxMemUsageBars, diff --git a/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.tsx b/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.tsx index fc0ebeb9b776..b2b7d1e058bf 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.tsx @@ -837,6 +837,13 @@ export class StatementDetails extends React.Component< Duration, )} /> + FixLong(Number(stmt.stats.exec_stats.cpu_sql_nanos?.mean)), }, + { + name: "kvCPUTime", + title: statisticsTableTitles.kvCPUTime(statType), + cell: kvCPUTimeBar, + sort: (stmt: AggregateStatistics) => + FixLong(Number(stmt.stats.kv_cpu_time_nanos?.mean)), + }, { name: "latencyMin", title: statisticsTableTitles.latencyMin(statType), diff --git a/pkg/ui/workspaces/cluster-ui/src/statsTableUtil/statsTableUtil.tsx b/pkg/ui/workspaces/cluster-ui/src/statsTableUtil/statsTableUtil.tsx index eea34c570c61..32f5d71bfa32 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statsTableUtil/statsTableUtil.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/statsTableUtil/statsTableUtil.tsx @@ -27,6 +27,7 @@ export const statisticsColumnLabels = { clientAddress: "Client IP Address", contention: "Contention Time", cpu: "SQL CPU Time", + kvCPUTime: "KV CPU Time", database: "Database", diagnostics: "Diagnostics", executionCount: "Execution Count", @@ -684,6 +685,28 @@ export const statisticsTableTitles: StatisticTableTitleType = { ); }, + kvCPUTime: (_: StatisticType) => { + return ( + +

+ Average KV CPU time spent executing within the specified time + interval. This can be thought of as KV work that is on the + critical path of serving the query. It does not include any + asynchronous replication related work. The gray bar indicates mean + KV CPU time. The blue bar indicates one standard deviation from + the mean. +

+ + } + > + {getLabel("kvCPUTime")} +
+ ); + }, maxMemUsage: (statType: StatisticType) => { let contentModifier = ""; let fingerprintModifier = ""; diff --git a/pkg/ui/workspaces/cluster-ui/src/transactionDetails/transactionDetailsUtils.tsx b/pkg/ui/workspaces/cluster-ui/src/transactionDetails/transactionDetailsUtils.tsx index fd5bbf408913..daa3f0ef9e59 100644 --- a/pkg/ui/workspaces/cluster-ui/src/transactionDetails/transactionDetailsUtils.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/transactionDetails/transactionDetailsUtils.tsx @@ -158,6 +158,12 @@ function addTransactionStats( countB, ), exec_stats: addExecStats(a.exec_stats, b.exec_stats), + kv_cpu_time_nanos: aggregateNumericStats( + a.kv_cpu_time_nanos, + b.kv_cpu_time_nanos, + countA, + countB, + ), }; } diff --git a/pkg/ui/workspaces/cluster-ui/src/transactionsTable/transactionsBarCharts.ts b/pkg/ui/workspaces/cluster-ui/src/transactionsTable/transactionsBarCharts.ts index 282d1384c5dc..6b10884d6e07 100644 --- a/pkg/ui/workspaces/cluster-ui/src/transactionsTable/transactionsBarCharts.ts +++ b/pkg/ui/workspaces/cluster-ui/src/transactionsTable/transactionsBarCharts.ts @@ -75,6 +75,15 @@ const cpuStdDev = bar(cx("cpu-dev"), (d: Transaction) => d.stats_data.stats.exec_stats.count, ), ); +const kvCPUTimeBar = [ + bar( + "kv-cpu-time", + (d: TransactionInfo) => d.stats_data.stats.kv_cpu_time_nanos?.mean, + ), +]; +const kvCPUTimeStdDev = bar(cx("kv-cpu-time-dev"), (d: Transaction) => + stdDevLong(d.stats_data.stats.kv_cpu_time_nanos, d.stats_data.stats.count), +); const maxMemUsageBar = [ bar("max-mem-usage", (d: TransactionInfo) => longToInt(d.stats_data.stats.exec_stats.max_mem_usage?.mean), @@ -138,6 +147,12 @@ export const transactionsCPUBarChart = barChartFactory( v => Duration(v), cpuStdDev, ); +export const transactionsKVCPUTimeBarChart = barChartFactory( + "grey", + kvCPUTimeBar, + v => Duration(v), + kvCPUTimeStdDev, +); export const transactionsMaxMemUsageBarChart = barChartFactory( "grey", maxMemUsageBar, diff --git a/pkg/ui/workspaces/cluster-ui/src/transactionsTable/transactionsTable.tsx b/pkg/ui/workspaces/cluster-ui/src/transactionsTable/transactionsTable.tsx index 8411de957a76..0b8446906efd 100644 --- a/pkg/ui/workspaces/cluster-ui/src/transactionsTable/transactionsTable.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/transactionsTable/transactionsTable.tsx @@ -40,6 +40,7 @@ import { transactionsServiceLatencyBarChart, transactionsContentionBarChart, transactionsCPUBarChart, + transactionsKVCPUTimeBarChart, transactionsMaxMemUsageBarChart, transactionsNetworkBytesBarChart, transactionsRetryBarChart, @@ -131,6 +132,10 @@ export function makeTransactionsColumns( transactions, sampledExecStatsBarChartOptions, ); + const kvCPUTimeBar = transactionsKVCPUTimeBarChart( + transactions, + defaultBarChartOptions, // kvCPUTime is always collected, it is not part of the sampled exec stats. + ); const maxMemUsageBar = transactionsMaxMemUsageBarChart( transactions, sampledExecStatsBarChartOptions, @@ -239,6 +244,14 @@ export function makeTransactionsColumns( sort: (item: TransactionInfo) => FixLong(Number(item.stats_data.stats.exec_stats.cpu_sql_nanos?.mean)), }, + { + name: "kvCPUTime", + title: statisticsTableTitles.kvCPUTime(statType), + cell: kvCPUTimeBar, + className: cx("statements-table__col-kv-cpu-time"), + sort: (item: TransactionInfo) => + FixLong(Number(item.stats_data.stats.kv_cpu_time_nanos?.mean)), + }, { name: "maxMemUsage", title: statisticsTableTitles.maxMemUsage(statType), diff --git a/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.ts b/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.ts index dcc417f1f46b..124f1708272c 100644 --- a/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.ts +++ b/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.ts @@ -231,6 +231,12 @@ export function addStatementStats( countA, countB, ), + kv_cpu_time_nanos: aggregateNumericStats( + a.kv_cpu_time_nanos, + b.kv_cpu_time_nanos, + countA, + countB, + ), rows_read: aggregateNumericStats(a.rows_read, b.rows_read, countA, countB), rows_written: aggregateNumericStats( a.rows_written,