raft.go 41 KB

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