Skip to content

Commit

Permalink
kv, gossip: remove misc deprecated system config code
Browse files Browse the repository at this point in the history
There is still some work left to actually remove `SystemConfigSpan`
or at least update `SystemConfigSpan` to contain just system.descriptor and
system.zones.

cockroachdb#76279

Release note: None
  • Loading branch information
RichardJCai committed Jun 3, 2022
1 parent 5a54758 commit 4b1be2e
Show file tree
Hide file tree
Showing 36 changed files with 32 additions and 887 deletions.
31 changes: 0 additions & 31 deletions pkg/gossip/gossip.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
16 changes: 0 additions & 16 deletions pkg/kv/kvclient/kvcoord/txn_coord_sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -994,20 +992,6 @@ func (tc *TxnCoordSender) setTxnAnchorKeyLocked(key roachpb.Key) error {
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()
Expand Down
6 changes: 1 addition & 5 deletions pkg/kv/kvclient/kvcoord/txn_interceptor_committer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -242,11 +242,7 @@ func TestTxnCommitterStripsInFlightWrites(t *testing.T) {
// parallel commits.
ba.Requests = nil
etArgsWithTrigger := etArgs
etArgsWithTrigger.InternalCommitTrigger = &roachpb.InternalCommitTrigger{
ModifiedSpanTrigger: &roachpb.ModifiedSpanTrigger{
SystemConfigSpan: true,
},
}
etArgsWithTrigger.InternalCommitTrigger = &roachpb.InternalCommitTrigger{}
ba.Add(&putArgs, &qiArgs, &etArgsWithTrigger)

mockSender.MockSend(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) {
Expand Down
37 changes: 0 additions & 37 deletions pkg/kv/kvserver/batcheval/cmd_end_transaction.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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{
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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)
}
Expand Down
132 changes: 0 additions & 132 deletions pkg/kv/kvserver/client_lease_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,16 +22,13 @@ 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"
"github.com/cockroachdb/cockroach/pkg/kv"
"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"
Expand Down Expand Up @@ -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)
Expand Down
9 changes: 1 addition & 8 deletions pkg/kv/kvserver/client_replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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())
}
}
Expand All @@ -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)
}
}
Expand Down
Loading

0 comments on commit 4b1be2e

Please sign in to comment.