-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
Copy pathclient_merge_test.go
4769 lines (4325 loc) · 170 KB
/
client_merge_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 kvserver_test
import (
"bytes"
"context"
"fmt"
"math"
"math/rand"
"reflect"
"regexp"
"sort"
"strings"
"sync"
"sync/atomic"
"testing"
"time"
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/config/zonepb"
"github.com/cockroachdb/cockroach/pkg/gossip"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/txnwait"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/rpc"
"github.com/cockroachdb/cockroach/pkg/rpc/nodedialer"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"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/randutil"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/gogo/protobuf/proto"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"go.etcd.io/etcd/raft/v3/raftpb"
)
func adminMergeArgs(key roachpb.Key) *roachpb.AdminMergeRequest {
return &roachpb.AdminMergeRequest{
RequestHeader: roachpb.RequestHeader{
Key: key,
},
}
}
// createSplitRanges issues an AdminSplit command for the key "b". It returns
// the descriptors for the ranges to the left and right of the split.
func createSplitRanges(
ctx context.Context, store *kvserver.Store,
) (*roachpb.RangeDescriptor, *roachpb.RangeDescriptor, error) {
args := adminSplitArgs(roachpb.Key("b"))
if _, err := kv.SendWrapped(ctx, store.TestSender(), args); err != nil {
return nil, nil, err.GoError()
}
lhsDesc := store.LookupReplica(roachpb.RKey("a")).Desc()
rhsDesc := store.LookupReplica(roachpb.RKey("c")).Desc()
if bytes.Equal(lhsDesc.StartKey, rhsDesc.StartKey) {
return nil, nil, fmt.Errorf("split ranges have the same start key: %q = %q",
lhsDesc.StartKey, rhsDesc.StartKey)
}
return lhsDesc, rhsDesc, nil
}
// TestStoreRangeMergeTwoEmptyRanges tries to merge two empty ranges together.
func TestStoreRangeMergeTwoEmptyRanges(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
ctx := context.Background()
tc := testcluster.StartTestCluster(t, 1,
base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
})
defer tc.Stopper().Stop(context.Background())
store := tc.GetFirstStoreFromServer(t, 0)
lhsDesc, _, err := createSplitRanges(ctx, store)
if err != nil {
t.Fatal(err)
}
// Merge the RHS back into the LHS.
args := adminMergeArgs(lhsDesc.StartKey.AsRawKey())
_, pErr := kv.SendWrapped(ctx, store.TestSender(), args)
if pErr != nil {
t.Fatal(pErr)
}
// Verify the merge by looking up keys from both ranges.
lhsRepl := store.LookupReplica(roachpb.RKey("a"))
rhsRepl := store.LookupReplica(roachpb.RKey("c"))
if !reflect.DeepEqual(lhsRepl, rhsRepl) {
t.Fatalf("ranges were not merged: %s != %s", lhsRepl, rhsRepl)
}
// The LHS has been split once and merged once, so it should have received
// two generation bumps.
if e, a := roachpb.RangeGeneration(2), lhsRepl.Desc().Generation; e != a {
t.Fatalf("expected LHS to have generation %d, but got %d", e, a)
}
}
func getEngineKeySet(t *testing.T, e storage.Engine) map[string]struct{} {
t.Helper()
// Have to scan local and global keys separately as mentioned in the comment
// for storage.Scan (because of its use of intentInterleavingIter).
kvs, err := storage.Scan(e, roachpb.KeyMin, keys.LocalMax, 0 /* max */)
if err != nil {
t.Fatal(err)
}
globalKVs, err := storage.Scan(e, keys.LocalMax, roachpb.KeyMax, 0 /* max */)
if err != nil {
t.Fatal(err)
}
kvs = append(kvs, globalKVs...)
out := map[string]struct{}{}
for _, kv := range kvs {
out[string(kv.Key.Key)] = struct{}{}
}
return out
}
// TestStoreRangeMergeMetadataCleanup tests that all metadata of a
// subsumed range is cleaned up on merge.
func TestStoreRangeMergeMetadataCleanup(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
ctx := context.Background()
tc := testcluster.StartTestCluster(t, 1,
base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
})
defer tc.Stopper().Stop(context.Background())
store := tc.GetFirstStoreFromServer(t, 0)
content := roachpb.Key("testing!")
// Write some values left of the proposed split key.
pArgs := putArgs(roachpb.Key("aaa"), content)
if _, pErr := kv.SendWrapped(ctx, store.TestSender(), pArgs); pErr != nil {
t.Fatal(pErr)
}
// Collect all the keys.
preKeys := getEngineKeySet(t, store.Engine())
// Split the range.
lhsDesc, rhsDesc, err := createSplitRanges(ctx, store)
if err != nil {
t.Fatal(err)
}
// Write some values right of the split key.
pArgs = putArgs(roachpb.Key("ccc"), content)
if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{
RangeID: rhsDesc.RangeID,
}, pArgs); pErr != nil {
t.Fatal(pErr)
}
// Merge the b range back into the a range.
args := adminMergeArgs(lhsDesc.StartKey.AsRawKey())
if _, pErr := kv.SendWrapped(ctx, store.TestSender(), args); pErr != nil {
t.Fatal(pErr)
}
// Collect all the keys again.
postKeys := getEngineKeySet(t, store.Engine())
// Compute the new keys.
for k := range preKeys {
delete(postKeys, k)
}
tombstoneKey := string(keys.RangeTombstoneKey(rhsDesc.RangeID))
if _, ok := postKeys[tombstoneKey]; !ok {
t.Errorf("tombstone key (%s) missing after merge", roachpb.Key(tombstoneKey))
}
delete(postKeys, tombstoneKey)
// Keep only the subsumed range's local range-ID keys.
localRangeKeyPrefix := string(keys.MakeRangeIDPrefix(rhsDesc.RangeID))
for k := range postKeys {
if !strings.HasPrefix(k, localRangeKeyPrefix) {
delete(postKeys, k)
}
}
if numKeys := len(postKeys); numKeys > 0 {
var buf bytes.Buffer
fmt.Fprintf(&buf, "%d keys were not cleaned up:\n", numKeys)
for k := range postKeys {
fmt.Fprintf(&buf, "%s (%q)\n", roachpb.Key(k), k)
}
t.Fatal(buf.String())
}
}
// TestStoreRangeMergeWithData attempts to merge two ranges, each containing
// data.
func TestStoreRangeMergeWithData(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
for _, retries := range []int64{0, 3} {
t.Run(fmt.Sprintf("retries=%d", retries), func(t *testing.T) {
mergeWithData(t, retries)
})
}
}
func mergeWithData(t *testing.T, retries int64) {
ctx := context.Background()
manualClock := hlc.NewHybridManualClock()
var store *kvserver.Store
// Maybe inject some retryable errors when the merge transaction commits.
testingRequestFilter := func(_ context.Context, ba roachpb.BatchRequest) *roachpb.Error {
for _, req := range ba.Requests {
if et := req.GetEndTxn(); et != nil && et.InternalCommitTrigger.GetMergeTrigger() != nil {
if atomic.AddInt64(&retries, -1) >= 0 {
return roachpb.NewError(
roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE, "filter err"))
}
}
if req.GetSubsume() != nil {
// Introduce targeted chaos by forcing a lease acquisition before
// Subsume can execute. This triggers an unusual code path where the
// lease acquisition, not Subsume, notices the merge and installs a
// mergeComplete channel on the replica.
manualClock.Increment(store.GetStoreConfig().LeaseExpiration())
}
}
return nil
}
serv, _, _ := serverutils.StartServer(t, base.TestServerArgs{
Knobs: base.TestingKnobs{
Store: &kvserver.StoreTestingKnobs{
DisableMergeQueue: true,
DisableSplitQueue: true,
TestingRequestFilter: testingRequestFilter,
},
Server: &server.TestingKnobs{
ClockSource: manualClock.UnixNano,
},
},
})
s := serv.(*server.TestServer)
defer s.Stopper().Stop(ctx)
store, err := s.Stores().GetStore(s.GetFirstStoreID())
require.NoError(t, err)
scratchKey, err := s.ScratchRangeWithExpirationLease()
repl := store.LookupReplica(roachpb.RKey(scratchKey))
require.NoError(t, err)
lhsDesc, rhsDesc, pErr := s.SplitRange(scratchKey.Next().Next())
require.NoError(t, pErr)
content := []byte("testing!")
// Write some values left and right of the proposed split key.
put := func(key roachpb.Key, rangeID roachpb.RangeID, value []byte) {
pArgs := putArgs(key, value)
if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{
RangeID: rangeID,
}, pArgs); pErr != nil {
t.Fatal(pErr)
}
}
verify := func(key roachpb.Key, rangeID roachpb.RangeID, value []byte) {
// Confirm the values are there.
gArgs := getArgs(key)
if reply, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{
RangeID: rangeID,
}, gArgs); pErr != nil {
} else if replyBytes, err := reply.(*roachpb.GetResponse).Value.GetBytes(); err != nil {
t.Fatal(err)
} else if !bytes.Equal(replyBytes, value) {
t.Fatalf("actual value %q did not match expected value %q", replyBytes, content)
}
}
put(lhsDesc.StartKey.Next().AsRawKey(), lhsDesc.RangeID, content)
put(rhsDesc.StartKey.Next().AsRawKey(), rhsDesc.RangeID, content)
verify(lhsDesc.StartKey.Next().AsRawKey(), lhsDesc.RangeID, content)
verify(rhsDesc.StartKey.Next().AsRawKey(), rhsDesc.RangeID, content)
// Merge the b range back into the a range.
args := adminMergeArgs(lhsDesc.StartKey.AsRawKey())
if _, pErr := kv.SendWrapped(ctx, store.TestSender(), args); pErr != nil {
t.Fatal(pErr)
}
// Verify no intents remains on range descriptor keys.
for _, key := range []roachpb.Key{keys.RangeDescriptorKey(lhsDesc.StartKey), keys.RangeDescriptorKey(rhsDesc.StartKey)} {
if _, _, err := storage.MVCCGet(
ctx, store.Engine(), key, store.Clock().Now(), storage.MVCCGetOptions{},
); err != nil {
t.Fatal(err)
}
}
// Verify the merge by looking up keys from both ranges.
lhsRepl := store.LookupReplica(lhsDesc.StartKey.Next())
rhsRepl := store.LookupReplica(rhsDesc.StartKey.Next())
if lhsRepl != rhsRepl {
t.Fatalf("ranges were not merged %+v=%+v", lhsRepl.Desc(), rhsRepl.Desc())
}
if startKey := lhsRepl.Desc().StartKey; !bytes.Equal(startKey, repl.Desc().StartKey) {
t.Fatalf("The start key is not equal to KeyMin %q=%q", startKey, roachpb.RKeyMin)
}
if endKey := rhsRepl.Desc().EndKey; !bytes.Equal(endKey, repl.Desc().EndKey) {
t.Fatalf("The end key is not equal to KeyMax %q=%q", endKey, roachpb.RKeyMax)
}
verify(lhsDesc.StartKey.Next().AsRawKey(), lhsRepl.RangeID, content)
verify(rhsDesc.StartKey.Next().AsRawKey(), rhsRepl.RangeID, content)
newContent := []byte("testing!better!")
// Put new values after the merge on both sides.
put(lhsDesc.StartKey.Next().AsRawKey(), lhsRepl.RangeID, newContent)
put(rhsDesc.StartKey.Next().AsRawKey(), rhsRepl.RangeID, newContent)
// Try to get the newly placed values.
verify(lhsDesc.StartKey.Next().AsRawKey(), lhsRepl.RangeID, newContent)
verify(rhsDesc.StartKey.Next().AsRawKey(), rhsRepl.RangeID, newContent)
gArgs := getArgs(lhsDesc.StartKey.Next().AsRawKey())
if _, pErr := kv.SendWrappedWith(ctx, store, roachpb.Header{
RangeID: rhsDesc.RangeID,
}, gArgs); !testutils.IsPError(
pErr, `was not found on s`,
) {
t.Fatalf("expected get on rhs to fail after merge, but got err=%v", pErr)
}
if atomic.LoadInt64(&retries) >= 0 {
t.Fatalf("%d retries remaining (expected less than zero)", retries)
}
}
// TestStoreRangeMergeTimestampCache verifies that the timestamp cache on the
// LHS is properly updated after a merge. The test contains a subtest for each
// of the combinations of the following boolean options:
//
// - disjointLeaseholders: configures whether or not the leaseholder of the
// LHS range is disjoint from the leaseholder of the RHS range. If false,
// the leaseholders are collocated before the merge is initiated.
//
// - throughSnapshot: configures whether or not the leaseholder of the LHS of
// the merge hears about and applies the merge through a Raft snapshot, as
// opposed to through normal Raft log application.
//
// - futureRead: configures whether or not the reads performed on the RHS range
// before the merge is initiated are performed in the future of present
// time using synthetic timestamps.
//
func TestStoreRangeMergeTimestampCache(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
skip.UnderShort(t)
testutils.RunTrueAndFalse(t, "disjoint-leaseholders", func(t *testing.T, disjointLeaseholders bool) {
testutils.RunTrueAndFalse(t, "through-snapshot", func(t *testing.T, throughSnapshot bool) {
testutils.RunTrueAndFalse(t, "future-read", func(t *testing.T, futureRead bool) {
mergeCheckingTimestampCaches(t, disjointLeaseholders, throughSnapshot, futureRead)
})
})
})
}
func mergeCheckingTimestampCaches(
t *testing.T, disjointLeaseholders, throughSnapshot, futureRead bool,
) {
// mergeCommitFilter is used to issue a sequence of operations on the LHS of
// a range merge immediately before it.
var mergeCommitFilter func()
// blockHBAndGCs is used to black hole Heartbeat and GC requests for the
// duration of the merge on the throughSnapshot path. Neither request type
// is needed and both can create issues by holding latches during the split
// leader-leaseholder state.
var blockHBAndGCs chan struct{}
var filterMu syncutil.Mutex
testingRequestFilter := func(_ context.Context, ba roachpb.BatchRequest) *roachpb.Error {
filterMu.Lock()
mergeCommitFilterCopy := mergeCommitFilter
blockHBAndGCsCopy := blockHBAndGCs
filterMu.Unlock()
for _, req := range ba.Requests {
switch v := req.GetInner().(type) {
case *roachpb.EndTxnRequest:
if v.InternalCommitTrigger.GetMergeTrigger() != nil {
if mergeCommitFilterCopy != nil {
mergeCommitFilterCopy()
}
}
case *roachpb.HeartbeatTxnRequest, *roachpb.GCRequest:
if blockHBAndGCsCopy != nil {
<-blockHBAndGCsCopy
}
}
}
return nil
}
// snapshotFilter is used to listen for the completion of a Raft snapshot.
var snapshotFilter func(kvserver.IncomingSnapshot)
beforeSnapshotSSTIngestion := func(
inSnap kvserver.IncomingSnapshot,
snapType kvserver.SnapshotRequest_Type,
_ []string,
) error {
filterMu.Lock()
snapshotFilterCopy := snapshotFilter
filterMu.Unlock()
if snapshotFilterCopy != nil {
snapshotFilterCopy(inSnap)
}
return nil
}
manualClock := hlc.NewHybridManualClock()
ctx := context.Background()
tc := testcluster.StartTestCluster(t, 3,
base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgs: base.TestServerArgs{
Knobs: base.TestingKnobs{
Server: &server.TestingKnobs{
ClockSource: manualClock.UnixNano,
},
Store: &kvserver.StoreTestingKnobs{
TestingRequestFilter: testingRequestFilter,
BeforeSnapshotSSTIngestion: beforeSnapshotSSTIngestion,
},
},
},
})
defer tc.Stopper().Stop(ctx)
lhsStore := tc.GetFirstStoreFromServer(t, 0)
var rhsStore *kvserver.Store
if disjointLeaseholders {
rhsStore = tc.GetFirstStoreFromServer(t, 1)
} else {
rhsStore = tc.GetFirstStoreFromServer(t, 0)
}
// Disable closed timestamps to ensure that any writes that are bumped to
// higher timestamps are bumped by the timestamp cache, as expected.
_, err := tc.ServerConn(0).Exec(`SET CLUSTER SETTING kv.closed_timestamp.target_duration = '24h'`)
require.NoError(t, err)
lhsDesc, rhsDesc, err := createSplitRanges(ctx, lhsStore)
require.NoError(t, err)
tc.AddVotersOrFatal(t, lhsDesc.StartKey.AsRawKey(), tc.Target(1), tc.Target(2))
tc.AddVotersOrFatal(t, rhsDesc.StartKey.AsRawKey(), tc.Target(1), tc.Target(2))
if disjointLeaseholders {
tc.TransferRangeLeaseOrFatal(t, *rhsDesc, tc.Target(1))
testutils.SucceedsSoon(t, func() error {
rhsRepl, err := rhsStore.GetReplica(rhsDesc.RangeID)
if err != nil {
return err
}
if !rhsRepl.OwnsValidLease(ctx, tc.Servers[1].Clock().NowAsClockTimestamp()) {
return errors.New("rhs store does not own valid lease for rhs range")
}
return nil
})
}
// Write a key to the RHS.
rhsKey := roachpb.Key("c")
if _, pErr := kv.SendWrappedWith(ctx, rhsStore, roachpb.Header{
RangeID: rhsDesc.RangeID,
}, incrementArgs(rhsKey, 1)); pErr != nil {
t.Fatal(pErr)
}
readTS := tc.Servers[0].Clock().Now()
if futureRead {
readTS = readTS.Add(500*time.Millisecond.Nanoseconds(), 0).WithSynthetic(true)
}
// Simulate a read on the RHS from a node with a newer clock.
var ba roachpb.BatchRequest
ba.Timestamp = readTS
ba.RangeID = rhsDesc.RangeID
ba.Add(getArgs(rhsKey))
if br, pErr := rhsStore.Send(ctx, ba); pErr != nil {
t.Fatal(pErr)
} else if v, err := br.Responses[0].GetGet().Value.GetInt(); err != nil {
t.Fatal(err)
} else if v != 1 {
t.Fatalf("expected 1, but got %d", v)
} else if br.Timestamp != readTS {
t.Fatalf("expected read to execute at %v, but executed at %v", readTS, br.Timestamp)
}
// Simulate a txn abort on the RHS from a node with a newer clock. Because
// the transaction record for the pushee was not yet written, this will bump
// the timestamp cache to record the abort.
pushee := roachpb.MakeTransaction("pushee", rhsKey, roachpb.MinUserPriority, readTS, 0)
pusher := roachpb.MakeTransaction("pusher", rhsKey, roachpb.MaxUserPriority, readTS, 0)
ba = roachpb.BatchRequest{}
ba.Timestamp = readTS.Next()
ba.RangeID = rhsDesc.RangeID
ba.Add(pushTxnArgs(&pusher, &pushee, roachpb.PUSH_ABORT))
if br, pErr := rhsStore.Send(ctx, ba); pErr != nil {
t.Fatal(pErr)
} else if txn := br.Responses[0].GetPushTxn().PusheeTxn; txn.Status != roachpb.ABORTED {
t.Fatalf("expected aborted pushee, but got %v", txn)
}
// Pause the cluster's clock. This accomplishes two things:
// 1. It ensures that if we force the LHS leaseholder to learn about the
// merge through a snapshot (throughSnapshot), the merge transaction is not
// allowed to expire and be aborted due to delayed txn heartbeats.
// 2. it ensures that if we performed a read at a future timestamp, the read
// time remains in the future, regardless of the passage of real time.
manualClock.Pause()
if !throughSnapshot {
// The easy case: merge the RHS back into the LHS normally.
args := adminMergeArgs(lhsDesc.StartKey.AsRawKey())
_, pErr := kv.SendWrapped(ctx, lhsStore.TestSender(), args)
require.Nil(t, pErr)
} else {
// The hard case: merge the RHS back into the LHS, but make sure the LHS
// leaseholder finds out about the merge through a Raft snapshot.
//
// To do this, we partition the leaseholder from the rest of its range.
// Once partitioned, we perform another write and truncate the Raft log
// on the two connected nodes. We then complete the range merge before
// removing the partition. This ensures that that when the leaseholder
// reconnects it will require a snapshot from Raft.
//
// But there's a wrinkle here that makes things more difficult: the
// leaseholder needs to play a role in coordinating the range merge and
// the log truncation, as it is the only replica that can propose such
// changes. To accommodate this, we put the range into a split
// leader-leaseholder state and lock down all communication between the
// two _except_ for forwarded proposal from the leaseholder to the
// leader. This allows the leaseholder to make proposals, even though it
// won't be able to hear their result. Because this is such a fragile
// state, we enter it as late as possible - after the merge begins and
// only upon receiving the merge's EndTxn request.
lhsKey := roachpb.Key("a")
var lhsStores []*kvserver.Store
var lhsRepls []*kvserver.Replica
for i := range tc.Servers {
s := tc.GetFirstStoreFromServer(t, i)
r := s.LookupReplica(roachpb.RKey(lhsKey))
lhsStores = append(lhsStores, s)
lhsRepls = append(lhsRepls, r)
}
// Applied to the leaseholder's raft transport during the partition.
partitionedLeaseholderFuncs := noopRaftHandlerFuncs()
partitionedLeaseholderFuncs.dropReq = func(*kvserver.RaftMessageRequest) bool {
// Ignore everything from new leader.
return true
}
// Applied to the leader and other follower's raft transport during the
// partition.
partitionedLeaderFuncs := noopRaftHandlerFuncs()
partitionedLeaderFuncs.dropReq = func(req *kvserver.RaftMessageRequest) bool {
// Ignore everything from leaseholder, except forwarded proposals.
return req.FromReplica.StoreID == lhsStore.StoreID() &&
req.Message.Type != raftpb.MsgProp
}
partitionedLeaderFuncs.dropHB = func(hb *kvserver.RaftHeartbeat) bool {
// Ignore heartbeats from leaseholder, results in campaign.
return hb.FromReplicaID == roachpb.ReplicaID(lhsRepls[0].RaftStatus().ID)
}
// Applied to leaseholder after the partition heals.
var truncIndex uint64
restoredLeaseholderFuncs := noopRaftHandlerFuncs()
restoredLeaseholderFuncs.dropReq = func(req *kvserver.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 - otherwise
// the leaseholder may catch up without needing a snapshot, tripping
// up the test.
//
// 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 < truncIndex
}
// Because we enter a split leader-leaseholder state, none of the
// operations we perform on the leaseholder will return. Instead, they
// will block for the duration of the partition, even after they have
// succeeded on the majority quorum. So we launch async goroutines to
// perform the write and the log truncation and only wait for them to
// complete after the partition heals.
incChan := make(chan *roachpb.Error, 1)
truncChan := make(chan *roachpb.Error, 1)
snapChan := make(chan kvserver.IncomingSnapshot, 1)
filterMu.Lock()
mergeCommitFilter = func() {
// Install leader-leaseholder partition.
for i, s := range lhsStores {
var funcs unreliableRaftHandlerFuncs
if i == 0 {
funcs = partitionedLeaseholderFuncs
} else {
funcs = partitionedLeaderFuncs
}
tc.Servers[i].RaftTransport().Listen(s.StoreID(), &unreliableRaftHandler{
rangeID: lhsDesc.GetRangeID(),
RaftMessageHandler: s,
unreliableRaftHandlerFuncs: funcs,
})
}
// Make sure the LHS range in uniquiesced so that it elects a new
// Raft leader after the partition is established.
for _, r := range lhsRepls {
r.UnquiesceAndWakeLeader()
}
// Issue an increment on the range. The leaseholder should evaluate
// the request and forward a proposal to the leader, but it should
// be the only replica that does not apply the proposal.
go func() {
incArgs := incrementArgs(lhsKey, 4)
_, pErr := kv.SendWrappedWith(ctx, lhsStore, roachpb.Header{RangeID: lhsDesc.RangeID}, incArgs)
incChan <- pErr
}()
// NB: the operation won't complete, so peek below Raft and wait for
// the result to apply on the majority quorum.
tc.WaitForValues(t, lhsKey, []int64{0, 4, 4})
// Truncate the log to eventually force a snapshot. Determining
// which log index to truncate is tricky. We need to make sure it is
// <= to the largest log index on the leaseholder or it will reject
// the request. But we also need to make sure it is <= to the
// largest log index on the leader, or it will panic. So we choose
// the minimum of these two and just pick the smallest "last index"
// in the range, which does the trick.
min := func(a, b uint64) uint64 {
if a < b {
return a
}
return b
}
minLastIndex := uint64(math.MaxUint64)
for _, r := range lhsRepls {
lastIndex, err := r.GetLastIndex()
require.NoError(t, err)
minLastIndex = min(minLastIndex, lastIndex)
}
// Truncate the log at index+1 (log entries < N are removed).
truncIndex = minLastIndex + 1
go func() {
truncArgs := truncateLogArgs(truncIndex, lhsDesc.RangeID)
truncArgs.Key = lhsKey
_, pErr := kv.SendWrappedWith(ctx, lhsStore, roachpb.Header{RangeID: lhsDesc.RangeID}, truncArgs)
truncChan <- pErr
}()
// NB: the operation won't complete, so peek below Raft and wait for
// the result to apply on the majority quorum.
testutils.SucceedsSoon(t, func() error {
for _, r := range lhsRepls[1:] {
firstIndex, err := r.GetFirstIndex()
require.NoError(t, err)
if firstIndex < truncIndex {
return errors.Errorf("truncate not applied, %d < %d", firstIndex, truncIndex)
}
}
return nil
})
}
// Begin blocking txn heartbeats and GC requests. They cause issues
// because they can grab latches and then get stuck once in the split
// leader-leaseholder state.
blockHBAndGCs = make(chan struct{})
// Install a filter to capture the Raft snapshot.
snapshotFilter = func(inSnap kvserver.IncomingSnapshot) {
if inSnap.State.Desc.RangeID == lhsDesc.RangeID {
snapChan <- inSnap
}
}
filterMu.Unlock()
// Merge the RHS back into the LHS.
mergeChan := make(chan *roachpb.Error, 1)
go func() {
args := adminMergeArgs(lhsDesc.StartKey.AsRawKey())
_, pErr := kv.SendWrapped(ctx, lhsStore.TestSender(), args)
mergeChan <- pErr
}()
// NB: the operation won't complete, so peek below Raft and wait for
// the result to apply on the majority quorum.
testutils.SucceedsSoon(t, func() error {
for _, r := range lhsRepls[1:] {
desc := r.Desc()
if !desc.EndKey.Equal(rhsDesc.EndKey) {
return errors.Errorf("merge not applied")
}
}
return nil
})
// Remove the partition. A snapshot to the leaseholder should follow.
// This snapshot will inform the leaseholder about the range merge.
for i, s := range lhsStores {
var h kvserver.RaftMessageHandler
if i == 0 {
h = &unreliableRaftHandler{
rangeID: lhsDesc.GetRangeID(),
RaftMessageHandler: s,
unreliableRaftHandlerFuncs: restoredLeaseholderFuncs,
}
} else {
h = s
}
tc.Servers[i].RaftTransport().Listen(s.StoreID(), h)
}
close(blockHBAndGCs)
t.Logf("waiting for snapshot to LHS leaseholder")
inSnap := <-snapChan
inSnapDesc := inSnap.State.Desc
require.Equal(t, lhsDesc.StartKey, inSnapDesc.StartKey)
require.Equal(t, rhsDesc.EndKey, inSnapDesc.EndKey)
// Wait for all async ops to complete.
for _, asyncRes := range []struct {
name string
ch chan *roachpb.Error
}{
{"increment", incChan},
{"truncate", truncChan},
{"merge", mergeChan},
} {
t.Logf("waiting for result of %s", asyncRes.name)
err := <-asyncRes.ch
require.NotNil(t, err, "%s should fail", asyncRes.name)
require.Regexp(t, "result is ambiguous", err, "%s's result should be ambiguous", asyncRes.name)
}
}
// After the merge, attempt to write under the read. The batch should get
// forwarded to a timestamp after the read.
ba = roachpb.BatchRequest{}
ba.Timestamp = readTS
ba.RangeID = lhsDesc.RangeID
ba.Add(incrementArgs(rhsKey, 1))
if br, pErr := lhsStore.Send(ctx, ba); pErr != nil {
t.Fatal(pErr)
} else if br.Timestamp.LessEq(readTS) {
t.Fatalf("expected write to execute after %v, but executed at %v", readTS, br.Timestamp)
}
// Attempt to create a transaction record for the pushee transaction, which
// was aborted before the merge. This should be rejected with a transaction
// aborted error. The reason will depend on whether the leaseholders were
// disjoint or not because disjoint leaseholders will lead to a loss of
// resolution in the timestamp cache. Similarly, the reason will depend on
// whether the LHS leaseholder learned about the merge through Raft log
// application or a Raft snapshot. Either way though, the transaction should
// not be allowed to create its record.
hb, hbH := heartbeatArgs(&pushee, tc.Servers[0].Clock().Now())
ba = roachpb.BatchRequest{}
ba.Header = hbH
ba.RangeID = lhsDesc.RangeID
ba.Add(hb)
var expReason roachpb.TransactionAbortedReason
if disjointLeaseholders || throughSnapshot {
expReason = roachpb.ABORT_REASON_TIMESTAMP_CACHE_REJECTED
} else {
expReason = roachpb.ABORT_REASON_ABORTED_RECORD_FOUND
}
if _, pErr := lhsStore.Send(ctx, ba); pErr == nil {
t.Fatalf("expected TransactionAbortedError(%s) but got %v", expReason, pErr)
} else if abortErr, ok := pErr.GetDetail().(*roachpb.TransactionAbortedError); !ok {
t.Fatalf("expected TransactionAbortedError(%s) but got %v", expReason, pErr)
} else if abortErr.Reason != expReason {
t.Fatalf("expected TransactionAbortedError(%s) but got %v", expReason, pErr)
}
}
// TestStoreRangeMergeTimestampCacheCausality verifies that range merges update
// the clock on the subsuming store as necessary to preserve causality.
//
// The test simulates a particularly diabolical sequence of events in which
// causality information is not communicated through the normal channels.
// Suppose two adjacent ranges, A and B, are collocated on S2, S3, and S4. (S1
// is omitted for consistency with the store numbering in the test itself.) S3
// holds the lease on A, while S4 holds the lease on B. Every store's clock
// starts at time T1.
//
// To merge A and B, S3 will launch a merge transaction that sends several RPCs
// to S4. Suppose that, just before S4 begins executing the Subsume request, a
// read sneaks in for some key K at a large timestamp T3. S4 will bump its clock
// from T1 to T3, so when the Subsume goes to determine the current time to use
// for the FreezeStart field in the Subsume response, it will use T3. When S3
// completes the merge, it will thus use T3 as the timestamp cache's low water
// mark for the keys that previously belonged to B.
//
// Importantly, S3 must also update its clock from T1 to T3. Otherwise, as this
// test demonstrates, it is possible for S3 to send a lease to another store, in
// this case S2, that begins at T2. S2 will then assume it is free to accept a
// write at T2, when in fact we already served a read at T3. This would be a
// serializability violation!
//
// Note that there are several mechanisms that *almost* prevent this problem. If
// the read of K at T3 occurs slightly earlier, the batch response for Subsume
// will set the Now field to T3, which S3 will use to bump its clock.
// (BatchResponse.Now is computed when the batch is received, not when it
// finishes executing.) If S3 receives a write for K at T2, it will a) properly
// bump the write to T4, because its timestamp cache is up to date, and then b)
// bump its clock to T4. Or if S4 were to send a single RPC to S3, S3 would bump
// its clock based on the BatchRequest.Timestamp.
//
// In short, this sequence of events is likely to be exceedingly unlikely in
// practice, but is subtle enough to warrant a test.
func TestStoreRangeMergeTimestampCacheCausality(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
ctx := context.Background()
var readTS hlc.Timestamp
rhsKey := roachpb.Key("c")
var tc *testcluster.TestCluster
testingRequestFilter := func(_ context.Context, ba roachpb.BatchRequest) *roachpb.Error {
if ba.IsSingleSubsumeRequest() {
// Before we execute a Subsume request, execute a read on the same store
// at a much higher timestamp.
gba := roachpb.BatchRequest{}
gba.RangeID = ba.RangeID
gba.Timestamp = ba.Timestamp.Add(42 /* wallTime */, 0 /* logical */)
gba.Add(getArgs(rhsKey))
store := tc.GetFirstStoreFromServer(t, int(ba.Header.Replica.StoreID-1))
gbr, pErr := store.Send(ctx, gba)
if pErr != nil {
t.Error(pErr) // different goroutine, so can't use t.Fatal
}
readTS = gbr.Timestamp
}
return nil
}
tc = testcluster.StartTestCluster(t, 4,
base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgs: base.TestServerArgs{
Knobs: base.TestingKnobs{
Store: &kvserver.StoreTestingKnobs{
TestingRequestFilter: testingRequestFilter,
},
},
},
})
defer tc.Stopper().Stop(context.Background())
distSender := tc.Servers[0].DistSender()
for _, key := range []roachpb.Key{roachpb.Key("a"), roachpb.Key("b")} {
if _, pErr := kv.SendWrapped(ctx, distSender, adminSplitArgs(key)); pErr != nil {
t.Fatal(pErr)
}
}
lhsRangeDesc := tc.GetFirstStoreFromServer(t, 0).LookupReplica(roachpb.RKey("a")).Desc()
rhsRangeDesc := tc.GetFirstStoreFromServer(t, 0).LookupReplica(roachpb.RKey("b")).Desc()
// Replicate [a, b) to s2, s3, and s4, and put the lease on s3.
tc.AddVotersOrFatal(t, lhsRangeDesc.StartKey.AsRawKey(), tc.Targets(1, 2, 3)...)
tc.TransferRangeLeaseOrFatal(t, *lhsRangeDesc, tc.Target(2))
tc.RemoveVotersOrFatal(t, lhsRangeDesc.StartKey.AsRawKey(), tc.Target(0))
// Replicate [b, Max) to s2, s3, and s4, and put the lease on s4.
tc.AddVotersOrFatal(t, rhsRangeDesc.StartKey.AsRawKey(), tc.Targets(1, 2, 3)...)
tc.TransferRangeLeaseOrFatal(t, *rhsRangeDesc, tc.Target(3))
tc.RemoveVotersOrFatal(t, rhsRangeDesc.StartKey.AsRawKey(), tc.Target(0))
// N.B. We isolate r1 on s1 so that node liveness heartbeats do not interfere
// with our precise clock management on s2, s3, and s4.
// Write a key to [b, Max).
if _, pErr := kv.SendWrapped(ctx, distSender, incrementArgs(rhsKey, 1)); pErr != nil {
t.Fatal(pErr)
}
// Wait for all relevant stores to have the same value. This indirectly
// ensures the lease transfers have applied on all relevant stores.
tc.WaitForValues(t, rhsKey, []int64{0, 1, 1, 1})
// Merge [a, b) and [b, Max). Our request filter above will intercept the
// merge and execute a read with a large timestamp immediately before the
// Subsume request executes.
if _, pErr := kv.SendWrappedWith(ctx, tc.GetFirstStoreFromServer(t, 2), roachpb.Header{
RangeID: lhsRangeDesc.RangeID,
}, adminMergeArgs(roachpb.Key("a"))); pErr != nil {
t.Fatal(pErr)
}
// Immediately transfer the lease on the merged range [a, Max) from s3 to s2.
// To test that it is, in fact, the merge trigger that properly bumps s3's
// clock, s3 must not send or receive any requests before it transfers the
// lease, as those requests could bump s3's clock through other code paths.
tc.TransferRangeLeaseOrFatal(t, *lhsRangeDesc, tc.Target(1))
testutils.SucceedsSoon(t, func() error {
lhsRepl1, err := tc.GetFirstStoreFromServer(t, 1).GetReplica(lhsRangeDesc.RangeID)
if err != nil {
return err
}
if !lhsRepl1.OwnsValidLease(ctx, tc.Servers[1].Clock().NowAsClockTimestamp()) {
return errors.New("s2 does not own valid lease for lhs range")
}
return nil
})
// Attempt to write at the same time as the read. The write's timestamp
// should be forwarded to after the read.
ba := roachpb.BatchRequest{}
ba.Timestamp = readTS
ba.RangeID = lhsRangeDesc.RangeID
ba.Add(incrementArgs(rhsKey, 1))
if br, pErr := tc.GetFirstStoreFromServer(t, 1).Send(ctx, ba); pErr != nil {
t.Fatal(pErr)
} else if br.Timestamp.LessEq(readTS) {
t.Fatalf("expected write to execute after %v, but executed at %v", readTS, br.Timestamp)
}
}
// TestStoreRangeMergeLastRange verifies that merging the last range fails.
func TestStoreRangeMergeLastRange(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
ctx := context.Background()
tc := testcluster.StartTestCluster(t, 1,
base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
})
defer tc.Stopper().Stop(context.Background())
store := tc.GetFirstStoreFromServer(t, 0)
// Find the range with the max key
lastKey := roachpb.RKeyMin
store.VisitReplicas(func(replica *kvserver.Replica) (wantMore bool) {
if lastKey.Less(replica.Desc().StartKey) {
lastKey = replica.Desc().StartKey
}
return true
})
// Merge last range.
_, pErr := kv.SendWrapped(ctx, store.TestSender(), adminMergeArgs(lastKey.AsRawKey()))
if !testutils.IsPError(pErr, "cannot merge final range") {
t.Fatalf("expected 'cannot merge final range' error; got %s", pErr)