diff --git a/docs/tech-notes/mvcc-range-tombstones.md b/docs/tech-notes/mvcc-range-tombstones.md index 402778a5c358..e5f07087d9fa 100644 --- a/docs/tech-notes/mvcc-range-tombstones.md +++ b/docs/tech-notes/mvcc-range-tombstones.md @@ -22,6 +22,9 @@ not yet been upgraded to support them (in the NB: MVCC range tombstones are not yet supported in transactions, since that would require ranged write intents. +NB: The `storage.mvcc.range_tombstones.enabled` cluster setting and the +`storage.CanUseMVCCRangeTombstones()` helper were removed in 23.1. + ## Motivation Previously, certain operations would rely on the non-MVCC methods `ClearRange` @@ -638,4 +641,4 @@ completeness: Related RFCs: * [Streaming Replication Between Clusters RFC](https://github.com/cockroachdb/cockroach/blob/master/docs/RFCS/20201119_streaming_cluster_to_cluster.md) -* [RFC Draft - Import Rollbacks without MVCC Timestamps](https://docs.google.com/document/d/16TbkFznqbsu3mialSw6o1sxOEn1pKhhJ9HTxNdw0-WE/edit#heading=h.bpox0bmkz77i) \ No newline at end of file +* [RFC Draft - Import Rollbacks without MVCC Timestamps](https://docs.google.com/document/d/16TbkFznqbsu3mialSw6o1sxOEn1pKhhJ9HTxNdw0-WE/edit#heading=h.bpox0bmkz77i) diff --git a/pkg/ccl/backupccl/tenant_backup_nemesis_test.go b/pkg/ccl/backupccl/tenant_backup_nemesis_test.go index d39e1eab8349..dc202351ff6b 100644 --- a/pkg/ccl/backupccl/tenant_backup_nemesis_test.go +++ b/pkg/ccl/backupccl/tenant_backup_nemesis_test.go @@ -71,9 +71,6 @@ func TestTenantBackupWithCanceledImport(t *testing.T) { ) defer hostClusterCleanupFn() - hostSQLDB.Exec(t, "SET CLUSTER SETTING storage.mvcc.range_tombstones.enabled = true") - hostSQLDB.Exec(t, "ALTER TENANT ALL SET CLUSTER SETTING storage.mvcc.range_tombstones.enabled = true") - tenant10, err := tc.Servers[0].StartTenant(ctx, base.TestTenantArgs{ TenantID: roachpb.MustMakeTenantID(10), TestingKnobs: base.TestingKnobs{ @@ -146,10 +143,6 @@ func TestTenantBackupNemesis(t *testing.T) { ) defer hostClusterCleanupFn() - // Range tombstones must be enabled for tenant backups to work correctly. - hostSQLDB.Exec(t, "SET CLUSTER SETTING storage.mvcc.range_tombstones.enabled = true") - hostSQLDB.Exec(t, "ALTER TENANT ALL SET CLUSTER SETTING storage.mvcc.range_tombstones.enabled = true") - tenant10, err := tc.Servers[0].StartTenant(ctx, base.TestTenantArgs{ TenantID: roachpb.MustMakeTenantID(10), TestingKnobs: base.TestingKnobs{ diff --git a/pkg/ccl/backupccl/testdata/backup-restore/in-progress-import-rollback b/pkg/ccl/backupccl/testdata/backup-restore/in-progress-import-rollback index 5cd89c3e29ab..6ef1bbd11c53 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/in-progress-import-rollback +++ b/pkg/ccl/backupccl/testdata/backup-restore/in-progress-import-rollback @@ -7,9 +7,7 @@ # - roll it back it back non-mvcc # - run an inc backup and ensure we reintroduce the table spans -# disabled to run within tenant as they don't have access to the -# storage.mvcc.range_tombstones.enabled cluster setting -new-cluster name=s1 disable-tenant +new-cluster name=s1 ---- ########### @@ -36,12 +34,6 @@ exec-sql SET CLUSTER SETTING kv.bulkio.write_metadata_sst.enabled = false; ---- - -exec-sql -SET CLUSTER SETTING storage.mvcc.range_tombstones.enabled = false; ----- - - exec-sql EXPORT INTO CSV 'nodelocal://0/export1/' FROM SELECT * FROM baz; ---- @@ -243,10 +235,6 @@ SET CLUSTER SETTING jobs.debug.pausepoints = 'import.after_ingest'; ---- -exec-sql -SET CLUSTER SETTING storage.mvcc.range_tombstones.enabled = true; ----- - exec-sql SET CLUSTER SETTING kv.bulkio.write_metadata_sst.enabled = false; ---- diff --git a/pkg/ccl/backupccl/testdata/backup-restore/restore-on-fail-or-cancel-fast-drop b/pkg/ccl/backupccl/testdata/backup-restore/restore-on-fail-or-cancel-fast-drop index 69acb8222eb2..99d65dd02e1c 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/restore-on-fail-or-cancel-fast-drop +++ b/pkg/ccl/backupccl/testdata/backup-restore/restore-on-fail-or-cancel-fast-drop @@ -19,10 +19,6 @@ BACKUP INTO 'nodelocal://1/cluster_backup'; new-cluster name=s2 nodes=1 share-io-dir=s1 disable-tenant ---- -exec-sql -SET CLUSTER SETTING storage.mvcc.range_tombstones.enabled = true; ----- - # Restore's OnFailOrCancel deletes descriptors which requires us to wait for no # versions of that descriptor to be leased before proceeding. Since our test fails # the job after the descriptors have been published, it's possible for them to be leased diff --git a/pkg/ccl/backupccl/testdata/backup-restore/restore-on-fail-or-cancel-retry b/pkg/ccl/backupccl/testdata/backup-restore/restore-on-fail-or-cancel-retry index 14a927808ba6..e97b2b93cc2d 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/restore-on-fail-or-cancel-retry +++ b/pkg/ccl/backupccl/testdata/backup-restore/restore-on-fail-or-cancel-retry @@ -1,5 +1,4 @@ -# disabled for tenants as they can't set storage.mvcc.range_tombstones.enabled -new-cluster name=s1 nodes=1 disable-tenant +new-cluster name=s1 nodes=1 ---- subtest restore-retry @@ -20,10 +19,6 @@ BACKUP INTO 'nodelocal://1/cluster_backup'; new-cluster name=s2 nodes=1 share-io-dir=s1 disable-tenant ---- -exec-sql -SET CLUSTER SETTING storage.mvcc.range_tombstones.enabled = true; ----- - exec-sql SELECT crdb_internal.set_vmodule('lease=3'); ---- diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index 593d16301426..10d28a5783e1 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -3061,7 +3061,6 @@ func TestChangefeedFailOnTableOffline(t *testing.T) { cdcTestNamedWithSystem(t, "reverted import fails changefeed with earlier cursor", func(t *testing.T, s TestServerWithSystem, f cdctest.TestFeedFactory) { sysSQLDB := sqlutils.MakeSQLRunner(s.SystemDB) sysSQLDB.Exec(t, "SET CLUSTER SETTING kv.bulk_io_write.small_write_size = '1'") - sysSQLDB.Exec(t, "SET CLUSTER SETTING storage.mvcc.range_tombstones.enabled = true") sqlDB := sqlutils.MakeSQLRunner(s.DB) sqlDB.Exec(t, `CREATE TABLE for_import (a INT PRIMARY KEY, b INT)`) diff --git a/pkg/cmd/roachtest/tests/backup.go b/pkg/cmd/roachtest/tests/backup.go index a106ddab50bc..95c6f6bdfdf9 100644 --- a/pkg/cmd/roachtest/tests/backup.go +++ b/pkg/cmd/roachtest/tests/backup.go @@ -1145,8 +1145,6 @@ func runBackupMVCCRangeTombstones(ctx context.Context, t test.Test, c cluster.Cl t.Status("configuring cluster") _, err := conn.Exec(`SET CLUSTER SETTING kv.bulk_ingest.max_index_buffer_size = '2gb'`) require.NoError(t, err) - _, err = conn.Exec(`SET CLUSTER SETTING storage.mvcc.range_tombstones.enabled = 't'`) - require.NoError(t, err) _, err = conn.Exec(`SET CLUSTER SETTING server.debug.default_vmodule = 'txn=2,sst_batcher=4, revert=2'`) require.NoError(t, err) diff --git a/pkg/cmd/roachtest/tests/clearrange.go b/pkg/cmd/roachtest/tests/clearrange.go index 6e622cd71bbb..dd3950fe7adb 100644 --- a/pkg/cmd/roachtest/tests/clearrange.go +++ b/pkg/cmd/roachtest/tests/clearrange.go @@ -27,63 +27,43 @@ import ( func registerClearRange(r registry.Registry) { for _, checks := range []bool{true, false} { - for _, rangeTombstones := range []bool{true, false} { - checks := checks - rangeTombstones := rangeTombstones - r.Add(registry.TestSpec{ - Name: fmt.Sprintf(`clearrange/checks=%t/rangeTs=%t`, checks, rangeTombstones), - Owner: registry.OwnerStorage, - // 5h for import, 90 for the test. The import should take closer - // to <3:30h but it varies. - Timeout: 5*time.Hour + 90*time.Minute, - Cluster: r.MakeClusterSpec(10, spec.CPU(16)), - Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { - runClearRange(ctx, t, c, checks, rangeTombstones) - }, - }) - - // Using a separate clearrange test on zfs instead of randomly - // using the same test, cause the Timeout might be different, - // and may need to be tweaked. - r.Add(registry.TestSpec{ - Name: fmt.Sprintf(`clearrange/zfs/checks=%t/rangeTs=%t`, checks, rangeTombstones), - Skip: "Consistently failing. See #68716 context.", - Owner: registry.OwnerStorage, - // 5h for import, 120 for the test. The import should take closer - // to <3:30h but it varies. - Timeout: 5*time.Hour + 120*time.Minute, - Cluster: r.MakeClusterSpec(10, spec.CPU(16), spec.SetFileSystem(spec.Zfs)), - EncryptionSupport: registry.EncryptionMetamorphic, - Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { - runClearRange(ctx, t, c, checks, rangeTombstones) - }, - }) - } + checks := checks + r.Add(registry.TestSpec{ + Name: fmt.Sprintf(`clearrange/checks=%t`, checks), + Owner: registry.OwnerStorage, + // 5h for import, 90 for the test. The import should take closer + // to <3:30h but it varies. + Timeout: 5*time.Hour + 90*time.Minute, + Cluster: r.MakeClusterSpec(10, spec.CPU(16)), + Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { + runClearRange(ctx, t, c, checks) + }, + }) + + // Using a separate clearrange test on zfs instead of randomly + // using the same test, cause the Timeout might be different, + // and may need to be tweaked. + r.Add(registry.TestSpec{ + Name: fmt.Sprintf(`clearrange/zfs/checks=%t`, checks), + Skip: "Consistently failing. See #68716 context.", + Owner: registry.OwnerStorage, + // 5h for import, 120 for the test. The import should take closer + // to <3:30h but it varies. + Timeout: 5*time.Hour + 120*time.Minute, + Cluster: r.MakeClusterSpec(10, spec.CPU(16), spec.SetFileSystem(spec.Zfs)), + EncryptionSupport: registry.EncryptionMetamorphic, + Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { + runClearRange(ctx, t, c, checks) + }, + }) } } - -func runClearRange( - ctx context.Context, - t test.Test, - c cluster.Cluster, - aggressiveChecks bool, - useRangeTombstones bool, -) { +func runClearRange(ctx context.Context, t test.Test, c cluster.Cluster, aggressiveChecks bool) { c.Put(ctx, t.Cockroach(), "./cockroach") t.Status("restoring fixture") c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings()) - { - conn := c.Conn(ctx, t.L(), 1) - if _, err := conn.ExecContext(ctx, - `SET CLUSTER SETTING storage.mvcc.range_tombstones.enabled = $1`, - useRangeTombstones); err != nil { - t.Fatal(err) - } - conn.Close() - } - // NB: on a 10 node cluster, this should take well below 3h. tBegin := timeutil.Now() c.Run(ctx, c.Node(1), "./cockroach", "workload", "fixtures", "import", "bank", diff --git a/pkg/cmd/roachtest/tests/import_cancellation.go b/pkg/cmd/roachtest/tests/import_cancellation.go index f854b144a83b..efd0ef3ca4d5 100644 --- a/pkg/cmd/roachtest/tests/import_cancellation.go +++ b/pkg/cmd/roachtest/tests/import_cancellation.go @@ -29,22 +29,18 @@ import ( ) func registerImportCancellation(r registry.Registry) { - for _, rangeTombstones := range []bool{true, false} { - r.Add(registry.TestSpec{ - Name: fmt.Sprintf(`import-cancellation/rangeTs=%t`, rangeTombstones), - Owner: registry.OwnerDisasterRecovery, - Timeout: 4 * time.Hour, - Cluster: r.MakeClusterSpec(6, spec.CPU(32)), - Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { - runImportCancellation(ctx, t, c, rangeTombstones) - }, - }) - } + r.Add(registry.TestSpec{ + Name: `import-cancellation`, + Owner: registry.OwnerDisasterRecovery, + Timeout: 4 * time.Hour, + Cluster: r.MakeClusterSpec(6, spec.CPU(32)), + Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { + runImportCancellation(ctx, t, c) + }, + }) } -func runImportCancellation( - ctx context.Context, t test.Test, c cluster.Cluster, rangeTombstones bool, -) { +func runImportCancellation(ctx context.Context, t test.Test, c cluster.Cluster) { c.Put(ctx, t.Cockroach(), "./cockroach") c.Put(ctx, t.DeprecatedWorkload(), "./workload") // required for tpch c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings()) @@ -76,15 +72,6 @@ func runImportCancellation( if _, err := conn.Exec(`SET CLUSTER SETTING kv.bulk_ingest.max_index_buffer_size = '2gb'`); err != nil { t.Fatal(err) } - // Enable MVCC Range tombstones, if required. - rtEnable := "f" - if rangeTombstones { - rtEnable = "t" - } - stmt := fmt.Sprintf(`SET CLUSTER SETTING storage.mvcc.range_tombstones.enabled = '%s'`, rtEnable) - if _, err := conn.Exec(stmt); err != nil { - t.Fatal(err) - } // Increase AddSSTable concurrency to speed up the imports. Otherwise the // lineitem (the largest tpch table) IMPORT will extend the test duration // significantly. diff --git a/pkg/cmd/roachtest/tests/mvcc_gc.go b/pkg/cmd/roachtest/tests/mvcc_gc.go index a3c8eb33d2df..9d80556f94d2 100644 --- a/pkg/cmd/roachtest/tests/mvcc_gc.go +++ b/pkg/cmd/roachtest/tests/mvcc_gc.go @@ -93,7 +93,6 @@ func runMVCCGC(ctx context.Context, t test.Test, c cluster.Cluster) { execSQLOrFail(fmt.Sprintf(`SET CLUSTER SETTING %s = $1`, name), value) } - setClusterSetting("storage.mvcc.range_tombstones.enabled", true) setClusterSetting("kv.protectedts.poll_interval", "5s") setClusterSetting("kv.mvcc_gc.queue_interval", "0s") diff --git a/pkg/settings/registry.go b/pkg/settings/registry.go index 05844f866d32..a1736e1bedc4 100644 --- a/pkg/settings/registry.go +++ b/pkg/settings/registry.go @@ -159,6 +159,7 @@ var retiredSettings = map[string]struct{}{ "sql.distsql.drain.cancel_after_wait.enabled": {}, "changefeed.active_protected_timestamps.enabled": {}, "jobs.scheduler.single_node_scheduler.enabled": {}, + "storage.mvcc.range_tombstones.enabled": {}, // renamed. "spanconfig.host_coalesce_adjacent.enabled": {}, "sql.defaults.experimental_stream_replication.enabled": {}, diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 4ae32ee2abe4..caf1e6ac0751 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -485,7 +485,6 @@ go_library( "//pkg/sql/syntheticprivilegecache", "//pkg/sql/types", "//pkg/sql/vtable", - "//pkg/storage", "//pkg/storage/enginepb", "//pkg/testutils/serverutils", "//pkg/upgrade", diff --git a/pkg/sql/gcjob/BUILD.bazel b/pkg/sql/gcjob/BUILD.bazel index a3ffd1532e86..cbc966ebda03 100644 --- a/pkg/sql/gcjob/BUILD.bazel +++ b/pkg/sql/gcjob/BUILD.bazel @@ -40,7 +40,6 @@ go_library( "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", "//pkg/sql/sqlerrors", - "//pkg/storage", "//pkg/util/admission/admissionpb", "//pkg/util/hlc", "//pkg/util/log", diff --git a/pkg/sql/gcjob/gc_job.go b/pkg/sql/gcjob/gc_job.go index c20ef5de7859..202bd88f5851 100644 --- a/pkg/sql/gcjob/gc_job.go +++ b/pkg/sql/gcjob/gc_job.go @@ -28,7 +28,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/isql" - "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" @@ -307,7 +306,7 @@ func (r schemaChangeGCResumer) Resume(ctx context.Context, execCtx interface{}) return err } - if !shouldUseDelRange(ctx, details, execCfg.Settings, execCfg.GCJobTestingKnobs) { + if !shouldUseDelRange(ctx, details, execCfg.Settings) { return r.legacyWaitAndClearTableData(ctx, execCfg, details, progress) } return r.deleteDataAndWaitForGC(ctx, execCfg, details, progress) @@ -432,7 +431,7 @@ func (r schemaChangeGCResumer) legacyWaitAndClearTableData( // Now that we've registered to be notified, check to see if we raced // with the new version becoming active. - if shouldUseDelRange(ctx, details, execCfg.Settings, execCfg.GCJobTestingKnobs) { + if shouldUseDelRange(ctx, details, execCfg.Settings) { return r.deleteDataAndWaitForGC(ctx, execCfg, details, progress) } @@ -448,9 +447,7 @@ func (r schemaChangeGCResumer) legacyWaitAndClearTableData( } // We'll be notified if the new version becomes active, so check and // see if it's now time to change to the new protocol. - if shouldUseDelRange( - ctx, details, execCfg.Settings, execCfg.GCJobTestingKnobs, - ) { + if shouldUseDelRange(ctx, details, execCfg.Settings) { return r.deleteDataAndWaitForGC(ctx, execCfg, details, progress) } @@ -500,17 +497,11 @@ func (r schemaChangeGCResumer) legacyWaitAndClearTableData( } func shouldUseDelRange( - ctx context.Context, - details *jobspb.SchemaChangeGCDetails, - s *cluster.Settings, - knobs *sql.GCJobTestingKnobs, + ctx context.Context, details *jobspb.SchemaChangeGCDetails, s *cluster.Settings, ) bool { // TODO(ajwerner): Adopt the DeleteRange protocol for tenant GC. return details.Tenant == nil && - s.Version.IsActive(ctx, clusterversion.TODODelete_V22_2UseDelRangeInGCJob) && - (storage.CanUseMVCCRangeTombstones(ctx, s) || - // Allow this testing knob to override the storage setting, for convenience. - knobs.SkipWaitingForMVCCGC) + s.Version.IsActive(ctx, clusterversion.TODODelete_V22_2UseDelRangeInGCJob) } // waitForWork waits until there is work to do given the gossipUpDateC, the diff --git a/pkg/sql/gcjob/gcjobnotifier/BUILD.bazel b/pkg/sql/gcjob/gcjobnotifier/BUILD.bazel index cf0441cd8fa3..d80e5c62d781 100644 --- a/pkg/sql/gcjob/gcjobnotifier/BUILD.bazel +++ b/pkg/sql/gcjob/gcjobnotifier/BUILD.bazel @@ -13,7 +13,6 @@ go_library( "//pkg/keys", "//pkg/roachpb", "//pkg/settings/cluster", - "//pkg/storage", "//pkg/util/log/logcrash", "//pkg/util/stop", "//pkg/util/syncutil", diff --git a/pkg/sql/gcjob/gcjobnotifier/notifier.go b/pkg/sql/gcjob/gcjobnotifier/notifier.go index 1e2a5e86f80f..abb9357ec885 100644 --- a/pkg/sql/gcjob/gcjobnotifier/notifier.go +++ b/pkg/sql/gcjob/gcjobnotifier/notifier.go @@ -23,7 +23,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -155,21 +154,12 @@ func (n *Notifier) run(_ context.Context) { versionSettingChanged <- struct{}{} } }) - tombstonesEnableChanges := make(chan struct{}, 1) - storage.MVCCRangeTombstonesEnabled.SetOnChange(&n.settings.SV, func(ctx context.Context) { - select { - case tombstonesEnableChanges <- struct{}{}: - default: - } - }) for { select { case <-n.stopper.ShouldQuiesce(): return case <-versionSettingChanged: n.notify() - case <-tombstonesEnableChanges: - n.notify() case <-systemConfigUpdateCh: n.maybeNotify() } diff --git a/pkg/sql/gcjob_test/BUILD.bazel b/pkg/sql/gcjob_test/BUILD.bazel index 40e297c9d2d3..fa80068145b0 100644 --- a/pkg/sql/gcjob_test/BUILD.bazel +++ b/pkg/sql/gcjob_test/BUILD.bazel @@ -36,7 +36,6 @@ go_test( "//pkg/sql/gcjob/gcjobnotifier", "//pkg/sql/isql", "//pkg/sql/sem/catid", - "//pkg/storage", "//pkg/testutils", "//pkg/testutils/jobutils", "//pkg/testutils/serverutils", diff --git a/pkg/sql/gcjob_test/gc_job_test.go b/pkg/sql/gcjob_test/gc_job_test.go index 8ce019a52492..1df7e20785ed 100644 --- a/pkg/sql/gcjob_test/gc_job_test.go +++ b/pkg/sql/gcjob_test/gc_job_test.go @@ -42,7 +42,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/gcjob/gcjobnotifier" "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" - "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/jobutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -278,7 +277,6 @@ func TestGCJobRetry(t *testing.T) { failed.Store(false) cs := cluster.MakeTestingClusterSettings() gcjob.EmptySpanPollInterval.Override(ctx, &cs.SV, 100*time.Millisecond) - storage.MVCCRangeTombstonesEnabled.Override(ctx, &cs.SV, true) params := base.TestServerArgs{Settings: cs} params.Knobs.JobsTestingKnobs = jobs.NewTestingKnobsWithShortIntervals() params.Knobs.Store = &kvserver.StoreTestingKnobs{ diff --git a/pkg/sql/importer/BUILD.bazel b/pkg/sql/importer/BUILD.bazel index df61b5f80c58..38d0d651bd15 100644 --- a/pkg/sql/importer/BUILD.bazel +++ b/pkg/sql/importer/BUILD.bazel @@ -91,7 +91,6 @@ go_library( "//pkg/sql/sqltelemetry", "//pkg/sql/stats", "//pkg/sql/types", - "//pkg/storage", "//pkg/util", "//pkg/util/bitarray", "//pkg/util/bufalloc", @@ -213,7 +212,6 @@ go_test( "//pkg/sql/stats", "//pkg/sql/tests", "//pkg/sql/types", - "//pkg/storage", "//pkg/testutils", "//pkg/testutils/datapathutils", "//pkg/testutils/jobutils", diff --git a/pkg/sql/importer/import_job.go b/pkg/sql/importer/import_job.go index ff7cf7e4027c..9244efbc1c4f 100644 --- a/pkg/sql/importer/import_job.go +++ b/pkg/sql/importer/import_job.go @@ -46,7 +46,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/stats" - "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/ioctx" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -1445,8 +1444,6 @@ func (r *importResumer) dropTables( return nil } - useDeleteRange := storage.CanUseMVCCRangeTombstones(ctx, r.settings) - var tableWasEmpty bool var intoTable catalog.TableDescriptor for _, tbl := range details.Tables { @@ -1483,46 +1480,22 @@ func (r *importResumer) dropTables( // it was rolled back to its pre-IMPORT state, and instead provide a manual // admin knob (e.g. ALTER TABLE REVERT TO SYSTEM TIME) if anything goes wrong. ts := hlc.Timestamp{WallTime: details.Walltime}.Prev() - if useDeleteRange { - predicates := roachpb.DeleteRangePredicates{StartTime: ts} - if err := sql.DeleteTableWithPredicate( - ctx, - execCfg.DB, - execCfg.Codec, - &execCfg.Settings.SV, - execCfg.DistSender, - intoTable, - predicates, sql.RevertTableDefaultBatchSize); err != nil { - return errors.Wrap(err, "rolling back IMPORT INTO in non empty table via DeleteRange") - } - } else { - // disallowShadowingBelow=writeTS used to write means no existing keys could - // have been covered by a key imported and the table was offline to other - // writes, so even if GC has run it would not have GC'ed any keys to which - // we need to revert, so we can safely ignore the target-time GC check. - const ignoreGC = true - if err := sql.RevertTables(ctx, txn.KV().DB(), execCfg, []catalog.TableDescriptor{intoTable}, ts, ignoreGC, - sql.RevertTableDefaultBatchSize); err != nil { - return errors.Wrap(err, "rolling back partially completed IMPORT via RevertRange") - } + predicates := roachpb.DeleteRangePredicates{StartTime: ts} + if err := sql.DeleteTableWithPredicate( + ctx, + execCfg.DB, + execCfg.Codec, + &execCfg.Settings.SV, + execCfg.DistSender, + intoTable, + predicates, sql.RevertTableDefaultBatchSize); err != nil { + return errors.Wrap(err, "rolling back IMPORT INTO in non empty table via DeleteRange") } } else if tableWasEmpty { - if useDeleteRange { - if err := gcjob.DeleteAllTableData( - ctx, execCfg.DB, execCfg.DistSender, execCfg.Codec, intoTable, - ); err != nil { - return errors.Wrap(err, "rolling back IMPORT INTO in empty table via DeleteRange") - } - } else { - // Set a DropTime on the table descriptor to differentiate it from an - // older-format (v1.1) descriptor. This enables ClearTableData to use a - // RangeClear for faster data removal, rather than removing by chunks. - intoTable.TableDesc().DropTime = int64(1) - if err := gcjob.ClearTableData( - ctx, execCfg.DB, execCfg.DistSender, execCfg.Codec, &execCfg.Settings.SV, intoTable, - ); err != nil { - return errors.Wrapf(err, "rolling back IMPORT INTO in empty table via ClearRange") - } + if err := gcjob.DeleteAllTableData( + ctx, execCfg.DB, execCfg.DistSender, execCfg.Codec, intoTable, + ); err != nil { + return errors.Wrap(err, "rolling back IMPORT INTO in empty table via DeleteRange") } } diff --git a/pkg/sql/importer/import_stmt_test.go b/pkg/sql/importer/import_stmt_test.go index c685cdb092e2..1facd0be4f17 100644 --- a/pkg/sql/importer/import_stmt_test.go +++ b/pkg/sql/importer/import_stmt_test.go @@ -61,7 +61,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/stats" "github.com/cockroachdb/cockroach/pkg/sql/tests" - "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" "github.com/cockroachdb/cockroach/pkg/testutils/jobutils" @@ -129,7 +128,6 @@ ORDER BY table_name sqlDB := sqlutils.MakeSQLRunner(db) sqlDB.Exec(t, `SET CLUSTER SETTING kv.bulk_ingest.batch_size = '10KB'`) - sqlDB.Exec(t, `SET CLUSTER SETTING storage.mvcc.range_tombstones.enabled = true`) tests := []struct { name string @@ -2060,9 +2058,6 @@ func TestFailedImportGC(t *testing.T) { kvDB := tc.Server(0).DB() sqlDB.Exec(t, `SET CLUSTER SETTING kv.bulk_ingest.batch_size = '10KB'`) - // The test assumes we'll use the MVCC range tombstone in the GC job. We need - // to set this cluster setting to make that true. - sqlDB.Exec(t, `SET CLUSTER SETTING storage.mvcc.range_tombstones.enabled = true`) forceFailure = true defer func() { forceFailure = false }() @@ -6260,7 +6255,6 @@ func TestImportPgDumpSchemas(t *testing.T) { baseDir := datapathutils.TestDataPath(t, "pgdump") mkArgs := func() base.TestServerArgs { s := cluster.MakeTestingClusterSettings() - storage.MVCCRangeTombstonesEnabled.Override(ctx, &s.SV, true) return base.TestServerArgs{ Settings: s, ExternalIODir: baseDir, diff --git a/pkg/sql/repair.go b/pkg/sql/repair.go index 65a726b473a2..6cf810eb2eb0 100644 --- a/pkg/sql/repair.go +++ b/pkg/sql/repair.go @@ -38,7 +38,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" "github.com/cockroachdb/cockroach/pkg/sql/privilege" - "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/ioctx" @@ -741,8 +740,7 @@ func (p *planner) ForceDeleteTableData(ctx context.Context, descID int64) error Key: tableSpan.Key, EndKey: tableSpan.EndKey, } b := &kv.Batch{} - if p.execCfg.Settings.Version.IsActive(ctx, clusterversion.TODODelete_V22_2UseDelRangeInGCJob) && - storage.CanUseMVCCRangeTombstones(ctx, p.execCfg.Settings) { + if p.execCfg.Settings.Version.IsActive(ctx, clusterversion.TODODelete_V22_2UseDelRangeInGCJob) { b.AddRawRequest(&roachpb.DeleteRangeRequest{ RequestHeader: requestHeader, UseRangeTombstone: true, diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 91e5c56c783d..d2c9a28ee6e6 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -22,7 +22,6 @@ import ( "sync" "time" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" @@ -74,30 +73,6 @@ var minWALSyncInterval = settings.RegisterDurationSetting( 0*time.Millisecond, ) -// MVCCRangeTombstonesEnabled enables writing of MVCC range tombstones. -// Currently, this is used for schema GC and import cancellation rollbacks. -// -// Note that any executing jobs may not pick up this change, so these need to be -// waited out before being certain that the setting has taken effect. -// -// If disabled after being enabled, this will prevent new range tombstones from -// being written, but already written tombstones will remain until GCed. The -// above note on jobs also applies in this case. -var MVCCRangeTombstonesEnabled = settings.RegisterBoolSetting( - settings.TenantReadOnly, - "storage.mvcc.range_tombstones.enabled", - "enables the use of MVCC range tombstones", - true) - -// CanUseMVCCRangeTombstones returns true if the caller can begin writing -// MVCC range tombstones, by setting DeleteRangeRequest.UseRangeTombstone. -// It requires the MVCCRangeTombstones version gate to be active, and the -// setting storage.mvcc.range_tombstones.enabled to be enabled. -func CanUseMVCCRangeTombstones(ctx context.Context, st *cluster.Settings) bool { - return st.Version.IsActive(ctx, clusterversion.TODODelete_V22_2MVCCRangeTombstones) && - MVCCRangeTombstonesEnabled.Get(&st.SV) -} - // MaxIntentsPerWriteIntentError sets maximum number of intents returned in // WriteIntentError in operations that return multiple intents per error. // Currently it is used in Scan, ReverseScan, and ExportToSST. diff --git a/pkg/upgrade/upgrades/BUILD.bazel b/pkg/upgrade/upgrades/BUILD.bazel index 717b8e363e35..798594aa532a 100644 --- a/pkg/upgrade/upgrades/BUILD.bazel +++ b/pkg/upgrade/upgrades/BUILD.bazel @@ -67,7 +67,6 @@ go_library( "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", "//pkg/sql/types", - "//pkg/storage", "//pkg/upgrade", "//pkg/upgrade/upgradebase", "//pkg/util/hlc", @@ -151,7 +150,6 @@ go_test( "//pkg/sql/sem/tree", "//pkg/sql/tests", "//pkg/sql/types", - "//pkg/storage", "//pkg/testutils", "//pkg/testutils/serverutils", "//pkg/testutils/skip", diff --git a/pkg/upgrade/upgrades/wait_for_del_range_in_gc_job.go b/pkg/upgrade/upgrades/wait_for_del_range_in_gc_job.go index ce0e7bc9241a..393cfb0cb1cd 100644 --- a/pkg/upgrade/upgrades/wait_for_del_range_in_gc_job.go +++ b/pkg/upgrade/upgrades/wait_for_del_range_in_gc_job.go @@ -21,7 +21,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" - "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/upgrade" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/retry" @@ -34,9 +33,6 @@ func waitForDelRangeInGCJob( ctx context.Context, _ clusterversion.ClusterVersion, deps upgrade.TenantDeps, ) error { for r := retry.StartWithCtx(ctx, retry.Options{}); r.Next(); { - if !storage.CanUseMVCCRangeTombstones(ctx, deps.Settings) { - return nil - } jobIDs, err := collectJobIDsFromQuery( ctx, deps.InternalExecutor, "wait-for-gc-job-upgrades", ` WITH jobs AS ( diff --git a/pkg/upgrade/upgrades/wait_for_del_range_in_gc_job_test.go b/pkg/upgrade/upgrades/wait_for_del_range_in_gc_job_test.go index 00fd9e7a584a..4b837f5a28f4 100644 --- a/pkg/upgrade/upgrades/wait_for_del_range_in_gc_job_test.go +++ b/pkg/upgrade/upgrades/wait_for_del_range_in_gc_job_test.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/server" "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/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -40,7 +39,6 @@ func TestWaitForDelRangeInGCJob(t *testing.T) { ctx := context.Background() settings := cluster.MakeTestingClusterSettingsWithVersions(v1, v0, false /* initializeVersion */) require.NoError(t, clusterversion.Initialize(ctx, v0, &settings.SV)) - storage.MVCCRangeTombstonesEnabled.Override(ctx, &settings.SV, true) testServer, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{ Settings: settings, Knobs: base.TestingKnobs{