Skip to content

Commit

Permalink
spanconfig: remove knob to disable span configs
Browse files Browse the repository at this point in the history
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`.
  • Loading branch information
andrewbaptist committed Sep 28, 2023
1 parent 6d1bad3 commit 124aaa3
Show file tree
Hide file tree
Showing 24 changed files with 126 additions and 253 deletions.
9 changes: 0 additions & 9 deletions pkg/base/test_server_args.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
1 change: 0 additions & 1 deletion pkg/jobs/job_info_storage_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -239,7 +239,6 @@ func TestAccessorsWithWrongSQLLivenessSession(t *testing.T) {
SkipJobBootstrap: true,
},
},
DisableSpanConfigs: true,
}

ctx := context.Background()
Expand Down
2 changes: 0 additions & 2 deletions pkg/jobs/registry_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -333,7 +333,6 @@ func TestCreateJobWritesToJobInfo(t *testing.T) {
SkipJobBootstrap: true,
},
},
DisableSpanConfigs: true,
}

ctx := context.Background()
Expand Down Expand Up @@ -520,7 +519,6 @@ func TestBatchJobsCreation(t *testing.T) {
SkipJobBootstrap: true,
},
},
DisableSpanConfigs: true,
}

ctx := context.Background()
Expand Down
2 changes: 0 additions & 2 deletions pkg/jobs/update_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,6 @@ func TestUpdaterUpdatesJobInfo(t *testing.T) {
SkipJobBootstrap: true,
},
},
DisableSpanConfigs: true,
}

ctx := context.Background()
Expand Down Expand Up @@ -197,7 +196,6 @@ func TestUpdateDoesNotReadPayloadFromJobsTable(t *testing.T) {
SkipJobBootstrap: true,
},
},
DisableSpanConfigs: true,
}

ctx := context.Background()
Expand Down
1 change: 0 additions & 1 deletion pkg/kv/kvserver/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
12 changes: 0 additions & 12 deletions pkg/kv/kvserver/client_spanconfigs_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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())
Expand Down Expand Up @@ -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())
Expand Down
34 changes: 24 additions & 10 deletions pkg/kv/kvserver/client_split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand All @@ -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 ()")
Expand All @@ -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)
Expand All @@ -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())
Expand All @@ -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))
Expand Down Expand Up @@ -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,
Expand Down
1 change: 0 additions & 1 deletion pkg/kv/kvserver/merge_queue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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())
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/mvcc_gc_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/protectedts/ptcache/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
14 changes: 6 additions & 8 deletions pkg/kv/kvserver/protectedts/ptcache/cache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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,
},
})
Expand Down
29 changes: 4 additions & 25 deletions pkg/kv/kvserver/store.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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",
Expand Down
19 changes: 0 additions & 19 deletions pkg/kv/kvserver/store_gossip.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
Expand All @@ -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)
Expand Down
9 changes: 0 additions & 9 deletions pkg/server/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down Expand Up @@ -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()
Expand Down Expand Up @@ -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)
}

Expand Down
8 changes: 0 additions & 8 deletions pkg/server/config_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
Expand Down Expand Up @@ -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)
}
Expand All @@ -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",
Expand Down
Loading

0 comments on commit 124aaa3

Please sign in to comment.