raft.go 26 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904
  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 the raft cluster. It
  50. // should only be set when starting a new raft cluster. Restarting raft from
  51. // previous configuration will panic if peers is set. peer is private and only
  52. // used for testing right now.
  53. peers []uint64
  54. // ElectionTick is the number of Node.Tick invocations that must pass between
  55. // elections. That is, if a follower does not receive any message from the
  56. // leader of current term before ElectionTick has elapsed, it will become
  57. // candidate and start an election. ElectionTick must be greater than
  58. // HeartbeatTick. We suggest ElectionTick = 10 * HeartbeatTick to avoid
  59. // unnecessary leader switching.
  60. ElectionTick int
  61. // HeartbeatTick is the number of Node.Tick invocations that must pass between
  62. // heartbeats. That is, a leader sends heartbeat messages to maintain its
  63. // leadership every HeartbeatTick ticks.
  64. HeartbeatTick int
  65. // Storage is the storage for raft. raft generates entries and states to be
  66. // stored in storage. raft reads the persisted entries and states out of
  67. // Storage when it needs. raft reads out the previous state and configuration
  68. // out of storage when restarting.
  69. Storage Storage
  70. // Applied is the last applied index. It should only be set when restarting
  71. // raft. raft will not return entries to the application smaller or equal to
  72. // Applied. If Applied is unset when restarting, raft might return previous
  73. // applied entries. This is a very application dependent configuration.
  74. Applied uint64
  75. // MaxSizePerMsg limits the max size of each append message. Smaller value
  76. // lowers the raft recovery cost(initial probing and message lost during normal
  77. // operation). On the other side, it might affect the throughput during normal
  78. // replication. Note: math.MaxUint64 for unlimited, 0 for at most one entry per
  79. // message.
  80. MaxSizePerMsg uint64
  81. // MaxInflightMsgs limits the max number of in-flight append messages during
  82. // optimistic replication phase. The application transportation layer usually
  83. // has its own sending buffer over TCP/UDP. Setting MaxInflightMsgs to avoid
  84. // overflowing that sending buffer. TODO (xiangli): feedback to application to
  85. // limit the proposal rate?
  86. MaxInflightMsgs int
  87. // CheckQuorum specifies if the leader should check quorum activity. Leader
  88. // steps down when quorum is not active for an electionTimeout.
  89. CheckQuorum bool
  90. // Logger is the logger used for raft log. For multinode which can host
  91. // multiple raft group, each raft group can have its 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. // maybeCommit attempts to advance the commit index. Returns true if
  318. // the commit index changed (in which case the caller should call
  319. // r.bcastAppend).
  320. func (r *raft) maybeCommit() bool {
  321. // TODO(bmizerany): optimize.. Currently naive
  322. mis := make(uint64Slice, 0, len(r.prs))
  323. for id := range r.prs {
  324. mis = append(mis, r.prs[id].Match)
  325. }
  326. sort.Sort(sort.Reverse(mis))
  327. mci := mis[r.quorum()-1]
  328. return r.raftLog.maybeCommit(mci, r.Term)
  329. }
  330. func (r *raft) reset(term uint64) {
  331. if r.Term != term {
  332. r.Term = term
  333. r.Vote = None
  334. }
  335. r.lead = None
  336. r.electionElapsed = 0
  337. r.heartbeatElapsed = 0
  338. r.votes = make(map[uint64]bool)
  339. for id := range r.prs {
  340. r.prs[id] = &Progress{Next: r.raftLog.lastIndex() + 1, ins: newInflights(r.maxInflight)}
  341. if id == r.id {
  342. r.prs[id].Match = r.raftLog.lastIndex()
  343. }
  344. }
  345. r.pendingConf = false
  346. }
  347. func (r *raft) appendEntry(es ...pb.Entry) {
  348. li := r.raftLog.lastIndex()
  349. for i := range es {
  350. es[i].Term = r.Term
  351. es[i].Index = li + 1 + uint64(i)
  352. }
  353. r.raftLog.append(es...)
  354. r.prs[r.id].maybeUpdate(r.raftLog.lastIndex())
  355. // Regardless of maybeCommit's return, our caller will call bcastAppend.
  356. r.maybeCommit()
  357. }
  358. // tickElection is run by followers and candidates after r.electionTimeout.
  359. func (r *raft) tickElection() {
  360. if !r.promotable() {
  361. r.electionElapsed = 0
  362. return
  363. }
  364. r.electionElapsed++
  365. if r.isElectionTimeout() {
  366. r.electionElapsed = 0
  367. r.Step(pb.Message{From: r.id, Type: pb.MsgHup})
  368. }
  369. }
  370. // tickHeartbeat is run by leaders to send a MsgBeat after r.heartbeatTimeout.
  371. func (r *raft) tickHeartbeat() {
  372. r.heartbeatElapsed++
  373. r.electionElapsed++
  374. if r.electionElapsed >= r.electionTimeout {
  375. r.electionElapsed = 0
  376. if r.checkQuorum {
  377. r.Step(pb.Message{From: r.id, Type: pb.MsgCheckQuorum})
  378. }
  379. }
  380. if r.state != StateLeader {
  381. return
  382. }
  383. if r.heartbeatElapsed >= r.heartbeatTimeout {
  384. r.heartbeatElapsed = 0
  385. r.Step(pb.Message{From: r.id, Type: pb.MsgBeat})
  386. }
  387. }
  388. func (r *raft) becomeFollower(term uint64, lead uint64) {
  389. r.step = stepFollower
  390. r.reset(term)
  391. r.tick = r.tickElection
  392. r.lead = lead
  393. r.state = StateFollower
  394. r.logger.Infof("%x became follower at term %d", r.id, r.Term)
  395. }
  396. func (r *raft) becomeCandidate() {
  397. // TODO(xiangli) remove the panic when the raft implementation is stable
  398. if r.state == StateLeader {
  399. panic("invalid transition [leader -> candidate]")
  400. }
  401. r.step = stepCandidate
  402. r.reset(r.Term + 1)
  403. r.tick = r.tickElection
  404. r.Vote = r.id
  405. r.state = StateCandidate
  406. r.logger.Infof("%x became candidate at term %d", r.id, r.Term)
  407. }
  408. func (r *raft) becomeLeader() {
  409. // TODO(xiangli) remove the panic when the raft implementation is stable
  410. if r.state == StateFollower {
  411. panic("invalid transition [follower -> leader]")
  412. }
  413. r.step = stepLeader
  414. r.reset(r.Term)
  415. r.tick = r.tickHeartbeat
  416. r.lead = r.id
  417. r.state = StateLeader
  418. ents, err := r.raftLog.entries(r.raftLog.committed+1, noLimit)
  419. if err != nil {
  420. r.logger.Panicf("unexpected error getting uncommitted entries (%v)", err)
  421. }
  422. for _, e := range ents {
  423. if e.Type != pb.EntryConfChange {
  424. continue
  425. }
  426. if r.pendingConf {
  427. panic("unexpected double uncommitted config entry")
  428. }
  429. r.pendingConf = true
  430. }
  431. r.appendEntry(pb.Entry{Data: nil})
  432. r.logger.Infof("%x became leader at term %d", r.id, r.Term)
  433. }
  434. func (r *raft) campaign() {
  435. r.becomeCandidate()
  436. if r.quorum() == r.poll(r.id, true) {
  437. r.becomeLeader()
  438. return
  439. }
  440. for id := range r.prs {
  441. if id == r.id {
  442. continue
  443. }
  444. r.logger.Infof("%x [logterm: %d, index: %d] sent vote request to %x at term %d",
  445. r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), id, r.Term)
  446. r.send(pb.Message{To: id, Type: pb.MsgVote, Index: r.raftLog.lastIndex(), LogTerm: r.raftLog.lastTerm()})
  447. }
  448. }
  449. func (r *raft) poll(id uint64, v bool) (granted int) {
  450. if v {
  451. r.logger.Infof("%x received vote from %x at term %d", r.id, id, r.Term)
  452. } else {
  453. r.logger.Infof("%x received vote rejection from %x at term %d", r.id, id, r.Term)
  454. }
  455. if _, ok := r.votes[id]; !ok {
  456. r.votes[id] = v
  457. }
  458. for _, vv := range r.votes {
  459. if vv {
  460. granted++
  461. }
  462. }
  463. return granted
  464. }
  465. func (r *raft) Step(m pb.Message) error {
  466. if m.Type == pb.MsgHup {
  467. if r.state != StateLeader {
  468. r.logger.Infof("%x is starting a new election at term %d", r.id, r.Term)
  469. r.campaign()
  470. } else {
  471. r.logger.Debugf("%x ignoring MsgHup because already leader", r.id)
  472. }
  473. return nil
  474. }
  475. switch {
  476. case m.Term == 0:
  477. // local message
  478. case m.Term > r.Term:
  479. lead := m.From
  480. if m.Type == pb.MsgVote {
  481. lead = None
  482. }
  483. r.logger.Infof("%x [term: %d] received a %s message with higher term from %x [term: %d]",
  484. r.id, r.Term, m.Type, m.From, m.Term)
  485. r.becomeFollower(m.Term, lead)
  486. case m.Term < r.Term:
  487. // ignore
  488. r.logger.Infof("%x [term: %d] ignored a %s message with lower term from %x [term: %d]",
  489. r.id, r.Term, m.Type, m.From, m.Term)
  490. return nil
  491. }
  492. r.step(r, m)
  493. return nil
  494. }
  495. type stepFunc func(r *raft, m pb.Message)
  496. func stepLeader(r *raft, m pb.Message) {
  497. // These message types do not require any progress for m.From.
  498. switch m.Type {
  499. case pb.MsgBeat:
  500. r.bcastHeartbeat()
  501. return
  502. case pb.MsgCheckQuorum:
  503. if !r.checkQuorumActive() {
  504. r.logger.Warningf("%x stepped down to follower since quorum is not active", r.id)
  505. r.becomeFollower(r.Term, None)
  506. }
  507. return
  508. case pb.MsgProp:
  509. if len(m.Entries) == 0 {
  510. r.logger.Panicf("%x stepped empty MsgProp", r.id)
  511. }
  512. if _, ok := r.prs[r.id]; !ok {
  513. // If we are not currently a member of the range (i.e. this node
  514. // was removed from the configuration while serving as leader),
  515. // drop any new proposals.
  516. return
  517. }
  518. for i, e := range m.Entries {
  519. if e.Type == pb.EntryConfChange {
  520. if r.pendingConf {
  521. m.Entries[i] = pb.Entry{Type: pb.EntryNormal}
  522. }
  523. r.pendingConf = true
  524. }
  525. }
  526. r.appendEntry(m.Entries...)
  527. r.bcastAppend()
  528. return
  529. case pb.MsgVote:
  530. r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %d",
  531. r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
  532. r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp, Reject: true})
  533. return
  534. }
  535. // All other message types require a progress for m.From (pr).
  536. pr, prOk := r.prs[m.From]
  537. if !prOk {
  538. r.logger.Debugf("no progress available for %x", m.From)
  539. return
  540. }
  541. switch m.Type {
  542. case pb.MsgAppResp:
  543. pr.RecentActive = true
  544. if m.Reject {
  545. r.logger.Debugf("%x received msgApp rejection(lastindex: %d) from %x for index %d",
  546. r.id, m.RejectHint, m.From, m.Index)
  547. if pr.maybeDecrTo(m.Index, m.RejectHint) {
  548. r.logger.Debugf("%x decreased progress of %x to [%s]", r.id, m.From, pr)
  549. if pr.State == ProgressStateReplicate {
  550. pr.becomeProbe()
  551. }
  552. r.sendAppend(m.From)
  553. }
  554. } else {
  555. oldPaused := pr.isPaused()
  556. if pr.maybeUpdate(m.Index) {
  557. switch {
  558. case pr.State == ProgressStateProbe:
  559. pr.becomeReplicate()
  560. case pr.State == ProgressStateSnapshot && pr.maybeSnapshotAbort():
  561. r.logger.Debugf("%x snapshot aborted, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
  562. pr.becomeProbe()
  563. case pr.State == ProgressStateReplicate:
  564. pr.ins.freeTo(m.Index)
  565. }
  566. if r.maybeCommit() {
  567. r.bcastAppend()
  568. } else if oldPaused {
  569. // update() reset the wait state on this node. If we had delayed sending
  570. // an update before, send it now.
  571. r.sendAppend(m.From)
  572. }
  573. }
  574. }
  575. case pb.MsgHeartbeatResp:
  576. pr.RecentActive = true
  577. // free one slot for the full inflights window to allow progress.
  578. if pr.State == ProgressStateReplicate && pr.ins.full() {
  579. pr.ins.freeFirstOne()
  580. }
  581. if pr.Match < r.raftLog.lastIndex() {
  582. r.sendAppend(m.From)
  583. }
  584. case pb.MsgSnapStatus:
  585. if pr.State != ProgressStateSnapshot {
  586. return
  587. }
  588. if !m.Reject {
  589. pr.becomeProbe()
  590. r.logger.Debugf("%x snapshot succeeded, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
  591. } else {
  592. pr.snapshotFailure()
  593. pr.becomeProbe()
  594. r.logger.Debugf("%x snapshot failed, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
  595. }
  596. // If snapshot finish, wait for the msgAppResp from the remote node before sending
  597. // out the next msgApp.
  598. // If snapshot failure, wait for a heartbeat interval before next try
  599. pr.pause()
  600. case pb.MsgUnreachable:
  601. // During optimistic replication, if the remote becomes unreachable,
  602. // there is huge probability that a MsgApp is lost.
  603. if pr.State == ProgressStateReplicate {
  604. pr.becomeProbe()
  605. }
  606. r.logger.Debugf("%x failed to send message to %x because it is unreachable [%s]", r.id, m.From, pr)
  607. }
  608. }
  609. func stepCandidate(r *raft, m pb.Message) {
  610. switch m.Type {
  611. case pb.MsgProp:
  612. r.logger.Infof("%x no leader at term %d; dropping proposal", r.id, r.Term)
  613. return
  614. case pb.MsgApp:
  615. r.becomeFollower(r.Term, m.From)
  616. r.handleAppendEntries(m)
  617. case pb.MsgHeartbeat:
  618. r.becomeFollower(r.Term, m.From)
  619. r.handleHeartbeat(m)
  620. case pb.MsgSnap:
  621. r.becomeFollower(m.Term, m.From)
  622. r.handleSnapshot(m)
  623. case pb.MsgVote:
  624. r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %d",
  625. r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
  626. r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp, Reject: true})
  627. case pb.MsgVoteResp:
  628. gr := r.poll(m.From, !m.Reject)
  629. r.logger.Infof("%x [quorum:%d] has received %d votes and %d vote rejections", r.id, r.quorum(), gr, len(r.votes)-gr)
  630. switch r.quorum() {
  631. case gr:
  632. r.becomeLeader()
  633. r.bcastAppend()
  634. case len(r.votes) - gr:
  635. r.becomeFollower(r.Term, None)
  636. }
  637. }
  638. }
  639. func stepFollower(r *raft, m pb.Message) {
  640. switch m.Type {
  641. case pb.MsgProp:
  642. if r.lead == None {
  643. r.logger.Infof("%x no leader at term %d; dropping proposal", r.id, r.Term)
  644. return
  645. }
  646. m.To = r.lead
  647. r.send(m)
  648. case pb.MsgApp:
  649. r.electionElapsed = 0
  650. r.lead = m.From
  651. r.handleAppendEntries(m)
  652. case pb.MsgHeartbeat:
  653. r.electionElapsed = 0
  654. r.lead = m.From
  655. r.handleHeartbeat(m)
  656. case pb.MsgSnap:
  657. r.electionElapsed = 0
  658. r.handleSnapshot(m)
  659. case pb.MsgVote:
  660. if (r.Vote == None || r.Vote == m.From) && r.raftLog.isUpToDate(m.Index, m.LogTerm) {
  661. r.electionElapsed = 0
  662. r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] voted for %x [logterm: %d, index: %d] at term %d",
  663. r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
  664. r.Vote = m.From
  665. r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp})
  666. } else {
  667. r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %d",
  668. r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
  669. r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp, Reject: true})
  670. }
  671. }
  672. }
  673. func (r *raft) handleAppendEntries(m pb.Message) {
  674. if m.Index < r.raftLog.committed {
  675. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed})
  676. return
  677. }
  678. if mlastIndex, ok := r.raftLog.maybeAppend(m.Index, m.LogTerm, m.Commit, m.Entries...); ok {
  679. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: mlastIndex})
  680. } else {
  681. r.logger.Debugf("%x [logterm: %d, index: %d] rejected msgApp [logterm: %d, index: %d] from %x",
  682. r.id, r.raftLog.zeroTermOnErrCompacted(r.raftLog.term(m.Index)), m.Index, m.LogTerm, m.Index, m.From)
  683. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: m.Index, Reject: true, RejectHint: r.raftLog.lastIndex()})
  684. }
  685. }
  686. func (r *raft) handleHeartbeat(m pb.Message) {
  687. r.raftLog.commitTo(m.Commit)
  688. r.send(pb.Message{To: m.From, Type: pb.MsgHeartbeatResp})
  689. }
  690. func (r *raft) handleSnapshot(m pb.Message) {
  691. sindex, sterm := m.Snapshot.Metadata.Index, m.Snapshot.Metadata.Term
  692. if r.restore(m.Snapshot) {
  693. r.logger.Infof("%x [commit: %d] restored 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.lastIndex()})
  696. } else {
  697. r.logger.Infof("%x [commit: %d] ignored 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.committed})
  700. }
  701. }
  702. // restore recovers the state machine from a snapshot. It restores the log and the
  703. // configuration of state machine.
  704. func (r *raft) restore(s pb.Snapshot) bool {
  705. if s.Metadata.Index <= r.raftLog.committed {
  706. return false
  707. }
  708. if r.raftLog.matchTerm(s.Metadata.Index, s.Metadata.Term) {
  709. r.logger.Infof("%x [commit: %d, lastindex: %d, lastterm: %d] fast-forwarded commit to snapshot [index: %d, term: %d]",
  710. r.id, r.raftLog.committed, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
  711. r.raftLog.commitTo(s.Metadata.Index)
  712. return false
  713. }
  714. r.logger.Infof("%x [commit: %d, lastindex: %d, lastterm: %d] starts to restore snapshot [index: %d, term: %d]",
  715. r.id, r.raftLog.committed, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
  716. r.raftLog.restore(s)
  717. r.prs = make(map[uint64]*Progress)
  718. for _, n := range s.Metadata.ConfState.Nodes {
  719. match, next := uint64(0), uint64(r.raftLog.lastIndex())+1
  720. if n == r.id {
  721. match = next - 1
  722. } else {
  723. match = 0
  724. }
  725. r.setProgress(n, match, next)
  726. r.logger.Infof("%x restored progress of %x [%s]", r.id, n, r.prs[n])
  727. }
  728. return true
  729. }
  730. // promotable indicates whether state machine can be promoted to leader,
  731. // which is true when its own id is in progress list.
  732. func (r *raft) promotable() bool {
  733. _, ok := r.prs[r.id]
  734. return ok
  735. }
  736. func (r *raft) addNode(id uint64) {
  737. if _, ok := r.prs[id]; ok {
  738. // Ignore any redundant addNode calls (which can happen because the
  739. // initial bootstrapping entries are applied twice).
  740. return
  741. }
  742. r.setProgress(id, 0, r.raftLog.lastIndex()+1)
  743. r.pendingConf = false
  744. }
  745. func (r *raft) removeNode(id uint64) {
  746. r.delProgress(id)
  747. r.pendingConf = false
  748. // do not try to commit or abort transferring if there is no nodes in the cluster.
  749. if len(r.prs) == 0 {
  750. return
  751. }
  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. // isElectionTimeout returns true if r.electionElapsed is greater than the
  774. // randomized election timeout in (electiontimeout, 2 * electiontimeout - 1).
  775. // Otherwise, it returns false.
  776. func (r *raft) isElectionTimeout() bool {
  777. d := r.electionElapsed - r.electionTimeout
  778. if d < 0 {
  779. return false
  780. }
  781. return d > r.rand.Int()%r.electionTimeout
  782. }
  783. // checkQuorumActive returns true if the quorum is active from
  784. // the view of the local raft state machine. Otherwise, it returns
  785. // false.
  786. // checkQuorumActive also resets all RecentActive to false.
  787. func (r *raft) checkQuorumActive() bool {
  788. var act int
  789. for id := range r.prs {
  790. if id == r.id { // self is always active
  791. act++
  792. continue
  793. }
  794. if r.prs[id].RecentActive {
  795. act++
  796. }
  797. r.prs[id].RecentActive = false
  798. }
  799. return act >= r.quorum()
  800. }