Skip to content

Commit 2c6258d

Browse files
committed
sql/hints, builtins: add two more statement_hints_cache builtins
Add new builtins `crdb_internal.clear_statement_hints_cache` and `crdb_internal.await_statement_hints_cache` which will be useful for testing. The first builtin clears the hintCache, similar to `clear_query_plan_cache` and `clear_table_stats_cache`. (I debated whether this builtin should also set hintedHashes to nil and restart the changefeed, but that seems too different from `clear_table_stats_cache`.) The second builtin blocks until the hints cache rangefeed watcher has caught up with the present, which usually takes a couple of seconds. This is handy when we want to test a statement using the hint immediately after adding the hint to system.statement_hints. Informs: #153633 Release note: None
1 parent 8d84db0 commit 2c6258d

File tree

9 files changed

+115
-0
lines changed

9 files changed

+115
-0
lines changed

pkg/sql/faketreeeval/BUILD.bazel

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

pkg/sql/faketreeeval/evalctx.go

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -14,6 +14,7 @@ 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"
1718
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
1819
"github.com/cockroachdb/cockroach/pkg/sql/hintpb"
1920
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
@@ -578,6 +579,12 @@ func (ep *DummyEvalPlanner) ClearQueryPlanCache() {}
578579
// ClearTableStatsCache is part of the eval.Planner interface.
579580
func (ep *DummyEvalPlanner) ClearTableStatsCache() {}
580581

582+
// ClearStatementHintsCache is part of the eval.Planner interface.
583+
func (ep *DummyEvalPlanner) ClearStatementHintsCache() {}
584+
585+
// AwaitStatementHintsCache is part of the eval.Planner interface.
586+
func (ep *DummyEvalPlanner) AwaitStatementHintsCache(ctx context.Context, st *cluster.Settings) {}
587+
581588
// RetryCounter is part of the eval.Planner interface.
582589
func (ep *DummyEvalPlanner) RetryCounter() int {
583590
return 0

pkg/sql/hints/BUILD.bazel

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -14,6 +14,8 @@ 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",
1719
"//pkg/roachpb",
1820
"//pkg/settings",
1921
"//pkg/settings/cluster",

pkg/sql/hints/hint_cache.go

Lines changed: 42 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,8 @@ 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"
2022
"github.com/cockroachdb/cockroach/pkg/roachpb"
2123
"github.com/cockroachdb/cockroach/pkg/settings"
2224
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
@@ -92,6 +94,10 @@ type StatementHintsCache struct {
9294
// generation is incremented any time the hint cache is updated by the
9395
// rangefeed.
9496
generation atomic.Int64
97+
98+
// frontier is the walltime of the rangefeed watcher frontier time, which is
99+
// polled by goroutines in Await.
100+
frontier atomic.Int64
95101
}
96102

97103
// cacheSize is the size of the entries to store in the cache.
@@ -139,6 +145,16 @@ func NewStatementHintsCache(
139145
return hintsCache
140146
}
141147

148+
// Clear removes all entries from the StatementHintsCache.
149+
func (c *StatementHintsCache) Clear() {
150+
defer c.generation.Add(1)
151+
c.mu.Lock()
152+
defer c.mu.Unlock()
153+
c.mu.hintCache.Clear()
154+
// TODO(michae2): Consider whether we should also delete hintedHashes and
155+
// restart the rangefeed.
156+
}
157+
142158
// ============================================================================
143159
// Populating the cache.
144160
// ============================================================================
@@ -214,6 +230,7 @@ func (c *StatementHintsCache) onUpdate(
214230
log.Dev.Info(ctx, "statement_hints rangefeed applying incremental update")
215231
c.handleIncrementalUpdate(ctx, update)
216232
}
233+
c.frontier.Store(update.Timestamp.WallTime)
217234
}
218235

