raft.go 44 KB

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