From e67c1903f45d68bdf5ecba031778a2b2246f8509 Mon Sep 17 00:00:00 2001 From: Swapneeth Gorantla Date: Tue, 19 Aug 2025 15:56:56 -0400 Subject: [PATCH 01/44] kvserver: add logging & metrics for decommissioning nudger Previously, the decommissioning nudger had limited observability, making it difficult to monitor its effectiveness and diagnose issues during node decommissioning operations. This was inadequate because operators couldn't track how many ranges were being enqueued for decommissioning, nor could they see when the nudger skipped ranges due to leaseholder status or invalid leases. To address this, this patch adds comprehensive logging and metrics: - Logs when the decommissioning nudger enqueues replicas with priority info - Tracks successful enqueues via DecommissioningNudgerEnqueueEnqueued metric - Tracks skipped enqueues via DecommissioningNudgerNotLeaseholderOrInvalidLease metric - Adds structured logging for debugging nudger behavior - Includes comprehensive test coverage for the new metrics TODO: - Figure out a better way to track decommissioning enqueue failures. Currently it's hard to "get as close as we can" to the source of the enqueue failures for logging & metrics purposes - this would require a better architecting of the code pathways to ensure we log and track failures as close as we can to where they occur Fixes: CRDB-51396 Release note: None --- docs/generated/metrics/metrics.html | 2 ++ pkg/kv/kvserver/metrics.go | 22 +++++++++++++ pkg/kv/kvserver/replica.go | 23 ++++++++++++- pkg/kv/kvserver/replicate_queue_test.go | 43 +++++++++++++++++++++++++ 4 files changed, 89 insertions(+), 1 deletion(-) diff --git a/docs/generated/metrics/metrics.html b/docs/generated/metrics/metrics.html index 323182f0bdd9..da0bec0ffd6d 100644 --- a/docs/generated/metrics/metrics.html +++ b/docs/generated/metrics/metrics.html @@ -580,6 +580,8 @@ STORAGErangekeycountCount of all range keys (e.g. MVCC range tombstones)KeysGAUGECOUNTAVGNONE STORAGErangesNumber of rangesRangesGAUGECOUNTAVGNONE STORAGEranges.decommissioningNumber of ranges with at lease one replica on a decommissioning nodeRangesGAUGECOUNTAVGNONE +STORAGEranges.decommissioning.nudger.enqueueNumber of enqueued enqueues of a range for decommissioning by the decommissioning nudger. Note: This metric tracks when the nudger attempts to enqueue, but the replica might not end up being enqueued by the priority queue due to various filtering or failure conditions.RangesCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEranges.decommissioning.nudger.not_leaseholder_or_invalid_leaseNumber of enqueues of a range for decommissioning by the decommissioning nudger that were not the leaseholder or had an invalid leaseRangesCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEranges.overreplicatedNumber of ranges with more live replicas than the replication targetRangesGAUGECOUNTAVGNONE STORAGEranges.unavailableNumber of ranges with fewer live replicas than needed for quorumRangesGAUGECOUNTAVGNONE STORAGEranges.underreplicatedNumber of ranges with fewer live replicas than the replication targetRangesGAUGECOUNTAVGNONE diff --git a/pkg/kv/kvserver/metrics.go b/pkg/kv/kvserver/metrics.go index 8a951e18e4dd..241df3cc4a82 100644 --- a/pkg/kv/kvserver/metrics.go +++ b/pkg/kv/kvserver/metrics.go @@ -154,6 +154,20 @@ var ( Unit: metric.Unit_COUNT, } + // Decommisioning nudger metrics. + metaDecommissioningNudgerEnqueue = metric.Metadata{ + Name: "ranges.decommissioning.nudger.enqueue", + Help: "Number of enqueued enqueues of a range for decommissioning by the decommissioning nudger. Note: This metric tracks when the nudger attempts to enqueue, but the replica might not end up being enqueued by the priority queue due to various filtering or failure conditions.", + Measurement: "Ranges", + Unit: metric.Unit_COUNT, + } + metaDecommissioningNudgerNotLeaseholderOrInvalidLease = metric.Metadata{ + Name: "ranges.decommissioning.nudger.not_leaseholder_or_invalid_lease", + Help: "Number of enqueues of a range for decommissioning by the decommissioning nudger that were not the leaseholder or had an invalid lease", + Measurement: "Ranges", + Unit: metric.Unit_COUNT, + } + // Lease request metrics. metaLeaseRequestSuccessCount = metric.Metadata{ Name: "leases.success", @@ -2657,6 +2671,10 @@ type StoreMetrics struct { OverReplicatedRangeCount *metric.Gauge DecommissioningRangeCount *metric.Gauge + // Decommissioning nudger metrics. + DecommissioningNudgerEnqueue *metric.Counter + DecommissioningNudgerNotLeaseholderOrInvalidLease *metric.Counter + // Lease request metrics for successful and failed lease requests. These // count proposals (i.e. it does not matter how many replicas apply the // lease). @@ -3365,6 +3383,10 @@ func newStoreMetrics(histogramWindow time.Duration) *StoreMetrics { OverReplicatedRangeCount: metric.NewGauge(metaOverReplicatedRangeCount), DecommissioningRangeCount: metric.NewGauge(metaDecommissioningRangeCount), + // Decommissioning nuder metrics. + DecommissioningNudgerEnqueue: metric.NewCounter(metaDecommissioningNudgerEnqueue), + DecommissioningNudgerNotLeaseholderOrInvalidLease: metric.NewCounter(metaDecommissioningNudgerNotLeaseholderOrInvalidLease), + // Lease request metrics. LeaseRequestSuccessCount: metric.NewCounter(metaLeaseRequestSuccessCount), LeaseRequestErrorCount: metric.NewCounter(metaLeaseRequestErrorCount), diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index ee5ef3a7f7e6..b26b98181e48 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -2875,7 +2875,17 @@ func (r *Replica) maybeEnqueueProblemRange( // updating metrics. if !isLeaseholder || !leaseValid { // The replicate queue will not process the replica without a valid lease. - // Nothing to do. + // Track when we skip enqueuing for these reasons. + boolToInt := func(b bool) int { + if b { + return 1 + } + return 0 + } + reasons := []string{"is not the leaseholder", "the lease is not valid"} + reason := reasons[boolToInt(isLeaseholder)] + log.KvDistribution.VInfof(ctx, 1, "not enqueuing replica %s because %s", r.Desc(), reason) + r.store.metrics.DecommissioningNudgerNotLeaseholderOrInvalidLease.Inc(1) return } @@ -2896,8 +2906,19 @@ func (r *Replica) maybeEnqueueProblemRange( // expect a race, however if the value changed underneath us we won't enqueue // the replica as we lost the race. if !r.lastProblemRangeReplicateEnqueueTime.CompareAndSwap(lastTime, now) { + // This race condition is expected to be rare. + log.KvDistribution.VInfof(ctx, 1, "not enqueuing replica %s due to race: "+ + "lastProblemRangeReplicateEnqueueTime was updated concurrently", r.Desc()) return } + // Log at default verbosity to ensure some indication the nudger is working + // (other logs have a verbosity of 1 which). + log.KvDistribution.Infof(ctx, "decommissioning nudger enqueuing replica %s "+ + "with priority %f", r.Desc(), + allocatorimpl.AllocatorReplaceDecommissioningVoter.Priority()) + r.store.metrics.DecommissioningNudgerEnqueue.Inc(1) + // TODO(dodeca12): Figure out a better way to track the + // decommissioning nudger enqueue failures/errors. r.store.replicateQueue.AddAsync(ctx, r, allocatorimpl.AllocatorReplaceDecommissioningVoter.Priority()) } diff --git a/pkg/kv/kvserver/replicate_queue_test.go b/pkg/kv/kvserver/replicate_queue_test.go index 751daea7b302..d4a697463258 100644 --- a/pkg/kv/kvserver/replicate_queue_test.go +++ b/pkg/kv/kvserver/replicate_queue_test.go @@ -2471,12 +2471,55 @@ func TestReplicateQueueDecommissionScannerDisabled(t *testing.T) { return nil }) + getDecommissioningNudgerMetricValue := func(t *testing.T, tc *testcluster.TestCluster, metricType string, + ) int64 { + var total int64 + + for i := 0; i < tc.NumServers(); i++ { + store := tc.GetFirstStoreFromServer(t, i) + var value int64 + + switch metricType { + case "decommissioning_ranges": + value = store.Metrics().DecommissioningRangeCount.Value() + case "enqueue": + value = store.Metrics().DecommissioningNudgerEnqueue.Count() + case "not_leaseholder_or_invalid_lease": + value = store.Metrics().DecommissioningNudgerNotLeaseholderOrInvalidLease.Count() + default: + t.Fatalf("unknown metric type: %s", metricType) + } + + total += value + } + return total + } + + initialDecommissioningRanges := getDecommissioningNudgerMetricValue(t, tc, "decommissioning_ranges") + // Now add a replica to the decommissioning node and then enable the // replicate queue. We expect that the replica will be removed after the // decommissioning replica is noticed via maybeEnqueueProblemRange. scratchKey := tc.ScratchRange(t) tc.AddVotersOrFatal(t, scratchKey, tc.Target(decommissioningSrvIdx)) tc.ToggleReplicateQueues(true /* active */) + + // Wait for the enqueue logic to trigger and validate metrics were updated. + testutils.SucceedsSoon(t, func() error { + afterDecommissioningRanges := getDecommissioningNudgerMetricValue(t, tc, "decommissioning_ranges") + afterEnqueued := getDecommissioningNudgerMetricValue(t, tc, "enqueue") + + if afterDecommissioningRanges <= initialDecommissioningRanges { + return errors.New("expected DecommissioningRangeCount to increase") + } + if afterEnqueued <= 0 { + return errors.New("expected DecommissioningNudgerEnqueueEnqueued to be greater than 0") + } + + return nil + }) + + // Verify that the decommissioning node has no replicas left. testutils.SucceedsSoon(t, func() error { var descs []*roachpb.RangeDescriptor tc.GetFirstStoreFromServer(t, decommissioningSrvIdx).VisitReplicas(func(r *kvserver.Replica) bool { From d63c58688b26dd26e971c47a3fcce0290ce9d4c4 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Mon, 25 Aug 2025 19:55:23 -0400 Subject: [PATCH 02/44] allocator: move isDecommissionAction to allocatorimpl This commit refactors isDecommissionAction into allocatorimpl for consistency with other similar helpers like allocatorActions.{Add,Replace,Remove}. This change has no behavior changes but to make future commits easier. --- pkg/kv/kvserver/allocator/allocatorimpl/allocator.go | 9 +++++++++ pkg/kv/kvserver/replicate_queue.go | 9 +-------- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go b/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go index 05d0ad86505a..21e35f510481 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go +++ b/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go @@ -163,6 +163,15 @@ func (a AllocatorAction) Remove() bool { a == AllocatorRemoveDecommissioningNonVoter } +// Decommissioning indicates an action replacing or removing a decommissioning +// replicas. +func (a AllocatorAction) Decommissioning() bool { + return a == AllocatorRemoveDecommissioningVoter || + a == AllocatorRemoveDecommissioningNonVoter || + a == AllocatorReplaceDecommissioningVoter || + a == AllocatorReplaceDecommissioningNonVoter +} + // TargetReplicaType returns that the action is for a voter or non-voter replica. func (a AllocatorAction) TargetReplicaType() TargetReplicaType { var t TargetReplicaType diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index 5988442a3917..01f8695e1ac6 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -921,19 +921,12 @@ func (rq *replicateQueue) processOneChange( } func maybeAnnotateDecommissionErr(err error, action allocatorimpl.AllocatorAction) error { - if err != nil && isDecommissionAction(action) { + if err != nil && action.Decommissioning() { err = decommissionPurgatoryError{err} } return err } -func isDecommissionAction(action allocatorimpl.AllocatorAction) bool { - return action == allocatorimpl.AllocatorRemoveDecommissioningVoter || - action == allocatorimpl.AllocatorRemoveDecommissioningNonVoter || - action == allocatorimpl.AllocatorReplaceDecommissioningVoter || - action == allocatorimpl.AllocatorReplaceDecommissioningNonVoter -} - // shedLease takes in a leaseholder replica, looks for a target for transferring // the lease and, if a suitable target is found (e.g. alive, not draining), // transfers the lease away. From 5417c92a354c2078593e667b241a6b74ce6168f8 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Mon, 25 Aug 2025 20:59:34 -0400 Subject: [PATCH 03/44] kvserver: simplify logging in maybeEnqueueProblemRange This commit simplifies the logging in `maybeEnqueueProblemRange` to log two booleans directly. --- pkg/kv/kvserver/replica.go | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index b26b98181e48..919c4f7245e4 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -2876,15 +2876,8 @@ func (r *Replica) maybeEnqueueProblemRange( if !isLeaseholder || !leaseValid { // The replicate queue will not process the replica without a valid lease. // Track when we skip enqueuing for these reasons. - boolToInt := func(b bool) int { - if b { - return 1 - } - return 0 - } - reasons := []string{"is not the leaseholder", "the lease is not valid"} - reason := reasons[boolToInt(isLeaseholder)] - log.KvDistribution.VInfof(ctx, 1, "not enqueuing replica %s because %s", r.Desc(), reason) + log.KvDistribution.Infof(ctx, "not enqueuing replica %s because isLeaseholder=%t, leaseValid=%t", + r.Desc(), isLeaseholder, leaseValid) r.store.metrics.DecommissioningNudgerNotLeaseholderOrInvalidLease.Inc(1) return } From 782caef1dc2e3bac171a235c574e33a850c3136f Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Mon, 25 Aug 2025 20:19:55 -0400 Subject: [PATCH 04/44] kvserver: fix comment when dropping due to exceeding size Previously, the comment on the queue incorrectly stated that it removes the lowest-priority element when exceeding its maximum size. This was misleading because heap only guarantees that the root is the highest priority, not that elements are globally ordered. This commit updates the comment to clarify that the removed element might not be the lowest priority. Ideally, we should drop the lowest priority element when exceeding queue size, but heap doesn't make this very easy. --- pkg/kv/kvserver/queue.go | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index 5d9da3efe1b4..1fb18891025d 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -763,8 +763,11 @@ func (bq *baseQueue) addInternal( item = &replicaItem{rangeID: desc.RangeID, replicaID: replicaID, priority: priority} bq.addLocked(item) - // If adding this replica has pushed the queue past its maximum size, - // remove the lowest priority element. + // If adding this replica has pushed the queue past its maximum size, remove + // an element. Note that it might not be the lowest priority since heap is not + // guaranteed to be globally ordered. Ideally, we would remove the lowest + // priority element, but it would require additional bookkeeping or a linear + // scan. if pqLen := bq.mu.priorityQ.Len(); pqLen > bq.maxSize { bq.removeLocked(bq.mu.priorityQ.sl[pqLen-1]) } From ae1ade8e6a272a074de16f346caf27bcf0330545 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Tue, 26 Aug 2025 08:32:51 -0400 Subject: [PATCH 05/44] kvserver: add logging for ranges dropped from base queue This commit adds logging for ranges dropped from the base queue due to exceeding max size, improving observability. The log is gated behind V(1) to avoid verbosity on nodes with many ranges. --- pkg/kv/kvserver/queue.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index 1fb18891025d..c1f479c4356f 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -769,7 +769,10 @@ func (bq *baseQueue) addInternal( // priority element, but it would require additional bookkeeping or a linear // scan. if pqLen := bq.mu.priorityQ.Len(); pqLen > bq.maxSize { - bq.removeLocked(bq.mu.priorityQ.sl[pqLen-1]) + replicaItemToDrop := bq.mu.priorityQ.sl[pqLen-1] + log.Dev.VInfof(ctx, 1, "dropping due to exceeding queue max size: priority=%0.3f, replica=%v", + priority, replicaItemToDrop.replicaID) + bq.removeLocked(replicaItemToDrop) } // Signal the processLoop that a replica has been added. select { From e91c47c7d81680ec6c011d1e9b71adfe2f7fbd16 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Thu, 21 Aug 2025 21:32:35 -0400 Subject: [PATCH 06/44] kvserver: plumb enqueue time priority This commit plumbs the enqueue time priority into baseQueue.processReplica, enabling comparison between the priority at enqueue time and at processing time. For now, we pass -1 in all cases except when processing replicas directly from the base queue, where -1 signals that priority verification should be skipped. No logic change has been made yet to check for priority inversion; future commits will extend processReplica to validate that processing priority has not differed significantly from the enqueue time priority. --- pkg/kv/kvserver/consistency_queue.go | 2 +- pkg/kv/kvserver/helpers_test.go | 2 +- pkg/kv/kvserver/lease_queue.go | 2 +- pkg/kv/kvserver/merge_queue.go | 4 ++-- pkg/kv/kvserver/mvcc_gc_queue.go | 2 +- pkg/kv/kvserver/mvcc_gc_queue_test.go | 12 +++++------ pkg/kv/kvserver/queue.go | 26 +++++++++++++---------- pkg/kv/kvserver/queue_concurrency_test.go | 2 +- pkg/kv/kvserver/queue_test.go | 10 ++++----- pkg/kv/kvserver/raft_log_queue.go | 2 +- pkg/kv/kvserver/raft_snapshot_queue.go | 2 +- pkg/kv/kvserver/replica_command.go | 2 +- pkg/kv/kvserver/replica_gc_queue.go | 2 +- pkg/kv/kvserver/replica_test.go | 2 +- pkg/kv/kvserver/replicate_queue.go | 15 ++++++++----- pkg/kv/kvserver/split_queue.go | 2 +- pkg/kv/kvserver/store.go | 6 +++--- pkg/kv/kvserver/ts_maintenance_queue.go | 2 +- 18 files changed, 53 insertions(+), 44 deletions(-) diff --git a/pkg/kv/kvserver/consistency_queue.go b/pkg/kv/kvserver/consistency_queue.go index 26f6e125c847..265b84bc6d61 100644 --- a/pkg/kv/kvserver/consistency_queue.go +++ b/pkg/kv/kvserver/consistency_queue.go @@ -165,7 +165,7 @@ func consistencyQueueShouldQueueImpl( // process() is called on every range for which this node is a lease holder. func (q *consistencyQueue) process( - ctx context.Context, repl *Replica, _ spanconfig.StoreReader, + ctx context.Context, repl *Replica, _ spanconfig.StoreReader, _ float64, ) (bool, error) { if q.interval() <= 0 { return false, nil diff --git a/pkg/kv/kvserver/helpers_test.go b/pkg/kv/kvserver/helpers_test.go index 1d4de6b9f477..d86736d6eea5 100644 --- a/pkg/kv/kvserver/helpers_test.go +++ b/pkg/kv/kvserver/helpers_test.go @@ -212,7 +212,7 @@ func manualQueue(s *Store, q queueImpl, repl *Replica) error { return fmt.Errorf("%s: system config not yet available", s) } ctx := repl.AnnotateCtx(context.Background()) - _, err := q.process(ctx, repl, cfg) + _, err := q.process(ctx, repl, cfg, -1 /*priorityAtEnqueue*/) return err } diff --git a/pkg/kv/kvserver/lease_queue.go b/pkg/kv/kvserver/lease_queue.go index 61e1cbe1813c..a79868e77420 100644 --- a/pkg/kv/kvserver/lease_queue.go +++ b/pkg/kv/kvserver/lease_queue.go @@ -114,7 +114,7 @@ func (lq *leaseQueue) shouldQueue( } func (lq *leaseQueue) process( - ctx context.Context, repl *Replica, confReader spanconfig.StoreReader, + ctx context.Context, repl *Replica, confReader spanconfig.StoreReader, _ float64, ) (processed bool, err error) { if tokenErr := repl.allocatorToken.TryAcquire(ctx, lq.name); tokenErr != nil { return false, tokenErr diff --git a/pkg/kv/kvserver/merge_queue.go b/pkg/kv/kvserver/merge_queue.go index ede6f669dcd5..fb288c2d7756 100644 --- a/pkg/kv/kvserver/merge_queue.go +++ b/pkg/kv/kvserver/merge_queue.go @@ -239,7 +239,7 @@ func (mq *mergeQueue) requestRangeStats( } func (mq *mergeQueue) process( - ctx context.Context, lhsRepl *Replica, confReader spanconfig.StoreReader, + ctx context.Context, lhsRepl *Replica, confReader spanconfig.StoreReader, _ float64, ) (processed bool, err error) { lhsDesc := lhsRepl.Desc() @@ -420,7 +420,7 @@ func (mq *mergeQueue) process( return false, rangeMergePurgatoryError{err} } if testingAggressiveConsistencyChecks { - if _, err := mq.store.consistencyQueue.process(ctx, lhsRepl, confReader); err != nil { + if _, err := mq.store.consistencyQueue.process(ctx, lhsRepl, confReader, -1 /*priorityAtEnqueue*/); err != nil { log.Warningf(ctx, "%v", err) } } diff --git a/pkg/kv/kvserver/mvcc_gc_queue.go b/pkg/kv/kvserver/mvcc_gc_queue.go index c207fb0bdd8e..df2d31eb1316 100644 --- a/pkg/kv/kvserver/mvcc_gc_queue.go +++ b/pkg/kv/kvserver/mvcc_gc_queue.go @@ -662,7 +662,7 @@ func (r *replicaGCer) GC( // 7. push these transactions (again, recreating txn entries). // 8. send a GCRequest. func (mgcq *mvccGCQueue) process( - ctx context.Context, repl *Replica, _ spanconfig.StoreReader, + ctx context.Context, repl *Replica, _ spanconfig.StoreReader, _ float64, ) (processed bool, err error) { // Record the CPU time processing the request for this replica. This is // recorded regardless of errors that are encountered. diff --git a/pkg/kv/kvserver/mvcc_gc_queue_test.go b/pkg/kv/kvserver/mvcc_gc_queue_test.go index 01542904412b..7221244dd03a 100644 --- a/pkg/kv/kvserver/mvcc_gc_queue_test.go +++ b/pkg/kv/kvserver/mvcc_gc_queue_test.go @@ -910,7 +910,7 @@ func testMVCCGCQueueProcessImpl(t *testing.T, snapshotBounds bool) { // Process through a scan queue. mgcq := newMVCCGCQueue(tc.store) - processed, err := mgcq.process(ctx, tc.repl, cfg) + processed, err := mgcq.process(ctx, tc.repl, cfg, -1 /* priorityAtEnqueue */) if err != nil { t.Fatal(err) } @@ -1161,7 +1161,7 @@ func TestMVCCGCQueueTransactionTable(t *testing.T) { t.Fatal(err) } - processed, err := mgcq.process(ctx, tc.repl, cfg) + processed, err := mgcq.process(ctx, tc.repl, cfg, -1 /* priorityAtEnqueue */) if err != nil { t.Fatal(err) } @@ -1178,7 +1178,7 @@ func TestMVCCGCQueueTransactionTable(t *testing.T) { if err != nil { return err } - if expGC := (sp.newStatus == -1); expGC { + if expGC := (sp.newStatus == -1 /* priorityAtEnqueue */); expGC { if expGC != !ok { return fmt.Errorf("%s: expected gc: %t, but found %s\n%s", strKey, expGC, txn, roachpb.Key(strKey)) } @@ -1295,7 +1295,7 @@ func TestMVCCGCQueueIntentResolution(t *testing.T) { t.Fatal(err) } mgcq := newMVCCGCQueue(tc.store) - processed, err := mgcq.process(ctx, tc.repl, confReader) + processed, err := mgcq.process(ctx, tc.repl, confReader, -1 /* priorityAtEnqueue */) if err != nil { t.Fatal(err) } @@ -1360,7 +1360,7 @@ func TestMVCCGCQueueLastProcessedTimestamps(t *testing.T) { // Process through a scan queue. mgcq := newMVCCGCQueue(tc.store) - processed, err := mgcq.process(ctx, tc.repl, confReader) + processed, err := mgcq.process(ctx, tc.repl, confReader, -1 /* priorityAtEnqueue */) if err != nil { t.Fatal(err) } @@ -1471,7 +1471,7 @@ func TestMVCCGCQueueChunkRequests(t *testing.T) { } tc.manualClock.Advance(conf.TTL() + 1) mgcq := newMVCCGCQueue(tc.store) - processed, err := mgcq.process(ctx, tc.repl, confReader) + processed, err := mgcq.process(ctx, tc.repl, confReader, -1 /* priorityAtEnqueue */) if err != nil { t.Fatal(err) } diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index c1f479c4356f..e09ffa4720b6 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -124,7 +124,9 @@ type replicaItem struct { replicaID roachpb.ReplicaID seq int // enforce FIFO order for equal priorities - // fields used when a replicaItem is enqueued in a priority queue. + // fields used when a replicaItem is enqueued in a priority queue. This field + // is preserved for purgatory queue as well since baseQueue.processReplica + // requies it. priority float64 index int // The index of the item in the heap, maintained by the heap.Interface methods @@ -261,7 +263,7 @@ type queueImpl interface { // queue-specific work on it. The Replica is guaranteed to be initialized. // We return a boolean to indicate if the Replica was processed successfully // (vs. it being a no-op or an error). - process(context.Context, *Replica, spanconfig.StoreReader) (processed bool, err error) + process(context.Context, *Replica, spanconfig.StoreReader, float64) (processed bool, err error) // processScheduled is called after async task was created to run process. // This function is called by the process loop synchronously. This method is @@ -873,10 +875,10 @@ func (bq *baseQueue) processLoop(stopper *stop.Stopper) { // Acquire from the process semaphore. bq.processSem <- struct{}{} - repl, priority := bq.pop() + repl, priorityAtEnqueue := bq.pop() if repl != nil { - bq.processOneAsyncAndReleaseSem(ctx, repl, stopper) - bq.impl.postProcessScheduled(ctx, repl, priority) + bq.processOneAsyncAndReleaseSem(ctx, repl, stopper, priorityAtEnqueue) + bq.impl.postProcessScheduled(ctx, repl, priorityAtEnqueue) } else { // Release semaphore if no replicas were available. <-bq.processSem @@ -904,7 +906,7 @@ func (bq *baseQueue) processLoop(stopper *stop.Stopper) { // processOneAsyncAndReleaseSem processes a replica if possible and releases the // processSem when the processing is complete. func (bq *baseQueue) processOneAsyncAndReleaseSem( - ctx context.Context, repl replicaInQueue, stopper *stop.Stopper, + ctx context.Context, repl replicaInQueue, stopper *stop.Stopper, priorityAtEnqueue float64, ) { ctx = repl.AnnotateCtx(ctx) taskName := bq.processOpName() + " [outer]" @@ -920,7 +922,7 @@ func (bq *baseQueue) processOneAsyncAndReleaseSem( // Release semaphore when finished processing. defer func() { <-bq.processSem }() start := timeutil.Now() - err := bq.processReplica(ctx, repl) + err := bq.processReplica(ctx, repl, priorityAtEnqueue) bq.recordProcessDuration(ctx, timeutil.Since(start)) bq.finishProcessingReplica(ctx, stopper, repl, err) }); err != nil { @@ -953,7 +955,9 @@ func (bq *baseQueue) recordProcessDuration(ctx context.Context, dur time.Duratio // // ctx should already be annotated by both bq.AnnotateCtx() and // repl.AnnotateCtx(). -func (bq *baseQueue) processReplica(ctx context.Context, repl replicaInQueue) error { +func (bq *baseQueue) processReplica( + ctx context.Context, repl replicaInQueue, priorityAtEnqueue float64, +) error { ctx, span := tracing.EnsureChildSpan(ctx, bq.Tracer, bq.processOpName()) defer span.Finish() @@ -977,7 +981,7 @@ func (bq *baseQueue) processReplica(ctx context.Context, repl replicaInQueue) er // it may not be and shouldQueue will be passed a nil realRepl. These tests // know what they're getting into so that's fine. realRepl, _ := repl.(*Replica) - processed, err := bq.impl.process(ctx, realRepl, conf) + processed, err := bq.impl.process(ctx, realRepl, conf, priorityAtEnqueue) if err != nil { return err } @@ -1318,7 +1322,7 @@ func (bq *baseQueue) processReplicasInPurgatory( if _, err := bq.replicaCanBeProcessed(ctx, repl, false); err != nil { bq.finishProcessingReplica(ctx, stopper, repl, err) } else { - err = bq.processReplica(ctx, repl) + err = bq.processReplica(ctx, repl, -1 /*priorityAtEnqueue*/) bq.finishProcessingReplica(ctx, stopper, repl, err) } }, @@ -1445,7 +1449,7 @@ func (bq *baseQueue) DrainQueue(ctx context.Context, stopper *stop.Stopper) { if _, err := bq.replicaCanBeProcessed(annotatedCtx, repl, false); err != nil { bq.finishProcessingReplica(annotatedCtx, stopper, repl, err) } else { - err = bq.processReplica(annotatedCtx, repl) + err = bq.processReplica(annotatedCtx, repl, -1 /*priorityAtEnqueue*/) bq.finishProcessingReplica(annotatedCtx, stopper, repl, err) } } diff --git a/pkg/kv/kvserver/queue_concurrency_test.go b/pkg/kv/kvserver/queue_concurrency_test.go index f224c6c4d5f0..c5d2fb1a06b3 100644 --- a/pkg/kv/kvserver/queue_concurrency_test.go +++ b/pkg/kv/kvserver/queue_concurrency_test.go @@ -140,7 +140,7 @@ func (fakeQueueImpl) shouldQueue( } func (fq fakeQueueImpl) process( - ctx context.Context, repl *Replica, confReader spanconfig.StoreReader, + ctx context.Context, repl *Replica, confReader spanconfig.StoreReader, _ float64, ) (bool, error) { return fq.pr(ctx, repl, confReader) } diff --git a/pkg/kv/kvserver/queue_test.go b/pkg/kv/kvserver/queue_test.go index 9443c2cb9e0e..0b65b363247e 100644 --- a/pkg/kv/kvserver/queue_test.go +++ b/pkg/kv/kvserver/queue_test.go @@ -57,7 +57,7 @@ func (tq *testQueueImpl) shouldQueue( } func (tq *testQueueImpl) process( - _ context.Context, _ *Replica, _ spanconfig.StoreReader, + _ context.Context, _ *Replica, _ spanconfig.StoreReader, _ float64, ) (bool, error) { defer atomic.AddInt32(&tq.processed, 1) if tq.err != nil { @@ -989,7 +989,7 @@ type processTimeoutQueueImpl struct { var _ queueImpl = &processTimeoutQueueImpl{} func (pq *processTimeoutQueueImpl) process( - ctx context.Context, r *Replica, _ spanconfig.StoreReader, + ctx context.Context, r *Replica, _ spanconfig.StoreReader, _ float64, ) (processed bool, err error) { <-ctx.Done() atomic.AddInt32(&pq.processed, 1) @@ -1119,7 +1119,7 @@ type processTimeQueueImpl struct { var _ queueImpl = &processTimeQueueImpl{} func (pq *processTimeQueueImpl) process( - _ context.Context, _ *Replica, _ spanconfig.StoreReader, + _ context.Context, _ *Replica, _ spanconfig.StoreReader, _ float64, ) (processed bool, err error) { time.Sleep(5 * time.Millisecond) return true, nil @@ -1343,13 +1343,13 @@ type parallelQueueImpl struct { var _ queueImpl = ¶llelQueueImpl{} func (pq *parallelQueueImpl) process( - ctx context.Context, repl *Replica, confReader spanconfig.StoreReader, + ctx context.Context, repl *Replica, confReader spanconfig.StoreReader, priority float64, ) (processed bool, err error) { atomic.AddInt32(&pq.processing, 1) if pq.processBlocker != nil { <-pq.processBlocker } - processed, err = pq.testQueueImpl.process(ctx, repl, confReader) + processed, err = pq.testQueueImpl.process(ctx, repl, confReader, priority) atomic.AddInt32(&pq.processing, -1) return processed, err } diff --git a/pkg/kv/kvserver/raft_log_queue.go b/pkg/kv/kvserver/raft_log_queue.go index caa389355c75..94445f0e49f0 100644 --- a/pkg/kv/kvserver/raft_log_queue.go +++ b/pkg/kv/kvserver/raft_log_queue.go @@ -658,7 +658,7 @@ func (rlq *raftLogQueue) shouldQueueImpl( // leader and if the total number of the range's raft log's stale entries // exceeds RaftLogQueueStaleThreshold. func (rlq *raftLogQueue) process( - ctx context.Context, r *Replica, _ spanconfig.StoreReader, + ctx context.Context, r *Replica, _ spanconfig.StoreReader, _ float64, ) (processed bool, err error) { decision, err := newTruncateDecision(ctx, r) if err != nil { diff --git a/pkg/kv/kvserver/raft_snapshot_queue.go b/pkg/kv/kvserver/raft_snapshot_queue.go index 34267fd83a05..77160bd4ad04 100644 --- a/pkg/kv/kvserver/raft_snapshot_queue.go +++ b/pkg/kv/kvserver/raft_snapshot_queue.go @@ -78,7 +78,7 @@ func (rq *raftSnapshotQueue) shouldQueue( } func (rq *raftSnapshotQueue) process( - ctx context.Context, repl *Replica, _ spanconfig.StoreReader, + ctx context.Context, repl *Replica, _ spanconfig.StoreReader, _ float64, ) (anyProcessed bool, _ error) { // If a follower requires a Raft snapshot, perform it. if status := repl.RaftStatus(); status != nil { diff --git a/pkg/kv/kvserver/replica_command.go b/pkg/kv/kvserver/replica_command.go index 7235e1bfd5a7..9d11b63b5b03 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -4229,7 +4229,7 @@ func (r *Replica) scatterRangeAndRandomizeLeases(ctx context.Context, randomizeL break } _, err = rq.processOneChange( - ctx, r, desc, conf, true /* scatter */, false, /* dryRun */ + ctx, r, desc, conf, true /* scatter */, false /* dryRun */, -1, /*priorityAtEnqueue*/ ) if err != nil { // If the error is expected to be transient, retry processing the range. diff --git a/pkg/kv/kvserver/replica_gc_queue.go b/pkg/kv/kvserver/replica_gc_queue.go index 9d6be6e639d5..fb7dfb4abee8 100644 --- a/pkg/kv/kvserver/replica_gc_queue.go +++ b/pkg/kv/kvserver/replica_gc_queue.go @@ -215,7 +215,7 @@ func replicaGCShouldQueueImpl(now, lastCheck hlc.Timestamp, isSuspect bool) (boo // process performs a consistent lookup on the range descriptor to see if we are // still a member of the range. func (rgcq *replicaGCQueue) process( - ctx context.Context, repl *Replica, _ spanconfig.StoreReader, + ctx context.Context, repl *Replica, _ spanconfig.StoreReader, _ float64, ) (processed bool, err error) { // Note that the Replicas field of desc is probably out of date, so // we should only use `desc` for its static fields like RangeID and diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 68c2638865f7..b3afdbfc839e 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -10494,7 +10494,7 @@ func TestConsistenctQueueErrorFromCheckConsistency(t *testing.T) { } for i := 0; i < 2; i++ { // Do this twice because it used to deadlock. See #25456. - processed, err := tc.store.consistencyQueue.process(ctx, tc.repl, confReader) + processed, err := tc.store.consistencyQueue.process(ctx, tc.repl, confReader, -1 /*priorityAtEnqueue*/) if !testutils.IsError(err, "boom") { t.Fatal(err) } diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index 01f8695e1ac6..2a0ee03ee579 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -631,7 +631,7 @@ func (rq *replicateQueue) shouldQueue( } func (rq *replicateQueue) process( - ctx context.Context, repl *Replica, confReader spanconfig.StoreReader, + ctx context.Context, repl *Replica, confReader spanconfig.StoreReader, priorityAtEnqueue float64, ) (processed bool, err error) { if tokenErr := repl.allocatorToken.TryAcquire(ctx, rq.name); tokenErr != nil { log.KvDistribution.VEventf(ctx, @@ -657,7 +657,7 @@ func (rq *replicateQueue) process( // usually signaling that a rebalancing reservation could not be made with the // selected target. for r := retry.StartWithCtx(ctx, retryOpts); r.Next(); { - requeue, err := rq.processOneChangeWithTracing(ctx, repl, desc, &conf) + requeue, err := rq.processOneChangeWithTracing(ctx, repl, desc, &conf, priorityAtEnqueue) if isSnapshotError(err) { // If ChangeReplicas failed because the snapshot failed, we attempt to // retry the operation. The most likely causes of the snapshot failing @@ -682,7 +682,7 @@ func (rq *replicateQueue) process( } if testingAggressiveConsistencyChecks { - if _, err := rq.store.consistencyQueue.process(ctx, repl, confReader); err != nil { + if _, err := rq.store.consistencyQueue.process(ctx, repl, confReader, -1 /*priorityAtEnqueue*/); err != nil { log.KvDistribution.Warningf(ctx, "%v", err) } } @@ -741,7 +741,11 @@ func filterTracingSpans(rec tracingpb.Recording, opNamesToFilter ...string) trac // logging the resulting traces to the DEV channel in the case of errors or // when the configured log traces threshold is exceeded. func (rq *replicateQueue) processOneChangeWithTracing( - ctx context.Context, repl *Replica, desc *roachpb.RangeDescriptor, conf *roachpb.SpanConfig, + ctx context.Context, + repl *Replica, + desc *roachpb.RangeDescriptor, + conf *roachpb.SpanConfig, + priorityAtEnqueue float64, ) (requeue bool, _ error) { processStart := timeutil.Now() startTracing := log.ExpensiveLogEnabled(ctx, 1) @@ -756,7 +760,7 @@ func (rq *replicateQueue) processOneChangeWithTracing( defer sp.Finish() requeue, err := rq.processOneChange(ctx, repl, desc, conf, - false /* scatter */, false, /* dryRun */ + false /* scatter */, false /* dryRun */, priorityAtEnqueue, ) processDuration := timeutil.Since(processStart) loggingThreshold := rq.logTracesThresholdFunc(rq.store.cfg.Settings, repl) @@ -863,6 +867,7 @@ func (rq *replicateQueue) processOneChange( desc *roachpb.RangeDescriptor, conf *roachpb.SpanConfig, scatter, dryRun bool, + priorityAtEnqueue float64, ) (requeue bool, _ error) { change, err := rq.planner.PlanOneChange( ctx, repl, desc, conf, plan.PlannerOptions{Scatter: scatter}) diff --git a/pkg/kv/kvserver/split_queue.go b/pkg/kv/kvserver/split_queue.go index 7b44caa6ab66..fa3d7ccaf632 100644 --- a/pkg/kv/kvserver/split_queue.go +++ b/pkg/kv/kvserver/split_queue.go @@ -218,7 +218,7 @@ var _ PurgatoryError = unsplittableRangeError{} // process synchronously invokes admin split for each proposed split key. func (sq *splitQueue) process( - ctx context.Context, r *Replica, confReader spanconfig.StoreReader, + ctx context.Context, r *Replica, confReader spanconfig.StoreReader, _ float64, ) (processed bool, err error) { processed, err = sq.processAttemptWithTracing(ctx, r, confReader) if errors.HasType(err, (*kvpb.ConditionFailedError)(nil)) { diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 8c62516bed63..b36ad53ef156 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -3897,7 +3897,7 @@ func (s *Store) ReplicateQueueDryRun( return collectAndFinish(), nil } _, err = s.replicateQueue.processOneChange( - ctx, repl, desc, conf, false /* scatter */, true, /* dryRun */ + ctx, repl, desc, conf, false /* scatter */, true /* dryRun */, -1, /*priorityAtEnqueue*/ ) if err != nil { log.Eventf(ctx, "error simulating allocator on replica %s: %s", repl, err) @@ -4076,7 +4076,7 @@ func (s *Store) Enqueue( } log.Eventf(ctx, "running %s.process", queueName) - processed, processErr := qImpl.process(ctx, repl, confReader) + processed, processErr := qImpl.process(ctx, repl, confReader, -1 /*priorityAtEnqueue*/) log.Eventf(ctx, "processed: %t (err: %v)", processed, processErr) return processErr, nil } @@ -4113,7 +4113,7 @@ func (s *Store) PurgeOutdatedReplicas(ctx context.Context, version roachpb.Versi g.GoCtx(func(ctx context.Context) error { defer alloc.Release() - processed, err := s.replicaGCQueue.process(ctx, repl, nil) + processed, err := s.replicaGCQueue.process(ctx, repl, nil, -1 /*priorityAtEnqueue*/) if err != nil { return errors.Wrapf(err, "on %s", repl.Desc()) } diff --git a/pkg/kv/kvserver/ts_maintenance_queue.go b/pkg/kv/kvserver/ts_maintenance_queue.go index 15feff9eed34..de7a89fed903 100644 --- a/pkg/kv/kvserver/ts_maintenance_queue.go +++ b/pkg/kv/kvserver/ts_maintenance_queue.go @@ -135,7 +135,7 @@ func (q *timeSeriesMaintenanceQueue) shouldQueue( } func (q *timeSeriesMaintenanceQueue) process( - ctx context.Context, repl *Replica, _ spanconfig.StoreReader, + ctx context.Context, repl *Replica, _ spanconfig.StoreReader, _ float64, ) (processed bool, err error) { desc := repl.Desc() eng := repl.store.StateEngine() From 4b419aaf576c45837cfdf56b4a706dad65c28565 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Thu, 21 Aug 2025 22:06:54 -0400 Subject: [PATCH 07/44] kvserver: remove priority reset during setProcessing MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Previously, a replicaItem’s priority was cleared when marked as processing, to indicate it was no longer in the priority queue. This behavior made sense when the purgatory queue did not track priorities. However, we now need to preserve priorities for items in purgatory as well since they will be calling into baseQueue.processReplica. This commit removes the priority reset in replicaItem.SetProcessing(), ensuring that the enqueue time priority is retained when replicas are popped from the heap and passed into the purgatory queue properly. No behavior change should happen from this change. --- pkg/kv/kvserver/queue.go | 1 - 1 file changed, 1 deletion(-) diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index e09ffa4720b6..cfa2c2f369eb 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -138,7 +138,6 @@ type replicaItem struct { // setProcessing moves the item from an enqueued state to a processing state. func (i *replicaItem) setProcessing() { - i.priority = 0 if i.index >= 0 { log.Fatalf(context.Background(), "r%d marked as processing but appears in prioQ", i.rangeID, From 8ceb25e00e8718649f841bab2fb14d746a3977ec Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Thu, 21 Aug 2025 22:13:42 -0400 Subject: [PATCH 08/44] kvserver: plumb priority at enqueue for purgatory queue Previously, replica items in the purgatory queue did not retain their enqueue time priority. This commit ensures that the priority is preserved so it can be passed to baseQueue.processReplica when processing items from purgatory. --- pkg/kv/kvserver/queue.go | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index cfa2c2f369eb..c38f5bf65992 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -1158,6 +1158,7 @@ func (bq *baseQueue) finishProcessingReplica( processing := item.processing callbacks := item.callbacks requeue := item.requeue + priority := item.priority item.callbacks = nil bq.removeFromReplicaSetLocked(repl.GetRangeID()) item = nil // prevent accidental use below @@ -1188,7 +1189,7 @@ func (bq *baseQueue) finishProcessingReplica( // purgatory. if purgErr, ok := IsPurgatoryError(err); ok { bq.mu.Lock() - bq.addToPurgatoryLocked(ctx, stopper, repl, purgErr) + bq.addToPurgatoryLocked(ctx, stopper, repl, purgErr, priority /*priorityAtEnqueue*/) bq.mu.Unlock() return } @@ -1208,7 +1209,11 @@ func (bq *baseQueue) finishProcessingReplica( // addToPurgatoryLocked adds the specified replica to the purgatory queue, which // holds replicas which have failed processing. func (bq *baseQueue) addToPurgatoryLocked( - ctx context.Context, stopper *stop.Stopper, repl replicaInQueue, purgErr PurgatoryError, + ctx context.Context, + stopper *stop.Stopper, + repl replicaInQueue, + purgErr PurgatoryError, + priorityAtEnqueue float64, ) { bq.mu.AssertHeld() @@ -1232,7 +1237,7 @@ func (bq *baseQueue) addToPurgatoryLocked( return } - item := &replicaItem{rangeID: repl.GetRangeID(), replicaID: repl.ReplicaID(), index: -1} + item := &replicaItem{rangeID: repl.GetRangeID(), replicaID: repl.ReplicaID(), index: -1, priority: priorityAtEnqueue} bq.mu.replicas[repl.GetRangeID()] = item defer func() { @@ -1321,7 +1326,7 @@ func (bq *baseQueue) processReplicasInPurgatory( if _, err := bq.replicaCanBeProcessed(ctx, repl, false); err != nil { bq.finishProcessingReplica(ctx, stopper, repl, err) } else { - err = bq.processReplica(ctx, repl, -1 /*priorityAtEnqueue*/) + err = bq.processReplica(ctx, repl, item.priority /*priorityAtEnqueue*/) bq.finishProcessingReplica(ctx, stopper, repl, err) } }, From 2c27b47be46960513da67a8cd5afb771009a1d49 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Tue, 26 Aug 2025 09:37:28 -0400 Subject: [PATCH 09/44] allocatorimpl: adds a priority assertion to computeAction MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This commit adds an assertion to Allocator.ComputeAction to ensure that priority is never -1 in cases where it shouldn’t be. Normally, ComputeAction returns action.Priority(), but we sometimes adjust the priority for specific actions like AllocatorAddVoter, AllocatorRemoveDeadVoter, and AllocatorRemoveVoter. A priority of -1 is a special case reserved for processing logic to run even if there’s a priority inversion. If the priority is not -1, the range may be re-queued to be processed with the correct priority. --- .../allocator/allocatorimpl/allocator.go | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go b/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go index 21e35f510481..07613d319dbc 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go +++ b/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -955,8 +956,21 @@ func (a *Allocator) ComputeAction( return action, action.Priority() } - return a.computeAction(ctx, storePool, conf, desc.Replicas().VoterDescriptors(), + action, priority = a.computeAction(ctx, storePool, conf, desc.Replicas().VoterDescriptors(), desc.Replicas().NonVoterDescriptors()) + // Ensure that priority is never -1. Typically, computeAction return + // action.Priority(), but we sometimes modify the priority for specific + // actions like AllocatorAddVoter, AllocatorRemoveDeadVoter, and + // AllocatorRemoveVoter. A priority of -1 is a special case, indicating that + // the caller expects the processing logic to be invoked even if there's a + // priority inversion. If the priority is not -1, the range might be re-queued + // to be processed with the correct priority. + if priority == -1 && buildutil.CrdbTestBuild { + log.Fatalf(ctx, "allocator returned -1 priority for range %s: %v", desc, action) + } else { + log.Warningf(ctx, "allocator returned -1 priority for range %s: %v", desc, action) + } + return action, priority } func (a *Allocator) computeAction( From e1ff3480a0e2bf361c96f5832ca3f8e015a61ec4 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Tue, 26 Aug 2025 13:14:18 -0400 Subject: [PATCH 10/44] allocatorimpl: add invariants on priority to base queue tests This commit adds additional invariants to verify the correctness of priority plumbing for range items in base queue tests. --- .../allocator/allocatorimpl/BUILD.bazel | 1 + pkg/kv/kvserver/queue.go | 11 +++- pkg/kv/kvserver/queue_concurrency_test.go | 2 +- pkg/kv/kvserver/queue_test.go | 52 ++++++++++++++++++- 4 files changed, 61 insertions(+), 5 deletions(-) diff --git a/pkg/kv/kvserver/allocator/allocatorimpl/BUILD.bazel b/pkg/kv/kvserver/allocator/allocatorimpl/BUILD.bazel index d432c584b732..d5fc9ca59070 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/BUILD.bazel +++ b/pkg/kv/kvserver/allocator/allocatorimpl/BUILD.bazel @@ -28,6 +28,7 @@ go_library( "//pkg/settings", "//pkg/settings/cluster", "//pkg/util/admission/admissionpb", + "//pkg/util/buildutil", "//pkg/util/log", "//pkg/util/metric", "//pkg/util/stop", diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index c38f5bf65992..70d08ffc4489 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -1095,19 +1095,24 @@ func IsPurgatoryError(err error) (PurgatoryError, bool) { } // assertInvariants codifies the guarantees upheld by the data structures in the -// base queue. In summary, a replica is one of: +// base queue. +// 1. In summary, a replica is one of: // - "queued" and in mu.replicas and mu.priorityQ // - "processing" and only in mu.replicas // - "purgatory" and in mu.replicas and mu.purgatory +// 2. For every item in bq.mu.priorityQ.sl, bq.mu.purgatory, and bq.mu.replicas, +// assertOnReplicaItem callback is called with the item. Note that we expect +// items in priorityQ and purgatory to be in replicas. // // Note that in particular, nothing is ever in both mu.priorityQ and // mu.purgatory. -func (bq *baseQueue) assertInvariants() { +func (bq *baseQueue) assertInvariants(assertOnReplicaItem func(item *replicaItem)) { bq.mu.Lock() defer bq.mu.Unlock() ctx := bq.AnnotateCtx(context.Background()) for _, item := range bq.mu.priorityQ.sl { + assertOnReplicaItem(item) if item.processing { log.Fatalf(ctx, "processing item found in prioQ: %v", item) } @@ -1120,6 +1125,7 @@ func (bq *baseQueue) assertInvariants() { } for rangeID := range bq.mu.purgatory { item, inReplicas := bq.mu.replicas[rangeID] + assertOnReplicaItem(item) if !inReplicas { log.Fatalf(ctx, "item found in purg but not in mu.replicas: %v", item) } @@ -1134,6 +1140,7 @@ func (bq *baseQueue) assertInvariants() { // that there aren't any non-processing replicas *only* in bq.mu.replicas. var nNotProcessing int for _, item := range bq.mu.replicas { + assertOnReplicaItem(item) if !item.processing { nNotProcessing++ } diff --git a/pkg/kv/kvserver/queue_concurrency_test.go b/pkg/kv/kvserver/queue_concurrency_test.go index c5d2fb1a06b3..e80774bb4ef4 100644 --- a/pkg/kv/kvserver/queue_concurrency_test.go +++ b/pkg/kv/kvserver/queue_concurrency_test.go @@ -112,7 +112,7 @@ func TestBaseQueueConcurrent(t *testing.T) { }) } g.Go(func() error { - bq.assertInvariants() + bq.assertInvariants(func(item *replicaItem) {}) return nil }) } diff --git a/pkg/kv/kvserver/queue_test.go b/pkg/kv/kvserver/queue_test.go index 0b65b363247e..a20ca8a071fc 100644 --- a/pkg/kv/kvserver/queue_test.go +++ b/pkg/kv/kvserver/queue_test.go @@ -224,6 +224,11 @@ func TestBaseQueueAddUpdateAndRemove(t *testing.T) { } else { bq.finishProcessingReplica(ctx, stopper, r2, nil) } + bq.assertInvariants(func(item *replicaItem) { + replica, err := bq.getReplica(item.rangeID) + require.NoError(t, err) + require.Equal(t, priorityMap[replica.(*Replica)], item.priority) + }) if v := bq.pending.Value(); v != 1 { t.Errorf("expected 1 pending replicas; got %d", v) } @@ -296,6 +301,11 @@ func TestBaseQueueAddUpdateAndRemove(t *testing.T) { if r, _ := bq.pop(); r != nil { t.Errorf("expected empty queue; got %v", r) } + bq.assertInvariants(func(item *replicaItem) { + replica, err := bq.getReplica(item.rangeID) + require.NoError(t, err) + require.Equal(t, priorityMap[replica.(*Replica)], item.priority) + }) // Try removing a replica. bq.maybeAdd(ctx, r1, hlc.ClockTimestamp{}) @@ -315,6 +325,11 @@ func TestBaseQueueAddUpdateAndRemove(t *testing.T) { if v := bq.pending.Value(); v != 0 { t.Errorf("expected 0 pending replicas; got %d", v) } + bq.assertInvariants(func(item *replicaItem) { + replica, err := bq.getReplica(item.rangeID) + require.NoError(t, err) + require.Equal(t, priorityMap[replica.(*Replica)], item.priority) + }) } // TestBaseQueueSamePriorityFIFO verifies that if multiple items are queued at @@ -540,11 +555,12 @@ func TestBaseQueueAddRemove(t *testing.T) { t.Fatal(err) } + const testPriority = 1.0 testQueue := &testQueueImpl{ blocker: make(chan struct{}, 1), shouldQueueFn: func(now hlc.ClockTimestamp, r *Replica) (shouldQueue bool, priority float64) { shouldQueue = true - priority = 1.0 + priority = testPriority return }, } @@ -552,7 +568,14 @@ func TestBaseQueueAddRemove(t *testing.T) { bq.Start(stopper) bq.maybeAdd(ctx, r, hlc.ClockTimestamp{}) + bq.assertInvariants(func(item *replicaItem) { + require.Equal(t, testPriority, item.priority) + }) + bq.MaybeRemove(r.RangeID) + bq.assertInvariants(func(item *replicaItem) { + require.Equal(t, testPriority, item.priority) + }) // Wake the queue close(testQueue.blocker) @@ -839,10 +862,19 @@ func TestBaseQueuePurgatory(t *testing.T) { bq.maybeAdd(context.Background(), r, hlc.ClockTimestamp{}) } + // Make sure priority is preserved during processing. + bq.assertInvariants(func(item *replicaItem) { + require.Equal(t, float64(item.rangeID), item.priority) + }) + testutils.SucceedsSoon(t, func() error { if pc := testQueue.getProcessed(); pc != replicaCount { return errors.Errorf("expected %d processed replicas; got %d", replicaCount, pc) } + // Make sure priorities are preserved with the purgatory queue. + bq.assertInvariants(func(item *replicaItem) { + require.Equal(t, float64(item.rangeID), item.priority) + }) // We have to loop checking the following conditions because the increment // of testQueue.processed does not happen atomically with the replica being // placed in purgatory. @@ -854,6 +886,9 @@ func TestBaseQueuePurgatory(t *testing.T) { if l := bq.Length(); l != 0 { return errors.Errorf("expected empty priorityQ; got %d", l) } + bq.assertInvariants(func(item *replicaItem) { + require.Equal(t, float64(item.rangeID), item.priority) + }) // Check metrics. if v := bq.successes.Count(); v != 0 { return errors.Errorf("expected 0 processed replicas; got %d", v) @@ -888,6 +923,9 @@ func TestBaseQueuePurgatory(t *testing.T) { if l := bq.Length(); l != 0 { return errors.Errorf("expected empty priorityQ; got %d", l) } + bq.assertInvariants(func(item *replicaItem) { + require.Equal(t, float64(item.rangeID), item.priority) + }) // Check metrics. if v := bq.successes.Count(); v != 0 { return errors.Errorf("expected 0 processed replicas; got %d", v) @@ -923,6 +961,9 @@ func TestBaseQueuePurgatory(t *testing.T) { if pc := testQueue.getProcessed(); pc != replicaCount*3-rmReplCount { return errors.Errorf("expected %d processed replicas; got %d", replicaCount*3-rmReplCount, pc) } + bq.assertInvariants(func(item *replicaItem) { + require.Equal(t, float64(item.rangeID), item.priority) + }) // Check metrics. if v := bq.successes.Count(); v != int64(replicaCount)-rmReplCount { return errors.Errorf("expected %d processed replicas; got %d", replicaCount-rmReplCount, v) @@ -966,6 +1007,9 @@ func TestBaseQueuePurgatory(t *testing.T) { if pc := testQueue.getProcessed(); pc != beforeProcessCount+1 { return errors.Errorf("expected %d processed replicas; got %d", beforeProcessCount+1, pc) } + bq.assertInvariants(func(item *replicaItem) { + require.Equal(t, float64(item.rangeID), item.priority) + }) if v := bq.successes.Count(); v != beforeSuccessCount+1 { return errors.Errorf("expected %d processed replicas; got %d", beforeSuccessCount+1, v) } @@ -1370,11 +1414,12 @@ func TestBaseQueueProcessConcurrently(t *testing.T) { repls := createReplicas(t, &tc, 3) r1, r2, r3 := repls[0], repls[1], repls[2] + const testPriority = 1 pQueue := ¶llelQueueImpl{ testQueueImpl: testQueueImpl{ blocker: make(chan struct{}, 1), shouldQueueFn: func(now hlc.ClockTimestamp, r *Replica) (shouldQueue bool, priority float64) { - return true, 1 + return true, testPriority }, }, processBlocker: make(chan struct{}, 1), @@ -1419,6 +1464,9 @@ func TestBaseQueueProcessConcurrently(t *testing.T) { pQueue.processBlocker <- struct{}{} assertProcessedAndProcessing(3, 0) + bq.assertInvariants(func(item *replicaItem) { + require.Equal(t, float64(testPriority), item.priority) + }) } // TestBaseQueueReplicaChange ensures that if a replica is added to the queue From a9e04983b8d464bcb08148257b4bab49a845e7f8 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Thu, 28 Aug 2025 12:21:58 -0400 Subject: [PATCH 11/44] allocator: correct logging for priority assertion This commit fixes an incorrect log statement in computeAction for priority assertions. The log was mistakenly emitted even when the priority was not -1. Related: https://github.com/cockroachdb/cockroach/pull/152512 Release note: none --- pkg/kv/kvserver/allocator/allocatorimpl/allocator.go | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go b/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go index 07613d319dbc..d220ea907d0d 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go +++ b/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go @@ -965,10 +965,12 @@ func (a *Allocator) ComputeAction( // the caller expects the processing logic to be invoked even if there's a // priority inversion. If the priority is not -1, the range might be re-queued // to be processed with the correct priority. - if priority == -1 && buildutil.CrdbTestBuild { - log.Fatalf(ctx, "allocator returned -1 priority for range %s: %v", desc, action) - } else { - log.Warningf(ctx, "allocator returned -1 priority for range %s: %v", desc, action) + if priority == -1 { + if buildutil.CrdbTestBuild { + log.Fatalf(ctx, "allocator returned -1 priority for range %s: %v", desc, action) + } else { + log.Warningf(ctx, "allocator returned -1 priority for range %s: %v", desc, action) + } } return action, priority } From 29a7f72260e873a9a4aa06e8dc66295976c9be2b Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Tue, 26 Aug 2025 06:48:18 -0400 Subject: [PATCH 12/44] kvserver: remove bq.replicaCanBeProcessed right before bq.processReplica MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Previously, we called bq.replicaCanBeProcessed with acquireLeaseIfNeeded = false before invoking bq.processReplica, which itself calls replicaCanBeProcessed with acquireLeaseIfNeeded = true. This looks incorrect and did not exist prior to https://github.com/cockroachdb/cockroach/commit/c9cf06893bf827a1752213aa3aebee2aaea35f13. It’s unclear how often lease renewal is actually going to be helpful here, but I removed these two calls since they were newly introduced and seem unintentional. Informs: https://github.com/cockroachdb/cockroach/issues/151292 Release note: none --- pkg/kv/kvserver/queue.go | 15 ++------------- 1 file changed, 2 insertions(+), 13 deletions(-) diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index 70d08ffc4489..05a60d0954e0 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -909,13 +909,6 @@ func (bq *baseQueue) processOneAsyncAndReleaseSem( ) { ctx = repl.AnnotateCtx(ctx) taskName := bq.processOpName() + " [outer]" - // Validate that the replica is still in a state that can be processed. If - // it is no longer processable, return immediately. - if _, err := bq.replicaCanBeProcessed(ctx, repl, false /*acquireLeaseIfNeeded */); err != nil { - bq.finishProcessingReplica(ctx, stopper, repl, err) - <-bq.processSem - return - } if err := stopper.RunAsyncTaskEx(ctx, stop.TaskOpts{TaskName: taskName}, func(ctx context.Context) { // Release semaphore when finished processing. @@ -1330,12 +1323,8 @@ func (bq *baseQueue) processReplicasInPurgatory( annotatedCtx := repl.AnnotateCtx(ctx) if stopper.RunTask( annotatedCtx, bq.processOpName(), func(ctx context.Context) { - if _, err := bq.replicaCanBeProcessed(ctx, repl, false); err != nil { - bq.finishProcessingReplica(ctx, stopper, repl, err) - } else { - err = bq.processReplica(ctx, repl, item.priority /*priorityAtEnqueue*/) - bq.finishProcessingReplica(ctx, stopper, repl, err) - } + err = bq.processReplica(ctx, repl, item.priority /*priorityAtEnqueue*/) + bq.finishProcessingReplica(ctx, stopper, repl, err) }, ) != nil { // NB: We do not need to worry about removing any unprocessed replicas From 11e980753b42cab145c683b83d1060654d59e8bc Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Thu, 28 Aug 2025 21:48:46 -0400 Subject: [PATCH 13/44] kvserver: add ReplicateQueueDroppedDueToSize Previously, we had limited observability into when queues drop replicas due to reaching their maximum size. This commit adds a metric to track and observe such events. --- docs/generated/metrics/metrics.html | 1 + pkg/kv/kvserver/metrics.go | 8 ++++++++ pkg/kv/kvserver/queue.go | 8 ++++++++ pkg/kv/kvserver/replicate_queue.go | 1 + 4 files changed, 18 insertions(+) diff --git a/docs/generated/metrics/metrics.html b/docs/generated/metrics/metrics.html index da0bec0ffd6d..7c4933b0c5cc 100644 --- a/docs/generated/metrics/metrics.html +++ b/docs/generated/metrics/metrics.html @@ -420,6 +420,7 @@ STORAGEqueue.replicate.addreplica.errorNumber of failed replica additions processed by the replicate queueReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.addreplica.successNumber of successful replica additions processed by the replicate queueReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.addvoterreplicaNumber of voter replica additions attempted by the replicate queueReplica AdditionsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEqueue.replicate.dropped_due_to_sizeNumber of replicas dropped due to the replicate queue exceeding its max sizeReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.nonvoterpromotionsNumber of non-voters promoted to voters by the replicate queuePromotions of Non Voters to VotersCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.pendingNumber of pending replicas in the replicate queueReplicasGAUGECOUNTAVGNONE STORAGEqueue.replicate.process.failureNumber of replicas which failed processing in the replicate queueReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE diff --git a/pkg/kv/kvserver/metrics.go b/pkg/kv/kvserver/metrics.go index 241df3cc4a82..8818f8712734 100644 --- a/pkg/kv/kvserver/metrics.go +++ b/pkg/kv/kvserver/metrics.go @@ -2052,6 +2052,12 @@ The messages are dropped to help these replicas to recover from I/O overload.`, Measurement: "Replicas", Unit: metric.Unit_COUNT, } + metaReplicateQueueDroppedDueToSize = metric.Metadata{ + Name: "queue.replicate.dropped_due_to_size", + Help: "Number of replicas dropped due to the replicate queue exceeding its max size", + Measurement: "Replicas", + Unit: metric.Unit_COUNT, + } metaReplicateQueueProcessingNanos = metric.Metadata{ Name: "queue.replicate.processingnanos", Help: "Nanoseconds spent processing replicas in the replicate queue", @@ -2955,6 +2961,7 @@ type StoreMetrics struct { ReplicateQueueSuccesses *metric.Counter ReplicateQueueFailures *metric.Counter ReplicateQueuePending *metric.Gauge + ReplicateQueueDroppedDueToSize *metric.Counter ReplicateQueueProcessingNanos *metric.Counter ReplicateQueuePurgatory *metric.Gauge SplitQueueSuccesses *metric.Counter @@ -3733,6 +3740,7 @@ func newStoreMetrics(histogramWindow time.Duration) *StoreMetrics { ReplicateQueueSuccesses: metric.NewCounter(metaReplicateQueueSuccesses), ReplicateQueueFailures: metric.NewCounter(metaReplicateQueueFailures), ReplicateQueuePending: metric.NewGauge(metaReplicateQueuePending), + ReplicateQueueDroppedDueToSize: metric.NewCounter(metaReplicateQueueDroppedDueToSize), ReplicateQueueProcessingNanos: metric.NewCounter(metaReplicateQueueProcessingNanos), ReplicateQueuePurgatory: metric.NewGauge(metaReplicateQueuePurgatory), SplitQueueSuccesses: metric.NewCounter(metaSplitQueueSuccesses), diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index 05a60d0954e0..cfb99f526d26 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -332,6 +332,11 @@ type queueConfig struct { failures *metric.Counter // pending is a gauge measuring current replica count pending. pending *metric.Gauge + // droppedDueToSize is a counter measuring replicas dropped due to + // exceeding the queue max size. + // NB: this metric may be nil for queues that are not interested in tracking + // this. + droppedDueToSize *metric.Counter // processingNanos is a counter measuring total nanoseconds spent processing // replicas. processingNanos *metric.Counter @@ -771,6 +776,9 @@ func (bq *baseQueue) addInternal( // scan. if pqLen := bq.mu.priorityQ.Len(); pqLen > bq.maxSize { replicaItemToDrop := bq.mu.priorityQ.sl[pqLen-1] + if bq.droppedDueToSize != nil { + bq.droppedDueToSize.Inc(1) + } log.Dev.VInfof(ctx, 1, "dropping due to exceeding queue max size: priority=%0.3f, replica=%v", priority, replicaItemToDrop.replicaID) bq.removeLocked(replicaItemToDrop) diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index 2a0ee03ee579..a72b668a40db 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -573,6 +573,7 @@ func newReplicateQueue(store *Store, allocator allocatorimpl.Allocator) *replica successes: store.metrics.ReplicateQueueSuccesses, failures: store.metrics.ReplicateQueueFailures, pending: store.metrics.ReplicateQueuePending, + droppedDueToSize: store.metrics.ReplicateQueueDroppedDueToSize, processingNanos: store.metrics.ReplicateQueueProcessingNanos, purgatory: store.metrics.ReplicateQueuePurgatory, disabledConfig: kvserverbase.ReplicateQueueEnabled, From a27a42ae6cff2375813ea12ef8f80095181ca09e Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Fri, 29 Aug 2025 16:07:38 -0400 Subject: [PATCH 14/44] kvserver: add ReplicateQueueMaxSize MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Previously, the maximum base queue size was hardcoded to defaultQueueMaxSize (10000). Since replica item structs are small, there’s little reason to enforce a fixed limit. This commit makes the replicate queue size configurable via a cluster setting ReplicateQueueMaxSize, allowing incremental and backport-friendly adjustments. Note that reducing the setting does not drop replicas appropirately; future commits will address this behavior. --- pkg/kv/kvserver/queue.go | 11 ++++++++++- pkg/kv/kvserver/replicate_queue.go | 24 ++++++++++++++++++++++++ 2 files changed, 34 insertions(+), 1 deletion(-) diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index cfb99f526d26..3c339f76b294 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -447,6 +447,7 @@ type baseQueue struct { purgatory map[roachpb.RangeID]PurgatoryError // Map of replicas to processing errors stopped bool disabled bool + maxSize int64 } } @@ -499,6 +500,7 @@ func newBaseQueue(name string, impl queueImpl, store *Store, cfg queueConfig) *b }, } bq.mu.replicas = map[roachpb.RangeID]*replicaItem{} + bq.mu.maxSize = int64(cfg.maxSize) bq.SetDisabled(!cfg.disabledConfig.Get(&store.cfg.Settings.SV)) cfg.disabledConfig.SetOnChange(&store.cfg.Settings.SV, func(ctx context.Context) { bq.SetDisabled(!cfg.disabledConfig.Get(&store.cfg.Settings.SV)) @@ -543,6 +545,13 @@ func (bq *baseQueue) SetDisabled(disabled bool) { bq.mu.Unlock() } +// SetMaxSize sets the max size of the queue. +func (bq *baseQueue) SetMaxSize(maxSize int64) { + bq.mu.Lock() + bq.mu.maxSize = maxSize + bq.mu.Unlock() +} + // lockProcessing locks all processing in the baseQueue. It returns // a function to unlock processing. func (bq *baseQueue) lockProcessing() func() { @@ -774,7 +783,7 @@ func (bq *baseQueue) addInternal( // guaranteed to be globally ordered. Ideally, we would remove the lowest // priority element, but it would require additional bookkeeping or a linear // scan. - if pqLen := bq.mu.priorityQ.Len(); pqLen > bq.maxSize { + if pqLen := bq.mu.priorityQ.Len(); int64(pqLen) > bq.mu.maxSize { replicaItemToDrop := bq.mu.priorityQ.sl[pqLen-1] if bq.droppedDueToSize != nil { bq.droppedDueToSize.Inc(1) diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index a72b668a40db..a8d659efa44e 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -100,6 +100,24 @@ var EnqueueProblemRangeInReplicateQueueInterval = settings.RegisterDurationSetti settings.NonNegativeDuration, ) +// ReplicateQueueMaxSize is a setting that controls the max size of the +// replicate queue. When this limit is exceeded, lower priority replicas (not +// guaranteed to be the lowest) are dropped from the queue. +var ReplicateQueueMaxSize = settings.RegisterIntSetting( + settings.ApplicationLevel, + "kv.replicate_queue.max_size", + "maximum number of replicas that can be queued for replicate queue processing; "+ + "when this limit is exceeded, lower priority (not guaranteed to be the lowest) "+ + "replicas are dropped from the queue", + defaultQueueMaxSize, + settings.WithValidateInt(func(v int64) error { + if v < defaultQueueMaxSize { + return errors.Errorf("cannot be set to a value lower than %d: %d", defaultQueueMaxSize, v) + } + return nil + }), +) + var ( metaReplicateQueueAddReplicaCount = metric.Metadata{ Name: "queue.replicate.addreplica", @@ -521,6 +539,7 @@ func (metrics *ReplicateQueueMetrics) trackResultByAllocatorAction( // additional replica to their range. type replicateQueue struct { *baseQueue + maxSize *settings.IntSetting metrics ReplicateQueueMetrics allocator allocatorimpl.Allocator storePool storepool.AllocatorStorePool @@ -545,6 +564,7 @@ func newReplicateQueue(store *Store, allocator allocatorimpl.Allocator) *replica storePool = store.cfg.StorePool } rq := &replicateQueue{ + maxSize: ReplicateQueueMaxSize, metrics: makeReplicateQueueMetrics(), planner: plan.NewReplicaPlanner(allocator, storePool, store.TestingKnobs().ReplicaPlannerKnobs), @@ -579,6 +599,10 @@ func newReplicateQueue(store *Store, allocator allocatorimpl.Allocator) *replica disabledConfig: kvserverbase.ReplicateQueueEnabled, }, ) + rq.baseQueue.SetMaxSize(ReplicateQueueMaxSize.Get(&store.cfg.Settings.SV)) + ReplicateQueueMaxSize.SetOnChange(&store.cfg.Settings.SV, func(ctx context.Context) { + rq.baseQueue.SetMaxSize(ReplicateQueueMaxSize.Get(&store.cfg.Settings.SV)) + }) updateFn := func() { select { case rq.updateCh <- timeutil.Now(): From 7d93bffcdb2d229923bae8007bf29cbd26f0b54a Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Thu, 28 Aug 2025 21:43:02 -0400 Subject: [PATCH 15/44] kvserver: add TestReplicateQueueMaxSize This commit adds tests to (1) verify metric updates when replicas are dropped from the queue, and (2) ensure the cluster setting for ReplicateQueueMaxSize works correctly. --- pkg/kv/kvserver/store_rebalancer_test.go | 49 ++++++++++++++++++++++++ 1 file changed, 49 insertions(+) diff --git a/pkg/kv/kvserver/store_rebalancer_test.go b/pkg/kv/kvserver/store_rebalancer_test.go index 929b13290e17..b7144d89e301 100644 --- a/pkg/kv/kvserver/store_rebalancer_test.go +++ b/pkg/kv/kvserver/store_rebalancer_test.go @@ -1860,3 +1860,52 @@ func TestingRaftStatusFn(desc *roachpb.RangeDescriptor, storeID roachpb.StoreID) } return status } + +// TestReplicateQueueMaxSize tests the max size of the replicate queue and +// verifies that replicas are dropped when the max size is exceeded. It also +// checks that the metric ReplicateQueueDroppedDueToSize is updated correctly. +func TestReplicateQueueMaxSize(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + tc := testContext{} + stopper := stop.NewStopper() + ctx := context.Background() + defer stopper.Stop(ctx) + tc.Start(ctx, t, stopper) + + r, err := tc.store.GetReplica(1) + require.NoError(t, err) + + stopper, _, _, a, _ := allocatorimpl.CreateTestAllocator(ctx, 10, true /* deterministic */) + defer stopper.Stop(ctx) + ReplicateQueueMaxSize.Override(ctx, &tc.store.cfg.Settings.SV, 1) + replicateQueue := newReplicateQueue(tc.store, a) + + // Function to add a replica and verify queue state + addReplicaAndVerify := func(rangeID roachpb.RangeID, expectedLength int, expectedDropped int64) { + r.Desc().RangeID = rangeID + enqueued, err := replicateQueue.testingAdd(context.Background(), r, 0.0) + require.NoError(t, err) + require.True(t, enqueued) + require.Equal(t, expectedLength, replicateQueue.Length()) + require.Equal(t, expectedDropped, replicateQueue.droppedDueToSize.Count()) + require.Equal(t, expectedDropped, tc.store.metrics.ReplicateQueueDroppedDueToSize.Count()) + } + + // First replica should be added. + addReplicaAndVerify(1 /* rangeID */, 1 /* expectedLength */, 0 /* expectedDropped */) + // Second replica should be dropped. + addReplicaAndVerify(2 /* rangeID */, 1 /* expectedLength */, 1 /* expectedDropped */) + // Third replica should be dropped. + addReplicaAndVerify(3 /* rangeID */, 1 /* expectedLength */, 2 /* expectedDropped */) + + // Increase the max size to 100 and add more replicas + ReplicateQueueMaxSize.Override(ctx, &tc.store.cfg.Settings.SV, 100) + for i := 2; i <= 100; i++ { + // Should be added. + addReplicaAndVerify(roachpb.RangeID(i+1 /* rangeID */), i /* expectedLength */, 2 /* expectedDropped */) + } + + // Add one more to exceed the max size. Should be dropped. + addReplicaAndVerify(102 /* rangeID */, 100 /* expectedLength */, 3 /* expectedDropped */) +} From 5810818afb786c708a2204ece5cf666f957bf0d6 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Fri, 29 Aug 2025 16:48:58 -0400 Subject: [PATCH 16/44] kvserver: drop excess replicas when lowering ReplicateQueueMaxSize MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Previously, the ReplicateQueueMaxSize cluster setting allowed dynamic adjustment of the replicate queue’s maximum size. However, decreasing this setting did not properly drop excess replicas. This commit fixes that by removing replicas when the queue’s max size is lowered. --- pkg/kv/kvserver/queue.go | 12 +++++++++++- pkg/kv/kvserver/store_rebalancer_test.go | 23 +++++++++++++++++++---- 2 files changed, 30 insertions(+), 5 deletions(-) diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index 3c339f76b294..95cc05d0fdcf 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -548,8 +548,18 @@ func (bq *baseQueue) SetDisabled(disabled bool) { // SetMaxSize sets the max size of the queue. func (bq *baseQueue) SetMaxSize(maxSize int64) { bq.mu.Lock() + defer bq.mu.Unlock() bq.mu.maxSize = maxSize - bq.mu.Unlock() + // Drop replicas until no longer exceeding the max size. Note: We call + // removeLocked to match the behavior of addInternal. In theory, only + // removeFromQueueLocked should be triggered in removeLocked, since the item + // is in the priority queue, it should not be processing or in the purgatory + // queue. To be safe, however, we use removeLocked. + for int64(bq.mu.priorityQ.Len()) > maxSize { + pqLen := bq.mu.priorityQ.Len() + bq.droppedDueToSize.Inc(1) + bq.removeLocked(bq.mu.priorityQ.sl[pqLen-1]) + } } // lockProcessing locks all processing in the baseQueue. It returns diff --git a/pkg/kv/kvserver/store_rebalancer_test.go b/pkg/kv/kvserver/store_rebalancer_test.go index b7144d89e301..f00d41592617 100644 --- a/pkg/kv/kvserver/store_rebalancer_test.go +++ b/pkg/kv/kvserver/store_rebalancer_test.go @@ -1881,15 +1881,19 @@ func TestReplicateQueueMaxSize(t *testing.T) { ReplicateQueueMaxSize.Override(ctx, &tc.store.cfg.Settings.SV, 1) replicateQueue := newReplicateQueue(tc.store, a) - // Function to add a replica and verify queue state + // Helper function to add a replica and verify queue state. + verify := func(expectedLength int, expectedDropped int64) { + require.Equal(t, expectedLength, replicateQueue.Length()) + require.Equal(t, expectedDropped, replicateQueue.droppedDueToSize.Count()) + require.Equal(t, expectedDropped, tc.store.metrics.ReplicateQueueDroppedDueToSize.Count()) + } + addReplicaAndVerify := func(rangeID roachpb.RangeID, expectedLength int, expectedDropped int64) { r.Desc().RangeID = rangeID enqueued, err := replicateQueue.testingAdd(context.Background(), r, 0.0) require.NoError(t, err) require.True(t, enqueued) - require.Equal(t, expectedLength, replicateQueue.Length()) - require.Equal(t, expectedDropped, replicateQueue.droppedDueToSize.Count()) - require.Equal(t, expectedDropped, tc.store.metrics.ReplicateQueueDroppedDueToSize.Count()) + verify(expectedLength, expectedDropped) } // First replica should be added. @@ -1908,4 +1912,15 @@ func TestReplicateQueueMaxSize(t *testing.T) { // Add one more to exceed the max size. Should be dropped. addReplicaAndVerify(102 /* rangeID */, 100 /* expectedLength */, 3 /* expectedDropped */) + + // Reset to the same size should not change the queue length. + ReplicateQueueMaxSize.Override(ctx, &tc.store.cfg.Settings.SV, 100) + verify(100 /* expectedLength */, 3 /* expectedDropped */) + + // Decrease the max size to 10 which should drop 90 replicas. + ReplicateQueueMaxSize.Override(ctx, &tc.store.cfg.Settings.SV, 10) + verify(10 /* expectedLength */, 93 /* expectedDropped: 3 + 90 */) + + // Should drop another one now that max size is 10. + addReplicaAndVerify(103 /* rangeID */, 10 /* expectedLength */, 94 /* expectedDropped: 3 + 90 + 1 */) } From 17baf84cad4f4f23ad829307065d6ba1567420f4 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Fri, 29 Aug 2025 17:10:10 -0400 Subject: [PATCH 17/44] kvserver: rename ReplicateQueueDroppedDueToSize to ReplicateQueueFull This commit improves the clarity around the naming and description of the metrics. --- docs/generated/metrics/metrics.html | 2 +- pkg/kv/kvserver/metrics.go | 10 +++++----- pkg/kv/kvserver/queue.go | 12 ++++++------ pkg/kv/kvserver/replicate_queue.go | 2 +- pkg/kv/kvserver/store_rebalancer_test.go | 4 ++-- 5 files changed, 15 insertions(+), 15 deletions(-) diff --git a/docs/generated/metrics/metrics.html b/docs/generated/metrics/metrics.html index 7c4933b0c5cc..d15b63699e36 100644 --- a/docs/generated/metrics/metrics.html +++ b/docs/generated/metrics/metrics.html @@ -420,13 +420,13 @@ STORAGEqueue.replicate.addreplica.errorNumber of failed replica additions processed by the replicate queueReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.addreplica.successNumber of successful replica additions processed by the replicate queueReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.addvoterreplicaNumber of voter replica additions attempted by the replicate queueReplica AdditionsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE -STORAGEqueue.replicate.dropped_due_to_sizeNumber of replicas dropped due to the replicate queue exceeding its max sizeReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.nonvoterpromotionsNumber of non-voters promoted to voters by the replicate queuePromotions of Non Voters to VotersCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.pendingNumber of pending replicas in the replicate queueReplicasGAUGECOUNTAVGNONE STORAGEqueue.replicate.process.failureNumber of replicas which failed processing in the replicate queueReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.process.successNumber of replicas successfully processed by the replicate queueReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.processingnanosNanoseconds spent processing replicas in the replicate queueProcessing TimeCOUNTERNANOSECONDSAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.purgatoryNumber of replicas in the replicate queue's purgatory, awaiting allocation optionsReplicasGAUGECOUNTAVGNONE +STORAGEqueue.replicate.queue_fullNumber of times a replica was dropped from the queue due to queue fullnessReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.rebalancenonvoterreplicaNumber of non-voter replica rebalancer-initiated additions attempted by the replicate queueReplica AdditionsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.rebalancereplicaNumber of replica rebalancer-initiated additions attempted by the replicate queueReplica AdditionsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.rebalancevoterreplicaNumber of voter replica rebalancer-initiated additions attempted by the replicate queueReplica AdditionsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE diff --git a/pkg/kv/kvserver/metrics.go b/pkg/kv/kvserver/metrics.go index 8818f8712734..ad9702e47b9b 100644 --- a/pkg/kv/kvserver/metrics.go +++ b/pkg/kv/kvserver/metrics.go @@ -2052,9 +2052,9 @@ The messages are dropped to help these replicas to recover from I/O overload.`, Measurement: "Replicas", Unit: metric.Unit_COUNT, } - metaReplicateQueueDroppedDueToSize = metric.Metadata{ - Name: "queue.replicate.dropped_due_to_size", - Help: "Number of replicas dropped due to the replicate queue exceeding its max size", + metaReplicateQueueFull = metric.Metadata{ + Name: "queue.replicate.queue_full", + Help: "Number of times a replica was dropped from the queue due to queue fullness", Measurement: "Replicas", Unit: metric.Unit_COUNT, } @@ -2961,7 +2961,7 @@ type StoreMetrics struct { ReplicateQueueSuccesses *metric.Counter ReplicateQueueFailures *metric.Counter ReplicateQueuePending *metric.Gauge - ReplicateQueueDroppedDueToSize *metric.Counter + ReplicateQueueFull *metric.Counter ReplicateQueueProcessingNanos *metric.Counter ReplicateQueuePurgatory *metric.Gauge SplitQueueSuccesses *metric.Counter @@ -3740,7 +3740,7 @@ func newStoreMetrics(histogramWindow time.Duration) *StoreMetrics { ReplicateQueueSuccesses: metric.NewCounter(metaReplicateQueueSuccesses), ReplicateQueueFailures: metric.NewCounter(metaReplicateQueueFailures), ReplicateQueuePending: metric.NewGauge(metaReplicateQueuePending), - ReplicateQueueDroppedDueToSize: metric.NewCounter(metaReplicateQueueDroppedDueToSize), + ReplicateQueueFull: metric.NewCounter(metaReplicateQueueFull), ReplicateQueueProcessingNanos: metric.NewCounter(metaReplicateQueueProcessingNanos), ReplicateQueuePurgatory: metric.NewGauge(metaReplicateQueuePurgatory), SplitQueueSuccesses: metric.NewCounter(metaSplitQueueSuccesses), diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index 95cc05d0fdcf..563df72ab10c 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -332,11 +332,11 @@ type queueConfig struct { failures *metric.Counter // pending is a gauge measuring current replica count pending. pending *metric.Gauge - // droppedDueToSize is a counter measuring replicas dropped due to - // exceeding the queue max size. + // full is a counter measuring replicas dropped due to exceeding the queue max + // size. // NB: this metric may be nil for queues that are not interested in tracking // this. - droppedDueToSize *metric.Counter + full *metric.Counter // processingNanos is a counter measuring total nanoseconds spent processing // replicas. processingNanos *metric.Counter @@ -557,7 +557,7 @@ func (bq *baseQueue) SetMaxSize(maxSize int64) { // queue. To be safe, however, we use removeLocked. for int64(bq.mu.priorityQ.Len()) > maxSize { pqLen := bq.mu.priorityQ.Len() - bq.droppedDueToSize.Inc(1) + bq.full.Inc(1) bq.removeLocked(bq.mu.priorityQ.sl[pqLen-1]) } } @@ -795,8 +795,8 @@ func (bq *baseQueue) addInternal( // scan. if pqLen := bq.mu.priorityQ.Len(); int64(pqLen) > bq.mu.maxSize { replicaItemToDrop := bq.mu.priorityQ.sl[pqLen-1] - if bq.droppedDueToSize != nil { - bq.droppedDueToSize.Inc(1) + if bq.full != nil { + bq.full.Inc(1) } log.Dev.VInfof(ctx, 1, "dropping due to exceeding queue max size: priority=%0.3f, replica=%v", priority, replicaItemToDrop.replicaID) diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index a8d659efa44e..a971b75a9380 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -593,7 +593,7 @@ func newReplicateQueue(store *Store, allocator allocatorimpl.Allocator) *replica successes: store.metrics.ReplicateQueueSuccesses, failures: store.metrics.ReplicateQueueFailures, pending: store.metrics.ReplicateQueuePending, - droppedDueToSize: store.metrics.ReplicateQueueDroppedDueToSize, + full: store.metrics.ReplicateQueueFull, processingNanos: store.metrics.ReplicateQueueProcessingNanos, purgatory: store.metrics.ReplicateQueuePurgatory, disabledConfig: kvserverbase.ReplicateQueueEnabled, diff --git a/pkg/kv/kvserver/store_rebalancer_test.go b/pkg/kv/kvserver/store_rebalancer_test.go index f00d41592617..7fb39eed30b2 100644 --- a/pkg/kv/kvserver/store_rebalancer_test.go +++ b/pkg/kv/kvserver/store_rebalancer_test.go @@ -1884,8 +1884,8 @@ func TestReplicateQueueMaxSize(t *testing.T) { // Helper function to add a replica and verify queue state. verify := func(expectedLength int, expectedDropped int64) { require.Equal(t, expectedLength, replicateQueue.Length()) - require.Equal(t, expectedDropped, replicateQueue.droppedDueToSize.Count()) - require.Equal(t, expectedDropped, tc.store.metrics.ReplicateQueueDroppedDueToSize.Count()) + require.Equal(t, expectedDropped, replicateQueue.full.Count()) + require.Equal(t, expectedDropped, tc.store.metrics.ReplicateQueueFull.Count()) } addReplicaAndVerify := func(rangeID roachpb.RangeID, expectedLength int, expectedDropped int64) { From 6a2ce5ba1074de5164c5089de42e42df6304ed65 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Tue, 26 Aug 2025 22:59:42 -0400 Subject: [PATCH 18/44] kvserver: add PriorityInversionRequeue This commit adds a new cluster setting PriorityInversionRequeue that controls whether the replicate queue should requeue replicas when their priority at enqueue time differs significantly from their priority at processing time (e.g. dropping from top 3 to the lowest priority). --- pkg/kv/kvserver/replicate_queue.go | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index a971b75a9380..ef0e83b8c24c 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -100,6 +100,19 @@ var EnqueueProblemRangeInReplicateQueueInterval = settings.RegisterDurationSetti settings.NonNegativeDuration, ) +// TODO(wenyihu6): move these cluster settings to kvserverbase + +// PriorityInversionRequeue is a setting that controls whether to requeue +// replicas when their priority at enqueue time and processing time is inverted +// too much (e.g. dropping from a repair action to AllocatorConsiderRebalance). +var PriorityInversionRequeue = settings.RegisterBoolSetting( + settings.SystemOnly, + "kv.priority_inversion_requeue_replicate_queue.enabled", + "whether the requeue replicas should requeue when enqueued for "+ + "repair action but ended up consider rebalancing during processing", + false, +) + // ReplicateQueueMaxSize is a setting that controls the max size of the // replicate queue. When this limit is exceeded, lower priority replicas (not // guaranteed to be the lowest) are dropped from the queue. From f2f1bec0eb5807db5a76bbf32bf0c7db8d4198c7 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Tue, 26 Aug 2025 23:11:05 -0400 Subject: [PATCH 19/44] kvserver: requeue on priority inversion for replicate queue Previously, a replica could enter the queue with high priority but, by the time it was processed, the action planned for this replica may have a low priority, causing us to perform low priority work. Specifically, we are mostly worried about cases where the priority changes from any of the repair actions to consider rebalance. Rebalancing could take a long time and block other ranges enqueued with actual repair action needed. This commit ensures that such replicas are requeued instead, avoiding priority inversions. --- .../allocator/allocatorimpl/allocator.go | 59 +++++++ .../allocator/allocatorimpl/allocator_test.go | 145 ++++++++++++++++++ pkg/kv/kvserver/replicate_queue.go | 20 +++ 3 files changed, 224 insertions(+) diff --git a/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go b/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go index d220ea907d0d..f86383dcef03 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go +++ b/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go @@ -3245,3 +3245,62 @@ func replDescsToStoreIDs(descs []roachpb.ReplicaDescriptor) []roachpb.StoreID { } return ret } + +// roundToNearestPriorityCategory rounds a priority to the nearest 100. n should be non-negative. +func roundToNearestPriorityCategory(n float64) float64 { + return math.Round(n/100.0) * 100 +} + +// WithinPriorityRange checks if a priority is within the range of possible +// priorities for the allocator actions. +func withinPriorityRange(priority float64) bool { + return AllocatorNoop.Priority() <= priority && priority <= AllocatorFinalizeAtomicReplicationChange.Priority() +} + +// CheckPriorityInversion checks if the priority at enqueue time is higher than +// the priority corresponding to the action computed at processing time. It +// returns whether there was a priority inversion and whether the caller should +// skip the processing of the range since the inversion is considered unfair. +// Currently, we only consider the inversion as unfair if it has gone from a +// repair action to lowest priority (AllocatorConsiderRebalance). We let +// AllocatorRangeUnavailable, AllocatorNoop pass through since they are noop. +// +// NB: If shouldRequeue is true, isInversion must be true. +func CheckPriorityInversion( + priorityAtEnqueue float64, actionAtProcessing AllocatorAction, +) (isInversion bool, shouldRequeue bool) { + // priorityAtEnqueue of -1 is a special case reserved for processing logic to + // run even if there’s a priority inversion. If the priority is not -1, the + // range may be re-queued to be processed with the correct priority. It is + // used for things that call into baseQueue.process without going through the + // replicate priority queue. For example, s.ReplicateQueueDryRun or + // r.scatterRangeAndRandomizeLeases. + + // NB: we need to check for when priorityAtEnqueue falls within the range + // of the allocator actions because store.Enqueue might enqueue things with + // a very high priority (1e5). In those cases, we do not want to requeue + // these actions or count it as an inversion. + if priorityAtEnqueue == -1 || !withinPriorityRange(priorityAtEnqueue) { + return false, false + } + + if priorityAtEnqueue > AllocatorConsiderRebalance.Priority() && actionAtProcessing == AllocatorConsiderRebalance { + return true, true + } + + // NB: Usually, the priority at enqueue time should correspond to + // action.Priority(). However, for AllocatorAddVoter, + // AllocatorRemoveDeadVoter, AllocatorRemoveVoter, the priority can be + // adjusted at enqueue time (See ComputeAction for more details). However, we + // expect the adjustment to be relatively small (<100). So we round the + // priority to the nearest 100 to compare against + // actionAtProcessing.Priority(). Without this rounding, we might treat going + // from 10000 to 999 as an inversion, but it was just due to the adjustment. + // Note that priorities at AllocatorFinalizeAtomicReplicationChange, + // AllocatorRemoveLearner, and AllocatorReplaceDeadVoter will be rounded to + // the same priority. They are so close to each other, so we don't really + // count it as an inversion among them. + normPriorityAtEnqueue := roundToNearestPriorityCategory(priorityAtEnqueue) + isInversion = normPriorityAtEnqueue > actionAtProcessing.Priority() + return isInversion, false +} diff --git a/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go b/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go index 6caec22f7f69..ac758ac91f8b 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go +++ b/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go @@ -9629,3 +9629,148 @@ func TestAllocatorRebalanceTargetVoterConstraintUnsatisfied(t *testing.T) { }) } } + +// TestRoundToNearestPriorityCategory tests the RoundToNearestPriorityCategory +// function. +func TestRoundToNearestPriorityCategory(t *testing.T) { + defer leaktest.AfterTest(t)() + + testCases := []struct { + name string + input float64 + expected float64 + }{ + { + name: "zero", + input: 0.0, + expected: 0.0, + }, + { + name: "exact multiple of 100", + input: 100.0, + expected: 100.0, + }, + { + name: "round down to nearest 100", + input: 149.0, + expected: 100.0, + }, + { + name: "round up to nearest 100", + input: 151.0, + expected: 200.0, + }, + { + name: "negative exact multiple of 100", + input: -200.0, + expected: -200.0, + }, + { + name: "negative round down to nearest 100", + input: -249.0, + expected: -200.0, + }, + { + name: "negative round up to nearest 100", + input: -251.0, + expected: -300.0, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + require.Equal(t, tc.expected, roundToNearestPriorityCategory(tc.input)) + }) + } +} + +// TestCheckPriorityInversion tests the CheckPriorityInversion function. +func TestCheckPriorityInversion(t *testing.T) { + defer leaktest.AfterTest(t)() + + for action := AllocatorNoop; action <= AllocatorFinalizeAtomicReplicationChange; action++ { + t.Run(action.String(), func(t *testing.T) { + if action == AllocatorConsiderRebalance || action == AllocatorNoop || action == AllocatorRangeUnavailable { + inversion, requeue := CheckPriorityInversion(action.Priority(), AllocatorConsiderRebalance) + require.False(t, inversion) + require.False(t, requeue) + } else { + inversion, requeue := CheckPriorityInversion(action.Priority(), AllocatorConsiderRebalance) + require.True(t, inversion) + require.True(t, requeue) + } + }) + } + + testCases := []struct { + name string + priorityAtEnqueue float64 + actionAtProcessing AllocatorAction + expectedInversion bool + expectedRequeue bool + }{ + { + name: "AllocatorNoop at processing is noop", + priorityAtEnqueue: AllocatorFinalizeAtomicReplicationChange.Priority(), + actionAtProcessing: AllocatorNoop, + expectedInversion: true, + expectedRequeue: false, + }, + { + name: "AllocatorRangeUnavailable at processing is noop", + priorityAtEnqueue: AllocatorFinalizeAtomicReplicationChange.Priority(), + actionAtProcessing: AllocatorRangeUnavailable, + expectedInversion: true, + expectedRequeue: false, + }, + { + name: "priority -1 bypasses", + priorityAtEnqueue: -1, + actionAtProcessing: AllocatorConsiderRebalance, + expectedInversion: false, + expectedRequeue: false, + }, + { + name: "above range priority(1e5)", + priorityAtEnqueue: 1e5, + actionAtProcessing: AllocatorConsiderRebalance, + expectedInversion: false, + expectedRequeue: false, + }, + { + name: "below range priority at -10", + priorityAtEnqueue: -10, + actionAtProcessing: -100, + expectedInversion: false, + expectedRequeue: false, + }, + { + name: "inversion but small priority changes", + priorityAtEnqueue: AllocatorFinalizeAtomicReplicationChange.Priority(), + actionAtProcessing: AllocatorReplaceDecommissioningNonVoter, + expectedInversion: true, + expectedRequeue: false, + }, + { + name: "inversion but small priority changes", + priorityAtEnqueue: AllocatorRemoveDeadVoter.Priority(), + actionAtProcessing: AllocatorAddNonVoter, + expectedInversion: true, + expectedRequeue: false, + }, + { + name: "inversion but small priority changes", + priorityAtEnqueue: AllocatorConsiderRebalance.Priority(), + actionAtProcessing: AllocatorNoop, + expectedInversion: false, + expectedRequeue: false, + }, + } + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + inversion, requeue := CheckPriorityInversion(tc.priorityAtEnqueue, tc.actionAtProcessing) + require.Equal(t, tc.expectedInversion, inversion) + require.Equal(t, tc.expectedRequeue, requeue) + }) + } +} diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index ef0e83b8c24c..aa649e7d8f4c 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -716,6 +716,10 @@ func (rq *replicateQueue) process( } if err != nil { + if requeue { + log.KvDistribution.VEventf(ctx, 1, "re-queuing on errors: %v", err) + rq.maybeAdd(ctx, repl, rq.store.Clock().NowAsClockTimestamp()) + } return false, err } @@ -909,6 +913,22 @@ func (rq *replicateQueue) processOneChange( ) (requeue bool, _ error) { change, err := rq.planner.PlanOneChange( ctx, repl, desc, conf, plan.PlannerOptions{Scatter: scatter}) + + inversion, shouldRequeue := allocatorimpl.CheckPriorityInversion(priorityAtEnqueue, change.Action) + if inversion { + log.KvDistribution.VInfof(ctx, 2, + "priority inversion during process: shouldRequeue = %t action=%s, priority=%v, enqueuePriority=%v", + shouldRequeue, change.Action, change.Action.Priority(), priorityAtEnqueue) + if shouldRequeue && PriorityInversionRequeue.Get(&rq.store.cfg.Settings.SV) { + // Return true here to requeue the range. We can't return an error here + // because rq.process only requeue when error is nil. See + // replicateQueue.process for more details. + return true /*requeue*/, maybeAnnotateDecommissionErr( + errors.Errorf("requing due to priority inversion: action=%s, priority=%v, enqueuePriority=%v", + change.Action, change.Action.Priority(), priorityAtEnqueue), change.Action) + } + } + // When there is an error planning a change, return the error immediately // and do not requeue. It is unlikely that the range or storepool state // will change quickly enough in order to not get the same error and From d02be73426446d5337ab1f0d81e5a9a69b966492 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Wed, 27 Aug 2025 22:23:44 -0400 Subject: [PATCH 20/44] kvserver: use priorityInversionLogEveryN MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Previously, replicateQueue used V(2) to log info on priority inverted replicas because I wanted visibility into every case without missing any replicas. On reflection, the individual cases aren’t that interesting - it’s the overall volume that matters, which we can track through metrics. This commit changes it so that we just rate limit priority inversions every 3 seconds. --- pkg/kv/kvserver/replicate_queue.go | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index aa649e7d8f4c..cb9a4ddfb6dc 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -903,6 +903,10 @@ func ShouldRequeue( return requeue } +// priorityInversionLogEveryN rate limits how often we log about priority +// inversion to avoid spams. +var priorityInversionLogEveryN = log.Every(3 * time.Second) + func (rq *replicateQueue) processOneChange( ctx context.Context, repl *Replica, @@ -916,9 +920,12 @@ func (rq *replicateQueue) processOneChange( inversion, shouldRequeue := allocatorimpl.CheckPriorityInversion(priorityAtEnqueue, change.Action) if inversion { - log.KvDistribution.VInfof(ctx, 2, - "priority inversion during process: shouldRequeue = %t action=%s, priority=%v, enqueuePriority=%v", - shouldRequeue, change.Action, change.Action.Priority(), priorityAtEnqueue) + if priorityInversionLogEveryN.ShouldLog() { + log.KvDistribution.Infof(ctx, + "priority inversion during process: shouldRequeue = %t action=%s, priority=%v, enqueuePriority=%v", + shouldRequeue, change.Action, change.Action.Priority(), priorityAtEnqueue) + } + if shouldRequeue && PriorityInversionRequeue.Get(&rq.store.cfg.Settings.SV) { // Return true here to requeue the range. We can't return an error here // because rq.process only requeue when error is nil. See From 0ac23868a31d4dfb6f86d3ad72fecd5ce5141152 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Fri, 29 Aug 2025 14:45:26 -0400 Subject: [PATCH 21/44] kvserver: improve comments for PriorityInversionRequeue This commit improves the comments for PriorityInversionRequeue and clarifies the contracts around action.Priority(). --- .../allocator/allocatorimpl/allocator.go | 84 ++++++++++++++++--- pkg/kv/kvserver/replicate_queue.go | 7 +- 2 files changed, 78 insertions(+), 13 deletions(-) diff --git a/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go b/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go index f86383dcef03..27c66b278cb4 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go +++ b/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go @@ -252,6 +252,20 @@ func (a AllocatorAction) SafeValue() {} // range. Within a given range, the ordering of the various checks inside // `Allocator.computeAction` determines which repair/rebalancing actions are // taken before the others. +// +// NB: Priorities should be non-negative and should be spaced in multiples of +// 100 unless you believe they should belong to the same priority category. +// AllocatorNoop should have the lowest priority. CheckPriorityInversion depends +// on this contract. In most cases, the allocator returns a priority that +// matches the definitions below. For AllocatorAddVoter, +// AllocatorRemoveDeadVoter, and AllocatorRemoveVoter, the priority may be +// adjusted (see ComputeAction for details), but the adjustment is expected to +// be small (<49). +// +// Exceptions: AllocatorFinalizeAtomicReplicationChange, AllocatorRemoveLearner, +// and AllocatorReplaceDeadVoter violates the spacing of 100. These cases +// predate this comment, so we allow them as they belong to the same general +// priority category. func (a AllocatorAction) Priority() float64 { switch a { case AllocatorFinalizeAtomicReplicationChange: @@ -975,6 +989,49 @@ func (a *Allocator) ComputeAction( return action, priority } +// computeAction determines the action to take on a range along with its +// priority. +// +// NB: The returned priority may include a small adjustment and therefore might +// not exactly match action.Priority(). See AllocatorAddVoter, +// AllocatorRemoveDeadVoter, AllocatorRemoveVoter below. The adjustment should +// be <49 with two assumptions below. New uses on this contract should be +// avoided since the assumptions are not strong guarantees (especially the +// second one). +// +// The claim that the adjustment is < 49 has two assumptions: +// 1. min(num_replicas,total_nodes) in zone configuration is < 98. +// 2. when ranges are not under-replicated, the difference between +// min(num_replicas,total_nodes)/2-1 and existing_replicas is < 49. +// +// neededVoters <= min(num_replicas,total_nodes) +// desiredQuorum = neededVoters/2-1 +// quorum = haveVoters/2-1 +// +// For AllocatorAddVoter, we know haveVoters < neededVoters +// adjustment = desiredQuorum-haveVoters = neededVoters/2-1-haveVoters +// To find the worst case (largest adjustment), +// 1. haveVoters = neededVoters-1, +// adjustment = neededVoters/2-1-(neededVoters-1) +// = neededVoters/2-neededVoters = -neededVoters/2 +// 2. haveVoters = 0 +// adjustement = neededVoters/2-1 +// +// In order for adjustment to be <49, neededVoters/2<49 => neededVoters<98. +// Hence the first assumption. +// +// For AllocatorRemoveDeadVoter, we know haveVoters >= neededVoters +// adjustment = desiredQuorum-haveVoters = neededVoters/2-1-haveVoters +// To find the worst case (largest adjustment), +// 1. neededVoters/2-1 is much larger than haveVoters: given haveVoters >= +// neededVoters, haveVoters/2-1 >= neededVoters/2-1. So this case is impossible. +// 2. neededVoters/2-1 is much smaller than haveVoters: since ranges could be +// over-replicated, theoretically speaking, there may be no upper bounds on +// haveVoters. In order for adjustment to be < 49, we can only make an +// assumption here that the difference between neededVoters/2-1 and haveVoters +// cannot be >= 49 in this case. +// +// For AllocatorRemoveVoter, adjustment is haveVoters%2 = 0 or 1 < 49. func (a *Allocator) computeAction( ctx context.Context, storePool storepool.AllocatorStorePool, @@ -3246,7 +3303,8 @@ func replDescsToStoreIDs(descs []roachpb.ReplicaDescriptor) []roachpb.StoreID { return ret } -// roundToNearestPriorityCategory rounds a priority to the nearest 100. n should be non-negative. +// roundToNearestPriorityCategory rounds a priority to the nearest 100. n should +// be non-negative. func roundToNearestPriorityCategory(n float64) float64 { return math.Round(n/100.0) * 100 } @@ -3257,15 +3315,21 @@ func withinPriorityRange(priority float64) bool { return AllocatorNoop.Priority() <= priority && priority <= AllocatorFinalizeAtomicReplicationChange.Priority() } -// CheckPriorityInversion checks if the priority at enqueue time is higher than -// the priority corresponding to the action computed at processing time. It -// returns whether there was a priority inversion and whether the caller should -// skip the processing of the range since the inversion is considered unfair. -// Currently, we only consider the inversion as unfair if it has gone from a -// repair action to lowest priority (AllocatorConsiderRebalance). We let -// AllocatorRangeUnavailable, AllocatorNoop pass through since they are noop. +// CheckPriorityInversion returns whether there was a priority inversion (and +// the range should not be processed at this time, since doing so could starve +// higher-priority items), and whether the caller should re-add the range to the +// queue (presumably under its new priority). A priority inversion happens if +// the priority at enqueue time is higher than the priority corresponding to the +// action computed at processing time. Caller should re-add the range to the +// queue if it has gone from a repair action to lowest priority +// (AllocatorConsiderRebalance). +// +// Note: Changing from AllocatorRangeUnavailable/AllocatorNoop to +// AllocatorConsiderRebalance is not treated as a priority inversion. Going from +// a repair action to AllocatorRangeUnavailable/AllocatorNoop is considered a +// priority inversion but shouldRequeue = false. // -// NB: If shouldRequeue is true, isInversion must be true. +// INVARIANT: shouldRequeue => isInversion func CheckPriorityInversion( priorityAtEnqueue float64, actionAtProcessing AllocatorAction, ) (isInversion bool, shouldRequeue bool) { @@ -3292,7 +3356,7 @@ func CheckPriorityInversion( // action.Priority(). However, for AllocatorAddVoter, // AllocatorRemoveDeadVoter, AllocatorRemoveVoter, the priority can be // adjusted at enqueue time (See ComputeAction for more details). However, we - // expect the adjustment to be relatively small (<100). So we round the + // expect the adjustment to be relatively small (<49). So we round the // priority to the nearest 100 to compare against // actionAtProcessing.Priority(). Without this rounding, we might treat going // from 10000 to 999 as an inversion, but it was just due to the adjustment. diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index cb9a4ddfb6dc..a7f683e19eee 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -105,6 +105,7 @@ var EnqueueProblemRangeInReplicateQueueInterval = settings.RegisterDurationSetti // PriorityInversionRequeue is a setting that controls whether to requeue // replicas when their priority at enqueue time and processing time is inverted // too much (e.g. dropping from a repair action to AllocatorConsiderRebalance). +// TODO(wenyihu6): flip default to true after landing 152596 to bake var PriorityInversionRequeue = settings.RegisterBoolSetting( settings.SystemOnly, "kv.priority_inversion_requeue_replicate_queue.enabled", @@ -927,9 +928,9 @@ func (rq *replicateQueue) processOneChange( } if shouldRequeue && PriorityInversionRequeue.Get(&rq.store.cfg.Settings.SV) { - // Return true here to requeue the range. We can't return an error here - // because rq.process only requeue when error is nil. See - // replicateQueue.process for more details. + // Return true to requeue the range. Return the error to ensure it is + // logged and tracked in replicate queue bq.failures metrics. See + // replicateQueue.process for details. return true /*requeue*/, maybeAnnotateDecommissionErr( errors.Errorf("requing due to priority inversion: action=%s, priority=%v, enqueuePriority=%v", change.Action, change.Action.Priority(), priorityAtEnqueue), change.Action) From b6bdd21bed34c9f22738b47a8a2a38211b3ed191 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Fri, 29 Aug 2025 14:45:45 -0400 Subject: [PATCH 22/44] allocator: small refactor for CheckPriorityInversion This commit refactors CheckPriorityInversion. --- .../allocator/allocatorimpl/allocator.go | 28 +++++++++---------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go b/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go index 27c66b278cb4..ab2857e0f14c 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go +++ b/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go @@ -139,6 +139,7 @@ const ( AllocatorConsiderRebalance AllocatorRangeUnavailable AllocatorFinalizeAtomicReplicationChange + AllocatorMaxPriority ) // Add indicates an action adding a replica. @@ -267,9 +268,12 @@ func (a AllocatorAction) SafeValue() {} // predate this comment, so we allow them as they belong to the same general // priority category. func (a AllocatorAction) Priority() float64 { + const maxPriority = 12002 switch a { + case AllocatorMaxPriority: + return maxPriority case AllocatorFinalizeAtomicReplicationChange: - return 12002 + return maxPriority case AllocatorRemoveLearner: return 12001 case AllocatorReplaceDeadVoter: @@ -3309,12 +3313,6 @@ func roundToNearestPriorityCategory(n float64) float64 { return math.Round(n/100.0) * 100 } -// WithinPriorityRange checks if a priority is within the range of possible -// priorities for the allocator actions. -func withinPriorityRange(priority float64) bool { - return AllocatorNoop.Priority() <= priority && priority <= AllocatorFinalizeAtomicReplicationChange.Priority() -} - // CheckPriorityInversion returns whether there was a priority inversion (and // the range should not be processed at this time, since doing so could starve // higher-priority items), and whether the caller should re-add the range to the @@ -3333,18 +3331,20 @@ func withinPriorityRange(priority float64) bool { func CheckPriorityInversion( priorityAtEnqueue float64, actionAtProcessing AllocatorAction, ) (isInversion bool, shouldRequeue bool) { - // priorityAtEnqueue of -1 is a special case reserved for processing logic to - // run even if there’s a priority inversion. If the priority is not -1, the - // range may be re-queued to be processed with the correct priority. It is - // used for things that call into baseQueue.process without going through the - // replicate priority queue. For example, s.ReplicateQueueDryRun or - // r.scatterRangeAndRandomizeLeases. + // NB: priorityAtEnqueue is -1 for callers such as scatter, dry runs, and + // manual queue runs. Priority inversion does not apply to these calls. + if priorityAtEnqueue == -1 { + return false, false + } // NB: we need to check for when priorityAtEnqueue falls within the range // of the allocator actions because store.Enqueue might enqueue things with // a very high priority (1e5). In those cases, we do not want to requeue // these actions or count it as an inversion. - if priorityAtEnqueue == -1 || !withinPriorityRange(priorityAtEnqueue) { + withinPriorityRange := func(priority float64) bool { + return AllocatorNoop.Priority() <= priority && priority <= AllocatorMaxPriority.Priority() + } + if !withinPriorityRange(priorityAtEnqueue) { return false, false } From d39cd7ea907f14f34301f75ac54f332b118037f7 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Fri, 29 Aug 2025 14:46:19 -0400 Subject: [PATCH 23/44] allocator: add TestAllocatorPriorityInvariance This commit adds the TestAllocatorPriorityInvariance test, which acts as a regression safeguard when new actions are added to AllocatorAction, ensuring the contract is upheld. See action.Priority() and ComputeAction() for more details on the contract. --- .../allocator/allocatorimpl/allocator_test.go | 33 +++++++++++++++++++ 1 file changed, 33 insertions(+) diff --git a/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go b/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go index ac758ac91f8b..7c0694d4b184 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go +++ b/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go @@ -9730,6 +9730,13 @@ func TestCheckPriorityInversion(t *testing.T) { expectedInversion: false, expectedRequeue: false, }, + { + name: "priority increase", + priorityAtEnqueue: 0, + actionAtProcessing: AllocatorFinalizeAtomicReplicationChange, + expectedInversion: false, + expectedRequeue: false, + }, { name: "above range priority(1e5)", priorityAtEnqueue: 1e5, @@ -9774,3 +9781,29 @@ func TestCheckPriorityInversion(t *testing.T) { }) } } + +// TestAllocatorPriorityInvariance verifies that allocator priorities remain +// spaced in multiples of 100. This prevents regressions against the contract +// relied on by CheckPriorityInversion. For details, see the comment above +// action.Priority(). +func TestAllocatorPriorityInvariance(t *testing.T) { + defer leaktest.AfterTest(t)() + + exceptions := map[AllocatorAction]struct{}{ + AllocatorFinalizeAtomicReplicationChange: {}, + AllocatorRemoveLearner: {}, + AllocatorReplaceDeadVoter: {}, + } + lowestPriority := AllocatorNoop.Priority() + for action := AllocatorNoop; action < AllocatorMaxPriority; action++ { + require.GreaterOrEqualf(t, action.Priority(), lowestPriority, + "priority %f is less than AllocatorNoop: likely violating contract", + action.Priority()) + if _, ok := exceptions[action]; !ok { + require.Equalf(t, int(action.Priority())%100, 0, + "priority %f is not a multiple of 100: likely violating contract", + action.Priority()) + + } + } +} From 50a330a759bae31c7ed44b50bbf5db13501ab890 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Fri, 29 Aug 2025 14:51:21 -0400 Subject: [PATCH 24/44] kvserver: guard inversion check and requeue behind PriorityInversionRequeue Previously, we introduced the PriorityInversionRequeue cluster setting, intended for backport, to handle cases where a range was enqueued with a high-priority repair action but, at processing time, a low-priority rebalance action was computed. In such cases, the caller re-adds the range to the queue under its updated priority. Although the cluster setting guards this requeue behavior, the inversion check always ran unconditionally, reducing backport safety. This commit updates the logic so that the cluster setting guard both the inversion check and the requeue behavior. --- pkg/kv/kvserver/replicate_queue.go | 30 +++++++++++++++--------------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index a7f683e19eee..3201ed7fd9ff 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -919,21 +919,21 @@ func (rq *replicateQueue) processOneChange( change, err := rq.planner.PlanOneChange( ctx, repl, desc, conf, plan.PlannerOptions{Scatter: scatter}) - inversion, shouldRequeue := allocatorimpl.CheckPriorityInversion(priorityAtEnqueue, change.Action) - if inversion { - if priorityInversionLogEveryN.ShouldLog() { - log.KvDistribution.Infof(ctx, - "priority inversion during process: shouldRequeue = %t action=%s, priority=%v, enqueuePriority=%v", - shouldRequeue, change.Action, change.Action.Priority(), priorityAtEnqueue) - } - - if shouldRequeue && PriorityInversionRequeue.Get(&rq.store.cfg.Settings.SV) { - // Return true to requeue the range. Return the error to ensure it is - // logged and tracked in replicate queue bq.failures metrics. See - // replicateQueue.process for details. - return true /*requeue*/, maybeAnnotateDecommissionErr( - errors.Errorf("requing due to priority inversion: action=%s, priority=%v, enqueuePriority=%v", - change.Action, change.Action.Priority(), priorityAtEnqueue), change.Action) + if PriorityInversionRequeue.Get(&rq.store.cfg.Settings.SV) { + if inversion, shouldRequeue := allocatorimpl.CheckPriorityInversion(priorityAtEnqueue, change.Action); inversion { + if priorityInversionLogEveryN.ShouldLog() { + log.KvDistribution.Infof(ctx, + "priority inversion during process: shouldRequeue = %t action=%s, priority=%v, enqueuePriority=%v", + shouldRequeue, change.Action, change.Action.Priority(), priorityAtEnqueue) + } + if shouldRequeue { + // Return true to requeue the range. Return the error to ensure it is + // logged and tracked in replicate queue bq.failures metrics. See + // replicateQueue.process for details. + return true /*requeue*/, maybeAnnotateDecommissionErr( + errors.Errorf("requing due to priority inversion: action=%s, priority=%v, enqueuePriority=%v", + change.Action, change.Action.Priority(), priorityAtEnqueue), change.Action) + } } } From f0ab31adc015e4bf87e6dc24f1ac2cbc2e0716e0 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Mon, 1 Sep 2025 13:46:22 -0400 Subject: [PATCH 25/44] kvserver: move priority inversion check before applyChange Previously, we checked for priority inversion before planning errors, which meant we could return requeue = true even when a planning error occurred. This commit changes it so that planning errors should take higher precedence over a priority inversion error. rq.processOneChange now returns early if there is a planning error and only check for priority inversion right before applying a change. --- pkg/kv/kvserver/replicate_queue.go | 40 ++++++++++++++++-------------- 1 file changed, 22 insertions(+), 18 deletions(-) diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index 3201ed7fd9ff..ca70ce1ff35a 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -919,24 +919,6 @@ func (rq *replicateQueue) processOneChange( change, err := rq.planner.PlanOneChange( ctx, repl, desc, conf, plan.PlannerOptions{Scatter: scatter}) - if PriorityInversionRequeue.Get(&rq.store.cfg.Settings.SV) { - if inversion, shouldRequeue := allocatorimpl.CheckPriorityInversion(priorityAtEnqueue, change.Action); inversion { - if priorityInversionLogEveryN.ShouldLog() { - log.KvDistribution.Infof(ctx, - "priority inversion during process: shouldRequeue = %t action=%s, priority=%v, enqueuePriority=%v", - shouldRequeue, change.Action, change.Action.Priority(), priorityAtEnqueue) - } - if shouldRequeue { - // Return true to requeue the range. Return the error to ensure it is - // logged and tracked in replicate queue bq.failures metrics. See - // replicateQueue.process for details. - return true /*requeue*/, maybeAnnotateDecommissionErr( - errors.Errorf("requing due to priority inversion: action=%s, priority=%v, enqueuePriority=%v", - change.Action, change.Action.Priority(), priorityAtEnqueue), change.Action) - } - } - } - // When there is an error planning a change, return the error immediately // and do not requeue. It is unlikely that the range or storepool state // will change quickly enough in order to not get the same error and @@ -961,6 +943,28 @@ func (rq *replicateQueue) processOneChange( return false, nil } + // At this point, planning returned no error, and we're not doing a dry run. + // Check for priority inversion if enabled. If detected, we may requeue the + // replica to return an error early to requeue the range instead to avoid + // starving other higher priority work. + if PriorityInversionRequeue.Get(&rq.store.cfg.Settings.SV) { + if inversion, shouldRequeue := allocatorimpl.CheckPriorityInversion(priorityAtEnqueue, change.Action); inversion { + if priorityInversionLogEveryN.ShouldLog() { + log.KvDistribution.Infof(ctx, + "priority inversion during process: shouldRequeue = %t action=%s, priority=%v, enqueuePriority=%v", + shouldRequeue, change.Action, change.Action.Priority(), priorityAtEnqueue) + } + if shouldRequeue { + // Return true to requeue the range. Return the error to ensure it is + // logged and tracked in replicate queue bq.failures metrics. See + // replicateQueue.process for details. + return true /*requeue*/, maybeAnnotateDecommissionErr( + errors.Errorf("requing due to priority inversion: action=%s, priority=%v, enqueuePriority=%v", + change.Action, change.Action.Priority(), priorityAtEnqueue), change.Action) + } + } + } + // Track the metrics generated during planning. These are not updated // directly during planning to avoid pushing the dryRun flag into every // function. From ae60761329c372362459cb93144b717bf990f174 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Tue, 2 Sep 2025 18:03:30 -0400 Subject: [PATCH 26/44] kvserver: check for requeue before error checking in rq.process Previously, we checked for requeue right before returning for both nil and non-nil errors, making the code harder to follow. This commit refactors replicateQueue.process to requeue replicas before checking for errors. --- pkg/kv/kvserver/replicate_queue.go | 18 +++++++----------- 1 file changed, 7 insertions(+), 11 deletions(-) diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index ca70ce1ff35a..84a976e9ee13 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -715,25 +715,21 @@ func (rq *replicateQueue) process( log.KvDistribution.Infof(ctx, "%v", err) continue } - + // At the time of writing, requeue => err == nil except for priority + // inversions. Priority inversion intentionally returns a priority inversion + // error along with requeue = true. + if requeue { + log.KvDistribution.VEventf(ctx, 1, "re-queuing: %v", err) + rq.maybeAdd(ctx, repl, rq.store.Clock().NowAsClockTimestamp()) + } if err != nil { - if requeue { - log.KvDistribution.VEventf(ctx, 1, "re-queuing on errors: %v", err) - rq.maybeAdd(ctx, repl, rq.store.Clock().NowAsClockTimestamp()) - } return false, err } - if testingAggressiveConsistencyChecks { if _, err := rq.store.consistencyQueue.process(ctx, repl, confReader, -1 /*priorityAtEnqueue*/); err != nil { log.KvDistribution.Warningf(ctx, "%v", err) } } - - if requeue { - log.KvDistribution.VEventf(ctx, 1, "re-queuing") - rq.maybeAdd(ctx, repl, rq.store.Clock().NowAsClockTimestamp()) - } return true, nil } From e7d5ae174e4dbfeaf5a92e936ed5c0818919fb46 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Sat, 30 Aug 2025 23:42:50 -0400 Subject: [PATCH 27/44] kvserver: use non-blocking send on errors for maybeBackpressureBatch maybeBackpressureBatch registers a callback with the split queue for replicas that are too large relative to their split size. This backpressures the range to stop it from growing and prevent new writes from blocking a pending split. The callback is invoked when the split queue finishes processing the replica. Previously, the error channel used in the callback had a size of 1 and performed blocking sends. This was safe because the base queue only sent a single error, and by the time maybeBackpressureBatch returned, the callback was guaranteed to have been consumed, and no additional sends would occur. Future commits will allow the callback to be invoked multiple times (although it should only twice at most). To be safe and avoid potential deadlocks from multiple sends after maybeBackpressureBatch already returns, this commit makes the error send non-blocking. If the channel is already full, the error is dropped, which is acceptable since we only care about observing the completion of the replica processing at least once. --- pkg/kv/kvserver/replica_backpressure.go | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/pkg/kv/kvserver/replica_backpressure.go b/pkg/kv/kvserver/replica_backpressure.go index 181781749a82..47668480346e 100644 --- a/pkg/kv/kvserver/replica_backpressure.go +++ b/pkg/kv/kvserver/replica_backpressure.go @@ -204,7 +204,14 @@ func (r *Replica) maybeBackpressureBatch(ctx context.Context, ba *kvpb.BatchRequ // Register a callback on an ongoing split for this range in the splitQueue. splitC := make(chan error, 1) if !r.store.splitQueue.MaybeAddCallback(r.RangeID, func(err error) { - splitC <- err + select { + case splitC <- err: + default: + // TODO(wenyihu6): should we add ctx timeout when invoking callbacks + // Drop the error if the channel is already full. This prevents + // blocking if the callback is invoked multiple times. + return + } }) { // No split ongoing. We may have raced with its completion. There's // no good way to prevent this race, so we conservatively allow the From af26a619388b55d3b938544fa9c9f75db2e9abfe Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Thu, 28 Aug 2025 23:31:43 -0400 Subject: [PATCH 28/44] kvserver: return baseQueueAsyncRateLimited from bq.Async baseQueue.Async may return immediately as a noop if the semaphore does not available capacity and the wait parameter is false. Previously, this case returned no error, leaving the caller unaware that the request was dropped. This commit changes the behavior to return a baseQueueAsyncRateLimited error, allowing callers to detect and handle the condition. --- pkg/kv/kvserver/queue.go | 25 +++++++++++++++++-------- pkg/kv/kvserver/replica.go | 10 +++++----- pkg/kv/kvserver/store_gossip.go | 4 ++-- 3 files changed, 24 insertions(+), 15 deletions(-) diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index 563df72ab10c..392ce4fecd26 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -608,18 +608,23 @@ type queueHelper interface { Add(ctx context.Context, repl replicaInQueue, prio float64) } +// baseQueueAsyncRateLimited indicates that the base queue async task was rate +// limited and the task was not executed. +var baseQueueAsyncRateLimited = errors.Newf("async task rate limited") + // Async is a more performant substitute for calling AddAsync or MaybeAddAsync // when many operations are going to be carried out. It invokes the given helper // function in a goroutine if semaphore capacity is available. If the semaphore -// is not available, the 'wait' parameter decides whether to wait or to return -// as a noop. Note that if the system is quiescing, fn may never be called in- -// dependent of the value of 'wait'. +// is at capacity, the 'wait' parameter determines whether to block until +// capacity becomes available or return immediately with an error. Note that if +// the system is shutting down, the function may not be executed regardless of +// the 'wait' value. // // The caller is responsible for ensuring that opName does not contain PII. // (Best is to pass a constant string.) func (bq *baseQueue) Async( ctx context.Context, opName string, wait bool, fn func(ctx context.Context, h queueHelper), -) { +) error { if log.V(3) { log.InfofDepth(ctx, 2, "%s", redact.Safe(opName)) } @@ -633,9 +638,13 @@ func (bq *baseQueue) Async( }, func(ctx context.Context) { fn(ctx, baseQueueHelper{bq}) - }); err != nil && bq.addLogN.ShouldLog() { - log.Infof(ctx, "rate limited in %s: %s", redact.Safe(opName), err) + }); err != nil { + if bq.addLogN.ShouldLog() { + log.Infof(ctx, "rate limited in %s: %s", redact.Safe(opName), err) + } + return baseQueueAsyncRateLimited } + return nil } // MaybeAddAsync offers the replica to the queue. The queue will only process a @@ -644,7 +653,7 @@ func (bq *baseQueue) Async( func (bq *baseQueue) MaybeAddAsync( ctx context.Context, repl replicaInQueue, now hlc.ClockTimestamp, ) { - bq.Async(ctx, "MaybeAdd", false /* wait */, func(ctx context.Context, h queueHelper) { + _ = bq.Async(ctx, "MaybeAdd", false /* wait */, func(ctx context.Context, h queueHelper) { h.MaybeAdd(ctx, repl, now) }) } @@ -653,7 +662,7 @@ func (bq *baseQueue) MaybeAddAsync( // for other operations to finish instead of turning into a noop (because // unlikely MaybeAdd, Add is not subject to being called opportunistically). func (bq *baseQueue) AddAsync(ctx context.Context, repl replicaInQueue, prio float64) { - bq.Async(ctx, "Add", true /* wait */, func(ctx context.Context, h queueHelper) { + _ = bq.Async(ctx, "Add", true /* wait */, func(ctx context.Context, h queueHelper) { h.Add(ctx, repl, prio) }) } diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 919c4f7245e4..31bc8b73c623 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -1256,25 +1256,25 @@ func (r *Replica) SetSpanConfig(conf roachpb.SpanConfig, sp roachpb.Span) bool { // impacted by changes to the SpanConfig. This should be called after any // changes to the span configs. func (r *Replica) MaybeQueue(ctx context.Context, now hlc.ClockTimestamp) { - r.store.splitQueue.Async(ctx, "span config update", true /* wait */, func(ctx context.Context, h queueHelper) { + _ = r.store.splitQueue.Async(ctx, "span config update", true /* wait */, func(ctx context.Context, h queueHelper) { h.MaybeAdd(ctx, r, now) }) - r.store.mergeQueue.Async(ctx, "span config update", true /* wait */, func(ctx context.Context, h queueHelper) { + _ = r.store.mergeQueue.Async(ctx, "span config update", true /* wait */, func(ctx context.Context, h queueHelper) { h.MaybeAdd(ctx, r, now) }) if EnqueueInMvccGCQueueOnSpanConfigUpdateEnabled.Get(&r.store.GetStoreConfig().Settings.SV) { - r.store.mvccGCQueue.Async(ctx, "span config update", true /* wait */, func(ctx context.Context, h queueHelper) { + _ = r.store.mvccGCQueue.Async(ctx, "span config update", true /* wait */, func(ctx context.Context, h queueHelper) { h.MaybeAdd(ctx, r, now) }) } - r.store.leaseQueue.Async(ctx, "span config update", true /* wait */, func(ctx context.Context, h queueHelper) { + _ = r.store.leaseQueue.Async(ctx, "span config update", true /* wait */, func(ctx context.Context, h queueHelper) { h.MaybeAdd(ctx, r, now) }) // The replicate queue has a relatively more expensive queue check // (shouldQueue), because it scales with the number of stores, and // performs more checks. if EnqueueInReplicateQueueOnSpanConfigUpdateEnabled.Get(&r.store.GetStoreConfig().Settings.SV) { - r.store.replicateQueue.Async(ctx, "span config update", true /* wait */, func(ctx context.Context, h queueHelper) { + _ = r.store.replicateQueue.Async(ctx, "span config update", true /* wait */, func(ctx context.Context, h queueHelper) { h.MaybeAdd(ctx, r, now) }) } diff --git a/pkg/kv/kvserver/store_gossip.go b/pkg/kv/kvserver/store_gossip.go index 3232d213b7f9..a50c70397b1d 100644 --- a/pkg/kv/kvserver/store_gossip.go +++ b/pkg/kv/kvserver/store_gossip.go @@ -206,10 +206,10 @@ func (s *Store) systemGossipUpdate(sysCfg *config.SystemConfig) { now := s.cfg.Clock.NowAsClockTimestamp() newStoreReplicaVisitor(s).Visit(func(repl *Replica) bool { if shouldQueue { - s.splitQueue.Async(ctx, "gossip update", true /* wait */, func(ctx context.Context, h queueHelper) { + _ = s.splitQueue.Async(ctx, "gossip update", true /* wait */, func(ctx context.Context, h queueHelper) { h.MaybeAdd(ctx, repl, now) }) - s.mergeQueue.Async(ctx, "gossip update", true /* wait */, func(ctx context.Context, h queueHelper) { + _ = s.mergeQueue.Async(ctx, "gossip update", true /* wait */, func(ctx context.Context, h queueHelper) { h.MaybeAdd(ctx, repl, now) }) } From d75b08eb11efdec778f0392323276c728282ec2a Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Thu, 28 Aug 2025 18:56:51 -0400 Subject: [PATCH 29/44] kvserver: add onProcessResult and onEnqueueResult to processCallback The base queue already supports registering callbacks that are invoked with the processing result of replicas once they are processed. However, replicas may fail before reaching that stage (e.g., failing to enqueue or dropped early). This commit extends the mechanism to also report enqueue results, allowing callers to detect failures earlier. Currently, only decommissioningNudger.maybeEnqueueProblemRange uses this. Note that one behavior change is introduced: previously, a registered callback would fire only once with the processing result and not again if the replica was later processed by the purgatory queue. With this change, the callback may now be invoked twice. --- pkg/kv/kvserver/mvcc_gc_queue.go | 2 +- pkg/kv/kvserver/queue.go | 170 ++++++++++++++++++---- pkg/kv/kvserver/queue_helpers_testutil.go | 2 +- pkg/kv/kvserver/replica.go | 31 +++- pkg/kv/kvserver/replica_backpressure.go | 21 +-- 5 files changed, 181 insertions(+), 45 deletions(-) diff --git a/pkg/kv/kvserver/mvcc_gc_queue.go b/pkg/kv/kvserver/mvcc_gc_queue.go index df2d31eb1316..afd250031ec0 100644 --- a/pkg/kv/kvserver/mvcc_gc_queue.go +++ b/pkg/kv/kvserver/mvcc_gc_queue.go @@ -888,7 +888,7 @@ func (mgcq *mvccGCQueue) scanReplicasForHiPriGCHints( if !isLeaseHolder { return true } - added, _ := mgcq.addInternal(ctx, desc, replica.ReplicaID(), deleteRangePriority) + added, _ := mgcq.addInternal(ctx, desc, replica.ReplicaID(), deleteRangePriority, noopProcessCallback) if added { mgcq.store.metrics.GCEnqueueHighPriority.Inc(1) foundReplicas++ diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index 392ce4fecd26..d203a88d486c 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -113,9 +113,57 @@ type PurgatoryError interface { PurgatoryErrorMarker() // dummy method for unique interface } -// processCallback is a hook that is called when a replica finishes processing. -// It is called with the result of the process attempt. -type processCallback func(error) +// noopProcessCallback is a processCallback that does nothing. +var noopProcessCallback = processCallback{ + onProcessResult: func(err error) {}, + onEnqueueResult: func(indexOnHeap int, err error) {}, +} + +// processCallback is a hook that is called when a replica is enqueued or +// finishes processing. +// +// NB: None of the fields below can be nil. Use noopProcessCallback if you do +// not need to register any callback. +// +// NB: These callbacks may be called multiple times: +// 1. onEnqueueResult may be called with error = nil first and called again with +// error = errDroppedDueToFullQueueSize when the replicaItem is later dropped +// before processing due to exceeding max queue size. +// 2. onProcessResult may be called with error first and sent to the purgatory +// queue and called again when the puragtory processes the replica. +// +// NB: It is not a strong guarantee that the callback will be executed since +// removeLocked or removeFromReplicaSetLocked may be called without executing +// the callbacks. That happens when the replica is destroyed or recreated with a +// new replica id. +// +// For now, the two use cases (decommissioning nudger and +// maybeBackpressureBatch) are okay with this behaviour. But adding new uses is +// discouraged without cleaning up the contract of processCallback. +// TODO(wenyihu6): consider clean the semantics up after backports +type processCallback struct { + // onProcessResult is called with the result of a process attempt. It is only + // invoked if the base queue gets a chance to process this replica. It may be + // invoked multiple times: first with a processing error and again with + // purgatory processing error. + onProcessResult func(err error) + + // onEnqueueResult is called with the result of the enqueue attempt. It is + // invoked when the range is added to the queue and if the range encounters + // any errors before getting a chance to be popped off the queue and getting + // processed. + // + // This may be invoked multiple times: first with error = nil when + // successfully enqueued at the beginning, and again with an error if the + // replica encounters any errors + // + // If error is nil, the index on the priority queue where this item sits is + // also passed in the callback. If error is non-nil, the index passed in the + // callback is -1. Note: indexOnHeap does not represent the item's exact rank + // by priority. It only reflects the item's position in the heap array, which + // gives a rough idea of where it sits in the priority hierarchy. + onEnqueueResult func(indexOnHeap int, err error) +} // A replicaItem holds a replica and metadata about its queue state and // processing state. @@ -146,8 +194,15 @@ func (i *replicaItem) setProcessing() { i.processing = true } -// registerCallback adds a new callback to be executed when the replicaItem -// finishes processing. +// registerCallback adds a new callback to be executed when the replicaItem is +// enqueued or finishes processing. There are two cases where the callback may +// be registered at: +// 1. bq.MaybeAddCallback: register the callback if the replicaItem has been +// added to bq.mu.replicas +// 2. bq.addInternal: register the callback if the replicaItem has not been +// added to bq.mu.replicas yet. +// Note that the contract here is tricky, so adding new uses is discouraged. See +// the comment on processCallback for more details. func (i *replicaItem) registerCallback(cb processCallback) { i.callbacks = append(i.callbacks, cb) } @@ -205,8 +260,13 @@ func (pq *priorityQueue) update(item *replicaItem, priority float64) { } var ( - errQueueDisabled = errors.New("queue disabled") - errQueueStopped = errors.New("queue stopped") + errQueueDisabled = errors.New("queue disabled") + errQueueStopped = errors.New("queue stopped") + errReplicaNotInitialized = errors.New("replica not initialized") + errReplicaAlreadyProcessing = errors.New("replica already processing") + errReplicaAlreadyInPurgatory = errors.New("replica in purgatory") + errReplicaAlreadyInQueue = errors.New("replica already in queue") + errDroppedDueToFullQueueSize = errors.New("queue full") ) func isExpectedQueueError(err error) bool { @@ -596,8 +656,10 @@ func (h baseQueueHelper) MaybeAdd( h.bq.maybeAdd(ctx, repl, now) } -func (h baseQueueHelper) Add(ctx context.Context, repl replicaInQueue, prio float64) { - _, err := h.bq.addInternal(ctx, repl.Desc(), repl.ReplicaID(), prio) +func (h baseQueueHelper) Add( + ctx context.Context, repl replicaInQueue, prio float64, processCallback processCallback, +) { + _, err := h.bq.addInternal(ctx, repl.Desc(), repl.ReplicaID(), prio, processCallback) if err != nil && log.V(1) { log.Infof(ctx, "during Add: %s", err) } @@ -605,7 +667,7 @@ func (h baseQueueHelper) Add(ctx context.Context, repl replicaInQueue, prio floa type queueHelper interface { MaybeAdd(ctx context.Context, repl replicaInQueue, now hlc.ClockTimestamp) - Add(ctx context.Context, repl replicaInQueue, prio float64) + Add(ctx context.Context, repl replicaInQueue, prio float64, processCallback processCallback) } // baseQueueAsyncRateLimited indicates that the base queue async task was rate @@ -658,12 +720,25 @@ func (bq *baseQueue) MaybeAddAsync( }) } +// MaybeAddAsyncWithCallback is the same as MaybeAddAsync, but allows the caller +// to register a process callback that will be invoked when the replica is +// enqueued or processed. +func (bq *baseQueue) AddAsyncWithCallback( + ctx context.Context, repl replicaInQueue, prio float64, processCallback processCallback, +) { + if err := bq.Async(ctx, "Add", true /* wait */, func(ctx context.Context, h queueHelper) { + h.Add(ctx, repl, prio, processCallback) + }); err != nil { + processCallback.onEnqueueResult(-1 /*indexOnHeap*/, err) + } +} + // AddAsync adds the replica to the queue. Unlike MaybeAddAsync, it will wait // for other operations to finish instead of turning into a noop (because // unlikely MaybeAdd, Add is not subject to being called opportunistically). func (bq *baseQueue) AddAsync(ctx context.Context, repl replicaInQueue, prio float64) { _ = bq.Async(ctx, "Add", true /* wait */, func(ctx context.Context, h queueHelper) { - h.Add(ctx, repl, prio) + h.Add(ctx, repl, prio, noopProcessCallback) }) } @@ -725,7 +800,7 @@ func (bq *baseQueue) maybeAdd(ctx context.Context, repl replicaInQueue, now hlc. return } } - _, err = bq.addInternal(ctx, repl.Desc(), repl.ReplicaID(), priority) + _, err = bq.addInternal(ctx, repl.Desc(), repl.ReplicaID(), priority, noopProcessCallback) if !isExpectedQueueError(err) { log.Errorf(ctx, "unable to add: %+v", err) } @@ -735,20 +810,26 @@ func (bq *baseQueue) maybeAdd(ctx context.Context, repl replicaInQueue, now hlc. // the replica is already queued at a lower priority, updates the existing // priority. Expects the queue lock to be held by caller. func (bq *baseQueue) addInternal( - ctx context.Context, desc *roachpb.RangeDescriptor, replicaID roachpb.ReplicaID, priority float64, + ctx context.Context, + desc *roachpb.RangeDescriptor, + replicaID roachpb.ReplicaID, + priority float64, + processCallback processCallback, ) (bool, error) { // NB: this is intentionally outside of bq.mu to avoid having to consider // lock ordering constraints. if !desc.IsInitialized() { // We checked this above in MaybeAdd(), but we need to check it // again for Add(). - return false, errors.New("replica not initialized") + processCallback.onEnqueueResult(-1 /*indexOnHeap*/, errReplicaNotInitialized) + return false, errReplicaNotInitialized } bq.mu.Lock() defer bq.mu.Unlock() if bq.mu.stopped { + processCallback.onEnqueueResult(-1 /*indexOnHeap*/, errQueueStopped) return false, errQueueStopped } @@ -761,12 +842,14 @@ func (bq *baseQueue) addInternal( if log.V(3) { log.Infof(ctx, "queue disabled") } + processCallback.onEnqueueResult(-1 /*indexOnHeap*/, errQueueDisabled) return false, errQueueDisabled } } // If the replica is currently in purgatory, don't re-add it. if _, ok := bq.mu.purgatory[desc.RangeID]; ok { + processCallback.onEnqueueResult(-1 /*indexOnHeap*/, errReplicaAlreadyInPurgatory) return false, nil } @@ -776,6 +859,7 @@ func (bq *baseQueue) addInternal( if item.processing { wasRequeued := item.requeue item.requeue = true + processCallback.onEnqueueResult(-1 /*indexOnHeap*/, errReplicaAlreadyProcessing) return !wasRequeued, nil } @@ -786,6 +870,9 @@ func (bq *baseQueue) addInternal( if log.V(1) { log.Infof(ctx, "updating priority: %0.3f -> %0.3f", item.priority, priority) } + // TODO(wenyihu6): will this introduce a lot of new memory allocation? + processCallback.onEnqueueResult(-1, /*indexOnHeap*/ + errors.Wrapf(errReplicaAlreadyInQueue, "priority=%.3f->%.3f", item.priority, priority)) bq.mu.priorityQ.update(item, priority) } return false, nil @@ -795,6 +882,7 @@ func (bq *baseQueue) addInternal( log.Infof(ctx, "adding: priority=%0.3f", priority) } item = &replicaItem{rangeID: desc.RangeID, replicaID: replicaID, priority: priority} + item.registerCallback(processCallback) bq.addLocked(item) // If adding this replica has pushed the queue past its maximum size, remove @@ -809,6 +897,11 @@ func (bq *baseQueue) addInternal( } log.Dev.VInfof(ctx, 1, "dropping due to exceeding queue max size: priority=%0.3f, replica=%v", priority, replicaItemToDrop.replicaID) + // TODO(wenyihu6): when we introduce base queue max size cluster setting, + // remember to invoke this callback when shrinking the size + for _, cb := range replicaItemToDrop.callbacks { + cb.onEnqueueResult(-1 /*indexOnHeap*/, errDroppedDueToFullQueueSize) + } bq.removeLocked(replicaItemToDrop) } // Signal the processLoop that a replica has been added. @@ -817,26 +910,41 @@ func (bq *baseQueue) addInternal( default: // No need to signal again. } + // Note: it may already be dropped or dropped afterwards. + processCallback.onEnqueueResult(item.index /*indexOnHeap*/, nil) return true, nil } // MaybeAddCallback adds a callback to be called when the specified range -// finishes processing if the range is in the queue. If the range is in -// purgatory, the callback is called immediately with the purgatory error. If -// the range is not in the queue (either waiting or processing), the method -// returns false. +// finishes processing. The replica can be in one of several states: // -// NB: If the replica this attaches to is dropped from an overfull queue, this -// callback is never called. This is surprising, but the single caller of this -// is okay with these semantics. Adding new uses is discouraged without cleaning -// up the contract of this method, but this code doesn't lend itself readily to -// upholding invariants so there may need to be some cleanup first. +// - waiting: not in mu.replicas +// Returns false and no callback is executed. +// +// - queued: in mu.replicas and mu.priorityQ +// Returns true and callback is executed when the replica is processed. +// +// - purgatory: in mu.replicas and mu.purgatory +// Returns true and the callback is called immediately with the purgatory error. +// Note that the callback may be invoked again when the purgatory finishes +// processing the replica. +// +// - processing: only in mu.replicas and currently being processed +// Returns true and callback is executed when processing completes. If the +// replica is currently being processed by the purgatory queue, it will not +// be in bq.mu.purgatory and the callback will only execute when the purgatory +// finishes processing the replica. +// +// NB: Adding new uses is discouraged without cleaning up the contract of +// processCallback. For example, removeFromReplicaSetLocked may be called +// without invoking these callbacks. See replicaItem.registerCallback for more +// details. func (bq *baseQueue) MaybeAddCallback(rangeID roachpb.RangeID, cb processCallback) bool { bq.mu.Lock() defer bq.mu.Unlock() if purgatoryErr, ok := bq.mu.purgatory[rangeID]; ok { - cb(purgatoryErr) + cb.onProcessResult(purgatoryErr) return true } if item, ok := bq.mu.replicas[rangeID]; ok { @@ -1206,7 +1314,7 @@ func (bq *baseQueue) finishProcessingReplica( // Call any registered callbacks. for _, cb := range callbacks { - cb(err) + cb.onProcessResult(err) } // Handle failures. @@ -1225,7 +1333,7 @@ func (bq *baseQueue) finishProcessingReplica( // purgatory. if purgErr, ok := IsPurgatoryError(err); ok { bq.mu.Lock() - bq.addToPurgatoryLocked(ctx, stopper, repl, purgErr, priority /*priorityAtEnqueue*/) + bq.addToPurgatoryLocked(ctx, stopper, repl, purgErr, priority /*priorityAtEnqueue*/, callbacks /*processCallback*/) bq.mu.Unlock() return } @@ -1250,6 +1358,7 @@ func (bq *baseQueue) addToPurgatoryLocked( repl replicaInQueue, purgErr PurgatoryError, priorityAtEnqueue float64, + processCallback []processCallback, ) { bq.mu.AssertHeld() @@ -1273,7 +1382,14 @@ func (bq *baseQueue) addToPurgatoryLocked( return } - item := &replicaItem{rangeID: repl.GetRangeID(), replicaID: repl.ReplicaID(), index: -1, priority: priorityAtEnqueue} + item := &replicaItem{ + rangeID: repl.GetRangeID(), + replicaID: repl.ReplicaID(), + index: -1, + priority: priorityAtEnqueue, + callbacks: processCallback, + } + bq.mu.replicas[repl.GetRangeID()] = item defer func() { diff --git a/pkg/kv/kvserver/queue_helpers_testutil.go b/pkg/kv/kvserver/queue_helpers_testutil.go index a667a15a43c0..5a05c4b5309d 100644 --- a/pkg/kv/kvserver/queue_helpers_testutil.go +++ b/pkg/kv/kvserver/queue_helpers_testutil.go @@ -18,7 +18,7 @@ import ( func (bq *baseQueue) testingAdd( ctx context.Context, repl replicaInQueue, priority float64, ) (bool, error) { - return bq.addInternal(ctx, repl.Desc(), repl.ReplicaID(), priority) + return bq.addInternal(ctx, repl.Desc(), repl.ReplicaID(), priority, noopProcessCallback) } func forceScanAndProcess(ctx context.Context, s *Store, q *baseQueue) error { diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 31bc8b73c623..319cbd50afcb 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -2904,16 +2904,33 @@ func (r *Replica) maybeEnqueueProblemRange( "lastProblemRangeReplicateEnqueueTime was updated concurrently", r.Desc()) return } - // Log at default verbosity to ensure some indication the nudger is working - // (other logs have a verbosity of 1 which). - log.KvDistribution.Infof(ctx, "decommissioning nudger enqueuing replica %s "+ - "with priority %f", r.Desc(), - allocatorimpl.AllocatorReplaceDecommissioningVoter.Priority()) r.store.metrics.DecommissioningNudgerEnqueue.Inc(1) // TODO(dodeca12): Figure out a better way to track the // decommissioning nudger enqueue failures/errors. - r.store.replicateQueue.AddAsync(ctx, r, - allocatorimpl.AllocatorReplaceDecommissioningVoter.Priority()) + r.store.replicateQueue.AddAsyncWithCallback(ctx, r, + allocatorimpl.AllocatorReplaceDecommissioningVoter.Priority(), processCallback{ + onEnqueueResult: func(indexOnHeap int, err error) { + if err != nil { + // TODO(wenyihu6): if we want to put these logs behind vmodule, move + // this function to another file so that we can avoid the spam on + // other logs. + log.KvDistribution.Infof(ctx, + "decommissioning nudger failed to enqueue range %v due to %v", r.Desc(), err) + } else { + log.KvDistribution.Infof(ctx, + "decommissioning nudger successfully enqueued range %v at index %d", r.Desc(), indexOnHeap) + } + }, + onProcessResult: func(err error) { + if err != nil { + log.KvDistribution.Infof(ctx, + "decommissioning nudger failed to process range %v due to %v", r.Desc(), err) + } else { + log.KvDistribution.Infof(ctx, + "decommissioning nudger successfully processed replica %s", r.Desc()) + } + }, + }) } // SendStreamStats sets the stats for the replica send streams that belong to diff --git a/pkg/kv/kvserver/replica_backpressure.go b/pkg/kv/kvserver/replica_backpressure.go index 47668480346e..b93f429f7a63 100644 --- a/pkg/kv/kvserver/replica_backpressure.go +++ b/pkg/kv/kvserver/replica_backpressure.go @@ -203,15 +203,18 @@ func (r *Replica) maybeBackpressureBatch(ctx context.Context, ba *kvpb.BatchRequ // Register a callback on an ongoing split for this range in the splitQueue. splitC := make(chan error, 1) - if !r.store.splitQueue.MaybeAddCallback(r.RangeID, func(err error) { - select { - case splitC <- err: - default: - // TODO(wenyihu6): should we add ctx timeout when invoking callbacks - // Drop the error if the channel is already full. This prevents - // blocking if the callback is invoked multiple times. - return - } + if !r.store.splitQueue.MaybeAddCallback(r.RangeID, processCallback{ + onEnqueueResult: func(rank int, err error) {}, + onProcessResult: func(err error) { + select { + case splitC <- err: + default: + // TODO(wenyihu6): should we add ctx timeout when invoking callbacks + // Drop the error if the channel is already full. This prevents + // blocking if the callback is invoked multiple times. + return + } + }, }) { // No split ongoing. We may have raced with its completion. There's // no good way to prevent this race, so we conservatively allow the From 317c0a2cf6cf8968769d517ae051e99217f49111 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Sat, 30 Aug 2025 18:30:41 -0400 Subject: [PATCH 30/44] kvserver: add TestBaseQueueCallback This commit adds TestBaseQueueCallbackOnEnqueueResult and TestBaseQueueCallbackOnProcessResult to verify that callbacks are correctly invoked with both enqueue and process results. --- pkg/kv/kvserver/queue_helpers_testutil.go | 9 + pkg/kv/kvserver/queue_test.go | 259 ++++++++++++++++++++++ 2 files changed, 268 insertions(+) diff --git a/pkg/kv/kvserver/queue_helpers_testutil.go b/pkg/kv/kvserver/queue_helpers_testutil.go index 5a05c4b5309d..6840934b701d 100644 --- a/pkg/kv/kvserver/queue_helpers_testutil.go +++ b/pkg/kv/kvserver/queue_helpers_testutil.go @@ -21,6 +21,15 @@ func (bq *baseQueue) testingAdd( return bq.addInternal(ctx, repl.Desc(), repl.ReplicaID(), priority, noopProcessCallback) } +// testingAddWithCallback is the same as testingAdd, but allows the caller to +// register a process callback that will be invoked when the replica is enqueued +// or processed. +func (bq *baseQueue) testingAddWithCallback( + ctx context.Context, repl replicaInQueue, priority float64, callback processCallback, +) (bool, error) { + return bq.addInternal(ctx, repl.Desc(), repl.ReplicaID(), priority, callback) +} + func forceScanAndProcess(ctx context.Context, s *Store, q *baseQueue) error { // Check that the system config is available. It is needed by many queues. If // it's not available, some queues silently fail to process any replicas, diff --git a/pkg/kv/kvserver/queue_test.go b/pkg/kv/kvserver/queue_test.go index a20ca8a071fc..f0b545f5c1b3 100644 --- a/pkg/kv/kvserver/queue_test.go +++ b/pkg/kv/kvserver/queue_test.go @@ -1292,6 +1292,265 @@ func TestBaseQueueDisable(t *testing.T) { } } +// TestBaseQueueCallbackOnEnqueueResult tests the callback onEnqueueResult for +// 1. successful case: the replica is successfully enqueued. +// 2. priority update: updates the priority of the replica and not enqueuing +// again. +// 3. disabled: queue is disabled and the replica is not enqueued. +// 4. stopped: queue is stopped and the replica is not enqueued. +// 5. already queued: the replica is already in the queue and not enqueued +// again. +// 6. purgatory: the replica is in purgatory and not enqueued again. +// 7. processing: the replica is already being processed and not enqueued again. +// 8. full queue: the queue is full and the replica is not enqueued again. +func TestBaseQueueCallbackOnEnqueueResult(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + tc := testContext{} + stopper := stop.NewStopper() + ctx := context.Background() + defer stopper.Stop(ctx) + tc.Start(ctx, t, stopper) + + t.Run("successfuladd", func(t *testing.T) { + testQueue := &testQueueImpl{} + bq := makeTestBaseQueue("test", testQueue, tc.store, queueConfig{maxSize: 1}) + r, err := tc.store.GetReplica(1) + require.NoError(t, err) + queued, _ := bq.testingAddWithCallback(ctx, r, 1.0, processCallback{ + onEnqueueResult: func(indexOnHeap int, err error) { + require.Equal(t, 0, indexOnHeap) + require.NoError(t, err) + }, + onProcessResult: func(err error) { + t.Fatal("unexpected call to onProcessResult") + }, + }) + require.True(t, queued) + }) + + t.Run("priority", func(t *testing.T) { + testQueue := &testQueueImpl{} + bq := makeTestBaseQueue("test", testQueue, tc.store, queueConfig{maxSize: 5}) + r, err := tc.store.GetReplica(1) + require.NoError(t, err) + priorities := []float64{5.0, 4.0, 8.0, 1.0, 3.0} + expectedIndices := []int{0, 1, 0, 3, 4} + // When inserting 5, [5], index 0. + // When inserting 4, [5, 4], index 1. + // When inserting 8, [8, 4, 5], index 0. + // When inserting 1, [8, 4, 5, 1], index 3. + // When inserting 3, [8, 4, 5, 1, 3], index 4. + for i, priority := range priorities { + r.Desc().RangeID = roachpb.RangeID(i + 1) + queued, _ := bq.testingAddWithCallback(ctx, r, priority, processCallback{ + onEnqueueResult: func(indexOnHeap int, err error) { + require.Equal(t, expectedIndices[i], indexOnHeap) + require.NoError(t, err) + }, + onProcessResult: func(err error) { + t.Fatal("unexpected call to onProcessResult") + }, + }) + require.True(t, queued) + } + // Set range id back to 1. + r.Desc().RangeID = 1 + }) + t.Run("disabled", func(t *testing.T) { + testQueue := &testQueueImpl{} + bq := makeTestBaseQueue("test", testQueue, tc.store, queueConfig{maxSize: 2}) + bq.SetDisabled(true) + r, err := tc.store.GetReplica(1) + require.NoError(t, err) + queued, _ := bq.testingAddWithCallback(ctx, r, 1.0, processCallback{ + onEnqueueResult: func(indexOnHeap int, err error) { + require.Equal(t, -1, indexOnHeap) + require.ErrorIs(t, err, errQueueDisabled) + }, + onProcessResult: func(err error) { + t.Fatal("unexpected call to onProcessResult") + }, + }) + require.False(t, queued) + }) + t.Run("stopped", func(t *testing.T) { + testQueue := &testQueueImpl{} + bq := makeTestBaseQueue("test", testQueue, tc.store, queueConfig{maxSize: 2}) + bq.mu.stopped = true + r, err := tc.store.GetReplica(1) + require.NoError(t, err) + queued, _ := bq.testingAddWithCallback(ctx, r, 1.0, processCallback{ + onEnqueueResult: func(indexOnHeap int, err error) { + require.Equal(t, -1, indexOnHeap) + require.ErrorIs(t, err, errQueueStopped) + }, + onProcessResult: func(err error) { + t.Fatal("unexpected call to onProcessResult") + }, + }) + require.False(t, queued) + }) + + t.Run("alreadyqueued", func(t *testing.T) { + testQueue := &testQueueImpl{} + bq := makeTestBaseQueue("test", testQueue, tc.store, queueConfig{maxSize: 2}) + r, err := tc.store.GetReplica(1) + require.NoError(t, err) + queued, _ := bq.testingAddWithCallback(ctx, r, 1.0, processCallback{ + onEnqueueResult: func(indexOnHeap int, err error) { + require.Equal(t, 0, indexOnHeap) + require.NoError(t, err) + }, + onProcessResult: func(err error) { + t.Fatal("unexpected call to onProcessResult") + }, + }) + require.True(t, queued) + + // Inserting again on the same range id should fail. + queued, _ = bq.testingAddWithCallback(ctx, r, 1.0, processCallback{ + onEnqueueResult: func(indexOnHeap int, err error) { + require.Equal(t, -1, indexOnHeap) + require.ErrorIs(t, err, errReplicaAlreadyInQueue) + }, + onProcessResult: func(err error) { + t.Fatal("unexpected call to onProcessResult") + }, + }) + require.False(t, queued) + }) + + t.Run("purgatory", func(t *testing.T) { + testQueue := &testQueueImpl{ + pChan: make(chan time.Time, 1), + } + bq := makeTestBaseQueue("test", testQueue, tc.store, queueConfig{maxSize: 2}) + r, err := tc.store.GetReplica(1) + require.NoError(t, err) + bq.mu.Lock() + bq.addToPurgatoryLocked(ctx, stopper, r, &testPurgatoryError{}, 1.0, nil) + bq.mu.Unlock() + // Inserting a range in purgatory should not enqueue again. + queued, _ := bq.testingAddWithCallback(ctx, r, 1.0, processCallback{ + onEnqueueResult: func(indexOnHeap int, err error) { + require.Equal(t, -1, indexOnHeap) + require.ErrorIs(t, err, errReplicaAlreadyInPurgatory) + }, + onProcessResult: func(err error) { + t.Fatal("unexpected call to onProcessResult") + }, + }) + require.False(t, queued) + }) + + t.Run("processing", func(t *testing.T) { + testQueue := &testQueueImpl{} + bq := makeTestBaseQueue("test", testQueue, tc.store, queueConfig{maxSize: 2}) + r, err := tc.store.GetReplica(1) + require.NoError(t, err) + item := &replicaItem{rangeID: r.Desc().RangeID, replicaID: r.ReplicaID(), index: -1} + item.setProcessing() + bq.addLocked(item) + // Inserting a range that is already being processed should not enqueue again. + requeued, _ := bq.testingAddWithCallback(ctx, r, 1.0, processCallback{ + onEnqueueResult: func(indexOnHeap int, err error) { + require.Equal(t, -1, indexOnHeap) + require.ErrorIs(t, err, errReplicaAlreadyProcessing) + }, + onProcessResult: func(err error) { + t.Fatal("unexpected call to onProcessResult") + }, + }) + require.True(t, requeued) + }) + t.Run("fullqueue", func(t *testing.T) { + testQueue := &testQueueImpl{} + bq := makeTestBaseQueue("test", testQueue, tc.store, queueConfig{maxSize: 0}) + r, err := tc.store.GetReplica(1) + require.NoError(t, err) + // Max size is 0, so the replica should not be enqueued. + queued, _ := bq.testingAddWithCallback(ctx, r, 1.0, processCallback{ + onEnqueueResult: func(indexOnHeap int, err error) { + // It may be called with err = nil. + if err != nil { + require.ErrorIs(t, err, errDroppedDueToFullQueueSize) + } + }, + onProcessResult: func(err error) { + t.Fatal("unexpected call to onProcessResult") + }, + }) + require.True(t, queued) + }) +} + +// TestBaseQueueCallbackOnProcessResult tests that the processCallback is +// invoked when the replica is processed and will be invoked again if the +// replica ends up in the purgatory queue and being processed again. +func TestBaseQueueCallbackOnProcessResult(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + tc := testContext{} + stopper := stop.NewStopper() + ctx := context.Background() + defer stopper.Stop(ctx) + tsc := TestStoreConfig(nil) + tc.StartWithStoreConfig(ctx, t, stopper, tsc) + + testQueue := &testQueueImpl{ + duration: time.Nanosecond, + pChan: make(chan time.Time, 1), + err: &testPurgatoryError{}, + } + + const replicaCount = 10 + repls := createReplicas(t, &tc, replicaCount) + + bq := makeTestBaseQueue("test", testQueue, tc.store, queueConfig{maxSize: replicaCount}) + bq.Start(stopper) + + var totalProcessedCalledWithErr atomic.Int32 + for _, r := range repls { + queued, _ := bq.testingAddWithCallback(context.Background(), r, 1.0, processCallback{ + onEnqueueResult: func(indexOnHeap int, err error) { + require.NoError(t, err) + }, + onProcessResult: func(err error) { + if err != nil { + totalProcessedCalledWithErr.Add(1) + } + }, + }) + require.True(t, queued) + } + + testutils.SucceedsSoon(t, func() error { + if pc := testQueue.getProcessed(); pc != replicaCount { + return errors.Errorf("expected %d processed replicas; got %d", replicaCount, pc) + } + + if totalProcessedCalledWithErr.Load() != int32(replicaCount) { + return errors.Errorf("expected %d processed replicas with err; got %d", replicaCount, totalProcessedCalledWithErr.Load()) + } + return nil + }) + + // Now, signal that purgatoried replicas should retry. + testQueue.pChan <- timeutil.Now() + + testutils.SucceedsSoon(t, func() error { + if pc := testQueue.getProcessed(); pc != replicaCount*2 { + return errors.Errorf("expected %d processed replicas; got %d", replicaCount, pc) + } + + if totalProcessedCalledWithErr.Load() != int32(replicaCount*2) { + return errors.Errorf("expected %d processed replicas with err; got %d", replicaCount, totalProcessedCalledWithErr.Load()) + } + return nil + }) +} + // TestQueueDisable verifies that setting the set of queue.enabled cluster // settings actually disables the base queue. This test works alongside // TestBaseQueueDisable to verify the entire disable workflow. From b54e16bc34e720ed2b561b24fbc4c5a7c50d1a38 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Mon, 1 Sep 2025 23:16:09 -0400 Subject: [PATCH 31/44] kvserver: better comments for on processCallback This commit updates the comments to better clarify the semantics and the contract of processCallback. --- pkg/kv/kvserver/queue.go | 99 ++++++++++++++++--------- pkg/kv/kvserver/replica_backpressure.go | 1 - 2 files changed, 63 insertions(+), 37 deletions(-) diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index d203a88d486c..6c199edf6932 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -125,44 +125,47 @@ var noopProcessCallback = processCallback{ // NB: None of the fields below can be nil. Use noopProcessCallback if you do // not need to register any callback. // -// NB: These callbacks may be called multiple times: -// 1. onEnqueueResult may be called with error = nil first and called again with -// error = errDroppedDueToFullQueueSize when the replicaItem is later dropped -// before processing due to exceeding max queue size. -// 2. onProcessResult may be called with error first and sent to the purgatory -// queue and called again when the puragtory processes the replica. +// The callback behavior depends on when it's registered. Currently, addInternal +// and MaybeAddCallback are the only two users. See comments above them for more +// details on the exact behaviour. // -// NB: It is not a strong guarantee that the callback will be executed since -// removeLocked or removeFromReplicaSetLocked may be called without executing -// the callbacks. That happens when the replica is destroyed or recreated with a -// new replica id. +// NB: Callback execution is not guaranteed since removeLocked or +// removeFromReplicaSetLocked may be called without executing callbacks. This +// happens when the replica is destroyed or recreated with a new replica ID. // // For now, the two use cases (decommissioning nudger and -// maybeBackpressureBatch) are okay with this behaviour. But adding new uses is -// discouraged without cleaning up the contract of processCallback. -// TODO(wenyihu6): consider clean the semantics up after backports +// maybeBackpressureBatch) are okay with the current behaviour. But adding new +// uses is discouraged without cleaning up the contract of processCallback. +// TODO(wenyihu6): consider cleaning up the semantics after backports type processCallback struct { - // onProcessResult is called with the result of a process attempt. It is only - // invoked if the base queue gets a chance to process this replica. It may be - // invoked multiple times: first with a processing error and again with - // purgatory processing error. - onProcessResult func(err error) - // onEnqueueResult is called with the result of the enqueue attempt. It is // invoked when the range is added to the queue and if the range encounters - // any errors before getting a chance to be popped off the queue and getting - // processed. - // - // This may be invoked multiple times: first with error = nil when - // successfully enqueued at the beginning, and again with an error if the - // replica encounters any errors + // any errors and being enqueued again before being processed. // // If error is nil, the index on the priority queue where this item sits is // also passed in the callback. If error is non-nil, the index passed in the // callback is -1. Note: indexOnHeap does not represent the item's exact rank // by priority. It only reflects the item's position in the heap array, which // gives a rough idea of where it sits in the priority hierarchy. + // + // - May be invoked multiple times: + // 1. Immediately after successful enqueue (err = nil). + // 2. If the replica is later dropped due to full queue (err = + // errDroppedDueToFullQueueSize). + // 3. If re-added with updated priority (err = nil, new heap index). + // 4. If the replica is already in the queue and processing. + // - May be skipped if the replica is already in queue and no priority changes + // occur. onEnqueueResult func(indexOnHeap int, err error) + + // onProcessResult is called with the result of any process attempts. It is + // only invoked if the base queue gets a chance to process this replica. + // + // - May be invoked multiple times if the replica goes through purgatory or + // re-processing. + // - May be skipped if the replica is removed with removeFromReplicaSetLocked + // or registered with a new replica id before processing begins. + onProcessResult func(err error) } // A replicaItem holds a replica and metadata about its queue state and @@ -720,9 +723,9 @@ func (bq *baseQueue) MaybeAddAsync( }) } -// MaybeAddAsyncWithCallback is the same as MaybeAddAsync, but allows the caller -// to register a process callback that will be invoked when the replica is -// enqueued or processed. +// AddAsyncWithCallback is the same as AddAsync, but allows the caller to +// register a process callback that will be invoked when the replica is enqueued +// or processed. func (bq *baseQueue) AddAsyncWithCallback( ctx context.Context, repl replicaInQueue, prio float64, processCallback processCallback, ) { @@ -809,6 +812,27 @@ func (bq *baseQueue) maybeAdd(ctx context.Context, repl replicaInQueue, now hlc. // addInternal adds the replica the queue with specified priority. If // the replica is already queued at a lower priority, updates the existing // priority. Expects the queue lock to be held by caller. +// +// processCallback allows the caller to register a callback that will be invoked +// when the replica is enqueued or processed. +// - If the replicaItem has not been added to bq.mu.replicas yet, the callback +// is registered and onEnqueueResult is invoked immediately with the result of +// the enqueue attempt. If successfully enqueued, onProcessResult will be +// invoked when processing completes. +// - If the replicaItem has already been added to bq.mu.replicas, no new +// callbacks will be registered. onEnqueueResult registered first time will be +// invoked with the result of enqueue attempts: +// 1. Already processing or in purgatory: invoked with +// errReplicaAlreadyProcessing/errReplicaAlreadyInPurgatory +// 2. Priority updated: invoked with error = nil and new heap index +// 3. Waiting in queue without priority change: not invoked +// 4. Dropped due to full queue: invoked with +// errDroppedDueToFullQueueSizeonEnqueueResult registered first time is +// invoked with the result of this enqueue attempt. +// 5. Other errors: invoked with the error. +// +// NB: callback invokation is not guanranteed since removeFromReplicaSetLocked +// may remove the replica from the queue at any time without invoking them. func (bq *baseQueue) addInternal( ctx context.Context, desc *roachpb.RangeDescriptor, @@ -922,18 +946,21 @@ func (bq *baseQueue) addInternal( // Returns false and no callback is executed. // // - queued: in mu.replicas and mu.priorityQ -// Returns true and callback is executed when the replica is processed. +// Returns true. onProcessResult is executed when the replica is processed. // // - purgatory: in mu.replicas and mu.purgatory -// Returns true and the callback is called immediately with the purgatory error. -// Note that the callback may be invoked again when the purgatory finishes -// processing the replica. +// Returns true and the onProcessResult is called immediately with the +// purgatory error. Note that the onProcessResult may be invoked again when +// the purgatory finishes processing the replica.. // // - processing: only in mu.replicas and currently being processed -// Returns true and callback is executed when processing completes. If the -// replica is currently being processed by the purgatory queue, it will not -// be in bq.mu.purgatory and the callback will only execute when the purgatory -// finishes processing the replica. +// Returns true and onProcessResult is executed when processing completes. +// If the replica is currently being processed by the purgatory queue, it +// will not be in bq.mu.purgatory and the onProcessResult will only execute +// when the purgatory finishes processing the replica. +// +// If it returns true, onEnqueueResult is invoked on subsequent invocations to +// addInternal as well. // // NB: Adding new uses is discouraged without cleaning up the contract of // processCallback. For example, removeFromReplicaSetLocked may be called diff --git a/pkg/kv/kvserver/replica_backpressure.go b/pkg/kv/kvserver/replica_backpressure.go index b93f429f7a63..ca0a585a5277 100644 --- a/pkg/kv/kvserver/replica_backpressure.go +++ b/pkg/kv/kvserver/replica_backpressure.go @@ -209,7 +209,6 @@ func (r *Replica) maybeBackpressureBatch(ctx context.Context, ba *kvpb.BatchRequ select { case splitC <- err: default: - // TODO(wenyihu6): should we add ctx timeout when invoking callbacks // Drop the error if the channel is already full. This prevents // blocking if the callback is invoked multiple times. return From b80f751934fc8d5da9378474bb9762fbfa94d82a Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Tue, 2 Sep 2025 14:36:45 -0400 Subject: [PATCH 32/44] kvserver: treat priority update as a success with onEnqueueResult Previously, bq.addInternal invoked processCallback.onEnqueueResult with a newly constructed error whenever a replica was already present and re-enqueued with a higher priority, since the priority had to be passed dynamically. This commit instead treats the case as a success and passes the updated heap index to onEnqueueResult, as the caller mainly cares about whether the replica is already in the queue and its new position. --- pkg/kv/kvserver/queue.go | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index 6c199edf6932..e07668d9a7a5 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -894,10 +894,9 @@ func (bq *baseQueue) addInternal( if log.V(1) { log.Infof(ctx, "updating priority: %0.3f -> %0.3f", item.priority, priority) } - // TODO(wenyihu6): will this introduce a lot of new memory allocation? - processCallback.onEnqueueResult(-1, /*indexOnHeap*/ - errors.Wrapf(errReplicaAlreadyInQueue, "priority=%.3f->%.3f", item.priority, priority)) bq.mu.priorityQ.update(item, priority) + // item.index should be updated now based on heap property now. + processCallback.onEnqueueResult(item.index /*indexOnHeap*/, nil) } return false, nil } From 8691d330294464b59895b5286f471f46d2a8cde0 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Tue, 2 Sep 2025 14:40:51 -0400 Subject: [PATCH 33/44] kvserver: rename processCallback processCallback to cb processCallback Previously, the variable name processCallback shadowed its type name, which was not ideal. This commit renames the variable to cb. --- pkg/kv/kvserver/queue.go | 40 +++++++++++------------ pkg/kv/kvserver/queue_helpers_testutil.go | 4 +-- 2 files changed, 22 insertions(+), 22 deletions(-) diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index e07668d9a7a5..43d473851014 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -660,9 +660,9 @@ func (h baseQueueHelper) MaybeAdd( } func (h baseQueueHelper) Add( - ctx context.Context, repl replicaInQueue, prio float64, processCallback processCallback, + ctx context.Context, repl replicaInQueue, prio float64, cb processCallback, ) { - _, err := h.bq.addInternal(ctx, repl.Desc(), repl.ReplicaID(), prio, processCallback) + _, err := h.bq.addInternal(ctx, repl.Desc(), repl.ReplicaID(), prio, cb) if err != nil && log.V(1) { log.Infof(ctx, "during Add: %s", err) } @@ -670,7 +670,7 @@ func (h baseQueueHelper) Add( type queueHelper interface { MaybeAdd(ctx context.Context, repl replicaInQueue, now hlc.ClockTimestamp) - Add(ctx context.Context, repl replicaInQueue, prio float64, processCallback processCallback) + Add(ctx context.Context, repl replicaInQueue, prio float64, cb processCallback) } // baseQueueAsyncRateLimited indicates that the base queue async task was rate @@ -727,12 +727,12 @@ func (bq *baseQueue) MaybeAddAsync( // register a process callback that will be invoked when the replica is enqueued // or processed. func (bq *baseQueue) AddAsyncWithCallback( - ctx context.Context, repl replicaInQueue, prio float64, processCallback processCallback, + ctx context.Context, repl replicaInQueue, prio float64, cb processCallback, ) { if err := bq.Async(ctx, "Add", true /* wait */, func(ctx context.Context, h queueHelper) { - h.Add(ctx, repl, prio, processCallback) + h.Add(ctx, repl, prio, cb) }); err != nil { - processCallback.onEnqueueResult(-1 /*indexOnHeap*/, err) + cb.onEnqueueResult(-1 /*indexOnHeap*/, err) } } @@ -838,14 +838,14 @@ func (bq *baseQueue) addInternal( desc *roachpb.RangeDescriptor, replicaID roachpb.ReplicaID, priority float64, - processCallback processCallback, + cb processCallback, ) (bool, error) { // NB: this is intentionally outside of bq.mu to avoid having to consider // lock ordering constraints. if !desc.IsInitialized() { // We checked this above in MaybeAdd(), but we need to check it // again for Add(). - processCallback.onEnqueueResult(-1 /*indexOnHeap*/, errReplicaNotInitialized) + cb.onEnqueueResult(-1 /*indexOnHeap*/, errReplicaNotInitialized) return false, errReplicaNotInitialized } @@ -853,7 +853,7 @@ func (bq *baseQueue) addInternal( defer bq.mu.Unlock() if bq.mu.stopped { - processCallback.onEnqueueResult(-1 /*indexOnHeap*/, errQueueStopped) + cb.onEnqueueResult(-1 /*indexOnHeap*/, errQueueStopped) return false, errQueueStopped } @@ -866,14 +866,14 @@ func (bq *baseQueue) addInternal( if log.V(3) { log.Infof(ctx, "queue disabled") } - processCallback.onEnqueueResult(-1 /*indexOnHeap*/, errQueueDisabled) + cb.onEnqueueResult(-1 /*indexOnHeap*/, errQueueDisabled) return false, errQueueDisabled } } // If the replica is currently in purgatory, don't re-add it. if _, ok := bq.mu.purgatory[desc.RangeID]; ok { - processCallback.onEnqueueResult(-1 /*indexOnHeap*/, errReplicaAlreadyInPurgatory) + cb.onEnqueueResult(-1 /*indexOnHeap*/, errReplicaAlreadyInPurgatory) return false, nil } @@ -883,7 +883,7 @@ func (bq *baseQueue) addInternal( if item.processing { wasRequeued := item.requeue item.requeue = true - processCallback.onEnqueueResult(-1 /*indexOnHeap*/, errReplicaAlreadyProcessing) + cb.onEnqueueResult(-1 /*indexOnHeap*/, errReplicaAlreadyProcessing) return !wasRequeued, nil } @@ -895,8 +895,8 @@ func (bq *baseQueue) addInternal( log.Infof(ctx, "updating priority: %0.3f -> %0.3f", item.priority, priority) } bq.mu.priorityQ.update(item, priority) - // item.index should be updated now based on heap property now. - processCallback.onEnqueueResult(item.index /*indexOnHeap*/, nil) + // item.index should be updated now based on heap property now. + cb.onEnqueueResult(item.index /*indexOnHeap*/, nil) } return false, nil } @@ -905,7 +905,7 @@ func (bq *baseQueue) addInternal( log.Infof(ctx, "adding: priority=%0.3f", priority) } item = &replicaItem{rangeID: desc.RangeID, replicaID: replicaID, priority: priority} - item.registerCallback(processCallback) + item.registerCallback(cb) bq.addLocked(item) // If adding this replica has pushed the queue past its maximum size, remove @@ -922,8 +922,8 @@ func (bq *baseQueue) addInternal( priority, replicaItemToDrop.replicaID) // TODO(wenyihu6): when we introduce base queue max size cluster setting, // remember to invoke this callback when shrinking the size - for _, cb := range replicaItemToDrop.callbacks { - cb.onEnqueueResult(-1 /*indexOnHeap*/, errDroppedDueToFullQueueSize) + for _, callback := range replicaItemToDrop.callbacks { + callback.onEnqueueResult(-1 /*indexOnHeap*/, errDroppedDueToFullQueueSize) } bq.removeLocked(replicaItemToDrop) } @@ -934,7 +934,7 @@ func (bq *baseQueue) addInternal( // No need to signal again. } // Note: it may already be dropped or dropped afterwards. - processCallback.onEnqueueResult(item.index /*indexOnHeap*/, nil) + cb.onEnqueueResult(item.index /*indexOnHeap*/, nil) return true, nil } @@ -1384,7 +1384,7 @@ func (bq *baseQueue) addToPurgatoryLocked( repl replicaInQueue, purgErr PurgatoryError, priorityAtEnqueue float64, - processCallback []processCallback, + cbs []processCallback, ) { bq.mu.AssertHeld() @@ -1413,7 +1413,7 @@ func (bq *baseQueue) addToPurgatoryLocked( replicaID: repl.ReplicaID(), index: -1, priority: priorityAtEnqueue, - callbacks: processCallback, + callbacks: cbs, } bq.mu.replicas[repl.GetRangeID()] = item diff --git a/pkg/kv/kvserver/queue_helpers_testutil.go b/pkg/kv/kvserver/queue_helpers_testutil.go index 6840934b701d..ecf2a09dc229 100644 --- a/pkg/kv/kvserver/queue_helpers_testutil.go +++ b/pkg/kv/kvserver/queue_helpers_testutil.go @@ -25,9 +25,9 @@ func (bq *baseQueue) testingAdd( // register a process callback that will be invoked when the replica is enqueued // or processed. func (bq *baseQueue) testingAddWithCallback( - ctx context.Context, repl replicaInQueue, priority float64, callback processCallback, + ctx context.Context, repl replicaInQueue, priority float64, cb processCallback, ) (bool, error) { - return bq.addInternal(ctx, repl.Desc(), repl.ReplicaID(), priority, callback) + return bq.addInternal(ctx, repl.Desc(), repl.ReplicaID(), priority, cb) } func forceScanAndProcess(ctx context.Context, s *Store, q *baseQueue) error { From ccb5a008ceac59cf11798775fcf4ada38c3b84b0 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Tue, 2 Sep 2025 14:44:27 -0400 Subject: [PATCH 34/44] kvserver: call cb.onEnqueueResult in defer on errors Previously, cb.onEnqueueResult was invoked inline before returning errors, which was less robust and required explicit calls. This commit refactors the code to invoke onEnqueueResult in a defer statement when returning a non-nil error. Note that the function may still call cb.onEnqueueResult with non-nil errors even when no error is returned, since we want visibility into those cases as well. --- pkg/kv/kvserver/queue.go | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index 43d473851014..4f46ec1450bd 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -839,13 +839,17 @@ func (bq *baseQueue) addInternal( replicaID roachpb.ReplicaID, priority float64, cb processCallback, -) (bool, error) { +) (added bool, err error) { + defer func() { + if err != nil { + cb.onEnqueueResult(-1 /* indexOnHeap */, err) + } + }() // NB: this is intentionally outside of bq.mu to avoid having to consider // lock ordering constraints. if !desc.IsInitialized() { // We checked this above in MaybeAdd(), but we need to check it // again for Add(). - cb.onEnqueueResult(-1 /*indexOnHeap*/, errReplicaNotInitialized) return false, errReplicaNotInitialized } @@ -853,7 +857,6 @@ func (bq *baseQueue) addInternal( defer bq.mu.Unlock() if bq.mu.stopped { - cb.onEnqueueResult(-1 /*indexOnHeap*/, errQueueStopped) return false, errQueueStopped } @@ -866,7 +869,6 @@ func (bq *baseQueue) addInternal( if log.V(3) { log.Infof(ctx, "queue disabled") } - cb.onEnqueueResult(-1 /*indexOnHeap*/, errQueueDisabled) return false, errQueueDisabled } } From 293983ea09efdc70edcf14133f05238fa20da90c Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Tue, 2 Sep 2025 18:17:48 -0400 Subject: [PATCH 35/44] fixup! kvserver: treat priority update as a success with onEnqueueResult --- pkg/kv/kvserver/queue.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index 4f46ec1450bd..f6573c0d34cf 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -897,7 +897,7 @@ func (bq *baseQueue) addInternal( log.Infof(ctx, "updating priority: %0.3f -> %0.3f", item.priority, priority) } bq.mu.priorityQ.update(item, priority) - // item.index should be updated now based on heap property now. + // item.index should be updated now based on heap property now. cb.onEnqueueResult(item.index /*indexOnHeap*/, nil) } return false, nil From af9aa957e70a90a2edc12f722a94d4e487fb1e45 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Tue, 2 Sep 2025 17:32:08 -0400 Subject: [PATCH 36/44] kvserver: allow logs from callbacks up to 15 replicas per updateReplicationGauges Previously, logs from the decommission nudger were not gated by a vmodule and could become spammy when many replicas were decommissioned at a low nudger frequency. This commit introduces a per-store budget, allowing logs from callbacks for up to 15 replicas per updateReplicationGauges call. Drawbacks of this approach: - Replicas are not visited in a sorted order, so we may be opening the floodgates from 15 different replicas each iteration. - Once a replica is permitted to log, its future logs from callbacks are not restricted. - If EnqueueProblemRangeInReplicateQueueInterval is set too low, 1 and 2 may become worse. For 1, we could consider visit the replica set with WithReplicasInOrder. I'm not sure about the overhead here since updateReplicationGauges is called periodically when collecting metrics. Here are the reasons that I think this approach is acceptable for now: - onEnqueueResult is unlikely to be reinvoked for replicas already in the queue unless they are processing or in purgatory (both are short-lived states we want visibility into). Once processed, replicas are removed from the set. onProcessResult should be called at most twice. For replicas merely waiting in the queue, the callback is not invoked, since their priority should not be actively updated. - We could cap logging per maybeEnqueueProblemRange, but granting full logging permission for each replica simplifies reasoning and gives complete visibility for specific replias. - In practice, escalations show that slow decommissioning usually involves <15 ranges, and EnqueueProblemRangeInReplicateQueueInterval is typically large (~15 minutes). --- pkg/kv/kvserver/replica.go | 46 ++++++++++++++++++++++++++------------ pkg/kv/kvserver/store.go | 12 +++++++++- 2 files changed, 43 insertions(+), 15 deletions(-) diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 319cbd50afcb..7dee2560b040 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -2867,8 +2867,9 @@ func (r *Replica) RefreshLeaderlessWatcherUnavailableStateForTesting( // manner via the replica scanner, see #130199. This functionality is disabled // by default for this reason. func (r *Replica) maybeEnqueueProblemRange( - ctx context.Context, now time.Time, leaseValid, isLeaseholder bool, + ctx context.Context, now time.Time, leaseValid, isLeaseholder bool, shouldLog bool, ) { + // The method expects the caller to provide whether the lease is valid and // the replica is the leaseholder for the range, so that it can avoid // unnecessary work. We expect this method to be called in the context of @@ -2910,24 +2911,41 @@ func (r *Replica) maybeEnqueueProblemRange( r.store.replicateQueue.AddAsyncWithCallback(ctx, r, allocatorimpl.AllocatorReplaceDecommissioningVoter.Priority(), processCallback{ onEnqueueResult: func(indexOnHeap int, err error) { - if err != nil { - // TODO(wenyihu6): if we want to put these logs behind vmodule, move - // this function to another file so that we can avoid the spam on - // other logs. - log.KvDistribution.Infof(ctx, - "decommissioning nudger failed to enqueue range %v due to %v", r.Desc(), err) + if shouldLog { + if err != nil { + log.KvDistribution.Infof(ctx, + "decommissioning nudger failed to enqueue range %v due to %v", r.Desc(), err) + } else { + log.KvDistribution.Infof(ctx, + "decommissioning nudger successfully enqueued range %v at index %d", r.Desc(), indexOnHeap) + } } else { - log.KvDistribution.Infof(ctx, - "decommissioning nudger successfully enqueued range %v at index %d", r.Desc(), indexOnHeap) + if err != nil { + log.KvDistribution.VInfof(ctx, 2, + "decommissioning nudger failed to enqueue range %v due to %v", r.Desc(), err) + } else { + log.KvDistribution.VInfof(ctx, 2, + "decommissioning nudger successfully enqueued range %v at index %d", r.Desc(), indexOnHeap) + } } }, onProcessResult: func(err error) { - if err != nil { - log.KvDistribution.Infof(ctx, - "decommissioning nudger failed to process range %v due to %v", r.Desc(), err) + if shouldLog { + if err != nil { + log.KvDistribution.Infof(ctx, + "decommissioning nudger failed to process range %v due to %v", r.Desc(), err) + } else { + log.KvDistribution.Infof(ctx, + "decommissioning nudger successfully processed replica %s", r.Desc()) + } } else { - log.KvDistribution.Infof(ctx, - "decommissioning nudger successfully processed replica %s", r.Desc()) + if err != nil { + log.KvDistribution.VInfof(ctx, 2, + "decommissioning nudger failed to process range %v due to %v", r.Desc(), err) + } else { + log.KvDistribution.VInfof(ctx, 2, + "decommissioning nudger successfully processed replica %s", r.Desc()) + } } }, }) diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index b36ad53ef156..9a195a24a10b 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -3403,6 +3403,12 @@ func (s *Store) updateReplicationGauges(ctx context.Context) error { ioOverload, _ = s.ioThreshold.t.Score() s.ioThreshold.Unlock() + // TODO(wenyihu6): it would be nicer if we can sort the replicas so that we + // can always get the nudger story on the same set of replicas, will this + // introduce a lot of overhead? For now, it seems fine since we usually see < + // 15 ranges on decommission stall. + var logBudgetOnDecommissioningNudger = 15 + // We want to avoid having to read this multiple times during the replica // visiting, so load it once up front for all nodes. livenessMap := s.cfg.NodeLiveness.ScanNodeVitalityFromCache() @@ -3472,7 +3478,11 @@ func (s *Store) updateReplicationGauges(ctx context.Context) error { if metrics.Decommissioning { // NB: Enqueue is disabled by default from here and throttled async if // enabled. - rep.maybeEnqueueProblemRange(ctx, goNow, metrics.LeaseValid, metrics.Leaseholder) + shouldLog := logBudgetOnDecommissioningNudger > 0 + if shouldLog { + logBudgetOnDecommissioningNudger-- + } + rep.maybeEnqueueProblemRange(ctx, goNow, metrics.LeaseValid, metrics.Leaseholder, shouldLog) decommissioningRangeCount++ } } From a9194dcaacfb388076d844082eefafd6d7bd9754 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Wed, 3 Sep 2025 12:59:46 -0400 Subject: [PATCH 37/44] kvserver: rename shouldLog to maybeLog and change vlevel to a var This commit renames shouldLog to maybeLog in maybeEnqueueProblemRange and refactors vmodule logging level to use a dynamic variable instead of an if statement to choose between Info and VInfo. --- pkg/kv/kvserver/replica.go | 46 +++++++++++++------------------------- pkg/kv/kvserver/store.go | 6 ++--- 2 files changed, 18 insertions(+), 34 deletions(-) diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 7dee2560b040..e9ee3baadb16 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -2867,7 +2867,7 @@ func (r *Replica) RefreshLeaderlessWatcherUnavailableStateForTesting( // manner via the replica scanner, see #130199. This functionality is disabled // by default for this reason. func (r *Replica) maybeEnqueueProblemRange( - ctx context.Context, now time.Time, leaseValid, isLeaseholder bool, shouldLog bool, + ctx context.Context, now time.Time, leaseValid, isLeaseholder bool, maybeLog bool, ) { // The method expects the caller to provide whether the lease is valid and @@ -2908,44 +2908,28 @@ func (r *Replica) maybeEnqueueProblemRange( r.store.metrics.DecommissioningNudgerEnqueue.Inc(1) // TODO(dodeca12): Figure out a better way to track the // decommissioning nudger enqueue failures/errors. + level := log.Level(2) + if maybeLog { + level = log.Level(0) + } r.store.replicateQueue.AddAsyncWithCallback(ctx, r, allocatorimpl.AllocatorReplaceDecommissioningVoter.Priority(), processCallback{ onEnqueueResult: func(indexOnHeap int, err error) { - if shouldLog { - if err != nil { - log.KvDistribution.Infof(ctx, - "decommissioning nudger failed to enqueue range %v due to %v", r.Desc(), err) - } else { - log.KvDistribution.Infof(ctx, - "decommissioning nudger successfully enqueued range %v at index %d", r.Desc(), indexOnHeap) - } + if err != nil { + log.KvDistribution.VInfof(ctx, level, + "decommissioning nudger failed to enqueue range %v due to %v", r.Desc(), err) } else { - if err != nil { - log.KvDistribution.VInfof(ctx, 2, - "decommissioning nudger failed to enqueue range %v due to %v", r.Desc(), err) - } else { - log.KvDistribution.VInfof(ctx, 2, - "decommissioning nudger successfully enqueued range %v at index %d", r.Desc(), indexOnHeap) - } + log.KvDistribution.VInfof(ctx, level, + "decommissioning nudger successfully enqueued range %v at index %d", r.Desc(), indexOnHeap) } }, onProcessResult: func(err error) { - if shouldLog { - if err != nil { - log.KvDistribution.Infof(ctx, - "decommissioning nudger failed to process range %v due to %v", r.Desc(), err) - } else { - log.KvDistribution.Infof(ctx, - "decommissioning nudger successfully processed replica %s", r.Desc()) - } + if err != nil { + log.KvDistribution.VInfof(ctx, level, + "decommissioning nudger failed to process range %v due to %v", r.Desc(), err) } else { - if err != nil { - log.KvDistribution.VInfof(ctx, 2, - "decommissioning nudger failed to process range %v due to %v", r.Desc(), err) - } else { - log.KvDistribution.VInfof(ctx, 2, - "decommissioning nudger successfully processed replica %s", r.Desc()) - } + log.KvDistribution.VInfof(ctx, level, + "decommissioning nudger successfully processed replica %s", r.Desc()) } }, }) diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 9a195a24a10b..d039225a1117 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -3478,11 +3478,11 @@ func (s *Store) updateReplicationGauges(ctx context.Context) error { if metrics.Decommissioning { // NB: Enqueue is disabled by default from here and throttled async if // enabled. - shouldLog := logBudgetOnDecommissioningNudger > 0 - if shouldLog { + maybeLog := logBudgetOnDecommissioningNudger > 0 + if maybeLog { logBudgetOnDecommissioningNudger-- } - rep.maybeEnqueueProblemRange(ctx, goNow, metrics.LeaseValid, metrics.Leaseholder, shouldLog) + rep.maybeEnqueueProblemRange(ctx, goNow, metrics.LeaseValid, metrics.Leaseholder, maybeLog) decommissioningRangeCount++ } } From efcf9cec7577e1300fbb794e10a4f621a7916f99 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Sat, 30 Aug 2025 21:54:21 -0400 Subject: [PATCH 38/44] kvserver: improve observability with decommission nudger Previously, we added the decommissioning nudger which nudges the leaseholder replica of decommissioning ranges to enqueue themselves into the replicate queue for decommissioning. However, we are still observing extended decommission stall with the nudger enabled. Observability was limited, and we could not easily tell whether replicas were successfully enqueued or processed. This commit improves observability by adding four metrics to track the enqueue and processing results of the decommissioning nudger: ranges.decommissioning.nudger.{enqueue,process}.{success,failure}. --- docs/generated/metrics/metrics.html | 6 ++++- pkg/kv/kvserver/metrics.go | 38 ++++++++++++++++++++++++++++- pkg/kv/kvserver/replica.go | 4 +++ 3 files changed, 46 insertions(+), 2 deletions(-) diff --git a/docs/generated/metrics/metrics.html b/docs/generated/metrics/metrics.html index d15b63699e36..54ef0436df3c 100644 --- a/docs/generated/metrics/metrics.html +++ b/docs/generated/metrics/metrics.html @@ -582,7 +582,11 @@ STORAGErangesNumber of rangesRangesGAUGECOUNTAVGNONE STORAGEranges.decommissioningNumber of ranges with at lease one replica on a decommissioning nodeRangesGAUGECOUNTAVGNONE STORAGEranges.decommissioning.nudger.enqueueNumber of enqueued enqueues of a range for decommissioning by the decommissioning nudger. Note: This metric tracks when the nudger attempts to enqueue, but the replica might not end up being enqueued by the priority queue due to various filtering or failure conditions.RangesCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE -STORAGEranges.decommissioning.nudger.not_leaseholder_or_invalid_leaseNumber of enqueues of a range for decommissioning by the decommissioning nudger that were not the leaseholder or had an invalid leaseRangesCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEranges.decommissioning.nudger.enqueue.failureNumber of ranges that failed to enqueue at the replicate queueRangesCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEranges.decommissioning.nudger.enqueue.successNumber of ranges that were successfully enqueued by the decommisioning nudgerRangesCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEranges.decommissioning.nudger.not_leaseholder_or_invalid_leaseNumber of ranges that were not the leaseholder or had an invalid lease at the decommissioning nudgerRangesCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEranges.decommissioning.nudger.process.failureNumber of ranges enqueued by the decommissioning nudger that failed to process by the replicate queueRangesCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEranges.decommissioning.nudger.process.successNumber of ranges enqueued by the decommissioning nudger that were successfully processed by the replicate queueRangesCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEranges.overreplicatedNumber of ranges with more live replicas than the replication targetRangesGAUGECOUNTAVGNONE STORAGEranges.unavailableNumber of ranges with fewer live replicas than needed for quorumRangesGAUGECOUNTAVGNONE STORAGEranges.underreplicatedNumber of ranges with fewer live replicas than the replication targetRangesGAUGECOUNTAVGNONE diff --git a/pkg/kv/kvserver/metrics.go b/pkg/kv/kvserver/metrics.go index ad9702e47b9b..cbc0c9a9b38a 100644 --- a/pkg/kv/kvserver/metrics.go +++ b/pkg/kv/kvserver/metrics.go @@ -161,9 +161,37 @@ var ( Measurement: "Ranges", Unit: metric.Unit_COUNT, } + metaDecommissioningNudgerEnqueueSuccess = metric.Metadata{ + Name: "ranges.decommissioning.nudger.enqueue.success", + Help: "Number of ranges that were successfully enqueued by the decommisioning nudger", + Measurement: "Ranges", + Unit: metric.Unit_COUNT, + LabeledName: "ranges.decommissioning.nudger.enqueue.success", + } + metaDecommissioningNudgerEnqueueFailure = metric.Metadata{ + Name: "ranges.decommissioning.nudger.enqueue.failure", + Help: "Number of ranges that failed to enqueue at the replicate queue", + Measurement: "Ranges", + Unit: metric.Unit_COUNT, + LabeledName: "ranges.decommissioning.nudger.enqueue.failure", + } + metaDecommissioningNudgerProcessSuccess = metric.Metadata{ + Name: "ranges.decommissioning.nudger.process.success", + Help: "Number of ranges enqueued by the decommissioning nudger that were successfully processed by the replicate queue", + Measurement: "Ranges", + Unit: metric.Unit_COUNT, + LabeledName: "ranges.decommissioning.nudger.process.success", + } + metaDecommissioningNudgerProcessFailure = metric.Metadata{ + Name: "ranges.decommissioning.nudger.process.failure", + Help: "Number of ranges enqueued by the decommissioning nudger that failed to process by the replicate queue", + Measurement: "Ranges", + Unit: metric.Unit_COUNT, + LabeledName: "ranges.decommissioning.nudger.process.failure", + } metaDecommissioningNudgerNotLeaseholderOrInvalidLease = metric.Metadata{ Name: "ranges.decommissioning.nudger.not_leaseholder_or_invalid_lease", - Help: "Number of enqueues of a range for decommissioning by the decommissioning nudger that were not the leaseholder or had an invalid lease", + Help: "Number of ranges that were not the leaseholder or had an invalid lease at the decommissioning nudger", Measurement: "Ranges", Unit: metric.Unit_COUNT, } @@ -2679,6 +2707,10 @@ type StoreMetrics struct { // Decommissioning nudger metrics. DecommissioningNudgerEnqueue *metric.Counter + DecommissioningNudgerEnqueueSuccess *metric.Counter + DecommissioningNudgerEnqueueFailure *metric.Counter + DecommissioningNudgerProcessSuccess *metric.Counter + DecommissioningNudgerProcessFailure *metric.Counter DecommissioningNudgerNotLeaseholderOrInvalidLease *metric.Counter // Lease request metrics for successful and failed lease requests. These @@ -3392,6 +3424,10 @@ func newStoreMetrics(histogramWindow time.Duration) *StoreMetrics { // Decommissioning nuder metrics. DecommissioningNudgerEnqueue: metric.NewCounter(metaDecommissioningNudgerEnqueue), + DecommissioningNudgerEnqueueSuccess: metric.NewCounter(metaDecommissioningNudgerEnqueueSuccess), + DecommissioningNudgerEnqueueFailure: metric.NewCounter(metaDecommissioningNudgerEnqueueFailure), + DecommissioningNudgerProcessSuccess: metric.NewCounter(metaDecommissioningNudgerProcessSuccess), + DecommissioningNudgerProcessFailure: metric.NewCounter(metaDecommissioningNudgerProcessFailure), DecommissioningNudgerNotLeaseholderOrInvalidLease: metric.NewCounter(metaDecommissioningNudgerNotLeaseholderOrInvalidLease), // Lease request metrics. diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index e9ee3baadb16..7828039ca264 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -2918,18 +2918,22 @@ func (r *Replica) maybeEnqueueProblemRange( if err != nil { log.KvDistribution.VInfof(ctx, level, "decommissioning nudger failed to enqueue range %v due to %v", r.Desc(), err) + r.store.metrics.DecommissioningNudgerEnqueueFailure.Inc(1) } else { log.KvDistribution.VInfof(ctx, level, "decommissioning nudger successfully enqueued range %v at index %d", r.Desc(), indexOnHeap) + r.store.metrics.DecommissioningNudgerEnqueueSuccess.Inc(1) } }, onProcessResult: func(err error) { if err != nil { log.KvDistribution.VInfof(ctx, level, "decommissioning nudger failed to process range %v due to %v", r.Desc(), err) + r.store.metrics.DecommissioningNudgerProcessFailure.Inc(1) } else { log.KvDistribution.VInfof(ctx, level, "decommissioning nudger successfully processed replica %s", r.Desc()) + r.store.metrics.DecommissioningNudgerProcessSuccess.Inc(1) } }, }) From a7fc88cd4f87638f48f2f3f34da649b524195c1c Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Tue, 2 Sep 2025 20:22:20 -0400 Subject: [PATCH 39/44] kvserver: add enqueue metrics to base queue Previously, observability into base queue enqueuing was limited to pending queue length and process results. This commit adds enqueue-specific metrics for the replicate queue: - queue.replicate.enqueue.add: counts replicas successfully added to the queue - queue.replicate.enqueue.failedprecondition: counts replicas that failed the replicaCanBeProcessed precondition check - queue.replicate.enqueue.noaction: counts replicas skipped because ShouldQueue determined no action was needed - queue.replicate.enqueue.unexpectederror: counts replicas that were expected to be enqueued (ShouldQueue returned true or the caller attempted a direct enqueue) but failed due to unexpected errors --- docs/generated/metrics/metrics.html | 4 +++ pkg/kv/kvserver/metrics.go | 35 ++++++++++++++++++ pkg/kv/kvserver/queue.go | 56 +++++++++++++++++++++++++++-- pkg/kv/kvserver/replicate_queue.go | 20 ++++++----- 4 files changed, 104 insertions(+), 11 deletions(-) diff --git a/docs/generated/metrics/metrics.html b/docs/generated/metrics/metrics.html index 54ef0436df3c..c729fc73c950 100644 --- a/docs/generated/metrics/metrics.html +++ b/docs/generated/metrics/metrics.html @@ -420,6 +420,10 @@ STORAGEqueue.replicate.addreplica.errorNumber of failed replica additions processed by the replicate queueReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.addreplica.successNumber of successful replica additions processed by the replicate queueReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.addvoterreplicaNumber of voter replica additions attempted by the replicate queueReplica AdditionsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEqueue.replicate.enqueue.addNumber of replicas successfully added to the replicate queueReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEqueue.replicate.enqueue.failedpreconditionNumber of replicas that failed the precondition checks and were therefore not added to the replicate queueReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEqueue.replicate.enqueue.noactionNumber of replicas for which ShouldQueue determined no action was needed and were therefore not added to the replicate queueReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEqueue.replicate.enqueue.unexpectederrorNumber of replicas that were expected to be enqueued (ShouldQueue returned true or the caller decided to add to the replicate queue directly), but failed to be enqueued due to unexpected errorsReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.nonvoterpromotionsNumber of non-voters promoted to voters by the replicate queuePromotions of Non Voters to VotersCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.pendingNumber of pending replicas in the replicate queueReplicasGAUGECOUNTAVGNONE STORAGEqueue.replicate.process.failureNumber of replicas which failed processing in the replicate queueReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE diff --git a/pkg/kv/kvserver/metrics.go b/pkg/kv/kvserver/metrics.go index cbc0c9a9b38a..971b38a2e990 100644 --- a/pkg/kv/kvserver/metrics.go +++ b/pkg/kv/kvserver/metrics.go @@ -2032,6 +2032,33 @@ The messages are dropped to help these replicas to recover from I/O overload.`, Measurement: "Processing Time", Unit: metric.Unit_NANOSECONDS, } + metaReplicateQueueEnqueueAdd = metric.Metadata{ + Name: "queue.replicate.enqueue.add", + Help: "Number of replicas successfully added to the replicate queue", + Measurement: "Replicas", + Unit: metric.Unit_COUNT, + } + metaReplicateQueueEnqueueFailedPrecondition = metric.Metadata{ + Name: "queue.replicate.enqueue.failedprecondition", + Help: "Number of replicas that failed the precondition checks and were therefore not added to the replicate " + + "queue", + Measurement: "Replicas", + Unit: metric.Unit_COUNT, + } + metaReplicateQueueEnqueueNoAction = metric.Metadata{ + Name: "queue.replicate.enqueue.noaction", + Help: "Number of replicas for which ShouldQueue determined no action was needed and were therefore not " + + "added to the replicate queue", + Measurement: "Replicas", + Unit: metric.Unit_COUNT, + } + metaReplicateQueueEnqueueUnexpectedError = metric.Metadata{ + Name: "queue.replicate.enqueue.unexpectederror", + Help: "Number of replicas that were expected to be enqueued (ShouldQueue returned true or the caller decided to " + + "add to the replicate queue directly), but failed to be enqueued due to unexpected errors", + Measurement: "Replicas", + Unit: metric.Unit_COUNT, + } metaLeaseQueueSuccesses = metric.Metadata{ Name: "queue.lease.process.success", Help: "Number of replicas successfully processed by the replica lease queue", @@ -2990,6 +3017,10 @@ type StoreMetrics struct { ReplicaGCQueueFailures *metric.Counter ReplicaGCQueuePending *metric.Gauge ReplicaGCQueueProcessingNanos *metric.Counter + ReplicateQueueEnqueueAdd *metric.Counter + ReplicateQueueEnqueueFailedPrecondition *metric.Counter + ReplicateQueueEnqueueNoAction *metric.Counter + ReplicateQueueEnqueueUnexpectedError *metric.Counter ReplicateQueueSuccesses *metric.Counter ReplicateQueueFailures *metric.Counter ReplicateQueuePending *metric.Gauge @@ -3773,6 +3804,10 @@ func newStoreMetrics(histogramWindow time.Duration) *StoreMetrics { ReplicaGCQueueFailures: metric.NewCounter(metaReplicaGCQueueFailures), ReplicaGCQueuePending: metric.NewGauge(metaReplicaGCQueuePending), ReplicaGCQueueProcessingNanos: metric.NewCounter(metaReplicaGCQueueProcessingNanos), + ReplicateQueueEnqueueAdd: metric.NewCounter(metaReplicateQueueEnqueueAdd), + ReplicateQueueEnqueueFailedPrecondition: metric.NewCounter(metaReplicateQueueEnqueueFailedPrecondition), + ReplicateQueueEnqueueNoAction: metric.NewCounter(metaReplicateQueueEnqueueNoAction), + ReplicateQueueEnqueueUnexpectedError: metric.NewCounter(metaReplicateQueueEnqueueUnexpectedError), ReplicateQueueSuccesses: metric.NewCounter(metaReplicateQueueSuccesses), ReplicateQueueFailures: metric.NewCounter(metaReplicateQueueFailures), ReplicateQueuePending: metric.NewGauge(metaReplicateQueuePending), diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index f6573c0d34cf..c380f615643a 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -388,7 +388,11 @@ type queueConfig struct { // replicas that have been destroyed but not GCed. processDestroyedReplicas bool // processTimeout returns the timeout for processing a replica. - processTimeoutFunc queueProcessTimeoutFunc + processTimeoutFunc queueProcessTimeoutFunc + enqueueAdd *metric.Counter + enqueueFailedPrecondition *metric.Counter + enqueueNoAction *metric.Counter + enqueueUnexpectedError *metric.Counter // successes is a counter of replicas processed successfully. successes *metric.Counter // failures is a counter of replicas which failed processing. @@ -733,6 +737,7 @@ func (bq *baseQueue) AddAsyncWithCallback( h.Add(ctx, repl, prio, cb) }); err != nil { cb.onEnqueueResult(-1 /*indexOnHeap*/, err) + bq.updateMetricsOnEnqueueUnexpectedError() } } @@ -740,9 +745,46 @@ func (bq *baseQueue) AddAsyncWithCallback( // for other operations to finish instead of turning into a noop (because // unlikely MaybeAdd, Add is not subject to being called opportunistically). func (bq *baseQueue) AddAsync(ctx context.Context, repl replicaInQueue, prio float64) { - _ = bq.Async(ctx, "Add", true /* wait */, func(ctx context.Context, h queueHelper) { + if err := bq.Async(ctx, "Add", true /* wait */, func(ctx context.Context, h queueHelper) { h.Add(ctx, repl, prio, noopProcessCallback) - }) + }); err != nil { + // We don't update metrics in MaybeAddAsync because we don't know if the + // replica should be queued at this point. We only count it as an unexpected + // error when we're certain the replica should be enqueued. In this case, + // the caller explicitly wants to add the replica to the queue directly, so + // we do update the metrics on unexpected error. + bq.updateMetricsOnEnqueueUnexpectedError() + } +} + +// updateMetricsOnEnqueueFailedPrecondition updates the metrics when a replica +// fails precondition checks (replicaCanBeProcessed) and should not be +// considered for enqueueing. This may include cases where the replica does not +// have a valid lease, is uninitialized, is destroyed, failed to retrieve span +// conf reader, or unsplit ranges. +func (bq *baseQueue) updateMetricsOnEnqueueFailedPrecondition() { + if bq.enqueueFailedPrecondition != nil { + bq.enqueueFailedPrecondition.Inc(1) + } +} + +// updateMetricsOnEnqueueNoAction updates the metrics when shouldQueue +// determines no action is needed and the replica is not added to the queue. +func (bq *baseQueue) updateMetricsOnEnqueueNoAction() { + if bq.enqueueNoAction != nil { + bq.enqueueNoAction.Inc(1) + } +} + +// updateMetricsOnEnqueueUnexpectedError updates the metrics when an unexpected +// error occurs during enqueue operations. This should be called for replicas +// that were expected to be enqueued (either had ShouldQueue return true or the +// caller explicitly requested to be added to the queue directly), but failed to +// be enqueued during the enqueue process (such as Async was rated limited). +func (bq *baseQueue) updateMetricsOnEnqueueUnexpectedError() { + if bq.enqueueUnexpectedError != nil { + bq.enqueueUnexpectedError.Inc(1) + } } func (bq *baseQueue) maybeAdd(ctx context.Context, repl replicaInQueue, now hlc.ClockTimestamp) { @@ -779,6 +821,7 @@ func (bq *baseQueue) maybeAdd(ctx context.Context, repl replicaInQueue, now hlc. // Load the system config if it's needed. confReader, err := bq.replicaCanBeProcessed(ctx, repl, false /* acquireLeaseIfNeeded */) if err != nil { + bq.updateMetricsOnEnqueueFailedPrecondition() return } @@ -788,6 +831,7 @@ func (bq *baseQueue) maybeAdd(ctx context.Context, repl replicaInQueue, now hlc. realRepl, _ := repl.(*Replica) should, priority := bq.impl.shouldQueue(ctx, now, realRepl, confReader) if !should { + bq.updateMetricsOnEnqueueNoAction() return } @@ -795,10 +839,12 @@ func (bq *baseQueue) maybeAdd(ctx context.Context, repl replicaInQueue, now hlc. if extConf != nil && extConf.Get(&bq.store.cfg.Settings.SV) { hasExternal, err := realRepl.HasExternalBytes() if err != nil { + bq.updateMetricsOnEnqueueUnexpectedError() log.Warningf(ctx, "could not determine if %s has external bytes: %s", realRepl, err) return } if hasExternal { + bq.updateMetricsOnEnqueueUnexpectedError() log.VInfof(ctx, 1, "skipping %s for %s because it has external bytes", bq.name, realRepl) return } @@ -841,8 +887,12 @@ func (bq *baseQueue) addInternal( cb processCallback, ) (added bool, err error) { defer func() { + if added && bq.enqueueAdd != nil { + bq.enqueueAdd.Inc(1) + } if err != nil { cb.onEnqueueResult(-1 /* indexOnHeap */, err) + bq.updateMetricsOnEnqueueUnexpectedError() } }() // NB: this is intentionally outside of bq.mu to avoid having to consider diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index 84a976e9ee13..1aa2c7702882 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -603,14 +603,18 @@ func newReplicateQueue(store *Store, allocator allocatorimpl.Allocator) *replica // so we use the raftSnapshotQueueTimeoutFunc. This function sets a // timeout based on the range size and the sending rate in addition // to consulting the setting which controls the minimum timeout. - processTimeoutFunc: makeRateLimitedTimeoutFunc(rebalanceSnapshotRate), - successes: store.metrics.ReplicateQueueSuccesses, - failures: store.metrics.ReplicateQueueFailures, - pending: store.metrics.ReplicateQueuePending, - full: store.metrics.ReplicateQueueFull, - processingNanos: store.metrics.ReplicateQueueProcessingNanos, - purgatory: store.metrics.ReplicateQueuePurgatory, - disabledConfig: kvserverbase.ReplicateQueueEnabled, + processTimeoutFunc: makeRateLimitedTimeoutFunc(rebalanceSnapshotRate), + enqueueAdd: store.metrics.ReplicateQueueEnqueueAdd, + enqueueFailedPrecondition: store.metrics.ReplicateQueueEnqueueFailedPrecondition, + enqueueNoAction: store.metrics.ReplicateQueueEnqueueNoAction, + enqueueUnexpectedError: store.metrics.ReplicateQueueEnqueueUnexpectedError, + successes: store.metrics.ReplicateQueueSuccesses, + failures: store.metrics.ReplicateQueueFailures, + pending: store.metrics.ReplicateQueuePending, + full: store.metrics.ReplicateQueueFull, + processingNanos: store.metrics.ReplicateQueueProcessingNanos, + purgatory: store.metrics.ReplicateQueuePurgatory, + disabledConfig: kvserverbase.ReplicateQueueEnabled, }, ) rq.baseQueue.SetMaxSize(ReplicateQueueMaxSize.Get(&store.cfg.Settings.SV)) From 094259eabd87eda3a7c496704d6f25f164e8a7db Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Wed, 3 Sep 2025 14:15:56 -0400 Subject: [PATCH 40/44] kvserver: move bq.enqueueAdd update to be outside of defer Previously, we updated bq.enqueueAdd inside the defer statement of addInternal. This was incorrect because we may return queued = true for a replica already processing and was marked for requeue. That replica would later be requeued in finishProcessingReplica, incrementing the metric again, lead to double counting. --- pkg/kv/kvserver/queue.go | 34 +++++++++++++++++++++++++++------- 1 file changed, 27 insertions(+), 7 deletions(-) diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index c380f615643a..3d1208a6bbbd 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -388,11 +388,21 @@ type queueConfig struct { // replicas that have been destroyed but not GCed. processDestroyedReplicas bool // processTimeout returns the timeout for processing a replica. - processTimeoutFunc queueProcessTimeoutFunc - enqueueAdd *metric.Counter + processTimeoutFunc queueProcessTimeoutFunc + // enqueueAdd is a counter of replicas that were successfully added to the + // queue. + enqueueAdd *metric.Counter + // enqueueFailedPrecondition is a counter of replicas that failed the + // precondition checks and were therefore not added to the queue. enqueueFailedPrecondition *metric.Counter - enqueueNoAction *metric.Counter - enqueueUnexpectedError *metric.Counter + // enqueueNoAction is a counter of replicas that had ShouldQueue determine no + // action was needed and were therefore not added to the queue. + enqueueNoAction *metric.Counter + // enqueueUnexpectedError is a counter of replicas that were expected to be + // enqueued (either had ShouldQueue return true or the caller explicitly + // requested to be added to the queue directly), but failed to be enqueued + // during the enqueue process (such as Async was rated limited). + enqueueUnexpectedError *metric.Counter // successes is a counter of replicas processed successfully. successes *metric.Counter // failures is a counter of replicas which failed processing. @@ -787,6 +797,14 @@ func (bq *baseQueue) updateMetricsOnEnqueueUnexpectedError() { } } +// updateMetricsOnEnqueueAdd updates the metrics when a replica is successfully +// added to the queue. +func (bq *baseQueue) updateMetricsOnEnqueueAdd() { + if bq.enqueueAdd != nil { + bq.enqueueAdd.Inc(1) + } +} + func (bq *baseQueue) maybeAdd(ctx context.Context, repl replicaInQueue, now hlc.ClockTimestamp) { ctx = repl.AnnotateCtx(ctx) ctx = bq.AnnotateCtx(ctx) @@ -887,9 +905,7 @@ func (bq *baseQueue) addInternal( cb processCallback, ) (added bool, err error) { defer func() { - if added && bq.enqueueAdd != nil { - bq.enqueueAdd.Inc(1) - } + // INVARIANT: added => err == nil. if err != nil { cb.onEnqueueResult(-1 /* indexOnHeap */, err) bq.updateMetricsOnEnqueueUnexpectedError() @@ -985,6 +1001,10 @@ func (bq *baseQueue) addInternal( default: // No need to signal again. } + // Note that we are bumping enqueueAdd here instead of during defer to avoid + // treating requeuing a processing replica as newly added. They will be + // re-added to the queue later which will double count them. + bq.updateMetricsOnEnqueueAdd() // Note: it may already be dropped or dropped afterwards. cb.onEnqueueResult(item.index /*indexOnHeap*/, nil) return true, nil From 26bacec73f0fdb69a9f835452152aba6eabe2c2b Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Wed, 3 Sep 2025 14:17:02 -0400 Subject: [PATCH 41/44] kvserver: test metrics in TestBaseQueueCallback* and TestReplicateQueueDecommissionScannerDisabled his commit extends TestBaseQueueCallback* and TestReplicateQueueDecommissionScannerDisabled to also verify metric updates. --- pkg/kv/kvserver/queue_test.go | 22 ++++++++++++++++++++-- pkg/kv/kvserver/replicate_queue_test.go | 13 ++++++++++--- 2 files changed, 30 insertions(+), 5 deletions(-) diff --git a/pkg/kv/kvserver/queue_test.go b/pkg/kv/kvserver/queue_test.go index f0b545f5c1b3..bb5709cbbb5d 100644 --- a/pkg/kv/kvserver/queue_test.go +++ b/pkg/kv/kvserver/queue_test.go @@ -110,6 +110,8 @@ func makeTestBaseQueue(name string, impl queueImpl, store *Store, cfg queueConfi cfg.pending = metric.NewGauge(metric.Metadata{Name: "pending"}) cfg.processingNanos = metric.NewCounter(metric.Metadata{Name: "processingnanos"}) cfg.purgatory = metric.NewGauge(metric.Metadata{Name: "purgatory"}) + cfg.enqueueAdd = metric.NewCounter(metric.Metadata{Name: "enqueueadd"}) + cfg.enqueueUnexpectedError = metric.NewCounter(metric.Metadata{Name: "enqueueunexpectederror"}) cfg.disabledConfig = testQueueEnabled return newBaseQueue(name, impl, store, cfg) } @@ -1326,6 +1328,7 @@ func TestBaseQueueCallbackOnEnqueueResult(t *testing.T) { t.Fatal("unexpected call to onProcessResult") }, }) + require.Equal(t, bq.enqueueAdd.Count(), int64(1)) require.True(t, queued) }) @@ -1352,6 +1355,7 @@ func TestBaseQueueCallbackOnEnqueueResult(t *testing.T) { t.Fatal("unexpected call to onProcessResult") }, }) + require.Equal(t, int64(i+1), bq.enqueueAdd.Count()) require.True(t, queued) } // Set range id back to 1. @@ -1372,6 +1376,8 @@ func TestBaseQueueCallbackOnEnqueueResult(t *testing.T) { t.Fatal("unexpected call to onProcessResult") }, }) + require.Equal(t, int64(0), bq.enqueueAdd.Count()) + require.Equal(t, int64(1), bq.enqueueUnexpectedError.Count()) require.False(t, queued) }) t.Run("stopped", func(t *testing.T) { @@ -1390,6 +1396,8 @@ func TestBaseQueueCallbackOnEnqueueResult(t *testing.T) { }, }) require.False(t, queued) + require.Equal(t, int64(0), bq.enqueueAdd.Count()) + require.Equal(t, int64(1), bq.enqueueUnexpectedError.Count()) }) t.Run("alreadyqueued", func(t *testing.T) { @@ -1407,6 +1415,8 @@ func TestBaseQueueCallbackOnEnqueueResult(t *testing.T) { }, }) require.True(t, queued) + require.Equal(t, int64(1), bq.enqueueAdd.Count()) + require.Equal(t, int64(0), bq.enqueueUnexpectedError.Count()) // Inserting again on the same range id should fail. queued, _ = bq.testingAddWithCallback(ctx, r, 1.0, processCallback{ @@ -1419,6 +1429,8 @@ func TestBaseQueueCallbackOnEnqueueResult(t *testing.T) { }, }) require.False(t, queued) + require.Equal(t, int64(1), bq.enqueueAdd.Count()) + require.Equal(t, int64(0), bq.enqueueUnexpectedError.Count()) }) t.Run("purgatory", func(t *testing.T) { @@ -1442,6 +1454,8 @@ func TestBaseQueueCallbackOnEnqueueResult(t *testing.T) { }, }) require.False(t, queued) + require.Equal(t, int64(0), bq.enqueueAdd.Count()) + require.Equal(t, int64(0), bq.enqueueUnexpectedError.Count()) }) t.Run("processing", func(t *testing.T) { @@ -1453,7 +1467,7 @@ func TestBaseQueueCallbackOnEnqueueResult(t *testing.T) { item.setProcessing() bq.addLocked(item) // Inserting a range that is already being processed should not enqueue again. - requeued, _ := bq.testingAddWithCallback(ctx, r, 1.0, processCallback{ + markedAsRequeued, _ := bq.testingAddWithCallback(ctx, r, 1.0, processCallback{ onEnqueueResult: func(indexOnHeap int, err error) { require.Equal(t, -1, indexOnHeap) require.ErrorIs(t, err, errReplicaAlreadyProcessing) @@ -1462,7 +1476,9 @@ func TestBaseQueueCallbackOnEnqueueResult(t *testing.T) { t.Fatal("unexpected call to onProcessResult") }, }) - require.True(t, requeued) + require.True(t, markedAsRequeued) + require.Equal(t, int64(0), bq.enqueueAdd.Count()) + require.Equal(t, int64(0), bq.enqueueUnexpectedError.Count()) }) t.Run("fullqueue", func(t *testing.T) { testQueue := &testQueueImpl{} @@ -1482,6 +1498,8 @@ func TestBaseQueueCallbackOnEnqueueResult(t *testing.T) { }, }) require.True(t, queued) + require.Equal(t, int64(1), bq.enqueueAdd.Count()) + require.Equal(t, int64(0), bq.enqueueUnexpectedError.Count()) }) } diff --git a/pkg/kv/kvserver/replicate_queue_test.go b/pkg/kv/kvserver/replicate_queue_test.go index d4a697463258..2a0ff5a812cd 100644 --- a/pkg/kv/kvserver/replicate_queue_test.go +++ b/pkg/kv/kvserver/replicate_queue_test.go @@ -2484,8 +2484,10 @@ func TestReplicateQueueDecommissionScannerDisabled(t *testing.T) { value = store.Metrics().DecommissioningRangeCount.Value() case "enqueue": value = store.Metrics().DecommissioningNudgerEnqueue.Count() - case "not_leaseholder_or_invalid_lease": - value = store.Metrics().DecommissioningNudgerNotLeaseholderOrInvalidLease.Count() + case "enqueue_success": + value = store.Metrics().DecommissioningNudgerEnqueueSuccess.Count() + case "process_success": + value = store.Metrics().DecommissioningNudgerProcessSuccess.Count() default: t.Fatalf("unknown metric type: %s", metricType) } @@ -2506,9 +2508,10 @@ func TestReplicateQueueDecommissionScannerDisabled(t *testing.T) { // Wait for the enqueue logic to trigger and validate metrics were updated. testutils.SucceedsSoon(t, func() error { + // TODO(wenyihu6): is there a race condition here where we might not observe + // decommissioning_ranges increasing? afterDecommissioningRanges := getDecommissioningNudgerMetricValue(t, tc, "decommissioning_ranges") afterEnqueued := getDecommissioningNudgerMetricValue(t, tc, "enqueue") - if afterDecommissioningRanges <= initialDecommissioningRanges { return errors.New("expected DecommissioningRangeCount to increase") } @@ -2531,4 +2534,8 @@ func TestReplicateQueueDecommissionScannerDisabled(t *testing.T) { } return nil }) + afterEnqueueSuccess := getDecommissioningNudgerMetricValue(t, tc, "enqueue_success") + require.Greater(t, afterEnqueueSuccess, int64(0)) + afterProcessSuccess := getDecommissioningNudgerMetricValue(t, tc, "process_success") + require.Greater(t, afterProcessSuccess, int64(0)) } From 2c8290437eb8d8f73e5051f3a044f86609c742df Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Tue, 26 Aug 2025 23:26:51 -0400 Subject: [PATCH 42/44] kvserver: plumb enqueue time priority This commit plumbs the enqueue time priority into baseQueue.processReplica, enabling comparison between the priority at enqueue time and at processing time. For now, we pass -1 in all cases except when processing replicas directly from the base queue, where -1 signals that priority verification should be skipped. No logic change has been made yet to check for priority inversion; future commits will extend processReplica to validate that processing priority has not differed significantly from the enqueue time priority. --- docs/generated/metrics/metrics.html | 16 +++ pkg/kv/kvserver/replicate_queue.go | 182 ++++++++++++++++++++++++++++ 2 files changed, 198 insertions(+) diff --git a/docs/generated/metrics/metrics.html b/docs/generated/metrics/metrics.html index c729fc73c950..797e7b62b59b 100644 --- a/docs/generated/metrics/metrics.html +++ b/docs/generated/metrics/metrics.html @@ -426,6 +426,22 @@ STORAGEqueue.replicate.enqueue.unexpectederrorNumber of replicas that were expected to be enqueued (ShouldQueue returned true or the caller decided to add to the replicate queue directly), but failed to be enqueued due to unexpected errorsReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.nonvoterpromotionsNumber of non-voters promoted to voters by the replicate queuePromotions of Non Voters to VotersCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.pendingNumber of pending replicas in the replicate queueReplicasGAUGECOUNTAVGNONE +STORAGEqueue.replicate.priority_inversion.addnonvoterNumber of priority inversions in the replicate queue that resulted in add non-voter action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEqueue.replicate.priority_inversion.addvoterNumber of priority inversions in the replicate queue that resulted in add voter action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEqueue.replicate.priority_inversion.considerrebalanceNumber of priority inversions in the replicate queue that resulted in consider rebalance action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEqueue.replicate.priority_inversion.noopNumber of priority inversions in the replicate queue that resulted in noop action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEqueue.replicate.priority_inversion.rangeunavailableNumber of priority inversions in the replicate queue that resulted in range unavailable action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEqueue.replicate.priority_inversion.removedeadnonvoterNumber of priority inversions in the replicate queue that resulted in remove dead non-voter action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEqueue.replicate.priority_inversion.removedeadvoterNumber of priority inversions in the replicate queue that resulted in remove dead voter action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEqueue.replicate.priority_inversion.removedecommissioningnonvoterNumber of priority inversions in the replicate queue that resulted in remove decommissioning non-voter action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEqueue.replicate.priority_inversion.removedecommissioningvoterNumber of priority inversions in the replicate queue that resulted in remove decommissioning voter action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEqueue.replicate.priority_inversion.removenonvoterNumber of priority inversions in the replicate queue that resulted in remove non-voter action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEqueue.replicate.priority_inversion.removevoterNumber of priority inversions in the replicate queue that resulted in remove voter action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEqueue.replicate.priority_inversion.replacedeadnonvoterNumber of priority inversions in the replicate queue that resulted in replace dead non-voter action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEqueue.replicate.priority_inversion.replacedecommissioningnonvoterNumber of priority inversions in the replicate queue that resulted in replace decommissioning non-voter action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEqueue.replicate.priority_inversion.replacedecommissioningvoterNumber of priority inversions in the replicate queue that resulted in replace decommissioning voter action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEqueue.replicate.priority_inversion.requeueNumber of priority inversions in the replicate queue that resulted in requeuing of the replicas. A priority inversion occurs when the priority at processing time ends up being lower than at enqueue time. When the priority has changed from a high priority repair action to rebalance, the change is requeued to avoid unfairness.ReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +STORAGEqueue.replicate.priority_inversion.totalTotal number of priority inversions in the replicate queue. A priority inversion occurs when the priority at processing time ends up being lower than at enqueue timeReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.process.failureNumber of replicas which failed processing in the replicate queueReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.process.successNumber of replicas successfully processed by the replicate queueReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.processingnanosNanoseconds spent processing replicas in the replicate queueProcessing TimeCOUNTERNANOSECONDSAVGNON_NEGATIVE_DERIVATIVE diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index 1aa2c7702882..413ad43363ca 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -319,6 +319,106 @@ var ( Measurement: "Replicas", Unit: metric.Unit_COUNT, } + metaReplicateQueueRequeueDueToPriorityInversion = metric.Metadata{ + Name: "queue.replicate.priority_inversion.requeue", + Help: "Number of priority inversions in the replicate queue that resulted in requeuing of the replicas. " + + "A priority inversion occurs when the priority at processing time ends up being lower " + + "than at enqueue time. When the priority has changed from a high priority repair action to rebalance, " + + "the change is requeued to avoid unfairness.", + Measurement: "Replicas", + Unit: metric.Unit_COUNT, + } + metaReplicateQueuePriorityInversionTotal = metric.Metadata{ + Name: "queue.replicate.priority_inversion.total", + Help: "Total number of priority inversions in the replicate queue. " + + "A priority inversion occurs when the priority at processing time ends up being lower than at enqueue time", + Measurement: "Replicas", + Unit: metric.Unit_COUNT, + } + metaReplicateQueuePriorityInversionForAddVoterCount = metric.Metadata{ + Name: "queue.replicate.priority_inversion.addvoter", + Help: "Number of priority inversions in the replicate queue that resulted in add voter action during processing", + Measurement: "Replicas", + Unit: metric.Unit_COUNT, + } + metaReplicateQueuePriorityInversionForReplaceDecommissioningVoterCount = metric.Metadata{ + Name: "queue.replicate.priority_inversion.replacedecommissioningvoter", + Help: "Number of priority inversions in the replicate queue that resulted in replace decommissioning voter action during processing", + Measurement: "Replicas", + Unit: metric.Unit_COUNT, + } + metaReplicateQueuePriorityInversionForRemoveDeadVoterCount = metric.Metadata{ + Name: "queue.replicate.priority_inversion.removedeadvoter", + Help: "Number of priority inversions in the replicate queue that resulted in remove dead voter action during processing", + Measurement: "Replicas", + Unit: metric.Unit_COUNT, + } + metaReplicateQueuePriorityInversionForRemoveDecommissioningVoterCount = metric.Metadata{ + Name: "queue.replicate.priority_inversion.removedecommissioningvoter", + Help: "Number of priority inversions in the replicate queue that resulted in remove decommissioning voter action during processing", + Measurement: "Replicas", + Unit: metric.Unit_COUNT, + } + metaReplicateQueuePriorityInversionForRemoveVoterCount = metric.Metadata{ + Name: "queue.replicate.priority_inversion.removevoter", + Help: "Number of priority inversions in the replicate queue that resulted in remove voter action during processing", + Measurement: "Replicas", + Unit: metric.Unit_COUNT, + } + metaReplicateQueuePriorityInversionForReplaceDeadNonVoterCount = metric.Metadata{ + Name: "queue.replicate.priority_inversion.replacedeadnonvoter", + Help: "Number of priority inversions in the replicate queue that resulted in replace dead non-voter action during processing", + Measurement: "Replicas", + Unit: metric.Unit_COUNT, + } + metaReplicateQueuePriorityInversionForAddNonVoterCount = metric.Metadata{ + Name: "queue.replicate.priority_inversion.addnonvoter", + Help: "Number of priority inversions in the replicate queue that resulted in add non-voter action during processing", + Measurement: "Replicas", + Unit: metric.Unit_COUNT, + } + metaReplicateQueuePriorityInversionForReplaceDecommissioningNonVoterCount = metric.Metadata{ + Name: "queue.replicate.priority_inversion.replacedecommissioningnonvoter", + Help: "Number of priority inversions in the replicate queue that resulted in replace decommissioning non-voter action during processing", + Measurement: "Replicas", + Unit: metric.Unit_COUNT, + } + metaReplicateQueuePriorityInversionForRemoveDeadNonVoterCount = metric.Metadata{ + Name: "queue.replicate.priority_inversion.removedeadnonvoter", + Help: "Number of priority inversions in the replicate queue that resulted in remove dead non-voter action during processing", + Measurement: "Replicas", + Unit: metric.Unit_COUNT, + } + metaReplicateQueuePriorityInversionForRemoveDecommissioningNonVoterCount = metric.Metadata{ + Name: "queue.replicate.priority_inversion.removedecommissioningnonvoter", + Help: "Number of priority inversions in the replicate queue that resulted in remove decommissioning non-voter action during processing", + Measurement: "Replicas", + Unit: metric.Unit_COUNT, + } + metaReplicateQueuePriorityInversionForRemoveNonVoterCount = metric.Metadata{ + Name: "queue.replicate.priority_inversion.removenonvoter", + Help: "Number of priority inversions in the replicate queue that resulted in remove non-voter action during processing", + Measurement: "Replicas", + Unit: metric.Unit_COUNT, + } + metaReplicateQueuePriorityInversionForConsiderRebalance = metric.Metadata{ + Name: "queue.replicate.priority_inversion.considerrebalance", + Help: "Number of priority inversions in the replicate queue that resulted in consider rebalance action during processing", + Measurement: "Replicas", + Unit: metric.Unit_COUNT, + } + metaReplicateQueuePriorityInversionForRangeUnavailable = metric.Metadata{ + Name: "queue.replicate.priority_inversion.rangeunavailable", + Help: "Number of priority inversions in the replicate queue that resulted in range unavailable action during processing", + Measurement: "Replicas", + Unit: metric.Unit_COUNT, + } + metaReplicateQueuePriorityInversionForNoop = metric.Metadata{ + Name: "queue.replicate.priority_inversion.noop", + Help: "Number of priority inversions in the replicate queue that resulted in noop action during processing", + Measurement: "Replicas", + Unit: metric.Unit_COUNT, + } ) // quorumError indicates a retryable error condition which sends replicas being @@ -341,6 +441,7 @@ func (e *quorumError) Error() string { func (*quorumError) PurgatoryErrorMarker() {} // ReplicateQueueMetrics is the set of metrics for the replicate queue. +// TODO(wenyihu6): metrics initialization could be cleaned up by using a map. type ReplicateQueueMetrics struct { AddReplicaCount *metric.Counter AddVoterReplicaCount *metric.Counter @@ -378,6 +479,27 @@ type ReplicateQueueMetrics struct { // TODO(sarkesian): Consider adding metrics for AllocatorRemoveLearner, // AllocatorConsiderRebalance, and AllocatorFinalizeAtomicReplicationChange // allocator actions. + + // Priority Inversion. Not tracked for + // AllocatorFinalizeAtomicReplicationChange, AllocatorRemoveLearner, + // AllocatorReplaceDeadVoter since they are the highest priority actions and + // cannot be inverted. (17 total actions-3=14) + RequeueDueToPriorityInversion *metric.Counter + PriorityInversionTotal *metric.Counter + PriorityInversionForAddVoterCount *metric.Counter + PriorityInversionForReplaceDecommissioningVoterCount *metric.Counter + PriorityInversionForRemoveDeadVoterCount *metric.Counter + PriorityInversionForRemoveDecommissioningVoterCount *metric.Counter + PriorityInversionForRemoveVoterCount *metric.Counter + PriorityInversionForReplaceDeadNonVoterCount *metric.Counter + PriorityInversionForAddNonVoterCount *metric.Counter + PriorityInversionForReplaceDecommissioningNonVoterCount *metric.Counter + PriorityInversionForRemoveDeadNonVoterCount *metric.Counter + PriorityInversionForRemoveDecommissioningNonVoterCount *metric.Counter + PriorityInversionForRemoveNonVoterCount *metric.Counter + PriorityInversionForConsiderRebalance *metric.Counter + PriorityInversionForRangeUnavailable *metric.Counter + PriorityInversionForNoop *metric.Counter } func makeReplicateQueueMetrics() ReplicateQueueMetrics { @@ -414,6 +536,23 @@ func makeReplicateQueueMetrics() ReplicateQueueMetrics { ReplaceDecommissioningReplicaErrorCount: metric.NewCounter(metaReplicateQueueReplaceDecommissioningReplicaErrorCount), RemoveDecommissioningReplicaSuccessCount: metric.NewCounter(metaReplicateQueueRemoveDecommissioningReplicaSuccessCount), RemoveDecommissioningReplicaErrorCount: metric.NewCounter(metaReplicateQueueRemoveDecommissioningReplicaErrorCount), + + RequeueDueToPriorityInversion: metric.NewCounter(metaReplicateQueueRequeueDueToPriorityInversion), + PriorityInversionTotal: metric.NewCounter(metaReplicateQueuePriorityInversionTotal), + PriorityInversionForAddVoterCount: metric.NewCounter(metaReplicateQueuePriorityInversionForAddVoterCount), + PriorityInversionForReplaceDecommissioningVoterCount: metric.NewCounter(metaReplicateQueuePriorityInversionForReplaceDecommissioningVoterCount), + PriorityInversionForRemoveDeadVoterCount: metric.NewCounter(metaReplicateQueuePriorityInversionForRemoveDeadVoterCount), + PriorityInversionForRemoveDecommissioningVoterCount: metric.NewCounter(metaReplicateQueuePriorityInversionForRemoveDecommissioningVoterCount), + PriorityInversionForRemoveVoterCount: metric.NewCounter(metaReplicateQueuePriorityInversionForRemoveVoterCount), + PriorityInversionForReplaceDeadNonVoterCount: metric.NewCounter(metaReplicateQueuePriorityInversionForReplaceDeadNonVoterCount), + PriorityInversionForAddNonVoterCount: metric.NewCounter(metaReplicateQueuePriorityInversionForAddNonVoterCount), + PriorityInversionForReplaceDecommissioningNonVoterCount: metric.NewCounter(metaReplicateQueuePriorityInversionForReplaceDecommissioningNonVoterCount), + PriorityInversionForRemoveDeadNonVoterCount: metric.NewCounter(metaReplicateQueuePriorityInversionForRemoveDeadNonVoterCount), + PriorityInversionForRemoveDecommissioningNonVoterCount: metric.NewCounter(metaReplicateQueuePriorityInversionForRemoveDecommissioningNonVoterCount), + PriorityInversionForRemoveNonVoterCount: metric.NewCounter(metaReplicateQueuePriorityInversionForRemoveNonVoterCount), + PriorityInversionForConsiderRebalance: metric.NewCounter(metaReplicateQueuePriorityInversionForConsiderRebalance), + PriorityInversionForRangeUnavailable: metric.NewCounter(metaReplicateQueuePriorityInversionForRangeUnavailable), + PriorityInversionForNoop: metric.NewCounter(metaReplicateQueuePriorityInversionForNoop), } } @@ -537,6 +676,47 @@ func (metrics *ReplicateQueueMetrics) trackErrorByAllocatorAction( } +// trackPriorityInversion tracks the action that the replicate queue ended up +// processing when the priority at enqueue time was higher than the priority at +// processing time. +func (metrics *ReplicateQueueMetrics) trackPriorityInversion( + actionAtProcessingTime allocatorimpl.AllocatorAction, +) { + metrics.PriorityInversionTotal.Inc(1) + switch actionAtProcessingTime { + case allocatorimpl.AllocatorAddVoter: + metrics.PriorityInversionForAddVoterCount.Inc(1) + case allocatorimpl.AllocatorReplaceDecommissioningVoter: + metrics.PriorityInversionForReplaceDecommissioningVoterCount.Inc(1) + case allocatorimpl.AllocatorRemoveDeadVoter: + metrics.PriorityInversionForRemoveDeadVoterCount.Inc(1) + case allocatorimpl.AllocatorRemoveDecommissioningVoter: + metrics.PriorityInversionForRemoveDecommissioningVoterCount.Inc(1) + case allocatorimpl.AllocatorRemoveVoter: + metrics.PriorityInversionForRemoveVoterCount.Inc(1) + case allocatorimpl.AllocatorReplaceDeadNonVoter: + metrics.PriorityInversionForReplaceDeadNonVoterCount.Inc(1) + case allocatorimpl.AllocatorAddNonVoter: + metrics.PriorityInversionForAddNonVoterCount.Inc(1) + case allocatorimpl.AllocatorReplaceDecommissioningNonVoter: + metrics.PriorityInversionForReplaceDecommissioningNonVoterCount.Inc(1) + case allocatorimpl.AllocatorRemoveDeadNonVoter: + metrics.PriorityInversionForRemoveDeadNonVoterCount.Inc(1) + case allocatorimpl.AllocatorRemoveDecommissioningNonVoter: + metrics.PriorityInversionForRemoveDecommissioningNonVoterCount.Inc(1) + case allocatorimpl.AllocatorRemoveNonVoter: + metrics.PriorityInversionForRemoveNonVoterCount.Inc(1) + case allocatorimpl.AllocatorConsiderRebalance: + metrics.PriorityInversionForConsiderRebalance.Inc(1) + case allocatorimpl.AllocatorRangeUnavailable: + metrics.PriorityInversionForRangeUnavailable.Inc(1) + case allocatorimpl.AllocatorNoop: + metrics.PriorityInversionForNoop.Inc(1) + default: + panic("unhandled default case") + } +} + // trackProcessResult increases the corresponding success/error count metric for // processing a particular allocator action through the replicate queue. func (metrics *ReplicateQueueMetrics) trackResultByAllocatorAction( @@ -949,12 +1129,14 @@ func (rq *replicateQueue) processOneChange( // starving other higher priority work. if PriorityInversionRequeue.Get(&rq.store.cfg.Settings.SV) { if inversion, shouldRequeue := allocatorimpl.CheckPriorityInversion(priorityAtEnqueue, change.Action); inversion { + rq.metrics.trackPriorityInversion(change.Action) if priorityInversionLogEveryN.ShouldLog() { log.KvDistribution.Infof(ctx, "priority inversion during process: shouldRequeue = %t action=%s, priority=%v, enqueuePriority=%v", shouldRequeue, change.Action, change.Action.Priority(), priorityAtEnqueue) } if shouldRequeue { + rq.metrics.RequeueDueToPriorityInversion.Inc(1) // Return true to requeue the range. Return the error to ensure it is // logged and tracked in replicate queue bq.failures metrics. See // replicateQueue.process for details. From 7fcad47994953e95de4e52c766430114cbcdbfb7 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Wed, 27 Aug 2025 06:26:18 -0400 Subject: [PATCH 43/44] kvserver: add TestPriorityInversionRequeue MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Previously, we added priority inversion requeuing mechanism. This commit adds a unit test that forces the race condition we suspected to be happening in escalations involving priority inversion and asserts that priority inversion occurs and that the replica is correctly requeued. Test set up: 1. range’s leaseholder replica is rebalanced from one store to another. 2. new leaseholder enqueues the replica for repair with high priority (e.g. to finalize the atomic replication change or remove a learner replica) 3. before processing, the old leaseholder completes the change (exits the joint config or removes the learner). 4. when the new leaseholder processes the replica, it computes a ConsiderRebalance action, resulting in a priority inversion and potentially blocking other high-priority work. --- pkg/kv/kvserver/queue.go | 3 + pkg/kv/kvserver/replicate_queue_test.go | 116 ++++++++++++++++++++++++ pkg/kv/kvserver/testing_knobs.go | 4 + 3 files changed, 123 insertions(+) diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index 3d1208a6bbbd..9dd4df7dbb2f 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -1001,6 +1001,9 @@ func (bq *baseQueue) addInternal( default: // No need to signal again. } + if postEnqueueInterceptor := bq.store.TestingKnobs().BaseQueuePostEnqueueInterceptor; postEnqueueInterceptor != nil { + postEnqueueInterceptor(bq.store.StoreID(), desc.RangeID) + } // Note that we are bumping enqueueAdd here instead of during defer to avoid // treating requeuing a processing replica as newly added. They will be // re-added to the queue later which will double count them. diff --git a/pkg/kv/kvserver/replicate_queue_test.go b/pkg/kv/kvserver/replicate_queue_test.go index 2a0ff5a812cd..eb86f3bfb041 100644 --- a/pkg/kv/kvserver/replicate_queue_test.go +++ b/pkg/kv/kvserver/replicate_queue_test.go @@ -2539,3 +2539,119 @@ func TestReplicateQueueDecommissionScannerDisabled(t *testing.T) { afterProcessSuccess := getDecommissioningNudgerMetricValue(t, tc, "process_success") require.Greater(t, afterProcessSuccess, int64(0)) } + +// TestPriorityInversionRequeue tests that the replicate queue correctly handles +// priority inversions by requeuing replicas when the PriorityInversionRequeue +// setting is enabled. +// +// This test specifically targets a race condition where: +// 1. A replica is enqueued for a high-priority repair action +// (FinalizeAtomicReplicationChange or RemoveLearner). +// 2. By the time the replica is processed, the repair is no longer needed and +// only a low-priority rebalance action (ConsiderRebalance) is computed. +// 3. This creates a priority inversion where a low-priority action blocks +// other higher-priority replicas in the queue from being processed. +// +// The race occurs during range rebalancing: +// 1. A leaseholder replica of a range is rebalanced from one store to another. +// 2. The new leaseholder enqueues the replica for repair (e.g. to finalize +// the atomic replication change or remove a learner replica). +// 3. Before processing, the old leaseholder has left the atomic joint config +// state or removed the learner replica. 4. When the new leaseholder processes +// the replica, it computes a ConsiderRebalance action, causing priority +// inversion. +// +// With PriorityInversionRequeue enabled, the queue should detect this condition +// and requeue the replica at the correct priority. The test validates this +// behavior through metrics that track priority inversions and requeuing events. +func TestPriorityInversionRequeue(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + settings := cluster.MakeTestingClusterSettings() + kvserver.PriorityInversionRequeue.Override(ctx, &settings.SV, true) + + var scratchRangeID int64 + atomic.StoreInt64(&scratchRangeID, -1) + require.NoError(t, log.SetVModule("queue=5,replicate_queue=5,replica_command=5,replicate=5,replica=5")) + + const newLeaseholderStoreAndNodeID = 4 + var waitUntilLeavingJoint = func() {} + + tc := testcluster.StartTestCluster(t, 4, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Settings: settings, + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + BaseQueueDisabledBypassFilter: func(rangeID roachpb.RangeID) bool { + // Disable the replicate queue except for the scratch range on the new leaseholder. + t.Logf("range %d is added to replicate queue store", rangeID) + return rangeID == roachpb.RangeID(atomic.LoadInt64(&scratchRangeID)) + }, + BaseQueuePostEnqueueInterceptor: func(storeID roachpb.StoreID, rangeID roachpb.RangeID) { + // After enqueuing, wait for the old leaseholder to leave the atomic + // joint config state or remove the learner replica to force the + // priority inversion. + t.Logf("waiting for %d to leave joint config", rangeID) + if storeID == 4 && rangeID == roachpb.RangeID(atomic.LoadInt64(&scratchRangeID)) { + waitUntilLeavingJoint() + } + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + scratchKey := tc.ScratchRange(t) + + // Wait until the old leaseholder has left the atomic joint config state or + // removed the learner replica. + waitUntilLeavingJoint = func() { + testutils.SucceedsSoon(t, func() error { + rangeDesc := tc.LookupRangeOrFatal(t, scratchKey) + replicas := rangeDesc.Replicas() + t.Logf("range %v: waiting to leave joint conf", rangeDesc) + if replicas.InAtomicReplicationChange() || len(replicas.LearnerDescriptors()) != 0 { + return errors.Newf("in between atomic changes: %v", replicas) + } + return nil + }) + } + + scratchRange := tc.LookupRangeOrFatal(t, scratchKey) + tc.AddVotersOrFatal(t, scratchRange.StartKey.AsRawKey(), tc.Targets(1, 2)...) + atomic.StoreInt64(&scratchRangeID, int64(scratchRange.RangeID)) + lh, err := tc.FindRangeLeaseHolder(scratchRange, nil) + require.NoError(t, err) + + // Rebalance the leaseholder replica to a new store. This will cause the race + // condition where the new leaseholder can enqueue a replica to replicate + // queue with high priority but compute a low priority action at processing + // time. + t.Logf("rebalancing range %d from s%d to s%d", scratchRange, lh.StoreID, newLeaseholderStoreAndNodeID) + _, err = tc.RebalanceVoter( + ctx, + scratchRange.StartKey.AsRawKey(), + roachpb.ReplicationTarget{StoreID: lh.StoreID, NodeID: lh.NodeID}, /* src */ + roachpb.ReplicationTarget{StoreID: newLeaseholderStoreAndNodeID, NodeID: newLeaseholderStoreAndNodeID}, /* dest */ + ) + require.NoError(t, err) + + // Wait until the priority inversion is detected and the replica is requeued. + testutils.SucceedsSoon(t, func() error { + store := tc.GetFirstStoreFromServer(t, 3) + if c := store.ReplicateQueueMetrics().PriorityInversionTotal.Count(); c == 0 { + return errors.New("expected non-zero priority inversion total count but got 0") + } + if c := store.ReplicateQueueMetrics().PriorityInversionForConsiderRebalance.Count(); c == 0 { + return errors.New("expected non-zero priority inversion count for consider rebalance but got 0") + } + if c := store.ReplicateQueueMetrics().RequeueDueToPriorityInversion.Count(); c == 0 { + return errors.New("expected to requeue due to priority inversion but got 0") + } + return nil + }) +} diff --git a/pkg/kv/kvserver/testing_knobs.go b/pkg/kv/kvserver/testing_knobs.go index 6c5dad7718ad..df4a4ed35a6d 100644 --- a/pkg/kv/kvserver/testing_knobs.go +++ b/pkg/kv/kvserver/testing_knobs.go @@ -530,6 +530,10 @@ type StoreTestingKnobs struct { // rangeID should ignore the queue being disabled, and be processed anyway. BaseQueueDisabledBypassFilter func(rangeID roachpb.RangeID) bool + // BaseQueuePostEnqueueInterceptor is called with the storeID and rangeID of + // the replica right after a replica is enqueued (before it is processed) + BaseQueuePostEnqueueInterceptor func(storeID roachpb.StoreID, rangeID roachpb.RangeID) + // InjectReproposalError injects an error in tryReproposeWithNewLeaseIndexRaftMuLocked. // If nil is returned, reproposal will be attempted. InjectReproposalError func(p *ProposalData) error From a93cbd81d293ad839b7dd7a356a2bd2693a7bfef Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Mon, 1 Sep 2025 14:30:43 -0400 Subject: [PATCH 44/44] kvserver: delete per action priority inversion metrics This commit removes per-action priority inversion metrics due to their high cardinality. We already have logging in place, which should provide sufficient observability. For now, we care about is priority inversion that leads to consider rebalance and requeuing the most. --- docs/generated/metrics/metrics.html | 14 -- pkg/kv/kvserver/replicate_queue.go | 168 +----------------------- pkg/kv/kvserver/replicate_queue_test.go | 4 +- 3 files changed, 7 insertions(+), 179 deletions(-) diff --git a/docs/generated/metrics/metrics.html b/docs/generated/metrics/metrics.html index 797e7b62b59b..50d99410f8fe 100644 --- a/docs/generated/metrics/metrics.html +++ b/docs/generated/metrics/metrics.html @@ -426,20 +426,6 @@ STORAGEqueue.replicate.enqueue.unexpectederrorNumber of replicas that were expected to be enqueued (ShouldQueue returned true or the caller decided to add to the replicate queue directly), but failed to be enqueued due to unexpected errorsReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.nonvoterpromotionsNumber of non-voters promoted to voters by the replicate queuePromotions of Non Voters to VotersCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.pendingNumber of pending replicas in the replicate queueReplicasGAUGECOUNTAVGNONE -STORAGEqueue.replicate.priority_inversion.addnonvoterNumber of priority inversions in the replicate queue that resulted in add non-voter action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE -STORAGEqueue.replicate.priority_inversion.addvoterNumber of priority inversions in the replicate queue that resulted in add voter action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE -STORAGEqueue.replicate.priority_inversion.considerrebalanceNumber of priority inversions in the replicate queue that resulted in consider rebalance action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE -STORAGEqueue.replicate.priority_inversion.noopNumber of priority inversions in the replicate queue that resulted in noop action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE -STORAGEqueue.replicate.priority_inversion.rangeunavailableNumber of priority inversions in the replicate queue that resulted in range unavailable action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE -STORAGEqueue.replicate.priority_inversion.removedeadnonvoterNumber of priority inversions in the replicate queue that resulted in remove dead non-voter action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE -STORAGEqueue.replicate.priority_inversion.removedeadvoterNumber of priority inversions in the replicate queue that resulted in remove dead voter action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE -STORAGEqueue.replicate.priority_inversion.removedecommissioningnonvoterNumber of priority inversions in the replicate queue that resulted in remove decommissioning non-voter action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE -STORAGEqueue.replicate.priority_inversion.removedecommissioningvoterNumber of priority inversions in the replicate queue that resulted in remove decommissioning voter action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE -STORAGEqueue.replicate.priority_inversion.removenonvoterNumber of priority inversions in the replicate queue that resulted in remove non-voter action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE -STORAGEqueue.replicate.priority_inversion.removevoterNumber of priority inversions in the replicate queue that resulted in remove voter action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE -STORAGEqueue.replicate.priority_inversion.replacedeadnonvoterNumber of priority inversions in the replicate queue that resulted in replace dead non-voter action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE -STORAGEqueue.replicate.priority_inversion.replacedecommissioningnonvoterNumber of priority inversions in the replicate queue that resulted in replace decommissioning non-voter action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE -STORAGEqueue.replicate.priority_inversion.replacedecommissioningvoterNumber of priority inversions in the replicate queue that resulted in replace decommissioning voter action during processingReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.priority_inversion.requeueNumber of priority inversions in the replicate queue that resulted in requeuing of the replicas. A priority inversion occurs when the priority at processing time ends up being lower than at enqueue time. When the priority has changed from a high priority repair action to rebalance, the change is requeued to avoid unfairness.ReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.priority_inversion.totalTotal number of priority inversions in the replicate queue. A priority inversion occurs when the priority at processing time ends up being lower than at enqueue timeReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE STORAGEqueue.replicate.process.failureNumber of replicas which failed processing in the replicate queueReplicasCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index 413ad43363ca..7d48cac86f8c 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -335,90 +335,6 @@ var ( Measurement: "Replicas", Unit: metric.Unit_COUNT, } - metaReplicateQueuePriorityInversionForAddVoterCount = metric.Metadata{ - Name: "queue.replicate.priority_inversion.addvoter", - Help: "Number of priority inversions in the replicate queue that resulted in add voter action during processing", - Measurement: "Replicas", - Unit: metric.Unit_COUNT, - } - metaReplicateQueuePriorityInversionForReplaceDecommissioningVoterCount = metric.Metadata{ - Name: "queue.replicate.priority_inversion.replacedecommissioningvoter", - Help: "Number of priority inversions in the replicate queue that resulted in replace decommissioning voter action during processing", - Measurement: "Replicas", - Unit: metric.Unit_COUNT, - } - metaReplicateQueuePriorityInversionForRemoveDeadVoterCount = metric.Metadata{ - Name: "queue.replicate.priority_inversion.removedeadvoter", - Help: "Number of priority inversions in the replicate queue that resulted in remove dead voter action during processing", - Measurement: "Replicas", - Unit: metric.Unit_COUNT, - } - metaReplicateQueuePriorityInversionForRemoveDecommissioningVoterCount = metric.Metadata{ - Name: "queue.replicate.priority_inversion.removedecommissioningvoter", - Help: "Number of priority inversions in the replicate queue that resulted in remove decommissioning voter action during processing", - Measurement: "Replicas", - Unit: metric.Unit_COUNT, - } - metaReplicateQueuePriorityInversionForRemoveVoterCount = metric.Metadata{ - Name: "queue.replicate.priority_inversion.removevoter", - Help: "Number of priority inversions in the replicate queue that resulted in remove voter action during processing", - Measurement: "Replicas", - Unit: metric.Unit_COUNT, - } - metaReplicateQueuePriorityInversionForReplaceDeadNonVoterCount = metric.Metadata{ - Name: "queue.replicate.priority_inversion.replacedeadnonvoter", - Help: "Number of priority inversions in the replicate queue that resulted in replace dead non-voter action during processing", - Measurement: "Replicas", - Unit: metric.Unit_COUNT, - } - metaReplicateQueuePriorityInversionForAddNonVoterCount = metric.Metadata{ - Name: "queue.replicate.priority_inversion.addnonvoter", - Help: "Number of priority inversions in the replicate queue that resulted in add non-voter action during processing", - Measurement: "Replicas", - Unit: metric.Unit_COUNT, - } - metaReplicateQueuePriorityInversionForReplaceDecommissioningNonVoterCount = metric.Metadata{ - Name: "queue.replicate.priority_inversion.replacedecommissioningnonvoter", - Help: "Number of priority inversions in the replicate queue that resulted in replace decommissioning non-voter action during processing", - Measurement: "Replicas", - Unit: metric.Unit_COUNT, - } - metaReplicateQueuePriorityInversionForRemoveDeadNonVoterCount = metric.Metadata{ - Name: "queue.replicate.priority_inversion.removedeadnonvoter", - Help: "Number of priority inversions in the replicate queue that resulted in remove dead non-voter action during processing", - Measurement: "Replicas", - Unit: metric.Unit_COUNT, - } - metaReplicateQueuePriorityInversionForRemoveDecommissioningNonVoterCount = metric.Metadata{ - Name: "queue.replicate.priority_inversion.removedecommissioningnonvoter", - Help: "Number of priority inversions in the replicate queue that resulted in remove decommissioning non-voter action during processing", - Measurement: "Replicas", - Unit: metric.Unit_COUNT, - } - metaReplicateQueuePriorityInversionForRemoveNonVoterCount = metric.Metadata{ - Name: "queue.replicate.priority_inversion.removenonvoter", - Help: "Number of priority inversions in the replicate queue that resulted in remove non-voter action during processing", - Measurement: "Replicas", - Unit: metric.Unit_COUNT, - } - metaReplicateQueuePriorityInversionForConsiderRebalance = metric.Metadata{ - Name: "queue.replicate.priority_inversion.considerrebalance", - Help: "Number of priority inversions in the replicate queue that resulted in consider rebalance action during processing", - Measurement: "Replicas", - Unit: metric.Unit_COUNT, - } - metaReplicateQueuePriorityInversionForRangeUnavailable = metric.Metadata{ - Name: "queue.replicate.priority_inversion.rangeunavailable", - Help: "Number of priority inversions in the replicate queue that resulted in range unavailable action during processing", - Measurement: "Replicas", - Unit: metric.Unit_COUNT, - } - metaReplicateQueuePriorityInversionForNoop = metric.Metadata{ - Name: "queue.replicate.priority_inversion.noop", - Help: "Number of priority inversions in the replicate queue that resulted in noop action during processing", - Measurement: "Replicas", - Unit: metric.Unit_COUNT, - } ) // quorumError indicates a retryable error condition which sends replicas being @@ -480,26 +396,9 @@ type ReplicateQueueMetrics struct { // AllocatorConsiderRebalance, and AllocatorFinalizeAtomicReplicationChange // allocator actions. - // Priority Inversion. Not tracked for - // AllocatorFinalizeAtomicReplicationChange, AllocatorRemoveLearner, - // AllocatorReplaceDeadVoter since they are the highest priority actions and - // cannot be inverted. (17 total actions-3=14) - RequeueDueToPriorityInversion *metric.Counter - PriorityInversionTotal *metric.Counter - PriorityInversionForAddVoterCount *metric.Counter - PriorityInversionForReplaceDecommissioningVoterCount *metric.Counter - PriorityInversionForRemoveDeadVoterCount *metric.Counter - PriorityInversionForRemoveDecommissioningVoterCount *metric.Counter - PriorityInversionForRemoveVoterCount *metric.Counter - PriorityInversionForReplaceDeadNonVoterCount *metric.Counter - PriorityInversionForAddNonVoterCount *metric.Counter - PriorityInversionForReplaceDecommissioningNonVoterCount *metric.Counter - PriorityInversionForRemoveDeadNonVoterCount *metric.Counter - PriorityInversionForRemoveDecommissioningNonVoterCount *metric.Counter - PriorityInversionForRemoveNonVoterCount *metric.Counter - PriorityInversionForConsiderRebalance *metric.Counter - PriorityInversionForRangeUnavailable *metric.Counter - PriorityInversionForNoop *metric.Counter + // Priority Inversion. + RequeueDueToPriorityInversion *metric.Counter + PriorityInversionTotal *metric.Counter } func makeReplicateQueueMetrics() ReplicateQueueMetrics { @@ -537,22 +436,8 @@ func makeReplicateQueueMetrics() ReplicateQueueMetrics { RemoveDecommissioningReplicaSuccessCount: metric.NewCounter(metaReplicateQueueRemoveDecommissioningReplicaSuccessCount), RemoveDecommissioningReplicaErrorCount: metric.NewCounter(metaReplicateQueueRemoveDecommissioningReplicaErrorCount), - RequeueDueToPriorityInversion: metric.NewCounter(metaReplicateQueueRequeueDueToPriorityInversion), - PriorityInversionTotal: metric.NewCounter(metaReplicateQueuePriorityInversionTotal), - PriorityInversionForAddVoterCount: metric.NewCounter(metaReplicateQueuePriorityInversionForAddVoterCount), - PriorityInversionForReplaceDecommissioningVoterCount: metric.NewCounter(metaReplicateQueuePriorityInversionForReplaceDecommissioningVoterCount), - PriorityInversionForRemoveDeadVoterCount: metric.NewCounter(metaReplicateQueuePriorityInversionForRemoveDeadVoterCount), - PriorityInversionForRemoveDecommissioningVoterCount: metric.NewCounter(metaReplicateQueuePriorityInversionForRemoveDecommissioningVoterCount), - PriorityInversionForRemoveVoterCount: metric.NewCounter(metaReplicateQueuePriorityInversionForRemoveVoterCount), - PriorityInversionForReplaceDeadNonVoterCount: metric.NewCounter(metaReplicateQueuePriorityInversionForReplaceDeadNonVoterCount), - PriorityInversionForAddNonVoterCount: metric.NewCounter(metaReplicateQueuePriorityInversionForAddNonVoterCount), - PriorityInversionForReplaceDecommissioningNonVoterCount: metric.NewCounter(metaReplicateQueuePriorityInversionForReplaceDecommissioningNonVoterCount), - PriorityInversionForRemoveDeadNonVoterCount: metric.NewCounter(metaReplicateQueuePriorityInversionForRemoveDeadNonVoterCount), - PriorityInversionForRemoveDecommissioningNonVoterCount: metric.NewCounter(metaReplicateQueuePriorityInversionForRemoveDecommissioningNonVoterCount), - PriorityInversionForRemoveNonVoterCount: metric.NewCounter(metaReplicateQueuePriorityInversionForRemoveNonVoterCount), - PriorityInversionForConsiderRebalance: metric.NewCounter(metaReplicateQueuePriorityInversionForConsiderRebalance), - PriorityInversionForRangeUnavailable: metric.NewCounter(metaReplicateQueuePriorityInversionForRangeUnavailable), - PriorityInversionForNoop: metric.NewCounter(metaReplicateQueuePriorityInversionForNoop), + RequeueDueToPriorityInversion: metric.NewCounter(metaReplicateQueueRequeueDueToPriorityInversion), + PriorityInversionTotal: metric.NewCounter(metaReplicateQueuePriorityInversionTotal), } } @@ -676,47 +561,6 @@ func (metrics *ReplicateQueueMetrics) trackErrorByAllocatorAction( } -// trackPriorityInversion tracks the action that the replicate queue ended up -// processing when the priority at enqueue time was higher than the priority at -// processing time. -func (metrics *ReplicateQueueMetrics) trackPriorityInversion( - actionAtProcessingTime allocatorimpl.AllocatorAction, -) { - metrics.PriorityInversionTotal.Inc(1) - switch actionAtProcessingTime { - case allocatorimpl.AllocatorAddVoter: - metrics.PriorityInversionForAddVoterCount.Inc(1) - case allocatorimpl.AllocatorReplaceDecommissioningVoter: - metrics.PriorityInversionForReplaceDecommissioningVoterCount.Inc(1) - case allocatorimpl.AllocatorRemoveDeadVoter: - metrics.PriorityInversionForRemoveDeadVoterCount.Inc(1) - case allocatorimpl.AllocatorRemoveDecommissioningVoter: - metrics.PriorityInversionForRemoveDecommissioningVoterCount.Inc(1) - case allocatorimpl.AllocatorRemoveVoter: - metrics.PriorityInversionForRemoveVoterCount.Inc(1) - case allocatorimpl.AllocatorReplaceDeadNonVoter: - metrics.PriorityInversionForReplaceDeadNonVoterCount.Inc(1) - case allocatorimpl.AllocatorAddNonVoter: - metrics.PriorityInversionForAddNonVoterCount.Inc(1) - case allocatorimpl.AllocatorReplaceDecommissioningNonVoter: - metrics.PriorityInversionForReplaceDecommissioningNonVoterCount.Inc(1) - case allocatorimpl.AllocatorRemoveDeadNonVoter: - metrics.PriorityInversionForRemoveDeadNonVoterCount.Inc(1) - case allocatorimpl.AllocatorRemoveDecommissioningNonVoter: - metrics.PriorityInversionForRemoveDecommissioningNonVoterCount.Inc(1) - case allocatorimpl.AllocatorRemoveNonVoter: - metrics.PriorityInversionForRemoveNonVoterCount.Inc(1) - case allocatorimpl.AllocatorConsiderRebalance: - metrics.PriorityInversionForConsiderRebalance.Inc(1) - case allocatorimpl.AllocatorRangeUnavailable: - metrics.PriorityInversionForRangeUnavailable.Inc(1) - case allocatorimpl.AllocatorNoop: - metrics.PriorityInversionForNoop.Inc(1) - default: - panic("unhandled default case") - } -} - // trackProcessResult increases the corresponding success/error count metric for // processing a particular allocator action through the replicate queue. func (metrics *ReplicateQueueMetrics) trackResultByAllocatorAction( @@ -1129,7 +973,7 @@ func (rq *replicateQueue) processOneChange( // starving other higher priority work. if PriorityInversionRequeue.Get(&rq.store.cfg.Settings.SV) { if inversion, shouldRequeue := allocatorimpl.CheckPriorityInversion(priorityAtEnqueue, change.Action); inversion { - rq.metrics.trackPriorityInversion(change.Action) + rq.metrics.PriorityInversionTotal.Inc(1) if priorityInversionLogEveryN.ShouldLog() { log.KvDistribution.Infof(ctx, "priority inversion during process: shouldRequeue = %t action=%s, priority=%v, enqueuePriority=%v", diff --git a/pkg/kv/kvserver/replicate_queue_test.go b/pkg/kv/kvserver/replicate_queue_test.go index eb86f3bfb041..a490cf8f8e85 100644 --- a/pkg/kv/kvserver/replicate_queue_test.go +++ b/pkg/kv/kvserver/replicate_queue_test.go @@ -2567,6 +2567,7 @@ func TestReplicateQueueDecommissionScannerDisabled(t *testing.T) { func TestPriorityInversionRequeue(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + skip.UnderDuress(t) ctx := context.Background() settings := cluster.MakeTestingClusterSettings() @@ -2646,9 +2647,6 @@ func TestPriorityInversionRequeue(t *testing.T) { if c := store.ReplicateQueueMetrics().PriorityInversionTotal.Count(); c == 0 { return errors.New("expected non-zero priority inversion total count but got 0") } - if c := store.ReplicateQueueMetrics().PriorityInversionForConsiderRebalance.Count(); c == 0 { - return errors.New("expected non-zero priority inversion count for consider rebalance but got 0") - } if c := store.ReplicateQueueMetrics().RequeueDueToPriorityInversion.Count(); c == 0 { return errors.New("expected to requeue due to priority inversion but got 0") }