raft.go 40 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080108110821083108410851086108710881089109010911092109310941095109610971098109911001101110211031104110511061107110811091110111111121113111411151116111711181119112011211122112311241125112611271128112911301131113211331134113511361137113811391140114111421143114411451146114711481149115011511152115311541155115611571158115911601161116211631164116511661167116811691170117111721173117411751176117711781179118011811182118311841185118611871188118911901191119211931194119511961197119811991200120112021203120412051206120712081209121012111212121312141215121612171218121912201221122212231224122512261227122812291230123112321233123412351236123712381239124012411242124312441245124612471248124912501251125212531254125512561257125812591260126112621263126412651266126712681269127012711272
  1. // 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. package raft
  15. import (
  16. "bytes"
  17. "errors"
  18. "fmt"
  19. "math"
  20. "math/rand"
  21. "sort"
  22. "strings"
  23. "sync"
  24. "time"
  25. pb "github.com/coreos/etcd/raft/raftpb"
  26. )
  27. // None is a placeholder node ID used when there is no leader.
  28. const None uint64 = 0
  29. const noLimit = math.MaxUint64
  30. // Possible values for StateType.
  31. const (
  32. StateFollower StateType = iota
  33. StateCandidate
  34. StateLeader
  35. StatePreCandidate
  36. numStates
  37. )
  38. type ReadOnlyOption int
  39. const (
  40. // ReadOnlySafe guarantees the linearizability of the read only request by
  41. // communicating with the quorum. It is the default and suggested option.
  42. ReadOnlySafe ReadOnlyOption = iota
  43. // ReadOnlyLeaseBased ensures linearizability of the read only request by
  44. // relying on the leader lease. It can be affected by clock drift.
  45. // If the clock drift is unbounded, leader might keep the lease longer than it
  46. // should (clock can move backward/pause without any bound). ReadIndex is not safe
  47. // in that case.
  48. ReadOnlyLeaseBased
  49. )
  50. // Possible values for CampaignType
  51. const (
  52. // campaignPreElection represents the first phase of a normal election when
  53. // Config.PreVote is true.
  54. campaignPreElection CampaignType = "CampaignPreElection"
  55. // campaignElection represents a normal (time-based) election (the second phase
  56. // of the election when Config.PreVote is true).
  57. campaignElection CampaignType = "CampaignElection"
  58. // campaignTransfer represents the type of leader transfer
  59. campaignTransfer CampaignType = "CampaignTransfer"
  60. )
  61. // lockedRand is a small wrapper around rand.Rand to provide
  62. // synchronization. Only the methods needed by the code are exposed
  63. // (e.g. Intn).
  64. type lockedRand struct {
  65. mu sync.Mutex
  66. rand *rand.Rand
  67. }
  68. func (r *lockedRand) Intn(n int) int {
  69. r.mu.Lock()
  70. v := r.rand.Intn(n)
  71. r.mu.Unlock()
  72. return v
  73. }
  74. var globalRand = &lockedRand{
  75. rand: rand.New(rand.NewSource(time.Now().UnixNano())),
  76. }
  77. // CampaignType represents the type of campaigning
  78. // the reason we use the type of string instead of uint64
  79. // is because it's simpler to compare and fill in raft entries
  80. type CampaignType string
  81. // StateType represents the role of a node in a cluster.
  82. type StateType uint64
  83. var stmap = [...]string{
  84. "StateFollower",
  85. "StateCandidate",
  86. "StateLeader",
  87. "StatePreCandidate",
  88. }
  89. func (st StateType) String() string {
  90. return stmap[uint64(st)]
  91. }
  92. // Config contains the parameters to start a raft.
  93. type Config struct {
  94. // ID is the identity of the local raft. ID cannot be 0.
  95. ID uint64
  96. // peers contains the IDs of all nodes (including self) in the raft cluster. It
  97. // should only be set when starting a new raft cluster. Restarting raft from
  98. // previous configuration will panic if peers is set. peer is private and only
  99. // used for testing right now.
  100. peers []uint64
  101. // ElectionTick is the number of Node.Tick invocations that must pass between
  102. // elections. That is, if a follower does not receive any message from the
  103. // leader of current term before ElectionTick has elapsed, it will become
  104. // candidate and start an election. ElectionTick must be greater than
  105. // HeartbeatTick. We suggest ElectionTick = 10 * HeartbeatTick to avoid
  106. // unnecessary leader switching.
  107. ElectionTick int
  108. // HeartbeatTick is the number of Node.Tick invocations that must pass between
  109. // heartbeats. That is, a leader sends heartbeat messages to maintain its
  110. // leadership every HeartbeatTick ticks.
  111. HeartbeatTick int
  112. // Storage is the storage for raft. raft generates entries and states to be
  113. // stored in storage. raft reads the persisted entries and states out of
  114. // Storage when it needs. raft reads out the previous state and configuration
  115. // out of storage when restarting.
  116. Storage Storage
  117. // Applied is the last applied index. It should only be set when restarting
  118. // raft. raft will not return entries to the application smaller or equal to
  119. // Applied. If Applied is unset when restarting, raft might return previous
  120. // applied entries. This is a very application dependent configuration.
  121. Applied uint64
  122. // MaxSizePerMsg limits the max size of each append message. Smaller value
  123. // lowers the raft recovery cost(initial probing and message lost during normal
  124. // operation). On the other side, it might affect the throughput during normal
  125. // replication. Note: math.MaxUint64 for unlimited, 0 for at most one entry per
  126. // message.
  127. MaxSizePerMsg uint64
  128. // MaxInflightMsgs limits the max number of in-flight append messages during
  129. // optimistic replication phase. The application transportation layer usually
  130. // has its own sending buffer over TCP/UDP. Setting MaxInflightMsgs to avoid
  131. // overflowing that sending buffer. TODO (xiangli): feedback to application to
  132. // limit the proposal rate?
  133. MaxInflightMsgs int
  134. // CheckQuorum specifies if the leader should check quorum activity. Leader
  135. // steps down when quorum is not active for an electionTimeout.
  136. CheckQuorum bool
  137. // PreVote enables the Pre-Vote algorithm described in raft thesis section
  138. // 9.6. This prevents disruption when a node that has been partitioned away
  139. // rejoins the cluster.
  140. PreVote bool
  141. // ReadOnlyOption specifies how the read only request is processed.
  142. //
  143. // ReadOnlySafe guarantees the linearizability of the read only request by
  144. // communicating with the quorum. It is the default and suggested option.
  145. //
  146. // ReadOnlyLeaseBased ensures linearizability of the read only request by
  147. // relying on the leader lease. It can be affected by clock drift.
  148. // If the clock drift is unbounded, leader might keep the lease longer than it
  149. // should (clock can move backward/pause without any bound). ReadIndex is not safe
  150. // in that case.
  151. ReadOnlyOption ReadOnlyOption
  152. // Logger is the logger used for raft log. For multinode which can host
  153. // multiple raft group, each raft group can have its own logger
  154. Logger Logger
  155. // DisableProposalForwarding set to true means that followers will drop
  156. // proposals, rather than forwarding them to the leader. One use case for
  157. // this feature would be in a situation where the Raft leader is used to
  158. // compute the data of a proposal, for example, adding a timestamp from a
  159. // hybrid logical clock to data in a monotonically increasing way. Forwarding
  160. // should be disabled to prevent a follower with an innaccurate hybrid
  161. // logical clock from assigning the timestamp and then forwarding the data
  162. // to the leader.
  163. DisableProposalForwarding bool
  164. }
  165. func (c *Config) validate() error {
  166. if c.ID == None {
  167. return errors.New("cannot use none as id")
  168. }
  169. if c.HeartbeatTick <= 0 {
  170. return errors.New("heartbeat tick must be greater than 0")
  171. }
  172. if c.ElectionTick <= c.HeartbeatTick {
  173. return errors.New("election tick must be greater than heartbeat tick")
  174. }
  175. if c.Storage == nil {
  176. return errors.New("storage cannot be nil")
  177. }
  178. if c.MaxInflightMsgs <= 0 {
  179. return errors.New("max inflight messages must be greater than 0")
  180. }
  181. if c.Logger == nil {
  182. c.Logger = raftLogger
  183. }
  184. return nil
  185. }
  186. type raft struct {
  187. id uint64
  188. Term uint64
  189. Vote uint64
  190. readStates []ReadState
  191. // the log
  192. raftLog *raftLog
  193. maxInflight int
  194. maxMsgSize uint64
  195. prs map[uint64]*Progress
  196. state StateType
  197. votes map[uint64]bool
  198. msgs []pb.Message
  199. // the leader id
  200. lead uint64
  201. // leadTransferee is id of the leader transfer target when its value is not zero.
  202. // Follow the procedure defined in raft thesis 3.10.
  203. leadTransferee uint64
  204. // New configuration is ignored if there exists unapplied configuration.
  205. pendingConf bool
  206. readOnly *readOnly
  207. // number of ticks since it reached last electionTimeout when it is leader
  208. // or candidate.
  209. // number of ticks since it reached last electionTimeout or received a
  210. // valid message from current leader when it is a follower.
  211. electionElapsed int
  212. // number of ticks since it reached last heartbeatTimeout.
  213. // only leader keeps heartbeatElapsed.
  214. heartbeatElapsed int
  215. checkQuorum bool
  216. preVote bool
  217. heartbeatTimeout int
  218. electionTimeout int
  219. // randomizedElectionTimeout is a random number between
  220. // [electiontimeout, 2 * electiontimeout - 1]. It gets reset
  221. // when raft changes its state to follower or candidate.
  222. randomizedElectionTimeout int
  223. disableProposalForwarding bool
  224. tick func()
  225. step stepFunc
  226. logger Logger
  227. }
  228. func newRaft(c *Config) *raft {
  229. if err := c.validate(); err != nil {
  230. panic(err.Error())
  231. }
  232. raftlog := newLog(c.Storage, c.Logger)
  233. hs, cs, err := c.Storage.InitialState()
  234. if err != nil {
  235. panic(err) // TODO(bdarnell)
  236. }
  237. peers := c.peers
  238. if len(cs.Nodes) > 0 {
  239. if len(peers) > 0 {
  240. // TODO(bdarnell): the peers argument is always nil except in
  241. // tests; the argument should be removed and these tests should be
  242. // updated to specify their nodes through a snapshot.
  243. panic("cannot specify both newRaft(peers) and ConfState.Nodes)")
  244. }
  245. peers = cs.Nodes
  246. }
  247. r := &raft{
  248. id: c.ID,
  249. lead: None,
  250. raftLog: raftlog,
  251. maxMsgSize: c.MaxSizePerMsg,
  252. maxInflight: c.MaxInflightMsgs,
  253. prs: make(map[uint64]*Progress),
  254. electionTimeout: c.ElectionTick,
  255. heartbeatTimeout: c.HeartbeatTick,
  256. logger: c.Logger,
  257. checkQuorum: c.CheckQuorum,
  258. preVote: c.PreVote,
  259. readOnly: newReadOnly(c.ReadOnlyOption),
  260. disableProposalForwarding: c.DisableProposalForwarding,
  261. }
  262. for _, p := range peers {
  263. r.prs[p] = &Progress{Next: 1, ins: newInflights(r.maxInflight)}
  264. }
  265. if !isHardStateEqual(hs, emptyState) {
  266. r.loadState(hs)
  267. }
  268. if c.Applied > 0 {
  269. raftlog.appliedTo(c.Applied)
  270. }
  271. r.becomeFollower(r.Term, None)
  272. var nodesStrs []string
  273. for _, n := range r.nodes() {
  274. nodesStrs = append(nodesStrs, fmt.Sprintf("%x", n))
  275. }
  276. r.logger.Infof("newRaft %x [peers: [%s], term: %d, commit: %d, applied: %d, lastindex: %d, lastterm: %d]",
  277. r.id, strings.Join(nodesStrs, ","), r.Term, r.raftLog.committed, r.raftLog.applied, r.raftLog.lastIndex(), r.raftLog.lastTerm())
  278. return r
  279. }
  280. func (r *raft) hasLeader() bool { return r.lead != None }
  281. func (r *raft) softState() *SoftState { return &SoftState{Lead: r.lead, RaftState: r.state} }
  282. func (r *raft) hardState() pb.HardState {
  283. return pb.HardState{
  284. Term: r.Term,
  285. Vote: r.Vote,
  286. Commit: r.raftLog.committed,
  287. }
  288. }
  289. func (r *raft) quorum() int { return len(r.prs)/2 + 1 }
  290. func (r *raft) nodes() []uint64 {
  291. nodes := make([]uint64, 0, len(r.prs))
  292. for id := range r.prs {
  293. nodes = append(nodes, id)
  294. }
  295. sort.Sort(uint64Slice(nodes))
  296. return nodes
  297. }
  298. // send persists state to stable storage and then sends to its mailbox.
  299. func (r *raft) send(m pb.Message) {
  300. m.From = r.id
  301. if m.Type == pb.MsgVote || m.Type == pb.MsgPreVote {
  302. if m.Term == 0 {
  303. // PreVote RPCs are sent at a term other than our actual term, so the code
  304. // that sends these messages is responsible for setting the term.
  305. panic(fmt.Sprintf("term should be set when sending %s", m.Type))
  306. }
  307. } else {
  308. if m.Term != 0 {
  309. panic(fmt.Sprintf("term should not be set when sending %s (was %d)", m.Type, m.Term))
  310. }
  311. // do not attach term to MsgProp, MsgReadIndex
  312. // proposals are a way to forward to the leader and
  313. // should be treated as local message.
  314. // MsgReadIndex is also forwarded to leader.
  315. if m.Type != pb.MsgProp && m.Type != pb.MsgReadIndex {
  316. m.Term = r.Term
  317. }
  318. }
  319. r.msgs = append(r.msgs, m)
  320. }
  321. // sendAppend sends RPC, with entries to the given peer.
  322. func (r *raft) sendAppend(to uint64) {
  323. pr := r.prs[to]
  324. if pr.IsPaused() {
  325. return
  326. }
  327. m := pb.Message{}
  328. m.To = to
  329. term, errt := r.raftLog.term(pr.Next - 1)
  330. ents, erre := r.raftLog.entries(pr.Next, r.maxMsgSize)
  331. if errt != nil || erre != nil { // send snapshot if we failed to get term or entries
  332. if !pr.RecentActive {
  333. r.logger.Debugf("ignore sending snapshot to %x since it is not recently active", to)
  334. return
  335. }
  336. m.Type = pb.MsgSnap
  337. snapshot, err := r.raftLog.snapshot()
  338. if err != nil {
  339. if err == ErrSnapshotTemporarilyUnavailable {
  340. r.logger.Debugf("%x failed to send snapshot to %x because snapshot is temporarily unavailable", r.id, to)
  341. return
  342. }
  343. panic(err) // TODO(bdarnell)
  344. }
  345. if IsEmptySnap(snapshot) {
  346. panic("need non-empty snapshot")
  347. }
  348. m.Snapshot = snapshot
  349. sindex, sterm := snapshot.Metadata.Index, snapshot.Metadata.Term
  350. r.logger.Debugf("%x [firstindex: %d, commit: %d] sent snapshot[index: %d, term: %d] to %x [%s]",
  351. r.id, r.raftLog.firstIndex(), r.raftLog.committed, sindex, sterm, to, pr)
  352. pr.becomeSnapshot(sindex)
  353. r.logger.Debugf("%x paused sending replication messages to %x [%s]", r.id, to, pr)
  354. } else {
  355. m.Type = pb.MsgApp
  356. m.Index = pr.Next - 1
  357. m.LogTerm = term
  358. m.Entries = ents
  359. m.Commit = r.raftLog.committed
  360. if n := len(m.Entries); n != 0 {
  361. switch pr.State {
  362. // optimistically increase the next when in ProgressStateReplicate
  363. case ProgressStateReplicate:
  364. last := m.Entries[n-1].Index
  365. pr.optimisticUpdate(last)
  366. pr.ins.add(last)
  367. case ProgressStateProbe:
  368. pr.pause()
  369. default:
  370. r.logger.Panicf("%x is sending append in unhandled state %s", r.id, pr.State)
  371. }
  372. }
  373. }
  374. r.send(m)
  375. }
  376. // sendHeartbeat sends an empty MsgApp
  377. func (r *raft) sendHeartbeat(to uint64, ctx []byte) {
  378. // Attach the commit as min(to.matched, r.committed).
  379. // When the leader sends out heartbeat message,
  380. // the receiver(follower) might not be matched with the leader
  381. // or it might not have all the committed entries.
  382. // The leader MUST NOT forward the follower's commit to
  383. // an unmatched index.
  384. commit := min(r.prs[to].Match, r.raftLog.committed)
  385. m := pb.Message{
  386. To: to,
  387. Type: pb.MsgHeartbeat,
  388. Commit: commit,
  389. Context: ctx,
  390. }
  391. r.send(m)
  392. }
  393. // bcastAppend sends RPC, with entries to all peers that are not up-to-date
  394. // according to the progress recorded in r.prs.
  395. func (r *raft) bcastAppend() {
  396. for id := range r.prs {
  397. if id == r.id {
  398. continue
  399. }
  400. r.sendAppend(id)
  401. }
  402. }
  403. // bcastHeartbeat sends RPC, without entries to all the peers.
  404. func (r *raft) bcastHeartbeat() {
  405. lastCtx := r.readOnly.lastPendingRequestCtx()
  406. if len(lastCtx) == 0 {
  407. r.bcastHeartbeatWithCtx(nil)
  408. } else {
  409. r.bcastHeartbeatWithCtx([]byte(lastCtx))
  410. }
  411. }
  412. func (r *raft) bcastHeartbeatWithCtx(ctx []byte) {
  413. for id := range r.prs {
  414. if id == r.id {
  415. continue
  416. }
  417. r.sendHeartbeat(id, ctx)
  418. }
  419. }
  420. // maybeCommit attempts to advance the commit index. Returns true if
  421. // the commit index changed (in which case the caller should call
  422. // r.bcastAppend).
  423. func (r *raft) maybeCommit() bool {
  424. // TODO(bmizerany): optimize.. Currently naive
  425. mis := make(uint64Slice, 0, len(r.prs))
  426. for id := range r.prs {
  427. mis = append(mis, r.prs[id].Match)
  428. }
  429. sort.Sort(sort.Reverse(mis))
  430. mci := mis[r.quorum()-1]
  431. return r.raftLog.maybeCommit(mci, r.Term)
  432. }
  433. func (r *raft) reset(term uint64) {
  434. if r.Term != term {
  435. r.Term = term
  436. r.Vote = None
  437. }
  438. r.lead = None
  439. r.electionElapsed = 0
  440. r.heartbeatElapsed = 0
  441. r.resetRandomizedElectionTimeout()
  442. r.abortLeaderTransfer()
  443. r.votes = make(map[uint64]bool)
  444. for id := range r.prs {
  445. r.prs[id] = &Progress{Next: r.raftLog.lastIndex() + 1, ins: newInflights(r.maxInflight)}
  446. if id == r.id {
  447. r.prs[id].Match = r.raftLog.lastIndex()
  448. }
  449. }
  450. r.pendingConf = false
  451. r.readOnly = newReadOnly(r.readOnly.option)
  452. }
  453. func (r *raft) appendEntry(es ...pb.Entry) {
  454. li := r.raftLog.lastIndex()
  455. for i := range es {
  456. es[i].Term = r.Term
  457. es[i].Index = li + 1 + uint64(i)
  458. }
  459. r.raftLog.append(es...)
  460. r.prs[r.id].maybeUpdate(r.raftLog.lastIndex())
  461. // Regardless of maybeCommit's return, our caller will call bcastAppend.
  462. r.maybeCommit()
  463. }
  464. // tickElection is run by followers and candidates after r.electionTimeout.
  465. func (r *raft) tickElection() {
  466. r.electionElapsed++
  467. if r.promotable() && r.pastElectionTimeout() {
  468. r.electionElapsed = 0
  469. r.Step(pb.Message{From: r.id, Type: pb.MsgHup})
  470. }
  471. }
  472. // tickHeartbeat is run by leaders to send a MsgBeat after r.heartbeatTimeout.
  473. func (r *raft) tickHeartbeat() {
  474. r.heartbeatElapsed++
  475. r.electionElapsed++
  476. if r.electionElapsed >= r.electionTimeout {
  477. r.electionElapsed = 0
  478. if r.checkQuorum {
  479. r.Step(pb.Message{From: r.id, Type: pb.MsgCheckQuorum})
  480. }
  481. // If current leader cannot transfer leadership in electionTimeout, it becomes leader again.
  482. if r.state == StateLeader && r.leadTransferee != None {
  483. r.abortLeaderTransfer()
  484. }
  485. }
  486. if r.state != StateLeader {
  487. return
  488. }
  489. if r.heartbeatElapsed >= r.heartbeatTimeout {
  490. r.heartbeatElapsed = 0
  491. r.Step(pb.Message{From: r.id, Type: pb.MsgBeat})
  492. }
  493. }
  494. func (r *raft) becomeFollower(term uint64, lead uint64) {
  495. r.step = stepFollower
  496. r.reset(term)
  497. r.tick = r.tickElection
  498. r.lead = lead
  499. r.state = StateFollower
  500. r.logger.Infof("%x became follower at term %d", r.id, r.Term)
  501. }
  502. func (r *raft) becomeCandidate() {
  503. // TODO(xiangli) remove the panic when the raft implementation is stable
  504. if r.state == StateLeader {
  505. panic("invalid transition [leader -> candidate]")
  506. }
  507. r.step = stepCandidate
  508. r.reset(r.Term + 1)
  509. r.tick = r.tickElection
  510. r.Vote = r.id
  511. r.state = StateCandidate
  512. r.logger.Infof("%x became candidate at term %d", r.id, r.Term)
  513. }
  514. func (r *raft) becomePreCandidate() {
  515. // TODO(xiangli) remove the panic when the raft implementation is stable
  516. if r.state == StateLeader {
  517. panic("invalid transition [leader -> pre-candidate]")
  518. }
  519. // Becoming a pre-candidate changes our step functions and state,
  520. // but doesn't change anything else. In particular it does not increase
  521. // r.Term or change r.Vote.
  522. r.step = stepCandidate
  523. r.tick = r.tickElection
  524. r.state = StatePreCandidate
  525. r.logger.Infof("%x became pre-candidate at term %d", r.id, r.Term)
  526. }
  527. func (r *raft) becomeLeader() {
  528. // TODO(xiangli) remove the panic when the raft implementation is stable
  529. if r.state == StateFollower {
  530. panic("invalid transition [follower -> leader]")
  531. }
  532. r.step = stepLeader
  533. r.reset(r.Term)
  534. r.tick = r.tickHeartbeat
  535. r.lead = r.id
  536. r.state = StateLeader
  537. ents, err := r.raftLog.entries(r.raftLog.committed+1, noLimit)
  538. if err != nil {
  539. r.logger.Panicf("unexpected error getting uncommitted entries (%v)", err)
  540. }
  541. nconf := numOfPendingConf(ents)
  542. if nconf > 1 {
  543. panic("unexpected multiple uncommitted config entry")
  544. }
  545. if nconf == 1 {
  546. r.pendingConf = true
  547. }
  548. r.appendEntry(pb.Entry{Data: nil})
  549. r.logger.Infof("%x became leader at term %d", r.id, r.Term)
  550. }
  551. func (r *raft) campaign(t CampaignType) {
  552. var term uint64
  553. var voteMsg pb.MessageType
  554. if t == campaignPreElection {
  555. r.becomePreCandidate()
  556. voteMsg = pb.MsgPreVote
  557. // PreVote RPCs are sent for the next term before we've incremented r.Term.
  558. term = r.Term + 1
  559. } else {
  560. r.becomeCandidate()
  561. voteMsg = pb.MsgVote
  562. term = r.Term
  563. }
  564. if r.quorum() == r.poll(r.id, voteRespMsgType(voteMsg), true) {
  565. // We won the election after voting for ourselves (which must mean that
  566. // this is a single-node cluster). Advance to the next state.
  567. if t == campaignPreElection {
  568. r.campaign(campaignElection)
  569. } else {
  570. r.becomeLeader()
  571. }
  572. return
  573. }
  574. for id := range r.prs {
  575. if id == r.id {
  576. continue
  577. }
  578. r.logger.Infof("%x [logterm: %d, index: %d] sent %s request to %x at term %d",
  579. r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), voteMsg, id, r.Term)
  580. var ctx []byte
  581. if t == campaignTransfer {
  582. ctx = []byte(t)
  583. }
  584. r.send(pb.Message{Term: term, To: id, Type: voteMsg, Index: r.raftLog.lastIndex(), LogTerm: r.raftLog.lastTerm(), Context: ctx})
  585. }
  586. }
  587. func (r *raft) poll(id uint64, t pb.MessageType, v bool) (granted int) {
  588. if v {
  589. r.logger.Infof("%x received %s from %x at term %d", r.id, t, id, r.Term)
  590. } else {
  591. r.logger.Infof("%x received %s rejection from %x at term %d", r.id, t, id, r.Term)
  592. }
  593. if _, ok := r.votes[id]; !ok {
  594. r.votes[id] = v
  595. }
  596. for _, vv := range r.votes {
  597. if vv {
  598. granted++
  599. }
  600. }
  601. return granted
  602. }
  603. func (r *raft) Step(m pb.Message) error {
  604. // Handle the message term, which may result in our stepping down to a follower.
  605. switch {
  606. case m.Term == 0:
  607. // local message
  608. case m.Term > r.Term:
  609. lead := m.From
  610. if m.Type == pb.MsgVote || m.Type == pb.MsgPreVote {
  611. force := bytes.Equal(m.Context, []byte(campaignTransfer))
  612. inLease := r.checkQuorum && r.lead != None && r.electionElapsed < r.electionTimeout
  613. if !force && inLease {
  614. // If a server receives a RequestVote request within the minimum election timeout
  615. // of hearing from a current leader, it does not update its term or grant its vote
  616. 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)",
  617. r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term, r.electionTimeout-r.electionElapsed)
  618. return nil
  619. }
  620. lead = None
  621. }
  622. switch {
  623. case m.Type == pb.MsgPreVote:
  624. // Never change our term in response to a PreVote
  625. case m.Type == pb.MsgPreVoteResp && !m.Reject:
  626. // We send pre-vote requests with a term in our future. If the
  627. // pre-vote is granted, we will increment our term when we get a
  628. // quorum. If it is not, the term comes from the node that
  629. // rejected our vote so we should become a follower at the new
  630. // term.
  631. default:
  632. r.logger.Infof("%x [term: %d] received a %s message with higher term from %x [term: %d]",
  633. r.id, r.Term, m.Type, m.From, m.Term)
  634. r.becomeFollower(m.Term, lead)
  635. }
  636. case m.Term < r.Term:
  637. if r.checkQuorum && (m.Type == pb.MsgHeartbeat || m.Type == pb.MsgApp) {
  638. // We have received messages from a leader at a lower term. It is possible
  639. // that these messages were simply delayed in the network, but this could
  640. // also mean that this node has advanced its term number during a network
  641. // partition, and it is now unable to either win an election or to rejoin
  642. // the majority on the old term. If checkQuorum is false, this will be
  643. // handled by incrementing term numbers in response to MsgVote with a
  644. // higher term, but if checkQuorum is true we may not advance the term on
  645. // MsgVote and must generate other messages to advance the term. The net
  646. // result of these two features is to minimize the disruption caused by
  647. // nodes that have been removed from the cluster's configuration: a
  648. // removed node will send MsgVotes (or MsgPreVotes) which will be ignored,
  649. // but it will not receive MsgApp or MsgHeartbeat, so it will not create
  650. // disruptive term increases
  651. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp})
  652. } else {
  653. // ignore other cases
  654. r.logger.Infof("%x [term: %d] ignored a %s message with lower term from %x [term: %d]",
  655. r.id, r.Term, m.Type, m.From, m.Term)
  656. }
  657. return nil
  658. }
  659. switch m.Type {
  660. case pb.MsgHup:
  661. if r.state != StateLeader {
  662. ents, err := r.raftLog.slice(r.raftLog.applied+1, r.raftLog.committed+1, noLimit)
  663. if err != nil {
  664. r.logger.Panicf("unexpected error getting unapplied entries (%v)", err)
  665. }
  666. if n := numOfPendingConf(ents); n != 0 && r.raftLog.committed > r.raftLog.applied {
  667. r.logger.Warningf("%x cannot campaign at term %d since there are still %d pending configuration changes to apply", r.id, r.Term, n)
  668. return nil
  669. }
  670. r.logger.Infof("%x is starting a new election at term %d", r.id, r.Term)
  671. if r.preVote {
  672. r.campaign(campaignPreElection)
  673. } else {
  674. r.campaign(campaignElection)
  675. }
  676. } else {
  677. r.logger.Debugf("%x ignoring MsgHup because already leader", r.id)
  678. }
  679. case pb.MsgVote, pb.MsgPreVote:
  680. // The m.Term > r.Term clause is for MsgPreVote. For MsgVote m.Term should
  681. // always equal r.Term.
  682. if (r.Vote == None || m.Term > r.Term || r.Vote == m.From) && r.raftLog.isUpToDate(m.Index, m.LogTerm) {
  683. r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] cast %s for %x [logterm: %d, index: %d] at term %d",
  684. r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term)
  685. r.send(pb.Message{To: m.From, Type: voteRespMsgType(m.Type)})
  686. if m.Type == pb.MsgVote {
  687. // Only record real votes.
  688. r.electionElapsed = 0
  689. r.Vote = m.From
  690. }
  691. } else {
  692. r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected %s from %x [logterm: %d, index: %d] at term %d",
  693. r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term)
  694. r.send(pb.Message{To: m.From, Type: voteRespMsgType(m.Type), Reject: true})
  695. }
  696. default:
  697. r.step(r, m)
  698. }
  699. return nil
  700. }
  701. type stepFunc func(r *raft, m pb.Message)
  702. func stepLeader(r *raft, m pb.Message) {
  703. // These message types do not require any progress for m.From.
  704. switch m.Type {
  705. case pb.MsgBeat:
  706. r.bcastHeartbeat()
  707. return
  708. case pb.MsgCheckQuorum:
  709. if !r.checkQuorumActive() {
  710. r.logger.Warningf("%x stepped down to follower since quorum is not active", r.id)
  711. r.becomeFollower(r.Term, None)
  712. }
  713. return
  714. case pb.MsgProp:
  715. if len(m.Entries) == 0 {
  716. r.logger.Panicf("%x stepped empty MsgProp", r.id)
  717. }
  718. if _, ok := r.prs[r.id]; !ok {
  719. // If we are not currently a member of the range (i.e. this node
  720. // was removed from the configuration while serving as leader),
  721. // drop any new proposals.
  722. return
  723. }
  724. if r.leadTransferee != None {
  725. r.logger.Debugf("%x [term %d] transfer leadership to %x is in progress; dropping proposal", r.id, r.Term, r.leadTransferee)
  726. return
  727. }
  728. for i, e := range m.Entries {
  729. if e.Type == pb.EntryConfChange {
  730. if r.pendingConf {
  731. r.logger.Infof("propose conf %s ignored since pending unapplied configuration", e.String())
  732. m.Entries[i] = pb.Entry{Type: pb.EntryNormal}
  733. }
  734. r.pendingConf = true
  735. }
  736. }
  737. r.appendEntry(m.Entries...)
  738. r.bcastAppend()
  739. return
  740. case pb.MsgReadIndex:
  741. if r.quorum() > 1 {
  742. if r.raftLog.zeroTermOnErrCompacted(r.raftLog.term(r.raftLog.committed)) != r.Term {
  743. // Reject read only request when this leader has not committed any log entry at its term.
  744. return
  745. }
  746. // thinking: use an interally defined context instead of the user given context.
  747. // We can express this in terms of the term and index instead of a user-supplied value.
  748. // This would allow multiple reads to piggyback on the same message.
  749. switch r.readOnly.option {
  750. case ReadOnlySafe:
  751. r.readOnly.addRequest(r.raftLog.committed, m)
  752. r.bcastHeartbeatWithCtx(m.Entries[0].Data)
  753. case ReadOnlyLeaseBased:
  754. var ri uint64
  755. if r.checkQuorum {
  756. ri = r.raftLog.committed
  757. }
  758. if m.From == None || m.From == r.id { // from local member
  759. r.readStates = append(r.readStates, ReadState{Index: r.raftLog.committed, RequestCtx: m.Entries[0].Data})
  760. } else {
  761. r.send(pb.Message{To: m.From, Type: pb.MsgReadIndexResp, Index: ri, Entries: m.Entries})
  762. }
  763. }
  764. } else {
  765. r.readStates = append(r.readStates, ReadState{Index: r.raftLog.committed, RequestCtx: m.Entries[0].Data})
  766. }
  767. return
  768. }
  769. // All other message types require a progress for m.From (pr).
  770. pr, prOk := r.prs[m.From]
  771. if !prOk {
  772. r.logger.Debugf("%x no progress available for %x", r.id, m.From)
  773. return
  774. }
  775. switch m.Type {
  776. case pb.MsgAppResp:
  777. pr.RecentActive = true
  778. if m.Reject {
  779. r.logger.Debugf("%x received msgApp rejection(lastindex: %d) from %x for index %d",
  780. r.id, m.RejectHint, m.From, m.Index)
  781. if pr.maybeDecrTo(m.Index, m.RejectHint) {
  782. r.logger.Debugf("%x decreased progress of %x to [%s]", r.id, m.From, pr)
  783. if pr.State == ProgressStateReplicate {
  784. pr.becomeProbe()
  785. }
  786. r.sendAppend(m.From)
  787. }
  788. } else {
  789. oldPaused := pr.IsPaused()
  790. if pr.maybeUpdate(m.Index) {
  791. switch {
  792. case pr.State == ProgressStateProbe:
  793. pr.becomeReplicate()
  794. case pr.State == ProgressStateSnapshot && pr.needSnapshotAbort():
  795. r.logger.Debugf("%x snapshot aborted, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
  796. pr.becomeProbe()
  797. case pr.State == ProgressStateReplicate:
  798. pr.ins.freeTo(m.Index)
  799. }
  800. if r.maybeCommit() {
  801. r.bcastAppend()
  802. } else if oldPaused {
  803. // update() reset the wait state on this node. If we had delayed sending
  804. // an update before, send it now.
  805. r.sendAppend(m.From)
  806. }
  807. // Transfer leadership is in progress.
  808. if m.From == r.leadTransferee && pr.Match == r.raftLog.lastIndex() {
  809. r.logger.Infof("%x sent MsgTimeoutNow to %x after received MsgAppResp", r.id, m.From)
  810. r.sendTimeoutNow(m.From)
  811. }
  812. }
  813. }
  814. case pb.MsgHeartbeatResp:
  815. pr.RecentActive = true
  816. pr.resume()
  817. // free one slot for the full inflights window to allow progress.
  818. if pr.State == ProgressStateReplicate && pr.ins.full() {
  819. pr.ins.freeFirstOne()
  820. }
  821. if pr.Match < r.raftLog.lastIndex() {
  822. r.sendAppend(m.From)
  823. }
  824. if r.readOnly.option != ReadOnlySafe || len(m.Context) == 0 {
  825. return
  826. }
  827. ackCount := r.readOnly.recvAck(m)
  828. if ackCount < r.quorum() {
  829. return
  830. }
  831. rss := r.readOnly.advance(m)
  832. for _, rs := range rss {
  833. req := rs.req
  834. if req.From == None || req.From == r.id { // from local member
  835. r.readStates = append(r.readStates, ReadState{Index: rs.index, RequestCtx: req.Entries[0].Data})
  836. } else {
  837. r.send(pb.Message{To: req.From, Type: pb.MsgReadIndexResp, Index: rs.index, Entries: req.Entries})
  838. }
  839. }
  840. case pb.MsgSnapStatus:
  841. if pr.State != ProgressStateSnapshot {
  842. return
  843. }
  844. if !m.Reject {
  845. pr.becomeProbe()
  846. r.logger.Debugf("%x snapshot succeeded, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
  847. } else {
  848. pr.snapshotFailure()
  849. pr.becomeProbe()
  850. r.logger.Debugf("%x snapshot failed, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
  851. }
  852. // If snapshot finish, wait for the msgAppResp from the remote node before sending
  853. // out the next msgApp.
  854. // If snapshot failure, wait for a heartbeat interval before next try
  855. pr.pause()
  856. case pb.MsgUnreachable:
  857. // During optimistic replication, if the remote becomes unreachable,
  858. // there is huge probability that a MsgApp is lost.
  859. if pr.State == ProgressStateReplicate {
  860. pr.becomeProbe()
  861. }
  862. r.logger.Debugf("%x failed to send message to %x because it is unreachable [%s]", r.id, m.From, pr)
  863. case pb.MsgTransferLeader:
  864. leadTransferee := m.From
  865. lastLeadTransferee := r.leadTransferee
  866. if lastLeadTransferee != None {
  867. if lastLeadTransferee == leadTransferee {
  868. r.logger.Infof("%x [term %d] transfer leadership to %x is in progress, ignores request to same node %x",
  869. r.id, r.Term, leadTransferee, leadTransferee)
  870. return
  871. }
  872. r.abortLeaderTransfer()
  873. r.logger.Infof("%x [term %d] abort previous transferring leadership to %x", r.id, r.Term, lastLeadTransferee)
  874. }
  875. if leadTransferee == r.id {
  876. r.logger.Debugf("%x is already leader. Ignored transferring leadership to self", r.id)
  877. return
  878. }
  879. // Transfer leadership to third party.
  880. r.logger.Infof("%x [term %d] starts to transfer leadership to %x", r.id, r.Term, leadTransferee)
  881. // Transfer leadership should be finished in one electionTimeout, so reset r.electionElapsed.
  882. r.electionElapsed = 0
  883. r.leadTransferee = leadTransferee
  884. if pr.Match == r.raftLog.lastIndex() {
  885. r.sendTimeoutNow(leadTransferee)
  886. r.logger.Infof("%x sends MsgTimeoutNow to %x immediately as %x already has up-to-date log", r.id, leadTransferee, leadTransferee)
  887. } else {
  888. r.sendAppend(leadTransferee)
  889. }
  890. }
  891. }
  892. // stepCandidate is shared by StateCandidate and StatePreCandidate; the difference is
  893. // whether they respond to MsgVoteResp or MsgPreVoteResp.
  894. func stepCandidate(r *raft, m pb.Message) {
  895. // Only handle vote responses corresponding to our candidacy (while in
  896. // StateCandidate, we may get stale MsgPreVoteResp messages in this term from
  897. // our pre-candidate state).
  898. var myVoteRespType pb.MessageType
  899. if r.state == StatePreCandidate {
  900. myVoteRespType = pb.MsgPreVoteResp
  901. } else {
  902. myVoteRespType = pb.MsgVoteResp
  903. }
  904. switch m.Type {
  905. case pb.MsgProp:
  906. r.logger.Infof("%x no leader at term %d; dropping proposal", r.id, r.Term)
  907. return
  908. case pb.MsgApp:
  909. r.becomeFollower(r.Term, m.From)
  910. r.handleAppendEntries(m)
  911. case pb.MsgHeartbeat:
  912. r.becomeFollower(r.Term, m.From)
  913. r.handleHeartbeat(m)
  914. case pb.MsgSnap:
  915. r.becomeFollower(m.Term, m.From)
  916. r.handleSnapshot(m)
  917. case myVoteRespType:
  918. gr := r.poll(m.From, m.Type, !m.Reject)
  919. 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)
  920. switch r.quorum() {
  921. case gr:
  922. if r.state == StatePreCandidate {
  923. r.campaign(campaignElection)
  924. } else {
  925. r.becomeLeader()
  926. r.bcastAppend()
  927. }
  928. case len(r.votes) - gr:
  929. r.becomeFollower(r.Term, None)
  930. }
  931. case pb.MsgTimeoutNow:
  932. r.logger.Debugf("%x [term %d state %v] ignored MsgTimeoutNow from %x", r.id, r.Term, r.state, m.From)
  933. }
  934. }
  935. func stepFollower(r *raft, m pb.Message) {
  936. switch m.Type {
  937. case pb.MsgProp:
  938. if r.lead == None {
  939. r.logger.Infof("%x no leader at term %d; dropping proposal", r.id, r.Term)
  940. return
  941. } else if r.disableProposalForwarding {
  942. r.logger.Infof("%x not forwarding to leader %x at term %d; dropping proposal", r.id, r.lead, r.Term)
  943. return
  944. }
  945. m.To = r.lead
  946. r.send(m)
  947. case pb.MsgApp:
  948. r.electionElapsed = 0
  949. r.lead = m.From
  950. r.handleAppendEntries(m)
  951. case pb.MsgHeartbeat:
  952. r.electionElapsed = 0
  953. r.lead = m.From
  954. r.handleHeartbeat(m)
  955. case pb.MsgSnap:
  956. r.electionElapsed = 0
  957. r.lead = m.From
  958. r.handleSnapshot(m)
  959. case pb.MsgTransferLeader:
  960. if r.lead == None {
  961. r.logger.Infof("%x no leader at term %d; dropping leader transfer msg", r.id, r.Term)
  962. return
  963. }
  964. m.To = r.lead
  965. r.send(m)
  966. case pb.MsgTimeoutNow:
  967. if r.promotable() {
  968. r.logger.Infof("%x [term %d] received MsgTimeoutNow from %x and starts an election to get leadership.", r.id, r.Term, m.From)
  969. // Leadership transfers never use pre-vote even if r.preVote is true; we
  970. // know we are not recovering from a partition so there is no need for the
  971. // extra round trip.
  972. r.campaign(campaignTransfer)
  973. } else {
  974. r.logger.Infof("%x received MsgTimeoutNow from %x but is not promotable", r.id, m.From)
  975. }
  976. case pb.MsgReadIndex:
  977. if r.lead == None {
  978. r.logger.Infof("%x no leader at term %d; dropping index reading msg", r.id, r.Term)
  979. return
  980. }
  981. m.To = r.lead
  982. r.send(m)
  983. case pb.MsgReadIndexResp:
  984. if len(m.Entries) != 1 {
  985. r.logger.Errorf("%x invalid format of MsgReadIndexResp from %x, entries count: %d", r.id, m.From, len(m.Entries))
  986. return
  987. }
  988. r.readStates = append(r.readStates, ReadState{Index: m.Index, RequestCtx: m.Entries[0].Data})
  989. }
  990. }
  991. func (r *raft) handleAppendEntries(m pb.Message) {
  992. if m.Index < r.raftLog.committed {
  993. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed})
  994. return
  995. }
  996. if mlastIndex, ok := r.raftLog.maybeAppend(m.Index, m.LogTerm, m.Commit, m.Entries...); ok {
  997. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: mlastIndex})
  998. } else {
  999. r.logger.Debugf("%x [logterm: %d, index: %d] rejected msgApp [logterm: %d, index: %d] from %x",
  1000. r.id, r.raftLog.zeroTermOnErrCompacted(r.raftLog.term(m.Index)), m.Index, m.LogTerm, m.Index, m.From)
  1001. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: m.Index, Reject: true, RejectHint: r.raftLog.lastIndex()})
  1002. }
  1003. }
  1004. func (r *raft) handleHeartbeat(m pb.Message) {
  1005. r.raftLog.commitTo(m.Commit)
  1006. r.send(pb.Message{To: m.From, Type: pb.MsgHeartbeatResp, Context: m.Context})
  1007. }
  1008. func (r *raft) handleSnapshot(m pb.Message) {
  1009. sindex, sterm := m.Snapshot.Metadata.Index, m.Snapshot.Metadata.Term
  1010. if r.restore(m.Snapshot) {
  1011. r.logger.Infof("%x [commit: %d] restored snapshot [index: %d, term: %d]",
  1012. r.id, r.raftLog.committed, sindex, sterm)
  1013. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.lastIndex()})
  1014. } else {
  1015. r.logger.Infof("%x [commit: %d] ignored snapshot [index: %d, term: %d]",
  1016. r.id, r.raftLog.committed, sindex, sterm)
  1017. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed})
  1018. }
  1019. }
  1020. // restore recovers the state machine from a snapshot. It restores the log and the
  1021. // configuration of state machine.
  1022. func (r *raft) restore(s pb.Snapshot) bool {
  1023. if s.Metadata.Index <= r.raftLog.committed {
  1024. return false
  1025. }
  1026. if r.raftLog.matchTerm(s.Metadata.Index, s.Metadata.Term) {
  1027. r.logger.Infof("%x [commit: %d, lastindex: %d, lastterm: %d] fast-forwarded commit to snapshot [index: %d, term: %d]",
  1028. r.id, r.raftLog.committed, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
  1029. r.raftLog.commitTo(s.Metadata.Index)
  1030. return false
  1031. }
  1032. r.logger.Infof("%x [commit: %d, lastindex: %d, lastterm: %d] starts to restore snapshot [index: %d, term: %d]",
  1033. r.id, r.raftLog.committed, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
  1034. r.raftLog.restore(s)
  1035. r.prs = make(map[uint64]*Progress)
  1036. for _, n := range s.Metadata.ConfState.Nodes {
  1037. match, next := uint64(0), r.raftLog.lastIndex()+1
  1038. if n == r.id {
  1039. match = next - 1
  1040. }
  1041. r.setProgress(n, match, next)
  1042. r.logger.Infof("%x restored progress of %x [%s]", r.id, n, r.prs[n])
  1043. }
  1044. return true
  1045. }
  1046. // promotable indicates whether state machine can be promoted to leader,
  1047. // which is true when its own id is in progress list.
  1048. func (r *raft) promotable() bool {
  1049. _, ok := r.prs[r.id]
  1050. return ok
  1051. }
  1052. func (r *raft) addNode(id uint64) {
  1053. r.pendingConf = false
  1054. if _, ok := r.prs[id]; ok {
  1055. // Ignore any redundant addNode calls (which can happen because the
  1056. // initial bootstrapping entries are applied twice).
  1057. return
  1058. }
  1059. r.setProgress(id, 0, r.raftLog.lastIndex()+1)
  1060. // When a node is first added, we should mark it as recently active.
  1061. // Otherwise, CheckQuorum may cause us to step down if it is invoked
  1062. // before the added node has a chance to communicate with us.
  1063. r.prs[id].RecentActive = true
  1064. }
  1065. func (r *raft) removeNode(id uint64) {
  1066. r.delProgress(id)
  1067. r.pendingConf = false
  1068. // do not try to commit or abort transferring if there is no nodes in the cluster.
  1069. if len(r.prs) == 0 {
  1070. return
  1071. }
  1072. // The quorum size is now smaller, so see if any pending entries can
  1073. // be committed.
  1074. if r.maybeCommit() {
  1075. r.bcastAppend()
  1076. }
  1077. // If the removed node is the leadTransferee, then abort the leadership transferring.
  1078. if r.state == StateLeader && r.leadTransferee == id {
  1079. r.abortLeaderTransfer()
  1080. }
  1081. }
  1082. func (r *raft) resetPendingConf() { r.pendingConf = false }
  1083. func (r *raft) setProgress(id, match, next uint64) {
  1084. r.prs[id] = &Progress{Next: next, Match: match, ins: newInflights(r.maxInflight)}
  1085. }
  1086. func (r *raft) delProgress(id uint64) {
  1087. delete(r.prs, id)
  1088. }
  1089. func (r *raft) loadState(state pb.HardState) {
  1090. if state.Commit < r.raftLog.committed || state.Commit > r.raftLog.lastIndex() {
  1091. r.logger.Panicf("%x state.commit %d is out of range [%d, %d]", r.id, state.Commit, r.raftLog.committed, r.raftLog.lastIndex())
  1092. }
  1093. r.raftLog.committed = state.Commit
  1094. r.Term = state.Term
  1095. r.Vote = state.Vote
  1096. }
  1097. // pastElectionTimeout returns true iff r.electionElapsed is greater
  1098. // than or equal to the randomized election timeout in
  1099. // [electiontimeout, 2 * electiontimeout - 1].
  1100. func (r *raft) pastElectionTimeout() bool {
  1101. return r.electionElapsed >= r.randomizedElectionTimeout
  1102. }
  1103. func (r *raft) resetRandomizedElectionTimeout() {
  1104. r.randomizedElectionTimeout = r.electionTimeout + globalRand.Intn(r.electionTimeout)
  1105. }
  1106. // checkQuorumActive returns true if the quorum is active from
  1107. // the view of the local raft state machine. Otherwise, it returns
  1108. // false.
  1109. // checkQuorumActive also resets all RecentActive to false.
  1110. func (r *raft) checkQuorumActive() bool {
  1111. var act int
  1112. for id := range r.prs {
  1113. if id == r.id { // self is always active
  1114. act++
  1115. continue
  1116. }
  1117. if r.prs[id].RecentActive {
  1118. act++
  1119. }
  1120. r.prs[id].RecentActive = false
  1121. }
  1122. return act >= r.quorum()
  1123. }
  1124. func (r *raft) sendTimeoutNow(to uint64) {
  1125. r.send(pb.Message{To: to, Type: pb.MsgTimeoutNow})
  1126. }
  1127. func (r *raft) abortLeaderTransfer() {
  1128. r.leadTransferee = None
  1129. }
  1130. func numOfPendingConf(ents []pb.Entry) int {
  1131. n := 0
  1132. for i := range ents {
  1133. if ents[i].Type == pb.EntryConfChange {
  1134. n++
  1135. }
  1136. }
  1137. return n
  1138. }