raft.go 32 KB

1234567891011121314151617181920212223242526272829303132333435363738394041424344454647484950515253545556575859606162636465666768697071727374757677787980818283848586878889909192939495969798991001011021031041051061071081091101111121131141151161171181191201211221231241251261271281291301311321331341351361371381391401411421431441451461471481491501511521531541551561571581591601611621631641651661671681691701711721731741751761771781791801811821831841851861871881891901911921931941951961971981992002012022032042052062072082092102112122132142152162172182192202212222232242252262272282292302312322332342352362372382392402412422432442452462472482492502512522532542552562572582592602612622632642652662672682692702712722732742752762772782792802812822832842852862872882892902912922932942952962972982993003013023033043053063073083093103113123133143153163173183193203213223233243253263273283293303313323333343353363373383393403413423433443453463473483493503513523533543553563573583593603613623633643653663673683693703713723733743753763773783793803813823833843853863873883893903913923933943953963973983994004014024034044054064074084094104114124134144154164174184194204214224234244254264274284294304314324334344354364374384394404414424434444454464474484494504514524534544554564574584594604614624634644654664674684694704714724734744754764774784794804814824834844854864874884894904914924934944954964974984995005015025035045055065075085095105115125135145155165175185195205215225235245255265275285295305315325335345355365375385395405415425435445455465475485495505515525535545555565575585595605615625635645655665675685695705715725735745755765775785795805815825835845855865875885895905915925935945955965975985996006016026036046056066076086096106116126136146156166176186196206216226236246256266276286296306316326336346356366376386396406416426436446456466476486496506516526536546556566576586596606616626636646656666676686696706716726736746756766776786796806816826836846856866876886896906916926936946956966976986997007017027037047057067077087097107117127137147157167177187197207217227237247257267277287297307317327337347357367377387397407417427437447457467477487497507517527537547557567577587597607617627637647657667677687697707717727737747757767777787797807817827837847857867877887897907917927937947957967977987998008018028038048058068078088098108118128138148158168178188198208218228238248258268278288298308318328338348358368378388398408418428438448458468478488498508518528538548558568578588598608618628638648658668678688698708718728738748758768778788798808818828838848858868878888898908918928938948958968978988999009019029039049059069079089099109119129139149159169179189199209219229239249259269279289299309319329339349359369379389399409419429439449459469479489499509519529539549559569579589599609619629639649659669679689699709719729739749759769779789799809819829839849859869879889899909919929939949959969979989991000100110021003100410051006100710081009101010111012101310141015101610171018101910201021102210231024102510261027102810291030103110321033103410351036103710381039104010411042104310441045104610471048104910501051105210531054105510561057105810591060106110621063106410651066106710681069
  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. nconf := numOfPendingConf(ents)
  453. if nconf > 1 {
  454. panic("unexpected multiple uncommitted config entry")
  455. }
  456. if nconf == 1 {
  457. r.pendingConf = true
  458. }
  459. r.appendEntry(pb.Entry{Data: nil})
  460. r.logger.Infof("%x became leader at term %d", r.id, r.Term)
  461. }
  462. func (r *raft) campaign(t CampaignType) {
  463. r.becomeCandidate()
  464. if r.quorum() == r.poll(r.id, true) {
  465. r.becomeLeader()
  466. return
  467. }
  468. for id := range r.prs {
  469. if id == r.id {
  470. continue
  471. }
  472. r.logger.Infof("%x [logterm: %d, index: %d] sent vote request to %x at term %d",
  473. r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), id, r.Term)
  474. var ctx []byte
  475. if t == campaignTransfer {
  476. ctx = []byte(t)
  477. }
  478. r.send(pb.Message{To: id, Type: pb.MsgVote, Index: r.raftLog.lastIndex(), LogTerm: r.raftLog.lastTerm(), Context: ctx})
  479. }
  480. }
  481. func (r *raft) poll(id uint64, v bool) (granted int) {
  482. if v {
  483. r.logger.Infof("%x received vote from %x at term %d", r.id, id, r.Term)
  484. } else {
  485. r.logger.Infof("%x received vote rejection from %x at term %d", r.id, id, r.Term)
  486. }
  487. if _, ok := r.votes[id]; !ok {
  488. r.votes[id] = v
  489. }
  490. for _, vv := range r.votes {
  491. if vv {
  492. granted++
  493. }
  494. }
  495. return granted
  496. }
  497. func (r *raft) Step(m pb.Message) error {
  498. if m.Type == pb.MsgHup {
  499. if r.state != StateLeader {
  500. ents, err := r.raftLog.slice(r.raftLog.applied+1, r.raftLog.committed+1, noLimit)
  501. if err != nil {
  502. r.logger.Panicf("unexpected error getting unapplied entries (%v)", err)
  503. }
  504. if n := numOfPendingConf(ents); n != 0 && r.raftLog.committed > r.raftLog.applied {
  505. r.logger.Warningf("%x cannot campaign at term %d since there are still %d pending configuration changes to apply", r.id, r.Term, n)
  506. return nil
  507. }
  508. r.logger.Infof("%x is starting a new election at term %d", r.id, r.Term)
  509. r.campaign(campaignElection)
  510. } else {
  511. r.logger.Debugf("%x ignoring MsgHup because already leader", r.id)
  512. }
  513. return nil
  514. }
  515. switch {
  516. case m.Term == 0:
  517. // local message
  518. case m.Term > r.Term:
  519. lead := m.From
  520. if m.Type == pb.MsgVote {
  521. force := bytes.Equal(m.Context, []byte(campaignTransfer))
  522. inLease := r.checkQuorum && r.lead != None && r.electionElapsed < r.electionTimeout
  523. if !force && inLease {
  524. // If a server receives a RequestVote request within the minimum election timeout
  525. // of hearing from a current leader, it does not update its term or grant its vote
  526. 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)",
  527. r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term, r.electionTimeout-r.electionElapsed)
  528. return nil
  529. }
  530. lead = None
  531. }
  532. r.logger.Infof("%x [term: %d] received a %s message with higher term from %x [term: %d]",
  533. r.id, r.Term, m.Type, m.From, m.Term)
  534. r.becomeFollower(m.Term, lead)
  535. case m.Term < r.Term:
  536. if r.checkQuorum && (m.Type == pb.MsgHeartbeat || m.Type == pb.MsgApp) {
  537. // We have received messages from a leader at a lower term. It is possible that these messages were
  538. // simply delayed in the network, but this could also mean that this node has advanced its term number
  539. // during a network partition, and it is now unable to either win an election or to rejoin the majority
  540. // on the old term. If checkQuorum is false, this will be handled by incrementing term numbers in response
  541. // to MsgVote with a higher term, but if checkQuorum is true we may not advance the term on MsgVote and
  542. // must generate other messages to advance the term. The net result of these two features is to minimize
  543. // the disruption caused by nodes that have been removed from the cluster's configuration: a removed node
  544. // will send MsgVotes which will be ignored, but it will not receive MsgApp or MsgHeartbeat, so it will not
  545. // create disruptive term increases
  546. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp})
  547. } else {
  548. // ignore other cases
  549. r.logger.Infof("%x [term: %d] ignored a %s message with lower term from %x [term: %d]",
  550. r.id, r.Term, m.Type, m.From, m.Term)
  551. }
  552. return nil
  553. }
  554. r.step(r, m)
  555. return nil
  556. }
  557. type stepFunc func(r *raft, m pb.Message)
  558. func stepLeader(r *raft, m pb.Message) {
  559. // These message types do not require any progress for m.From.
  560. switch m.Type {
  561. case pb.MsgBeat:
  562. r.bcastHeartbeat()
  563. return
  564. case pb.MsgCheckQuorum:
  565. if !r.checkQuorumActive() {
  566. r.logger.Warningf("%x stepped down to follower since quorum is not active", r.id)
  567. r.becomeFollower(r.Term, None)
  568. }
  569. return
  570. case pb.MsgProp:
  571. if len(m.Entries) == 0 {
  572. r.logger.Panicf("%x stepped empty MsgProp", r.id)
  573. }
  574. if _, ok := r.prs[r.id]; !ok {
  575. // If we are not currently a member of the range (i.e. this node
  576. // was removed from the configuration while serving as leader),
  577. // drop any new proposals.
  578. return
  579. }
  580. if r.leadTransferee != None {
  581. r.logger.Debugf("%x [term %d] transfer leadership to %x is in progress; dropping proposal", r.id, r.Term, r.leadTransferee)
  582. return
  583. }
  584. for i, e := range m.Entries {
  585. if e.Type == pb.EntryConfChange {
  586. if r.pendingConf {
  587. m.Entries[i] = pb.Entry{Type: pb.EntryNormal}
  588. }
  589. r.pendingConf = true
  590. }
  591. }
  592. r.appendEntry(m.Entries...)
  593. r.bcastAppend()
  594. return
  595. case pb.MsgVote:
  596. r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %d",
  597. r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
  598. r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp, Reject: true})
  599. return
  600. case pb.MsgReadIndex:
  601. ri := None
  602. if r.checkQuorum {
  603. ri = r.raftLog.committed
  604. }
  605. if m.From == None || m.From == r.id { // from local member
  606. r.readState.Index = ri
  607. r.readState.RequestCtx = m.Entries[0].Data
  608. } else {
  609. r.send(pb.Message{To: m.From, Type: pb.MsgReadIndexResp, Index: ri, Entries: m.Entries})
  610. }
  611. return
  612. }
  613. // All other message types require a progress for m.From (pr).
  614. pr, prOk := r.prs[m.From]
  615. if !prOk {
  616. r.logger.Debugf("%x no progress available for %x", r.id, m.From)
  617. return
  618. }
  619. switch m.Type {
  620. case pb.MsgAppResp:
  621. pr.RecentActive = true
  622. if m.Reject {
  623. r.logger.Debugf("%x received msgApp rejection(lastindex: %d) from %x for index %d",
  624. r.id, m.RejectHint, m.From, m.Index)
  625. if pr.maybeDecrTo(m.Index, m.RejectHint) {
  626. r.logger.Debugf("%x decreased progress of %x to [%s]", r.id, m.From, pr)
  627. if pr.State == ProgressStateReplicate {
  628. pr.becomeProbe()
  629. }
  630. r.sendAppend(m.From)
  631. }
  632. } else {
  633. oldPaused := pr.isPaused()
  634. if pr.maybeUpdate(m.Index) {
  635. switch {
  636. case pr.State == ProgressStateProbe:
  637. pr.becomeReplicate()
  638. case pr.State == ProgressStateSnapshot && pr.needSnapshotAbort():
  639. r.logger.Debugf("%x snapshot aborted, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
  640. pr.becomeProbe()
  641. case pr.State == ProgressStateReplicate:
  642. pr.ins.freeTo(m.Index)
  643. }
  644. if r.maybeCommit() {
  645. r.bcastAppend()
  646. } else if oldPaused {
  647. // update() reset the wait state on this node. If we had delayed sending
  648. // an update before, send it now.
  649. r.sendAppend(m.From)
  650. }
  651. // Transfer leadership is in progress.
  652. if m.From == r.leadTransferee && pr.Match == r.raftLog.lastIndex() {
  653. r.logger.Infof("%x sent MsgTimeoutNow to %x after received MsgAppResp", r.id, m.From)
  654. r.sendTimeoutNow(m.From)
  655. }
  656. }
  657. }
  658. case pb.MsgHeartbeatResp:
  659. pr.RecentActive = true
  660. // free one slot for the full inflights window to allow progress.
  661. if pr.State == ProgressStateReplicate && pr.ins.full() {
  662. pr.ins.freeFirstOne()
  663. }
  664. if pr.Match < r.raftLog.lastIndex() {
  665. r.sendAppend(m.From)
  666. }
  667. case pb.MsgSnapStatus:
  668. if pr.State != ProgressStateSnapshot {
  669. return
  670. }
  671. if !m.Reject {
  672. pr.becomeProbe()
  673. r.logger.Debugf("%x snapshot succeeded, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
  674. } else {
  675. pr.snapshotFailure()
  676. pr.becomeProbe()
  677. r.logger.Debugf("%x snapshot failed, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
  678. }
  679. // If snapshot finish, wait for the msgAppResp from the remote node before sending
  680. // out the next msgApp.
  681. // If snapshot failure, wait for a heartbeat interval before next try
  682. pr.pause()
  683. case pb.MsgUnreachable:
  684. // During optimistic replication, if the remote becomes unreachable,
  685. // there is huge probability that a MsgApp is lost.
  686. if pr.State == ProgressStateReplicate {
  687. pr.becomeProbe()
  688. }
  689. r.logger.Debugf("%x failed to send message to %x because it is unreachable [%s]", r.id, m.From, pr)
  690. case pb.MsgTransferLeader:
  691. leadTransferee := m.From
  692. lastLeadTransferee := r.leadTransferee
  693. if lastLeadTransferee != None {
  694. if lastLeadTransferee == leadTransferee {
  695. r.logger.Infof("%x [term %d] transfer leadership to %x is in progress, ignores request to same node %x",
  696. r.id, r.Term, leadTransferee, leadTransferee)
  697. return
  698. }
  699. r.abortLeaderTransfer()
  700. r.logger.Infof("%x [term %d] abort previous transferring leadership to %x", r.id, r.Term, lastLeadTransferee)
  701. }
  702. if leadTransferee == r.id {
  703. r.logger.Debugf("%x is already leader. Ignored transferring leadership to self", r.id)
  704. return
  705. }
  706. // Transfer leadership to third party.
  707. r.logger.Infof("%x [term %d] starts to transfer leadership to %x", r.id, r.Term, leadTransferee)
  708. // Transfer leadership should be finished in one electionTimeout, so reset r.electionElapsed.
  709. r.electionElapsed = 0
  710. r.leadTransferee = leadTransferee
  711. if pr.Match == r.raftLog.lastIndex() {
  712. r.sendTimeoutNow(leadTransferee)
  713. r.logger.Infof("%x sends MsgTimeoutNow to %x immediately as %x already has up-to-date log", r.id, leadTransferee, leadTransferee)
  714. } else {
  715. r.sendAppend(leadTransferee)
  716. }
  717. }
  718. }
  719. func stepCandidate(r *raft, m pb.Message) {
  720. switch m.Type {
  721. case pb.MsgProp:
  722. r.logger.Infof("%x no leader at term %d; dropping proposal", r.id, r.Term)
  723. return
  724. case pb.MsgApp:
  725. r.becomeFollower(r.Term, m.From)
  726. r.handleAppendEntries(m)
  727. case pb.MsgHeartbeat:
  728. r.becomeFollower(r.Term, m.From)
  729. r.handleHeartbeat(m)
  730. case pb.MsgSnap:
  731. r.becomeFollower(m.Term, m.From)
  732. r.handleSnapshot(m)
  733. case pb.MsgVote:
  734. r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %d",
  735. r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
  736. r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp, Reject: true})
  737. case pb.MsgVoteResp:
  738. gr := r.poll(m.From, !m.Reject)
  739. r.logger.Infof("%x [quorum:%d] has received %d votes and %d vote rejections", r.id, r.quorum(), gr, len(r.votes)-gr)
  740. switch r.quorum() {
  741. case gr:
  742. r.becomeLeader()
  743. r.bcastAppend()
  744. case len(r.votes) - gr:
  745. r.becomeFollower(r.Term, None)
  746. }
  747. case pb.MsgTimeoutNow:
  748. r.logger.Debugf("%x [term %d state %v] ignored MsgTimeoutNow from %x", r.id, r.Term, r.state, m.From)
  749. }
  750. }
  751. func stepFollower(r *raft, m pb.Message) {
  752. switch m.Type {
  753. case pb.MsgProp:
  754. if r.lead == None {
  755. r.logger.Infof("%x no leader at term %d; dropping proposal", r.id, r.Term)
  756. return
  757. }
  758. m.To = r.lead
  759. r.send(m)
  760. case pb.MsgApp:
  761. r.electionElapsed = 0
  762. r.lead = m.From
  763. r.handleAppendEntries(m)
  764. case pb.MsgHeartbeat:
  765. r.electionElapsed = 0
  766. r.lead = m.From
  767. r.handleHeartbeat(m)
  768. case pb.MsgSnap:
  769. r.electionElapsed = 0
  770. r.lead = m.From
  771. r.handleSnapshot(m)
  772. case pb.MsgVote:
  773. if (r.Vote == None || r.Vote == m.From) && r.raftLog.isUpToDate(m.Index, m.LogTerm) {
  774. r.electionElapsed = 0
  775. r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] voted for %x [logterm: %d, index: %d] at term %d",
  776. r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
  777. r.Vote = m.From
  778. r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp})
  779. } else {
  780. r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %d",
  781. r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
  782. r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp, Reject: true})
  783. }
  784. case pb.MsgTransferLeader:
  785. if r.lead == None {
  786. r.logger.Infof("%x no leader at term %d; dropping leader transfer msg", r.id, r.Term)
  787. return
  788. }
  789. m.To = r.lead
  790. r.send(m)
  791. case pb.MsgTimeoutNow:
  792. r.logger.Infof("%x [term %d] received MsgTimeoutNow from %x and starts an election to get leadership.", r.id, r.Term, m.From)
  793. r.campaign(campaignTransfer)
  794. case pb.MsgReadIndex:
  795. if r.lead == None {
  796. r.logger.Infof("%x no leader at term %d; dropping index reading msg", r.id, r.Term)
  797. return
  798. }
  799. m.To = r.lead
  800. r.send(m)
  801. case pb.MsgReadIndexResp:
  802. if len(m.Entries) != 1 {
  803. r.logger.Errorf("%x invalid format of MsgReadIndexResp from %x, entries count: %d", r.id, m.From, len(m.Entries))
  804. return
  805. }
  806. r.readState.Index = m.Index
  807. r.readState.RequestCtx = m.Entries[0].Data
  808. }
  809. }
  810. func (r *raft) handleAppendEntries(m pb.Message) {
  811. if m.Index < r.raftLog.committed {
  812. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed})
  813. return
  814. }
  815. if mlastIndex, ok := r.raftLog.maybeAppend(m.Index, m.LogTerm, m.Commit, m.Entries...); ok {
  816. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: mlastIndex})
  817. } else {
  818. r.logger.Debugf("%x [logterm: %d, index: %d] rejected msgApp [logterm: %d, index: %d] from %x",
  819. r.id, r.raftLog.zeroTermOnErrCompacted(r.raftLog.term(m.Index)), m.Index, m.LogTerm, m.Index, m.From)
  820. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: m.Index, Reject: true, RejectHint: r.raftLog.lastIndex()})
  821. }
  822. }
  823. func (r *raft) handleHeartbeat(m pb.Message) {
  824. r.raftLog.commitTo(m.Commit)
  825. r.send(pb.Message{To: m.From, Type: pb.MsgHeartbeatResp})
  826. }
  827. func (r *raft) handleSnapshot(m pb.Message) {
  828. sindex, sterm := m.Snapshot.Metadata.Index, m.Snapshot.Metadata.Term
  829. if r.restore(m.Snapshot) {
  830. r.logger.Infof("%x [commit: %d] restored snapshot [index: %d, term: %d]",
  831. r.id, r.raftLog.committed, sindex, sterm)
  832. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.lastIndex()})
  833. } else {
  834. r.logger.Infof("%x [commit: %d] ignored snapshot [index: %d, term: %d]",
  835. r.id, r.raftLog.committed, sindex, sterm)
  836. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed})
  837. }
  838. }
  839. // restore recovers the state machine from a snapshot. It restores the log and the
  840. // configuration of state machine.
  841. func (r *raft) restore(s pb.Snapshot) bool {
  842. if s.Metadata.Index <= r.raftLog.committed {
  843. return false
  844. }
  845. if r.raftLog.matchTerm(s.Metadata.Index, s.Metadata.Term) {
  846. r.logger.Infof("%x [commit: %d, lastindex: %d, lastterm: %d] fast-forwarded commit to snapshot [index: %d, term: %d]",
  847. r.id, r.raftLog.committed, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
  848. r.raftLog.commitTo(s.Metadata.Index)
  849. return false
  850. }
  851. r.logger.Infof("%x [commit: %d, lastindex: %d, lastterm: %d] starts to restore snapshot [index: %d, term: %d]",
  852. r.id, r.raftLog.committed, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
  853. r.raftLog.restore(s)
  854. r.prs = make(map[uint64]*Progress)
  855. for _, n := range s.Metadata.ConfState.Nodes {
  856. match, next := uint64(0), r.raftLog.lastIndex()+1
  857. if n == r.id {
  858. match = next - 1
  859. }
  860. r.setProgress(n, match, next)
  861. r.logger.Infof("%x restored progress of %x [%s]", r.id, n, r.prs[n])
  862. }
  863. return true
  864. }
  865. // promotable indicates whether state machine can be promoted to leader,
  866. // which is true when its own id is in progress list.
  867. func (r *raft) promotable() bool {
  868. _, ok := r.prs[r.id]
  869. return ok
  870. }
  871. func (r *raft) addNode(id uint64) {
  872. if _, ok := r.prs[id]; ok {
  873. // Ignore any redundant addNode calls (which can happen because the
  874. // initial bootstrapping entries are applied twice).
  875. return
  876. }
  877. r.setProgress(id, 0, r.raftLog.lastIndex()+1)
  878. r.pendingConf = false
  879. }
  880. func (r *raft) removeNode(id uint64) {
  881. r.delProgress(id)
  882. r.pendingConf = false
  883. // do not try to commit or abort transferring if there is no nodes in the cluster.
  884. if len(r.prs) == 0 {
  885. return
  886. }
  887. // The quorum size is now smaller, so see if any pending entries can
  888. // be committed.
  889. if r.maybeCommit() {
  890. r.bcastAppend()
  891. }
  892. // If the removed node is the leadTransferee, then abort the leadership transferring.
  893. if r.state == StateLeader && r.leadTransferee == id {
  894. r.abortLeaderTransfer()
  895. }
  896. }
  897. func (r *raft) resetPendingConf() { r.pendingConf = false }
  898. func (r *raft) setProgress(id, match, next uint64) {
  899. r.prs[id] = &Progress{Next: next, Match: match, ins: newInflights(r.maxInflight)}
  900. }
  901. func (r *raft) delProgress(id uint64) {
  902. delete(r.prs, id)
  903. }
  904. func (r *raft) loadState(state pb.HardState) {
  905. if state.Commit < r.raftLog.committed || state.Commit > r.raftLog.lastIndex() {
  906. r.logger.Panicf("%x state.commit %d is out of range [%d, %d]", r.id, state.Commit, r.raftLog.committed, r.raftLog.lastIndex())
  907. }
  908. r.raftLog.committed = state.Commit
  909. r.Term = state.Term
  910. r.Vote = state.Vote
  911. }
  912. // pastElectionTimeout returns true iff r.electionElapsed is greater
  913. // than or equal to the randomized election timeout in
  914. // [electiontimeout, 2 * electiontimeout - 1].
  915. func (r *raft) pastElectionTimeout() bool {
  916. return r.electionElapsed >= r.randomizedElectionTimeout
  917. }
  918. func (r *raft) resetRandomizedElectionTimeout() {
  919. r.randomizedElectionTimeout = r.electionTimeout + r.rand.Intn(r.electionTimeout)
  920. }
  921. // checkQuorumActive returns true if the quorum is active from
  922. // the view of the local raft state machine. Otherwise, it returns
  923. // false.
  924. // checkQuorumActive also resets all RecentActive to false.
  925. func (r *raft) checkQuorumActive() bool {
  926. var act int
  927. for id := range r.prs {
  928. if id == r.id { // self is always active
  929. act++
  930. continue
  931. }
  932. if r.prs[id].RecentActive {
  933. act++
  934. }
  935. r.prs[id].RecentActive = false
  936. }
  937. return act >= r.quorum()
  938. }
  939. func (r *raft) sendTimeoutNow(to uint64) {
  940. r.send(pb.Message{To: to, Type: pb.MsgTimeoutNow})
  941. }
  942. func (r *raft) abortLeaderTransfer() {
  943. r.leadTransferee = None
  944. }
  945. func numOfPendingConf(ents []pb.Entry) int {
  946. n := 0
  947. for i := range ents {
  948. if ents[i].Type == pb.EntryConfChange {
  949. n++
  950. }
  951. }
  952. return n
  953. }