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