raft.go 25 KB

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