Skip to content

Commit 7d15c71

Browse files
craig[bot]DrewKimball
andcommitted
Merge #155251
155251: sql/hints: add facilities to invalidate query plans when their hints change r=DrewKimball a=DrewKimball #### sql/hints: add statement hint cache generation This commit adds a generation to the statement hints cache, which is incremented every time the cache processes a rangefeed event. The generation is used as part of the `cat.DependencyDigest`, used to decide whether an expensive staleness check is necessary for a cached plan. Informs #148160 Release note: None #### sql/hints: include row ID when fetching hints The row ID of each hint is now included with the hints fetched from the hint cache. In a future PR, cached query plans will store the row IDs of the hints they retrieved from the cache, and will compare against the re-fetched IDs to determine if the query plan is stale (e.g. hints were added or removed). Informs #148160 Release note: None Co-authored-by: Drew Kimball <[email protected]>
2 parents c5ac4c9 + 0937148 commit 7d15c71

File tree

5 files changed

+205
-57
lines changed

5 files changed

+205
-57
lines changed

pkg/sql/hints/hint_cache.go

Lines changed: 49 additions & 28 deletions
Original file line numberDiff line numberDiff line change
@@ -9,6 +9,7 @@ import (
99
"context"
1010
"hash/fnv"
1111
"sync"
12+
"sync/atomic"
1213
"time"
1314

1415
"github.com/cockroachdb/cockroach/pkg/keys"
@@ -87,6 +88,10 @@ type StatementHintsCache struct {
8788

8889
// Used to read hints from the database when they are not in the cache.
8990
db descs.DB
91+
92+
// generation is incremented any time the hint cache is updated by the
93+
// rangefeed.
94+
generation atomic.Int64
9095
}
9196

9297
// cacheSize is the size of the entries to store in the cache.
@@ -214,6 +219,7 @@ func (c *StatementHintsCache) onUpdate(
214219
// handleInitialScan builds the hintedHashes map and adds it to
215220
// StatementHintsCache after the initial scan completes.
216221
func (c *StatementHintsCache) handleInitialScan(update rangefeedcache.Update[*bufferEvent]) {
222+
defer c.generation.Add(1)
217223
hintedHashes := make(map[int64]hlc.Timestamp)
218224
for _, ev := range update.Events {
219225
if ev.del {
@@ -246,6 +252,7 @@ func (c *StatementHintsCache) handleIncrementalUpdate(
246252
// Avoid synchronization when we're just bumping the resolved timestamp.
247253
return
248254
}
255+
defer c.generation.Add(1)
249256
c.mu.Lock()
250257
defer c.mu.Unlock()
251258
for _, ev := range update.Events {
@@ -309,8 +316,6 @@ func (c *StatementHintsCache) checkHashHasHintsAsync(
309316
// Unset the timestamp to indicate that the refresh is done.
310317
c.mu.hintedHashes[hash] = hlc.Timestamp{}
311318
} else {
312-
// There may be a hintCache entry with no hints; drop it.
313-
c.mu.hintCache.Del(hash)
314319
delete(c.mu.hintedHashes, hash)
315320
}
316321
return true
@@ -378,18 +383,25 @@ var _ rangefeedbuffer.Event = &bufferEvent{}
378383
// Reading from the cache.
379384
// ============================================================================
380385

386+
// GetGeneration returns the current generation, which will change if any
387+
// modifications happen to the cache.
388+
func (c *StatementHintsCache) GetGeneration() int64 {
389+
return c.generation.Load()
390+
}
391+
381392
// MaybeGetStatementHints attempts to retrieve the hints for the given statement
382-
// fingerprint. It returns nil if the statement has no hints, or there was an
383-
// error retrieving them.
393+
// fingerprint, along with the unique ID of each hint (for invalidating cached
394+
// plans). It returns nil if the statement has no hints, or there was an error
395+
// retrieving them.
384396
func (c *StatementHintsCache) MaybeGetStatementHints(
385397
ctx context.Context, statementFingerprint string,
386-
) (hints []StatementHint) {
398+
) (hints []StatementHint, ids []int64) {
387399
hash := fnv.New64()
388400
_, err := hash.Write([]byte(statementFingerprint))
389401
if err != nil {
390402
// This should never happen for 64-bit FNV-1.
391403
log.Dev.Errorf(ctx, "failed to compute hash for statement fingerprint: %v", err)
392-
return nil
404+
return nil, nil
393405
}
394406
statementHash := int64(hash.Sum64())
395407
c.mu.Lock()
@@ -399,7 +411,7 @@ func (c *StatementHintsCache) MaybeGetStatementHints(
399411
// unconditionally check hintCache.
400412
if _, ok := c.mu.hintedHashes[statementHash]; !ok {
401413
// There are no plan hints for this query.
402-
return nil
414+
return nil, nil
403415
}
404416
}
405417
e, ok := c.mu.hintCache.Get(statementHash)
@@ -441,7 +453,7 @@ func (c *StatementHintsCache) maybeWaitForRefreshLocked(
441453
// released while reading from the db, and then reacquired.
442454
func (c *StatementHintsCache) addCacheEntryLocked(
443455
ctx context.Context, statementHash int64, statementFingerprint string,
444-
) []StatementHint {
456+
) (hints []StatementHint, ids []int64) {
445457
c.mu.AssertHeld()
446458

447459
// Add a cache entry that other queries can find and wait on until we have the
@@ -470,20 +482,24 @@ func (c *StatementHintsCache) addCacheEntryLocked(
470482
// Remove the cache entry so that the next caller retries the query.
471483
c.mu.hintCache.Del(statementHash)
472484
log.VEventf(ctx, 1, "encountered error while reading hints for query %s", statementFingerprint)
473-
return nil
485+
return nil, nil
474486
}
475487
return entry.getMatchingHints(statementFingerprint)
476488
}
477489

478490
// getStatementHintsFromDB queries the system.statement_hints table for hints
479491
// matching the given fingerprint hash. It is able to handle the case when
480492
// multiple fingerprints match the hash, as well as the case when there are no
481-
// hints for the fingerprint.
493+
// hints for the fingerprint. Results are ordered by row ID.
482494
func (c *StatementHintsCache) getStatementHintsFromDB(
483495
ctx context.Context, statementHash int64, entry *cacheEntry,
484496
) (retErr error) {
485497
const opName = "get-plan-hints"
486-
const getHintsStmt = `SELECT "fingerprint", "hint" FROM system.statement_hints WHERE "hash" = $1`
498+
const getHintsStmt = `
499+
SELECT "row_id", "fingerprint", "hint"
500+
FROM system.statement_hints
501+
WHERE "hash" = $1
502+
ORDER BY "row_id" ASC`
487503
it, err := c.db.Executor().QueryIteratorEx(
488504
ctx, opName, nil /* txn */, sessiondata.NodeUserSessionDataOverride,
489505
getHintsStmt, statementHash,
@@ -500,11 +516,13 @@ func (c *StatementHintsCache) getStatementHintsFromDB(
500516
return err
501517
}
502518
datums := it.Cur()
503-
fingerprint := string(tree.MustBeDString(datums[0]))
504-
hint, err := NewStatementHint([]byte(tree.MustBeDBytes(datums[1])))
519+
rowID := int64(tree.MustBeDInt(datums[0]))
520+
fingerprint := string(tree.MustBeDString(datums[1]))
521+
hint, err := NewStatementHint([]byte(tree.MustBeDBytes(datums[2])))
505522
if err != nil {
506523
return err
507524
}
525+
entry.ids = append(entry.ids, rowID)
508526
entry.fingerprints = append(entry.fingerprints, fingerprint)
509527
entry.hints = append(entry.hints, hint)
510528
}
@@ -517,30 +535,33 @@ type cacheEntry struct {
517535
mustWait bool
518536
waitCond sync.Cond
519537

520-
// fingerprints and hints have the same length. fingerprints[i] is the
521-
// statement fingerprint to which hints[i] applies. We have to keep track of
522-
// the fingerprints to resolve hash collisions. Note that a single fingerprint
523-
// can have multiple hints, in which case there will be duplicate entries in
524-
// the fingerprints slice.
538+
// hints, fingerprints, and ids have the same length. fingerprints[i] is the
539+
// statement fingerprint to which hints[i] applies, while ids[i] uniquely
540+
// identifies a hint in the system table. They are kept in order of id.
541+
//
542+
// We track the hint ID for invalidating cached query plans after a hint is
543+
// added or removed. We track the fingerprint to resolve hash collisions. Note
544+
// that a single fingerprint can have multiple hints, in which case there will
545+
// be duplicate entries in the fingerprints slice.
525546
// TODO(drewk): consider de-duplicating the fingerprint strings to reduce
526547
// memory usage.
527-
fingerprints []string
528548
hints []StatementHint
549+
fingerprints []string
550+
ids []int64
529551
}
530552

531-
// getMatchingHints returns the plan hints for the given fingerprint, or nil if
532-
// they don't exist.
533-
func (entry *cacheEntry) getMatchingHints(statementFingerprint string) []StatementHint {
534-
var res []StatementHint
553+
// getMatchingHints returns the plan hints and row IDs for the given
554+
// fingerprint, or nil if they don't exist. The results are in order of row ID.
555+
func (entry *cacheEntry) getMatchingHints(
556+
statementFingerprint string,
557+
) (hints []StatementHint, ids []int64) {
535558
for i := range entry.hints {
536559
if entry.fingerprints[i] == statementFingerprint {
537-
// TODO(drewk,michae2): we could do something smarter here to avoid
538-
// allocations. For example, order by fingerprint in the original query
539-
// so that we can return a slice here.
540-
res = append(res, entry.hints[i])
560+
hints = append(hints, entry.hints[i])
561+
ids = append(ids, entry.ids[i])
541562
}
542563
}
543-
return res
564+
return hints, ids
544565
}
545566

546567
// ============================================================================

0 commit comments

Comments
 (0)