219236
// handleInitialScan builds the hintedHashes map and adds it to
@@ -251,6 +268,7 @@ func (c *StatementHintsCache) handleIncrementalUpdate(
251268
defer c.generation.Add(1)
252269
c.mu.Lock()
253270
defer c.mu.Unlock()
271+
254272
for _, ev := range update.Events {
255273
// Drop outdated entries from hintCache.
256274
c.mu.hintCache.Del(ev.hash)
@@ -358,6 +376,30 @@ var _ rangefeedbuffer.Event = &bufferEvent{}
358376
// Reading from the cache.
359377
// ============================================================================
360378

379+
// Await blocks until the StatementHintsCache's rangefeed watcher catches up
380+
// with the present. After Await returns, MaybeGetStatementHints should
381+
// accurately reflect all hints that were modified before the call to Await
382+
// (assuming the ctx was not canceled).
383+
func (c *StatementHintsCache) Await(ctx context.Context, st *cluster.Settings) {
384+
// The frontier timestamp comes from the rangefeed, and could be up to
385+
// kv.closed_timestamp.target_duration +
386+
// kv.rangefeed.closed_timestamp_refresh_interval behind the present.
387+
targetDuration := closedts.TargetDuration.Get(&st.SV)
388+
refreshInterval := kvserver.RangeFeedRefreshInterval.Get(&st.SV)
389+
const fudge = 10 * time.Millisecond
390+
waitUntil := c.clock.Now().AddDuration(targetDuration + refreshInterval + fudge).WallTime
391+
392+
// Await is only used for testing, so we don't need to wake up immediately. We
393+
// can get away with polling the frontier time.
394+
for frontier := c.frontier.Load(); frontier < waitUntil; frontier = c.frontier.Load() {
395+
select {
396+
case <-ctx.Done():
397+
return
398+
case <-time.After(time.Duration(waitUntil-frontier) * time.Nanosecond):
399+
}
400+
}
401+
}
402+
361403
// GetGeneration returns the current generation, which will change if any
362404
// modifications happen to the cache.
363405
func (c *StatementHintsCache) GetGeneration() int64 {

pkg/sql/hints/hint_table.go

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -102,5 +102,8 @@ func InsertHintIntoDB(
102102
if err != nil {
103103
return 0, err
104104
}
105+
// TODO(michae2,drewk): Consider calling
106+
// StatementHintsCache.handleIncrementalUpdate here to eagerly update the
107+
// local node's cache.
105108
return int64(tree.MustBeDInt(row[0])), nil
106109
}

pkg/sql/planner.go

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@ import (
1919
"github.com/cockroachdb/cockroach/pkg/roachpb"
2020
"github.com/cockroachdb/cockroach/pkg/security/username"
2121
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
22+
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
2223
"github.com/cockroachdb/cockroach/pkg/spanconfig"
2324
"github.com/cockroachdb/cockroach/pkg/sql/auditlogging"
2425
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
@@ -1090,6 +1091,20 @@ func (p *planner) ClearTableStatsCache() {
10901091
}
10911092
}
10921093

1094+
// ClearStatementHintsCache is part of the eval.Planner interface.
1095+
func (p *planner) ClearStatementHintsCache() {
1096+
if p.execCfg.StatementHintsCache != nil {
1097+
p.execCfg.StatementHintsCache.Clear()
1098+
}
1099+
}
1100+
1101+
// AwaitStatementHintsCache is part of the eval.Planner interface.
1102+
func (p *planner) AwaitStatementHintsCache(ctx context.Context, st *cluster.Settings) {
1103+
if p.execCfg.StatementHintsCache != nil {
1104+
p.execCfg.StatementHintsCache.Await(ctx, st)
1105+
}
1106+
}
1107+
10931108
// innerPlansMustUseLeafTxn returns true if inner plans must use a leaf
10941109
// transaction.
10951110
func (p *planner) innerPlansMustUseLeafTxn() bool {

pkg/sql/sem/builtins/builtins.go

Lines changed: 34 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -9714,6 +9714,40 @@ WHERE object_id = table_descriptor_id
97149714
Volatility: volatility.Volatile,
97159715
},
97169716
),
9717+
"crdb_internal.clear_statement_hints_cache": makeBuiltin(
9718+
tree.FunctionProperties{
9719+
Category: builtinconstants.CategorySystemRepair,
9720+
Undocumented: true,
9721+
DistsqlBlocklist: true, // applicable only on the gateway
9722+
},
9723+
tree.Overload{
9724+
Types: tree.ParamTypes{},
9725+
ReturnType: tree.FixedReturnType(types.Void),
9726+
Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) {
9727+
evalCtx.Planner.ClearStatementHintsCache()
9728+
return tree.DVoidDatum, nil
9729+
},
9730+
Info: `This function is used to clear the statement hints cache on the gateway node`,
9731+
Volatility: volatility.Volatile,
9732+
},
9733+
),
9734+
"crdb_internal.await_statement_hints_cache": makeBuiltin(
9735+
tree.FunctionProperties{
9736+
Category: builtinconstants.CategorySystemRepair,
9737+
Undocumented: true,
9738+
DistsqlBlocklist: true, // applicable only on the gateway
9739+
},
9740+
tree.Overload{
9741+
Types: tree.ParamTypes{},
9742+
ReturnType: tree.FixedReturnType(types.Void),
9743+
Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) {
9744+
evalCtx.Planner.AwaitStatementHintsCache(ctx, evalCtx.Settings)
9745+
return tree.DVoidDatum, nil
9746+
},
9747+
Info: `This function is used to await the statement hints cache on the gateway node`,
9748+
Volatility: volatility.Volatile,
9749+
},
9750+
),
97179751
}
97189752

97199753
var lengthImpls = func(incBitOverload bool) builtinDefinition {

pkg/sql/sem/builtins/fixed_oids.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2861,6 +2861,8 @@ var builtinOidsArray = []string{
28612861
2906: `levenshtein_less_equal(source: string, target: string, ins_cost: int, del_cost: int, sub_cost: int, max_d: int) -> int`,
28622862
2907: `crdb_internal.request_transaction_bundle(transaction_fingerprint_id: string, sampling_probability: float, min_execution_latency: interval, expires_after: interval, redacted: bool) -> tuple{int AS request_id, bool AS created}`,
28632863
2908: `crdb_internal.inject_hint(statement_fingerprint: string, donor_sql: string) -> int`,
2864+
2909: `crdb_internal.clear_statement_hints_cache() -> void`,
2865+
2910: `crdb_internal.await_statement_hints_cache() -> void`,
28642866
}
28652867

28662868
var builtinOidsBySignature map[string]oid.Oid

pkg/sql/sem/eval/deps.go

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -13,6 +13,7 @@ import (
1313
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
1414
"github.com/cockroachdb/cockroach/pkg/roachpb"
1515
"github.com/cockroachdb/cockroach/pkg/security/username"
16+
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
1617
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
1718
"github.com/cockroachdb/cockroach/pkg/sql/hintpb"
1819
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice"
@@ -457,6 +458,14 @@ type Planner interface {
457458
// ClearTableStatsCache removes all entries from the node's table stats cache.
458459
ClearTableStatsCache()
459460

461+
// ClearStatementHintsCache removes all entries from the node's statement
462+
// hints cache.
463+
ClearStatementHintsCache()
464+
465+
// AwaitStatementHintsCache waits for the node's statement hints cache to
466+
// catch up with recent hint injections.
467+
AwaitStatementHintsCache(ctx context.Context, st *cluster.Settings)
468+
460469
// RetryCounter is the number of times this statement has been retried.
461470
RetryCounter() int
462471

0 commit comments

Comments
 (0)