raft.go 26 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902
  1. // Copyright 2015 CoreOS, Inc.
  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. "errors"
  17. "fmt"
  18. "math"
  19. "math/rand"
  20. "sort"
  21. "strings"
  22. pb "github.com/coreos/etcd/raft/raftpb"
  23. )
  24. // None is a placeholder node ID used when there is no leader.
  25. const None uint64 = 0
  26. const noLimit = math.MaxUint64
  27. var ErrSnapshotTemporarilyUnavailable = errors.New("snapshot is temporarily unavailable")
  28. // Possible values for StateType.
  29. const (
  30. StateFollower StateType = iota
  31. StateCandidate
  32. StateLeader
  33. )
  34. // StateType represents the role of a node in a cluster.
  35. type StateType uint64
  36. var stmap = [...]string{
  37. "StateFollower",
  38. "StateCandidate",
  39. "StateLeader",
  40. }
  41. func (st StateType) String() string {
  42. return stmap[uint64(st)]
  43. }
  44. // Config contains the parameters to start a raft.
  45. type Config struct {
  46. // ID is the identity of the local raft. ID cannot be 0.
  47. ID uint64
  48. // peers contains the IDs of all nodes (including self) in the raft cluster. It
  49. // should only be set when starting a new raft cluster. Restarting raft from
  50. // previous configuration will panic if peers is set. peer is private and only
  51. // used for testing right now.
  52. peers []uint64
  53. // ElectionTick is the number of Node.Tick invocations that must pass between
  54. // elections. That is, if a follower does not receive any message from the
  55. // leader of current term before ElectionTick has elapsed, it will become
  56. // candidate and start an election. ElectionTick must be greater than
  57. // HeartbeatTick. We suggest ElectionTick = 10 * HeartbeatTick to avoid
  58. // unnecessary leader switching.
  59. ElectionTick int
  60. // HeartbeatTick is the number of Node.Tick invocations that must pass between
  61. // heartbeats. That is, a leader sends heartbeat messages to maintain its
  62. // leadership every HeartbeatTick ticks.
  63. HeartbeatTick int
  64. // Storage is the storage for raft. raft generates entries and states to be
  65. // stored in storage. raft reads the persisted entries and states out of
  66. // Storage when it needs. raft reads out the previous state and configuration
  67. // out of storage when restarting.
  68. Storage Storage
  69. // Applied is the last applied index. It should only be set when restarting
  70. // raft. raft will not return entries to the application smaller or equal to
  71. // Applied. If Applied is unset when restarting, raft might return previous
  72. // applied entries. This is a very application dependent configuration.
  73. Applied uint64
  74. // MaxSizePerMsg limits the max size of each append message. Smaller value
  75. // lowers the raft recovery cost(initial probing and message lost during normal
  76. // operation). On the other side, it might affect the throughput during normal
  77. // replication. Note: math.MaxUint64 for unlimited, 0 for at most one entry per
  78. // message.
  79. MaxSizePerMsg uint64
  80. // MaxInflightMsgs limits the max number of in-flight append messages during
  81. // optimistic replication phase. The application transportation layer usually
  82. // has its own sending buffer over TCP/UDP. Setting MaxInflightMsgs to avoid
  83. // overflowing that sending buffer. TODO (xiangli): feedback to application to
  84. // limit the proposal rate?
  85. MaxInflightMsgs int
  86. // CheckQuorum specifies if the leader should check quorum activity. Leader
  87. // steps down when quorum is not active for an electionTimeout.
  88. CheckQuorum bool
  89. // Logger is the logger used for raft log. For multinode which can host
  90. // multiple raft group, each raft group can have its own logger
  91. Logger Logger
  92. }
  93. func (c *Config) validate() error {
  94. if c.ID == None {
  95. return errors.New("cannot use none as id")
  96. }
  97. if c.HeartbeatTick <= 0 {
  98. return errors.New("heartbeat tick must be greater than 0")
  99. }
  100. if c.ElectionTick <= c.HeartbeatTick {
  101. return errors.New("election tick must be greater than heartbeat tick")
  102. }
  103. if c.Storage == nil {
  104. return errors.New("storage cannot be nil")
  105. }
  106. if c.MaxInflightMsgs <= 0 {
  107. return errors.New("max inflight messages must be greater than 0")
  108. }
  109. if c.Logger == nil {
  110. c.Logger = raftLogger
  111. }
  112. return nil
  113. }
  114. type raft struct {
  115. id uint64
  116. Term uint64
  117. Vote uint64
  118. // the log
  119. raftLog *raftLog
  120. maxInflight int
  121. maxMsgSize uint64
  122. prs map[uint64]*Progress
  123. state StateType
  124. votes map[uint64]bool
  125. msgs []pb.Message
  126. // the leader id
  127. lead uint64
  128. // New configuration is ignored if there exists unapplied configuration.
  129. pendingConf bool
  130. // number of ticks since it reached last electionTimeout when it is leader
  131. // or candidate.
  132. // number of ticks since it reached last electionTimeout or received a
  133. // valid message from current leader when it is a follower.
  134. electionElapsed int
  135. // number of ticks since it reached last heartbeatTimeout.
  136. // only leader keeps heartbeatElapsed.
  137. heartbeatElapsed int
  138. checkQuorum bool
  139. heartbeatTimeout int
  140. electionTimeout int
  141. // randomizedElectionTimeout is a random number between
  142. // [electiontimeout, 2 * electiontimeout - 1]. It gets reset
  143. // when raft changes its state to follower or candidate.
  144. randomizedElectionTimeout int
  145. rand *rand.Rand
  146. tick func()
  147. step stepFunc
  148. logger Logger
  149. }
  150. func newRaft(c *Config) *raft {
  151. if err := c.validate(); err != nil {
  152. panic(err.Error())
  153. }
  154. raftlog := newLog(c.Storage, c.Logger)
  155. hs, cs, err := c.Storage.InitialState()
  156. if err != nil {
  157. panic(err) // TODO(bdarnell)
  158. }
  159. peers := c.peers
  160. if len(cs.Nodes) > 0 {
  161. if len(peers) > 0 {
  162. // TODO(bdarnell): the peers argument is always nil except in
  163. // tests; the argument should be removed and these tests should be
  164. // updated to specify their nodes through a snapshot.
  165. panic("cannot specify both newRaft(peers) and ConfState.Nodes)")
  166. }
  167. peers = cs.Nodes
  168. }
  169. r := &raft{
  170. id: c.ID,
  171. lead: None,
  172. raftLog: raftlog,
  173. maxMsgSize: c.MaxSizePerMsg,
  174. maxInflight: c.MaxInflightMsgs,
  175. prs: make(map[uint64]*Progress),
  176. electionTimeout: c.ElectionTick,
  177. heartbeatTimeout: c.HeartbeatTick,
  178. logger: c.Logger,
  179. checkQuorum: c.CheckQuorum,
  180. }
  181. r.rand = rand.New(rand.NewSource(int64(c.ID)))
  182. for _, p := range peers {
  183. r.prs[p] = &Progress{Next: 1, ins: newInflights(r.maxInflight)}
  184. }
  185. if !isHardStateEqual(hs, emptyState) {
  186. r.loadState(hs)
  187. }
  188. if c.Applied > 0 {
  189. raftlog.appliedTo(c.Applied)
  190. }
  191. r.becomeFollower(r.Term, None)
  192. nodesStrs := make([]string, 0)
  193. for _, n := range r.nodes() {
  194. nodesStrs = append(nodesStrs, fmt.Sprintf("%x", n))
  195. }
  196. r.logger.Infof("newRaft %x [peers: [%s], term: %d, commit: %d, applied: %d, lastindex: %d, lastterm: %d]",
  197. r.id, strings.Join(nodesStrs, ","), r.Term, r.raftLog.committed, r.raftLog.applied, r.raftLog.lastIndex(), r.raftLog.lastTerm())
  198. return r
  199. }
  200. func (r *raft) hasLeader() bool { return r.lead != None }
  201. func (r *raft) softState() *SoftState { return &SoftState{Lead: r.lead, RaftState: r.state} }
  202. func (r *raft) hardState() pb.HardState {
  203. return pb.HardState{
  204. Term: r.Term,
  205. Vote: r.Vote,
  206. Commit: r.raftLog.committed,
  207. }
  208. }
  209. func (r *raft) quorum() int { return len(r.prs)/2 + 1 }
  210. func (r *raft) nodes() []uint64 {
  211. nodes := make([]uint64, 0, len(r.prs))
  212. for id := range r.prs {
  213. nodes = append(nodes, id)
  214. }
  215. sort.Sort(uint64Slice(nodes))
  216. return nodes
  217. }
  218. // send persists state to stable storage and then sends to its mailbox.
  219. func (r *raft) send(m pb.Message) {
  220. m.From = r.id
  221. // do not attach term to MsgProp
  222. // proposals are a way to forward to the leader and
  223. // should be treated as local message.
  224. if m.Type != pb.MsgProp {
  225. m.Term = r.Term
  226. }
  227. r.msgs = append(r.msgs, m)
  228. }
  229. // sendAppend sends RPC, with entries to the given peer.
  230. func (r *raft) sendAppend(to uint64) {
  231. pr := r.prs[to]
  232. if pr.isPaused() {
  233. return
  234. }
  235. m := pb.Message{}
  236. m.To = to
  237. term, errt := r.raftLog.term(pr.Next - 1)
  238. ents, erre := r.raftLog.entries(pr.Next, r.maxMsgSize)
  239. if errt != nil || erre != nil { // send snapshot if we failed to get term or entries
  240. if !pr.RecentActive {
  241. r.logger.Debugf("ignore sending snapshot to %x since it is not recently active", to)
  242. return
  243. }
  244. m.Type = pb.MsgSnap
  245. snapshot, err := r.raftLog.snapshot()
  246. if err != nil {
  247. if err == ErrSnapshotTemporarilyUnavailable {
  248. r.logger.Debugf("%x failed to send snapshot to %x because snapshot is temporarily unavailable", r.id, to)
  249. return
  250. }
  251. panic(err) // TODO(bdarnell)
  252. }
  253. if IsEmptySnap(snapshot) {
  254. panic("need non-empty snapshot")
  255. }
  256. m.Snapshot = snapshot
  257. sindex, sterm := snapshot.Metadata.Index, snapshot.Metadata.Term
  258. r.logger.Debugf("%x [firstindex: %d, commit: %d] sent snapshot[index: %d, term: %d] to %x [%s]",
  259. r.id, r.raftLog.firstIndex(), r.raftLog.committed, sindex, sterm, to, pr)
  260. pr.becomeSnapshot(sindex)
  261. r.logger.Debugf("%x paused sending replication messages to %x [%s]", r.id, to, pr)
  262. } else {
  263. m.Type = pb.MsgApp
  264. m.Index = pr.Next - 1
  265. m.LogTerm = term
  266. m.Entries = ents
  267. m.Commit = r.raftLog.committed
  268. if n := len(m.Entries); n != 0 {
  269. switch pr.State {
  270. // optimistically increase the next when in ProgressStateReplicate
  271. case ProgressStateReplicate:
  272. last := m.Entries[n-1].Index
  273. pr.optimisticUpdate(last)
  274. pr.ins.add(last)
  275. case ProgressStateProbe:
  276. pr.pause()
  277. default:
  278. r.logger.Panicf("%x is sending append in unhandled state %s", r.id, pr.State)
  279. }
  280. }
  281. }
  282. r.send(m)
  283. }
  284. // sendHeartbeat sends an empty MsgApp
  285. func (r *raft) sendHeartbeat(to uint64) {
  286. // Attach the commit as min(to.matched, r.committed).
  287. // When the leader sends out heartbeat message,
  288. // the receiver(follower) might not be matched with the leader
  289. // or it might not have all the committed entries.
  290. // The leader MUST NOT forward the follower's commit to
  291. // an unmatched index.
  292. commit := min(r.prs[to].Match, r.raftLog.committed)
  293. m := pb.Message{
  294. To: to,
  295. Type: pb.MsgHeartbeat,
  296. Commit: commit,
  297. }
  298. r.send(m)
  299. }
  300. // bcastAppend sends RPC, with entries to all peers that are not up-to-date
  301. // according to the progress recorded in r.prs.
  302. func (r *raft) bcastAppend() {
  303. for id := range r.prs {
  304. if id == r.id {
  305. continue
  306. }
  307. r.sendAppend(id)
  308. }
  309. }
  310. // bcastHeartbeat sends RPC, without entries to all the peers.
  311. func (r *raft) bcastHeartbeat() {
  312. for id := range r.prs {
  313. if id == r.id {
  314. continue
  315. }
  316. r.sendHeartbeat(id)
  317. r.prs[id].resume()
  318. }
  319. }
  320. // maybeCommit attempts to advance the commit index. Returns true if
  321. // the commit index changed (in which case the caller should call
  322. // r.bcastAppend).
  323. func (r *raft) maybeCommit() bool {
  324. // TODO(bmizerany): optimize.. Currently naive
  325. mis := make(uint64Slice, 0, len(r.prs))
  326. for id := range r.prs {
  327. mis = append(mis, r.prs[id].Match)
  328. }
  329. sort.Sort(sort.Reverse(mis))
  330. mci := mis[r.quorum()-1]
  331. return r.raftLog.maybeCommit(mci, r.Term)
  332. }
  333. func (r *raft) reset(term uint64) {
  334. if r.Term != term {
  335. r.Term = term
  336. r.Vote = None
  337. }
  338. r.lead = None
  339. r.electionElapsed = 0
  340. r.heartbeatElapsed = 0
  341. r.resetRandomizedElectionTimeout()
  342. r.votes = make(map[uint64]bool)
  343. for id := range r.prs {
  344. r.prs[id] = &Progress{Next: r.raftLog.lastIndex() + 1, ins: newInflights(r.maxInflight)}
  345. if id == r.id {
  346. r.prs[id].Match = r.raftLog.lastIndex()
  347. }
  348. }
  349. r.pendingConf = false
  350. }
  351. func (r *raft) appendEntry(es ...pb.Entry) {
  352. li := r.raftLog.lastIndex()
  353. for i := range es {
  354. es[i].Term = r.Term
  355. es[i].Index = li + 1 + uint64(i)
  356. }
  357. r.raftLog.append(es...)
  358. r.prs[r.id].maybeUpdate(r.raftLog.lastIndex())
  359. // Regardless of maybeCommit's return, our caller will call bcastAppend.
  360. r.maybeCommit()
  361. }
  362. // tickElection is run by followers and candidates after r.electionTimeout.
  363. func (r *raft) tickElection() {
  364. if !r.promotable() {
  365. r.electionElapsed = 0
  366. return
  367. }
  368. r.electionElapsed++
  369. if r.pastElectionTimeout() {
  370. r.electionElapsed = 0
  371. r.Step(pb.Message{From: r.id, Type: pb.MsgHup})
  372. }
  373. }
  374. // tickHeartbeat is run by leaders to send a MsgBeat after r.heartbeatTimeout.
  375. func (r *raft) tickHeartbeat() {
  376. r.heartbeatElapsed++
  377. r.electionElapsed++
  378. if r.electionElapsed >= r.electionTimeout {
  379. r.electionElapsed = 0
  380. if r.checkQuorum {
  381. r.Step(pb.Message{From: r.id, Type: pb.MsgCheckQuorum})
  382. }
  383. }
  384. if r.state != StateLeader {
  385. return
  386. }
  387. if r.heartbeatElapsed >= r.heartbeatTimeout {
  388. r.heartbeatElapsed = 0
  389. r.Step(pb.Message{From: r.id, Type: pb.MsgBeat})
  390. }
  391. }
  392. func (r *raft) becomeFollower(term uint64, lead uint64) {
  393. r.step = stepFollower
  394. r.reset(term)
  395. r.tick = r.tickElection
  396. r.lead = lead
  397. r.state = StateFollower
  398. r.logger.Infof("%x became follower at term %d", r.id, r.Term)
  399. }
  400. func (r *raft) becomeCandidate() {
  401. // TODO(xiangli) remove the panic when the raft implementation is stable
  402. if r.state == StateLeader {
  403. panic("invalid transition [leader -> candidate]")
  404. }
  405. r.step = stepCandidate
  406. r.reset(r.Term + 1)
  407. r.tick = r.tickElection
  408. r.Vote = r.id
  409. r.state = StateCandidate
  410. r.logger.Infof("%x became candidate at term %d", r.id, r.Term)
  411. }
  412. func (r *raft) becomeLeader() {
  413. // TODO(xiangli) remove the panic when the raft implementation is stable
  414. if r.state == StateFollower {
  415. panic("invalid transition [follower -> leader]")
  416. }
  417. r.step = stepLeader
  418. r.reset(r.Term)
  419. r.tick = r.tickHeartbeat
  420. r.lead = r.id
  421. r.state = StateLeader
  422. ents, err := r.raftLog.entries(r.raftLog.committed+1, noLimit)
  423. if err != nil {
  424. r.logger.Panicf("unexpected error getting uncommitted entries (%v)", err)
  425. }
  426. for _, e := range ents {
  427. if e.Type != pb.EntryConfChange {
  428. continue
  429. }
  430. if r.pendingConf {
  431. panic("unexpected double uncommitted config entry")
  432. }
  433. r.pendingConf = true
  434. }
  435. r.appendEntry(pb.Entry{Data: nil})
  436. r.logger.Infof("%x became leader at term %d", r.id, r.Term)
  437. }
  438. func (r *raft) campaign() {
  439. r.becomeCandidate()
  440. if r.quorum() == r.poll(r.id, true) {
  441. r.becomeLeader()
  442. return
  443. }
  444. for id := range r.prs {
  445. if id == r.id {
  446. continue
  447. }
  448. r.logger.Infof("%x [logterm: %d, index: %d] sent vote request to %x at term %d",
  449. r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), id, r.Term)
  450. r.send(pb.Message{To: id, Type: pb.MsgVote, Index: r.raftLog.lastIndex(), LogTerm: r.raftLog.lastTerm()})
  451. }
  452. }
  453. func (r *raft) poll(id uint64, v bool) (granted int) {
  454. if v {
  455. r.logger.Infof("%x received vote from %x at term %d", r.id, id, r.Term)
  456. } else {
  457. r.logger.Infof("%x received vote rejection from %x at term %d", r.id, id, r.Term)
  458. }
  459. if _, ok := r.votes[id]; !ok {
  460. r.votes[id] = v
  461. }
  462. for _, vv := range r.votes {
  463. if vv {
  464. granted++
  465. }
  466. }
  467. return granted
  468. }
  469. func (r *raft) Step(m pb.Message) error {
  470. if m.Type == pb.MsgHup {
  471. if r.state != StateLeader {
  472. r.logger.Infof("%x is starting a new election at term %d", r.id, r.Term)
  473. r.campaign()
  474. } else {
  475. r.logger.Debugf("%x ignoring MsgHup because already leader", r.id)
  476. }
  477. return nil
  478. }
  479. switch {
  480. case m.Term == 0:
  481. // local message
  482. case m.Term > r.Term:
  483. lead := m.From
  484. if m.Type == pb.MsgVote {
  485. lead = None
  486. }
  487. r.logger.Infof("%x [term: %d] received a %s message with higher term from %x [term: %d]",
  488. r.id, r.Term, m.Type, m.From, m.Term)
  489. r.becomeFollower(m.Term, lead)
  490. case m.Term < r.Term:
  491. // ignore
  492. r.logger.Infof("%x [term: %d] ignored a %s message with lower term from %x [term: %d]",
  493. r.id, r.Term, m.Type, m.From, m.Term)
  494. return nil
  495. }
  496. r.step(r, m)
  497. return nil
  498. }
  499. type stepFunc func(r *raft, m pb.Message)
  500. func stepLeader(r *raft, m pb.Message) {
  501. // These message types do not require any progress for m.From.
  502. switch m.Type {
  503. case pb.MsgBeat:
  504. r.bcastHeartbeat()
  505. return
  506. case pb.MsgCheckQuorum:
  507. if !r.checkQuorumActive() {
  508. r.logger.Warningf("%x stepped down to follower since quorum is not active", r.id)
  509. r.becomeFollower(r.Term, None)
  510. }
  511. return
  512. case pb.MsgProp:
  513. if len(m.Entries) == 0 {
  514. r.logger.Panicf("%x stepped empty MsgProp", r.id)
  515. }
  516. if _, ok := r.prs[r.id]; !ok {
  517. // If we are not currently a member of the range (i.e. this node
  518. // was removed from the configuration while serving as leader),
  519. // drop any new proposals.
  520. return
  521. }
  522. for i, e := range m.Entries {
  523. if e.Type == pb.EntryConfChange {
  524. if r.pendingConf {
  525. m.Entries[i] = pb.Entry{Type: pb.EntryNormal}
  526. }
  527. r.pendingConf = true
  528. }
  529. }
  530. r.appendEntry(m.Entries...)
  531. r.bcastAppend()
  532. return
  533. case pb.MsgVote:
  534. r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %d",
  535. r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
  536. r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp, Reject: true})
  537. return
  538. }
  539. // All other message types require a progress for m.From (pr).
  540. pr, prOk := r.prs[m.From]
  541. if !prOk {
  542. r.logger.Debugf("no progress available for %x", m.From)
  543. return
  544. }
  545. switch m.Type {
  546. case pb.MsgAppResp:
  547. pr.RecentActive = true
  548. if m.Reject {
  549. r.logger.Debugf("%x received msgApp rejection(lastindex: %d) from %x for index %d",
  550. r.id, m.RejectHint, m.From, m.Index)
  551. if pr.maybeDecrTo(m.Index, m.RejectHint) {
  552. r.logger.Debugf("%x decreased progress of %x to [%s]", r.id, m.From, pr)
  553. if pr.State == ProgressStateReplicate {
  554. pr.becomeProbe()
  555. }
  556. r.sendAppend(m.From)
  557. }
  558. } else {
  559. oldPaused := pr.isPaused()
  560. if pr.maybeUpdate(m.Index) {
  561. switch {
  562. case pr.State == ProgressStateProbe:
  563. pr.becomeReplicate()
  564. case pr.State == ProgressStateSnapshot && pr.maybeSnapshotAbort():
  565. r.logger.Debugf("%x snapshot aborted, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
  566. pr.becomeProbe()
  567. case pr.State == ProgressStateReplicate:
  568. pr.ins.freeTo(m.Index)
  569. }
  570. if r.maybeCommit() {
  571. r.bcastAppend()
  572. } else if oldPaused {
  573. // update() reset the wait state on this node. If we had delayed sending
  574. // an update before, send it now.
  575. r.sendAppend(m.From)
  576. }
  577. }
  578. }
  579. case pb.MsgHeartbeatResp:
  580. pr.RecentActive = true
  581. // free one slot for the full inflights window to allow progress.
  582. if pr.State == ProgressStateReplicate && pr.ins.full() {
  583. pr.ins.freeFirstOne()
  584. }
  585. if pr.Match < r.raftLog.lastIndex() {
  586. r.sendAppend(m.From)
  587. }
  588. case pb.MsgSnapStatus:
  589. if pr.State != ProgressStateSnapshot {
  590. return
  591. }
  592. if !m.Reject {
  593. pr.becomeProbe()
  594. r.logger.Debugf("%x snapshot succeeded, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
  595. } else {
  596. pr.snapshotFailure()
  597. pr.becomeProbe()
  598. r.logger.Debugf("%x snapshot failed, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
  599. }
  600. // If snapshot finish, wait for the msgAppResp from the remote node before sending
  601. // out the next msgApp.
  602. // If snapshot failure, wait for a heartbeat interval before next try
  603. pr.pause()
  604. case pb.MsgUnreachable:
  605. // During optimistic replication, if the remote becomes unreachable,
  606. // there is huge probability that a MsgApp is lost.
  607. if pr.State == ProgressStateReplicate {
  608. pr.becomeProbe()
  609. }
  610. r.logger.Debugf("%x failed to send message to %x because it is unreachable [%s]", r.id, m.From, pr)
  611. }
  612. }
  613. func stepCandidate(r *raft, m pb.Message) {
  614. switch m.Type {
  615. case pb.MsgProp:
  616. r.logger.Infof("%x no leader at term %d; dropping proposal", r.id, r.Term)
  617. return
  618. case pb.MsgApp:
  619. r.becomeFollower(r.Term, m.From)
  620. r.handleAppendEntries(m)
  621. case pb.MsgHeartbeat:
  622. r.becomeFollower(r.Term, m.From)
  623. r.handleHeartbeat(m)
  624. case pb.MsgSnap:
  625. r.becomeFollower(m.Term, m.From)
  626. r.handleSnapshot(m)
  627. case pb.MsgVote:
  628. r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %d",
  629. r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
  630. r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp, Reject: true})
  631. case pb.MsgVoteResp:
  632. gr := r.poll(m.From, !m.Reject)
  633. r.logger.Infof("%x [quorum:%d] has received %d votes and %d vote rejections", r.id, r.quorum(), gr, len(r.votes)-gr)
  634. switch r.quorum() {
  635. case gr:
  636. r.becomeLeader()
  637. r.bcastAppend()
  638. case len(r.votes) - gr:
  639. r.becomeFollower(r.Term, None)
  640. }
  641. }
  642. }
  643. func stepFollower(r *raft, m pb.Message) {
  644. switch m.Type {
  645. case pb.MsgProp:
  646. if r.lead == None {
  647. r.logger.Infof("%x no leader at term %d; dropping proposal", r.id, r.Term)
  648. return
  649. }
  650. m.To = r.lead
  651. r.send(m)
  652. case pb.MsgApp:
  653. r.electionElapsed = 0
  654. r.lead = m.From
  655. r.handleAppendEntries(m)
  656. case pb.MsgHeartbeat:
  657. r.electionElapsed = 0
  658. r.lead = m.From
  659. r.handleHeartbeat(m)
  660. case pb.MsgSnap:
  661. r.electionElapsed = 0
  662. r.handleSnapshot(m)
  663. case pb.MsgVote:
  664. if (r.Vote == None || r.Vote == m.From) && r.raftLog.isUpToDate(m.Index, m.LogTerm) {
  665. r.electionElapsed = 0
  666. r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] voted for %x [logterm: %d, index: %d] at term %d",
  667. r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
  668. r.Vote = m.From
  669. r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp})
  670. } else {
  671. r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %d",
  672. r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
  673. r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp, Reject: true})
  674. }
  675. }
  676. }
  677. func (r *raft) handleAppendEntries(m pb.Message) {
  678. if m.Index < r.raftLog.committed {
  679. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed})
  680. return
  681. }
  682. if mlastIndex, ok := r.raftLog.maybeAppend(m.Index, m.LogTerm, m.Commit, m.Entries...); ok {
  683. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: mlastIndex})
  684. } else {
  685. r.logger.Debugf("%x [logterm: %d, index: %d] rejected msgApp [logterm: %d, index: %d] from %x",
  686. r.id, r.raftLog.zeroTermOnErrCompacted(r.raftLog.term(m.Index)), m.Index, m.LogTerm, m.Index, m.From)
  687. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: m.Index, Reject: true, RejectHint: r.raftLog.lastIndex()})
  688. }
  689. }
  690. func (r *raft) handleHeartbeat(m pb.Message) {
  691. r.raftLog.commitTo(m.Commit)
  692. r.send(pb.Message{To: m.From, Type: pb.MsgHeartbeatResp})
  693. }
  694. func (r *raft) handleSnapshot(m pb.Message) {
  695. sindex, sterm := m.Snapshot.Metadata.Index, m.Snapshot.Metadata.Term
  696. if r.restore(m.Snapshot) {
  697. r.logger.Infof("%x [commit: %d] restored snapshot [index: %d, term: %d]",
  698. r.id, r.raftLog.committed, sindex, sterm)
  699. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.lastIndex()})
  700. } else {
  701. r.logger.Infof("%x [commit: %d] ignored snapshot [index: %d, term: %d]",
  702. r.id, r.raftLog.committed, sindex, sterm)
  703. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed})
  704. }
  705. }
  706. // restore recovers the state machine from a snapshot. It restores the log and the
  707. // configuration of state machine.
  708. func (r *raft) restore(s pb.Snapshot) bool {
  709. if s.Metadata.Index <= r.raftLog.committed {
  710. return false
  711. }
  712. if r.raftLog.matchTerm(s.Metadata.Index, s.Metadata.Term) {
  713. r.logger.Infof("%x [commit: %d, lastindex: %d, lastterm: %d] fast-forwarded commit to snapshot [index: %d, term: %d]",
  714. r.id, r.raftLog.committed, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
  715. r.raftLog.commitTo(s.Metadata.Index)
  716. return false
  717. }
  718. r.logger.Infof("%x [commit: %d, lastindex: %d, lastterm: %d] starts to restore snapshot [index: %d, term: %d]",
  719. r.id, r.raftLog.committed, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
  720. r.raftLog.restore(s)
  721. r.prs = make(map[uint64]*Progress)
  722. for _, n := range s.Metadata.ConfState.Nodes {
  723. match, next := uint64(0), uint64(r.raftLog.lastIndex())+1
  724. if n == r.id {
  725. match = next - 1
  726. } else {
  727. match = 0
  728. }
  729. r.setProgress(n, match, next)
  730. r.logger.Infof("%x restored progress of %x [%s]", r.id, n, r.prs[n])
  731. }
  732. return true
  733. }
  734. // promotable indicates whether state machine can be promoted to leader,
  735. // which is true when its own id is in progress list.
  736. func (r *raft) promotable() bool {
  737. _, ok := r.prs[r.id]
  738. return ok
  739. }
  740. func (r *raft) addNode(id uint64) {
  741. if _, ok := r.prs[id]; ok {
  742. // Ignore any redundant addNode calls (which can happen because the
  743. // initial bootstrapping entries are applied twice).
  744. return
  745. }
  746. r.setProgress(id, 0, r.raftLog.lastIndex()+1)
  747. r.pendingConf = false
  748. }
  749. func (r *raft) removeNode(id uint64) {
  750. r.delProgress(id)
  751. r.pendingConf = false
  752. // The quorum size is now smaller, so see if any pending entries can
  753. // be committed.
  754. if r.maybeCommit() {
  755. r.bcastAppend()
  756. }
  757. }
  758. func (r *raft) resetPendingConf() { r.pendingConf = false }
  759. func (r *raft) setProgress(id, match, next uint64) {
  760. r.prs[id] = &Progress{Next: next, Match: match, ins: newInflights(r.maxInflight)}
  761. }
  762. func (r *raft) delProgress(id uint64) {
  763. delete(r.prs, id)
  764. }
  765. func (r *raft) loadState(state pb.HardState) {
  766. if state.Commit < r.raftLog.committed || state.Commit > r.raftLog.lastIndex() {
  767. r.logger.Panicf("%x state.commit %d is out of range [%d, %d]", r.id, state.Commit, r.raftLog.committed, r.raftLog.lastIndex())
  768. }
  769. r.raftLog.committed = state.Commit
  770. r.Term = state.Term
  771. r.Vote = state.Vote
  772. }
  773. // pastElectionTimeout returns true iff r.electionElapsed is greater
  774. // than or equal to the randomized election timeout in
  775. // [electiontimeout, 2 * electiontimeout - 1].
  776. func (r *raft) pastElectionTimeout() bool {
  777. return r.electionElapsed >= r.randomizedElectionTimeout
  778. }
  779. func (r *raft) resetRandomizedElectionTimeout() {
  780. r.randomizedElectionTimeout = r.electionTimeout + r.rand.Intn(r.electionTimeout)
  781. }
  782. // checkQuorumActive returns true if the quorum is active from
  783. // the view of the local raft state machine. Otherwise, it returns
  784. // false.
  785. // checkQuorumActive also resets all RecentActive to false.
  786. func (r *raft) checkQuorumActive() bool {
  787. var act int
  788. for id := range r.prs {
  789. if id == r.id { // self is always active
  790. act++
  791. continue
  792. }
  793. if r.prs[id].RecentActive {
  794. act++
  795. }
  796. r.prs[id].RecentActive = false
  797. }
  798. return act >= r.quorum()
  799. }