Skip to content

Commit

Permalink
kv: forbid direct voter removal without demotion
Browse files Browse the repository at this point in the history
Informs #129796.
Informs #125355.

Perform the same assertion as we have in pkg/raft, but for cases where
DisableConfChangeValidation is set to true.

Release note: None
  • Loading branch information
nvanbenschoten committed Oct 14, 2024
1 parent e8fc001 commit 686d59d
Show file tree
Hide file tree
Showing 2 changed files with 87 additions and 1 deletion.
63 changes: 63 additions & 0 deletions pkg/kv/kvserver/client_raft_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4084,6 +4084,69 @@ func TestLeaseHolderRemoveSelf(t *testing.T) {
}
}

// TestVoterRemovalWithoutDemotion verifies that a voter replica cannot be
// removed directly without first being demoted to a learner.
func TestVoterRemovalWithoutDemotion(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

// Inject a filter which skips the demotion of a voter replica when removing
// it from the range. This will trigger the raft-level check which ensures
// that a voter replica cannot be removed directly.
type noDemotionKey struct{}
var removalTarget roachpb.ReplicationTarget
testingProposalFilter := func(args kvserverbase.ProposalFilterArgs) *kvpb.Error {
if args.Ctx.Value(noDemotionKey{}) != nil {
if state := args.Cmd.ReplicatedEvalResult.State; state != nil && state.Desc != nil {
repl, ok := state.Desc.GetReplicaDescriptor(removalTarget.StoreID)
if ok && repl.Type == roachpb.VOTER_DEMOTING_LEARNER {
t.Logf("intercepting proposal, skipping voter demotion: %+v", args.Cmd)
_, ok := state.Desc.RemoveReplica(repl.NodeID, repl.StoreID)
require.True(t, ok)
}
}
}
return nil
}

ctx := context.Background()
tc := testcluster.StartTestCluster(t, 2,
base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgs: base.TestServerArgs{
Knobs: base.TestingKnobs{
Store: &kvserver.StoreTestingKnobs{
TestingProposalFilter: testingProposalFilter,
},
},
},
})
defer tc.Stopper().Stop(ctx)
removalTarget = tc.Target(1)

key := []byte("a")
tc.SplitRangeOrFatal(t, key)
tc.AddVotersOrFatal(t, key, removalTarget)

var beforeDesc roachpb.RangeDescriptor
db := tc.Servers[0].SystemLayer().DB()
require.NoError(t, db.GetProto(ctx, keys.RangeDescriptorKey(key), &beforeDesc))

// First attempt to remove the voter without demotion. Should fail.
expectedErr := "cannot remove voter .* directly; must first demote to learner"
noDemotionCtx := context.WithValue(ctx, noDemotionKey{}, struct{}{})
removeVoter := kvpb.MakeReplicationChanges(roachpb.REMOVE_VOTER, removalTarget)
_, err := db.AdminChangeReplicas(noDemotionCtx, key, beforeDesc, removeVoter)
require.Error(t, err)
require.Regexp(t, expectedErr, err)

// Now demote the voter to a learner and then remove it.
desc, err := db.AdminChangeReplicas(ctx, key, beforeDesc, removeVoter)
require.NoError(t, err)
_, ok := desc.GetReplicaDescriptor(removalTarget.StoreID)
require.False(t, ok)
}

// TestRemovedReplicaError verifies that a replica that has been removed from a
// range returns a RangeNotFoundError if it receives a request for that range
// (not RaftGroupDeletedError, and even before the ReplicaGCQueue has run).
Expand Down
25 changes: 24 additions & 1 deletion pkg/kv/kvserver/replica_raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -528,7 +528,30 @@ func checkReplicationChangeAllowed(
return err
}

// TODO(nvanbenschoten): check against direct voter removal.
// Check against direct voter removal. Voters must first be demoted to
// learners before they can be removed for at least two reasons:
// 1. the leader (or any voter) may be needed to vote for a candidate who
// has not yet applied the configuration change. This is a liveness issue
// if the leader/voter is immediately removed without stepping down to a
// learner first and waiting for a second configuration change to
// succeed.
// For details, see: https://github.com/cockroachdb/cockroach/pull/42251.
// 2. the leader may have fortified its leadership term, binding the
// liveness of the leader replica to the leader's store's store liveness
// heartbeats. Removal of the leader replica from a store while that
// store continues to heartbeat in the store liveness fabric will lead to
// the leader disappearing without any other replica deciding that the
// leader is gone and stepping up to campaign.
//
// This same check exists in the pkg/raft library, but we disable it with
// DisableConfChangeValidation.
for _, repl := range desc.Replicas().Voters().Descriptors() {
if _, ok := proposedDesc.Replicas().GetReplicaDescriptorByID(repl.ReplicaID); !ok {
err := errors.Errorf("cannot remove voter %s directly; must first demote to learner", repl)
err = errors.Mark(err, errMarkInvalidReplicationChange)
return err
}
}

return nil
}
Expand Down

0 comments on commit 686d59d

Please sign in to comment.