Skip to content

Commit e1c7c3d

Browse files
committed
kvserver: drop RemoveReplica opts parameter
All callers pass in the same opts, so no need in making it configurable. Epic: none Release note: none
1 parent cad9544 commit e1c7c3d

File tree

7 files changed

+45
-84
lines changed

7 files changed

+45
-84
lines changed

pkg/kv/kvserver/client_raft_test.go

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -3087,9 +3087,9 @@ func TestReplicaRemovalCampaign(t *testing.T) {
30873087

30883088
if td.remove {
30893089
// Simulate second replica being transferred by removing it.
3090-
if err := store0.RemoveReplica(ctx, replica2, replica2.Desc().NextReplicaID, redact.SafeString(t.Name()), kvserver.RemoveOptions{
3091-
DestroyData: true,
3092-
}); err != nil {
3090+
if err := store0.RemoveReplica(
3091+
ctx, replica2, replica2.Desc().NextReplicaID, redact.SafeString(t.Name()),
3092+
); err != nil {
30933093
t.Fatal(err)
30943094
}
30953095
}

pkg/kv/kvserver/mvcc_gc_queue_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1511,7 +1511,7 @@ func TestMVCCGCQueueGroupsRangeDeletions(t *testing.T) {
15111511
store := createTestStoreWithConfig(ctx, t, stopper, testStoreOpts{createSystemRanges: false}, &cfg)
15121512
r, err := store.GetReplica(roachpb.RangeID(1))
15131513
require.NoError(t, err)
1514-
require.NoError(t, store.RemoveReplica(ctx, r, r.Desc().NextReplicaID, redact.SafeString(t.Name()), RemoveOptions{DestroyData: true}))
1514+
require.NoError(t, store.RemoveReplica(ctx, r, r.Desc().NextReplicaID, redact.SafeString(t.Name())))
15151515
// Add replica without hint.
15161516
r1 := createReplica(store, roachpb.RangeID(100), key("a"), key("b"))
15171517
require.NoError(t, store.AddReplica(r1))

pkg/kv/kvserver/queue_test.go

Lines changed: 5 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -123,11 +123,8 @@ func createReplicas(t *testing.T, tc *testContext, num int) []*Replica {
123123
if err != nil {
124124
t.Fatal(err)
125125
}
126-
if err := tc.store.RemoveReplica(context.Background(), repl1, repl1.Desc().NextReplicaID, redact.SafeString(t.Name()), RemoveOptions{
127-
DestroyData: true,
128-
}); err != nil {
129-
t.Fatal(err)
130-
}
126+
require.NoError(t, tc.store.RemoveReplica(context.Background(),
127+
repl1, repl1.Desc().NextReplicaID, redact.SafeString(t.Name())))
131128

132129
repls := make([]*Replica, num)
133130
for i := 0; i < num; i++ {
@@ -708,7 +705,7 @@ func TestAcceptsUnsplitRanges(t *testing.T) {
708705
// Remove replica for range 1 since it encompasses the entire keyspace.
709706
repl1, err := s.GetReplica(1)
710707
require.NoError(t, err)
711-
require.NoError(t, s.RemoveReplica(ctx, repl1, repl1.Desc().NextReplicaID, redact.SafeString(t.Name()), RemoveOptions{DestroyData: true}))
708+
require.NoError(t, s.RemoveReplica(ctx, repl1, repl1.Desc().NextReplicaID, redact.SafeString(t.Name())))
712709

713710
// This range can never be split due to zone configs boundaries.
714711
neverSplits := createReplica(s, 2, roachpb.RKeyMin, maxWontSplitAddr)
@@ -910,11 +907,8 @@ func TestBaseQueuePurgatory(t *testing.T) {
910907
// the replica set. The number of processed replicas will be 2 less.
911908
const rmReplCount = 2
912909
repls[0].replicaID = 2
913-
if err := tc.store.RemoveReplica(ctx, repls[1], repls[1].Desc().NextReplicaID, redact.SafeString(t.Name()), RemoveOptions{
914-
DestroyData: true,
915-
}); err != nil {
916-
t.Fatal(err)
917-
}
910+
require.NoError(t, tc.store.RemoveReplica(ctx,
911+
repls[1], repls[1].Desc().NextReplicaID, redact.SafeString(t.Name())))
918912

919913
// Remove error and reprocess.
920914
testQueue.err = nil

pkg/kv/kvserver/replica_gc_queue.go

Lines changed: 4 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -315,9 +315,7 @@ func (rgcq *replicaGCQueue) process(
315315
// possible if we currently think we're processing a pre-emptive snapshot
316316
// but discover in RemoveReplica that this range has since been added and
317317
// knows that.
318-
if err := repl.store.RemoveReplica(ctx, repl, nextReplicaID, "MVCC GC queue", RemoveOptions{
319-
DestroyData: true,
320-
}); err != nil {
318+
if err := repl.store.RemoveReplica(ctx, repl, nextReplicaID, "MVCC GC queue"); err != nil {
321319
// Should never get an error from RemoveReplica.
322320
const format = "error during replicaGC: %v"
323321
logcrash.ReportOrPanic(ctx, &repl.store.ClusterSettings().SV, format, err)
@@ -359,9 +357,9 @@ func (rgcq *replicaGCQueue) process(
359357
// A tombstone is written with a value of mergedTombstoneReplicaID because
360358
// we know the range to have been merged. See the Merge case of
361359
// runPreApplyTriggers() for details.
362-
if err := repl.store.RemoveReplica(ctx, repl, mergedTombstoneReplicaID, "dangling subsume via MVCC GC queue", RemoveOptions{
363-
DestroyData: true,
364-
}); err != nil {
360+
if err := repl.store.RemoveReplica(
361+
ctx, repl, mergedTombstoneReplicaID, "dangling subsume via MVCC GC queue",
362+
); err != nil {
365363
return false, err
366364
}
367365
}

pkg/kv/kvserver/replica_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -9226,7 +9226,7 @@ func TestCancelPendingCommands(t *testing.T) {
92269226
t.Fatalf("command finished earlier than expected with error %v", pErr)
92279227
default:
92289228
}
9229-
require.NoError(t, tc.store.RemoveReplica(ctx, tc.repl, tc.repl.Desc().NextReplicaID, redact.SafeString(t.Name()), RemoveOptions{DestroyData: true}))
9229+
require.NoError(t, tc.store.RemoveReplica(ctx, tc.repl, tc.repl.Desc().NextReplicaID, redact.SafeString(t.Name())))
92309230
pErr := <-errChan
92319231
if _, ok := pErr.GetDetail().(*kvpb.AmbiguousResultError); !ok {
92329232
t.Errorf("expected AmbiguousResultError, got %v", pErr)

pkg/kv/kvserver/store_remove_replica.go

Lines changed: 4 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -33,16 +33,11 @@ type RemoveOptions struct {
3333
// removal decision is passed in. Removal is aborted if the replica ID has
3434
// advanced to or beyond the NextReplicaID since the removal decision was made.
3535
//
36-
// If opts.DestroyReplica is false, replica.destroyRaftMuLocked is not called.
37-
//
3836
// The passed replica must be initialized.
3937
func (s *Store) RemoveReplica(
40-
ctx context.Context,
41-
rep *Replica,
42-
nextReplicaID roachpb.ReplicaID,
43-
reason redact.SafeString,
44-
opts RemoveOptions,
38+
ctx context.Context, rep *Replica, nextReplicaID roachpb.ReplicaID, reason redact.SafeString,
4539
) error {
40+
opts := RemoveOptions{DestroyData: true}
4641
rep.raftMu.Lock()
4742
defer rep.raftMu.Unlock()
4843
if opts.InsertPlaceholder {
@@ -77,6 +72,8 @@ func (s *Store) removeReplicaRaftMuLocked(
7772
// removeInitializedReplicaRaftMuLocked is the implementation of RemoveReplica,
7873
// which is sometimes called directly when the necessary lock is already held.
7974
// It requires that Replica.raftMu is held and that s.mu is not held.
75+
//
76+
// If opts.DestroyData is false, replica.destroyRaftMuLocked is not called.
8077
func (s *Store) removeInitializedReplicaRaftMuLocked(
8178
ctx context.Context,
8279
rep *Replica,

pkg/kv/kvserver/store_test.go

Lines changed: 27 additions & 55 deletions
Original file line numberDiff line numberDiff line change
@@ -71,6 +71,7 @@ import (
7171
"github.com/cockroachdb/errors"
7272
"github.com/cockroachdb/redact"
7373
"github.com/kr/pretty"
74+
"github.com/stretchr/testify/assert"
7475
"github.com/stretchr/testify/require"
7576
"golang.org/x/exp/slices"
7677
"golang.org/x/sync/errgroup"
@@ -620,11 +621,7 @@ func TestStoreAddRemoveRanges(t *testing.T) {
620621
t.Error(err)
621622
}
622623
// Remove range 1.
623-
if err := store.RemoveReplica(ctx, repl1, repl1.Desc().NextReplicaID, redact.SafeString(t.Name()), RemoveOptions{
624-
DestroyData: true,
625-
}); err != nil {
626-
t.Error(err)
627-
}
624+
assert.NoError(t, store.RemoveReplica(ctx, repl1, repl1.Desc().NextReplicaID, redact.SafeString(t.Name())))
628625
// Create a new range (id=2).
629626
repl2 := createReplica(store, 2, roachpb.RKey("a"), roachpb.RKey("b"))
630627
if err := store.AddReplica(repl2); err != nil {
@@ -636,11 +633,7 @@ func TestStoreAddRemoveRanges(t *testing.T) {
636633
t.Fatal("expected error re-adding same range")
637634
}
638635
// Try to remove range 1 again.
639-
if err := store.RemoveReplica(ctx, repl1, repl1.Desc().NextReplicaID, redact.SafeString(t.Name()), RemoveOptions{
640-
DestroyData: true,
641-
}); err != nil {
642-
t.Fatalf("didn't expect error re-removing same range: %v", err)
643-
}
636+
require.NoError(t, store.RemoveReplica(ctx, repl1, repl1.Desc().NextReplicaID, redact.SafeString(t.Name())))
644637
// Try to add a range with previously-used (but now removed) ID.
645638
repl2Dup := createReplica(store, 1, roachpb.RKey("a"), roachpb.RKey("b"))
646639
if err := store.AddReplica(repl2Dup); err == nil {
@@ -742,26 +735,29 @@ func TestStoreRemoveReplicaDestroy(t *testing.T) {
742735
t.Fatal(err)
743736
}
744737

745-
// Can't remove Replica with DestroyData false because this requires the destroyStatus
746-
// to already have been set by the caller (but we didn't).
747-
require.ErrorContains(t, store.RemoveReplica(ctx, repl1, repl1.Desc().NextReplicaID, redact.SafeString(t.Name()), RemoveOptions{
748-
DestroyData: false,
749-
}), `replica not marked as destroyed`)
738+
rmWithoutData := func() error {
739+
repl1.raftMu.Lock()
740+
defer repl1.raftMu.Unlock()
741+
_, err := store.removeInitializedReplicaRaftMuLocked(
742+
ctx, repl1, repl1.Desc().NextReplicaID, redact.SafeString(t.Name()),
743+
RemoveOptions{DestroyData: false},
744+
)
745+
return err
746+
}
747+
// Can't remove Replica with DestroyData false because this requires the
748+
// destroyStatus to already have been set by the caller (but we didn't).
749+
require.ErrorContains(t, rmWithoutData(), `replica not marked as destroyed`)
750750

751751
// Remove the Replica twice, as this should be idempotent.
752752
// NB: we rely on this idempotency today (as @tbg found out when he accidentally
753753
// removed it).
754754
for i := 0; i < 2; i++ {
755-
require.NoError(t, store.RemoveReplica(ctx, repl1, repl1.Desc().NextReplicaID, redact.SafeString(t.Name()), RemoveOptions{
756-
DestroyData: true,
757-
}), "%d", i)
755+
require.NoError(t, store.RemoveReplica(ctx, repl1, repl1.Desc().NextReplicaID, redact.SafeString(t.Name())), "%d", i)
758756
}
759757

760-
// However, if we have DestroyData=false, caller is expected to be the unique first "destroyer"
761-
// of the Replica.
762-
require.ErrorContains(t, store.RemoveReplica(ctx, repl1, repl1.Desc().NextReplicaID, redact.SafeString(t.Name()), RemoveOptions{
763-
DestroyData: false,
764-
}), `does not exist`)
758+
// However, if we have DestroyData=false, caller is expected to be the unique
759+
// first "destroyer" of the Replica.
760+
require.ErrorContains(t, rmWithoutData(), `does not exist`)
765761

766762
// Verify that removal of a replica marks it as destroyed so that future raft
767763
// commands on the Replica will silently be dropped.
@@ -802,11 +798,7 @@ func TestStoreReplicaVisitor(t *testing.T) {
802798
if err != nil {
803799
t.Error(err)
804800
}
805-
if err := store.RemoveReplica(ctx, repl1, repl1.Desc().NextReplicaID, redact.SafeString(t.Name()), RemoveOptions{
806-
DestroyData: true,
807-
}); err != nil {
808-
t.Error(err)
809-
}
801+
assert.NoError(t, store.RemoveReplica(ctx, repl1, repl1.Desc().NextReplicaID, redact.SafeString(t.Name())))
810802

811803
// Add 10 new ranges.
812804
const newCount = 10
@@ -883,14 +875,8 @@ func TestMarkReplicaInitialized(t *testing.T) {
883875

884876
// Clobber the existing range so we can test overlaps that aren't KeyMin or KeyMax.
885877
repl1, err := store.GetReplica(1)
886-
if err != nil {
887-
t.Error(err)
888-
}
889-
if err := store.RemoveReplica(ctx, repl1, repl1.Desc().NextReplicaID, redact.SafeString(t.Name()), RemoveOptions{
890-
DestroyData: true,
891-
}); err != nil {
892-
t.Error(err)
893-
}
878+
assert.NoError(t, err)
879+
assert.NoError(t, store.RemoveReplica(ctx, repl1, repl1.Desc().NextReplicaID, redact.SafeString(t.Name())))
894880

895881
repl := createReplica(store, roachpb.RangeID(2), roachpb.RKey("a"), roachpb.RKey("c"))
896882
if err := store.AddReplica(repl); err != nil {
@@ -2754,14 +2740,8 @@ func TestMaybeRemove(t *testing.T) {
27542740
store.WaitForInit()
27552741

27562742
repl, err := store.GetReplica(1)
2757-
if err != nil {
2758-
t.Error(err)
2759-
}
2760-
if err := store.RemoveReplica(ctx, repl, repl.Desc().NextReplicaID, redact.SafeString(t.Name()), RemoveOptions{
2761-
DestroyData: true,
2762-
}); err != nil {
2763-
t.Error(err)
2764-
}
2743+
assert.NoError(t, err)
2744+
assert.NoError(t, store.RemoveReplica(ctx, repl, repl.Desc().NextReplicaID, redact.SafeString(t.Name())))
27652745
// MaybeRemove is called.
27662746
removedRng := <-fq.maybeRemovedRngs
27672747
if removedRng != repl.RangeID {
@@ -2873,14 +2853,8 @@ func TestStoreRangePlaceholders(t *testing.T) {
28732853

28742854
// Clobber the existing range so we can test non-overlapping placeholders.
28752855
repl1, err := s.GetReplica(1)
2876-
if err != nil {
2877-
t.Error(err)
2878-
}
2879-
if err := s.RemoveReplica(ctx, repl1, repl1.Desc().NextReplicaID, redact.SafeString(t.Name()), RemoveOptions{
2880-
DestroyData: true,
2881-
}); err != nil {
2882-
t.Error(err)
2883-
}
2856+
assert.NoError(t, err)
2857+
assert.NoError(t, s.RemoveReplica(ctx, repl1, repl1.Desc().NextReplicaID, redact.SafeString(t.Name())))
28842858

28852859
repID := roachpb.RangeID(2)
28862860
rep := createReplica(s, repID, roachpb.RKeyMin, roachpb.RKey("c"))
@@ -3012,9 +2986,7 @@ func TestStoreRemovePlaceholderOnRaftIgnored(t *testing.T) {
30122986
repl1, err := s.GetReplica(1)
30132987
desc := repl1.Desc()
30142988
require.NoError(t, err)
3015-
require.NoError(t, s.RemoveReplica(ctx, repl1, desc.NextReplicaID, redact.SafeString(t.Name()), RemoveOptions{
3016-
DestroyData: true,
3017-
}))
2989+
require.NoError(t, s.RemoveReplica(ctx, repl1, desc.NextReplicaID, redact.SafeString(t.Name())))
30182990

30192991
// Wrap the snapshot in a minimal header. The request will be dropped because
30202992
// replica 2 is not in the ConfState.

0 commit comments

Comments
 (0)