1. 背景
在Basic Raft算法中,當(dāng)一個Follower與其他節(jié)點網(wǎng)絡(luò)隔離燥滑,如下圖所示:
Follower_2在electionTimeout沒收到心跳之后,會發(fā)起選舉,并轉(zhuǎn)為Candidate士骤。每次發(fā)起選舉時忆蚀,會把Term加一。由于網(wǎng)絡(luò)隔離嘁扼,它既不會被選成Leader信粮,也不會收到Leader的消息,而是會一直不斷地發(fā)起選舉趁啸。Term會不斷增大强缘。
一段時間之后,這個節(jié)點的Term會非常大不傅。在網(wǎng)絡(luò)恢復(fù)之后旅掂,這個節(jié)點會把它的Term傳播到集群的其他節(jié)點,導(dǎo)致其他節(jié)點更新自己的term访娶,變?yōu)镕ollower辞友。然后觸發(fā)重新選主,但這個舊的Follower_2節(jié)點由于其日志不是最新震肮,并不會成為Leader称龙。整個集群被這個網(wǎng)絡(luò)隔離過的舊節(jié)點擾亂,顯然需要避免的戳晌。
2. Provote算法
Raft作者博士論文《CONSENSUS: BRIDGING THEORY AND PRACTICE》的第9.6節(jié) "Preventing disruptions when a server rejoins the cluster"提到了PreVote算法的大概實現(xiàn)思路鲫尊。
在PreVote算法中,Candidate首先要確認(rèn)自己能贏得集群中大多數(shù)節(jié)點的投票沦偎,這樣才會把自己的term增加疫向,然后發(fā)起真正的投票咳蔚。其他投票節(jié)點同意發(fā)起選舉的條件是(同時滿足下面兩個條件):
- 沒有收到有效領(lǐng)導(dǎo)的心跳,至少有一次選舉超時搔驼。
- Candidate的日志足夠新(Term更大谈火,或者Term相同raft index更大)。
PreVote算法解決了網(wǎng)絡(luò)分區(qū)節(jié)點在重新加入時舌涨,會中斷集群的問題糯耍。在PreVote算法中,網(wǎng)絡(luò)分區(qū)節(jié)點由于無法獲得大部分節(jié)點的許可囊嘉,因此無法增加其Term温技。然后當(dāng)它重新加入集群時,它仍然無法遞增其Term扭粱,因為其他服務(wù)器將一直收到來自Leader節(jié)點的定期心跳信息舵鳞。一旦該服務(wù)器從領(lǐng)導(dǎo)者接收到心跳,它將返回到Follower狀態(tài)琢蛤,Term和Leader一致蜓堕。
3. Etcd的Provote實現(xiàn)流程
Etcd針對發(fā)起PreVote的節(jié)點增加了一個角色狀態(tài):StatePreCandidate。
const (
StateFollower StateType = iota
StateCandidate
StateLeader
StatePreCandidate
numStates
)
3.1 節(jié)點發(fā)起PreVote流程
1.首先節(jié)點超時博其,會進(jìn)入Step函數(shù)套才,然后觸發(fā)選舉流程,如果配置了prevote贺奠,則會進(jìn)入預(yù)選舉流程,代碼片段如下所示:
case pb.MsgHup:
if r.state != StateLeader {
ents, err := r.raftLog.slice(r.raftLog.applied+1, r.raftLog.committed+1, noLimit)
if err != nil {
r.logger.Panicf("unexpected error getting unapplied entries (%v)", err)
}
if n := numOfPendingConf(ents); n != 0 && r.raftLog.committed > r.raftLog.applied {
r.logger.Warningf("%x cannot campaign at term %d since there are still %d pending configuration changes to apply", r.id, r.Term, n)
return nil
}
r.logger.Infof("%x is starting a new election at term %d", r.id, r.Term)
if r.preVote {
r.campaign(campaignPreElection)
} else {
r.campaign(campaignElection)
}
} else {
r.logger.Debugf("%x ignoring MsgHup because already leader", r.id)
}
2.節(jié)點調(diào)用r.campaign(campaignPreElection)错忱,發(fā)送投票請求儡率。函數(shù)流程如下所示:
func (r *raft) campaign(t CampaignType) {
var term uint64
var voteMsg pb.MessageType
if t == campaignPreElection {
r.becomePreCandidate()
voteMsg = pb.MsgPreVote
// PreVote RPCs are sent for the next term before we've incremented r.Term.
//關(guān)鍵點:這里raft的term不會增加,先以r.Term + 1詢問其他節(jié)點以清,而不增加自己的真實term
term = r.Term + 1
} else {
r.becomeCandidate()
voteMsg = pb.MsgVote
term = r.Term
}
//檢查投票是否過半儿普,第一次進(jìn)入該函數(shù)不會執(zhí)行這段邏輯。
//流程3掷倔,會統(tǒng)計投票結(jié)果
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
}
//向其他節(jié)點發(fā)送投票請求
for id := range r.prs {
if id == r.id {
continue
}
r.logger.Infof("%x [logterm: %d, index: %d] sent %s request to %x at term %d",
r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), voteMsg, id, r.Term)
var ctx []byte
if t == campaignTransfer {
ctx = []byte(t)
}
r.send(pb.Message{Term: term, To: id, Type: voteMsg, Index: r.raftLog.lastIndex(), LogTerm: r.raftLog.lastTerm(), Context: ctx})
}
}
3.當(dāng)發(fā)起prevote節(jié)點收到響應(yīng)消息以后眉孩,會進(jìn)入stepCandidate函數(shù),stepCandidate函數(shù)是PreCandidate狀態(tài)和Candidate狀態(tài)共用的勒葱。當(dāng)收到其他節(jié)點對投票的響應(yīng)時浪汪,重新計算自己的票數(shù)。如果達(dá)到大多數(shù)凛虽,PreCandidate會變?yōu)镃andidate狀態(tài)死遭,發(fā)起真正的選舉。代碼片段如下所示:
func stepCandidate(r *raft, m pb.Message) error {
// 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 {
...
case myVoteRespType:
//統(tǒng)計贊成票和反對票
gr := r.poll(m.From, m.Type, !m.Reject)
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() {
case gr:
//當(dāng)贊成票過半后凯旋,PreVote直接轉(zhuǎn)入第二個階段:正式選舉
if r.state == StatePreCandidate {
r.campaign(campaignElection)
} else {
//如果已經(jīng)是StateCandidate,則直接變?yōu)長eader呀潭,選舉結(jié)束钉迷。
r.becomeLeader()
r.bcastAppend()
}
case len(r.votes) - gr:
// pb.MsgPreVoteResp contains future term of pre-candidate
// m.Term > r.Term; reuse r.Term
//如果反對票已過半,這直接變?yōu)镕ollower钠署,并且不增加term
r.becomeFollower(r.Term, None)
}
...
}
return nil
}
3.2 節(jié)點響應(yīng)PreVote流程
節(jié)點收到Prevote請求糠聪,都會進(jìn)入Step函數(shù),然后做相應(yīng)的響應(yīng)處理:
1.如果當(dāng)前節(jié)點未選舉超時谐鼎,并且存在Leader舰蟆,則不響應(yīng)投票請求
2.如果滿足投票要求,并且日志最新该面,則投贊成票夭苗,否則投反對票。
func (r *raft) Step(m pb.Message) error {
// Handle the message term, which may result in our stepping down to a follower.
switch {
...
//#1
case m.Term > r.Term:
if m.Type == pb.MsgVote || m.Type == pb.MsgPreVote {
force := bytes.Equal(m.Context, []byte(campaignTransfer))
inLease := r.checkQuorum && r.lead != None && r.electionElapsed < r.electionTimeout
if !force && inLease {
// 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
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.Type, m.From, m.LogTerm, m.Index, r.Term, r.electionTimeout-r.electionElapsed)
return nil
}
}
...
case pb.MsgVote, pb.MsgPreVote:
...
//#2
// We can vote if this is a repeat of a vote we've already cast...
canVote := r.Vote == m.From ||
// ...we haven't voted and we don't think there's a leader yet in this term...
(r.Vote == None && r.lead == None) ||
// ...or this is a PreVote for a future term...
(m.Type == pb.MsgPreVote && m.Term > r.Term)
// ...and we believe the candidate is up to date.
if canVote && r.raftLog.isUpToDate(m.Index, m.LogTerm) {
...
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
r.Vote = m.From
} else {
...
r.send(pb.Message{To: m.From, Term: r.Term, Type: voteRespMsgType(m.Type), Reject: true})
}
4. 總結(jié)
Prevote是一個典型的2PC協(xié)議隔缀,第一階段先征求其他節(jié)點是否同意選舉题造,如果同意選舉則發(fā)起真正的選舉操作,否則降為Follower角色猾瘸。這樣就避免了網(wǎng)絡(luò)分區(qū)節(jié)點重新加入集群界赔,觸發(fā)不必要的選舉操作。