raft.go 35 KB

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