Skip to content

Commit 0937148

Browse files
committed
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
1 parent 59e8ced commit 0937148

File tree

2 files changed

+88
-53
lines changed

2 files changed

+88
-53
lines changed

pkg/sql/hints/hint_cache.go

Lines changed: 36 additions & 26 deletions
Original file line numberDiff line numberDiff line change
@@ -390,17 +390,18 @@ func (c *StatementHintsCache) GetGeneration() int64 {
390390
}
391391

392392
// MaybeGetStatementHints attempts to retrieve the hints for the given statement
393-
// fingerprint. It returns nil if the statement has no hints, or there was an
394-
// 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.
395396
func (c *StatementHintsCache) MaybeGetStatementHints(
396397
ctx context.Context, statementFingerprint string,
397-
) (hints []StatementHint) {
398+
) (hints []StatementHint, ids []int64) {
398399
hash := fnv.New64()
399400
_, err := hash.Write([]byte(statementFingerprint))
400401
if err != nil {
401402
// This should never happen for 64-bit FNV-1.
402403
log.Dev.Errorf(ctx, "failed to compute hash for statement fingerprint: %v", err)
403-
return nil
404+
return nil, nil
404405
}
405406
statementHash := int64(hash.Sum64())
406407
c.mu.Lock()
@@ -410,7 +411,7 @@ func (c *StatementHintsCache) MaybeGetStatementHints(
410411
// unconditionally check hintCache.
411412
if _, ok := c.mu.hintedHashes[statementHash]; !ok {
412413
// There are no plan hints for this query.
413-
return nil
414+
return nil, nil
414415
}
415416
}
416417
e, ok := c.mu.hintCache.Get(statementHash)
@@ -452,7 +453,7 @@ func (c *StatementHintsCache) maybeWaitForRefreshLocked(
452453
// released while reading from the db, and then reacquired.
453454
func (c *StatementHintsCache) addCacheEntryLocked(
454455
ctx context.Context, statementHash int64, statementFingerprint string,
455-
) []StatementHint {
456+
) (hints []StatementHint, ids []int64) {
456457
c.mu.AssertHeld()
457458

458459
// Add a cache entry that other queries can find and wait on until we have the
@@ -481,20 +482,24 @@ func (c *StatementHintsCache) addCacheEntryLocked(
481482
// Remove the cache entry so that the next caller retries the query.
482483
c.mu.hintCache.Del(statementHash)
483484
log.VEventf(ctx, 1, "encountered error while reading hints for query %s", statementFingerprint)
484-
return nil
485+
return nil, nil
485486
}
486487
return entry.getMatchingHints(statementFingerprint)
487488
}
488489

489490
// getStatementHintsFromDB queries the system.statement_hints table for hints
490491
// matching the given fingerprint hash. It is able to handle the case when
491492
// multiple fingerprints match the hash, as well as the case when there are no
492-
// hints for the fingerprint.
493+
// hints for the fingerprint. Results are ordered by row ID.
493494
func (c *StatementHintsCache) getStatementHintsFromDB(
494495
ctx context.Context, statementHash int64, entry *cacheEntry,
495496
) (retErr error) {
496497
const opName = "get-plan-hints"
497-
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`
498503
it, err := c.db.Executor().QueryIteratorEx(
499504
ctx, opName, nil /* txn */, sessiondata.NodeUserSessionDataOverride,
500505
getHintsStmt, statementHash,
@@ -511,11 +516,13 @@ func (c *StatementHintsCache) getStatementHintsFromDB(
511516
return err
512517
}
513518
datums := it.Cur()
514-
fingerprint := string(tree.MustBeDString(datums[0]))
515-
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])))
516522
if err != nil {
517523
return err
518524
}
525+
entry.ids = append(entry.ids, rowID)
519526
entry.fingerprints = append(entry.fingerprints, fingerprint)
520527
entry.hints = append(entry.hints, hint)
521528
}
@@ -528,30 +535,33 @@ type cacheEntry struct {
528535
mustWait bool
529536
waitCond sync.Cond
530537

531-
// fingerprints and hints have the same length. fingerprints[i] is the
532-
// statement fingerprint to which hints[i] applies. We have to keep track of
533-
// the fingerprints to resolve hash collisions. Note that a single fingerprint
534-
// can have multiple hints, in which case there will be duplicate entries in
535-
// 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.
536546
// TODO(drewk): consider de-duplicating the fingerprint strings to reduce
537547
// memory usage.
538-
fingerprints []string
539548
hints []StatementHint
549+
fingerprints []string
550+
ids []int64
540551
}
541552

542-
// getMatchingHints returns the plan hints for the given fingerprint, or nil if
543-
// they don't exist.
544-
func (entry *cacheEntry) getMatchingHints(statementFingerprint string) []StatementHint {
545-
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) {
546558
for i := range entry.hints {
547559
if entry.fingerprints[i] == statementFingerprint {
548-
// TODO(drewk,michae2): we could do something smarter here to avoid
549-
// allocations. For example, order by fingerprint in the original query
550-
// so that we can return a slice here.
551-
res = append(res, entry.hints[i])
560+
hints = append(hints, entry.hints[i])
561+
ids = append(ids, entry.ids[i])
552562
}
553563
}
554-
return res
564+
return hints, ids
555565
}
556566

557567
// ============================================================================

pkg/sql/hints/hint_cache_test.go

Lines changed: 52 additions & 27 deletions
Original file line numberDiff line numberDiff line change
@@ -59,7 +59,7 @@ func TestHintCacheBasic(t *testing.T) {
5959
// Query for hints on a statement that has no hints.
6060
nonHintedFingerprint := "SELECT x FROM t WHERE y = $1"
6161
require.False(t, hc.TestingHashHasHints(computeHash(t, nonHintedFingerprint)))
62-
require.Nil(t, hc.MaybeGetStatementHints(ctx, nonHintedFingerprint))
62+
requireHintsCount(t, hc, ctx, nonHintedFingerprint, 0)
6363

6464
// Add a hint for another statement.
6565
fingerprint2 := "SELECT c FROM t WHERE d = $1"
@@ -122,43 +122,43 @@ func TestHintCacheLRU(t *testing.T) {
122122

123123
// Access the first two fingerprints to populate the cache.
124124
// This should result in 2 table reads.
125-
require.NotNil(t, hc.MaybeGetStatementHints(ctx, fingerprints[0]))
125+
requireHintsCount(t, hc, ctx, fingerprints[0], 1)
126126
require.Equal(t, 1, hc.TestingNumTableReads())
127127

128-
require.NotNil(t, hc.MaybeGetStatementHints(ctx, fingerprints[1]))
128+
requireHintsCount(t, hc, ctx, fingerprints[1], 1)
129129
require.Equal(t, 2, hc.TestingNumTableReads())
130130

131131
// Access the same fingerprints again - should be served from cache with no
132132
// additional reads.
133-
require.NotNil(t, hc.MaybeGetStatementHints(ctx, fingerprints[0]))
134-
require.NotNil(t, hc.MaybeGetStatementHints(ctx, fingerprints[1]))
133+
requireHintsCount(t, hc, ctx, fingerprints[0], 1)
134+
requireHintsCount(t, hc, ctx, fingerprints[1], 1)
135135
require.Equal(t, 2, hc.TestingNumTableReads())
136136

137137
// Access the third fingerprint. This should evict the first (LRU) due to
138138
// cache size limit of 2, resulting in one more table read.
139-
require.NotNil(t, hc.MaybeGetStatementHints(ctx, fingerprints[2]))
139+
requireHintsCount(t, hc, ctx, fingerprints[2], 1)
140140
require.Equal(t, 3, hc.TestingNumTableReads())
141141

142142
// Access the first fingerprint again. Since it was evicted, this should
143143
// result in another table read on the first access.
144-
require.NotNil(t, hc.MaybeGetStatementHints(ctx, fingerprints[0]))
144+
requireHintsCount(t, hc, ctx, fingerprints[0], 1)
145145
require.Equal(t, 4, hc.TestingNumTableReads())
146146

147147
// Access the second fingerprint. It should have been evicted by now, so
148148
// another table read on the first access.
149-
require.NotNil(t, hc.MaybeGetStatementHints(ctx, fingerprints[1]))
149+
requireHintsCount(t, hc, ctx, fingerprints[1], 1)
150150
require.Equal(t, 5, hc.TestingNumTableReads())
151151

152152
// The first and second fingerprint should now be cached, so accessing them
153153
// again should not increase table reads.
154-
require.NotNil(t, hc.MaybeGetStatementHints(ctx, fingerprints[0]))
155-
require.NotNil(t, hc.MaybeGetStatementHints(ctx, fingerprints[1]))
154+
requireHintsCount(t, hc, ctx, fingerprints[0], 1)
155+
requireHintsCount(t, hc, ctx, fingerprints[1], 1)
156156
require.Equal(t, 5, hc.TestingNumTableReads())
157157

158158
// Access the third fingerprint again - should have been evicted, so the first
159159
// access should cause a table read.
160-
require.NotNil(t, hc.MaybeGetStatementHints(ctx, fingerprints[2]))
161-
require.NotNil(t, hc.MaybeGetStatementHints(ctx, fingerprints[2]))
160+
requireHintsCount(t, hc, ctx, fingerprints[2], 1)
161+
requireHintsCount(t, hc, ctx, fingerprints[2], 1)
162162
require.Equal(t, 6, hc.TestingNumTableReads())
163163
}
164164

@@ -204,15 +204,15 @@ func TestHintCacheInitialScan(t *testing.T) {
204204
require.True(t, hc.TestingHashHasHints(hashes[0]))
205205
require.True(t, hc.TestingHashHasHints(hashes[1]))
206206
require.True(t, hc.TestingHashHasHints(hashes[2]))
207-
require.Len(t, hc.MaybeGetStatementHints(ctx, fingerprints[0]), 2)
208-
require.Len(t, hc.MaybeGetStatementHints(ctx, fingerprints[1]), 1)
209-
require.Len(t, hc.MaybeGetStatementHints(ctx, fingerprints[2]), 1)
207+
requireHintsCount(t, hc, ctx, fingerprints[0], 2)
208+
requireHintsCount(t, hc, ctx, fingerprints[1], 1)
209+
requireHintsCount(t, hc, ctx, fingerprints[2], 1)
210210

211211
// Query for a fingerprint with no hints.
212212
nonHintedFingerprint := "SELECT x FROM t WHERE y = $1"
213213
nonHintedHash := computeHash(t, nonHintedFingerprint)
214214
require.False(t, hc.TestingHashHasHints(nonHintedHash))
215-
require.Nil(t, hc.MaybeGetStatementHints(ctx, nonHintedFingerprint))
215+
requireHintsCount(t, hc, ctx, nonHintedFingerprint, 0)
216216

217217
// After the initial scan, new hints should still be detected via rangefeed.
218218
fingerprint4 := "SELECT g FROM t WHERE h = $1"
@@ -274,8 +274,8 @@ func TestHintCacheMultiNode(t *testing.T) {
274274
require.Equal(t, 1, hc.TestingHashCount())
275275

276276
// Verify that fingerprint1 still has hints but fingerprint2 doesn't.
277-
require.Len(t, hc.MaybeGetStatementHints(ctx, fingerprint1), 1)
278-
require.Nil(t, hc.MaybeGetStatementHints(ctx, fingerprint2))
277+
requireHintsCount(t, hc, ctx, fingerprint1, 1)
278+
requireHintsCount(t, hc, ctx, fingerprint2, 0)
279279
}
280280

281281
// TestHintCacheMultiTenant tests that hint caches for different tenants are
@@ -317,24 +317,24 @@ func TestHintCacheMultiTenant(t *testing.T) {
317317
insertStatementHint(t, r1, fingerprint1)
318318
waitForUpdateOnFingerprintHash(t, ctx, hc1, fingerprint1, 1)
319319
require.Equal(t, 1, hc1.TestingHashCount())
320-
require.Len(t, hc1.MaybeGetStatementHints(ctx, fingerprint1), 1)
320+
requireHintsCount(t, hc1, ctx, fingerprint1, 1)
321321

322322
// Tenant2's cache should remain empty.
323323
require.Equal(t, 0, hc2.TestingHashCount())
324324
require.False(t, hc2.TestingHashHasHints(computeHash(t, fingerprint1)))
325-
require.Nil(t, hc2.MaybeGetStatementHints(ctx, fingerprint1))
325+
requireHintsCount(t, hc2, ctx, fingerprint1, 0)
326326

327327
// Insert a different hint for tenant2.
328328
fingerprint2 := "SELECT c FROM t WHERE d = $1"
329329
insertStatementHint(t, r2, fingerprint2)
330330
waitForUpdateOnFingerprintHash(t, ctx, hc2, fingerprint2, 1)
331331
require.Equal(t, 1, hc2.TestingHashCount())
332-
require.Len(t, hc2.MaybeGetStatementHints(ctx, fingerprint2), 1)
332+
requireHintsCount(t, hc2, ctx, fingerprint2, 1)
333333

334334
// Tenant1's cache should still only have its original hint.
335335
require.Equal(t, 1, hc1.TestingHashCount())
336336
require.False(t, hc1.TestingHashHasHints(computeHash(t, fingerprint2)))
337-
require.Nil(t, hc1.MaybeGetStatementHints(ctx, fingerprint2))
337+
requireHintsCount(t, hc1, ctx, fingerprint2, 0)
338338

339339
// Insert the same fingerprint in both tenants - they should be independent.
340340
fingerprintShared := "SELECT x FROM t WHERE y = $1"
@@ -352,8 +352,8 @@ func TestHintCacheMultiTenant(t *testing.T) {
352352
// Verify that the shared fingerprint has different counts in each tenant.
353353
require.True(t, hc1.TestingHashHasHints(computeHash(t, fingerprintShared)))
354354
require.True(t, hc2.TestingHashHasHints(computeHash(t, fingerprintShared)))
355-
require.Len(t, hc1.MaybeGetStatementHints(ctx, fingerprintShared), 2)
356-
require.Len(t, hc2.MaybeGetStatementHints(ctx, fingerprintShared), 1)
355+
requireHintsCount(t, hc1, ctx, fingerprintShared, 2)
356+
requireHintsCount(t, hc2, ctx, fingerprintShared, 1)
357357

358358
// Delete one hint from tenant1's shared fingerprint.
359359
deleteStatementHints(t, r1, fingerprintShared, 1)
@@ -370,8 +370,8 @@ func TestHintCacheMultiTenant(t *testing.T) {
370370
// Tenant1 should still have the hint, but tenant2 should not.
371371
require.True(t, hc1.TestingHashHasHints(computeHash(t, fingerprintShared)))
372372
require.False(t, hc2.TestingHashHasHints(computeHash(t, fingerprintShared)))
373-
require.Len(t, hc1.MaybeGetStatementHints(ctx, fingerprintShared), 1)
374-
require.Nil(t, hc2.MaybeGetStatementHints(ctx, fingerprintShared))
373+
requireHintsCount(t, hc1, ctx, fingerprintShared, 1)
374+
requireHintsCount(t, hc2, ctx, fingerprintShared, 0)
375375
}
376376

377377
// TestHintCacheGeneration tests that the cache generation is incremented
@@ -505,14 +505,39 @@ func waitForUpdateOnFingerprintHash(
505505
if hasHints := hc.TestingHashHasHints(hash); (expected > 0) != hasHints {
506506
return errors.Errorf("expected hash %d with hasHints=%t, got hasHints=%t", hash, expected > 0, hasHints)
507507
}
508-
hints := hc.MaybeGetStatementHints(ctx, fingerprint)
508+
hints, ids := hc.MaybeGetStatementHints(ctx, fingerprint)
509509
if len(hints) != expected {
510510
return errors.Errorf("expected %d hints for fingerprint %q, got %d", expected, fingerprint, len(hints))
511511
}
512+
checkIDOrder(t, ids)
512513
return nil
513514
})
514515
}
515516

517+
// requireHintsCount verifies that MaybeGetStatementHints returns the expected number of hints and IDs.
518+
func requireHintsCount(
519+
t *testing.T,
520+
hc *hints.StatementHintsCache,
521+
ctx context.Context,
522+
fingerprint string,
523+
expectedCount int,
524+
) {
525+
hints, ids := hc.MaybeGetStatementHints(ctx, fingerprint)
526+
require.Len(t, hints, expectedCount)
527+
require.Len(t, ids, expectedCount)
528+
checkIDOrder(t, ids)
529+
}
530+
531+
// checkIDOrder verifies that the row IDs are in ascending order.
532+
func checkIDOrder(t *testing.T, ids []int64) {
533+
t.Helper()
534+
for i := 1; i < len(ids); i++ {
535+
if ids[i] <= ids[i-1] {
536+
t.Fatalf("expected IDs to be in ascending order, got %v", ids)
537+
}
538+
}
539+
}
540+
516541
// insertStatementHint inserts an empty statement hint into the
517542
// system.statement_hints table.
518543
func insertStatementHint(t *testing.T, r *sqlutils.SQLRunner, fingerprint string) {

0 commit comments

Comments
 (0)