blob: e1e6a16d4a09b19fef74814dd37e60b27cc95907 [file] [log] [blame]
Stephane Barbarie260a5632019-02-26 16:12:49 -05001// Copyright 2015 The etcd Authors
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7// http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15package raft
16
17import (
18 "bytes"
19 "errors"
20 "fmt"
21 "math"
22 "math/rand"
23 "sort"
24 "strings"
25 "sync"
26 "time"
27
28 pb "go.etcd.io/etcd/raft/raftpb"
29)
30
31// None is a placeholder node ID used when there is no leader.
32const None uint64 = 0
33const noLimit = math.MaxUint64
34
35// Possible values for StateType.
36const (
37 StateFollower StateType = iota
38 StateCandidate
39 StateLeader
40 StatePreCandidate
41 numStates
42)
43
44type ReadOnlyOption int
45
46const (
47 // ReadOnlySafe guarantees the linearizability of the read only request by
48 // communicating with the quorum. It is the default and suggested option.
49 ReadOnlySafe ReadOnlyOption = iota
50 // ReadOnlyLeaseBased ensures linearizability of the read only request by
51 // relying on the leader lease. It can be affected by clock drift.
52 // If the clock drift is unbounded, leader might keep the lease longer than it
53 // should (clock can move backward/pause without any bound). ReadIndex is not safe
54 // in that case.
55 ReadOnlyLeaseBased
56)
57
58// Possible values for CampaignType
59const (
60 // campaignPreElection represents the first phase of a normal election when
61 // Config.PreVote is true.
62 campaignPreElection CampaignType = "CampaignPreElection"
63 // campaignElection represents a normal (time-based) election (the second phase
64 // of the election when Config.PreVote is true).
65 campaignElection CampaignType = "CampaignElection"
66 // campaignTransfer represents the type of leader transfer
67 campaignTransfer CampaignType = "CampaignTransfer"
68)
69
70// ErrProposalDropped is returned when the proposal is ignored by some cases,
71// so that the proposer can be notified and fail fast.
72var ErrProposalDropped = errors.New("raft proposal dropped")
73
74// lockedRand is a small wrapper around rand.Rand to provide
75// synchronization among multiple raft groups. Only the methods needed
76// by the code are exposed (e.g. Intn).
77type lockedRand struct {
78 mu sync.Mutex
79 rand *rand.Rand
80}
81
82func (r *lockedRand) Intn(n int) int {
83 r.mu.Lock()
84 v := r.rand.Intn(n)
85 r.mu.Unlock()
86 return v
87}
88
89var globalRand = &lockedRand{
90 rand: rand.New(rand.NewSource(time.Now().UnixNano())),
91}
92
93// CampaignType represents the type of campaigning
94// the reason we use the type of string instead of uint64
95// is because it's simpler to compare and fill in raft entries
96type CampaignType string
97
98// StateType represents the role of a node in a cluster.
99type StateType uint64
100
101var stmap = [...]string{
102 "StateFollower",
103 "StateCandidate",
104 "StateLeader",
105 "StatePreCandidate",
106}
107
108func (st StateType) String() string {
109 return stmap[uint64(st)]
110}
111
112// Config contains the parameters to start a raft.
113type Config struct {
114 // ID is the identity of the local raft. ID cannot be 0.
115 ID uint64
116
117 // peers contains the IDs of all nodes (including self) in the raft cluster. It
118 // should only be set when starting a new raft cluster. Restarting raft from
119 // previous configuration will panic if peers is set. peer is private and only
120 // used for testing right now.
121 peers []uint64
122
123 // learners contains the IDs of all learner nodes (including self if the
124 // local node is a learner) in the raft cluster. learners only receives
125 // entries from the leader node. It does not vote or promote itself.
126 learners []uint64
127
128 // ElectionTick is the number of Node.Tick invocations that must pass between
129 // elections. That is, if a follower does not receive any message from the
130 // leader of current term before ElectionTick has elapsed, it will become
131 // candidate and start an election. ElectionTick must be greater than
132 // HeartbeatTick. We suggest ElectionTick = 10 * HeartbeatTick to avoid
133 // unnecessary leader switching.
134 ElectionTick int
135 // HeartbeatTick is the number of Node.Tick invocations that must pass between
136 // heartbeats. That is, a leader sends heartbeat messages to maintain its
137 // leadership every HeartbeatTick ticks.
138 HeartbeatTick int
139
140 // Storage is the storage for raft. raft generates entries and states to be
141 // stored in storage. raft reads the persisted entries and states out of
142 // Storage when it needs. raft reads out the previous state and configuration
143 // out of storage when restarting.
144 Storage Storage
145 // Applied is the last applied index. It should only be set when restarting
146 // raft. raft will not return entries to the application smaller or equal to
147 // Applied. If Applied is unset when restarting, raft might return previous
148 // applied entries. This is a very application dependent configuration.
149 Applied uint64
150
151 // MaxSizePerMsg limits the max byte size of each append message. Smaller
152 // value lowers the raft recovery cost(initial probing and message lost
153 // during normal operation). On the other side, it might affect the
154 // throughput during normal replication. Note: math.MaxUint64 for unlimited,
155 // 0 for at most one entry per message.
156 MaxSizePerMsg uint64
157 // MaxCommittedSizePerReady limits the size of the committed entries which
158 // can be applied.
159 MaxCommittedSizePerReady uint64
160 // MaxUncommittedEntriesSize limits the aggregate byte size of the
161 // uncommitted entries that may be appended to a leader's log. Once this
162 // limit is exceeded, proposals will begin to return ErrProposalDropped
163 // errors. Note: 0 for no limit.
164 MaxUncommittedEntriesSize uint64
165 // MaxInflightMsgs limits the max number of in-flight append messages during
166 // optimistic replication phase. The application transportation layer usually
167 // has its own sending buffer over TCP/UDP. Setting MaxInflightMsgs to avoid
168 // overflowing that sending buffer. TODO (xiangli): feedback to application to
169 // limit the proposal rate?
170 MaxInflightMsgs int
171
172 // CheckQuorum specifies if the leader should check quorum activity. Leader
173 // steps down when quorum is not active for an electionTimeout.
174 CheckQuorum bool
175
176 // PreVote enables the Pre-Vote algorithm described in raft thesis section
177 // 9.6. This prevents disruption when a node that has been partitioned away
178 // rejoins the cluster.
179 PreVote bool
180
181 // ReadOnlyOption specifies how the read only request is processed.
182 //
183 // ReadOnlySafe guarantees the linearizability of the read only request by
184 // communicating with the quorum. It is the default and suggested option.
185 //
186 // ReadOnlyLeaseBased ensures linearizability of the read only request by
187 // relying on the leader lease. It can be affected by clock drift.
188 // If the clock drift is unbounded, leader might keep the lease longer than it
189 // should (clock can move backward/pause without any bound). ReadIndex is not safe
190 // in that case.
191 // CheckQuorum MUST be enabled if ReadOnlyOption is ReadOnlyLeaseBased.
192 ReadOnlyOption ReadOnlyOption
193
194 // Logger is the logger used for raft log. For multinode which can host
195 // multiple raft group, each raft group can have its own logger
196 Logger Logger
197
198 // DisableProposalForwarding set to true means that followers will drop
199 // proposals, rather than forwarding them to the leader. One use case for
200 // this feature would be in a situation where the Raft leader is used to
201 // compute the data of a proposal, for example, adding a timestamp from a
202 // hybrid logical clock to data in a monotonically increasing way. Forwarding
203 // should be disabled to prevent a follower with an inaccurate hybrid
204 // logical clock from assigning the timestamp and then forwarding the data
205 // to the leader.
206 DisableProposalForwarding bool
207}
208
209func (c *Config) validate() error {
210 if c.ID == None {
211 return errors.New("cannot use none as id")
212 }
213
214 if c.HeartbeatTick <= 0 {
215 return errors.New("heartbeat tick must be greater than 0")
216 }
217
218 if c.ElectionTick <= c.HeartbeatTick {
219 return errors.New("election tick must be greater than heartbeat tick")
220 }
221
222 if c.Storage == nil {
223 return errors.New("storage cannot be nil")
224 }
225
226 if c.MaxUncommittedEntriesSize == 0 {
227 c.MaxUncommittedEntriesSize = noLimit
228 }
229
230 // default MaxCommittedSizePerReady to MaxSizePerMsg because they were
231 // previously the same parameter.
232 if c.MaxCommittedSizePerReady == 0 {
233 c.MaxCommittedSizePerReady = c.MaxSizePerMsg
234 }
235
236 if c.MaxInflightMsgs <= 0 {
237 return errors.New("max inflight messages must be greater than 0")
238 }
239
240 if c.Logger == nil {
241 c.Logger = raftLogger
242 }
243
244 if c.ReadOnlyOption == ReadOnlyLeaseBased && !c.CheckQuorum {
245 return errors.New("CheckQuorum must be enabled when ReadOnlyOption is ReadOnlyLeaseBased")
246 }
247
248 return nil
249}
250
251type raft struct {
252 id uint64
253
254 Term uint64
255 Vote uint64
256
257 readStates []ReadState
258
259 // the log
260 raftLog *raftLog
261
262 maxMsgSize uint64
263 maxUncommittedSize uint64
264 maxInflight int
265 prs map[uint64]*Progress
266 learnerPrs map[uint64]*Progress
267 matchBuf uint64Slice
268
269 state StateType
270
271 // isLearner is true if the local raft node is a learner.
272 isLearner bool
273
274 votes map[uint64]bool
275
276 msgs []pb.Message
277
278 // the leader id
279 lead uint64
280 // leadTransferee is id of the leader transfer target when its value is not zero.
281 // Follow the procedure defined in raft thesis 3.10.
282 leadTransferee uint64
283 // Only one conf change may be pending (in the log, but not yet
284 // applied) at a time. This is enforced via pendingConfIndex, which
285 // is set to a value >= the log index of the latest pending
286 // configuration change (if any). Config changes are only allowed to
287 // be proposed if the leader's applied index is greater than this
288 // value.
289 pendingConfIndex uint64
290 // an estimate of the size of the uncommitted tail of the Raft log. Used to
291 // prevent unbounded log growth. Only maintained by the leader. Reset on
292 // term changes.
293 uncommittedSize uint64
294
295 readOnly *readOnly
296
297 // number of ticks since it reached last electionTimeout when it is leader
298 // or candidate.
299 // number of ticks since it reached last electionTimeout or received a
300 // valid message from current leader when it is a follower.
301 electionElapsed int
302
303 // number of ticks since it reached last heartbeatTimeout.
304 // only leader keeps heartbeatElapsed.
305 heartbeatElapsed int
306
307 checkQuorum bool
308 preVote bool
309
310 heartbeatTimeout int
311 electionTimeout int
312 // randomizedElectionTimeout is a random number between
313 // [electiontimeout, 2 * electiontimeout - 1]. It gets reset
314 // when raft changes its state to follower or candidate.
315 randomizedElectionTimeout int
316 disableProposalForwarding bool
317
318 tick func()
319 step stepFunc
320
321 logger Logger
322}
323
324func newRaft(c *Config) *raft {
325 if err := c.validate(); err != nil {
326 panic(err.Error())
327 }
328 raftlog := newLogWithSize(c.Storage, c.Logger, c.MaxCommittedSizePerReady)
329 hs, cs, err := c.Storage.InitialState()
330 if err != nil {
331 panic(err) // TODO(bdarnell)
332 }
333 peers := c.peers
334 learners := c.learners
335 if len(cs.Nodes) > 0 || len(cs.Learners) > 0 {
336 if len(peers) > 0 || len(learners) > 0 {
337 // TODO(bdarnell): the peers argument is always nil except in
338 // tests; the argument should be removed and these tests should be
339 // updated to specify their nodes through a snapshot.
340 panic("cannot specify both newRaft(peers, learners) and ConfState.(Nodes, Learners)")
341 }
342 peers = cs.Nodes
343 learners = cs.Learners
344 }
345 r := &raft{
346 id: c.ID,
347 lead: None,
348 isLearner: false,
349 raftLog: raftlog,
350 maxMsgSize: c.MaxSizePerMsg,
351 maxInflight: c.MaxInflightMsgs,
352 maxUncommittedSize: c.MaxUncommittedEntriesSize,
353 prs: make(map[uint64]*Progress),
354 learnerPrs: make(map[uint64]*Progress),
355 electionTimeout: c.ElectionTick,
356 heartbeatTimeout: c.HeartbeatTick,
357 logger: c.Logger,
358 checkQuorum: c.CheckQuorum,
359 preVote: c.PreVote,
360 readOnly: newReadOnly(c.ReadOnlyOption),
361 disableProposalForwarding: c.DisableProposalForwarding,
362 }
363 for _, p := range peers {
364 r.prs[p] = &Progress{Next: 1, ins: newInflights(r.maxInflight)}
365 }
366 for _, p := range learners {
367 if _, ok := r.prs[p]; ok {
368 panic(fmt.Sprintf("node %x is in both learner and peer list", p))
369 }
370 r.learnerPrs[p] = &Progress{Next: 1, ins: newInflights(r.maxInflight), IsLearner: true}
371 if r.id == p {
372 r.isLearner = true
373 }
374 }
375
376 if !isHardStateEqual(hs, emptyState) {
377 r.loadState(hs)
378 }
379 if c.Applied > 0 {
380 raftlog.appliedTo(c.Applied)
381 }
382 r.becomeFollower(r.Term, None)
383
384 var nodesStrs []string
385 for _, n := range r.nodes() {
386 nodesStrs = append(nodesStrs, fmt.Sprintf("%x", n))
387 }
388
389 r.logger.Infof("newRaft %x [peers: [%s], term: %d, commit: %d, applied: %d, lastindex: %d, lastterm: %d]",
390 r.id, strings.Join(nodesStrs, ","), r.Term, r.raftLog.committed, r.raftLog.applied, r.raftLog.lastIndex(), r.raftLog.lastTerm())
391 return r
392}
393
394func (r *raft) hasLeader() bool { return r.lead != None }
395
396func (r *raft) softState() *SoftState { return &SoftState{Lead: r.lead, RaftState: r.state} }
397
398func (r *raft) hardState() pb.HardState {
399 return pb.HardState{
400 Term: r.Term,
401 Vote: r.Vote,
402 Commit: r.raftLog.committed,
403 }
404}
405
406func (r *raft) quorum() int { return len(r.prs)/2 + 1 }
407
408func (r *raft) nodes() []uint64 {
409 nodes := make([]uint64, 0, len(r.prs))
410 for id := range r.prs {
411 nodes = append(nodes, id)
412 }
413 sort.Sort(uint64Slice(nodes))
414 return nodes
415}
416
417func (r *raft) learnerNodes() []uint64 {
418 nodes := make([]uint64, 0, len(r.learnerPrs))
419 for id := range r.learnerPrs {
420 nodes = append(nodes, id)
421 }
422 sort.Sort(uint64Slice(nodes))
423 return nodes
424}
425
426// send persists state to stable storage and then sends to its mailbox.
427func (r *raft) send(m pb.Message) {
428 m.From = r.id
429 if m.Type == pb.MsgVote || m.Type == pb.MsgVoteResp || m.Type == pb.MsgPreVote || m.Type == pb.MsgPreVoteResp {
430 if m.Term == 0 {
431 // All {pre-,}campaign messages need to have the term set when
432 // sending.
433 // - MsgVote: m.Term is the term the node is campaigning for,
434 // non-zero as we increment the term when campaigning.
435 // - MsgVoteResp: m.Term is the new r.Term if the MsgVote was
436 // granted, non-zero for the same reason MsgVote is
437 // - MsgPreVote: m.Term is the term the node will campaign,
438 // non-zero as we use m.Term to indicate the next term we'll be
439 // campaigning for
440 // - MsgPreVoteResp: m.Term is the term received in the original
441 // MsgPreVote if the pre-vote was granted, non-zero for the
442 // same reasons MsgPreVote is
443 panic(fmt.Sprintf("term should be set when sending %s", m.Type))
444 }
445 } else {
446 if m.Term != 0 {
447 panic(fmt.Sprintf("term should not be set when sending %s (was %d)", m.Type, m.Term))
448 }
449 // do not attach term to MsgProp, MsgReadIndex
450 // proposals are a way to forward to the leader and
451 // should be treated as local message.
452 // MsgReadIndex is also forwarded to leader.
453 if m.Type != pb.MsgProp && m.Type != pb.MsgReadIndex {
454 m.Term = r.Term
455 }
456 }
457 r.msgs = append(r.msgs, m)
458}
459
460func (r *raft) getProgress(id uint64) *Progress {
461 if pr, ok := r.prs[id]; ok {
462 return pr
463 }
464
465 return r.learnerPrs[id]
466}
467
468// sendAppend sends an append RPC with new entries (if any) and the
469// current commit index to the given peer.
470func (r *raft) sendAppend(to uint64) {
471 r.maybeSendAppend(to, true)
472}
473
474// maybeSendAppend sends an append RPC with new entries to the given peer,
475// if necessary. Returns true if a message was sent. The sendIfEmpty
476// argument controls whether messages with no entries will be sent
477// ("empty" messages are useful to convey updated Commit indexes, but
478// are undesirable when we're sending multiple messages in a batch).
479func (r *raft) maybeSendAppend(to uint64, sendIfEmpty bool) bool {
480 pr := r.getProgress(to)
481 if pr.IsPaused() {
482 return false
483 }
484 m := pb.Message{}
485 m.To = to
486
487 term, errt := r.raftLog.term(pr.Next - 1)
488 ents, erre := r.raftLog.entries(pr.Next, r.maxMsgSize)
489 if len(ents) == 0 && !sendIfEmpty {
490 return false
491 }
492
493 if errt != nil || erre != nil { // send snapshot if we failed to get term or entries
494 if !pr.RecentActive {
495 r.logger.Debugf("ignore sending snapshot to %x since it is not recently active", to)
496 return false
497 }
498
499 m.Type = pb.MsgSnap
500 snapshot, err := r.raftLog.snapshot()
501 if err != nil {
502 if err == ErrSnapshotTemporarilyUnavailable {
503 r.logger.Debugf("%x failed to send snapshot to %x because snapshot is temporarily unavailable", r.id, to)
504 return false
505 }
506 panic(err) // TODO(bdarnell)
507 }
508 if IsEmptySnap(snapshot) {
509 panic("need non-empty snapshot")
510 }
511 m.Snapshot = snapshot
512 sindex, sterm := snapshot.Metadata.Index, snapshot.Metadata.Term
513 r.logger.Debugf("%x [firstindex: %d, commit: %d] sent snapshot[index: %d, term: %d] to %x [%s]",
514 r.id, r.raftLog.firstIndex(), r.raftLog.committed, sindex, sterm, to, pr)
515 pr.becomeSnapshot(sindex)
516 r.logger.Debugf("%x paused sending replication messages to %x [%s]", r.id, to, pr)
517 } else {
518 m.Type = pb.MsgApp
519 m.Index = pr.Next - 1
520 m.LogTerm = term
521 m.Entries = ents
522 m.Commit = r.raftLog.committed
523 if n := len(m.Entries); n != 0 {
524 switch pr.State {
525 // optimistically increase the next when in ProgressStateReplicate
526 case ProgressStateReplicate:
527 last := m.Entries[n-1].Index
528 pr.optimisticUpdate(last)
529 pr.ins.add(last)
530 case ProgressStateProbe:
531 pr.pause()
532 default:
533 r.logger.Panicf("%x is sending append in unhandled state %s", r.id, pr.State)
534 }
535 }
536 }
537 r.send(m)
538 return true
539}
540
541// sendHeartbeat sends a heartbeat RPC to the given peer.
542func (r *raft) sendHeartbeat(to uint64, ctx []byte) {
543 // Attach the commit as min(to.matched, r.committed).
544 // When the leader sends out heartbeat message,
545 // the receiver(follower) might not be matched with the leader
546 // or it might not have all the committed entries.
547 // The leader MUST NOT forward the follower's commit to
548 // an unmatched index.
549 commit := min(r.getProgress(to).Match, r.raftLog.committed)
550 m := pb.Message{
551 To: to,
552 Type: pb.MsgHeartbeat,
553 Commit: commit,
554 Context: ctx,
555 }
556
557 r.send(m)
558}
559
560func (r *raft) forEachProgress(f func(id uint64, pr *Progress)) {
561 for id, pr := range r.prs {
562 f(id, pr)
563 }
564
565 for id, pr := range r.learnerPrs {
566 f(id, pr)
567 }
568}
569
570// bcastAppend sends RPC, with entries to all peers that are not up-to-date
571// according to the progress recorded in r.prs.
572func (r *raft) bcastAppend() {
573 r.forEachProgress(func(id uint64, _ *Progress) {
574 if id == r.id {
575 return
576 }
577
578 r.sendAppend(id)
579 })
580}
581
582// bcastHeartbeat sends RPC, without entries to all the peers.
583func (r *raft) bcastHeartbeat() {
584 lastCtx := r.readOnly.lastPendingRequestCtx()
585 if len(lastCtx) == 0 {
586 r.bcastHeartbeatWithCtx(nil)
587 } else {
588 r.bcastHeartbeatWithCtx([]byte(lastCtx))
589 }
590}
591
592func (r *raft) bcastHeartbeatWithCtx(ctx []byte) {
593 r.forEachProgress(func(id uint64, _ *Progress) {
594 if id == r.id {
595 return
596 }
597 r.sendHeartbeat(id, ctx)
598 })
599}
600
601// maybeCommit attempts to advance the commit index. Returns true if
602// the commit index changed (in which case the caller should call
603// r.bcastAppend).
604func (r *raft) maybeCommit() bool {
605 // Preserving matchBuf across calls is an optimization
606 // used to avoid allocating a new slice on each call.
607 if cap(r.matchBuf) < len(r.prs) {
608 r.matchBuf = make(uint64Slice, len(r.prs))
609 }
610 mis := r.matchBuf[:len(r.prs)]
611 idx := 0
612 for _, p := range r.prs {
613 mis[idx] = p.Match
614 idx++
615 }
616 sort.Sort(mis)
617 mci := mis[len(mis)-r.quorum()]
618 return r.raftLog.maybeCommit(mci, r.Term)
619}
620
621func (r *raft) reset(term uint64) {
622 if r.Term != term {
623 r.Term = term
624 r.Vote = None
625 }
626 r.lead = None
627
628 r.electionElapsed = 0
629 r.heartbeatElapsed = 0
630 r.resetRandomizedElectionTimeout()
631
632 r.abortLeaderTransfer()
633
634 r.votes = make(map[uint64]bool)
635 r.forEachProgress(func(id uint64, pr *Progress) {
636 *pr = Progress{Next: r.raftLog.lastIndex() + 1, ins: newInflights(r.maxInflight), IsLearner: pr.IsLearner}
637 if id == r.id {
638 pr.Match = r.raftLog.lastIndex()
639 }
640 })
641
642 r.pendingConfIndex = 0
643 r.uncommittedSize = 0
644 r.readOnly = newReadOnly(r.readOnly.option)
645}
646
647func (r *raft) appendEntry(es ...pb.Entry) (accepted bool) {
648 li := r.raftLog.lastIndex()
649 for i := range es {
650 es[i].Term = r.Term
651 es[i].Index = li + 1 + uint64(i)
652 }
653 // Track the size of this uncommitted proposal.
654 if !r.increaseUncommittedSize(es) {
655 r.logger.Debugf(
656 "%x appending new entries to log would exceed uncommitted entry size limit; dropping proposal",
657 r.id,
658 )
659 // Drop the proposal.
660 return false
661 }
662 // use latest "last" index after truncate/append
663 li = r.raftLog.append(es...)
664 r.getProgress(r.id).maybeUpdate(li)
665 // Regardless of maybeCommit's return, our caller will call bcastAppend.
666 r.maybeCommit()
667 return true
668}
669
670// tickElection is run by followers and candidates after r.electionTimeout.
671func (r *raft) tickElection() {
672 r.electionElapsed++
673
674 if r.promotable() && r.pastElectionTimeout() {
675 r.electionElapsed = 0
676 r.Step(pb.Message{From: r.id, Type: pb.MsgHup})
677 }
678}
679
680// tickHeartbeat is run by leaders to send a MsgBeat after r.heartbeatTimeout.
681func (r *raft) tickHeartbeat() {
682 r.heartbeatElapsed++
683 r.electionElapsed++
684
685 if r.electionElapsed >= r.electionTimeout {
686 r.electionElapsed = 0
687 if r.checkQuorum {
688 r.Step(pb.Message{From: r.id, Type: pb.MsgCheckQuorum})
689 }
690 // If current leader cannot transfer leadership in electionTimeout, it becomes leader again.
691 if r.state == StateLeader && r.leadTransferee != None {
692 r.abortLeaderTransfer()
693 }
694 }
695
696 if r.state != StateLeader {
697 return
698 }
699
700 if r.heartbeatElapsed >= r.heartbeatTimeout {
701 r.heartbeatElapsed = 0
702 r.Step(pb.Message{From: r.id, Type: pb.MsgBeat})
703 }
704}
705
706func (r *raft) becomeFollower(term uint64, lead uint64) {
707 r.step = stepFollower
708 r.reset(term)
709 r.tick = r.tickElection
710 r.lead = lead
711 r.state = StateFollower
712 r.logger.Infof("%x became follower at term %d", r.id, r.Term)
713}
714
715func (r *raft) becomeCandidate() {
716 // TODO(xiangli) remove the panic when the raft implementation is stable
717 if r.state == StateLeader {
718 panic("invalid transition [leader -> candidate]")
719 }
720 r.step = stepCandidate
721 r.reset(r.Term + 1)
722 r.tick = r.tickElection
723 r.Vote = r.id
724 r.state = StateCandidate
725 r.logger.Infof("%x became candidate at term %d", r.id, r.Term)
726}
727
728func (r *raft) becomePreCandidate() {
729 // TODO(xiangli) remove the panic when the raft implementation is stable
730 if r.state == StateLeader {
731 panic("invalid transition [leader -> pre-candidate]")
732 }
733 // Becoming a pre-candidate changes our step functions and state,
734 // but doesn't change anything else. In particular it does not increase
735 // r.Term or change r.Vote.
736 r.step = stepCandidate
737 r.votes = make(map[uint64]bool)
738 r.tick = r.tickElection
739 r.lead = None
740 r.state = StatePreCandidate
741 r.logger.Infof("%x became pre-candidate at term %d", r.id, r.Term)
742}
743
744func (r *raft) becomeLeader() {
745 // TODO(xiangli) remove the panic when the raft implementation is stable
746 if r.state == StateFollower {
747 panic("invalid transition [follower -> leader]")
748 }
749 r.step = stepLeader
750 r.reset(r.Term)
751 r.tick = r.tickHeartbeat
752 r.lead = r.id
753 r.state = StateLeader
754 // Followers enter replicate mode when they've been successfully probed
755 // (perhaps after having received a snapshot as a result). The leader is
756 // trivially in this state. Note that r.reset() has initialized this
757 // progress with the last index already.
758 r.prs[r.id].becomeReplicate()
759
760 // Conservatively set the pendingConfIndex to the last index in the
761 // log. There may or may not be a pending config change, but it's
762 // safe to delay any future proposals until we commit all our
763 // pending log entries, and scanning the entire tail of the log
764 // could be expensive.
765 r.pendingConfIndex = r.raftLog.lastIndex()
766
767 emptyEnt := pb.Entry{Data: nil}
768 if !r.appendEntry(emptyEnt) {
769 // This won't happen because we just called reset() above.
770 r.logger.Panic("empty entry was dropped")
771 }
772 // As a special case, don't count the initial empty entry towards the
773 // uncommitted log quota. This is because we want to preserve the
774 // behavior of allowing one entry larger than quota if the current
775 // usage is zero.
776 r.reduceUncommittedSize([]pb.Entry{emptyEnt})
777 r.logger.Infof("%x became leader at term %d", r.id, r.Term)
778}
779
780func (r *raft) campaign(t CampaignType) {
781 var term uint64
782 var voteMsg pb.MessageType
783 if t == campaignPreElection {
784 r.becomePreCandidate()
785 voteMsg = pb.MsgPreVote
786 // PreVote RPCs are sent for the next term before we've incremented r.Term.
787 term = r.Term + 1
788 } else {
789 r.becomeCandidate()
790 voteMsg = pb.MsgVote
791 term = r.Term
792 }
793 if r.quorum() == r.poll(r.id, voteRespMsgType(voteMsg), true) {
794 // We won the election after voting for ourselves (which must mean that
795 // this is a single-node cluster). Advance to the next state.
796 if t == campaignPreElection {
797 r.campaign(campaignElection)
798 } else {
799 r.becomeLeader()
800 }
801 return
802 }
803 for id := range r.prs {
804 if id == r.id {
805 continue
806 }
807 r.logger.Infof("%x [logterm: %d, index: %d] sent %s request to %x at term %d",
808 r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), voteMsg, id, r.Term)
809
810 var ctx []byte
811 if t == campaignTransfer {
812 ctx = []byte(t)
813 }
814 r.send(pb.Message{Term: term, To: id, Type: voteMsg, Index: r.raftLog.lastIndex(), LogTerm: r.raftLog.lastTerm(), Context: ctx})
815 }
816}
817
818func (r *raft) poll(id uint64, t pb.MessageType, v bool) (granted int) {
819 if v {
820 r.logger.Infof("%x received %s from %x at term %d", r.id, t, id, r.Term)
821 } else {
822 r.logger.Infof("%x received %s rejection from %x at term %d", r.id, t, id, r.Term)
823 }
824 if _, ok := r.votes[id]; !ok {
825 r.votes[id] = v
826 }
827 for _, vv := range r.votes {
828 if vv {
829 granted++
830 }
831 }
832 return granted
833}
834
835func (r *raft) Step(m pb.Message) error {
836 // Handle the message term, which may result in our stepping down to a follower.
837 switch {
838 case m.Term == 0:
839 // local message
840 case m.Term > r.Term:
841 if m.Type == pb.MsgVote || m.Type == pb.MsgPreVote {
842 force := bytes.Equal(m.Context, []byte(campaignTransfer))
843 inLease := r.checkQuorum && r.lead != None && r.electionElapsed < r.electionTimeout
844 if !force && inLease {
845 // If a server receives a RequestVote request within the minimum election timeout
846 // of hearing from a current leader, it does not update its term or grant its vote
847 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)",
848 r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term, r.electionTimeout-r.electionElapsed)
849 return nil
850 }
851 }
852 switch {
853 case m.Type == pb.MsgPreVote:
854 // Never change our term in response to a PreVote
855 case m.Type == pb.MsgPreVoteResp && !m.Reject:
856 // We send pre-vote requests with a term in our future. If the
857 // pre-vote is granted, we will increment our term when we get a
858 // quorum. If it is not, the term comes from the node that
859 // rejected our vote so we should become a follower at the new
860 // term.
861 default:
862 r.logger.Infof("%x [term: %d] received a %s message with higher term from %x [term: %d]",
863 r.id, r.Term, m.Type, m.From, m.Term)
864 if m.Type == pb.MsgApp || m.Type == pb.MsgHeartbeat || m.Type == pb.MsgSnap {
865 r.becomeFollower(m.Term, m.From)
866 } else {
867 r.becomeFollower(m.Term, None)
868 }
869 }
870
871 case m.Term < r.Term:
872 if (r.checkQuorum || r.preVote) && (m.Type == pb.MsgHeartbeat || m.Type == pb.MsgApp) {
873 // We have received messages from a leader at a lower term. It is possible
874 // that these messages were simply delayed in the network, but this could
875 // also mean that this node has advanced its term number during a network
876 // partition, and it is now unable to either win an election or to rejoin
877 // the majority on the old term. If checkQuorum is false, this will be
878 // handled by incrementing term numbers in response to MsgVote with a
879 // higher term, but if checkQuorum is true we may not advance the term on
880 // MsgVote and must generate other messages to advance the term. The net
881 // result of these two features is to minimize the disruption caused by
882 // nodes that have been removed from the cluster's configuration: a
883 // removed node will send MsgVotes (or MsgPreVotes) which will be ignored,
884 // but it will not receive MsgApp or MsgHeartbeat, so it will not create
885 // disruptive term increases, by notifying leader of this node's activeness.
886 // The above comments also true for Pre-Vote
887 //
888 // When follower gets isolated, it soon starts an election ending
889 // up with a higher term than leader, although it won't receive enough
890 // votes to win the election. When it regains connectivity, this response
891 // with "pb.MsgAppResp" of higher term would force leader to step down.
892 // However, this disruption is inevitable to free this stuck node with
893 // fresh election. This can be prevented with Pre-Vote phase.
894 r.send(pb.Message{To: m.From, Type: pb.MsgAppResp})
895 } else if m.Type == pb.MsgPreVote {
896 // Before Pre-Vote enable, there may have candidate with higher term,
897 // but less log. After update to Pre-Vote, the cluster may deadlock if
898 // we drop messages with a lower term.
899 r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected %s from %x [logterm: %d, index: %d] at term %d",
900 r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term)
901 r.send(pb.Message{To: m.From, Term: r.Term, Type: pb.MsgPreVoteResp, Reject: true})
902 } else {
903 // ignore other cases
904 r.logger.Infof("%x [term: %d] ignored a %s message with lower term from %x [term: %d]",
905 r.id, r.Term, m.Type, m.From, m.Term)
906 }
907 return nil
908 }
909
910 switch m.Type {
911 case pb.MsgHup:
912 if r.state != StateLeader {
913 ents, err := r.raftLog.slice(r.raftLog.applied+1, r.raftLog.committed+1, noLimit)
914 if err != nil {
915 r.logger.Panicf("unexpected error getting unapplied entries (%v)", err)
916 }
917 if n := numOfPendingConf(ents); n != 0 && r.raftLog.committed > r.raftLog.applied {
918 r.logger.Warningf("%x cannot campaign at term %d since there are still %d pending configuration changes to apply", r.id, r.Term, n)
919 return nil
920 }
921
922 r.logger.Infof("%x is starting a new election at term %d", r.id, r.Term)
923 if r.preVote {
924 r.campaign(campaignPreElection)
925 } else {
926 r.campaign(campaignElection)
927 }
928 } else {
929 r.logger.Debugf("%x ignoring MsgHup because already leader", r.id)
930 }
931
932 case pb.MsgVote, pb.MsgPreVote:
933 if r.isLearner {
934 // TODO: learner may need to vote, in case of node down when confchange.
935 r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] ignored %s from %x [logterm: %d, index: %d] at term %d: learner can not vote",
936 r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term)
937 return nil
938 }
939 // We can vote if this is a repeat of a vote we've already cast...
940 canVote := r.Vote == m.From ||
941 // ...we haven't voted and we don't think there's a leader yet in this term...
942 (r.Vote == None && r.lead == None) ||
943 // ...or this is a PreVote for a future term...
944 (m.Type == pb.MsgPreVote && m.Term > r.Term)
945 // ...and we believe the candidate is up to date.
946 if canVote && r.raftLog.isUpToDate(m.Index, m.LogTerm) {
947 r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] cast %s for %x [logterm: %d, index: %d] at term %d",
948 r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term)
949 // When responding to Msg{Pre,}Vote messages we include the term
950 // from the message, not the local term. To see why consider the
951 // case where a single node was previously partitioned away and
952 // it's local term is now of date. If we include the local term
953 // (recall that for pre-votes we don't update the local term), the
954 // (pre-)campaigning node on the other end will proceed to ignore
955 // the message (it ignores all out of date messages).
956 // The term in the original message and current local term are the
957 // same in the case of regular votes, but different for pre-votes.
958 r.send(pb.Message{To: m.From, Term: m.Term, Type: voteRespMsgType(m.Type)})
959 if m.Type == pb.MsgVote {
960 // Only record real votes.
961 r.electionElapsed = 0
962 r.Vote = m.From
963 }
964 } else {
965 r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected %s from %x [logterm: %d, index: %d] at term %d",
966 r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term)
967 r.send(pb.Message{To: m.From, Term: r.Term, Type: voteRespMsgType(m.Type), Reject: true})
968 }
969
970 default:
971 err := r.step(r, m)
972 if err != nil {
973 return err
974 }
975 }
976 return nil
977}
978
979type stepFunc func(r *raft, m pb.Message) error
980
981func stepLeader(r *raft, m pb.Message) error {
982 // These message types do not require any progress for m.From.
983 switch m.Type {
984 case pb.MsgBeat:
985 r.bcastHeartbeat()
986 return nil
987 case pb.MsgCheckQuorum:
988 if !r.checkQuorumActive() {
989 r.logger.Warningf("%x stepped down to follower since quorum is not active", r.id)
990 r.becomeFollower(r.Term, None)
991 }
992 return nil
993 case pb.MsgProp:
994 if len(m.Entries) == 0 {
995 r.logger.Panicf("%x stepped empty MsgProp", r.id)
996 }
997 if _, ok := r.prs[r.id]; !ok {
998 // If we are not currently a member of the range (i.e. this node
999 // was removed from the configuration while serving as leader),
1000 // drop any new proposals.
1001 return ErrProposalDropped
1002 }
1003 if r.leadTransferee != None {
1004 r.logger.Debugf("%x [term %d] transfer leadership to %x is in progress; dropping proposal", r.id, r.Term, r.leadTransferee)
1005 return ErrProposalDropped
1006 }
1007
1008 for i, e := range m.Entries {
1009 if e.Type == pb.EntryConfChange {
1010 if r.pendingConfIndex > r.raftLog.applied {
1011 r.logger.Infof("propose conf %s ignored since pending unapplied configuration [index %d, applied %d]",
1012 e.String(), r.pendingConfIndex, r.raftLog.applied)
1013 m.Entries[i] = pb.Entry{Type: pb.EntryNormal}
1014 } else {
1015 r.pendingConfIndex = r.raftLog.lastIndex() + uint64(i) + 1
1016 }
1017 }
1018 }
1019
1020 if !r.appendEntry(m.Entries...) {
1021 return ErrProposalDropped
1022 }
1023 r.bcastAppend()
1024 return nil
1025 case pb.MsgReadIndex:
1026 if r.quorum() > 1 {
1027 if r.raftLog.zeroTermOnErrCompacted(r.raftLog.term(r.raftLog.committed)) != r.Term {
1028 // Reject read only request when this leader has not committed any log entry at its term.
1029 return nil
1030 }
1031
1032 // thinking: use an interally defined context instead of the user given context.
1033 // We can express this in terms of the term and index instead of a user-supplied value.
1034 // This would allow multiple reads to piggyback on the same message.
1035 switch r.readOnly.option {
1036 case ReadOnlySafe:
1037 r.readOnly.addRequest(r.raftLog.committed, m)
1038 r.bcastHeartbeatWithCtx(m.Entries[0].Data)
1039 case ReadOnlyLeaseBased:
1040 ri := r.raftLog.committed
1041 if m.From == None || m.From == r.id { // from local member
1042 r.readStates = append(r.readStates, ReadState{Index: r.raftLog.committed, RequestCtx: m.Entries[0].Data})
1043 } else {
1044 r.send(pb.Message{To: m.From, Type: pb.MsgReadIndexResp, Index: ri, Entries: m.Entries})
1045 }
1046 }
1047 } else {
1048 r.readStates = append(r.readStates, ReadState{Index: r.raftLog.committed, RequestCtx: m.Entries[0].Data})
1049 }
1050
1051 return nil
1052 }
1053
1054 // All other message types require a progress for m.From (pr).
1055 pr := r.getProgress(m.From)
1056 if pr == nil {
1057 r.logger.Debugf("%x no progress available for %x", r.id, m.From)
1058 return nil
1059 }
1060 switch m.Type {
1061 case pb.MsgAppResp:
1062 pr.RecentActive = true
1063
1064 if m.Reject {
1065 r.logger.Debugf("%x received msgApp rejection(lastindex: %d) from %x for index %d",
1066 r.id, m.RejectHint, m.From, m.Index)
1067 if pr.maybeDecrTo(m.Index, m.RejectHint) {
1068 r.logger.Debugf("%x decreased progress of %x to [%s]", r.id, m.From, pr)
1069 if pr.State == ProgressStateReplicate {
1070 pr.becomeProbe()
1071 }
1072 r.sendAppend(m.From)
1073 }
1074 } else {
1075 oldPaused := pr.IsPaused()
1076 if pr.maybeUpdate(m.Index) {
1077 switch {
1078 case pr.State == ProgressStateProbe:
1079 pr.becomeReplicate()
1080 case pr.State == ProgressStateSnapshot && pr.needSnapshotAbort():
1081 r.logger.Debugf("%x snapshot aborted, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
1082 // Transition back to replicating state via probing state
1083 // (which takes the snapshot into account). If we didn't
1084 // move to replicating state, that would only happen with
1085 // the next round of appends (but there may not be a next
1086 // round for a while, exposing an inconsistent RaftStatus).
1087 pr.becomeProbe()
1088 pr.becomeReplicate()
1089 case pr.State == ProgressStateReplicate:
1090 pr.ins.freeTo(m.Index)
1091 }
1092
1093 if r.maybeCommit() {
1094 r.bcastAppend()
1095 } else if oldPaused {
1096 // If we were paused before, this node may be missing the
1097 // latest commit index, so send it.
1098 r.sendAppend(m.From)
1099 }
1100 // We've updated flow control information above, which may
1101 // allow us to send multiple (size-limited) in-flight messages
1102 // at once (such as when transitioning from probe to
1103 // replicate, or when freeTo() covers multiple messages). If
1104 // we have more entries to send, send as many messages as we
1105 // can (without sending empty messages for the commit index)
1106 for r.maybeSendAppend(m.From, false) {
1107 }
1108 // Transfer leadership is in progress.
1109 if m.From == r.leadTransferee && pr.Match == r.raftLog.lastIndex() {
1110 r.logger.Infof("%x sent MsgTimeoutNow to %x after received MsgAppResp", r.id, m.From)
1111 r.sendTimeoutNow(m.From)
1112 }
1113 }
1114 }
1115 case pb.MsgHeartbeatResp:
1116 pr.RecentActive = true
1117 pr.resume()
1118
1119 // free one slot for the full inflights window to allow progress.
1120 if pr.State == ProgressStateReplicate && pr.ins.full() {
1121 pr.ins.freeFirstOne()
1122 }
1123 if pr.Match < r.raftLog.lastIndex() {
1124 r.sendAppend(m.From)
1125 }
1126
1127 if r.readOnly.option != ReadOnlySafe || len(m.Context) == 0 {
1128 return nil
1129 }
1130
1131 ackCount := r.readOnly.recvAck(m)
1132 if ackCount < r.quorum() {
1133 return nil
1134 }
1135
1136 rss := r.readOnly.advance(m)
1137 for _, rs := range rss {
1138 req := rs.req
1139 if req.From == None || req.From == r.id { // from local member
1140 r.readStates = append(r.readStates, ReadState{Index: rs.index, RequestCtx: req.Entries[0].Data})
1141 } else {
1142 r.send(pb.Message{To: req.From, Type: pb.MsgReadIndexResp, Index: rs.index, Entries: req.Entries})
1143 }
1144 }
1145 case pb.MsgSnapStatus:
1146 if pr.State != ProgressStateSnapshot {
1147 return nil
1148 }
1149 if !m.Reject {
1150 pr.becomeProbe()
1151 r.logger.Debugf("%x snapshot succeeded, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
1152 } else {
1153 pr.snapshotFailure()
1154 pr.becomeProbe()
1155 r.logger.Debugf("%x snapshot failed, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
1156 }
1157 // If snapshot finish, wait for the msgAppResp from the remote node before sending
1158 // out the next msgApp.
1159 // If snapshot failure, wait for a heartbeat interval before next try
1160 pr.pause()
1161 case pb.MsgUnreachable:
1162 // During optimistic replication, if the remote becomes unreachable,
1163 // there is huge probability that a MsgApp is lost.
1164 if pr.State == ProgressStateReplicate {
1165 pr.becomeProbe()
1166 }
1167 r.logger.Debugf("%x failed to send message to %x because it is unreachable [%s]", r.id, m.From, pr)
1168 case pb.MsgTransferLeader:
1169 if pr.IsLearner {
1170 r.logger.Debugf("%x is learner. Ignored transferring leadership", r.id)
1171 return nil
1172 }
1173 leadTransferee := m.From
1174 lastLeadTransferee := r.leadTransferee
1175 if lastLeadTransferee != None {
1176 if lastLeadTransferee == leadTransferee {
1177 r.logger.Infof("%x [term %d] transfer leadership to %x is in progress, ignores request to same node %x",
1178 r.id, r.Term, leadTransferee, leadTransferee)
1179 return nil
1180 }
1181 r.abortLeaderTransfer()
1182 r.logger.Infof("%x [term %d] abort previous transferring leadership to %x", r.id, r.Term, lastLeadTransferee)
1183 }
1184 if leadTransferee == r.id {
1185 r.logger.Debugf("%x is already leader. Ignored transferring leadership to self", r.id)
1186 return nil
1187 }
1188 // Transfer leadership to third party.
1189 r.logger.Infof("%x [term %d] starts to transfer leadership to %x", r.id, r.Term, leadTransferee)
1190 // Transfer leadership should be finished in one electionTimeout, so reset r.electionElapsed.
1191 r.electionElapsed = 0
1192 r.leadTransferee = leadTransferee
1193 if pr.Match == r.raftLog.lastIndex() {
1194 r.sendTimeoutNow(leadTransferee)
1195 r.logger.Infof("%x sends MsgTimeoutNow to %x immediately as %x already has up-to-date log", r.id, leadTransferee, leadTransferee)
1196 } else {
1197 r.sendAppend(leadTransferee)
1198 }
1199 }
1200 return nil
1201}
1202
1203// stepCandidate is shared by StateCandidate and StatePreCandidate; the difference is
1204// whether they respond to MsgVoteResp or MsgPreVoteResp.
1205func stepCandidate(r *raft, m pb.Message) error {
1206 // Only handle vote responses corresponding to our candidacy (while in
1207 // StateCandidate, we may get stale MsgPreVoteResp messages in this term from
1208 // our pre-candidate state).
1209 var myVoteRespType pb.MessageType
1210 if r.state == StatePreCandidate {
1211 myVoteRespType = pb.MsgPreVoteResp
1212 } else {
1213 myVoteRespType = pb.MsgVoteResp
1214 }
1215 switch m.Type {
1216 case pb.MsgProp:
1217 r.logger.Infof("%x no leader at term %d; dropping proposal", r.id, r.Term)
1218 return ErrProposalDropped
1219 case pb.MsgApp:
1220 r.becomeFollower(m.Term, m.From) // always m.Term == r.Term
1221 r.handleAppendEntries(m)
1222 case pb.MsgHeartbeat:
1223 r.becomeFollower(m.Term, m.From) // always m.Term == r.Term
1224 r.handleHeartbeat(m)
1225 case pb.MsgSnap:
1226 r.becomeFollower(m.Term, m.From) // always m.Term == r.Term
1227 r.handleSnapshot(m)
1228 case myVoteRespType:
1229 gr := r.poll(m.From, m.Type, !m.Reject)
1230 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)
1231 switch r.quorum() {
1232 case gr:
1233 if r.state == StatePreCandidate {
1234 r.campaign(campaignElection)
1235 } else {
1236 r.becomeLeader()
1237 r.bcastAppend()
1238 }
1239 case len(r.votes) - gr:
1240 // pb.MsgPreVoteResp contains future term of pre-candidate
1241 // m.Term > r.Term; reuse r.Term
1242 r.becomeFollower(r.Term, None)
1243 }
1244 case pb.MsgTimeoutNow:
1245 r.logger.Debugf("%x [term %d state %v] ignored MsgTimeoutNow from %x", r.id, r.Term, r.state, m.From)
1246 }
1247 return nil
1248}
1249
1250func stepFollower(r *raft, m pb.Message) error {
1251 switch m.Type {
1252 case pb.MsgProp:
1253 if r.lead == None {
1254 r.logger.Infof("%x no leader at term %d; dropping proposal", r.id, r.Term)
1255 return ErrProposalDropped
1256 } else if r.disableProposalForwarding {
1257 r.logger.Infof("%x not forwarding to leader %x at term %d; dropping proposal", r.id, r.lead, r.Term)
1258 return ErrProposalDropped
1259 }
1260 m.To = r.lead
1261 r.send(m)
1262 case pb.MsgApp:
1263 r.electionElapsed = 0
1264 r.lead = m.From
1265 r.handleAppendEntries(m)
1266 case pb.MsgHeartbeat:
1267 r.electionElapsed = 0
1268 r.lead = m.From
1269 r.handleHeartbeat(m)
1270 case pb.MsgSnap:
1271 r.electionElapsed = 0
1272 r.lead = m.From
1273 r.handleSnapshot(m)
1274 case pb.MsgTransferLeader:
1275 if r.lead == None {
1276 r.logger.Infof("%x no leader at term %d; dropping leader transfer msg", r.id, r.Term)
1277 return nil
1278 }
1279 m.To = r.lead
1280 r.send(m)
1281 case pb.MsgTimeoutNow:
1282 if r.promotable() {
1283 r.logger.Infof("%x [term %d] received MsgTimeoutNow from %x and starts an election to get leadership.", r.id, r.Term, m.From)
1284 // Leadership transfers never use pre-vote even if r.preVote is true; we
1285 // know we are not recovering from a partition so there is no need for the
1286 // extra round trip.
1287 r.campaign(campaignTransfer)
1288 } else {
1289 r.logger.Infof("%x received MsgTimeoutNow from %x but is not promotable", r.id, m.From)
1290 }
1291 case pb.MsgReadIndex:
1292 if r.lead == None {
1293 r.logger.Infof("%x no leader at term %d; dropping index reading msg", r.id, r.Term)
1294 return nil
1295 }
1296 m.To = r.lead
1297 r.send(m)
1298 case pb.MsgReadIndexResp:
1299 if len(m.Entries) != 1 {
1300 r.logger.Errorf("%x invalid format of MsgReadIndexResp from %x, entries count: %d", r.id, m.From, len(m.Entries))
1301 return nil
1302 }
1303 r.readStates = append(r.readStates, ReadState{Index: m.Index, RequestCtx: m.Entries[0].Data})
1304 }
1305 return nil
1306}
1307
1308func (r *raft) handleAppendEntries(m pb.Message) {
1309 if m.Index < r.raftLog.committed {
1310 r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed})
1311 return
1312 }
1313
1314 if mlastIndex, ok := r.raftLog.maybeAppend(m.Index, m.LogTerm, m.Commit, m.Entries...); ok {
1315 r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: mlastIndex})
1316 } else {
1317 r.logger.Debugf("%x [logterm: %d, index: %d] rejected msgApp [logterm: %d, index: %d] from %x",
1318 r.id, r.raftLog.zeroTermOnErrCompacted(r.raftLog.term(m.Index)), m.Index, m.LogTerm, m.Index, m.From)
1319 r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: m.Index, Reject: true, RejectHint: r.raftLog.lastIndex()})
1320 }
1321}
1322
1323func (r *raft) handleHeartbeat(m pb.Message) {
1324 r.raftLog.commitTo(m.Commit)
1325 r.send(pb.Message{To: m.From, Type: pb.MsgHeartbeatResp, Context: m.Context})
1326}
1327
1328func (r *raft) handleSnapshot(m pb.Message) {
1329 sindex, sterm := m.Snapshot.Metadata.Index, m.Snapshot.Metadata.Term
1330 if r.restore(m.Snapshot) {
1331 r.logger.Infof("%x [commit: %d] restored snapshot [index: %d, term: %d]",
1332 r.id, r.raftLog.committed, sindex, sterm)
1333 r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.lastIndex()})
1334 } else {
1335 r.logger.Infof("%x [commit: %d] ignored snapshot [index: %d, term: %d]",
1336 r.id, r.raftLog.committed, sindex, sterm)
1337 r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed})
1338 }
1339}
1340
1341// restore recovers the state machine from a snapshot. It restores the log and the
1342// configuration of state machine.
1343func (r *raft) restore(s pb.Snapshot) bool {
1344 if s.Metadata.Index <= r.raftLog.committed {
1345 return false
1346 }
1347 if r.raftLog.matchTerm(s.Metadata.Index, s.Metadata.Term) {
1348 r.logger.Infof("%x [commit: %d, lastindex: %d, lastterm: %d] fast-forwarded commit to snapshot [index: %d, term: %d]",
1349 r.id, r.raftLog.committed, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
1350 r.raftLog.commitTo(s.Metadata.Index)
1351 return false
1352 }
1353
1354 // The normal peer can't become learner.
1355 if !r.isLearner {
1356 for _, id := range s.Metadata.ConfState.Learners {
1357 if id == r.id {
1358 r.logger.Errorf("%x can't become learner when restores snapshot [index: %d, term: %d]", r.id, s.Metadata.Index, s.Metadata.Term)
1359 return false
1360 }
1361 }
1362 }
1363
1364 r.logger.Infof("%x [commit: %d, lastindex: %d, lastterm: %d] starts to restore snapshot [index: %d, term: %d]",
1365 r.id, r.raftLog.committed, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
1366
1367 r.raftLog.restore(s)
1368 r.prs = make(map[uint64]*Progress)
1369 r.learnerPrs = make(map[uint64]*Progress)
1370 r.restoreNode(s.Metadata.ConfState.Nodes, false)
1371 r.restoreNode(s.Metadata.ConfState.Learners, true)
1372 return true
1373}
1374
1375func (r *raft) restoreNode(nodes []uint64, isLearner bool) {
1376 for _, n := range nodes {
1377 match, next := uint64(0), r.raftLog.lastIndex()+1
1378 if n == r.id {
1379 match = next - 1
1380 r.isLearner = isLearner
1381 }
1382 r.setProgress(n, match, next, isLearner)
1383 r.logger.Infof("%x restored progress of %x [%s]", r.id, n, r.getProgress(n))
1384 }
1385}
1386
1387// promotable indicates whether state machine can be promoted to leader,
1388// which is true when its own id is in progress list.
1389func (r *raft) promotable() bool {
1390 _, ok := r.prs[r.id]
1391 return ok
1392}
1393
1394func (r *raft) addNode(id uint64) {
1395 r.addNodeOrLearnerNode(id, false)
1396}
1397
1398func (r *raft) addLearner(id uint64) {
1399 r.addNodeOrLearnerNode(id, true)
1400}
1401
1402func (r *raft) addNodeOrLearnerNode(id uint64, isLearner bool) {
1403 pr := r.getProgress(id)
1404 if pr == nil {
1405 r.setProgress(id, 0, r.raftLog.lastIndex()+1, isLearner)
1406 } else {
1407 if isLearner && !pr.IsLearner {
1408 // can only change Learner to Voter
1409 r.logger.Infof("%x ignored addLearner: do not support changing %x from raft peer to learner.", r.id, id)
1410 return
1411 }
1412
1413 if isLearner == pr.IsLearner {
1414 // Ignore any redundant addNode calls (which can happen because the
1415 // initial bootstrapping entries are applied twice).
1416 return
1417 }
1418
1419 // change Learner to Voter, use origin Learner progress
1420 delete(r.learnerPrs, id)
1421 pr.IsLearner = false
1422 r.prs[id] = pr
1423 }
1424
1425 if r.id == id {
1426 r.isLearner = isLearner
1427 }
1428
1429 // When a node is first added, we should mark it as recently active.
1430 // Otherwise, CheckQuorum may cause us to step down if it is invoked
1431 // before the added node has a chance to communicate with us.
1432 pr = r.getProgress(id)
1433 pr.RecentActive = true
1434}
1435
1436func (r *raft) removeNode(id uint64) {
1437 r.delProgress(id)
1438
1439 // do not try to commit or abort transferring if there is no nodes in the cluster.
1440 if len(r.prs) == 0 && len(r.learnerPrs) == 0 {
1441 return
1442 }
1443
1444 // The quorum size is now smaller, so see if any pending entries can
1445 // be committed.
1446 if r.maybeCommit() {
1447 r.bcastAppend()
1448 }
1449 // If the removed node is the leadTransferee, then abort the leadership transferring.
1450 if r.state == StateLeader && r.leadTransferee == id {
1451 r.abortLeaderTransfer()
1452 }
1453}
1454
1455func (r *raft) setProgress(id, match, next uint64, isLearner bool) {
1456 if !isLearner {
1457 delete(r.learnerPrs, id)
1458 r.prs[id] = &Progress{Next: next, Match: match, ins: newInflights(r.maxInflight)}
1459 return
1460 }
1461
1462 if _, ok := r.prs[id]; ok {
1463 panic(fmt.Sprintf("%x unexpected changing from voter to learner for %x", r.id, id))
1464 }
1465 r.learnerPrs[id] = &Progress{Next: next, Match: match, ins: newInflights(r.maxInflight), IsLearner: true}
1466}
1467
1468func (r *raft) delProgress(id uint64) {
1469 delete(r.prs, id)
1470 delete(r.learnerPrs, id)
1471}
1472
1473func (r *raft) loadState(state pb.HardState) {
1474 if state.Commit < r.raftLog.committed || state.Commit > r.raftLog.lastIndex() {
1475 r.logger.Panicf("%x state.commit %d is out of range [%d, %d]", r.id, state.Commit, r.raftLog.committed, r.raftLog.lastIndex())
1476 }
1477 r.raftLog.committed = state.Commit
1478 r.Term = state.Term
1479 r.Vote = state.Vote
1480}
1481
1482// pastElectionTimeout returns true iff r.electionElapsed is greater
1483// than or equal to the randomized election timeout in
1484// [electiontimeout, 2 * electiontimeout - 1].
1485func (r *raft) pastElectionTimeout() bool {
1486 return r.electionElapsed >= r.randomizedElectionTimeout
1487}
1488
1489func (r *raft) resetRandomizedElectionTimeout() {
1490 r.randomizedElectionTimeout = r.electionTimeout + globalRand.Intn(r.electionTimeout)
1491}
1492
1493// checkQuorumActive returns true if the quorum is active from
1494// the view of the local raft state machine. Otherwise, it returns
1495// false.
1496// checkQuorumActive also resets all RecentActive to false.
1497func (r *raft) checkQuorumActive() bool {
1498 var act int
1499
1500 r.forEachProgress(func(id uint64, pr *Progress) {
1501 if id == r.id { // self is always active
1502 act++
1503 return
1504 }
1505
1506 if pr.RecentActive && !pr.IsLearner {
1507 act++
1508 }
1509
1510 pr.RecentActive = false
1511 })
1512
1513 return act >= r.quorum()
1514}
1515
1516func (r *raft) sendTimeoutNow(to uint64) {
1517 r.send(pb.Message{To: to, Type: pb.MsgTimeoutNow})
1518}
1519
1520func (r *raft) abortLeaderTransfer() {
1521 r.leadTransferee = None
1522}
1523
1524// increaseUncommittedSize computes the size of the proposed entries and
1525// determines whether they would push leader over its maxUncommittedSize limit.
1526// If the new entries would exceed the limit, the method returns false. If not,
1527// the increase in uncommitted entry size is recorded and the method returns
1528// true.
1529func (r *raft) increaseUncommittedSize(ents []pb.Entry) bool {
1530 var s uint64
1531 for _, e := range ents {
1532 s += uint64(PayloadSize(e))
1533 }
1534
1535 if r.uncommittedSize > 0 && r.uncommittedSize+s > r.maxUncommittedSize {
1536 // If the uncommitted tail of the Raft log is empty, allow any size
1537 // proposal. Otherwise, limit the size of the uncommitted tail of the
1538 // log and drop any proposal that would push the size over the limit.
1539 return false
1540 }
1541 r.uncommittedSize += s
1542 return true
1543}
1544
1545// reduceUncommittedSize accounts for the newly committed entries by decreasing
1546// the uncommitted entry size limit.
1547func (r *raft) reduceUncommittedSize(ents []pb.Entry) {
1548 if r.uncommittedSize == 0 {
1549 // Fast-path for followers, who do not track or enforce the limit.
1550 return
1551 }
1552
1553 var s uint64
1554 for _, e := range ents {
1555 s += uint64(PayloadSize(e))
1556 }
1557 if s > r.uncommittedSize {
1558 // uncommittedSize may underestimate the size of the uncommitted Raft
1559 // log tail but will never overestimate it. Saturate at 0 instead of
1560 // allowing overflow.
1561 r.uncommittedSize = 0
1562 } else {
1563 r.uncommittedSize -= s
1564 }
1565}
1566
1567func numOfPendingConf(ents []pb.Entry) int {
1568 n := 0
1569 for i := range ents {
1570 if ents[i].Type == pb.EntryConfChange {
1571 n++
1572 }
1573 }
1574 return n
1575}