raft: add test for leadership transfer in joint configuration

This commit is contained in:
shralex 2021-10-22 14:11:26 -07:00
parent 3b1ecc4cbe
commit ea3c86ef5b
5 changed files with 577 additions and 4 deletions

View File

@ -82,9 +82,14 @@ func (env *InteractionEnv) Handle(t *testing.T, d datadriven.TestData) string {
//
// raft-log 3
err = env.handleRaftLog(t, d)
case "raft-state":
// Print Raft state of all nodes (whether the node is leading,
// following, etc.). The information for node n is based on
// n's view.
err = env.handleRaftState()
case "stabilize":
// Deliver messages to and run process-ready on the set of IDs until
// no more work is to be done.
// no more work is to be done. If no ids are given, all nodes are used.
//
// Example:
//
@ -104,6 +109,13 @@ func (env *InteractionEnv) Handle(t *testing.T, d datadriven.TestData) string {
//
// tick-heartbeat 3
err = env.handleTickHeartbeat(t, d)
case "transfer-leadership":
// Transfer the Raft leader.
//
// Example:
//
// transfer-leadership from=1 to=4
err = env.handleTransferLeadership(t, d)
case "propose":
// Propose an entry.
//
@ -112,16 +124,27 @@ func (env *InteractionEnv) Handle(t *testing.T, d datadriven.TestData) string {
// propose 1 foo
err = env.handlePropose(t, d)
case "propose-conf-change":
// Propose a configuration change.
// Propose a configuration change, or transition out of a previously
// proposed joint configuration change that requested explicit
// transitions. When adding nodes, this command can be used to
// logically add nodes to the configuration, but add-nodes is needed
// to "create" the nodes.
//
// propose-conf-change node_id [v1=<bool>] [transition=<string>]
// command string
// See ConfChangesFromString for command string format.
// Arguments are:
// node_id - the node proposing the configuration change.
// v1 - make one change at a time, false by default.
// transition - "auto" (the default), "explicit" or "implicit".
// Example:
//
// propose-conf-change transition=explicit
// propose-conf-change 1 transition=explicit
// v1 v3 l4 r5
//
// Example:
//
// propose-conf-change v1=true
// propose-conf-change 2 v1=true
// v5
err = env.handleProposeConfChange(t, d)
default:

View File

@ -0,0 +1,48 @@
// Copyright 2021 The etcd Authors
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package rafttest
import (
"fmt"
"go.etcd.io/etcd/raft/v3"
)
// isVoter checks whether node id is in the voter list within st.
func isVoter(id uint64, st raft.Status) bool {
idMap := st.Config.Voters.IDs()
for idx := range idMap {
if id == idx {
return true
}
}
return false
}
// handleRaftState pretty-prints the raft state for all nodes to the output buffer.
// For each node, the information is based on its own configuration view.
func (env *InteractionEnv) handleRaftState() error {
for _, n := range env.Nodes {
st := n.Status()
var voterStatus string
if isVoter(st.ID, st) {
voterStatus = "(Voter)"
} else {
voterStatus = "(Non-Voter)"
}
fmt.Fprintf(env.Output, "%d: %s %s\n", st.ID, st.RaftState, voterStatus)
}
return nil
}

View File

@ -0,0 +1,41 @@
// Copyright 2021 The etcd Authors
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package rafttest
import (
"testing"
"github.com/cockroachdb/datadriven"
)
func (env *InteractionEnv) handleTransferLeadership(t *testing.T, d datadriven.TestData) error {
var from, to uint64
d.ScanArgs(t, "from", &from)
d.ScanArgs(t, "to", &to)
if from == 0 || from > uint64(len(env.Nodes)) {
t.Fatalf(`expected valid "from" argument`)
}
if to == 0 || to > uint64(len(env.Nodes)) {
t.Fatalf(`expected valid "to" argument`)
}
return env.transferLeadership(from, to)
}
// Initiate leadership transfer.
func (env *InteractionEnv) transferLeadership(from, to uint64) error {
fromIdx := from - 1
env.Nodes[fromIdx].TransferLeader(to)
return nil
}

View File

@ -21,12 +21,24 @@ log-level debug
----
ok
raft-state
----
1: StateLeader (Voter)
2: StateFollower (Voter)
3: StateFollower (Voter)
# Start removing n1.
propose-conf-change 1 v1=true
r1
----
ok
raft-state
----
1: StateLeader (Voter)
2: StateFollower (Voter)
3: StateFollower (Voter)
# Propose an extra entry which will be sent out together with the conf change.
propose 1 foo
----
@ -95,6 +107,12 @@ stabilize 1
1->3 MsgApp Term:1 Log:1/6 Commit:5
INFO 1 switched to configuration voters=(2 3)
raft-state
----
1: StateLeader (Non-Voter)
2: StateFollower (Voter)
3: StateFollower (Voter)
# n2 responds, n3 doesn't yet. Quorum for 'bar' should not be reached...
stabilize 2
----
@ -222,3 +240,10 @@ stabilize
> 1 receiving messages
2->1 MsgHeartbeatResp Term:1 Log:0/0
3->1 MsgHeartbeatResp Term:1 Log:0/0
# Just confirming the issue above - leader does not automatically step down.
raft-state
----
1: StateLeader (Non-Voter)
2: StateFollower (Voter)
3: StateFollower (Voter)

View File

@ -0,0 +1,436 @@
# Run a V2 membership change that removes the leader and adds another voter as
# a single operation, using joint consensus and explicitly determining when to
# transition out of the joint config. Leadership is transferred to new joiner
# while in the joint config. After the reconfiguration completes, we verify
# that the removed leader cannot campaign to become leader.
# We'll turn this back on after the boilerplate.
log-level none
----
ok
# Bootstrap n1, n2, n3.
add-nodes 3 voters=(1,2,3) index=2
----
ok
# n1 campaigns to become leader.
campaign 1
----
ok
stabilize
----
ok (quiet)
log-level info
----
ok
raft-state
----
1: StateLeader (Voter)
2: StateFollower (Voter)
3: StateFollower (Voter)
log-level info
----
ok
# create n4
add-nodes 1
----
INFO 4 switched to configuration voters=()
INFO 4 became follower at term 0
INFO newRaft 4 [peers: [], term: 0, commit: 0, applied: 0, lastindex: 0, lastterm: 0]
# Start reconfiguration to remove n1 and add n4.
propose-conf-change 1 v1=false transition=explicit
r1 v4
----
ok
# Enter joint config.
stabilize
----
> 1 handling Ready
Ready MustSync=true:
Entries:
1/4 EntryConfChangeV2 r1 v4
Messages:
1->2 MsgApp Term:1 Log:1/3 Commit:3 Entries:[1/4 EntryConfChangeV2 r1 v4]
1->3 MsgApp Term:1 Log:1/3 Commit:3 Entries:[1/4 EntryConfChangeV2 r1 v4]
> 2 receiving messages
1->2 MsgApp Term:1 Log:1/3 Commit:3 Entries:[1/4 EntryConfChangeV2 r1 v4]
> 3 receiving messages
1->3 MsgApp Term:1 Log:1/3 Commit:3 Entries:[1/4 EntryConfChangeV2 r1 v4]
> 2 handling Ready
Ready MustSync=true:
Entries:
1/4 EntryConfChangeV2 r1 v4
Messages:
2->1 MsgAppResp Term:1 Log:0/4
> 3 handling Ready
Ready MustSync=true:
Entries:
1/4 EntryConfChangeV2 r1 v4
Messages:
3->1 MsgAppResp Term:1 Log:0/4
> 1 receiving messages
2->1 MsgAppResp Term:1 Log:0/4
3->1 MsgAppResp Term:1 Log:0/4
> 1 handling Ready
Ready MustSync=false:
HardState Term:1 Vote:1 Commit:4
CommittedEntries:
1/4 EntryConfChangeV2 r1 v4
Messages:
1->2 MsgApp Term:1 Log:1/4 Commit:4
1->3 MsgApp Term:1 Log:1/4 Commit:4
INFO 1 switched to configuration voters=(2 3 4)&&(1 2 3)
> 2 receiving messages
1->2 MsgApp Term:1 Log:1/4 Commit:4
> 3 receiving messages
1->3 MsgApp Term:1 Log:1/4 Commit:4
> 1 handling Ready
Ready MustSync=false:
Messages:
1->4 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 r1 v4]
> 2 handling Ready
Ready MustSync=false:
HardState Term:1 Vote:1 Commit:4
CommittedEntries:
1/4 EntryConfChangeV2 r1 v4
Messages:
2->1 MsgAppResp Term:1 Log:0/4
INFO 2 switched to configuration voters=(2 3 4)&&(1 2 3)
> 3 handling Ready
Ready MustSync=false:
HardState Term:1 Vote:1 Commit:4
CommittedEntries:
1/4 EntryConfChangeV2 r1 v4
Messages:
3->1 MsgAppResp Term:1 Log:0/4
INFO 3 switched to configuration voters=(2 3 4)&&(1 2 3)
> 1 receiving messages
2->1 MsgAppResp Term:1 Log:0/4
3->1 MsgAppResp Term:1 Log:0/4
> 4 receiving messages
1->4 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 r1 v4]
INFO 4 [term: 0] received a MsgApp message with higher term from 1 [term: 1]
INFO 4 became follower at term 1
> 4 handling Ready
Ready MustSync=true:
Lead:1 State:StateFollower
HardState Term:1 Commit:0
Messages:
4->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0)
> 1 receiving messages
4->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0)
> 1 handling Ready
Ready MustSync=false:
Messages:
1->4 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[2 3 4] VotersOutgoing:[1 2 3] Learners:[] LearnersNext:[] AutoLeave:false
> 4 receiving messages
1->4 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[2 3 4] VotersOutgoing:[1 2 3] Learners:[] LearnersNext:[] AutoLeave:false
INFO log [committed=0, applied=0, unstable.offset=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1]
INFO 4 switched to configuration voters=(2 3 4)&&(1 2 3)
INFO 4 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1]
INFO 4 [commit: 4] restored snapshot [index: 4, term: 1]
> 4 handling Ready
Ready MustSync=false:
HardState Term:1 Commit:4
Snapshot Index:4 Term:1 ConfState:Voters:[2 3 4] VotersOutgoing:[1 2 3] Learners:[] LearnersNext:[] AutoLeave:false
Messages:
4->1 MsgAppResp Term:1 Log:0/4
> 1 receiving messages
4->1 MsgAppResp Term:1 Log:0/4
> 1 handling Ready
Ready MustSync=false:
Messages:
1->4 MsgApp Term:1 Log:1/4 Commit:4
> 4 receiving messages
1->4 MsgApp Term:1 Log:1/4 Commit:4
> 4 handling Ready
Ready MustSync=false:
Messages:
4->1 MsgAppResp Term:1 Log:0/4
> 1 receiving messages
4->1 MsgAppResp Term:1 Log:0/4
# Transfer leadership while in the joint config.
transfer-leadership from=1 to=4
----
INFO 1 [term 1] starts to transfer leadership to 4
INFO 1 sends MsgTimeoutNow to 4 immediately as 4 already has up-to-date log
# Leadership transfer wasn't processed yet.
raft-state
----
1: StateLeader (Voter)
2: StateFollower (Voter)
3: StateFollower (Voter)
4: StateFollower (Voter)
# Leadership transfer is happening here.
stabilize
----
> 1 handling Ready
Ready MustSync=false:
Messages:
1->4 MsgTimeoutNow Term:1 Log:0/0
> 4 receiving messages
1->4 MsgTimeoutNow Term:1 Log:0/0
INFO 4 [term 1] received MsgTimeoutNow from 1 and starts an election to get leadership.
INFO 4 is starting a new election at term 1
INFO 4 became candidate at term 2
INFO 4 received MsgVoteResp from 4 at term 2
INFO 4 [logterm: 1, index: 4] sent MsgVote request to 1 at term 2
INFO 4 [logterm: 1, index: 4] sent MsgVote request to 2 at term 2
INFO 4 [logterm: 1, index: 4] sent MsgVote request to 3 at term 2
> 4 handling Ready
Ready MustSync=true:
Lead:0 State:StateCandidate
HardState Term:2 Vote:4 Commit:4
Messages:
4->1 MsgVote Term:2 Log:1/4
4->2 MsgVote Term:2 Log:1/4
4->3 MsgVote Term:2 Log:1/4
> 1 receiving messages
4->1 MsgVote Term:2 Log:1/4
INFO 1 [term: 1] received a MsgVote message with higher term from 4 [term: 2]
INFO 1 became follower at term 2
INFO 1 [logterm: 1, index: 4, vote: 0] cast MsgVote for 4 [logterm: 1, index: 4] at term 2
> 2 receiving messages
4->2 MsgVote Term:2 Log:1/4
INFO 2 [term: 1] received a MsgVote message with higher term from 4 [term: 2]
INFO 2 became follower at term 2
INFO 2 [logterm: 1, index: 4, vote: 0] cast MsgVote for 4 [logterm: 1, index: 4] at term 2
> 3 receiving messages
4->3 MsgVote Term:2 Log:1/4
INFO 3 [term: 1] received a MsgVote message with higher term from 4 [term: 2]
INFO 3 became follower at term 2
INFO 3 [logterm: 1, index: 4, vote: 0] cast MsgVote for 4 [logterm: 1, index: 4] at term 2
> 1 handling Ready
Ready MustSync=true:
Lead:0 State:StateFollower
HardState Term:2 Vote:4 Commit:4
Messages:
1->4 MsgVoteResp Term:2 Log:0/0
> 2 handling Ready
Ready MustSync=true:
Lead:0 State:StateFollower
HardState Term:2 Vote:4 Commit:4
Messages:
2->4 MsgVoteResp Term:2 Log:0/0
> 3 handling Ready
Ready MustSync=true:
Lead:0 State:StateFollower
HardState Term:2 Vote:4 Commit:4
Messages:
3->4 MsgVoteResp Term:2 Log:0/0
> 4 receiving messages
1->4 MsgVoteResp Term:2 Log:0/0
INFO 4 received MsgVoteResp from 1 at term 2
INFO 4 has received 2 MsgVoteResp votes and 0 vote rejections
2->4 MsgVoteResp Term:2 Log:0/0
INFO 4 received MsgVoteResp from 2 at term 2
INFO 4 has received 3 MsgVoteResp votes and 0 vote rejections
INFO 4 became leader at term 2
3->4 MsgVoteResp Term:2 Log:0/0
> 4 handling Ready
Ready MustSync=true:
Lead:4 State:StateLeader
Entries:
2/5 EntryNormal ""
Messages:
4->1 MsgApp Term:2 Log:1/4 Commit:4 Entries:[2/5 EntryNormal ""]
4->2 MsgApp Term:2 Log:1/4 Commit:4 Entries:[2/5 EntryNormal ""]
4->3 MsgApp Term:2 Log:1/4 Commit:4 Entries:[2/5 EntryNormal ""]
> 1 receiving messages
4->1 MsgApp Term:2 Log:1/4 Commit:4 Entries:[2/5 EntryNormal ""]
> 2 receiving messages
4->2 MsgApp Term:2 Log:1/4 Commit:4 Entries:[2/5 EntryNormal ""]
> 3 receiving messages
4->3 MsgApp Term:2 Log:1/4 Commit:4 Entries:[2/5 EntryNormal ""]
> 1 handling Ready
Ready MustSync=true:
Lead:4 State:StateFollower
Entries:
2/5 EntryNormal ""
Messages:
1->4 MsgAppResp Term:2 Log:0/5
> 2 handling Ready
Ready MustSync=true:
Lead:4 State:StateFollower
Entries:
2/5 EntryNormal ""
Messages:
2->4 MsgAppResp Term:2 Log:0/5
> 3 handling Ready
Ready MustSync=true:
Lead:4 State:StateFollower
Entries:
2/5 EntryNormal ""
Messages:
3->4 MsgAppResp Term:2 Log:0/5
> 4 receiving messages
1->4 MsgAppResp Term:2 Log:0/5
2->4 MsgAppResp Term:2 Log:0/5
3->4 MsgAppResp Term:2 Log:0/5
> 4 handling Ready
Ready MustSync=false:
HardState Term:2 Vote:4 Commit:5
CommittedEntries:
2/5 EntryNormal ""
Messages:
4->1 MsgApp Term:2 Log:2/5 Commit:4
4->1 MsgApp Term:2 Log:2/5 Commit:5
4->2 MsgApp Term:2 Log:2/5 Commit:5
4->3 MsgApp Term:2 Log:2/5 Commit:5
> 1 receiving messages
4->1 MsgApp Term:2 Log:2/5 Commit:4
4->1 MsgApp Term:2 Log:2/5 Commit:5
> 2 receiving messages
4->2 MsgApp Term:2 Log:2/5 Commit:5
> 3 receiving messages
4->3 MsgApp Term:2 Log:2/5 Commit:5
> 1 handling Ready
Ready MustSync=false:
HardState Term:2 Vote:4 Commit:5
CommittedEntries:
2/5 EntryNormal ""
Messages:
1->4 MsgAppResp Term:2 Log:0/5
1->4 MsgAppResp Term:2 Log:0/5
> 2 handling Ready
Ready MustSync=false:
HardState Term:2 Vote:4 Commit:5
CommittedEntries:
2/5 EntryNormal ""
Messages:
2->4 MsgAppResp Term:2 Log:0/5
> 3 handling Ready
Ready MustSync=false:
HardState Term:2 Vote:4 Commit:5
CommittedEntries:
2/5 EntryNormal ""
Messages:
3->4 MsgAppResp Term:2 Log:0/5
> 4 receiving messages
1->4 MsgAppResp Term:2 Log:0/5
1->4 MsgAppResp Term:2 Log:0/5
2->4 MsgAppResp Term:2 Log:0/5
3->4 MsgAppResp Term:2 Log:0/5
# Leadership transfer succeeded.
raft-state
----
1: StateFollower (Voter)
2: StateFollower (Voter)
3: StateFollower (Voter)
4: StateLeader (Voter)
# n4 will propose a transition out of the joint config.
propose-conf-change 4
----
ok
# The group commits the command and everyone switches to the final config.
stabilize
----
> 4 handling Ready
Ready MustSync=true:
Entries:
2/6 EntryConfChangeV2
Messages:
4->1 MsgApp Term:2 Log:2/5 Commit:5 Entries:[2/6 EntryConfChangeV2]
4->2 MsgApp Term:2 Log:2/5 Commit:5 Entries:[2/6 EntryConfChangeV2]
4->3 MsgApp Term:2 Log:2/5 Commit:5 Entries:[2/6 EntryConfChangeV2]
> 1 receiving messages
4->1 MsgApp Term:2 Log:2/5 Commit:5 Entries:[2/6 EntryConfChangeV2]
> 2 receiving messages
4->2 MsgApp Term:2 Log:2/5 Commit:5 Entries:[2/6 EntryConfChangeV2]
> 3 receiving messages
4->3 MsgApp Term:2 Log:2/5 Commit:5 Entries:[2/6 EntryConfChangeV2]
> 1 handling Ready
Ready MustSync=true:
Entries:
2/6 EntryConfChangeV2
Messages:
1->4 MsgAppResp Term:2 Log:0/6
> 2 handling Ready
Ready MustSync=true:
Entries:
2/6 EntryConfChangeV2
Messages:
2->4 MsgAppResp Term:2 Log:0/6
> 3 handling Ready
Ready MustSync=true:
Entries:
2/6 EntryConfChangeV2
Messages:
3->4 MsgAppResp Term:2 Log:0/6
> 4 receiving messages
1->4 MsgAppResp Term:2 Log:0/6
2->4 MsgAppResp Term:2 Log:0/6
3->4 MsgAppResp Term:2 Log:0/6
> 4 handling Ready
Ready MustSync=false:
HardState Term:2 Vote:4 Commit:6
CommittedEntries:
2/6 EntryConfChangeV2
Messages:
4->1 MsgApp Term:2 Log:2/6 Commit:6
4->2 MsgApp Term:2 Log:2/6 Commit:6
4->3 MsgApp Term:2 Log:2/6 Commit:6
INFO 4 switched to configuration voters=(2 3 4)
> 1 receiving messages
4->1 MsgApp Term:2 Log:2/6 Commit:6
> 2 receiving messages
4->2 MsgApp Term:2 Log:2/6 Commit:6
> 3 receiving messages
4->3 MsgApp Term:2 Log:2/6 Commit:6
> 1 handling Ready
Ready MustSync=false:
HardState Term:2 Vote:4 Commit:6
CommittedEntries:
2/6 EntryConfChangeV2
Messages:
1->4 MsgAppResp Term:2 Log:0/6
INFO 1 switched to configuration voters=(2 3 4)
> 2 handling Ready
Ready MustSync=false:
HardState Term:2 Vote:4 Commit:6
CommittedEntries:
2/6 EntryConfChangeV2
Messages:
2->4 MsgAppResp Term:2 Log:0/6
INFO 2 switched to configuration voters=(2 3 4)
> 3 handling Ready
Ready MustSync=false:
HardState Term:2 Vote:4 Commit:6
CommittedEntries:
2/6 EntryConfChangeV2
Messages:
3->4 MsgAppResp Term:2 Log:0/6
INFO 3 switched to configuration voters=(2 3 4)
> 4 receiving messages
1->4 MsgAppResp Term:2 Log:0/6
raft: cannot step as peer not found
2->4 MsgAppResp Term:2 Log:0/6
3->4 MsgAppResp Term:2 Log:0/6
# n1 is out of the configuration.
raft-state
----
1: StateFollower (Non-Voter)
2: StateFollower (Voter)
3: StateFollower (Voter)
4: StateLeader (Voter)
# Make sure n1 cannot campaign to become leader.
campaign 1
----
WARN 1 is unpromotable and can not campaign