mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
Merge pull request #8288 from irfansharif/pre-vote
raft: introduce/fix TestNodeWithSmallerTermCanCompleteElection
This commit is contained in:
commit
93826f2f78
29
raft/raft.go
29
raft/raft.go
@ -355,10 +355,20 @@ func (r *raft) nodes() []uint64 {
|
||||
// send persists state to stable storage and then sends to its mailbox.
|
||||
func (r *raft) send(m pb.Message) {
|
||||
m.From = r.id
|
||||
if m.Type == pb.MsgVote || m.Type == pb.MsgPreVote {
|
||||
if m.Type == pb.MsgVote || m.Type == pb.MsgVoteResp || m.Type == pb.MsgPreVote || m.Type == pb.MsgPreVoteResp {
|
||||
if m.Term == 0 {
|
||||
// PreVote RPCs are sent at a term other than our actual term, so the code
|
||||
// that sends these messages is responsible for setting the term.
|
||||
// All {pre-,}campaign messages need to have the term set when
|
||||
// sending.
|
||||
// - MsgVote: m.Term is the term the node is campaigning for,
|
||||
// non-zero as we increment the term when campaigning.
|
||||
// - MsgVoteResp: m.Term is the new r.Term if the MsgVote was
|
||||
// granted, non-zero for the same reason MsgVote is
|
||||
// - MsgPreVote: m.Term is the term the node will campaign,
|
||||
// non-zero as we use m.Term to indicate the next term we'll be
|
||||
// campaigning for
|
||||
// - MsgPreVoteResp: m.Term is the term received in the original
|
||||
// MsgPreVote if the pre-vote was granted, non-zero for the
|
||||
// same reasons MsgPreVote is
|
||||
panic(fmt.Sprintf("term should be set when sending %s", m.Type))
|
||||
}
|
||||
} else {
|
||||
@ -774,7 +784,16 @@ func (r *raft) Step(m pb.Message) error {
|
||||
if (r.Vote == None || m.Term > r.Term || r.Vote == m.From) && r.raftLog.isUpToDate(m.Index, m.LogTerm) {
|
||||
r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] cast %s for %x [logterm: %d, index: %d] at term %d",
|
||||
r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term)
|
||||
r.send(pb.Message{To: m.From, Type: voteRespMsgType(m.Type)})
|
||||
// When responding to Msg{Pre,}Vote messages we include the term
|
||||
// from the message, not the local term. To see why consider the
|
||||
// case where a single node was previously partitioned away and
|
||||
// it's local term is now of date. If we include the local term
|
||||
// (recall that for pre-votes we don't update the local term), the
|
||||
// (pre-)campaigning node on the other end will proceed to ignore
|
||||
// the message (it ignores all out of date messages).
|
||||
// The term in the original message and current local term are the
|
||||
// same in the case of regular votes, but different for pre-votes.
|
||||
r.send(pb.Message{To: m.From, Term: m.Term, Type: voteRespMsgType(m.Type)})
|
||||
if m.Type == pb.MsgVote {
|
||||
// Only record real votes.
|
||||
r.electionElapsed = 0
|
||||
@ -783,7 +802,7 @@ func (r *raft) Step(m pb.Message) error {
|
||||
} else {
|
||||
r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected %s from %x [logterm: %d, index: %d] at term %d",
|
||||
r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term)
|
||||
r.send(pb.Message{To: m.From, Type: voteRespMsgType(m.Type), Reject: true})
|
||||
r.send(pb.Message{To: m.From, Term: r.Term, Type: voteRespMsgType(m.Type), Reject: true})
|
||||
}
|
||||
|
||||
default:
|
||||
|
@ -1364,7 +1364,7 @@ func TestRecvMsgVote(t *testing.T) {
|
||||
func testRecvMsgVote(t *testing.T, msgType pb.MessageType) {
|
||||
tests := []struct {
|
||||
state StateType
|
||||
i, term uint64
|
||||
index, logTerm uint64
|
||||
voteFor uint64
|
||||
wreject bool
|
||||
}{
|
||||
@ -1396,6 +1396,13 @@ func testRecvMsgVote(t *testing.T, msgType pb.MessageType) {
|
||||
{StateCandidate, 3, 3, 1, true},
|
||||
}
|
||||
|
||||
max := func(a, b uint64) uint64 {
|
||||
if a > b {
|
||||
return a
|
||||
}
|
||||
return b
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
sm := newTestRaft(1, []uint64{1}, 10, 1, NewMemoryStorage())
|
||||
sm.state = tt.state
|
||||
@ -1413,7 +1420,17 @@ func testRecvMsgVote(t *testing.T, msgType pb.MessageType) {
|
||||
unstable: unstable{offset: 3},
|
||||
}
|
||||
|
||||
sm.Step(pb.Message{Type: msgType, From: 2, Index: tt.i, LogTerm: tt.term})
|
||||
// raft.Term is greater than or equal to raft.raftLog.lastTerm. In this
|
||||
// test we're only testing MsgVote responses when the campaigning node
|
||||
// has a different raft log compared to the recipient node.
|
||||
// Additionally we're verifying behaviour when the recipient node has
|
||||
// already given out its vote for its current term. We're not testing
|
||||
// what the recipient node does when receiving a message with a
|
||||
// different term number, so we simply initialize both term numbers to
|
||||
// be the same.
|
||||
term := max(sm.raftLog.lastTerm(), tt.logTerm)
|
||||
sm.Term = term
|
||||
sm.Step(pb.Message{Type: msgType, Term: term, From: 2, Index: tt.index, LogTerm: tt.logTerm})
|
||||
|
||||
msgs := sm.readMessages()
|
||||
if g := len(msgs); g != 1 {
|
||||
@ -3094,6 +3111,104 @@ func TestTransferNonMember(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
// TestNodeWithSmallerTermCanCompleteElection tests the scenario where a node
|
||||
// that has been partitioned away (and fallen behind) rejoins the cluster at
|
||||
// about the same time the leader node gets partitioned away.
|
||||
// Previously the cluster would come to a standstill when run with PreVote
|
||||
// enabled.
|
||||
func TestNodeWithSmallerTermCanCompleteElection(t *testing.T) {
|
||||
n1 := newTestRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
|
||||
n2 := newTestRaft(2, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
|
||||
n3 := newTestRaft(3, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
|
||||
|
||||
n1.becomeFollower(1, None)
|
||||
n2.becomeFollower(1, None)
|
||||
n3.becomeFollower(1, None)
|
||||
|
||||
n1.preVote = true
|
||||
n2.preVote = true
|
||||
n3.preVote = true
|
||||
|
||||
// cause a network partition to isolate node 3
|
||||
nt := newNetwork(n1, n2, n3)
|
||||
nt.cut(1, 3)
|
||||
nt.cut(2, 3)
|
||||
|
||||
nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
|
||||
|
||||
sm := nt.peers[1].(*raft)
|
||||
if sm.state != StateLeader {
|
||||
t.Errorf("peer 1 state: %s, want %s", sm.state, StateLeader)
|
||||
}
|
||||
|
||||
sm = nt.peers[2].(*raft)
|
||||
if sm.state != StateFollower {
|
||||
t.Errorf("peer 2 state: %s, want %s", sm.state, StateFollower)
|
||||
}
|
||||
|
||||
nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
|
||||
sm = nt.peers[3].(*raft)
|
||||
if sm.state != StatePreCandidate {
|
||||
t.Errorf("peer 3 state: %s, want %s", sm.state, StatePreCandidate)
|
||||
}
|
||||
|
||||
nt.send(pb.Message{From: 2, To: 2, Type: pb.MsgHup})
|
||||
|
||||
// check whether the term values are expected
|
||||
// a.Term == 3
|
||||
// b.Term == 3
|
||||
// c.Term == 1
|
||||
sm = nt.peers[1].(*raft)
|
||||
if sm.Term != 3 {
|
||||
t.Errorf("peer 1 term: %d, want %d", sm.Term, 3)
|
||||
}
|
||||
|
||||
sm = nt.peers[2].(*raft)
|
||||
if sm.Term != 3 {
|
||||
t.Errorf("peer 2 term: %d, want %d", sm.Term, 3)
|
||||
}
|
||||
|
||||
sm = nt.peers[3].(*raft)
|
||||
if sm.Term != 1 {
|
||||
t.Errorf("peer 3 term: %d, want %d", sm.Term, 1)
|
||||
}
|
||||
|
||||
// check state
|
||||
// a == follower
|
||||
// b == leader
|
||||
// c == pre-candidate
|
||||
sm = nt.peers[1].(*raft)
|
||||
if sm.state != StateFollower {
|
||||
t.Errorf("peer 1 state: %s, want %s", sm.state, StateFollower)
|
||||
}
|
||||
sm = nt.peers[2].(*raft)
|
||||
if sm.state != StateLeader {
|
||||
t.Errorf("peer 2 state: %s, want %s", sm.state, StateLeader)
|
||||
}
|
||||
sm = nt.peers[3].(*raft)
|
||||
if sm.state != StatePreCandidate {
|
||||
t.Errorf("peer 3 state: %s, want %s", sm.state, StatePreCandidate)
|
||||
}
|
||||
|
||||
sm.logger.Infof("going to bring back peer 3 and kill peer 2")
|
||||
// recover the network then immediately isolate b which is currently
|
||||
// the leader, this is to emulate the crash of b.
|
||||
nt.recover()
|
||||
nt.cut(2, 1)
|
||||
nt.cut(2, 3)
|
||||
|
||||
// call for election
|
||||
nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
|
||||
nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
|
||||
|
||||
// do we have a leader?
|
||||
sma := nt.peers[1].(*raft)
|
||||
smb := nt.peers[3].(*raft)
|
||||
if sma.state != StateLeader && smb.state != StateLeader {
|
||||
t.Errorf("no leader")
|
||||
}
|
||||
}
|
||||
|
||||
func entsWithConfig(configFunc func(*Config), terms ...uint64) *raft {
|
||||
storage := NewMemoryStorage()
|
||||
for i, term := range terms {
|
||||
|
Loading…
x
Reference in New Issue
Block a user