Skip to content

Commit c956f73

Browse files
craig[bot]michae2
andcommitted
Merge #158096
158096: sql/*: add hint injection r=DrewKimball a=michae2 **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 --- **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. --- **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 Co-authored-by: Michael Erickson <michae2@cockroachlabs.com>
2 parents 3c88993 + b61eb64 commit c956f73

File tree

20 files changed

+1024
-117
lines changed

20 files changed

+1024
-117
lines changed

pkg/sql/conn_executor_exec.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -565,6 +565,7 @@ func (ex *connExecutor) execStmtInOpenState(
565565
stmt.Hints = ps.Hints
566566
stmt.HintIDs = ps.HintIDs
567567
stmt.HintsGeneration = ps.HintsGeneration
568+
stmt.ASTWithInjectedHints = ps.ASTWithInjectedHints
568569
stmt.ReloadHintsIfStale(ctx, stmtFingerprintFmtMask, statementHintsCache)
569570
res.ResetStmtType(ps.AST)
570571

@@ -1465,6 +1466,7 @@ func (ex *connExecutor) execStmtInOpenStateWithPausablePortal(
14651466
vars.stmt.Hints = ps.Hints
14661467
vars.stmt.HintIDs = ps.HintIDs
14671468
vars.stmt.HintsGeneration = ps.HintsGeneration
1469+
vars.stmt.ASTWithInjectedHints = ps.ASTWithInjectedHints
14681470
vars.stmt.ReloadHintsIfStale(ctx, stmtFingerprintFmtMask, statementHintsCache)
14691471
res.ResetStmtType(ps.AST)
14701472

pkg/sql/conn_executor_prepare.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -247,6 +247,7 @@ func (ex *connExecutor) prepare(
247247
prepared.Hints = stmt.Hints
248248
prepared.HintIDs = stmt.HintIDs
249249
prepared.HintsGeneration = stmt.HintsGeneration
250+
prepared.ASTWithInjectedHints = stmt.ASTWithInjectedHints
250251

251252
// Point to the prepared state, which can be further populated during query
252253
// preparation.

pkg/sql/faketreeeval/evalctx.go

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -603,6 +603,16 @@ func (ep *DummyEvalPlanner) InsertStatementHint(
603603
return 0, nil
604604
}
605605

606+
// UsingHintInjection is part of the eval.Planner interface.
607+
func (ep *DummyEvalPlanner) UsingHintInjection() bool {
608+
return false
609+
}
610+
611+
// GetHintIDs is part of the eval.Planner interface.
612+
func (ep *DummyEvalPlanner) GetHintIDs() []int64 {
613+
return nil
614+
}
615+
606616
// DummyPrivilegedAccessor implements the tree.PrivilegedAccessor interface by returning errors.
607617
type DummyPrivilegedAccessor struct{}
608618

pkg/sql/hints/BUILD.bazel

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -24,12 +24,14 @@ go_library(
2424
"//pkg/sql/catalog/systemschema",
2525
"//pkg/sql/hintpb",
2626
"//pkg/sql/isql",
27+
"//pkg/sql/parserutils",
2728
"//pkg/sql/rowenc",
2829
"//pkg/sql/sem/tree",
2930
"//pkg/sql/sessiondata",
3031
"//pkg/sql/types",
3132
"//pkg/util/buildutil",
3233
"//pkg/util/cache",
34+
"//pkg/util/errorutil",
3335
"//pkg/util/hlc",
3436
"//pkg/util/log",
3537
"//pkg/util/metamorphic",
@@ -59,11 +61,14 @@ go_test(
5961
"//pkg/security/securityassets",
6062
"//pkg/security/securitytest",
6163
"//pkg/server",
64+
"//pkg/settings/cluster",
6265
"//pkg/sql",
6366
"//pkg/sql/catalog/descs",
6467
"//pkg/sql/hintpb",
6568
"//pkg/sql/isql",
69+
"//pkg/sql/parserutils",
6670
"//pkg/sql/randgen",
71+
"//pkg/sql/sem/tree",
6772
"//pkg/sql/stats",
6873
"//pkg/storage/fs",
6974
"//pkg/testutils",

pkg/sql/hints/hint_cache.go

Lines changed: 11 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,6 @@ import (
2525
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
2626
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
2727
"github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema"
28-
"github.com/cockroachdb/cockroach/pkg/sql/hintpb"
2928
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
3029
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
3130
"github.com/cockroachdb/cockroach/pkg/sql/types"
@@ -406,8 +405,8 @@ func (c *StatementHintsCache) GetGeneration() int64 {
406405
// plans). It returns nil if the statement has no hints, or there was an error
407406
// retrieving them.
408407
func (c *StatementHintsCache) MaybeGetStatementHints(
409-
ctx context.Context, statementFingerprint string,
410-
) (hints []hintpb.StatementHintUnion, ids []int64) {
408+
ctx context.Context, statementFingerprint string, fingerprintFlags tree.FmtFlags,
409+
) (hints []Hint, ids []int64) {
411410
hash := fnv.New64()
412411
_, err := hash.Write([]byte(statementFingerprint))
413412
if err != nil {
@@ -430,7 +429,7 @@ func (c *StatementHintsCache) MaybeGetStatementHints(
430429
if !ok {
431430
// The plan hints were evicted from the cache. Retrieve them from the
432431
// database and add them to the cache.
433-
return c.addCacheEntryLocked(ctx, statementHash, statementFingerprint)
432+
return c.addCacheEntryLocked(ctx, statementHash, statementFingerprint, fingerprintFlags)
434433
}
435434
entry := e.(*cacheEntry)
436435
c.maybeWaitForRefreshLocked(ctx, entry, statementHash)
@@ -464,8 +463,11 @@ func (c *StatementHintsCache) maybeWaitForRefreshLocked(
464463
// other queries to wait for the result via sync.Cond. Note that the lock is
465464
// released while reading from the db, and then reacquired.
466465
func (c *StatementHintsCache) addCacheEntryLocked(
467-
ctx context.Context, statementHash int64, statementFingerprint string,
468-
) (hints []hintpb.StatementHintUnion, ids []int64) {
466+
ctx context.Context,
467+
statementHash int64,
468+
statementFingerprint string,
469+
fingerprintFlags tree.FmtFlags,
470+
) (hints []Hint, ids []int64) {
469471
c.mu.AssertHeld()
470472

471473
// Add a cache entry that other queries can find and wait on until we have the
@@ -483,7 +485,7 @@ func (c *StatementHintsCache) addCacheEntryLocked(
483485
defer c.mu.Lock()
484486
log.VEventf(ctx, 1, "reading hints for query %s", statementFingerprint)
485487
entry.ids, entry.fingerprints, entry.hints, err =
486-
GetStatementHintsFromDB(ctx, c.db.Executor(), statementHash)
488+
GetStatementHintsFromDB(ctx, c.db.Executor(), statementHash, fingerprintFlags)
487489
log.VEventf(ctx, 1, "finished reading hints for query %s", statementFingerprint)
488490
}()
489491

@@ -517,16 +519,14 @@ type cacheEntry struct {
517519
// be duplicate entries in the fingerprints slice.
518520
// TODO(drewk): consider de-duplicating the fingerprint strings to reduce
519521
// memory usage.
520-
hints []hintpb.StatementHintUnion
522+
hints []Hint
521523
fingerprints []string
522524
ids []int64
523525
}
524526

525527
// getMatchingHints returns the plan hints and row IDs for the given
526528
// fingerprint, or nil if they don't exist. The results are in order of row ID.
527-
func (entry *cacheEntry) getMatchingHints(
528-
statementFingerprint string,
529-
) (hints []hintpb.StatementHintUnion, ids []int64) {
529+
func (entry *cacheEntry) getMatchingHints(statementFingerprint string) (hints []Hint, ids []int64) {
530530
for i := range entry.hints {
531531
if entry.fingerprints[i] == statementFingerprint {
532532
hints = append(hints, entry.hints[i])

0 commit comments

Comments
 (0)