raft.go 38 KB

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