Skip to content

Commit b8111a6

Browse files
craig[bot]DrewKimball
andcommitted
Merge #157160
157160: sql: improve observability for statements with hints r=DrewKimball a=DrewKimball #### 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. Epic: None Release note: None #### explain: display number of hints for EXPLAIN ANALYZE This commit adds a top-level field to the `EXPLAIN ANALYZE` output showing the number of hints applied to the statement if nonzero. Informs #121502 Release note (sql change): EXPLAIN ANALYZE will now display the number of hints from `system.statement_hints` applied to the executed statement. #### 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. Co-authored-by: Drew Kimball <drewk@cockroachlabs.com>
2 parents 8ff4da1 + 1d578a3 commit b8111a6

File tree

26 files changed

+154
-25
lines changed

26 files changed

+154
-25
lines changed

pkg/sql/appstatspb/app_stats.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -214,6 +214,7 @@ func (s *StatementStatistics) Add(other *StatementStatistics) {
214214
s.Count += other.Count
215215
s.FailureCount += other.FailureCount
216216
s.GenericCount += other.GenericCount
217+
s.StmtHintsCount += other.StmtHintsCount
217218
}
218219

219220
// AlmostEqual compares two StatementStatistics and their contained NumericStats

pkg/sql/appstatspb/app_stats.proto

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -138,6 +138,10 @@ message StatementStatistics {
138138
// generic_count is the count of executions that used a generic query plan.
139139
optional int64 generic_count = 36 [(gogoproto.nullable) = false];
140140

141+
// stmt_hints_count is the count of executions that applied a hint from the
142+
// system.statement_hints table.
143+
optional int64 stmt_hints_count = 37 [(gogoproto.nullable) = false];
144+
141145
// Note: be sure to update `sql/app_stats.go` when adding/removing fields here!
142146

143147
reserved 13, 14, 17, 18, 19, 20;

pkg/sql/executor_statement_metrics.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -225,6 +225,7 @@ func (ex *connExecutor) recordStatementSummary(
225225
FingerprintID: stmtFingerprintID,
226226
QuerySummary: stmt.StmtSummary,
227227
Generic: flags.IsSet(planFlagGeneric),
228+
AppliedStmtHints: len(stmt.Hints) > 0,
228229
DistSQL: flags.ShouldBeDistributed(),
229230
Vec: flags.IsSet(planFlagVectorized),
230231
ImplicitTxn: implicitTxn,

pkg/sql/explain_plan.go

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -120,6 +120,10 @@ func (e *explainPlanNode) startExec(params runParams) error {
120120
}
121121
}
122122

123+
if len(params.p.stmt.Hints) > 0 {
124+
ob.AddStmtHintCount(uint64(len(params.p.stmt.Hints)))
125+
}
126+
123127
if e.options.Flags[tree.ExplainFlagJSON] {
124128
// For the JSON flag, we only want to emit the diagram JSON.
125129
rows = []string{diagramJSON}

pkg/sql/faketreeeval/BUILD.bazel

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -10,7 +10,6 @@ go_library(
1010
"//pkg/jobs/jobspb",
1111
"//pkg/roachpb",
1212
"//pkg/security/username",
13-
"//pkg/settings/cluster",
1413
"//pkg/sql/catalog/descpb",
1514
"//pkg/sql/hintpb",
1615
"//pkg/sql/pgwire/pgcode",

pkg/sql/faketreeeval/evalctx.go

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -14,7 +14,6 @@ import (
1414
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
1515
"github.com/cockroachdb/cockroach/pkg/roachpb"
1616
"github.com/cockroachdb/cockroach/pkg/security/username"
17-
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
1817
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
1918
"github.com/cockroachdb/cockroach/pkg/sql/hintpb"
2019
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
@@ -583,7 +582,7 @@ func (ep *DummyEvalPlanner) ClearTableStatsCache() {}
583582
func (ep *DummyEvalPlanner) ClearStatementHintsCache() {}
584583

585584
// AwaitStatementHintsCache is part of the eval.Planner interface.
586-
func (ep *DummyEvalPlanner) AwaitStatementHintsCache(ctx context.Context, st *cluster.Settings) {}
585+
func (ep *DummyEvalPlanner) AwaitStatementHintsCache(ctx context.Context) {}
587586

588587
// RetryCounter is part of the eval.Planner interface.
589588
func (ep *DummyEvalPlanner) RetryCounter() int {

pkg/sql/hints/BUILD.bazel

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -14,8 +14,6 @@ go_library(
1414
"//pkg/kv/kvclient/rangefeed/rangefeedbuffer",
1515
"//pkg/kv/kvclient/rangefeed/rangefeedcache",
1616
"//pkg/kv/kvpb",
17-
"//pkg/kv/kvserver",
18-
"//pkg/kv/kvserver/closedts",
1917
"//pkg/roachpb",
2018
"//pkg/settings",
2119
"//pkg/settings/cluster",

pkg/sql/hints/hint_cache.go

Lines changed: 6 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -17,8 +17,6 @@ import (
1717
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedbuffer"
1818
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedcache"
1919
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
20-
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
21-
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts"
2220
"github.com/cockroachdb/cockroach/pkg/roachpb"
2321
"github.com/cockroachdb/cockroach/pkg/settings"
2422
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
@@ -381,22 +379,18 @@ var _ rangefeedbuffer.Event = &bufferEvent{}
381379
// with the present. After Await returns, MaybeGetStatementHints should
382380
// accurately reflect all hints that were modified before the call to Await
383381
// (assuming the ctx was not canceled).
384-
func (c *StatementHintsCache) Await(ctx context.Context, st *cluster.Settings) {
385-
// The frontier timestamp comes from the rangefeed, and could be up to
386-
// kv.closed_timestamp.target_duration +
387-
// kv.rangefeed.closed_timestamp_refresh_interval behind the present.
388-
targetDuration := closedts.TargetDuration.Get(&st.SV)
389-
refreshInterval := kvserver.RangeFeedRefreshInterval.Get(&st.SV)
390-
const fudge = 10 * time.Millisecond
391-
waitUntil := c.clock.Now().AddDuration(targetDuration + refreshInterval + fudge).WallTime
382+
func (c *StatementHintsCache) Await(ctx context.Context) {
383+
// Wait in intervals of at least 100 milliseconds to avoid busy-waiting.
384+
const minWait = time.Millisecond * 100
385+
waitUntil := c.clock.Now().WallTime
392386

393387
// Await is only used for testing, so we don't need to wake up immediately. We
394388
// can get away with polling the frontier time.
395-
for frontier := c.frontier.Load(); frontier < waitUntil; frontier = c.frontier.Load() {
389+
for frontier := c.frontier.Load(); frontier <= waitUntil; frontier = c.frontier.Load() {
396390
select {
397391
case <-ctx.Done():
398392
return
399-
case <-time.After(time.Duration(waitUntil-frontier) * time.Nanosecond):
393+
case <-time.After(max(time.Duration(waitUntil-frontier), minWait)):
400394
}
401395
}
402396
}

pkg/sql/instrumentation.go

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -207,6 +207,10 @@ type instrumentationHelper struct {
207207
// stats scanned by this query.
208208
nanosSinceStatsForecasted time.Duration
209209

210+
// stmtHintsCount is the number of hints from system.statement_hints applied
211+
// to the statement.
212+
stmtHintsCount uint64
213+
210214
// retryCount is the number of times the transaction was retried.
211215
retryCount uint64
212216

@@ -429,6 +433,7 @@ func (ih *instrumentationHelper) Setup(
429433
ih.implicitTxn = implicitTxn
430434
ih.txnPriority = txnPriority
431435
ih.txnBufferedWritesEnabled = p.txn.BufferedWritesEnabled()
436+
ih.stmtHintsCount = uint64(len(stmt.Hints))
432437
ih.retryCount = uint64(retryCount)
433438
ih.codec = cfg.Codec
434439
ih.origCtx = ctx
@@ -861,6 +866,7 @@ func (ih *instrumentationHelper) emitExplainAnalyzePlanToOutputBuilder(
861866
ob.AddDistribution(ih.distribution.String())
862867
ob.AddVectorized(ih.vectorized)
863868
ob.AddPlanType(ih.generic, ih.optimized)
869+
ob.AddStmtHintCount(ih.stmtHintsCount)
864870
ob.AddRetryCount("transaction", ih.retryCount)
865871
ob.AddRetryTime("transaction", phaseTimes.GetTransactionRetryLatency())
866872
ob.AddRetryCount("statement", ih.retryStmtCount)

pkg/sql/logictest/testdata/logic_test/statement_hint_builtins

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,5 @@
11
# LogicTest: !local-mixed-25.2 !local-mixed-25.3 !local-prepared
2+
# cluster-opt: disable-mvcc-range-tombstones-for-point-deletes
23

34
statement ok
45
CREATE TABLE xy (x INT PRIMARY KEY, y INT, INDEX (y));

0 commit comments

Comments
 (0)