forked from etcd-io/raft
-
Notifications
You must be signed in to change notification settings - Fork 0
/
raft_test.go
4153 lines (3465 loc) · 126 KB
/
raft_test.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 (
"fmt"
"math"
"math/rand"
"strings"
"testing"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
pb "go.etcd.io/raft/v3/raftpb"
"go.etcd.io/raft/v3/tracker"
)
// nextEnts returns the appliable entries and updates the applied index.
func nextEnts(r *raft, s *MemoryStorage) (ents []pb.Entry) {
// Append unstable entries.
s.Append(r.raftLog.nextUnstableEnts())
r.raftLog.stableTo(r.raftLog.lastEntryID())
// Run post-append steps.
r.advanceMessagesAfterAppend()
// Return committed entries.
ents = r.raftLog.nextCommittedEnts(true)
r.raftLog.appliedTo(r.raftLog.committed, 0 /* size */)
return ents
}
func mustAppendEntry(r *raft, ents ...pb.Entry) {
if !r.appendEntry(ents...) {
panic("entry unexpectedly dropped")
}
}
type stateMachine interface {
Step(m pb.Message) error
readMessages() []pb.Message
advanceMessagesAfterAppend()
}
func (r *raft) readMessages() []pb.Message {
r.advanceMessagesAfterAppend()
msgs := r.msgs
r.msgs = nil
return msgs
}
func (r *raft) advanceMessagesAfterAppend() {
for {
msgs := r.takeMessagesAfterAppend()
if len(msgs) == 0 {
break
}
r.stepOrSend(msgs)
}
}
func (r *raft) takeMessagesAfterAppend() []pb.Message {
msgs := r.msgsAfterAppend
r.msgsAfterAppend = nil
return msgs
}
func (r *raft) stepOrSend(msgs []pb.Message) error {
for _, m := range msgs {
if m.To == r.id {
if err := r.Step(m); err != nil {
return err
}
} else {
r.msgs = append(r.msgs, m)
}
}
return nil
}
func TestProgressLeader(t *testing.T) {
s := newTestMemoryStorage(withPeers(1, 2))
r := newTestRaft(1, 5, 1, s)
r.becomeCandidate()
r.becomeLeader()
r.trk.Progress[2].BecomeReplicate()
// Send proposals to r1. The first 5 entries should be queued in the unstable log.
propMsg := pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("foo")}}}
for i := 0; i < 5; i++ {
require.NoError(t, r.Step(propMsg), "#%d", i)
}
require.Zero(t, r.trk.Progress[1].Match)
ents := r.raftLog.nextUnstableEnts()
require.Len(t, ents, 6)
require.Len(t, ents[0].Data, 0)
require.Equal(t, "foo", string(ents[5].Data))
r.advanceMessagesAfterAppend()
require.Equal(t, uint64(6), r.trk.Progress[1].Match)
require.Equal(t, uint64(7), r.trk.Progress[1].Next)
}
// TestProgressResumeByHeartbeatResp ensures raft.heartbeat reset progress.paused by heartbeat response.
func TestProgressResumeByHeartbeatResp(t *testing.T) {
r := newTestRaft(1, 5, 1, newTestMemoryStorage(withPeers(1, 2)))
r.becomeCandidate()
r.becomeLeader()
r.trk.Progress[2].MsgAppFlowPaused = true
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgBeat})
assert.True(t, r.trk.Progress[2].MsgAppFlowPaused)
r.trk.Progress[2].BecomeReplicate()
assert.False(t, r.trk.Progress[2].MsgAppFlowPaused)
r.trk.Progress[2].MsgAppFlowPaused = true
r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgHeartbeatResp})
assert.False(t, r.trk.Progress[2].MsgAppFlowPaused)
}
func TestProgressPaused(t *testing.T) {
r := newTestRaft(1, 5, 1, newTestMemoryStorage(withPeers(1, 2)))
r.becomeCandidate()
r.becomeLeader()
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
ms := r.readMessages()
assert.Len(t, ms, 1)
}
func TestProgressFlowControl(t *testing.T) {
cfg := newTestConfig(1, 5, 1, newTestMemoryStorage(withPeers(1, 2)))
cfg.MaxInflightMsgs = 3
cfg.MaxSizePerMsg = 2048
cfg.MaxInflightBytes = 9000 // A little over MaxInflightMsgs * MaxSizePerMsg.
r := newRaft(cfg)
r.becomeCandidate()
r.becomeLeader()
// Throw away all the messages relating to the initial election.
r.readMessages()
// While node 2 is in probe state, propose a bunch of entries.
r.trk.Progress[2].BecomeProbe()
blob := []byte(strings.Repeat("a", 1000))
large := []byte(strings.Repeat("b", 5000))
for i := 0; i < 22; i++ {
blob := blob
if i >= 10 && i < 16 { // Temporarily send large messages.
blob = large
}
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: blob}}})
}
ms := r.readMessages()
// First append has two entries: the empty entry to confirm the
// election, and the first proposal (only one proposal gets sent
// because we're in probe state).
require.Len(t, ms, 1)
require.Equal(t, pb.MsgApp, ms[0].Type)
require.Len(t, ms[0].Entries, 2)
require.Empty(t, ms[0].Entries[0].Data)
require.Len(t, ms[0].Entries[1].Data, 1000)
ackAndVerify := func(index uint64, expEntries ...int) uint64 {
r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgAppResp, Index: index})
ms := r.readMessages()
require.Equal(t, len(expEntries), len(ms))
for i, m := range ms {
assert.Equal(t, pb.MsgApp, m.Type, "#%d", i)
assert.Len(t, m.Entries, expEntries[i], "#%d", i)
}
last := ms[len(ms)-1].Entries
if len(last) == 0 {
return index
}
return last[len(last)-1].Index
}
// When this append is acked, we change to replicate state and can
// send multiple messages at once.
index := ackAndVerify(ms[0].Entries[1].Index, 2, 2, 2)
// Ack all three of those messages together and get another 3 messages. The
// third message contains a single large entry, in contrast to 2 before.
index = ackAndVerify(index, 2, 1, 1)
// All subsequent messages contain one large entry, and we cap at 2 messages
// because it overflows MaxInflightBytes.
index = ackAndVerify(index, 1, 1)
index = ackAndVerify(index, 1, 1)
// Start getting small messages again.
index = ackAndVerify(index, 1, 2, 2)
ackAndVerify(index, 2)
}
func TestUncommittedEntryLimit(t *testing.T) {
// Use a relatively large number of entries here to prevent regression of a
// bug which computed the size before it was fixed. This test would fail
// with the bug, either because we'd get dropped proposals earlier than we
// expect them, or because the final tally ends up nonzero. (At the time of
// writing, the former).
const maxEntries = 1024
testEntry := pb.Entry{Data: []byte("testdata")}
maxEntrySize := maxEntries * payloadSize(testEntry)
require.Zero(t, payloadSize(pb.Entry{Data: nil}))
cfg := newTestConfig(1, 5, 1, newTestMemoryStorage(withPeers(1, 2, 3)))
cfg.MaxUncommittedEntriesSize = uint64(maxEntrySize)
cfg.MaxInflightMsgs = 2 * 1024 // avoid interference
r := newRaft(cfg)
r.becomeCandidate()
r.becomeLeader()
require.Zero(t, r.uncommittedSize)
// Set the two followers to the replicate state. Commit to tail of log.
const numFollowers = 2
r.trk.Progress[2].BecomeReplicate()
r.trk.Progress[3].BecomeReplicate()
r.uncommittedSize = 0
// Send proposals to r1. The first 5 entries should be appended to the log.
propMsg := pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{testEntry}}
propEnts := make([]pb.Entry, maxEntries)
for i := 0; i < maxEntries; i++ {
require.NoError(t, r.Step(propMsg), "#%d", i)
propEnts[i] = testEntry
}
// Send one more proposal to r1. It should be rejected.
require.Equal(t, ErrProposalDropped, r.Step(propMsg))
// Read messages and reduce the uncommitted size as if we had committed
// these entries.
ms := r.readMessages()
require.Len(t, ms, maxEntries*numFollowers)
r.reduceUncommittedSize(payloadsSize(propEnts))
require.Zero(t, r.uncommittedSize)
// Send a single large proposal to r1. Should be accepted even though it
// pushes us above the limit because we were beneath it before the proposal.
propEnts = make([]pb.Entry, 2*maxEntries)
for i := range propEnts {
propEnts[i] = testEntry
}
propMsgLarge := pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: propEnts}
require.NoError(t, r.Step(propMsgLarge))
// Send one more proposal to r1. It should be rejected, again.
require.Equal(t, ErrProposalDropped, r.Step(propMsg))
// But we can always append an entry with no Data. This is used both for the
// leader's first empty entry and for auto-transitioning out of joint config
// states.
require.NoError(t, r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}}))
// Read messages and reduce the uncommitted size as if we had committed
// these entries.
ms = r.readMessages()
require.Len(t, ms, 2*numFollowers)
r.reduceUncommittedSize(payloadsSize(propEnts))
require.Zero(t, r.uncommittedSize)
}
func TestLeaderElection(t *testing.T) {
testLeaderElection(t, false)
}
func TestLeaderElectionPreVote(t *testing.T) {
testLeaderElection(t, true)
}
func testLeaderElection(t *testing.T, preVote bool) {
var cfg func(*Config)
candState := StateCandidate
candTerm := uint64(1)
if preVote {
cfg = preVoteConfig
// In pre-vote mode, an election that fails to complete
// leaves the node in pre-candidate state without advancing
// the term.
candState = StatePreCandidate
candTerm = 0
}
tests := []struct {
*network
state StateType
expTerm uint64
}{
{newNetworkWithConfig(cfg, nil, nil, nil), StateLeader, 1},
{newNetworkWithConfig(cfg, nil, nil, nopStepper), StateLeader, 1},
{newNetworkWithConfig(cfg, nil, nopStepper, nopStepper), candState, candTerm},
{newNetworkWithConfig(cfg, nil, nopStepper, nopStepper, nil), candState, candTerm},
{newNetworkWithConfig(cfg, nil, nopStepper, nopStepper, nil, nil), StateLeader, 1},
// three logs further along than 0, but in the same term so rejections
// are returned instead of the votes being ignored.
{newNetworkWithConfig(cfg,
nil, entsWithConfig(cfg, 1), entsWithConfig(cfg, 1), entsWithConfig(cfg, 1, 1), nil),
StateFollower, 1},
}
for i, tt := range tests {
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
sm := tt.network.peers[1].(*raft)
assert.Equal(t, tt.state, sm.state, "#%d", i)
assert.Equal(t, tt.expTerm, sm.Term, "#%d", i)
}
}
// TestLearnerElectionTimeout verfies that the leader should not start election even
// when times out.
func TestLearnerElectionTimeout(t *testing.T) {
n1 := newTestLearnerRaft(1, 10, 1, newTestMemoryStorage(withPeers(1), withLearners(2)))
n2 := newTestLearnerRaft(2, 10, 1, newTestMemoryStorage(withPeers(1), withLearners(2)))
n1.becomeFollower(1, None)
n2.becomeFollower(1, None)
// n2 is learner. Learner should not start election even when times out.
setRandomizedElectionTimeout(n2, n2.electionTimeout)
for i := 0; i < n2.electionTimeout; i++ {
n2.tick()
}
assert.Equal(t, StateFollower, n2.state)
}
// TestLearnerPromotion verifies that the learner should not election until
// it is promoted to a normal peer.
func TestLearnerPromotion(t *testing.T) {
n1 := newTestLearnerRaft(1, 10, 1, newTestMemoryStorage(withPeers(1), withLearners(2)))
n2 := newTestLearnerRaft(2, 10, 1, newTestMemoryStorage(withPeers(1), withLearners(2)))
n1.becomeFollower(1, None)
n2.becomeFollower(1, None)
nt := newNetwork(n1, n2)
assert.NotEqual(t, StateLeader, n1.state)
// n1 should become leader
setRandomizedElectionTimeout(n1, n1.electionTimeout)
for i := 0; i < n1.electionTimeout; i++ {
n1.tick()
}
n1.advanceMessagesAfterAppend()
assert.Equal(t, StateLeader, n1.state)
assert.Equal(t, StateFollower, n2.state)
nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgBeat})
n1.applyConfChange(pb.ConfChange{NodeID: 2, Type: pb.ConfChangeAddNode}.AsV2())
n2.applyConfChange(pb.ConfChange{NodeID: 2, Type: pb.ConfChangeAddNode}.AsV2())
assert.False(t, n2.isLearner)
// n2 start election, should become leader
setRandomizedElectionTimeout(n2, n2.electionTimeout)
for i := 0; i < n2.electionTimeout; i++ {
n2.tick()
}
n2.advanceMessagesAfterAppend()
nt.send(pb.Message{From: 2, To: 2, Type: pb.MsgBeat})
assert.Equal(t, StateFollower, n1.state)
assert.Equal(t, StateLeader, n2.state)
}
// TestLearnerCanVote checks that a learner can vote when it receives a valid Vote request.
// See (*raft).Step for why this is necessary and correct behavior.
func TestLearnerCanVote(t *testing.T) {
n2 := newTestLearnerRaft(2, 10, 1, newTestMemoryStorage(withPeers(1), withLearners(2)))
n2.becomeFollower(1, None)
n2.Step(pb.Message{From: 1, To: 2, Term: 2, Type: pb.MsgVote, LogTerm: 11, Index: 11})
msgs := n2.readMessages()
require.Len(t, msgs, 1)
require.Equal(t, msgs[0].Type, pb.MsgVoteResp)
require.False(t, msgs[0].Reject, "expected learner to not reject vote")
}
func TestLeaderCycle(t *testing.T) {
testLeaderCycle(t, false)
}
func TestLeaderCyclePreVote(t *testing.T) {
testLeaderCycle(t, true)
}
// testLeaderCycle verifies that each node in a cluster can campaign
// and be elected in turn. This ensures that elections (including
// pre-vote) work when not starting from a clean slate (as they do in
// TestLeaderElection)
func testLeaderCycle(t *testing.T, preVote bool) {
var cfg func(*Config)
if preVote {
cfg = preVoteConfig
}
n := newNetworkWithConfig(cfg, nil, nil, nil)
for campaignerID := uint64(1); campaignerID <= 3; campaignerID++ {
n.send(pb.Message{From: campaignerID, To: campaignerID, Type: pb.MsgHup})
for _, peer := range n.peers {
sm := peer.(*raft)
if sm.id == campaignerID {
assert.Equal(t, StateLeader, sm.state, "preVote=%v: campaigning node %d", preVote, sm.id)
} else {
assert.Equal(t, StateFollower, sm.state, "preVote=%v: campaigning node %d, current node %d", preVote, campaignerID, sm.id)
}
}
}
}
// TestLeaderElectionOverwriteNewerLogs tests a scenario in which a
// newly-elected leader does *not* have the newest (i.e. highest term)
// log entries, and must overwrite higher-term log entries with
// lower-term ones.
func TestLeaderElectionOverwriteNewerLogs(t *testing.T) {
testLeaderElectionOverwriteNewerLogs(t, false)
}
func TestLeaderElectionOverwriteNewerLogsPreVote(t *testing.T) {
testLeaderElectionOverwriteNewerLogs(t, true)
}
func testLeaderElectionOverwriteNewerLogs(t *testing.T, preVote bool) {
var cfg func(*Config)
if preVote {
cfg = preVoteConfig
}
// This network represents the results of the following sequence of
// events:
// - Node 1 won the election in term 1.
// - Node 1 replicated a log entry to node 2 but died before sending
// it to other nodes.
// - Node 3 won the second election in term 2.
// - Node 3 wrote an entry to its logs but died without sending it
// to any other nodes.
//
// At this point, nodes 1, 2, and 3 all have uncommitted entries in
// their logs and could win an election at term 3. The winner's log
// entry overwrites the losers'. (TestLeaderSyncFollowerLog tests
// the case where older log entries are overwritten, so this test
// focuses on the case where the newer entries are lost).
n := newNetworkWithConfig(cfg,
entsWithConfig(cfg, 1), // Node 1: Won first election
entsWithConfig(cfg, 1), // Node 2: Got logs from node 1
entsWithConfig(cfg, 2), // Node 3: Won second election
votedWithConfig(cfg, 3, 2), // Node 4: Voted but didn't get logs
votedWithConfig(cfg, 3, 2)) // Node 5: Voted but didn't get logs
// Node 1 campaigns. The election fails because a quorum of nodes
// know about the election that already happened at term 2. Node 1's
// term is pushed ahead to 2.
n.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
sm1 := n.peers[1].(*raft)
assert.Equal(t, StateFollower, sm1.state)
assert.Equal(t, uint64(2), sm1.Term)
// Node 1 campaigns again with a higher term. This time it succeeds.
n.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
assert.Equal(t, StateLeader, sm1.state)
assert.Equal(t, uint64(3), sm1.Term)
// Now all nodes agree on a log entry with term 1 at index 1 (and
// term 3 at index 2).
for i := range n.peers {
sm := n.peers[i].(*raft)
entries := sm.raftLog.allEntries()
require.Len(t, entries, 2)
assert.Equal(t, uint64(1), entries[0].Term)
assert.Equal(t, uint64(3), entries[1].Term)
}
}
func TestVoteFromAnyState(t *testing.T) {
testVoteFromAnyState(t, pb.MsgVote)
}
func TestPreVoteFromAnyState(t *testing.T) {
testVoteFromAnyState(t, pb.MsgPreVote)
}
func testVoteFromAnyState(t *testing.T, vt pb.MessageType) {
for st := StateType(0); st < numStates; st++ {
r := newTestRaft(1, 10, 1, newTestMemoryStorage(withPeers(1, 2, 3)))
r.Term = 1
switch st {
case StateFollower:
r.becomeFollower(r.Term, 3)
case StatePreCandidate:
r.becomePreCandidate()
case StateCandidate:
r.becomeCandidate()
case StateLeader:
r.becomeCandidate()
r.becomeLeader()
}
// Note that setting our state above may have advanced r.Term
// past its initial value.
origTerm := r.Term
newTerm := r.Term + 1
msg := pb.Message{
From: 2,
To: 1,
Type: vt,
Term: newTerm,
LogTerm: newTerm,
Index: 42,
}
assert.NoError(t, r.Step(msg), "%s,%s", vt, st)
msgs := r.readMessages()
if assert.Len(t, msgs, 1, "%s,%s", vt, st) {
resp := msgs[0]
assert.Equal(t, voteRespMsgType(vt), resp.Type, "%s,%s", vt, st)
assert.False(t, resp.Reject, "%s,%s", vt, st)
}
// If this was a real vote, we reset our state and term.
if vt == pb.MsgVote {
assert.Equal(t, StateFollower, r.state, "%s,%s", vt, st)
assert.Equal(t, newTerm, r.Term, "%s,%s", vt, st)
assert.Equal(t, uint64(2), r.Vote, "%s,%s", vt, st)
} else {
// In a prevote, nothing changes.
assert.Equal(t, st, r.state, "%s,%s", vt, st)
assert.Equal(t, origTerm, r.Term, "%s,%s", vt, st)
// if st == StateFollower or StatePreCandidate, r hasn't voted yet.
// In StateCandidate or StateLeader, it's voted for itself.
assert.True(t, r.Vote == None || r.Vote == 1, "%s,%s: vote %d, want %d or 1", vt, st, r.Vote, None)
}
}
}
func TestLogReplication(t *testing.T) {
tests := []struct {
*network
msgs []pb.Message
wcommitted uint64
}{
{
newNetwork(nil, nil, nil),
[]pb.Message{
{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}},
},
2,
},
{
newNetwork(nil, nil, nil),
[]pb.Message{
{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}},
{From: 1, To: 2, Type: pb.MsgHup},
{From: 1, To: 2, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}},
},
4,
},
}
for i, tt := range tests {
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
for _, m := range tt.msgs {
tt.send(m)
}
for j, x := range tt.network.peers {
sm := x.(*raft)
assert.Equal(t, tt.wcommitted, sm.raftLog.committed, "#%d.%d", i, j)
var ents []pb.Entry
for _, e := range nextEnts(sm, tt.network.storage[j]) {
if e.Data != nil {
ents = append(ents, e)
}
}
var props []pb.Message
for _, m := range tt.msgs {
if m.Type == pb.MsgProp {
props = append(props, m)
}
}
for k, m := range props {
assert.Equal(t, m.Entries[0].Data, ents[k].Data, "#%d.%d", i, j)
}
}
}
}
// TestLearnerLogReplication tests that a learner can receive entries from the leader.
func TestLearnerLogReplication(t *testing.T) {
s1 := newTestMemoryStorage(withPeers(1), withLearners(2))
n1 := newTestLearnerRaft(1, 10, 1, s1)
n2 := newTestLearnerRaft(2, 10, 1, newTestMemoryStorage(withPeers(1), withLearners(2)))
nt := newNetwork(n1, n2)
nt.t = t
n1.becomeFollower(1, None)
n2.becomeFollower(1, None)
setRandomizedElectionTimeout(n1, n1.electionTimeout)
for i := 0; i < n1.electionTimeout; i++ {
n1.tick()
}
n1.advanceMessagesAfterAppend()
nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgBeat})
// n1 is leader and n2 is learner
assert.Equal(t, StateLeader, n1.state)
assert.True(t, n2.isLearner)
nextCommitted := uint64(2)
{
nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
}
assert.Equal(t, nextCommitted, n1.raftLog.committed)
assert.Equal(t, n1.raftLog.committed, n2.raftLog.committed)
match := n1.trk.Progress[2].Match
assert.Equal(t, n2.raftLog.committed, match)
}
func TestSingleNodeCommit(t *testing.T) {
s := newTestMemoryStorage(withPeers(1))
cfg := newTestConfig(1, 10, 1, s)
r := newRaft(cfg)
tt := newNetwork(r)
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}})
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}})
sm := tt.peers[1].(*raft)
assert.Equal(t, uint64(3), sm.raftLog.committed)
}
// TestCannotCommitWithoutNewTermEntry tests the entries cannot be committed
// when leader changes, no new proposal comes in and ChangeTerm proposal is
// filtered.
func TestCannotCommitWithoutNewTermEntry(t *testing.T) {
tt := newNetwork(nil, nil, nil, nil, nil)
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
// 0 cannot reach 2,3,4
tt.cut(1, 3)
tt.cut(1, 4)
tt.cut(1, 5)
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}})
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}})
sm := tt.peers[1].(*raft)
assert.Equal(t, uint64(1), sm.raftLog.committed)
// network recovery
tt.recover()
// avoid committing ChangeTerm proposal
tt.ignore(pb.MsgApp)
// elect 2 as the new leader with term 2
tt.send(pb.Message{From: 2, To: 2, Type: pb.MsgHup})
// no log entries from previous term should be committed
sm = tt.peers[2].(*raft)
assert.Equal(t, uint64(1), sm.raftLog.committed)
tt.recover()
// send heartbeat; reset wait
tt.send(pb.Message{From: 2, To: 2, Type: pb.MsgBeat})
// append an entry at current term
tt.send(pb.Message{From: 2, To: 2, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}})
// expect the committed to be advanced
assert.Equal(t, uint64(5), sm.raftLog.committed)
}
// TestCommitWithoutNewTermEntry tests the entries could be committed
// when leader changes, no new proposal comes in.
func TestCommitWithoutNewTermEntry(t *testing.T) {
tt := newNetwork(nil, nil, nil, nil, nil)
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
// 0 cannot reach 3,4,5
tt.cut(1, 3)
tt.cut(1, 4)
tt.cut(1, 5)
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}})
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}})
sm := tt.peers[1].(*raft)
assert.Equal(t, uint64(1), sm.raftLog.committed)
// network recovery
tt.recover()
// elect 2 as the new leader with term 2
// after append a ChangeTerm entry from the current term, all entries
// should be committed
tt.send(pb.Message{From: 2, To: 2, Type: pb.MsgHup})
assert.Equal(t, uint64(4), sm.raftLog.committed)
}
func TestDuelingCandidates(t *testing.T) {
s1 := newTestMemoryStorage(withPeers(1, 2, 3))
s2 := newTestMemoryStorage(withPeers(1, 2, 3))
s3 := newTestMemoryStorage(withPeers(1, 2, 3))
a := newTestRaft(1, 10, 1, s1)
b := newTestRaft(2, 10, 1, s2)
c := newTestRaft(3, 10, 1, s3)
nt := newNetwork(a, b, c)
nt.cut(1, 3)
nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
// 1 becomes leader since it receives votes from 1 and 2
sm := nt.peers[1].(*raft)
assert.Equal(t, StateLeader, sm.state)
// 3 stays as candidate since it receives a vote from 3 and a rejection from 2
sm = nt.peers[3].(*raft)
assert.Equal(t, StateCandidate, sm.state)
nt.recover()
// candidate 3 now increases its term and tries to vote again
// we expect it to disrupt the leader 1 since it has a higher term
// 3 will be follower again since both 1 and 2 rejects its vote request since 3 does not have a long enough log
nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
assert.Equal(t, StateFollower, sm.state)
tests := []struct {
sm *raft
state StateType
term uint64
lastIndex uint64
}{
{a, StateFollower, 2, 1},
{b, StateFollower, 2, 1},
{c, StateFollower, 2, 0},
}
for i, tt := range tests {
assert.Equal(t, tt.state, tt.sm.state, "#%d", i)
assert.Equal(t, tt.term, tt.sm.Term, "#%d", i)
assert.Equal(t, tt.lastIndex, tt.sm.raftLog.lastIndex(), "#%d", i)
}
}
func TestDuelingPreCandidates(t *testing.T) {
cfgA := newTestConfig(1, 10, 1, newTestMemoryStorage(withPeers(1, 2, 3)))
cfgB := newTestConfig(2, 10, 1, newTestMemoryStorage(withPeers(1, 2, 3)))
cfgC := newTestConfig(3, 10, 1, newTestMemoryStorage(withPeers(1, 2, 3)))
cfgA.PreVote = true
cfgB.PreVote = true
cfgC.PreVote = true
a := newRaft(cfgA)
b := newRaft(cfgB)
c := newRaft(cfgC)
nt := newNetwork(a, b, c)
nt.t = t
nt.cut(1, 3)
nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
// 1 becomes leader since it receives votes from 1 and 2
sm := nt.peers[1].(*raft)
assert.Equal(t, StateLeader, sm.state)
// 3 campaigns then reverts to follower when its PreVote is rejected
sm = nt.peers[3].(*raft)
assert.Equal(t, StateFollower, sm.state)
nt.recover()
// Candidate 3 now increases its term and tries to vote again.
// With PreVote, it does not disrupt the leader.
nt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
tests := []struct {
sm *raft
state StateType
term uint64
lastIndex uint64
}{
{a, StateLeader, 1, 1},
{b, StateFollower, 1, 1},
{c, StateFollower, 1, 0},
}
for i, tt := range tests {
assert.Equal(t, tt.state, tt.sm.state, "#%d", i)
assert.Equal(t, tt.term, tt.sm.Term, "#%d", i)
assert.Equal(t, tt.lastIndex, tt.sm.raftLog.lastIndex(), "#%d", i)
}
}
func TestCandidateConcede(t *testing.T) {
tt := newNetwork(nil, nil, nil)
tt.isolate(1)
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
tt.send(pb.Message{From: 3, To: 3, Type: pb.MsgHup})
// heal the partition
tt.recover()
// send heartbeat; reset wait
tt.send(pb.Message{From: 3, To: 3, Type: pb.MsgBeat})
data := []byte("force follower")
// send a proposal to 3 to flush out a MsgApp to 1
tt.send(pb.Message{From: 3, To: 3, Type: pb.MsgProp, Entries: []pb.Entry{{Data: data}}})
// send heartbeat; flush out commit
tt.send(pb.Message{From: 3, To: 3, Type: pb.MsgBeat})
a := tt.peers[1].(*raft)
assert.Equal(t, StateFollower, a.state)
assert.Equal(t, uint64(1), a.Term)
wantLog := ltoa(newLog(&MemoryStorage{
ents: []pb.Entry{{}, {Data: nil, Term: 1, Index: 1}, {Term: 1, Index: 2, Data: data}},
}, nil))
for i, p := range tt.peers {
if sm, ok := p.(*raft); ok {
l := ltoa(sm.raftLog)
assert.Empty(t, diffu(wantLog, l), "#%d", i)
} else {
t.Logf("#%d: empty log", i)
}
}
}
func TestSingleNodeCandidate(t *testing.T) {
tt := newNetwork(nil)
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
sm := tt.peers[1].(*raft)
assert.Equal(t, StateLeader, sm.state)
}
func TestSingleNodePreCandidate(t *testing.T) {
tt := newNetworkWithConfig(preVoteConfig, nil)
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
sm := tt.peers[1].(*raft)
assert.Equal(t, StateLeader, sm.state)
}
func TestOldMessages(t *testing.T) {
tt := newNetwork(nil, nil, nil)
// make 0 leader @ term 3
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
tt.send(pb.Message{From: 2, To: 2, Type: pb.MsgHup})
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
// pretend we're an old leader trying to make progress; this entry is expected to be ignored.
tt.send(pb.Message{From: 2, To: 1, Type: pb.MsgApp, Term: 2, Entries: index(3).terms(2)})
// commit a new entry
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
ents := index(0).terms(0, 1, 2, 3, 3)
ents[4].Data = []byte("somedata")
ilog := newLog(&MemoryStorage{ents: ents}, nil)
base := ltoa(ilog)
for i, p := range tt.peers {
if sm, ok := p.(*raft); ok {
l := ltoa(sm.raftLog)
assert.Empty(t, diffu(base, l), "#%d", i)
} else {
t.Logf("#%d: empty log", i)
}
}
}
// TestOldMessagesReply - optimization - reply with new term.
func TestProposal(t *testing.T) {
tests := []struct {
*network
success bool
}{
{newNetwork(nil, nil, nil), true},
{newNetwork(nil, nil, nopStepper), true},
{newNetwork(nil, nopStepper, nopStepper), false},
{newNetwork(nil, nopStepper, nopStepper, nil), false},
{newNetwork(nil, nopStepper, nopStepper, nil, nil), true},
}
for j, tt := range tests {
send := func(m pb.Message) {
defer func() {
// only recover if we expect it to panic (success==false)
if !tt.success {
e := recover()
if e != nil {
t.Logf("#%d: err: %s", j, e)
}
}
}()
tt.send(m)
}
data := []byte("somedata")
// promote 1 to become leader
send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: data}}})
r := tt.network.peers[1].(*raft)
wantLog := newLog(NewMemoryStorage(), raftLogger)
if tt.success {
wantLog = newLog(&MemoryStorage{
ents: []pb.Entry{{}, {Data: nil, Term: 1, Index: 1}, {Term: 1, Index: 2, Data: data}},
}, nil)
}
base := ltoa(wantLog)
for i, p := range tt.peers {
if sm, ok := p.(*raft); ok {
l := ltoa(sm.raftLog)
assert.Empty(t, diffu(base, l), "#%d, peer %d", j, i)
} else {
t.Logf("#%d: peer %d empty log", j, i)
}
}
assert.Equal(t, uint64(1), r.Term, "#%d", j)
}
}
func TestProposalByProxy(t *testing.T) {
data := []byte("somedata")
tests := []*network{
newNetwork(nil, nil, nil),
newNetwork(nil, nil, nopStepper),
}
for j, tt := range tests {
// promote 0 the leader
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
// propose via follower
tt.send(pb.Message{From: 2, To: 2, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
wantLog := newLog(&MemoryStorage{
ents: []pb.Entry{{}, {Data: nil, Term: 1, Index: 1}, {Term: 1, Data: data, Index: 2}},
}, nil)
base := ltoa(wantLog)
for i, p := range tt.peers {
if sm, ok := p.(*raft); ok {
l := ltoa(sm.raftLog)
assert.Empty(t, diffu(base, l), "#%d.%d", j, i)
} else {
t.Logf("#%d: peer %d empty log", j, i)
}
}
sm := tt.peers[1].(*raft)
assert.Equal(t, uint64(1), sm.Term, "#%d", j)
}
}
func TestCommit(t *testing.T) {
tests := []struct {
matches []uint64
logs []pb.Entry
smTerm uint64
w uint64
}{
// single