diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index a1af43cb2d0f..5939471df8ab 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -263,8 +263,6 @@ const ( // on lease transfer Raft proposals. New leaseholders now forward their clock // directly to the new lease start time. DontProposeWriteTimestampForLeaseTransfers - // TenantSettingsTable adds the system table for tracking tenant usage. - TenantSettingsTable // EnablePebbleFormatVersionBlockProperties enables a new Pebble SSTable // format version for block property collectors. // NB: this cluster version is paired with PebbleFormatBlockPropertyCollector @@ -540,10 +538,6 @@ var versionsSingleton = keyedVersions{ Key: DontProposeWriteTimestampForLeaseTransfers, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 60}, }, - { - Key: TenantSettingsTable, - Version: roachpb.Version{Major: 21, Minor: 2, Internal: 62}, - }, { Key: EnablePebbleFormatVersionBlockProperties, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 64}, diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 6403cde27bac..2826696add53 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -38,48 +38,47 @@ func _() { _ = x[AddRaftAppliedIndexTermMigration-27] _ = x[PostAddRaftAppliedIndexTermMigration-28] _ = x[DontProposeWriteTimestampForLeaseTransfers-29] - _ = x[TenantSettingsTable-30] - _ = x[EnablePebbleFormatVersionBlockProperties-31] - _ = x[DisableSystemConfigGossipTrigger-32] - _ = x[MVCCIndexBackfiller-33] - _ = x[EnableLeaseHolderRemoval-34] - _ = x[BackupResolutionInJob-35] - _ = x[LooselyCoupledRaftLogTruncation-36] - _ = x[ChangefeedIdleness-37] - _ = x[BackupDoesNotOverwriteLatestAndCheckpoint-38] - _ = x[EnableDeclarativeSchemaChanger-39] - _ = x[RowLevelTTL-40] - _ = x[PebbleFormatSplitUserKeysMarked-41] - _ = x[IncrementalBackupSubdir-42] - _ = x[DateStyleIntervalStyleCastRewrite-43] - _ = x[EnableNewStoreRebalancer-44] - _ = x[ClusterLocksVirtualTable-45] - _ = x[AutoStatsTableSettings-46] - _ = x[ForecastStats-47] - _ = x[SuperRegions-48] - _ = x[EnableNewChangefeedOptions-49] - _ = x[SpanCountTable-50] - _ = x[PreSeedSpanCountTable-51] - _ = x[SeedSpanCountTable-52] - _ = x[V22_1-53] - _ = x[Start22_2-54] - _ = x[LocalTimestamps-55] - _ = x[EnsurePebbleFormatVersionRangeKeys-56] - _ = x[EnablePebbleFormatVersionRangeKeys-57] - _ = x[TrigramInvertedIndexes-58] - _ = x[RemoveGrantPrivilege-59] - _ = x[MVCCRangeTombstones-60] - _ = x[UpgradeSequenceToBeReferencedByID-61] - _ = x[SampledStmtDiagReqs-62] - _ = x[AddSSTableTombstones-63] - _ = x[SystemPrivilegesTable-64] - _ = x[EnablePredicateProjectionChangefeed-65] - _ = x[AlterSystemSQLInstancesAddLocality-66] + _ = x[EnablePebbleFormatVersionBlockProperties-30] + _ = x[DisableSystemConfigGossipTrigger-31] + _ = x[MVCCIndexBackfiller-32] + _ = x[EnableLeaseHolderRemoval-33] + _ = x[BackupResolutionInJob-34] + _ = x[LooselyCoupledRaftLogTruncation-35] + _ = x[ChangefeedIdleness-36] + _ = x[BackupDoesNotOverwriteLatestAndCheckpoint-37] + _ = x[EnableDeclarativeSchemaChanger-38] + _ = x[RowLevelTTL-39] + _ = x[PebbleFormatSplitUserKeysMarked-40] + _ = x[IncrementalBackupSubdir-41] + _ = x[DateStyleIntervalStyleCastRewrite-42] + _ = x[EnableNewStoreRebalancer-43] + _ = x[ClusterLocksVirtualTable-44] + _ = x[AutoStatsTableSettings-45] + _ = x[ForecastStats-46] + _ = x[SuperRegions-47] + _ = x[EnableNewChangefeedOptions-48] + _ = x[SpanCountTable-49] + _ = x[PreSeedSpanCountTable-50] + _ = x[SeedSpanCountTable-51] + _ = x[V22_1-52] + _ = x[Start22_2-53] + _ = x[LocalTimestamps-54] + _ = x[EnsurePebbleFormatVersionRangeKeys-55] + _ = x[EnablePebbleFormatVersionRangeKeys-56] + _ = x[TrigramInvertedIndexes-57] + _ = x[RemoveGrantPrivilege-58] + _ = x[MVCCRangeTombstones-59] + _ = x[UpgradeSequenceToBeReferencedByID-60] + _ = x[SampledStmtDiagReqs-61] + _ = x[AddSSTableTombstones-62] + _ = x[SystemPrivilegesTable-63] + _ = x[EnablePredicateProjectionChangefeed-64] + _ = x[AlterSystemSQLInstancesAddLocality-65] } -const _Key_name = "V21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersTenantSettingsTableEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalBackupResolutionInJobLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirDateStyleIntervalStyleCastRewriteEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsForecastStatsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocality" +const _Key_name = "V21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalBackupResolutionInJobLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirDateStyleIntervalStyleCastRewriteEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsForecastStatsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocality" -var _Key_index = [...]uint16{0, 5, 14, 36, 54, 76, 113, 152, 166, 207, 233, 252, 286, 298, 329, 353, 374, 402, 432, 460, 481, 494, 513, 547, 585, 619, 651, 687, 719, 755, 797, 816, 856, 888, 907, 931, 952, 983, 1001, 1042, 1072, 1083, 1114, 1137, 1170, 1194, 1218, 1240, 1253, 1265, 1291, 1305, 1326, 1344, 1349, 1358, 1373, 1407, 1441, 1463, 1483, 1502, 1535, 1554, 1574, 1595, 1630, 1664} +var _Key_index = [...]uint16{0, 5, 14, 36, 54, 76, 113, 152, 166, 207, 233, 252, 286, 298, 329, 353, 374, 402, 432, 460, 481, 494, 513, 547, 585, 619, 651, 687, 719, 755, 797, 837, 869, 888, 912, 933, 964, 982, 1023, 1053, 1064, 1095, 1118, 1151, 1175, 1199, 1221, 1234, 1246, 1272, 1286, 1307, 1325, 1330, 1339, 1354, 1388, 1422, 1444, 1464, 1483, 1516, 1535, 1555, 1576, 1611, 1645} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/server/tenantsettingswatcher/BUILD.bazel b/pkg/server/tenantsettingswatcher/BUILD.bazel index e58e52f3a9b7..1f7be226f406 100644 --- a/pkg/server/tenantsettingswatcher/BUILD.bazel +++ b/pkg/server/tenantsettingswatcher/BUILD.bazel @@ -11,7 +11,6 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/server/tenantsettingswatcher", visibility = ["//visibility:public"], deps = [ - "//pkg/clusterversion", "//pkg/keys", "//pkg/kv/kvclient/rangefeed", "//pkg/kv/kvclient/rangefeed/rangefeedbuffer", @@ -44,7 +43,6 @@ go_test( embed = [":tenantsettingswatcher"], deps = [ "//pkg/base", - "//pkg/clusterversion", "//pkg/keys", "//pkg/roachpb", "//pkg/security/securityassets", @@ -53,12 +51,10 @@ go_test( "//pkg/settings", "//pkg/sql", "//pkg/sql/catalog", - "//pkg/testutils", "//pkg/testutils/serverutils", "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", "//pkg/util/leaktest", - "@com_github_cockroachdb_errors//:errors", "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/server/tenantsettingswatcher/watcher.go b/pkg/server/tenantsettingswatcher/watcher.go index 5c121e5c4dde..6799331ab377 100644 --- a/pkg/server/tenantsettingswatcher/watcher.go +++ b/pkg/server/tenantsettingswatcher/watcher.go @@ -12,9 +12,7 @@ package tenantsettingswatcher import ( "context" - "sync" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedbuffer" @@ -78,56 +76,14 @@ func New( // Start will start the Watcher. // -// If the current cluster version indicates that we have a tenant settings -// table, this function sets up the rangefeed and waits for the initial scan. An -// error will be returned if the initial table scan hits an error, the context -// is canceled or the stopper is stopped prior to the initial data being -// retrieved. -// -// Otherwise, Start sets up a background task that waits for the right version -// and starts the rangefeed when appropriate. WaitUntilStarted can be used to -// wait for the rangefeed setup. +// This function sets up the rangefeed and waits for the initial scan. An error +// will be returned if the initial table scan hits an error, the context is +// canceled or the stopper is stopped prior to the initial data being retrieved. func (w *Watcher) Start(ctx context.Context, sysTableResolver catalog.SystemTableIDResolver) error { w.startCh = make(chan struct{}) - if w.st.Version.IsActive(ctx, clusterversion.TenantSettingsTable) { - // We are not in a mixed-version scenario; start the rangefeed now. - w.startErr = w.startRangeFeed(ctx, sysTableResolver) - close(w.startCh) - return w.startErr - } - // Set up an on-change callback that closes this channel once the version - // supports tenant settings. - versionOkCh := make(chan struct{}) - var once sync.Once - w.st.Version.SetOnChange(func(ctx context.Context, newVersion clusterversion.ClusterVersion) { - if newVersion.IsActive(clusterversion.TenantSettingsTable) { - once.Do(func() { - close(versionOkCh) - }) - } - }) - // Now check the version again, in case the version changed just before - // SetOnChange. - if w.st.Version.IsActive(ctx, clusterversion.TenantSettingsTable) { - w.startErr = w.startRangeFeed(ctx, sysTableResolver) - close(w.startCh) - return w.startErr - } - return w.stopper.RunAsyncTask(ctx, "tenantsettingswatcher-start", func(ctx context.Context) { - log.Infof(ctx, "tenantsettingswatcher waiting for the appropriate version") - select { - case <-versionOkCh: - case <-w.stopper.ShouldQuiesce(): - return - } - log.Infof(ctx, "tenantsettingswatcher can now start") - w.startErr = w.startRangeFeed(ctx, sysTableResolver) - if w.startErr != nil { - // We are not equipped to handle this error asynchronously. - log.Warningf(ctx, "error starting tenantsettingswatcher rangefeed: %v", w.startErr) - } - close(w.startCh) - }) + w.startErr = w.startRangeFeed(ctx, sysTableResolver) + close(w.startCh) + return w.startErr } // startRangeFeed starts the range feed and waits for the initial table scan. An @@ -237,11 +193,6 @@ func (w *Watcher) startRangeFeed( // WaitForStart waits until the rangefeed is set up. Returns an error if the // rangefeed setup failed. -// -// If the cluster version does not support tenant settings, returns immediately -// with no error. Note that it is still legal to call GetTenantOverrides and -// GetAllTenantOverrides in this state. When the cluster version is upgraded, -// the settings will start being updated. func (w *Watcher) WaitForStart(ctx context.Context) error { // Fast path check. select { @@ -252,12 +203,6 @@ func (w *Watcher) WaitForStart(ctx context.Context) error { if w.startCh == nil { return errors.AssertionFailedf("Start() was not yet called") } - if !w.st.Version.IsActive(ctx, clusterversion.TenantSettingsTable) { - // If this happens, then we are running new tenant code against a host - // cluster that was not fully upgraded. - log.Warningf(ctx, "tenant requested settings before host cluster version upgrade") - return nil - } select { case <-w.startCh: return w.startErr diff --git a/pkg/server/tenantsettingswatcher/watcher_test.go b/pkg/server/tenantsettingswatcher/watcher_test.go index 7b7a18920d76..2cfcaefe3ba9 100644 --- a/pkg/server/tenantsettingswatcher/watcher_test.go +++ b/pkg/server/tenantsettingswatcher/watcher_test.go @@ -18,18 +18,13 @@ 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/server" "github.com/cockroachdb/cockroach/pkg/server/tenantsettingswatcher" - "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" ) @@ -117,87 +112,3 @@ func TestWatcher(t *testing.T) { t3Overrides, _ = w.GetTenantOverrides(t3) expect(t3Overrides, "qux=qux-t3") } - -// TestWatcherWaitForVersion verifies that watcher startup waits for the cluster -// version to be upgraded. -func TestWatcherWaitForVersion(t *testing.T) { - defer leaktest.AfterTest(t)() - ctx := context.Background() - - var oldVersion = clusterversion.ByKey(clusterversion.V21_2) - - disableUpgradeCh := make(chan struct{}) - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ - ServerArgs: base.TestServerArgs{ - Knobs: base.TestingKnobs{ - Server: &server.TestingKnobs{ - BinaryVersionOverride: oldVersion, - DisableAutomaticVersionUpgrade: disableUpgradeCh, - }, - }, - }, - }) - defer tc.Stopper().Stop(ctx) - - s0 := tc.Server(0) - w := tenantsettingswatcher.New( - s0.Clock(), - s0.ExecutorConfig().(sql.ExecutorConfig).RangeFeedFactory, - s0.Stopper(), - s0.ClusterSettings(), - ) - - // Start should go in async mode and wait for the version. - err := w.Start(ctx, s0.SystemTableIDResolver().(catalog.SystemTableIDResolver)) - require.NoError(t, err) - - // Allow upgrade, wait for the table to be created. - close(disableUpgradeCh) - db := tc.ServerConn(0) - testutils.SucceedsSoon(t, func() error { - row := db.QueryRow("SELECT count(*) FROM [SHOW TABLES FROM system] WHERE table_name = 'tenant_settings'") - if row.Err() != nil { - return row.Err() - } - var count int - if err := row.Scan(&count); err != nil { - return err - } - if count == 0 { - return errors.Errorf("tenant_settings table does not exist") - } - return nil - }) - // Wait for watcher start. - waitForStartCh := make(chan error) - go func() { - waitForStartCh <- w.WaitForStart(ctx) - }() - select { - case err := <-waitForStartCh: - if err != nil { - t.Fatalf("WaitForStart error: %v", err) - } - case <-time.After(45 * time.Second): - t.Fatalf("WaitForStart did not return after upgrade was allowed") - } - - // Set an override and make sure the watcher is working. - _, ch := w.GetAllTenantOverrides() - r := sqlutils.MakeSQLRunner(db) - r.Exec(t, `INSERT INTO system.tenant_settings (tenant_id, name, value, value_type) VALUES (0, 'foo', 'foo', 's')`) - // Wait for the update. - select { - case <-ch: - overrides, _ := w.GetAllTenantOverrides() - expected := roachpb.TenantSetting{ - Name: "foo", - Value: settings.EncodedValue{Value: "foo", Type: "s"}, - } - if len(overrides) != 1 || overrides[0] != expected { - t.Fatalf("invalid overrides %v", overrides) - } - case <-time.After(45 * time.Second): - t.Fatalf("Did not receive updated tenant overrides") - } -} diff --git a/pkg/sql/tenant.go b/pkg/sql/tenant.go index 445f09708804..7ccc46e33db4 100644 --- a/pkg/sql/tenant.go +++ b/pkg/sql/tenant.go @@ -448,13 +448,11 @@ func GCTenantSync(ctx context.Context, execCfg *ExecutorConfig, info *descpb.Ten return errors.Wrapf(err, "deleting tenant %d usage", info.ID) } - if execCfg.Settings.Version.IsActive(ctx, clusterversion.TenantSettingsTable) { - if _, err := execCfg.InternalExecutor.ExecEx( - ctx, "delete-tenant-settings", txn, sessiondata.NodeUserSessionDataOverride, - `DELETE FROM system.tenant_settings WHERE tenant_id = $1`, info.ID, - ); err != nil { - return errors.Wrapf(err, "deleting tenant %d settings", info.ID) - } + if _, err := execCfg.InternalExecutor.ExecEx( + ctx, "delete-tenant-settings", txn, sessiondata.NodeUserSessionDataOverride, + `DELETE FROM system.tenant_settings WHERE tenant_id = $1`, info.ID, + ); err != nil { + return errors.Wrapf(err, "deleting tenant %d settings", info.ID) } if !execCfg.Settings.Version.IsActive(ctx, clusterversion.PreSeedTenantSpanConfigs) { diff --git a/pkg/upgrade/upgrades/BUILD.bazel b/pkg/upgrade/upgrades/BUILD.bazel index 5d807c4ff2f4..bd4a6ef4d6a8 100644 --- a/pkg/upgrade/upgrades/BUILD.bazel +++ b/pkg/upgrade/upgrades/BUILD.bazel @@ -20,7 +20,6 @@ go_library( "seed_tenant_span_configs.go", "span_count_table.go", "system_privileges.go", - "tenant_settings.go", "upgrade_sequence_to_be_referenced_by_ID.go", "upgrades.go", ], diff --git a/pkg/upgrade/upgrades/tenant_settings.go b/pkg/upgrade/upgrades/tenant_settings.go deleted file mode 100644 index 1e84f930efd3..000000000000 --- a/pkg/upgrade/upgrades/tenant_settings.go +++ /dev/null @@ -1,34 +0,0 @@ -// Copyright 2021 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 upgrades - -import ( - "context" - - "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/jobs" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" - "github.com/cockroachdb/cockroach/pkg/upgrade" -) - -// tenantSettingsTableMigration creates the system.tenant_settings table (for the -// system tenant). -func tenantSettingsTableMigration( - ctx context.Context, _ clusterversion.ClusterVersion, d upgrade.TenantDeps, _ *jobs.Job, -) error { - // Only create the table on the system tenant. - if !d.Codec.ForSystemTenant() { - return nil - } - return createSystemTable( - ctx, d.DB, d.Codec, systemschema.TenantSettingsTable, - ) -} diff --git a/pkg/upgrade/upgrades/upgrades.go b/pkg/upgrade/upgrades/upgrades.go index 4aae684dc307..8c678f24dea7 100644 --- a/pkg/upgrade/upgrades/upgrades.go +++ b/pkg/upgrade/upgrades/upgrades.go @@ -121,12 +121,6 @@ var upgrades = []upgrade.Upgrade{ toCV(clusterversion.PostAddRaftAppliedIndexTermMigration), postRaftAppliedIndexTermMigration, ), - upgrade.NewTenantUpgrade( - "add the system.tenant_settings table", - toCV(clusterversion.TenantSettingsTable), - NoPrecondition, - tenantSettingsTableMigration, - ), upgrade.NewTenantUpgrade( "add the system.span_count table", toCV(clusterversion.SpanCountTable),