raft.go 30 KB

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