From 1b7d90478fba25a6af5015125b9ab0964233f793 Mon Sep 17 00:00:00 2001 From: Michael Erickson Date: Mon, 17 Nov 2025 23:47:52 -0800 Subject: [PATCH 1/3] sql: add parseHint step when loading hint into hints cache When loading an external statement hint into the statement hints cache, we might need to call some function to get the hint ready for use. (For hint injections, this function is `tree.NewHintInjectionDonor` which parses and walks the donor statement fingerprint.) This function could fail, in which case we want to skip over the hint but not return an error from `GetStatementHintsFromDB`. This function could succeed but create some extra state which we need to save. This commit adds a new `parseHint` step which calls any functions needed to get the hint ready, and creates a new `hints.Hint` struct which holds the object(s) created when parsing hints. (These are analogous to `parseStats` and `TableStatistic` from the stats cache.) Informs: #153633 Release note: None --- pkg/sql/hints/BUILD.bazel | 2 + pkg/sql/hints/hint_cache.go | 22 +++++------ pkg/sql/hints/hint_table.go | 75 +++++++++++++++++++++++++++++++++---- pkg/sql/prep/BUILD.bazel | 2 +- pkg/sql/prep/metadata.go | 4 +- pkg/sql/statement.go | 7 ++-- 6 files changed, 87 insertions(+), 25 deletions(-) diff --git a/pkg/sql/hints/BUILD.bazel b/pkg/sql/hints/BUILD.bazel index 196bea76c061..f5f7c561bacc 100644 --- a/pkg/sql/hints/BUILD.bazel +++ b/pkg/sql/hints/BUILD.bazel @@ -24,12 +24,14 @@ go_library( "//pkg/sql/catalog/systemschema", "//pkg/sql/hintpb", "//pkg/sql/isql", + "//pkg/sql/parser", "//pkg/sql/rowenc", "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", "//pkg/sql/types", "//pkg/util/buildutil", "//pkg/util/cache", + "//pkg/util/errorutil", "//pkg/util/hlc", "//pkg/util/log", "//pkg/util/metamorphic", diff --git a/pkg/sql/hints/hint_cache.go b/pkg/sql/hints/hint_cache.go index f684fb2aee63..28785bd6dcdb 100644 --- a/pkg/sql/hints/hint_cache.go +++ b/pkg/sql/hints/hint_cache.go @@ -25,7 +25,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" - "github.com/cockroachdb/cockroach/pkg/sql/hintpb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -406,8 +405,8 @@ func (c *StatementHintsCache) GetGeneration() int64 { // plans). It returns nil if the statement has no hints, or there was an error // retrieving them. func (c *StatementHintsCache) MaybeGetStatementHints( - ctx context.Context, statementFingerprint string, -) (hints []hintpb.StatementHintUnion, ids []int64) { + ctx context.Context, statementFingerprint string, fingerprintFlags tree.FmtFlags, +) (hints []Hint, ids []int64) { hash := fnv.New64() _, err := hash.Write([]byte(statementFingerprint)) if err != nil { @@ -430,7 +429,7 @@ func (c *StatementHintsCache) MaybeGetStatementHints( if !ok { // The plan hints were evicted from the cache. Retrieve them from the // database and add them to the cache. - return c.addCacheEntryLocked(ctx, statementHash, statementFingerprint) + return c.addCacheEntryLocked(ctx, statementHash, statementFingerprint, fingerprintFlags) } entry := e.(*cacheEntry) c.maybeWaitForRefreshLocked(ctx, entry, statementHash) @@ -464,8 +463,11 @@ func (c *StatementHintsCache) maybeWaitForRefreshLocked( // other queries to wait for the result via sync.Cond. Note that the lock is // released while reading from the db, and then reacquired. func (c *StatementHintsCache) addCacheEntryLocked( - ctx context.Context, statementHash int64, statementFingerprint string, -) (hints []hintpb.StatementHintUnion, ids []int64) { + ctx context.Context, + statementHash int64, + statementFingerprint string, + fingerprintFlags tree.FmtFlags, +) (hints []Hint, ids []int64) { c.mu.AssertHeld() // Add a cache entry that other queries can find and wait on until we have the @@ -483,7 +485,7 @@ func (c *StatementHintsCache) addCacheEntryLocked( defer c.mu.Lock() log.VEventf(ctx, 1, "reading hints for query %s", statementFingerprint) entry.ids, entry.fingerprints, entry.hints, err = - GetStatementHintsFromDB(ctx, c.db.Executor(), statementHash) + GetStatementHintsFromDB(ctx, c.db.Executor(), statementHash, fingerprintFlags) log.VEventf(ctx, 1, "finished reading hints for query %s", statementFingerprint) }() @@ -517,16 +519,14 @@ type cacheEntry struct { // be duplicate entries in the fingerprints slice. // TODO(drewk): consider de-duplicating the fingerprint strings to reduce // memory usage. - hints []hintpb.StatementHintUnion + hints []Hint fingerprints []string ids []int64 } // getMatchingHints returns the plan hints and row IDs for the given // fingerprint, or nil if they don't exist. The results are in order of row ID. -func (entry *cacheEntry) getMatchingHints( - statementFingerprint string, -) (hints []hintpb.StatementHintUnion, ids []int64) { +func (entry *cacheEntry) getMatchingHints(statementFingerprint string) (hints []Hint, ids []int64) { for i := range entry.hints { if entry.fingerprints[i] == statementFingerprint { hints = append(hints, entry.hints[i]) diff --git a/pkg/sql/hints/hint_table.go b/pkg/sql/hints/hint_table.go index 297521a4ec7f..bf4018972df3 100644 --- a/pkg/sql/hints/hint_table.go +++ b/pkg/sql/hints/hint_table.go @@ -10,11 +10,23 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/hintpb" "github.com/cockroachdb/cockroach/pkg/sql/isql" + "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/util/errorutil" + "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) +// Hint represents an unmarshaled hint that is ready to apply to statements. +type Hint struct { + hintpb.StatementHintUnion + + // HintInjectionDonor is the fully parsed donor statement fingerprint used for + // hint injection. + HintInjectionDonor *tree.HintInjectionDonor +} + // CheckForStatementHintsInDB queries the system.statement_hints table to // determine if there are any hints for the given fingerprint hash. The caller // must be able to retry if an error is returned. @@ -45,9 +57,13 @@ func CheckForStatementHintsInDB( // fingerprints[i] is the statement fingerprint to which hints[i] applies, while // hintIDs[i] uniquely identifies a hint in the system table. The results are in // order of hint ID. +// +// GetStatementHintsFromDB will return an error if the query returns an +// error. If one of the hints cannot be unmarshalled, the hint (and associated +// fingerprint and ID) will be skipped but no error will be returned. func GetStatementHintsFromDB( - ctx context.Context, ex isql.Executor, statementHash int64, -) (hintIDs []int64, fingerprints []string, hints []hintpb.StatementHintUnion, retErr error) { + ctx context.Context, ex isql.Executor, statementHash int64, fingerprintFlags tree.FmtFlags, +) (hintIDs []int64, fingerprints []string, hints []Hint, retErr error) { const opName = "get-plan-hints" const getHintsStmt = ` SELECT "row_id", "fingerprint", "hint" @@ -72,18 +88,58 @@ func GetStatementHintsFromDB( if !ok { break } - datums := it.Cur() - hintIDs = append(hintIDs, int64(tree.MustBeDInt(datums[0]))) - fingerprints = append(fingerprints, string(tree.MustBeDString(datums[1]))) - hint, err := hintpb.FromBytes([]byte(tree.MustBeDBytes(datums[2]))) + hintID, fingerprint, hint, err := parseHint(it.Cur(), fingerprintFlags) if err != nil { - return nil, nil, nil, err + log.Dev.Warningf( + ctx, "could not decode hint ID %v for statement hash %v fingerprint %v: %v", + hintID, statementHash, fingerprint, err, + ) + continue } + hintIDs = append(hintIDs, hintID) + fingerprints = append(fingerprints, fingerprint) hints = append(hints, hint) } return hintIDs, fingerprints, hints, nil } +func parseHint( + datums tree.Datums, fingerprintFlags tree.FmtFlags, +) (hintID int64, fingerprint string, hint Hint, err error) { + defer func() { + if r := recover(); r != nil { + // In the event of a "safe" panic, we only want to log the error and + // continue executing the query without this hint. This is only possible + // because the code does not update shared state and does not manipulate + // locks. + if ok, e := errorutil.ShouldCatch(r); ok { + err = e + } else { + // Other panic objects can't be considered "safe" and thus are + // propagated as crashes that terminate the session. + panic(r) + } + } + }() + hintID = int64(tree.MustBeDInt(datums[0])) + fingerprint = string(tree.MustBeDString(datums[1])) + hint.StatementHintUnion, err = hintpb.FromBytes([]byte(tree.MustBeDBytes(datums[2]))) + if err != nil { + return hintID, fingerprint, Hint{}, err + } + if hint.InjectHints != nil && hint.InjectHints.DonorSQL != "" { + donorStmt, err := parser.ParseOne(hint.InjectHints.DonorSQL) + if err != nil { + return hintID, fingerprint, Hint{}, err + } + hint.HintInjectionDonor, err = tree.NewHintInjectionDonor(donorStmt.AST, fingerprintFlags) + if err != nil { + return hintID, fingerprint, Hint{}, err + } + } + return hintID, fingerprint, hint, nil +} + // InsertHintIntoDB inserts a statement hint into the system.statement_hints // table. It returns the hint ID of the newly inserted hint if successful. func InsertHintIntoDB( @@ -107,3 +163,8 @@ func InsertHintIntoDB( // local node's cache. return int64(tree.MustBeDInt(row[0])), nil } + +func (hint *Hint) Size() int { + // TODO(michae2): add size of HintInjectionDonor + return hint.StatementHintUnion.Size() +} diff --git a/pkg/sql/prep/BUILD.bazel b/pkg/sql/prep/BUILD.bazel index e5b2fb1df781..3df3ed2321f1 100644 --- a/pkg/sql/prep/BUILD.bazel +++ b/pkg/sql/prep/BUILD.bazel @@ -11,7 +11,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/sql/catalog/colinfo", - "//pkg/sql/hintpb", + "//pkg/sql/hints", "//pkg/sql/opt/memo", "//pkg/sql/parser/statements", "//pkg/sql/sem/tree", diff --git a/pkg/sql/prep/metadata.go b/pkg/sql/prep/metadata.go index f9233d2d0337..f554f0b93fee 100644 --- a/pkg/sql/prep/metadata.go +++ b/pkg/sql/prep/metadata.go @@ -9,7 +9,7 @@ import ( "unsafe" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" - "github.com/cockroachdb/cockroach/pkg/sql/hintpb" + "github.com/cockroachdb/cockroach/pkg/sql/hints" "github.com/cockroachdb/cockroach/pkg/sql/parser/statements" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -44,7 +44,7 @@ type Metadata struct { // Hints are any external statement hints from the system.statement_hints // table that could apply to this statement, based on the statement // fingerprint. - Hints []hintpb.StatementHintUnion + Hints []hints.Hint // HintIDs are the IDs of any external statement hints, which are used for // invalidation of cached plans. diff --git a/pkg/sql/statement.go b/pkg/sql/statement.go index 92ea60fa9f6b..0d0621202cdc 100644 --- a/pkg/sql/statement.go +++ b/pkg/sql/statement.go @@ -11,7 +11,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/clusterunique" - "github.com/cockroachdb/cockroach/pkg/sql/hintpb" "github.com/cockroachdb/cockroach/pkg/sql/hints" "github.com/cockroachdb/cockroach/pkg/sql/parser/statements" "github.com/cockroachdb/cockroach/pkg/sql/prep" @@ -46,7 +45,7 @@ type Statement struct { // Hints are any external statement hints from the system.statement_hints // table that could apply to this statement, based on the statement // fingerprint. - Hints []hintpb.StatementHintUnion + Hints []hints.Hint // HintIDs are the IDs of any external statement hints, which are used for // invalidation of cached plans. @@ -133,7 +132,7 @@ func (s Statement) String() string { func (s *Statement) ReloadHintsIfStale( ctx context.Context, fmtFlags tree.FmtFlags, statementHintsCache *hints.StatementHintsCache, ) bool { - var hints []hintpb.StatementHintUnion + var hints []hints.Hint var hintIDs []int64 var hintsGeneration int64 if statementHintsCache != nil { @@ -154,7 +153,7 @@ func (s *Statement) ReloadHintsIfStale( case *tree.Prepare: hintStmtFingerprint = tree.FormatStatementHideConstants(e.Statement, fmtFlags) } - hints, hintIDs = statementHintsCache.MaybeGetStatementHints(ctx, hintStmtFingerprint) + hints, hintIDs = statementHintsCache.MaybeGetStatementHints(ctx, hintStmtFingerprint, fmtFlags) } if slices.Equal(hintIDs, s.HintIDs) { return false From 6daf24d4fd0d1fcb37e57bafaf0e92863a76041f Mon Sep 17 00:00:00 2001 From: Michael Erickson Date: Tue, 18 Nov 2025 22:37:51 -0800 Subject: [PATCH 2/3] sql/*: add hint injection 1. During `ReloadHintsIfStale` we now call `Validate` and `InjectHints` using the donor to perform the AST rewrite. We save the rewritten AST in the statement separately from the original AST. 2. We wrap `prepareUsingOptimizer` and `makeOptimizerPlan` with functions that first try preparing / planning with injected hints, and then try again without injected hints in case the injected hints are invalid. With these two pieces we can now actually perform hint injection. Fixes: #153633 Release note (sql change): A new "hint injection" ability has been added, which allows operators to dynamically inject inline hints into statements, without modifying the text of those statements. Hints can be injected using the builtin function `crdb_internal.inject_hint` with the target statement fingerprint to rewrite. For example, to add an index hint to the statement `SELECT * FROM my_table WHERE col = 3`, use: ``` SELECT crdb_internal.inject_hint( 'SELECT * FROM my_table WHERE col = _', 'SELECT * FROM my_table@my_table_col_idx WHERE col = _' ); ``` Whenever a statement is executed matching statement fingerprint `SELECT * FROM my_table WHERE col = _`, it will first be rewritten to include the injected index hint. --- pkg/sql/conn_executor_exec.go | 2 + pkg/sql/conn_executor_prepare.go | 1 + .../logic_test/statement_hint_builtins | 264 ++++++++++++++++++ pkg/sql/plan_opt.go | 64 ++++- pkg/sql/planner.go | 6 + pkg/sql/prep/metadata.go | 6 + pkg/sql/statement.go | 93 +++++- 7 files changed, 416 insertions(+), 20 deletions(-) diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index ef39c35d850e..41c4e270d6ee 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -565,6 +565,7 @@ func (ex *connExecutor) execStmtInOpenState( stmt.Hints = ps.Hints stmt.HintIDs = ps.HintIDs stmt.HintsGeneration = ps.HintsGeneration + stmt.ASTWithInjectedHints = ps.ASTWithInjectedHints stmt.ReloadHintsIfStale(ctx, stmtFingerprintFmtMask, statementHintsCache) res.ResetStmtType(ps.AST) @@ -1465,6 +1466,7 @@ func (ex *connExecutor) execStmtInOpenStateWithPausablePortal( vars.stmt.Hints = ps.Hints vars.stmt.HintIDs = ps.HintIDs vars.stmt.HintsGeneration = ps.HintsGeneration + vars.stmt.ASTWithInjectedHints = ps.ASTWithInjectedHints vars.stmt.ReloadHintsIfStale(ctx, stmtFingerprintFmtMask, statementHintsCache) res.ResetStmtType(ps.AST) diff --git a/pkg/sql/conn_executor_prepare.go b/pkg/sql/conn_executor_prepare.go index 97d7ac207841..7f43210ba174 100644 --- a/pkg/sql/conn_executor_prepare.go +++ b/pkg/sql/conn_executor_prepare.go @@ -247,6 +247,7 @@ func (ex *connExecutor) prepare( prepared.Hints = stmt.Hints prepared.HintIDs = stmt.HintIDs prepared.HintsGeneration = stmt.HintsGeneration + prepared.ASTWithInjectedHints = stmt.ASTWithInjectedHints // Point to the prepared state, which can be further populated during query // preparation. diff --git a/pkg/sql/logictest/testdata/logic_test/statement_hint_builtins b/pkg/sql/logictest/testdata/logic_test/statement_hint_builtins index 4d63b4234b39..de33ccce6cef 100644 --- a/pkg/sql/logictest/testdata/logic_test/statement_hint_builtins +++ b/pkg/sql/logictest/testdata/logic_test/statement_hint_builtins @@ -179,3 +179,267 @@ FROM crdb_internal.feature_usage WHERE feature_name = 'sql.session.statement-hints' ---- 6 + +statement ok +DEALLOCATE ALL + +# Testcases for hint injection. + +statement ok +CREATE TABLE abc (a INT PRIMARY KEY, b INT, c INT, INDEX (b)) + +# Temporary, until next commit. +statement ok +SET CLUSTER SETTING sql.query_cache.enabled = off + +# Try some simple hint injections. First, an index hint. + +statement ok +SELECT crdb_internal.inject_hint( + 'SELECT a FROM abc WHERE a = _', + 'SELECT a FROM abc@abc_b_idx WHERE a = _' +) + +statement ok +SELECT crdb_internal.await_statement_hints_cache() + +statement ok +SET tracing = on + +statement ok +SELECT a FROM abc WHERE a = 5 + +statement ok +SET tracing = off + +query T +SELECT regexp_replace(message, E'\\d+', 'x') FROM [SHOW TRACE FOR SESSION] WHERE message LIKE '%injected hints%' +---- +injected hints from external statement hint x +trying planning with injected hints + +query T +EXPLAIN SELECT a FROM abc WHERE a = 10 +---- +distribution: local +vectorized: true +statement hints count: 1 +· +• filter +│ filter: a = 10 +│ +└── • scan + missing stats + table: abc@abc_b_idx + spans: FULL SCAN + +# Try injecting a join hint. + +statement ok +SELECT crdb_internal.inject_hint( + 'SELECT a, x FROM abc JOIN xy ON y = b WHERE a = _', + 'SELECT a, x FROM abc INNER HASH JOIN xy ON y = b WHERE a = _' +) + +statement ok +SELECT crdb_internal.await_statement_hints_cache() + +statement ok +SET tracing = on + +statement ok +SELECT a, x FROM abc JOIN xy ON y = b WHERE a = 5 + +statement ok +SET tracing = off + +query T +SELECT regexp_replace(message, E'\\d+', 'x') FROM [SHOW TRACE FOR SESSION] WHERE message LIKE '%injected hints%' +---- +injected hints from external statement hint x +trying planning with injected hints + +query T +EXPLAIN SELECT a, x FROM abc JOIN xy ON y = b WHERE a = 10 +---- +distribution: local +vectorized: true +statement hints count: 1 +· +• hash join +│ equality: (b) = (y) +│ left cols are key +│ +├── • scan +│ missing stats +│ table: abc@abc_pkey +│ spans: [/10 - /10] +│ +└── • scan + missing stats + table: xy@xy_pkey + spans: FULL SCAN + +# Try removing a hint. + +statement ok +SELECT crdb_internal.inject_hint( + 'SELECT a FROM abc@abc_pkey WHERE b = _', + 'SELECT a FROM abc WHERE b = _' +) + +statement ok +SELECT crdb_internal.await_statement_hints_cache() + +statement ok +SET tracing = on + +statement ok +SELECT a FROM abc@abc_pkey WHERE b = 5 + +statement ok +SET tracing = off + +query T +SELECT regexp_replace(message, E'\\d+', 'x') FROM [SHOW TRACE FOR SESSION] WHERE message LIKE '%injected hints%' +---- +injected hints from external statement hint x +trying planning with injected hints + +query T +EXPLAIN SELECT a FROM abc@abc_pkey WHERE b = 10 +---- +distribution: local +vectorized: true +statement hints count: 1 +· +• scan + missing stats + table: abc@abc_b_idx + spans: [/10 - /10] + +# Check that we do not use an invalid injected index hint. + +statement ok +SELECT crdb_internal.inject_hint( + 'SELECT a + _ FROM abc WHERE a = _', + 'SELECT a + _ FROM abc@foo WHERE a = _' +) + +statement ok +SELECT crdb_internal.await_statement_hints_cache() + +statement ok +SET tracing = on + +statement ok +SELECT a + 1 FROM abc WHERE a = 5 + +statement ok +SET tracing = off + +query T +SELECT regexp_replace(message, E'\\d+', 'x') FROM [SHOW TRACE FOR SESSION] WHERE message LIKE '%injected hints%' +---- +injected hints from external statement hint x +trying planning with injected hints +planning with injected hints failed with: index "foo" not found +falling back to planning without injected hints + +query T +EXPLAIN SELECT a + 1 FROM abc WHERE a = 10 +---- +distribution: local +vectorized: true +statement hints count: 1 +· +• render +│ +└── • scan + missing stats + table: abc@abc_pkey + spans: [/10 - /10] + +# Check that we do not use an unsatisfiable injected hint. + +statement ok +SELECT crdb_internal.inject_hint( + 'SELECT c FROM xy JOIN abc ON c = y WHERE x = _', + 'SELECT c FROM xy INNER LOOKUP JOIN abc ON c = y WHERE x = _' +) + +statement ok +SELECT crdb_internal.await_statement_hints_cache() + +statement ok +SET tracing = on + +statement ok +SELECT c FROM xy JOIN abc ON c = y WHERE x = 5 + +statement ok +SET tracing = off + +query T +SELECT regexp_replace(message, E'\\d+', 'x') FROM [SHOW TRACE FOR SESSION] WHERE message LIKE '%injected hints%' +---- +injected hints from external statement hint x +trying planning with injected hints +planning with injected hints failed with: could not produce a query plan conforming to the LOOKUP JOIN hint +falling back to planning without injected hints + +query T +EXPLAIN SELECT c FROM xy JOIN abc ON c = y WHERE x = 10 +---- +distribution: local +vectorized: true +statement hints count: 1 +· +• hash join +│ equality: (c) = (y) +│ right cols are key +│ +├── • scan +│ missing stats +│ table: abc@abc_pkey +│ spans: FULL SCAN +│ +└── • scan + missing stats + table: xy@xy_pkey + spans: [/10 - /10] + +# Try a prepared statement with an injected hint. + +statement ok +SELECT crdb_internal.inject_hint( + 'SELECT c FROM abc WHERE b > _', + 'SELECT c FROM abc@{NO_INDEX_JOIN} WHERE b > _' +) + +statement ok +SELECT crdb_internal.await_statement_hints_cache() + +statement ok +SET tracing = on + +statement ok +PREPARE p AS SELECT c FROM abc WHERE b > $1 + +statement ok +EXECUTE p (5) + +statement ok +SET tracing = off + +query T +SELECT regexp_replace(message, E'\\d+', 'x') FROM [SHOW TRACE FOR SESSION] WHERE message LIKE '%injected hints%' +---- +injected hints from external statement hint x +injected hints from external statement hint x +trying preparing with injected hints +trying planning with injected hints + +# Temporary, until next commit. +statement ok +RESET CLUSTER SETTING sql.query_cache.enabled diff --git a/pkg/sql/plan_opt.go b/pkg/sql/plan_opt.go index af69b8dd06d2..4d7c7cbc3b37 100644 --- a/pkg/sql/plan_opt.go +++ b/pkg/sql/plan_opt.go @@ -56,10 +56,33 @@ var queryCacheEnabled = settings.RegisterBoolSetting( func (p *planner) prepareUsingOptimizer( ctx context.Context, origin prep.StatementOrigin, ) (planFlags, error) { - stmt := &p.stmt - opc := &p.optPlanningCtx + + // If there are externally-injected hints, first try preparing with the + // injected hints. + if p.stmt.ASTWithInjectedHints != nil { + opc.log(ctx, "trying preparing with injected hints") + p.usingHintInjection = true + opc.reset(ctx) + flags, err := p.prepareUsingOptimizerInternal(ctx, origin) + p.usingHintInjection = false + if err == nil { + return flags, nil + } + // If semantic analysis failed, try preparing again without injected hints. + log.Eventf(ctx, "preparing with injected hints failed with: %v", err) + opc.log(ctx, "falling back to preparing without injected hints") + } opc.reset(ctx) + return p.prepareUsingOptimizerInternal(ctx, origin) +} + +func (p *planner) prepareUsingOptimizerInternal( + ctx context.Context, origin prep.StatementOrigin, +) (planFlags, error) { + stmt := &p.stmt + opc := &p.optPlanningCtx + if origin == prep.StatementOriginSessionMigration { opc.flags.Set(planFlagSessionMigration) } @@ -156,8 +179,9 @@ func (p *planner) prepareUsingOptimizer( stmt.Hints = pm.Hints stmt.HintIDs = pm.HintIDs stmt.HintsGeneration = pm.HintsGeneration + stmt.ASTWithInjectedHints = pm.ASTWithInjectedHints if cachedData.Memo.IsOptimized() { - // A cache, fully optimized memo is an "ideal generic + // A cached, fully optimized memo is an "ideal generic // memo". stmt.Prepared.GenericMemo = cachedData.Memo stmt.Prepared.IdealGenericPlan = true @@ -262,7 +286,29 @@ func (p *planner) makeOptimizerPlan(ctx context.Context) error { p.curPlan.init(&p.stmt, &p.instrumentation) opc := &p.optPlanningCtx + + // If there are externally-injected hints, first try planning with the + // injected hints. + if p.stmt.ASTWithInjectedHints != nil { + opc.log(ctx, "trying planning with injected hints") + p.usingHintInjection = true + opc.reset(ctx) + err := p.makeOptimizerPlanInternal(ctx) + p.usingHintInjection = false + if err == nil { + return nil + } + // If semantic analysis or optimization failed, try planning again without + // injected hints. + log.Eventf(ctx, "planning with injected hints failed with: %v", err) + opc.log(ctx, "falling back to planning without injected hints") + } opc.reset(ctx) + return p.makeOptimizerPlanInternal(ctx) +} + +func (p *planner) makeOptimizerPlanInternal(ctx context.Context) error { + opc := &p.optPlanningCtx execMemo, err := opc.buildExecMemo(ctx) if err != nil { @@ -510,7 +556,11 @@ func (opc *optPlanningCtx) buildReusableMemo( // that there's even less to do during the EXECUTE phase. // f := opc.optimizer.Factory() - bld := optbuilder.New(ctx, &p.semaCtx, p.EvalContext(), opc.catalog, f, opc.p.stmt.AST) + ast := opc.p.stmt.AST + if opc.p.usingHintInjection { + ast = opc.p.stmt.ASTWithInjectedHints + } + bld := optbuilder.New(ctx, &p.semaCtx, p.EvalContext(), opc.catalog, f, ast) bld.KeepPlaceholders = true if opc.flags.IsSet(planFlagSessionMigration) { bld.SkipAOST = true @@ -870,7 +920,11 @@ func (opc *optPlanningCtx) buildExecMemo(ctx context.Context) (_ *memo.Memo, _ e // available. f := opc.optimizer.Factory() f.FoldingControl().AllowStableFolds() - bld := optbuilder.New(ctx, &p.semaCtx, p.EvalContext(), opc.catalog, f, opc.p.stmt.AST) + ast := opc.p.stmt.AST + if opc.p.usingHintInjection { + ast = opc.p.stmt.ASTWithInjectedHints + } + bld := optbuilder.New(ctx, &p.semaCtx, p.EvalContext(), opc.catalog, f, ast) if err := bld.Build(); err != nil { return nil, err } diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index ea6c7a9acea5..66fe45522a4b 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -321,6 +321,10 @@ type planner struct { // skipUnsafeInternalsCheck is used to skip the check that the // planner is not used for unsafe internal statements. skipUnsafeInternalsCheck bool + + // usingHintInjection is true if we're passing the rewritten AST with injected + // hints into optbuild. It is only set during planning. + usingHintInjection bool } // hasFlowForPausablePortal returns true if the planner is for re-executing a @@ -1001,6 +1005,8 @@ func (p *planner) resetPlanner( p.autoRetryCounter = 0 p.autoRetryStmtReason = nil p.autoRetryStmtCounter = 0 + + p.usingHintInjection = false } // GetReplicationStreamManager returns a ReplicationStreamManager. diff --git a/pkg/sql/prep/metadata.go b/pkg/sql/prep/metadata.go index f554f0b93fee..1d6d4e2d28da 100644 --- a/pkg/sql/prep/metadata.go +++ b/pkg/sql/prep/metadata.go @@ -53,6 +53,9 @@ type Metadata struct { // HintsGeneration is the generation of the hints cache at the time the // hints were retrieved, used for invalidation of cached plans. HintsGeneration int64 + + // ASTWithInjectedHints is the AST rewritten with injected hints. + ASTWithInjectedHints tree.Statement } // MemoryEstimate returns an estimation (in bytes) of how much memory is used by @@ -63,6 +66,9 @@ func (pm *Metadata) MemoryEstimate() int64 { // We don't have a good way of estimating the size of the AST. Just assume // it's a small multiple of the string length. res += 2 * int64(len(pm.SQL)) + if pm.ASTWithInjectedHints != nil { + res += 2 * int64(len(pm.SQL)) + } res += int64(len(pm.StatementNoConstants)) diff --git a/pkg/sql/statement.go b/pkg/sql/statement.go index 0d0621202cdc..3a44e0436601 100644 --- a/pkg/sql/statement.go +++ b/pkg/sql/statement.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/prep" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlcommenter" + "github.com/cockroachdb/cockroach/pkg/util/log" ) // Statement contains a statement with optional expected result columns and metadata. @@ -54,6 +55,10 @@ type Statement struct { // HintsGeneration is the generation of the hints cache at the time the // hints were retrieved, used for invalidation of cached plans. HintsGeneration int64 + + // ASTWithInjectedHints is the AST rewritten with injected hints. It is + // computed from the hints and AST during ReloadHintsIfStale and cached here. + ASTWithInjectedHints tree.Statement } // makeStatement creates a Statement with the metadata necessary to execute the @@ -106,16 +111,17 @@ func makeStatementFromPrepared( tags = sqlcommenter.ExtractQueryTags(comments[cl-1]) } s := Statement{ - Statement: prepared.Statement, - Prepared: prepared, - ExpectedTypes: prepared.Columns, - StmtNoConstants: prepared.StatementNoConstants, - StmtSummary: prepared.StatementSummary, - QueryID: queryID, - QueryTags: tags, - Hints: prepared.Hints, - HintIDs: prepared.HintIDs, - HintsGeneration: prepared.HintsGeneration, + Statement: prepared.Statement, + Prepared: prepared, + ExpectedTypes: prepared.Columns, + StmtNoConstants: prepared.StatementNoConstants, + StmtSummary: prepared.StatementSummary, + QueryID: queryID, + QueryTags: tags, + Hints: prepared.Hints, + HintIDs: prepared.HintIDs, + HintsGeneration: prepared.HintsGeneration, + ASTWithInjectedHints: prepared.ASTWithInjectedHints, } s.ReloadHintsIfStale(ctx, fmtFlags, statementHintsCache) return s @@ -141,17 +147,25 @@ func (s *Statement) ReloadHintsIfStale( if hintsGeneration == s.HintsGeneration { return false } + ast := s.AST if statementHintsCache != nil { hintStmtFingerprint := s.StmtNoConstants - switch e := s.Statement.AST.(type) { + // Unwrap wrapper statements before checking for hints. We try to make this + // unwrapping match what is done for crdb_internal.statement_statistics / + // the SQL Activity page. + switch e := s.AST.(type) { case *tree.CopyTo: - hintStmtFingerprint = tree.FormatStatementHideConstants(e.Statement, fmtFlags) + ast = e.Statement + hintStmtFingerprint = tree.FormatStatementHideConstants(ast, fmtFlags) case *tree.Explain: - hintStmtFingerprint = tree.FormatStatementHideConstants(e.Statement, fmtFlags) + ast = e.Statement + hintStmtFingerprint = tree.FormatStatementHideConstants(ast, fmtFlags) case *tree.ExplainAnalyze: - hintStmtFingerprint = tree.FormatStatementHideConstants(e.Statement, fmtFlags) + ast = e.Statement + hintStmtFingerprint = tree.FormatStatementHideConstants(ast, fmtFlags) case *tree.Prepare: - hintStmtFingerprint = tree.FormatStatementHideConstants(e.Statement, fmtFlags) + ast = e.Statement + hintStmtFingerprint = tree.FormatStatementHideConstants(ast, fmtFlags) } hints, hintIDs = statementHintsCache.MaybeGetStatementHints(ctx, hintStmtFingerprint, fmtFlags) } @@ -161,5 +175,54 @@ func (s *Statement) ReloadHintsIfStale( s.Hints = hints s.HintIDs = hintIDs s.HintsGeneration = hintsGeneration + s.ASTWithInjectedHints = nil + + log.Eventf( + ctx, "loaded %d external statement hints from the statement hints cache: %v", + len(s.Hints), s.HintIDs, + ) + + for i, hint := range s.Hints { + if hd := hint.HintInjectionDonor; hd != nil { + if err := hd.Validate(ast, fmtFlags); err != nil { + log.Eventf( + ctx, "failed to validate hint injection donor from external statement hint %v: %v", + s.HintIDs[i], err, + ) + continue + } + injectedAST, injected, err := hd.InjectHints(ast) + if err != nil { + log.Eventf( + ctx, "failed to inject hints from external statement hint %v: %v", s.HintIDs[i], err, + ) + continue + } + if injected { + log.Eventf(ctx, "injected hints from external statement hint %v", s.HintIDs[i]) + // If we unwrapped the AST above, we need to re-wrap the rewritten AST here. + s.ASTWithInjectedHints = injectedAST + switch e := s.AST.(type) { + case *tree.CopyTo: + copyTo := *e + copyTo.Statement = injectedAST + s.ASTWithInjectedHints = ©To + case *tree.Explain: + explain := *e + explain.Statement = injectedAST + s.ASTWithInjectedHints = &explain + case *tree.ExplainAnalyze: + explain := *e + explain.Statement = injectedAST + s.ASTWithInjectedHints = &explain + case *tree.Prepare: + prepare := *e + prepare.Statement = injectedAST + s.ASTWithInjectedHints = &prepare + } + break + } + } + } return true } From fbfcd7d39ac7819a47e235892b4f8ab727c08a78 Mon Sep 17 00:00:00 2001 From: Michael Erickson Date: Wed, 19 Nov 2025 15:07:30 -0800 Subject: [PATCH 3/3] sql/*: invalidate cached memos after hint injection changes If we build a memo with hint injection, and then later we realize that memo won't work (maybe because we discover the hint is unsatisfiable during execution of a prepared statement) we need to invalidate the cached memo. To do this, add a usingHintInjection field which tells the memo staleness check whether we're trying with or without hint injection. Also, in a related but separate change, this commit adds all matching HintIDs to the optimizer metadata so that we don't invalidate cached memos if the hintsGeneration changed due to some unrelated statement hints changing. Informs: #153633 Release note: None --- pkg/sql/faketreeeval/evalctx.go | 10 + .../logic_test/statement_hint_builtins | 247 +++++++++++++++++- pkg/sql/opt/memo/memo.go | 3 + pkg/sql/opt/metadata.go | 18 +- pkg/sql/plan_opt.go | 2 + pkg/sql/planner.go | 10 + pkg/sql/sem/eval/deps.go | 8 + 7 files changed, 288 insertions(+), 10 deletions(-) diff --git a/pkg/sql/faketreeeval/evalctx.go b/pkg/sql/faketreeeval/evalctx.go index 976e87fe297c..719180db45b2 100644 --- a/pkg/sql/faketreeeval/evalctx.go +++ b/pkg/sql/faketreeeval/evalctx.go @@ -603,6 +603,16 @@ func (ep *DummyEvalPlanner) InsertStatementHint( return 0, nil } +// UsingHintInjection is part of the eval.Planner interface. +func (ep *DummyEvalPlanner) UsingHintInjection() bool { + return false +} + +// GetHintIDs is part of the eval.Planner interface. +func (ep *DummyEvalPlanner) GetHintIDs() []int64 { + return nil +} + // DummyPrivilegedAccessor implements the tree.PrivilegedAccessor interface by returning errors. type DummyPrivilegedAccessor struct{} diff --git a/pkg/sql/logictest/testdata/logic_test/statement_hint_builtins b/pkg/sql/logictest/testdata/logic_test/statement_hint_builtins index de33ccce6cef..fc720c722fb2 100644 --- a/pkg/sql/logictest/testdata/logic_test/statement_hint_builtins +++ b/pkg/sql/logictest/testdata/logic_test/statement_hint_builtins @@ -188,10 +188,6 @@ DEALLOCATE ALL statement ok CREATE TABLE abc (a INT PRIMARY KEY, b INT, c INT, INDEX (b)) -# Temporary, until next commit. -statement ok -SET CLUSTER SETTING sql.query_cache.enabled = off - # Try some simple hint injections. First, an index hint. statement ok @@ -411,7 +407,7 @@ statement hints count: 1 # Try a prepared statement with an injected hint. -statement ok +let $hint_p1 SELECT crdb_internal.inject_hint( 'SELECT c FROM abc WHERE b > _', 'SELECT c FROM abc@{NO_INDEX_JOIN} WHERE b > _' @@ -424,10 +420,105 @@ statement ok SET tracing = on statement ok -PREPARE p AS SELECT c FROM abc WHERE b > $1 +PREPARE p1 AS SELECT c FROM abc WHERE b > $1 + +statement ok +EXECUTE p1 (5) + +statement ok +SET tracing = off + +query T +SELECT regexp_replace(message, E'\\d+', 'x') FROM [SHOW TRACE FOR SESSION] WHERE message LIKE '%injected hints%' +---- +injected hints from external statement hint x +injected hints from external statement hint x +trying preparing with injected hints +trying planning with injected hints + +# Try injecting a hint between prepare and execute. + +statement ok +SET tracing = on + +statement ok +PREPARE p2 AS SELECT c + 1 FROM abc WHERE b > $1 + +statement ok +SET tracing = off + +# (This should be empty.) +query T +SELECT regexp_replace(message, E'\\d+', 'x') FROM [SHOW TRACE FOR SESSION] WHERE message LIKE '%injected hints%' +---- + +statement ok +SELECT crdb_internal.inject_hint( + 'SELECT c + _ FROM abc WHERE b > _', + 'SELECT c + _ FROM abc@{FORCE_INDEX=abc_pkey,DESC} WHERE b > _' +) + +statement ok +SELECT crdb_internal.await_statement_hints_cache() + +statement ok +SET tracing = on + +statement ok +EXECUTE p2 (5) + +statement ok +SET tracing = off + +query T +SELECT regexp_replace(message, E'\\d+', 'x') FROM [SHOW TRACE FOR SESSION] WHERE message LIKE '%injected hints%' +---- +injected hints from external statement hint x +trying planning with injected hints + +# Try removing an injected hint between prepare and execute. + +# (Re-use p1.) +statement ok +DELETE FROM system.statement_hints WHERE row_id = $hint_p1 + +statement ok +SELECT crdb_internal.await_statement_hints_cache() + +statement ok +SET tracing = on + +statement ok +EXECUTE p1 (6) + +statement ok +SET tracing = off + +# (This should be empty.) +query T +SELECT regexp_replace(message, E'\\d+', 'x') FROM [SHOW TRACE FOR SESSION] WHERE message LIKE '%injected hints%' +---- + +# Check that we do not use an invalid index hint injected into a prepared +# statement. + +statement ok +SELECT crdb_internal.inject_hint( + 'SELECT sum(a) FROM abc WHERE c = _', + 'SELECT sum(a) FROM abc@abc_foo WHERE c = _' +) + +statement ok +SELECT crdb_internal.await_statement_hints_cache() + +statement ok +SET tracing = on + +statement ok +PREPARE p3 AS SELECT sum(a) FROM abc WHERE c = $1 statement ok -EXECUTE p (5) +EXECUTE p3 (5) statement ok SET tracing = off @@ -438,8 +529,146 @@ SELECT regexp_replace(message, E'\\d+', 'x') FROM [SHOW TRACE FOR SESSION] WHERE injected hints from external statement hint x injected hints from external statement hint x trying preparing with injected hints +preparing with injected hints failed with: index "abc_foo" not found +falling back to preparing without injected hints trying planning with injected hints +planning with injected hints failed with: index "abc_foo" not found +falling back to planning without injected hints + +# Check that we do not use an unsatisfiable hint injected into a prepared +# statement. -# Temporary, until next commit. statement ok -RESET CLUSTER SETTING sql.query_cache.enabled +SELECT crdb_internal.inject_hint( + 'SELECT max(a) FROM abc WHERE (b = _) AND (c = _)', + 'SELECT max(a) FROM abc@{FORCE_ZIGZAG} WHERE (b = _) AND (c = _)' +) + +statement ok +SELECT crdb_internal.await_statement_hints_cache() + +statement ok +SET tracing = on + +statement ok +PREPARE p4 AS SELECT max(a) FROM abc WHERE b = $1 AND c = $2 + +statement ok +EXECUTE p4 (5, 6) + +statement ok +SET tracing = off + +query T +SELECT regexp_replace(message, E'\\d+', 'x') FROM [SHOW TRACE FOR SESSION] WHERE message LIKE '%injected hints%' +---- +injected hints from external statement hint x +injected hints from external statement hint x +trying preparing with injected hints +trying planning with injected hints +planning with injected hints failed with: could not produce a query plan conforming to the FORCE_ZIGZAG hint +falling back to planning without injected hints + +# Check that we can inject hints into generic query plans. + +statement ok +SET plan_cache_mode = force_generic_plan + +statement ok +SELECT crdb_internal.inject_hint( + 'SELECT a * _ FROM abc WHERE b > _', + 'SELECT a * _ FROM abc@abc_pkey WHERE b > _' +) + +statement ok +SELECT crdb_internal.await_statement_hints_cache() + +statement ok +SET tracing = on + +statement ok +PREPARE p5 AS SELECT a * 2 FROM abc WHERE b > $1 + +statement ok +EXECUTE p5 (5) + +statement ok +EXECUTE p5 (6) + +statement ok +SET tracing = off + +query T +SELECT regexp_replace(message, E'\\d+', 'x') FROM [SHOW TRACE FOR SESSION] WHERE message LIKE '%injected hints%' OR message LIKE '%(generic)%' +---- +injected hints from external statement hint x +injected hints from external statement hint x +trying preparing with injected hints +trying planning with injected hints +optimizing (generic) +trying planning with injected hints + +# Check that we can inject hints when using plan_cache_mode=auto. + +statement ok +SET plan_cache_mode = auto + +statement ok +SELECT crdb_internal.inject_hint( + 'SELECT a - b FROM abc WHERE b = _', + 'SELECT a - b FROM abc@abc_pkey WHERE b = _' +) + +statement ok +SELECT crdb_internal.await_statement_hints_cache() + +statement ok +SET tracing = on + +statement ok +PREPARE p6 AS SELECT a - b FROM abc WHERE b = $1 + +statement ok +EXECUTE p6 (5) + +statement ok +EXECUTE p6 (6) + +statement ok +EXECUTE p6 (7) + +statement ok +EXECUTE p6 (8) + +statement ok +EXECUTE p6 (9) + +statement ok +EXECUTE p6 (10) + +statement ok +EXECUTE p6 (11) + +statement ok +SET tracing = off + +query T +SELECT regexp_replace(message, E'\\d+', 'x') FROM [SHOW TRACE FOR SESSION] WHERE message LIKE '%injected hints%' OR message LIKE '%(generic)%' +---- +injected hints from external statement hint x +injected hints from external statement hint x +trying preparing with injected hints +trying planning with injected hints +trying planning with injected hints +trying planning with injected hints +trying planning with injected hints +trying planning with injected hints +trying planning with injected hints +optimizing (generic) +trying planning with injected hints + +statement ok +RESET plan_cache_mode + +statement ok +DEALLOCATE ALL diff --git a/pkg/sql/opt/memo/memo.go b/pkg/sql/opt/memo/memo.go index 7bbd811d0de8..51a6bfbd08e8 100644 --- a/pkg/sql/opt/memo/memo.go +++ b/pkg/sql/opt/memo/memo.go @@ -215,6 +215,7 @@ type Memo struct { clampLowHistogramSelectivity bool clampInequalitySelectivity bool useMaxFrequencySelectivity bool + usingHintInjection bool // txnIsoLevel is the isolation level under which the plan was created. This // affects the planning of some locking operations, so it must be included in @@ -330,6 +331,7 @@ func (m *Memo) Init(ctx context.Context, evalCtx *eval.Context) { clampLowHistogramSelectivity: evalCtx.SessionData().OptimizerClampLowHistogramSelectivity, clampInequalitySelectivity: evalCtx.SessionData().OptimizerClampInequalitySelectivity, useMaxFrequencySelectivity: evalCtx.SessionData().OptimizerUseMaxFrequencySelectivity, + usingHintInjection: evalCtx.Planner.UsingHintInjection(), txnIsoLevel: evalCtx.TxnIsoLevel, } m.metadata.Init() @@ -509,6 +511,7 @@ func (m *Memo) IsStale( m.clampLowHistogramSelectivity != evalCtx.SessionData().OptimizerClampLowHistogramSelectivity || m.clampInequalitySelectivity != evalCtx.SessionData().OptimizerClampInequalitySelectivity || m.useMaxFrequencySelectivity != evalCtx.SessionData().OptimizerUseMaxFrequencySelectivity || + m.usingHintInjection != evalCtx.Planner.UsingHintInjection() || m.txnIsoLevel != evalCtx.TxnIsoLevel { return true, nil } diff --git a/pkg/sql/opt/metadata.go b/pkg/sql/opt/metadata.go index 9de39affd0b4..dceb6fa7e0d7 100644 --- a/pkg/sql/opt/metadata.go +++ b/pkg/sql/opt/metadata.go @@ -9,6 +9,7 @@ import ( "context" "fmt" "math/bits" + "slices" "strings" "github.com/cockroachdb/cockroach/pkg/clusterversion" @@ -148,6 +149,9 @@ type Metadata struct { // execution. rlsMeta RowLevelSecurityMeta + // hintIDs are the external statement hints that match this statement. + hintIDs []int64 + digest struct { syncutil.Mutex depDigest cat.DependencyDigest @@ -254,7 +258,7 @@ func (md *Metadata) CopyFrom(from *Metadata, copyScalarFn func(Expr) Expr) { len(md.sequences) != 0 || len(md.views) != 0 || len(md.userDefinedTypes) != 0 || len(md.userDefinedTypesSlice) != 0 || len(md.dataSourceDeps) != 0 || len(md.routineDeps) != 0 || len(md.objectRefsByName) != 0 || len(md.privileges) != 0 || - len(md.builtinRefsByName) != 0 || md.rlsMeta.IsInitialized { + len(md.builtinRefsByName) != 0 || md.rlsMeta.IsInitialized || len(md.hintIDs) != 0 { panic(errors.AssertionFailedf("CopyFrom requires empty destination")) } md.schemas = append(md.schemas, from.schemas...) @@ -337,6 +341,8 @@ func (md *Metadata) CopyFrom(from *Metadata, copyScalarFn func(Expr) Expr) { md.withBindings = nil md.rlsMeta = from.rlsMeta.Copy() + + md.hintIDs = append(md.hintIDs, from.hintIDs...) } // MDDepName stores either the unresolved DataSourceName or the StableID from @@ -602,6 +608,11 @@ func (md *Metadata) CheckDependencies( return upToDate, err } + // Check that external statement hints have not changed. + if !slices.Equal(md.hintIDs, evalCtx.Planner.GetHintIDs()) { + return false, nil + } + // Update the digest after a full dependency check, since our fast // check did not succeed. if evalCtx.SessionData().CatalogDigestStalenessCheckEnabled { @@ -1280,3 +1291,8 @@ func (md *Metadata) checkRLSDependencies( // check already accounts for changes in the table descriptor version. return true, nil } + +// SetHintIDs copies the given matching hintIDs into the metadata. +func (md *Metadata) SetHintIDs(hintIDs []int64) { + md.hintIDs = append(md.hintIDs, hintIDs...) +} diff --git a/pkg/sql/plan_opt.go b/pkg/sql/plan_opt.go index 4d7c7cbc3b37..0f27046fb942 100644 --- a/pkg/sql/plan_opt.go +++ b/pkg/sql/plan_opt.go @@ -560,6 +560,7 @@ func (opc *optPlanningCtx) buildReusableMemo( if opc.p.usingHintInjection { ast = opc.p.stmt.ASTWithInjectedHints } + f.Metadata().SetHintIDs(opc.p.GetHintIDs()) bld := optbuilder.New(ctx, &p.semaCtx, p.EvalContext(), opc.catalog, f, ast) bld.KeepPlaceholders = true if opc.flags.IsSet(planFlagSessionMigration) { @@ -924,6 +925,7 @@ func (opc *optPlanningCtx) buildExecMemo(ctx context.Context) (_ *memo.Memo, _ e if opc.p.usingHintInjection { ast = opc.p.stmt.ASTWithInjectedHints } + f.Metadata().SetHintIDs(opc.p.GetHintIDs()) bld := optbuilder.New(ctx, &p.semaCtx, p.EvalContext(), opc.catalog, f, ast) if err := bld.Build(); err != nil { return nil, err diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index 66fe45522a4b..f0bc15afd05b 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -1148,3 +1148,13 @@ func (p *planner) InsertStatementHint( ) (int64, error) { return hints.InsertHintIntoDB(ctx, p.InternalSQLTxn(), statementFingerprint, hint) } + +// UsingHintInjection is part of the eval.Planner interface. +func (p *planner) UsingHintInjection() bool { + return p.usingHintInjection +} + +// GetHintIDs is part of the eval.Planner interface. +func (p *planner) GetHintIDs() []int64 { + return p.stmt.HintIDs +} diff --git a/pkg/sql/sem/eval/deps.go b/pkg/sql/sem/eval/deps.go index 7191ce3a596c..2e8551b76dfe 100644 --- a/pkg/sql/sem/eval/deps.go +++ b/pkg/sql/sem/eval/deps.go @@ -476,6 +476,14 @@ type Planner interface { // the system.statement_hints table. It returns the hint ID of the newly // created hint. InsertStatementHint(ctx context.Context, statementFingerprint string, hint hintpb.StatementHintUnion) (int64, error) + + // UsingHintInjection returns whether we are planning with externally-injected + // hints. + UsingHintInjection() bool + + // GetHintIDs returns the external statement hints we're using for this + // statement. + GetHintIDs() []int64 } // InternalRows is an iterator interface that's exposed by the internal