raft.go 52 KB

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