Skip to content

Commit 87da966

Browse files
craig[bot]aayushshah15
andcommitted
Merge #81005
81005: kvserver: retry failures to rebalance decommissioning replicas r=aayushshah15 a=aayushshah15 Related to #80993 Relates to #79453 This commit makes it such that failures to rebalance replicas on decommissioning nodes no longer move the replica out of the replicateQueue as they previously used to. Instead, these failures now put these replicas into the replicateQueue's purgatory, which will retry these replicas every minute. All this is intended to improve the speed of decommissioning towards its tail end, since previously, failures to rebalance these replicas meant that they were only retried after about 10 minutes. Release note: None Co-authored-by: Aayush Shah <[email protected]>
2 parents c9c8a7c + 6f5122b commit 87da966

14 files changed

+242
-74
lines changed

pkg/kv/kvserver/allocator_impl_test.go

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,6 @@ import (
2626
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
2727
"github.com/cockroachdb/cockroach/pkg/util/log"
2828
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
29-
"github.com/cockroachdb/errors"
3029
"go.etcd.io/etcd/raft/v3"
3130
"go.etcd.io/etcd/raft/v3/tracker"
3231
)
@@ -254,7 +253,7 @@ func TestAllocatorThrottled(t *testing.T) {
254253

255254
// First test to make sure we would send the replica to purgatory.
256255
_, _, err := a.AllocateVoter(ctx, simpleSpanConfig, []roachpb.ReplicaDescriptor{}, nil)
257-
if !errors.HasInterface(err, (*purgatoryError)(nil)) {
256+
if _, ok := IsPurgatoryError(err); !ok {
258257
t.Fatalf("expected a purgatory error, got: %+v", err)
259258
}
260259

@@ -278,7 +277,7 @@ func TestAllocatorThrottled(t *testing.T) {
278277
storeDetail.ThrottledUntil = timeutil.Now().Add(24 * time.Hour)
279278
a.StorePool.DetailsMu.Unlock()
280279
_, _, err = a.AllocateVoter(ctx, simpleSpanConfig, []roachpb.ReplicaDescriptor{}, nil)
281-
if errors.HasInterface(err, (*purgatoryError)(nil)) {
280+
if _, ok := IsPurgatoryError(err); ok {
282281
t.Fatalf("expected a non purgatory error, got: %+v", err)
283282
}
284283
}

pkg/kv/kvserver/consistency_queue.go

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -238,3 +238,7 @@ func (q *consistencyQueue) timer(duration time.Duration) time.Duration {
238238
func (*consistencyQueue) purgatoryChan() <-chan time.Time {
239239
return nil
240240
}
241+
242+
func (*consistencyQueue) updateChan() <-chan time.Time {
243+
return nil
244+
}

pkg/kv/kvserver/merge_queue.go

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -178,7 +178,7 @@ type rangeMergePurgatoryError struct{ error }
178178

179179
func (rangeMergePurgatoryError) PurgatoryErrorMarker() {}
180180

181-
var _ purgatoryError = rangeMergePurgatoryError{}
181+
var _ PurgatoryError = rangeMergePurgatoryError{}
182182

183183
func (mq *mergeQueue) requestRangeStats(
184184
ctx context.Context, key roachpb.Key,
@@ -433,3 +433,7 @@ func (mq *mergeQueue) timer(time.Duration) time.Duration {
433433
func (mq *mergeQueue) purgatoryChan() <-chan time.Time {
434434
return mq.purgChan
435435
}
436+
437+
func (mq *mergeQueue) updateChan() <-chan time.Time {
438+
return nil
439+
}

pkg/kv/kvserver/mvcc_gc_queue.go

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -644,3 +644,7 @@ func (*mvccGCQueue) timer(_ time.Duration) time.Duration {
644644
func (*mvccGCQueue) purgatoryChan() <-chan time.Time {
645645
return nil
646646
}
647+
648+
func (*mvccGCQueue) updateChan() <-chan time.Time {
649+
return nil
650+
}

pkg/kv/kvserver/queue.go

Lines changed: 77 additions & 54 deletions
Original file line numberDiff line numberDiff line change
@@ -100,10 +100,10 @@ func makeRateLimitedTimeoutFunc(rateSetting *settings.ByteSizeSetting) queueProc
100100
// the operations's timeout.
101101
const permittedRangeScanSlowdown = 10
102102

103-
// a purgatoryError indicates a replica processing failure which indicates
104-
// the replica can be placed into purgatory for faster retries when the
105-
// failure condition changes.
106-
type purgatoryError interface {
103+
// PurgatoryError indicates a replica processing failure which indicates the
104+
// replica can be placed into purgatory for faster retries than the replica
105+
// scanner's interval.
106+
type PurgatoryError interface {
107107
error
108108
PurgatoryErrorMarker() // dummy method for unique interface
109109
}
@@ -270,6 +270,11 @@ type queueImpl interface {
270270
// purgatory due to failures. If purgatoryChan returns nil, failing
271271
// replicas are not sent to purgatory.
272272
purgatoryChan() <-chan time.Time
273+
274+
// updateChan returns a channel that is signalled whenever there is an update
275+
// to the cluster state that might impact the replicas in the queue's
276+
// purgatory.
277+
updateChan() <-chan time.Time
273278
}
274279

275280
// queueProcessTimeoutFunc controls the timeout for queue processing for a
@@ -380,7 +385,7 @@ type queueConfig struct {
380385
//
381386
// A queueImpl can opt into a purgatory by returning a non-nil channel from the
382387
// `purgatoryChan` method. A replica is put into purgatory when the `process`
383-
// method returns an error with a `purgatoryError` as an entry somewhere in the
388+
// method returns an error with a `PurgatoryError` as an entry somewhere in the
384389
// `Cause` chain. A replica in purgatory is not processed again until the
385390
// channel is signaled, at which point every replica in purgatory is immediately
386391
// processed. This catchup is run without the `timer` rate limiting but shares
@@ -414,7 +419,7 @@ type baseQueue struct {
414419
syncutil.Mutex // Protects all variables in the mu struct
415420
replicas map[roachpb.RangeID]*replicaItem // Map from RangeID to replicaItem
416421
priorityQ priorityQueue // The priority queue
417-
purgatory map[roachpb.RangeID]purgatoryError // Map of replicas to processing errors
422+
purgatory map[roachpb.RangeID]PurgatoryError // Map of replicas to processing errors
418423
stopped bool
419424
// Some tests in this package disable queues.
420425
disabled bool
@@ -987,8 +992,9 @@ func isBenign(err error) bool {
987992
return errors.HasType(err, (*benignError)(nil))
988993
}
989994

990-
func isPurgatoryError(err error) (purgatoryError, bool) {
991-
var purgErr purgatoryError
995+
// IsPurgatoryError returns true iff the given error is a purgatory error.
996+
func IsPurgatoryError(err error) (PurgatoryError, bool) {
997+
var purgErr PurgatoryError
992998
return purgErr, errors.As(err, &purgErr)
993999
}
9941000

@@ -1084,7 +1090,7 @@ func (bq *baseQueue) finishProcessingReplica(
10841090
// the failing replica to purgatory. Note that even if the item was
10851091
// scheduled to be requeued, we ignore this if we add the replica to
10861092
// purgatory.
1087-
if purgErr, ok := isPurgatoryError(err); ok {
1093+
if purgErr, ok := IsPurgatoryError(err); ok {
10881094
bq.mu.Lock()
10891095
bq.addToPurgatoryLocked(ctx, stopper, repl, purgErr)
10901096
bq.mu.Unlock()
@@ -1106,7 +1112,7 @@ func (bq *baseQueue) finishProcessingReplica(
11061112
// addToPurgatoryLocked adds the specified replica to the purgatory queue, which
11071113
// holds replicas which have failed processing.
11081114
func (bq *baseQueue) addToPurgatoryLocked(
1109-
ctx context.Context, stopper *stop.Stopper, repl replicaInQueue, purgErr purgatoryError,
1115+
ctx context.Context, stopper *stop.Stopper, repl replicaInQueue, purgErr PurgatoryError,
11101116
) {
11111117
bq.mu.AssertHeld()
11121118

@@ -1144,7 +1150,7 @@ func (bq *baseQueue) addToPurgatoryLocked(
11441150
}
11451151

11461152
// Otherwise, create purgatory and start processing.
1147-
bq.mu.purgatory = map[roachpb.RangeID]purgatoryError{
1153+
bq.mu.purgatory = map[roachpb.RangeID]PurgatoryError{
11481154
repl.GetRangeID(): purgErr,
11491155
}
11501156

@@ -1153,51 +1159,14 @@ func (bq *baseQueue) addToPurgatoryLocked(
11531159
ticker := time.NewTicker(purgatoryReportInterval)
11541160
for {
11551161
select {
1162+
case <-bq.impl.updateChan():
1163+
if bq.processReplicasInPurgatory(ctx, stopper) {
1164+
return
1165+
}
11561166
case <-bq.impl.purgatoryChan():
1157-
func() {
1158-
// Acquire from the process semaphore, release when done.
1159-
bq.processSem <- struct{}{}
1160-
defer func() { <-bq.processSem }()
1161-
1162-
// Remove all items from purgatory into a copied slice.
1163-
bq.mu.Lock()
1164-
ranges := make([]*replicaItem, 0, len(bq.mu.purgatory))
1165-
for rangeID := range bq.mu.purgatory {
1166-
item := bq.mu.replicas[rangeID]
1167-
if item == nil {
1168-
log.Fatalf(ctx, "r%d is in purgatory but not in replicas", rangeID)
1169-
}
1170-
item.setProcessing()
1171-
ranges = append(ranges, item)
1172-
bq.removeFromPurgatoryLocked(item)
1173-
}
1174-
bq.mu.Unlock()
1175-
1176-
for _, item := range ranges {
1177-
repl, err := bq.getReplica(item.rangeID)
1178-
if err != nil || item.replicaID != repl.ReplicaID() {
1179-
continue
1180-
}
1181-
annotatedCtx := repl.AnnotateCtx(ctx)
1182-
if stopper.RunTask(
1183-
annotatedCtx, bq.processOpName(), func(ctx context.Context) {
1184-
err := bq.processReplica(ctx, repl)
1185-
bq.finishProcessingReplica(ctx, stopper, repl, err)
1186-
}) != nil {
1187-
return
1188-
}
1189-
}
1190-
}()
1191-
1192-
// Clean up purgatory, if empty.
1193-
bq.mu.Lock()
1194-
if len(bq.mu.purgatory) == 0 {
1195-
log.Infof(ctx, "purgatory is now empty")
1196-
bq.mu.purgatory = nil
1197-
bq.mu.Unlock()
1167+
if bq.processReplicasInPurgatory(ctx, stopper) {
11981168
return
11991169
}
1200-
bq.mu.Unlock()
12011170
case <-ticker.C:
12021171
// Report purgatory status.
12031172
bq.mu.Lock()
@@ -1213,7 +1182,61 @@ func (bq *baseQueue) addToPurgatoryLocked(
12131182
return
12141183
}
12151184
}
1216-
})
1185+
},
1186+
)
1187+
}
1188+
1189+
// processReplicasInPurgatory processes replicas currently in the queue's
1190+
// purgatory.
1191+
func (bq *baseQueue) processReplicasInPurgatory(
1192+
ctx context.Context, stopper *stop.Stopper,
1193+
) (purgatoryCleared bool) {
1194+
func() {
1195+
// Acquire from the process semaphore, release when done.
1196+
bq.processSem <- struct{}{}
1197+
defer func() { <-bq.processSem }()
1198+
1199+
// Remove all items from purgatory into a copied slice.
1200+
bq.mu.Lock()
1201+
ranges := make([]*replicaItem, 0, len(bq.mu.purgatory))
1202+
for rangeID := range bq.mu.purgatory {
1203+
item := bq.mu.replicas[rangeID]
1204+
if item == nil {
1205+
log.Fatalf(ctx, "r%d is in purgatory but not in replicas", rangeID)
1206+
}
1207+
item.setProcessing()
1208+
ranges = append(ranges, item)
1209+
bq.removeFromPurgatoryLocked(item)
1210+
}
1211+
bq.mu.Unlock()
1212+
1213+
for _, item := range ranges {
1214+
repl, err := bq.getReplica(item.rangeID)
1215+
if err != nil || item.replicaID != repl.ReplicaID() {
1216+
continue
1217+
}
1218+
annotatedCtx := repl.AnnotateCtx(ctx)
1219+
if stopper.RunTask(
1220+
annotatedCtx, bq.processOpName(), func(ctx context.Context) {
1221+
err := bq.processReplica(ctx, repl)
1222+
bq.finishProcessingReplica(ctx, stopper, repl, err)
1223+
},
1224+
) != nil {
1225+
return
1226+
}
1227+
}
1228+
}()
1229+
1230+
// Clean up purgatory, if empty.
1231+
bq.mu.Lock()
1232+
if len(bq.mu.purgatory) == 0 {
1233+
log.Infof(ctx, "purgatory is now empty")
1234+
bq.mu.purgatory = nil
1235+
bq.mu.Unlock()
1236+
return true /* purgatoryCleared */
1237+
}
1238+
bq.mu.Unlock()
1239+
return false /* purgatoryCleared */
12171240
}
12181241

12191242
// pop dequeues the highest priority replica, if any, in the queue. The

pkg/kv/kvserver/queue_concurrency_test.go

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -151,6 +151,10 @@ func (fakeQueueImpl) purgatoryChan() <-chan time.Time {
151151
return time.After(time.Nanosecond)
152152
}
153153

154+
func (fakeQueueImpl) updateChan() <-chan time.Time {
155+
return nil
156+
}
157+
154158
type fakeReplica struct {
155159
rangeID roachpb.RangeID
156160
replicaID roachpb.ReplicaID

pkg/kv/kvserver/queue_test.go

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -84,6 +84,10 @@ func (tq *testQueueImpl) purgatoryChan() <-chan time.Time {
8484
return tq.pChan
8585
}
8686

87+
func (tq *testQueueImpl) updateChan() <-chan time.Time {
88+
return nil
89+
}
90+
8791
func makeTestBaseQueue(name string, impl queueImpl, store *Store, cfg queueConfig) *baseQueue {
8892
if !cfg.acceptsUnsplitRanges {
8993
// Needed in order to pass the validation in newBaseQueue.

pkg/kv/kvserver/raft_log_queue.go

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -743,6 +743,10 @@ func (*raftLogQueue) purgatoryChan() <-chan time.Time {
743743
return nil
744744
}
745745

746+
func (*raftLogQueue) updateChan() <-chan time.Time {
747+
return nil
748+
}
749+
746750
func isLooselyCoupledRaftLogTruncationEnabled(
747751
ctx context.Context, settings *cluster.Settings,
748752
) bool {

pkg/kv/kvserver/raft_snapshot_queue.go

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -171,3 +171,7 @@ func (*raftSnapshotQueue) timer(_ time.Duration) time.Duration {
171171
func (rq *raftSnapshotQueue) purgatoryChan() <-chan time.Time {
172172
return nil
173173
}
174+
175+
func (rq *raftSnapshotQueue) updateChan() <-chan time.Time {
176+
return nil
177+
}

pkg/kv/kvserver/replica_gc_queue.go

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -373,3 +373,7 @@ func (*replicaGCQueue) timer(_ time.Duration) time.Duration {
373373
func (*replicaGCQueue) purgatoryChan() <-chan time.Time {
374374
return nil
375375
}
376+
377+
func (*replicaGCQueue) updateChan() <-chan time.Time {
378+
return nil
379+
}

0 commit comments

Comments
 (0)