Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions pkg/sql/appstatspb/app_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/appstatspb/app_stats.proto
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/executor_statement_metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/explain_plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -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}
Expand Down
1 change: 0 additions & 1 deletion pkg/sql/faketreeeval/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
3 changes: 1 addition & 2 deletions pkg/sql/faketreeeval/evalctx.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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 {
Expand Down
2 changes: 0 additions & 2 deletions pkg/sql/hints/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
18 changes: 6 additions & 12 deletions pkg/sql/hints/hint_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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)):
}
}
}
Expand Down
6 changes: 6 additions & 0 deletions pkg/sql/instrumentation.go
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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)
Expand Down
Original file line number Diff line number Diff line change
@@ -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));
Expand Down
38 changes: 38 additions & 0 deletions pkg/sql/opt/exec/execbuilder/testdata/explain
Original file line number Diff line number Diff line change
Expand Up @@ -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-
44 changes: 42 additions & 2 deletions pkg/sql/opt/exec/execbuilder/testdata/explain_analyze
Original file line number Diff line number Diff line change
@@ -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
Expand Down Expand Up @@ -176,7 +176,7 @@ quality of service: regular
│ │ └── • virtual table
│ │ sql nodes: <hidden>
│ │ regions: <hidden>
│ │ actual row count: 361
│ │ actual row count: 362
│ │ execution time: 0µs
│ │ table: pg_class@primary
│ │
Expand Down Expand Up @@ -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: <hidden>
vectorized: <hidden>
plan type: custom
statement hints count: 1
maximum memory usage: <hidden>
DistSQL network usage: <hidden>
regions: <hidden>
isolation level: serializable
priority: normal
quality of service: regular
·
• scan
sql nodes: <hidden>
kv nodes: <hidden>
regions: <hidden>
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 - ]
9 changes: 9 additions & 0 deletions pkg/sql/opt/exec/explain/output.go
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand Down
5 changes: 2 additions & 3 deletions pkg/sql/planner.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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)
}
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/sem/builtins/builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -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`,
Expand Down
3 changes: 1 addition & 2 deletions pkg/sql/sem/eval/deps.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,7 @@ func TestSQLStatsJsonEncoding(t *testing.T) {
"firstAttemptCnt": {{.Int64}},
"failureCount": {{.Int64}},
"genericCount": {{.Int64}},
"stmtHintsCount": {{.Int64}},
"maxRetries": {{.Int64}},
"lastExecAt": "{{stringifyTime .Time}}",
"numRows": {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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)},
}
}
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/sqlstats/ssmemstorage/ss_mem_writer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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))
Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -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)
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/sqlstats/ssprovider.go
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,7 @@ type RecordedStmtStats struct {
AutoRetryCount int
Failed bool
Generic bool
AppliedStmtHints bool
AutoRetryReason error
RowsAffected int
IdleLatencySec float64
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -193,6 +193,10 @@ function ExplainPlan({
label="Generic Query Plan"
value={RenderCount(plan.stats.generic_count, plan.stats.count)}
/>
<SummaryCardItem
label="Statement Hints"
value={RenderCount(plan.stats.stmt_hints_count, plan.stats.count)}
/>
<SummaryCardItem
label="Distributed"
value={RenderCount(
Expand Down
Loading
Loading