Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
8 changes: 0 additions & 8 deletions pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -901,10 +901,6 @@ func (ex *connExecutor) execStmtInOpenState(
typeHints[i] = resolved
}
}
var statementHintsCache *hints.StatementHintsCache
if ex.executorType != executorTypeInternal {
statementHintsCache = ex.server.cfg.StatementHintsCache
}
prepStmt := makeStatement(
ctx,
statements.Statement[tree.Statement]{
Expand Down Expand Up @@ -1883,10 +1879,6 @@ func (ex *connExecutor) execStmtInOpenStateWithPausablePortal(
typeHints[i] = resolved
}
}
var statementHintsCache *hints.StatementHintsCache
if ex.executorType != executorTypeInternal {
statementHintsCache = ex.server.cfg.StatementHintsCache
}
prepStmt := makeStatement(
ctx,
statements.Statement[tree.Statement]{
Expand Down
5 changes: 3 additions & 2 deletions pkg/sql/hints/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -58,17 +58,18 @@ go_test(
deps = [
":hints",
"//pkg/base",
"//pkg/kv/kvclient/rangefeed",
"//pkg/security/securityassets",
"//pkg/security/securitytest",
"//pkg/server",
"//pkg/sql/catalog",
"//pkg/sql",
"//pkg/sql/catalog/descs",
"//pkg/sql/hintpb",
"//pkg/sql/isql",
"//pkg/sql/randgen",
"//pkg/sql/stats",
"//pkg/storage/fs",
"//pkg/testutils",
"//pkg/testutils/listenerutil",
"//pkg/testutils/serverutils",
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
Expand Down
9 changes: 5 additions & 4 deletions pkg/sql/hints/hint_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -109,8 +109,8 @@ var cacheSize = settings.RegisterIntSetting(
"number of hint entries to store in the LRU",
metamorphic.ConstantWithTestChoice[int64](
"sql.hints.statement_hints_cache_size",
1024, /* defaultValue */
1, 2, 3, 8, 128, 4096, /* otherValues */
1024, /* defaultValue */
0, 1, 2, 3, 8, 128, 4096, /* otherValues */
),
settings.NonNegativeInt,
)
Expand Down Expand Up @@ -322,8 +322,9 @@ func (c *StatementHintsCache) checkHashHasHintsAsync(
c.mu.Lock()
defer c.mu.Unlock()
if refreshTS.Forward(c.mu.hintedHashes[hash]) {
// The refresh timestamp was bumped by a rangefeed event. Retry at the
// new timestamp.
// The refresh timestamp was bumped by a rangefeed event.
// Retry at the new timestamp (refreshTS has been updated in
// place).
return false
}
if hasHints {
Expand Down
96 changes: 51 additions & 45 deletions pkg/sql/hints/hint_cache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,12 +12,13 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/hintpb"
"github.com/cockroachdb/cockroach/pkg/sql/hints"
"github.com/cockroachdb/cockroach/pkg/storage/fs"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/listenerutil"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
Expand All @@ -41,8 +42,7 @@ func TestHintCacheBasic(t *testing.T) {
r := sqlutils.MakeSQLRunner(db)
setTestDefaults(t, srv)

// Create a hints cache.
hc := createHintsCache(t, ctx, ts)
hc := ts.ExecutorConfig().(sql.ExecutorConfig).StatementHintsCache
require.Equal(t, 0, hc.TestingHashCount())

// Insert a hint for a statement. The cache should soon contain the hash.
Expand Down Expand Up @@ -96,7 +96,7 @@ func TestHintCacheLRU(t *testing.T) {
// Set cache size to 2 for testing eviction.
r.Exec(t, "SET CLUSTER SETTING sql.hints.statement_hints_cache_size = 2")

hc := createHintsCache(t, ctx, ts)
hc := ts.ExecutorConfig().(sql.ExecutorConfig).StatementHintsCache
require.Equal(t, 0, hc.TestingHashCount())

// Create test data: 3 different fingerprints.
Expand All @@ -121,46 +121,51 @@ func TestHintCacheLRU(t *testing.T) {
return nil
})

// Before the initial scan is complete, all queries unconditionally check
// the hint cache and might perform DB reads, so we need to ignore all reads
// that happened already.
ignoredReads := hc.TestingNumTableReads()

// Access the first two fingerprints to populate the cache.
// This should result in 2 table reads.
requireHintsCount(t, hc, ctx, fingerprints[0], 1)
require.Equal(t, 1, hc.TestingNumTableReads())
require.Equal(t, ignoredReads+1, hc.TestingNumTableReads())

requireHintsCount(t, hc, ctx, fingerprints[1], 1)
require.Equal(t, 2, hc.TestingNumTableReads())
require.Equal(t, ignoredReads+2, hc.TestingNumTableReads())

// Access the same fingerprints again - should be served from cache with no
// additional reads.
requireHintsCount(t, hc, ctx, fingerprints[0], 1)
requireHintsCount(t, hc, ctx, fingerprints[1], 1)
require.Equal(t, 2, hc.TestingNumTableReads())
require.Equal(t, ignoredReads+2, hc.TestingNumTableReads())

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

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

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

// The first and second fingerprint should now be cached, so accessing them
// again should not increase table reads.
requireHintsCount(t, hc, ctx, fingerprints[0], 1)
requireHintsCount(t, hc, ctx, fingerprints[1], 1)
require.Equal(t, 5, hc.TestingNumTableReads())
require.Equal(t, ignoredReads+5, hc.TestingNumTableReads())

// Access the third fingerprint again - should have been evicted, so the first
// access should cause a table read.
requireHintsCount(t, hc, ctx, fingerprints[2], 1)
requireHintsCount(t, hc, ctx, fingerprints[2], 1)
require.Equal(t, 6, hc.TestingNumTableReads())
require.Equal(t, ignoredReads+6, hc.TestingNumTableReads())
}

// TestHintCacheInitialScan tests that a new cache correctly populates from
Expand All @@ -169,14 +174,29 @@ func TestHintCacheInitialScan(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

listenerReg := listenerutil.NewListenerRegistry()
defer listenerReg.Close()
stickyVFSRegistry := fs.NewStickyRegistry()

ctx := context.Background()
srv, db, _ := serverutils.StartServer(t, base.TestServerArgs{})
defer srv.Stopper().Stop(ctx)
ts := srv.ApplicationLayer()
r := sqlutils.MakeSQLRunner(db)
setTestDefaults(t, srv)
tc := serverutils.StartCluster(t, 1, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
Knobs: base.TestingKnobs{
Server: &server.TestingKnobs{
// Sticky vfs is needed for cluster restart.
StickyVFSRegistry: stickyVFSRegistry,
},
},
},
// A listener is required for cluster restart.
ReusableListenerReg: listenerReg,
})
defer tc.Stopper().Stop(ctx)
ts := tc.ApplicationLayer(0)
r := sqlutils.MakeSQLRunner(ts.SQLConn(t))
setTestDefaults(t, tc.Server(0))

// Insert multiple hints into the system table BEFORE creating the cache.
// Insert multiple hints into the system table.
fingerprints := []string{
"SELECT a FROM t WHERE b = $1",
"SELECT c FROM t WHERE d = $1",
Expand All @@ -191,9 +211,12 @@ func TestHintCacheInitialScan(t *testing.T) {
// Insert multiple hints for the first fingerprint.
insertStatementHint(t, r, fingerprints[0])

// Now create a hints cache - it should populate from existing hints during
// initial scan.
hc := createHintsCache(t, ctx, ts)
// Restart the cluster to trigger the initial scan for the hints cache.
require.NoError(t, tc.Restart())
ts = tc.ApplicationLayer(0)
r = sqlutils.MakeSQLRunner(ts.SQLConn(t))

hc := ts.ExecutorConfig().(sql.ExecutorConfig).StatementHintsCache

// The cache should have all the hashes once the initial scan completes.
testutils.SucceedsSoon(t, func() error {
Expand Down Expand Up @@ -246,8 +269,8 @@ func TestHintCacheMultiNode(t *testing.T) {
r2 := sqlutils.MakeSQLRunner(tc.ServerConn(2))
setTestDefaults(t, tc.Server(0))

// Create a hints cache on node 0.
hc := createHintsCache(t, ctx, ts)
// Use the hints cache from node 0.
hc := ts.ExecutorConfig().(sql.ExecutorConfig).StatementHintsCache
require.Equal(t, 0, hc.TestingHashCount())

// Insert hints from node 1.
Expand Down Expand Up @@ -307,9 +330,8 @@ func TestHintCacheMultiTenant(t *testing.T) {
r1 := sqlutils.MakeSQLRunner(tenantConn1)
r2 := sqlutils.MakeSQLRunner(tenantConn2)

// Create hints caches for both tenants.
hc1 := createHintsCache(t, ctx, tenant1)
hc2 := createHintsCache(t, ctx, tenant2)
hc1 := tenant1.ExecutorConfig().(sql.ExecutorConfig).StatementHintsCache
hc2 := tenant2.ExecutorConfig().(sql.ExecutorConfig).StatementHintsCache
require.Equal(t, 0, hc1.TestingHashCount())
require.Equal(t, 0, hc2.TestingHashCount())

Expand Down Expand Up @@ -390,8 +412,7 @@ func TestHintCacheGeneration(t *testing.T) {
r := sqlutils.MakeSQLRunner(db)
setTestDefaults(t, srv)

// Create a hints cache.
hc := createHintsCache(t, ctx, ts)
hc := ts.ExecutorConfig().(sql.ExecutorConfig).StatementHintsCache

// Helper that retrieves the generation and verifies that it doesn't change
// over a short period.
Expand Down Expand Up @@ -475,21 +496,6 @@ func setTestDefaults(t *testing.T, srv serverutils.TestServerInterface) {
r.Exec(t, "SET CLUSTER SETTING kv.rangefeed.closed_timestamp_refresh_interval = '10ms'")
}

func createHintsCache(
t *testing.T, ctx context.Context, ts serverutils.ApplicationLayerInterface,
) *hints.StatementHintsCache {
hc := hints.NewStatementHintsCache(
ts.Clock(),
ts.RangeFeedFactory().(*rangefeed.Factory),
ts.AppStopper(),
ts.Codec(),
ts.InternalDB().(descs.DB),
ts.ClusterSettings(),
)
require.NoError(t, hc.Start(ctx, ts.SystemTableIDResolver().(catalog.SystemTableIDResolver)))
return hc
}

// waitForUpdateOnFingerprintHash waits for the cache to automatically refresh
// to reflect a hint insertion or deletion.
func waitForUpdateOnFingerprintHash(
Expand Down