-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
client_raft_test.go
4799 lines (4280 loc) · 158 KB
/
client_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 Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
package storage_test
import (
"bytes"
"context"
"fmt"
"math"
"math/rand"
"reflect"
"runtime"
"sync"
"sync/atomic"
"testing"
"time"
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/gossip"
"github.com/cockroachdb/cockroach/pkg/internal/client"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/rpc"
"github.com/cockroachdb/cockroach/pkg/rpc/nodedialer"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/engine"
"github.com/cockroachdb/cockroach/pkg/storage/stateloader"
"github.com/cockroachdb/cockroach/pkg/storage/storagebase"
"github.com/cockroachdb/cockroach/pkg/storage/storagepb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/humanizeutil"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/pkg/errors"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"go.etcd.io/etcd/raft"
"go.etcd.io/etcd/raft/raftpb"
"google.golang.org/grpc"
)
// mustGetInt decodes an int64 value from the bytes field of the receiver
// and panics if the bytes field is not 0 or 8 bytes in length.
func mustGetInt(v *roachpb.Value) int64 {
if v == nil {
return 0
}
i, err := v.GetInt()
if err != nil {
panic(err)
}
return i
}
// TestStoreRecoverFromEngine verifies that the store recovers all ranges and their contents
// after being stopped and recreated.
func TestStoreRecoverFromEngine(t *testing.T) {
defer leaktest.AfterTest(t)()
storeCfg := storage.TestStoreConfig(nil /* clock */)
storeCfg.TestingKnobs.DisableSplitQueue = true
storeCfg.TestingKnobs.DisableMergeQueue = true
const rangeID = roachpb.RangeID(1)
splitKey := roachpb.Key("m")
key1 := roachpb.Key("a")
key2 := roachpb.Key("z")
engineStopper := stop.NewStopper()
defer engineStopper.Stop(context.TODO())
eng := engine.NewInMem(roachpb.Attributes{}, 1<<20)
engineStopper.AddCloser(eng)
var rangeID2 roachpb.RangeID
get := func(store *storage.Store, rangeID roachpb.RangeID, key roachpb.Key) int64 {
args := getArgs(key)
resp, err := client.SendWrappedWith(context.Background(), store.TestSender(), roachpb.Header{
RangeID: rangeID,
}, args)
if err != nil {
t.Fatal(err)
}
return mustGetInt(resp.(*roachpb.GetResponse).Value)
}
validate := func(store *storage.Store) {
if val := get(store, rangeID, key1); val != 13 {
t.Errorf("key %q: expected 13 but got %v", key1, val)
}
if val := get(store, rangeID2, key2); val != 28 {
t.Errorf("key %q: expected 28 but got %v", key2, val)
}
}
// First, populate the store with data across two ranges. Each range contains commands
// that both predate and postdate the split.
func() {
stopper := stop.NewStopper()
defer stopper.Stop(context.TODO())
store := createTestStoreWithOpts(t,
testStoreOpts{
eng: eng,
cfg: &storeCfg,
// This test was written before the test stores were able to start with
// more than one range and is not prepared to handle many ranges.
dontCreateSystemRanges: true,
},
stopper)
increment := func(rangeID roachpb.RangeID, key roachpb.Key, value int64) (*roachpb.IncrementResponse, *roachpb.Error) {
args := incrementArgs(key, value)
resp, err := client.SendWrappedWith(context.Background(), store.TestSender(), roachpb.Header{
RangeID: rangeID,
}, args)
incResp, _ := resp.(*roachpb.IncrementResponse)
return incResp, err
}
if _, err := increment(rangeID, key1, 2); err != nil {
t.Fatal(err)
}
if _, err := increment(rangeID, key2, 5); err != nil {
t.Fatal(err)
}
splitArgs := adminSplitArgs(splitKey)
if _, err := client.SendWrapped(context.Background(), store.TestSender(), splitArgs); err != nil {
t.Fatal(err)
}
rangeID2 = store.LookupReplica(roachpb.RKey(key2)).RangeID
if rangeID2 == rangeID {
t.Fatal("got same range id after split")
}
if _, err := increment(rangeID, key1, 11); err != nil {
t.Fatal(err)
}
if _, err := increment(rangeID2, key2, 23); err != nil {
t.Fatal(err)
}
validate(store)
}()
// Now create a new store with the same engine and make sure the expected data is present.
// We must use the same clock because a newly-created manual clock will be behind the one
// we wrote with and so will see stale MVCC data.
store := createTestStoreWithOpts(t,
testStoreOpts{
dontBootstrap: true,
eng: eng,
cfg: &storeCfg,
},
engineStopper)
// Raft processing is initialized lazily; issue a no-op write request on each key to
// ensure that is has been started.
incArgs := incrementArgs(key1, 0)
if _, err := client.SendWrapped(context.Background(), store.TestSender(), incArgs); err != nil {
t.Fatal(err)
}
incArgs = incrementArgs(key2, 0)
if _, err := client.SendWrappedWith(context.Background(), store.TestSender(), roachpb.Header{
RangeID: rangeID2,
}, incArgs); err != nil {
t.Fatal(err)
}
validate(store)
}
// TestStoreRecoverWithErrors verifies that even commands that fail are marked as
// applied so they are not retried after recovery.
func TestStoreRecoverWithErrors(t *testing.T) {
defer leaktest.AfterTest(t)()
storeCfg := storage.TestStoreConfig(nil)
// Splits can cause our chosen keys to end up on ranges other than range 1,
// and trying to handle that complicates the test without providing any
// added benefit.
storeCfg.TestingKnobs.DisableSplitQueue = true
eng := engine.NewInMem(roachpb.Attributes{}, 1<<20)
defer eng.Close()
numIncrements := 0
func() {
stopper := stop.NewStopper()
defer stopper.Stop(context.TODO())
keyA := roachpb.Key("a")
storeCfg := storeCfg // copy
storeCfg.TestingKnobs.EvalKnobs.TestingEvalFilter =
func(filterArgs storagebase.FilterArgs) *roachpb.Error {
_, ok := filterArgs.Req.(*roachpb.IncrementRequest)
if ok && filterArgs.Req.Header().Key.Equal(keyA) {
numIncrements++
}
return nil
}
store := createTestStoreWithOpts(
t,
testStoreOpts{eng: eng, cfg: &storeCfg},
stopper)
// Write a bytes value so the increment will fail.
putArgs := putArgs(keyA, []byte("asdf"))
if _, err := client.SendWrapped(context.Background(), store.TestSender(), putArgs); err != nil {
t.Fatal(err)
}
// Try and fail to increment the key. It is important for this test that the
// failure be the last thing in the raft log when the store is stopped.
incArgs := incrementArgs(keyA, 42)
if _, err := client.SendWrapped(context.Background(), store.TestSender(), incArgs); err == nil {
t.Fatal("did not get expected error")
}
}()
if numIncrements != 1 {
t.Fatalf("expected 1 increments; was %d", numIncrements)
}
stopper := stop.NewStopper()
defer stopper.Stop(context.TODO())
// Recover from the engine.
store := createTestStoreWithOpts(t,
testStoreOpts{
dontBootstrap: true,
eng: eng,
cfg: &storeCfg,
},
stopper)
// Issue a no-op write to lazily initialize raft on the range.
keyB := roachpb.Key("b")
incArgs := incrementArgs(keyB, 0)
if _, err := client.SendWrapped(context.Background(), store.TestSender(), incArgs); err != nil {
t.Fatal(err)
}
// No additional increments were performed on key A during recovery.
if numIncrements != 1 {
t.Fatalf("expected 1 increments; was %d", numIncrements)
}
}
// TestReplicateRange verifies basic replication functionality by creating two stores
// and a range, replicating the range to the second store, and reading its data there.
func TestReplicateRange(t *testing.T) {
defer leaktest.AfterTest(t)()
mtc := &multiTestContext{
// This test was written before the multiTestContext started creating many
// system ranges at startup, and hasn't been update to take that into
// account.
startWithSingleRange: true,
}
defer mtc.Stop()
mtc.Start(t, 2)
// Issue a command on the first node before replicating.
incArgs := incrementArgs([]byte("a"), 5)
if _, err := client.SendWrapped(context.Background(), mtc.stores[0].TestSender(), incArgs); err != nil {
t.Fatal(err)
}
repl, err := mtc.stores[0].GetReplica(1)
if err != nil {
t.Fatal(err)
}
chgs := roachpb.MakeReplicationChanges(roachpb.ADD_REPLICA, roachpb.ReplicationTarget{
NodeID: mtc.stores[1].Ident.NodeID,
StoreID: mtc.stores[1].Ident.StoreID,
})
if _, err := repl.ChangeReplicas(context.Background(), repl.Desc(), storage.SnapshotRequest_REBALANCE, storagepb.ReasonRangeUnderReplicated, "", chgs); err != nil {
t.Fatal(err)
}
// Verify no intent remains on range descriptor key.
key := keys.RangeDescriptorKey(repl.Desc().StartKey)
desc := roachpb.RangeDescriptor{}
if ok, err := engine.MVCCGetProto(context.Background(), mtc.stores[0].Engine(), key,
mtc.stores[0].Clock().Now(), &desc, engine.MVCCGetOptions{}); err != nil {
t.Fatal(err)
} else if !ok {
t.Fatalf("range descriptor key %s was not found", key)
}
// Verify that in time, no intents remain on meta addressing
// keys, and that range descriptor on the meta records is correct.
testutils.SucceedsSoon(t, func() error {
meta2 := keys.RangeMetaKey(roachpb.RKeyMax)
meta1 := keys.RangeMetaKey(meta2)
for _, key := range []roachpb.RKey{meta2, meta1} {
metaDesc := roachpb.RangeDescriptor{}
if ok, err := engine.MVCCGetProto(context.Background(), mtc.stores[0].Engine(), key.AsRawKey(),
mtc.stores[0].Clock().Now(), &metaDesc, engine.MVCCGetOptions{}); err != nil {
return err
} else if !ok {
return errors.Errorf("failed to resolve %s", key.AsRawKey())
}
if !reflect.DeepEqual(metaDesc, desc) {
return errors.Errorf("descs not equal: %+v != %+v", metaDesc, desc)
}
}
return nil
})
// Verify that the same data is available on the replica.
testutils.SucceedsSoon(t, func() error {
getArgs := getArgs([]byte("a"))
if reply, err := client.SendWrappedWith(context.Background(), mtc.stores[1].TestSender(), roachpb.Header{
ReadConsistency: roachpb.INCONSISTENT,
}, getArgs); err != nil {
return errors.Errorf("failed to read data: %s", err)
} else if e, v := int64(5), mustGetInt(reply.(*roachpb.GetResponse).Value); v != e {
return errors.Errorf("failed to read correct data: expected %d, got %d", e, v)
}
return nil
})
}
// TestRestoreReplicas ensures that consensus group membership is properly
// persisted to disk and restored when a node is stopped and restarted.
func TestRestoreReplicas(t *testing.T) {
defer leaktest.AfterTest(t)()
t.Skip("https://github.com/cockroachdb/cockroach/issues/40351")
sc := storage.TestStoreConfig(nil)
// Disable periodic gossip activities. The periodic gossiping of the first
// range can cause spurious lease transfers which cause this test to fail.
sc.TestingKnobs.DisablePeriodicGossips = true
// Allow a replica to use the lease it had before a restart; we don't want
// this test to deal with needing to acquire new leases after the restart.
sc.TestingKnobs.DontPreventUseOfOldLeaseOnStart = true
mtc := &multiTestContext{
storeConfig: &sc,
// This test was written before the multiTestContext started creating many
// system ranges at startup, and hasn't been update to take that into
// account.
startWithSingleRange: true,
}
defer mtc.Stop()
mtc.Start(t, 2)
firstRng, err := mtc.stores[0].GetReplica(1)
if err != nil {
t.Fatal(err)
}
// Perform an increment before replication to ensure that commands are not
// repeated on restarts.
incArgs := incrementArgs([]byte("a"), 23)
if _, err := client.SendWrapped(context.Background(), mtc.stores[0].TestSender(), incArgs); err != nil {
t.Fatal(err)
}
chgs := roachpb.MakeReplicationChanges(roachpb.ADD_REPLICA, roachpb.ReplicationTarget{
NodeID: mtc.stores[1].Ident.NodeID,
StoreID: mtc.stores[1].Ident.StoreID,
})
if _, err := firstRng.ChangeReplicas(context.Background(), firstRng.Desc(), storage.SnapshotRequest_REBALANCE, storagepb.ReasonRangeUnderReplicated, "", chgs); err != nil {
t.Fatal(err)
}
mtc.restart()
// Send a command on each store. The original store (the lease holder still)
// will succeed.
incArgs = incrementArgs([]byte("a"), 5)
if _, err := client.SendWrapped(context.Background(), mtc.stores[0].TestSender(), incArgs); err != nil {
t.Fatal(err)
}
// The follower will return a not lease holder error, indicating the command
// should be forwarded to the lease holder.
incArgs = incrementArgs([]byte("a"), 11)
{
_, pErr := client.SendWrapped(context.Background(), mtc.stores[1].TestSender(), incArgs)
if _, ok := pErr.GetDetail().(*roachpb.NotLeaseHolderError); !ok {
t.Fatalf("expected not lease holder error; got %s", pErr)
}
}
// Send again, this time to first store.
if _, pErr := client.SendWrapped(context.Background(), mtc.stores[0].TestSender(), incArgs); pErr != nil {
t.Fatal(pErr)
}
testutils.SucceedsSoon(t, func() error {
getArgs := getArgs([]byte("a"))
if reply, err := client.SendWrappedWith(context.Background(), mtc.stores[1].TestSender(), roachpb.Header{
ReadConsistency: roachpb.INCONSISTENT,
}, getArgs); err != nil {
return errors.Errorf("failed to read data: %s", err)
} else if e, v := int64(39), mustGetInt(reply.(*roachpb.GetResponse).Value); v != e {
return errors.Errorf("failed to read correct data: expected %d, got %d", e, v)
}
return nil
})
// Both replicas have a complete list in Desc.Replicas
for i, store := range mtc.stores {
repl, err := store.GetReplica(1)
if err != nil {
t.Fatal(err)
}
desc := repl.Desc()
if len(desc.InternalReplicas) != 2 {
t.Fatalf("store %d: expected 2 replicas, found %d", i, len(desc.InternalReplicas))
}
if desc.InternalReplicas[0].NodeID != mtc.stores[0].Ident.NodeID {
t.Errorf("store %d: expected replica[0].NodeID == %d, was %d",
i, mtc.stores[0].Ident.NodeID, desc.InternalReplicas[0].NodeID)
}
}
}
// TODO(bdarnell): more aggressive testing here; especially with
// proposer-evaluated KV, what this test does is much less as it doesn't
// exercise the path in which the replica change fails at *apply* time (we only
// test the failfast path), in which case the replica change isn't even
// proposed.
func TestFailedReplicaChange(t *testing.T) {
defer leaktest.AfterTest(t)()
var runFilter atomic.Value
runFilter.Store(true)
sc := storage.TestStoreConfig(nil)
sc.TestingKnobs.EvalKnobs.TestingEvalFilter = func(filterArgs storagebase.FilterArgs) *roachpb.Error {
if runFilter.Load().(bool) {
if et, ok := filterArgs.Req.(*roachpb.EndTransactionRequest); ok && et.Commit {
return roachpb.NewErrorWithTxn(errors.Errorf("boom"), filterArgs.Hdr.Txn)
}
}
return nil
}
mtc := &multiTestContext{storeConfig: &sc}
defer mtc.Stop()
mtc.Start(t, 2)
repl, err := mtc.stores[0].GetReplica(1)
if err != nil {
t.Fatal(err)
}
chgs := roachpb.MakeReplicationChanges(roachpb.ADD_REPLICA, roachpb.ReplicationTarget{
NodeID: mtc.stores[1].Ident.NodeID,
StoreID: mtc.stores[1].Ident.StoreID,
})
if _, err := repl.ChangeReplicas(context.Background(), repl.Desc(), storage.SnapshotRequest_REBALANCE, storagepb.ReasonRangeUnderReplicated, "", chgs); !testutils.IsError(err, "boom") {
t.Fatalf("did not get expected error: %+v", err)
}
// After the aborted transaction, r.Desc was not updated.
// TODO(bdarnell): expose and inspect raft's internal state.
if replicas := repl.Desc().InternalReplicas; len(replicas) != 1 {
t.Fatalf("expected 1 replica, found %v", replicas)
}
// The pending config change flag was cleared, so a subsequent attempt
// can succeed.
runFilter.Store(false)
// The first failed replica change has laid down intents. Make sure those
// are pushable by making the transaction abandoned.
mtc.manualClock.Increment(10 * base.DefaultTxnHeartbeatInterval.Nanoseconds())
if _, err := repl.ChangeReplicas(context.Background(), repl.Desc(), storage.SnapshotRequest_REBALANCE, storagepb.ReasonRangeUnderReplicated, "", chgs); err != nil {
t.Fatal(err)
}
// Wait for the range to sync to both replicas (mainly so leaktest doesn't
// complain about goroutines involved in the process).
testutils.SucceedsSoon(t, func() error {
for _, store := range mtc.stores {
rang, err := store.GetReplica(1)
if err != nil {
return err
}
if replicas := rang.Desc().InternalReplicas; len(replicas) <= 1 {
return errors.Errorf("expected > 1 replicas; got %v", replicas)
}
}
return nil
})
}
// We can truncate the old log entries and a new replica will be brought up from a snapshot.
func TestReplicateAfterTruncation(t *testing.T) {
defer leaktest.AfterTest(t)()
mtc := &multiTestContext{
// This test was written before the multiTestContext started creating many
// system ranges at startup, and hasn't been update to take that into
// account.
startWithSingleRange: true,
}
defer mtc.Stop()
mtc.Start(t, 2)
repl, err := mtc.stores[0].GetReplica(1)
if err != nil {
t.Fatal(err)
}
// Issue a command on the first node before replicating.
incArgs := incrementArgs([]byte("a"), 5)
if _, err := client.SendWrapped(context.Background(), mtc.stores[0].TestSender(), incArgs); err != nil {
t.Fatal(err)
}
// Get that command's log index.
index, err := repl.GetLastIndex()
if err != nil {
t.Fatal(err)
}
// Truncate the log at index+1 (log entries < N are removed, so this includes
// the increment).
truncArgs := truncateLogArgs(index+1, 1)
if _, err := client.SendWrapped(context.Background(), mtc.stores[0].TestSender(), truncArgs); err != nil {
t.Fatal(err)
}
// Issue a second command post-truncation.
incArgs = incrementArgs([]byte("a"), 11)
if _, err := client.SendWrapped(context.Background(), mtc.stores[0].TestSender(), incArgs); err != nil {
t.Fatal(err)
}
// Now add the second replica.
chgs := roachpb.MakeReplicationChanges(roachpb.ADD_REPLICA, roachpb.ReplicationTarget{
NodeID: mtc.stores[1].Ident.NodeID,
StoreID: mtc.stores[1].Ident.StoreID,
})
if _, err := repl.ChangeReplicas(context.Background(), repl.Desc(), storage.SnapshotRequest_REBALANCE, storagepb.ReasonRangeUnderReplicated, "", chgs); err != nil {
t.Fatal(err)
}
// Once it catches up, the effects of both commands can be seen.
testutils.SucceedsSoon(t, func() error {
getArgs := getArgs([]byte("a"))
if reply, err := client.SendWrappedWith(context.Background(), mtc.stores[1].TestSender(), roachpb.Header{
ReadConsistency: roachpb.INCONSISTENT,
}, getArgs); err != nil {
return errors.Errorf("failed to read data: %s", err)
} else if e, v := int64(16), mustGetInt(reply.(*roachpb.GetResponse).Value); v != e {
return errors.Errorf("failed to read correct data: expected %d, got %d", e, v)
}
return nil
})
repl2, err := mtc.stores[1].GetReplica(1)
if err != nil {
t.Fatal(err)
}
testutils.SucceedsSoon(t, func() error {
if mvcc, mvcc2 := repl.GetMVCCStats(), repl2.GetMVCCStats(); mvcc2 != mvcc {
return errors.Errorf("expected stats on new range:\n%+v\not equal old:\n%+v", mvcc2, mvcc)
}
return nil
})
// Send a third command to verify that the log states are synced up so the
// new node can accept new commands.
incArgs = incrementArgs([]byte("a"), 23)
if _, err := client.SendWrapped(context.Background(), mtc.stores[0].TestSender(), incArgs); err != nil {
t.Fatal(err)
}
testutils.SucceedsSoon(t, func() error {
getArgs := getArgs([]byte("a"))
if reply, err := client.SendWrappedWith(context.Background(), mtc.stores[1].TestSender(), roachpb.Header{
ReadConsistency: roachpb.INCONSISTENT,
}, getArgs); err != nil {
return errors.Errorf("failed to read data: %s", err)
} else if e, v := int64(39), mustGetInt(reply.(*roachpb.GetResponse).Value); v != e {
return errors.Errorf("failed to read correct data: expected %d, got %d", e, v)
}
return nil
})
}
func TestRaftLogSizeAfterTruncation(t *testing.T) {
defer leaktest.AfterTest(t)()
mtc := &multiTestContext{
// This test was written before the multiTestContext started creating many
// system ranges at startup, and hasn't been update to take that into
// account.
startWithSingleRange: true,
}
defer mtc.Stop()
mtc.Start(t, 1)
const rangeID = 1
repl, err := mtc.stores[0].GetReplica(rangeID)
if err != nil {
t.Fatal(err)
}
key := []byte("a")
incArgs := incrementArgs(key, 5)
if _, err := client.SendWrapped(
context.Background(), mtc.stores[0].TestSender(), incArgs); err != nil {
t.Fatal(err)
}
index, err := repl.GetLastIndex()
if err != nil {
t.Fatal(err)
}
// Verifies the recomputed log size against what we track in `r.mu.raftLogSize`.
assertCorrectRaftLogSize := func() error {
// Recompute under raft lock so that the log doesn't change while we
// compute its size.
repl.RaftLock()
realSize, err := storage.ComputeRaftLogSize(
context.Background(), repl.RangeID, repl.Engine(), repl.SideloadedRaftMuLocked(),
)
size, _ := repl.GetRaftLogSize()
repl.RaftUnlock()
if err != nil {
t.Fatal(err)
}
// If the size isn't trusted, it won't have to match (and in fact
// likely won't). In this test, this is because the upreplication
// elides old Raft log entries in the snapshot it uses.
if size != realSize {
return fmt.Errorf("%s: raft log claims size %d, but is in fact %d", repl, size, realSize)
}
return nil
}
assert.NoError(t, assertCorrectRaftLogSize())
truncArgs := truncateLogArgs(index+1, 1)
if _, err := client.SendWrapped(
context.Background(), mtc.stores[0].TestSender(), truncArgs); err != nil {
t.Fatal(err)
}
// Note that if there were multiple nodes, the Raft log sizes would not
// be correct for the followers as they would have received a shorter
// Raft log than the leader.
assert.NoError(t, assertCorrectRaftLogSize())
}
// TestSnapshotAfterTruncation tests that Raft will properly send a
// non-preemptive snapshot when a node is brought up and the log has been
// truncated.
func TestSnapshotAfterTruncation(t *testing.T) {
defer leaktest.AfterTest(t)()
for _, changeTerm := range []bool{false, true} {
name := "sameTerm"
if changeTerm {
name = "differentTerm"
}
t.Run(name, func(t *testing.T) {
mtc := &multiTestContext{
// This test was written before the multiTestContext started creating many
// system ranges at startup, and hasn't been update to take that into
// account.
startWithSingleRange: true,
}
defer mtc.Stop()
mtc.Start(t, 3)
const stoppedStore = 1
repl0, err := mtc.stores[0].GetReplica(1)
if err != nil {
t.Fatal(err)
}
key := roachpb.Key("a")
incA := int64(5)
incB := int64(7)
incAB := incA + incB
// Set up a key to replicate across the cluster. We're going to modify this
// key and truncate the raft logs from that command after killing one of the
// nodes to check that it gets the new value after it comes up.
incArgs := incrementArgs(key, incA)
if _, err := client.SendWrapped(context.Background(), mtc.stores[0].TestSender(), incArgs); err != nil {
t.Fatal(err)
}
mtc.replicateRange(1, 1, 2)
mtc.waitForValues(key, []int64{incA, incA, incA})
// Now kill one store, increment the key on the other stores and truncate
// their logs to make sure that when store 1 comes back up it will require a
// non-preemptive snapshot from Raft.
mtc.stopStore(stoppedStore)
incArgs = incrementArgs(key, incB)
if _, err := client.SendWrapped(context.Background(), mtc.stores[0].TestSender(), incArgs); err != nil {
t.Fatal(err)
}
mtc.waitForValues(key, []int64{incAB, incA, incAB})
index, err := repl0.GetLastIndex()
if err != nil {
t.Fatal(err)
}
// Truncate the log at index+1 (log entries < N are removed, so this
// includes the increment).
truncArgs := truncateLogArgs(index+1, 1)
if _, err := client.SendWrapped(context.Background(), mtc.stores[0].TestSender(), truncArgs); err != nil {
t.Fatal(err)
}
if changeTerm {
for i := range mtc.stores {
if i != stoppedStore {
// Stop and restart all the live stores, which guarantees that
// we won't be in the same term we started with.
mtc.stopStore(i)
mtc.restartStore(i)
// Disable the snapshot queue on the live stores so that
// stoppedStore won't get a snapshot as soon as it starts
// back up.
mtc.stores[i].SetRaftSnapshotQueueActive(false)
}
}
// Restart the stopped store and wait for raft
// election/heartbeat traffic to settle down. Specifically, we
// need stoppedStore to know about the new term number before
// the snapshot is sent to reproduce #13506. If the snapshot
// happened before it learned the term, it would accept the
// snapshot no matter what term it contained.
//
// We do not wait for the store to successfully heartbeat
// because it is not expected to succeed in cases where the
// other two stores have already completed their leader
// election. In this case, a successful heartbeat won't be
// possible until we re-enable snapshots.
mtc.restartStoreWithoutHeartbeat(stoppedStore)
testutils.SucceedsSoon(t, func() error {
hasLeader := false
term := uint64(0)
for i := range mtc.stores {
repl, err := mtc.stores[i].GetReplica(1)
if err != nil {
return err
}
status := repl.RaftStatus()
if status == nil {
return errors.New("raft status not initialized")
}
if status.RaftState == raft.StateLeader {
hasLeader = true
}
if term == 0 {
term = status.Term
} else if status.Term != term {
return errors.Errorf("terms do not agree: %d vs %d", status.Term, term)
}
}
if !hasLeader {
return errors.New("no leader")
}
return nil
})
// Turn the queues back on and wait for the snapshot to be sent and processed.
for i, store := range mtc.stores {
if i != stoppedStore {
store.SetRaftSnapshotQueueActive(true)
if err := store.ForceRaftSnapshotQueueProcess(); err != nil {
t.Fatal(err)
}
}
}
} else { // !changeTerm
mtc.restartStore(stoppedStore)
}
mtc.waitForValues(key, []int64{incAB, incAB, incAB})
testutils.SucceedsSoon(t, func() error {
// Verify that the cached index and term (Replica.mu.last{Index,Term}))
// on all of the replicas is the same. #18327 fixed an issue where the
// cached term was left unchanged after applying a snapshot leading to a
// persistently unavailable range.
repl0, err = mtc.stores[0].GetReplica(1)
if err != nil {
t.Fatal(err)
}
expectedLastIndex, _ := repl0.GetLastIndex()
expectedLastTerm := repl0.GetCachedLastTerm()
for i := 1; i < len(mtc.stores); i++ {
repl1, err := mtc.stores[i].GetReplica(1)
if err != nil {
return err
}
if lastIndex, _ := repl1.GetLastIndex(); expectedLastIndex != lastIndex {
return fmt.Errorf("%d: expected last index %d, but found %d", i, expectedLastIndex, lastIndex)
}
if lastTerm := repl1.GetCachedLastTerm(); expectedLastTerm != lastTerm {
return fmt.Errorf("%d: expected last term %d, but found %d", i, expectedLastTerm, lastTerm)
}
}
return nil
})
})
}
}
// TestSnapshotAfterTruncationWithUncommittedTail is similar in spirit to
// TestSnapshotAfterTruncation/differentTerm. However, it differs in that we
// take care to ensure that the partitioned Replica has a long uncommitted tail
// of Raft entries that is not entirely overwritten by the snapshot it receives
// after the partition heals. If the recipient of the snapshot did not purge its
// Raft entry cache when receiving the snapshot, it could get stuck repeatedly
// rejecting attempts to catch it up. This serves as a regression test for the
// bug seen in #37056.
func TestSnapshotAfterTruncationWithUncommittedTail(t *testing.T) {
defer leaktest.AfterTest(t)()
ctx := context.Background()
mtc := &multiTestContext{
// This test was written before the multiTestContext started creating many
// system ranges at startup, and hasn't been update to take that into
// account.
startWithSingleRange: true,
}
defer mtc.Stop()
mtc.Start(t, 3)
key := roachpb.Key("a")
incA := int64(5)
incB := int64(7)
incC := int64(9)
incAB := incA + incB
incABC := incAB + incC
// Set up a key to replicate across the cluster. We're going to modify this
// key and truncate the raft logs from that command after partitioning one
// of the nodes to check that it gets the new value after it reconnects.
// We're then going to continue modifying this key to make sure that the
// temporarily partitioned node can continue to receive updates.
incArgs := incrementArgs(key, incA)
if _, pErr := client.SendWrapped(ctx, mtc.stores[0].TestSender(), incArgs); pErr != nil {
t.Fatal(pErr)
}
mtc.replicateRange(1, 1, 2)
mtc.waitForValues(key, []int64{incA, incA, incA})
// We partition the original leader from the other two replicas. This allows
// us to build up a large uncommitted Raft log on the partitioned node.
const partStore = 0
partRepl, err := mtc.stores[partStore].GetReplica(1)
if err != nil {
t.Fatal(err)
}
partReplDesc, err := partRepl.GetReplicaDescriptor()
if err != nil {
t.Fatal(err)
}
partReplSender := mtc.stores[partStore].TestSender()
// Partition the original leader from its followers. We do this by installing
// unreliableRaftHandler listeners on all three Stores. The handler on the
// partitioned store filters out all messages while the handler on the other
// two stores only filters out messages from the partitioned store. The
// configuration looks like:
//
// [0]
// x x
// / \
// x x
// [1]<---->[2]
//
for _, s := range []int{0, 1, 2} {
h := &unreliableRaftHandler{rangeID: 1, RaftMessageHandler: mtc.stores[s]}
if s != partStore {
// Only filter messages from the partitioned store on the other
// two stores.
h.dropReq = func(req *storage.RaftMessageRequest) bool {
return req.FromReplica.StoreID == partRepl.StoreID()
}
h.dropHB = func(hb *storage.RaftHeartbeat) bool {
return hb.FromReplicaID == partReplDesc.ReplicaID
}
}
mtc.transport.Listen(mtc.stores[s].Ident.StoreID, h)
}
// Perform a series of writes on the partitioned replica. The writes will
// not succeed before their context is canceled, but they will be appended
// to the partitioned replica's Raft log because it is currently the Raft
// leader.
g := ctxgroup.WithContext(ctx)
for i := 0; i < 32; i++ {
otherKey := roachpb.Key(fmt.Sprintf("other-%d", i))
g.GoCtx(func(ctx context.Context) error {
cCtx, cancel := context.WithTimeout(ctx, 50*time.Millisecond)
defer cancel()
incArgsOther := incrementArgs(otherKey, 1)
if _, pErr := client.SendWrapped(cCtx, partReplSender, incArgsOther); pErr == nil {
return errors.New("unexpected success")
} else if !testutils.IsPError(pErr, "context deadline exceeded") {
return pErr.GoError()
}
return nil
})
}
if err := g.Wait(); err != nil {
t.Fatal(err)
}
// Transfer the lease to one of the followers and perform a write. The
// partition ensures that this will require a Raft leadership change.
const newLeaderStore = partStore + 1
newLeaderRepl, err := mtc.stores[newLeaderStore].GetReplica(1)
if err != nil {
t.Fatal(err)
}
newLeaderReplSender := mtc.stores[newLeaderStore].TestSender()
incArgs = incrementArgs(key, incB)
testutils.SucceedsSoon(t, func() error {
mtc.advanceClock(ctx)
_, pErr := client.SendWrapped(ctx, newLeaderReplSender, incArgs)
if _, ok := pErr.GetDetail().(*roachpb.NotLeaseHolderError); ok {
return pErr.GoError()
} else if pErr != nil {
t.Fatal(pErr)
}
return nil
})
mtc.waitForValues(key, []int64{incA, incAB, incAB})
index, err := newLeaderRepl.GetLastIndex()
if err != nil {
t.Fatal(err)
}
// Truncate the log at index+1 (log entries < N are removed, so this
// includes the increment).
truncArgs := truncateLogArgs(index+1, 1)
testutils.SucceedsSoon(t, func() error {
mtc.advanceClock(ctx)
_, pErr := client.SendWrapped(ctx, newLeaderReplSender, truncArgs)
if _, ok := pErr.GetDetail().(*roachpb.NotLeaseHolderError); ok {
return pErr.GoError()
} else if pErr != nil {
t.Fatal(pErr)
}
return nil
})
snapsMetric := mtc.stores[partStore].Metrics().RangeSnapshotsNormalApplied
snapsBefore := snapsMetric.Count()
// Remove the partition. Snapshot should follow.
for _, s := range []int{0, 1, 2} {
mtc.transport.Listen(mtc.stores[s].Ident.StoreID, &unreliableRaftHandler{
rangeID: 1,
RaftMessageHandler: mtc.stores[s],
dropReq: func(req *storage.RaftMessageRequest) bool {
// Make sure that even going forward no MsgApp for what we just truncated can
// make it through. The Raft transport is asynchronous so this is necessary
// to make the test pass reliably.
// NB: the Index on the message is the log index that _precedes_ any of the
// entries in the MsgApp, so filter where msg.Index < index, not <= index.
return req.Message.Type == raftpb.MsgApp && req.Message.Index < index
},
dropHB: func(*storage.RaftHeartbeat) bool { return false },
dropResp: func(*storage.RaftMessageResponse) bool { return false },
})
}
// The partitioned replica should catch up after a snapshot.
testutils.SucceedsSoon(t, func() error {
snapsAfter := snapsMetric.Count()
if !(snapsAfter > snapsBefore) {
return errors.New("expected at least 1 snapshot to catch the partitioned replica up")
}
return nil
})
mtc.waitForValues(key, []int64{incAB, incAB, incAB})
// Perform another write. The partitioned replica should be able to receive