raft.go 44 KB

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