raft.go 32 KB

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