diff --git a/pkg/config/system.go b/pkg/config/system.go index 2a38770400f7..1283a7ae7cbf 100644 --- a/pkg/config/system.go +++ b/pkg/config/system.go @@ -81,6 +81,9 @@ type zoneEntry struct { // saying whether or not it should be considered for splitting at all. // A database descriptor or a table view descriptor are examples of IDs // that should not be considered for splits. +// NB: SystemConfig can be updated to only contain system.descriptor and +// system.zones. We still need SystemConfig for SystemConfigProvider which is +// used in replication reports and the opt catalog. type SystemConfig struct { SystemConfigEntries DefaultZoneConfig *zonepb.ZoneConfig diff --git a/pkg/gossip/gossip.go b/pkg/gossip/gossip.go index 97c7f117d8c0..8160efbd4c65 100644 --- a/pkg/gossip/gossip.go +++ b/pkg/gossip/gossip.go @@ -1145,37 +1145,6 @@ func (g *Gossip) RegisterCallback(pattern string, method Callback, opts ...Callb } } -// DeprecatedGetSystemConfig returns the local unmarshaled version of the system config. -// Returns nil if the system config hasn't been set yet. -// -// TODO(ajwerner): Remove this in 22.2. -func (g *Gossip) DeprecatedGetSystemConfig() *config.SystemConfig { - g.systemConfigMu.RLock() - defer g.systemConfigMu.RUnlock() - return g.systemConfig -} - -// DeprecatedRegisterSystemConfigChannel registers a channel to signify updates for the -// system config. It is notified after registration (if a system config is -// already set), and whenever a new system config is successfully unmarshaled. -// -// TODO(ajwerner): Remove this in 22.2. -func (g *Gossip) DeprecatedRegisterSystemConfigChannel() <-chan struct{} { - // Create channel that receives new system config notifications. - // The channel has a size of 1 to prevent gossip from having to block on it. - c := make(chan struct{}, 1) - - g.systemConfigMu.Lock() - defer g.systemConfigMu.Unlock() - g.systemConfigChannels = append(g.systemConfigChannels, c) - - // Notify the channel right away if we have a config. - if g.systemConfig != nil { - c <- struct{}{} - } - return c -} - // updateSystemConfig is the raw gossip info callback. Unmarshal the // system config, and if successful, send on each system config // channel. diff --git a/pkg/kv/kvclient/kvcoord/txn_coord_sender.go b/pkg/kv/kvclient/kvcoord/txn_coord_sender.go index cb6f71799d48..5fcd8c96fc52 100644 --- a/pkg/kv/kvclient/kvcoord/txn_coord_sender.go +++ b/pkg/kv/kvclient/kvcoord/txn_coord_sender.go @@ -11,12 +11,10 @@ package kvcoord import ( - "bytes" "context" "fmt" "runtime/debug" - "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -984,30 +982,6 @@ func sanityCheckErrWithTxn( return err } -// setTxnAnchorKey sets the key at which to anchor the transaction record. The -// transaction anchor key defaults to the first key written in a transaction. -func (tc *TxnCoordSender) setTxnAnchorKeyLocked(key roachpb.Key) error { - if len(tc.mu.txn.Key) != 0 { - return errors.Errorf("transaction anchor key already set") - } - tc.mu.txn.Key = key - return nil -} - -// AnchorOnSystemConfigRange is part of the client.TxnSender interface. -func (tc *TxnCoordSender) AnchorOnSystemConfigRange() error { - tc.mu.Lock() - defer tc.mu.Unlock() - // Allow this to be called more than once. - if bytes.Equal(tc.mu.txn.Key, keys.SystemConfigSpan.Key) { - return nil - } - // The system-config trigger must be run on the system-config range which - // means any transaction with the trigger set needs to be anchored to the - // system-config range. - return tc.setTxnAnchorKeyLocked(keys.SystemConfigSpan.Key) -} - // TxnStatus is part of the client.TxnSender interface. func (tc *TxnCoordSender) TxnStatus() roachpb.TransactionStatus { tc.mu.Lock() diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_committer_test.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_committer_test.go index 52a132054af0..c0a58c820536 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_committer_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_committer_test.go @@ -243,9 +243,7 @@ func TestTxnCommitterStripsInFlightWrites(t *testing.T) { ba.Requests = nil etArgsWithTrigger := etArgs etArgsWithTrigger.InternalCommitTrigger = &roachpb.InternalCommitTrigger{ - ModifiedSpanTrigger: &roachpb.ModifiedSpanTrigger{ - SystemConfigSpan: true, - }, + ModifiedSpanTrigger: &roachpb.ModifiedSpanTrigger{NodeLivenessSpan: &roachpb.Span{}}, } ba.Add(&putArgs, &qiArgs, &etArgsWithTrigger) diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go index 734c8b3065d3..76f975c4b892 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go @@ -18,7 +18,6 @@ import ( "sync/atomic" "time" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/abortspan" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" @@ -425,17 +424,6 @@ func EndTxn( if err := txnResult.MergeAndDestroy(triggerResult); err != nil { return result.Result{}, err } - } else if reply.Txn.Status == roachpb.ABORTED { - // If this is the system config span and we're aborted, add a trigger to - // potentially gossip now that we've removed an intent. This is important - // to deal with cases where previously committed values were not gossipped - // due to an outstanding intent. - if cArgs.EvalCtx.ContainsKey(keys.SystemConfigSpan.Key) && - !cArgs.EvalCtx.ClusterSettings().Version.IsActive( - ctx, clusterversion.DisableSystemConfigGossipTrigger, - ) { - txnResult.Local.MaybeGossipSystemConfigIfHaveFailure = true - } } return txnResult, nil @@ -691,31 +679,6 @@ func RunCommitTrigger( } if ct.GetModifiedSpanTrigger() != nil { var pd result.Result - if ct.ModifiedSpanTrigger.SystemConfigSpan { - // Check if we need to gossip the system config. - // NOTE: System config gossiping can only execute correctly if - // the transaction record is located on the range that contains - // the system span. If a transaction is created which modifies - // both system *and* non-system data, it should be ensured that - // the transaction record itself is on the system span. This can - // be done by making sure a system key is the first key touched - // in the transaction. - if rec.ContainsKey(keys.SystemConfigSpan.Key) { - if err := pd.MergeAndDestroy( - result.Result{ - Local: result.LocalResult{ - MaybeGossipSystemConfig: true, - }, - }, - ); err != nil { - return result.Result{}, err - } - } else { - log.Errorf(ctx, "System configuration span was modified, but the "+ - "modification trigger is executing on a non-system range. "+ - "Configuration changes will not be gossiped.") - } - } if nlSpan := ct.ModifiedSpanTrigger.NodeLivenessSpan; nlSpan != nil { if err := pd.MergeAndDestroy( result.Result{ diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go index 923b91a015cf..fa90387cba91 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go @@ -1154,7 +1154,7 @@ func TestCommitWaitBeforeIntentResolutionIfCommitTrigger(t *testing.T) { } if commitTrigger { req.InternalCommitTrigger = &roachpb.InternalCommitTrigger{ - ModifiedSpanTrigger: &roachpb.ModifiedSpanTrigger{SystemConfigSpan: true}, + ModifiedSpanTrigger: &roachpb.ModifiedSpanTrigger{NodeLivenessSpan: &roachpb.Span{}}, } } var resp roachpb.EndTxnResponse @@ -1175,7 +1175,7 @@ func TestCommitWaitBeforeIntentResolutionIfCommitTrigger(t *testing.T) { if cfg.expError { require.Error(t, err) - require.Regexp(t, `txn .* with modified-span \(system-config\) commit trigger needs commit wait`, err) + require.Regexp(t, `txn .* with modified-span \(node-liveness\) commit trigger needs commit wait`, err) } else { require.NoError(t, err) } diff --git a/pkg/kv/kvserver/client_lease_test.go b/pkg/kv/kvserver/client_lease_test.go index 89a03e6cee07..b490ce628f67 100644 --- a/pkg/kv/kvserver/client_lease_test.go +++ b/pkg/kv/kvserver/client_lease_test.go @@ -22,8 +22,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" @@ -31,7 +29,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -96,135 +93,6 @@ func TestStoreRangeLease(t *testing.T) { } } -// TestStoreGossipSystemData verifies that the system-config and node-liveness -// data is gossiped at startup in the mixed version state. -// -// TODO(ajwerner): Delete this test in 22.2. -func TestStoreGossipSystemData(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - zcfg := zonepb.DefaultZoneConfig() - version := clusterversion.ByKey(clusterversion.DisableSystemConfigGossipTrigger - 1) - settings := cluster.MakeTestingClusterSettingsWithVersions( - version, version, false, /* initializeVersion */ - ) - serverArgs := base.TestServerArgs{ - Settings: settings, - Knobs: base.TestingKnobs{ - Store: &kvserver.StoreTestingKnobs{ - DisableMergeQueue: true, - }, - Server: &server.TestingKnobs{ - DefaultZoneConfigOverride: &zcfg, - BinaryVersionOverride: version, - DisableAutomaticVersionUpgrade: make(chan struct{}), - }, - }, - } - tc := testcluster.StartTestCluster(t, 1, - base.TestClusterArgs{ - ReplicationMode: base.ReplicationManual, - ServerArgs: serverArgs, - }, - ) - defer tc.Stopper().Stop(context.Background()) - - store := tc.GetFirstStoreFromServer(t, 0) - splitKey := keys.SystemConfigSplitKey - tc.SplitRangeOrFatal(t, splitKey) - if _, err := store.DB().Inc(context.Background(), splitKey, 1); err != nil { - t.Fatalf("failed to increment: %+v", err) - } - - getSystemConfig := func(s *kvserver.Store) *config.SystemConfig { - systemConfig := s.Gossip().DeprecatedGetSystemConfig() - return systemConfig - } - getNodeLiveness := func(s *kvserver.Store) livenesspb.Liveness { - var liveness livenesspb.Liveness - if err := s.Gossip().GetInfoProto(gossip.MakeNodeLivenessKey(1), &liveness); err == nil { - return liveness - } - return livenesspb.Liveness{} - } - - // Restart the store and verify that both the system-config and node-liveness - // data is gossiped. - tc.AddAndStartServer(t, serverArgs) - tc.StopServer(0) - - testutils.SucceedsSoon(t, func() error { - if !getSystemConfig(tc.GetFirstStoreFromServer(t, 1)).DefaultZoneConfig.Equal(zcfg) { - return errors.New("system config not gossiped") - } - if getNodeLiveness(tc.GetFirstStoreFromServer(t, 1)) == (livenesspb.Liveness{}) { - return errors.New("node liveness not gossiped") - } - return nil - }) -} - -// TestGossipSystemConfigOnLeaseChange verifies that the system-config gets -// re-gossiped on lease transfer even if it hasn't changed. This helps prevent -// situations where a previous leaseholder can restart and not receive the -// system config because it was the original source of it within the gossip -// network. This test only applies in the mixed version state. -// -// TODO(ajwerner): Remove this test in 22.2. -func TestGossipSystemConfigOnLeaseChange(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - const numStores = 3 - tc := testcluster.StartTestCluster(t, numStores, - base.TestClusterArgs{ - ReplicationMode: base.ReplicationManual, - ServerArgs: base.TestServerArgs{ - Knobs: base.TestingKnobs{ - Store: &kvserver.StoreTestingKnobs{ - DisableMergeQueue: true, - }, - Server: &server.TestingKnobs{ - BinaryVersionOverride: clusterversion.ByKey( - clusterversion.DisableSystemConfigGossipTrigger - 1, - ), - DisableAutomaticVersionUpgrade: make(chan struct{}), - }, - }, - }, - }, - ) - defer tc.Stopper().Stop(context.Background()) - - key := keys.SystemConfigSpan.Key - tc.AddVotersOrFatal(t, key, tc.Target(1), tc.Target(2)) - - initialStoreIdx := -1 - for i := range tc.Servers { - if tc.GetFirstStoreFromServer(t, i).Gossip().InfoOriginatedHere(gossip.KeyDeprecatedSystemConfig) { - initialStoreIdx = i - } - } - if initialStoreIdx == -1 { - t.Fatalf("no store has gossiped system config; gossip contents: %+v", tc.GetFirstStoreFromServer(t, 0).Gossip().GetInfoStatus()) - } - - newStoreIdx := (initialStoreIdx + 1) % numStores - if err := tc.TransferRangeLease(tc.LookupRangeOrFatal(t, key), tc.Target(newStoreIdx)); err != nil { - t.Fatalf("Unexpected error %v", err) - } - testutils.SucceedsSoon(t, func() error { - if tc.GetFirstStoreFromServer(t, initialStoreIdx).Gossip().InfoOriginatedHere(gossip.KeyDeprecatedSystemConfig) { - return errors.New("system config still most recently gossiped by original leaseholder") - } - if !tc.GetFirstStoreFromServer(t, newStoreIdx).Gossip().InfoOriginatedHere(gossip.KeyDeprecatedSystemConfig) { - return errors.New("system config not most recently gossiped by new leaseholder") - } - return nil - }) -} - func TestGossipNodeLivenessOnLeaseChange(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index d46f34a312b7..8ba63652fe75 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -3790,10 +3790,6 @@ func TestStrictGCEnforcement(t *testing.T) { s := tc.Server(i) _, r := getFirstStoreReplica(t, s, tableKey) if c := r.SpanConfig(); c.TTL().Seconds() != (time.Duration(exp) * time.Second).Seconds() { - _, sysCfg := getFirstStoreReplica(t, tc.Server(i), keys.SystemConfigSpan.Key) - sysCfg.RaftLock() - require.NoError(t, sysCfg.MaybeGossipSystemConfigRaftMuLocked(ctx)) - sysCfg.RaftUnlock() return errors.Errorf("expected %d, got %f", exp, c.TTL().Seconds()) } } @@ -3805,11 +3801,8 @@ func TestStrictGCEnforcement(t *testing.T) { sqlDB.Exec(t, `SET CLUSTER SETTING kv.gc_ttl.strict_enforcement.enabled = `+fmt.Sprint(val)) testutils.SucceedsSoon(t, func() error { for i := 0; i < tc.NumServers(); i++ { - s, r := getFirstStoreReplica(t, tc.Server(i), keys.SystemConfigSpan.Key) + s, _ := getFirstStoreReplica(t, tc.Server(i), keys.SystemConfigSpan.Key) if kvserver.StrictGCEnforcement.Get(&s.ClusterSettings().SV) != val { - r.RaftLock() - require.NoError(t, r.MaybeGossipSystemConfigRaftMuLocked(ctx)) - r.RaftUnlock() return errors.Errorf("expected %v, got %v", val, !val) } } diff --git a/pkg/kv/kvserver/gossip_test.go b/pkg/kv/kvserver/gossip_test.go index 4abc0f22810b..a5e833dd6041 100644 --- a/pkg/kv/kvserver/gossip_test.go +++ b/pkg/kv/kvserver/gossip_test.go @@ -18,15 +18,9 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/security/username" - "github.com/cockroachdb/cockroach/pkg/server" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" @@ -34,7 +28,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/retry" - "github.com/stretchr/testify/require" ) func TestGossipFirstRange(t *testing.T) { @@ -211,92 +204,3 @@ func TestGossipHandlesReplacedNode(t *testing.T) { } } } - -// TestGossipAfterAbortOfSystemConfigTransactionAfterFailureDueToIntents tests -// that failures to gossip the system config due to intents are rectified when -// later intents are aborted. -// -// Note that this tests the gossip functionality only in the mixed version -// state. After the release is finalized, these gossip triggers will no longer -// happen. -// -// TODO(ajwerner): Delete this test in 22.2. -func TestGossipAfterAbortOfSystemConfigTransactionAfterFailureDueToIntents(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - ctx := context.Background() - settings := cluster.MakeTestingClusterSettingsWithVersions( - clusterversion.TestingBinaryMinSupportedVersion, - clusterversion.TestingBinaryMinSupportedVersion, - false, - ) - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ - ServerArgs: base.TestServerArgs{ - Settings: settings, - Knobs: base.TestingKnobs{ - Store: &kvserver.StoreTestingKnobs{ - DisableMergeQueue: true, - }, - Server: &server.TestingKnobs{ - BinaryVersionOverride: clusterversion.TestingBinaryMinSupportedVersion, - DisableAutomaticVersionUpgrade: make(chan struct{}), - }, - }, - }, - }) - defer tc.Stopper().Stop(ctx) - require.NoError(t, tc.WaitForFullReplication()) - - db := tc.Server(0).DB() - - txA := db.NewTxn(ctx, "a") - txB := db.NewTxn(ctx, "b") - - require.NoError(t, txA.DeprecatedSetSystemConfigTrigger(true /* forSystemTenant */)) - db1000 := dbdesc.NewInitial(1000, "1000", username.AdminRoleName()) - require.NoError(t, txA.Put(ctx, - keys.SystemSQLCodec.DescMetadataKey(1000), - db1000.DescriptorProto())) - - require.NoError(t, txB.DeprecatedSetSystemConfigTrigger(true /* forSystemTenant */)) - db2000 := dbdesc.NewInitial(2000, "2000", username.AdminRoleName()) - require.NoError(t, txB.Put(ctx, - keys.SystemSQLCodec.DescMetadataKey(2000), - db2000.DescriptorProto())) - - const someTime = 10 * time.Millisecond - clearNotifictions := func(ch <-chan struct{}) { - for { - select { - case <-ch: - case <-time.After(someTime): - return - } - } - } - systemConfChangeCh := tc.Server(0).GossipI().(*gossip.Gossip).DeprecatedRegisterSystemConfigChannel() - clearNotifictions(systemConfChangeCh) - require.NoError(t, txB.Commit(ctx)) - select { - case <-systemConfChangeCh: - // This case is rare but happens sometimes. We gossip the node liveness - // in a bunch of cases so we just let the test finish here. The important - // thing is that sometimes we get to the next phase. - t.Log("got unexpected update. This can happen for a variety of " + - "reasons like lease transfers. The test is exiting without testing anything") - return - case <-time.After(someTime): - // Did not expect an update so this is the happy case - } - // Roll back the transaction which had laid down the intent which blocked the - // earlier gossip update, make sure we get a gossip notification now. - const aLongTime = 20 * someTime - require.NoError(t, txA.Rollback(ctx)) - select { - case <-systemConfChangeCh: - // Got an update. - case <-time.After(aLongTime): - t.Fatal("expected update") - } -} diff --git a/pkg/kv/kvserver/mvcc_gc_queue_test.go b/pkg/kv/kvserver/mvcc_gc_queue_test.go index 84c22e4d2251..9afde86d9d75 100644 --- a/pkg/kv/kvserver/mvcc_gc_queue_test.go +++ b/pkg/kv/kvserver/mvcc_gc_queue_test.go @@ -585,9 +585,9 @@ func TestMVCCGCQueueProcess(t *testing.T) { } } - cfg := tc.gossip.DeprecatedGetSystemConfig() - if cfg == nil { - t.Fatal("config not set") + cfg, err := tc.store.GetConfReader(ctx) + if err != nil { + t.Fatal(err) } // The total size of the GC'able versions of the keys and values in Info. @@ -870,9 +870,9 @@ func TestMVCCGCQueueTransactionTable(t *testing.T) { // Run GC. mgcq := newMVCCGCQueue(tc.store) - cfg := tc.gossip.DeprecatedGetSystemConfig() - if cfg == nil { - t.Fatal("config not set") + cfg, err := tc.store.GetConfReader(ctx) + if err != nil { + t.Fatal(err) } processed, err := mgcq.process(ctx, tc.repl, cfg) diff --git a/pkg/kv/kvserver/queue_test.go b/pkg/kv/kvserver/queue_test.go index acb3ff5da833..8ecc69133dc5 100644 --- a/pkg/kv/kvserver/queue_test.go +++ b/pkg/kv/kvserver/queue_test.go @@ -691,20 +691,21 @@ func TestAcceptsUnsplitRanges(t *testing.T) { bq.Start(stopper) // Check our config. - var sysCfg *config.SystemConfig + var cfg spanconfig.StoreReader testutils.SucceedsSoon(t, func() error { - sysCfg = s.cfg.Gossip.DeprecatedGetSystemConfig() - if sysCfg == nil { + cfg, err = bq.store.GetConfReader(ctx) + require.NoError(t, err) + if cfg == nil { return errors.New("system config not yet present") } return nil }) neverSplitsDesc := neverSplits.Desc() - if sysCfg.NeedsSplit(ctx, neverSplitsDesc.StartKey, neverSplitsDesc.EndKey) { + if cfg.NeedsSplit(ctx, neverSplitsDesc.StartKey, neverSplitsDesc.EndKey) { t.Fatal("System config says range needs to be split") } willSplitDesc := willSplit.Desc() - if sysCfg.NeedsSplit(ctx, willSplitDesc.StartKey, willSplitDesc.EndKey) { + if cfg.NeedsSplit(ctx, willSplitDesc.StartKey, willSplitDesc.EndKey) { t.Fatal("System config says range needs to be split") } @@ -736,11 +737,11 @@ func TestAcceptsUnsplitRanges(t *testing.T) { // Check our config. neverSplitsDesc = neverSplits.Desc() - if sysCfg.NeedsSplit(ctx, neverSplitsDesc.StartKey, neverSplitsDesc.EndKey) { + if cfg.NeedsSplit(ctx, neverSplitsDesc.StartKey, neverSplitsDesc.EndKey) { t.Fatal("System config says range needs to be split") } willSplitDesc = willSplit.Desc() - if !sysCfg.NeedsSplit(ctx, willSplitDesc.StartKey, willSplitDesc.EndKey) { + if !cfg.NeedsSplit(ctx, willSplitDesc.StartKey, willSplitDesc.EndKey) { t.Fatal("System config says range does not need to be split") } diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 70344ec484e8..d38220cfb52e 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -2003,18 +2003,6 @@ func (r *Replica) GetExternalStorageFromURI( return r.store.cfg.ExternalStorageFromURI(ctx, uri, user) } -func (r *Replica) markSystemConfigGossipSuccess() { - r.mu.Lock() - defer r.mu.Unlock() - r.mu.failureToGossipSystemConfig = false -} - -func (r *Replica) markSystemConfigGossipFailed() { - r.mu.Lock() - defer r.mu.Unlock() - r.mu.failureToGossipSystemConfig = true -} - // GetResponseMemoryAccount implements the batcheval.EvalContext interface. func (r *Replica) GetResponseMemoryAccount() *mon.BoundAccount { // Return an empty account, which places no limits. Places where a real diff --git a/pkg/kv/kvserver/replica_gossip.go b/pkg/kv/kvserver/replica_gossip.go index a8a4da9e7757..476d70f492a5 100644 --- a/pkg/kv/kvserver/replica_gossip.go +++ b/pkg/kv/kvserver/replica_gossip.go @@ -13,7 +13,6 @@ package kvserver import ( "context" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" @@ -65,91 +64,6 @@ func (r *Replica) shouldGossip(ctx context.Context) bool { return r.OwnsValidLease(ctx, r.store.Clock().NowAsClockTimestamp()) } -// MaybeGossipSystemConfigRaftMuLocked scans the entire SystemConfig span and -// gossips it. Further calls come from the trigger on EndTxn or range lease -// acquisition. -// -// Note that MaybeGossipSystemConfigRaftMuLocked gossips information only when -// the lease is actually held. The method does not request a range lease here -// since RequestLease and applyRaftCommand call the method and we need to avoid -// deadlocking in redirectOnOrAcquireLease. -// -// MaybeGossipSystemConfigRaftMuLocked must only be called from Raft commands -// while holding the raftMu (which provide the necessary serialization to avoid -// data races). -// -// TODO(nvanbenschoten,bdarnell): even though this is best effort, we should log -// louder when we continually fail to gossip system config. -// -// TODO(ajwerner): Remove this in 22.2. -func (r *Replica) MaybeGossipSystemConfigRaftMuLocked(ctx context.Context) error { - if r.ClusterSettings().Version.IsActive( - ctx, clusterversion.DisableSystemConfigGossipTrigger, - ) { - return nil - } - r.raftMu.AssertHeld() - if r.store.Gossip() == nil { - log.VEventf(ctx, 2, "not gossiping system config because gossip isn't initialized") - return nil - } - if !r.IsInitialized() { - log.VEventf(ctx, 2, "not gossiping system config because the replica isn't initialized") - return nil - } - if !r.ContainsKey(keys.SystemConfigSpan.Key) { - log.VEventf(ctx, 3, - "not gossiping system config because the replica doesn't contain the system config's start key") - return nil - } - if !r.shouldGossip(ctx) { - log.VEventf(ctx, 2, "not gossiping system config because the replica doesn't hold the lease") - return nil - } - - // TODO(marc): check for bad split in the middle of the SystemConfig span. - loadedCfg, err := r.loadSystemConfig(ctx) - if err != nil { - if errors.Is(err, errSystemConfigIntent) { - log.VEventf(ctx, 2, "not gossiping system config because intents were found on SystemConfigSpan") - r.markSystemConfigGossipFailed() - return nil - } - return errors.Wrap(err, "could not load SystemConfig span") - } - - if gossipedCfg := r.store.Gossip().DeprecatedGetSystemConfig(); gossipedCfg != nil && - gossipedCfg.Equal(loadedCfg) && - r.store.Gossip().InfoOriginatedHere(gossip.KeyDeprecatedSystemConfig) { - log.VEventf(ctx, 2, "not gossiping unchanged system config") - // Clear the failure bit if all intents have been resolved but there's - // nothing new to gossip. - r.markSystemConfigGossipSuccess() - return nil - } - - log.VEventf(ctx, 2, "gossiping system config") - if err := r.store.Gossip().AddInfoProto(gossip.KeyDeprecatedSystemConfig, loadedCfg, 0); err != nil { - return errors.Wrap(err, "failed to gossip system config") - } - r.markSystemConfigGossipSuccess() - return nil -} - -// MaybeGossipSystemConfigIfHaveFailureRaftMuLocked is a trigger to gossip the -// system config due to an abort of a transaction keyed in the system config -// span. It will call MaybeGossipSystemConfigRaftMuLocked if -// failureToGossipSystemConfig is true. -func (r *Replica) MaybeGossipSystemConfigIfHaveFailureRaftMuLocked(ctx context.Context) error { - r.mu.RLock() - failed := r.mu.failureToGossipSystemConfig - r.mu.RUnlock() - if !failed { - return nil - } - return r.MaybeGossipSystemConfigRaftMuLocked(ctx) -} - // MaybeGossipNodeLivenessRaftMuLocked gossips information for all node liveness // records stored on this range. To scan and gossip, this replica must hold the // lease to a range which contains some or all of the node liveness records. diff --git a/pkg/kv/kvserver/replica_proposal.go b/pkg/kv/kvserver/replica_proposal.go index 47088aea8345..b52ec9fd2d2c 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -401,9 +401,6 @@ func (r *Replica) leasePostApplyLocked( // Nothing to do. return } - if err := r.MaybeGossipSystemConfigRaftMuLocked(ctx); err != nil { - log.Errorf(ctx, "%v", err) - } if err := r.MaybeGossipNodeLivenessRaftMuLocked(ctx, keys.NodeLivenessSpan); err != nil { log.Errorf(ctx, "%v", err) } @@ -596,26 +593,6 @@ func (r *Replica) handleReadWriteLocalEvalResult(ctx context.Context, lResult re lResult.MaybeAddToSplitQueue = false } - // The gossip triggers below require raftMu to be held, but - // handleReadWriteLocalEvalResult() may be called from non-Raft code paths (in - // particular for noop proposals). LocalResult.RequiresRaft() will force - // results that set these gossip triggers to always go via Raft such that - // raftMu is held. The triggers assert that callers hold the mutex during race - // tests via raftMu.AssertHeld(). - if lResult.MaybeGossipSystemConfig { - if err := r.MaybeGossipSystemConfigRaftMuLocked(ctx); err != nil { - log.Errorf(ctx, "%v", err) - } - lResult.MaybeGossipSystemConfig = false - } - - if lResult.MaybeGossipSystemConfigIfHaveFailure { - if err := r.MaybeGossipSystemConfigIfHaveFailureRaftMuLocked(ctx); err != nil { - log.Errorf(ctx, "%v", err) - } - lResult.MaybeGossipSystemConfigIfHaveFailure = false - } - if lResult.MaybeGossipNodeLiveness != nil { if err := r.MaybeGossipNodeLivenessRaftMuLocked(ctx, *lResult.MaybeGossipNodeLiveness); err != nil { log.Errorf(ctx, "%v", err) diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 00f5287fd3f3..9a069683fc59 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -30,7 +30,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/cli/exit" "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" @@ -218,8 +217,6 @@ func (tc *testContext) StartWithStoreConfigAndVersion( tc.transport = store.cfg.Transport tc.engine = store.engine tc.store = store - // TODO(tbg): see if this is needed. Would like to remove it. - require.NoError(t, tc.initConfigs(t)) } func (tc *testContext) Sender() kv.Sender { @@ -252,27 +249,6 @@ func (tc *testContext) SendWrapped(args roachpb.Request) (roachpb.Response, *roa return tc.SendWrappedWith(roachpb.Header{}, args) } -// initConfigs creates default configuration entries. -// -// TODO(ajwerner): Remove this in 22.2. -func (tc *testContext) initConfigs(t testing.TB) error { - // Put an empty system config into gossip so that gossip callbacks get - // run. We're using a fake config, but it's hooked into SystemConfig. - if err := tc.gossip.AddInfoProto(gossip.KeyDeprecatedSystemConfig, - &config.SystemConfigEntries{}, 0); err != nil { - return err - } - - testutils.SucceedsSoon(t, func() error { - if cfg := tc.gossip.DeprecatedGetSystemConfig(); cfg == nil { - return errors.Errorf("expected system config to be set") - } - return nil - }) - - return nil -} - // addBogusReplicaToRangeDesc modifies the range descriptor to include a second // replica. This is useful for tests that want to pretend they're transferring // the range lease away, as the lease can only be obtained by Replicas which are @@ -1195,14 +1171,6 @@ func TestReplicaGossipConfigsOnLease(t *testing.T) { t.Fatal(err) } - // If this actually failed, we would have gossiped from MVCCPutProto. - // Unlikely, but why not check. - if cfg := tc.gossip.DeprecatedGetSystemConfig(); cfg != nil { - if nv := len(cfg.Values); nv == 1 && cfg.Values[nv-1].Key.Equal(key) { - t.Errorf("unexpected gossip of system config: %s", cfg) - } - } - // Expire our own lease which we automagically acquired due to being // first range and config holder. tc.manualClock.Set(leaseExpiry(tc.repl)) @@ -1221,12 +1189,6 @@ func TestReplicaGossipConfigsOnLease(t *testing.T) { tc.manualClock.Increment(11 + int64(tc.Clock().MaxOffset())) // advance time now = tc.Clock().NowAsClockTimestamp() - ch := tc.gossip.DeprecatedRegisterSystemConfigChannel() - select { - case <-ch: - default: - } - // Give lease to this range. if err := sendLeaseRequest(tc.repl, &roachpb.Lease{ Start: now.ToTimestamp().Add(11, 0).UnsafeToClockTimestamp(), @@ -1239,24 +1201,6 @@ func TestReplicaGossipConfigsOnLease(t *testing.T) { }); err != nil { t.Fatal(err) } - - testutils.SucceedsSoon(t, func() error { - sysCfg := tc.gossip.DeprecatedGetSystemConfig() - if sysCfg == nil { - return errors.Errorf("no system config yet") - } - var found bool - for _, cur := range sysCfg.Values { - if key.Equal(cur.Key) { - found = true - break - } - } - if !found { - return errors.Errorf("key %s not found in SystemConfig", key) - } - return nil - }) } // TestReplicaTSCacheLowWaterOnLease verifies that the low water mark @@ -1554,20 +1498,6 @@ func TestReplicaGossipFirstRange(t *testing.T) { } } -// TestReplicaGossipAllConfigs verifies that all config types are gossiped. -func TestReplicaGossipAllConfigs(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - ctx := context.Background() - tc := testContext{} - stopper := stop.NewStopper() - defer stopper.Stop(ctx) - tc.Start(ctx, t, stopper) - if cfg := tc.gossip.DeprecatedGetSystemConfig(); cfg == nil { - t.Fatal("config not set") - } -} - func getArgs(key []byte) roachpb.GetRequest { return roachpb.GetRequest{ RequestHeader: roachpb.RequestHeader{ @@ -10426,10 +10356,13 @@ func TestConsistenctQueueErrorFromCheckConsistency(t *testing.T) { tc := testContext{} tc.StartWithStoreConfig(ctx, t, stopper, cfg) + confReader, err := tc.store.GetConfReader(ctx) + if err != nil { + t.Fatal(err) + } for i := 0; i < 2; i++ { // Do this twice because it used to deadlock. See #25456. - sysCfg := tc.store.Gossip().DeprecatedGetSystemConfig() - processed, err := tc.store.consistencyQueue.process(ctx, tc.repl, sysCfg) + processed, err := tc.store.consistencyQueue.process(ctx, tc.repl, confReader) if !testutils.IsError(err, "boom") { t.Fatal(err) } diff --git a/pkg/kv/kvserver/split_queue_test.go b/pkg/kv/kvserver/split_queue_test.go index 9e75db36e0a6..27111c5ecaad 100644 --- a/pkg/kv/kvserver/split_queue_test.go +++ b/pkg/kv/kvserver/split_queue_test.go @@ -70,9 +70,9 @@ func TestSplitQueueShouldQueue(t *testing.T) { {roachpb.RKey(keys.SystemSQLCodec.TablePrefix(2001)), roachpb.RKeyMax, 32<<20 + 1, 64 << 20, true, 1}, } - cfg := tc.gossip.DeprecatedGetSystemConfig() - if cfg == nil { - t.Fatal("config not set") + cfg, err := tc.store.GetConfReader(ctx) + if err != nil { + t.Fatal(err) } for i, test := range testCases { // Create a replica for testing that is not hooked up to the store. This diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 581b8a95fa18..71f8d9068d49 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -1068,6 +1068,7 @@ type StoreConfig struct { // SpanConfigsDisabled determines whether we're able to use the span configs // infrastructure or not. + // TODO(richardjcai): We can likely remove this. SpanConfigsDisabled bool // Used to subscribe to span configuration changes, keeping up-to-date a // data structure useful for retrieving span configs. Only available if @@ -2193,7 +2194,6 @@ func (s *Store) GetConfReader(ctx context.Context) (spanconfig.StoreReader, erro if s.cfg.SpanConfigsDisabled || !spanconfigstore.EnabledSetting.Get(&s.ClusterSettings().SV) || - !s.cfg.Settings.Version.IsActive(ctx, clusterversion.EnableSpanConfigStore) || s.TestingKnobs().UseSystemConfigSpanForQueues { sysCfg := s.cfg.SystemConfigProvider.GetSystemConfig() diff --git a/pkg/kv/mock_transactional_sender.go b/pkg/kv/mock_transactional_sender.go index 0af00a2da39c..d807546ad2a8 100644 --- a/pkg/kv/mock_transactional_sender.go +++ b/pkg/kv/mock_transactional_sender.go @@ -16,7 +16,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/errors" ) // MockTransactionalSender allows a function to be used as a TxnSender. @@ -66,11 +65,6 @@ func (m *MockTransactionalSender) UpdateRootWithLeafFinalState( panic("unimplemented") } -// AnchorOnSystemConfigRange is part of the TxnSender interface. -func (m *MockTransactionalSender) AnchorOnSystemConfigRange() error { - return errors.New("unimplemented") -} - // TxnStatus is part of the TxnSender interface. func (m *MockTransactionalSender) TxnStatus() roachpb.TransactionStatus { return m.txn.Status diff --git a/pkg/kv/sender.go b/pkg/kv/sender.go index 71fc61d7590c..25450f472284 100644 --- a/pkg/kv/sender.go +++ b/pkg/kv/sender.go @@ -96,17 +96,6 @@ type Sender interface { type TxnSender interface { Sender - // AnchorOnSystemConfigRange ensures that the transaction record, - // if/when it will be created, will be created on the system config - // range. This is useful because some commit triggers only work when - // the EndTxn is evaluated on that range. - // - // An error is returned if the transaction's key has already been - // set by anything other than a previous call to this function - // (i.e. if the transaction already performed any writes). - // It is allowed to call this method multiple times. - AnchorOnSystemConfigRange() error - // GetLeafTxnInputState retrieves the input state necessary and // sufficient to initialize a LeafTxn from the current RootTxn. // diff --git a/pkg/kv/txn.go b/pkg/kv/txn.go index 70282aaffc1b..c6c6a5125032 100644 --- a/pkg/kv/txn.go +++ b/pkg/kv/txn.go @@ -63,9 +63,6 @@ type Txn struct { // commitTriggers are run upon successful commit. commitTriggers []func(ctx context.Context) - // systemConfigTrigger is set to true when modifying keys from the SystemConfig - // span. This sets the SystemConfigTrigger on EndTxnRequest. - systemConfigTrigger bool // mu holds fields that need to be synchronized for concurrent request execution. mu struct { @@ -402,30 +399,6 @@ func (txn *Txn) RequiredFrontier() hlc.Timestamp { return txn.mu.sender.RequiredFrontier() } -// DeprecatedSetSystemConfigTrigger sets the system db trigger to true on this transaction. -// This will impact the EndTxnRequest. Note that this method takes a boolean -// argument indicating whether this transaction is intended for the system -// tenant. Only transactions for the system tenant need to set the system config -// trigger which is used to gossip updates to the system config to KV servers. -// The KV servers need access to an up-to-date system config in order to -// determine split points and zone configurations. -func (txn *Txn) DeprecatedSetSystemConfigTrigger(forSystemTenant bool) error { - if txn.typ != RootTxn { - return errors.AssertionFailedf("DeprecatedSetSystemConfigTrigger() called on leaf txn") - } - if !forSystemTenant { - return nil - } - - txn.mu.Lock() - defer txn.mu.Unlock() - if err := txn.mu.sender.AnchorOnSystemConfigRange(); err != nil { - return err - } - txn.systemConfigTrigger = true - return nil -} - // DisablePipelining instructs the transaction not to pipeline requests. It // should rarely be necessary to call this method. // @@ -701,7 +674,7 @@ func (txn *Txn) commit(ctx context.Context) error { // to reduce contention by releasing locks. In multi-tenant settings, it // will be subject to admission control, and the zero CreateTime will give // it preference within the tenant. - et := endTxnReq(true /* commit */, txn.deadline(), txn.systemConfigTrigger) + et := endTxnReq(true, txn.deadline()) ba := roachpb.BatchRequest{Requests: et.unionArr[:]} _, pErr := txn.Send(ctx, ba) if pErr == nil { @@ -757,7 +730,7 @@ func (txn *Txn) CommitInBatch(ctx context.Context, b *Batch) error { if txn != b.txn { return errors.Errorf("a batch b can only be committed by b.txn") } - et := endTxnReq(true /* commit */, txn.deadline(), txn.systemConfigTrigger) + et := endTxnReq(true, txn.deadline()) b.growReqs(1) b.reqs[len(b.reqs)-1].Value = &et.union b.initResult(1 /* calls */, 0, b.raw, nil) @@ -877,7 +850,7 @@ func (txn *Txn) rollback(ctx context.Context) *roachpb.Error { // order to reduce contention by releasing locks. In multi-tenant // settings, it will be subject to admission control, and the zero // CreateTime will give it preference within the tenant. - et := endTxnReq(false /* commit */, nil /* deadline */, false /* systemConfigTrigger */) + et := endTxnReq(false, nil /* deadline */) ba := roachpb.BatchRequest{Requests: et.unionArr[:]} _, pErr := txn.Send(ctx, ba) if pErr == nil { @@ -903,7 +876,7 @@ func (txn *Txn) rollback(ctx context.Context) *roachpb.Error { // order to reduce contention by releasing locks. In multi-tenant // settings, it will be subject to admission control, and the zero // CreateTime will give it preference within the tenant. - et := endTxnReq(false /* commit */, nil /* deadline */, false /* systemConfigTrigger */) + et := endTxnReq(false, nil /* deadline */) ba := roachpb.BatchRequest{Requests: et.unionArr[:]} _ = contextutil.RunWithTimeout(ctx, "async txn rollback", asyncRollbackTimeout, func(ctx context.Context) error { @@ -945,17 +918,10 @@ type endTxnReqAlloc struct { unionArr [1]roachpb.RequestUnion } -func endTxnReq(commit bool, deadline *hlc.Timestamp, hasTrigger bool) *endTxnReqAlloc { +func endTxnReq(commit bool, deadline *hlc.Timestamp) *endTxnReqAlloc { alloc := new(endTxnReqAlloc) alloc.req.Commit = commit alloc.req.Deadline = deadline - if hasTrigger { - alloc.req.InternalCommitTrigger = &roachpb.InternalCommitTrigger{ - ModifiedSpanTrigger: &roachpb.ModifiedSpanTrigger{ - SystemConfigSpan: true, - }, - } - } alloc.union.EndTxn = &alloc.req alloc.unionArr[0].Value = &alloc.union return alloc diff --git a/pkg/kv/txn_test.go b/pkg/kv/txn_test.go index 8c2470a5bfbd..d55805698d1b 100644 --- a/pkg/kv/txn_test.go +++ b/pkg/kv/txn_test.go @@ -538,27 +538,6 @@ func TestUpdateDeadlineMaybe(t *testing.T) { } } -// Test that, if DeprecatedSetSystemConfigTrigger() fails, the systemConfigTrigger has not -// been set. -func TestAnchoringErrorNoTrigger(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - ctx := context.Background() - stopper := stop.NewStopper() - defer stopper.Stop(ctx) - - mc := hlc.NewManualClock(1) - clock := hlc.NewClock(mc, time.Nanosecond /* maxOffset */) - db := NewDB(log.MakeTestingAmbientCtxWithNewTracer(), MakeMockTxnSenderFactory( - func(context.Context, *roachpb.Transaction, roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { - return nil, nil - }), clock, stopper) - txn := NewTxn(ctx, db, 0 /* gatewayNodeID */) - require.EqualError(t, txn.DeprecatedSetSystemConfigTrigger(true /* forSystemTenant */), "unimplemented") - require.False(t, txn.systemConfigTrigger) -} - // TestTxnNegotiateAndSend tests the behavior of NegotiateAndSend, both when the // server-side fast path is possible (for single-range reads) and when it is not // (for cross-range reads). diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index 35f1c56de3d2..39c964412ced 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -855,8 +855,6 @@ func (ct InternalCommitTrigger) Kind() redact.SafeString { return "change-replicas" case ct.ModifiedSpanTrigger != nil: switch { - case ct.ModifiedSpanTrigger.SystemConfigSpan: - return "modified-span (system-config)" case ct.ModifiedSpanTrigger.NodeLivenessSpan != nil: return "modified-span (node-liveness)" default: diff --git a/pkg/roachpb/data.proto b/pkg/roachpb/data.proto index 46d84c2acf74..060a73be797d 100644 --- a/pkg/roachpb/data.proto +++ b/pkg/roachpb/data.proto @@ -249,7 +249,7 @@ message ChangeReplicasTrigger { // ModifiedSpanTrigger indicates that a specific span has been modified. // This can be used to trigger scan-and-gossip for the given span. message ModifiedSpanTrigger { - bool system_config_span = 1; + reserved 1; // node_liveness_span is set to indicate that node liveness records // need re-gossiping after modification or range lease updates. The // span is set to a single key when nodes update their liveness records diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index dc5157f6d345..2bf0e54e7fa9 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -400,8 +400,6 @@ go_test( "//pkg/settings/cluster", "//pkg/spanconfig", "//pkg/sql", - "//pkg/sql/catalog/catalogkeys", - "//pkg/sql/catalog/dbdesc", "//pkg/sql/catalog/descpb", "//pkg/sql/execinfrapb", "//pkg/sql/idxusage", diff --git a/pkg/server/server_test.go b/pkg/server/server_test.go index 8eb10126bada..01c6c4b2a101 100644 --- a/pkg/server/server_test.go +++ b/pkg/server/server_test.go @@ -11,7 +11,6 @@ package server import ( - "bytes" "compress/gzip" "context" "fmt" @@ -32,22 +31,15 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/cli/exit" - "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/server/status/statuspb" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -396,109 +388,6 @@ func TestAcceptEncoding(t *testing.T) { } } -// TestSystemConfigGossip tests that system config gossip works in the mixed -// version state. After the 22.1 release is finalized, system config gossip -// will no longer occur. -// -// TODO(ajwerner): Delete this test in 22.2. -func TestSystemConfigGossip(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - ctx := context.Background() - settings := cluster.MakeTestingClusterSettingsWithVersions( - clusterversion.TestingBinaryMinSupportedVersion, - clusterversion.TestingBinaryMinSupportedVersion, - false, - ) - serverArgs := base.TestServerArgs{ - Settings: settings, - Knobs: base.TestingKnobs{ - Store: &kvserver.StoreTestingKnobs{ - DisableMergeQueue: true, - }, - Server: &TestingKnobs{ - BinaryVersionOverride: clusterversion.TestingBinaryMinSupportedVersion, - DisableAutomaticVersionUpgrade: make(chan struct{}), - }, - }, - } - s, _, kvDB := serverutils.StartServer(t, serverArgs) - defer s.Stopper().Stop(ctx) - ts := s.(*TestServer) - - key := catalogkeys.MakeDescMetadataKey(keys.SystemSQLCodec, descpb.ID(keys.MaxSystemConfigDescID+1)) - valAt := func(i int) *descpb.Descriptor { - return dbdesc.NewInitial( - descpb.ID(i), "foo", username.AdminRoleName(), - ).DescriptorProto() - } - - // Register a callback for gossip updates. - resultChan := ts.Gossip().DeprecatedRegisterSystemConfigChannel() - - // The span gets gossiped when it first shows up. - select { - case <-resultChan: - - case <-time.After(500 * time.Millisecond): - t.Fatal("did not receive gossip message") - } - - // Write a system key with the transaction marked as having a Gossip trigger. - if err := kvDB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - if err := txn.DeprecatedSetSystemConfigTrigger(true /* forSystemTenant */); err != nil { - return err - } - return txn.Put(ctx, key, valAt(2)) - }); err != nil { - t.Fatal(err) - } - - // This has to be wrapped in a SucceedSoon because system upgrades on the - // testserver's startup can trigger system config updates without the key we - // wrote. - testutils.SucceedsSoon(t, func() error { - // New system config received. - var systemConfig *config.SystemConfig - select { - case <-resultChan: - systemConfig = ts.gossip.DeprecatedGetSystemConfig() - - case <-time.After(500 * time.Millisecond): - return errors.Errorf("did not receive gossip message") - } - - // Now check the new config. - var val *roachpb.Value - for _, kv := range systemConfig.Values { - if bytes.Equal(key, kv.Key) { - val = &kv.Value - break - } - } - if val == nil { - return errors.Errorf("key not found in gossiped info") - } - - // Make sure the returned value is valAt(2). - var got descpb.Descriptor - if err := val.GetProto(&got); err != nil { - return err - } - - _, expected, _, _ := descpb.FromDescriptor(valAt(2)) - _, db, _, _ := descpb.FromDescriptor(&got) - if db == nil { - panic(errors.Errorf("found nil database: %v", got)) - } - if !reflect.DeepEqual(*db, *expected) { - panic(errors.Errorf("mismatch: expected %+v, got %+v", *expected, *db)) - } - return nil - }) -} - func TestListenerFileCreation(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/spanconfig/spanconfigstore/store.go b/pkg/spanconfig/spanconfigstore/store.go index 02552a9b1450..6beff543e175 100644 --- a/pkg/spanconfig/spanconfigstore/store.go +++ b/pkg/spanconfig/spanconfigstore/store.go @@ -28,6 +28,7 @@ import ( // 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(richardjcai): We can likely remove this. var EnabledSetting = settings.RegisterBoolSetting( settings.SystemOnly, "spanconfig.store.enabled", diff --git a/pkg/sql/catalog/descs/txn.go b/pkg/sql/catalog/descs/txn.go index 80717a3e6c0c..6226a8c679c1 100644 --- a/pkg/sql/catalog/descs/txn.go +++ b/pkg/sql/catalog/descs/txn.go @@ -16,7 +16,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -82,15 +81,6 @@ func (cf *CollectionFactory) Txn( deletedDescs = catalog.DescriptorIDSet{} descsCol = cf.MakeCollection(ctx, nil /* temporarySchemaProvider */, nil /* monitor */) defer descsCol.ReleaseAll(ctx) - if !cf.settings.Version.IsActive( - ctx, clusterversion.DisableSystemConfigGossipTrigger, - ) { - if err := txn.DeprecatedSetSystemConfigTrigger( - cf.leaseMgr.Codec().ForSystemTenant(), - ); err != nil { - return err - } - } if err := f(ctx, txn, &descsCol); err != nil { return err } diff --git a/pkg/sql/gcjob/BUILD.bazel b/pkg/sql/gcjob/BUILD.bazel index 13c143c4a819..0d5ce27d9dd9 100644 --- a/pkg/sql/gcjob/BUILD.bazel +++ b/pkg/sql/gcjob/BUILD.bazel @@ -15,7 +15,6 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/gcjob", visibility = ["//visibility:public"], deps = [ - "//pkg/clusterversion", "//pkg/config", "//pkg/config/zonepb", "//pkg/jobs", diff --git a/pkg/sql/gcjob/descriptor_utils.go b/pkg/sql/gcjob/descriptor_utils.go index 63dc4171f798..9b4f5d303bb0 100644 --- a/pkg/sql/gcjob/descriptor_utils.go +++ b/pkg/sql/gcjob/descriptor_utils.go @@ -13,7 +13,6 @@ package gcjob import ( "context" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" @@ -75,13 +74,6 @@ func deleteDatabaseZoneConfig( return nil } return db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - if !settings.Version.IsActive( - ctx, clusterversion.DisableSystemConfigGossipTrigger, - ) { - if err := txn.DeprecatedSetSystemConfigTrigger(codec.ForSystemTenant()); err != nil { - return err - } - } b := &kv.Batch{} // Delete the zone config entry for the dropped database associated with the diff --git a/pkg/sql/opt/exec/execbuilder/BUILD.bazel b/pkg/sql/opt/exec/execbuilder/BUILD.bazel index 53a095e28194..6db700a483ce 100644 --- a/pkg/sql/opt/exec/execbuilder/BUILD.bazel +++ b/pkg/sql/opt/exec/execbuilder/BUILD.bazel @@ -14,7 +14,6 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder", visibility = ["//visibility:public"], deps = [ - "//pkg/clusterversion", "//pkg/server/telemetry", "//pkg/sql/catalog/colinfo", "//pkg/sql/catalog/descpb", diff --git a/pkg/sql/opt/exec/execbuilder/relational.go b/pkg/sql/opt/exec/execbuilder/relational.go index a8503b9354cb..aaae2d467235 100644 --- a/pkg/sql/opt/exec/execbuilder/relational.go +++ b/pkg/sql/opt/exec/execbuilder/relational.go @@ -15,7 +15,6 @@ import ( "context" "fmt" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -153,23 +152,6 @@ func (b *Builder) buildRelational(e memo.RelExpr) (execPlan, error) { // Mark the statement as containing DDL for use // in the SQL executor. b.IsDDL = true - - // This will set the system DB trigger for transactions containing - // schema-modifying statements that have no effect, such as - // `BEGIN; INSERT INTO ...; CREATE TABLE IF NOT EXISTS ...; COMMIT;` - // where the table already exists. This will generate some false schema - // cache refreshes, but that's expected to be quite rare in practice. - if !b.evalCtx.Settings.Version.IsActive( - b.evalCtx.Ctx(), clusterversion.DisableSystemConfigGossipTrigger, - ) { - if err := b.evalCtx.Txn.DeprecatedSetSystemConfigTrigger(b.evalCtx.Codec.ForSystemTenant()); err != nil { - return execPlan{}, errors.WithSecondaryError( - unimplemented.NewWithIssuef(26508, - "the first schema change statement in a transaction must precede any writes"), - err) - } - } - } if opt.IsMutationOp(e) { diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index 938312ff1945..d75389b0c010 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -1293,10 +1293,6 @@ func (ef *execFactory) ConstructInsert( tabDesc := table.(*optTable).desc cols := makeColList(table, insertColOrdSet) - if err := ef.planner.maybeSetSystemConfig(tabDesc.GetID()); err != nil { - return nil, err - } - // Create the table inserter, which does the bulk of the work. internal := ef.planner.SessionData().Internal ri, err := row.MakeInserter( @@ -1368,10 +1364,6 @@ func (ef *execFactory) ConstructInsertFastPath( tabDesc := table.(*optTable).desc cols := makeColList(table, insertColOrdSet) - if err := ef.planner.maybeSetSystemConfig(tabDesc.GetID()); err != nil { - return nil, err - } - // Create the table inserter, which does the bulk of the work. internal := ef.planner.SessionData().Internal ri, err := row.MakeInserter( @@ -1465,10 +1457,6 @@ func (ef *execFactory) ConstructUpdate( tabDesc := table.(*optTable).desc fetchCols := makeColList(table, fetchColOrdSet) - if err := ef.planner.maybeSetSystemConfig(tabDesc.GetID()); err != nil { - return nil, err - } - // Add each column to update as a sourceSlot. The CBO only uses scalarSlot, // since it compiles tuples and subqueries into a simple sequence of target // columns. @@ -1580,10 +1568,6 @@ func (ef *execFactory) ConstructUpsert( fetchCols := makeColList(table, fetchColOrdSet) updateCols := makeColList(table, updateColOrdSet) - if err := ef.planner.maybeSetSystemConfig(tabDesc.GetID()); err != nil { - return nil, err - } - // Create the table inserter, which does the bulk of the insert-related work. internal := ef.planner.SessionData().Internal ri, err := row.MakeInserter( @@ -1677,10 +1661,6 @@ func (ef *execFactory) ConstructDelete( tabDesc := table.(*optTable).desc fetchCols := makeColList(table, fetchColOrdSet) - if err := ef.planner.maybeSetSystemConfig(tabDesc.GetID()); err != nil { - return nil, err - } - // Create the table deleter, which does the bulk of the work. In the HP, // the deleter derives the columns that need to be fetched. By contrast, the // CBO will have already determined the set of fetch columns, and passes @@ -1742,10 +1722,6 @@ func (ef *execFactory) ConstructDeleteRange( var sb span.Builder sb.Init(ef.planner.EvalContext(), ef.planner.ExecCfg().Codec, tabDesc, tabDesc.GetPrimaryIndex()) - if err := ef.planner.maybeSetSystemConfig(tabDesc.GetID()); err != nil { - return nil, err - } - spans, err := sb.SpansFromConstraint(indexConstraint, span.NoopSplitter()) if err != nil { return nil, err diff --git a/pkg/sql/plan.go b/pkg/sql/plan.go index cab4d1b19723..3b35c4a260f2 100644 --- a/pkg/sql/plan.go +++ b/pkg/sql/plan.go @@ -13,10 +13,8 @@ package sql import ( "context" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/execstats" "github.com/cockroachdb/cockroach/pkg/sql/opt/exec" @@ -539,19 +537,6 @@ func (p *planner) maybePlanHook(ctx context.Context, stmt tree.Statement) (planN return nil, nil } -// Mark transaction as operating on the system DB if the descriptor id -// is within the SystemConfig range. -func (p *planner) maybeSetSystemConfig(id descpb.ID) error { - if !descpb.IsSystemConfigID(id) || p.execCfg.Settings.Version.IsActive( - p.EvalContext().Ctx(), clusterversion.DisableSystemConfigGossipTrigger, - ) { - return nil - } - // Mark transaction as operating on the system DB. - // Only the system tenant marks the SystemConfigTrigger. - return p.txn.DeprecatedSetSystemConfigTrigger(p.execCfg.Codec.ForSystemTenant()) -} - // planFlags is used throughout the planning code to keep track of various // events or decisions along the way. type planFlags uint32 diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index cb485b0d0e02..81b6a32c58a0 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -3051,13 +3051,6 @@ func DeleteTableDescAndZoneConfig( ) error { log.Infof(ctx, "removing table descriptor and zone config for table %d", tableDesc.GetID()) return db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - if !settings.Version.IsActive( - ctx, clusterversion.DisableSystemConfigGossipTrigger, - ) { - if err := txn.DeprecatedSetSystemConfigTrigger(codec.ForSystemTenant()); err != nil { - return err - } - } b := &kv.Batch{} // Delete the descriptor. diff --git a/pkg/sql/tests/BUILD.bazel b/pkg/sql/tests/BUILD.bazel index 7b08df9099a5..0148051f2559 100644 --- a/pkg/sql/tests/BUILD.bazel +++ b/pkg/sql/tests/BUILD.bazel @@ -5,15 +5,12 @@ go_library( srcs = [ "command_filters.go", "data.go", - "end_txn_trigger.go", "server_params.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/sql/tests", visibility = ["//visibility:public"], deps = [ "//pkg/base", - "//pkg/clusterversion", - "//pkg/keys", "//pkg/kv", "//pkg/kv/kvserver", "//pkg/kv/kvserver/kvserverbase", diff --git a/pkg/sql/tests/end_txn_trigger.go b/pkg/sql/tests/end_txn_trigger.go deleted file mode 100644 index f2bfb66f6859..000000000000 --- a/pkg/sql/tests/end_txn_trigger.go +++ /dev/null @@ -1,68 +0,0 @@ -// Copyright 2017 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 tests - -import ( - "bytes" - - "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/errors" -) - -// CheckEndTxnTrigger verifies that an EndTxnRequest that includes intents for -// the SystemDB keys sets the proper trigger. -// -// TODO(ajwerner): Remove this in 22.2. It only applies to the mixed-version -// state. -func CheckEndTxnTrigger(args kvserverbase.FilterArgs) *roachpb.Error { - req, ok := args.Req.(*roachpb.EndTxnRequest) - if !ok { - return nil - } - - if !req.Commit { - // This is a rollback: skip trigger verification. - return nil - } - - modifiedSpanTrigger := req.InternalCommitTrigger.GetModifiedSpanTrigger() - modifiedSystemConfigSpan := modifiedSpanTrigger != nil && modifiedSpanTrigger.SystemConfigSpan - - var hasSystemKey bool - for _, span := range req.LockSpans { - if bytes.Compare(span.Key, keys.SystemConfigSpan.Key) >= 0 && - bytes.Compare(span.Key, keys.SystemConfigSpan.EndKey) < 0 { - hasSystemKey = true - break - } - } - // If the transaction in question has intents in the system span, then - // modifiedSystemConfigSpan should always be true. However, it is possible - // for modifiedSystemConfigSpan to be set, even though no system keys are - // present. This can occur with certain conditional DDL statements (e.g. - // "CREATE TABLE IF NOT EXISTS"), which set the SystemConfigTrigger - // aggressively but may not actually end up changing the system DB depending - // on the current state. - // For more information, see the related comment at the beginning of - // planner.makePlan(). - if hasSystemKey && - !(clusterversion.ClusterVersion{Version: args.Version}). - IsActive(clusterversion.DisableSystemConfigGossipTrigger) && - !modifiedSystemConfigSpan { - return roachpb.NewError(errors.Errorf("EndTxn hasSystemKey=%t, but hasSystemConfigTrigger=%t", - hasSystemKey, modifiedSystemConfigSpan)) - } - - return nil -} diff --git a/pkg/sql/tests/server_params.go b/pkg/sql/tests/server_params.go index 8e3746ad15d0..a08c08063dba 100644 --- a/pkg/sql/tests/server_params.go +++ b/pkg/sql/tests/server_params.go @@ -25,7 +25,6 @@ import ( // if the EndTxn checks are important. func CreateTestServerParams() (base.TestServerArgs, *CommandFilters) { var cmdFilters CommandFilters - cmdFilters.AppendFilter(CheckEndTxnTrigger, true) params := base.TestServerArgs{} params.Knobs = CreateTestingKnobs() params.Knobs.Store = &kvserver.StoreTestingKnobs{ diff --git a/pkg/sql/txn_restart_test.go b/pkg/sql/txn_restart_test.go index 257c82c79b20..047d8dfdd672 100644 --- a/pkg/sql/txn_restart_test.go +++ b/pkg/sql/txn_restart_test.go @@ -1237,7 +1237,6 @@ func TestFlushUncommitedDescriptorCacheOnRestart(t *testing.T) { defer log.Scope(t).Close(t) var cmdFilters tests.CommandFilters - cmdFilters.AppendFilter(tests.CheckEndTxnTrigger, true) testKey := []byte("test_key") testingKnobs := &kvserver.StoreTestingKnobs{ EvalKnobs: kvserverbase.BatchEvalTestingKnobs{