raft.go 56 KB

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