diff --git a/pkg/base/config.go b/pkg/base/config.go index 8d22ccd1b782..6929b69909d9 100644 --- a/pkg/base/config.go +++ b/pkg/base/config.go @@ -515,6 +515,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/batcheval/cmd_revert_range_test.go b/pkg/kv/kvserver/batcheval/cmd_revert_range_test.go index 77b74559fd43..a6eb546db687 100644 --- a/pkg/kv/kvserver/batcheval/cmd_revert_range_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_revert_range_test.go @@ -55,7 +55,7 @@ func getStats(t *testing.T, reader storage.Reader) enginepb.MVCCStats { // createTestPebbleEngine returns a new in-memory Pebble storage engine. func createTestPebbleEngine(ctx context.Context) (storage.Engine, error) { return storage.Open(ctx, storage.InMemory(), - storage.MaxSize(1<<20), storage.SettingsForTesting()) + storage.MaxSize(1<<20), storage.ForTesting) } var engineImpls = []struct { diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index f7fba5856bb7..05c639974999 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -238,7 +238,7 @@ func (tc *testContext) StartWithStoreConfigAndVersion( storage.InMemory(), storage.Attributes(roachpb.Attributes{Attrs: []string{"dc1", "mem"}}), storage.MaxSize(1<<20), - storage.SettingsForTesting()) + storage.ForTesting) if err != nil { t.Fatal(err) } @@ -6357,12 +6357,6 @@ func verifyRangeStats( return nil } -func accountForTxnDidNotUpdateMeta(t *testing.T, eng storage.Engine) bool { - accountFor, err := eng.SafeToWriteSeparatedIntents(context.Background()) - require.NoError(t, err) - return accountFor -} - // TestRangeStatsComputation verifies that commands executed against a // range update the range stat counters. The stat values are // empirically derived; we're really just testing that they increment @@ -6431,22 +6425,17 @@ func TestRangeStatsComputation(t *testing.T) { } expMS = baseStats expMS.Add(enginepb.MVCCStats{ - LiveBytes: 101, + LiveBytes: 103, KeyBytes: 28, - ValBytes: 73, + ValBytes: 75, IntentBytes: 23, LiveCount: 2, KeyCount: 2, ValCount: 2, IntentCount: 1, }) - if accountForTxnDidNotUpdateMeta(t, tc.engine) { - // Account for TxnDidNotUpdateMeta - expMS.LiveBytes += 2 - expMS.ValBytes += 2 - if tc.engine.IsSeparatedIntentsEnabledForTesting(ctx) { - expMS.SeparatedIntentCount++ - } + if tc.engine.IsSeparatedIntentsEnabledForTesting(ctx) { + expMS.SeparatedIntentCount++ } if err := verifyRangeStats(tc.engine, tc.repl.RangeID, expMS); err != nil { t.Fatal(err) diff --git a/pkg/kv/kvserver/spanset/batch.go b/pkg/kv/kvserver/spanset/batch.go index 3185dcbb5749..d8d4678518da 100644 --- a/pkg/kv/kvserver/spanset/batch.go +++ b/pkg/kv/kvserver/spanset/batch.go @@ -644,10 +644,6 @@ func (s spanSetWriter) PutEngineKey(key storage.EngineKey, value []byte) error { return s.w.PutEngineKey(key, value) } -func (s spanSetWriter) SafeToWriteSeparatedIntents(ctx context.Context) (bool, error) { - return s.w.SafeToWriteSeparatedIntents(ctx) -} - func (s spanSetWriter) LogData(data []byte) error { return s.w.LogData(data) } diff --git a/pkg/kv/kvserver/testing_knobs.go b/pkg/kv/kvserver/testing_knobs.go index cc5674ab477d..4466034deb86 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 fae083a383d5..346d14cf9804 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" @@ -44,26 +45,28 @@ func TestSeparatedIntentsMigration(t *testing.T) { clusterversion.ByKey(clusterversion.SeparatedIntentsMigration-1), false, /* initializeVersion */ ) - storage.SeparatedIntentsEnabled.Override(ctx, &settings.SV, false) - stickyEngineRegistry := server.NewStickyInMemEnginesRegistry() - defer stickyEngineRegistry.CloseAllStickyInMemEngines() const numServers int = 3 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, }, + Store: storeKnobs, }, } } @@ -143,7 +146,8 @@ func TestSeparatedIntentsMigration(t *testing.T) { } require.True(t, interleavedIntentFound) - 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) tdb = tc.ServerConn(0) diff --git a/pkg/server/config.go b/pkg/server/config.go index ebffbe86fd6f..2f290ca0c0f5 100644 --- a/pkg/server/config.go +++ b/pkg/server/config.go @@ -479,6 +479,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 @@ -542,12 +544,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/settings_cache_test.go b/pkg/server/settings_cache_test.go index d535484ec5f3..a239bcbf1e61 100644 --- a/pkg/server/settings_cache_test.go +++ b/pkg/server/settings_cache_test.go @@ -44,7 +44,7 @@ func TestCachedSettingsStoreAndLoad(t *testing.T) { ctx := context.Background() engine, err := storage.Open(ctx, storage.InMemory(), storage.MaxSize(512<<20 /* 512 MiB */), - storage.SettingsForTesting()) + storage.ForTesting) require.NoError(t, err) defer engine.Close() diff --git a/pkg/server/settingswatcher/BUILD.bazel b/pkg/server/settingswatcher/BUILD.bazel index 17d426f3d009..bd33266cf6b2 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/sticky_engine.go b/pkg/server/sticky_engine.go index 03a4df8420a0..a7cff27f4f93 100644 --- a/pkg/server/sticky_engine.go +++ b/pkg/server/sticky_engine.go @@ -112,7 +112,7 @@ func (registry *stickyInMemEnginesRegistryImpl) GetOrCreateStickyInMemEngine( storage.Attributes(spec.Attributes), storage.CacheSize(cfg.CacheSize), storage.MaxSize(spec.Size.InBytes), - storage.SettingsForTesting()) + storage.ForTesting) engineEntry := &stickyInMemEngine{ id: spec.StickyInMemoryEngineID, diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index 48d703622af0..1c2e9d009528 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -15,7 +15,6 @@ import ( "context" "encoding/base64" "fmt" - "math/rand" "net/http" "net/http/cookiejar" "net/url" @@ -56,7 +55,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/ts" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/retry" @@ -129,10 +127,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 c2b59b724af3..8756714f3866 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -35,12 +35,12 @@ go_library( "sst_writer.go", "temp_dir.go", "temp_engine.go", + "testing_knobs.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/storage", visibility = ["//visibility:public"], deps = [ "//pkg/base", - "//pkg/clusterversion", "//pkg/keys", "//pkg/kv/kvserver/concurrency/lock", "//pkg/kv/kvserver/diskmap", diff --git a/pkg/storage/bench_pebble_test.go b/pkg/storage/bench_pebble_test.go index 6c38a1f35efe..4de8f5101172 100644 --- a/pkg/storage/bench_pebble_test.go +++ b/pkg/storage/bench_pebble_test.go @@ -17,7 +17,6 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -31,9 +30,7 @@ func setupMVCCPebble(b testing.TB, dir string) Engine { peb, err := Open( context.Background(), Filesystem(dir), - CacheSize(testCacheSize), - Settings(makeSettingsForSeparatedIntents( - false /* oldClusterVersion */, true /* enabled */))) + CacheSize(testCacheSize)) if err != nil { b.Fatalf("could not create new pebble instance at %s: %+v", dir, err) } @@ -41,15 +38,14 @@ func setupMVCCPebble(b testing.TB, dir string) Engine { } func setupMVCCInMemPebble(b testing.TB, loc string) Engine { - return setupMVCCInMemPebbleWithSettings(b, makeSettingsForSeparatedIntents( - false /* oldClusterVersion */, true /* enabled */)) + return setupMVCCInMemPebbleWithSeparatedIntents(b, false /* disabledSeparatedIntents */) } -func setupMVCCInMemPebbleWithSettings(b testing.TB, settings *cluster.Settings) Engine { +func setupMVCCInMemPebbleWithSeparatedIntents(b testing.TB, disableSeparatedIntents bool) Engine { peb, err := Open( context.Background(), InMemory(), - Settings(settings), + SetSeparatedIntents(disableSeparatedIntents), CacheSize(testCacheSize)) if err != nil { b.Fatalf("could not create new in-mem pebble instance: %+v", err) diff --git a/pkg/storage/bench_test.go b/pkg/storage/bench_test.go index 2c0f2e4a1f9d..bafed6f1d74f 100644 --- a/pkg/storage/bench_test.go +++ b/pkg/storage/bench_test.go @@ -212,8 +212,7 @@ func BenchmarkIntentScan(b *testing.B) { b.Run(fmt.Sprintf("versions=%d", numVersions), func(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)) + eng := setupMVCCInMemPebbleWithSeparatedIntents(b, sep) numFlushedVersions := (percentFlushed * numVersions) / 100 setupKeysWithIntent(b, eng, numVersions, numFlushedVersions, false /* resolveAll */) lower := makeKey(nil, 0) @@ -276,8 +275,7 @@ func BenchmarkScanAllIntentsResolved(b *testing.B) { b.Run(fmt.Sprintf("versions=%d", numVersions), func(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)) + eng := setupMVCCInMemPebbleWithSeparatedIntents(b, sep) numFlushedVersions := (percentFlushed * numVersions) / 100 setupKeysWithIntent(b, eng, numVersions, numFlushedVersions, true /* resolveAll */) lower := makeKey(nil, 0) @@ -347,8 +345,7 @@ func BenchmarkScanOneAllIntentsResolved(b *testing.B) { b.Run(fmt.Sprintf("versions=%d", numVersions), func(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)) + eng := setupMVCCInMemPebbleWithSeparatedIntents(b, sep) numFlushedVersions := (percentFlushed * numVersions) / 100 setupKeysWithIntent(b, eng, numVersions, numFlushedVersions, true /* resolveAll */) lower := makeKey(nil, 0) @@ -399,8 +396,7 @@ func BenchmarkIntentResolution(b *testing.B) { b.Run(fmt.Sprintf("versions=%d", numVersions), func(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)) + eng := setupMVCCInMemPebbleWithSeparatedIntents(b, sep) numFlushedVersions := (percentFlushed * numVersions) / 100 lockUpdate := setupKeysWithIntent(b, eng, numVersions, numFlushedVersions, false /* resolveAll */) keys := make([]roachpb.Key, numIntentKeys) @@ -443,8 +439,7 @@ func BenchmarkIntentRangeResolution(b *testing.B) { b.Run(fmt.Sprintf("versions=%d", numVersions), func(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)) + eng := setupMVCCInMemPebbleWithSeparatedIntents(b, 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.go b/pkg/storage/engine.go index 37241f5af0d7..9ef8349fa063 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -654,11 +654,6 @@ type Writer interface { // // It is safe to modify the contents of the arguments after Put returns. PutEngineKey(key EngineKey, value []byte) error - // SafeToWriteSeparatedIntents is only for internal use in the storage - // package. Returns an error if the callee does not know whether it is safe. - // This method is temporary, to handle the transition from clusters where - // not all nodes understand separated intents. - SafeToWriteSeparatedIntents(ctx context.Context) (bool, error) // LogData adds the specified data to the RocksDB WAL. The data is // uninterpreted by RocksDB (i.e. not added to the memtable or sstables). diff --git a/pkg/storage/engine_test.go b/pkg/storage/engine_test.go index 9c6e9b3346c7..faad997c1c1b 100644 --- a/pkg/storage/engine_test.go +++ b/pkg/storage/engine_test.go @@ -1639,7 +1639,7 @@ func TestScanSeparatedIntents(t *testing.T) { for name, enableSeparatedIntents := range map[string]bool{"interleaved": false, "separated": true} { t.Run(name, func(t *testing.T) { eng, err := Open(ctx, InMemory(), CacheSize(1<<20 /* 1 MiB */), - Settings(makeSettingsForSeparatedIntents(false, enableSeparatedIntents))) + SetSeparatedIntents(!enableSeparatedIntents)) require.NoError(t, err) defer eng.Close() diff --git a/pkg/storage/in_mem.go b/pkg/storage/in_mem.go index 3ab2c49fa8f7..6f1922f89e35 100644 --- a/pkg/storage/in_mem.go +++ b/pkg/storage/in_mem.go @@ -12,11 +12,7 @@ package storage import ( "context" - "math/rand" - "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/pebble/vfs" ) @@ -50,31 +46,9 @@ func InMemFromFS(ctx context.Context, fs vfs.FS, dir string, opts ...ConfigOptio // the default configuration. The caller must call the engine's Close method // when the engine is no longer needed. func NewDefaultInMemForTesting() Engine { - eng, err := Open(context.Background(), InMemory(), SettingsForTesting(), MaxSize(1<<20)) + eng, err := Open(context.Background(), InMemory(), ForTesting, MaxSize(1<<20)) if err != nil { panic(err) } return eng } - -// 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. -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) -} - -func makeSettingsForSeparatedIntents(oldClusterVersion bool, enabled 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..fafd3e66fc9d 100644 --- a/pkg/storage/intent_reader_writer.go +++ b/pkg/storage/intent_reader_writer.go @@ -14,44 +14,14 @@ import ( "context" "sync" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "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. @@ -59,32 +29,17 @@ var SeparatedIntentsEnabled = settings.RegisterBoolSetting( // PutIntent, ClearIntent, ClearMVCCRangeAndIntents. type intentDemuxWriter struct { w Writer - // Must be non-nil if this intentDemuxWriter is used. We do the checking - // lazily when methods are called since the clients of intentDemuxWriter - // initialize it up-front, but don't know if they are being used by code - // that cares about intents (e.g. a temporary Engine used for disk-spilling - // during query execution will never read-write intents). - settings *cluster.Settings - cachedSettingsAreValid bool - clusterVersionIsRecentEnoughCached bool - writeSeparatedIntentsCached bool + // Can only be set to true by testing knobs to disable writing of separated + // intents. + disableSeparatedIntents bool } func wrapIntentWriter( - ctx context.Context, w Writer, settings *cluster.Settings, isLongLived bool, + ctx context.Context, w Writer, disableSeparatedIntents bool, ) intentDemuxWriter { - idw := intentDemuxWriter{w: w, settings: settings} - if !isLongLived && settings != nil { - // Cache the settings for performance. - idw.cachedSettingsAreValid = true - // 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 := intentDemuxWriter{w: w} + idw.disableSeparatedIntents = disableSeparatedIntents return idw } @@ -98,9 +53,6 @@ func (idw intentDemuxWriter) ClearIntent( txnUUID uuid.UUID, buf []byte, ) (_ []byte, separatedIntentCountDelta int, _ error) { - if idw.settings == nil { - return nil, 0, errors.AssertionFailedf("intentDemuxWriter not configured with cluster.Setttings") - } switch state { case ExistingIntentInterleaved: return buf, 0, idw.w.ClearUnversioned(key) @@ -132,22 +84,7 @@ func (idw intentDemuxWriter) PutIntent( txnUUID uuid.UUID, buf []byte, ) (_ []byte, separatedIntentCountDelta int, _ error) { - if idw.settings == nil { - return nil, 0, errors.AssertionFailedf("intentDemuxWriter not configured with cluster.Setttings") - } - var writeSeparatedIntents bool - if idw.cachedSettingsAreValid { - // Fast-path - writeSeparatedIntents = idw.clusterVersionIsRecentEnoughCached && idw.writeSeparatedIntentsCached - } else { - // Slow-path, when doing writes on the Engine directly. This should not be - // performance sensitive code. - writeSeparatedIntents = - // Be resilient to the version not yet being initialized. - !idw.settings.Version.ActiveVersionOrEmpty(ctx).Less( - clusterversion.ByKey(clusterversion.SeparatedIntents)) && - SeparatedIntentsEnabled.Get(&idw.settings.SV) - } + writeSeparatedIntents := !idw.disableSeparatedIntents var engineKey EngineKey if state == ExistingIntentSeparated || writeSeparatedIntents { engineKey, buf = LockTableKey{ @@ -194,9 +131,6 @@ func (idw intentDemuxWriter) PutIntent( func (idw intentDemuxWriter) ClearMVCCRangeAndIntents( start, end roachpb.Key, buf []byte, ) ([]byte, error) { - if idw.settings == nil { - return nil, errors.AssertionFailedf("intentDemuxWriter not configured with cluster.Setttings") - } err := idw.w.ClearRawRange(start, end) if err != nil { return buf, err @@ -206,20 +140,6 @@ func (idw intentDemuxWriter) ClearMVCCRangeAndIntents( return buf, idw.w.ClearRawRange(lstart, lend) } -func (idw intentDemuxWriter) safeToWriteSeparatedIntents(ctx context.Context) (bool, error) { - if idw.settings == nil { - return false, - errors.Errorf( - "intentDemuxWriter without cluster.Settings does not support SafeToWriteSeparatedIntents") - } - if idw.cachedSettingsAreValid { - return idw.clusterVersionIsRecentEnoughCached, nil - } - // Be resilient to the version not yet being initialized. - return !idw.settings.Version.ActiveVersionOrEmpty(ctx).Less( - clusterversion.ByKey(clusterversion.SeparatedIntents)), nil -} - // wrappableReader is used to implement a wrapped Reader. A wrapped Reader // should be used and immediately discarded. It maintains no state of its own // between calls. diff --git a/pkg/storage/intent_reader_writer_test.go b/pkg/storage/intent_reader_writer_test.go index 620a739057dd..ed49434a2063 100644 --- a/pkg/storage/intent_reader_writer_test.go +++ b/pkg/storage/intent_reader_writer_test.go @@ -215,9 +215,7 @@ func TestIntentDemuxWriter(t *testing.T) { // This is a low-level test that explicitly wraps the writer, so it // 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 */) + w = wrapIntentWriter(context.Background(), &pw, !separated) return "" case "put-intent": pw.reset() diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index f98516271263..69326ca5e52e 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -1079,28 +1079,20 @@ func (b *putBuffer) putIntentMeta( return 0, 0, 0, errors.AssertionFailedf( "meta.Timestamp != meta.Txn.WriteTimestamp: %s != %s", meta.Timestamp, meta.Txn.WriteTimestamp) } - safe, err := writer.SafeToWriteSeparatedIntents(ctx) - if err != nil { - return 0, 0, 0, err - } - if safe { - // All nodes in this cluster understand separated intents, so can fiddle - // with TxnDidNotUpdateMeta, which is not understood by older nodes (which - // are no longer present, and will never again be present). - // - // NB: the parameter txnDidNotUpdateMeta is about what happened prior to - // this Put, and is passed through to writer below. The field - // TxnDidNotUpdateMeta, in the MVCCMetadata we are about to write, - // includes what happened in this Put. - if state == NoExistingIntent { - meta.TxnDidNotUpdateMeta = &trueValue - } else { - // Absence represents false. - meta.TxnDidNotUpdateMeta = nil - } + // All nodes in this cluster understand separated intents, so can fiddle + // with TxnDidNotUpdateMeta, which is not understood by older nodes (which + // are no longer present, and will never again be present). + // + // NB: the parameter txnDidNotUpdateMeta is about what happened prior to + // this Put, and is passed through to writer below. The field + // TxnDidNotUpdateMeta, in the MVCCMetadata we are about to write, + // includes what happened in this Put. + if state == NoExistingIntent { + meta.TxnDidNotUpdateMeta = &trueValue + } else { + // Absence represents false. + meta.TxnDidNotUpdateMeta = nil } - // Else disallowSeparatedIntents, so don't set MVCCMetadata.TxnDidNotUpdateMeta - // for compatibility in mixed version clusters. bytes, err := b.marshalMeta(meta) if err != nil { diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index e69d894fd861..2842f2166ab1 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -94,32 +94,22 @@ func TestMVCCHistories(t *testing.T) { datadriven.Walk(t, "testdata/mvcc_histories", func(t *testing.T, path string) { // Default to random behavior wrt cluster version and separated // intents. - oldClusterVersion := rand.Intn(2) == 0 enabledSeparated := rand.Intn(2) == 0 overridden := false if strings.Contains(path, "_disallow_separated") { - oldClusterVersion = true - enabledSeparated = false - overridden = true - } - if strings.Contains(path, "_allow_separated") { - oldClusterVersion = false enabledSeparated = false overridden = true } if strings.Contains(path, "_enable_separated") { - oldClusterVersion = false enabledSeparated = true overridden = true } if !overridden { log.Infof(context.Background(), - "randomly setting oldClusterVersion: %t, enableSeparated: %t", - oldClusterVersion, enabledSeparated) + "randomly setting enableSeparated: %t", enabledSeparated) } - settings := makeSettingsForSeparatedIntents(oldClusterVersion, enabledSeparated) // We start from a clean slate in every test file. - engine, err := Open(ctx, InMemory(), CacheSize(1<<20 /* 1 MiB */), Settings(settings)) + engine, err := Open(ctx, InMemory(), CacheSize(1<<20 /* 1 MiB */), SetSeparatedIntents(!enabledSeparated)) if err != nil { t.Fatal(err) } diff --git a/pkg/storage/mvcc_incremental_iterator_test.go b/pkg/storage/mvcc_incremental_iterator_test.go index ab85d1aaa6ca..18b869dcfe6d 100644 --- a/pkg/storage/mvcc_incremental_iterator_test.go +++ b/pkg/storage/mvcc_incremental_iterator_test.go @@ -912,7 +912,7 @@ func TestMVCCIncrementalIteratorIntentStraddlesSStables(t *testing.T) { // regular MVCCPut operation to generate these keys, which we'll later be // copying into manually created sstables. ctx := context.Background() - db1, err := Open(ctx, InMemory(), SettingsForTesting()) + db1, err := Open(ctx, InMemory(), ForTesting) require.NoError(t, err) defer db1.Close() @@ -948,7 +948,7 @@ func TestMVCCIncrementalIteratorIntentStraddlesSStables(t *testing.T) { // // SSTable 2: // b@2 - db2, err := Open(ctx, InMemory(), SettingsForTesting()) + db2, err := Open(ctx, InMemory(), ForTesting) require.NoError(t, err) defer db2.Close() diff --git a/pkg/storage/mvcc_stats_test.go b/pkg/storage/mvcc_stats_test.go index 147cacbafd25..12369011325f 100644 --- a/pkg/storage/mvcc_stats_test.go +++ b/pkg/storage/mvcc_stats_test.go @@ -76,12 +76,6 @@ func assertEqLocal(t *testing.T, rw ReadWriter, debug string, ms, expMS *enginep assertEqImpl(t, rw, debug, false /* globalKeys */, ms, expMS) } -func accountForTxnDidNotUpdateMeta(t *testing.T, eng Engine) bool { - accountFor, err := eng.SafeToWriteSeparatedIntents(context.Background()) - require.NoError(t, err) - return accountFor -} - // TestMVCCStatsDeleteCommitMovesTimestamp exercises the case in which a value // is written, later deleted via an intent and the deletion committed at an even // higher timestamp. This exercises subtleties related to the implicit push of @@ -197,12 +191,9 @@ func TestMVCCStatsPutCommitMovesTimestamp(t *testing.T) { Txn: &txn.TxnMeta, }).Size()) var separatedIntentCount int64 - if accountForTxnDidNotUpdateMeta(t, engine) { - // Account for TxnDidNotUpdateMeta - mValSize += 2 - if engine.IsSeparatedIntentsEnabledForTesting(ctx) { - separatedIntentCount = 1 - } + mValSize += 2 + if engine.IsSeparatedIntentsEnabledForTesting(ctx) { + separatedIntentCount = 1 } vKeySize := MVCCVersionTimestampSize // 12 vValSize := int64(len(value.RawBytes)) // 10 @@ -286,12 +277,9 @@ func TestMVCCStatsPutPushMovesTimestamp(t *testing.T) { Txn: &txn.TxnMeta, }).Size()) var separatedIntentCount int64 - if accountForTxnDidNotUpdateMeta(t, engine) { - // Account for TxnDidNotUpdateMeta - mValSize += 2 - if engine.IsSeparatedIntentsEnabledForTesting(ctx) { - separatedIntentCount = 1 - } + mValSize += 2 + if engine.IsSeparatedIntentsEnabledForTesting(ctx) { + separatedIntentCount = 1 } vKeySize := MVCCVersionTimestampSize // 12 vValSize := int64(len(value.RawBytes)) // 10 @@ -320,10 +308,8 @@ func TestMVCCStatsPutPushMovesTimestamp(t *testing.T) { ); err != nil { t.Fatal(err) } - if accountForTxnDidNotUpdateMeta(t, engine) { - // Account for removal of TxnDidNotUpdateMeta - mValSize -= 2 - } + // Account for removal of TxnDidNotUpdateMeta + mValSize -= 2 expAggMS := enginepb.MVCCStats{ LastUpdateNanos: 4e9, @@ -388,12 +374,9 @@ func TestMVCCStatsDeleteMovesTimestamp(t *testing.T) { }).Size()) require.EqualValues(t, mVal1Size, 46) var separatedIntentCount int64 - if accountForTxnDidNotUpdateMeta(t, engine) { - // Account for TxnDidNotUpdateMeta - mVal1Size += 2 - if engine.IsSeparatedIntentsEnabledForTesting(ctx) { - separatedIntentCount = 1 - } + mVal1Size += 2 + if engine.IsSeparatedIntentsEnabledForTesting(ctx) { + separatedIntentCount = 1 } // TODO(sumeer): this is the first put at ts1, so why are we using this m1ValSize @@ -404,10 +387,7 @@ func TestMVCCStatsDeleteMovesTimestamp(t *testing.T) { Txn: &txn.TxnMeta, }).Size()) require.EqualValues(t, m1ValSize, 46) - if accountForTxnDidNotUpdateMeta(t, engine) { - // Account for TxnDidNotUpdateMeta - m1ValSize += 2 - } + m1ValSize += 2 vKeySize := MVCCVersionTimestampSize require.EqualValues(t, vKeySize, 12) @@ -518,12 +498,9 @@ func TestMVCCStatsPutMovesDeletionTimestamp(t *testing.T) { }).Size()) require.EqualValues(t, mVal1Size, 46) var separatedIntentCount int64 - if accountForTxnDidNotUpdateMeta(t, engine) { - // Account for TxnDidNotUpdateMeta - mVal1Size += 2 - if engine.IsSeparatedIntentsEnabledForTesting(ctx) { - separatedIntentCount = 1 - } + mVal1Size += 2 + if engine.IsSeparatedIntentsEnabledForTesting(ctx) { + separatedIntentCount = 1 } vKeySize := MVCCVersionTimestampSize @@ -654,12 +631,10 @@ func TestMVCCStatsDelDelCommitMovesTimestamp(t *testing.T) { }).Size()) require.EqualValues(t, mValSize, 46) var separatedIntentCount int64 - if accountForTxnDidNotUpdateMeta(t, engine) { - // Account for TxnDidNotUpdateMeta - mValSize += 2 - if engine.IsSeparatedIntentsEnabledForTesting(ctx) { - separatedIntentCount = 1 - } + // Account for TxnDidNotUpdateMeta + mValSize += 2 + if engine.IsSeparatedIntentsEnabledForTesting(ctx) { + separatedIntentCount = 1 } expMS = enginepb.MVCCStats{ @@ -810,12 +785,9 @@ func TestMVCCStatsPutDelPutMovesTimestamp(t *testing.T) { }).Size()) require.EqualValues(t, mValSize, 46) var separatedIntentCount int64 - if accountForTxnDidNotUpdateMeta(t, engine) { - // Account for TxnDidNotUpdateMeta - mValSize += 2 - if engine.IsSeparatedIntentsEnabledForTesting(ctx) { - separatedIntentCount = 1 - } + mValSize += 2 + if engine.IsSeparatedIntentsEnabledForTesting(ctx) { + separatedIntentCount = 1 } expMS = enginepb.MVCCStats{ @@ -1037,12 +1009,9 @@ func TestMVCCStatsPutIntentTimestampNotPutTimestamp(t *testing.T) { Txn: &txn.TxnMeta, }).Size()) var separatedIntentCount int64 - if accountForTxnDidNotUpdateMeta(t, engine) { - // Account for TxnDidNotUpdateMeta - m1ValSize += 2 - if engine.IsSeparatedIntentsEnabledForTesting(ctx) { - separatedIntentCount = 1 - } + m1ValSize += 2 + if engine.IsSeparatedIntentsEnabledForTesting(ctx) { + separatedIntentCount = 1 } vKeySize := MVCCVersionTimestampSize // 12 vValSize := int64(len(value.RawBytes)) // 10 @@ -1233,10 +1202,7 @@ func TestMVCCStatsTxnSysPutPut(t *testing.T) { Txn: &txn.TxnMeta, }).Size()) require.EqualValues(t, mValSize, 46) - if accountForTxnDidNotUpdateMeta(t, engine) { - // Account for TxnDidNotUpdateMeta - mValSize += 2 - } + mValSize += 2 vKeySize := MVCCVersionTimestampSize require.EqualValues(t, vKeySize, 12) @@ -1326,10 +1292,7 @@ func TestMVCCStatsTxnSysPutAbort(t *testing.T) { Txn: &txn.TxnMeta, }).Size()) require.EqualValues(t, mValSize, 46) - if accountForTxnDidNotUpdateMeta(t, engine) { - // Account for TxnDidNotUpdateMeta - mValSize += 2 - } + mValSize += 2 vKeySize := MVCCVersionTimestampSize require.EqualValues(t, vKeySize, 12) diff --git a/pkg/storage/open.go b/pkg/storage/open.go index 33c535b30663..29f0136478e7 100644 --- a/pkg/storage/open.go +++ b/pkg/storage/open.go @@ -12,10 +12,12 @@ package storage import ( "context" + "math/rand" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/pebble" "github.com/cockroachdb/pebble/vfs" ) @@ -48,6 +50,20 @@ var MustExist ConfigOption = func(cfg *engineConfig) error { return nil } +// ForTesting configures the engine for use in testing. It may randomize some +// config options to improve test coverage +var ForTesting ConfigOption = func(cfg *engineConfig) error { + if cfg.Settings == nil { + cfg.Settings = cluster.MakeTestingClusterSettings() + } + disableSeparatedIntents := rand.Intn(2) == 0 + log.Infof(context.Background(), + "engine creation is randomly setting disableSeparatedIntents: %t", + disableSeparatedIntents) + cfg.DisableSeparatedIntents = disableSeparatedIntents + return nil +} + // Attributes configures the engine's attributes. func Attributes(attrs roachpb.Attributes) ConfigOption { return func(cfg *engineConfig) error { @@ -101,11 +117,13 @@ func Hook(hookFunc func(*base.StorageConfig) error) ConfigOption { } } -// SettingsForTesting configures the engine's cluster settings for an engine -// used in testing. It may randomize some cluster settings to improve test -// coverage. -func SettingsForTesting() ConfigOption { - return Settings(makeRandomSettingsForSeparatedIntents()) +// SetSeparatedIntents sets the config option(s) for separated intents. If the +// bool argument is true, separated intents are _not_ written. +func SetSeparatedIntents(disable bool) ConfigOption { + return func(cfg *engineConfig) error { + cfg.DisableSeparatedIntents = disable + return nil + } } // A Location describes where the storage engine's data will be written. A @@ -164,5 +182,8 @@ func Open(ctx context.Context, loc Location, opts ...ConfigOption) (*Pebble, err cfg.Opts.Cache = pebble.NewCache(*cfg.cacheSize) defer cfg.Opts.Cache.Unref() } + if cfg.Settings == nil { + cfg.Settings = cluster.MakeClusterSettings() + } return NewPebble(ctx, cfg.PebbleConfig) } diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index ea6cd5fef6a5..9b1161c93797 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -27,7 +27,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -461,6 +460,9 @@ type Pebble struct { diskSlowCount int64 diskStallCount int64 + // Copied from testing knobs. + disableSeparatedIntents bool + // Relevant options copied over from pebble.Options. fs vfs.FS logger pebble.Logger @@ -576,16 +578,17 @@ func NewPebble(ctx context.Context, cfg PebbleConfig) (*Pebble, error) { depth: 1, } 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, - storeIDPebbleLog: storeIDContainer, + 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, + storeIDPebbleLog: storeIDContainer, + disableSeparatedIntents: cfg.DisableSeparatedIntents, } cfg.Opts.EventListener = pebble.TeeEventListener( pebble.MakeLoggingEventListener(pebbleLogger{ @@ -595,7 +598,7 @@ func NewPebble(ctx context.Context, cfg PebbleConfig) (*Pebble, error) { p.makeMetricEventListener(ctx), ) p.eventListener = &cfg.Opts.EventListener - p.wrappedIntentWriter = wrapIntentWriter(ctx, p, cfg.Settings, true /* isLongLived */) + p.wrappedIntentWriter = wrapIntentWriter(ctx, p, cfg.DisableSeparatedIntents) db, err := pebble.Open(cfg.StorageConfig.Dir, cfg.Opts) if err != nil { @@ -915,17 +918,9 @@ func (p *Pebble) PutEngineKey(key EngineKey, value []byte) error { return p.db.Set(key.Encode(), value, pebble.Sync) } -// SafeToWriteSeparatedIntents implements the Engine interface. -func (p *Pebble) SafeToWriteSeparatedIntents(ctx context.Context) (bool, error) { - // This is not fast. Pebble should not be used by writers that want - // performance. They should use pebbleBatch. - return p.wrappedIntentWriter.safeToWriteSeparatedIntents(ctx) -} - // 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) + return !p.disableSeparatedIntents } func (p *Pebble) put(key MVCCKey, value []byte) error { @@ -1150,7 +1145,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.disableSeparatedIntents) } // NewReadOnly implements the Engine interface. @@ -1160,7 +1157,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.disableSeparatedIntents) } // NewSnapshot implements the Engine interface. @@ -1630,10 +1627,6 @@ func (p *pebbleReadOnly) PutEngineKey(key EngineKey, value []byte) error { panic("not implemented") } -func (p *pebbleReadOnly) SafeToWriteSeparatedIntents(context.Context) (bool, error) { - panic("not implemented") -} - func (p *pebbleReadOnly) LogData(data []byte) error { panic("not implemented") } diff --git a/pkg/storage/pebble_batch.go b/pkg/storage/pebble_batch.go index f37a1ed967fb..9064858712c9 100644 --- a/pkg/storage/pebble_batch.go +++ b/pkg/storage/pebble_batch.go @@ -16,7 +16,6 @@ import ( "sync" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -67,7 +66,7 @@ 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, disableSeparatedIntents bool, ) *pebbleBatch { pb := pebbleBatchPool.Get().(*pebbleBatch) *pb = pebbleBatch{ @@ -96,8 +95,7 @@ func newPebbleBatch( }, writeOnly: writeOnly, } - pb.wrappedIntentWriter = - wrapIntentWriter(context.Background(), pb, settings, false /* isLongLived */) + pb.wrappedIntentWriter = wrapIntentWriter(context.Background(), pb, disableSeparatedIntents) return pb } @@ -476,11 +474,6 @@ func (p *pebbleBatch) PutEngineKey(key EngineKey, value []byte) error { return p.batch.Set(p.buf, value, nil) } -// SafeToWriteSeparatedIntents implements the Batch interface. -func (p *pebbleBatch) SafeToWriteSeparatedIntents(ctx context.Context) (bool, error) { - return p.wrappedIntentWriter.safeToWriteSeparatedIntents(ctx) -} - func (p *pebbleBatch) put(key MVCCKey, value []byte) error { if len(key.Key) == 0 { return emptyKeyError() diff --git a/pkg/storage/pebble_mvcc_scanner_test.go b/pkg/storage/pebble_mvcc_scanner_test.go index 43b126c3edaf..240e8b42935d 100644 --- a/pkg/storage/pebble_mvcc_scanner_test.go +++ b/pkg/storage/pebble_mvcc_scanner_test.go @@ -32,11 +32,9 @@ import ( func TestMVCCScanWithManyVersionsAndSeparatedIntents(t *testing.T) { defer leaktest.AfterTest(t)() - // Force separated intents for writing. + // We default to separated intents enabled. eng, err := Open(context.Background(), InMemory(), - CacheSize(1<<20), - Settings(makeSettingsForSeparatedIntents( - false /* oldClusterVersion */, true /* enabled */))) + CacheSize(1<<20)) require.NoError(t, err) defer eng.Close() diff --git a/pkg/storage/sst_writer.go b/pkg/storage/sst_writer.go index a23c8221c8aa..d54fa0c7e824 100644 --- a/pkg/storage/sst_writer.go +++ b/pkg/storage/sst_writer.go @@ -180,11 +180,6 @@ func (fw *SSTWriter) PutEngineKey(key EngineKey, value []byte) error { return fw.fw.Set(fw.scratch, value) } -// SafeToWriteSeparatedIntents implements the Writer interface. -func (fw *SSTWriter) SafeToWriteSeparatedIntents(context.Context) (bool, error) { - return false, errors.Errorf("SSTWriter does not support SafeToWriteSeparatedIntents") -} - // put puts a kv entry into the sstable being built. An error is returned if it // is not greater than any previously added entry (according to the comparator // configured during writer creation). `Close` cannot have been called. diff --git a/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_allow_separated b/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_allow_separated deleted file mode 100644 index c79c4ff750b2..000000000000 --- a/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_allow_separated +++ /dev/null @@ -1,110 +0,0 @@ -run ok -txn_begin t=A ts=123 ----- ->> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=123.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=123.000000000,0 wto=false gul=0,0 - -# Write value1. - -run ok -with t=A - txn_step - cput k=k v=v ----- ->> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=123.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=123.000000000,0 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=123.000000000,0 min=0,0 seq=1} ts=123.000000000,0 del=false klen=12 vlen=6 -data: "k"/123.000000000,0 -> /BYTES/v - -# Now, overwrite value1 with value2 from same txn; should see value1 -# as pre-existing value. - -run ok -with t=A - txn_step - cput k=k v=v2 cond=v ----- ->> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=123.000000000,0 min=0,0 seq=2} lock=true stat=PENDING rts=123.000000000,0 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=123.000000000,0 min=0,0 seq=2} ts=123.000000000,0 del=false klen=12 vlen=7 ih={{1 /BYTES/v}} -data: "k"/123.000000000,0 -> /BYTES/v2 - -# Writing value3 from a new epoch should see nil again. - -run ok -with t=A - txn_restart - txn_step - cput k=k v=v3 ----- ->> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=123.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=123.000000000,0 wto=false gul=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=123.000000000,0 min=0,0 seq=1} ts=123.000000000,0 del=false klen=12 vlen=7 -data: "k"/123.000000000,0 -> /BYTES/v3 - -# Commit value3 at a later timestamp. - -run ok -with t=A - txn_advance ts=124 - resolve_intent k=k - txn_remove ----- ->> at end: -data: "k"/124.000000000,0 -> /BYTES/v3 - -# Write value4 with an old timestamp without txn...should get a write -# too old error. - -run error -cput k=k v=v4 cond=v3 ts=123 ----- ->> at end: -data: "k"/124.000000000,1 -> /BYTES/v4 -data: "k"/124.000000000,0 -> /BYTES/v3 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 123.000000000,0 too old; wrote at 124.000000000,1 - -# Reset for next test - -run ok -clear_range k=k end=-k ----- ->> at end: - - -# From TxnCoordSenderRetries, -# "multi-range batch with forwarded timestamp and cput and delete range" - -# First txn attempt - -run ok -# Before txn start: -put k=c v=value ts=1 -with t=A - txn_begin ts=2 - txn_step - cput k=c v=cput cond=value ----- ->> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 -meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=1} ts=2.000000000,0 del=false klen=12 vlen=9 -data: "c"/2.000000000,0 -> /BYTES/cput -data: "c"/1.000000000,0 -> /BYTES/value - -# Restart and retry cput. It should succeed. - -run trace ok -with t=A - txn_restart ts=3 - txn_step - cput k=c v=cput cond=value ----- ->> txn_restart ts=3 t=A -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=3.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=3.000000000,0 wto=false gul=0,0 ->> txn_step t=A -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=3.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=3.000000000,0 wto=false gul=0,0 ->> cput k=c v=cput cond=value t=A -called PutIntent("c", _, ExistingIntentInterleaved, TDNUM(true), 00000000-0000-0000-0000-000000000002) -meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=9 -data: "c"/3.000000000,0 -> /BYTES/cput -data: "c"/1.000000000,0 -> /BYTES/value diff --git a/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_disallow_separated b/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_disallow_separated index ff74edce0867..c79c4ff750b2 100644 --- a/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_disallow_separated +++ b/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_disallow_separated @@ -104,7 +104,7 @@ txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=3.000000000,0 min=0, >> txn_step t=A txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=3.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=3.000000000,0 wto=false gul=0,0 >> cput k=c v=cput cond=value t=A -called PutIntent("c", _, ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000002) +called PutIntent("c", _, ExistingIntentInterleaved, TDNUM(true), 00000000-0000-0000-0000-000000000002) meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=9 data: "c"/3.000000000,0 -> /BYTES/cput data: "c"/1.000000000,0 -> /BYTES/value diff --git a/pkg/storage/testdata/mvcc_histories/intent_history_allow_separated b/pkg/storage/testdata/mvcc_histories/intent_history_allow_separated deleted file mode 100644 index 3950dc62f08d..000000000000 --- a/pkg/storage/testdata/mvcc_histories/intent_history_allow_separated +++ /dev/null @@ -1,58 +0,0 @@ -## Write the base (default) value. - -run ok -with t=A - txn_begin ts=1 - put k=a v=default resolve - txn_remove ----- ->> at end: -data: "a"/1.000000000,0 -> /BYTES/default - -## See how the intent history evolves throughout the test. - -run trace ok -with t=A - txn_begin ts=2 - with k=a - put v=first - txn_step - put v=second - txn_step n=2 - del - txn_step n=6 - put v=first - resolve_intent ----- ->> txn_begin ts=2 t=A -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 ->> put v=first k=a t=A -called PutIntent("a", _, NoExistingIntent, TDNUM(true), 00000000-0000-0000-0000-000000000002) -meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=0} ts=2.000000000,0 del=false klen=12 vlen=10 -data: "a"/2.000000000,0 -> /BYTES/first -data: "a"/1.000000000,0 -> /BYTES/default ->> txn_step k=a t=A -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 ->> put v=second k=a t=A -called PutIntent("a", _, ExistingIntentInterleaved, TDNUM(true), 00000000-0000-0000-0000-000000000002) -meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=1} ts=2.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} -data: "a"/2.000000000,0 -> /BYTES/second -data: "a"/1.000000000,0 -> /BYTES/default ->> txn_step n=2 k=a t=A -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=3} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 ->> del k=a t=A -called PutIntent("a", _, ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000002) -meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=3} ts=2.000000000,0 del=true klen=12 vlen=0 ih={{0 /BYTES/first}{1 /BYTES/second}} -data: "a"/2.000000000,0 -> / -data: "a"/1.000000000,0 -> /BYTES/default ->> txn_step n=6 k=a t=A -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=9} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 ->> put v=first k=a t=A -called PutIntent("a", _, ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000002) -meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=9} ts=2.000000000,0 del=false klen=12 vlen=10 ih={{0 /BYTES/first}{1 /BYTES/second}{3 /}} -data: "a"/2.000000000,0 -> /BYTES/first -data: "a"/1.000000000,0 -> /BYTES/default ->> resolve_intent k=a t=A -called ClearIntent("a", ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000002) -data: "a"/2.000000000,0 -> /BYTES/first -data: "a"/1.000000000,0 -> /BYTES/default diff --git a/pkg/storage/testdata/mvcc_histories/intent_history_disallow_separated b/pkg/storage/testdata/mvcc_histories/intent_history_disallow_separated index 2c72c88eb80a..3950dc62f08d 100644 --- a/pkg/storage/testdata/mvcc_histories/intent_history_disallow_separated +++ b/pkg/storage/testdata/mvcc_histories/intent_history_disallow_separated @@ -34,7 +34,7 @@ data: "a"/1.000000000,0 -> /BYTES/default >> txn_step k=a t=A txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 >> put v=second k=a t=A -called PutIntent("a", _, ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000002) +called PutIntent("a", _, ExistingIntentInterleaved, TDNUM(true), 00000000-0000-0000-0000-000000000002) meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=1} ts=2.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} data: "a"/2.000000000,0 -> /BYTES/second data: "a"/1.000000000,0 -> /BYTES/default diff --git a/pkg/storage/testdata/mvcc_histories/intent_with_write_tracing_allow_separated b/pkg/storage/testdata/mvcc_histories/intent_with_write_tracing_allow_separated deleted file mode 100644 index dd76b908dcd0..000000000000 --- a/pkg/storage/testdata/mvcc_histories/intent_with_write_tracing_allow_separated +++ /dev/null @@ -1,87 +0,0 @@ -run trace ok -with t=A - txn_begin ts=2 - put k=k1 v=v1 ----- ->> txn_begin ts=2 t=A -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 ->> put k=k1 v=v1 t=A -called PutIntent("k1", _, NoExistingIntent, TDNUM(true), 00000000-0000-0000-0000-000000000001) -meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=0} ts=2.000000000,0 del=false klen=12 vlen=7 -data: "k1"/2.000000000,0 -> /BYTES/v1 - -run trace ok -with t=A - txn_advance ts=3 - txn_step - put k=k1 v=v1 - put k=k2 v=v2 ----- ->> txn_advance ts=3 t=A -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 ->> txn_step t=A -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 ->> put k=k1 v=v1 t=A -called PutIntent("k1", _, ExistingIntentInterleaved, TDNUM(true), 00000000-0000-0000-0000-000000000001) -meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} -data: "k1"/3.000000000,0 -> /BYTES/v1 ->> put k=k2 v=v2 t=A -called PutIntent("k2", _, NoExistingIntent, TDNUM(true), 00000000-0000-0000-0000-000000000001) -meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} -data: "k1"/3.000000000,0 -> /BYTES/v1 -meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 -data: "k2"/3.000000000,0 -> /BYTES/v2 - -run trace ok -put k=k3 v=v3 ts=1 ----- ->> put k=k3 v=v3 ts=1 -meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} -data: "k1"/3.000000000,0 -> /BYTES/v1 -meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 -data: "k2"/3.000000000,0 -> /BYTES/v2 -data: "k3"/1.000000000,0 -> /BYTES/v3 - -run trace ok -with t=A - put k=k3 v=v33 ----- ->> put k=k3 v=v33 t=A -called PutIntent("k3", _, NoExistingIntent, TDNUM(true), 00000000-0000-0000-0000-000000000001) -meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} -data: "k1"/3.000000000,0 -> /BYTES/v1 -meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 -data: "k2"/3.000000000,0 -> /BYTES/v2 -meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=8 -data: "k3"/3.000000000,0 -> /BYTES/v33 -data: "k3"/1.000000000,0 -> /BYTES/v3 - -# transactionDidNotUpdateMeta (TDNUM) is false below for k2 and k3 since -# disallowSeparatedIntents=true causes mvcc.go to always set it to false to maintain -# consistency in a mixed version cluster. -run trace ok -with t=A - resolve_intent k=k1 - resolve_intent k=k2 status=ABORTED - resolve_intent k=k3 status=ABORTED - txn_remove ----- ->> resolve_intent k=k1 t=A -called ClearIntent("k1", ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000001) -data: "k1"/3.000000000,0 -> /BYTES/v1 -meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 -data: "k2"/3.000000000,0 -> /BYTES/v2 -meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=8 -data: "k3"/3.000000000,0 -> /BYTES/v33 -data: "k3"/1.000000000,0 -> /BYTES/v3 ->> resolve_intent k=k2 status=ABORTED t=A -called ClearIntent("k2", ExistingIntentInterleaved, TDNUM(true), 00000000-0000-0000-0000-000000000001) -data: "k1"/3.000000000,0 -> /BYTES/v1 -meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=8 -data: "k3"/3.000000000,0 -> /BYTES/v33 -data: "k3"/1.000000000,0 -> /BYTES/v3 ->> resolve_intent k=k3 status=ABORTED t=A -called ClearIntent("k3", ExistingIntentInterleaved, TDNUM(true), 00000000-0000-0000-0000-000000000001) -data: "k1"/3.000000000,0 -> /BYTES/v1 -data: "k3"/1.000000000,0 -> /BYTES/v3 ->> txn_remove t=A diff --git a/pkg/storage/testdata/mvcc_histories/intent_with_write_tracing_disallow_separated b/pkg/storage/testdata/mvcc_histories/intent_with_write_tracing_disallow_separated index 0c796fdd99e7..dd76b908dcd0 100644 --- a/pkg/storage/testdata/mvcc_histories/intent_with_write_tracing_disallow_separated +++ b/pkg/storage/testdata/mvcc_histories/intent_with_write_tracing_disallow_separated @@ -22,7 +22,7 @@ txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0, >> txn_step t=A txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 >> put k=k1 v=v1 t=A -called PutIntent("k1", _, ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000001) +called PutIntent("k1", _, ExistingIntentInterleaved, TDNUM(true), 00000000-0000-0000-0000-000000000001) meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} data: "k1"/3.000000000,0 -> /BYTES/v1 >> put k=k2 v=v2 t=A @@ -75,13 +75,13 @@ meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000, data: "k3"/3.000000000,0 -> /BYTES/v33 data: "k3"/1.000000000,0 -> /BYTES/v3 >> resolve_intent k=k2 status=ABORTED t=A -called ClearIntent("k2", ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000001) +called ClearIntent("k2", ExistingIntentInterleaved, TDNUM(true), 00000000-0000-0000-0000-000000000001) data: "k1"/3.000000000,0 -> /BYTES/v1 meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=8 data: "k3"/3.000000000,0 -> /BYTES/v33 data: "k3"/1.000000000,0 -> /BYTES/v3 >> resolve_intent k=k3 status=ABORTED t=A -called ClearIntent("k3", ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000001) +called ClearIntent("k3", ExistingIntentInterleaved, TDNUM(true), 00000000-0000-0000-0000-000000000001) data: "k1"/3.000000000,0 -> /BYTES/v1 data: "k3"/1.000000000,0 -> /BYTES/v3 >> txn_remove t=A diff --git a/pkg/storage/testdata/mvcc_histories/no_read_after_abort_allow_separated b/pkg/storage/testdata/mvcc_histories/no_read_after_abort_allow_separated deleted file mode 100644 index e62ee5e72250..000000000000 --- a/pkg/storage/testdata/mvcc_histories/no_read_after_abort_allow_separated +++ /dev/null @@ -1,30 +0,0 @@ -## Simple txn that aborts. - -run trace ok -with t=A k=a - txn_begin ts=22 - put v=cde - resolve_intent status=ABORTED - txn_remove ----- ->> txn_begin ts=22 t=A k=a -txn: "A" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=22.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=22.000000000,0 wto=false gul=0,0 ->> put v=cde t=A k=a -called PutIntent("a", _, NoExistingIntent, TDNUM(true), 00000000-0000-0000-0000-000000000001) -meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=22.000000000,0 min=0,0 seq=0} ts=22.000000000,0 del=false klen=12 vlen=8 -data: "a"/22.000000000,0 -> /BYTES/cde ->> resolve_intent status=ABORTED t=A k=a -called ClearIntent("a", ExistingIntentInterleaved, TDNUM(true), 00000000-0000-0000-0000-000000000001) - ->> txn_remove t=A k=a - -# Cannot read aborted value. - -run ok -with t=A - txn_begin ts=23 - get k=a - txn_remove ----- -get: "a" -> ->> at end: diff --git a/pkg/storage/testdata/mvcc_histories/no_read_after_abort_disallow_separated b/pkg/storage/testdata/mvcc_histories/no_read_after_abort_disallow_separated index 6c04fa257eaa..e62ee5e72250 100644 --- a/pkg/storage/testdata/mvcc_histories/no_read_after_abort_disallow_separated +++ b/pkg/storage/testdata/mvcc_histories/no_read_after_abort_disallow_separated @@ -14,7 +14,7 @@ called PutIntent("a", _, NoExistingIntent, TDNUM(true), 00000000-0000-0000-0000- meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=22.000000000,0 min=0,0 seq=0} ts=22.000000000,0 del=false klen=12 vlen=8 data: "a"/22.000000000,0 -> /BYTES/cde >> resolve_intent status=ABORTED t=A k=a -called ClearIntent("a", ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000001) +called ClearIntent("a", ExistingIntentInterleaved, TDNUM(true), 00000000-0000-0000-0000-000000000001) >> txn_remove t=A k=a diff --git a/pkg/storage/testdata/mvcc_histories/read_after_write_allow_separated b/pkg/storage/testdata/mvcc_histories/read_after_write_allow_separated deleted file mode 100644 index 93247b9b82c5..000000000000 --- a/pkg/storage/testdata/mvcc_histories/read_after_write_allow_separated +++ /dev/null @@ -1,91 +0,0 @@ -## A simple txn that commits. - -run trace ok -with t=A - txn_begin ts=11 - with k=a - put v=abc - get - resolve_intent ----- ->> txn_begin ts=11 t=A -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=11.000000000,0 wto=false gul=0,0 ->> put v=abc k=a t=A -called PutIntent("a", _, NoExistingIntent, TDNUM(true), 00000000-0000-0000-0000-000000000001) -meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} ts=11.000000000,0 del=false klen=12 vlen=8 -data: "a"/11.000000000,0 -> /BYTES/abc ->> get k=a t=A -get: "a" -> /BYTES/abc @11.000000000,0 ->> resolve_intent k=a t=A -called ClearIntent("a", ExistingIntentInterleaved, TDNUM(true), 00000000-0000-0000-0000-000000000001) -data: "a"/11.000000000,0 -> /BYTES/abc - -run ok -with t=A resolve - put k=a/1 v=eee - put k=b v=fff - put k=b/2 v=ggg - put k=c v=hhh - txn_remove ----- ->> at end: -data: "a"/11.000000000,0 -> /BYTES/abc -data: "a/1"/11.000000000,0 -> /BYTES/eee -data: "b"/11.000000000,0 -> /BYTES/fff -data: "b/2"/11.000000000,0 -> /BYTES/ggg -data: "c"/11.000000000,0 -> /BYTES/hhh - -# Reads previous writes, transactional. - -run ok -with t=A - txn_begin ts=11 - get k=a ----- -get: "a" -> /BYTES/abc @11.000000000,0 ->> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=11.000000000,0 wto=false gul=0,0 - -run trace ok -with t=A - scan k=a end==b - scan k=a end=+a - scan k=a end=-a - scan k=a end=+b - scan k=a end==b - scan k=a end=-b - txn_remove ----- ->> scan k=a end==b t=A -scan: "a" -> /BYTES/abc @11.000000000,0 -scan: "a/1" -> /BYTES/eee @11.000000000,0 ->> scan k=a end=+a t=A -scan: "a" -> /BYTES/abc @11.000000000,0 ->> scan k=a end=-a t=A -scan: "a" -> /BYTES/abc @11.000000000,0 -scan: "a/1" -> /BYTES/eee @11.000000000,0 ->> scan k=a end=+b t=A -scan: "a" -> /BYTES/abc @11.000000000,0 -scan: "a/1" -> /BYTES/eee @11.000000000,0 -scan: "b" -> /BYTES/fff @11.000000000,0 ->> scan k=a end==b t=A -scan: "a" -> /BYTES/abc @11.000000000,0 -scan: "a/1" -> /BYTES/eee @11.000000000,0 ->> scan k=a end=-b t=A -scan: "a" -> /BYTES/abc @11.000000000,0 -scan: "a/1" -> /BYTES/eee @11.000000000,0 -scan: "b" -> /BYTES/fff @11.000000000,0 -scan: "b/2" -> /BYTES/ggg @11.000000000,0 ->> txn_remove t=A - - -## A simple txn anchored at some arbitrary key. - -run trace ok -with t=A k=a - txn_begin ts=1 - txn_remove ----- ->> txn_begin ts=1 t=A k=a -txn: "A" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=1.000000000,0 wto=false gul=0,0 ->> txn_remove t=A k=a diff --git a/pkg/storage/testdata/mvcc_histories/read_after_write_disallow_separated b/pkg/storage/testdata/mvcc_histories/read_after_write_disallow_separated index 4ee072133cd0..93247b9b82c5 100644 --- a/pkg/storage/testdata/mvcc_histories/read_after_write_disallow_separated +++ b/pkg/storage/testdata/mvcc_histories/read_after_write_disallow_separated @@ -17,7 +17,7 @@ data: "a"/11.000000000,0 -> /BYTES/abc >> get k=a t=A get: "a" -> /BYTES/abc @11.000000000,0 >> resolve_intent k=a t=A -called ClearIntent("a", ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000001) +called ClearIntent("a", ExistingIntentInterleaved, TDNUM(true), 00000000-0000-0000-0000-000000000001) data: "a"/11.000000000,0 -> /BYTES/abc run ok 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/localtestcluster/local_test_cluster.go b/pkg/testutils/localtestcluster/local_test_cluster.go index c8aa27643cbe..46175c467373 100644 --- a/pkg/testutils/localtestcluster/local_test_cluster.go +++ b/pkg/testutils/localtestcluster/local_test_cluster.go @@ -139,7 +139,7 @@ func (ltc *LocalTestCluster) Start(t testing.TB, baseCtx *base.Config, initFacto storage.InMemory(), storage.CacheSize(0), storage.MaxSize(50<<20 /* 50 MiB */), - storage.SettingsForTesting()) + storage.ForTesting) if err != nil { t.Fatal(err) } diff --git a/pkg/testutils/testcluster/testcluster.go b/pkg/testutils/testcluster/testcluster.go index a3ee0ed7c563..4ee0e409ad3d 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) } }