forked from etcd-io/raft
-
Notifications
You must be signed in to change notification settings - Fork 0
/
raft.go
2158 lines (1972 loc) · 80.2 KB
/
raft.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// Copyright 2015 The etcd Authors
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package raft
import (
"bytes"
"crypto/rand"
"errors"
"fmt"
"math"
"math/big"
"sort"
"strings"
"sync"
"go.etcd.io/raft/v3/confchange"
"go.etcd.io/raft/v3/quorum"
pb "go.etcd.io/raft/v3/raftpb"
"go.etcd.io/raft/v3/tracker"
)
const (
// None is a placeholder node ID used when there is no leader.
None uint64 = 0
// LocalAppendThread is a reference to a local thread that saves unstable
// log entries and snapshots to stable storage. The identifier is used as a
// target for MsgStorageAppend messages when AsyncStorageWrites is enabled.
LocalAppendThread uint64 = math.MaxUint64
// LocalApplyThread is a reference to a local thread that applies committed
// log entries to the local state machine. The identifier is used as a
// target for MsgStorageApply messages when AsyncStorageWrites is enabled.
LocalApplyThread uint64 = math.MaxUint64 - 1
)
// Possible values for StateType.
const (
StateFollower StateType = iota
StateCandidate
StateLeader
StatePreCandidate
numStates
)
type ReadOnlyOption int
const (
// ReadOnlySafe guarantees the linearizability of the read only request by
// communicating with the quorum. It is the default and suggested option.
ReadOnlySafe ReadOnlyOption = iota
// ReadOnlyLeaseBased ensures linearizability of the read only request by
// relying on the leader lease. It can be affected by clock drift.
// If the clock drift is unbounded, leader might keep the lease longer than it
// should (clock can move backward/pause without any bound). ReadIndex is not safe
// in that case.
ReadOnlyLeaseBased
)
// Possible values for CampaignType
const (
// campaignPreElection represents the first phase of a normal election when
// Config.PreVote is true.
campaignPreElection CampaignType = "CampaignPreElection"
// campaignElection represents a normal (time-based) election (the second phase
// of the election when Config.PreVote is true).
campaignElection CampaignType = "CampaignElection"
// campaignTransfer represents the type of leader transfer
campaignTransfer CampaignType = "CampaignTransfer"
)
const noLimit = math.MaxUint64
// ErrProposalDropped is returned when the proposal is ignored by some cases,
// so that the proposer can be notified and fail fast.
var ErrProposalDropped = errors.New("raft proposal dropped")
// lockedRand is a small wrapper around rand.Rand to provide
// synchronization among multiple raft groups. Only the methods needed
// by the code are exposed (e.g. Intn).
type lockedRand struct {
mu sync.Mutex
}
func (r *lockedRand) Intn(n int) int {
r.mu.Lock()
v, _ := rand.Int(rand.Reader, big.NewInt(int64(n)))
r.mu.Unlock()
return int(v.Int64())
}
var globalRand = &lockedRand{}
// CampaignType represents the type of campaigning
// the reason we use the type of string instead of uint64
// is because it's simpler to compare and fill in raft entries
type CampaignType string
// StateType represents the role of a node in a cluster.
type StateType uint64
var stmap = [...]string{
"StateFollower",
"StateCandidate",
"StateLeader",
"StatePreCandidate",
}
func (st StateType) String() string {
return stmap[st]
}
// Config contains the parameters to start a raft.
type Config struct {
// ID is the identity of the local raft. ID cannot be 0.
ID uint64
// ElectionTick is the number of Node.Tick invocations that must pass between
// elections. That is, if a follower does not receive any message from the
// leader of current term before ElectionTick has elapsed, it will become
// candidate and start an election. ElectionTick must be greater than
// HeartbeatTick. We suggest ElectionTick = 10 * HeartbeatTick to avoid
// unnecessary leader switching.
ElectionTick int
// HeartbeatTick is the number of Node.Tick invocations that must pass between
// heartbeats. That is, a leader sends heartbeat messages to maintain its
// leadership every HeartbeatTick ticks.
HeartbeatTick int
// Storage is the storage for raft. raft generates entries and states to be
// stored in storage. raft reads the persisted entries and states out of
// Storage when it needs. raft reads out the previous state and configuration
// out of storage when restarting.
Storage Storage
// Applied is the last applied index. It should only be set when restarting
// raft. raft will not return entries to the application smaller or equal to
// Applied. If Applied is unset when restarting, raft might return previous
// applied entries. This is a very application dependent configuration.
Applied uint64
// AsyncStorageWrites configures the raft node to write to its local storage
// (raft log and state machine) using a request/response message passing
// interface instead of the default Ready/Advance function call interface.
// Local storage messages can be pipelined and processed asynchronously
// (with respect to Ready iteration), facilitating reduced interference
// between Raft proposals and increased batching of log appends and state
// machine application. As a result, use of asynchronous storage writes can
// reduce end-to-end commit latency and increase maximum throughput.
//
// When true, the Ready.Message slice will include MsgStorageAppend and
// MsgStorageApply messages. The messages will target a LocalAppendThread
// and a LocalApplyThread, respectively. Messages to the same target must be
// reliably processed in order. In other words, they can't be dropped (like
// messages over the network) and those targeted at the same thread can't be
// reordered. Messages to different targets can be processed in any order.
//
// MsgStorageAppend carries Raft log entries to append, election votes /
// term changes / updated commit indexes to persist, and snapshots to apply.
// All writes performed in service of a MsgStorageAppend must be durable
// before response messages are delivered. However, if the MsgStorageAppend
// carries no response messages, durability is not required. The message
// assumes the role of the Entries, HardState, and Snapshot fields in Ready.
//
// MsgStorageApply carries committed entries to apply. Writes performed in
// service of a MsgStorageApply need not be durable before response messages
// are delivered. The message assumes the role of the CommittedEntries field
// in Ready.
//
// Local messages each carry one or more response messages which should be
// delivered after the corresponding storage write has been completed. These
// responses may target the same node or may target other nodes. The storage
// threads are not responsible for understanding the response messages, only
// for delivering them to the correct target after performing the storage
// write.
AsyncStorageWrites bool
// MaxSizePerMsg limits the max byte size of each append message. Smaller
// value lowers the raft recovery cost(initial probing and message lost
// during normal operation). On the other side, it might affect the
// throughput during normal replication. Note: math.MaxUint64 for unlimited,
// 0 for at most one entry per message.
MaxSizePerMsg uint64
// MaxCommittedSizePerReady limits the size of the committed entries which
// can be applying at the same time.
//
// Despite its name (preserved for compatibility), this quota applies across
// Ready structs to encompass all outstanding entries in unacknowledged
// MsgStorageApply messages when AsyncStorageWrites is enabled.
MaxCommittedSizePerReady uint64
// MaxUncommittedEntriesSize limits the aggregate byte size of the
// uncommitted entries that may be appended to a leader's log. Once this
// limit is exceeded, proposals will begin to return ErrProposalDropped
// errors. Note: 0 for no limit.
MaxUncommittedEntriesSize uint64
// MaxInflightMsgs limits the max number of in-flight append messages during
// optimistic replication phase. The application transportation layer usually
// has its own sending buffer over TCP/UDP. Setting MaxInflightMsgs to avoid
// overflowing that sending buffer. TODO (xiangli): feedback to application to
// limit the proposal rate?
MaxInflightMsgs int
// MaxInflightBytes limits the number of in-flight bytes in append messages.
// Complements MaxInflightMsgs. Ignored if zero.
//
// This effectively bounds the bandwidth-delay product. Note that especially
// in high-latency deployments setting this too low can lead to a dramatic
// reduction in throughput. For example, with a peer that has a round-trip
// latency of 100ms to the leader and this setting is set to 1 MB, there is a
// throughput limit of 10 MB/s for this group. With RTT of 400ms, this drops
// to 2.5 MB/s. See Little's law to understand the maths behind.
MaxInflightBytes uint64
// CheckQuorum specifies if the leader should check quorum activity. Leader
// steps down when quorum is not active for an electionTimeout.
CheckQuorum bool
// PreVote enables the Pre-Vote algorithm described in raft thesis section
// 9.6. This prevents disruption when a node that has been partitioned away
// rejoins the cluster.
PreVote bool
// ReadOnlyOption specifies how the read only request is processed.
//
// ReadOnlySafe guarantees the linearizability of the read only request by
// communicating with the quorum. It is the default and suggested option.
//
// ReadOnlyLeaseBased ensures linearizability of the read only request by
// relying on the leader lease. It can be affected by clock drift.
// If the clock drift is unbounded, leader might keep the lease longer than it
// should (clock can move backward/pause without any bound). ReadIndex is not safe
// in that case.
// CheckQuorum MUST be enabled if ReadOnlyOption is ReadOnlyLeaseBased.
ReadOnlyOption ReadOnlyOption
// Logger is the logger used for raft log. For multinode which can host
// multiple raft group, each raft group can have its own logger
Logger Logger
// DisableProposalForwarding set to true means that followers will drop
// proposals, rather than forwarding them to the leader. One use case for
// this feature would be in a situation where the Raft leader is used to
// compute the data of a proposal, for example, adding a timestamp from a
// hybrid logical clock to data in a monotonically increasing way. Forwarding
// should be disabled to prevent a follower with an inaccurate hybrid
// logical clock from assigning the timestamp and then forwarding the data
// to the leader.
DisableProposalForwarding bool
// DisableConfChangeValidation turns off propose-time verification of
// configuration changes against the currently active configuration of the
// raft instance. These checks are generally sensible (cannot leave a joint
// config unless in a joint config, et cetera) but they have false positives
// because the active configuration may not be the most recent
// configuration. This is because configurations are activated during log
// application, and even the leader can trail log application by an
// unbounded number of entries.
// Symmetrically, the mechanism has false negatives - because the check may
// not run against the "actual" config that will be the predecessor of the
// newly proposed one, the check may pass but the new config may be invalid
// when it is being applied. In other words, the checks are best-effort.
//
// Users should *not* use this option unless they have a reliable mechanism
// (above raft) that serializes and verifies configuration changes. If an
// invalid configuration change enters the log and gets applied, a panic
// will result.
//
// This option may be removed once false positives are no longer possible.
// See: https://github.com/etcd-io/raft/issues/80
DisableConfChangeValidation bool
// StepDownOnRemoval makes the leader step down when it is removed from the
// group or demoted to a learner.
//
// This behavior will become unconditional in the future. See:
// https://github.com/etcd-io/raft/issues/83
StepDownOnRemoval bool
// raft state tracer
TraceLogger TraceLogger
}
func (c *Config) validate() error {
if c.ID == None {
return errors.New("cannot use none as id")
}
if IsLocalMsgTarget(c.ID) {
return errors.New("cannot use local target as id")
}
if c.HeartbeatTick <= 0 {
return errors.New("heartbeat tick must be greater than 0")
}
if c.ElectionTick <= c.HeartbeatTick {
return errors.New("election tick must be greater than heartbeat tick")
}
if c.Storage == nil {
return errors.New("storage cannot be nil")
}
if c.MaxUncommittedEntriesSize == 0 {
c.MaxUncommittedEntriesSize = noLimit
}
// default MaxCommittedSizePerReady to MaxSizePerMsg because they were
// previously the same parameter.
if c.MaxCommittedSizePerReady == 0 {
c.MaxCommittedSizePerReady = c.MaxSizePerMsg
}
if c.MaxInflightMsgs <= 0 {
return errors.New("max inflight messages must be greater than 0")
}
if c.MaxInflightBytes == 0 {
c.MaxInflightBytes = noLimit
} else if c.MaxInflightBytes < c.MaxSizePerMsg {
return errors.New("max inflight bytes must be >= max message size")
}
if c.Logger == nil {
c.Logger = getLogger()
}
if c.ReadOnlyOption == ReadOnlyLeaseBased && !c.CheckQuorum {
return errors.New("CheckQuorum must be enabled when ReadOnlyOption is ReadOnlyLeaseBased")
}
return nil
}
type raft struct {
id uint64
Term uint64
Vote uint64
readStates []ReadState
// the log
raftLog *raftLog
maxMsgSize entryEncodingSize
maxUncommittedSize entryPayloadSize
trk tracker.ProgressTracker
state StateType
// isLearner is true if the local raft node is a learner.
isLearner bool
// msgs contains the list of messages that should be sent out immediately to
// other nodes.
//
// Messages in this list must target other nodes.
msgs []pb.Message
// msgsAfterAppend contains the list of messages that should be sent after
// the accumulated unstable state (e.g. term, vote, []entry, and snapshot)
// has been persisted to durable storage. This includes waiting for any
// unstable state that is already in the process of being persisted (i.e.
// has already been handed out in a prior Ready struct) to complete.
//
// Messages in this list may target other nodes or may target this node.
//
// Messages in this list have the type MsgAppResp, MsgVoteResp, or
// MsgPreVoteResp. See the comment in raft.send for details.
msgsAfterAppend []pb.Message
// the leader id
lead uint64
// leadTransferee is id of the leader transfer target when its value is not zero.
// Follow the procedure defined in raft thesis 3.10.
leadTransferee uint64
// Only one conf change may be pending (in the log, but not yet
// applied) at a time. This is enforced via pendingConfIndex, which
// is set to a value >= the log index of the latest pending
// configuration change (if any). Config changes are only allowed to
// be proposed if the leader's applied index is greater than this
// value.
pendingConfIndex uint64
// disableConfChangeValidation is Config.DisableConfChangeValidation,
// see there for details.
disableConfChangeValidation bool
// an estimate of the size of the uncommitted tail of the Raft log. Used to
// prevent unbounded log growth. Only maintained by the leader. Reset on
// term changes.
uncommittedSize entryPayloadSize
readOnly *readOnly
// number of ticks since it reached last electionTimeout when it is leader
// or candidate.
// number of ticks since it reached last electionTimeout or received a
// valid message from current leader when it is a follower.
electionElapsed int
// number of ticks since it reached last heartbeatTimeout.
// only leader keeps heartbeatElapsed.
heartbeatElapsed int
checkQuorum bool
preVote bool
heartbeatTimeout int
electionTimeout int
// randomizedElectionTimeout is a random number between
// [electiontimeout, 2 * electiontimeout - 1]. It gets reset
// when raft changes its state to follower or candidate.
randomizedElectionTimeout int
disableProposalForwarding bool
stepDownOnRemoval bool
tick func()
step stepFunc
logger Logger
// pendingReadIndexMessages is used to store messages of type MsgReadIndex
// that can't be answered as new leader didn't committed any log in
// current term. Those will be handled as fast as first log is committed in
// current term.
pendingReadIndexMessages []pb.Message
traceLogger TraceLogger
}
func newRaft(c *Config) *raft {
if err := c.validate(); err != nil {
panic(err.Error())
}
raftlog := newLogWithSize(c.Storage, c.Logger, entryEncodingSize(c.MaxCommittedSizePerReady))
hs, cs, err := c.Storage.InitialState()
if err != nil {
panic(err) // TODO(bdarnell)
}
r := &raft{
id: c.ID,
lead: None,
isLearner: false,
raftLog: raftlog,
maxMsgSize: entryEncodingSize(c.MaxSizePerMsg),
maxUncommittedSize: entryPayloadSize(c.MaxUncommittedEntriesSize),
trk: tracker.MakeProgressTracker(c.MaxInflightMsgs, c.MaxInflightBytes),
electionTimeout: c.ElectionTick,
heartbeatTimeout: c.HeartbeatTick,
logger: c.Logger,
checkQuorum: c.CheckQuorum,
preVote: c.PreVote,
readOnly: newReadOnly(c.ReadOnlyOption),
disableProposalForwarding: c.DisableProposalForwarding,
disableConfChangeValidation: c.DisableConfChangeValidation,
stepDownOnRemoval: c.StepDownOnRemoval,
traceLogger: c.TraceLogger,
}
traceInitState(r)
lastID := r.raftLog.lastEntryID()
cfg, trk, err := confchange.Restore(confchange.Changer{
Tracker: r.trk,
LastIndex: lastID.index,
}, cs)
if err != nil {
panic(err)
}
assertConfStatesEquivalent(r.logger, cs, r.switchToConfig(cfg, trk))
if !IsEmptyHardState(hs) {
r.loadState(hs)
}
if c.Applied > 0 {
raftlog.appliedTo(c.Applied, 0 /* size */)
}
r.becomeFollower(r.Term, None)
var nodesStrs []string
for _, n := range r.trk.VoterNodes() {
nodesStrs = append(nodesStrs, fmt.Sprintf("%x", n))
}
// TODO(pav-kv): it should be ok to simply print %+v for lastID.
r.logger.Infof("newRaft %x [peers: [%s], term: %d, commit: %d, applied: %d, lastindex: %d, lastterm: %d]",
r.id, strings.Join(nodesStrs, ","), r.Term, r.raftLog.committed, r.raftLog.applied, lastID.index, lastID.term)
return r
}
func (r *raft) hasLeader() bool { return r.lead != None }
func (r *raft) softState() SoftState { return SoftState{Lead: r.lead, RaftState: r.state} }
func (r *raft) hardState() pb.HardState {
return pb.HardState{
Term: r.Term,
Vote: r.Vote,
Commit: r.raftLog.committed,
}
}
// send schedules persisting state to a stable storage and AFTER that
// sending the message (as part of next Ready message processing).
func (r *raft) send(m pb.Message) {
if m.From == None {
m.From = r.id
}
if m.Type == pb.MsgVote || m.Type == pb.MsgVoteResp || m.Type == pb.MsgPreVote || m.Type == pb.MsgPreVoteResp {
if m.Term == 0 {
// All {pre-,}campaign messages need to have the term set when
// sending.
// - MsgVote: m.Term is the term the node is campaigning for,
// non-zero as we increment the term when campaigning.
// - MsgVoteResp: m.Term is the new r.Term if the MsgVote was
// granted, non-zero for the same reason MsgVote is
// - MsgPreVote: m.Term is the term the node will campaign,
// non-zero as we use m.Term to indicate the next term we'll be
// campaigning for
// - MsgPreVoteResp: m.Term is the term received in the original
// MsgPreVote if the pre-vote was granted, non-zero for the
// same reasons MsgPreVote is
r.logger.Panicf("term should be set when sending %s", m.Type)
}
} else {
if m.Term != 0 {
r.logger.Panicf("term should not be set when sending %s (was %d)", m.Type, m.Term)
}
// do not attach term to MsgProp, MsgReadIndex
// proposals are a way to forward to the leader and
// should be treated as local message.
// MsgReadIndex is also forwarded to leader.
if m.Type != pb.MsgProp && m.Type != pb.MsgReadIndex {
m.Term = r.Term
}
}
if m.Type == pb.MsgAppResp || m.Type == pb.MsgVoteResp || m.Type == pb.MsgPreVoteResp {
// If async storage writes are enabled, messages added to the msgs slice
// are allowed to be sent out before unstable state (e.g. log entry
// writes and election votes) have been durably synced to the local
// disk.
//
// For most message types, this is not an issue. However, response
// messages that relate to "voting" on either leader election or log
// appends require durability before they can be sent. It would be
// incorrect to publish a vote in an election before that vote has been
// synced to stable storage locally. Similarly, it would be incorrect to
// acknowledge a log append to the leader before that entry has been
// synced to stable storage locally.
//
// Per the Raft thesis, section 3.8 Persisted state and server restarts:
//
// > Raft servers must persist enough information to stable storage to
// > survive server restarts safely. In particular, each server persists
// > its current term and vote; this is necessary to prevent the server
// > from voting twice in the same term or replacing log entries from a
// > newer leader with those from a deposed leader. Each server also
// > persists new log entries before they are counted towards the entries’
// > commitment; this prevents committed entries from being lost or
// > “uncommitted” when servers restart
//
// To enforce this durability requirement, these response messages are
// queued to be sent out as soon as the current collection of unstable
// state (the state that the response message was predicated upon) has
// been durably persisted. This unstable state may have already been
// passed to a Ready struct whose persistence is in progress or may be
// waiting for the next Ready struct to begin being written to Storage.
// These messages must wait for all of this state to be durable before
// being published.
//
// Rejected responses (m.Reject == true) present an interesting case
// where the durability requirement is less unambiguous. A rejection may
// be predicated upon unstable state. For instance, a node may reject a
// vote for one peer because it has already begun syncing its vote for
// another peer. Or it may reject a vote from one peer because it has
// unstable log entries that indicate that the peer is behind on its
// log. In these cases, it is likely safe to send out the rejection
// response immediately without compromising safety in the presence of a
// server restart. However, because these rejections are rare and
// because the safety of such behavior has not been formally verified,
// we err on the side of safety and omit a `&& !m.Reject` condition
// above.
r.msgsAfterAppend = append(r.msgsAfterAppend, m)
traceSendMessage(r, &m)
} else {
if m.To == r.id {
r.logger.Panicf("message should not be self-addressed when sending %s", m.Type)
}
r.msgs = append(r.msgs, m)
traceSendMessage(r, &m)
}
}
// sendAppend sends an append RPC with new entries (if any) and the
// current commit index to the given peer.
func (r *raft) sendAppend(to uint64) {
r.maybeSendAppend(to, true)
}
// maybeSendAppend sends an append RPC with new entries to the given peer,
// if necessary. Returns true if a message was sent. The sendIfEmpty
// argument controls whether messages with no entries will be sent
// ("empty" messages are useful to convey updated Commit indexes, but
// are undesirable when we're sending multiple messages in a batch).
//
// TODO(pav-kv): make invocation of maybeSendAppend stateless. The Progress
// struct contains all the state necessary for deciding whether to send a
// message.
func (r *raft) maybeSendAppend(to uint64, sendIfEmpty bool) bool {
pr := r.trk.Progress[to]
if pr.IsPaused() {
return false
}
prevIndex := pr.Next - 1
prevTerm, err := r.raftLog.term(prevIndex)
if err != nil {
// The log probably got truncated at >= pr.Next, so we can't catch up the
// follower log anymore. Send a snapshot instead.
return r.maybeSendSnapshot(to, pr)
}
var ents []pb.Entry
// In a throttled StateReplicate only send empty MsgApp, to ensure progress.
// Otherwise, if we had a full Inflights and all inflight messages were in
// fact dropped, replication to that follower would stall. Instead, an empty
// MsgApp will eventually reach the follower (heartbeats responses prompt the
// leader to send an append), allowing it to be acked or rejected, both of
// which will clear out Inflights.
if pr.State != tracker.StateReplicate || !pr.Inflights.Full() {
ents, err = r.raftLog.entries(pr.Next, r.maxMsgSize)
}
if len(ents) == 0 && !sendIfEmpty {
return false
}
// TODO(pav-kv): move this check up to where err is returned.
if err != nil { // send a snapshot if we failed to get the entries
return r.maybeSendSnapshot(to, pr)
}
// Send the actual MsgApp otherwise, and update the progress accordingly.
r.send(pb.Message{
To: to,
Type: pb.MsgApp,
Index: prevIndex,
LogTerm: prevTerm,
Entries: ents,
Commit: r.raftLog.committed,
})
pr.SentEntries(len(ents), uint64(payloadsSize(ents)))
pr.SentCommit(r.raftLog.committed)
return true
}
// maybeSendSnapshot fetches a snapshot from Storage, and sends it to the given
// node. Returns true iff the snapshot message has been emitted successfully.
func (r *raft) maybeSendSnapshot(to uint64, pr *tracker.Progress) bool {
if !pr.RecentActive {
r.logger.Debugf("ignore sending snapshot to %x since it is not recently active", to)
return false
}
snapshot, err := r.raftLog.snapshot()
if err != nil {
if err == ErrSnapshotTemporarilyUnavailable {
r.logger.Debugf("%x failed to send snapshot to %x because snapshot is temporarily unavailable", r.id, to)
return false
}
panic(err) // TODO(bdarnell)
}
if IsEmptySnap(snapshot) {
panic("need non-empty snapshot")
}
sindex, sterm := snapshot.Metadata.Index, snapshot.Metadata.Term
r.logger.Debugf("%x [firstindex: %d, commit: %d] sent snapshot[index: %d, term: %d] to %x [%s]",
r.id, r.raftLog.firstIndex(), r.raftLog.committed, sindex, sterm, to, pr)
pr.BecomeSnapshot(sindex)
r.logger.Debugf("%x paused sending replication messages to %x [%s]", r.id, to, pr)
r.send(pb.Message{To: to, Type: pb.MsgSnap, Snapshot: &snapshot})
return true
}
// sendHeartbeat sends a heartbeat RPC to the given peer.
func (r *raft) sendHeartbeat(to uint64, ctx []byte) {
pr := r.trk.Progress[to]
// Attach the commit as min(to.matched, r.committed).
// When the leader sends out heartbeat message,
// the receiver(follower) might not be matched with the leader
// or it might not have all the committed entries.
// The leader MUST NOT forward the follower's commit to
// an unmatched index.
commit := min(pr.Match, r.raftLog.committed)
r.send(pb.Message{
To: to,
Type: pb.MsgHeartbeat,
Commit: commit,
Context: ctx,
})
pr.SentCommit(commit)
}
// bcastAppend sends RPC, with entries to all peers that are not up-to-date
// according to the progress recorded in r.trk.
func (r *raft) bcastAppend() {
r.trk.Visit(func(id uint64, _ *tracker.Progress) {
if id == r.id {
return
}
r.sendAppend(id)
})
}
// bcastHeartbeat sends RPC, without entries to all the peers.
func (r *raft) bcastHeartbeat() {
lastCtx := r.readOnly.lastPendingRequestCtx()
if len(lastCtx) == 0 {
r.bcastHeartbeatWithCtx(nil)
} else {
r.bcastHeartbeatWithCtx([]byte(lastCtx))
}
}
func (r *raft) bcastHeartbeatWithCtx(ctx []byte) {
r.trk.Visit(func(id uint64, _ *tracker.Progress) {
if id == r.id {
return
}
r.sendHeartbeat(id, ctx)
})
}
func (r *raft) appliedTo(index uint64, size entryEncodingSize) {
oldApplied := r.raftLog.applied
newApplied := max(index, oldApplied)
r.raftLog.appliedTo(newApplied, size)
if r.trk.Config.AutoLeave && newApplied >= r.pendingConfIndex && r.state == StateLeader {
// If the current (and most recent, at least for this leader's term)
// configuration should be auto-left, initiate that now. We use a
// nil Data which unmarshals into an empty ConfChangeV2 and has the
// benefit that appendEntry can never refuse it based on its size
// (which registers as zero).
m, err := confChangeToMsg(nil)
if err != nil {
panic(err)
}
// NB: this proposal can't be dropped due to size, but can be
// dropped if a leadership transfer is in progress. We'll keep
// checking this condition on each applied entry, so either the
// leadership transfer will succeed and the new leader will leave
// the joint configuration, or the leadership transfer will fail,
// and we will propose the config change on the next advance.
if err := r.Step(m); err != nil {
r.logger.Debugf("not initiating automatic transition out of joint configuration %s: %v", r.trk.Config, err)
} else {
r.logger.Infof("initiating automatic transition out of joint configuration %s", r.trk.Config)
}
}
}
func (r *raft) appliedSnap(snap *pb.Snapshot) {
index := snap.Metadata.Index
r.raftLog.stableSnapTo(index)
r.appliedTo(index, 0 /* size */)
}
// maybeCommit attempts to advance the commit index. Returns true if the commit
// index changed (in which case the caller should call r.bcastAppend). This can
// only be called in StateLeader.
func (r *raft) maybeCommit() bool {
defer traceCommit(r)
return r.raftLog.maybeCommit(entryID{term: r.Term, index: r.trk.Committed()})
}
func (r *raft) reset(term uint64) {
if r.Term != term {
r.Term = term
r.Vote = None
}
r.lead = None
r.electionElapsed = 0
r.heartbeatElapsed = 0
r.resetRandomizedElectionTimeout()
r.abortLeaderTransfer()
r.trk.ResetVotes()
r.trk.Visit(func(id uint64, pr *tracker.Progress) {
*pr = tracker.Progress{
Match: 0,
Next: r.raftLog.lastIndex() + 1,
Inflights: tracker.NewInflights(r.trk.MaxInflight, r.trk.MaxInflightBytes),
IsLearner: pr.IsLearner,
}
if id == r.id {
pr.Match = r.raftLog.lastIndex()
}
})
r.pendingConfIndex = 0
r.uncommittedSize = 0
r.readOnly = newReadOnly(r.readOnly.option)
}
func (r *raft) appendEntry(es ...pb.Entry) (accepted bool) {
li := r.raftLog.lastIndex()
for i := range es {
es[i].Term = r.Term
es[i].Index = li + 1 + uint64(i)
}
// Track the size of this uncommitted proposal.
if !r.increaseUncommittedSize(es) {
r.logger.Warningf(
"%x appending new entries to log would exceed uncommitted entry size limit; dropping proposal",
r.id,
)
// Drop the proposal.
return false
}
traceReplicate(r, es...)
// use latest "last" index after truncate/append
li = r.raftLog.append(es...)
// The leader needs to self-ack the entries just appended once they have
// been durably persisted (since it doesn't send an MsgApp to itself). This
// response message will be added to msgsAfterAppend and delivered back to
// this node after these entries have been written to stable storage. When
// handled, this is roughly equivalent to:
//
// r.trk.Progress[r.id].MaybeUpdate(e.Index)
// if r.maybeCommit() {
// r.bcastAppend()
// }
r.send(pb.Message{To: r.id, Type: pb.MsgAppResp, Index: li})
return true
}
// tickElection is run by followers and candidates after r.electionTimeout.
func (r *raft) tickElection() {
r.electionElapsed++
if r.promotable() && r.pastElectionTimeout() {
r.electionElapsed = 0
if err := r.Step(pb.Message{From: r.id, Type: pb.MsgHup}); err != nil {
r.logger.Debugf("error occurred during election: %v", err)
}
}
}
// tickHeartbeat is run by leaders to send a MsgBeat after r.heartbeatTimeout.
func (r *raft) tickHeartbeat() {
r.heartbeatElapsed++
r.electionElapsed++
if r.electionElapsed >= r.electionTimeout {
r.electionElapsed = 0
if r.checkQuorum {
if err := r.Step(pb.Message{From: r.id, Type: pb.MsgCheckQuorum}); err != nil {
r.logger.Debugf("error occurred during checking sending heartbeat: %v", err)
}
}
// If current leader cannot transfer leadership in electionTimeout, it becomes leader again.
if r.state == StateLeader && r.leadTransferee != None {
r.abortLeaderTransfer()
}
}
if r.state != StateLeader {
return
}
if r.heartbeatElapsed >= r.heartbeatTimeout {
r.heartbeatElapsed = 0
if err := r.Step(pb.Message{From: r.id, Type: pb.MsgBeat}); err != nil {
r.logger.Debugf("error occurred during checking sending heartbeat: %v", err)
}
}
}
func (r *raft) becomeFollower(term uint64, lead uint64) {
r.step = stepFollower
r.reset(term)
r.tick = r.tickElection
r.lead = lead
r.state = StateFollower
r.logger.Infof("%x became follower at term %d", r.id, r.Term)
traceBecomeFollower(r)
}
func (r *raft) becomeCandidate() {
// TODO(xiangli) remove the panic when the raft implementation is stable
if r.state == StateLeader {
panic("invalid transition [leader -> candidate]")
}
r.step = stepCandidate
r.reset(r.Term + 1)
r.tick = r.tickElection
r.Vote = r.id
r.state = StateCandidate
r.logger.Infof("%x became candidate at term %d", r.id, r.Term)
traceBecomeCandidate(r)
}
func (r *raft) becomePreCandidate() {
// TODO(xiangli) remove the panic when the raft implementation is stable
if r.state == StateLeader {
panic("invalid transition [leader -> pre-candidate]")
}
// Becoming a pre-candidate changes our step functions and state,
// but doesn't change anything else. In particular it does not increase
// r.Term or change r.Vote.
r.step = stepCandidate
r.trk.ResetVotes()
r.tick = r.tickElection
r.lead = None
r.state = StatePreCandidate
r.logger.Infof("%x became pre-candidate at term %d", r.id, r.Term)
}
func (r *raft) becomeLeader() {
// TODO(xiangli) remove the panic when the raft implementation is stable
if r.state == StateFollower {
panic("invalid transition [follower -> leader]")
}
r.step = stepLeader
r.reset(r.Term)
r.tick = r.tickHeartbeat
r.lead = r.id
r.state = StateLeader
// Followers enter replicate mode when they've been successfully probed
// (perhaps after having received a snapshot as a result). The leader is
// trivially in this state. Note that r.reset() has initialized this
// progress with the last index already.
pr := r.trk.Progress[r.id]
pr.BecomeReplicate()
// The leader always has RecentActive == true; MsgCheckQuorum makes sure to
// preserve this.
pr.RecentActive = true
// Conservatively set the pendingConfIndex to the last index in the
// log. There may or may not be a pending config change, but it's
// safe to delay any future proposals until we commit all our
// pending log entries, and scanning the entire tail of the log
// could be expensive.
r.pendingConfIndex = r.raftLog.lastIndex()
traceBecomeLeader(r)
emptyEnt := pb.Entry{Data: nil}
if !r.appendEntry(emptyEnt) {
// This won't happen because we just called reset() above.
r.logger.Panic("empty entry was dropped")
}
// The payloadSize of an empty entry is 0 (see TestPayloadSizeOfEmptyEntry),
// so the preceding log append does not count against the uncommitted log
// quota of the new leader. In other words, after the call to appendEntry,
// r.uncommittedSize is still 0.
r.logger.Infof("%x became leader at term %d", r.id, r.Term)
}
func (r *raft) hup(t CampaignType) {
if r.state == StateLeader {
r.logger.Debugf("%x ignoring MsgHup because already leader", r.id)
return
}
if !r.promotable() {
r.logger.Warningf("%x is unpromotable and can not campaign", r.id)
return
}
if r.hasUnappliedConfChanges() {
r.logger.Warningf("%x cannot campaign at term %d since there are still pending configuration changes to apply", r.id, r.Term)
return
}
r.logger.Infof("%x is starting a new election at term %d", r.id, r.Term)
r.campaign(t)
}
// errBreak is a sentinel error used to break a callback-based loop.
var errBreak = errors.New("break")
func (r *raft) hasUnappliedConfChanges() bool {
if r.raftLog.applied >= r.raftLog.committed { // in fact applied == committed
return false
}
found := false
// Scan all unapplied committed entries to find a config change. Paginate the