Skip to content

Commit d0a7b61

Browse files
craig[bot]pav-kv
andcommitted
Merge #150318
150318: kvserver: drop opts in replica removal r=tbg a=pav-kv All callers pass in the same opts, so no need in making it configurable. After this PR, `removeInitializedReplicaRaftMuLocked` is the only remaining call accepting `RemoveOptions`, which makes the scope/purpose of these options clearer. This also does a couple of drive-by clean-ups in replica destruction stack. Epic: CRDB-49111 Co-authored-by: Pavel Kalinnikov <[email protected]>
2 parents ac1db4f + 5a32ea3 commit d0a7b61

13 files changed

+68
-132
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_application_result.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -698,7 +698,7 @@ func (r *Replica) handleChangeReplicasResult(
698698
// NB: postDestroyRaftMuLocked requires that the batch which removed the data
699699
// be durably synced to disk, which we have.
700700
// See replicaAppBatch.ApplyToStateMachine().
701-
if err := r.postDestroyRaftMuLocked(ctx, r.GetMVCCStats()); err != nil {
701+
if err := r.postDestroyRaftMuLocked(ctx); err != nil {
702702
log.Fatalf(ctx, "failed to run Replica postDestroy: %v", err)
703703
}
704704

pkg/kv/kvserver/replica_destroy.go

Lines changed: 7 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -14,7 +14,6 @@ import (
1414
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/apply"
1515
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage"
1616
"github.com/cockroachdb/cockroach/pkg/roachpb"
17-
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
1817
"github.com/cockroachdb/cockroach/pkg/util/log"
1918
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
2019
)
@@ -65,20 +64,12 @@ func (s destroyStatus) Removed() bool {
6564
// don't know the current replica ID.
6665
const mergedTombstoneReplicaID roachpb.ReplicaID = math.MaxInt32
6766

68-
func (r *Replica) postDestroyRaftMuLocked(ctx context.Context, ms enginepb.MVCCStats) error {
69-
// TODO(tschottdorf): at node startup, we should remove all on-disk
70-
// directories belonging to replicas which aren't present. A crash before a
71-
// call to postDestroyRaftMuLocked will currently leave the files around
72-
// forever.
73-
//
74-
// TODO(tbg): coming back in 2021, the above should be outdated. The ReplicaID
75-
// is set on creation and never changes over the lifetime of a Replica. Also,
76-
// the replica is always contained in its descriptor. So this code below should
77-
// be removable.
78-
//
79-
// TODO(pavelkalinnikov): coming back in 2023, the above may still happen if:
80-
// (1) state machine syncs, (2) OS crashes before (3) sideloaded was able to
81-
// sync the files removal. The files should be cleaned up on restart.
67+
// postDestroyRaftMuLocked is called after the replica destruction is durably
68+
// written to Pebble.
69+
func (r *Replica) postDestroyRaftMuLocked(ctx context.Context) error {
70+
// TODO(#136416): at node startup, we should remove all on-disk directories
71+
// belonging to replicas which aren't present. A crash before a call to
72+
// postDestroyRaftMuLocked will currently leave the files around forever.
8273
if err := r.logStorage.ls.Sideload.Clear(ctx); err != nil {
8374
return err
8475
}
@@ -135,7 +126,7 @@ func (r *Replica) destroyRaftMuLocked(ctx context.Context, nextReplicaID roachpb
135126
}
136127
commitTime := timeutil.Now()
137128

138-
if err := r.postDestroyRaftMuLocked(ctx, ms); err != nil {
129+
if err := r.postDestroyRaftMuLocked(ctx); err != nil {
139130
return err
140131
}
141132
if r.IsInitialized() {

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_raftstorage.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -815,7 +815,7 @@ func (r *Replica) clearSubsumedReplicaInMemoryData(
815815
phs = append(phs, ph)
816816
// We removed sr's data when we committed the batch. Finish subsumption by
817817
// updating the in-memory bookkeping.
818-
if err := sr.postDestroyRaftMuLocked(ctx, sr.GetMVCCStats()); err != nil {
818+
if err := sr.postDestroyRaftMuLocked(ctx); err != nil {
819819
return nil, err
820820
}
821821
}

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_create_replica.go

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -116,9 +116,9 @@ func (s *Store) tryGetReplica(
116116
}
117117

118118
repl.mu.RUnlock()
119-
if err := s.removeReplicaRaftMuLocked(ctx, repl, id.ReplicaID, "superseded by newer Replica", RemoveOptions{
120-
DestroyData: true,
121-
}); err != nil {
119+
if err := s.removeReplicaRaftMuLocked(
120+
ctx, repl, id.ReplicaID, "superseded by newer Replica",
121+
); err != nil {
122122
log.Fatalf(ctx, "failed to remove replica: %v", err)
123123
}
124124
repl.raftMu.Unlock()

pkg/kv/kvserver/store_merge.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -138,7 +138,7 @@ func (s *Store) MergeRange(
138138
return errors.Wrap(err, "cannot remove range")
139139
}
140140

141-
if err := rightRepl.postDestroyRaftMuLocked(ctx, rightRepl.GetMVCCStats()); err != nil {
141+
if err := rightRepl.postDestroyRaftMuLocked(ctx); err != nil {
142142
return err
143143
}
144144

0 commit comments

Comments
 (0)