From 35a1a5a113220214c3e88ee67200f8485cfdd099 Mon Sep 17 00:00:00 2001 From: Drew Kimball Date: Mon, 10 Nov 2025 20:46:02 -0600 Subject: [PATCH 1/3] builtins: decrease wait time for await_statement_hints_cache Previously, `crdb_internal.await_statement_hints_cache` waited until the hint cache frontier timestamp reached `now()` plus the closed timestamp target duration and refresh interval. The frontier timestamp indicates that we've seen all events up to that timestamp, so it's OK to just wait until `now()` instead. This commit also fixes a flake in the `statement_hint_builtins` logic test due to a test flag `UseRangeTombstonesForPointDeletes`. Epic: None Release note: None --- pkg/sql/faketreeeval/BUILD.bazel | 1 - pkg/sql/faketreeeval/evalctx.go | 3 +-- pkg/sql/hints/BUILD.bazel | 2 -- pkg/sql/hints/hint_cache.go | 18 ++++++------------ .../logic_test/statement_hint_builtins | 1 + pkg/sql/planner.go | 5 ++--- pkg/sql/sem/builtins/builtins.go | 2 +- pkg/sql/sem/eval/deps.go | 3 +-- 8 files changed, 12 insertions(+), 23 deletions(-) diff --git a/pkg/sql/faketreeeval/BUILD.bazel b/pkg/sql/faketreeeval/BUILD.bazel index d8a4c36dade1..39fa9f0d7e28 100644 --- a/pkg/sql/faketreeeval/BUILD.bazel +++ b/pkg/sql/faketreeeval/BUILD.bazel @@ -10,7 +10,6 @@ go_library( "//pkg/jobs/jobspb", "//pkg/roachpb", "//pkg/security/username", - "//pkg/settings/cluster", "//pkg/sql/catalog/descpb", "//pkg/sql/hintpb", "//pkg/sql/pgwire/pgcode", diff --git a/pkg/sql/faketreeeval/evalctx.go b/pkg/sql/faketreeeval/evalctx.go index 5795d935ee71..976e87fe297c 100644 --- a/pkg/sql/faketreeeval/evalctx.go +++ b/pkg/sql/faketreeeval/evalctx.go @@ -14,7 +14,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/hintpb" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" @@ -583,7 +582,7 @@ func (ep *DummyEvalPlanner) ClearTableStatsCache() {} func (ep *DummyEvalPlanner) ClearStatementHintsCache() {} // AwaitStatementHintsCache is part of the eval.Planner interface. -func (ep *DummyEvalPlanner) AwaitStatementHintsCache(ctx context.Context, st *cluster.Settings) {} +func (ep *DummyEvalPlanner) AwaitStatementHintsCache(ctx context.Context) {} // RetryCounter is part of the eval.Planner interface. func (ep *DummyEvalPlanner) RetryCounter() int { diff --git a/pkg/sql/hints/BUILD.bazel b/pkg/sql/hints/BUILD.bazel index 98a6b1bf7a48..196bea76c061 100644 --- a/pkg/sql/hints/BUILD.bazel +++ b/pkg/sql/hints/BUILD.bazel @@ -14,8 +14,6 @@ go_library( "//pkg/kv/kvclient/rangefeed/rangefeedbuffer", "//pkg/kv/kvclient/rangefeed/rangefeedcache", "//pkg/kv/kvpb", - "//pkg/kv/kvserver", - "//pkg/kv/kvserver/closedts", "//pkg/roachpb", "//pkg/settings", "//pkg/settings/cluster", diff --git a/pkg/sql/hints/hint_cache.go b/pkg/sql/hints/hint_cache.go index 9a0590c345b8..f684fb2aee63 100644 --- a/pkg/sql/hints/hint_cache.go +++ b/pkg/sql/hints/hint_cache.go @@ -17,8 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedbuffer" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedcache" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -381,22 +379,18 @@ var _ rangefeedbuffer.Event = &bufferEvent{} // with the present. After Await returns, MaybeGetStatementHints should // accurately reflect all hints that were modified before the call to Await // (assuming the ctx was not canceled). -func (c *StatementHintsCache) Await(ctx context.Context, st *cluster.Settings) { - // The frontier timestamp comes from the rangefeed, and could be up to - // kv.closed_timestamp.target_duration + - // kv.rangefeed.closed_timestamp_refresh_interval behind the present. - targetDuration := closedts.TargetDuration.Get(&st.SV) - refreshInterval := kvserver.RangeFeedRefreshInterval.Get(&st.SV) - const fudge = 10 * time.Millisecond - waitUntil := c.clock.Now().AddDuration(targetDuration + refreshInterval + fudge).WallTime +func (c *StatementHintsCache) Await(ctx context.Context) { + // Wait in intervals of at least 100 milliseconds to avoid busy-waiting. + const minWait = time.Millisecond * 100 + waitUntil := c.clock.Now().WallTime // Await is only used for testing, so we don't need to wake up immediately. We // can get away with polling the frontier time. - for frontier := c.frontier.Load(); frontier < waitUntil; frontier = c.frontier.Load() { + for frontier := c.frontier.Load(); frontier <= waitUntil; frontier = c.frontier.Load() { select { case <-ctx.Done(): return - case <-time.After(time.Duration(waitUntil-frontier) * time.Nanosecond): + case <-time.After(max(time.Duration(waitUntil-frontier), minWait)): } } } diff --git a/pkg/sql/logictest/testdata/logic_test/statement_hint_builtins b/pkg/sql/logictest/testdata/logic_test/statement_hint_builtins index 4715675c7fbf..8dddfc92cee2 100644 --- a/pkg/sql/logictest/testdata/logic_test/statement_hint_builtins +++ b/pkg/sql/logictest/testdata/logic_test/statement_hint_builtins @@ -1,4 +1,5 @@ # LogicTest: !local-mixed-25.2 !local-mixed-25.3 !local-prepared +# cluster-opt: disable-mvcc-range-tombstones-for-point-deletes statement ok CREATE TABLE xy (x INT PRIMARY KEY, y INT, INDEX (y)); diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index 2bd5d90bcb58..ea6c7a9acea5 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -19,7 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/server/serverpb" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/sql/auditlogging" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -1099,9 +1098,9 @@ func (p *planner) ClearStatementHintsCache() { } // AwaitStatementHintsCache is part of the eval.Planner interface. -func (p *planner) AwaitStatementHintsCache(ctx context.Context, st *cluster.Settings) { +func (p *planner) AwaitStatementHintsCache(ctx context.Context) { if p.execCfg.StatementHintsCache != nil { - p.execCfg.StatementHintsCache.Await(ctx, st) + p.execCfg.StatementHintsCache.Await(ctx) } } diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index 6cfd5c22ece8..7774bca54a19 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -9741,7 +9741,7 @@ WHERE object_id = table_descriptor_id Types: tree.ParamTypes{}, ReturnType: tree.FixedReturnType(types.Void), Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) { - evalCtx.Planner.AwaitStatementHintsCache(ctx, evalCtx.Settings) + evalCtx.Planner.AwaitStatementHintsCache(ctx) return tree.DVoidDatum, nil }, Info: `This function is used to await the statement hints cache on the gateway node`, diff --git a/pkg/sql/sem/eval/deps.go b/pkg/sql/sem/eval/deps.go index 34f6de9bd4ba..7191ce3a596c 100644 --- a/pkg/sql/sem/eval/deps.go +++ b/pkg/sql/sem/eval/deps.go @@ -13,7 +13,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/hintpb" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" @@ -464,7 +463,7 @@ type Planner interface { // AwaitStatementHintsCache waits for the node's statement hints cache to // catch up with recent hint injections. - AwaitStatementHintsCache(ctx context.Context, st *cluster.Settings) + AwaitStatementHintsCache(ctx context.Context) // RetryCounter is the number of times this statement has been retried. RetryCounter() int From a4e6a42a81d5283a8b91edf695fad3da68ee3e38 Mon Sep 17 00:00:00 2001 From: Drew Kimball Date: Mon, 10 Nov 2025 20:50:36 -0600 Subject: [PATCH 2/3] explain: display number of hints for EXPLAIN This commit adds a top-level field to the output of `EXPLAIN` and `EXPLAIN ANALYZE` showing the number of hints applied to the statement, if nonzero. Informs #121502 Release note (sql change): EXPLAIN and EXPLAIN ANALYZE will now display the number of hints from `system.statement_hints` applied to the executed statement. --- pkg/sql/explain_plan.go | 4 ++ pkg/sql/instrumentation.go | 6 +++ pkg/sql/opt/exec/execbuilder/testdata/explain | 38 ++++++++++++++++ .../exec/execbuilder/testdata/explain_analyze | 44 ++++++++++++++++++- pkg/sql/opt/exec/explain/output.go | 9 ++++ 5 files changed, 99 insertions(+), 2 deletions(-) diff --git a/pkg/sql/explain_plan.go b/pkg/sql/explain_plan.go index 32a7694b7190..280d85a29e59 100644 --- a/pkg/sql/explain_plan.go +++ b/pkg/sql/explain_plan.go @@ -120,6 +120,10 @@ func (e *explainPlanNode) startExec(params runParams) error { } } + if len(params.p.stmt.Hints) > 0 { + ob.AddStmtHintCount(uint64(len(params.p.stmt.Hints))) + } + if e.options.Flags[tree.ExplainFlagJSON] { // For the JSON flag, we only want to emit the diagram JSON. rows = []string{diagramJSON} diff --git a/pkg/sql/instrumentation.go b/pkg/sql/instrumentation.go index 09947c7bd974..658146e5d69b 100644 --- a/pkg/sql/instrumentation.go +++ b/pkg/sql/instrumentation.go @@ -207,6 +207,10 @@ type instrumentationHelper struct { // stats scanned by this query. nanosSinceStatsForecasted time.Duration + // stmtHintsCount is the number of hints from system.statement_hints applied + // to the statement. + stmtHintsCount uint64 + // retryCount is the number of times the transaction was retried. retryCount uint64 @@ -429,6 +433,7 @@ func (ih *instrumentationHelper) Setup( ih.implicitTxn = implicitTxn ih.txnPriority = txnPriority ih.txnBufferedWritesEnabled = p.txn.BufferedWritesEnabled() + ih.stmtHintsCount = uint64(len(stmt.Hints)) ih.retryCount = uint64(retryCount) ih.codec = cfg.Codec ih.origCtx = ctx @@ -861,6 +866,7 @@ func (ih *instrumentationHelper) emitExplainAnalyzePlanToOutputBuilder( ob.AddDistribution(ih.distribution.String()) ob.AddVectorized(ih.vectorized) ob.AddPlanType(ih.generic, ih.optimized) + ob.AddStmtHintCount(ih.stmtHintsCount) ob.AddRetryCount("transaction", ih.retryCount) ob.AddRetryTime("transaction", phaseTimes.GetTransactionRetryLatency()) ob.AddRetryCount("statement", ih.retryStmtCount) diff --git a/pkg/sql/opt/exec/execbuilder/testdata/explain b/pkg/sql/opt/exec/execbuilder/testdata/explain index 0748d62e0351..aa488fc43f1e 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/explain +++ b/pkg/sql/opt/exec/execbuilder/testdata/explain @@ -2573,3 +2573,41 @@ vectorized: true table: t_float@t_float_pkey spans: /1/0 /2/0 /3/0 /4/0 /5/0 /6/0 /7/0 /8/0 /9/0 /10/0 /11/0 /12/0 /13/0 /14/0 /15/0 /16/0 /17/0 /18/0 /19/0 /20/0 … (20 more) parallel + +# Verify that the number of applied statement hints is displayed. +statement ok +CREATE TABLE t_hints (k INT PRIMARY KEY, v INT, INDEX (v)) + +statement ok +SELECT crdb_internal.inject_hint( + 'SELECT k FROM t_hints WHERE k >= _', + 'SELECT k FROM t_hints@t_hints_v_idx WHERE k >= _' +) + +statement ok +SELECT crdb_internal.await_statement_hints_cache() + +query T +EXPLAIN SELECT k FROM t_hints WHERE k >= 100 +---- +distribution: local +vectorized: true +statement hints count: 1 +· +• scan + missing stats + table: t_hints@t_hints_pkey + spans: [/100 - ] + +query T +EXPLAIN (VERBOSE) SELECT k FROM t_hints WHERE k >= 100 +---- +distribution: local +vectorized: true +statement hints count: 1 +· +• scan + columns: (k) + estimated row count: 333 (missing stats) + table: t_hints@t_hints_pkey + spans: /100- diff --git a/pkg/sql/opt/exec/execbuilder/testdata/explain_analyze b/pkg/sql/opt/exec/execbuilder/testdata/explain_analyze index e482707f0a2d..0bbfaa5d7060 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/explain_analyze +++ b/pkg/sql/opt/exec/execbuilder/testdata/explain_analyze @@ -1,7 +1,7 @@ # LogicTest: local statement ok -CREATE TABLE kv (k INT PRIMARY KEY, v INT, FAMILY (k, v)) +CREATE TABLE kv (k INT PRIMARY KEY, v INT, FAMILY (k, v), INDEX (v)) query T EXPLAIN ANALYZE (PLAN) SELECT k FROM kv WHERE k >= 2 @@ -176,7 +176,7 @@ quality of service: regular │ │ └── • virtual table │ │ sql nodes: │ │ regions: - │ │ actual row count: 361 + │ │ actual row count: 362 │ │ execution time: 0µs │ │ table: pg_class@primary │ │ @@ -276,3 +276,43 @@ quality of service: regular actual row count: 1 execution time: 0µs size: 2 columns, 1 row + +# Verify that the number of applied statement hints is displayed. +statement ok +SELECT crdb_internal.inject_hint( + 'SELECT k FROM kv WHERE k >= _', + 'SELECT k FROM kv@kv_v_idx WHERE k >= _' +) + +statement ok +SELECT crdb_internal.await_statement_hints_cache() + +query T +EXPLAIN ANALYZE SELECT k FROM kv WHERE k >= 100 +---- +planning time: 10µs +execution time: 100µs +distribution: +vectorized: +plan type: custom +statement hints count: 1 +maximum memory usage: +DistSQL network usage: +regions: +isolation level: serializable +priority: normal +quality of service: regular +· +• scan + sql nodes: + kv nodes: + regions: + actual row count: 0 + KV time: 0µs + KV rows decoded: 0 + KV bytes read: 0 B + KV gRPC calls: 0 + estimated max memory allocated: 0 B + missing stats + table: kv@kv_pkey + spans: [/100 - ] diff --git a/pkg/sql/opt/exec/explain/output.go b/pkg/sql/opt/exec/explain/output.go index e4c62c7f26ef..33e75bc44ff9 100644 --- a/pkg/sql/opt/exec/explain/output.go +++ b/pkg/sql/opt/exec/explain/output.go @@ -316,6 +316,15 @@ func (ob *OutputBuilder) AddPlanType(generic, optimized bool) { } } +// AddStmtHintCount adds a top-level field displaying the number of statement +// hints applied to the query. Cannot be called while inside a node. +func (ob *OutputBuilder) AddStmtHintCount(hintCount uint64) { + if hintCount == 0 { + return + } + ob.AddTopLevelField("statement hints count", string(humanizeutil.Count(hintCount))) +} + // AddPlanningTime adds a top-level planning time field. Cannot be called // while inside a node. func (ob *OutputBuilder) AddPlanningTime(delta time.Duration) { From 1d578a31a6bab1919f61affc7b3f924e8ce8a2fe Mon Sep 17 00:00:00 2001 From: Drew Kimball Date: Mon, 10 Nov 2025 20:51:47 -0600 Subject: [PATCH 3/3] ui: show whether statement hints were applied in DB console This commit adds a field to the "plan details" shown for statements in the "statement activity" page of the DB console. The new field indicates whether any statement hints from `system.statement_hints` were applied to the statement. Informs #121502 Release note (sql change): Plan details in the "statement activity" page of the DB console now show whether any hints from `system.statement_hints` werer applied to the statement execution. --- pkg/sql/appstatspb/app_stats.go | 1 + pkg/sql/appstatspb/app_stats.proto | 4 ++++ pkg/sql/executor_statement_metrics.go | 1 + .../sqlstatsutil/json_encoding_test.go | 1 + .../sqlstatsutil/json_impl.go | 1 + .../sqlstats/ssmemstorage/ss_mem_writer.go | 3 +++ .../ssmemstorage/ss_mem_writer_test.go | 3 +++ pkg/sql/sqlstats/ssprovider.go | 1 + .../planDetails/planDetails.tsx | 4 ++++ .../planDetails/plansTable.tsx | 20 +++++++++++++++++++ .../statementDetails.fixture.ts | 2 ++ .../cluster-ui/src/util/appStats/appStats.ts | 1 + .../src/views/statements/statements.spec.tsx | 1 + 13 files changed, 43 insertions(+) diff --git a/pkg/sql/appstatspb/app_stats.go b/pkg/sql/appstatspb/app_stats.go index 99d22d174dfc..61668b01d34e 100644 --- a/pkg/sql/appstatspb/app_stats.go +++ b/pkg/sql/appstatspb/app_stats.go @@ -214,6 +214,7 @@ func (s *StatementStatistics) Add(other *StatementStatistics) { s.Count += other.Count s.FailureCount += other.FailureCount s.GenericCount += other.GenericCount + s.StmtHintsCount += other.StmtHintsCount } // AlmostEqual compares two StatementStatistics and their contained NumericStats diff --git a/pkg/sql/appstatspb/app_stats.proto b/pkg/sql/appstatspb/app_stats.proto index 5ffd003c458d..d18300a08b7c 100644 --- a/pkg/sql/appstatspb/app_stats.proto +++ b/pkg/sql/appstatspb/app_stats.proto @@ -138,6 +138,10 @@ message StatementStatistics { // generic_count is the count of executions that used a generic query plan. optional int64 generic_count = 36 [(gogoproto.nullable) = false]; + // stmt_hints_count is the count of executions that applied a hint from the + // system.statement_hints table. + optional int64 stmt_hints_count = 37 [(gogoproto.nullable) = false]; + // Note: be sure to update `sql/app_stats.go` when adding/removing fields here! reserved 13, 14, 17, 18, 19, 20; diff --git a/pkg/sql/executor_statement_metrics.go b/pkg/sql/executor_statement_metrics.go index 92ee3eb327da..3342db2e7e64 100644 --- a/pkg/sql/executor_statement_metrics.go +++ b/pkg/sql/executor_statement_metrics.go @@ -225,6 +225,7 @@ func (ex *connExecutor) recordStatementSummary( FingerprintID: stmtFingerprintID, QuerySummary: stmt.StmtSummary, Generic: flags.IsSet(planFlagGeneric), + AppliedStmtHints: len(stmt.Hints) > 0, DistSQL: flags.ShouldBeDistributed(), Vec: flags.IsSet(planFlagVectorized), ImplicitTxn: implicitTxn, diff --git a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_encoding_test.go b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_encoding_test.go index fe584db6a94c..ea6851def635 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_encoding_test.go +++ b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_encoding_test.go @@ -54,6 +54,7 @@ func TestSQLStatsJsonEncoding(t *testing.T) { "firstAttemptCnt": {{.Int64}}, "failureCount": {{.Int64}}, "genericCount": {{.Int64}}, + "stmtHintsCount": {{.Int64}}, "maxRetries": {{.Int64}}, "lastExecAt": "{{stringifyTime .Time}}", "numRows": { diff --git a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_impl.go b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_impl.go index c3e227c47250..3d76eac536f8 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_impl.go +++ b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_impl.go @@ -345,6 +345,7 @@ func (s *innerStmtStats) jsonFields() jsonFields { {"lastErrorCode", (*jsonString)(&s.LastErrorCode)}, {"failureCount", (*jsonInt)(&s.FailureCount)}, {"genericCount", (*jsonInt)(&s.GenericCount)}, + {"stmtHintsCount", (*jsonInt)(&s.StmtHintsCount)}, {"sqlType", (*jsonString)(&s.SQLType)}, } } diff --git a/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go b/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go index 7c4cc2ed69dc..6867261232e4 100644 --- a/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go +++ b/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go @@ -80,6 +80,9 @@ func (s *Container) RecordStatement(ctx context.Context, value *sqlstats.Recorde if value.Generic { stats.mu.data.GenericCount++ } + if value.AppliedStmtHints { + stats.mu.data.StmtHintsCount++ + } if value.AutoRetryCount == 0 { stats.mu.data.FirstAttemptCount++ } else if int64(value.AutoRetryCount) > stats.mu.data.MaxRetries { diff --git a/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer_test.go b/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer_test.go index 8d3b8b459b58..020db798ec4b 100644 --- a/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer_test.go +++ b/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer_test.go @@ -119,6 +119,7 @@ func TestContainer_Add(t *testing.T) { RowsWritten: 5, Failed: true, Generic: true, + AppliedStmtHints: true, StatementError: errors.New("test error"), } require.NoError(t, src.RecordStatement(ctx, stmtStats)) @@ -161,6 +162,7 @@ func TestContainer_Add(t *testing.T) { RowsWritten: 80, Failed: true, Generic: true, + AppliedStmtHints: true, StatementError: errors.New("test error"), } reducedTxnFingerprintID := appstatspb.TransactionFingerprintID(321) @@ -208,6 +210,7 @@ func verifyStmtStatsMultiple( require.Equal(t, destStmtStats.mu.data.Count, int64(count)) require.Equal(t, destStmtStats.mu.data.FailureCount, int64(count)) require.Equal(t, destStmtStats.mu.data.GenericCount, int64(count)) + require.Equal(t, destStmtStats.mu.data.StmtHintsCount, int64(count)) require.InEpsilon(t, float64(stmtStats.RowsAffected), destStmtStats.mu.data.NumRows.Mean, epsilon) require.InEpsilon(t, float64(stmtStats.RowsAffected), destStmtStats.mu.data.NumRows.Mean, epsilon) require.InEpsilon(t, stmtStats.IdleLatencySec, destStmtStats.mu.data.IdleLat.Mean, epsilon) diff --git a/pkg/sql/sqlstats/ssprovider.go b/pkg/sql/sqlstats/ssprovider.go index 84d76115f062..06d8d89078fd 100644 --- a/pkg/sql/sqlstats/ssprovider.go +++ b/pkg/sql/sqlstats/ssprovider.go @@ -71,6 +71,7 @@ type RecordedStmtStats struct { AutoRetryCount int Failed bool Generic bool + AppliedStmtHints bool AutoRetryReason error RowsAffected int IdleLatencySec float64 diff --git a/pkg/ui/workspaces/cluster-ui/src/statementDetails/planDetails/planDetails.tsx b/pkg/ui/workspaces/cluster-ui/src/statementDetails/planDetails/planDetails.tsx index 8b99e847c5a3..d07aac18820b 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementDetails/planDetails/planDetails.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/statementDetails/planDetails/planDetails.tsx @@ -193,6 +193,10 @@ function ExplainPlan({ label="Generic Query Plan" value={RenderCount(plan.stats.generic_count, plan.stats.count)} /> + ); }, + stmtHints: () => { + return ( + + {planDetailsColumnLabels.stmtHints} + + ); + }, distSQL: () => { return ( RenderCount(item.stats.generic_count, item.stats.count), }, + { + name: "stmtHints", + title: planDetailsTableTitles.stmtHints(), + cell: (item: PlanHashStats) => + RenderCount(item.stats.stmt_hints_count, item.stats.count), + sort: (item: PlanHashStats) => + RenderCount(item.stats.stmt_hints_count, item.stats.count), + }, { name: "distSQL", title: planDetailsTableTitles.distSQL(), diff --git a/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.fixture.ts b/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.fixture.ts index fdf48f0f5cfe..f2d9933fa3b7 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.fixture.ts +++ b/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.fixture.ts @@ -31,6 +31,7 @@ const statementDetailsNoData: StatementDetailsResponse = { count: new Long(0), failure_count: new Long(0), generic_count: new Long(0), + stmt_hints_count: new Long(0), first_attempt_count: new Long(0), max_retries: new Long(0), legacy_last_err: "", @@ -88,6 +89,7 @@ const statementDetailsData: StatementDetailsResponse = { count: new Long(5), failure_count: new Long(2), generic_count: new Long(1), + stmt_hints_count: new Long(1), first_attempt_count: new Long(5), max_retries: new Long(0), legacy_last_err: "", 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 7dddee0bbbf1..dcc417f1f46b 100644 --- a/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.ts +++ b/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.ts @@ -203,6 +203,7 @@ export function addStatementStats( count: a.count.add(b.count), failure_count: a.failure_count.add(b.failure_count), generic_count: a.generic_count.add(b.generic_count), + stmt_hints_count: a.stmt_hints_count.add(b.stmt_hints_count), first_attempt_count: a.first_attempt_count.add(b.first_attempt_count), max_retries: a.max_retries.greaterThan(b.max_retries) ? a.max_retries diff --git a/pkg/ui/workspaces/db-console/src/views/statements/statements.spec.tsx b/pkg/ui/workspaces/db-console/src/views/statements/statements.spec.tsx index 57d212d1fd63..88da1f70e62c 100644 --- a/pkg/ui/workspaces/db-console/src/views/statements/statements.spec.tsx +++ b/pkg/ui/workspaces/db-console/src/views/statements/statements.spec.tsx @@ -296,6 +296,7 @@ function makeStats(): Required { last_error_code: "", failure_count: Long.fromNumber(0), generic_count: Long.fromNumber(0), + stmt_hints_count: Long.fromNumber(0), }; }