From 124aaa3b00349f4240c3a078301686c2d8883eda Mon Sep 17 00:00:00 2001 From: Andrew Baptist Date: Fri, 4 Aug 2023 10:13:36 -0400 Subject: [PATCH] spanconfig: remove knob to disable span configs Span configs are integral to correct operation of queues and the ability to disable this infrastructure is no longer necessary. This commit removes the customer facing knob `spanconfig.store.enabled` as well as the environment setting `COCKROACH_DISABLE_SPAN_CONFIGS` and modifies all the tests to stop using this setting. `store_test.go` still sets `cfg.SpanConfigDisabled` but all other tests are converted to use `ManagerDisableJobCreation` instead which keeps the span config infrastructure but disables the internal job to reconcile span configs after certain changes like an upgrade. Epic: none Release note (ops change): Removes the cluster setting `spanconfig.store.enabled` and the ability to use `COCKROACH_DISABLE_SPAN_CONFIGS`. --- pkg/base/test_server_args.go | 9 -- pkg/jobs/job_info_storage_test.go | 1 - pkg/jobs/registry_test.go | 2 - pkg/jobs/update_test.go | 2 - pkg/kv/kvserver/BUILD.bazel | 1 - pkg/kv/kvserver/client_spanconfigs_test.go | 12 -- pkg/kv/kvserver/client_split_test.go | 34 +++-- pkg/kv/kvserver/merge_queue_test.go | 1 - pkg/kv/kvserver/mvcc_gc_queue.go | 2 +- .../kvserver/protectedts/ptcache/BUILD.bazel | 1 + .../protectedts/ptcache/cache_test.go | 14 +- pkg/kv/kvserver/store.go | 29 +---- pkg/kv/kvserver/store_gossip.go | 19 --- pkg/server/config.go | 9 -- pkg/server/config_test.go | 8 -- pkg/server/server.go | 121 ++++++++---------- pkg/server/server_controller_new_server.go | 1 - pkg/server/server_sql.go | 70 +++++----- pkg/server/testserver.go | 3 - pkg/spanconfig/spanconfigmanager/manager.go | 5 +- pkg/spanconfig/spanconfigstore/store.go | 14 -- pkg/sql/logictest/logic.go | 15 --- pkg/upgrade/upgrades/BUILD.bazel | 1 + .../backfill_job_info_table_migration_test.go | 5 +- 24 files changed, 126 insertions(+), 253 deletions(-) diff --git a/pkg/base/test_server_args.go b/pkg/base/test_server_args.go index f4a4d2f06b06..82066b16109f 100644 --- a/pkg/base/test_server_args.go +++ b/pkg/base/test_server_args.go @@ -144,15 +144,6 @@ type TestServerArgs struct { // If set, a TraceDir is initialized at the provided path. TraceDir string - // DisableSpanConfigs disables the use of the span configs infrastructure - // (in favor of the gossiped system config span). It's equivalent to setting - // COCKROACH_DISABLE_SPAN_CONFIGS, and is only intended for tests written - // with the system config span in mind. - // - // TODO(irfansharif): Remove all uses of this when we rip out the system - // config span. - DisableSpanConfigs bool - // DefaultTestTenant determines whether a test's application // workload will be redirected to a virtual cluster (secondary // tenant) automatically. diff --git a/pkg/jobs/job_info_storage_test.go b/pkg/jobs/job_info_storage_test.go index fe99dabcc6eb..12e8880ccc50 100644 --- a/pkg/jobs/job_info_storage_test.go +++ b/pkg/jobs/job_info_storage_test.go @@ -239,7 +239,6 @@ func TestAccessorsWithWrongSQLLivenessSession(t *testing.T) { SkipJobBootstrap: true, }, }, - DisableSpanConfigs: true, } ctx := context.Background() diff --git a/pkg/jobs/registry_test.go b/pkg/jobs/registry_test.go index 9eba6220d15e..c2e2f642cb20 100644 --- a/pkg/jobs/registry_test.go +++ b/pkg/jobs/registry_test.go @@ -333,7 +333,6 @@ func TestCreateJobWritesToJobInfo(t *testing.T) { SkipJobBootstrap: true, }, }, - DisableSpanConfigs: true, } ctx := context.Background() @@ -520,7 +519,6 @@ func TestBatchJobsCreation(t *testing.T) { SkipJobBootstrap: true, }, }, - DisableSpanConfigs: true, } ctx := context.Background() diff --git a/pkg/jobs/update_test.go b/pkg/jobs/update_test.go index b1a553053f56..34e7ee5c5600 100644 --- a/pkg/jobs/update_test.go +++ b/pkg/jobs/update_test.go @@ -54,7 +54,6 @@ func TestUpdaterUpdatesJobInfo(t *testing.T) { SkipJobBootstrap: true, }, }, - DisableSpanConfigs: true, } ctx := context.Background() @@ -197,7 +196,6 @@ func TestUpdateDoesNotReadPayloadFromJobsTable(t *testing.T) { SkipJobBootstrap: true, }, }, - DisableSpanConfigs: true, } ctx := context.Background() diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index d090be12b28f..d2a657e48366 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -453,7 +453,6 @@ go_test( "//pkg/sql/isql", "//pkg/sql/rowenc/keyside", "//pkg/sql/sem/tree", - "//pkg/sql/sessiondata", "//pkg/sql/sqlstats", "//pkg/storage", "//pkg/storage/enginepb", diff --git a/pkg/kv/kvserver/client_spanconfigs_test.go b/pkg/kv/kvserver/client_spanconfigs_test.go index c8b9a748e9a5..5797c838f467 100644 --- a/pkg/kv/kvserver/client_spanconfigs_test.go +++ b/pkg/kv/kvserver/client_spanconfigs_test.go @@ -22,8 +22,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigstore" - "github.com/cockroachdb/cockroach/pkg/sql/isql" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -64,11 +62,6 @@ func TestSpanConfigUpdateAppliedToReplica(t *testing.T) { s := serverutils.StartServerOnly(t, args) defer s.Stopper().Stop(context.Background()) - _, err := s.InternalExecutor().(isql.Executor).ExecEx(ctx, "inline-exec", nil, - sessiondata.RootUserSessionDataOverride, - `SET CLUSTER SETTING spanconfig.store.enabled = true`) - require.NoError(t, err) - key, err := s.ScratchRange() require.NoError(t, err) store, err := s.GetStores().(*kvserver.Stores).GetStore(s.GetFirstStoreID()) @@ -133,11 +126,6 @@ func TestFallbackSpanConfigOverride(t *testing.T) { s := serverutils.StartServerOnly(t, args) defer s.Stopper().Stop(context.Background()) - _, err := s.InternalDB().(isql.DB).Executor().ExecEx(ctx, "inline-exec", nil, - sessiondata.RootUserSessionDataOverride, - `SET CLUSTER SETTING spanconfig.store.enabled = true`) - require.NoError(t, err) - key, err := s.ScratchRange() require.NoError(t, err) store, err := s.GetStores().(*kvserver.Stores).GetStore(s.GetFirstStoreID()) diff --git a/pkg/kv/kvserver/client_split_test.go b/pkg/kv/kvserver/client_split_test.go index 5bff9fb4d475..1d8823b5152c 100644 --- a/pkg/kv/kvserver/client_split_test.go +++ b/pkg/kv/kvserver/client_split_test.go @@ -3629,11 +3629,17 @@ func TestStoreRangeSplitAndMergeWithGlobalReads(t *testing.T) { } return nil } + // Set global reads. + zoneConfig := zonepb.DefaultZoneConfig() + zoneConfig.GlobalReads = proto.Bool(true) ctx := context.Background() s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{ - DisableSpanConfigs: true, Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + DefaultZoneConfigOverride: &zoneConfig, + }, + Store: &kvserver.StoreTestingKnobs{ DisableMergeQueue: true, TestingResponseFilter: respFilter, @@ -3660,11 +3666,6 @@ func TestStoreRangeSplitAndMergeWithGlobalReads(t *testing.T) { _, pErr := kv.SendWrapped(ctx, store.TestSender(), splitArgs) require.Nil(t, pErr) - // Set global reads. - zoneConfig := zonepb.DefaultZoneConfig() - zoneConfig.GlobalReads = proto.Bool(true) - config.TestingSetZoneConfig(config.ObjectID(descID), zoneConfig) - // Perform a write to the system config span being watched by // the SystemConfigProvider. tdb.Exec(t, "CREATE TABLE foo ()") @@ -3676,6 +3677,19 @@ func TestStoreRangeSplitAndMergeWithGlobalReads(t *testing.T) { return nil }) + // The commit wait count is 1 due to the split above since global reads are + // set for the default config. + var splitCount = int64(1) + testutils.SucceedsSoon(t, func() error { + if splitCount != store.Metrics().CommitWaitsBeforeCommitTrigger.Count() { + return errors.Errorf("commit wait count is %d", store.Metrics().CommitWaitsBeforeCommitTrigger.Count()) + } + if splitCount != atomic.LoadInt64(&splitsWithSyntheticTS) { + return errors.Errorf("num splits is %d", atomic.LoadInt64(&splitsWithSyntheticTS)) + } + return nil + }) + // Write to the range, which has the effect of bumping the closed timestamp. pArgs := putArgs(descKey, []byte("foo")) _, pErr = kv.SendWrapped(ctx, store.TestSender(), pArgs) @@ -3686,8 +3700,9 @@ func TestStoreRangeSplitAndMergeWithGlobalReads(t *testing.T) { splitArgs = adminSplitArgs(splitKey) _, pErr = kv.SendWrapped(ctx, store.TestSender(), splitArgs) require.Nil(t, pErr) - require.Equal(t, int64(1), store.Metrics().CommitWaitsBeforeCommitTrigger.Count()) - require.Equal(t, int64(1), atomic.LoadInt64(&splitsWithSyntheticTS)) + splitCount++ + require.Equal(t, splitCount, store.Metrics().CommitWaitsBeforeCommitTrigger.Count()) + require.Equal(t, splitCount, atomic.LoadInt64(&splitsWithSyntheticTS)) repl := store.LookupReplica(roachpb.RKey(splitKey)) require.Equal(t, splitKey, repl.Desc().StartKey.AsRawKey()) @@ -3696,7 +3711,7 @@ func TestStoreRangeSplitAndMergeWithGlobalReads(t *testing.T) { mergeArgs := adminMergeArgs(descKey) _, pErr = kv.SendWrapped(ctx, store.TestSender(), mergeArgs) require.Nil(t, pErr) - require.Equal(t, int64(2), store.Metrics().CommitWaitsBeforeCommitTrigger.Count()) + require.Equal(t, splitCount+1, store.Metrics().CommitWaitsBeforeCommitTrigger.Count()) require.Equal(t, int64(1), atomic.LoadInt64(&mergesWithSyntheticTS)) repl = store.LookupReplica(roachpb.RKey(splitKey)) @@ -3876,7 +3891,6 @@ func TestLBSplitUnsafeKeys(t *testing.T) { } s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{ - DisableSpanConfigs: true, Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ LoadBasedSplittingOverrideKey: overrideLBSplitFn, diff --git a/pkg/kv/kvserver/merge_queue_test.go b/pkg/kv/kvserver/merge_queue_test.go index 0c787317f9bf..6913d5da2af7 100644 --- a/pkg/kv/kvserver/merge_queue_test.go +++ b/pkg/kv/kvserver/merge_queue_test.go @@ -37,7 +37,6 @@ func TestMergeQueueShouldQueue(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) tsc := TestStoreConfig(nil) - tsc.SpanConfigsDisabled = true testCtx.StartWithStoreConfig(ctx, t, stopper, tsc) mq := newMergeQueue(testCtx.store, testCtx.store.DB()) diff --git a/pkg/kv/kvserver/mvcc_gc_queue.go b/pkg/kv/kvserver/mvcc_gc_queue.go index 749c8e6f7abc..9baf40681984 100644 --- a/pkg/kv/kvserver/mvcc_gc_queue.go +++ b/pkg/kv/kvserver/mvcc_gc_queue.go @@ -119,7 +119,7 @@ var mvccGCQueueHighPriInterval = settings.RegisterDurationSetting( ) // EnqueueInMvccGCQueueOnSpanConfigUpdateEnabled controls whether replicas -// are enqueued into the mvcc queue, following a span config update which +// are enqueued into the mvcc gc queue, following a span config update which // affects the replica. // TODO(baptist): Enable this once we have better AC control and have verified // this doesn't cause any overload problems. diff --git a/pkg/kv/kvserver/protectedts/ptcache/BUILD.bazel b/pkg/kv/kvserver/protectedts/ptcache/BUILD.bazel index 67d7c4647ba3..c801d056efe9 100644 --- a/pkg/kv/kvserver/protectedts/ptcache/BUILD.bazel +++ b/pkg/kv/kvserver/protectedts/ptcache/BUILD.bazel @@ -44,6 +44,7 @@ go_test( "//pkg/security/securityassets", "//pkg/security/securitytest", "//pkg/server", + "//pkg/spanconfig", "//pkg/sql", "//pkg/sql/catalog/systemschema", "//pkg/sql/isql", diff --git a/pkg/kv/kvserver/protectedts/ptcache/cache_test.go b/pkg/kv/kvserver/protectedts/ptcache/cache_test.go index 76cc51299e66..4d36715d3631 100644 --- a/pkg/kv/kvserver/protectedts/ptcache/cache_test.go +++ b/pkg/kv/kvserver/protectedts/ptcache/cache_test.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptstorage" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" "github.com/cockroachdb/cockroach/pkg/sql/isql" @@ -159,18 +160,15 @@ func TestRefresh(t *testing.T) { } srv := serverutils.StartServerOnly(t, base.TestServerArgs{ - // Disable span configs to avoid measuring protected timestamp lookups - // performed by the AUTO SPAN CONFIG RECONCILIATION job. - DisableSpanConfigs: true, - - // It's not possible to create secondary tenants when span - // configs are disabled. - DefaultTestTenant: base.TestNeedsTightIntegrationBetweenAPIsAndTestingKnobs, - Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ TestingRequestFilter: st.requestFilter, }, + // Disable span configs to avoid measuring protected timestamp lookups + // performed by the AUTO SPAN CONFIG RECONCILIATION job. + SpanConfig: &spanconfig.TestingKnobs{ + ManagerDisableJobCreation: true, + }, ProtectedTS: ptsKnobs, }, }) diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 0c0ecfc84154..6538915dfa6c 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -1217,11 +1217,11 @@ type StoreConfig struct { // SpanConfigsDisabled determines whether we're able to use the span configs // infrastructure or not. // - // TODO(irfansharif): We can remove this. + // TODO(baptist): Don't add any future uses of this. Will be removed soon. SpanConfigsDisabled bool + // Used to subscribe to span configuration changes, keeping up-to-date a - // data structure useful for retrieving span configs. Only available if - // SpanConfigsDisabled is unset. + // data structure useful for retrieving span configs. SpanConfigSubscriber spanconfig.KVSubscriber // SharedStorageEnabled stores whether this store is configured with a // shared.Storage instance and can accept shared snapshots. @@ -2225,20 +2225,6 @@ func (s *Store) Start(ctx context.Context, stopper *stop.Stopper) error { s.onSpanConfigUpdate(ctx, update) }) - // When toggling between the system config span and the span - // configs infrastructure, we want to re-apply configs on all - // replicas from whatever the new source is. - spanconfigstore.EnabledSetting.SetOnChange(&s.ClusterSettings().SV, func(ctx context.Context) { - enabled := spanconfigstore.EnabledSetting.Get(&s.ClusterSettings().SV) - if enabled { - s.applyAllFromSpanConfigStore(ctx) - } else if scp := s.cfg.SystemConfigProvider; scp != nil { - if sc := scp.GetSystemConfig(); sc != nil { - s.systemGossipUpdate(sc) - } - } - }) - // We also want to do it when the fallback config setting is changed. spanconfigstore.FallbackConfigOverride.SetOnChange(&s.ClusterSettings().SV, func(ctx context.Context) { s.applyAllFromSpanConfigStore(ctx) @@ -2284,10 +2270,7 @@ func (s *Store) GetConfReader(ctx context.Context) (spanconfig.StoreReader, erro return s.cfg.TestingKnobs.ConfReaderInterceptor(), nil } - if s.cfg.SpanConfigsDisabled || - !spanconfigstore.EnabledSetting.Get(&s.ClusterSettings().SV) || - s.TestingKnobs().UseSystemConfigSpanForQueues { - + if s.cfg.SpanConfigsDisabled || s.TestingKnobs().UseSystemConfigSpanForQueues { sysCfg := s.cfg.SystemConfigProvider.GetSystemConfig() if sysCfg == nil { return nil, errSpanConfigsUnavailable @@ -2487,10 +2470,6 @@ func (s *Store) removeReplicaWithRangefeed(rangeID roachpb.RangeID) { // onSpanConfigUpdate is the callback invoked whenever this store learns of a // span config update. func (s *Store) onSpanConfigUpdate(ctx context.Context, updated roachpb.Span) { - if !spanconfigstore.EnabledSetting.Get(&s.ClusterSettings().SV) { - return - } - sp, err := keys.SpanAddr(updated) if err != nil { log.Errorf(ctx, "skipped applying update (%s), unexpected error resolving span address: %v", diff --git a/pkg/kv/kvserver/store_gossip.go b/pkg/kv/kvserver/store_gossip.go index 4d33e385f5ec..b555bc32eb4b 100644 --- a/pkg/kv/kvserver/store_gossip.go +++ b/pkg/kv/kvserver/store_gossip.go @@ -21,7 +21,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigstore" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/retry" @@ -164,10 +163,6 @@ var errSpanConfigsUnavailable = errors.New("span configs not available") // TODO(kvoli): Refactor this function to sit on the store gossip struct, // rather than on the store. func (s *Store) systemGossipUpdate(sysCfg *config.SystemConfig) { - if !s.cfg.SpanConfigsDisabled && spanconfigstore.EnabledSetting.Get(&s.ClusterSettings().SV) { - return // nothing to do - } - ctx := s.AnnotateCtx(context.Background()) s.computeInitialMetrics.Do(func() { // Metrics depend in part on the system config. Compute them as soon as we @@ -187,20 +182,6 @@ func (s *Store) systemGossipUpdate(sysCfg *config.SystemConfig) { // be split or merged. now := s.cfg.Clock.NowAsClockTimestamp() newStoreReplicaVisitor(s).Visit(func(repl *Replica) bool { - key := repl.Desc().StartKey - conf, err := sysCfg.GetSpanConfigForKey(ctx, key) - if err != nil { - if log.V(1) { - log.Infof(context.TODO(), "failed to get span config for key %s", key) - } - conf = s.cfg.DefaultSpanConfig - } - - if s.cfg.SpanConfigsDisabled || - !spanconfigstore.EnabledSetting.Get(&s.ClusterSettings().SV) { - repl.SetSpanConfig(conf) - } - if shouldQueue { s.splitQueue.Async(ctx, "gossip update", true /* wait */, func(ctx context.Context, h queueHelper) { h.MaybeAdd(ctx, repl, now) diff --git a/pkg/server/config.go b/pkg/server/config.go index 4bc31d30477f..966b21d7c254 100644 --- a/pkg/server/config.go +++ b/pkg/server/config.go @@ -200,11 +200,6 @@ type BaseConfig struct { // instantiate stores. StorageEngine enginepb.EngineType - // SpanConfigsDisabled disables the use of the span configs infrastructure. - // - // Environment Variable: COCKROACH_DISABLE_SPAN_CONFIGS - SpanConfigsDisabled bool - // TestingKnobs is used for internal test controls only. TestingKnobs base.TestingKnobs @@ -650,9 +645,6 @@ func (cfg *Config) String() string { if cfg.Linearizable { fmt.Fprintln(w, "linearizable\t", cfg.Linearizable) } - if !cfg.SpanConfigsDisabled { - fmt.Fprintln(w, "span configs enabled\t", !cfg.SpanConfigsDisabled) - } _ = w.Flush() return buf.String() @@ -934,7 +926,6 @@ func (cfg *BaseConfig) InsecureWebAccess() bool { } func (cfg *Config) readSQLEnvironmentVariables() { - cfg.SpanConfigsDisabled = envutil.EnvOrDefaultBool("COCKROACH_DISABLE_SPAN_CONFIGS", cfg.SpanConfigsDisabled) cfg.Linearizable = envutil.EnvOrDefaultBool("COCKROACH_EXPERIMENTAL_LINEARIZABLE", cfg.Linearizable) } diff --git a/pkg/server/config_test.go b/pkg/server/config_test.go index edcc477c534e..e73d7e45b287 100644 --- a/pkg/server/config_test.go +++ b/pkg/server/config_test.go @@ -99,9 +99,6 @@ func TestReadEnvironmentVariables(t *testing.T) { if err := os.Unsetenv("COCKROACH_EXPERIMENTAL_LINEARIZABLE"); err != nil { t.Fatal(err) } - if err := os.Unsetenv("COCKROACH_DISABLE_SPAN_CONFIGS"); err != nil { - t.Fatal(err) - } if err := os.Unsetenv("COCKROACH_SCAN_INTERVAL"); err != nil { t.Fatal(err) } @@ -139,10 +136,6 @@ func TestReadEnvironmentVariables(t *testing.T) { // Set all the environment variables to valid values and ensure they are set // correctly. - if err := os.Setenv("COCKROACH_DISABLE_SPAN_CONFIGS", "true"); err != nil { - t.Fatal(err) - } - cfgExpected.SpanConfigsDisabled = true if err := os.Setenv("COCKROACH_EXPERIMENTAL_LINEARIZABLE", "true"); err != nil { t.Fatal(err) } @@ -167,7 +160,6 @@ func TestReadEnvironmentVariables(t *testing.T) { } for _, envVar := range []string{ - "COCKROACH_DISABLE_SPAN_CONFIGS", "COCKROACH_EXPERIMENTAL_LINEARIZABLE", "COCKROACH_SCAN_INTERVAL", "COCKROACH_SCAN_MIN_IDLE_TIME", diff --git a/pkg/server/server.go b/pkg/server/server.go index 111b028a2c2e..b53e41eefad9 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -745,75 +745,59 @@ func NewServer(cfg Config, stopper *stop.Stopper) (serverctl.ServerStartupInterf // tenant records. kvAccessorForTenantRecords spanconfig.KVAccessor } - if !cfg.SpanConfigsDisabled { - spanConfigKnobs, _ := cfg.TestingKnobs.SpanConfig.(*spanconfig.TestingKnobs) - if spanConfigKnobs != nil && spanConfigKnobs.StoreKVSubscriberOverride != nil { - spanConfig.subscriber = spanConfigKnobs.StoreKVSubscriberOverride - } else { - // We use the span configs infra to control whether rangefeeds are - // enabled on a given range. At the moment this only applies to - // system tables (on both host and secondary tenants). We need to - // consider two things: - // - The sql-side reconciliation process runs asynchronously. When - // the config for a given range is requested, we might not yet have - // it, thus falling back to the static config below. - // - Various internal subsystems rely on rangefeeds to function. - // - // Consequently, we configure our static fallback config to actually - // allow rangefeeds. As the sql-side reconciliation process kicks - // off, it'll install the actual configs that we'll later consult. - // For system table ranges we install configs that allow for - // rangefeeds. Until then, we simply allow rangefeeds when a more - // targeted config is not found. - fallbackConf := cfg.DefaultZoneConfig.AsSpanConfig() - fallbackConf.RangefeedEnabled = true - // We do the same for opting out of strict GC enforcement; it - // really only applies to user table ranges - fallbackConf.GCPolicy.IgnoreStrictEnforcement = true - - spanConfig.subscriber = spanconfigkvsubscriber.New( - clock, - rangeFeedFactory, - keys.SpanConfigurationsTableID, - 1<<20, /* 1 MB */ - fallbackConf, - cfg.Settings, - spanconfigstore.NewBoundsReader(tenantCapabilitiesWatcher), - spanConfigKnobs, - nodeRegistry, - ) - } - - scKVAccessor := spanconfigkvaccessor.New( - db, internalExecutor, cfg.Settings, clock, - systemschema.SpanConfigurationsTableName.FQString(), - spanConfigKnobs, - ) - spanConfig.kvAccessor, spanConfig.kvAccessorForTenantRecords = scKVAccessor, scKVAccessor - spanConfig.reporter = spanconfigreporter.New( - nodeLiveness, - storePool, - spanConfig.subscriber, - rangedesc.NewScanner(db), + spanConfigKnobs, _ := cfg.TestingKnobs.SpanConfig.(*spanconfig.TestingKnobs) + if spanConfigKnobs != nil && spanConfigKnobs.StoreKVSubscriberOverride != nil { + spanConfig.subscriber = spanConfigKnobs.StoreKVSubscriberOverride + } else { + // We use the span configs infra to control whether rangefeeds are + // enabled on a given range. At the moment this only applies to + // system tables (on both host and secondary tenants). We need to + // consider two things: + // - The sql-side reconciliation process runs asynchronously. When + // the config for a given range is requested, we might not yet have + // it, thus falling back to the static config below. + // - Various internal subsystems rely on rangefeeds to function. + // + // Consequently, we configure our static fallback config to actually + // allow rangefeeds. As the sql-side reconciliation process kicks + // off, it'll install the actual configs that we'll later consult. + // For system table ranges we install configs that allow for + // rangefeeds. Until then, we simply allow rangefeeds when a more + // targeted config is not found. + fallbackConf := cfg.DefaultZoneConfig.AsSpanConfig() + fallbackConf.RangefeedEnabled = true + // We do the same for opting out of strict GC enforcement; it + // really only applies to user table ranges + fallbackConf.GCPolicy.IgnoreStrictEnforcement = true + + spanConfig.subscriber = spanconfigkvsubscriber.New( + clock, + rangeFeedFactory, + keys.SpanConfigurationsTableID, + 1<<20, /* 1 MB */ + fallbackConf, cfg.Settings, + spanconfigstore.NewBoundsReader(tenantCapabilitiesWatcher), spanConfigKnobs, + nodeRegistry, ) - } else { - // If the spanconfigs infrastructure is disabled, there should be no - // reconciliation jobs or RPCs issued against the infrastructure. Plug - // in a disabled spanconfig.KVAccessor that would error out for - // unexpected use. - spanConfig.kvAccessor = spanconfigkvaccessor.DisabledKVAccessor - - // Ditto for the spanconfig.Reporter. - spanConfig.reporter = spanconfigreporter.DisabledReporter - - // Use a no-op accessor where tenant records are created/destroyed. - spanConfig.kvAccessorForTenantRecords = spanconfigkvaccessor.NoopKVAccessor - - spanConfig.subscriber = spanconfigkvsubscriber.NewNoopSubscriber(clock) } + scKVAccessor := spanconfigkvaccessor.New( + db, internalExecutor, cfg.Settings, clock, + systemschema.SpanConfigurationsTableName.FQString(), + spanConfigKnobs, + ) + spanConfig.kvAccessor, spanConfig.kvAccessorForTenantRecords = scKVAccessor, scKVAccessor + spanConfig.reporter = spanconfigreporter.New( + nodeLiveness, + storePool, + spanConfig.subscriber, + rangedesc.NewScanner(db), + cfg.Settings, + spanConfigKnobs, + ) + var protectedTSReader spanconfig.ProtectedTSReader if cfg.TestingKnobs.SpanConfig != nil && cfg.TestingKnobs.SpanConfig.(*spanconfig.TestingKnobs).ProtectedTSReaderOverrideFn != nil { @@ -867,7 +851,6 @@ func NewServer(cfg Config, stopper *stop.Stopper) (serverctl.ServerStartupInterf SharedStorageEnabled: cfg.SharedStorage != "", SystemConfigProvider: systemConfigWatcher, SpanConfigSubscriber: spanConfig.subscriber, - SpanConfigsDisabled: cfg.SpanConfigsDisabled, SnapshotApplyLimit: cfg.SnapshotApplyLimit, SnapshotSendLimit: cfg.SnapshotSendLimit, RangeLogWriter: rangeLogWriter, @@ -2006,11 +1989,9 @@ func (s *topLevelServer) PreStart(ctx context.Context) error { return err } - if !s.cfg.SpanConfigsDisabled && s.spanConfigSubscriber != nil { - if subscriber, ok := s.spanConfigSubscriber.(*spanconfigkvsubscriber.KVSubscriber); ok { - if err := subscriber.Start(workersCtx, s.stopper); err != nil { - return err - } + if subscriber, ok := s.spanConfigSubscriber.(*spanconfigkvsubscriber.KVSubscriber); ok { + if err := subscriber.Start(workersCtx, s.stopper); err != nil { + return err } } // Start garbage collecting system events. diff --git a/pkg/server/server_controller_new_server.go b/pkg/server/server_controller_new_server.go index 3c168301c00b..f0c4e1405525 100644 --- a/pkg/server/server_controller_new_server.go +++ b/pkg/server/server_controller_new_server.go @@ -243,7 +243,6 @@ func makeSharedProcessTenantServerConfig( baseCfg.StorageEngine = kvServerCfg.BaseConfig.StorageEngine baseCfg.TestingInsecureWebAccess = kvServerCfg.BaseConfig.TestingInsecureWebAccess baseCfg.Locality = kvServerCfg.BaseConfig.Locality - baseCfg.SpanConfigsDisabled = kvServerCfg.BaseConfig.SpanConfigsDisabled baseCfg.EnableDemoLoginEndpoint = kvServerCfg.BaseConfig.EnableDemoLoginEndpoint baseCfg.DefaultZoneConfig = kvServerCfg.BaseConfig.DefaultZoneConfig baseCfg.HeapProfileDirName = kvServerCfg.BaseConfig.HeapProfileDirName diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 961bbf19c3f9..efaa4a4b0034 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -1260,44 +1260,40 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { execCfg.UpgradeTestingKnobs = knobs } - if !codec.ForSystemTenant() || !cfg.SpanConfigsDisabled { - // Instantiate a span config manager. If we're the host tenant we'll - // only do it unless COCKROACH_DISABLE_SPAN_CONFIGS is set. - spanConfigKnobs, _ := cfg.TestingKnobs.SpanConfig.(*spanconfig.TestingKnobs) - spanConfig.sqlTranslatorFactory = spanconfigsqltranslator.NewFactory( - execCfg.ProtectedTimestampProvider, codec, spanConfigKnobs, - ) - spanConfig.sqlWatcher = spanconfigsqlwatcher.New( - codec, - cfg.Settings, - cfg.rangeFeedFactory, - 1<<20, /* 1 MB bufferMemLimit */ - cfg.stopper, - // TODO(irfansharif): What should this no-op cadence be? - 30*time.Second, /* checkpointNoopsEvery */ - spanConfigKnobs, - ) - spanConfigReconciler := spanconfigreconciler.New( - spanConfig.sqlWatcher, - spanConfig.sqlTranslatorFactory, - cfg.spanConfigAccessor, - execCfg, - codec, - cfg.TenantID, - cfg.Settings, - spanConfigKnobs, - ) - spanConfig.manager = spanconfigmanager.New( - cfg.internalDB, - jobRegistry, - cfg.stopper, - cfg.Settings, - spanConfigReconciler, - spanConfigKnobs, - ) + // Instantiate a span config manager. + spanConfig.sqlTranslatorFactory = spanconfigsqltranslator.NewFactory( + execCfg.ProtectedTimestampProvider, codec, spanConfigKnobs, + ) + spanConfig.sqlWatcher = spanconfigsqlwatcher.New( + codec, + cfg.Settings, + cfg.rangeFeedFactory, + 1<<20, /* 1 MB bufferMemLimit */ + cfg.stopper, + // TODO(irfansharif): What should this no-op cadence be? + 30*time.Second, /* checkpointNoopsEvery */ + spanConfigKnobs, + ) + spanConfigReconciler := spanconfigreconciler.New( + spanConfig.sqlWatcher, + spanConfig.sqlTranslatorFactory, + cfg.spanConfigAccessor, + execCfg, + codec, + cfg.TenantID, + cfg.Settings, + spanConfigKnobs, + ) + spanConfig.manager = spanconfigmanager.New( + cfg.internalDB, + jobRegistry, + cfg.stopper, + cfg.Settings, + spanConfigReconciler, + spanConfigKnobs, + ) - execCfg.SpanConfigReconciler = spanConfigReconciler - } + execCfg.SpanConfigReconciler = spanConfigReconciler execCfg.SpanConfigKVAccessor = cfg.spanConfigAccessor execCfg.SpanConfigLimiter = spanConfig.limiter execCfg.SpanConfigSplitter = spanConfig.splitter diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index f5554d65b0ce..343cf122c8eb 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -254,9 +254,6 @@ func makeTestConfigFromParams(params base.TestServerArgs) Config { if params.EnableDemoLoginEndpoint { cfg.EnableDemoLoginEndpoint = true } - if params.DisableSpanConfigs { - cfg.SpanConfigsDisabled = true - } if params.SnapshotApplyLimit != 0 { cfg.SnapshotApplyLimit = params.SnapshotApplyLimit } diff --git a/pkg/spanconfig/spanconfigmanager/manager.go b/pkg/spanconfig/spanconfigmanager/manager.go index 2d1d02009549..123b8c682acd 100644 --- a/pkg/spanconfig/spanconfigmanager/manager.go +++ b/pkg/spanconfig/spanconfigmanager/manager.go @@ -30,8 +30,7 @@ import ( // checkReconciliationJobInterval is a cluster setting to control how often we // check if the span config reconciliation job exists. If it's not found, it // will be started. It has no effect unless -// spanconfig.reconciliation_job.enabled is configured. For host -// tenants, COCKROACH_DISABLE_SPAN_CONFIGS must not be set. +// spanconfig.reconciliation_job.enabled is configured. var checkReconciliationJobInterval = settings.RegisterDurationSetting( settings.TenantWritable, "spanconfig.reconciliation_job.check_interval", @@ -41,8 +40,6 @@ var checkReconciliationJobInterval = settings.RegisterDurationSetting( ) // jobEnabledSetting gates the activation of the span config reconciliation job. -// For the host tenant it has no effect if COCKROACH_DISABLE_SPAN_CONFIGS is -// set. // // TODO(irfansharif): This should be a tenant read-only setting once the work // for #73349 is completed. diff --git a/pkg/spanconfig/spanconfigstore/store.go b/pkg/spanconfig/spanconfigstore/store.go index f644724bb3db..c2257878755c 100644 --- a/pkg/spanconfig/spanconfigstore/store.go +++ b/pkg/spanconfig/spanconfigstore/store.go @@ -25,20 +25,6 @@ import ( "github.com/cockroachdb/errors" ) -// EnabledSetting is a hidden cluster setting to enable the use of the span -// configs infrastructure in KV. It switches each store in the cluster from -// using the gossip backed system config span to instead using the span configs -// infrastructure. It has no effect if COCKROACH_DISABLE_SPAN_CONFIGS -// is set. -// -// TODO(irfansharif): We should remove this. -var EnabledSetting = settings.RegisterBoolSetting( - settings.SystemOnly, - "spanconfig.store.enabled", - `use the span config infrastructure in KV instead of the system config span`, - true, -) - // FallbackConfigOverride is a hidden cluster setting to override the fallback // config used for ranges with no explicit span configs set. var FallbackConfigOverride = settings.RegisterProtobufSetting( diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go index 61cfe40acf0b..e2b14cccf5e6 100644 --- a/pkg/sql/logictest/logic.go +++ b/pkg/sql/logictest/logic.go @@ -208,8 +208,6 @@ import ( // # cluster-opt: opt1 opt2 // // The options are: -// - disable-span-config: If specified, the span configs infrastructure will be -// disabled. // - tracing-off: If specified, tracing defaults to being turned off. This is // used to override the environment, which may ask for tracing to be on by // default. @@ -1939,17 +1937,6 @@ type clusterOpt interface { apply(args *base.TestServerArgs) } -// clusterOptDisableSpanConfigs corresponds to the disable-span-configs -// directive. -type clusterOptDisableSpanConfigs struct{} - -var _ clusterOpt = clusterOptDisableSpanConfigs{} - -// apply implements the clusterOpt interface. -func (c clusterOptDisableSpanConfigs) apply(args *base.TestServerArgs) { - args.DisableSpanConfigs = true -} - // clusterOptTracingOff corresponds to the tracing-off directive. type clusterOptTracingOff struct{} @@ -2136,8 +2123,6 @@ func readClusterOptions(t *testing.T, path string) []clusterOpt { var res []clusterOpt parseDirectiveOptions(t, path, clusterDirective, func(opt string) { switch opt { - case "disable-span-configs": - res = append(res, clusterOptDisableSpanConfigs{}) case "tracing-off": res = append(res, clusterOptTracingOff{}) case "ignore-tenant-strict-gc-enforcement": diff --git a/pkg/upgrade/upgrades/BUILD.bazel b/pkg/upgrade/upgrades/BUILD.bazel index 490542269c39..b02fa27135fe 100644 --- a/pkg/upgrade/upgrades/BUILD.bazel +++ b/pkg/upgrade/upgrades/BUILD.bazel @@ -161,6 +161,7 @@ go_test( "//pkg/security/username", "//pkg/server", "//pkg/settings/cluster", + "//pkg/spanconfig", "//pkg/sql", "//pkg/sql/catalog", "//pkg/sql/catalog/catalogkeys", diff --git a/pkg/upgrade/upgrades/backfill_job_info_table_migration_test.go b/pkg/upgrade/upgrades/backfill_job_info_table_migration_test.go index 4aebc75d7356..640d550384e1 100644 --- a/pkg/upgrade/upgrades/backfill_job_info_table_migration_test.go +++ b/pkg/upgrade/upgrades/backfill_job_info_table_migration_test.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" @@ -42,12 +43,14 @@ func TestBackfillJobsInfoTable(t *testing.T) { clusterArgs := base.TestClusterArgs{ // Disable all automatic jobs creation and adoption. ServerArgs: base.TestServerArgs{ - DisableSpanConfigs: true, Knobs: base.TestingKnobs{ Server: &server.TestingKnobs{ DisableAutomaticVersionUpgrade: make(chan struct{}), BinaryVersionOverride: clusterversion.ByKey(clusterversion.V22_2), }, + SpanConfig: &spanconfig.TestingKnobs{ + ManagerDisableJobCreation: true, + }, }, }, }