raft.go 25 KB

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