Skip to content

Commit

Permalink
raft: add test for leadership transfer in joint configuration
Browse files Browse the repository at this point in the history
  • Loading branch information
shralex committed Oct 25, 2021
1 parent 67e747d commit 088717f
Show file tree
Hide file tree
Showing 5 changed files with 578 additions and 5 deletions.
33 changes: 28 additions & 5 deletions rafttest/interaction_env_handler.go
Original file line number Diff line number Diff line change
Expand Up @@ -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:
//
Expand All @@ -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.
//
Expand All @@ -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:
Expand Down
48 changes: 48 additions & 0 deletions rafttest/interaction_env_handler_raftstate.go
Original file line number Diff line number Diff line change
@@ -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
}
41 changes: 41 additions & 0 deletions rafttest/interaction_env_handler_transfer_leadership.go
Original file line number Diff line number Diff line change
@@ -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
}
25 changes: 25 additions & 0 deletions testdata/confchange_v1_remove_leader.txt
Original file line number Diff line number Diff line change
Expand Up @@ -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
----
Expand Down Expand Up @@ -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
----
Expand Down Expand Up @@ -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)
Loading

0 comments on commit 088717f

Please sign in to comment.