etcd-raft(一) 领导者选举 主要是讲述了raft集群从启动到选出Leader的过程,Leader主要负责接受提议,将提议以日志(Entry)的形式顺序的同步到Follower,同步到集群多数节点的日志就可以被提交,然后将提交的日志交给应用层,最后通过Ready结构传递到应用层,应用层将提议Apply到数据库中。
日志广播
raft节点在成功竞选Leader之后,首先会调用bcastAppend,像所有其他节点发送追加日志的请求,bcastAppend实现主要是遍历所有的raft节点,结合该节点的日志同步进度Progress,构造同步日志的消息,如下:
// bcastAppend sends RPC, with entries to all peers that are not up-to-date
// according to the progress recorded in r.prs.
func (r *raft) bcastAppend() {
r.forEachProgress(func(id uint64, _ *Progress) {
if id == r.id {
return
}
r.sendAppend(id)
})
}
ho// maybeSendAppend sends an append RPC with new entries to the given peer,
// if necessary. Returns true if a message was sent. The sendIfEmpty
// argument controls whether messages with no entries will be sent
// ("empty" messages are useful to convey updated Commit indexes, but
// are undesirable when we're sending multiple messages in a batch).
func (r *raft) maybeSendAppend(to uint64, sendIfEmpty bool) bool {
pr := r.getProgress(to)
if pr.IsPaused() {
return false
}
m := pb.Message{}
m.To = to
term, errt := r.raftLog.term(pr.Next - 1)
ents, erre := r.raftLog.entries(pr.Next, r.maxMsgSize)
if len(ents) == 0 && !sendIfEmpty {
return false
}
if errt != nil || erre != nil { // send snapshot if we failed to get term or entries
// 先忽略同步snapshot的情况,后面有记录
} else {
m.Type = pb.MsgApp
m.Index = pr.Next - 1
m.LogTerm = term
m.Entries = ents
m.Commit = r.raftLog.committed
if n := len(m.Entries); n != 0 {
switch pr.State {
// optimistically increase the next when in ProgressStateReplicate
case ProgressStateReplicate:
last := m.Entries[n-1].Index
pr.optimisticUpdate(last)
pr.ins.add(last)
case ProgressStateProbe:
pr.pause()
default:
r.logger.Panicf("%x is sending append in unhandled state %s", r.id, pr.State)
}
}
}
r.send(m)
return true
}
Follower的日志同步进度维护在Progress对象中,通过pr := r.getProgress(to)获取Follower的日志同步状态信息,根据日志同步状态及Leader的现有日志情况,构造日志同步消息,将消息传递给transport发送出去,消息的具体字段如:
Type:消息类型,MsgApp表明本次消息是用来同步日志的
Index:最后一次拷贝到改Follower上的日志index
Term:最后一次拷贝到改Follower上的日志的term
Entries:本次需要同步的日志Entry数组
Commit:在Leader上已经提交的最后一条日志的Index
Follower接收消息
日志追加消息从Leader发送出去之后,经过transport层到达Follower中,处理追加日志消息的逻辑主要在stepFollower,代码如下:
func stepFollower(r *raft, m pb.Message) error {
switch m.Type {
// ...
case pb.MsgApp:
r.electionElapsed = 0
r.lead = m.From
r.handleAppendEntries(m)
// other case ...
}
return nil
}
在MsgApp的消息处理逻辑中可以看到electionElapsed计数器也会被清零,日志追加请求也可以理解为Leader的一次心跳包,此外Follower对日志的处理逻辑在handleAppendEntries函数中,主要处理逻辑如下:
func (r *raft) handleAppendEntries(m pb.Message) {
if m.Index < r.raftLog.committed {
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed})
return
}
if mlastIndex, ok := r.raftLog.maybeAppend(m.Index, m.LogTerm, m.Commit, m.Entries...); ok {
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: mlastIndex})
} else {
r.logger.Debugf("%x [logterm: %d, index: %d] rejected msgApp [logterm: %d, index: %d] from %x",
r.id, r.raftLog.zeroTermOnErrCompacted(r.raftLog.term(m.Index)), m.Index, m.LogTerm, m.Index, m.From)
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: m.Index, Reject: true, RejectHint: r.raftLog.lastIndex()})
}
}
在handleAppendEntries函数中,存在一个日志拷贝进度协商的问题,因为在Leader刚刚上任时,所有Follower的进度都是假设以Leader自身的日志进度一致,Follower还有可能接收过其他Leader的日志追加请求,因此,同步给Follower的日志可能和Follower现有日志出现index不连续、日志index-term不匹配的情况,因此,Follower需要结合自身拥有的日志情况,对日志追加请求里面的日志条目,进行可追加与否的判断,并将追加结果及判断结果反馈给Leader,具体在Follower上对日志追加请求的处理分为如下几种情况。
拷贝已经提交的日志
如果Leader向Follower拷贝了Follower已经提交的日志,Follower通知Leader从其committed的日志index开始拷贝,Leader收到反馈后会将改Follower的拷贝进度更新到committed,代码如下:
if m.Index < r.raftLog.committed {
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed})
return
}
Leader同步的日志中和Follower有交集
Follower除了已经提交的日志之外,可能还保存若干条未提交的日志,因为Leader在发送追加日志请求时携带了如下两个字段:
Index:最后一次拷贝到改Follower上的日志index
Term:最后一次拷贝到改Follower上的日志的term
所以,可以通过查询这两个字段是否在Index和term里面存在来判断Follower保存的未提交的日志和Leader的日志是否有交集,如果有交集,就会通过findConflict寻找可以append的第一条日志index,然后将index及之后的日志Append到raftlog中(由于raft的日志强Leader属性,可能会覆盖来自其他Leader的日志,这些被覆盖的日志index和新Leader一样,但是term不一样),具体的逻辑如下:
func (l *raftLog) maybeAppend(index, logTerm, committed uint64, ents ...pb.Entry) (lastnewi uint64, ok bool) {
if l.matchTerm(index, logTerm) {
lastnewi = index + uint64(len(ents))
ci := l.findConflict(ents)
switch {
case ci == 0:
case ci <= l.committed:
l.logger.Panicf("entry %d conflict with committed entry [committed(%d)]", ci, l.committed)
default:
offset := index + 1
l.append(ents[ci-offset:]...)
}
l.commitTo(min(committed, lastnewi))
return lastnewi, true
}
return 0, false
}
func (l *raftLog) findConflict(ents []pb.Entry) uint64 {
for _, ne := range ents {
if !l.matchTerm(ne.Index, ne.Term) {
if ne.Index <= l.lastIndex() {
l.logger.Infof("found conflict at index %d [existing term: %d, conflicting term: %d]",
ne.Index, l.zeroTermOnErrCompacted(l.term(ne.Index)), ne.Term)
}
return ne.Index
}
}
return 0
}
通过maybeAppend中matchTerm如果判断日志存在交集,就进行日志冲突匹配,由于日志index是递增的,如果Follower接收过其他任期Leader的日志,可能出现index相同但term不同的情况,因此通过findConflict找到开始出现冲突的index,Follower上index及其之后的日志将会被新Leader的日志覆盖,每一个任期内Follower的日志始终以Leader的为准,之前任期Leader未提交的日志,且新Leader上没有的日志将会被新Leader的日志覆盖。
Leader同步的日志中和Follower无交集
如果Follower的日志index和Leader的日志无交集,那么可能Follower的日志比较比较旧,此时Follower会直接拒绝Leader追加日志的请求,并向Leader反馈,自己拥有的最后一条日志的index,让Leader下次从这个位置开始拷贝日志,具体消息内容如下:
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: m.Index, Reject: true, RejectHint: r.raftLog.lastIndex()})
Leader处理MsgAppResp
从上面Follower接收日志的情况来看,Follower的响应分为两类:拒绝日志追加请求,返回RejectHint、接受日志追加请求。
日志追加请求被拒绝
此情况下,Leader会将该Follower待同步日志Next更新为拒绝消息里的Index及RejectHint中较小值,然后继续调用sendAppend在新的Next值上进行新一轮探测(调整Next值,如果Follower的日志很旧,可能导致后续再次进行日志同步时,Next-1的日志已经被Leader压缩掉,可能会导致Leader向Follower同步snapshot),具体代码如下:
func stepLeader(r *raft, m pb.Message) error {
// ...
// All other message types require a progress for m.From (pr).
pr := r.getProgress(m.From)
if pr == nil {
r.logger.Debugf("%x no progress available for %x", r.id, m.From)
return nil
}
switch m.Type {
case pb.MsgAppResp:
pr.RecentActive = true
if m.Reject {
r.logger.Debugf("%x received msgApp rejection(lastindex: %d) from %x for index %d",
r.id, m.RejectHint, m.From, m.Index)
if pr.maybeDecrTo(m.Index, m.RejectHint) {
r.logger.Debugf("%x decreased progress of %x to [%s]", r.id, m.From, pr)
if pr.State == ProgressStateReplicate {
pr.becomeProbe()
}
r.sendAppend(m.From)
}
} else {
// ...
}
// other case
}
return nil
}
// maybeDecrTo returns false if the given to index comes from an out of order message.
// Otherwise it decreases the progress next index to min(rejected, last) and returns true.
func (pr *Progress) maybeDecrTo(rejected, last uint64) bool {
if pr.State == ProgressStateReplicate {
// the rejection must be stale if the progress has matched and "rejected"
// is smaller than "match".
if rejected <= pr.Match {
return false
}
// directly decrease next to match + 1
pr.Next = pr.Match + 1
return true
}
// the rejection must be stale if "rejected" does not match next - 1
if pr.Next-1 != rejected {
return false
}
if pr.Next = min(rejected, last+1); pr.Next < 1 {
pr.Next = 1
}
pr.resume()
return true
}
日志追加请求被接受
Leader根据响应中Index(Follower和Leader匹配成功的日志index),被Follower接收的最后一条日志Index调整该Follower日志同步进度的Match及Next值,见maybeUpdate,Progress中的Match值在Follower第一次被接受同步的日志记录后,被调整为Leader和Follower匹配成功的Index,如果出现日配探测成功,Follower日志同步状态进入ProgressStateReplicate状态,具体如下:
func stepLeader(r *raft, m pb.Message) error {
// ...
// All other message types require a progress for m.From (pr).
pr := r.getProgress(m.From)
if pr == nil {
r.logger.Debugf("%x no progress available for %x", r.id, m.From)
return nil
}
switch m.Type {
case pb.MsgAppResp:
pr.RecentActive = true
if m.Reject {
// 消息被拒绝的处理场景
} else {
oldPaused := pr.IsPaused()
if pr.maybeUpdate(m.Index) {
switch {
case pr.State == ProgressStateProbe:
pr.becomeReplicate()
case pr.State == ProgressStateSnapshot && pr.needSnapshotAbort():
r.logger.Debugf("%x snapshot aborted, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
// Transition back to replicating state via probing state
// (which takes the snapshot into account). If we didn't
// move to replicating state, that would only happen with
// the next round of appends (but there may not be a next
// round for a while, exposing an inconsistent RaftStatus).
pr.becomeProbe()
pr.becomeReplicate()
case pr.State == ProgressStateReplicate:
pr.ins.freeTo(m.Index)
}
if r.maybeCommit() {
r.bcastAppend()
} else if oldPaused {
// If we were paused before, this node may be missing the
// latest commit index, so send it.
r.sendAppend(m.From)
}
// We've updated flow control information above, which may
// allow us to send multiple (size-limited) in-flight messages
// at once (such as when transitioning from probe to
// replicate, or when freeTo() covers multiple messages). If
// we have more entries to send, send as many messages as we
// can (without sending empty messages for the commit index)
for r.maybeSendAppend(m.From, false) {
}
// Transfer leadership is in progress.
if m.From == r.leadTransferee && pr.Match == r.raftLog.lastIndex() {
r.logger.Infof("%x sent MsgTimeoutNow to %x after received MsgAppResp", r.id, m.From)
r.sendTimeoutNow(m.From)
}
}
}
// other case ...
}
return nil
}
// maybeUpdate returns false if the given n index comes from an outdated message.
// Otherwise it updates the progress and returns true.
func (pr *Progress) maybeUpdate(n uint64) bool {
var updated bool
if pr.Match < n {
pr.Match = n
updated = true
pr.resume()
}
if pr.Next < n+1 {
pr.Next = n + 1
}
return updated
}
任何一次日志在任何一个Follower同步成功,都有可能出现新的日志被拷贝到大多数节点上,因此,日志同步成功时,Leader还会判断是否有新的日志已经被拷贝到raft集群的大多数节点上,如果有新的Entry被拷贝到集群quorum个节点上,就提交这些日志,更新自身committed值,然后会通知其他Followe提交日志,具体代码如下:
if r.maybeCommit() {
r.bcastAppend()
}
// maybeCommit attempts to advance the commit index. Returns true if
// the commit index changed (in which case the caller should call
// r.bcastAppend).
func (r *raft) maybeCommit() bool {
// Preserving matchBuf across calls is an optimization
// used to avoid allocating a new slice on each call.
if cap(r.matchBuf) < len(r.prs) {
r.matchBuf = make(uint64Slice, len(r.prs))
}
r.matchBuf = r.matchBuf[:len(r.prs)]
idx := 0
for _, p := range r.prs {
r.matchBuf[idx] = p.Match
idx++
}
sort.Sort(&r.matchBuf)
mci := r.matchBuf[len(r.matchBuf)-r.quorum()]
return r.raftLog.maybeCommit(mci, r.Term)
}
但是需要注意,Leader只会提交它当前任期的日志,所以即使Leader拥有的日志如果不是其任期产生的是不会被提交的,判断逻辑在maybeCommit里面,具体可以见raft论文[1]:5.4.2
func (l *raftLog) maybeCommit(maxIndex, term uint64) bool {
if maxIndex > l.committed && l.zeroTermOnErrCompacted(l.term(maxIndex)) == term {
l.commitTo(maxIndex)
return true
}
return false
}
同步snapshot
etcd里面日志会被定期的压缩compact,如果Follower的日志比较落后,需要拷贝的日志已经被压缩,Leader就会尝试同步snapshot的方式向该Follower同步数据,Leader向Follower同步数据的同时,会将其同步状态切换到:ProgressStateSnapshot,具体代码如下:
// maybeSendAppend sends an append RPC with new entries to the given peer,
// if necessary. Returns true if a message was sent. The sendIfEmpty
// argument controls whether messages with no entries will be sent
// ("empty" messages are useful to convey updated Commit indexes, but
// are undesirable when we're sending multiple messages in a batch).
func (r *raft) maybeSendAppend(to uint64, sendIfEmpty bool) bool {
pr := r.getProgress(to)
if pr.IsPaused() {
return false
}
m := pb.Message{}
m.To = to
term, errt := r.raftLog.term(pr.Next - 1)
ents, erre := r.raftLog.entries(pr.Next, r.maxMsgSize)
if len(ents) == 0 && !sendIfEmpty {
return false
}
if errt != nil || erre != nil { // send snapshot if we failed to get term or entries
if !pr.RecentActive {
r.logger.Debugf("ignore sending snapshot to %x since it is not recently active", to)
return false
}
m.Type = pb.MsgSnap
snapshot, err := r.raftLog.snapshot()
if err != nil {
if err == ErrSnapshotTemporarilyUnavailable {
r.logger.Debugf("%x failed to send snapshot to %x because snapshot is temporarily unavailable", r.id, to)
return false
}
panic(err) // TODO(bdarnell)
}
if IsEmptySnap(snapshot) {
panic("need non-empty snapshot")
}
m.Snapshot = snapshot
sindex, sterm := snapshot.Metadata.Index, snapshot.Metadata.Term
r.logger.Debugf("%x [firstindex: %d, commit: %d] sent snapshot[index: %d, term: %d] to %x [%s]",
r.id, r.raftLog.firstIndex(), r.raftLog.committed, sindex, sterm, to, pr)
pr.becomeSnapshot(sindex)
r.logger.Debugf("%x paused sending replication messages to %x [%s]", r.id, to, pr)
} else {
// ....
}
r.send(m)
return true
}
Follower处理snapshot
Follower处理snapshot同步消息的逻辑主要在stepFollower里面,具体如下:
func stepFollower(r *raft, m pb.Message) error {
switch m.Type {
// ...
case pb.MsgSnap:
r.electionElapsed = 0
r.lead = m.From
r.handleSnapshot(m)
// other case
}
return nil
}
func (r *raft) handleSnapshot(m pb.Message) {
sindex, sterm := m.Snapshot.Metadata.Index, m.Snapshot.Metadata.Term
if r.restore(m.Snapshot) {
r.logger.Infof("%x [commit: %d] restored snapshot [index: %d, term: %d]",
r.id, r.raftLog.committed, sindex, sterm)
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.lastIndex()})
} else {
r.logger.Infof("%x [commit: %d] ignored snapshot [index: %d, term: %d]",
r.id, r.raftLog.committed, sindex, sterm)
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed})
}
}
// restore recovers the state machine from a snapshot. It restores the log and the
// configuration of state machine.
func (r *raft) restore(s pb.Snapshot) bool {
if s.Metadata.Index <= r.raftLog.committed {
return false
}
if r.raftLog.matchTerm(s.Metadata.Index, s.Metadata.Term) {
r.logger.Infof("%x [commit: %d, lastindex: %d, lastterm: %d] fast-forwarded commit to snapshot [index: %d, term: %d]",
r.id, r.raftLog.committed, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
r.raftLog.commitTo(s.Metadata.Index)
return false
}
// The normal peer can't become learner.
if !r.isLearner {
for _, id := range s.Metadata.ConfState.Learners {
if id == r.id {
r.logger.Errorf("%x can't become learner when restores snapshot [index: %d, term: %d]", r.id, s.Metadata.Index, s.Metadata.Term)
return false
}
}
}
r.logger.Infof("%x [commit: %d, lastindex: %d, lastterm: %d] starts to restore snapshot [index: %d, term: %d]",
r.id, r.raftLog.committed, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
r.raftLog.restore(s)
r.prs = make(map[uint64]*Progress)
r.learnerPrs = make(map[uint64]*Progress)
r.restoreNode(s.Metadata.ConfState.Nodes, false)
r.restoreNode(s.Metadata.ConfState.Learners, true)
return true
}
存储snapshot的逻辑主要在raftlog的restrore方法中,如果snapshot里面最后一条日志的index已经被提交,改snapshot不会被存储下来。
如果snapshot里面最后一条日志的index及term能够和当前Follower的raftlog里面未提交的日志匹配上,由于snapshot里面打包的都是已经commited且applied的日志,所以Follower可以将其日志提交到该index处。
如果不是上面两种情况,raft将snapshot存储到raftlog中,更新committed,并将目前拥有的最后一条日志的index作为接收的日志index响应给Leader,具体逻辑如:
func (l *raftLog) restore(s pb.Snapshot) {
l.logger.Infof("log [%s] starts to restore snapshot [index: %d, term: %d]", l, s.Metadata.Index, s.Metadata.Term)
l.committed = s.Metadata.Index
l.unstable.restore(s)
}
func (u *unstable) restore(s pb.Snapshot) {
u.offset = s.Metadata.Index + 1
u.entries = nil
u.snapshot = &s
}
参考
- raft原文 raft.github.io/raft.pdf