raft.go 41 KB

1234567891011121314151617181920212223242526272829303132333435363738394041424344454647484950515253545556575859606162636465666768697071727374757677787980818283848586878889909192939495969798991001011021031041051061071081091101111121131141151161171181191201211221231241251261271281291301311321331341351361371381391401411421431441451461471481491501511521531541551561571581591601611621631641651661671681691701711721731741751761771781791801811821831841851861871881891901911921931941951961971981992002012022032042052062072082092102112122132142152162172182192202212222232242252262272282292302312322332342352362372382392402412422432442452462472482492502512522532542552562572582592602612622632642652662672682692702712722732742752762772782792802812822832842852862872882892902912922932942952962972982993003013023033043053063073083093103113123133143153163173183193203213223233243253263273283293303313323333343353363373383393403413423433443453463473483493503513523533543553563573583593603613623633643653663673683693703713723733743753763773783793803813823833843853863873883893903913923933943953963973983994004014024034044054064074084094104114124134144154164174184194204214224234244254264274284294304314324334344354364374384394404414424434444454464474484494504514524534544554564574584594604614624634644654664674684694704714724734744754764774784794804814824834844854864874884894904914924934944954964974984995005015025035045055065075085095105115125135145155165175185195205215225235245255265275285295305315325335345355365375385395405415425435445455465475485495505515525535545555565575585595605615625635645655665675685695705715725735745755765775785795805815825835845855865875885895905915925935945955965975985996006016026036046056066076086096106116126136146156166176186196206216226236246256266276286296306316326336346356366376386396406416426436446456466476486496506516526536546556566576586596606616626636646656666676686696706716726736746756766776786796806816826836846856866876886896906916926936946956966976986997007017027037047057067077087097107117127137147157167177187197207217227237247257267277287297307317327337347357367377387397407417427437447457467477487497507517527537547557567577587597607617627637647657667677687697707717727737747757767777787797807817827837847857867877887897907917927937947957967977987998008018028038048058068078088098108118128138148158168178188198208218228238248258268278288298308318328338348358368378388398408418428438448458468478488498508518528538548558568578588598608618628638648658668678688698708718728738748758768778788798808818828838848858868878888898908918928938948958968978988999009019029039049059069079089099109119129139149159169179189199209219229239249259269279289299309319329339349359369379389399409419429439449459469479489499509519529539549559569579589599609619629639649659669679689699709719729739749759769779789799809819829839849859869879889899909919929939949959969979989991000100110021003100410051006100710081009101010111012101310141015101610171018101910201021102210231024102510261027102810291030103110321033103410351036103710381039104010411042104310441045104610471048104910501051105210531054105510561057105810591060106110621063106410651066106710681069107010711072107310741075107610771078107910801081108210831084108510861087108810891090109110921093109410951096109710981099110011011102110311041105110611071108110911101111111211131114111511161117111811191120112111221123112411251126112711281129113011311132113311341135113611371138113911401141114211431144114511461147114811491150115111521153115411551156115711581159116011611162116311641165116611671168116911701171117211731174117511761177117811791180118111821183118411851186118711881189119011911192119311941195119611971198119912001201120212031204120512061207120812091210121112121213121412151216121712181219122012211222122312241225122612271228122912301231123212331234123512361237123812391240124112421243124412451246124712481249125012511252125312541255125612571258125912601261126212631264126512661267126812691270127112721273127412751276127712781279128012811282128312841285128612871288128912901291
  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.tick = r.tickElection
  534. r.state = StatePreCandidate
  535. r.logger.Infof("%x became pre-candidate at term %d", r.id, r.Term)
  536. }
  537. func (r *raft) becomeLeader() {
  538. // TODO(xiangli) remove the panic when the raft implementation is stable
  539. if r.state == StateFollower {
  540. panic("invalid transition [follower -> leader]")
  541. }
  542. r.step = stepLeader
  543. r.reset(r.Term)
  544. r.tick = r.tickHeartbeat
  545. r.lead = r.id
  546. r.state = StateLeader
  547. ents, err := r.raftLog.entries(r.raftLog.committed+1, noLimit)
  548. if err != nil {
  549. r.logger.Panicf("unexpected error getting uncommitted entries (%v)", err)
  550. }
  551. nconf := numOfPendingConf(ents)
  552. if nconf > 1 {
  553. panic("unexpected multiple uncommitted config entry")
  554. }
  555. if nconf == 1 {
  556. r.pendingConf = true
  557. }
  558. r.appendEntry(pb.Entry{Data: nil})
  559. r.logger.Infof("%x became leader at term %d", r.id, r.Term)
  560. }
  561. func (r *raft) campaign(t CampaignType) {
  562. var term uint64
  563. var voteMsg pb.MessageType
  564. if t == campaignPreElection {
  565. r.becomePreCandidate()
  566. voteMsg = pb.MsgPreVote
  567. // PreVote RPCs are sent for the next term before we've incremented r.Term.
  568. term = r.Term + 1
  569. } else {
  570. r.becomeCandidate()
  571. voteMsg = pb.MsgVote
  572. term = r.Term
  573. }
  574. if r.quorum() == r.poll(r.id, voteRespMsgType(voteMsg), true) {
  575. // We won the election after voting for ourselves (which must mean that
  576. // this is a single-node cluster). Advance to the next state.
  577. if t == campaignPreElection {
  578. r.campaign(campaignElection)
  579. } else {
  580. r.becomeLeader()
  581. }
  582. return
  583. }
  584. for id := range r.prs {
  585. if id == r.id {
  586. continue
  587. }
  588. r.logger.Infof("%x [logterm: %d, index: %d] sent %s request to %x at term %d",
  589. r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), voteMsg, id, r.Term)
  590. var ctx []byte
  591. if t == campaignTransfer {
  592. ctx = []byte(t)
  593. }
  594. r.send(pb.Message{Term: term, To: id, Type: voteMsg, Index: r.raftLog.lastIndex(), LogTerm: r.raftLog.lastTerm(), Context: ctx})
  595. }
  596. }
  597. func (r *raft) poll(id uint64, t pb.MessageType, v bool) (granted int) {
  598. if v {
  599. r.logger.Infof("%x received %s from %x at term %d", r.id, t, id, r.Term)
  600. } else {
  601. r.logger.Infof("%x received %s rejection from %x at term %d", r.id, t, id, r.Term)
  602. }
  603. if _, ok := r.votes[id]; !ok {
  604. r.votes[id] = v
  605. }
  606. for _, vv := range r.votes {
  607. if vv {
  608. granted++
  609. }
  610. }
  611. return granted
  612. }
  613. func (r *raft) Step(m pb.Message) error {
  614. // Handle the message term, which may result in our stepping down to a follower.
  615. switch {
  616. case m.Term == 0:
  617. // local message
  618. case m.Term > r.Term:
  619. lead := m.From
  620. if m.Type == pb.MsgVote || m.Type == pb.MsgPreVote {
  621. force := bytes.Equal(m.Context, []byte(campaignTransfer))
  622. inLease := r.checkQuorum && r.lead != None && r.electionElapsed < r.electionTimeout
  623. if !force && inLease {
  624. // If a server receives a RequestVote request within the minimum election timeout
  625. // of hearing from a current leader, it does not update its term or grant its vote
  626. 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)",
  627. r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term, r.electionTimeout-r.electionElapsed)
  628. return nil
  629. }
  630. lead = None
  631. }
  632. switch {
  633. case m.Type == pb.MsgPreVote:
  634. // Never change our term in response to a PreVote
  635. case m.Type == pb.MsgPreVoteResp && !m.Reject:
  636. // We send pre-vote requests with a term in our future. If the
  637. // pre-vote is granted, we will increment our term when we get a
  638. // quorum. If it is not, the term comes from the node that
  639. // rejected our vote so we should become a follower at the new
  640. // term.
  641. default:
  642. r.logger.Infof("%x [term: %d] received a %s message with higher term from %x [term: %d]",
  643. r.id, r.Term, m.Type, m.From, m.Term)
  644. r.becomeFollower(m.Term, lead)
  645. }
  646. case m.Term < r.Term:
  647. if r.checkQuorum && (m.Type == pb.MsgHeartbeat || m.Type == pb.MsgApp) {
  648. // We have received messages from a leader at a lower term. It is possible
  649. // that these messages were simply delayed in the network, but this could
  650. // also mean that this node has advanced its term number during a network
  651. // partition, and it is now unable to either win an election or to rejoin
  652. // the majority on the old term. If checkQuorum is false, this will be
  653. // handled by incrementing term numbers in response to MsgVote with a
  654. // higher term, but if checkQuorum is true we may not advance the term on
  655. // MsgVote and must generate other messages to advance the term. The net
  656. // result of these two features is to minimize the disruption caused by
  657. // nodes that have been removed from the cluster's configuration: a
  658. // removed node will send MsgVotes (or MsgPreVotes) which will be ignored,
  659. // but it will not receive MsgApp or MsgHeartbeat, so it will not create
  660. // disruptive term increases
  661. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp})
  662. } else {
  663. // ignore other cases
  664. r.logger.Infof("%x [term: %d] ignored a %s message with lower term from %x [term: %d]",
  665. r.id, r.Term, m.Type, m.From, m.Term)
  666. }
  667. return nil
  668. }
  669. switch m.Type {
  670. case pb.MsgHup:
  671. if r.state != StateLeader {
  672. ents, err := r.raftLog.slice(r.raftLog.applied+1, r.raftLog.committed+1, noLimit)
  673. if err != nil {
  674. r.logger.Panicf("unexpected error getting unapplied entries (%v)", err)
  675. }
  676. if n := numOfPendingConf(ents); n != 0 && r.raftLog.committed > r.raftLog.applied {
  677. r.logger.Warningf("%x cannot campaign at term %d since there are still %d pending configuration changes to apply", r.id, r.Term, n)
  678. return nil
  679. }
  680. r.logger.Infof("%x is starting a new election at term %d", r.id, r.Term)
  681. if r.preVote {
  682. r.campaign(campaignPreElection)
  683. } else {
  684. r.campaign(campaignElection)
  685. }
  686. } else {
  687. r.logger.Debugf("%x ignoring MsgHup because already leader", r.id)
  688. }
  689. case pb.MsgVote, pb.MsgPreVote:
  690. // The m.Term > r.Term clause is for MsgPreVote. For MsgVote m.Term should
  691. // always equal r.Term.
  692. if (r.Vote == None || m.Term > r.Term || r.Vote == m.From) && r.raftLog.isUpToDate(m.Index, m.LogTerm) {
  693. r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] cast %s for %x [logterm: %d, index: %d] at term %d",
  694. r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term)
  695. // When responding to Msg{Pre,}Vote messages we include the term
  696. // from the message, not the local term. To see why consider the
  697. // case where a single node was previously partitioned away and
  698. // it's local term is now of date. If we include the local term
  699. // (recall that for pre-votes we don't update the local term), the
  700. // (pre-)campaigning node on the other end will proceed to ignore
  701. // the message (it ignores all out of date messages).
  702. // The term in the original message and current local term are the
  703. // same in the case of regular votes, but different for pre-votes.
  704. r.send(pb.Message{To: m.From, Term: m.Term, Type: voteRespMsgType(m.Type)})
  705. if m.Type == pb.MsgVote {
  706. // Only record real votes.
  707. r.electionElapsed = 0
  708. r.Vote = m.From
  709. }
  710. } else {
  711. r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected %s from %x [logterm: %d, index: %d] at term %d",
  712. r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term)
  713. r.send(pb.Message{To: m.From, Term: r.Term, Type: voteRespMsgType(m.Type), Reject: true})
  714. }
  715. default:
  716. r.step(r, m)
  717. }
  718. return nil
  719. }
  720. type stepFunc func(r *raft, m pb.Message)
  721. func stepLeader(r *raft, m pb.Message) {
  722. // These message types do not require any progress for m.From.
  723. switch m.Type {
  724. case pb.MsgBeat:
  725. r.bcastHeartbeat()
  726. return
  727. case pb.MsgCheckQuorum:
  728. if !r.checkQuorumActive() {
  729. r.logger.Warningf("%x stepped down to follower since quorum is not active", r.id)
  730. r.becomeFollower(r.Term, None)
  731. }
  732. return
  733. case pb.MsgProp:
  734. if len(m.Entries) == 0 {
  735. r.logger.Panicf("%x stepped empty MsgProp", r.id)
  736. }
  737. if _, ok := r.prs[r.id]; !ok {
  738. // If we are not currently a member of the range (i.e. this node
  739. // was removed from the configuration while serving as leader),
  740. // drop any new proposals.
  741. return
  742. }
  743. if r.leadTransferee != None {
  744. r.logger.Debugf("%x [term %d] transfer leadership to %x is in progress; dropping proposal", r.id, r.Term, r.leadTransferee)
  745. return
  746. }
  747. for i, e := range m.Entries {
  748. if e.Type == pb.EntryConfChange {
  749. if r.pendingConf {
  750. r.logger.Infof("propose conf %s ignored since pending unapplied configuration", e.String())
  751. m.Entries[i] = pb.Entry{Type: pb.EntryNormal}
  752. }
  753. r.pendingConf = true
  754. }
  755. }
  756. r.appendEntry(m.Entries...)
  757. r.bcastAppend()
  758. return
  759. case pb.MsgReadIndex:
  760. if r.quorum() > 1 {
  761. if r.raftLog.zeroTermOnErrCompacted(r.raftLog.term(r.raftLog.committed)) != r.Term {
  762. // Reject read only request when this leader has not committed any log entry at its term.
  763. return
  764. }
  765. // thinking: use an interally defined context instead of the user given context.
  766. // We can express this in terms of the term and index instead of a user-supplied value.
  767. // This would allow multiple reads to piggyback on the same message.
  768. switch r.readOnly.option {
  769. case ReadOnlySafe:
  770. r.readOnly.addRequest(r.raftLog.committed, m)
  771. r.bcastHeartbeatWithCtx(m.Entries[0].Data)
  772. case ReadOnlyLeaseBased:
  773. var ri uint64
  774. if r.checkQuorum {
  775. ri = r.raftLog.committed
  776. }
  777. if m.From == None || m.From == r.id { // from local member
  778. r.readStates = append(r.readStates, ReadState{Index: r.raftLog.committed, RequestCtx: m.Entries[0].Data})
  779. } else {
  780. r.send(pb.Message{To: m.From, Type: pb.MsgReadIndexResp, Index: ri, Entries: m.Entries})
  781. }
  782. }
  783. } else {
  784. r.readStates = append(r.readStates, ReadState{Index: r.raftLog.committed, RequestCtx: m.Entries[0].Data})
  785. }
  786. return
  787. }
  788. // All other message types require a progress for m.From (pr).
  789. pr, prOk := r.prs[m.From]
  790. if !prOk {
  791. r.logger.Debugf("%x no progress available for %x", r.id, m.From)
  792. return
  793. }
  794. switch m.Type {
  795. case pb.MsgAppResp:
  796. pr.RecentActive = true
  797. if m.Reject {
  798. r.logger.Debugf("%x received msgApp rejection(lastindex: %d) from %x for index %d",
  799. r.id, m.RejectHint, m.From, m.Index)
  800. if pr.maybeDecrTo(m.Index, m.RejectHint) {
  801. r.logger.Debugf("%x decreased progress of %x to [%s]", r.id, m.From, pr)
  802. if pr.State == ProgressStateReplicate {
  803. pr.becomeProbe()
  804. }
  805. r.sendAppend(m.From)
  806. }
  807. } else {
  808. oldPaused := pr.IsPaused()
  809. if pr.maybeUpdate(m.Index) {
  810. switch {
  811. case pr.State == ProgressStateProbe:
  812. pr.becomeReplicate()
  813. case pr.State == ProgressStateSnapshot && pr.needSnapshotAbort():
  814. r.logger.Debugf("%x snapshot aborted, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
  815. pr.becomeProbe()
  816. case pr.State == ProgressStateReplicate:
  817. pr.ins.freeTo(m.Index)
  818. }
  819. if r.maybeCommit() {
  820. r.bcastAppend()
  821. } else if oldPaused {
  822. // update() reset the wait state on this node. If we had delayed sending
  823. // an update before, send it now.
  824. r.sendAppend(m.From)
  825. }
  826. // Transfer leadership is in progress.
  827. if m.From == r.leadTransferee && pr.Match == r.raftLog.lastIndex() {
  828. r.logger.Infof("%x sent MsgTimeoutNow to %x after received MsgAppResp", r.id, m.From)
  829. r.sendTimeoutNow(m.From)
  830. }
  831. }
  832. }
  833. case pb.MsgHeartbeatResp:
  834. pr.RecentActive = true
  835. pr.resume()
  836. // free one slot for the full inflights window to allow progress.
  837. if pr.State == ProgressStateReplicate && pr.ins.full() {
  838. pr.ins.freeFirstOne()
  839. }
  840. if pr.Match < r.raftLog.lastIndex() {
  841. r.sendAppend(m.From)
  842. }
  843. if r.readOnly.option != ReadOnlySafe || len(m.Context) == 0 {
  844. return
  845. }
  846. ackCount := r.readOnly.recvAck(m)
  847. if ackCount < r.quorum() {
  848. return
  849. }
  850. rss := r.readOnly.advance(m)
  851. for _, rs := range rss {
  852. req := rs.req
  853. if req.From == None || req.From == r.id { // from local member
  854. r.readStates = append(r.readStates, ReadState{Index: rs.index, RequestCtx: req.Entries[0].Data})
  855. } else {
  856. r.send(pb.Message{To: req.From, Type: pb.MsgReadIndexResp, Index: rs.index, Entries: req.Entries})
  857. }
  858. }
  859. case pb.MsgSnapStatus:
  860. if pr.State != ProgressStateSnapshot {
  861. return
  862. }
  863. if !m.Reject {
  864. pr.becomeProbe()
  865. r.logger.Debugf("%x snapshot succeeded, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
  866. } else {
  867. pr.snapshotFailure()
  868. pr.becomeProbe()
  869. r.logger.Debugf("%x snapshot failed, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
  870. }
  871. // If snapshot finish, wait for the msgAppResp from the remote node before sending
  872. // out the next msgApp.
  873. // If snapshot failure, wait for a heartbeat interval before next try
  874. pr.pause()
  875. case pb.MsgUnreachable:
  876. // During optimistic replication, if the remote becomes unreachable,
  877. // there is huge probability that a MsgApp is lost.
  878. if pr.State == ProgressStateReplicate {
  879. pr.becomeProbe()
  880. }
  881. r.logger.Debugf("%x failed to send message to %x because it is unreachable [%s]", r.id, m.From, pr)
  882. case pb.MsgTransferLeader:
  883. leadTransferee := m.From
  884. lastLeadTransferee := r.leadTransferee
  885. if lastLeadTransferee != None {
  886. if lastLeadTransferee == leadTransferee {
  887. r.logger.Infof("%x [term %d] transfer leadership to %x is in progress, ignores request to same node %x",
  888. r.id, r.Term, leadTransferee, leadTransferee)
  889. return
  890. }
  891. r.abortLeaderTransfer()
  892. r.logger.Infof("%x [term %d] abort previous transferring leadership to %x", r.id, r.Term, lastLeadTransferee)
  893. }
  894. if leadTransferee == r.id {
  895. r.logger.Debugf("%x is already leader. Ignored transferring leadership to self", r.id)
  896. return
  897. }
  898. // Transfer leadership to third party.
  899. r.logger.Infof("%x [term %d] starts to transfer leadership to %x", r.id, r.Term, leadTransferee)
  900. // Transfer leadership should be finished in one electionTimeout, so reset r.electionElapsed.
  901. r.electionElapsed = 0
  902. r.leadTransferee = leadTransferee
  903. if pr.Match == r.raftLog.lastIndex() {
  904. r.sendTimeoutNow(leadTransferee)
  905. r.logger.Infof("%x sends MsgTimeoutNow to %x immediately as %x already has up-to-date log", r.id, leadTransferee, leadTransferee)
  906. } else {
  907. r.sendAppend(leadTransferee)
  908. }
  909. }
  910. }
  911. // stepCandidate is shared by StateCandidate and StatePreCandidate; the difference is
  912. // whether they respond to MsgVoteResp or MsgPreVoteResp.
  913. func stepCandidate(r *raft, m pb.Message) {
  914. // Only handle vote responses corresponding to our candidacy (while in
  915. // StateCandidate, we may get stale MsgPreVoteResp messages in this term from
  916. // our pre-candidate state).
  917. var myVoteRespType pb.MessageType
  918. if r.state == StatePreCandidate {
  919. myVoteRespType = pb.MsgPreVoteResp
  920. } else {
  921. myVoteRespType = pb.MsgVoteResp
  922. }
  923. switch m.Type {
  924. case pb.MsgProp:
  925. r.logger.Infof("%x no leader at term %d; dropping proposal", r.id, r.Term)
  926. return
  927. case pb.MsgApp:
  928. r.becomeFollower(r.Term, m.From)
  929. r.handleAppendEntries(m)
  930. case pb.MsgHeartbeat:
  931. r.becomeFollower(r.Term, m.From)
  932. r.handleHeartbeat(m)
  933. case pb.MsgSnap:
  934. r.becomeFollower(m.Term, m.From)
  935. r.handleSnapshot(m)
  936. case myVoteRespType:
  937. gr := r.poll(m.From, m.Type, !m.Reject)
  938. 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)
  939. switch r.quorum() {
  940. case gr:
  941. if r.state == StatePreCandidate {
  942. r.campaign(campaignElection)
  943. } else {
  944. r.becomeLeader()
  945. r.bcastAppend()
  946. }
  947. case len(r.votes) - gr:
  948. r.becomeFollower(r.Term, None)
  949. }
  950. case pb.MsgTimeoutNow:
  951. r.logger.Debugf("%x [term %d state %v] ignored MsgTimeoutNow from %x", r.id, r.Term, r.state, m.From)
  952. }
  953. }
  954. func stepFollower(r *raft, m pb.Message) {
  955. switch m.Type {
  956. case pb.MsgProp:
  957. if r.lead == None {
  958. r.logger.Infof("%x no leader at term %d; dropping proposal", r.id, r.Term)
  959. return
  960. } else if r.disableProposalForwarding {
  961. r.logger.Infof("%x not forwarding to leader %x at term %d; dropping proposal", r.id, r.lead, r.Term)
  962. return
  963. }
  964. m.To = r.lead
  965. r.send(m)
  966. case pb.MsgApp:
  967. r.electionElapsed = 0
  968. r.lead = m.From
  969. r.handleAppendEntries(m)
  970. case pb.MsgHeartbeat:
  971. r.electionElapsed = 0
  972. r.lead = m.From
  973. r.handleHeartbeat(m)
  974. case pb.MsgSnap:
  975. r.electionElapsed = 0
  976. r.lead = m.From
  977. r.handleSnapshot(m)
  978. case pb.MsgTransferLeader:
  979. if r.lead == None {
  980. r.logger.Infof("%x no leader at term %d; dropping leader transfer msg", r.id, r.Term)
  981. return
  982. }
  983. m.To = r.lead
  984. r.send(m)
  985. case pb.MsgTimeoutNow:
  986. if r.promotable() {
  987. r.logger.Infof("%x [term %d] received MsgTimeoutNow from %x and starts an election to get leadership.", r.id, r.Term, m.From)
  988. // Leadership transfers never use pre-vote even if r.preVote is true; we
  989. // know we are not recovering from a partition so there is no need for the
  990. // extra round trip.
  991. r.campaign(campaignTransfer)
  992. } else {
  993. r.logger.Infof("%x received MsgTimeoutNow from %x but is not promotable", r.id, m.From)
  994. }
  995. case pb.MsgReadIndex:
  996. if r.lead == None {
  997. r.logger.Infof("%x no leader at term %d; dropping index reading msg", r.id, r.Term)
  998. return
  999. }
  1000. m.To = r.lead
  1001. r.send(m)
  1002. case pb.MsgReadIndexResp:
  1003. if len(m.Entries) != 1 {
  1004. r.logger.Errorf("%x invalid format of MsgReadIndexResp from %x, entries count: %d", r.id, m.From, len(m.Entries))
  1005. return
  1006. }
  1007. r.readStates = append(r.readStates, ReadState{Index: m.Index, RequestCtx: m.Entries[0].Data})
  1008. }
  1009. }
  1010. func (r *raft) handleAppendEntries(m pb.Message) {
  1011. if m.Index < r.raftLog.committed {
  1012. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed})
  1013. return
  1014. }
  1015. if mlastIndex, ok := r.raftLog.maybeAppend(m.Index, m.LogTerm, m.Commit, m.Entries...); ok {
  1016. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: mlastIndex})
  1017. } else {
  1018. r.logger.Debugf("%x [logterm: %d, index: %d] rejected msgApp [logterm: %d, index: %d] from %x",
  1019. r.id, r.raftLog.zeroTermOnErrCompacted(r.raftLog.term(m.Index)), m.Index, m.LogTerm, m.Index, m.From)
  1020. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: m.Index, Reject: true, RejectHint: r.raftLog.lastIndex()})
  1021. }
  1022. }
  1023. func (r *raft) handleHeartbeat(m pb.Message) {
  1024. r.raftLog.commitTo(m.Commit)
  1025. r.send(pb.Message{To: m.From, Type: pb.MsgHeartbeatResp, Context: m.Context})
  1026. }
  1027. func (r *raft) handleSnapshot(m pb.Message) {
  1028. sindex, sterm := m.Snapshot.Metadata.Index, m.Snapshot.Metadata.Term
  1029. if r.restore(m.Snapshot) {
  1030. r.logger.Infof("%x [commit: %d] restored snapshot [index: %d, term: %d]",
  1031. r.id, r.raftLog.committed, sindex, sterm)
  1032. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.lastIndex()})
  1033. } else {
  1034. r.logger.Infof("%x [commit: %d] ignored snapshot [index: %d, term: %d]",
  1035. r.id, r.raftLog.committed, sindex, sterm)
  1036. r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed})
  1037. }
  1038. }
  1039. // restore recovers the state machine from a snapshot. It restores the log and the
  1040. // configuration of state machine.
  1041. func (r *raft) restore(s pb.Snapshot) bool {
  1042. if s.Metadata.Index <= r.raftLog.committed {
  1043. return false
  1044. }
  1045. if r.raftLog.matchTerm(s.Metadata.Index, s.Metadata.Term) {
  1046. r.logger.Infof("%x [commit: %d, lastindex: %d, lastterm: %d] fast-forwarded commit to snapshot [index: %d, term: %d]",
  1047. r.id, r.raftLog.committed, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
  1048. r.raftLog.commitTo(s.Metadata.Index)
  1049. return false
  1050. }
  1051. r.logger.Infof("%x [commit: %d, lastindex: %d, lastterm: %d] starts to restore snapshot [index: %d, term: %d]",
  1052. r.id, r.raftLog.committed, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
  1053. r.raftLog.restore(s)
  1054. r.prs = make(map[uint64]*Progress)
  1055. for _, n := range s.Metadata.ConfState.Nodes {
  1056. match, next := uint64(0), r.raftLog.lastIndex()+1
  1057. if n == r.id {
  1058. match = next - 1
  1059. }
  1060. r.setProgress(n, match, next)
  1061. r.logger.Infof("%x restored progress of %x [%s]", r.id, n, r.prs[n])
  1062. }
  1063. return true
  1064. }
  1065. // promotable indicates whether state machine can be promoted to leader,
  1066. // which is true when its own id is in progress list.
  1067. func (r *raft) promotable() bool {
  1068. _, ok := r.prs[r.id]
  1069. return ok
  1070. }
  1071. func (r *raft) addNode(id uint64) {
  1072. r.pendingConf = false
  1073. if _, ok := r.prs[id]; ok {
  1074. // Ignore any redundant addNode calls (which can happen because the
  1075. // initial bootstrapping entries are applied twice).
  1076. return
  1077. }
  1078. r.setProgress(id, 0, r.raftLog.lastIndex()+1)
  1079. // When a node is first added, we should mark it as recently active.
  1080. // Otherwise, CheckQuorum may cause us to step down if it is invoked
  1081. // before the added node has a chance to communicate with us.
  1082. r.prs[id].RecentActive = true
  1083. }
  1084. func (r *raft) removeNode(id uint64) {
  1085. r.delProgress(id)
  1086. r.pendingConf = false
  1087. // do not try to commit or abort transferring if there is no nodes in the cluster.
  1088. if len(r.prs) == 0 {
  1089. return
  1090. }
  1091. // The quorum size is now smaller, so see if any pending entries can
  1092. // be committed.
  1093. if r.maybeCommit() {
  1094. r.bcastAppend()
  1095. }
  1096. // If the removed node is the leadTransferee, then abort the leadership transferring.
  1097. if r.state == StateLeader && r.leadTransferee == id {
  1098. r.abortLeaderTransfer()
  1099. }
  1100. }
  1101. func (r *raft) resetPendingConf() { r.pendingConf = false }
  1102. func (r *raft) setProgress(id, match, next uint64) {
  1103. r.prs[id] = &Progress{Next: next, Match: match, ins: newInflights(r.maxInflight)}
  1104. }
  1105. func (r *raft) delProgress(id uint64) {
  1106. delete(r.prs, id)
  1107. }
  1108. func (r *raft) loadState(state pb.HardState) {
  1109. if state.Commit < r.raftLog.committed || state.Commit > r.raftLog.lastIndex() {
  1110. r.logger.Panicf("%x state.commit %d is out of range [%d, %d]", r.id, state.Commit, r.raftLog.committed, r.raftLog.lastIndex())
  1111. }
  1112. r.raftLog.committed = state.Commit
  1113. r.Term = state.Term
  1114. r.Vote = state.Vote
  1115. }
  1116. // pastElectionTimeout returns true iff r.electionElapsed is greater
  1117. // than or equal to the randomized election timeout in
  1118. // [electiontimeout, 2 * electiontimeout - 1].
  1119. func (r *raft) pastElectionTimeout() bool {
  1120. return r.electionElapsed >= r.randomizedElectionTimeout
  1121. }
  1122. func (r *raft) resetRandomizedElectionTimeout() {
  1123. r.randomizedElectionTimeout = r.electionTimeout + globalRand.Intn(r.electionTimeout)
  1124. }
  1125. // checkQuorumActive returns true if the quorum is active from
  1126. // the view of the local raft state machine. Otherwise, it returns
  1127. // false.
  1128. // checkQuorumActive also resets all RecentActive to false.
  1129. func (r *raft) checkQuorumActive() bool {
  1130. var act int
  1131. for id := range r.prs {
  1132. if id == r.id { // self is always active
  1133. act++
  1134. continue
  1135. }
  1136. if r.prs[id].RecentActive {
  1137. act++
  1138. }
  1139. r.prs[id].RecentActive = false
  1140. }
  1141. return act >= r.quorum()
  1142. }
  1143. func (r *raft) sendTimeoutNow(to uint64) {
  1144. r.send(pb.Message{To: to, Type: pb.MsgTimeoutNow})
  1145. }
  1146. func (r *raft) abortLeaderTransfer() {
  1147. r.leadTransferee = None
  1148. }
  1149. func numOfPendingConf(ents []pb.Entry) int {
  1150. n := 0
  1151. for i := range ents {
  1152. if ents[i].Type == pb.EntryConfChange {
  1153. n++
  1154. }
  1155. }
  1156. return n
  1157. }