diff --git a/pkg/base/config.go b/pkg/base/config.go index dd7a759d0c1b..db6c11f58fec 100644 --- a/pkg/base/config.go +++ b/pkg/base/config.go @@ -527,6 +527,9 @@ type StorageConfig struct { // UseFileRegistry is true if the file registry is needed (eg: encryption-at-rest). // This may force the store version to versionFileRegistry if currently lower. UseFileRegistry bool + // DisableSeparatedIntents is true if separated intents should not be written + // by intent writers. Only true for tests. + DisableSeparatedIntents bool // EncryptionOptions is a serialized protobuf set by Go CCL code and passed // through to C CCL code to set up encryption-at-rest. Must be set if and // only if encryption is enabled, otherwise left empty. diff --git a/pkg/kv/kvserver/testing_knobs.go b/pkg/kv/kvserver/testing_knobs.go index 0470d0831854..b5e68f91369f 100644 --- a/pkg/kv/kvserver/testing_knobs.go +++ b/pkg/kv/kvserver/testing_knobs.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/tenantrate" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/txnwait" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/syncutil" ) @@ -35,6 +36,7 @@ type StoreTestingKnobs struct { TxnWaitKnobs txnwait.TestingKnobs ConsistencyTestingKnobs ConsistencyTestingKnobs TenantRateKnobs tenantrate.TestingKnobs + StorageKnobs storage.TestingKnobs // TestingRequestFilter is called before evaluating each request on a // replica. The filter is run before the request acquires latches, so diff --git a/pkg/migration/migrations/separated_intents_external_test.go b/pkg/migration/migrations/separated_intents_external_test.go index 7e612f64b836..b67c86cda47c 100644 --- a/pkg/migration/migrations/separated_intents_external_test.go +++ b/pkg/migration/migrations/separated_intents_external_test.go @@ -12,7 +12,8 @@ package migrations_test import ( "context" - "strconv" + "fmt" + "path" "testing" "time" @@ -38,29 +39,31 @@ func TestSeparatedIntentsMigration(t *testing.T) { defer cancel() settings := cluster.MakeTestingClusterSettingsWithVersions( clusterversion.ByKey(clusterversion.SeparatedIntentsMigration), - clusterversion.ByKey(clusterversion.SeparatedIntentsMigration - 1), + clusterversion.ByKey(clusterversion.SeparatedIntentsMigration-1), false, /* initializeVersion */ ) - storage.SeparatedIntentsEnabled.Override(ctx, &settings.SV, false) - stickyEngineRegistry := server.NewStickyInMemEnginesRegistry() - defer stickyEngineRegistry.CloseAllStickyInMemEngines() const numServers int = 5 stickyServerArgs := make(map[int]base.TestServerArgs) + storeKnobs := &kvserver.StoreTestingKnobs{ + StorageKnobs: storage.TestingKnobs{DisableSeparatedIntents: true}, + } + tempDir, cleanup := testutils.TempDir(t) + defer cleanup() for i := 0; i < numServers; i++ { stickyServerArgs[i] = base.TestServerArgs{ Settings: settings, StoreSpecs: []base.StoreSpec{ { - InMemory: true, - StickyInMemoryEngineID: strconv.FormatInt(int64(i), 10), + InMemory: false, + Path: path.Join(tempDir, fmt.Sprintf("engine-%d", i)), }, }, Knobs: base.TestingKnobs{ Server: &server.TestingKnobs{ DisableAutomaticVersionUpgrade: 1, - BinaryVersionOverride: clusterversion.ByKey(clusterversion.SeparatedIntentsMigration-1), - StickyEngineRegistry: stickyEngineRegistry, + BinaryVersionOverride: clusterversion.ByKey(clusterversion.SeparatedIntentsMigration - 1), }, + Store: storeKnobs, }, } } @@ -134,7 +137,8 @@ func TestSeparatedIntentsMigration(t *testing.T) { } require.Greater(t, interleavedIntentCount, 0) - storage.SeparatedIntentsEnabled.Override(ctx, &settings.SV, true) + // Start writing separated intents. + storeKnobs.StorageKnobs.DisableSeparatedIntents = false require.NoError(t, tc.Restart()) time.Sleep(10 * time.Second) require.NoError(t, tc.WaitForFullReplication()) diff --git a/pkg/server/config.go b/pkg/server/config.go index aa05ab1e43da..9d08edcfac2e 100644 --- a/pkg/server/config.go +++ b/pkg/server/config.go @@ -476,6 +476,8 @@ func (cfg *Config) CreateEngines(ctx context.Context) (Engines, error) { skipSizeCheck := cfg.TestingKnobs.Store != nil && cfg.TestingKnobs.Store.(*kvserver.StoreTestingKnobs).SkipMinSizeCheck + disableSeparatedIntents := cfg.TestingKnobs.Store != nil && + cfg.TestingKnobs.Store.(*kvserver.StoreTestingKnobs).StorageKnobs.DisableSeparatedIntents for i, spec := range cfg.Stores.Specs { log.Eventf(ctx, "initializing %+v", spec) var sizeInBytes = spec.Size.InBytes @@ -530,12 +532,13 @@ func (cfg *Config) CreateEngines(ctx context.Context) (Engines, error) { i, humanizeutil.IBytes(sizeInBytes), openFileLimitPerStore)) storageConfig := base.StorageConfig{ - Attrs: spec.Attributes, - Dir: spec.Path, - MaxSize: sizeInBytes, - Settings: cfg.Settings, - UseFileRegistry: spec.UseFileRegistry, - EncryptionOptions: spec.EncryptionOptions, + Attrs: spec.Attributes, + Dir: spec.Path, + MaxSize: sizeInBytes, + Settings: cfg.Settings, + UseFileRegistry: spec.UseFileRegistry, + DisableSeparatedIntents: disableSeparatedIntents, + EncryptionOptions: spec.EncryptionOptions, } pebbleConfig := storage.PebbleConfig{ StorageConfig: storageConfig, diff --git a/pkg/server/settingswatcher/BUILD.bazel b/pkg/server/settingswatcher/BUILD.bazel index aa0675d92085..0d13707799c6 100644 --- a/pkg/server/settingswatcher/BUILD.bazel +++ b/pkg/server/settingswatcher/BUILD.bazel @@ -50,7 +50,6 @@ go_test( "//pkg/settings", "//pkg/settings/cluster", "//pkg/sql", - "//pkg/storage", "//pkg/testutils", "//pkg/testutils/serverutils", "//pkg/testutils/sqlutils", diff --git a/pkg/server/settingswatcher/settings_watcher_external_test.go b/pkg/server/settingswatcher/settings_watcher_external_test.go index f223ce3de73d..65a903bf1503 100644 --- a/pkg/server/settingswatcher/settings_watcher_external_test.go +++ b/pkg/server/settingswatcher/settings_watcher_external_test.go @@ -21,7 +21,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" - "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" @@ -93,9 +92,6 @@ func TestSettingWatcher(t *testing.T) { s0.ExecutorConfig().(sql.ExecutorConfig).RangeFeedFactory, tc.Stopper()) require.NoError(t, sw.Start(ctx)) - // TestCluster randomizes the value of SeparatedIntentsEnabled, so set it to - // the same as in fakeSettings for the subsequent equality check. - storage.SeparatedIntentsEnabled.Override(ctx, &s0.ClusterSettings().SV, storage.SeparatedIntentsEnabled.Get(&fakeSettings.SV)) require.NoError(t, checkSettingsValuesMatch(s0.ClusterSettings(), fakeSettings)) for k, v := range toSet { tdb.Exec(t, "SET CLUSTER SETTING "+k+" = $1", v[1]) diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index e4677785fce9..f71999c63ae3 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -15,7 +15,6 @@ import ( "context" "encoding/base64" "fmt" - "math/rand" "net" "net/http" "net/http/cookiejar" @@ -129,10 +128,6 @@ func makeTestConfigFromParams(params base.TestServerArgs) Config { st := params.Settings if params.Settings == nil { st = cluster.MakeClusterSettings() - enabledSeparated := rand.Intn(2) == 0 - log.Infof(context.Background(), - "test Config is randomly setting enabledSeparated: %t", enabledSeparated) - storage.SeparatedIntentsEnabled.Override(context.Background(), &st.SV, enabledSeparated) } st.ExternalIODir = params.ExternalIODir cfg := makeTestConfig(st) diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index b01a8ab14f0f..888ea60f5317 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -35,6 +35,7 @@ go_library( "stacks.go", "temp_dir.go", "temp_engine.go", + "testing_knobs.go", ], # List out all the c-dependencies pkg/storage depends on. cdeps = [ diff --git a/pkg/storage/batch_test.go b/pkg/storage/batch_test.go index 1af9d1c1807a..fe152d7e8d1a 100644 --- a/pkg/storage/batch_test.go +++ b/pkg/storage/batch_test.go @@ -1106,6 +1106,7 @@ func TestDecodeKey(t *testing.T) { 1<<20, /* cacheSize */ 512<<20, /* storeSize */ nil, /* settings */ + nil, /* knobs */ ) defer e.Close() diff --git a/pkg/storage/bench_pebble_test.go b/pkg/storage/bench_pebble_test.go index e900470a187d..048431239441 100644 --- a/pkg/storage/bench_pebble_test.go +++ b/pkg/storage/bench_pebble_test.go @@ -42,7 +42,7 @@ func setupMVCCPebble(b testing.TB, dir string) Engine { StorageConfig: base.StorageConfig{ Dir: dir, Settings: makeSettingsForSeparatedIntents( - false /* oldClusterVersion */, true /* enabled */), + false /* oldClusterVersion */), }, Opts: opts, }) @@ -54,10 +54,12 @@ func setupMVCCPebble(b testing.TB, dir string) Engine { func setupMVCCInMemPebble(b testing.TB, loc string) Engine { return setupMVCCInMemPebbleWithSettings(b, makeSettingsForSeparatedIntents( - false /* oldClusterVersion */, true /* enabled */)) + false /* oldClusterVersion */), false /* disabledSeparatedIntents */) } -func setupMVCCInMemPebbleWithSettings(b testing.TB, settings *cluster.Settings) Engine { +func setupMVCCInMemPebbleWithSettings( + b testing.TB, settings *cluster.Settings, disableSeparatedIntents bool, +) Engine { opts := DefaultPebbleOptions() opts.FS = vfs.NewMem() opts.Cache = pebble.NewCache(testCacheSize) @@ -68,7 +70,8 @@ func setupMVCCInMemPebbleWithSettings(b testing.TB, settings *cluster.Settings) PebbleConfig{ Opts: opts, StorageConfig: base.StorageConfig{ - Settings: settings, + Settings: settings, + DisableSeparatedIntents: disableSeparatedIntents, }, }) if err != nil { diff --git a/pkg/storage/bench_test.go b/pkg/storage/bench_test.go index 1db285c3a0d1..6b408898f051 100644 --- a/pkg/storage/bench_test.go +++ b/pkg/storage/bench_test.go @@ -214,7 +214,7 @@ func BenchmarkIntentScan(b *testing.B) { for _, percentFlushed := range []int{0, 50, 80, 90, 100} { b.Run(fmt.Sprintf("percent-flushed=%d", percentFlushed), func(b *testing.B) { eng := setupMVCCInMemPebbleWithSettings( - b, makeSettingsForSeparatedIntents(false, sep)) + b, makeSettingsForSeparatedIntents(false), sep) numFlushedVersions := (percentFlushed * numVersions) / 100 setupKeysWithIntent(b, eng, numVersions, numFlushedVersions, false /* resolveAll */) lower := makeKey(nil, 0) @@ -278,7 +278,7 @@ func BenchmarkScanAllIntentsResolved(b *testing.B) { for _, percentFlushed := range []int{0, 50, 90, 100} { b.Run(fmt.Sprintf("percent-flushed=%d", percentFlushed), func(b *testing.B) { eng := setupMVCCInMemPebbleWithSettings( - b, makeSettingsForSeparatedIntents(false, sep)) + b, makeSettingsForSeparatedIntents(false), sep) numFlushedVersions := (percentFlushed * numVersions) / 100 setupKeysWithIntent(b, eng, numVersions, numFlushedVersions, true /* resolveAll */) lower := makeKey(nil, 0) @@ -349,7 +349,7 @@ func BenchmarkScanOneAllIntentsResolved(b *testing.B) { for _, percentFlushed := range []int{0, 50, 90, 100} { b.Run(fmt.Sprintf("percent-flushed=%d", percentFlushed), func(b *testing.B) { eng := setupMVCCInMemPebbleWithSettings( - b, makeSettingsForSeparatedIntents(false, sep)) + b, makeSettingsForSeparatedIntents(false), sep) numFlushedVersions := (percentFlushed * numVersions) / 100 setupKeysWithIntent(b, eng, numVersions, numFlushedVersions, true /* resolveAll */) lower := makeKey(nil, 0) @@ -401,7 +401,7 @@ func BenchmarkIntentResolution(b *testing.B) { for _, percentFlushed := range []int{0, 50, 80, 90, 100} { b.Run(fmt.Sprintf("percent-flushed=%d", percentFlushed), func(b *testing.B) { eng := setupMVCCInMemPebbleWithSettings( - b, makeSettingsForSeparatedIntents(false, sep)) + b, makeSettingsForSeparatedIntents(false), sep) numFlushedVersions := (percentFlushed * numVersions) / 100 lockUpdate := setupKeysWithIntent(b, eng, numVersions, numFlushedVersions, false /* resolveAll */) keys := make([]roachpb.Key, numIntentKeys) @@ -445,7 +445,7 @@ func BenchmarkIntentRangeResolution(b *testing.B) { for _, percentFlushed := range []int{0, 50, 80, 90, 100} { b.Run(fmt.Sprintf("percent-flushed=%d", percentFlushed), func(b *testing.B) { eng := setupMVCCInMemPebbleWithSettings( - b, makeSettingsForSeparatedIntents(false, sep)) + b, makeSettingsForSeparatedIntents(false), sep) numFlushedVersions := (percentFlushed * numVersions) / 100 lockUpdate := setupKeysWithIntent(b, eng, numVersions, numFlushedVersions, false /* resolveAll */) keys := make([]roachpb.Key, numIntentKeys+1) diff --git a/pkg/storage/engine_test.go b/pkg/storage/engine_test.go index 7979b5e90b21..5a5b5673308d 100644 --- a/pkg/storage/engine_test.go +++ b/pkg/storage/engine_test.go @@ -1542,6 +1542,7 @@ func TestSupportsPrev(t *testing.T) { 1<<20, /* cacheSize */ 512<<20, /* storeSize */ nil, /* settings */ + nil, /* knobs */ ) defer eng.Close() runTest(t, eng, engineTest{ @@ -1665,13 +1666,15 @@ func TestScanSeparatedIntents(t *testing.T) { for name, enableSeparatedIntents := range map[string]bool{"interleaved": false, "separated": true} { t.Run(name, func(t *testing.T) { - settings := makeSettingsForSeparatedIntents(false, enableSeparatedIntents) + settings := makeSettingsForSeparatedIntents(false) + knobs := &TestingKnobs{DisableSeparatedIntents: !enableSeparatedIntents} eng := newPebbleInMem( ctx, roachpb.Attributes{}, 1<<20, /* cacheSize */ 512<<20, /* storeSize */ settings, + knobs, ) defer eng.Close() diff --git a/pkg/storage/in_mem.go b/pkg/storage/in_mem.go index aa04e246036c..e7672b9d84c1 100644 --- a/pkg/storage/in_mem.go +++ b/pkg/storage/in_mem.go @@ -30,7 +30,7 @@ func NewInMem( cacheSize, storeSize int64, settings *cluster.Settings, ) Engine { - return newPebbleInMem(ctx, attrs, cacheSize, storeSize, settings) + return newPebbleInMem(ctx, attrs, cacheSize, storeSize, settings, nil /* knobs */) } // The ForTesting functions randomize the settings for separated intents. This @@ -48,7 +48,12 @@ func NewInMem( // must call the engine's Close method when the engine is no longer needed. func NewInMemForTesting(ctx context.Context, attrs roachpb.Attributes, storeSize int64) Engine { settings := MakeRandomSettingsForSeparatedIntents() - return newPebbleInMem(ctx, attrs, 0 /* cacheSize */, storeSize, settings) + disableSeparatedIntents := rand.Intn(2) == 0 + knobs := &TestingKnobs{DisableSeparatedIntents: disableSeparatedIntents} + log.Infof(context.Background(), + "engine creation is randomly setting disableSeparatedIntents: %t", + disableSeparatedIntents) + return newPebbleInMem(ctx, attrs, 0 /* cacheSize */, storeSize, settings, knobs /* knobs */) } // NewDefaultInMemForTesting allocates and returns a new, opened in-memory engine with @@ -59,23 +64,21 @@ func NewDefaultInMemForTesting() Engine { } // MakeRandomSettingsForSeparatedIntents makes settings for which it randomly -// picks whether the cluster understands separated intents, and if yes, -// whether to write separated intents. Once made, these setting do not change. +// picks whether the cluster understands and writes separated intents. +// Once made, these setting do not change. func MakeRandomSettingsForSeparatedIntents() *cluster.Settings { oldClusterVersion := rand.Intn(2) == 0 - enabledSeparated := rand.Intn(2) == 0 log.Infof(context.Background(), - "engine creation is randomly setting oldClusterVersion: %t, enabledSeparated: %t", - oldClusterVersion, enabledSeparated) - return makeSettingsForSeparatedIntents(oldClusterVersion, enabledSeparated) + "engine creation is randomly setting oldClusterVersion: %t", + oldClusterVersion) + return makeSettingsForSeparatedIntents(oldClusterVersion) } -func makeSettingsForSeparatedIntents(oldClusterVersion bool, enabled bool) *cluster.Settings { +func makeSettingsForSeparatedIntents(oldClusterVersion bool) *cluster.Settings { version := clusterversion.ByKey(clusterversion.SeparatedIntents) if oldClusterVersion { version = clusterversion.ByKey(clusterversion.V20_2) } settings := cluster.MakeTestingClusterSettingsWithVersions(version, version, true) - SeparatedIntentsEnabled.Override(context.TODO(), &settings.SV, enabled) return settings } diff --git a/pkg/storage/intent_reader_writer.go b/pkg/storage/intent_reader_writer.go index e1c03f514ade..3aa67e520a97 100644 --- a/pkg/storage/intent_reader_writer.go +++ b/pkg/storage/intent_reader_writer.go @@ -18,40 +18,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" ) -// SeparatedIntentsEnabled controls whether separated intents are written. A -// true setting is also gated on clusterversion.SeparatedIntents. After all -// nodes in a cluster are at or beyond clusterversion.SeparatedIntents, -// different nodes will see the version state transition at different times. -// Even nodes that have not yet seen the transition need to be able to read -// separated intents and to write over separated intents (due to a lease -// transfer from a node that has seen the transition to one that has not). -// Therefore, the clusterversion and the value of this setting do not affect -// whether intentDemuxWriter or intentInterleavingReader are used. They only -// affect whether intentDemuxWriter will write separated intents. As expected, -// this value can be set to false to disable writing of separated intents. -// -// Currently there is no long-running migration to replace all interleaved -// intents with separated intents, but we expect that when a cluster has been -// running with this flag set to true for some time, most ranges will only -// have separated intents. Similarly, setting this to false will gradually -// cause most ranges to only have interleaved intents. -// -// TODO(bilal): Remove this setting and replace it with a testing knob, as we -// no longer expect this setting to be toggled to false outside of tests. -var SeparatedIntentsEnabled = settings.RegisterBoolSetting( - "storage.transaction.separated_intents.enabled", - "if enabled, intents will be written to a separate lock table, instead of being "+ - "interleaved with MVCC values", - true, -) - // This file defines wrappers for Reader and Writer, and functions to do the // wrapping, which depend on the configuration settings above. @@ -68,11 +40,15 @@ type intentDemuxWriter struct { cachedSettingsAreValid bool clusterVersionIsRecentEnoughCached bool - writeSeparatedIntentsCached bool + disableSeparatedIntents bool } func wrapIntentWriter( - ctx context.Context, w Writer, settings *cluster.Settings, isLongLived bool, + ctx context.Context, + w Writer, + settings *cluster.Settings, + isLongLived bool, + disableSeparatedIntents bool, ) intentDemuxWriter { idw := intentDemuxWriter{w: w, settings: settings} if !isLongLived && settings != nil { @@ -81,10 +57,8 @@ func wrapIntentWriter( // Be resilient to the version not yet being initialized. idw.clusterVersionIsRecentEnoughCached = !idw.settings.Version.ActiveVersionOrEmpty(ctx).Less( clusterversion.ByKey(clusterversion.SeparatedIntents)) - - idw.writeSeparatedIntentsCached = - SeparatedIntentsEnabled.Get(&idw.settings.SV) } + idw.disableSeparatedIntents = disableSeparatedIntents return idw } @@ -138,7 +112,7 @@ func (idw intentDemuxWriter) PutIntent( var writeSeparatedIntents bool if idw.cachedSettingsAreValid { // Fast-path - writeSeparatedIntents = idw.clusterVersionIsRecentEnoughCached && idw.writeSeparatedIntentsCached + writeSeparatedIntents = idw.clusterVersionIsRecentEnoughCached && !idw.disableSeparatedIntents } else { // Slow-path, when doing writes on the Engine directly. This should not be // performance sensitive code. @@ -146,7 +120,7 @@ func (idw intentDemuxWriter) PutIntent( // Be resilient to the version not yet being initialized. !idw.settings.Version.ActiveVersionOrEmpty(ctx).Less( clusterversion.ByKey(clusterversion.SeparatedIntents)) && - SeparatedIntentsEnabled.Get(&idw.settings.SV) + !idw.disableSeparatedIntents } var engineKey EngineKey if state == ExistingIntentSeparated || writeSeparatedIntents { diff --git a/pkg/storage/intent_reader_writer_test.go b/pkg/storage/intent_reader_writer_test.go index 620a739057dd..acd0dd22fda6 100644 --- a/pkg/storage/intent_reader_writer_test.go +++ b/pkg/storage/intent_reader_writer_test.go @@ -216,8 +216,8 @@ func TestIntentDemuxWriter(t *testing.T) { // doesn't matter how the original call to createTestPebbleEngine // behaved in terms of separated intents config. w = wrapIntentWriter(context.Background(), &pw, - makeSettingsForSeparatedIntents(false /* oldClusterVersion */, separated), - false /* isLongLived */) + makeSettingsForSeparatedIntents(false /* oldClusterVersion */), + false /* isLongLived */, !separated) return "" case "put-intent": pw.reset() diff --git a/pkg/storage/multi_iterator_test.go b/pkg/storage/multi_iterator_test.go index 84d7ce03c448..12849e35964f 100644 --- a/pkg/storage/multi_iterator_test.go +++ b/pkg/storage/multi_iterator_test.go @@ -33,6 +33,7 @@ func TestMultiIterator(t *testing.T) { 1<<20, /* cacheSize */ 512<<20, /* storeSize */ nil, /* settings */ + nil, /* knobs */ ) defer pebble.Close() diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index 4093dc194d0b..7648b053a15a 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -117,9 +117,17 @@ func TestMVCCHistories(t *testing.T) { "randomly setting oldClusterVersion: %t, enableSeparated: %t", oldClusterVersion, enabledSeparated) } - settings := makeSettingsForSeparatedIntents(oldClusterVersion, enabledSeparated) + settings := makeSettingsForSeparatedIntents(oldClusterVersion) // We start from a clean slate in every test file. - engine := createTestPebbleEngineWithSettings(settings) + knobs := &TestingKnobs{DisableSeparatedIntents: !enabledSeparated} + engine := newPebbleInMem( + context.Background(), + roachpb.Attributes{}, + 1<<20, /* cacheSize */ + 512<<20, /* storeSize */ + settings, + knobs, /* knobs */ + ) defer engine.Close() reportDataEntries := func(buf *bytes.Buffer) error { diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index 224ad408d934..8ecb45b424b4 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -91,6 +91,7 @@ func createTestPebbleEngineWithSettings(settings *cluster.Settings) Engine { 1<<20, /* cacheSize */ 512<<20, /* storeSize */ settings, + nil, /* knobs */ ) } diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index a925900d8a63..4df7479bdd83 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -454,6 +454,9 @@ type Pebble struct { // GetStats. Updated and retrieved atomically. diskSlowCount, diskStallCount uint64 + // Copied from testing knobs. + disableSeparatedIntents bool + // Relevant options copied over from pebble.Options. fs vfs.FS logger pebble.Logger @@ -546,19 +549,21 @@ func NewPebble(ctx context.Context, cfg PebbleConfig) (*Pebble, error) { depth: 2, // skip over the EventListener stack frame }) p := &Pebble{ - path: cfg.Dir, - auxDir: auxDir, - maxSize: cfg.MaxSize, - attrs: cfg.Attrs, - settings: cfg.Settings, - statsHandler: statsHandler, - fileRegistry: fileRegistry, - fs: cfg.Opts.FS, - logger: cfg.Opts.Logger, + path: cfg.Dir, + auxDir: auxDir, + maxSize: cfg.MaxSize, + attrs: cfg.Attrs, + settings: cfg.Settings, + statsHandler: statsHandler, + fileRegistry: fileRegistry, + fs: cfg.Opts.FS, + logger: cfg.Opts.Logger, + disableSeparatedIntents: cfg.DisableSeparatedIntents, } p.connectEventMetrics(ctx, &cfg.Opts.EventListener) p.eventListener = &cfg.Opts.EventListener - p.wrappedIntentWriter = wrapIntentWriter(ctx, p, cfg.Settings, true /* isLongLived */) + p.wrappedIntentWriter = wrapIntentWriter( + ctx, p, cfg.Settings, true /* isLongLived */, cfg.DisableSeparatedIntents) db, err := pebble.Open(cfg.StorageConfig.Dir, cfg.Opts) if err != nil { @@ -574,19 +579,25 @@ func newPebbleInMem( attrs roachpb.Attributes, cacheSize, storeSize int64, settings *cluster.Settings, + knobs *TestingKnobs, ) *Pebble { opts := DefaultPebbleOptions() opts.Cache = pebble.NewCache(cacheSize) defer opts.Cache.Unref() + disableSeparatedIntents := false + if knobs != nil { + disableSeparatedIntents = knobs.DisableSeparatedIntents + } opts.FS = vfs.NewMem() db, err := NewPebble( ctx, PebbleConfig{ StorageConfig: base.StorageConfig{ - Attrs: attrs, - MaxSize: storeSize, - Settings: settings, + Attrs: attrs, + MaxSize: storeSize, + Settings: settings, + DisableSeparatedIntents: disableSeparatedIntents, }, Opts: opts, }) @@ -904,7 +915,7 @@ func (p *Pebble) SafeToWriteSeparatedIntents(ctx context.Context) (bool, error) // IsSeparatedIntentsEnabledForTesting implements the Engine interface. func (p *Pebble) IsSeparatedIntentsEnabledForTesting(ctx context.Context) bool { return !p.settings.Version.ActiveVersionOrEmpty(ctx).Less( - clusterversion.ByKey(clusterversion.SeparatedIntents)) && SeparatedIntentsEnabled.Get(&p.settings.SV) + clusterversion.ByKey(clusterversion.SeparatedIntents)) && !p.disableSeparatedIntents } func (p *Pebble) put(key MVCCKey, value []byte) error { @@ -1042,7 +1053,9 @@ func (p *Pebble) GetAuxiliaryDir() string { // NewBatch implements the Engine interface. func (p *Pebble) NewBatch() Batch { - return newPebbleBatch(p.db, p.db.NewIndexedBatch(), false /* writeOnly */, p.settings) + return newPebbleBatch( + p.db, p.db.NewIndexedBatch(), false /* writeOnly */, p.settings, + p.disableSeparatedIntents) } // NewReadOnly implements the Engine interface. @@ -1052,7 +1065,7 @@ func (p *Pebble) NewReadOnly() ReadWriter { // NewUnindexedBatch implements the Engine interface. func (p *Pebble) NewUnindexedBatch(writeOnly bool) Batch { - return newPebbleBatch(p.db, p.db.NewBatch(), writeOnly, p.settings) + return newPebbleBatch(p.db, p.db.NewBatch(), writeOnly, p.settings, p.disableSeparatedIntents) } // NewSnapshot implements the Engine interface. diff --git a/pkg/storage/pebble_batch.go b/pkg/storage/pebble_batch.go index f23819cc6348..a5d464cd8f84 100644 --- a/pkg/storage/pebble_batch.go +++ b/pkg/storage/pebble_batch.go @@ -66,7 +66,11 @@ var pebbleBatchPool = sync.Pool{ // Instantiates a new pebbleBatch. func newPebbleBatch( - db *pebble.DB, batch *pebble.Batch, writeOnly bool, settings *cluster.Settings, + db *pebble.DB, + batch *pebble.Batch, + writeOnly bool, + settings *cluster.Settings, + disableSeparatedIntents bool, ) *pebbleBatch { pb := pebbleBatchPool.Get().(*pebbleBatch) *pb = pebbleBatch{ @@ -95,8 +99,9 @@ func newPebbleBatch( }, writeOnly: writeOnly, } - pb.wrappedIntentWriter = - wrapIntentWriter(context.Background(), pb, settings, false /* isLongLived */) + pb.wrappedIntentWriter = wrapIntentWriter( + context.Background(), pb, settings, false, /* isLongLived */ + disableSeparatedIntents) return pb } diff --git a/pkg/storage/pebble_mvcc_scanner_test.go b/pkg/storage/pebble_mvcc_scanner_test.go index 934042d88e74..23e63f019fb5 100644 --- a/pkg/storage/pebble_mvcc_scanner_test.go +++ b/pkg/storage/pebble_mvcc_scanner_test.go @@ -30,8 +30,7 @@ func TestMVCCScanWithManyVersionsAndSeparatedIntents(t *testing.T) { defer leaktest.AfterTest(t)() // Force separated intents for writing. - settings := makeSettingsForSeparatedIntents( - false /* oldClusterVersion */, true /* enabled */) + settings := makeSettingsForSeparatedIntents(false /* oldClusterVersion */) eng := createTestPebbleEngineWithSettings(settings) defer eng.Close() diff --git a/pkg/storage/pebble_test.go b/pkg/storage/pebble_test.go index 55020acba970..da0d3a939294 100644 --- a/pkg/storage/pebble_test.go +++ b/pkg/storage/pebble_test.go @@ -431,6 +431,7 @@ func TestPebbleDiskSlowEmit(t *testing.T) { 1<<20, /* cacheSize */ 512<<20, /* storeSize */ settings, + nil, /* knobs */ ) defer p.Close() diff --git a/pkg/storage/testing_knobs.go b/pkg/storage/testing_knobs.go new file mode 100644 index 000000000000..40079447027f --- /dev/null +++ b/pkg/storage/testing_knobs.go @@ -0,0 +1,19 @@ +// Copyright 2018 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package storage + +// TestingKnobs can be passed when instantiating a storage engine. Settings here +// are used to change behavior in tests. +type TestingKnobs struct { + // DisableSeparatedIntents disables the writing of separated intents. Only + // used in tests to check handling of interleaved intents. + DisableSeparatedIntents bool +} diff --git a/pkg/testutils/testcluster/testcluster.go b/pkg/testutils/testcluster/testcluster.go index 3b714673fd86..00663842d4a7 100644 --- a/pkg/testutils/testcluster/testcluster.go +++ b/pkg/testutils/testcluster/testcluster.go @@ -1383,7 +1383,7 @@ func (tc *TestCluster) RestartServerWithInspect(idx int, inspect func(s *server. } for i, specs := range serverArgs.StoreSpecs { - if specs.StickyInMemoryEngineID == "" { + if specs.InMemory && specs.StickyInMemoryEngineID == "" { return errors.Errorf("failed to restart Server %d, because a restart can only be used on a server with a sticky engine", i) } }