raft: Implement the PreVote RPC described in thesis section 9.6

This prevents disruption when a node that has been partitioned
away rejoins the cluster.

Fixes #6522
This commit is contained in:
Ben Darnell 2016-10-10 14:32:40 +08:00
parent ca87a13b18
commit 73cae7abd0
7 changed files with 482 additions and 198 deletions

View File

@ -257,6 +257,12 @@ stale log entries:
If candidate receives majority of votes of denials, it reverts back to If candidate receives majority of votes of denials, it reverts back to
follower. follower.
'MsgPreVote' and 'MsgPreVoteResp' are used in an optional two-phase election
protocol. When Config.PreVote is true, a pre-election is carried out first
(using the same rules as a regular election, and no node increases its term
number unless the pre-election indicates that the campaigining node would win.
This minimizes disruption when a partitioned node rejoins the cluster.
'MsgSnap' requests to install a snapshot message. When a node has just 'MsgSnap' requests to install a snapshot message. When a node has just
become a leader or the leader receives 'MsgProp' message, it calls become a leader or the leader receives 'MsgProp' message, it calls
'bcastAppend' method, which then calls 'sendAppend' method to each 'bcastAppend' method, which then calls 'sendAppend' method to each

View File

@ -37,6 +37,7 @@ const (
StateFollower StateType = iota StateFollower StateType = iota
StateCandidate StateCandidate
StateLeader StateLeader
StatePreCandidate
) )
type ReadOnlyOption int type ReadOnlyOption int
@ -55,7 +56,11 @@ const (
// Possible values for CampaignType // Possible values for CampaignType
const ( const (
// campaignElection represents the type of normal election // campaignPreElection represents the first phase of a normal election when
// Config.PreVote is true.
campaignPreElection CampaignType = "CampaignPreElection"
// campaignElection represents a normal (time-based) election (the second phase
// of the election when Config.PreVote is true).
campaignElection CampaignType = "CampaignElection" campaignElection CampaignType = "CampaignElection"
// campaignTransfer represents the type of leader transfer // campaignTransfer represents the type of leader transfer
campaignTransfer CampaignType = "CampaignTransfer" campaignTransfer CampaignType = "CampaignTransfer"
@ -92,6 +97,7 @@ var stmap = [...]string{
"StateFollower", "StateFollower",
"StateCandidate", "StateCandidate",
"StateLeader", "StateLeader",
"StatePreCandidate",
} }
func (st StateType) String() string { func (st StateType) String() string {
@ -149,6 +155,11 @@ type Config struct {
// steps down when quorum is not active for an electionTimeout. // steps down when quorum is not active for an electionTimeout.
CheckQuorum bool CheckQuorum bool
// PreVote enables the Pre-Vote algorithm described in raft thesis section
// 9.6. This prevents disruption when a node that has been partitioned away
// rejoins the cluster.
PreVote bool
// ReadOnlyOption specifies how the read only request is processed. // ReadOnlyOption specifies how the read only request is processed.
// //
// ReadOnlySafe guarantees the linearizability of the read only request by // ReadOnlySafe guarantees the linearizability of the read only request by
@ -236,6 +247,7 @@ type raft struct {
heartbeatElapsed int heartbeatElapsed int
checkQuorum bool checkQuorum bool
preVote bool
heartbeatTimeout int heartbeatTimeout int
electionTimeout int electionTimeout int
@ -280,6 +292,7 @@ func newRaft(c *Config) *raft {
heartbeatTimeout: c.HeartbeatTick, heartbeatTimeout: c.HeartbeatTick,
logger: c.Logger, logger: c.Logger,
checkQuorum: c.CheckQuorum, checkQuorum: c.CheckQuorum,
preVote: c.PreVote,
readOnly: newReadOnly(c.ReadOnlyOption), readOnly: newReadOnly(c.ReadOnlyOption),
} }
for _, p := range peers { for _, p := range peers {
@ -329,11 +342,22 @@ func (r *raft) nodes() []uint64 {
// send persists state to stable storage and then sends to its mailbox. // send persists state to stable storage and then sends to its mailbox.
func (r *raft) send(m pb.Message) { func (r *raft) send(m pb.Message) {
m.From = r.id m.From = r.id
// do not attach term to MsgProp if m.Type == pb.MsgVote || m.Type == pb.MsgPreVote {
// proposals are a way to forward to the leader and if m.Term == 0 {
// should be treated as local message. // PreVote RPCs are sent at a term other than our actual term, so the code
if m.Type != pb.MsgProp { // that sends these messages is responsible for setting the term.
m.Term = r.Term panic(fmt.Sprintf("term should be set when sending %s", m.Type))
}
} else {
if m.Term != 0 {
panic(fmt.Sprintf("term should not be set when sending %s (was %d)", m.Type, m.Term))
}
// do not attach term to MsgProp
// proposals are a way to forward to the leader and
// should be treated as local message.
if m.Type != pb.MsgProp {
m.Term = r.Term
}
} }
r.msgs = append(r.msgs, m) r.msgs = append(r.msgs, m)
} }
@ -555,6 +579,20 @@ func (r *raft) becomeCandidate() {
r.logger.Infof("%x became candidate at term %d", r.id, r.Term) r.logger.Infof("%x became candidate at term %d", r.id, r.Term)
} }
func (r *raft) becomePreCandidate() {
// TODO(xiangli) remove the panic when the raft implementation is stable
if r.state == StateLeader {
panic("invalid transition [leader -> pre-candidate]")
}
// Becoming a pre-candidate changes our step functions and state,
// but doesn't change anything else. In particular it does not increase
// r.Term or change r.Vote.
r.step = stepCandidate
r.tick = r.tickElection
r.state = StatePreCandidate
r.logger.Infof("%x became pre-candidate at term %d", r.id, r.Term)
}
func (r *raft) becomeLeader() { func (r *raft) becomeLeader() {
// TODO(xiangli) remove the panic when the raft implementation is stable // TODO(xiangli) remove the panic when the raft implementation is stable
if r.state == StateFollower { if r.state == StateFollower {
@ -583,31 +621,48 @@ func (r *raft) becomeLeader() {
} }
func (r *raft) campaign(t CampaignType) { func (r *raft) campaign(t CampaignType) {
r.becomeCandidate() var term uint64
if r.quorum() == r.poll(r.id, true) { var voteMsg pb.MessageType
r.becomeLeader() if t == campaignPreElection {
r.becomePreCandidate()
voteMsg = pb.MsgPreVote
// PreVote RPCs are sent for the next term before we've incremented r.Term.
term = r.Term + 1
} else {
r.becomeCandidate()
voteMsg = pb.MsgVote
term = r.Term
}
if r.quorum() == r.poll(r.id, voteRespMsgType(voteMsg), true) {
// We won the election after voting for ourselves (which must mean that
// this is a single-node cluster). Advance to the next state.
if t == campaignPreElection {
r.campaign(campaignElection)
} else {
r.becomeLeader()
}
return return
} }
for id := range r.prs { for id := range r.prs {
if id == r.id { if id == r.id {
continue continue
} }
r.logger.Infof("%x [logterm: %d, index: %d] sent vote request to %x at term %d", r.logger.Infof("%x [logterm: %d, index: %d] sent %s request to %x at term %d",
r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), id, r.Term) r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), voteMsg, id, r.Term)
var ctx []byte var ctx []byte
if t == campaignTransfer { if t == campaignTransfer {
ctx = []byte(t) ctx = []byte(t)
} }
r.send(pb.Message{To: id, Type: pb.MsgVote, Index: r.raftLog.lastIndex(), LogTerm: r.raftLog.lastTerm(), Context: ctx}) r.send(pb.Message{Term: term, To: id, Type: voteMsg, Index: r.raftLog.lastIndex(), LogTerm: r.raftLog.lastTerm(), Context: ctx})
} }
} }
func (r *raft) poll(id uint64, v bool) (granted int) { func (r *raft) poll(id uint64, t pb.MessageType, v bool) (granted int) {
if v { if v {
r.logger.Infof("%x received vote from %x at term %d", r.id, id, r.Term) r.logger.Infof("%x received %s from %x at term %d", r.id, t, id, r.Term)
} else { } else {
r.logger.Infof("%x received vote rejection from %x at term %d", r.id, id, r.Term) r.logger.Infof("%x received %s rejection from %x at term %d", r.id, t, id, r.Term)
} }
if _, ok := r.votes[id]; !ok { if _, ok := r.votes[id]; !ok {
r.votes[id] = v r.votes[id] = v
@ -633,7 +688,11 @@ func (r *raft) Step(m pb.Message) error {
} }
r.logger.Infof("%x is starting a new election at term %d", r.id, r.Term) r.logger.Infof("%x is starting a new election at term %d", r.id, r.Term)
r.campaign(campaignElection) if r.preVote {
r.campaign(campaignPreElection)
} else {
r.campaign(campaignElection)
}
} else { } else {
r.logger.Debugf("%x ignoring MsgHup because already leader", r.id) r.logger.Debugf("%x ignoring MsgHup because already leader", r.id)
} }
@ -645,32 +704,47 @@ func (r *raft) Step(m pb.Message) error {
// local message // local message
case m.Term > r.Term: case m.Term > r.Term:
lead := m.From lead := m.From
if m.Type == pb.MsgVote { if m.Type == pb.MsgVote || m.Type == pb.MsgPreVote {
force := bytes.Equal(m.Context, []byte(campaignTransfer)) force := bytes.Equal(m.Context, []byte(campaignTransfer))
inLease := r.checkQuorum && r.lead != None && r.electionElapsed < r.electionTimeout inLease := r.checkQuorum && r.lead != None && r.electionElapsed < r.electionTimeout
if !force && inLease { if !force && inLease {
// If a server receives a RequestVote request within the minimum election timeout // If a server receives a RequestVote request within the minimum election timeout
// of hearing from a current leader, it does not update its term or grant its vote // of hearing from a current leader, it does not update its term or grant its vote
r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] ignored vote from %x [logterm: %d, index: %d] at term %d: lease is not expired (remaining ticks: %d)", r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] ignored %s from %x [logterm: %d, index: %d] at term %d: lease is not expired (remaining ticks: %d)",
r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term, r.electionTimeout-r.electionElapsed) r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term, r.electionTimeout-r.electionElapsed)
return nil return nil
} }
lead = None lead = None
} }
r.logger.Infof("%x [term: %d] received a %s message with higher term from %x [term: %d]", switch {
r.id, r.Term, m.Type, m.From, m.Term) case m.Type == pb.MsgPreVote:
r.becomeFollower(m.Term, lead) // Never change our term in response to a PreVote
case m.Type == pb.MsgPreVoteResp && !m.Reject:
// We send pre-vote requests with a term in our future. If the
// pre-vote is granted, we will increment our term when we get a
// quorum. If it is not, the term comes from the node that
// rejected our vote so we should become a follower at the new
// term.
default:
r.logger.Infof("%x [term: %d] received a %s message with higher term from %x [term: %d]",
r.id, r.Term, m.Type, m.From, m.Term)
r.becomeFollower(m.Term, lead)
}
case m.Term < r.Term: case m.Term < r.Term:
if r.checkQuorum && (m.Type == pb.MsgHeartbeat || m.Type == pb.MsgApp) { if r.checkQuorum && (m.Type == pb.MsgHeartbeat || m.Type == pb.MsgApp) {
// We have received messages from a leader at a lower term. It is possible that these messages were // We have received messages from a leader at a lower term. It is possible
// simply delayed in the network, but this could also mean that this node has advanced its term number // that these messages were simply delayed in the network, but this could
// during a network partition, and it is now unable to either win an election or to rejoin the majority // also mean that this node has advanced its term number during a network
// on the old term. If checkQuorum is false, this will be handled by incrementing term numbers in response // partition, and it is now unable to either win an election or to rejoin
// to MsgVote with a higher term, but if checkQuorum is true we may not advance the term on MsgVote and // the majority on the old term. If checkQuorum is false, this will be
// must generate other messages to advance the term. The net result of these two features is to minimize // handled by incrementing term numbers in response to MsgVote with a
// the disruption caused by nodes that have been removed from the cluster's configuration: a removed node // higher term, but if checkQuorum is true we may not advance the term on
// will send MsgVotes which will be ignored, but it will not receive MsgApp or MsgHeartbeat, so it will not // MsgVote and must generate other messages to advance the term. The net
// create disruptive term increases // result of these two features is to minimize the disruption caused by
// nodes that have been removed from the cluster's configuration: a
// removed node will send MsgVotes (or MsgPreVotes) which will be ignored,
// but it will not receive MsgApp or MsgHeartbeat, so it will not create
// disruptive term increases
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp}) r.send(pb.Message{To: m.From, Type: pb.MsgAppResp})
} else { } else {
// ignore other cases // ignore other cases
@ -723,10 +797,10 @@ func stepLeader(r *raft, m pb.Message) {
r.appendEntry(m.Entries...) r.appendEntry(m.Entries...)
r.bcastAppend() r.bcastAppend()
return return
case pb.MsgVote: case pb.MsgVote, pb.MsgPreVote:
r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %d", 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.From, m.LogTerm, m.Index, r.Term) 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: pb.MsgVoteResp, Reject: true}) r.send(pb.Message{To: m.From, Type: voteRespMsgType(m.Type), Reject: true})
return return
case pb.MsgReadIndex: case pb.MsgReadIndex:
if r.quorum() > 1 { if r.quorum() > 1 {
@ -884,7 +958,18 @@ func stepLeader(r *raft, m pb.Message) {
} }
} }
// stepCandidate is shared by StateCandidate and StatePreCandidate; the difference is
// whether they respond to MsgVoteResp or MsgPreVoteResp.
func stepCandidate(r *raft, m pb.Message) { func stepCandidate(r *raft, m pb.Message) {
// Only handle vote responses corresponding to our candidacy (while in
// StateCandidate, we may get stale MsgPreVoteResp messages in this term from
// our pre-candidate state).
var myVoteRespType pb.MessageType
if r.state == StatePreCandidate {
myVoteRespType = pb.MsgPreVoteResp
} else {
myVoteRespType = pb.MsgVoteResp
}
switch m.Type { switch m.Type {
case pb.MsgProp: case pb.MsgProp:
r.logger.Infof("%x no leader at term %d; dropping proposal", r.id, r.Term) r.logger.Infof("%x no leader at term %d; dropping proposal", r.id, r.Term)
@ -898,22 +983,27 @@ func stepCandidate(r *raft, m pb.Message) {
case pb.MsgSnap: case pb.MsgSnap:
r.becomeFollower(m.Term, m.From) r.becomeFollower(m.Term, m.From)
r.handleSnapshot(m) r.handleSnapshot(m)
case pb.MsgVote: case pb.MsgVote, pb.MsgPreVote:
r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %d", 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.From, m.LogTerm, m.Index, r.Term) 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: pb.MsgVoteResp, Reject: true}) r.send(pb.Message{To: m.From, Type: voteRespMsgType(m.Type), Reject: true})
case pb.MsgVoteResp: case myVoteRespType:
gr := r.poll(m.From, !m.Reject) gr := r.poll(m.From, m.Type, !m.Reject)
r.logger.Infof("%x [quorum:%d] has received %d votes and %d vote rejections", r.id, r.quorum(), gr, len(r.votes)-gr) r.logger.Infof("%x [quorum:%d] has received %d %s votes and %d vote rejections", r.id, r.quorum(), gr, m.Type, len(r.votes)-gr)
switch r.quorum() { switch r.quorum() {
case gr: case gr:
r.becomeLeader() if r.state == StatePreCandidate {
r.bcastAppend() r.campaign(campaignElection)
} else {
r.becomeLeader()
r.bcastAppend()
}
case len(r.votes) - gr: case len(r.votes) - gr:
r.becomeFollower(r.Term, None) r.becomeFollower(r.Term, None)
} }
case pb.MsgTimeoutNow: case pb.MsgTimeoutNow:
r.logger.Debugf("%x [term %d state %v] ignored MsgTimeoutNow from %x", r.id, r.Term, r.state, m.From) r.logger.Debugf("%x [term %d state %v] ignored MsgTimeoutNow from %x", r.id, r.Term, r.state, m.From)
} }
} }
@ -938,17 +1028,17 @@ func stepFollower(r *raft, m pb.Message) {
r.electionElapsed = 0 r.electionElapsed = 0
r.lead = m.From r.lead = m.From
r.handleSnapshot(m) r.handleSnapshot(m)
case pb.MsgVote: case pb.MsgVote, pb.MsgPreVote:
if (r.Vote == None || r.Vote == m.From) && r.raftLog.isUpToDate(m.Index, m.LogTerm) { if (r.Vote == None || m.Term > r.Term || r.Vote == m.From) && r.raftLog.isUpToDate(m.Index, m.LogTerm) {
r.electionElapsed = 0 r.electionElapsed = 0
r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] voted for %x [logterm: %d, index: %d] at term %d", 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.From, m.LogTerm, m.Index, r.Term) r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term)
r.Vote = m.From r.Vote = m.From
r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp}) r.send(pb.Message{To: m.From, Type: voteRespMsgType(m.Type)})
} else { } else {
r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %d", 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.From, m.LogTerm, m.Index, r.Term) 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: pb.MsgVoteResp, Reject: true}) r.send(pb.Message{To: m.From, Type: voteRespMsgType(m.Type), Reject: true})
} }
case pb.MsgTransferLeader: case pb.MsgTransferLeader:
if r.lead == None { if r.lead == None {
@ -959,6 +1049,9 @@ func stepFollower(r *raft, m pb.Message) {
r.send(m) r.send(m)
case pb.MsgTimeoutNow: case pb.MsgTimeoutNow:
r.logger.Infof("%x [term %d] received MsgTimeoutNow from %x and starts an election to get leadership.", r.id, r.Term, m.From) r.logger.Infof("%x [term %d] received MsgTimeoutNow from %x and starts an election to get leadership.", r.id, r.Term, m.From)
// Leadership transfers never use pre-vote even if r.preVote is true; we
// know we are not recovering from a partition so there is no need for the
// extra round trip.
r.campaign(campaignTransfer) r.campaign(campaignTransfer)
case pb.MsgReadIndex: case pb.MsgReadIndex:
if r.lead == None { if r.lead == None {

View File

@ -288,33 +288,80 @@ func TestProgressPaused(t *testing.T) {
} }
func TestLeaderElection(t *testing.T) { func TestLeaderElection(t *testing.T) {
tests := []struct { for i, preVote := range []bool{false, true} {
*network var cfg func(*Config)
state StateType if preVote {
expTerm uint64 cfg = preVoteConfig
}{
{newNetwork(nil, nil, nil), StateLeader, 1},
{newNetwork(nil, nil, nopStepper), StateLeader, 1},
{newNetwork(nil, nopStepper, nopStepper), StateCandidate, 1},
{newNetwork(nil, nopStepper, nopStepper, nil), StateCandidate, 1},
{newNetwork(nil, nopStepper, nopStepper, nil, nil), StateLeader, 1},
// three logs further along than 0, but in the same term so rejections
// are returned instead of the votes being ignored.
{newNetwork(nil, ents(1), ents(1), ents(1, 1), nil), StateFollower, 1},
// logs converge
{newNetwork(ents(1), nil, ents(2), ents(1), nil), StateLeader, 2},
}
for i, tt := range tests {
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
sm := tt.network.peers[1].(*raft)
if sm.state != tt.state {
t.Errorf("#%d: state = %s, want %s", i, sm.state, tt.state)
} }
if g := sm.Term; g != tt.expTerm { tests := []struct {
t.Errorf("#%d: term = %d, want %d", i, g, tt.expTerm) *network
state StateType
expTerm uint64
}{
{newNetworkWithConfig(cfg, nil, nil, nil), StateLeader, 1},
{newNetworkWithConfig(cfg, nil, nil, nopStepper), StateLeader, 1},
{newNetworkWithConfig(cfg, nil, nopStepper, nopStepper), StateCandidate, 1},
{newNetworkWithConfig(cfg, nil, nopStepper, nopStepper, nil), StateCandidate, 1},
{newNetworkWithConfig(cfg, nil, nopStepper, nopStepper, nil, nil), StateLeader, 1},
// three logs further along than 0, but in the same term so rejections
// are returned instead of the votes being ignored.
{newNetworkWithConfig(cfg, nil, ents(1), ents(1), ents(1, 1), nil), StateFollower, 1},
// logs converge
{newNetworkWithConfig(cfg, ents(1), nil, ents(2), ents(1), nil), StateLeader, 2},
}
for j, tt := range tests {
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
sm := tt.network.peers[1].(*raft)
var expState StateType
var expTerm uint64
if tt.state == StateCandidate && preVote {
// In pre-vote mode, an election that fails to complete
// leaves the node in pre-candidate state without advancing
// the term.
expState = StatePreCandidate
expTerm = 0
} else {
expState = tt.state
expTerm = tt.expTerm
}
if sm.state != expState {
t.Errorf("#%d.%d: state = %s, want %s", i, j, sm.state, expState)
}
if g := sm.Term; g != expTerm {
t.Errorf("#%d.%d: term = %d, want %d", i, j, g, expTerm)
}
}
}
}
// TestLeaderCycle verifies that each node in a cluster can campaign
// and be elected in turn. This ensures that elections (including
// pre-vote) work when not starting from a clean slate (as they do in
// TestLeaderElection)
func TestLeaderCycle(t *testing.T) {
for _, preVote := range []bool{false, true} {
var cfg func(*Config)
if preVote {
cfg = preVoteConfig
}
n := newNetworkWithConfig(cfg, nil, nil, nil)
for campaignerID := uint64(1); campaignerID <= 3; campaignerID++ {
n.send(pb.Message{From: campaignerID, To: campaignerID, Type: pb.MsgHup})
for _, peer := range n.peers {
sm := peer.(*raft)
if sm.id == campaignerID && sm.state != StateLeader {
t.Errorf("preVote=%v: campaigning node %d state = %v, want StateLeader",
preVote, sm.id, sm.state)
} else if sm.id != campaignerID && sm.state != StateFollower {
t.Errorf("preVote=%v: after campaign of node %d, "+
"node %d had state = %v, want StateFollower",
preVote, campaignerID, sm.id, sm.state)
}
}
} }
} }
} }
@ -532,6 +579,76 @@ func TestDuelingCandidates(t *testing.T) {
} }
} }
func TestDuelingPreCandidates(t *testing.T) {
cfgA := newTestConfig(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
cfgB := newTestConfig(2, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
cfgC := newTestConfig(3, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
cfgA.PreVote = true
cfgB.PreVote = true
cfgC.PreVote = true
a := newRaft(cfgA)
b := newRaft(cfgB)
c := newRaft(cfgC)
nt := newNetwork(a, b, c)
nt.cut(1, 3)
nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
// 1 becomes leader since it receives votes from 1 and 2
sm := nt.peers[1].(*raft)
if sm.state != StateLeader {
t.Errorf("state = %s, want %s", sm.state, StateLeader)
}
// 3 campaigns then reverts to follower when its PreVote is rejected
sm = nt.peers[3].(*raft)
if sm.state != StateFollower {
t.Errorf("state = %s, want %s", sm.state, StateFollower)
}
nt.recover()
// Candidate 3 now increases its term and tries to vote again.
// With PreVote, it does not disrupt the leader.
nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
wlog := &raftLog{
storage: &MemoryStorage{ents: []pb.Entry{{}, {Data: nil, Term: 1, Index: 1}}},
committed: 1,
unstable: unstable{offset: 2},
}
tests := []struct {
sm *raft
state StateType
term uint64
raftLog *raftLog
}{
{a, StateLeader, 1, wlog},
{b, StateFollower, 1, wlog},
{c, StateFollower, 1, newLog(NewMemoryStorage(), raftLogger)},
}
for i, tt := range tests {
if g := tt.sm.state; g != tt.state {
t.Errorf("#%d: state = %s, want %s", i, g, tt.state)
}
if g := tt.sm.Term; g != tt.term {
t.Errorf("#%d: term = %d, want %d", i, g, tt.term)
}
base := ltoa(tt.raftLog)
if sm, ok := nt.peers[1+uint64(i)].(*raft); ok {
l := ltoa(sm.raftLog)
if g := diffu(base, l); g != "" {
t.Errorf("#%d: diff:\n%s", i, g)
}
} else {
t.Logf("#%d: empty log", i)
}
}
}
func TestCandidateConcede(t *testing.T) { func TestCandidateConcede(t *testing.T) {
tt := newNetwork(nil, nil, nil) tt := newNetwork(nil, nil, nil)
tt.isolate(1) tt.isolate(1)
@ -586,6 +703,16 @@ func TestSingleNodeCandidate(t *testing.T) {
} }
} }
func TestSingleNodePreCandidate(t *testing.T) {
tt := newNetworkWithConfig(preVoteConfig, nil)
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
sm := tt.peers[1].(*raft)
if sm.state != StateLeader {
t.Errorf("state = %d, want %d", sm.state, StateLeader)
}
}
func TestOldMessages(t *testing.T) { func TestOldMessages(t *testing.T) {
tt := newNetwork(nil, nil, nil) tt := newNetwork(nil, nil, nil)
// make 0 leader @ term 3 // make 0 leader @ term 3
@ -1022,65 +1149,71 @@ func TestMsgAppRespWaitReset(t *testing.T) {
} }
func TestRecvMsgVote(t *testing.T) { func TestRecvMsgVote(t *testing.T) {
tests := []struct { for i, msgType := range []pb.MessageType{pb.MsgVote, pb.MsgPreVote} {
state StateType tests := []struct {
i, term uint64 state StateType
voteFor uint64 i, term uint64
wreject bool voteFor uint64
}{ wreject bool
{StateFollower, 0, 0, None, true}, }{
{StateFollower, 0, 1, None, true}, {StateFollower, 0, 0, None, true},
{StateFollower, 0, 2, None, true}, {StateFollower, 0, 1, None, true},
{StateFollower, 0, 3, None, false}, {StateFollower, 0, 2, None, true},
{StateFollower, 0, 3, None, false},
{StateFollower, 1, 0, None, true}, {StateFollower, 1, 0, None, true},
{StateFollower, 1, 1, None, true}, {StateFollower, 1, 1, None, true},
{StateFollower, 1, 2, None, true}, {StateFollower, 1, 2, None, true},
{StateFollower, 1, 3, None, false}, {StateFollower, 1, 3, None, false},
{StateFollower, 2, 0, None, true}, {StateFollower, 2, 0, None, true},
{StateFollower, 2, 1, None, true}, {StateFollower, 2, 1, None, true},
{StateFollower, 2, 2, None, false}, {StateFollower, 2, 2, None, false},
{StateFollower, 2, 3, None, false}, {StateFollower, 2, 3, None, false},
{StateFollower, 3, 0, None, true}, {StateFollower, 3, 0, None, true},
{StateFollower, 3, 1, None, true}, {StateFollower, 3, 1, None, true},
{StateFollower, 3, 2, None, false}, {StateFollower, 3, 2, None, false},
{StateFollower, 3, 3, None, false}, {StateFollower, 3, 3, None, false},
{StateFollower, 3, 2, 2, false}, {StateFollower, 3, 2, 2, false},
{StateFollower, 3, 2, 1, true}, {StateFollower, 3, 2, 1, true},
{StateLeader, 3, 3, 1, true}, {StateLeader, 3, 3, 1, true},
{StateCandidate, 3, 3, 1, true}, {StatePreCandidate, 3, 3, 1, true},
} {StateCandidate, 3, 3, 1, true},
for i, tt := range tests {
sm := newTestRaft(1, []uint64{1}, 10, 1, NewMemoryStorage())
sm.state = tt.state
switch tt.state {
case StateFollower:
sm.step = stepFollower
case StateCandidate:
sm.step = stepCandidate
case StateLeader:
sm.step = stepLeader
}
sm.Vote = tt.voteFor
sm.raftLog = &raftLog{
storage: &MemoryStorage{ents: []pb.Entry{{}, {Index: 1, Term: 2}, {Index: 2, Term: 2}}},
unstable: unstable{offset: 3},
} }
sm.Step(pb.Message{Type: pb.MsgVote, From: 2, Index: tt.i, LogTerm: tt.term}) for j, tt := range tests {
sm := newTestRaft(1, []uint64{1}, 10, 1, NewMemoryStorage())
sm.state = tt.state
switch tt.state {
case StateFollower:
sm.step = stepFollower
case StateCandidate, StatePreCandidate:
sm.step = stepCandidate
case StateLeader:
sm.step = stepLeader
}
sm.Vote = tt.voteFor
sm.raftLog = &raftLog{
storage: &MemoryStorage{ents: []pb.Entry{{}, {Index: 1, Term: 2}, {Index: 2, Term: 2}}},
unstable: unstable{offset: 3},
}
msgs := sm.readMessages() sm.Step(pb.Message{Type: msgType, From: 2, Index: tt.i, LogTerm: tt.term})
if g := len(msgs); g != 1 {
t.Fatalf("#%d: len(msgs) = %d, want 1", i, g) msgs := sm.readMessages()
continue if g := len(msgs); g != 1 {
} t.Fatalf("#%d.%d: len(msgs) = %d, want 1", i, j, g)
if g := msgs[0].Reject; g != tt.wreject { continue
t.Errorf("#%d, m.Reject = %v, want %v", i, g, tt.wreject) }
if g := msgs[0].Type; g != voteRespMsgType(msgType) {
t.Errorf("#%d.%d, m.Type = %v, want %v", i, j, g, voteRespMsgType(msgType))
}
if g := msgs[0].Reject; g != tt.wreject {
t.Errorf("#%d.%d, m.Reject = %v, want %v", i, j, g, tt.wreject)
}
} }
} }
} }
@ -1094,14 +1227,22 @@ func TestStateTransition(t *testing.T) {
wlead uint64 wlead uint64
}{ }{
{StateFollower, StateFollower, true, 1, None}, {StateFollower, StateFollower, true, 1, None},
{StateFollower, StatePreCandidate, true, 0, None},
{StateFollower, StateCandidate, true, 1, None}, {StateFollower, StateCandidate, true, 1, None},
{StateFollower, StateLeader, false, 0, None}, {StateFollower, StateLeader, false, 0, None},
{StatePreCandidate, StateFollower, true, 0, None},
{StatePreCandidate, StatePreCandidate, true, 0, None},
{StatePreCandidate, StateCandidate, true, 1, None},
{StatePreCandidate, StateLeader, true, 0, 1},
{StateCandidate, StateFollower, true, 0, None}, {StateCandidate, StateFollower, true, 0, None},
{StateCandidate, StatePreCandidate, true, 0, None},
{StateCandidate, StateCandidate, true, 1, None}, {StateCandidate, StateCandidate, true, 1, None},
{StateCandidate, StateLeader, true, 0, 1}, {StateCandidate, StateLeader, true, 0, 1},
{StateLeader, StateFollower, true, 1, None}, {StateLeader, StateFollower, true, 1, None},
{StateLeader, StatePreCandidate, false, 0, None},
{StateLeader, StateCandidate, false, 1, None}, {StateLeader, StateCandidate, false, 1, None},
{StateLeader, StateLeader, true, 0, 1}, {StateLeader, StateLeader, true, 0, 1},
} }
@ -1122,6 +1263,8 @@ func TestStateTransition(t *testing.T) {
switch tt.to { switch tt.to {
case StateFollower: case StateFollower:
sm.becomeFollower(tt.wterm, tt.wlead) sm.becomeFollower(tt.wterm, tt.wlead)
case StatePreCandidate:
sm.becomePreCandidate()
case StateCandidate: case StateCandidate:
sm.becomeCandidate() sm.becomeCandidate()
case StateLeader: case StateLeader:
@ -1147,6 +1290,7 @@ func TestAllServerStepdown(t *testing.T) {
windex uint64 windex uint64
}{ }{
{StateFollower, StateFollower, 3, 0}, {StateFollower, StateFollower, 3, 0},
{StatePreCandidate, StateFollower, 3, 0},
{StateCandidate, StateFollower, 3, 0}, {StateCandidate, StateFollower, 3, 0},
{StateLeader, StateFollower, 3, 1}, {StateLeader, StateFollower, 3, 1},
} }
@ -2176,23 +2320,27 @@ func TestRaftNodes(t *testing.T) {
} }
func TestCampaignWhileLeader(t *testing.T) { func TestCampaignWhileLeader(t *testing.T) {
r := newTestRaft(1, []uint64{1}, 5, 1, NewMemoryStorage()) for i, preVote := range []bool{false, true} {
if r.state != StateFollower { cfg := newTestConfig(1, []uint64{1}, 5, 1, NewMemoryStorage())
t.Errorf("expected new node to be follower but got %s", r.state) cfg.PreVote = preVote
} r := newRaft(cfg)
// We don't call campaign() directly because it comes after the check if r.state != StateFollower {
// for our current state. t.Errorf("#%d: expected new node to be follower but got %s", i, r.state)
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) }
if r.state != StateLeader { // We don't call campaign() directly because it comes after the check
t.Errorf("expected single-node election to become leader but got %s", r.state) // for our current state.
} r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
term := r.Term if r.state != StateLeader {
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) t.Errorf("#%d: expected single-node election to become leader but got %s", i, r.state)
if r.state != StateLeader { }
t.Errorf("expected to remain leader but got %s", r.state) term := r.Term
} r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
if r.Term != term { if r.state != StateLeader {
t.Errorf("expected to remain in term %v but got %v", term, r.Term) t.Errorf("#%d: expected to remain leader but got %s", i, r.state)
}
if r.Term != term {
t.Errorf("#%d: expected to remain in term %v but got %v", i, term, r.Term)
}
} }
} }
@ -2603,6 +2751,12 @@ type network struct {
// A *stateMachine will get its k, id. // A *stateMachine will get its k, id.
// When using stateMachine, the address list is always [1, n]. // When using stateMachine, the address list is always [1, n].
func newNetwork(peers ...stateMachine) *network { func newNetwork(peers ...stateMachine) *network {
return newNetworkWithConfig(nil, peers...)
}
// newNetworkWithConfig is like newNetwork but calls the given func to
// modify the configuration of any state machines it creates.
func newNetworkWithConfig(configFunc func(*Config), peers ...stateMachine) *network {
size := len(peers) size := len(peers)
peerAddrs := idsBySize(size) peerAddrs := idsBySize(size)
@ -2614,7 +2768,11 @@ func newNetwork(peers ...stateMachine) *network {
switch v := p.(type) { switch v := p.(type) {
case nil: case nil:
nstorage[id] = NewMemoryStorage() nstorage[id] = NewMemoryStorage()
sm := newTestRaft(id, peerAddrs, 10, 1, nstorage[id]) cfg := newTestConfig(id, peerAddrs, 10, 1, nstorage[id])
if configFunc != nil {
configFunc(cfg)
}
sm := newRaft(cfg)
npeers[id] = sm npeers[id] = sm
case *raft: case *raft:
v.id = id v.id = id
@ -2638,6 +2796,10 @@ func newNetwork(peers ...stateMachine) *network {
} }
} }
func preVoteConfig(c *Config) {
c.PreVote = true
}
func (nw *network) send(msgs ...pb.Message) { func (nw *network) send(msgs ...pb.Message) {
for len(msgs) > 0 { for len(msgs) > 0 {
m := msgs[0] m := msgs[0]

View File

@ -92,6 +92,8 @@ const (
MsgTimeoutNow MessageType = 14 MsgTimeoutNow MessageType = 14
MsgReadIndex MessageType = 15 MsgReadIndex MessageType = 15
MsgReadIndexResp MessageType = 16 MsgReadIndexResp MessageType = 16
MsgPreVote MessageType = 17
MsgPreVoteResp MessageType = 18
) )
var MessageType_name = map[int32]string{ var MessageType_name = map[int32]string{
@ -112,6 +114,8 @@ var MessageType_name = map[int32]string{
14: "MsgTimeoutNow", 14: "MsgTimeoutNow",
15: "MsgReadIndex", 15: "MsgReadIndex",
16: "MsgReadIndexResp", 16: "MsgReadIndexResp",
17: "MsgPreVote",
18: "MsgPreVoteResp",
} }
var MessageType_value = map[string]int32{ var MessageType_value = map[string]int32{
"MsgHup": 0, "MsgHup": 0,
@ -131,6 +135,8 @@ var MessageType_value = map[string]int32{
"MsgTimeoutNow": 14, "MsgTimeoutNow": 14,
"MsgReadIndex": 15, "MsgReadIndex": 15,
"MsgReadIndexResp": 16, "MsgReadIndexResp": 16,
"MsgPreVote": 17,
"MsgPreVoteResp": 18,
} }
func (x MessageType) Enum() *MessageType { func (x MessageType) Enum() *MessageType {
@ -1836,54 +1842,55 @@ var (
) )
var fileDescriptorRaft = []byte{ var fileDescriptorRaft = []byte{
// 776 bytes of a gzipped FileDescriptorProto // 788 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x64, 0x54, 0xcd, 0x6e, 0xdb, 0x38, 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x64, 0x54, 0xcd, 0x6e, 0xdb, 0x46,
0x10, 0xb6, 0x64, 0xf9, 0x6f, 0xe4, 0x38, 0x0c, 0xe3, 0x5d, 0x10, 0x41, 0xe0, 0xf5, 0x1a, 0x7b, 0x10, 0x16, 0x29, 0xea, 0x6f, 0x28, 0xcb, 0xab, 0xb5, 0x5a, 0x2c, 0x0c, 0x43, 0x55, 0x85, 0x1e,
0x30, 0xb2, 0x48, 0x76, 0xd7, 0x87, 0x3d, 0xf4, 0x96, 0xd8, 0x05, 0x12, 0xa0, 0x0e, 0x5a, 0xc7, 0x04, 0x17, 0x76, 0x5b, 0x1d, 0x7a, 0xe8, 0xcd, 0x96, 0x0a, 0xd8, 0x40, 0x65, 0xb8, 0xb2, 0xdc,
0xe9, 0xa1, 0x45, 0x51, 0x30, 0x16, 0x2d, 0xbb, 0x8d, 0x44, 0x81, 0xa2, 0xd3, 0xe4, 0x52, 0xf4, 0x43, 0x83, 0x20, 0x58, 0x8b, 0x2b, 0x4a, 0x89, 0xc9, 0x25, 0x96, 0x2b, 0xc7, 0xbe, 0x04, 0x79,
0x01, 0xfa, 0x00, 0xbd, 0xf4, 0x19, 0xfa, 0x1a, 0x39, 0xe6, 0x09, 0x8a, 0x26, 0x7d, 0x91, 0x82, 0x80, 0x3c, 0x40, 0x2e, 0x79, 0x1f, 0x1f, 0x0d, 0xe4, 0x1e, 0xc4, 0xce, 0x8b, 0x04, 0xbb, 0x5c,
0x14, 0x65, 0x4b, 0xf1, 0x8d, 0xfc, 0xbe, 0xe1, 0xcc, 0x37, 0xdf, 0x8c, 0x04, 0x20, 0xe8, 0x54, 0x4a, 0xa4, 0x75, 0xdb, 0xfd, 0xbe, 0xd9, 0x99, 0x6f, 0xbe, 0x19, 0x12, 0x40, 0xd0, 0x99, 0x3c,
0x1e, 0x44, 0x82, 0x4b, 0x8e, 0xcb, 0xea, 0x1c, 0x5d, 0xec, 0x34, 0x7d, 0xee, 0x73, 0x0d, 0xfd, 0x8c, 0x04, 0x97, 0x1c, 0x97, 0xd5, 0x39, 0xba, 0xda, 0x6d, 0xf9, 0xdc, 0xe7, 0x1a, 0xfa, 0x4d,
0xa3, 0x4e, 0x09, 0xdb, 0xf9, 0x08, 0xa5, 0xa7, 0xa1, 0x14, 0x37, 0xf8, 0x6f, 0x70, 0xc6, 0x37, 0x9d, 0x12, 0xb6, 0xfb, 0x0e, 0x4a, 0x7f, 0x87, 0x52, 0xdc, 0xe1, 0x5f, 0xc1, 0x99, 0xdc, 0x45,
0x11, 0x23, 0x56, 0xdb, 0xea, 0x36, 0x7a, 0x5b, 0x07, 0xc9, 0xab, 0x03, 0x4d, 0x2a, 0xe2, 0xc8, 0x8c, 0x58, 0x1d, 0xab, 0xd7, 0xe8, 0x37, 0x0f, 0x93, 0x57, 0x87, 0x9a, 0x54, 0xc4, 0xb1, 0x73,
0xb9, 0xfd, 0xfe, 0x47, 0x61, 0xe4, 0xc8, 0x9b, 0x88, 0x61, 0x02, 0xce, 0x98, 0x89, 0x80, 0xd8, 0xff, 0xe5, 0xa7, 0xc2, 0xd8, 0x91, 0x77, 0x11, 0xc3, 0x04, 0x9c, 0x09, 0x13, 0x01, 0xb1, 0x3b,
0x6d, 0xab, 0xeb, 0x2c, 0x19, 0x26, 0x02, 0xbc, 0x03, 0xa5, 0x93, 0xd0, 0x63, 0xd7, 0xa4, 0x98, 0x56, 0xcf, 0x59, 0x31, 0x4c, 0x04, 0x78, 0x17, 0x4a, 0xa7, 0xa1, 0xc7, 0x6e, 0x49, 0x31, 0x43,
0xa1, 0x4a, 0x73, 0x05, 0x61, 0x0c, 0xce, 0x80, 0x4a, 0x4a, 0x9c, 0xb6, 0xd5, 0xad, 0x8f, 0x1c, 0x95, 0x16, 0x0a, 0xc2, 0x18, 0x9c, 0x21, 0x95, 0x94, 0x38, 0x1d, 0xab, 0x57, 0x1f, 0x3b, 0x1e,
0x8f, 0x4a, 0xda, 0xf9, 0x64, 0x01, 0x3a, 0x0b, 0x69, 0x14, 0xcf, 0xb8, 0x1c, 0x32, 0x49, 0x15, 0x95, 0xb4, 0xfb, 0xde, 0x02, 0x74, 0x11, 0xd2, 0x28, 0x9e, 0x73, 0x39, 0x62, 0x92, 0x2a, 0x10,
0x88, 0xff, 0x07, 0x98, 0xf0, 0x70, 0xfa, 0x36, 0x96, 0x54, 0x26, 0x8a, 0xdc, 0x95, 0xa2, 0x3e, 0xff, 0x09, 0x30, 0xe5, 0xe1, 0xec, 0x55, 0x2c, 0xa9, 0x4c, 0x14, 0xb9, 0x6b, 0x45, 0x03, 0x1e,
0x0f, 0xa7, 0x67, 0x8a, 0x30, 0xc9, 0x6b, 0x93, 0x14, 0x50, 0xc5, 0x75, 0xa5, 0x9c, 0x2e, 0x53, 0xce, 0x2e, 0x14, 0x61, 0x92, 0xd7, 0xa6, 0x29, 0xa0, 0x8a, 0xeb, 0x4a, 0x39, 0x5d, 0xa6, 0x38,
0x9c, 0x80, 0x16, 0x98, 0xd3, 0xa5, 0x91, 0xce, 0x2b, 0xa8, 0xa6, 0x0a, 0x94, 0x44, 0xa5, 0x40, 0x01, 0x2d, 0x30, 0xa7, 0x4b, 0x23, 0xdd, 0xff, 0xa1, 0x9a, 0x2a, 0x50, 0x12, 0x95, 0x02, 0x5d,
0xd7, 0x34, 0x12, 0xf1, 0x13, 0xa8, 0x06, 0x46, 0x99, 0x4e, 0xec, 0xf6, 0x48, 0xaa, 0xe5, 0xb1, 0xd3, 0x48, 0xc4, 0x7f, 0x41, 0x35, 0x30, 0xca, 0x74, 0x62, 0xb7, 0x4f, 0x52, 0x2d, 0xcf, 0x95,
0x72, 0x93, 0x77, 0x19, 0xdf, 0xf9, 0x5a, 0x84, 0xca, 0x90, 0xc5, 0x31, 0xf5, 0x19, 0xde, 0x07, 0x9b, 0xbc, 0xab, 0xf8, 0xee, 0xa7, 0x22, 0x54, 0x46, 0x2c, 0x8e, 0xa9, 0xcf, 0xf0, 0x01, 0x68,
0x6d, 0x9e, 0x71, 0x78, 0x3b, 0xcd, 0x61, 0xe8, 0x35, 0x8f, 0x9b, 0x60, 0x4b, 0x9e, 0xeb, 0xc4, 0xf3, 0x8c, 0xc3, 0x3b, 0x69, 0x0e, 0x43, 0x6f, 0x78, 0xdc, 0x02, 0x5b, 0xf2, 0x5c, 0x27, 0xb6,
0x96, 0x5c, 0xb5, 0x31, 0x15, 0xfc, 0x51, 0x1b, 0x0a, 0x59, 0x36, 0xe8, 0xac, 0xcd, 0xa4, 0x05, 0xe4, 0xaa, 0x8d, 0x99, 0xe0, 0xcf, 0xda, 0x50, 0xc8, 0xaa, 0x41, 0x67, 0x63, 0x26, 0x6d, 0xa8,
0x95, 0x4b, 0xee, 0xeb, 0x81, 0x95, 0x32, 0x64, 0x0a, 0xae, 0x6c, 0x2b, 0xaf, 0xdb, 0xb6, 0x0f, 0x5c, 0x73, 0x5f, 0x0f, 0xac, 0x94, 0x21, 0x53, 0x70, 0x6d, 0x5b, 0x79, 0xd3, 0xb6, 0x03, 0xa8,
0x15, 0x16, 0x4a, 0x31, 0x67, 0x31, 0xa9, 0xb4, 0x8b, 0x5d, 0xb7, 0xb7, 0x91, 0xdb, 0x8c, 0x34, 0xb0, 0x50, 0x8a, 0x05, 0x8b, 0x49, 0xa5, 0x53, 0xec, 0xb9, 0xfd, 0xad, 0xdc, 0x66, 0xa4, 0xa9,
0x95, 0x89, 0xc1, 0xbb, 0x50, 0x9e, 0xf0, 0x20, 0x98, 0x4b, 0x52, 0xcd, 0xe4, 0x32, 0x18, 0xee, 0x4c, 0x0c, 0xde, 0x83, 0xf2, 0x94, 0x07, 0xc1, 0x42, 0x92, 0x6a, 0x26, 0x97, 0xc1, 0x70, 0x1f,
0x41, 0x35, 0x36, 0x8e, 0x91, 0x9a, 0x76, 0x12, 0x3d, 0x76, 0x32, 0x75, 0x30, 0x8d, 0x53, 0x19, 0xaa, 0xb1, 0x71, 0x8c, 0xd4, 0xb4, 0x93, 0xe8, 0xb9, 0x93, 0xa9, 0x83, 0x69, 0x9c, 0xca, 0x28,
0x05, 0x7b, 0xc7, 0x26, 0x92, 0x40, 0xdb, 0xea, 0x56, 0xd3, 0x8c, 0x09, 0x86, 0xff, 0x02, 0x48, 0xd8, 0x6b, 0x36, 0x95, 0x04, 0x3a, 0x56, 0xaf, 0x9a, 0x66, 0x4c, 0x30, 0xfc, 0x0b, 0x40, 0x72,
0x4e, 0xc7, 0xf3, 0x50, 0x12, 0x37, 0x53, 0x33, 0x83, 0x63, 0x02, 0x95, 0x09, 0x0f, 0x25, 0xbb, 0x3a, 0x59, 0x84, 0x92, 0xb8, 0x99, 0x9a, 0x19, 0x1c, 0x13, 0xa8, 0x4c, 0x79, 0x28, 0xd9, 0xad,
0x96, 0xa4, 0xae, 0x07, 0x9b, 0x5e, 0x3b, 0x6f, 0xa0, 0x76, 0x4c, 0x85, 0x97, 0xac, 0x4f, 0xea, 0x24, 0x75, 0x3d, 0xd8, 0xf4, 0xda, 0x7d, 0x09, 0xb5, 0x13, 0x2a, 0xbc, 0x64, 0x7d, 0x52, 0x07,
0xa0, 0xb5, 0xe6, 0x20, 0x01, 0xe7, 0x8a, 0x4b, 0x96, 0xdf, 0x77, 0x85, 0x64, 0x1a, 0x2e, 0xae, 0xad, 0x0d, 0x07, 0x09, 0x38, 0x37, 0x5c, 0xb2, 0xfc, 0xbe, 0x2b, 0x24, 0xd3, 0x70, 0x71, 0xb3,
0x37, 0xdc, 0xf9, 0x13, 0x6a, 0xcb, 0x75, 0xc5, 0x4d, 0x28, 0x85, 0xdc, 0x63, 0x31, 0xb1, 0xda, 0xe1, 0xee, 0xcf, 0x50, 0x5b, 0xad, 0x2b, 0x6e, 0x41, 0x29, 0xe4, 0x1e, 0x8b, 0x89, 0xd5, 0x29,
0xc5, 0xae, 0x33, 0x4a, 0x2e, 0x9d, 0xcf, 0x16, 0x80, 0x8a, 0xe9, 0xcf, 0x68, 0xe8, 0xeb, 0xa9, 0xf6, 0x9c, 0x71, 0x72, 0xe9, 0x7e, 0xb0, 0x00, 0x54, 0xcc, 0x60, 0x4e, 0x43, 0x5f, 0x4f, 0xfd,
0x9f, 0x0c, 0x72, 0x0a, 0xec, 0xf9, 0x00, 0xff, 0x6b, 0x3e, 0x4e, 0x5b, 0xaf, 0xce, 0xef, 0xd9, 0x74, 0x98, 0x53, 0x60, 0x2f, 0x86, 0xf8, 0x77, 0xf3, 0x71, 0xda, 0x7a, 0x75, 0x7e, 0xcc, 0x7e,
0x4f, 0x21, 0x79, 0xb7, 0xb6, 0x3d, 0xbb, 0x50, 0x3e, 0xe5, 0x1e, 0x3b, 0x19, 0xe4, 0x75, 0x85, 0x0a, 0xc9, 0xbb, 0x8d, 0xed, 0xd9, 0x83, 0xf2, 0x19, 0xf7, 0xd8, 0xe9, 0x30, 0xaf, 0x2b, 0xd4,
0x1a, 0x53, 0x86, 0xf4, 0x8d, 0x21, 0x4e, 0xce, 0x90, 0xbd, 0xff, 0xa0, 0xb6, 0xfc, 0xe4, 0xf1, 0x98, 0x32, 0x64, 0x60, 0x0c, 0x71, 0x72, 0x86, 0xec, 0xff, 0x01, 0xb5, 0xd5, 0x27, 0x8f, 0xb7,
0x26, 0xb8, 0xfa, 0x72, 0xca, 0x45, 0x40, 0x2f, 0x51, 0x01, 0x6f, 0xc3, 0xa6, 0x06, 0x56, 0x85, 0xc1, 0xd5, 0x97, 0x33, 0x2e, 0x02, 0x7a, 0x8d, 0x0a, 0x78, 0x07, 0xb6, 0x35, 0xb0, 0x2e, 0x8c,
0x91, 0xb5, 0xf7, 0xcd, 0x06, 0x37, 0xb3, 0xc4, 0x18, 0xa0, 0x3c, 0x8c, 0xfd, 0xe3, 0x45, 0x84, 0xac, 0xfd, 0xcf, 0x36, 0xb8, 0x99, 0x25, 0xc6, 0x00, 0xe5, 0x51, 0xec, 0x9f, 0x2c, 0x23, 0x54,
0x0a, 0xd8, 0x85, 0xca, 0x30, 0xf6, 0x8f, 0x18, 0x95, 0xc8, 0x32, 0x97, 0xe7, 0x82, 0x47, 0xc8, 0xc0, 0x2e, 0x54, 0x46, 0xb1, 0x7f, 0xcc, 0xa8, 0x44, 0x96, 0xb9, 0x9c, 0x0b, 0x1e, 0x21, 0xdb,
0x36, 0x51, 0x87, 0x51, 0x84, 0x8a, 0xb8, 0x01, 0x90, 0x9c, 0x47, 0x2c, 0x8e, 0x90, 0x63, 0x02, 0x44, 0x1d, 0x45, 0x11, 0x2a, 0xe2, 0x06, 0x40, 0x72, 0x1e, 0xb3, 0x38, 0x42, 0x8e, 0x09, 0xfc,
0x5f, 0x72, 0xc9, 0x50, 0x49, 0x89, 0x30, 0x17, 0xcd, 0x96, 0x0d, 0xab, 0x16, 0x06, 0x55, 0x30, 0x8f, 0x4b, 0x86, 0x4a, 0x4a, 0x84, 0xb9, 0x68, 0xb6, 0x6c, 0x58, 0xb5, 0x30, 0xa8, 0x82, 0x11,
0x82, 0xba, 0x2a, 0xc6, 0xa8, 0x90, 0x17, 0xaa, 0x4a, 0x15, 0x37, 0x01, 0x65, 0x11, 0xfd, 0xa8, 0xd4, 0x55, 0x31, 0x46, 0x85, 0xbc, 0x52, 0x55, 0xaa, 0xb8, 0x05, 0x28, 0x8b, 0xe8, 0x47, 0x35,
0x86, 0x31, 0x34, 0x86, 0xb1, 0x7f, 0x1e, 0x0a, 0x46, 0x27, 0x33, 0x7a, 0x71, 0xc9, 0x10, 0xe0, 0x8c, 0xa1, 0x31, 0x8a, 0xfd, 0xcb, 0x50, 0x30, 0x3a, 0x9d, 0xd3, 0xab, 0x6b, 0x86, 0x00, 0x37,
0x2d, 0xd8, 0x30, 0x89, 0xd4, 0x80, 0x16, 0x31, 0x72, 0x4d, 0x58, 0x7f, 0xc6, 0x26, 0xef, 0x5f, 0x61, 0xcb, 0x24, 0x52, 0x03, 0x5a, 0xc6, 0xc8, 0x35, 0x61, 0x83, 0x39, 0x9b, 0xbe, 0xf9, 0x77,
0x2c, 0xb8, 0x58, 0x04, 0xa8, 0x8e, 0x7f, 0x83, 0xad, 0x61, 0xec, 0x8f, 0x05, 0x0d, 0xe3, 0x29, 0xc9, 0xc5, 0x32, 0x40, 0x75, 0xfc, 0x03, 0x34, 0x47, 0xb1, 0x3f, 0x11, 0x34, 0x8c, 0x67, 0x4c,
0x13, 0xcf, 0x18, 0xf5, 0x98, 0x40, 0x1b, 0xe6, 0xf5, 0x78, 0x1e, 0x30, 0xbe, 0x90, 0xa7, 0xfc, 0xfc, 0xc3, 0xa8, 0xc7, 0x04, 0xda, 0x32, 0xaf, 0x27, 0x8b, 0x80, 0xf1, 0xa5, 0x3c, 0xe3, 0x6f,
0x03, 0x6a, 0x18, 0x31, 0x23, 0x46, 0x3d, 0xfd, 0x0f, 0x44, 0x9b, 0x46, 0xcc, 0x12, 0xd1, 0x62, 0x51, 0xc3, 0x88, 0x19, 0x33, 0xea, 0xe9, 0x7f, 0x20, 0xda, 0x36, 0x62, 0x56, 0x88, 0x16, 0x83,
0xd0, 0xde, 0x6b, 0x68, 0xe4, 0x47, 0xa7, 0x6a, 0xac, 0x90, 0x43, 0xcf, 0x53, 0xb3, 0x43, 0x05, 0x4c, 0xbf, 0xe7, 0x82, 0xe9, 0x16, 0x9b, 0xa6, 0xaa, 0xb9, 0xeb, 0x18, 0xbc, 0xff, 0x02, 0x1a,
0x4c, 0xa0, 0xb9, 0x82, 0x47, 0x2c, 0xe0, 0x57, 0x4c, 0x33, 0x56, 0x9e, 0x39, 0x8f, 0x3c, 0x2a, 0xf9, 0xf1, 0x2a, 0x1d, 0x6b, 0xe4, 0xc8, 0xf3, 0xd4, 0x7c, 0x51, 0x01, 0x13, 0x68, 0xad, 0xe1,
0x13, 0xc6, 0x3e, 0x22, 0xb7, 0xf7, 0xad, 0xc2, 0xdd, 0x7d, 0xab, 0x70, 0xfb, 0xd0, 0xb2, 0xee, 0x31, 0x0b, 0xf8, 0x0d, 0xd3, 0x8c, 0x95, 0x67, 0x2e, 0x23, 0x8f, 0xca, 0x84, 0xb1, 0x8f, 0xc9,
0x1e, 0x5a, 0xd6, 0x8f, 0x87, 0x96, 0xf5, 0xe5, 0x67, 0xab, 0xf0, 0x2b, 0x00, 0x00, 0xff, 0xff, 0xfd, 0x63, 0xbb, 0xf0, 0xf0, 0xd8, 0x2e, 0xdc, 0x3f, 0xb5, 0xad, 0x87, 0xa7, 0xb6, 0xf5, 0xf5,
0xdc, 0x70, 0xcb, 0xda, 0x16, 0x06, 0x00, 0x00, 0xa9, 0x6d, 0x7d, 0xfc, 0xd6, 0x2e, 0x7c, 0x0f, 0x00, 0x00, 0xff, 0xff, 0xaf, 0x8d, 0x65, 0xcb,
0x3a, 0x06, 0x00, 0x00,
} }

View File

@ -50,6 +50,8 @@ enum MessageType {
MsgTimeoutNow = 14; MsgTimeoutNow = 14;
MsgReadIndex = 15; MsgReadIndex = 15;
MsgReadIndexResp = 16; MsgReadIndexResp = 16;
MsgPreVote = 17;
MsgPreVoteResp = 18;
} }
message Message { message Message {

View File

@ -52,7 +52,19 @@ func IsLocalMsg(msgt pb.MessageType) bool {
} }
func IsResponseMsg(msgt pb.MessageType) bool { func IsResponseMsg(msgt pb.MessageType) bool {
return msgt == pb.MsgAppResp || msgt == pb.MsgVoteResp || msgt == pb.MsgHeartbeatResp || msgt == pb.MsgUnreachable return msgt == pb.MsgAppResp || msgt == pb.MsgVoteResp || msgt == pb.MsgHeartbeatResp || msgt == pb.MsgUnreachable || msgt == pb.MsgPreVoteResp
}
// voteResponseType maps vote and prevote message types to their corresponding responses.
func voteRespMsgType(msgt pb.MessageType) pb.MessageType {
switch msgt {
case pb.MsgVote:
return pb.MsgVoteResp
case pb.MsgPreVote:
return pb.MsgPreVoteResp
default:
panic(fmt.Sprintf("not a vote message: %s", msgt))
}
} }
// EntryFormatter can be implemented by the application to provide human-readable formatting // EntryFormatter can be implemented by the application to provide human-readable formatting

View File

@ -93,6 +93,8 @@ func TestIsLocalMsg(t *testing.T) {
{pb.MsgTimeoutNow, false}, {pb.MsgTimeoutNow, false},
{pb.MsgReadIndex, false}, {pb.MsgReadIndex, false},
{pb.MsgReadIndexResp, false}, {pb.MsgReadIndexResp, false},
{pb.MsgPreVote, false},
{pb.MsgPreVoteResp, false},
} }
for i, tt := range tests { for i, tt := range tests {