From 8c4f0bd3e1bc9db60f3de5f433235f64dbbcac2b 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.yaml | 18 +++++++++++ pkg/kv/kvserver/metrics.go | 25 ++++++++++++++ pkg/kv/kvserver/replica.go | 23 ++++++++++++- pkg/kv/kvserver/replicate_queue_test.go | 43 +++++++++++++++++++++++++ 4 files changed, 108 insertions(+), 1 deletion(-) diff --git a/docs/generated/metrics/metrics.yaml b/docs/generated/metrics/metrics.yaml index 7468d712c98d..cb037cf145b3 100644 --- a/docs/generated/metrics/metrics.yaml +++ b/docs/generated/metrics/metrics.yaml @@ -15269,6 +15269,24 @@ layers: unit: COUNT aggregation: AVG derivative: NONE + - name: ranges.decommissioning.nudger.enqueue + exported_name: ranges_decommissioning_nudger_enqueue + labeled_name: 'ranges.decommissioning.nudger.enqueue{status: enqueue}' + description: '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.' + y_axis_label: Ranges + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE + - name: ranges.decommissioning.nudger.not_leaseholder_or_invalid_lease + exported_name: ranges_decommissioning_nudger_not_leaseholder_or_invalid_lease + labeled_name: ranges.decommissioning.nudger.not_leaseholder_or_invalid_lease + description: Number of enqueues of a range for decommissioning by the decommissioning nudger that were not the leaseholder or had an invalid lease + y_axis_label: Ranges + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE - name: ranges.overreplicated exported_name: ranges_overreplicated description: Number of ranges with more live replicas than the replication target diff --git a/pkg/kv/kvserver/metrics.go b/pkg/kv/kvserver/metrics.go index f67935a19592..d73498d2923c 100644 --- a/pkg/kv/kvserver/metrics.go +++ b/pkg/kv/kvserver/metrics.go @@ -171,6 +171,23 @@ 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, + LabeledName: "ranges.decommissioning.nudger.enqueue", + StaticLabels: metric.MakeLabelPairs(metric.LabelStatus, "enqueue"), + } + 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, + LabeledName: "ranges.decommissioning.nudger.not_leaseholder_or_invalid_lease", + } + // Lease request metrics. metaLeaseRequestSuccessCount = metric.Metadata{ Name: "leases.success", @@ -2809,6 +2826,10 @@ type StoreMetrics struct { DecommissioningRangeCount *metric.Gauge RangeClosedTimestampPolicyCount [ctpb.MAX_CLOSED_TIMESTAMP_POLICY]*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). @@ -3518,6 +3539,10 @@ func newStoreMetrics(histogramWindow time.Duration) *StoreMetrics { DecommissioningRangeCount: metric.NewGauge(metaDecommissioningRangeCount), RangeClosedTimestampPolicyCount: makePolicyRefresherMetrics(), + // 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 80994e204133..4d578dfa6056 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -2911,7 +2911,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 } @@ -2932,8 +2942,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 5b538cfddc9d..07fdb2f13a50 100644 --- a/pkg/kv/kvserver/replicate_queue_test.go +++ b/pkg/kv/kvserver/replicate_queue_test.go @@ -2479,12 +2479,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 fbd4d45b180add3d6feae5ee7db64e2050f356b5 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 2fd0ec0ed68e..2c1a19cd497b 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -923,19 +923,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 f1fcfb698e3b89bc707078f5114299295d014762 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 4d578dfa6056..b1b7fca397df 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -2912,15 +2912,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 abb0d68efb2b7e376f53396e97713f00b03b5847 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 668519caab6f..1e518f30609b 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -766,8 +766,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 07a169471fabe9cea8bdb29d806bd68273fc3716 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 1e518f30609b..22a3fc339898 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -772,7 +772,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 e8a3906ebe3ecf8008b2fb618e50dbb646e65bf2 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 c62a658ddc9d..dcc014381cc6 100644 --- a/pkg/kv/kvserver/consistency_queue.go +++ b/pkg/kv/kvserver/consistency_queue.go @@ -164,7 +164,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 52b3490e1f4a..fa70cd4e612c 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 f40b135d5095..41fbdfdcf544 100644 --- a/pkg/kv/kvserver/lease_queue.go +++ b/pkg/kv/kvserver/lease_queue.go @@ -112,7 +112,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 71302c3a3fce..3e6391439327 100644 --- a/pkg/kv/kvserver/merge_queue.go +++ b/pkg/kv/kvserver/merge_queue.go @@ -238,7 +238,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() @@ -419,7 +419,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 19fe90515d42..ec3aa80c8731 100644 --- a/pkg/kv/kvserver/mvcc_gc_queue.go +++ b/pkg/kv/kvserver/mvcc_gc_queue.go @@ -660,7 +660,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 2eef8559678b..356cc99b47c7 100644 --- a/pkg/kv/kvserver/mvcc_gc_queue_test.go +++ b/pkg/kv/kvserver/mvcc_gc_queue_test.go @@ -911,7 +911,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) } @@ -1162,7 +1162,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) } @@ -1179,7 +1179,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)) } @@ -1296,7 +1296,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) } @@ -1361,7 +1361,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) } @@ -1472,7 +1472,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 22a3fc339898..f56d38cfab14 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -123,7 +123,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 @@ -260,7 +262,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 @@ -876,10 +878,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 @@ -907,7 +909,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]" @@ -923,7 +925,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 { @@ -956,7 +958,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() @@ -980,7 +984,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 } @@ -1321,7 +1325,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) } }, @@ -1448,7 +1452,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 ae85607059e5..7e2bde657257 100644 --- a/pkg/kv/kvserver/queue_test.go +++ b/pkg/kv/kvserver/queue_test.go @@ -58,7 +58,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 { @@ -990,7 +990,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) @@ -1120,7 +1120,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 @@ -1344,13 +1344,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 67461cbd2bff..8f62459e3103 100644 --- a/pkg/kv/kvserver/raft_log_queue.go +++ b/pkg/kv/kvserver/raft_log_queue.go @@ -659,7 +659,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 c44c3195d387..2b5533db79ee 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -4228,7 +4228,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 bedb0d948d85..27e85c94991b 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 b9544d4974f5..fceb2ea331d9 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -10308,7 +10308,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 2c1a19cd497b..6f9d8b7c6bcb 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -633,7 +633,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, @@ -659,7 +659,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 @@ -684,7 +684,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) } } @@ -743,7 +743,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) @@ -758,7 +762,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) @@ -865,6 +869,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 08ea3e3143e8..ee12500bb2f0 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 c1b0c40cafc2..04e569d46593 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -3895,7 +3895,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) @@ -4074,7 +4074,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 } @@ -4111,7 +4111,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 ead1c7ee02a29333394773a6b763dae54c5868fc 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 f56d38cfab14..e4fbb07ded31 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -137,7 +137,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 ab8b76164c643df6848254114efeb47e809c0076 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 e4fbb07ded31..3d914ec95a73 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -1161,6 +1161,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 @@ -1191,7 +1192,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 } @@ -1211,7 +1212,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() @@ -1235,7 +1240,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() { @@ -1324,7 +1329,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 42066dccbeaf16b4928b0c7f9d0e1057728b3190 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 602982c950821cd5830a5bab4b8224d2b49a6fd7 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 3d914ec95a73..493874c3a7ca 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -1098,19 +1098,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) } @@ -1123,6 +1128,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) } @@ -1137,6 +1143,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 7e2bde657257..6edf85fab7bc 100644 --- a/pkg/kv/kvserver/queue_test.go +++ b/pkg/kv/kvserver/queue_test.go @@ -225,6 +225,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) } @@ -297,6 +302,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{}) @@ -316,6 +326,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 @@ -541,11 +556,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 }, } @@ -553,7 +569,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) @@ -840,10 +863,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. @@ -855,6 +887,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) @@ -889,6 +924,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) @@ -924,6 +962,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) @@ -967,6 +1008,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) } @@ -1371,11 +1415,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), @@ -1420,6 +1465,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 5bccff0982bcbc029f46e976fd6940f206d5778b 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 c0a4ec7ee6f4036cd6b395a3e860b3f328e7b1a5 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 493874c3a7ca..241cffe8fff3 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -912,13 +912,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. @@ -1333,12 +1326,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 755a74bf64eaf7024b37c838df53b6afbe3c4a63 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.yaml | 8 ++++++++ pkg/kv/kvserver/metrics.go | 8 ++++++++ pkg/kv/kvserver/queue.go | 8 ++++++++ pkg/kv/kvserver/replicate_queue.go | 1 + 4 files changed, 25 insertions(+) diff --git a/docs/generated/metrics/metrics.yaml b/docs/generated/metrics/metrics.yaml index cb037cf145b3..c8288067ff81 100644 --- a/docs/generated/metrics/metrics.yaml +++ b/docs/generated/metrics/metrics.yaml @@ -13847,6 +13847,14 @@ layers: unit: COUNT aggregation: AVG derivative: NON_NEGATIVE_DERIVATIVE + - name: queue.replicate.dropped_due_to_size + exported_name: queue_replicate_dropped_due_to_size + description: Number of replicas dropped due to the replicate queue exceeding its max size + y_axis_label: Replicas + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE - name: queue.replicate.nonvoterpromotions exported_name: queue_replicate_nonvoterpromotions description: Number of non-voters promoted to voters by the replicate queue diff --git a/pkg/kv/kvserver/metrics.go b/pkg/kv/kvserver/metrics.go index d73498d2923c..99996cf24af2 100644 --- a/pkg/kv/kvserver/metrics.go +++ b/pkg/kv/kvserver/metrics.go @@ -2163,6 +2163,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", @@ -3119,6 +3125,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 @@ -3898,6 +3905,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 241cffe8fff3..74e2f91385f7 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -331,6 +331,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 @@ -774,6 +779,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 6f9d8b7c6bcb..40c70e10b992 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -575,6 +575,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 01c4c41b826f8239163af7ebff6c7488397cf9a2 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 74e2f91385f7..4a1bdc653662 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -446,6 +446,7 @@ type baseQueue struct { purgatory map[roachpb.RangeID]PurgatoryError // Map of replicas to processing errors stopped bool disabled bool + maxSize int64 } } @@ -498,6 +499,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)) @@ -542,6 +544,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() { @@ -777,7 +786,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 40c70e10b992..610f0e9c87a6 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -99,6 +99,24 @@ var EnqueueProblemRangeInReplicateQueueInterval = settings.RegisterDurationSetti 0, ) +// 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", @@ -523,6 +541,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 @@ -547,6 +566,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), @@ -581,6 +601,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 7afd95771b79a9e3e2e9d54a7b440feb91472b41 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 12d31e80523c..0e7848d2c412 100644 --- a/pkg/kv/kvserver/store_rebalancer_test.go +++ b/pkg/kv/kvserver/store_rebalancer_test.go @@ -1863,3 +1863,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 6c02ea647172d25d6b1abd73d523e45a611a2c59 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 4a1bdc653662..eb029048cb5a 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -547,8 +547,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 0e7848d2c412..fa83fb3bb99b 100644 --- a/pkg/kv/kvserver/store_rebalancer_test.go +++ b/pkg/kv/kvserver/store_rebalancer_test.go @@ -1884,15 +1884,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. @@ -1911,4 +1915,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 ca42283252776a2765ad94465e919a3ecc0f6d35 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.yaml | 16 ++++++++-------- 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, 22 insertions(+), 22 deletions(-) diff --git a/docs/generated/metrics/metrics.yaml b/docs/generated/metrics/metrics.yaml index c8288067ff81..59e5bb2d8f03 100644 --- a/docs/generated/metrics/metrics.yaml +++ b/docs/generated/metrics/metrics.yaml @@ -13847,14 +13847,6 @@ layers: unit: COUNT aggregation: AVG derivative: NON_NEGATIVE_DERIVATIVE - - name: queue.replicate.dropped_due_to_size - exported_name: queue_replicate_dropped_due_to_size - description: Number of replicas dropped due to the replicate queue exceeding its max size - y_axis_label: Replicas - type: COUNTER - unit: COUNT - aggregation: AVG - derivative: NON_NEGATIVE_DERIVATIVE - name: queue.replicate.nonvoterpromotions exported_name: queue_replicate_nonvoterpromotions description: Number of non-voters promoted to voters by the replicate queue @@ -13903,6 +13895,14 @@ layers: unit: COUNT aggregation: AVG derivative: NONE + - name: queue.replicate.queue_full + exported_name: queue_replicate_queue_full + description: Number of times a replica was dropped from the queue due to queue fullness + y_axis_label: Replicas + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE - name: queue.replicate.rebalancenonvoterreplica exported_name: queue_replicate_rebalancenonvoterreplica description: Number of non-voter replica rebalancer-initiated additions attempted by the replicate queue diff --git a/pkg/kv/kvserver/metrics.go b/pkg/kv/kvserver/metrics.go index 99996cf24af2..b583371d59b4 100644 --- a/pkg/kv/kvserver/metrics.go +++ b/pkg/kv/kvserver/metrics.go @@ -2163,9 +2163,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, } @@ -3125,7 +3125,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 @@ -3905,7 +3905,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 eb029048cb5a..47e323b617ad 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -331,11 +331,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 @@ -556,7 +556,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]) } } @@ -798,8 +798,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 610f0e9c87a6..7a8a17c7d3a3 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -595,7 +595,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 fa83fb3bb99b..93f3a64e56c0 100644 --- a/pkg/kv/kvserver/store_rebalancer_test.go +++ b/pkg/kv/kvserver/store_rebalancer_test.go @@ -1887,8 +1887,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 0a2a5a930839305580938af0a0d52efd287302bc 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 7a8a17c7d3a3..63452d3573e4 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -99,6 +99,19 @@ var EnqueueProblemRangeInReplicateQueueInterval = settings.RegisterDurationSetti 0, ) +// 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 8887a72b2796ae62870b3f823c1805799622d84f 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 ff6be6cf0e67..fb3aed2a220c 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go +++ b/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go @@ -9475,3 +9475,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 63452d3573e4..96b0796aa9d9 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -718,6 +718,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 } @@ -911,6 +915,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 761aebcfc2853ea307812164a1c5160f77b48a07 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 96b0796aa9d9..fe31d76674d7 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -905,6 +905,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, @@ -918,9 +922,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 35137acca4e1f95cc6856f89bb3704ee96df7fc9 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 fe31d76674d7..72a9a6ef1911 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -104,6 +104,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", @@ -929,9 +930,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 72edb88f2698043ffedf140618e45ce10cee94bc 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 215629ed23b64610291e0be5be9cb9878cafe1b9 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 fb3aed2a220c..cddab320468a 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go +++ b/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go @@ -9576,6 +9576,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, @@ -9620,3 +9627,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 89e73294d6892d95592b84513e68269b8f3683bc 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 72a9a6ef1911..7ed528297586 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -921,21 +921,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 af1b99c702d415f856de310919bba770c8718850 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 7ed528297586..d624e6c97f4b 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -921,24 +921,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 @@ -963,6 +945,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 01b62410cbf59aa4f8f787568258ce557c277b92 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 d624e6c97f4b..58464f05afcb 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -717,25 +717,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 b991142737c7b258838b49d830a7bd5255737ff3 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 0a4e5abebfe6..aa127cf3b981 100644 --- a/pkg/kv/kvserver/replica_backpressure.go +++ b/pkg/kv/kvserver/replica_backpressure.go @@ -208,7 +208,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 d823cdcf236dae2a8fa84abf8208f371e1bee6d3 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 | 20 +++++++++++++------- pkg/kv/kvserver/replica.go | 10 +++++----- pkg/kv/kvserver/store_gossip.go | 4 ++-- 3 files changed, 20 insertions(+), 14 deletions(-) diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index 47e323b617ad..0e9f4f05de54 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -607,18 +607,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,12 +638,13 @@ func (bq *baseQueue) Async( if bq.addLogN.ShouldLog() { log.Infof(ctx, "rate limited in %s: %s", redact.Safe(opName), err) } - return + return baseQueueAsyncRateLimited } go func(ctx context.Context) { defer hdl.Activate(ctx).Release(ctx) fn(ctx, baseQueueHelper{bq}) }(bgCtx) + return nil } // MaybeAddAsync offers the replica to the queue. The queue will only process a @@ -647,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) }) } @@ -656,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 b1b7fca397df..0166523e3cba 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -1172,25 +1172,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 1d2e4c8b543238de74bac20d6823c2896f63dee9 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 ec3aa80c8731..556dd1ea0194 100644 --- a/pkg/kv/kvserver/mvcc_gc_queue.go +++ b/pkg/kv/kvserver/mvcc_gc_queue.go @@ -878,7 +878,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 0e9f4f05de54..8f19e7a50e0f 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -112,9 +112,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. @@ -145,8 +193,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) } @@ -204,8 +259,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 { @@ -595,8 +655,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) } @@ -604,7 +666,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 2ffb427e16d9..063da4301d78 100644 --- a/pkg/kv/kvserver/queue_helpers_testutil.go +++ b/pkg/kv/kvserver/queue_helpers_testutil.go @@ -19,7 +19,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 0166523e3cba..aa31d4339b15 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -2940,16 +2940,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 aa127cf3b981..894510950ac7 100644 --- a/pkg/kv/kvserver/replica_backpressure.go +++ b/pkg/kv/kvserver/replica_backpressure.go @@ -207,15 +207,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 cb9ee71a463156300008907e3ee66e87dca649cc 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 063da4301d78..53936d05d4c4 100644 --- a/pkg/kv/kvserver/queue_helpers_testutil.go +++ b/pkg/kv/kvserver/queue_helpers_testutil.go @@ -22,6 +22,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 6edf85fab7bc..4a376de69505 100644 --- a/pkg/kv/kvserver/queue_test.go +++ b/pkg/kv/kvserver/queue_test.go @@ -1293,6 +1293,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 ef48cb0f789a21d3473c0346e2e714e27240707c 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 8f19e7a50e0f..2faacaf28112 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -124,44 +124,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 894510950ac7..c1b8ac4cec63 100644 --- a/pkg/kv/kvserver/replica_backpressure.go +++ b/pkg/kv/kvserver/replica_backpressure.go @@ -213,7 +213,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 d6b63f1c0663a211086624afc0201e6653715248 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 2faacaf28112..78b85f56d4be 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 c7c06eac100d84f07a7c7843f0ac81ad16ddae9d 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 78b85f56d4be..302225d09bcc 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -659,9 +659,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) } @@ -669,7 +669,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 53936d05d4c4..6c3858d27604 100644 --- a/pkg/kv/kvserver/queue_helpers_testutil.go +++ b/pkg/kv/kvserver/queue_helpers_testutil.go @@ -26,9 +26,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 ff593989cd16abbe5d92600d39c637135f50403c 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 302225d09bcc..a3afa567f5bd 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 52b461b62deb12d9cce39dcd57a01efdb0ce3f58 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 a3afa567f5bd..7f0a44f04f3c 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 cd44a09a4b9f520b81ffcf1c5a7a7008823dc83e 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 aa31d4339b15..a06bf912186a 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -2903,8 +2903,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 @@ -2946,24 +2947,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 04e569d46593..c22a992f9354 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -3390,6 +3390,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() @@ -3460,7 +3466,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 b04d2058779e248d63628791898288a81dfe1aa1 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 a06bf912186a..cbbde30181a6 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -2903,7 +2903,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 @@ -2944,44 +2944,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 c22a992f9354..c8a4f6b6b034 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -3466,11 +3466,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 b669c80bf6e24c36c9734b9ff42a551503cc00cc 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.yaml | 38 ++++++++++++++++++++++++++++- pkg/kv/kvserver/metrics.go | 38 ++++++++++++++++++++++++++++- pkg/kv/kvserver/replica.go | 4 +++ 3 files changed, 78 insertions(+), 2 deletions(-) diff --git a/docs/generated/metrics/metrics.yaml b/docs/generated/metrics/metrics.yaml index 59e5bb2d8f03..9f57e165814a 100644 --- a/docs/generated/metrics/metrics.yaml +++ b/docs/generated/metrics/metrics.yaml @@ -15286,10 +15286,46 @@ layers: unit: COUNT aggregation: AVG derivative: NON_NEGATIVE_DERIVATIVE + - name: ranges.decommissioning.nudger.enqueue.failure + exported_name: ranges_decommissioning_nudger_enqueue_failure + labeled_name: ranges.decommissioning.nudger.enqueue.failure + description: Number of ranges that failed to enqueue at the replicate queue + y_axis_label: Ranges + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE + - name: ranges.decommissioning.nudger.enqueue.success + exported_name: ranges_decommissioning_nudger_enqueue_success + labeled_name: ranges.decommissioning.nudger.enqueue.success + description: Number of ranges that were successfully enqueued by the decommisioning nudger + y_axis_label: Ranges + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE - name: ranges.decommissioning.nudger.not_leaseholder_or_invalid_lease exported_name: ranges_decommissioning_nudger_not_leaseholder_or_invalid_lease labeled_name: ranges.decommissioning.nudger.not_leaseholder_or_invalid_lease - description: Number of enqueues of a range for decommissioning by the decommissioning nudger that were not the leaseholder or had an invalid lease + description: Number of ranges that were not the leaseholder or had an invalid lease at the decommissioning nudger + y_axis_label: Ranges + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE + - name: ranges.decommissioning.nudger.process.failure + exported_name: ranges_decommissioning_nudger_process_failure + labeled_name: ranges.decommissioning.nudger.process.failure + description: Number of ranges enqueued by the decommissioning nudger that failed to process by the replicate queue + y_axis_label: Ranges + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE + - name: ranges.decommissioning.nudger.process.success + exported_name: ranges_decommissioning_nudger_process_success + labeled_name: ranges.decommissioning.nudger.process.success + description: Number of ranges enqueued by the decommissioning nudger that were successfully processed by the replicate queue y_axis_label: Ranges type: COUNTER unit: COUNT diff --git a/pkg/kv/kvserver/metrics.go b/pkg/kv/kvserver/metrics.go index b583371d59b4..a1897a7fa4d7 100644 --- a/pkg/kv/kvserver/metrics.go +++ b/pkg/kv/kvserver/metrics.go @@ -180,9 +180,37 @@ var ( LabeledName: "ranges.decommissioning.nudger.enqueue", StaticLabels: metric.MakeLabelPairs(metric.LabelStatus, "enqueue"), } + 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, LabeledName: "ranges.decommissioning.nudger.not_leaseholder_or_invalid_lease", @@ -2834,6 +2862,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 @@ -3548,6 +3580,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 cbbde30181a6..c1562e2eaa91 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -2954,18 +2954,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 b61dc8a4ae544f3a74adff44ed087957013f0ccc 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.yaml | 32 +++++++++++++++++ pkg/kv/kvserver/metrics.go | 35 ++++++++++++++++++ pkg/kv/kvserver/queue.go | 56 +++++++++++++++++++++++++++-- pkg/kv/kvserver/replicate_queue.go | 20 ++++++----- 4 files changed, 132 insertions(+), 11 deletions(-) diff --git a/docs/generated/metrics/metrics.yaml b/docs/generated/metrics/metrics.yaml index 9f57e165814a..72c6b3ce3b75 100644 --- a/docs/generated/metrics/metrics.yaml +++ b/docs/generated/metrics/metrics.yaml @@ -13847,6 +13847,38 @@ layers: unit: COUNT aggregation: AVG derivative: NON_NEGATIVE_DERIVATIVE + - name: queue.replicate.enqueue.add + exported_name: queue_replicate_enqueue_add + description: Number of replicas successfully added to the replicate queue + y_axis_label: Replicas + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE + - name: queue.replicate.enqueue.failedprecondition + exported_name: queue_replicate_enqueue_failedprecondition + description: Number of replicas that failed the precondition checks and were therefore not added to the replicate queue + y_axis_label: Replicas + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE + - name: queue.replicate.enqueue.noaction + exported_name: queue_replicate_enqueue_noaction + description: Number of replicas for which ShouldQueue determined no action was needed and were therefore not added to the replicate queue + y_axis_label: Replicas + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE + - name: queue.replicate.enqueue.unexpectederror + exported_name: queue_replicate_enqueue_unexpectederror + description: 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 + y_axis_label: Replicas + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE - name: queue.replicate.nonvoterpromotions exported_name: queue_replicate_nonvoterpromotions description: Number of non-voters promoted to voters by the replicate queue diff --git a/pkg/kv/kvserver/metrics.go b/pkg/kv/kvserver/metrics.go index a1897a7fa4d7..5e039340f155 100644 --- a/pkg/kv/kvserver/metrics.go +++ b/pkg/kv/kvserver/metrics.go @@ -2143,6 +2143,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", @@ -3154,6 +3181,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 @@ -3938,6 +3969,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 7f0a44f04f3c..f81192d8e9fb 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -387,7 +387,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 58464f05afcb..314792aa09b3 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -605,14 +605,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 a15adf126c776b3aa00d6e02aaa12a4777825378 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 f81192d8e9fb..a75d5da4e8ad 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -387,11 +387,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 8b07f87f86e71856811a275e3bb4f414346d9483 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 4a376de69505..091bd46fbc01 100644 --- a/pkg/kv/kvserver/queue_test.go +++ b/pkg/kv/kvserver/queue_test.go @@ -111,6 +111,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) } @@ -1327,6 +1329,7 @@ func TestBaseQueueCallbackOnEnqueueResult(t *testing.T) { t.Fatal("unexpected call to onProcessResult") }, }) + require.Equal(t, bq.enqueueAdd.Count(), int64(1)) require.True(t, queued) }) @@ -1353,6 +1356,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. @@ -1373,6 +1377,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) { @@ -1391,6 +1397,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) { @@ -1408,6 +1416,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{ @@ -1420,6 +1430,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) { @@ -1443,6 +1455,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) { @@ -1454,7 +1468,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) @@ -1463,7 +1477,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{} @@ -1483,6 +1499,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 07fdb2f13a50..fde0f95e0c87 100644 --- a/pkg/kv/kvserver/replicate_queue_test.go +++ b/pkg/kv/kvserver/replicate_queue_test.go @@ -2492,8 +2492,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) } @@ -2514,9 +2516,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") } @@ -2539,4 +2542,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 608060bd02010a9cf39584c4d2772dacbfa75d98 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Tue, 26 Aug 2025 23:26:51 -0400 Subject: [PATCH 42/44] kvserver: track priority inversion in replicate queue metrics Previously, replicas could be enqueued at a high priority but end up processing a lower-priority actions, causing priority inversion and unfairness to other replicas behind them that needs a repair action. This commit adds metrics to track such cases. In addition, this commit also adds metrics to track when replicas are requeued in the replicate queue due to a priority inversion from a repair action to a rebalance action. --- docs/generated/metrics/metrics.yaml | 128 +++++++++++++++++++ pkg/kv/kvserver/replicate_queue.go | 182 ++++++++++++++++++++++++++++ 2 files changed, 310 insertions(+) diff --git a/docs/generated/metrics/metrics.yaml b/docs/generated/metrics/metrics.yaml index 72c6b3ce3b75..3627ae2e3a9e 100644 --- a/docs/generated/metrics/metrics.yaml +++ b/docs/generated/metrics/metrics.yaml @@ -13895,6 +13895,134 @@ layers: unit: COUNT aggregation: AVG derivative: NONE + - name: queue.replicate.priority_inversion.addnonvoter + exported_name: queue_replicate_priority_inversion_addnonvoter + description: Number of priority inversions in the replicate queue that resulted in add non-voter action during processing + y_axis_label: Replicas + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE + - name: queue.replicate.priority_inversion.addvoter + exported_name: queue_replicate_priority_inversion_addvoter + description: Number of priority inversions in the replicate queue that resulted in add voter action during processing + y_axis_label: Replicas + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE + - name: queue.replicate.priority_inversion.considerrebalance + exported_name: queue_replicate_priority_inversion_considerrebalance + description: Number of priority inversions in the replicate queue that resulted in consider rebalance action during processing + y_axis_label: Replicas + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE + - name: queue.replicate.priority_inversion.noop + exported_name: queue_replicate_priority_inversion_noop + description: Number of priority inversions in the replicate queue that resulted in noop action during processing + y_axis_label: Replicas + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE + - name: queue.replicate.priority_inversion.rangeunavailable + exported_name: queue_replicate_priority_inversion_rangeunavailable + description: Number of priority inversions in the replicate queue that resulted in range unavailable action during processing + y_axis_label: Replicas + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE + - name: queue.replicate.priority_inversion.removedeadnonvoter + exported_name: queue_replicate_priority_inversion_removedeadnonvoter + description: Number of priority inversions in the replicate queue that resulted in remove dead non-voter action during processing + y_axis_label: Replicas + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE + - name: queue.replicate.priority_inversion.removedeadvoter + exported_name: queue_replicate_priority_inversion_removedeadvoter + description: Number of priority inversions in the replicate queue that resulted in remove dead voter action during processing + y_axis_label: Replicas + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE + - name: queue.replicate.priority_inversion.removedecommissioningnonvoter + exported_name: queue_replicate_priority_inversion_removedecommissioningnonvoter + description: Number of priority inversions in the replicate queue that resulted in remove decommissioning non-voter action during processing + y_axis_label: Replicas + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE + - name: queue.replicate.priority_inversion.removedecommissioningvoter + exported_name: queue_replicate_priority_inversion_removedecommissioningvoter + description: Number of priority inversions in the replicate queue that resulted in remove decommissioning voter action during processing + y_axis_label: Replicas + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE + - name: queue.replicate.priority_inversion.removenonvoter + exported_name: queue_replicate_priority_inversion_removenonvoter + description: Number of priority inversions in the replicate queue that resulted in remove non-voter action during processing + y_axis_label: Replicas + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE + - name: queue.replicate.priority_inversion.removevoter + exported_name: queue_replicate_priority_inversion_removevoter + description: Number of priority inversions in the replicate queue that resulted in remove voter action during processing + y_axis_label: Replicas + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE + - name: queue.replicate.priority_inversion.replacedeadnonvoter + exported_name: queue_replicate_priority_inversion_replacedeadnonvoter + description: Number of priority inversions in the replicate queue that resulted in replace dead non-voter action during processing + y_axis_label: Replicas + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE + - name: queue.replicate.priority_inversion.replacedecommissioningnonvoter + exported_name: queue_replicate_priority_inversion_replacedecommissioningnonvoter + description: Number of priority inversions in the replicate queue that resulted in replace decommissioning non-voter action during processing + y_axis_label: Replicas + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE + - name: queue.replicate.priority_inversion.replacedecommissioningvoter + exported_name: queue_replicate_priority_inversion_replacedecommissioningvoter + description: Number of priority inversions in the replicate queue that resulted in replace decommissioning voter action during processing + y_axis_label: Replicas + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE + - name: queue.replicate.priority_inversion.requeue + exported_name: queue_replicate_priority_inversion_requeue + description: 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. + y_axis_label: Replicas + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE + - name: queue.replicate.priority_inversion.total + exported_name: queue_replicate_priority_inversion_total + description: 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 + y_axis_label: Replicas + type: COUNTER + unit: COUNT + aggregation: AVG + derivative: NON_NEGATIVE_DERIVATIVE - name: queue.replicate.process.failure exported_name: queue_replicate_process_failure description: Number of replicas which failed processing in the replicate queue diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index 314792aa09b3..1923f8094a40 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -321,6 +321,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 @@ -343,6 +443,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 @@ -380,6 +481,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 { @@ -416,6 +538,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), } } @@ -539,6 +678,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( @@ -951,12 +1131,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 81d63db375c49204d2a4008996ef998aecb16690 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 a75d5da4e8ad..4d4f3f65b57d 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 fde0f95e0c87..e8c822561b91 100644 --- a/pkg/kv/kvserver/replicate_queue_test.go +++ b/pkg/kv/kvserver/replicate_queue_test.go @@ -2547,3 +2547,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 ba2db43cad63..8a1495d4a799 100644 --- a/pkg/kv/kvserver/testing_knobs.go +++ b/pkg/kv/kvserver/testing_knobs.go @@ -535,6 +535,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 7ee07bc7dbf2f71a1f734e442e8e10022f8c3487 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.yaml | 112 ---------------- pkg/kv/kvserver/replicate_queue.go | 168 +----------------------- pkg/kv/kvserver/replicate_queue_test.go | 4 +- 3 files changed, 7 insertions(+), 277 deletions(-) diff --git a/docs/generated/metrics/metrics.yaml b/docs/generated/metrics/metrics.yaml index 3627ae2e3a9e..0a806657c1df 100644 --- a/docs/generated/metrics/metrics.yaml +++ b/docs/generated/metrics/metrics.yaml @@ -13895,118 +13895,6 @@ layers: unit: COUNT aggregation: AVG derivative: NONE - - name: queue.replicate.priority_inversion.addnonvoter - exported_name: queue_replicate_priority_inversion_addnonvoter - description: Number of priority inversions in the replicate queue that resulted in add non-voter action during processing - y_axis_label: Replicas - type: COUNTER - unit: COUNT - aggregation: AVG - derivative: NON_NEGATIVE_DERIVATIVE - - name: queue.replicate.priority_inversion.addvoter - exported_name: queue_replicate_priority_inversion_addvoter - description: Number of priority inversions in the replicate queue that resulted in add voter action during processing - y_axis_label: Replicas - type: COUNTER - unit: COUNT - aggregation: AVG - derivative: NON_NEGATIVE_DERIVATIVE - - name: queue.replicate.priority_inversion.considerrebalance - exported_name: queue_replicate_priority_inversion_considerrebalance - description: Number of priority inversions in the replicate queue that resulted in consider rebalance action during processing - y_axis_label: Replicas - type: COUNTER - unit: COUNT - aggregation: AVG - derivative: NON_NEGATIVE_DERIVATIVE - - name: queue.replicate.priority_inversion.noop - exported_name: queue_replicate_priority_inversion_noop - description: Number of priority inversions in the replicate queue that resulted in noop action during processing - y_axis_label: Replicas - type: COUNTER - unit: COUNT - aggregation: AVG - derivative: NON_NEGATIVE_DERIVATIVE - - name: queue.replicate.priority_inversion.rangeunavailable - exported_name: queue_replicate_priority_inversion_rangeunavailable - description: Number of priority inversions in the replicate queue that resulted in range unavailable action during processing - y_axis_label: Replicas - type: COUNTER - unit: COUNT - aggregation: AVG - derivative: NON_NEGATIVE_DERIVATIVE - - name: queue.replicate.priority_inversion.removedeadnonvoter - exported_name: queue_replicate_priority_inversion_removedeadnonvoter - description: Number of priority inversions in the replicate queue that resulted in remove dead non-voter action during processing - y_axis_label: Replicas - type: COUNTER - unit: COUNT - aggregation: AVG - derivative: NON_NEGATIVE_DERIVATIVE - - name: queue.replicate.priority_inversion.removedeadvoter - exported_name: queue_replicate_priority_inversion_removedeadvoter - description: Number of priority inversions in the replicate queue that resulted in remove dead voter action during processing - y_axis_label: Replicas - type: COUNTER - unit: COUNT - aggregation: AVG - derivative: NON_NEGATIVE_DERIVATIVE - - name: queue.replicate.priority_inversion.removedecommissioningnonvoter - exported_name: queue_replicate_priority_inversion_removedecommissioningnonvoter - description: Number of priority inversions in the replicate queue that resulted in remove decommissioning non-voter action during processing - y_axis_label: Replicas - type: COUNTER - unit: COUNT - aggregation: AVG - derivative: NON_NEGATIVE_DERIVATIVE - - name: queue.replicate.priority_inversion.removedecommissioningvoter - exported_name: queue_replicate_priority_inversion_removedecommissioningvoter - description: Number of priority inversions in the replicate queue that resulted in remove decommissioning voter action during processing - y_axis_label: Replicas - type: COUNTER - unit: COUNT - aggregation: AVG - derivative: NON_NEGATIVE_DERIVATIVE - - name: queue.replicate.priority_inversion.removenonvoter - exported_name: queue_replicate_priority_inversion_removenonvoter - description: Number of priority inversions in the replicate queue that resulted in remove non-voter action during processing - y_axis_label: Replicas - type: COUNTER - unit: COUNT - aggregation: AVG - derivative: NON_NEGATIVE_DERIVATIVE - - name: queue.replicate.priority_inversion.removevoter - exported_name: queue_replicate_priority_inversion_removevoter - description: Number of priority inversions in the replicate queue that resulted in remove voter action during processing - y_axis_label: Replicas - type: COUNTER - unit: COUNT - aggregation: AVG - derivative: NON_NEGATIVE_DERIVATIVE - - name: queue.replicate.priority_inversion.replacedeadnonvoter - exported_name: queue_replicate_priority_inversion_replacedeadnonvoter - description: Number of priority inversions in the replicate queue that resulted in replace dead non-voter action during processing - y_axis_label: Replicas - type: COUNTER - unit: COUNT - aggregation: AVG - derivative: NON_NEGATIVE_DERIVATIVE - - name: queue.replicate.priority_inversion.replacedecommissioningnonvoter - exported_name: queue_replicate_priority_inversion_replacedecommissioningnonvoter - description: Number of priority inversions in the replicate queue that resulted in replace decommissioning non-voter action during processing - y_axis_label: Replicas - type: COUNTER - unit: COUNT - aggregation: AVG - derivative: NON_NEGATIVE_DERIVATIVE - - name: queue.replicate.priority_inversion.replacedecommissioningvoter - exported_name: queue_replicate_priority_inversion_replacedecommissioningvoter - description: Number of priority inversions in the replicate queue that resulted in replace decommissioning voter action during processing - y_axis_label: Replicas - type: COUNTER - unit: COUNT - aggregation: AVG - derivative: NON_NEGATIVE_DERIVATIVE - name: queue.replicate.priority_inversion.requeue exported_name: queue_replicate_priority_inversion_requeue description: 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. diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index 1923f8094a40..6e8376095369 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -337,90 +337,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 @@ -482,26 +398,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 { @@ -539,22 +438,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), } } @@ -678,47 +563,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( @@ -1131,7 +975,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 e8c822561b91..c318855dd889 100644 --- a/pkg/kv/kvserver/replicate_queue_test.go +++ b/pkg/kv/kvserver/replicate_queue_test.go @@ -2575,6 +2575,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() @@ -2654,9 +2655,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") }