From 5eba0473b3abdc09d4eccacd14c4365bdca987f0 Mon Sep 17 00:00:00 2001 From: Aayush Shah Date: Sat, 15 May 2021 20:45:36 -0400 Subject: [PATCH 1/7] kvserver: clean up convergesScore computation during rebalancing Release justification: Fixes high priority bug Release note: None --- pkg/kv/kvserver/allocator_scorer.go | 56 +++++++++++++----------- pkg/kv/kvserver/allocator_scorer_test.go | 12 ++--- 2 files changed, 36 insertions(+), 32 deletions(-) diff --git a/pkg/kv/kvserver/allocator_scorer.go b/pkg/kv/kvserver/allocator_scorer.go index 195e50dc6722..8f847fceb17d 100644 --- a/pkg/kv/kvserver/allocator_scorer.go +++ b/pkg/kv/kvserver/allocator_scorer.go @@ -507,16 +507,13 @@ func rankedCandidateListForRemoval( continue } diversityScore := diversityRemovalScore(s.StoreID, existingStoreLocalities) + // If removing this candidate replica does not converge the store + // stats to their means, we make it less attractive for removal by + // adding 1 to the constraint score. Note that when selecting a + // candidate for removal the candidates with the lowest scores are + // more likely to be removed. + convergesScore := rebalanceFromConvergesScore(sl, s.Capacity) balanceScore := balanceScore(sl, s.Capacity, options) - var convergesScore int - if !rebalanceFromConvergesOnMean(sl, s.Capacity) { - // If removing this candidate replica does not converge the store - // stats to their means, we make it less attractive for removal by - // adding 1 to the constraint score. Note that when selecting a - // candidate for removal the candidates with the lowest scores are - // more likely to be removed. - convergesScore = 1 - } candidates = append(candidates, candidate{ store: s, valid: constraintsOK, @@ -752,13 +749,10 @@ func rankedCandidateListForRebalancing( continue } balanceScore := balanceScore(comparable.sl, existing.store.Capacity, options) - var convergesScore int - if !rebalanceFromConvergesOnMean(comparable.sl, existing.store.Capacity) { - // Similarly to in rankedCandidateListForRemoval, any replica whose - // removal would not converge the range stats to their means is given a - // constraint score boost of 1 to make it less attractive for removal. - convergesScore = 1 - } + // Similarly to in rankedCandidateListForRemoval, any replica whose + // removal would not converge the range stats to their means is given a + // constraint score boost of 1 to make it less attractive for removal. + convergesScore := rebalanceFromConvergesScore(comparable.sl, existing.store.Capacity) existing.convergesScore = convergesScore existing.balanceScore = balanceScore existing.rangeCount = int(existing.store.Capacity.RangeCount) @@ -777,12 +771,8 @@ func rankedCandidateListForRebalancing( s := cand.store cand.fullDisk = !rebalanceToMaxCapacityCheck(s) cand.balanceScore = balanceScore(comparable.sl, s.Capacity, options) - if rebalanceToConvergesOnMean(comparable.sl, s.Capacity) { - // This is the counterpart of !rebalanceFromConvergesOnMean from - // the existing candidates. Candidates whose addition would - // converge towards the range count mean are promoted. - cand.convergesScore = 1 - } else if !needRebalance { + cand.convergesScore = rebalanceToConvergesScore(comparable.sl, s.Capacity) + if !needRebalance && cand.convergesScore == 0 { // Only consider this candidate if we must rebalance due to constraint, // disk fullness, or diversity reasons. log.VEventf(ctx, 3, "not considering %+v as a candidate for range %+v: score=%s storeList=%+v", @@ -1373,12 +1363,26 @@ func underfullThreshold(mean float64, thresholdFraction float64) float64 { return mean - math.Max(mean*thresholdFraction, minRangeRebalanceThreshold) } -func rebalanceFromConvergesOnMean(sl StoreList, sc roachpb.StoreCapacity) bool { - return rebalanceConvergesOnMean(sl, sc, sc.RangeCount-1) +// rebalanceFromConvergesScore returns 1 if rebalancing a replica away from `sc` +// will _not_ converge its range count towards the mean range count of stores in +// `sl`. When we're considering whether to rebalance a replica away from a store +// or not, we want to give it a "boost" (i.e. make it a less likely candidate +// for removal) if it doesn't further our goal to converge range counts towards +// the mean. +func rebalanceFromConvergesScore(sl StoreList, sc roachpb.StoreCapacity) int { + if rebalanceConvergesOnMean(sl, sc, sc.RangeCount-1) { + return 0 + } + return 1 } -func rebalanceToConvergesOnMean(sl StoreList, sc roachpb.StoreCapacity) bool { - return rebalanceConvergesOnMean(sl, sc, sc.RangeCount+1) +// rebalanceToConvergesScore returns 1 if rebalancing a replica to `sc` will +// converge its range count towards the mean of all the stores inside `sl`. +func rebalanceToConvergesScore(sl StoreList, sc roachpb.StoreCapacity) int { + if rebalanceConvergesOnMean(sl, sc, sc.RangeCount+1) { + return 1 + } + return 0 } func rebalanceConvergesOnMean(sl StoreList, sc roachpb.StoreCapacity, newRangeCount int32) bool { diff --git a/pkg/kv/kvserver/allocator_scorer_test.go b/pkg/kv/kvserver/allocator_scorer_test.go index cb9d0fb5ec74..062b8241d707 100644 --- a/pkg/kv/kvserver/allocator_scorer_test.go +++ b/pkg/kv/kvserver/allocator_scorer_test.go @@ -1518,23 +1518,23 @@ func TestRebalanceConvergesOnMean(t *testing.T) { }{ {0, true, false}, {900, true, false}, - {900, true, false}, {999, true, false}, {1000, false, false}, {1001, false, true}, {2000, false, true}, - {900, true, false}, } for i, tc := range testCases { sc := roachpb.StoreCapacity{ RangeCount: tc.rangeCount, } - if a, e := rebalanceToConvergesOnMean(storeList, sc), tc.toConverges; a != e { - t.Errorf("%d: rebalanceToConvergesOnMean(storeList, %+v) got %t; want %t", i, sc, a, e) + if a, e := rebalanceToConvergesScore(storeList, sc) == 1, tc.toConverges; a != e { + t.Errorf("%d: rebalanceToConvergesScore(storeList, %+v) got %t; want %t", i, sc, a, e) } - if a, e := rebalanceFromConvergesOnMean(storeList, sc), tc.fromConverges; a != e { - t.Errorf("%d: rebalanceFromConvergesOnMean(storeList, %+v) got %t; want %t", i, sc, a, e) + // NB: Any replica whose removal would not converge the range count to the + // mean is given a score of 1 to make it less attractive for removal. + if a, e := rebalanceFromConvergesScore(storeList, sc) == 0, tc.fromConverges; a != e { + t.Errorf("%d: rebalanceFromConvergesScore(storeList, %+v) got %t; want %t", i, sc, a, e) } } } From b88a2b797e7906d5465eb58681d5f61170af2500 Mon Sep 17 00:00:00 2001 From: Aayush Shah Date: Sat, 15 May 2021 22:06:33 -0400 Subject: [PATCH 2/7] kvserver: allow computing `balanceScore` off of QPS for rebalancing Previously, the replica rebalancing logic inside the allocator only computed `balanceScore` (a score of whether a store is overfull, underfull or balanced based on some signal) based on range count. This commit augments the replica rebalancing logic to support an option to allow computing `balanceScore` based on QPS instead. When the `balanceScore` is being computed off of QPS, we disable `convergesScore` (which we can only compute off of RangeCount and would typically take precedence over `balanceScore`). A future commit in this patchset will leverage this option in the `StoreRebalancer` to make zone-aware rebalancing decisions based on QPS. Release justification: Fixes high priority bug Release note: None --- pkg/kv/kvserver/allocator.go | 30 +- pkg/kv/kvserver/allocator_scorer.go | 230 ++++++++----- pkg/kv/kvserver/allocator_scorer_test.go | 64 +++- pkg/kv/kvserver/allocator_test.go | 412 ++++++++++++++++++----- pkg/kv/kvserver/replica_command.go | 9 +- pkg/kv/kvserver/replicate_queue.go | 17 +- pkg/kv/kvserver/store_rebalancer.go | 11 +- 7 files changed, 582 insertions(+), 191 deletions(-) diff --git a/pkg/kv/kvserver/allocator.go b/pkg/kv/kvserver/allocator.go index 57f40953e89b..c9e9387275b7 100644 --- a/pkg/kv/kvserver/allocator.go +++ b/pkg/kv/kvserver/allocator.go @@ -838,7 +838,7 @@ func (a *Allocator) allocateTargetFromList( log.VEventf(ctx, 3, "allocate %s: %s", targetType, candidates) if target := candidates.selectGood(a.randGen); target != nil { log.VEventf(ctx, 3, "add target: %s", target) - details := decisionDetails{Target: target.compactString(options)} + details := decisionDetails{Target: target.compactString()} detailsBytes, err := json.Marshal(details) if err != nil { log.Warningf(ctx, "failed to marshal details for choosing allocate target: %+v", err) @@ -858,6 +858,7 @@ func (a Allocator) simulateRemoveTarget( existingNonVoters []roachpb.ReplicaDescriptor, rangeUsageInfo RangeUsageInfo, targetType targetReplicaType, + options scorerOptions, ) (roachpb.ReplicaDescriptor, string, error) { // Update statistics first // TODO(a-robinson): This could theoretically interfere with decisions made by other goroutines, @@ -874,7 +875,7 @@ func (a Allocator) simulateRemoveTarget( ) log.VEventf(ctx, 3, "simulating which voter would be removed after adding s%d", targetStore) - return a.RemoveVoter(ctx, conf, candidates, existingVoters, existingNonVoters) + return a.RemoveVoter(ctx, conf, candidates, existingVoters, existingNonVoters, options) case nonVoterTarget: a.storePool.updateLocalStoreAfterRebalance(targetStore, rangeUsageInfo, roachpb.ADD_NON_VOTER) defer a.storePool.updateLocalStoreAfterRebalance( @@ -884,7 +885,7 @@ func (a Allocator) simulateRemoveTarget( ) log.VEventf(ctx, 3, "simulating which non-voter would be removed after adding s%d", targetStore) - return a.RemoveNonVoter(ctx, conf, candidates, existingVoters, existingNonVoters) + return a.RemoveNonVoter(ctx, conf, candidates, existingVoters, existingNonVoters, options) default: panic(fmt.Sprintf("unknown targetReplicaType: %s", t)) } @@ -897,6 +898,7 @@ func (a Allocator) removeTarget( existingVoters []roachpb.ReplicaDescriptor, existingNonVoters []roachpb.ReplicaDescriptor, targetType targetReplicaType, + options scorerOptions, ) (roachpb.ReplicaDescriptor, string, error) { if len(candidates) == 0 { return roachpb.ReplicaDescriptor{}, "", errors.Errorf("must supply at least one" + @@ -914,7 +916,6 @@ func (a Allocator) removeTarget( existingReplicas, conf.NumReplicas, conf.Constraints) analyzedVoterConstraints := constraint.AnalyzeConstraints(ctx, a.storePool.getStoreDescriptor, existingVoters, conf.GetNumVoters(), conf.VoterConstraints) - options := a.scorerOptions() var constraintsChecker constraintsCheckFn switch t := targetType; t { @@ -945,7 +946,7 @@ func (a Allocator) removeTarget( for _, exist := range existingReplicas { if exist.StoreID == bad.store.StoreID { log.VEventf(ctx, 3, "remove target: %s", bad) - details := decisionDetails{Target: bad.compactString(options)} + details := decisionDetails{Target: bad.compactString()} detailsBytes, err := json.Marshal(details) if err != nil { log.Warningf(ctx, "failed to marshal details for choosing remove target: %+v", err) @@ -968,6 +969,7 @@ func (a Allocator) RemoveVoter( voterCandidates []roachpb.ReplicaDescriptor, existingVoters []roachpb.ReplicaDescriptor, existingNonVoters []roachpb.ReplicaDescriptor, + options scorerOptions, ) (roachpb.ReplicaDescriptor, string, error) { return a.removeTarget( ctx, @@ -976,6 +978,7 @@ func (a Allocator) RemoveVoter( existingVoters, existingNonVoters, voterTarget, + options, ) } @@ -990,6 +993,7 @@ func (a Allocator) RemoveNonVoter( nonVoterCandidates []roachpb.ReplicaDescriptor, existingVoters []roachpb.ReplicaDescriptor, existingNonVoters []roachpb.ReplicaDescriptor, + options scorerOptions, ) (roachpb.ReplicaDescriptor, string, error) { return a.removeTarget( ctx, @@ -998,6 +1002,7 @@ func (a Allocator) RemoveNonVoter( existingVoters, existingNonVoters, nonVoterTarget, + options, ) } @@ -1009,7 +1014,8 @@ func (a Allocator) rebalanceTarget( rangeUsageInfo RangeUsageInfo, filter storeFilter, targetType targetReplicaType, -) (add roachpb.ReplicationTarget, remove roachpb.ReplicationTarget, details string, ok bool) { + options scorerOptions, +) (add, remove roachpb.ReplicationTarget, details string, ok bool) { sl, _, _ := a.storePool.getStoreList(filter) existingReplicas := append(existingVoters, existingNonVoters...) @@ -1049,7 +1055,6 @@ func (a Allocator) rebalanceTarget( log.Fatalf(ctx, "unsupported targetReplicaType: %v", t) } - options := a.scorerOptions() replicaSetForDiversityCalc := getReplicasForDiversityCalc(targetType, existingVoters, existingReplicas) results := rankedCandidateListForRebalancing( ctx, @@ -1116,6 +1121,7 @@ func (a Allocator) rebalanceTarget( otherReplicaSet, rangeUsageInfo, targetType, + options, ) if err != nil { log.Warningf(ctx, "simulating removal of %s failed: %+v", targetType, err) @@ -1134,7 +1140,7 @@ func (a Allocator) rebalanceTarget( // Compile the details entry that will be persisted into system.rangelog for // debugging/auditability purposes. dDetails := decisionDetails{ - Target: target.compactString(options), + Target: target.compactString(), Existing: existingCandidates.compactString(options), } detailsBytes, err := json.Marshal(dDetails) @@ -1185,7 +1191,8 @@ func (a Allocator) RebalanceVoter( existingVoters, existingNonVoters []roachpb.ReplicaDescriptor, rangeUsageInfo RangeUsageInfo, filter storeFilter, -) (add roachpb.ReplicationTarget, remove roachpb.ReplicationTarget, details string, ok bool) { + options scorerOptions, +) (add, remove roachpb.ReplicationTarget, details string, ok bool) { return a.rebalanceTarget( ctx, conf, @@ -1195,6 +1202,7 @@ func (a Allocator) RebalanceVoter( rangeUsageInfo, filter, voterTarget, + options, ) } @@ -1217,7 +1225,8 @@ func (a Allocator) RebalanceNonVoter( existingVoters, existingNonVoters []roachpb.ReplicaDescriptor, rangeUsageInfo RangeUsageInfo, filter storeFilter, -) (add roachpb.ReplicationTarget, remove roachpb.ReplicationTarget, details string, ok bool) { + options scorerOptions, +) (add, remove roachpb.ReplicationTarget, details string, ok bool) { return a.rebalanceTarget( ctx, conf, @@ -1227,6 +1236,7 @@ func (a Allocator) RebalanceNonVoter( rangeUsageInfo, filter, nonVoterTarget, + options, ) } diff --git a/pkg/kv/kvserver/allocator_scorer.go b/pkg/kv/kvserver/allocator_scorer.go index 8f847fceb17d..086c6e523f6c 100644 --- a/pkg/kv/kvserver/allocator_scorer.go +++ b/pkg/kv/kvserver/allocator_scorer.go @@ -16,7 +16,6 @@ import ( "fmt" "math" "sort" - "strconv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/constraint" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -93,22 +92,6 @@ type scorerOptions struct { qpsRebalanceThreshold float64 // only considered if non-zero } -type balanceDimensions struct { - ranges rangeCountStatus -} - -func (bd *balanceDimensions) totalScore() float64 { - return float64(bd.ranges) -} - -func (bd balanceDimensions) String() string { - return strconv.Itoa(int(bd.ranges)) -} - -func (bd balanceDimensions) compactString(options scorerOptions) string { - return fmt.Sprintf("%d", bd.ranges) -} - // candidate store for allocation. type candidate struct { store roachpb.StoreDescriptor @@ -117,14 +100,14 @@ type candidate struct { necessary bool diversityScore float64 convergesScore int - balanceScore balanceDimensions + balanceScore balanceStatus rangeCount int details string } func (c candidate) String() string { str := fmt.Sprintf("s%d, valid:%t, fulldisk:%t, necessary:%t, diversity:%.2f, converges:%d, "+ - "balance:%s, rangeCount:%d, queriesPerSecond:%.2f", + "balance:%d, rangeCount:%d, queriesPerSecond:%.2f", c.store.StoreID, c.valid, c.fullDisk, c.necessary, c.diversityScore, c.convergesScore, c.balanceScore, c.rangeCount, c.store.Capacity.QueriesPerSecond) if c.details != "" { @@ -133,7 +116,7 @@ func (c candidate) String() string { return str } -func (c candidate) compactString(options scorerOptions) string { +func (c candidate) compactString() string { var buf bytes.Buffer fmt.Fprintf(&buf, "s%d", c.store.StoreID) if !c.valid { @@ -148,8 +131,8 @@ func (c candidate) compactString(options scorerOptions) string { if c.diversityScore != 0 { fmt.Fprintf(&buf, ", diversity:%.2f", c.diversityScore) } - fmt.Fprintf(&buf, ", converges:%d, balance:%s, rangeCount:%d", - c.convergesScore, c.balanceScore.compactString(options), c.rangeCount) + fmt.Fprintf(&buf, ", converges:%d, balance:%d, rangeCount:%d", + c.convergesScore, c.balanceScore, c.rangeCount) if c.details != "" { fmt.Fprintf(&buf, ", details:(%s)", c.details) } @@ -197,11 +180,11 @@ func (c candidate) compare(o candidate) float64 { } return -(2 + float64(o.convergesScore-c.convergesScore)/10.0) } - if !scoresAlmostEqual(c.balanceScore.totalScore(), o.balanceScore.totalScore()) { - if c.balanceScore.totalScore() > o.balanceScore.totalScore() { - return 1 + (c.balanceScore.totalScore()-o.balanceScore.totalScore())/10.0 + if c.balanceScore != o.balanceScore { + if c.balanceScore > o.balanceScore { + return 1 + (float64(c.balanceScore-o.balanceScore))/10.0 } - return -(1 + (o.balanceScore.totalScore()-c.balanceScore.totalScore())/10.0) + return -(1 + (float64(o.balanceScore-c.balanceScore))/10.0) } // Sometimes we compare partially-filled in candidates, e.g. those with // diversity scores filled in but not balance scores or range counts. This @@ -239,7 +222,7 @@ func (cl candidateList) compactString(options scorerOptions) string { buffer.WriteRune('[') for _, c := range cl { buffer.WriteRune('\n') - buffer.WriteString(c.compactString(options)) + buffer.WriteString(c.compactString()) } buffer.WriteRune(']') return buffer.String() @@ -263,7 +246,7 @@ func (c byScoreAndID) Len() int { return len(c) } func (c byScoreAndID) Less(i, j int) bool { if scoresAlmostEqual(c[i].diversityScore, c[j].diversityScore) && c[i].convergesScore == c[j].convergesScore && - scoresAlmostEqual(c[i].balanceScore.totalScore(), c[j].balanceScore.totalScore()) && + c[i].balanceScore == c[j].balanceScore && c[i].rangeCount == c[j].rangeCount && c[i].necessary == c[j].necessary && c[i].fullDisk == c[j].fullDisk && @@ -295,7 +278,8 @@ func (cl candidateList) best() candidateList { for i := 1; i < len(cl); i++ { if cl[i].necessary == cl[0].necessary && scoresAlmostEqual(cl[i].diversityScore, cl[0].diversityScore) && - cl[i].convergesScore == cl[0].convergesScore { + cl[i].convergesScore == cl[0].convergesScore && + cl[i].balanceScore == cl[0].balanceScore { continue } return cl[:i] @@ -325,11 +309,12 @@ func (cl candidateList) worst() candidateList { } } } - // Find the worst constraint/locality/converges values. + // Find the worst constraint/locality/converges/balanceScore values. for i := len(cl) - 2; i >= 0; i-- { if cl[i].necessary == cl[len(cl)-1].necessary && scoresAlmostEqual(cl[i].diversityScore, cl[len(cl)-1].diversityScore) && - cl[i].convergesScore == cl[len(cl)-1].convergesScore { + cl[i].convergesScore == cl[len(cl)-1].convergesScore && + cl[i].balanceScore == cl[len(cl)-1].balanceScore { continue } return cl[i+1:] @@ -455,13 +440,13 @@ func rankedCandidateListForAllocation( balanceScore := balanceScore(candidateStores, s.Capacity, options) var convergesScore int if options.qpsRebalanceThreshold > 0 { - if s.Capacity.QueriesPerSecond < underfullThreshold( - candidateStores.candidateQueriesPerSecond.mean, options.qpsRebalanceThreshold) { + if s.Capacity.QueriesPerSecond < underfullQPSThreshold( + options, candidateStores.candidateQueriesPerSecond.mean) { convergesScore = 1 } else if s.Capacity.QueriesPerSecond < candidateStores.candidateQueriesPerSecond.mean { convergesScore = 0 - } else if s.Capacity.QueriesPerSecond < overfullThreshold( - candidateStores.candidateQueriesPerSecond.mean, options.qpsRebalanceThreshold) { + } else if s.Capacity.QueriesPerSecond < overfullQPSThreshold( + options, candidateStores.candidateQueriesPerSecond.mean) { convergesScore = -1 } else { convergesScore = -2 @@ -512,7 +497,7 @@ func rankedCandidateListForRemoval( // adding 1 to the constraint score. Note that when selecting a // candidate for removal the candidates with the lowest scores are // more likely to be removed. - convergesScore := rebalanceFromConvergesScore(sl, s.Capacity) + convergesScore := rebalanceFromConvergesScore(sl, s.Capacity, options) balanceScore := balanceScore(sl, s.Capacity, options) candidates = append(candidates, candidate{ store: s, @@ -719,7 +704,7 @@ func rankedCandidateListForRebalancing( // // TODO(a-robinson): Some moderate refactoring could extract this logic // out into the loop below, avoiding duplicate balanceScore calculations. - if shouldRebalanceBasedOnRangeCount(ctx, existing.store, sl, options) { + if shouldRebalanceBasedOnThresholds(ctx, existing.store, sl, options) { shouldRebalanceCheck = true break } @@ -752,7 +737,7 @@ func rankedCandidateListForRebalancing( // Similarly to in rankedCandidateListForRemoval, any replica whose // removal would not converge the range stats to their means is given a // constraint score boost of 1 to make it less attractive for removal. - convergesScore := rebalanceFromConvergesScore(comparable.sl, existing.store.Capacity) + convergesScore := rebalanceFromConvergesScore(comparable.sl, existing.store.Capacity, options) existing.convergesScore = convergesScore existing.balanceScore = balanceScore existing.rangeCount = int(existing.store.Capacity.RangeCount) @@ -771,14 +756,7 @@ func rankedCandidateListForRebalancing( s := cand.store cand.fullDisk = !rebalanceToMaxCapacityCheck(s) cand.balanceScore = balanceScore(comparable.sl, s.Capacity, options) - cand.convergesScore = rebalanceToConvergesScore(comparable.sl, s.Capacity) - if !needRebalance && cand.convergesScore == 0 { - // Only consider this candidate if we must rebalance due to constraint, - // disk fullness, or diversity reasons. - log.VEventf(ctx, 3, "not considering %+v as a candidate for range %+v: score=%s storeList=%+v", - s, existingReplicasForType, cand.balanceScore, comparable.sl) - continue - } + cand.convergesScore = rebalanceToConvergesScore(comparable.sl, s.Capacity, options) cand.rangeCount = int(s.Capacity.RangeCount) candidates = append(candidates, cand) } @@ -874,12 +852,54 @@ func betterRebalanceTarget(target1, existing1, target2, existing2 *candidate) *c return target1 } -// shouldRebalanceBasedOnRangeCount returns whether the specified store is a +// shouldRebalanceBasedOnThresholds returns whether the specified store is a // candidate for having a replica removed from it given the candidate store -// list. -func shouldRebalanceBasedOnRangeCount( +// list. This method returns true if there are any ranges that are on stores +// that lie _outside of the [underfullThreshold, overfullThreshold] window_ for +// the given signal (QPS or range count). +// +// NB: If the given `options` have `qpsRebalanceThreshold` set, this method +// makes its determination based on QPS. Otherwise, we fall back on using range +// count as a signal. +func shouldRebalanceBasedOnThresholds( ctx context.Context, store roachpb.StoreDescriptor, sl StoreList, options scorerOptions, ) bool { + if options.qpsRebalanceThreshold > 0 { + overfullThreshold := overfullQPSThreshold(options, sl.candidateQueriesPerSecond.mean) + if store.Capacity.QueriesPerSecond > overfullThreshold { + log.VEventf( + ctx, + 2, + "s%d: should-rebalance(QPS-overfull): QPS=%.2f, mean=%.2f, overfull-threshold=%.2f", + store.StoreID, + store.Capacity.QueriesPerSecond, + sl.candidateQueriesPerSecond.mean, + overfullThreshold, + ) + return true + } + if store.Capacity.QueriesPerSecond > sl.candidateQueriesPerSecond.mean { + underfullThreshold := underfullQPSThreshold(options, sl.candidateQueriesPerSecond.mean) + for _, desc := range sl.stores { + if desc.Capacity.QueriesPerSecond < underfullThreshold { + log.VEventf( + ctx, + 2, + "s%d: should-rebalance(better-fit-QPS=s%d): QPS=%.2f, otherQPS=%.2f, mean=%.2f, underfull-threshold=%.2f", + store.StoreID, + desc.StoreID, + store.Capacity.QueriesPerSecond, + desc.Capacity.QueriesPerSecond, + sl.candidateQueriesPerSecond.mean, + underfullThreshold, + ) + return true + } + } + } + return false + } + overfullThreshold := int32(math.Ceil(overfullRangeThreshold(options, sl.candidateRanges.mean))) if store.Capacity.RangeCount > overfullThreshold { log.VEventf(ctx, 2, @@ -887,7 +907,6 @@ func shouldRebalanceBasedOnRangeCount( store.StoreID, store.Capacity.RangeCount, sl.candidateRanges.mean, overfullThreshold) return true } - if float64(store.Capacity.RangeCount) > sl.candidateRanges.mean { underfullThreshold := int32(math.Floor(underfullRangeThreshold(options, sl.candidateRanges.mean))) for _, desc := range sl.stores { @@ -901,7 +920,6 @@ func shouldRebalanceBasedOnRangeCount( } } } - // If we reached this point, we're happy with the range where it is. return false } @@ -1324,68 +1342,114 @@ func diversityRebalanceFromScore( return sumScore / float64(numSamples) } -type rangeCountStatus int +type balanceStatus int const ( - overfull rangeCountStatus = -1 - balanced rangeCountStatus = 0 - underfull rangeCountStatus = 1 + overfull balanceStatus = -2 + moreThanMean balanceStatus = -1 + lessThanMean balanceStatus = 0 + underfull balanceStatus = 1 ) // balanceScore returns an arbitrarily scaled score where higher scores are for -// stores where the range is a better fit based on various balance factors -// like range count, disk usage, and QPS. -func balanceScore(sl StoreList, sc roachpb.StoreCapacity, options scorerOptions) balanceDimensions { - var dimensions balanceDimensions - if float64(sc.RangeCount) > overfullRangeThreshold(options, sl.candidateRanges.mean) { - dimensions.ranges = overfull - } else if float64(sc.RangeCount) < underfullRangeThreshold(options, sl.candidateRanges.mean) { - dimensions.ranges = underfull +// stores where the range is a better fit based on balance factors like range +// count or QPS. +// +// If the given options have qpsRebalanceThreshold set, we use that for +// computing the balanceScore, otherwise we use the `RangeCount`. +// +// TODO(aayush): It would be nice to be able to compose the two dimensions of +// balanceScore (QPS or RangeCount) and allow the `options` to simply specify an +// order of precedence. Callers who care about the balancing of QPS across +// stores (like the StoreRebalancer) could ask for QPS to take precedence over +// RangeCount, and vice-versa. +func balanceScore(sl StoreList, sc roachpb.StoreCapacity, options scorerOptions) balanceStatus { + if options.qpsRebalanceThreshold > 0 { + maxQPS := overfullQPSThreshold(options, sl.candidateQueriesPerSecond.mean) + minQPS := underfullQPSThreshold(options, sl.candidateQueriesPerSecond.mean) + curQPS := sc.QueriesPerSecond + if curQPS < minQPS { + return underfull + } else if curQPS < sl.candidateQueriesPerSecond.mean { + return lessThanMean + } else if curQPS < maxQPS { + return moreThanMean + } else { + return overfull + } + } + maxRangeCount := overfullRangeThreshold(options, sl.candidateRanges.mean) + minRangeCount := underfullRangeThreshold(options, sl.candidateRanges.mean) + curRangeCount := float64(sc.RangeCount) + if curRangeCount < minRangeCount { + return underfull + } else if curRangeCount < sl.candidateRanges.mean { + return lessThanMean + } else if curRangeCount < maxRangeCount { + return moreThanMean } else { - dimensions.ranges = balanced + return overfull } - return dimensions } func overfullRangeThreshold(options scorerOptions, mean float64) float64 { - return overfullThreshold(mean, options.rangeRebalanceThreshold) + return mean + math.Max(mean*options.rangeRebalanceThreshold, minRangeRebalanceThreshold) } func underfullRangeThreshold(options scorerOptions, mean float64) float64 { - return underfullThreshold(mean, options.rangeRebalanceThreshold) + return mean - math.Max(mean*options.rangeRebalanceThreshold, minRangeRebalanceThreshold) } -func overfullThreshold(mean float64, thresholdFraction float64) float64 { - return mean + math.Max(mean*thresholdFraction, minRangeRebalanceThreshold) +func overfullQPSThreshold(options scorerOptions, mean float64) float64 { + return math.Max(mean*(1+options.qpsRebalanceThreshold), mean+minQPSThresholdDifference) } -func underfullThreshold(mean float64, thresholdFraction float64) float64 { - return mean - math.Max(mean*thresholdFraction, minRangeRebalanceThreshold) +func underfullQPSThreshold(options scorerOptions, mean float64) float64 { + return math.Min(mean*(1-options.qpsRebalanceThreshold), mean-minQPSThresholdDifference) } -// rebalanceFromConvergesScore returns 1 if rebalancing a replica away from `sc` -// will _not_ converge its range count towards the mean range count of stores in -// `sl`. When we're considering whether to rebalance a replica away from a store -// or not, we want to give it a "boost" (i.e. make it a less likely candidate -// for removal) if it doesn't further our goal to converge range counts towards -// the mean. -func rebalanceFromConvergesScore(sl StoreList, sc roachpb.StoreCapacity) int { - if rebalanceConvergesOnMean(sl, sc, sc.RangeCount-1) { +// rebalanceFromConvergesScore returns a 1 iff rebalancing a replica away from +// `sc` will _not_ converge its range count towards the mean of stores in `sl`. +// When we're considering whether to rebalance a replica away from a store or +// not, we want to give it a "boost" (i.e. make it a less likely candidate for +// removal) if it doesn't further our goal to converge range count towards the +// mean. +func rebalanceFromConvergesScore( + sl StoreList, sc roachpb.StoreCapacity, options scorerOptions, +) int { + if options.qpsRebalanceThreshold > 0 { + // If `qpsRebalanceThreshold` is set, we disable the `convergesScore` + // because we only have access to the QPS of the current (i.e. usually the + // leaseholder) replica. We do not have access to the QPS of other replicas, + // and when this method is called, the caller doesn't yet know which replica + // is going to be rebalanced. Thus, we cannot actually compute the + // `convergesScroe` when rebalancing off of QPS. + return 0 + } + if rebalanceConvergesRangeCountOnMean(sl, sc, sc.RangeCount-1) { return 0 } return 1 } // rebalanceToConvergesScore returns 1 if rebalancing a replica to `sc` will -// converge its range count towards the mean of all the stores inside `sl`. -func rebalanceToConvergesScore(sl StoreList, sc roachpb.StoreCapacity) int { - if rebalanceConvergesOnMean(sl, sc, sc.RangeCount+1) { +// converge its range count towards the mean of all the stores inside +// `sl`. +func rebalanceToConvergesScore(sl StoreList, sc roachpb.StoreCapacity, options scorerOptions) int { + if options.qpsRebalanceThreshold > 0 { + // if `qpsRebalanceThreshold` is set, we disable the `convergesScore`. + // See comment inside `rebalanceFromConvergesScore()` for details. + return 0 + } + if rebalanceConvergesRangeCountOnMean(sl, sc, sc.RangeCount+1) { return 1 } return 0 } -func rebalanceConvergesOnMean(sl StoreList, sc roachpb.StoreCapacity, newRangeCount int32) bool { +func rebalanceConvergesRangeCountOnMean( + sl StoreList, sc roachpb.StoreCapacity, newRangeCount int32, +) bool { return convergesOnMean(float64(sc.RangeCount), float64(newRangeCount), sl.candidateRanges.mean) } diff --git a/pkg/kv/kvserver/allocator_scorer_test.go b/pkg/kv/kvserver/allocator_scorer_test.go index 062b8241d707..efb4afff4411 100644 --- a/pkg/kv/kvserver/allocator_scorer_test.go +++ b/pkg/kv/kvserver/allocator_scorer_test.go @@ -1193,7 +1193,7 @@ func TestShouldRebalanceDiversity(t *testing.T) { actual := len(targets) > 0 if actual != tc.expected { t.Errorf( - "%d: shouldRebalanceBasedOnRangeCount on s%d with replicas on %v got %t, expected %t", + "%d: shouldRebalanceBasedOnThresholds on s%d with replicas on %v got %t, expected %t", i, tc.s.StoreID, tc.existingNodeIDs, @@ -1489,21 +1489,59 @@ func TestBalanceScore(t *testing.T) { testCases := []struct { sc roachpb.StoreCapacity - expected float64 + expected balanceStatus + }{ + {sEmpty, underfull}, + {sMean, moreThanMean}, + {sRangesOverfull, overfull}, + {sRangesUnderfull, underfull}, + } + for i, tc := range testCases { + if a, e := balanceScore(storeList, tc.sc, options), tc.expected; a != e { + t.Errorf("%d: balanceScore(storeList, %+v) got %d; want %d", i, tc.sc, a, e) + } + } +} + +func TestRebalanceBalanceScoreOnQPS(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + storeList := StoreList{ + candidateQueriesPerSecond: stat{mean: 1000}, + } + + testCases := []struct { + QPS float64 + expBalanceScore balanceStatus }{ - {sEmpty, 1}, - {sMean, 0}, - {sRangesOverfull, -1}, - {sRangesUnderfull, 1}, + {0, underfull}, + {900, lessThanMean}, + {999, lessThanMean}, + {1000, moreThanMean}, + {1001, moreThanMean}, + {2000, overfull}, } + for i, tc := range testCases { - if a, e := balanceScore(storeList, tc.sc, options), tc.expected; a.totalScore() != e { - t.Errorf("%d: balanceScore(storeList, %+v) got %s; want %.2f", i, tc.sc, a, e) + sc := roachpb.StoreCapacity{ + QueriesPerSecond: tc.QPS, + } + options := scorerOptions{ + qpsRebalanceThreshold: 0.1, + } + if a, e := balanceScore(storeList, sc, options), tc.expBalanceScore; a != e { + t.Errorf("%d: rebalanceToConvergesScore(storeList, %+v) got %d; want %d", i, sc, a, e) + } + // NB: Any replica whose removal would not converge the QPS to the mean is + // given a score of 1 to make it less attractive for removal. + if a, e := balanceScore(storeList, sc, options), tc.expBalanceScore; a != e { + t.Errorf("%d: rebalanceFromConvergesScore(storeList, %+v) got %d; want %d", i, sc, a, e) } } } -func TestRebalanceConvergesOnMean(t *testing.T) { +func TestRebalanceConvergesRangeCountOnMean(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -1528,12 +1566,16 @@ func TestRebalanceConvergesOnMean(t *testing.T) { sc := roachpb.StoreCapacity{ RangeCount: tc.rangeCount, } - if a, e := rebalanceToConvergesScore(storeList, sc) == 1, tc.toConverges; a != e { + if a, e := rebalanceToConvergesScore( + storeList, sc, scorerOptions{}, + ) == 1, tc.toConverges; a != e { t.Errorf("%d: rebalanceToConvergesScore(storeList, %+v) got %t; want %t", i, sc, a, e) } // NB: Any replica whose removal would not converge the range count to the // mean is given a score of 1 to make it less attractive for removal. - if a, e := rebalanceFromConvergesScore(storeList, sc) == 0, tc.fromConverges; a != e { + if a, e := rebalanceFromConvergesScore( + storeList, sc, scorerOptions{}, + ) == 0, tc.fromConverges; a != e { t.Errorf("%d: rebalanceFromConvergesScore(storeList, %+v) got %t; want %t", i, sc, a, e) } } diff --git a/pkg/kv/kvserver/allocator_test.go b/pkg/kv/kvserver/allocator_test.go index f8345b12b178..7925b998ac5f 100644 --- a/pkg/kv/kvserver/allocator_test.go +++ b/pkg/kv/kvserver/allocator_test.go @@ -713,6 +713,7 @@ func TestAllocatorMultipleStoresPerNode(t *testing.T) { nil, rangeUsageInfo, storeFilterThrottled, + a.scorerOptions(), ) if e, a := tc.expectTargetRebalance, ok; e != a { t.Errorf( @@ -777,7 +778,16 @@ func TestAllocatorMultipleStoresPerNodeLopsided(t *testing.T) { // After that we should not be seeing replicas move. var rangeUsageInfo RangeUsageInfo for i := 1; i < 40; i++ { - add, remove, _, ok := a.RebalanceVoter(context.Background(), emptySpanConfig(), nil, ranges[i].InternalReplicas, nil, rangeUsageInfo, storeFilterThrottled) + add, remove, _, ok := a.RebalanceVoter( + context.Background(), + emptySpanConfig(), + nil, + ranges[i].InternalReplicas, + nil, + rangeUsageInfo, + storeFilterThrottled, + a.scorerOptions(), + ) if ok { // Update the descriptor. newReplicas := make([]roachpb.ReplicaDescriptor, 0, len(ranges[i].InternalReplicas)) @@ -809,7 +819,16 @@ func TestAllocatorMultipleStoresPerNodeLopsided(t *testing.T) { // We dont expect any range wanting to move since the system should have // reached a stable state at this point. for i := 1; i < 40; i++ { - _, _, _, ok := a.RebalanceVoter(context.Background(), emptySpanConfig(), nil, ranges[i].InternalReplicas, nil, rangeUsageInfo, storeFilterThrottled) + _, _, _, ok := a.RebalanceVoter( + context.Background(), + emptySpanConfig(), + nil, + ranges[i].InternalReplicas, + nil, + rangeUsageInfo, + storeFilterThrottled, + a.scorerOptions(), + ) require.False(t, ok) } } @@ -872,7 +891,16 @@ func TestAllocatorRebalance(t *testing.T) { // Every rebalance target must be either store 1 or 2. for i := 0; i < 10; i++ { var rangeUsageInfo RangeUsageInfo - target, _, _, ok := a.RebalanceVoter(ctx, emptySpanConfig(), nil, []roachpb.ReplicaDescriptor{{NodeID: 3, StoreID: 3}}, nil, rangeUsageInfo, storeFilterThrottled) + target, _, _, ok := a.RebalanceVoter( + ctx, + emptySpanConfig(), + nil, + []roachpb.ReplicaDescriptor{{NodeID: 3, StoreID: 3}}, + nil, + rangeUsageInfo, + storeFilterThrottled, + a.scorerOptions(), + ) if !ok { i-- // loop until we find 10 candidates continue @@ -884,14 +912,14 @@ func TestAllocatorRebalance(t *testing.T) { } } - // Verify shouldRebalanceBasedOnRangeCount results. + // Verify shouldRebalanceBasedOnThresholds results. for i, store := range stores { desc, ok := a.storePool.getStoreDescriptor(store.StoreID) if !ok { t.Fatalf("%d: unable to get store %d descriptor", i, store.StoreID) } sl, _, _ := a.storePool.getStoreList(storeFilterThrottled) - result := shouldRebalanceBasedOnRangeCount(ctx, desc, sl, a.scorerOptions()) + result := shouldRebalanceBasedOnThresholds(ctx, desc, sl, a.scorerOptions()) if expResult := (i >= 2); expResult != result { t.Errorf("%d: expected rebalance %t; got %t; desc %+v; sl: %+v", i, expResult, result, desc, sl) } @@ -1016,7 +1044,16 @@ func TestAllocatorRebalanceTarget(t *testing.T) { } } for i := 0; i < 10; i++ { - result, _, details, ok := a.RebalanceVoter(context.Background(), emptySpanConfig(), status, replicas, nil, rangeUsageInfo, storeFilterThrottled) + result, _, details, ok := a.RebalanceVoter( + context.Background(), + emptySpanConfig(), + status, + replicas, + nil, + rangeUsageInfo, + storeFilterThrottled, + a.scorerOptions(), + ) if ok { t.Fatalf("expected no rebalance, but got target s%d; details: %s", result.StoreID, details) } @@ -1029,7 +1066,16 @@ func TestAllocatorRebalanceTarget(t *testing.T) { stores[2].Capacity.RangeCount = 46 sg.GossipStores(stores, t) for i := 0; i < 10; i++ { - target, _, details, ok := a.RebalanceVoter(context.Background(), emptySpanConfig(), status, replicas, nil, rangeUsageInfo, storeFilterThrottled) + target, _, details, ok := a.RebalanceVoter( + context.Background(), + emptySpanConfig(), + status, + replicas, + nil, + rangeUsageInfo, + storeFilterThrottled, + a.scorerOptions(), + ) if ok { t.Fatalf("expected no rebalance, but got target s%d; details: %s", target.StoreID, details) } @@ -1039,7 +1085,16 @@ func TestAllocatorRebalanceTarget(t *testing.T) { stores[1].Capacity.RangeCount = 44 sg.GossipStores(stores, t) for i := 0; i < 10; i++ { - target, origin, details, ok := a.RebalanceVoter(context.Background(), emptySpanConfig(), status, replicas, nil, rangeUsageInfo, storeFilterThrottled) + target, origin, details, ok := a.RebalanceVoter( + context.Background(), + emptySpanConfig(), + status, + replicas, + nil, + rangeUsageInfo, + storeFilterThrottled, + a.scorerOptions(), + ) expTo := stores[1].StoreID expFrom := stores[0].StoreID if !ok || target.StoreID != expTo || origin.StoreID != expFrom { @@ -1109,7 +1164,16 @@ func TestAllocatorRebalanceDeadNodes(t *testing.T) { for _, c := range testCases { t.Run("", func(t *testing.T) { var rangeUsageInfo RangeUsageInfo - target, _, _, ok := a.RebalanceVoter(ctx, emptySpanConfig(), nil, c.existing, nil, rangeUsageInfo, storeFilterThrottled) + target, _, _, ok := a.RebalanceVoter( + ctx, + emptySpanConfig(), + nil, + c.existing, + nil, + rangeUsageInfo, + storeFilterThrottled, + a.scorerOptions(), + ) if c.expected > 0 { if !ok { t.Fatalf("expected %d, but found nil", c.expected) @@ -1245,20 +1309,131 @@ func TestAllocatorRebalanceThrashing(t *testing.T) { }) sl, _, _ := a.storePool.getStoreList(storeFilterThrottled) - // Verify shouldRebalanceBasedOnRangeCount returns the expected value. + // Verify shouldRebalanceBasedOnThresholds returns the expected value. for j, store := range stores { desc, ok := a.storePool.getStoreDescriptor(store.StoreID) if !ok { t.Fatalf("[store %d]: unable to get store %d descriptor", j, store.StoreID) } - if a, e := shouldRebalanceBasedOnRangeCount(context.Background(), desc, sl, a.scorerOptions()), cluster[j].shouldRebalanceFrom; a != e { - t.Errorf("[store %d]: shouldRebalanceBasedOnRangeCount %t != expected %t", store.StoreID, a, e) + if a, e := shouldRebalanceBasedOnThresholds(context.Background(), desc, sl, a.scorerOptions()), cluster[j].shouldRebalanceFrom; a != e { + t.Errorf("[store %d]: shouldRebalanceBasedOnThresholds %t != expected %t", store.StoreID, a, e) } } }) } } +// TestAllocatorRebalanceByQPS verifies that rebalance targets are chosen by QPS +// in the event that some stores lie outside the [underfull, overfull] window +// for QPS. +func TestAllocatorRebalanceByQPS(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + allStoresEqual := []*roachpb.StoreDescriptor{ + { + StoreID: 1, + Node: roachpb.NodeDescriptor{NodeID: 1}, + Capacity: roachpb.StoreCapacity{RangeCount: 1000, QueriesPerSecond: 1000}, + }, + { + StoreID: 2, + Node: roachpb.NodeDescriptor{NodeID: 2}, + Capacity: roachpb.StoreCapacity{RangeCount: 1000, QueriesPerSecond: 1000}, + }, + { + StoreID: 3, + Node: roachpb.NodeDescriptor{NodeID: 3}, + Capacity: roachpb.StoreCapacity{RangeCount: 1000, QueriesPerSecond: 1000}, + }, + { + StoreID: 4, + Node: roachpb.NodeDescriptor{NodeID: 4}, + Capacity: roachpb.StoreCapacity{RangeCount: 1000, QueriesPerSecond: 1000}, + }, + } + oneOverfullAndOneUnderfull := []*roachpb.StoreDescriptor{ + { + StoreID: 1, + Node: roachpb.NodeDescriptor{NodeID: 1}, + Capacity: roachpb.StoreCapacity{RangeCount: 1000, QueriesPerSecond: 1200}, + }, + { + StoreID: 2, + Node: roachpb.NodeDescriptor{NodeID: 2}, + Capacity: roachpb.StoreCapacity{RangeCount: 1000, QueriesPerSecond: 1000}, + }, + { + StoreID: 3, + Node: roachpb.NodeDescriptor{NodeID: 3}, + Capacity: roachpb.StoreCapacity{RangeCount: 1000, QueriesPerSecond: 1000}, + }, + { + StoreID: 4, + Node: roachpb.NodeDescriptor{NodeID: 4}, + Capacity: roachpb.StoreCapacity{RangeCount: 1000, QueriesPerSecond: 800}, + }, + } + + type testCase struct { + testStores []*roachpb.StoreDescriptor + expectRebalance bool + expectedAddTarget, expectedRemoveTarget roachpb.StoreID + } + tests := []testCase{ + { + // We don't expect any QPS based rebalancing when all stores are serving + // the same QPS. + testStores: allStoresEqual, + expectRebalance: false, + }, + { + // When one store is overfull and another is underfull, we expect a QPS + // based rebalance from the overfull store to the underfull store. + testStores: oneOverfullAndOneUnderfull, + expectRebalance: true, + expectedRemoveTarget: roachpb.StoreID(1), + expectedAddTarget: roachpb.StoreID(4), + }, + } + + for _, subtest := range tests { + stopper, g, _, a, _ := createTestAllocator(10, false /* deterministic */) + defer stopper.Stop(context.Background()) + gossiputil.NewStoreGossiper(g).GossipStores(subtest.testStores, t) + ctx := context.Background() + var rangeUsageInfo RangeUsageInfo + options := scorerOptions{ + qpsRebalanceThreshold: 0.1, + } + add, remove, _, ok := a.RebalanceVoter( + ctx, + emptySpanConfig(), + nil, + []roachpb.ReplicaDescriptor{{StoreID: subtest.testStores[0].StoreID}}, + nil, + rangeUsageInfo, + storeFilterThrottled, + options, + ) + if subtest.expectRebalance { + require.True(t, ok) + require.Equal(t, subtest.expectedAddTarget, add.StoreID) + require.Equal(t, subtest.expectedRemoveTarget, remove.StoreID) + // Verify shouldRebalanceBasedOnThresholds results. + desc, ok := a.storePool.getStoreDescriptor(remove.StoreID) + if !ok { + t.Fatalf("unable to get store %d descriptor", remove.StoreID) + } + sl, _, _ := a.storePool.getStoreList(storeFilterThrottled) + result := shouldRebalanceBasedOnThresholds(ctx, desc, sl, options) + require.True(t, result) + } else { + require.False(t, ok) + } + } +} + // TestAllocatorRebalanceByCount verifies that rebalance targets are // chosen by range counts in the event that available capacities // exceed the maxAvailCapacityThreshold. @@ -1299,20 +1474,29 @@ func TestAllocatorRebalanceByCount(t *testing.T) { // Every rebalance target must be store 4 (or nil for case of missing the only option). for i := 0; i < 10; i++ { var rangeUsageInfo RangeUsageInfo - result, _, _, ok := a.RebalanceVoter(ctx, emptySpanConfig(), nil, []roachpb.ReplicaDescriptor{{StoreID: stores[0].StoreID}}, nil, rangeUsageInfo, storeFilterThrottled) + result, _, _, ok := a.RebalanceVoter( + ctx, + emptySpanConfig(), + nil, + []roachpb.ReplicaDescriptor{{StoreID: stores[0].StoreID}}, + nil, + rangeUsageInfo, + storeFilterThrottled, + a.scorerOptions(), + ) if ok && result.StoreID != 4 { t.Errorf("expected store 4; got %d", result.StoreID) } } - // Verify shouldRebalanceBasedOnRangeCount results. + // Verify shouldRebalanceBasedOnThresholds results. for i, store := range stores { desc, ok := a.storePool.getStoreDescriptor(store.StoreID) if !ok { t.Fatalf("%d: unable to get store %d descriptor", i, store.StoreID) } sl, _, _ := a.storePool.getStoreList(storeFilterThrottled) - result := shouldRebalanceBasedOnRangeCount(ctx, desc, sl, a.scorerOptions()) + result := shouldRebalanceBasedOnThresholds(ctx, desc, sl, a.scorerOptions()) if expResult := (i < 3); expResult != result { t.Errorf("%d: expected rebalance %t; got %t", i, expResult, result) } @@ -1843,7 +2027,15 @@ func TestAllocatorRebalanceDifferentLocalitySizes(t *testing.T) { for i, tc := range testCases { var rangeUsageInfo RangeUsageInfo - result, _, details, ok := a.RebalanceVoter(ctx, emptySpanConfig(), nil, tc.existing, nil, rangeUsageInfo, storeFilterThrottled) + result, _, details, ok := a.RebalanceVoter( + ctx, + emptySpanConfig(), + nil, + tc.existing, + nil, + rangeUsageInfo, + storeFilterThrottled, + a.scorerOptions()) var resultID roachpb.StoreID if ok { resultID = result.StoreID @@ -1906,7 +2098,16 @@ func TestAllocatorRebalanceDifferentLocalitySizes(t *testing.T) { for i, tc := range testCases2 { log.Infof(ctx, "case #%d", i) var rangeUsageInfo RangeUsageInfo - result, _, details, ok := a.RebalanceVoter(ctx, emptySpanConfig(), nil, tc.existing, nil, rangeUsageInfo, storeFilterThrottled) + result, _, details, ok := a.RebalanceVoter( + ctx, + emptySpanConfig(), + nil, + tc.existing, + nil, + rangeUsageInfo, + storeFilterThrottled, + a.scorerOptions(), + ) var gotExpected bool if !ok { gotExpected = (tc.expected == nil) @@ -2419,6 +2620,7 @@ func TestAllocatorRemoveBasedOnDiversity(t *testing.T) { c.existingVoters, /* voterCandidates */ c.existingVoters, c.existingNonVoters, + a.scorerOptions(), ) require.NoError(t, err) @@ -2434,9 +2636,10 @@ func TestAllocatorRemoveBasedOnDiversity(t *testing.T) { targetVoter, _, err = a.RemoveVoter( context.Background(), emptySpanConfig(), - c.existingVoters, /* voterCandidates */ c.existingVoters, - nil, /* existingNonVoters */ + c.existingVoters, + nil, + a.scorerOptions(), ) require.NoError(t, err) require.Truef(t, checkReplExists(targetVoter, c.expVoterRemovals), @@ -2449,6 +2652,7 @@ func TestAllocatorRemoveBasedOnDiversity(t *testing.T) { c.existingNonVoters, /* nonVoterCandidates */ c.existingVoters, c.existingNonVoters, + a.scorerOptions(), ) require.NoError(t, err) require.True(t, checkReplExists(targetNonVoter, c.expNonVoterRemovals)) @@ -2714,7 +2918,16 @@ func TestAllocatorRebalanceTargetLocality(t *testing.T) { } } var rangeUsageInfo RangeUsageInfo - target, _, details, ok := a.RebalanceVoter(context.Background(), emptySpanConfig(), nil, existingRepls, nil, rangeUsageInfo, storeFilterThrottled) + target, _, details, ok := a.RebalanceVoter( + context.Background(), + emptySpanConfig(), + nil, + existingRepls, + nil, + rangeUsageInfo, + storeFilterThrottled, + a.scorerOptions(), + ) if !ok { t.Fatalf("%d: RebalanceVoter(%v) returned no target store; details: %s", i, c.existing, details) } @@ -3680,7 +3893,9 @@ func TestAllocatorRebalanceNonVoters(t *testing.T) { test.existingVoters, test.existingNonVoters, rangeUsageInfo, - storeFilterThrottled) + storeFilterThrottled, + a.scorerOptions(), + ) if test.expectNoAction { require.True(t, !ok) } else { @@ -3736,6 +3951,7 @@ func TestVotersCanRebalanceToNonVoterStores(t *testing.T) { existingNonVoters, rangeUsageInfo, storeFilterThrottled, + a.scorerOptions(), ) require.Truef(t, ok, "no action taken") @@ -4556,7 +4772,16 @@ func TestRebalanceCandidatesNumReplicasConstraints(t *testing.T) { } else { // Also verify that RebalanceVoter picks out one of the best options as // the final rebalance choice. - target, _, details, ok := a.RebalanceVoter(context.Background(), conf, nil, existingRepls, nil, rangeUsageInfo, storeFilterThrottled) + target, _, details, ok := a.RebalanceVoter( + context.Background(), + conf, + nil, + existingRepls, + nil, + rangeUsageInfo, + storeFilterThrottled, + a.scorerOptions(), + ) var found bool if !ok && len(tc.validTargets) == 0 { found = true @@ -4932,8 +5157,14 @@ func TestAllocatorRemoveTargetBasedOnCapacity(t *testing.T) { // Repeat this test 10 times, it should always be either store 2 or 3. for i := 0; i < 10; i++ { - targetRepl, _, err := a.RemoveVoter(ctx, emptySpanConfig(), replicas, replicas, - nil) + targetRepl, _, err := a.RemoveVoter( + ctx, + emptySpanConfig(), + replicas, + replicas, + nil, + a.scorerOptions(), + ) if err != nil { t.Fatal(err) } @@ -6816,7 +7047,16 @@ func TestAllocatorRebalanceAway(t *testing.T) { } var rangeUsageInfo RangeUsageInfo - actual, _, _, ok := a.RebalanceVoter(ctx, roachpb.SpanConfig{Constraints: []roachpb.ConstraintsConjunction{constraints}}, nil, existingReplicas, nil, rangeUsageInfo, storeFilterThrottled) + actual, _, _, ok := a.RebalanceVoter( + ctx, + roachpb.SpanConfig{Constraints: []roachpb.ConstraintsConjunction{constraints}}, + nil, + existingReplicas, + nil, + rangeUsageInfo, + storeFilterThrottled, + a.scorerOptions(), + ) if tc.expected == nil && ok { t.Errorf("rebalancing to the incorrect store, expected nil, got %d", actual.StoreID) @@ -6976,7 +7216,16 @@ func TestAllocatorFullDisks(t *testing.T) { // Rebalance until there's no more rebalancing to do. if ts.Capacity.RangeCount > 0 { var rangeUsageInfo RangeUsageInfo - target, _, details, ok := alloc.RebalanceVoter(ctx, emptySpanConfig(), nil, []roachpb.ReplicaDescriptor{{NodeID: ts.Node.NodeID, StoreID: ts.StoreID}}, nil, rangeUsageInfo, storeFilterThrottled) + target, _, details, ok := alloc.RebalanceVoter( + ctx, + emptySpanConfig(), + nil, + []roachpb.ReplicaDescriptor{{NodeID: ts.Node.NodeID, StoreID: ts.StoreID}}, + nil, + rangeUsageInfo, + storeFilterThrottled, + alloc.scorerOptions(), + ) if ok { if log.V(1) { log.Infof(ctx, "rebalancing to %v; details: %s", target, details) @@ -7099,7 +7348,16 @@ func Example_rebalancing() { for j := 0; j < len(testStores); j++ { ts := &testStores[j] var rangeUsageInfo RangeUsageInfo - target, _, details, ok := alloc.RebalanceVoter(ctx, emptySpanConfig(), nil, []roachpb.ReplicaDescriptor{{NodeID: ts.Node.NodeID, StoreID: ts.StoreID}}, nil, rangeUsageInfo, storeFilterThrottled) + target, _, details, ok := alloc.RebalanceVoter( + ctx, + emptySpanConfig(), + nil, + []roachpb.ReplicaDescriptor{{NodeID: ts.Node.NodeID, StoreID: ts.StoreID}}, + nil, + rangeUsageInfo, + storeFilterThrottled, + alloc.scorerOptions(), + ) if ok { log.Infof(ctx, "rebalancing to %v; details: %s", target, details) testStores[j].rebalance(&testStores[int(target.StoreID)], alloc.randGen.Int63n(1<<20)) @@ -7136,55 +7394,55 @@ func Example_rebalancing() { // | gen | store 0 | store 1 | store 2 | store 3 | store 4 | store 5 | store 6 | store 7 | store 8 | store 9 | store 10 | store 11 | store 12 | store 13 | store 14 | store 15 | store 16 | store 17 | store 18 | store 19 | // +-----+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+ // | 0 | 2 100% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | - // | 2 | 4 100% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | - // | 4 | 6 100% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | - // | 6 | 8 100% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | - // | 8 | 10 100% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | - // | 10 | 10 68% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 1 11% | 0 0% | 2 20% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | - // | 12 | 10 34% | 1 2% | 0 0% | 2 18% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 3 14% | 0 0% | 3 27% | 0 0% | 0 0% | 1 2% | 0 0% | 0 0% | 0 0% | - // | 14 | 10 22% | 3 7% | 0 0% | 4 18% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 1 3% | 0 0% | 4 11% | 2 1% | 4 22% | 0 0% | 0 0% | 3 8% | 1 0% | 1 4% | 0 0% | - // | 16 | 10 12% | 5 10% | 0 0% | 5 13% | 0 0% | 2 4% | 1 3% | 2 1% | 0 0% | 3 6% | 1 0% | 5 8% | 4 5% | 5 12% | 0 0% | 0 0% | 5 9% | 3 4% | 3 8% | 0 0% | - // | 18 | 10 5% | 6 7% | 3 1% | 6 9% | 3 4% | 4 6% | 3 5% | 4 2% | 2 1% | 4 5% | 3 2% | 6 7% | 5 3% | 6 10% | 3 2% | 4 3% | 6 6% | 4 5% | 4 6% | 2 1% | - // | 20 | 10 4% | 8 6% | 5 2% | 8 8% | 5 4% | 6 6% | 5 5% | 6 3% | 5 3% | 6 6% | 5 2% | 8 6% | 7 4% | 8 9% | 5 3% | 6 4% | 8 6% | 6 3% | 6 5% | 5 3% | - // | 22 | 11 4% | 10 6% | 7 2% | 10 7% | 7 4% | 8 6% | 7 5% | 8 4% | 7 3% | 8 5% | 8 2% | 10 6% | 9 4% | 10 8% | 7 3% | 8 3% | 10 6% | 8 3% | 8 5% | 7 4% | - // | 24 | 13 5% | 12 6% | 9 2% | 12 6% | 9 4% | 10 5% | 9 5% | 10 4% | 9 4% | 10 5% | 10 3% | 12 6% | 11 4% | 12 7% | 9 3% | 10 3% | 12 6% | 10 3% | 10 5% | 9 4% | - // | 26 | 15 5% | 14 6% | 11 3% | 14 6% | 11 4% | 12 5% | 11 4% | 12 4% | 11 4% | 12 5% | 12 3% | 14 6% | 13 4% | 14 6% | 11 4% | 12 3% | 14 6% | 12 3% | 12 5% | 11 4% | - // | 28 | 17 5% | 16 5% | 13 4% | 16 6% | 13 4% | 14 5% | 13 4% | 14 4% | 13 4% | 14 5% | 14 3% | 16 5% | 15 5% | 16 6% | 13 4% | 14 3% | 16 5% | 14 4% | 14 4% | 13 4% | - // | 30 | 19 5% | 18 6% | 15 3% | 18 6% | 15 4% | 16 5% | 15 4% | 16 4% | 15 4% | 16 5% | 16 3% | 18 5% | 17 5% | 18 5% | 15 4% | 16 3% | 18 6% | 16 4% | 16 5% | 15 4% | - // | 32 | 21 4% | 20 5% | 17 4% | 20 6% | 17 4% | 18 6% | 17 5% | 18 4% | 17 4% | 18 5% | 18 3% | 20 5% | 19 4% | 20 6% | 17 4% | 18 3% | 20 6% | 18 4% | 18 4% | 17 4% | - // | 34 | 23 4% | 22 6% | 19 4% | 22 6% | 19 4% | 20 5% | 19 5% | 20 4% | 19 4% | 20 4% | 20 4% | 22 5% | 21 4% | 22 5% | 19 4% | 20 3% | 22 6% | 20 3% | 20 4% | 19 5% | - // | 36 | 25 4% | 24 5% | 21 4% | 24 7% | 21 4% | 22 5% | 21 5% | 22 4% | 21 4% | 22 4% | 22 4% | 24 5% | 23 4% | 24 5% | 21 4% | 22 4% | 24 6% | 22 4% | 22 4% | 21 5% | - // | 38 | 27 4% | 26 5% | 23 4% | 26 6% | 23 4% | 24 5% | 23 5% | 24 4% | 23 4% | 24 4% | 24 4% | 26 5% | 25 5% | 26 5% | 23 4% | 24 4% | 26 6% | 24 4% | 24 5% | 23 5% | - // | 40 | 29 4% | 28 5% | 25 4% | 28 6% | 25 4% | 26 5% | 25 5% | 26 4% | 25 4% | 26 4% | 26 4% | 28 5% | 27 4% | 28 5% | 25 4% | 26 4% | 28 5% | 26 4% | 26 4% | 25 5% | - // | 42 | 31 4% | 30 5% | 27 4% | 30 6% | 27 4% | 28 5% | 27 5% | 28 4% | 27 4% | 28 4% | 28 4% | 30 5% | 29 4% | 30 5% | 27 4% | 28 4% | 30 5% | 28 4% | 28 4% | 27 5% | - // | 44 | 33 4% | 32 5% | 29 4% | 32 6% | 29 4% | 30 5% | 29 5% | 30 3% | 29 4% | 30 4% | 30 4% | 32 5% | 31 4% | 32 6% | 29 4% | 30 4% | 32 5% | 30 4% | 30 4% | 29 5% | - // | 46 | 35 4% | 34 5% | 31 4% | 34 6% | 31 4% | 32 5% | 31 5% | 32 3% | 31 4% | 32 5% | 32 4% | 34 5% | 33 4% | 34 6% | 31 4% | 32 4% | 34 5% | 32 4% | 32 4% | 31 5% | - // | 48 | 37 4% | 36 4% | 33 4% | 36 5% | 33 4% | 34 5% | 33 5% | 34 3% | 33 4% | 34 5% | 34 4% | 36 5% | 35 4% | 36 6% | 33 4% | 34 4% | 36 5% | 34 4% | 34 5% | 33 5% | - // | 50 | 39 4% | 38 4% | 35 4% | 38 5% | 35 4% | 36 5% | 35 5% | 36 3% | 35 4% | 36 5% | 36 4% | 38 5% | 37 4% | 38 5% | 35 4% | 36 4% | 38 5% | 36 4% | 36 5% | 35 5% | - // | 52 | 41 4% | 40 5% | 37 4% | 40 5% | 37 4% | 38 5% | 37 5% | 38 3% | 37 4% | 38 5% | 38 4% | 40 5% | 39 4% | 40 5% | 37 5% | 38 4% | 40 5% | 38 4% | 38 5% | 37 5% | - // | 54 | 43 4% | 42 5% | 39 4% | 42 5% | 39 4% | 40 5% | 39 5% | 40 3% | 39 4% | 40 5% | 40 4% | 42 5% | 41 4% | 42 5% | 39 5% | 40 4% | 42 5% | 40 4% | 40 5% | 39 5% | - // | 56 | 45 4% | 44 5% | 41 4% | 44 5% | 41 4% | 42 5% | 41 5% | 42 4% | 41 4% | 42 5% | 42 4% | 44 5% | 43 4% | 44 5% | 41 5% | 42 4% | 44 5% | 42 4% | 42 5% | 41 5% | - // | 58 | 47 4% | 46 5% | 43 4% | 46 5% | 43 4% | 44 5% | 43 5% | 44 4% | 43 4% | 44 5% | 44 4% | 46 5% | 45 5% | 46 5% | 43 5% | 44 4% | 46 5% | 44 4% | 44 5% | 43 5% | - // | 60 | 49 4% | 48 5% | 45 3% | 48 5% | 45 4% | 46 5% | 45 5% | 46 4% | 45 4% | 46 5% | 46 4% | 48 5% | 47 5% | 48 5% | 45 5% | 46 4% | 48 5% | 46 5% | 46 4% | 45 5% | - // | 62 | 51 4% | 50 5% | 47 4% | 50 5% | 47 4% | 48 5% | 47 5% | 48 4% | 47 4% | 48 5% | 48 5% | 50 5% | 49 5% | 50 5% | 47 5% | 48 4% | 50 5% | 48 5% | 48 4% | 47 5% | - // | 64 | 53 4% | 52 5% | 49 3% | 52 5% | 49 4% | 50 5% | 49 5% | 50 4% | 49 4% | 50 5% | 50 5% | 52 5% | 51 5% | 52 5% | 49 5% | 50 4% | 52 5% | 50 4% | 50 4% | 49 5% | - // | 66 | 55 4% | 54 5% | 51 4% | 54 5% | 51 4% | 52 5% | 51 5% | 52 4% | 51 4% | 52 5% | 52 5% | 54 5% | 53 5% | 54 5% | 51 5% | 52 5% | 54 5% | 52 4% | 52 4% | 51 5% | - // | 68 | 57 4% | 56 5% | 53 4% | 56 5% | 53 4% | 54 5% | 53 5% | 54 4% | 53 4% | 54 5% | 54 5% | 56 5% | 55 5% | 56 5% | 53 5% | 54 5% | 56 5% | 54 4% | 54 4% | 53 5% | - // | 70 | 59 4% | 58 5% | 55 4% | 58 5% | 55 4% | 56 5% | 55 5% | 56 4% | 55 4% | 56 5% | 56 4% | 58 5% | 57 5% | 58 5% | 55 5% | 56 5% | 58 5% | 56 4% | 56 4% | 55 5% | - // | 72 | 61 4% | 60 5% | 57 4% | 60 5% | 57 4% | 58 5% | 57 5% | 58 4% | 57 4% | 58 5% | 58 4% | 60 5% | 59 5% | 60 5% | 57 4% | 58 5% | 60 5% | 58 5% | 58 4% | 57 5% | - // | 74 | 63 4% | 62 5% | 59 4% | 62 5% | 59 4% | 60 5% | 59 5% | 60 4% | 59 4% | 60 5% | 60 4% | 62 5% | 61 5% | 62 5% | 59 4% | 60 5% | 62 5% | 60 5% | 60 4% | 59 5% | - // | 76 | 65 4% | 64 5% | 61 4% | 64 5% | 61 4% | 62 5% | 61 5% | 62 4% | 61 4% | 62 5% | 62 4% | 64 5% | 63 5% | 64 5% | 61 5% | 62 5% | 64 4% | 62 5% | 62 4% | 61 5% | - // | 78 | 67 4% | 66 5% | 63 4% | 66 5% | 63 4% | 64 5% | 63 5% | 64 4% | 63 4% | 64 5% | 64 4% | 66 5% | 65 5% | 66 5% | 63 4% | 64 5% | 66 5% | 64 5% | 64 4% | 63 5% | - // | 80 | 69 4% | 68 5% | 65 4% | 68 5% | 65 4% | 66 5% | 65 5% | 66 4% | 65 4% | 66 5% | 66 4% | 68 5% | 67 4% | 68 5% | 65 4% | 66 5% | 68 5% | 66 5% | 66 4% | 65 5% | - // | 82 | 71 4% | 70 5% | 67 4% | 70 5% | 67 4% | 68 5% | 67 5% | 68 4% | 67 4% | 68 5% | 68 4% | 70 5% | 69 5% | 70 5% | 67 4% | 68 4% | 70 5% | 68 4% | 68 4% | 67 5% | - // | 84 | 73 4% | 72 5% | 69 4% | 72 5% | 69 4% | 70 5% | 69 5% | 70 4% | 69 4% | 70 5% | 70 4% | 72 5% | 71 4% | 72 5% | 69 4% | 70 4% | 72 5% | 70 4% | 70 4% | 69 5% | - // | 86 | 75 4% | 74 5% | 71 4% | 74 5% | 71 4% | 72 5% | 71 5% | 72 4% | 71 4% | 72 5% | 72 4% | 74 5% | 73 4% | 74 5% | 71 4% | 72 4% | 74 5% | 72 4% | 72 4% | 71 5% | - // | 88 | 77 4% | 76 4% | 73 4% | 76 5% | 73 4% | 74 5% | 73 5% | 74 4% | 73 4% | 74 5% | 74 4% | 76 5% | 75 4% | 76 5% | 73 4% | 74 4% | 76 5% | 74 4% | 74 4% | 73 5% | - // | 90 | 79 4% | 78 4% | 75 4% | 78 5% | 75 4% | 76 5% | 75 5% | 76 4% | 75 4% | 76 5% | 76 4% | 78 5% | 77 5% | 78 5% | 75 5% | 76 4% | 78 5% | 76 4% | 76 4% | 75 5% | - // | 92 | 81 4% | 80 4% | 77 4% | 80 5% | 77 4% | 78 5% | 77 5% | 78 4% | 77 4% | 78 5% | 78 4% | 80 5% | 79 5% | 80 5% | 77 5% | 78 4% | 80 5% | 78 4% | 78 4% | 77 5% | - // | 94 | 83 4% | 82 4% | 79 4% | 82 5% | 79 4% | 80 5% | 79 5% | 80 4% | 79 4% | 80 5% | 80 4% | 82 5% | 81 5% | 82 5% | 79 4% | 80 4% | 82 5% | 80 4% | 80 4% | 79 5% | - // | 96 | 85 4% | 84 4% | 81 4% | 84 5% | 81 5% | 82 5% | 81 5% | 82 4% | 81 4% | 82 5% | 82 4% | 84 5% | 83 5% | 84 5% | 81 5% | 82 4% | 84 5% | 82 4% | 82 4% | 81 5% | - // | 98 | 87 4% | 86 4% | 83 4% | 86 5% | 83 5% | 84 5% | 83 5% | 84 4% | 83 4% | 84 5% | 84 4% | 86 5% | 85 5% | 86 5% | 83 5% | 84 4% | 86 5% | 84 4% | 84 4% | 83 5% | + // | 2 | 3 77% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 1 22% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | + // | 4 | 3 35% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 2 27% | 0 0% | 0 0% | 0 0% | 0 0% | 3 30% | 0 0% | 0 0% | 0 0% | 0 0% | 1 5% | 0 0% | 0 0% | + // | 6 | 3 20% | 0 0% | 0 0% | 0 0% | 1 5% | 1 3% | 0 0% | 3 15% | 0 0% | 0 0% | 1 6% | 0 0% | 3 24% | 2 5% | 0 0% | 2 10% | 0 0% | 3 9% | 0 0% | 0 0% | + // | 8 | 4 10% | 0 0% | 0 0% | 0 0% | 3 10% | 3 7% | 0 0% | 4 8% | 0 0% | 0 0% | 3 7% | 1 3% | 4 14% | 4 5% | 0 0% | 4 10% | 1 3% | 4 11% | 1 3% | 1 3% | + // | 10 | 5 6% | 5 10% | 0 0% | 1 1% | 4 6% | 4 4% | 1 0% | 5 3% | 0 0% | 2 1% | 4 5% | 3 4% | 5 13% | 5 4% | 0 0% | 5 11% | 3 4% | 5 7% | 3 7% | 3 5% | + // | 12 | 6 6% | 6 8% | 6 8% | 3 1% | 5 4% | 5 3% | 3 2% | 6 1% | 4 4% | 4 3% | 5 3% | 5 5% | 6 9% | 6 4% | 3 1% | 6 8% | 5 5% | 6 4% | 5 7% | 5 6% | + // | 14 | 8 5% | 8 7% | 8 8% | 5 2% | 7 5% | 7 4% | 5 3% | 8 1% | 6 4% | 6 3% | 7 3% | 7 4% | 8 9% | 8 4% | 5 3% | 8 7% | 7 4% | 8 4% | 7 6% | 7 4% | + // | 16 | 10 5% | 10 8% | 10 8% | 7 2% | 9 4% | 9 4% | 7 3% | 10 2% | 8 4% | 8 3% | 9 3% | 9 5% | 10 7% | 10 4% | 7 3% | 10 7% | 9 4% | 10 4% | 9 5% | 9 3% | + // | 18 | 12 5% | 12 7% | 12 7% | 9 3% | 11 5% | 11 4% | 9 4% | 12 3% | 10 5% | 10 3% | 11 3% | 11 4% | 12 6% | 12 5% | 9 3% | 12 6% | 11 5% | 12 4% | 11 5% | 11 3% | + // | 20 | 14 5% | 14 6% | 14 7% | 11 3% | 13 4% | 13 5% | 11 3% | 14 3% | 12 5% | 12 3% | 13 3% | 13 4% | 14 6% | 14 5% | 11 3% | 14 6% | 13 5% | 14 5% | 13 6% | 13 4% | + // | 22 | 16 5% | 16 6% | 16 7% | 13 3% | 15 4% | 15 4% | 13 4% | 16 3% | 14 4% | 14 4% | 15 3% | 15 5% | 16 5% | 16 5% | 13 3% | 16 6% | 15 5% | 16 5% | 15 5% | 15 4% | + // | 24 | 18 5% | 18 6% | 18 7% | 15 3% | 17 4% | 17 4% | 15 4% | 18 4% | 16 4% | 16 4% | 17 3% | 17 5% | 18 5% | 18 5% | 15 4% | 18 5% | 17 5% | 18 4% | 17 5% | 17 4% | + // | 26 | 20 5% | 20 5% | 20 6% | 17 3% | 19 4% | 19 4% | 17 3% | 20 3% | 18 4% | 18 4% | 19 3% | 19 5% | 20 5% | 20 5% | 17 4% | 20 5% | 19 5% | 20 4% | 19 5% | 19 4% | + // | 28 | 22 5% | 22 6% | 22 6% | 19 4% | 21 4% | 21 4% | 19 4% | 22 4% | 20 5% | 20 4% | 21 3% | 21 5% | 22 5% | 22 6% | 19 4% | 22 5% | 21 5% | 22 4% | 21 5% | 21 4% | + // | 30 | 24 6% | 24 5% | 24 6% | 21 4% | 23 4% | 23 5% | 21 4% | 24 4% | 22 4% | 22 4% | 23 3% | 23 5% | 24 5% | 24 5% | 21 3% | 24 5% | 23 5% | 24 4% | 23 5% | 23 4% | + // | 32 | 26 6% | 26 5% | 26 6% | 23 4% | 25 4% | 25 5% | 23 4% | 26 4% | 24 5% | 24 4% | 25 3% | 25 5% | 26 5% | 26 5% | 23 4% | 26 5% | 25 5% | 26 4% | 25 5% | 25 4% | + // | 34 | 28 5% | 28 5% | 28 6% | 25 4% | 27 4% | 27 5% | 25 4% | 28 4% | 26 5% | 26 4% | 27 3% | 27 5% | 28 5% | 28 5% | 25 4% | 28 5% | 27 5% | 28 4% | 27 5% | 27 4% | + // | 36 | 30 5% | 30 5% | 30 6% | 27 4% | 29 4% | 29 5% | 27 4% | 30 4% | 28 5% | 28 4% | 29 4% | 29 5% | 30 5% | 30 5% | 27 4% | 30 5% | 29 5% | 30 4% | 29 5% | 29 4% | + // | 38 | 32 5% | 32 5% | 32 6% | 29 5% | 31 4% | 31 5% | 29 4% | 32 4% | 30 5% | 30 4% | 31 4% | 31 5% | 32 5% | 32 5% | 29 4% | 32 5% | 31 5% | 32 4% | 31 5% | 31 4% | + // | 40 | 34 5% | 34 5% | 34 6% | 31 5% | 33 4% | 33 4% | 31 4% | 34 4% | 32 4% | 32 4% | 33 4% | 33 4% | 34 5% | 34 5% | 31 4% | 34 5% | 33 5% | 34 4% | 33 4% | 33 4% | + // | 42 | 36 5% | 36 5% | 36 6% | 33 5% | 35 3% | 35 4% | 33 4% | 36 4% | 34 5% | 34 4% | 35 4% | 35 5% | 36 5% | 36 5% | 33 4% | 36 5% | 35 5% | 36 4% | 35 4% | 35 4% | + // | 44 | 38 5% | 38 5% | 38 6% | 35 5% | 37 4% | 37 4% | 35 4% | 38 4% | 36 5% | 36 4% | 37 4% | 37 5% | 38 5% | 38 5% | 35 4% | 38 5% | 37 5% | 38 4% | 37 4% | 37 4% | + // | 46 | 40 5% | 40 5% | 40 5% | 37 4% | 39 4% | 39 4% | 37 4% | 40 4% | 38 5% | 38 4% | 39 4% | 39 5% | 40 5% | 40 4% | 37 4% | 40 5% | 39 5% | 40 4% | 39 4% | 39 4% | + // | 48 | 42 5% | 42 5% | 42 6% | 39 5% | 41 4% | 41 4% | 39 4% | 42 4% | 40 5% | 40 4% | 41 4% | 41 5% | 42 5% | 42 5% | 39 4% | 42 5% | 41 5% | 42 4% | 41 4% | 41 4% | + // | 50 | 44 5% | 44 5% | 44 5% | 41 4% | 43 4% | 43 4% | 41 4% | 44 4% | 42 5% | 42 4% | 43 4% | 43 5% | 44 5% | 44 5% | 41 4% | 44 5% | 43 5% | 44 4% | 43 4% | 43 4% | + // | 52 | 46 5% | 46 5% | 46 5% | 43 4% | 45 4% | 45 4% | 43 4% | 46 4% | 44 5% | 44 4% | 45 4% | 45 5% | 46 5% | 46 5% | 43 4% | 46 5% | 45 5% | 46 4% | 45 5% | 45 4% | + // | 54 | 48 5% | 48 5% | 48 5% | 45 4% | 47 4% | 47 4% | 45 4% | 48 4% | 46 5% | 46 4% | 47 4% | 47 5% | 48 5% | 48 5% | 45 4% | 48 5% | 47 5% | 48 4% | 47 5% | 47 4% | + // | 56 | 50 5% | 50 5% | 50 5% | 47 4% | 49 4% | 49 4% | 47 4% | 50 4% | 48 5% | 48 4% | 49 4% | 49 5% | 50 5% | 50 5% | 47 4% | 50 5% | 49 5% | 50 4% | 49 5% | 49 4% | + // | 58 | 52 5% | 52 5% | 52 5% | 49 4% | 51 4% | 51 4% | 49 4% | 52 5% | 50 4% | 50 4% | 51 4% | 51 5% | 52 5% | 52 4% | 49 4% | 52 5% | 51 5% | 52 4% | 51 5% | 51 4% | + // | 60 | 54 5% | 54 5% | 54 5% | 51 4% | 53 4% | 53 4% | 51 4% | 54 5% | 52 4% | 52 4% | 53 4% | 53 5% | 54 5% | 54 5% | 51 4% | 54 5% | 53 5% | 54 4% | 53 5% | 53 4% | + // | 62 | 56 5% | 56 5% | 56 5% | 53 4% | 55 4% | 55 4% | 53 4% | 56 5% | 54 4% | 54 4% | 55 4% | 55 5% | 56 5% | 56 5% | 53 4% | 56 5% | 55 5% | 56 4% | 55 5% | 55 4% | + // | 64 | 58 5% | 58 5% | 58 5% | 55 4% | 57 4% | 57 4% | 55 4% | 58 4% | 56 4% | 56 4% | 57 4% | 57 5% | 58 5% | 58 4% | 55 4% | 58 5% | 57 5% | 58 4% | 57 5% | 57 4% | + // | 66 | 60 5% | 60 5% | 60 5% | 57 4% | 59 4% | 59 4% | 57 4% | 60 4% | 58 4% | 58 4% | 59 4% | 59 5% | 60 5% | 60 4% | 57 5% | 60 4% | 59 5% | 60 4% | 59 5% | 59 4% | + // | 68 | 62 5% | 62 5% | 62 5% | 59 4% | 61 4% | 61 4% | 59 4% | 62 4% | 60 4% | 60 4% | 61 4% | 61 5% | 62 5% | 62 4% | 59 5% | 62 5% | 61 5% | 62 4% | 61 5% | 61 4% | + // | 70 | 64 5% | 64 5% | 64 5% | 61 4% | 63 4% | 63 5% | 61 4% | 64 4% | 62 4% | 62 4% | 63 4% | 63 5% | 64 5% | 64 4% | 61 5% | 64 5% | 63 5% | 64 4% | 63 5% | 63 4% | + // | 72 | 66 5% | 66 5% | 66 5% | 63 4% | 65 4% | 65 4% | 63 4% | 66 4% | 64 4% | 64 4% | 65 4% | 65 5% | 66 5% | 66 4% | 63 5% | 66 4% | 65 5% | 66 4% | 65 5% | 65 4% | + // | 74 | 68 5% | 68 5% | 68 5% | 65 4% | 67 4% | 67 4% | 65 4% | 68 4% | 66 5% | 66 4% | 67 4% | 67 5% | 68 5% | 68 4% | 65 5% | 68 4% | 67 5% | 68 4% | 67 5% | 67 4% | + // | 76 | 70 5% | 70 5% | 70 5% | 67 4% | 69 4% | 69 4% | 67 4% | 70 4% | 68 5% | 68 4% | 69 4% | 69 5% | 70 5% | 70 4% | 67 4% | 70 5% | 69 5% | 70 4% | 69 5% | 69 4% | + // | 78 | 72 5% | 72 5% | 72 5% | 69 4% | 71 4% | 71 4% | 69 4% | 72 4% | 70 5% | 70 4% | 71 4% | 71 5% | 72 5% | 72 4% | 69 4% | 72 5% | 71 5% | 72 4% | 71 5% | 71 4% | + // | 80 | 74 5% | 74 5% | 74 5% | 71 5% | 73 4% | 73 4% | 71 5% | 74 4% | 72 5% | 72 4% | 73 4% | 73 5% | 74 5% | 74 4% | 71 4% | 74 5% | 73 5% | 74 4% | 73 4% | 73 4% | + // | 82 | 76 5% | 76 5% | 76 5% | 73 5% | 75 4% | 75 4% | 73 5% | 76 4% | 74 5% | 74 4% | 75 4% | 75 5% | 76 5% | 76 4% | 73 4% | 76 5% | 75 5% | 76 4% | 75 4% | 75 4% | + // | 84 | 78 5% | 78 5% | 78 5% | 75 5% | 77 4% | 77 4% | 75 5% | 78 4% | 76 4% | 76 4% | 77 4% | 77 5% | 78 5% | 78 4% | 75 4% | 78 5% | 77 5% | 78 4% | 77 4% | 77 4% | + // | 86 | 80 5% | 80 5% | 80 5% | 77 4% | 79 4% | 79 4% | 77 5% | 80 4% | 78 5% | 78 4% | 79 4% | 79 5% | 80 5% | 80 4% | 77 4% | 80 5% | 79 5% | 80 4% | 79 4% | 79 4% | + // | 88 | 82 5% | 82 5% | 82 5% | 79 4% | 81 4% | 81 4% | 79 5% | 82 4% | 80 5% | 80 4% | 81 4% | 81 5% | 82 5% | 82 4% | 79 4% | 82 5% | 81 5% | 82 4% | 81 4% | 81 4% | + // | 90 | 84 5% | 84 5% | 84 5% | 81 4% | 83 4% | 83 4% | 81 5% | 84 4% | 82 5% | 82 4% | 83 4% | 83 5% | 84 5% | 84 4% | 81 4% | 84 5% | 83 5% | 84 4% | 83 4% | 83 4% | + // | 92 | 86 5% | 86 5% | 86 5% | 83 4% | 85 4% | 85 4% | 83 5% | 86 4% | 84 5% | 84 4% | 85 4% | 85 5% | 86 5% | 86 5% | 83 4% | 86 5% | 85 5% | 86 4% | 85 4% | 85 4% | + // | 94 | 88 5% | 88 5% | 88 5% | 85 4% | 87 4% | 87 4% | 85 5% | 88 4% | 86 5% | 86 4% | 87 4% | 87 5% | 88 5% | 88 4% | 85 4% | 88 5% | 87 5% | 88 4% | 87 4% | 87 4% | + // | 96 | 90 5% | 90 5% | 90 5% | 87 5% | 89 4% | 89 4% | 87 5% | 90 4% | 88 4% | 88 4% | 89 4% | 89 5% | 90 5% | 90 4% | 87 4% | 90 5% | 89 5% | 90 4% | 89 4% | 89 4% | + // | 98 | 92 5% | 92 5% | 92 5% | 89 5% | 91 4% | 91 4% | 89 5% | 92 4% | 90 4% | 90 4% | 91 4% | 91 5% | 92 5% | 92 4% | 89 4% | 92 5% | 91 5% | 92 4% | 91 4% | 91 4% | // +-----+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+ - // Total bytes=894061338, ranges=1708 + // Total bytes=957538798, ranges=1840 } diff --git a/pkg/kv/kvserver/replica_command.go b/pkg/kv/kvserver/replica_command.go index 142bde8cf25f..2b7572ff4667 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -2970,8 +2970,13 @@ func (s *Store) relocateOne( // overreplicated. If we asked it instead to remove s3 from (s1,s2,s3) it // may not want to do that due to constraints. targetStore, _, err := s.allocator.removeTarget( - ctx, conf, args.targetsToRemove(), existingVoters, - existingNonVoters, args.targetType, + ctx, + conf, + args.targetsToRemove(), + existingVoters, + existingNonVoters, + args.targetType, + s.allocator.scorerOptions(), ) if err != nil { return nil, nil, errors.Wrapf( diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index 316605241257..7d4042f2c8fe 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -252,6 +252,7 @@ func (rq *replicateQueue) shouldQueue( nonVoterReplicas, rangeUsageInfo, storeFilterThrottled, + rq.allocator.scorerOptions(), ) if ok { log.VEventf(ctx, 2, "rebalance target found for voter, enqueuing") @@ -265,6 +266,7 @@ func (rq *replicateQueue) shouldQueue( nonVoterReplicas, rangeUsageInfo, storeFilterThrottled, + rq.allocator.scorerOptions(), ) if ok { log.VEventf(ctx, 2, "rebalance target found for non-voter, enqueuing") @@ -781,7 +783,14 @@ func (rq *replicateQueue) findRemoveVoter( rangeRaftProgress(repl.RaftStatus(), existingVoters))} } - return rq.allocator.RemoveVoter(ctx, zone, candidates, existingVoters, existingNonVoters) + return rq.allocator.RemoveVoter( + ctx, + zone, + candidates, + existingVoters, + existingNonVoters, + rq.allocator.scorerOptions(), + ) } // maybeTransferLeaseAway is called whenever a replica on a given store is @@ -891,7 +900,9 @@ func (rq *replicateQueue) removeNonVoter( ctx, conf, existingNonVoters, - existingVoters, existingNonVoters, + existingVoters, + existingNonVoters, + rq.allocator.scorerOptions(), ) if err != nil { return false, err @@ -1074,6 +1085,7 @@ func (rq *replicateQueue) considerRebalance( existingNonVoters, rangeUsageInfo, storeFilterThrottled, + rq.allocator.scorerOptions(), ) if !ok { // If there was nothing to do for the set of voting replicas on this @@ -1087,6 +1099,7 @@ func (rq *replicateQueue) considerRebalance( existingNonVoters, rangeUsageInfo, storeFilterThrottled, + rq.allocator.scorerOptions(), ) rebalanceTargetType = nonVoterTarget } diff --git a/pkg/kv/kvserver/store_rebalancer.go b/pkg/kv/kvserver/store_rebalancer.go index 01282257ca08..0466796dde94 100644 --- a/pkg/kv/kvserver/store_rebalancer.go +++ b/pkg/kv/kvserver/store_rebalancer.go @@ -208,13 +208,12 @@ func (sr *StoreRebalancer) Start(ctx context.Context, stopper *stop.Stopper) { func (sr *StoreRebalancer) rebalanceStore( ctx context.Context, mode LBRebalancingMode, storeList StoreList, ) { - qpsThresholdFraction := qpsRebalanceThreshold.Get(&sr.st.SV) - // First check if we should transfer leases away to better balance QPS. - qpsMinThreshold := math.Min(storeList.candidateQueriesPerSecond.mean*(1-qpsThresholdFraction), - storeList.candidateQueriesPerSecond.mean-minQPSThresholdDifference) - qpsMaxThreshold := math.Max(storeList.candidateQueriesPerSecond.mean*(1+qpsThresholdFraction), - storeList.candidateQueriesPerSecond.mean+minQPSThresholdDifference) + options := scorerOptions{ + qpsRebalanceThreshold: qpsRebalanceThreshold.Get(&sr.st.SV), + } + qpsMinThreshold := underfullQPSThreshold(options, storeList.candidateQueriesPerSecond.mean) + qpsMaxThreshold := overfullQPSThreshold(options, storeList.candidateQueriesPerSecond.mean) var localDesc *roachpb.StoreDescriptor for i := range storeList.stores { From 3f4ed4ed1745ee9f00e8cb7c0b041c2f9f74e89c Mon Sep 17 00:00:00 2001 From: Aayush Shah Date: Sun, 16 May 2021 03:59:57 -0400 Subject: [PATCH 3/7] kvserver: sharpen computation of load based signals for replica removal This commit improves the computation of `convergesScore` and `balanceScore` during replica removal by computing these scores only in relation to the set of candidates that are the least diverse (i.e. the candidates that are actually being considered for removal). This is necessary for these load based signals to be meaningful in heterogeneously loaded localities. Release justification: Fixes high priority bug Release note: None --- pkg/kv/kvserver/allocator.go | 2 +- pkg/kv/kvserver/allocator_scorer.go | 97 +++++++++++++++++++----- pkg/kv/kvserver/allocator_test.go | 110 +++++++++++++++++++++++++++- 3 files changed, 185 insertions(+), 24 deletions(-) diff --git a/pkg/kv/kvserver/allocator.go b/pkg/kv/kvserver/allocator.go index c9e9387275b7..495058ce51c5 100644 --- a/pkg/kv/kvserver/allocator.go +++ b/pkg/kv/kvserver/allocator.go @@ -934,7 +934,7 @@ func (a Allocator) removeTarget( } replicaSetForDiversityCalc := getReplicasForDiversityCalc(targetType, existingVoters, existingReplicas) - rankedCandidates := rankedCandidateListForRemoval( + rankedCandidates := candidateListForRemoval( candidateStoreList, constraintsChecker, a.storePool.getLocalitiesByStore(replicaSetForDiversityCalc), diff --git a/pkg/kv/kvserver/allocator_scorer.go b/pkg/kv/kvserver/allocator_scorer.go index 086c6e523f6c..0cd7640d5022 100644 --- a/pkg/kv/kvserver/allocator_scorer.go +++ b/pkg/kv/kvserver/allocator_scorer.go @@ -287,8 +287,12 @@ func (cl candidateList) best() candidateList { return cl } -// worst returns all the elements in a sorted (by score reversed) candidate -// list that share the lowest constraint score. +// worst returns all the elements in a sorted (by score reversed) candidate list +// that share the lowest constraint score (for instance, the set of candidates +// that result in the lowest diversity score for the range, or the set of +// candidates that are on heavily loaded stores and thus, have the lowest +// `balanceScore`). This means that the resulting candidateList will contain all +// candidates that should be considered equally for removal. func (cl candidateList) worst() candidateList { if len(cl) <= 1 { return cl @@ -470,17 +474,25 @@ func rankedCandidateListForAllocation( return candidates } -// rankedCandidateListForRemoval creates a candidate list of all existing -// replicas' stores ordered from least qualified for removal to most qualified. +// candidateListForRemoval creates a candidate list of the existing +// replicas' stores that are most qualified for a removal. Callers trying to +// remove a replica from a range are expected to randomly pick a candidate from +// the result set of this method. +// +// This is determined based on factors like (in order of precedence) constraints +// conformance, disk fullness, the diversity score of the range without the +// given replica, as well as load-based factors like range count or QPS of the +// host store. +// // Stores that are marked as not valid, are in violation of a required criteria. -func rankedCandidateListForRemoval( - sl StoreList, +func candidateListForRemoval( + existingReplsStoreList StoreList, constraintsCheck constraintsCheckFn, existingStoreLocalities map[roachpb.StoreID]roachpb.Locality, options scorerOptions, ) candidateList { var candidates candidateList - for _, s := range sl.stores { + for _, s := range existingReplsStoreList.stores { constraintsOK, necessary := constraintsCheck(s) if !constraintsOK { candidates = append(candidates, candidate{ @@ -492,22 +504,12 @@ func rankedCandidateListForRemoval( continue } diversityScore := diversityRemovalScore(s.StoreID, existingStoreLocalities) - // If removing this candidate replica does not converge the store - // stats to their means, we make it less attractive for removal by - // adding 1 to the constraint score. Note that when selecting a - // candidate for removal the candidates with the lowest scores are - // more likely to be removed. - convergesScore := rebalanceFromConvergesScore(sl, s.Capacity, options) - balanceScore := balanceScore(sl, s.Capacity, options) candidates = append(candidates, candidate{ store: s, valid: constraintsOK, necessary: necessary, fullDisk: !maxCapacityCheck(s), diversityScore: diversityScore, - convergesScore: convergesScore, - balanceScore: balanceScore, - rangeCount: int(s.Capacity.RangeCount), }) } if options.deterministic { @@ -515,6 +517,63 @@ func rankedCandidateListForRemoval( } else { sort.Sort(sort.Reverse(byScore(candidates))) } + // We compute the converges and balance scores only in relation to stores that + // are the top candidates for removal based on diversity (i.e. only among + // candidates that are non-diverse relative to the rest of the replicas). + // + // This is because, in the face of heterogeneously loaded localities, + // load-based scores (convergesScore, balanceScore) relative to the entire set + // of stores are going to be misleading / inaccurate. To see this, consider a + // situation with 4 replicas where 2 of the replicas are in the same locality: + // + // Region A: [1, 2] + // Region B: [3] + // Region C: [4] + // + // In such an example, replicas 1 and 2 would have the worst diversity scores + // and thus, we'd pick one of them for removal. However, if we were computing + // the balanceScore and convergesScore across all replicas and region A was + // explicitly configured to have a heavier load profile than regions B and C, + // both these replicas would likely have a discrete balanceScore of + // `moreThanMean`. Effectively, this would mean that we would randomly pick + // one of these for removal. This would be unfortunate as we might have had a + // better removal candidate if we were just comparing these stats among the 2 + // replicas that are being considered for removal (for instance, replica 1 may + // actually be a better candidate for removal because it is on a store that + // has more replicas than the store of replica 2). + // + // Computing balance and convergence scores only relative to replicas that + // actually being considered for removal lets us make more accurate removal + // decisions in a cluster with heterogeneously loaded localities. In + // homogeneously loaded clusters, this ends up being roughly equivalent to + // computing convergence and balance scores across all stores. + candidates = candidates.worst() + removalCandidateStores := make([]roachpb.StoreDescriptor, 0, len(candidates)) + for _, cand := range candidates { + removalCandidateStores = append(removalCandidateStores, cand.store) + } + removalCandidateStoreList := makeStoreList(removalCandidateStores) + for i := range candidates { + // If removing this candidate replica does not converge the store + // stats to their mean, we make it less attractive for removal by + // adding 1 to the constraint score. Note that when selecting a + // candidate for removal the candidates with the lowest scores are + // more likely to be removed. + candidates[i].convergesScore = rebalanceFromConvergesScore( + removalCandidateStoreList, candidates[i].store.Capacity, options, + ) + candidates[i].balanceScore = balanceScore( + removalCandidateStoreList, candidates[i].store.Capacity, options, + ) + candidates[i].rangeCount = int(candidates[i].store.Capacity.RangeCount) + } + // Re-sort to account for the ordering changes resulting from the addition of + // convergesScore, balanceScore, etc. + if options.deterministic { + sort.Sort(sort.Reverse(byScoreAndID(candidates))) + } else { + sort.Sort(sort.Reverse(byScore(candidates))) + } return candidates } @@ -734,8 +793,8 @@ func rankedCandidateListForRebalancing( continue } balanceScore := balanceScore(comparable.sl, existing.store.Capacity, options) - // Similarly to in rankedCandidateListForRemoval, any replica whose - // removal would not converge the range stats to their means is given a + // Similarly to in candidateListForRemoval, any replica whose + // removal would not converge the range stats to their mean is given a // constraint score boost of 1 to make it less attractive for removal. convergesScore := rebalanceFromConvergesScore(comparable.sl, existing.store.Capacity, options) existing.convergesScore = convergesScore diff --git a/pkg/kv/kvserver/allocator_test.go b/pkg/kv/kvserver/allocator_test.go index 7925b998ac5f..e2d2c92acd32 100644 --- a/pkg/kv/kvserver/allocator_test.go +++ b/pkg/kv/kvserver/allocator_test.go @@ -1434,6 +1434,108 @@ func TestAllocatorRebalanceByQPS(t *testing.T) { } } +func TestAllocatorRemoveBasedOnQPS(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + region := func(regionName string) roachpb.Locality { + return roachpb.Locality{ + Tiers: []roachpb.Tier{ + {Key: "region", Value: regionName}, + }, + } + } + twoOverfullStores := []*roachpb.StoreDescriptor{ + { + StoreID: 1, + Node: roachpb.NodeDescriptor{NodeID: 1, Locality: region("a")}, + Capacity: roachpb.StoreCapacity{RangeCount: 1000, QueriesPerSecond: 1200}, + }, + { + StoreID: 2, + Node: roachpb.NodeDescriptor{NodeID: 2, Locality: region("a")}, + Capacity: roachpb.StoreCapacity{RangeCount: 1000, QueriesPerSecond: 1400}, + }, + { + StoreID: 3, + Node: roachpb.NodeDescriptor{NodeID: 3, Locality: region("b")}, + Capacity: roachpb.StoreCapacity{RangeCount: 1000, QueriesPerSecond: 1000}, + }, + { + StoreID: 4, + Node: roachpb.NodeDescriptor{NodeID: 4, Locality: region("c")}, + Capacity: roachpb.StoreCapacity{RangeCount: 1000, QueriesPerSecond: 1000}, + }, + { + StoreID: 5, + Node: roachpb.NodeDescriptor{NodeID: 5, Locality: region("c")}, + Capacity: roachpb.StoreCapacity{RangeCount: 1000, QueriesPerSecond: 800}, + }, + } + + replicas := func(ids ...roachpb.StoreID) (repls []roachpb.ReplicaDescriptor) { + for _, id := range ids { + repls = append( + repls, + roachpb.ReplicaDescriptor{ + NodeID: roachpb.NodeID(id), + StoreID: id, + ReplicaID: roachpb.ReplicaID(id), + }, + ) + } + return repls + } + type testCase struct { + testStores []*roachpb.StoreDescriptor + existingRepls []roachpb.ReplicaDescriptor + expectedRemoveTarget roachpb.StoreID + } + tests := []testCase{ + { + // Expect store 1 to be removed since it is fielding the most QPS out of + // all the existing replicas. + testStores: twoOverfullStores, + existingRepls: replicas(1, 3, 4), + expectedRemoveTarget: roachpb.StoreID(1), + }, + { + // Expect store 2 to be removed since it is serving more QPS than the only + // other store that's "comparable" to it (store 1). + testStores: twoOverfullStores, + existingRepls: replicas(1, 2, 3, 4, 5), + expectedRemoveTarget: roachpb.StoreID(2), + }, + { + // Expect store 4 to be removed because it is serving more QPS than store + // 5, which is its only comparable store. + testStores: twoOverfullStores, + existingRepls: replicas(2, 3, 4, 5), + expectedRemoveTarget: roachpb.StoreID(4), + }, + } + + for _, subtest := range tests { + stopper, g, _, a, _ := createTestAllocator(10, false /* deterministic */) + defer stopper.Stop(context.Background()) + gossiputil.NewStoreGossiper(g).GossipStores(subtest.testStores, t) + ctx := context.Background() + options := scorerOptions{ + qpsRebalanceThreshold: 0.1, + } + remove, _, err := a.RemoveVoter( + ctx, + zonepb.EmptyCompleteZoneConfig(), + subtest.existingRepls, + subtest.existingRepls, + nil, + options, + ) + require.NoError(t, err) + require.Equal(t, subtest.expectedRemoveTarget, remove.StoreID) + } +} + // TestAllocatorRebalanceByCount verifies that rebalance targets are // chosen by range counts in the event that available capacities // exceed the maxAvailCapacityThreshold. @@ -3727,12 +3829,12 @@ func TestRemoveCandidatesNumReplicasConstraints(t *testing.T) { // Check behavior in a span config where `voter_constraints` are empty. checkFn := voterConstraintsCheckerForRemoval(analyzed, constraint.EmptyAnalyzedConstraints) - candidates := rankedCandidateListForRemoval(sl, + candidates := candidateListForRemoval(sl, checkFn, a.storePool.getLocalitiesByStore(existingRepls), a.scorerOptions()) if !expectedStoreIDsMatch(tc.expected, candidates.worst()) { - t.Errorf("%d (with `constraints`): expected rankedCandidateListForRemoval(%v)"+ + t.Errorf("%d (with `constraints`): expected candidateListForRemoval(%v)"+ " = %v, but got %v\n for candidates %v", testIdx, tc.existing, tc.expected, candidates.worst(), candidates) } @@ -3740,12 +3842,12 @@ func TestRemoveCandidatesNumReplicasConstraints(t *testing.T) { // Check that we'd see the same result if the same constraints were // specified as `voter_constraints`. checkFn = voterConstraintsCheckerForRemoval(constraint.EmptyAnalyzedConstraints, analyzed) - candidates = rankedCandidateListForRemoval(sl, + candidates = candidateListForRemoval(sl, checkFn, a.storePool.getLocalitiesByStore(existingRepls), a.scorerOptions()) if !expectedStoreIDsMatch(tc.expected, candidates.worst()) { - t.Errorf("%d (with `voter_constraints`): expected rankedCandidateListForRemoval(%v)"+ + t.Errorf("%d (with `voter_constraints`): expected candidateListForRemoval(%v)"+ " = %v, but got %v\n for candidates %v", testIdx, tc.existing, tc.expected, candidates.worst(), candidates) } From d61182842d878f66160ea4dc89e59df549f3af46 Mon Sep 17 00:00:00 2001 From: Aayush Shah Date: Mon, 3 May 2021 05:21:23 -0400 Subject: [PATCH 4/7] kvserver: actuate load-based replica rebalancing under heterogeneous localities This commit teaches the `StoreRebalancer` to make load-based rebalancing decisions that are meaningful within the context of the replication constraints placed on the ranges being relocated and the set of stores that can legally receive replicas for such ranges. Previously, the `StoreRebalancer` would compute the QPS underfull and overfull thresholds based on the overall average QPS being served by all stores in the cluster. Notably, this included stores that were in replication zones that would not satisfy required constraints for the range being considered for rebalancing. This meant that the store rebalancer would effectively never be able to rebalance ranges within the stores inside heavily loaded replication zones (since all the _valid_ stores would be above the overfull thresholds). This patch is a move away from the bespoke relocation logic in the `StoreRebalancer`. Instead, we have the `StoreRebalancer` rely on the rebalancing logic used by the `replicateQueue` that already has the machinery to compute load based signals for candidates _relative to other comparable stores_. The main difference here is that the `StoreRebalancer` uses this machinery to promote convergence of QPS across stores, whereas the `replicateQueue` uses it to promote convergence of range counts. A series of preceeding commits in this patchset generalize the existing replica rebalancing logic, and this commit teaches the `StoreRebalancer` to use it. This generalization also addresses another key limitation (see #62922) of the `StoreRebalancer` regarding its inability to make partial improvements to a range. Previously, if the `StoreRebalancer` couldn't move a range _entirely_ off of overfull stores, it would give up and not even move the subset of replicas it could. This is no longer the case. Resolves #61883 Resolves #62992 Release justification: Fixes high priority bug Release note (performance improvement): QPS-based replica rebalancing is now aware of different constraints placed on different replication zones. This means that heterogeneously loaded replication zones (for instance, regions) will achieve a more even distribution of QPS within the stores inside each such zone. /cc @cockroachdb/kv --- pkg/kv/kvserver/allocator.go | 2 +- pkg/kv/kvserver/allocator_scorer.go | 43 +- pkg/kv/kvserver/allocator_test.go | 2 +- pkg/kv/kvserver/store_pool.go | 5 +- pkg/kv/kvserver/store_rebalancer.go | 520 +++++--------- pkg/kv/kvserver/store_rebalancer_test.go | 845 ++++++++++++++--------- 6 files changed, 727 insertions(+), 690 deletions(-) diff --git a/pkg/kv/kvserver/allocator.go b/pkg/kv/kvserver/allocator.go index 495058ce51c5..9d834fcaad84 100644 --- a/pkg/kv/kvserver/allocator.go +++ b/pkg/kv/kvserver/allocator.go @@ -1083,7 +1083,7 @@ func (a Allocator) rebalanceTarget( return zero, zero, "", false } - // Add a fake new replica to our copy of the range descriptor so that we can + // Add a fake new replica to our copy of the replica descriptor so that we can // simulate the removal logic. If we decide not to go with this target, note // that this needs to be removed from desc before we try any other target. newReplica := roachpb.ReplicaDescriptor{ diff --git a/pkg/kv/kvserver/allocator_scorer.go b/pkg/kv/kvserver/allocator_scorer.go index 0cd7640d5022..3956a752a169 100644 --- a/pkg/kv/kvserver/allocator_scorer.go +++ b/pkg/kv/kvserver/allocator_scorer.go @@ -86,10 +86,14 @@ var rangeRebalanceThreshold = func() *settings.FloatSetting { return s }() +// TODO(aayush): Maybe turn this into an interface with one implementation +// that cares about range count and another that cares about QPS, so its +// impossible to misuse. type scorerOptions struct { deterministic bool rangeRebalanceThreshold float64 - qpsRebalanceThreshold float64 // only considered if non-zero + // Only used if `rangeRebalanceThreshold` is not set. + qpsRebalanceThreshold float64 } // candidate store for allocation. @@ -442,26 +446,11 @@ func rankedCandidateListForAllocation( } diversityScore := diversityAllocateScore(s, existingStoreLocalities) balanceScore := balanceScore(candidateStores, s.Capacity, options) - var convergesScore int - if options.qpsRebalanceThreshold > 0 { - if s.Capacity.QueriesPerSecond < underfullQPSThreshold( - options, candidateStores.candidateQueriesPerSecond.mean) { - convergesScore = 1 - } else if s.Capacity.QueriesPerSecond < candidateStores.candidateQueriesPerSecond.mean { - convergesScore = 0 - } else if s.Capacity.QueriesPerSecond < overfullQPSThreshold( - options, candidateStores.candidateQueriesPerSecond.mean) { - convergesScore = -1 - } else { - convergesScore = -2 - } - } candidates = append(candidates, candidate{ store: s, valid: constraintsOK, necessary: necessary, diversityScore: diversityScore, - convergesScore: convergesScore, balanceScore: balanceScore, rangeCount: int(s.Capacity.RangeCount), }) @@ -792,11 +781,11 @@ func rankedCandidateListForRebalancing( existingCandidates = append(existingCandidates, existing) continue } - balanceScore := balanceScore(comparable.sl, existing.store.Capacity, options) // Similarly to in candidateListForRemoval, any replica whose // removal would not converge the range stats to their mean is given a // constraint score boost of 1 to make it less attractive for removal. convergesScore := rebalanceFromConvergesScore(comparable.sl, existing.store.Capacity, options) + balanceScore := balanceScore(comparable.sl, existing.store.Capacity, options) existing.convergesScore = convergesScore existing.balanceScore = balanceScore existing.rangeCount = int(existing.store.Capacity.RangeCount) @@ -1314,7 +1303,7 @@ func diversityAllocateScore( } // diversityRemovalScore returns a value between 0 and 1 based on how desirable -// it would be to remove a node's replica of a range. A higher score indicates +// it would be to remove a store's replica of a range. A higher score indicates // that the node is a better fit (i.e. keeping it around is good for diversity). func diversityRemovalScore( storeID roachpb.StoreID, existingStoreLocalities map[roachpb.StoreID]roachpb.Locality, @@ -1322,8 +1311,9 @@ func diversityRemovalScore( var sumScore float64 var numSamples int locality := existingStoreLocalities[storeID] - // We don't need to calculate the overall diversityScore for the range, because the original overall diversityScore - // of this range is always the same. + // We don't need to calculate the overall diversityScore for the range, + // because the original overall diversityScore of this range is always the + // same. for otherStoreID, otherLocality := range existingStoreLocalities { if otherStoreID == storeID { continue @@ -1339,11 +1329,10 @@ func diversityRemovalScore( } // diversityRebalanceScore returns a value between 0 and 1 based on how -// desirable it would be to rebalance away from an existing store to the target -// store. Because the store to be removed isn't specified, this assumes that +// desirable it would be to rebalance to `store` from any of the existing +// stores. Because the store to be removed isn't specified, this assumes that // the worst-fitting store from a diversity perspective will be removed. A -// higher score indicates that the provided store is a better fit for the -// range. +// higher score indicates that the provided store is a better fit for the range. func diversityRebalanceScore( store roachpb.StoreDescriptor, existingStoreLocalities map[roachpb.StoreID]roachpb.Locality, ) float64 { @@ -1371,20 +1360,20 @@ func diversityRebalanceScore( func diversityRebalanceFromScore( store roachpb.StoreDescriptor, fromStoreID roachpb.StoreID, - existingNodeLocalities map[roachpb.StoreID]roachpb.Locality, + existingStoreLocalities map[roachpb.StoreID]roachpb.Locality, ) float64 { // Compute the pairwise diversity score of all replicas that will exist // after adding store and removing fromNodeID. var sumScore float64 var numSamples int - for storeID, locality := range existingNodeLocalities { + for storeID, locality := range existingStoreLocalities { if storeID == fromStoreID { continue } newScore := store.Locality().DiversityScore(locality) sumScore += newScore numSamples++ - for otherStoreID, otherLocality := range existingNodeLocalities { + for otherStoreID, otherLocality := range existingStoreLocalities { // Only compare pairs of replicas where otherNodeID > nodeID to avoid // computing the diversity score between each pair of localities twice. if otherStoreID <= storeID || otherStoreID == fromStoreID { diff --git a/pkg/kv/kvserver/allocator_test.go b/pkg/kv/kvserver/allocator_test.go index e2d2c92acd32..0054b5694f88 100644 --- a/pkg/kv/kvserver/allocator_test.go +++ b/pkg/kv/kvserver/allocator_test.go @@ -1525,7 +1525,7 @@ func TestAllocatorRemoveBasedOnQPS(t *testing.T) { } remove, _, err := a.RemoveVoter( ctx, - zonepb.EmptyCompleteZoneConfig(), + emptySpanConfig(), subtest.existingRepls, subtest.existingRepls, nil, diff --git a/pkg/kv/kvserver/store_pool.go b/pkg/kv/kvserver/store_pool.go index d77eb725a890..929836db218d 100644 --- a/pkg/kv/kvserver/store_pool.go +++ b/pkg/kv/kvserver/store_pool.go @@ -793,8 +793,9 @@ func (sl StoreList) String() string { return buf.String() } -// filter takes a store list and filters it using the passed in constraints. It -// maintains the original order of the passed in store list. +// filter takes a store list and removes stores that would be explicitly invalid +// under the given set of constraints. It maintains the original order of the +// passed in store list. func (sl StoreList) filter(constraints []roachpb.ConstraintsConjunction) StoreList { if len(constraints) == 0 { return sl diff --git a/pkg/kv/kvserver/store_rebalancer.go b/pkg/kv/kvserver/store_rebalancer.go index 0466796dde94..c98251d3a9b3 100644 --- a/pkg/kv/kvserver/store_rebalancer.go +++ b/pkg/kv/kvserver/store_rebalancer.go @@ -12,7 +12,6 @@ package kvserver import ( "context" - "fmt" "math" "math/rand" "sort" @@ -27,7 +26,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/timeutil" - "github.com/cockroachdb/redact" "go.etcd.io/etcd/raft/v3" ) @@ -205,20 +203,43 @@ func (sr *StoreRebalancer) Start(ctx context.Context, stopper *stop.Stopper) { }) } +// NB: The StoreRebalancer only cares about the convergence of QPS across +// stores, not the convergence of range count. So, we don't use the allocator's +// `scorerOptions` here, which sets the range count rebalance threshold. +// Instead, we use our own implementation of `scorerOptions` that promotes QPS +// balance. +func (sr *StoreRebalancer) scorerOptions() scorerOptions { + return scorerOptions{ + deterministic: sr.rq.allocator.storePool.deterministic, + qpsRebalanceThreshold: qpsRebalanceThreshold.Get(&sr.st.SV), + } +} + +// rebalanceStore iterates through the top K hottest ranges on this store and +// for each such range, performs a lease transfer if it determines that that +// will improve QPS balance across the stores in the cluster. After it runs out +// of leases to transfer away (i.e. because it couldn't find better +// replacements), it considers these ranges for replica rebalancing. +// +// TODO(aayush): We don't try to move replicas or leases away from the local +// store unless it is fielding more than the overfull threshold of QPS based off +// of all the stores in the cluster. Is this desirable? Should we be more +// aggressive? func (sr *StoreRebalancer) rebalanceStore( - ctx context.Context, mode LBRebalancingMode, storeList StoreList, + ctx context.Context, mode LBRebalancingMode, allStoresList StoreList, ) { // First check if we should transfer leases away to better balance QPS. options := scorerOptions{ qpsRebalanceThreshold: qpsRebalanceThreshold.Get(&sr.st.SV), } - qpsMinThreshold := underfullQPSThreshold(options, storeList.candidateQueriesPerSecond.mean) - qpsMaxThreshold := overfullQPSThreshold(options, storeList.candidateQueriesPerSecond.mean) + qpsMinThreshold := underfullQPSThreshold(options, allStoresList.candidateQueriesPerSecond.mean) + qpsMaxThreshold := overfullQPSThreshold(options, allStoresList.candidateQueriesPerSecond.mean) var localDesc *roachpb.StoreDescriptor - for i := range storeList.stores { - if storeList.stores[i].StoreID == sr.rq.store.StoreID() { - localDesc = &storeList.stores[i] + for i := range allStoresList.stores { + if allStoresList.stores[i].StoreID == sr.rq.store.StoreID() { + localDesc = &allStoresList.stores[i] + break } } if localDesc == nil { @@ -228,21 +249,21 @@ func (sr *StoreRebalancer) rebalanceStore( if !(localDesc.Capacity.QueriesPerSecond > qpsMaxThreshold) { log.VEventf(ctx, 1, "local QPS %.2f is below max threshold %.2f (mean=%.2f); no rebalancing needed", - localDesc.Capacity.QueriesPerSecond, qpsMaxThreshold, storeList.candidateQueriesPerSecond.mean) + localDesc.Capacity.QueriesPerSecond, qpsMaxThreshold, allStoresList.candidateQueriesPerSecond.mean) return } var replicasToMaybeRebalance []replicaWithStats - storeMap := storeListToMap(storeList) + storeMap := storeListToMap(allStoresList) log.Infof(ctx, "considering load-based lease transfers for s%d with %.2f qps (mean=%.2f, upperThreshold=%.2f)", - localDesc.StoreID, localDesc.Capacity.QueriesPerSecond, storeList.candidateQueriesPerSecond.mean, qpsMaxThreshold) + localDesc.StoreID, localDesc.Capacity.QueriesPerSecond, allStoresList.candidateQueriesPerSecond.mean, qpsMaxThreshold) hottestRanges := sr.replRankings.topQPS() for localDesc.Capacity.QueriesPerSecond > qpsMaxThreshold { replWithStats, target, considerForRebalance := sr.chooseLeaseToTransfer( - ctx, &hottestRanges, localDesc, storeList, storeMap, qpsMinThreshold, qpsMaxThreshold) + ctx, &hottestRanges, localDesc, allStoresList, storeMap, qpsMinThreshold, qpsMaxThreshold) replicasToMaybeRebalance = append(replicasToMaybeRebalance, considerForRebalance...) if replWithStats.repl == nil { break @@ -273,7 +294,7 @@ func (sr *StoreRebalancer) rebalanceStore( if !(localDesc.Capacity.QueriesPerSecond > qpsMaxThreshold) { log.Infof(ctx, "load-based lease transfers successfully brought s%d down to %.2f qps (mean=%.2f, upperThreshold=%.2f)", - localDesc.StoreID, localDesc.Capacity.QueriesPerSecond, storeList.candidateQueriesPerSecond.mean, qpsMaxThreshold) + localDesc.StoreID, localDesc.Capacity.QueriesPerSecond, allStoresList.candidateQueriesPerSecond.mean, qpsMaxThreshold) return } @@ -296,10 +317,8 @@ func (sr *StoreRebalancer) rebalanceStore( ctx, &replicasToMaybeRebalance, localDesc, - storeList, - storeMap, - qpsMinThreshold, - qpsMaxThreshold) + allStoresList, + ) if replWithStats.repl == nil { log.Infof(ctx, "ran out of replicas worth transferring and qps (%.2f) is still above desired threshold (%.2f); will check again soon", @@ -357,7 +376,7 @@ func (sr *StoreRebalancer) rebalanceStore( log.Infof(ctx, "load-based replica transfers successfully brought s%d down to %.2f qps (mean=%.2f, upperThreshold=%.2f)", - localDesc.StoreID, localDesc.Capacity.QueriesPerSecond, storeList.candidateQueriesPerSecond.mean, qpsMaxThreshold) + localDesc.StoreID, localDesc.Capacity.QueriesPerSecond, allStoresList.candidateQueriesPerSecond.mean, qpsMaxThreshold) } // TODO(a-robinson): Should we take the number of leases on each store into @@ -385,7 +404,13 @@ func (sr *StoreRebalancer) chooseLeaseToTransfer( return replicaWithStats{}, roachpb.ReplicaDescriptor{}, considerForRebalance } - if shouldNotMoveAway(ctx, replWithStats, localDesc, now, minQPS) { + if !replWithStats.repl.OwnsValidLease(ctx, now) { + log.VEventf(ctx, 3, "store doesn't own the lease for r%d", replWithStats.repl.RangeID) + continue + } + if localDesc.Capacity.QueriesPerSecond-replWithStats.qps < minQPS { + log.VEventf(ctx, 3, "moving r%d's %.2f qps would bring s%d below the min threshold (%.2f)", + replWithStats.repl.RangeID, replWithStats.qps, localDesc.StoreID, minQPS) continue } @@ -480,35 +505,22 @@ func (sr *StoreRebalancer) chooseLeaseToTransfer( } } -// rangeRebalanceContext represents a snapshot of a range's state during the -// StoreRebalancer's attempt to rebalance it based on QPS. +// rangeRebalanceContext represents a snapshot of a replicas's state along with +// the state of the cluster during the StoreRebalancer's attempt to rebalance it +// based on QPS. type rangeRebalanceContext struct { - replWithStats replicaWithStats - rangeDesc *roachpb.RangeDescriptor - conf roachpb.SpanConfig - clusterNodes int - numDesiredVoters, numDesiredNonVoters int -} + replWithStats replicaWithStats + rangeDesc *roachpb.RangeDescriptor + conf roachpb.SpanConfig -func (rbc *rangeRebalanceContext) numDesiredReplicas(targetType targetReplicaType) int { - switch targetType { - case voterTarget: - return rbc.numDesiredVoters - case nonVoterTarget: - return rbc.numDesiredNonVoters - default: - panic(fmt.Sprintf("unknown targetReplicaType %s", targetType)) - } + qpsThresholdFraction float64 } func (sr *StoreRebalancer) chooseRangeToRebalance( ctx context.Context, hottestRanges *[]replicaWithStats, localDesc *roachpb.StoreDescriptor, - storeList StoreList, - storeMap map[roachpb.StoreID]*roachpb.StoreDescriptor, - minQPS float64, - maxQPS float64, + allStoresList StoreList, ) (replWithStats replicaWithStats, voterTargets, nonVoterTargets []roachpb.ReplicationTarget) { now := sr.rq.store.Clock().NowAsClockTimestamp() for { @@ -522,10 +534,6 @@ func (sr *StoreRebalancer) chooseRangeToRebalance( return replicaWithStats{}, nil, nil } - if shouldNotMoveAway(ctx, replWithStats, localDesc, now, minQPS) { - continue - } - // Don't bother moving ranges whose QPS is below some small fraction of the // store's QPS (unless the store has extra ranges to spare anyway). It's // just unnecessary churn with no benefit to move ranges responsible for, @@ -544,79 +552,49 @@ func (sr *StoreRebalancer) chooseRangeToRebalance( continue } - log.VEventf(ctx, 3, "considering replica rebalance for r%d with %.2f qps", - replWithStats.repl.GetRangeID(), replWithStats.qps) rangeDesc, conf := replWithStats.repl.DescAndSpanConfig() clusterNodes := sr.rq.allocator.storePool.ClusterNodeCount() numDesiredVoters := GetNeededVoters(conf.GetNumVoters(), clusterNodes) numDesiredNonVoters := GetNeededNonVoters(numDesiredVoters, int(conf.GetNumNonVoters()), clusterNodes) - if rs := rangeDesc.Replicas(); numDesiredVoters != len(rs.VoterDescriptors()) || - numDesiredNonVoters != len(rs.NonVoterDescriptors()) { - // If the StoreRebalancer is allowed past this point, it may accidentally - // downreplicate and this can cause unavailable ranges. - // - // See: https://github.com/cockroachdb/cockroach/issues/54444#issuecomment-707706553 - log.VEventf(ctx, 3, "range needs up/downreplication; not considering rebalance") + if expected, actual := numDesiredVoters, len(rangeDesc.Replicas().VoterDescriptors()); expected != actual { + log.VEventf( + ctx, + 3, + "r%d is either over or under replicated (expected %d voters, found %d); ignoring", + rangeDesc.RangeID, + expected, + actual, + ) continue } - - rebalanceCtx := rangeRebalanceContext{ - replWithStats: replWithStats, - rangeDesc: rangeDesc, - conf: conf, - clusterNodes: clusterNodes, - numDesiredVoters: numDesiredVoters, - numDesiredNonVoters: numDesiredNonVoters, - } - targetVoterRepls, targetNonVoterRepls := sr.getRebalanceCandidatesBasedOnQPS( - ctx, rebalanceCtx, localDesc, storeMap, storeList, minQPS, maxQPS, - ) - - // If we couldn't find enough valid targets, forget about this range. - // - // TODO(a-robinson): Support more incremental improvements -- move what we - // can if it makes things better even if it isn't great. For example, - // moving one of the other existing replicas that's on a store with less - // qps than the max threshold but above the mean would help in certain - // locality configurations. - if len(targetVoterRepls) < rebalanceCtx.numDesiredVoters { - log.VEventf(ctx, 3, "couldn't find enough voter rebalance targets for r%d (%d/%d)", - rangeDesc.RangeID, len(targetVoterRepls), rebalanceCtx.numDesiredVoters) + if expected, actual := numDesiredNonVoters, len(rangeDesc.Replicas().NonVoterDescriptors()); expected != actual { + log.VEventf( + ctx, + 3, + "r%d is either over or under replicated (expected %d non-voters, found %d); ignoring", + rangeDesc.RangeID, + expected, + actual, + ) continue } - if len(targetNonVoterRepls) < rebalanceCtx.numDesiredNonVoters { - log.VEventf(ctx, 3, "couldn't find enough non-voter rebalance targets for r%d (%d/%d)", - rangeDesc.RangeID, len(targetNonVoterRepls), rebalanceCtx.numDesiredNonVoters) - continue + rebalanceCtx := rangeRebalanceContext{ + replWithStats: replWithStats, + rangeDesc: rangeDesc, + conf: conf, } - // If the new set of replicas has lower diversity scores than the existing - // set, we don't continue with the rebalance. since we want to ensure we - // don't hurt locality diversity just to improve QPS. - // - // 1. Ensure that diversity among voting replicas is not hurt by this - // rebalancing decision. - if sr.worsensDiversity( - ctx, - rangeDesc.GetRangeID(), - rangeDesc.Replicas().VoterDescriptors(), - targetVoterRepls, - true, /* onlyVoters */ - ) { - continue - } - // 2. Ensure that diversity among all replicas is not hurt by this decision. - allTargetRepls := append(targetVoterRepls, targetNonVoterRepls...) - if sr.worsensDiversity( - ctx, - rangeDesc.GetRangeID(), - rangeDesc.Replicas().Descriptors(), - allTargetRepls, - false, /* onlyVoters */ - ) { + if !replWithStats.repl.OwnsValidLease(ctx, now) { + log.VEventf(ctx, 3, "store doesn't own the lease for r%d", replWithStats.repl.RangeID) continue } + log.VEventf(ctx, 3, "considering replica rebalance for r%d with %.2f qps", + replWithStats.repl.GetRangeID(), replWithStats.qps) + + targetVoterRepls, targetNonVoterRepls := sr.getRebalanceTargetsBasedOnQPS(ctx, rebalanceCtx) + storeDescMap := storeListToMap(allStoresList) + // Pick the voter with the least QPS to be leaseholder; // RelocateRange transfers the lease to the first provided target. newLeaseIdx := 0 @@ -634,7 +612,7 @@ func (sr *StoreRebalancer) chooseRangeToRebalance( } } - storeDesc, ok := storeMap[targetVoterRepls[i].StoreID] + storeDesc, ok := storeDescMap[targetVoterRepls[i].StoreID] if ok && storeDesc.Capacity.QueriesPerSecond < newLeaseQPS { newLeaseIdx = i newLeaseQPS = storeDesc.Capacity.QueriesPerSecond @@ -647,258 +625,126 @@ func (sr *StoreRebalancer) chooseRangeToRebalance( } } -// worsensDiversity returns true iff the diversity score of `currentRepls` is -// higher than `targetRepls` (either among just the set of voting replicas, or -// across all replicas in the range -- determined by `onlyVoters`). -func (sr *StoreRebalancer) worsensDiversity( - ctx context.Context, - rangeID roachpb.RangeID, - currentRepls, targetRepls []roachpb.ReplicaDescriptor, - onlyVoters bool, -) bool { - curDiversity := rangeDiversityScore( - sr.rq.allocator.storePool.getLocalitiesByStore(currentRepls), - ) - newDiversity := rangeDiversityScore( - sr.rq.allocator.storePool.getLocalitiesByStore(targetRepls), - ) - replicaStr := "replica" - if onlyVoters { - replicaStr = "voting replica" - } - if curDiversity > newDiversity { - log.VEventf( - ctx, - 3, - "new %s diversity %.2f for r%d worse than current diversity %.2f; not rebalancing", - replicaStr, - newDiversity, - rangeID, - curDiversity, - ) - return true - } - return false -} - -// getRebalanceCandidatesBasedOnQPS returns a list of rebalance targets for +// getRebalanceTargetsBasedOnQPS returns a list of rebalance targets for // voting and non-voting replicas on the range that match the relevant // constraints on the range and would further the goal of balancing the QPS on -// the stores in this cluster. In case there aren't enough stores that meet the -// constraints and are valid rebalance candidates based on QPS, the list of -// targets returned may contain fewer-than-required replicas. -// -// NB: `localStoreDesc` is expected to be the leaseholder of the range being -// operated on. -func (sr *StoreRebalancer) getRebalanceCandidatesBasedOnQPS( - ctx context.Context, - rebalanceCtx rangeRebalanceContext, - localStoreDesc *roachpb.StoreDescriptor, - storeMap map[roachpb.StoreID]*roachpb.StoreDescriptor, - storeList StoreList, - minQPS, maxQPS float64, +// the stores in this cluster. +func (sr *StoreRebalancer) getRebalanceTargetsBasedOnQPS( + ctx context.Context, rbCtx rangeRebalanceContext, ) (finalVoterTargets, finalNonVoterTargets []roachpb.ReplicaDescriptor) { - options := sr.rq.allocator.scorerOptions() - options.qpsRebalanceThreshold = qpsRebalanceThreshold.Get(&sr.st.SV) - - // Decide which voting / non-voting replicas we want to keep around and find - // rebalance targets for the rest. - partialVoterTargets := sr.pickReplsToKeep( - ctx, - rebalanceCtx, - nil, /* replsToExclude */ - localStoreDesc, - storeMap, - maxQPS, - voterTarget, - ) - finalVoterTargets = sr.pickRemainingRepls( - ctx, - rebalanceCtx, - partialVoterTargets, - nil, /* partialNonVoterTargets */ - storeMap, - storeList, - options, - minQPS, maxQPS, - voterTarget, - ) - - partialNonVoterTargets := sr.pickReplsToKeep( - ctx, - rebalanceCtx, - // NB: `finalVoterTargets` may contain replicas that are part of the - // existing set of non-voter targets, so we make sure that we don't keep - // those replicas around in `partialNonVoterTargets`. - finalVoterTargets, - localStoreDesc, - storeMap, - maxQPS, - nonVoterTarget, - ) - finalNonVoterTargets = sr.pickRemainingRepls( - ctx, - rebalanceCtx, - finalVoterTargets, - partialNonVoterTargets, - storeMap, - storeList, - options, - minQPS, - maxQPS, - nonVoterTarget, - ) - - return finalVoterTargets, finalNonVoterTargets -} - -// pickRemainingRepls determines the set of rebalance targets to fill in the -// rest of `partial{Voter,NonVoter}Targets` such that the resulting set contains -// exactly as many replicas as dictated by the zone configs. -// -// The caller is expected to synthesize the set of -// `partial{Voter,NonVoter}Targets` via `StoreRebalancer.pickReplsToKeep`. -func (sr *StoreRebalancer) pickRemainingRepls( - ctx context.Context, - rebalanceCtx rangeRebalanceContext, - partialVoterTargets, partialNonVoterTargets []roachpb.ReplicaDescriptor, - storeMap map[roachpb.StoreID]*roachpb.StoreDescriptor, - storeList StoreList, - options scorerOptions, - minQPS, maxQPS float64, - targetType targetReplicaType, -) []roachpb.ReplicaDescriptor { - // Alias the slice that corresponds to the set of replicas that is being - // appended to. This is because we want subsequent calls to - // `allocateTargetFromList` to observe the results of previous calls (note the - // append to the slice referenced by `finalTargetsForType`). - var finalTargetsForType *[]roachpb.ReplicaDescriptor - switch targetType { - case voterTarget: - finalTargetsForType = &partialVoterTargets - case nonVoterTarget: - finalTargetsForType = &partialNonVoterTargets - default: - log.Fatalf(ctx, "unknown targetReplicaType: %s", targetType) - } - for len(*finalTargetsForType) < rebalanceCtx.numDesiredReplicas(targetType) { - // Use the preexisting Allocate{Non}Voter logic to ensure that - // considerations such as zone constraints, locality diversity, and full - // disk come into play. - target, _ := sr.rq.allocator.allocateTargetFromList( + options := sr.scorerOptions() + finalVoterTargets = rbCtx.rangeDesc.Replicas().VoterDescriptors() + finalNonVoterTargets = rbCtx.rangeDesc.Replicas().NonVoterDescriptors() + + // NB: We attempt to rebalance N times for N replicas as we may want to + // replace all of them (they could all be on suboptimal stores). + for i := 0; i < len(finalVoterTargets); i++ { + // TODO(aayush): Figure out a way to plumb the `details` here into + // `AdminRelocateRange` so that these decisions show up in system.rangelog + add, remove, _, shouldRebalance := sr.rq.allocator.rebalanceTarget( ctx, - storeList, - rebalanceCtx.conf, - partialVoterTargets, - partialNonVoterTargets, + rbCtx.conf, + rbCtx.replWithStats.repl.RaftStatus(), + finalVoterTargets, finalNonVoterTargets, + rangeUsageInfoForRepl(rbCtx.replWithStats.repl), + storeFilterSuspect, + voterTarget, options, - // The store rebalancer should never need to perform lateral relocations, - // so we ask the allocator to disregard all the nodes that exist in - // `partial{Non}VoterTargets`. - false, /* allowMultipleReplsPerNode */ - targetType, ) - if target == nil { + if !shouldRebalance { log.VEventf( - ctx, 3, "no rebalance %ss found to replace the current store for r%d", - targetType, rebalanceCtx.rangeDesc.RangeID, + ctx, + 3, + "no more rebalancing opportunities for r%d voters that improve QPS balance", + rbCtx.rangeDesc.RangeID, ) break } - - meanQPS := storeList.candidateQueriesPerSecond.mean - if sr.shouldNotMoveTo( + log.VEventf( ctx, - storeMap, - rebalanceCtx.replWithStats, - target.StoreID, - meanQPS, - minQPS, - maxQPS, - ) { - // NB: If the target store returned by the allocator is not fit to - // receive a new replica due to balancing reasons, there is no point - // continuing with this loop since we'd expect future calls to - // `allocateTargetFromList` to return the same target. - break - } - - *finalTargetsForType = append(*finalTargetsForType, roachpb.ReplicaDescriptor{ - NodeID: target.Node.NodeID, - StoreID: target.StoreID, - }) - } - return *finalTargetsForType -} + 3, + "rebalancing voter (qps=%.2f) for r%d on %v to %v in order to improve QPS balance", + rbCtx.replWithStats.qps, + rbCtx.rangeDesc.RangeID, + remove, + add, + ) -// pickReplsToKeep determines the set of existing replicas for a range which -// should _not_ be rebalanced (because they belong to stores that aren't -// overloaded). -func (sr *StoreRebalancer) pickReplsToKeep( - ctx context.Context, - rebalanceCtx rangeRebalanceContext, - replsToExclude []roachpb.ReplicaDescriptor, - localStoreDesc *roachpb.StoreDescriptor, - storeMap map[roachpb.StoreID]*roachpb.StoreDescriptor, - maxQPS float64, - targetType targetReplicaType, -) (partialTargetRepls []roachpb.ReplicaDescriptor) { - shouldExclude := func(repl roachpb.ReplicaDescriptor) bool { - for _, excluded := range replsToExclude { - if repl.StoreID == excluded.StoreID { - return true + afterVoters := make([]roachpb.ReplicaDescriptor, 0, len(finalVoterTargets)) + afterNonVoters := make([]roachpb.ReplicaDescriptor, 0, len(finalNonVoterTargets)) + for _, voter := range finalVoterTargets { + if voter.StoreID == remove.StoreID { + afterVoters = append( + afterVoters, roachpb.ReplicaDescriptor{ + StoreID: add.StoreID, + NodeID: add.NodeID, + }) + } else { + afterVoters = append(afterVoters, voter) } } - return false - } - - var currentReplsForType []roachpb.ReplicaDescriptor - switch targetType { - case voterTarget: - currentReplsForType = rebalanceCtx.rangeDesc.Replicas().VoterDescriptors() - case nonVoterTarget: - currentReplsForType = rebalanceCtx.rangeDesc.Replicas().NonVoterDescriptors() - default: - log.Fatalf(ctx, "unknown targetReplicaType: %s", targetType) + // Voters are allowed to relocate to stores that have non-voters, which may + // displace them. + for _, nonVoter := range finalNonVoterTargets { + if nonVoter.StoreID == add.StoreID { + afterNonVoters = append(afterNonVoters, roachpb.ReplicaDescriptor{ + StoreID: remove.StoreID, + NodeID: remove.NodeID, + }) + } else { + afterNonVoters = append(afterNonVoters, nonVoter) + } + } + // Pretend that we've executed upon this rebalancing decision. + finalVoterTargets = afterVoters + finalNonVoterTargets = afterNonVoters } - // Check the existing replicas, keeping around those that aren't overloaded. - for i := range currentReplsForType { - if shouldExclude(currentReplsForType[i]) || - currentReplsForType[i].StoreID == localStoreDesc.StoreID { - continue + for i := 0; i < len(finalNonVoterTargets); i++ { + add, remove, _, shouldRebalance := sr.rq.allocator.rebalanceTarget( + ctx, + rbCtx.conf, + rbCtx.replWithStats.repl.RaftStatus(), + finalVoterTargets, finalNonVoterTargets, + rangeUsageInfoForRepl(rbCtx.replWithStats.repl), + storeFilterSuspect, + nonVoterTarget, + options, + ) + if !shouldRebalance { + log.VEventf( + ctx, + 3, + "no more rebalancing opportunities for r%d non-voters that improve QPS balance", + rbCtx.rangeDesc.RangeID, + ) + break } - - // Keep the replica in the range if we don't know its QPS or if its QPS is - // below the upper threshold. Punishing stores not in our store map could - // cause mass evictions if the storePool gets out of sync. - storeDesc, ok := storeMap[currentReplsForType[i].StoreID] - if !ok || storeDesc.Capacity.QueriesPerSecond < maxQPS { - if log.V(3) { - var reason redact.RedactableString - if ok { - reason = redact.Sprintf( - " (qps %.2f vs max %.2f)", - storeDesc.Capacity.QueriesPerSecond, - maxQPS, - ) - } - log.VEventf( - ctx, - 3, - "keeping %s r%d/%d on s%d%s", - targetType, - rebalanceCtx.rangeDesc.RangeID, - currentReplsForType[i].ReplicaID, - currentReplsForType[i].StoreID, - reason, - ) + log.VEventf( + ctx, + 3, + "rebalancing non-voter (qps=%.2f) for r%d on %v to %v in order to improve QPS balance", + rbCtx.replWithStats.qps, + rbCtx.rangeDesc.RangeID, + remove, + add, + ) + var newNonVoters []roachpb.ReplicaDescriptor + for _, nonVoter := range finalNonVoterTargets { + if nonVoter.StoreID == remove.StoreID { + newNonVoters = append( + newNonVoters, roachpb.ReplicaDescriptor{ + StoreID: add.StoreID, + NodeID: add.NodeID, + }) + } else { + newNonVoters = append(newNonVoters, nonVoter) } - - partialTargetRepls = append(partialTargetRepls, currentReplsForType[i]) } + // Pretend that we've executed upon this rebalancing decision. + finalNonVoterTargets = newNonVoters } - return partialTargetRepls + return finalVoterTargets, finalNonVoterTargets } func shouldNotMoveAway( diff --git a/pkg/kv/kvserver/store_rebalancer_test.go b/pkg/kv/kvserver/store_rebalancer_test.go index 98fbe8f5d08f..bf460da4e368 100644 --- a/pkg/kv/kvserver/store_rebalancer_test.go +++ b/pkg/kv/kvserver/store_rebalancer_test.go @@ -12,7 +12,10 @@ package kvserver import ( "context" + "fmt" + "math/rand" "reflect" + "strings" "testing" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -27,10 +30,173 @@ import ( "go.etcd.io/etcd/raft/v3/tracker" ) +const defaultQPSRebalanceThreshold = 0.25 + var ( - // noLocalityStores specifies a set of stores where s5 is - // under-utilized in terms of QPS, s2-s4 are in the middle, and s1 is - // over-utilized. + // multiRegionStores specifies a set of stores across 3 regions. These stores + // are arranged in descending order of the QPS they are receiving. Store 1 is + // the most heavily loaded, and store 9 is the least heavily loaded store. + // Consequently, region "a" is fielding the most QPS whereas region "c" is + // fielding the least. + multiRegionStores = []*roachpb.StoreDescriptor{ + { + StoreID: 1, + Node: roachpb.NodeDescriptor{ + NodeID: 1, + Locality: roachpb.Locality{ + []roachpb.Tier{ + { + Key: "region", + Value: "a", + }, + }, + }, + }, + Capacity: roachpb.StoreCapacity{ + QueriesPerSecond: 3000, + }, + }, + { + StoreID: 2, + Node: roachpb.NodeDescriptor{ + NodeID: 2, + Locality: roachpb.Locality{ + []roachpb.Tier{ + { + Key: "region", + Value: "a", + }, + }, + }, + }, + Capacity: roachpb.StoreCapacity{ + QueriesPerSecond: 2800, + }, + }, + { + StoreID: 3, + Node: roachpb.NodeDescriptor{ + NodeID: 3, + Locality: roachpb.Locality{ + []roachpb.Tier{ + { + Key: "region", + Value: "a", + }, + }, + }, + }, + Capacity: roachpb.StoreCapacity{ + QueriesPerSecond: 2600, + }, + }, + { + StoreID: 4, + Node: roachpb.NodeDescriptor{ + NodeID: 4, + Locality: roachpb.Locality{ + []roachpb.Tier{ + { + Key: "region", + Value: "b", + }, + }, + }, + }, + Capacity: roachpb.StoreCapacity{ + QueriesPerSecond: 2400, + }, + }, + { + StoreID: 5, + Node: roachpb.NodeDescriptor{ + NodeID: 5, + Locality: roachpb.Locality{ + []roachpb.Tier{ + { + Key: "region", + Value: "b", + }, + }, + }, + }, + Capacity: roachpb.StoreCapacity{ + QueriesPerSecond: 2200, + }, + }, + { + StoreID: 6, + Node: roachpb.NodeDescriptor{ + NodeID: 6, + Locality: roachpb.Locality{ + []roachpb.Tier{ + { + Key: "region", + Value: "b", + }, + }, + }, + }, + Capacity: roachpb.StoreCapacity{ + QueriesPerSecond: 2000, + }, + }, + { + StoreID: 7, + Node: roachpb.NodeDescriptor{ + NodeID: 7, + Locality: roachpb.Locality{ + []roachpb.Tier{ + { + Key: "region", + Value: "c", + }, + }, + }, + }, + Capacity: roachpb.StoreCapacity{ + QueriesPerSecond: 1800, + }, + }, + { + StoreID: 8, + Node: roachpb.NodeDescriptor{ + NodeID: 8, + Locality: roachpb.Locality{ + []roachpb.Tier{ + { + Key: "region", + Value: "c", + }, + }, + }, + }, + Capacity: roachpb.StoreCapacity{ + QueriesPerSecond: 1600, + }, + }, + { + StoreID: 9, + Node: roachpb.NodeDescriptor{ + NodeID: 9, + Locality: roachpb.Locality{ + []roachpb.Tier{ + { + Key: "region", + Value: "c", + }, + }, + }, + }, + Capacity: roachpb.StoreCapacity{ + QueriesPerSecond: 1400, + }, + }, + } + + // noLocalityStores specifies a set of stores that do not have any associated + // locality tags, where s5 is under-utilized in terms of QPS, s2-s4 are in the + // middle, and s1 is over-utilized. noLocalityStores = []*roachpb.StoreDescriptor{ { StoreID: 1, @@ -200,362 +366,398 @@ func TestChooseLeaseToTransfer(t *testing.T) { } } -func TestChooseRangeToRebalance(t *testing.T) { +func randomNoLocalityStores( + numNodes int, qpsMultiplier float64, +) (stores []*roachpb.StoreDescriptor, qpsMean float64) { + var totalQPS float64 + for i := 1; i <= numNodes; i++ { + qps := rand.Float64() * qpsMultiplier + stores = append( + stores, &roachpb.StoreDescriptor{ + StoreID: roachpb.StoreID(i), + Node: roachpb.NodeDescriptor{NodeID: roachpb.NodeID(i)}, + Capacity: roachpb.StoreCapacity{QueriesPerSecond: qps}, + }, + ) + totalQPS = totalQPS + qps + } + return stores, totalQPS / float64(numNodes) +} + +func logSummary( + ctx context.Context, allStores, deadStores []*roachpb.StoreDescriptor, meanQPS float64, +) { + var summary strings.Builder + for _, store := range allStores { + summary.WriteString( + fmt.Sprintf("s%d: %.2f qps", store.StoreID, store.Capacity.QueriesPerSecond), + ) + for _, dead := range deadStores { + if dead.StoreID == store.StoreID { + summary.WriteString(" (dead)") + } + } + summary.WriteString("\n") + } + summary.WriteString(fmt.Sprintf("overall-mean: %.2f", meanQPS)) + log.Infof(ctx, "generated random store list:\n%s", summary.String()) +} + +func TestChooseRangeToRebalanceRandom(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - ctx := context.Background() - stopper := stop.NewStopper() - defer stopper.Stop(ctx) - - stopper, g, _, a, _ := createTestAllocator(10, false /* deterministic */) - defer stopper.Stop(context.Background()) - gossiputil.NewStoreGossiper(g).GossipStores(noLocalityStores, t) - storeList, _, _ := a.storePool.getStoreList(storeFilterThrottled) - storeMap := storeListToMap(storeList) + const ( + numIterations = 10 + + qpsMultiplier = 2000 + numVoters = 3 + numNonVoters = 3 + numNodes = 12 + numDeadNodes = 3 + perReplicaQPS = 100 + qpsRebalanceThreshold = 0.25 + + epsilon = 1 + ) + + for i := 0; i < numIterations; i++ { + t.Run(fmt.Sprintf("%d", i+1), func(t *testing.T) { + ctx := context.Background() + stopper, g, _, a, _ := createTestAllocator(numNodes, false /* deterministic */) + defer stopper.Stop(context.Background()) + + stores, actualQPSMean := randomNoLocalityStores(numNodes, qpsMultiplier) + deadStores := stores[len(stores)-numDeadNodes:] + logSummary(ctx, stores, deadStores, actualQPSMean) + meanQPS := func(targets []roachpb.StoreID) float64 { + var totalQPS float64 + for _, store := range stores { + for _, target := range targets { + if target == store.StoreID { + totalQPS = totalQPS + store.Capacity.QueriesPerSecond + break + } + } + } + return totalQPS / float64(len(stores)) + } - const minQPS = 800 - const maxQPS = 1200 + // Test setup boilerplate. + gossiputil.NewStoreGossiper(g).GossipStores(stores, t) + storeList, _, _ := a.storePool.getStoreList(storeFilterThrottled) + localDesc := *stores[0] + cfg := TestStoreConfig(nil) + s := createTestStoreWithoutStart(t, stopper, testStoreOpts{createSystemRanges: true}, &cfg) + s.Ident = &roachpb.StoreIdent{StoreID: localDesc.StoreID} + rq := newReplicateQueue(s, a) + rr := newReplicaRankings() + sr := NewStoreRebalancer(cfg.AmbientCtx, cfg.Settings, rq, rr) + // Rather than trying to populate every Replica with a real raft group in + // order to pass replicaIsBehind checks, fake out the function for getting + // raft status with one that always returns all replicas as up to date. + sr.getRaftStatusFn = func(r *Replica) *raft.Status { + status := &raft.Status{ + Progress: make(map[uint64]tracker.Progress), + } + status.Lead = uint64(r.ReplicaID()) + status.Commit = 1 + for _, replica := range r.Desc().InternalReplicas { + status.Progress[uint64(replica.ReplicaID)] = tracker.Progress{ + Match: 1, + State: tracker.StateReplicate, + } + } + return status + } + a.storePool.isStoreReadyForRoutineReplicaTransfer = func(_ context.Context, this roachpb.StoreID) bool { + for _, deadStore := range deadStores { + // NodeID match StoreIDs here, so this comparison is valid. + if deadStore.StoreID == this { + return false + } + } + return true + } + s.cfg.DefaultSpanConfig.NumVoters = int32(numVoters) + s.cfg.DefaultSpanConfig.NumReplicas = int32(numVoters + numNonVoters) + // Place voters on the first `numVoters` stores and place non-voters on the + // next `numNonVoters` stores. + var voterStores, nonVoterStores []roachpb.StoreID + for i := 0; i < numVoters; i++ { + voterStores = append(voterStores, stores[i].StoreID) + } + for i := numVoters; i < numVoters+numNonVoters; i++ { + nonVoterStores = append(nonVoterStores, stores[i].StoreID) + } + loadRanges( + rr, s, []testRange{ + {voters: voterStores, nonVoters: nonVoterStores, qps: perReplicaQPS}, + }, + ) + hottestRanges := rr.topQPS() + _, voterTargets, nonVoterTargets := sr.chooseRangeToRebalance( + ctx, + &hottestRanges, + &localDesc, + storeList, + ) + var rebalancedVoterStores, rebalancedNonVoterStores []roachpb.StoreID + for _, target := range voterTargets { + rebalancedVoterStores = append(rebalancedVoterStores, target.StoreID) + } + for _, target := range nonVoterTargets { + rebalancedNonVoterStores = append(rebalancedNonVoterStores, target.StoreID) + } + log.Infof( + ctx, + "rebalanced voters from %v to %v: %.2f qps -> %.2f qps", + voterStores, + voterTargets, + meanQPS(voterStores), + meanQPS(rebalancedVoterStores), + ) + log.Infof( + ctx, + "rebalanced non-voters from %v to %v: %.2f qps -> %.2f qps", + nonVoterStores, + nonVoterTargets, + meanQPS(nonVoterStores), + meanQPS(rebalancedNonVoterStores), + ) + if r, o := meanQPS(rebalancedVoterStores), meanQPS(voterStores); r-o > epsilon { + t.Errorf("voters were rebalanced onto a set of stores with higher QPS (%.2f to %.2f)", o, r) + } + previousMean := meanQPS(append(voterStores, nonVoterStores...)) + newMean := meanQPS(append(rebalancedVoterStores, rebalancedNonVoterStores...)) + log.Infof( + ctx, + "rebalanced range from stores with %.2f average qps to %.2f average qps", + previousMean, + newMean, + ) + if newMean-previousMean > epsilon { + t.Errorf("replicas were rebalanced onto a set of stores with higher QPS (%.2f to %.2f)", previousMean, newMean) + } + }) + } +} - localDesc := *noLocalityStores[0] - cfg := TestStoreConfig(nil) - cfg.Gossip = g - s := createTestStoreWithoutStart(t, stopper, testStoreOpts{createSystemRanges: true}, &cfg) - s.Ident = &roachpb.StoreIdent{StoreID: localDesc.StoreID} - rq := newReplicateQueue(s, a) - rr := newReplicaRankings() +func TestChooseRangeToRebalanceAcrossHeterogeneousZones(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) - sr := NewStoreRebalancer(cfg.AmbientCtx, cfg.Settings, rq, rr) + ctx := context.Background() + stopper := stop.NewStopper() + defer stopper.Stop(ctx) - // Rather than trying to populate every Replica with a real raft group in - // order to pass replicaIsBehind checks, fake out the function for getting - // raft status with one that always returns all replicas as up to date. - sr.getRaftStatusFn = func(r *Replica) *raft.Status { - status := &raft.Status{ - Progress: make(map[uint64]tracker.Progress), - } - status.Lead = uint64(r.ReplicaID()) - status.Commit = 1 - for _, replica := range r.Desc().InternalReplicas { - status.Progress[uint64(replica.ReplicaID)] = tracker.Progress{ - Match: 1, - State: tracker.StateReplicate, - } + constraint := func(region string, numReplicas int32) roachpb.ConstraintsConjunction { + return roachpb.ConstraintsConjunction{ + NumReplicas: numReplicas, + Constraints: []roachpb.Constraint{ + { + Type: roachpb.Constraint_REQUIRED, + Key: "region", + Value: region, + }, + }, } - return status + } + + oneReplicaPerRegion := []roachpb.ConstraintsConjunction{ + constraint("a", 1), + constraint("b", 1), + constraint("c", 1), + } + twoReplicasInHotRegion := []roachpb.ConstraintsConjunction{ + constraint("a", 2), + } + allReplicasInHotRegion := []roachpb.ConstraintsConjunction{ + constraint("a", 3), + } + twoReplicasInSecondHottestRegion := []roachpb.ConstraintsConjunction{ + constraint("b", 2), } testCases := []struct { - voters, nonVoters []roachpb.StoreID - // stores that are not to be considered for rebalancing - nonLive []roachpb.StoreID - qps float64 - // the first listed voter target is expected to be the leaseholder - expectedRebalancedVoters, expectedRebalancedNonVoters []roachpb.StoreID + name string + voters, nonVoters []roachpb.StoreID + voterConstraints, constraints []roachpb.ConstraintsConjunction + + // the first listed voter target is expected to be the leaseholder. + expRebalancedVoters, expRebalancedNonVoters []roachpb.StoreID }{ - { - voters: []roachpb.StoreID{1}, - nonVoters: nil, - nonLive: nil, - qps: 100, - expectedRebalancedVoters: []roachpb.StoreID{5}, - expectedRebalancedNonVoters: nil, - }, - // If s5 is unavailable, s4 is the next best guess. - { - voters: []roachpb.StoreID{1}, - nonVoters: nil, - nonLive: []roachpb.StoreID{5}, - qps: 100, - expectedRebalancedVoters: []roachpb.StoreID{4}, - expectedRebalancedNonVoters: nil, - }, - { - voters: []roachpb.StoreID{1}, - nonVoters: nil, - nonLive: []roachpb.StoreID{4, 5}, - qps: 100, - expectedRebalancedVoters: []roachpb.StoreID{}, - expectedRebalancedNonVoters: nil, - }, - { - voters: []roachpb.StoreID{1}, - nonVoters: nil, - nonLive: nil, - qps: 500, - expectedRebalancedVoters: []roachpb.StoreID{5}, - expectedRebalancedNonVoters: nil, - }, - { - voters: []roachpb.StoreID{1}, - nonVoters: nil, - nonLive: []roachpb.StoreID{5}, - qps: 500, - expectedRebalancedVoters: []roachpb.StoreID{}, - expectedRebalancedNonVoters: nil, - }, - { - voters: []roachpb.StoreID{1}, - nonVoters: nil, - nonLive: nil, - qps: 800, - expectedRebalancedVoters: nil, - expectedRebalancedNonVoters: nil, - }, - { - voters: []roachpb.StoreID{1}, - nonVoters: nil, - nonLive: nil, - qps: 1.5, - expectedRebalancedVoters: []roachpb.StoreID{5}, - expectedRebalancedNonVoters: nil, - }, - { - voters: []roachpb.StoreID{1}, - nonVoters: nil, - nonLive: []roachpb.StoreID{5}, - qps: 1.5, - expectedRebalancedVoters: []roachpb.StoreID{4}, - expectedRebalancedNonVoters: nil, - }, - { - voters: []roachpb.StoreID{1}, - nonVoters: nil, - nonLive: nil, - qps: 1.49, - expectedRebalancedVoters: nil, - expectedRebalancedNonVoters: nil, - }, - { - voters: []roachpb.StoreID{1, 2}, - nonVoters: nil, - nonLive: nil, - qps: 100, - expectedRebalancedVoters: []roachpb.StoreID{5, 2}, - expectedRebalancedNonVoters: nil, - }, - { - voters: []roachpb.StoreID{1, 2}, - nonVoters: nil, - nonLive: []roachpb.StoreID{5}, - qps: 100, - expectedRebalancedVoters: []roachpb.StoreID{4, 2}, - expectedRebalancedNonVoters: nil, - }, - { - voters: []roachpb.StoreID{1, 3}, - nonVoters: nil, - nonLive: nil, - qps: 100, - expectedRebalancedVoters: []roachpb.StoreID{5, 3}, - expectedRebalancedNonVoters: nil, - }, - { - voters: []roachpb.StoreID{1, 4}, - nonVoters: nil, - nonLive: nil, - qps: 100, - expectedRebalancedVoters: []roachpb.StoreID{5, 4}, - expectedRebalancedNonVoters: nil, - }, - { - voters: []roachpb.StoreID{1, 2}, - nonVoters: nil, - nonLive: nil, - qps: 800, - expectedRebalancedVoters: nil, - expectedRebalancedNonVoters: nil, - }, - { - voters: []roachpb.StoreID{1, 2}, - nonVoters: nil, - nonLive: nil, - qps: 1.49, - expectedRebalancedVoters: nil, - expectedRebalancedNonVoters: nil, - }, - { - voters: []roachpb.StoreID{1, 4, 5}, - nonVoters: nil, - nonLive: nil, - qps: 500, - expectedRebalancedVoters: nil, - expectedRebalancedNonVoters: nil, - }, - { - voters: []roachpb.StoreID{1, 4, 5}, - nonVoters: nil, - nonLive: nil, - qps: 100, - expectedRebalancedVoters: nil, - expectedRebalancedNonVoters: nil, - }, - { - voters: []roachpb.StoreID{1, 3, 5}, - nonVoters: nil, - nonLive: nil, - qps: 500, - expectedRebalancedVoters: nil, - expectedRebalancedNonVoters: nil, - }, - { - voters: []roachpb.StoreID{1, 3, 4}, - nonVoters: nil, - nonLive: nil, - qps: 500, - expectedRebalancedVoters: []roachpb.StoreID{5, 4, 3}, - expectedRebalancedNonVoters: nil, - }, - { - voters: []roachpb.StoreID{1, 3, 5}, - nonVoters: nil, - nonLive: nil, - qps: 100, - expectedRebalancedVoters: []roachpb.StoreID{5, 4, 3}, - expectedRebalancedNonVoters: nil, - }, - { - voters: []roachpb.StoreID{1, 3, 5}, - nonVoters: nil, - nonLive: []roachpb.StoreID{4}, - qps: 100, - expectedRebalancedVoters: nil, - expectedRebalancedNonVoters: nil, - }, - // Rebalancing to s2 isn't chosen even though it's better than s1 because it's above the mean. - { - voters: []roachpb.StoreID{1, 3, 4, 5}, - nonVoters: nil, - nonLive: nil, - qps: 100, - expectedRebalancedVoters: nil, - expectedRebalancedNonVoters: nil, - }, - { - voters: []roachpb.StoreID{1, 2, 4, 5}, - nonVoters: nil, - nonLive: nil, - qps: 100, - expectedRebalancedVoters: nil, - expectedRebalancedNonVoters: nil, - }, - { - voters: []roachpb.StoreID{1, 2, 3, 5}, - nonVoters: nil, - nonLive: nil, - qps: 100, - expectedRebalancedVoters: []roachpb.StoreID{5, 4, 3, 2}, - expectedRebalancedNonVoters: nil, - }, - { - voters: []roachpb.StoreID{1, 2, 3, 4}, - nonVoters: nil, - nonLive: nil, - qps: 100, - expectedRebalancedVoters: []roachpb.StoreID{5, 4, 3, 2}, - expectedRebalancedNonVoters: nil, - }, - { - // Don't bother moving any replicas around since it won't make much of a - // difference. See `minQPSFraction` inside `chooseRangeToRebalance()`. - voters: []roachpb.StoreID{1}, - nonVoters: []roachpb.StoreID{2, 3, 4}, - nonLive: nil, - qps: 1, - expectedRebalancedVoters: nil, - expectedRebalancedNonVoters: nil, - }, - { - // None of the stores are worth moving to because they will be above the - // maxQPS after the move. - voters: []roachpb.StoreID{1}, - nonVoters: []roachpb.StoreID{2, 3, 4}, - nonLive: nil, - qps: 1000, - expectedRebalancedVoters: nil, - expectedRebalancedNonVoters: nil, - }, - { - voters: []roachpb.StoreID{1}, - nonVoters: []roachpb.StoreID{2, 3, 4}, - nonLive: nil, - qps: 100, - expectedRebalancedVoters: []roachpb.StoreID{5}, - expectedRebalancedNonVoters: []roachpb.StoreID{4, 3, 2}, - }, - // Voters may rebalance to stores that have a non-voter, and those - // displaced non-voters will be rebalanced to other valid stores. - { - voters: []roachpb.StoreID{1}, - nonVoters: []roachpb.StoreID{5}, - nonLive: nil, - qps: 100, - expectedRebalancedVoters: []roachpb.StoreID{5}, - expectedRebalancedNonVoters: []roachpb.StoreID{4}, - }, - { - voters: []roachpb.StoreID{1}, - nonVoters: []roachpb.StoreID{5, 2, 3}, - nonLive: nil, - qps: 100, - expectedRebalancedVoters: []roachpb.StoreID{5}, - expectedRebalancedNonVoters: []roachpb.StoreID{2, 3, 4}, - }, - { - // Voters may rebalance to stores that have a non-voter, but only if the - // displaced non-voters can be rebalanced to other underfull (based on - // QPS) stores. Note that stores 1 and 2 are above the maxQPS and the - // meanQPS, respectively, so non-voters cannot be rebalanced to them. - voters: []roachpb.StoreID{1, 2}, - nonVoters: []roachpb.StoreID{5, 4, 3}, - nonLive: nil, - qps: 100, - expectedRebalancedVoters: nil, - expectedRebalancedNonVoters: nil, + // All the replicas are already on the best possible stores. No + // rebalancing should be attempted. + { + name: "no rebalance", + voters: []roachpb.StoreID{3, 6, 9}, + constraints: oneReplicaPerRegion, + expRebalancedVoters: []roachpb.StoreID{9, 6, 3}, + }, + // A replica is in a heavily loaded region, on a relatively heavily loaded + // store. We expect it to be moved to a less busy store within the same + // region. + { + name: "rebalance one replica within heavy region", + voters: []roachpb.StoreID{1, 6, 9}, + constraints: oneReplicaPerRegion, + expRebalancedVoters: []roachpb.StoreID{9, 6, 3}, + }, + // Two replicas are in the hot region, both on relatively heavily loaded + // nodes. We expect one of those replicas to be moved to a less busy store + // within the same region. + { + name: "rebalance two replicas out of three within heavy region", + voters: []roachpb.StoreID{1, 2, 9}, + constraints: twoReplicasInHotRegion, + expRebalancedVoters: []roachpb.StoreID{9, 2, 3}, + }, + { + name: "rebalance two replicas out of five within heavy region", + voters: []roachpb.StoreID{1, 2, 6, 8, 9}, + constraints: twoReplicasInHotRegion, + // NB: Because of the diversity heuristic we won't rebalance to node 7. + expRebalancedVoters: []roachpb.StoreID{9, 3, 6, 8, 2}, + }, + // In the absence of any constraints, ensure that as long as diversity is + // maximized, replicas on hot stores are rebalanced to cooler stores within + // the same region. + { + // Within the hottest region, expect rebalance from the hottest node (n1) + // to the coolest node (n3). + name: "QPS balance without constraints", + voters: []roachpb.StoreID{1, 5, 8}, + expRebalancedVoters: []roachpb.StoreID{8, 5, 3}, + }, + { + // Within the second hottest region, expect rebalance from the hottest + // node (n4) to the coolest node (n6). + name: "QPS balance without constraints", + voters: []roachpb.StoreID{8, 4, 3}, + expRebalancedVoters: []roachpb.StoreID{8, 6, 3}, + }, + + // Multi-region database configurations. + { + name: "primary region with highest QPS, zone survival, one non-voter on hot node", + voters: []roachpb.StoreID{1, 2, 3}, + nonVoters: []roachpb.StoreID{4, 9}, + // Pin all voters to the hottest region (region A) and have overall + // constraints require at least one replica per each region. + voterConstraints: allReplicasInHotRegion, + constraints: oneReplicaPerRegion, + + expRebalancedVoters: []roachpb.StoreID{3, 2, 1}, + // NB: Expect the non-voter on node 4 (hottest node in region B) to move + // to node 6 (least hot region in region B). + expRebalancedNonVoters: []roachpb.StoreID{6, 9}, + }, + { + name: "primary region with second highest QPS, region survival, one voter on sub-optimal node", + voters: []roachpb.StoreID{3, 4, 5, 8, 9}, + // Pin two voters to the second hottest region (region B) and have overall + // constraints require at least one replica per each region. + voterConstraints: twoReplicasInSecondHottestRegion, + constraints: oneReplicaPerRegion, + // NB: Expect the voter on node 4 (hottest node in region B) to move to + // node 6 (least hot region in region B). + expRebalancedVoters: []roachpb.StoreID{9, 5, 6, 8, 3}, + }, + { + name: "primary region with highest QPS, region survival, two voters on sub-optimal nodes", + voters: []roachpb.StoreID{1, 2, 3, 4, 9}, + // Pin two voters to the hottest region (region A) and have overall + // constraints require at least one replica per each region. + voterConstraints: twoReplicasInHotRegion, + constraints: oneReplicaPerRegion, + // NB: We've got 3 voters in the hottest region, but we only need 2. We + // expect that one of the voters from the hottest region will be moved to + // the least hot region. + expRebalancedVoters: []roachpb.StoreID{9, 2, 4, 8, 3}, }, } - for _, tc := range testCases { - t.Run("", func(t *testing.T) { - a.storePool.isStoreReadyForRoutineReplicaTransfer = func(_ context.Context, storeID roachpb.StoreID) bool { - for _, s := range tc.nonLive { - if s == storeID { - return false + t.Run(tc.name, func(t *testing.T) { + // Boilerplate for test setup. + stopper, g, _, a, _ := createTestAllocator(10, false /* deterministic */) + defer stopper.Stop(context.Background()) + gossiputil.NewStoreGossiper(g).GossipStores(multiRegionStores, t) + storeList, _, _ := a.storePool.getStoreList(storeFilterThrottled) + + var localDesc roachpb.StoreDescriptor + for _, store := range multiRegionStores { + if store.StoreID == tc.voters[0] { + localDesc = *store + } + } + cfg := TestStoreConfig(nil) + s := createTestStoreWithoutStart(t, stopper, testStoreOpts{createSystemRanges: true}, &cfg) + s.Ident = &roachpb.StoreIdent{StoreID: localDesc.StoreID} + rq := newReplicateQueue(s, a) + rr := newReplicaRankings() + + sr := NewStoreRebalancer(cfg.AmbientCtx, cfg.Settings, rq, rr) + + // Rather than trying to populate every Replica with a real raft group in + // order to pass replicaIsBehind checks, fake out the function for getting + // raft status with one that always returns all replicas as up to date. + sr.getRaftStatusFn = func(r *Replica) *raft.Status { + status := &raft.Status{ + Progress: make(map[uint64]tracker.Progress), + } + status.Lead = uint64(r.ReplicaID()) + status.Commit = 1 + for _, replica := range r.Desc().InternalReplicas { + status.Progress[uint64(replica.ReplicaID)] = tracker.Progress{ + Match: 1, + State: tracker.StateReplicate, } } - return true + return status } s.cfg.DefaultSpanConfig.NumVoters = int32(len(tc.voters)) s.cfg.DefaultSpanConfig.NumReplicas = int32(len(tc.voters) + len(tc.nonVoters)) + s.cfg.DefaultSpanConfig.Constraints = tc.constraints + s.cfg.DefaultSpanConfig.VoterConstraints = tc.voterConstraints + const testingQPS = float64(50) loadRanges( rr, s, []testRange{ - {voters: tc.voters, nonVoters: tc.nonVoters, qps: tc.qps}, + {voters: tc.voters, nonVoters: tc.nonVoters, qps: testingQPS}, }, ) hottestRanges := rr.topQPS() _, voterTargets, nonVoterTargets := sr.chooseRangeToRebalance( - ctx, &hottestRanges, &localDesc, storeList, storeMap, minQPS, maxQPS, + ctx, + &hottestRanges, + &localDesc, + storeList, ) - require.Len(t, voterTargets, len(tc.expectedRebalancedVoters)) - if len(voterTargets) > 0 && voterTargets[0].StoreID != tc.expectedRebalancedVoters[0] { + require.Len(t, voterTargets, len(tc.expRebalancedVoters)) + if len(voterTargets) > 0 && voterTargets[0].StoreID != tc.expRebalancedVoters[0] { t.Errorf("chooseRangeToRebalance(existing=%v, qps=%f) chose s%d as leaseholder; want s%v", - tc.voters, tc.qps, voterTargets[0], tc.expectedRebalancedVoters[0]) + tc.voters, testingQPS, voterTargets[0], tc.expRebalancedVoters[0]) } voterStoreIDs := make([]roachpb.StoreID, len(voterTargets)) for i, target := range voterTargets { voterStoreIDs[i] = target.StoreID } - require.ElementsMatch(t, voterStoreIDs, tc.expectedRebalancedVoters) - - require.Len(t, nonVoterTargets, len(tc.expectedRebalancedNonVoters)) + require.ElementsMatch(t, voterStoreIDs, tc.expRebalancedVoters) + // Check that things "still work" when `VoterConstraints` are used + // instead. + s.cfg.DefaultSpanConfig.Constraints = []roachpb.ConstraintsConjunction{} + s.cfg.DefaultSpanConfig.VoterConstraints = tc.constraints + require.ElementsMatch(t, voterStoreIDs, tc.expRebalancedVoters) + + require.Len(t, nonVoterTargets, len(tc.expRebalancedNonVoters)) nonVoterStoreIDs := make([]roachpb.StoreID, len(nonVoterTargets)) for i, target := range nonVoterTargets { nonVoterStoreIDs[i] = target.StoreID } - require.ElementsMatch(t, nonVoterStoreIDs, tc.expectedRebalancedNonVoters) + require.ElementsMatch(t, nonVoterStoreIDs, tc.expRebalancedNonVoters) }) } } @@ -631,10 +833,9 @@ func TestNoLeaseTransferToBehindReplicas(t *testing.T) { hottestRanges = rr.topQPS() repl = hottestRanges[0].repl - _, targets, _ := sr.chooseRangeToRebalance( - ctx, &hottestRanges, &localDesc, storeList, storeMap, minQPS, maxQPS) + _, targets, _ := sr.chooseRangeToRebalance(ctx, &hottestRanges, &localDesc, storeList) expectTargets := []roachpb.ReplicationTarget{ - {NodeID: 4, StoreID: 4}, {NodeID: 5, StoreID: 5}, {NodeID: 3, StoreID: 3}, + {NodeID: 4, StoreID: 4}, {NodeID: 3, StoreID: 3}, {NodeID: 5, StoreID: 5}, } if !reflect.DeepEqual(targets, expectTargets) { t.Errorf("got targets %v for range with RaftStatus %v; want %v", From 3efcecf5f3143e686e14bc1c7ed6874b9a090cab Mon Sep 17 00:00:00 2001 From: Aayush Shah Date: Mon, 31 May 2021 19:19:03 -0400 Subject: [PATCH 5/7] kvserver: refactor allocator's scorer options This commit turns the allocator's `scorerOptions` into an interface that has two implementations: one that promotes the balancing of range count across comparable stores, and another that promotes the balancing of QPS across comparable stores. The replicateQueue uses the former, whereas the `StoreRebalancer` uses the latter. Release justification: Fixes high priority bug Release note: None --- pkg/kv/kvserver/allocator.go | 6 +- pkg/kv/kvserver/allocator_scorer.go | 428 +++++++++++++---------- pkg/kv/kvserver/allocator_scorer_test.go | 39 ++- pkg/kv/kvserver/allocator_test.go | 18 +- pkg/kv/kvserver/store_rebalancer.go | 18 +- pkg/kv/kvserver/store_rebalancer_test.go | 13 +- 6 files changed, 301 insertions(+), 221 deletions(-) diff --git a/pkg/kv/kvserver/allocator.go b/pkg/kv/kvserver/allocator.go index 9d834fcaad84..574462c09eb3 100644 --- a/pkg/kv/kvserver/allocator.go +++ b/pkg/kv/kvserver/allocator.go @@ -794,7 +794,7 @@ func (a *Allocator) allocateTargetFromList( candidateStores StoreList, conf roachpb.SpanConfig, existingVoters, existingNonVoters []roachpb.ReplicaDescriptor, - options scorerOptions, + options rangeCountScorerOptions, allowMultipleReplsPerNode bool, targetType targetReplicaType, ) (*roachpb.StoreDescriptor, string) { @@ -1240,8 +1240,8 @@ func (a Allocator) RebalanceNonVoter( ) } -func (a *Allocator) scorerOptions() scorerOptions { - return scorerOptions{ +func (a *Allocator) scorerOptions() rangeCountScorerOptions { + return rangeCountScorerOptions{ deterministic: a.storePool.deterministic, rangeRebalanceThreshold: rangeRebalanceThreshold.Get(&a.storePool.st.SV), } diff --git a/pkg/kv/kvserver/allocator_scorer.go b/pkg/kv/kvserver/allocator_scorer.go index 3956a752a169..260f924263b4 100644 --- a/pkg/kv/kvserver/allocator_scorer.go +++ b/pkg/kv/kvserver/allocator_scorer.go @@ -86,16 +86,218 @@ var rangeRebalanceThreshold = func() *settings.FloatSetting { return s }() -// TODO(aayush): Maybe turn this into an interface with one implementation -// that cares about range count and another that cares about QPS, so its -// impossible to misuse. -type scorerOptions struct { +type scorerOptions interface { + // deterministic is set by tests to have the allocator methods sort their + // results by constraints score as well as by store IDs, as opposed to just + // the score. + deterministicForTesting() bool + // shouldRebalanceBasedOnThresholds returns whether the specified store is a + // candidate for having a replica removed from it given the candidate store + // list based on either range count or QPS. This method returns true if any of + // the following is true: + // 1. `store` is overfull for the given signal. + // 2. `store` is above the mean for the given signal, and at least one other + // store is underfull for the given signal. + shouldRebalanceBasedOnThresholds( + ctx context.Context, store roachpb.StoreDescriptor, sl StoreList, + ) bool + // balanceScore returns a discrete score (`balanceStatus`) based on whether + // the store represented by `sc` classifies as underfull, lessThanEqualToMean, + // moreThanMean or overfull relative to all the stores in `sl`. + balanceScore(sl StoreList, sc roachpb.StoreCapacity) balanceStatus + // rebalanceFromConvergenceScore assigns a convergence score to the store + // referred to by `sc` based on whether moving a replica away from this store + // would converge its stats towards the mean. If moving the replica away from + // `sc` would not converge its stats towards the mean, a high convergence + // score is assigned, which would make it less likely for us to pick this + // store's replica to move away. + rebalanceFromConvergesScore(sl StoreList, sc roachpb.StoreCapacity) int + // rebalanceToConvergesScore is similar to `rebalanceFromConvergesScore` but + // it assigns a high convergence score iff moving a replica to the store + // referred to by `sc` will converge its stats towards the mean. This makes it + // more likely for us to pick this store as the rebalance target. + rebalanceToConvergesScore(sl StoreList, sc roachpb.StoreCapacity) int +} + +// rangeCountScorerOptions is used by the replicateQueue to tell the Allocator's +// rebalancing machinery to base its balance/convergence scores on range counts. +// This means that the resulting rebalancing decisions will further the goal of +// converging range counts across stores in the cluster. +type rangeCountScorerOptions struct { deterministic bool rangeRebalanceThreshold float64 - // Only used if `rangeRebalanceThreshold` is not set. +} + +func (o rangeCountScorerOptions) deterministicForTesting() bool { + return o.deterministic +} + +func (o rangeCountScorerOptions) shouldRebalanceBasedOnThresholds( + ctx context.Context, store roachpb.StoreDescriptor, sl StoreList, +) bool { + overfullThreshold := int32(math.Ceil(overfullRangeThreshold(o, sl.candidateRanges.mean))) + // 1. We rebalance if `store` is too far above the mean (i.e. stores + // that are overfull). + if store.Capacity.RangeCount > overfullThreshold { + log.VEventf(ctx, 2, + "s%d: should-rebalance(ranges-overfull): rangeCount=%d, mean=%.2f, overfull-threshold=%d", + store.StoreID, store.Capacity.RangeCount, sl.candidateRanges.mean, overfullThreshold) + return true + } + // 2. We rebalance if `store` isn't overfull, but it is above the mean and + // there is at least one other store that is "underfull" (i.e. too far below + // the mean). + if float64(store.Capacity.RangeCount) > sl.candidateRanges.mean { + underfullThreshold := int32(math.Floor(underfullRangeThreshold(o, sl.candidateRanges.mean))) + for _, desc := range sl.stores { + if desc.Capacity.RangeCount < underfullThreshold { + log.VEventf(ctx, 2, + "s%d: should-rebalance(better-fit-ranges=s%d): rangeCount=%d, otherRangeCount=%d, "+ + "mean=%.2f, underfull-threshold=%d", + store.StoreID, desc.StoreID, store.Capacity.RangeCount, desc.Capacity.RangeCount, + sl.candidateRanges.mean, underfullThreshold) + return true + } + } + } + // If we reached this point, we're happy with the range where it is. + return false +} + +func (o rangeCountScorerOptions) balanceScore( + sl StoreList, sc roachpb.StoreCapacity, +) balanceStatus { + maxRangeCount := overfullRangeThreshold(o, sl.candidateRanges.mean) + minRangeCount := underfullRangeThreshold(o, sl.candidateRanges.mean) + curRangeCount := float64(sc.RangeCount) + if curRangeCount < minRangeCount { + return underfull + } else if curRangeCount <= sl.candidateRanges.mean { + return lessThanEqualToMean + } else if curRangeCount < maxRangeCount { + return moreThanMean + } else { + return overfull + } +} + +// rebalanceFromConvergesScore returns 1 iff rebalancing a replica away from +// `sc` will _not_ converge its range count towards the mean of stores in `sl`. +// When we're considering whether to rebalance a replica away from a store or +// not, we want to give it a "boost" (i.e. make it a less likely candidate for +// removal) if it doesn't further our goal to converge range count towards the +// mean. +func (o rangeCountScorerOptions) rebalanceFromConvergesScore( + sl StoreList, sc roachpb.StoreCapacity, +) int { + if !rebalanceConvergesRangeCountOnMean(sl, sc, sc.RangeCount-1) { + return 1 + } + return 0 +} + +// rebalanceToConvergesScore returns 1 if rebalancing a replica to `sc` will +// converge its range count towards the mean of all the stores inside +// `sl`. +func (o rangeCountScorerOptions) rebalanceToConvergesScore( + sl StoreList, sc roachpb.StoreCapacity, +) int { + if rebalanceConvergesRangeCountOnMean(sl, sc, sc.RangeCount+1) { + return 1 + } + return 0 +} + +// qpsScorerOptions is used by the StoreRebalancer to tell the Allocator's +// rebalancing machinery to base its balance/convergence scores on +// queries-per-second. This means that the resulting rebalancing decisions will +// further the goal of converging QPS across stores in the cluster. +type qpsScorerOptions struct { + deterministic bool qpsRebalanceThreshold float64 } +func (o qpsScorerOptions) deterministicForTesting() bool { + return o.deterministic +} + +func (o qpsScorerOptions) shouldRebalanceBasedOnThresholds( + ctx context.Context, store roachpb.StoreDescriptor, sl StoreList, +) bool { + // 1. We rebalance if `store` is too far above the mean (i.e. stores + // that are overfull). + overfullThreshold := overfullQPSThreshold(o, sl.candidateQueriesPerSecond.mean) + if store.Capacity.QueriesPerSecond > overfullThreshold { + log.VEventf( + ctx, + 2, + "s%d: should-rebalance(QPS-overfull): QPS=%.2f, mean=%.2f, overfull-threshold=%.2f", + store.StoreID, + store.Capacity.QueriesPerSecond, + sl.candidateQueriesPerSecond.mean, + overfullThreshold, + ) + return true + } + // 2. We rebalance if `store` isn't overfull, but it is above the mean and + // there is at least one other store that is "underfull" (i.e. too far below + // the mean). + if store.Capacity.QueriesPerSecond > sl.candidateQueriesPerSecond.mean { + underfullThreshold := underfullQPSThreshold(o, sl.candidateQueriesPerSecond.mean) + for _, desc := range sl.stores { + if desc.Capacity.QueriesPerSecond < underfullThreshold { + log.VEventf( + ctx, + 2, + "s%d: should-rebalance(better-fit-QPS=s%d): QPS=%.2f, otherQPS=%.2f, mean=%.2f, underfull-threshold=%.2f", + store.StoreID, + desc.StoreID, + store.Capacity.QueriesPerSecond, + desc.Capacity.QueriesPerSecond, + sl.candidateQueriesPerSecond.mean, + underfullThreshold, + ) + return true + } + } + } + // If we reached this point, we're happy with the range where it is. + return false +} + +func (o qpsScorerOptions) balanceScore(sl StoreList, sc roachpb.StoreCapacity) balanceStatus { + maxQPS := overfullQPSThreshold(o, sl.candidateQueriesPerSecond.mean) + minQPS := underfullQPSThreshold(o, sl.candidateQueriesPerSecond.mean) + curQPS := sc.QueriesPerSecond + if curQPS < minQPS { + return underfull + } else if curQPS <= sl.candidateQueriesPerSecond.mean { + return lessThanEqualToMean + } else if curQPS < maxQPS { + return moreThanMean + } else { + return overfull + } +} + +func (o qpsScorerOptions) rebalanceFromConvergesScore(_ StoreList, _ roachpb.StoreCapacity) int { + // When using `qpsScorerOptions` (i.e. when rebalancing with the goal of + // promoting QPS convergence), we disable the `convergesScore` because we only + // have access to the QPS of the current (i.e. usually the leaseholder) + // replica. We do not have access to the QPS of other replicas, and when this + // method is called, the caller doesn't yet know which replica is going to be + // rebalanced. Thus, we cannot actually compute the `convergesScore` when + // rebalancing off of QPS. + return 0 +} + +func (o qpsScorerOptions) rebalanceToConvergesScore(_ StoreList, _ roachpb.StoreCapacity) int { + // When using `qpsScorerOptions` (i.e. when rebalancing with the goal of + // promoting QPS convergence), we disable the `convergesScore`. See comment + // inside `rebalanceFromConvergesScore()` for details. + return 0 +} + // candidate store for allocation. type candidate struct { store roachpb.StoreDescriptor @@ -412,7 +614,7 @@ func rankedCandidateListForAllocation( existingStoreLocalities map[roachpb.StoreID]roachpb.Locality, isStoreValidForRoutineReplicaTransfer func(context.Context, roachpb.StoreID) bool, allowMultipleReplsPerNode bool, - options scorerOptions, + options rangeCountScorerOptions, ) candidateList { var candidates candidateList existingReplTargets := roachpb.MakeReplicaSet(existingReplicas).ReplicationTargets() @@ -445,7 +647,7 @@ func rankedCandidateListForAllocation( continue } diversityScore := diversityAllocateScore(s, existingStoreLocalities) - balanceScore := balanceScore(candidateStores, s.Capacity, options) + balanceScore := options.balanceScore(candidateStores, s.Capacity) candidates = append(candidates, candidate{ store: s, valid: constraintsOK, @@ -455,7 +657,7 @@ func rankedCandidateListForAllocation( rangeCount: int(s.Capacity.RangeCount), }) } - if options.deterministic { + if options.deterministicForTesting() { sort.Sort(sort.Reverse(byScoreAndID(candidates))) } else { sort.Sort(sort.Reverse(byScore(candidates))) @@ -501,7 +703,7 @@ func candidateListForRemoval( diversityScore: diversityScore, }) } - if options.deterministic { + if options.deterministicForTesting() { sort.Sort(sort.Reverse(byScoreAndID(candidates))) } else { sort.Sort(sort.Reverse(byScore(candidates))) @@ -524,12 +726,12 @@ func candidateListForRemoval( // the balanceScore and convergesScore across all replicas and region A was // explicitly configured to have a heavier load profile than regions B and C, // both these replicas would likely have a discrete balanceScore of - // `moreThanMean`. Effectively, this would mean that we would randomly pick - // one of these for removal. This would be unfortunate as we might have had a - // better removal candidate if we were just comparing these stats among the 2 - // replicas that are being considered for removal (for instance, replica 1 may - // actually be a better candidate for removal because it is on a store that - // has more replicas than the store of replica 2). + // `moreThanMean`. Effectively, this would mean that we would + // randomly pick one of these for removal. This would be unfortunate as we + // might have had a better removal candidate if we were just comparing these + // stats among the 2 replicas that are being considered for removal (for + // instance, replica 1 may actually be a better candidate for removal because + // it is on a store that has more replicas than the store of replica 2). // // Computing balance and convergence scores only relative to replicas that // actually being considered for removal lets us make more accurate removal @@ -548,17 +750,17 @@ func candidateListForRemoval( // adding 1 to the constraint score. Note that when selecting a // candidate for removal the candidates with the lowest scores are // more likely to be removed. - candidates[i].convergesScore = rebalanceFromConvergesScore( - removalCandidateStoreList, candidates[i].store.Capacity, options, + candidates[i].convergesScore = options.rebalanceFromConvergesScore( + removalCandidateStoreList, candidates[i].store.Capacity, ) - candidates[i].balanceScore = balanceScore( - removalCandidateStoreList, candidates[i].store.Capacity, options, + candidates[i].balanceScore = options.balanceScore( + removalCandidateStoreList, candidates[i].store.Capacity, ) candidates[i].rangeCount = int(candidates[i].store.Capacity.RangeCount) } // Re-sort to account for the ordering changes resulting from the addition of // convergesScore, balanceScore, etc. - if options.deterministic { + if options.deterministicForTesting() { sort.Sort(sort.Reverse(byScoreAndID(candidates))) } else { sort.Sort(sort.Reverse(byScore(candidates))) @@ -712,7 +914,7 @@ func rankedCandidateListForRebalancing( } } } - if options.deterministic { + if options.deterministicForTesting() { sort.Sort(sort.Reverse(byScoreAndID(comparableCands))) } else { sort.Sort(sort.Reverse(byScore(comparableCands))) @@ -752,7 +954,7 @@ func rankedCandidateListForRebalancing( // // TODO(a-robinson): Some moderate refactoring could extract this logic // out into the loop below, avoiding duplicate balanceScore calculations. - if shouldRebalanceBasedOnThresholds(ctx, existing.store, sl, options) { + if options.shouldRebalanceBasedOnThresholds(ctx, existing.store, sl) { shouldRebalanceCheck = true break } @@ -784,8 +986,8 @@ func rankedCandidateListForRebalancing( // Similarly to in candidateListForRemoval, any replica whose // removal would not converge the range stats to their mean is given a // constraint score boost of 1 to make it less attractive for removal. - convergesScore := rebalanceFromConvergesScore(comparable.sl, existing.store.Capacity, options) - balanceScore := balanceScore(comparable.sl, existing.store.Capacity, options) + convergesScore := options.rebalanceFromConvergesScore(comparable.sl, existing.store.Capacity) + balanceScore := options.balanceScore(comparable.sl, existing.store.Capacity) existing.convergesScore = convergesScore existing.balanceScore = balanceScore existing.rangeCount = int(existing.store.Capacity.RangeCount) @@ -803,8 +1005,8 @@ func rankedCandidateListForRebalancing( // rebalance candidates. s := cand.store cand.fullDisk = !rebalanceToMaxCapacityCheck(s) - cand.balanceScore = balanceScore(comparable.sl, s.Capacity, options) - cand.convergesScore = rebalanceToConvergesScore(comparable.sl, s.Capacity, options) + cand.balanceScore = options.balanceScore(comparable.sl, s.Capacity) + cand.convergesScore = options.rebalanceToConvergesScore(comparable.sl, s.Capacity) cand.rangeCount = int(s.Capacity.RangeCount) candidates = append(candidates, cand) } @@ -813,7 +1015,7 @@ func rankedCandidateListForRebalancing( continue } - if options.deterministic { + if options.deterministicForTesting() { sort.Sort(sort.Reverse(byScoreAndID(existingCandidates))) sort.Sort(sort.Reverse(byScoreAndID(candidates))) } else { @@ -900,78 +1102,6 @@ func betterRebalanceTarget(target1, existing1, target2, existing2 *candidate) *c return target1 } -// shouldRebalanceBasedOnThresholds returns whether the specified store is a -// candidate for having a replica removed from it given the candidate store -// list. This method returns true if there are any ranges that are on stores -// that lie _outside of the [underfullThreshold, overfullThreshold] window_ for -// the given signal (QPS or range count). -// -// NB: If the given `options` have `qpsRebalanceThreshold` set, this method -// makes its determination based on QPS. Otherwise, we fall back on using range -// count as a signal. -func shouldRebalanceBasedOnThresholds( - ctx context.Context, store roachpb.StoreDescriptor, sl StoreList, options scorerOptions, -) bool { - if options.qpsRebalanceThreshold > 0 { - overfullThreshold := overfullQPSThreshold(options, sl.candidateQueriesPerSecond.mean) - if store.Capacity.QueriesPerSecond > overfullThreshold { - log.VEventf( - ctx, - 2, - "s%d: should-rebalance(QPS-overfull): QPS=%.2f, mean=%.2f, overfull-threshold=%.2f", - store.StoreID, - store.Capacity.QueriesPerSecond, - sl.candidateQueriesPerSecond.mean, - overfullThreshold, - ) - return true - } - if store.Capacity.QueriesPerSecond > sl.candidateQueriesPerSecond.mean { - underfullThreshold := underfullQPSThreshold(options, sl.candidateQueriesPerSecond.mean) - for _, desc := range sl.stores { - if desc.Capacity.QueriesPerSecond < underfullThreshold { - log.VEventf( - ctx, - 2, - "s%d: should-rebalance(better-fit-QPS=s%d): QPS=%.2f, otherQPS=%.2f, mean=%.2f, underfull-threshold=%.2f", - store.StoreID, - desc.StoreID, - store.Capacity.QueriesPerSecond, - desc.Capacity.QueriesPerSecond, - sl.candidateQueriesPerSecond.mean, - underfullThreshold, - ) - return true - } - } - } - return false - } - - overfullThreshold := int32(math.Ceil(overfullRangeThreshold(options, sl.candidateRanges.mean))) - if store.Capacity.RangeCount > overfullThreshold { - log.VEventf(ctx, 2, - "s%d: should-rebalance(ranges-overfull): rangeCount=%d, mean=%.2f, overfull-threshold=%d", - store.StoreID, store.Capacity.RangeCount, sl.candidateRanges.mean, overfullThreshold) - return true - } - if float64(store.Capacity.RangeCount) > sl.candidateRanges.mean { - underfullThreshold := int32(math.Floor(underfullRangeThreshold(options, sl.candidateRanges.mean))) - for _, desc := range sl.stores { - if desc.Capacity.RangeCount < underfullThreshold { - log.VEventf(ctx, 2, - "s%d: should-rebalance(better-fit-ranges=s%d): rangeCount=%d, otherRangeCount=%d, "+ - "mean=%.2f, underfull-threshold=%d", - store.StoreID, desc.StoreID, store.Capacity.RangeCount, desc.Capacity.RangeCount, - sl.candidateRanges.mean, underfullThreshold) - return true - } - } - } - // If we reached this point, we're happy with the range where it is. - return false -} - // nodeHasReplica returns true if the provided NodeID contains an entry in // the provided list of existing replicas. func nodeHasReplica(nodeID roachpb.NodeID, existing []roachpb.ReplicationTarget) bool { @@ -1393,106 +1523,26 @@ func diversityRebalanceFromScore( type balanceStatus int const ( - overfull balanceStatus = -2 - moreThanMean balanceStatus = -1 - lessThanMean balanceStatus = 0 - underfull balanceStatus = 1 + overfull balanceStatus = -2 + moreThanMean balanceStatus = -1 + lessThanEqualToMean balanceStatus = 0 + underfull balanceStatus = 1 ) -// balanceScore returns an arbitrarily scaled score where higher scores are for -// stores where the range is a better fit based on balance factors like range -// count or QPS. -// -// If the given options have qpsRebalanceThreshold set, we use that for -// computing the balanceScore, otherwise we use the `RangeCount`. -// -// TODO(aayush): It would be nice to be able to compose the two dimensions of -// balanceScore (QPS or RangeCount) and allow the `options` to simply specify an -// order of precedence. Callers who care about the balancing of QPS across -// stores (like the StoreRebalancer) could ask for QPS to take precedence over -// RangeCount, and vice-versa. -func balanceScore(sl StoreList, sc roachpb.StoreCapacity, options scorerOptions) balanceStatus { - if options.qpsRebalanceThreshold > 0 { - maxQPS := overfullQPSThreshold(options, sl.candidateQueriesPerSecond.mean) - minQPS := underfullQPSThreshold(options, sl.candidateQueriesPerSecond.mean) - curQPS := sc.QueriesPerSecond - if curQPS < minQPS { - return underfull - } else if curQPS < sl.candidateQueriesPerSecond.mean { - return lessThanMean - } else if curQPS < maxQPS { - return moreThanMean - } else { - return overfull - } - } - maxRangeCount := overfullRangeThreshold(options, sl.candidateRanges.mean) - minRangeCount := underfullRangeThreshold(options, sl.candidateRanges.mean) - curRangeCount := float64(sc.RangeCount) - if curRangeCount < minRangeCount { - return underfull - } else if curRangeCount < sl.candidateRanges.mean { - return lessThanMean - } else if curRangeCount < maxRangeCount { - return moreThanMean - } else { - return overfull - } -} - -func overfullRangeThreshold(options scorerOptions, mean float64) float64 { +func overfullRangeThreshold(options rangeCountScorerOptions, mean float64) float64 { return mean + math.Max(mean*options.rangeRebalanceThreshold, minRangeRebalanceThreshold) } -func underfullRangeThreshold(options scorerOptions, mean float64) float64 { +func underfullRangeThreshold(options rangeCountScorerOptions, mean float64) float64 { return mean - math.Max(mean*options.rangeRebalanceThreshold, minRangeRebalanceThreshold) } -func overfullQPSThreshold(options scorerOptions, mean float64) float64 { - return math.Max(mean*(1+options.qpsRebalanceThreshold), mean+minQPSThresholdDifference) +func overfullQPSThreshold(options qpsScorerOptions, mean float64) float64 { + return mean + math.Max(mean*options.qpsRebalanceThreshold, minQPSThresholdDifference) } -func underfullQPSThreshold(options scorerOptions, mean float64) float64 { - return math.Min(mean*(1-options.qpsRebalanceThreshold), mean-minQPSThresholdDifference) -} - -// rebalanceFromConvergesScore returns a 1 iff rebalancing a replica away from -// `sc` will _not_ converge its range count towards the mean of stores in `sl`. -// When we're considering whether to rebalance a replica away from a store or -// not, we want to give it a "boost" (i.e. make it a less likely candidate for -// removal) if it doesn't further our goal to converge range count towards the -// mean. -func rebalanceFromConvergesScore( - sl StoreList, sc roachpb.StoreCapacity, options scorerOptions, -) int { - if options.qpsRebalanceThreshold > 0 { - // If `qpsRebalanceThreshold` is set, we disable the `convergesScore` - // because we only have access to the QPS of the current (i.e. usually the - // leaseholder) replica. We do not have access to the QPS of other replicas, - // and when this method is called, the caller doesn't yet know which replica - // is going to be rebalanced. Thus, we cannot actually compute the - // `convergesScroe` when rebalancing off of QPS. - return 0 - } - if rebalanceConvergesRangeCountOnMean(sl, sc, sc.RangeCount-1) { - return 0 - } - return 1 -} - -// rebalanceToConvergesScore returns 1 if rebalancing a replica to `sc` will -// converge its range count towards the mean of all the stores inside -// `sl`. -func rebalanceToConvergesScore(sl StoreList, sc roachpb.StoreCapacity, options scorerOptions) int { - if options.qpsRebalanceThreshold > 0 { - // if `qpsRebalanceThreshold` is set, we disable the `convergesScore`. - // See comment inside `rebalanceFromConvergesScore()` for details. - return 0 - } - if rebalanceConvergesRangeCountOnMean(sl, sc, sc.RangeCount+1) { - return 1 - } - return 0 +func underfullQPSThreshold(options qpsScorerOptions, mean float64) float64 { + return mean - math.Max(mean*options.qpsRebalanceThreshold, minQPSThresholdDifference) } func rebalanceConvergesRangeCountOnMean( diff --git a/pkg/kv/kvserver/allocator_scorer_test.go b/pkg/kv/kvserver/allocator_scorer_test.go index efb4afff4411..5c3ebfc4588c 100644 --- a/pkg/kv/kvserver/allocator_scorer_test.go +++ b/pkg/kv/kvserver/allocator_scorer_test.go @@ -1036,7 +1036,7 @@ func TestShouldRebalanceDiversity(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - options := scorerOptions{} + options := rangeCountScorerOptions{} newStore := func(id int, locality roachpb.Locality) roachpb.StoreDescriptor { return roachpb.StoreDescriptor{ StoreID: roachpb.StoreID(id), @@ -1462,11 +1462,13 @@ func TestDiversityScoreEquivalence(t *testing.T) { } } -func TestBalanceScore(t *testing.T) { +func TestBalanceScoreByRangeCount(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - options := scorerOptions{} + options := rangeCountScorerOptions{ + rangeRebalanceThreshold: 0.1, + } storeList := StoreList{ candidateRanges: stat{mean: 1000}, } @@ -1486,18 +1488,24 @@ func TestBalanceScore(t *testing.T) { sRangesOverfull.RangeCount = 1500 sRangesUnderfull := sMean sRangesUnderfull.RangeCount = 500 + sRangesLessThanMean := sMean + sRangesLessThanMean.RangeCount = 900 + sRangesMoreThanMean := sMean + sRangesMoreThanMean.RangeCount = 1099 testCases := []struct { sc roachpb.StoreCapacity expected balanceStatus }{ {sEmpty, underfull}, - {sMean, moreThanMean}, + {sRangesLessThanMean, lessThanEqualToMean}, + {sMean, lessThanEqualToMean}, + {sRangesMoreThanMean, moreThanMean}, {sRangesOverfull, overfull}, {sRangesUnderfull, underfull}, } for i, tc := range testCases { - if a, e := balanceScore(storeList, tc.sc, options), tc.expected; a != e { + if a, e := options.balanceScore(storeList, tc.sc), tc.expected; a != e { t.Errorf("%d: balanceScore(storeList, %+v) got %d; want %d", i, tc.sc, a, e) } } @@ -1516,9 +1524,9 @@ func TestRebalanceBalanceScoreOnQPS(t *testing.T) { expBalanceScore balanceStatus }{ {0, underfull}, - {900, lessThanMean}, - {999, lessThanMean}, - {1000, moreThanMean}, + {900, lessThanEqualToMean}, + {999, lessThanEqualToMean}, + {1000, lessThanEqualToMean}, {1001, moreThanMean}, {2000, overfull}, } @@ -1527,15 +1535,15 @@ func TestRebalanceBalanceScoreOnQPS(t *testing.T) { sc := roachpb.StoreCapacity{ QueriesPerSecond: tc.QPS, } - options := scorerOptions{ + options := qpsScorerOptions{ qpsRebalanceThreshold: 0.1, } - if a, e := balanceScore(storeList, sc, options), tc.expBalanceScore; a != e { + if a, e := options.balanceScore(storeList, sc), tc.expBalanceScore; a != e { t.Errorf("%d: rebalanceToConvergesScore(storeList, %+v) got %d; want %d", i, sc, a, e) } // NB: Any replica whose removal would not converge the QPS to the mean is // given a score of 1 to make it less attractive for removal. - if a, e := balanceScore(storeList, sc, options), tc.expBalanceScore; a != e { + if a, e := options.balanceScore(storeList, sc), tc.expBalanceScore; a != e { t.Errorf("%d: rebalanceFromConvergesScore(storeList, %+v) got %d; want %d", i, sc, a, e) } } @@ -1562,19 +1570,20 @@ func TestRebalanceConvergesRangeCountOnMean(t *testing.T) { {2000, false, true}, } + options := rangeCountScorerOptions{} for i, tc := range testCases { sc := roachpb.StoreCapacity{ RangeCount: tc.rangeCount, } - if a, e := rebalanceToConvergesScore( - storeList, sc, scorerOptions{}, + if a, e := options.rebalanceToConvergesScore( + storeList, sc, ) == 1, tc.toConverges; a != e { t.Errorf("%d: rebalanceToConvergesScore(storeList, %+v) got %t; want %t", i, sc, a, e) } // NB: Any replica whose removal would not converge the range count to the // mean is given a score of 1 to make it less attractive for removal. - if a, e := rebalanceFromConvergesScore( - storeList, sc, scorerOptions{}, + if a, e := options.rebalanceFromConvergesScore( + storeList, sc, ) == 0, tc.fromConverges; a != e { t.Errorf("%d: rebalanceFromConvergesScore(storeList, %+v) got %t; want %t", i, sc, a, e) } diff --git a/pkg/kv/kvserver/allocator_test.go b/pkg/kv/kvserver/allocator_test.go index 0054b5694f88..8d36cbb18a47 100644 --- a/pkg/kv/kvserver/allocator_test.go +++ b/pkg/kv/kvserver/allocator_test.go @@ -919,7 +919,7 @@ func TestAllocatorRebalance(t *testing.T) { t.Fatalf("%d: unable to get store %d descriptor", i, store.StoreID) } sl, _, _ := a.storePool.getStoreList(storeFilterThrottled) - result := shouldRebalanceBasedOnThresholds(ctx, desc, sl, a.scorerOptions()) + result := a.scorerOptions().shouldRebalanceBasedOnThresholds(ctx, desc, sl) if expResult := (i >= 2); expResult != result { t.Errorf("%d: expected rebalance %t; got %t; desc %+v; sl: %+v", i, expResult, result, desc, sl) } @@ -1315,8 +1315,12 @@ func TestAllocatorRebalanceThrashing(t *testing.T) { if !ok { t.Fatalf("[store %d]: unable to get store %d descriptor", j, store.StoreID) } - if a, e := shouldRebalanceBasedOnThresholds(context.Background(), desc, sl, a.scorerOptions()), cluster[j].shouldRebalanceFrom; a != e { - t.Errorf("[store %d]: shouldRebalanceBasedOnThresholds %t != expected %t", store.StoreID, a, e) + if a, e := a.scorerOptions().shouldRebalanceBasedOnThresholds( + context.Background(), desc, sl, + ), cluster[j].shouldRebalanceFrom; a != e { + t.Errorf( + "[store %d]: shouldRebalanceBasedOnThresholds %t != expected %t", store.StoreID, a, e, + ) } } }) @@ -1403,7 +1407,7 @@ func TestAllocatorRebalanceByQPS(t *testing.T) { gossiputil.NewStoreGossiper(g).GossipStores(subtest.testStores, t) ctx := context.Background() var rangeUsageInfo RangeUsageInfo - options := scorerOptions{ + options := qpsScorerOptions{ qpsRebalanceThreshold: 0.1, } add, remove, _, ok := a.RebalanceVoter( @@ -1426,7 +1430,7 @@ func TestAllocatorRebalanceByQPS(t *testing.T) { t.Fatalf("unable to get store %d descriptor", remove.StoreID) } sl, _, _ := a.storePool.getStoreList(storeFilterThrottled) - result := shouldRebalanceBasedOnThresholds(ctx, desc, sl, options) + result := options.shouldRebalanceBasedOnThresholds(ctx, desc, sl) require.True(t, result) } else { require.False(t, ok) @@ -1520,7 +1524,7 @@ func TestAllocatorRemoveBasedOnQPS(t *testing.T) { defer stopper.Stop(context.Background()) gossiputil.NewStoreGossiper(g).GossipStores(subtest.testStores, t) ctx := context.Background() - options := scorerOptions{ + options := qpsScorerOptions{ qpsRebalanceThreshold: 0.1, } remove, _, err := a.RemoveVoter( @@ -1598,7 +1602,7 @@ func TestAllocatorRebalanceByCount(t *testing.T) { t.Fatalf("%d: unable to get store %d descriptor", i, store.StoreID) } sl, _, _ := a.storePool.getStoreList(storeFilterThrottled) - result := shouldRebalanceBasedOnThresholds(ctx, desc, sl, a.scorerOptions()) + result := a.scorerOptions().shouldRebalanceBasedOnThresholds(ctx, desc, sl) if expResult := (i < 3); expResult != result { t.Errorf("%d: expected rebalance %t; got %t", i, expResult, result) } diff --git a/pkg/kv/kvserver/store_rebalancer.go b/pkg/kv/kvserver/store_rebalancer.go index c98251d3a9b3..dd2650143886 100644 --- a/pkg/kv/kvserver/store_rebalancer.go +++ b/pkg/kv/kvserver/store_rebalancer.go @@ -209,7 +209,7 @@ func (sr *StoreRebalancer) Start(ctx context.Context, stopper *stop.Stopper) { // Instead, we use our own implementation of `scorerOptions` that promotes QPS // balance. func (sr *StoreRebalancer) scorerOptions() scorerOptions { - return scorerOptions{ + return qpsScorerOptions{ deterministic: sr.rq.allocator.storePool.deterministic, qpsRebalanceThreshold: qpsRebalanceThreshold.Get(&sr.st.SV), } @@ -229,8 +229,9 @@ func (sr *StoreRebalancer) rebalanceStore( ctx context.Context, mode LBRebalancingMode, allStoresList StoreList, ) { // First check if we should transfer leases away to better balance QPS. - options := scorerOptions{ - qpsRebalanceThreshold: qpsRebalanceThreshold.Get(&sr.st.SV), + options, ok := sr.scorerOptions().(qpsScorerOptions) + if !ok { + log.Fatalf(ctx, "expected the `StoreRebalancer` to be using a `qpsScorerOptions`") } qpsMinThreshold := underfullQPSThreshold(options, allStoresList.candidateQueriesPerSecond.mean) qpsMaxThreshold := overfullQPSThreshold(options, allStoresList.candidateQueriesPerSecond.mean) @@ -318,6 +319,7 @@ func (sr *StoreRebalancer) rebalanceStore( &replicasToMaybeRebalance, localDesc, allStoresList, + sr.scorerOptions(), ) if replWithStats.repl == nil { log.Infof(ctx, @@ -521,6 +523,7 @@ func (sr *StoreRebalancer) chooseRangeToRebalance( hottestRanges *[]replicaWithStats, localDesc *roachpb.StoreDescriptor, allStoresList StoreList, + options scorerOptions, ) (replWithStats replicaWithStats, voterTargets, nonVoterTargets []roachpb.ReplicationTarget) { now := sr.rq.store.Clock().NowAsClockTimestamp() for { @@ -592,7 +595,11 @@ func (sr *StoreRebalancer) chooseRangeToRebalance( log.VEventf(ctx, 3, "considering replica rebalance for r%d with %.2f qps", replWithStats.repl.GetRangeID(), replWithStats.qps) - targetVoterRepls, targetNonVoterRepls := sr.getRebalanceTargetsBasedOnQPS(ctx, rebalanceCtx) + targetVoterRepls, targetNonVoterRepls := sr.getRebalanceTargetsBasedOnQPS( + ctx, + rebalanceCtx, + options, + ) storeDescMap := storeListToMap(allStoresList) // Pick the voter with the least QPS to be leaseholder; @@ -630,9 +637,8 @@ func (sr *StoreRebalancer) chooseRangeToRebalance( // constraints on the range and would further the goal of balancing the QPS on // the stores in this cluster. func (sr *StoreRebalancer) getRebalanceTargetsBasedOnQPS( - ctx context.Context, rbCtx rangeRebalanceContext, + ctx context.Context, rbCtx rangeRebalanceContext, options scorerOptions, ) (finalVoterTargets, finalNonVoterTargets []roachpb.ReplicaDescriptor) { - options := sr.scorerOptions() finalVoterTargets = rbCtx.rangeDesc.Replicas().VoterDescriptors() finalNonVoterTargets = rbCtx.rangeDesc.Replicas().NonVoterDescriptors() diff --git a/pkg/kv/kvserver/store_rebalancer_test.go b/pkg/kv/kvserver/store_rebalancer_test.go index bf460da4e368..8bb5c91f87df 100644 --- a/pkg/kv/kvserver/store_rebalancer_test.go +++ b/pkg/kv/kvserver/store_rebalancer_test.go @@ -501,6 +501,10 @@ func TestChooseRangeToRebalanceRandom(t *testing.T) { &hottestRanges, &localDesc, storeList, + qpsScorerOptions{ + deterministic: false, + qpsRebalanceThreshold: qpsRebalanceThreshold, + }, ) var rebalancedVoterStores, rebalancedNonVoterStores []roachpb.StoreID for _, target := range voterTargets { @@ -733,6 +737,7 @@ func TestChooseRangeToRebalanceAcrossHeterogeneousZones(t *testing.T) { &hottestRanges, &localDesc, storeList, + qpsScorerOptions{deterministic: true, qpsRebalanceThreshold: 0.05}, ) require.Len(t, voterTargets, len(tc.expRebalancedVoters)) @@ -833,7 +838,13 @@ func TestNoLeaseTransferToBehindReplicas(t *testing.T) { hottestRanges = rr.topQPS() repl = hottestRanges[0].repl - _, targets, _ := sr.chooseRangeToRebalance(ctx, &hottestRanges, &localDesc, storeList) + _, targets, _ := sr.chooseRangeToRebalance( + ctx, + &hottestRanges, + &localDesc, + storeList, + qpsScorerOptions{deterministic: true, qpsRebalanceThreshold: 0.25}, + ) expectTargets := []roachpb.ReplicationTarget{ {NodeID: 4, StoreID: 4}, {NodeID: 3, StoreID: 3}, {NodeID: 5, StoreID: 5}, } From 153fd6bf1f20759e635908ba38a82d73d885974c Mon Sep 17 00:00:00 2001 From: Aayush Shah Date: Mon, 31 May 2021 21:30:37 -0400 Subject: [PATCH 6/7] kvserver: rename `StoreList.filter` This commit renames `StoreList`'s `filter()` method to `excludeInvalid()` as the existing name was ambiguous. Release justification: Fixes high priority bug Release note: None --- pkg/kv/kvserver/allocator.go | 8 ++++---- pkg/kv/kvserver/allocator_scorer.go | 4 ++-- pkg/kv/kvserver/allocator_scorer_test.go | 2 +- pkg/kv/kvserver/store_pool.go | 6 +++--- pkg/kv/kvserver/store_pool_test.go | 6 +++--- pkg/kv/kvserver/store_rebalancer.go | 4 ++-- 6 files changed, 15 insertions(+), 15 deletions(-) diff --git a/pkg/kv/kvserver/allocator.go b/pkg/kv/kvserver/allocator.go index 574462c09eb3..25c1657ec1c1 100644 --- a/pkg/kv/kvserver/allocator.go +++ b/pkg/kv/kvserver/allocator.go @@ -1276,8 +1276,8 @@ func (a *Allocator) TransferLeaseTarget( alwaysAllowDecisionWithoutStats bool, ) roachpb.ReplicaDescriptor { sl, _, _ := a.storePool.getStoreList(storeFilterSuspect) - sl = sl.filter(conf.Constraints) - sl = sl.filter(conf.VoterConstraints) + sl = sl.excludeInvalid(conf.Constraints) + sl = sl.excludeInvalid(conf.VoterConstraints) // The only thing we use the storeList for is for the lease mean across the // eligible stores, make that explicit here. candidateLeasesMean := sl.candidateLeases.mean @@ -1445,8 +1445,8 @@ func (a *Allocator) ShouldTransferLease( } sl, _, _ := a.storePool.getStoreList(storeFilterSuspect) - sl = sl.filter(conf.Constraints) - sl = sl.filter(conf.VoterConstraints) + sl = sl.excludeInvalid(conf.Constraints) + sl = sl.excludeInvalid(conf.VoterConstraints) log.VEventf(ctx, 3, "ShouldTransferLease (lease-holder=%d):\n%s", leaseStoreID, sl) // Only consider live, non-draining, non-suspect replicas. diff --git a/pkg/kv/kvserver/allocator_scorer.go b/pkg/kv/kvserver/allocator_scorer.go index 260f924263b4..855186c933a8 100644 --- a/pkg/kv/kvserver/allocator_scorer.go +++ b/pkg/kv/kvserver/allocator_scorer.go @@ -1365,9 +1365,9 @@ func containsStore(stores []roachpb.StoreID, target roachpb.StoreID) bool { return false } -// constraintsCheck returns true iff the provided store would be a valid in a +// isStoreValid returns true iff the provided store would be a valid in a // range with the provided constraints. -func constraintsCheck( +func isStoreValid( store roachpb.StoreDescriptor, constraints []roachpb.ConstraintsConjunction, ) bool { if len(constraints) == 0 { diff --git a/pkg/kv/kvserver/allocator_scorer_test.go b/pkg/kv/kvserver/allocator_scorer_test.go index 5c3ebfc4588c..e9d43d9bd20f 100644 --- a/pkg/kv/kvserver/allocator_scorer_test.go +++ b/pkg/kv/kvserver/allocator_scorer_test.go @@ -651,7 +651,7 @@ func TestConstraintsCheck(t *testing.T) { for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { for _, s := range testStores { - valid := constraintsCheck(s, tc.constraints) + valid := isStoreValid(s, tc.constraints) ok := tc.expected[s.StoreID] if valid != ok { t.Errorf("expected store %d to be %t, but got %t", s.StoreID, ok, valid) diff --git a/pkg/kv/kvserver/store_pool.go b/pkg/kv/kvserver/store_pool.go index 929836db218d..44c00b4e52fd 100644 --- a/pkg/kv/kvserver/store_pool.go +++ b/pkg/kv/kvserver/store_pool.go @@ -793,16 +793,16 @@ func (sl StoreList) String() string { return buf.String() } -// filter takes a store list and removes stores that would be explicitly invalid +// excludeInvalid takes a store list and removes stores that would be explicitly invalid // under the given set of constraints. It maintains the original order of the // passed in store list. -func (sl StoreList) filter(constraints []roachpb.ConstraintsConjunction) StoreList { +func (sl StoreList) excludeInvalid(constraints []roachpb.ConstraintsConjunction) StoreList { if len(constraints) == 0 { return sl } var filteredDescs []roachpb.StoreDescriptor for _, store := range sl.stores { - if ok := constraintsCheck(store, constraints); ok { + if ok := isStoreValid(store, constraints); ok { filteredDescs = append(filteredDescs, store) } } diff --git a/pkg/kv/kvserver/store_pool_test.go b/pkg/kv/kvserver/store_pool_test.go index 3010375795ae..310b1ea7a843 100644 --- a/pkg/kv/kvserver/store_pool_test.go +++ b/pkg/kv/kvserver/store_pool_test.go @@ -172,7 +172,7 @@ func verifyStoreList( sl, aliveStoreCount, throttled = sp.getStoreListFromIDs(storeIDs, filter) } throttledStoreCount := len(throttled) - sl = sl.filter(constraints) + sl = sl.excludeInvalid(constraints) if aliveStoreCount != expectedAliveStoreCount { return errors.Errorf("expected AliveStoreCount %d does not match actual %d", expectedAliveStoreCount, aliveStoreCount) @@ -216,7 +216,7 @@ func TestStorePoolGetStoreList(t *testing.T) { required := []string{"ssd", "dc"} // Nothing yet. sl, _, _ := sp.getStoreList(storeFilterNone) - sl = sl.filter(constraints) + sl = sl.excludeInvalid(constraints) if len(sl.stores) != 0 { t.Errorf("expected no stores, instead %+v", sl.stores) } @@ -485,7 +485,7 @@ func TestStoreListFilter(t *testing.T) { } } - filtered := sl.filter(constraints) + filtered := sl.excludeInvalid(constraints) if !reflect.DeepEqual(expected, filtered.stores) { t.Errorf("did not get expected stores %s", pretty.Diff(expected, filtered.stores)) } diff --git a/pkg/kv/kvserver/store_rebalancer.go b/pkg/kv/kvserver/store_rebalancer.go index dd2650143886..55ab046c8d55 100644 --- a/pkg/kv/kvserver/store_rebalancer.go +++ b/pkg/kv/kvserver/store_rebalancer.go @@ -483,8 +483,8 @@ func (sr *StoreRebalancer) chooseLeaseToTransfer( continue } - filteredStoreList := storeList.filter(conf.Constraints) - filteredStoreList = storeList.filter(conf.VoterConstraints) + filteredStoreList := storeList.excludeInvalid(conf.Constraints) + filteredStoreList = storeList.excludeInvalid(conf.VoterConstraints) if sr.rq.allocator.followTheWorkloadPrefersLocal( ctx, filteredStoreList, From d61f474635f814bc420a61a11e6660b8882e05f0 Mon Sep 17 00:00:00 2001 From: Aayush Shah Date: Mon, 23 Aug 2021 22:10:41 -0400 Subject: [PATCH 7/7] kvserver: promote QPS convergence during load-based lease rebalancing This commit augments `TransferLeaseTarget()` by adding a mode that picks the best lease transfer target that would lead to QPS convergence across the stores that have a replica for a given range. This commit implements a strategy that predicates lease transfer decisions on whether they would serve to reduce the QPS delta between existing replicas' stores. Resolves https://github.com/cockroachdb/cockroach/issues/31135 Release justification: Fixes high priority bug Release note (bug fix): Previously, the store rebalancer was unable to rebalance leases for hot ranges that received a disproportionate amount of traffic relative to the rest of the cluster. This often led to prolonged single node hotspots in certain workloads that led to hot ranges. This bug is now fixed. --- pkg/kv/kvserver/allocator.go | 269 +++++++++++++++++------ pkg/kv/kvserver/allocator_test.go | 156 ++++++++----- pkg/kv/kvserver/replica_stats.go | 12 + pkg/kv/kvserver/replicate_queue.go | 18 +- pkg/kv/kvserver/store.go | 15 +- pkg/kv/kvserver/store_rebalancer.go | 207 ++++++----------- pkg/kv/kvserver/store_rebalancer_test.go | 195 ++++++++++++---- pkg/kv/kvserver/testing_knobs.go | 9 + 8 files changed, 560 insertions(+), 321 deletions(-) diff --git a/pkg/kv/kvserver/allocator.go b/pkg/kv/kvserver/allocator.go index 25c1657ec1c1..694347335798 100644 --- a/pkg/kv/kvserver/allocator.go +++ b/pkg/kv/kvserver/allocator.go @@ -354,11 +354,15 @@ type Allocator struct { storePool *StorePool nodeLatencyFn func(addr string) (time.Duration, bool) randGen allocatorRand + + knobs *AllocatorTestingKnobs } // MakeAllocator creates a new allocator using the specified StorePool. func MakeAllocator( - storePool *StorePool, nodeLatencyFn func(addr string) (time.Duration, bool), + storePool *StorePool, + nodeLatencyFn func(addr string) (time.Duration, bool), + knobs *AllocatorTestingKnobs, ) Allocator { var randSource rand.Source // There are number of test cases that make a test store but don't add @@ -373,6 +377,7 @@ func MakeAllocator( storePool: storePool, nodeLatencyFn: nodeLatencyFn, randGen: makeAllocatorRand(randSource), + knobs: knobs, } } @@ -1271,15 +1276,16 @@ func (a *Allocator) TransferLeaseTarget( GetRangeID() roachpb.RangeID }, stats *replicaStats, - checkTransferLeaseSource bool, - checkCandidateFullness bool, - alwaysAllowDecisionWithoutStats bool, + forceDecisionWithoutStats bool, + opts transferLeaseOptions, ) roachpb.ReplicaDescriptor { + allStoresList, _, _ := a.storePool.getStoreList(storeFilterNone) + storeDescMap := storeListToMap(allStoresList) + sl, _, _ := a.storePool.getStoreList(storeFilterSuspect) sl = sl.excludeInvalid(conf.Constraints) sl = sl.excludeInvalid(conf.VoterConstraints) - // The only thing we use the storeList for is for the lease mean across the - // eligible stores, make that explicit here. + candidateLeasesMean := sl.candidateLeases.mean source, ok := a.storePool.getStoreDescriptor(leaseRepl.StoreID()) @@ -1292,6 +1298,7 @@ func (a *Allocator) TransferLeaseTarget( // store matches, it's where the lease should be (unless the preferred store // is the current one and checkTransferLeaseSource is false). var preferred []roachpb.ReplicaDescriptor + checkTransferLeaseSource := opts.checkTransferLeaseSource if checkTransferLeaseSource { preferred = a.preferredLeaseholders(conf, existing) } else { @@ -1328,24 +1335,26 @@ func (a *Allocator) TransferLeaseTarget( // Only consider live, non-draining, non-suspect replicas. existing, _ = a.storePool.liveAndDeadReplicas(existing, false /* includeSuspectAndDrainingStores */) - // Only proceed with the lease transfer if we are also the raft leader (we - // already know we are the leaseholder at this point), and only consider - // replicas that are in `StateReplicate` as potential candidates. - // - // NB: The RaftStatus() only returns a non-empty and non-nil result on the - // Raft leader (since Raft followers do not track the progress of other - // replicas, only the leader does). - // - // NB: On every Raft tick, we try to ensure that leadership is collocated with - // leaseholdership (see - // Replica.maybeTransferRaftLeadershipToLeaseholderLocked()). This means that - // on a range that is not already borked (i.e. can accept writes), periods of - // leader/leaseholder misalignment should be ephemeral and rare. We choose to - // be pessimistic here and choose to bail on the lease transfer, as opposed to - // potentially transferring the lease to a replica that may be waiting for a - // snapshot (which will wedge the range until the replica applies that - // snapshot). - existing = excludeReplicasInNeedOfSnapshots(ctx, leaseRepl.RaftStatus(), existing) + if a.knobs == nil || !a.knobs.AllowLeaseTransfersToReplicasNeedingSnapshots { + // Only proceed with the lease transfer if we are also the raft leader (we + // already know we are the leaseholder at this point), and only consider + // replicas that are in `StateReplicate` as potential candidates. + // + // NB: The RaftStatus() only returns a non-empty and non-nil result on the + // Raft leader (since Raft followers do not track the progress of other + // replicas, only the leader does). + // + // NB: On every Raft tick, we try to ensure that leadership is collocated with + // leaseholdership (see + // Replica.maybeTransferRaftLeadershipToLeaseholderLocked()). This means that + // on a range that is not already borked (i.e. can accept writes), periods of + // leader/leaseholder misalignment should be ephemeral and rare. We choose to + // be pessimistic here and choose to bail on the lease transfer, as opposed to + // potentially transferring the lease to a replica that may be waiting for a + // snapshot (which will wedge the range until the replica applies that + // snapshot). + existing = excludeReplicasInNeedOfSnapshots(ctx, leaseRepl.RaftStatus(), existing) + } // Short-circuit if there are no valid targets out there. if len(existing) == 0 || (len(existing) == 1 && existing[0].StoreID == leaseRepl.StoreID()) { @@ -1353,65 +1362,172 @@ func (a *Allocator) TransferLeaseTarget( return roachpb.ReplicaDescriptor{} } - // Try to pick a replica to transfer the lease to while also determining - // whether we actually should be transferring the lease. The transfer - // decision is only needed if we've been asked to check the source. - transferDec, repl := a.shouldTransferLeaseUsingStats( - ctx, source, existing, stats, nil, candidateLeasesMean, - ) - if checkTransferLeaseSource { - switch transferDec { - case shouldNotTransfer: - if !alwaysAllowDecisionWithoutStats { - return roachpb.ReplicaDescriptor{} + switch g := opts.goal; g { + case followTheWorkload: + // Try to pick a replica to transfer the lease to while also determining + // whether we actually should be transferring the lease. The transfer + // decision is only needed if we've been asked to check the source. + transferDec, repl := a.shouldTransferLeaseForAccessLocality( + ctx, source, existing, stats, nil, candidateLeasesMean, + ) + if checkTransferLeaseSource { + switch transferDec { + case shouldNotTransfer: + if !forceDecisionWithoutStats { + return roachpb.ReplicaDescriptor{} + } + fallthrough + case decideWithoutStats: + if !a.shouldTransferLeaseForLeaseCountConvergence(ctx, sl, source, existing) { + return roachpb.ReplicaDescriptor{} + } + case shouldTransfer: + default: + log.Fatalf(ctx, "unexpected transfer decision %d with replica %+v", transferDec, repl) + } + } + if repl != (roachpb.ReplicaDescriptor{}) { + return repl + } + fallthrough + + case leaseCountConvergence: + // Fall back to logic that doesn't take request counts and latency into + // account if the counts/latency-based logic couldn't pick a best replica. + candidates := make([]roachpb.ReplicaDescriptor, 0, len(existing)) + var bestOption roachpb.ReplicaDescriptor + bestOptionLeaseCount := int32(math.MaxInt32) + for _, repl := range existing { + if leaseRepl.StoreID() == repl.StoreID { + continue + } + storeDesc, ok := a.storePool.getStoreDescriptor(repl.StoreID) + if !ok { + continue } - fallthrough - case decideWithoutStats: - if !a.shouldTransferLeaseWithoutStats(ctx, sl, source, existing) { - return roachpb.ReplicaDescriptor{} + if !opts.checkCandidateFullness || float64(storeDesc.Capacity.LeaseCount) < candidateLeasesMean-0.5 { + candidates = append(candidates, repl) + } else if storeDesc.Capacity.LeaseCount < bestOptionLeaseCount { + bestOption = repl + bestOptionLeaseCount = storeDesc.Capacity.LeaseCount } - case shouldTransfer: - default: - log.Fatalf(ctx, "unexpected transfer decision %d with replica %+v", transferDec, repl) } - } + if len(candidates) == 0 { + // If we aren't supposed to be considering the current leaseholder (e.g. + // because we need to remove this replica for some reason), return + // our best option if we otherwise wouldn't want to do anything. + if !checkTransferLeaseSource { + return bestOption + } + return roachpb.ReplicaDescriptor{} + } + a.randGen.Lock() + defer a.randGen.Unlock() + return candidates[a.randGen.Intn(len(candidates))] + + case qpsConvergence: + // When the goal is to further QPS convergence across stores, we ensure that + // any lease transfer decision we make *reduces the delta between the store + // serving the highest QPS and the store serving the lowest QPS* among our + // list of candidates. + + // Create a separate map of store_id -> qps that we can manipulate in order + // to simulate the resulting QPS distribution of various potential lease + // transfer decisions. + storeQPSMap := make(map[roachpb.StoreID]float64) + for _, storeDesc := range storeDescMap { + storeQPSMap[storeDesc.StoreID] = storeDesc.Capacity.QueriesPerSecond + } - if repl != (roachpb.ReplicaDescriptor{}) { - return repl + leaseholderStoreQPS, ok := storeQPSMap[leaseRepl.StoreID()] + if !ok { + log.VEventf( + ctx, 3, "cannot find store descriptor for leaseholder s%d;"+ + " skipping this range", leaseRepl.StoreID(), + ) + return roachpb.ReplicaDescriptor{} + } + + leaseholderReplQPS, _ := stats.avgQPS() + currentDelta := getQPSDelta(storeQPSMap, existing) + bestOption := getCandidateWithMinQPS(storeQPSMap, existing) + if bestOption != (roachpb.ReplicaDescriptor{}) && + // It is always beneficial to transfer the lease to the coldest candidate + // if the range's own qps is smaller than the difference between the + // leaseholder store and the candidate store. This will always drive down + // the difference between those two stores, which should always drive down + // the difference between the store serving the highest QPS and the store + // serving the lowest QPS. + // + // TODO(aayush): We should think about whether we need any padding here. + // Not adding any sort of padding could make this a little sensitive, but + // there are some downsides to doing so. If the padding here is too high, + // we're going to keep ignoring opportunities for lease transfers for + // ranges with low QPS. This can add up and prevent us from achieving + // convergence in cases where we're dealing with a ton of very low-QPS + // ranges. + (leaseholderStoreQPS-leaseholderReplQPS) > storeQPSMap[bestOption.StoreID] { + storeQPSMap[leaseRepl.StoreID()] -= leaseholderReplQPS + storeQPSMap[bestOption.StoreID] += leaseholderReplQPS + minDelta := getQPSDelta(storeQPSMap, existing) + log.VEventf( + ctx, + 3, + "lease transfer to s%d would reduce the QPS delta between this ranges' stores from %.2f to %.2f", + bestOption.StoreID, + currentDelta, + minDelta, + ) + return bestOption + } + return roachpb.ReplicaDescriptor{} + default: + log.Fatalf(ctx, "unexpected lease transfer goal %d", g) } + panic("unreachable") +} - // Fall back to logic that doesn't take request counts and latency into - // account if the counts/latency-based logic couldn't pick a best replica. - candidates := make([]roachpb.ReplicaDescriptor, 0, len(existing)) - var bestOption roachpb.ReplicaDescriptor - bestOptionLeaseCount := int32(math.MaxInt32) +// getCandidateWithMinQPS returns the `ReplicaDescriptor` that belongs to the +// store serving the lowest QPS among all the `existing` replicas. +func getCandidateWithMinQPS( + storeQPSMap map[roachpb.StoreID]float64, existing []roachpb.ReplicaDescriptor, +) roachpb.ReplicaDescriptor { + minCandidateQPS := math.MaxFloat64 + var candidateWithMin roachpb.ReplicaDescriptor for _, repl := range existing { - if leaseRepl.StoreID() == repl.StoreID { + candidateQPS, ok := storeQPSMap[repl.StoreID] + if !ok { continue } - storeDesc, ok := a.storePool.getStoreDescriptor(repl.StoreID) + if minCandidateQPS > candidateQPS { + minCandidateQPS = candidateQPS + candidateWithMin = repl + } + } + return candidateWithMin +} + +// getQPSDelta returns the difference between the store serving the highest QPS +// and the store serving the lowest QPS, among the set of stores that have an +// `existing` replica. +func getQPSDelta( + storeQPSMap map[roachpb.StoreID]float64, existing []roachpb.ReplicaDescriptor, +) float64 { + maxCandidateQPS := float64(0) + minCandidateQPS := math.MaxFloat64 + for _, repl := range existing { + candidateQPS, ok := storeQPSMap[repl.StoreID] if !ok { continue } - if !checkCandidateFullness || float64(storeDesc.Capacity.LeaseCount) < candidateLeasesMean-0.5 { - candidates = append(candidates, repl) - } else if storeDesc.Capacity.LeaseCount < bestOptionLeaseCount { - bestOption = repl - bestOptionLeaseCount = storeDesc.Capacity.LeaseCount + if maxCandidateQPS < candidateQPS { + maxCandidateQPS = candidateQPS } - } - if len(candidates) == 0 { - // If we aren't supposed to be considering the current leaseholder (e.g. - // because we need to remove this replica for some reason), return - // our best option if we otherwise wouldn't want to do anything. - if !checkTransferLeaseSource { - return bestOption + if minCandidateQPS > candidateQPS { + minCandidateQPS = candidateQPS } - return roachpb.ReplicaDescriptor{} } - a.randGen.Lock() - defer a.randGen.Unlock() - return candidates[a.randGen.Intn(len(candidates))] + return maxCandidateQPS - minCandidateQPS } // ShouldTransferLease returns true if the specified store is overfull in terms @@ -1457,7 +1573,14 @@ func (a *Allocator) ShouldTransferLease( return false } - transferDec, _ := a.shouldTransferLeaseUsingStats(ctx, source, existing, stats, nil, sl.candidateLeases.mean) + transferDec, _ := a.shouldTransferLeaseForAccessLocality( + ctx, + source, + existing, + stats, + nil, + sl.candidateLeases.mean, + ) var result bool switch transferDec { case shouldNotTransfer: @@ -1465,7 +1588,7 @@ func (a *Allocator) ShouldTransferLease( case shouldTransfer: result = true case decideWithoutStats: - result = a.shouldTransferLeaseWithoutStats(ctx, sl, source, existing) + result = a.shouldTransferLeaseForLeaseCountConvergence(ctx, sl, source, existing) default: log.Fatalf(ctx, "unexpected transfer decision %d", transferDec) } @@ -1483,7 +1606,7 @@ func (a Allocator) followTheWorkloadPrefersLocal( stats *replicaStats, ) bool { adjustments := make(map[roachpb.StoreID]float64) - decision, _ := a.shouldTransferLeaseUsingStats(ctx, source, existing, stats, adjustments, sl.candidateLeases.mean) + decision, _ := a.shouldTransferLeaseForAccessLocality(ctx, source, existing, stats, adjustments, sl.candidateLeases.mean) if decision == decideWithoutStats { return false } @@ -1497,7 +1620,7 @@ func (a Allocator) followTheWorkloadPrefersLocal( return false } -func (a Allocator) shouldTransferLeaseUsingStats( +func (a Allocator) shouldTransferLeaseForAccessLocality( ctx context.Context, source roachpb.StoreDescriptor, existing []roachpb.ReplicaDescriptor, @@ -1681,7 +1804,7 @@ func loadBasedLeaseRebalanceScore( return totalScore, rebalanceAdjustment } -func (a Allocator) shouldTransferLeaseWithoutStats( +func (a Allocator) shouldTransferLeaseForLeaseCountConvergence( ctx context.Context, sl StoreList, source roachpb.StoreDescriptor, diff --git a/pkg/kv/kvserver/allocator_test.go b/pkg/kv/kvserver/allocator_test.go index 8d36cbb18a47..f3a991fe7fc5 100644 --- a/pkg/kv/kvserver/allocator_test.go +++ b/pkg/kv/kvserver/allocator_test.go @@ -373,6 +373,15 @@ func replicas(storeIDs ...roachpb.StoreID) []roachpb.ReplicaDescriptor { // use in tests. Stopper must be stopped by the caller. func createTestAllocator( numNodes int, deterministic bool, +) (*stop.Stopper, *gossip.Gossip, *StorePool, Allocator, *hlc.ManualClock) { + return createTestAllocatorWithKnobs(numNodes, deterministic, nil /* knobs */) +} + +// createTestAllocatorWithKnobs is like `createTestAllocator`, but allows the +// caller to pass in custom AllocatorTestingKnobs. Stopper must be stopped by +// the caller. +func createTestAllocatorWithKnobs( + numNodes int, deterministic bool, knobs *AllocatorTestingKnobs, ) (*stop.Stopper, *gossip.Gossip, *StorePool, Allocator, *hlc.ManualClock) { stopper, g, manual, storePool, _ := createTestStorePool( TestTimeUntilStoreDeadOff, deterministic, @@ -381,7 +390,9 @@ func createTestAllocator( a := MakeAllocator( storePool, func(string) (time.Duration, bool) { return 0, true - }) + }, + knobs, + ) return stopper, g, storePool, a, manual } @@ -1700,10 +1711,12 @@ func TestAllocatorTransferLeaseTarget(t *testing.T) { replicationFactor: 3, storeID: c.leaseholder, }, - nil, /* replicaStats */ - c.check, - true, /* checkCandidateFullness */ - false, /* alwaysAllowDecisionWithoutStats */ + nil, /* stats */ + false, /* forceDecisionWithoutStats */ + transferLeaseOptions{ + checkTransferLeaseSource: c.check, + checkCandidateFullness: true, + }, ) if c.expected != target.StoreID { t.Fatalf("expected %d, but found %d", c.expected, target.StoreID) @@ -1811,9 +1824,11 @@ func TestAllocatorTransferLeaseToReplicasNeedingSnapshot(t *testing.T) { c.existing, repl, nil, - c.checkSource, - true, /* checkCandidateFullness */ false, /* alwaysAllowDecisionWithoutStats */ + transferLeaseOptions{ + checkTransferLeaseSource: c.checkSource, + checkCandidateFullness: true, + }, ) if c.transferTarget != target.StoreID { t.Fatalf("expected %d, but found %d", c.transferTarget, target.StoreID) @@ -1901,10 +1916,12 @@ func TestAllocatorTransferLeaseTargetConstraints(t *testing.T) { replicationFactor: 3, storeID: c.leaseholder, }, - nil, /* replicaStats */ - true, - true, /* checkCandidateFullness */ - false, /* alwaysAllowDecisionWithoutStats */ + nil, /* stats */ + false, /* forceDecisionWithoutStats */ + transferLeaseOptions{ + checkTransferLeaseSource: true, + checkCandidateFullness: true, + }, ) if c.expected != target.StoreID { t.Fatalf("expected %d, but found %d", c.expected, target.StoreID) @@ -1922,9 +1939,11 @@ func TestAllocatorTransferLeaseTargetDraining(t *testing.T) { TestTimeUntilStoreDeadOff, true, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_LIVE) - a := MakeAllocator(storePool, func(string) (time.Duration, bool) { - return 0, true - }) + a := MakeAllocator( + storePool, func(string) (time.Duration, bool) { + return 0, true + }, nil, /* knobs */ + ) defer stopper.Stop(context.Background()) // 3 stores where the lease count for each store is equal to 100x the store @@ -2003,10 +2022,12 @@ func TestAllocatorTransferLeaseTargetDraining(t *testing.T) { replicationFactor: 3, storeID: c.leaseholder, }, - nil, /* replicaStats */ - c.check, - true, /* checkCandidateFullness */ - false, /* alwaysAllowDecisionWithoutStats */ + nil, /* stats */ + false, /* forceDecisionWithoutStats */ + transferLeaseOptions{ + checkTransferLeaseSource: c.check, + checkCandidateFullness: true, + }, ) if c.expected != target.StoreID { t.Fatalf("expected %d, but found %d", c.expected, target.StoreID) @@ -2293,9 +2314,11 @@ func TestAllocatorShouldTransferLeaseDraining(t *testing.T) { TestTimeUntilStoreDeadOff, true, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_LIVE) - a := MakeAllocator(storePool, func(string) (time.Duration, bool) { - return 0, true - }) + a := MakeAllocator( + storePool, func(string) (time.Duration, bool) { + return 0, true + }, nil, /* knobs */ + ) defer stopper.Stop(context.Background()) // 4 stores where the lease count for each store is equal to 10x the store @@ -2355,9 +2378,11 @@ func TestAllocatorShouldTransferSuspected(t *testing.T) { TestTimeUntilStoreDeadOff, true, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_LIVE) - a := MakeAllocator(storePool, func(string) (time.Duration, bool) { - return 0, true - }) + a := MakeAllocator( + storePool, func(string) (time.Duration, bool) { + return 0, true + }, nil, /* knobs */ + ) defer stopper.Stop(context.Background()) var stores []*roachpb.StoreDescriptor @@ -2533,10 +2558,12 @@ func TestAllocatorLeasePreferences(t *testing.T) { replicationFactor: 5, storeID: c.leaseholder, }, - nil, /* replicaStats */ - true, /* checkTransferLeaseSource */ - true, /* checkCandidateFullness */ - false, /* alwaysAllowDecisionWithoutStats */ + nil, /* stats */ + false, /* forceDecisionWithoutStats */ + transferLeaseOptions{ + checkTransferLeaseSource: true, + checkCandidateFullness: true, + }, ) if c.expectedCheckTrue != target.StoreID { t.Errorf("expected s%d for check=true, but found %v", c.expectedCheckTrue, target) @@ -2549,10 +2576,12 @@ func TestAllocatorLeasePreferences(t *testing.T) { replicationFactor: 5, storeID: c.leaseholder, }, - nil, /* replicaStats */ - false, /* checkTransferLeaseSource */ - true, /* checkCandidateFullness */ - false, /* alwaysAllowDecisionWithoutStats */ + nil, /* stats */ + false, /* forceDecisionWithoutStats */ + transferLeaseOptions{ + checkTransferLeaseSource: false, + checkCandidateFullness: true, + }, ) if c.expectedCheckFalse != target.StoreID { t.Errorf("expected s%d for check=false, but found %v", c.expectedCheckFalse, target) @@ -2635,10 +2664,12 @@ func TestAllocatorLeasePreferencesMultipleStoresPerLocality(t *testing.T) { replicationFactor: 6, storeID: c.leaseholder, }, - nil, /* replicaStats */ - true, /* checkTransferLeaseSource */ - true, /* checkCandidateFullness */ - false, /* alwaysAllowDecisionWithoutStats */ + nil, /* stats */ + false, /* forceDecisionWithoutStats */ + transferLeaseOptions{ + checkTransferLeaseSource: true, + checkCandidateFullness: true, + }, ) if c.expectedCheckTrue != target.StoreID { t.Errorf("expected s%d for check=true, but found %v", c.expectedCheckTrue, target) @@ -2651,10 +2682,12 @@ func TestAllocatorLeasePreferencesMultipleStoresPerLocality(t *testing.T) { replicationFactor: 6, storeID: c.leaseholder, }, - nil, /* replicaStats */ - false, /* checkTransferLeaseSource */ - true, /* checkCandidateFullness */ - false, /* alwaysAllowDecisionWithoutStats */ + nil, /* stats */ + false, /* forceDecisionWithoutStats */ + transferLeaseOptions{ + checkTransferLeaseSource: false, + checkCandidateFullness: true, + }, ) if c.expectedCheckFalse != target.StoreID { t.Errorf("expected s%d for check=false, but found %v", c.expectedCheckFalse, target) @@ -5057,9 +5090,11 @@ func TestAllocatorTransferLeaseTargetLoadBased(t *testing.T) { for _, c := range testCases { t.Run("", func(t *testing.T) { - a := MakeAllocator(storePool, func(addr string) (time.Duration, bool) { - return c.latency[addr], true - }) + a := MakeAllocator( + storePool, func(addr string) (time.Duration, bool) { + return c.latency[addr], true + }, nil, /* knobs */ + ) target := a.TransferLeaseTarget( context.Background(), emptySpanConfig(), @@ -5069,9 +5104,12 @@ func TestAllocatorTransferLeaseTargetLoadBased(t *testing.T) { storeID: c.leaseholder, }, c.stats, - c.check, - true, /* checkCandidateFullness */ - false, /* alwaysAllowDecisionWithoutStats */ + false, + transferLeaseOptions{ + checkTransferLeaseSource: c.check, + checkCandidateFullness: true, + dryRun: false, + }, ) if c.expected != target.StoreID { t.Errorf("expected %d, got %d", c.expected, target.StoreID) @@ -6631,9 +6669,11 @@ func TestAllocatorComputeActionDynamicNumReplicas(t *testing.T) { TestTimeUntilStoreDeadOff, false, /* deterministic */ func() int { return numNodes }, livenesspb.NodeLivenessStatus_LIVE) - a := MakeAllocator(sp, func(string) (time.Duration, bool) { - return 0, true - }) + a := MakeAllocator( + sp, func(string) (time.Duration, bool) { + return 0, true + }, nil, /* knobs */ + ) ctx := context.Background() defer stopper.Stop(ctx) @@ -6738,7 +6778,7 @@ func TestAllocatorComputeActionNoStorePool(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - a := MakeAllocator(nil /* storePool */, nil /* rpcContext */) + a := MakeAllocator(nil /* storePool */, nil /* nodeLatencyFn */, nil /* knobs */) action, priority := a.ComputeAction(context.Background(), roachpb.SpanConfig{}, nil) if action != AllocatorNoop { t.Errorf("expected AllocatorNoop, but got %v", action) @@ -7258,9 +7298,11 @@ func TestAllocatorFullDisks(t *testing.T) { mockNodeLiveness.nodeLivenessFunc, false, /* deterministic */ ) - alloc := MakeAllocator(sp, func(string) (time.Duration, bool) { - return 0, false - }) + alloc := MakeAllocator( + sp, func(string) (time.Duration, bool) { + return 0, false + }, nil, /* knobs */ + ) var wg sync.WaitGroup g.RegisterCallback(gossip.MakePrefixPattern(gossip.KeyStorePrefix), @@ -7404,9 +7446,11 @@ func Example_rebalancing() { newMockNodeLiveness(livenesspb.NodeLivenessStatus_LIVE).nodeLivenessFunc, /* deterministic */ true, ) - alloc := MakeAllocator(sp, func(string) (time.Duration, bool) { - return 0, false - }) + alloc := MakeAllocator( + sp, func(string) (time.Duration, bool) { + return 0, false + }, nil, /* knobs */ + ) var wg sync.WaitGroup g.RegisterCallback(gossip.MakePrefixPattern(gossip.KeyStorePrefix), diff --git a/pkg/kv/kvserver/replica_stats.go b/pkg/kv/kvserver/replica_stats.go index 77f889d9fbba..51d09555eed0 100644 --- a/pkg/kv/kvserver/replica_stats.go +++ b/pkg/kv/kvserver/replica_stats.go @@ -57,6 +57,9 @@ type replicaStats struct { requests [6]perLocalityCounts lastRotate time.Time lastReset time.Time + + // Testing only. + avgQPSForTesting float64 } } @@ -186,6 +189,9 @@ func (rs *replicaStats) avgQPS() (float64, time.Duration) { rs.mu.Lock() defer rs.mu.Unlock() + if rs.mu.avgQPSForTesting != 0 { + return rs.mu.avgQPSForTesting, 0 + } rs.maybeRotateLocked(now) @@ -224,3 +230,9 @@ func (rs *replicaStats) resetRequestCounts() { rs.mu.lastRotate = timeutil.Unix(0, rs.clock.PhysicalNow()) rs.mu.lastReset = rs.mu.lastRotate } + +func (rs *replicaStats) setAvgQPSForTesting(qps float64) { + rs.mu.Lock() + defer rs.mu.Unlock() + rs.mu.avgQPSForTesting = qps +} diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index 7d4042f2c8fe..734f514b0c8d 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -1164,6 +1164,7 @@ func (rq *replicateQueue) considerRebalance( desc, conf, transferLeaseOptions{ + goal: followTheWorkload, checkTransferLeaseSource: true, checkCandidateFullness: true, dryRun: dryRun, @@ -1271,7 +1272,19 @@ func replicationChangesForRebalance( return chgs, performingSwap, nil } +// transferLeaseGoal dictates whether a call to TransferLeaseTarget should +// improve locality of access, convergence of lease counts or convergence of +// QPS. +type transferLeaseGoal int + +const ( + followTheWorkload transferLeaseGoal = iota + leaseCountConvergence + qpsConvergence +) + type transferLeaseOptions struct { + goal transferLeaseGoal checkTransferLeaseSource bool checkCandidateFullness bool dryRun bool @@ -1320,9 +1333,8 @@ func (rq *replicateQueue) shedLease( desc.Replicas().VoterDescriptors(), repl, repl.leaseholderStats, - opts.checkTransferLeaseSource, - opts.checkCandidateFullness, - false, /* alwaysAllowDecisionWithoutStats */ + false, /* forceDecisionWithoutStats */ + opts, ) if target == (roachpb.ReplicaDescriptor{}) { return noSuitableTarget, nil diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 1c40e0d5b58c..5b476a2e7941 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -835,11 +835,18 @@ func NewStore( ctSender: cfg.ClosedTimestampSender, } if cfg.RPCContext != nil { - s.allocator = MakeAllocator(cfg.StorePool, cfg.RPCContext.RemoteClocks.Latency) + s.allocator = MakeAllocator( + cfg.StorePool, + cfg.RPCContext.RemoteClocks.Latency, + cfg.TestingKnobs.AllocatorKnobs, + ) } else { - s.allocator = MakeAllocator(cfg.StorePool, func(string) (time.Duration, bool) { - return 0, false - }) + s.allocator = MakeAllocator( + cfg.StorePool, func(string) (time.Duration, bool) { + return 0, false + }, + cfg.TestingKnobs.AllocatorKnobs, + ) } s.replRankings = newReplicaRankings() diff --git a/pkg/kv/kvserver/store_rebalancer.go b/pkg/kv/kvserver/store_rebalancer.go index 55ab046c8d55..d28f86450a8a 100644 --- a/pkg/kv/kvserver/store_rebalancer.go +++ b/pkg/kv/kvserver/store_rebalancer.go @@ -14,14 +14,12 @@ import ( "context" "math" "math/rand" - "sort" "time" "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/contextutil" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/stop" @@ -233,7 +231,8 @@ func (sr *StoreRebalancer) rebalanceStore( if !ok { log.Fatalf(ctx, "expected the `StoreRebalancer` to be using a `qpsScorerOptions`") } - qpsMinThreshold := underfullQPSThreshold(options, allStoresList.candidateQueriesPerSecond.mean) + // We only bother rebalancing stores that are fielding more than the + // cluster-level overfull threshold of QPS. qpsMaxThreshold := overfullQPSThreshold(options, allStoresList.candidateQueriesPerSecond.mean) var localDesc *roachpb.StoreDescriptor @@ -264,14 +263,17 @@ func (sr *StoreRebalancer) rebalanceStore( hottestRanges := sr.replRankings.topQPS() for localDesc.Capacity.QueriesPerSecond > qpsMaxThreshold { replWithStats, target, considerForRebalance := sr.chooseLeaseToTransfer( - ctx, &hottestRanges, localDesc, allStoresList, storeMap, qpsMinThreshold, qpsMaxThreshold) + ctx, + &hottestRanges, + localDesc, + allStoresList, + storeMap, + ) replicasToMaybeRebalance = append(replicasToMaybeRebalance, considerForRebalance...) if replWithStats.repl == nil { break } - log.VEventf(ctx, 1, "transferring r%d (%.2f qps) to s%d to better balance load", - replWithStats.repl.RangeID, replWithStats.qps, target.StoreID) timeout := sr.rq.processTimeoutFunc(sr.st, replWithStats.repl) if err := contextutil.RunWithTimeout(ctx, "transfer lease", timeout, func(ctx context.Context) error { return sr.rq.transferLease(ctx, replWithStats.repl, target, replWithStats.qps) @@ -381,16 +383,12 @@ func (sr *StoreRebalancer) rebalanceStore( localDesc.StoreID, localDesc.Capacity.QueriesPerSecond, allStoresList.candidateQueriesPerSecond.mean, qpsMaxThreshold) } -// TODO(a-robinson): Should we take the number of leases on each store into -// account here or just continue to let that happen in allocator.go? func (sr *StoreRebalancer) chooseLeaseToTransfer( ctx context.Context, hottestRanges *[]replicaWithStats, localDesc *roachpb.StoreDescriptor, storeList StoreList, storeMap map[roachpb.StoreID]*roachpb.StoreDescriptor, - minQPS float64, - maxQPS float64, ) (replicaWithStats, roachpb.ReplicaDescriptor, []replicaWithStats) { var considerForRebalance []replicaWithStats now := sr.rq.store.Clock().NowAsClockTimestamp() @@ -410,11 +408,6 @@ func (sr *StoreRebalancer) chooseLeaseToTransfer( log.VEventf(ctx, 3, "store doesn't own the lease for r%d", replWithStats.repl.RangeID) continue } - if localDesc.Capacity.QueriesPerSecond-replWithStats.qps < minQPS { - log.VEventf(ctx, 3, "moving r%d's %.2f qps would bring s%d below the min threshold (%.2f)", - replWithStats.repl.RangeID, replWithStats.qps, localDesc.StoreID, minQPS) - continue - } // Don't bother moving leases whose QPS is below some small fraction of the // store's QPS (unless the store has extra leases to spare anyway). It's @@ -423,7 +416,7 @@ func (sr *StoreRebalancer) chooseLeaseToTransfer( const minQPSFraction = .001 if replWithStats.qps < localDesc.Capacity.QueriesPerSecond*minQPSFraction && float64(localDesc.Capacity.LeaseCount) <= storeList.candidateLeases.mean { - log.VEventf(ctx, 5, "r%d's %.2f qps is too little to matter relative to s%d's %.2f total qps", + log.VEventf(ctx, 3, "r%d's %.2f qps is too little to matter relative to s%d's %.2f total qps", replWithStats.repl.RangeID, replWithStats.qps, localDesc.StoreID, localDesc.Capacity.QueriesPerSecond) continue } @@ -436,74 +429,68 @@ func (sr *StoreRebalancer) chooseLeaseToTransfer( // Learners or non-voters aren't allowed to become leaseholders or raft // leaders, so only consider the `Voter` replicas. candidates := desc.Replicas().DeepCopy().VoterDescriptors() - sort.Slice(candidates, func(i, j int) bool { - var iQPS, jQPS float64 - if desc := storeMap[candidates[i].StoreID]; desc != nil { - iQPS = desc.Capacity.QueriesPerSecond - } - if desc := storeMap[candidates[j].StoreID]; desc != nil { - jQPS = desc.Capacity.QueriesPerSecond - } - return iQPS < jQPS - }) - - var raftStatus *raft.Status - - preferred := sr.rq.allocator.preferredLeaseholders(conf, candidates) - - // Filter both the list of preferred stores as well as the list of all - // candidate replicas to only consider live (non-suspect, non-draining) - // nodes. - const includeSuspectAndDrainingStores = false - preferred, _ = sr.rq.allocator.storePool.liveAndDeadReplicas(preferred, includeSuspectAndDrainingStores) - candidates, _ = sr.rq.allocator.storePool.liveAndDeadReplicas(candidates, includeSuspectAndDrainingStores) - - for _, candidate := range candidates { - if candidate.StoreID == localDesc.StoreID { - continue - } - - meanQPS := storeList.candidateQueriesPerSecond.mean - if sr.shouldNotMoveTo(ctx, storeMap, replWithStats, candidate.StoreID, meanQPS, minQPS, maxQPS) { - continue - } - if raftStatus == nil { - raftStatus = sr.getRaftStatusFn(replWithStats.repl) - } - if replicaIsBehind(raftStatus, candidate.ReplicaID) { - log.VEventf(ctx, 3, "%v is behind or this store isn't the raft leader for r%d; raftStatus: %v", - candidate, desc.RangeID, raftStatus) - continue - } + // Only consider replicas that are not lagging behind the leader in order to + // avoid hurting QPS in the short term. This is a stronger check than what + // `TransferLeaseTarget` performs (it only excludes replicas that are + // waiting for a snapshot). + candidates = filterBehindReplicas(ctx, sr.getRaftStatusFn(replWithStats.repl), candidates) - if len(preferred) > 0 && !storeHasReplica(candidate.StoreID, roachpb.MakeReplicaSet(preferred).ReplicationTargets()) { - log.VEventf(ctx, 3, "s%d not a preferred leaseholder for r%d; preferred: %v", - candidate.StoreID, desc.RangeID, preferred) - continue - } + candidate := sr.rq.allocator.TransferLeaseTarget( + ctx, + conf, + candidates, + replWithStats.repl, + replWithStats.repl.leaseholderStats, + true, /* forceDecisionWithoutStats */ + transferLeaseOptions{ + goal: qpsConvergence, + checkTransferLeaseSource: true, + }, + ) - filteredStoreList := storeList.excludeInvalid(conf.Constraints) - filteredStoreList = storeList.excludeInvalid(conf.VoterConstraints) - if sr.rq.allocator.followTheWorkloadPrefersLocal( + if candidate == (roachpb.ReplicaDescriptor{}) { + log.VEventf( ctx, - filteredStoreList, - *localDesc, - candidate.StoreID, - candidates, - replWithStats.repl.leaseholderStats, - ) { - log.VEventf(ctx, 3, "r%d is on s%d due to follow-the-workload; skipping", - desc.RangeID, localDesc.StoreID) - continue - } - - return replWithStats, candidate, considerForRebalance + 3, + "could not find a better lease transfer target for r%d; considering replica rebalance instead", + desc.RangeID, + ) + considerForRebalance = append(considerForRebalance, replWithStats) + continue } - // If none of the other replicas are valid lease transfer targets, consider - // this range for replica rebalancing. - considerForRebalance = append(considerForRebalance, replWithStats) + filteredStoreList := storeList.excludeInvalid(conf.Constraints) + filteredStoreList = storeList.excludeInvalid(conf.VoterConstraints) + if sr.rq.allocator.followTheWorkloadPrefersLocal( + ctx, + filteredStoreList, + *localDesc, + candidate.StoreID, + candidates, + replWithStats.repl.leaseholderStats, + ) { + log.VEventf( + ctx, 3, "r%d is on s%d due to follow-the-workload; considering replica rebalance instead", + desc.RangeID, localDesc.StoreID, + ) + considerForRebalance = append(considerForRebalance, replWithStats) + continue + } + if targetStore, ok := storeMap[candidate.StoreID]; ok { + log.VEventf( + ctx, + 1, + "transferring lease for r%d (qps=%.2f) to store s%d (qps=%.2f) from local store s%d (qps=%.2f)", + desc.RangeID, + replWithStats.qps, + targetStore.StoreID, + targetStore.Capacity.QueriesPerSecond, + localDesc.StoreID, + localDesc.Capacity.QueriesPerSecond, + ) + } + return replWithStats, candidate, considerForRebalance } } @@ -514,8 +501,6 @@ type rangeRebalanceContext struct { replWithStats replicaWithStats rangeDesc *roachpb.RangeDescriptor conf roachpb.SpanConfig - - qpsThresholdFraction float64 } func (sr *StoreRebalancer) chooseRangeToRebalance( @@ -753,68 +738,6 @@ func (sr *StoreRebalancer) getRebalanceTargetsBasedOnQPS( return finalVoterTargets, finalNonVoterTargets } -func shouldNotMoveAway( - ctx context.Context, - replWithStats replicaWithStats, - localDesc *roachpb.StoreDescriptor, - now hlc.ClockTimestamp, - minQPS float64, -) bool { - if !replWithStats.repl.OwnsValidLease(ctx, now) { - log.VEventf(ctx, 3, "store doesn't own the lease for r%d", replWithStats.repl.RangeID) - return true - } - if localDesc.Capacity.QueriesPerSecond-replWithStats.qps < minQPS { - log.VEventf(ctx, 3, "moving r%d's %.2f qps would bring s%d below the min threshold (%.2f)", - replWithStats.repl.RangeID, replWithStats.qps, localDesc.StoreID, minQPS) - return true - } - return false -} - -func (sr *StoreRebalancer) shouldNotMoveTo( - ctx context.Context, - storeMap map[roachpb.StoreID]*roachpb.StoreDescriptor, - replWithStats replicaWithStats, - candidateStoreID roachpb.StoreID, - meanQPS float64, - minQPS float64, - maxQPS float64, -) bool { - candidateStore, ok := storeMap[candidateStoreID] - if !ok { - log.VEventf(ctx, 3, "missing store descriptor for s%d", candidateStoreID) - return true - } - - newCandidateQPS := candidateStore.Capacity.QueriesPerSecond + replWithStats.qps - if candidateStore.Capacity.QueriesPerSecond < minQPS { - if newCandidateQPS > maxQPS { - log.VEventf(ctx, 3, - "r%d's %.2f qps would push s%d over the max threshold (%.2f) with %.2f qps afterwards", - replWithStats.repl.RangeID, replWithStats.qps, candidateStoreID, maxQPS, newCandidateQPS) - return true - } - } else if newCandidateQPS > meanQPS { - log.VEventf(ctx, 3, - "r%d's %.2f qps would push s%d over the mean (%.2f) with %.2f qps afterwards", - replWithStats.repl.RangeID, replWithStats.qps, candidateStoreID, meanQPS, newCandidateQPS) - return true - } - - // If the target store is on a separate node, we will also care - // about node liveness. - targetNodeID := candidateStore.Node.NodeID - if targetNodeID != sr.rq.store.Ident.NodeID { - if !sr.rq.allocator.storePool.isStoreReadyForRoutineReplicaTransfer(ctx, candidateStore.StoreID) { - log.VEventf(ctx, 3, - "refusing to transfer replica to n%d/s%d", targetNodeID, candidateStore.StoreID) - return true - } - } - return false -} - func storeListToMap(sl StoreList) map[roachpb.StoreID]*roachpb.StoreDescriptor { storeMap := make(map[roachpb.StoreID]*roachpb.StoreDescriptor) for i := range sl.stores { diff --git a/pkg/kv/kvserver/store_rebalancer_test.go b/pkg/kv/kvserver/store_rebalancer_test.go index 8bb5c91f87df..d2dd5facfdec 100644 --- a/pkg/kv/kvserver/store_rebalancer_test.go +++ b/pkg/kv/kvserver/store_rebalancer_test.go @@ -30,8 +30,6 @@ import ( "go.etcd.io/etcd/raft/v3/tracker" ) -const defaultQPSRebalanceThreshold = 0.25 - var ( // multiRegionStores specifies a set of stores across 3 regions. These stores // are arranged in descending order of the QPS they are receiving. Store 1 is @@ -271,7 +269,10 @@ func loadRanges(rr *replicaRankings, s *Store, ranges []testRange) { // TODO(a-robinson): The below three lines won't be needed once the old // rangeInfo code is ripped out of the allocator. repl.mu.state.Stats = &enginepb.MVCCStats{} + repl.leaseholderStats = newReplicaStats(s.Clock(), nil) + repl.leaseholderStats.setAvgQPSForTesting(r.qps) + repl.writeStats = newReplicaStats(s.Clock(), nil) acc.addReplica(replicaWithStats{ repl: repl, @@ -289,15 +290,18 @@ func TestChooseLeaseToTransfer(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) - stopper, g, _, a, _ := createTestAllocator(10, false /* deterministic */) + stopper, g, _, a, _ := createTestAllocatorWithKnobs( + 10, false /* deterministic */, &AllocatorTestingKnobs{ + // Let the allocator pick lease transfer targets that are replicas in need + // of snapshots, in order to avoid mocking out a fake raft group for the + // `replicaMayNeedSnapshot` checks inside `TransferLeaseTarget`. + AllowLeaseTransfersToReplicasNeedingSnapshots: true, + }, + ) defer stopper.Stop(context.Background()) gossiputil.NewStoreGossiper(g).GossipStores(noLocalityStores, t) storeList, _, _ := a.storePool.getStoreList(storeFilterThrottled) storeMap := storeListToMap(storeList) - - const minQPS = 800 - const maxQPS = 1200 - localDesc := *noLocalityStores[0] cfg := TestStoreConfig(nil) cfg.Gossip = g @@ -331,38 +335,143 @@ func TestChooseLeaseToTransfer(t *testing.T) { qps float64 expectTarget roachpb.StoreID }{ - {[]roachpb.StoreID{1}, 100, 0}, - {[]roachpb.StoreID{1, 2}, 100, 0}, - {[]roachpb.StoreID{1, 3}, 100, 0}, - {[]roachpb.StoreID{1, 4}, 100, 4}, - {[]roachpb.StoreID{1, 5}, 100, 5}, - {[]roachpb.StoreID{5, 1}, 100, 0}, - {[]roachpb.StoreID{1, 2}, 200, 0}, - {[]roachpb.StoreID{1, 3}, 200, 0}, - {[]roachpb.StoreID{1, 4}, 200, 0}, - {[]roachpb.StoreID{1, 5}, 200, 5}, - {[]roachpb.StoreID{1, 2}, 500, 0}, - {[]roachpb.StoreID{1, 3}, 500, 0}, - {[]roachpb.StoreID{1, 4}, 500, 0}, - {[]roachpb.StoreID{1, 5}, 500, 5}, - {[]roachpb.StoreID{1, 5}, 600, 5}, - {[]roachpb.StoreID{1, 5}, 700, 5}, - {[]roachpb.StoreID{1, 5}, 800, 0}, - {[]roachpb.StoreID{1, 4}, 1.5, 4}, - {[]roachpb.StoreID{1, 5}, 1.5, 5}, - {[]roachpb.StoreID{1, 4}, 1.49, 0}, - {[]roachpb.StoreID{1, 5}, 1.49, 0}, + { + storeIDs: []roachpb.StoreID{1}, + qps: 100, + expectTarget: 0, + }, + { + storeIDs: []roachpb.StoreID{1, 2}, + qps: 100, + expectTarget: 2, + }, + { + storeIDs: []roachpb.StoreID{1, 3}, + qps: 100, + expectTarget: 3, + }, + { + storeIDs: []roachpb.StoreID{1, 4}, + qps: 100, + expectTarget: 4, + }, + { + storeIDs: []roachpb.StoreID{1, 5}, + qps: 100, + expectTarget: 5, + }, + { + storeIDs: []roachpb.StoreID{5, 1}, + qps: 100, + expectTarget: 0, + }, + { + storeIDs: []roachpb.StoreID{1, 2}, + qps: 200, + expectTarget: 2, + }, + { + storeIDs: []roachpb.StoreID{1, 3}, + qps: 200, + expectTarget: 3, + }, + { + storeIDs: []roachpb.StoreID{1, 4}, + qps: 200, + expectTarget: 4, + }, + { + storeIDs: []roachpb.StoreID{1, 5}, + qps: 200, + expectTarget: 5, + }, + { + storeIDs: []roachpb.StoreID{1, 2}, + qps: 500, + expectTarget: 0, + }, + { + storeIDs: []roachpb.StoreID{1, 3}, + qps: 500, + expectTarget: 0, + }, + { + storeIDs: []roachpb.StoreID{1, 4}, + qps: 500, + expectTarget: 4, + }, + { + storeIDs: []roachpb.StoreID{1, 5}, + qps: 500, + expectTarget: 5, + }, + { + storeIDs: []roachpb.StoreID{1, 5}, + qps: 600, + expectTarget: 5, + }, + { + storeIDs: []roachpb.StoreID{1, 5}, + qps: 700, + expectTarget: 5, + }, + { + storeIDs: []roachpb.StoreID{1, 5}, + qps: 800, + expectTarget: 5, + }, + { + storeIDs: []roachpb.StoreID{1, 4, 5}, + qps: 800, + expectTarget: 5, + }, + { + storeIDs: []roachpb.StoreID{1, 3, 4, 5}, + qps: 800, + expectTarget: 5, + }, + { + storeIDs: []roachpb.StoreID{1, 4}, + qps: 1.5, + expectTarget: 4, + }, + { + storeIDs: []roachpb.StoreID{1, 5}, + qps: 1.5, + expectTarget: 5, + }, + { + storeIDs: []roachpb.StoreID{1, 4}, + qps: 1.49, + expectTarget: 0, + }, + { + storeIDs: []roachpb.StoreID{1, 5}, + qps: 1.49, + expectTarget: 0, + }, + { + storeIDs: []roachpb.StoreID{1, 2, 3, 4}, + qps: 1500, + expectTarget: 0, + }, + { + storeIDs: []roachpb.StoreID{1, 2, 3, 4, 5}, + qps: 1500, + expectTarget: 0, + }, } for _, tc := range testCases { - loadRanges(rr, s, []testRange{{voters: tc.storeIDs, qps: tc.qps}}) - hottestRanges := rr.topQPS() - _, target, _ := sr.chooseLeaseToTransfer( - ctx, &hottestRanges, &localDesc, storeList, storeMap, minQPS, maxQPS) - if target.StoreID != tc.expectTarget { - t.Errorf("got target store %d for range with replicas %v and %f qps; want %d", - target.StoreID, tc.storeIDs, tc.qps, tc.expectTarget) - } + t.Run("", func(t *testing.T) { + loadRanges(rr, s, []testRange{{voters: tc.storeIDs, qps: tc.qps}}) + hottestRanges := rr.topQPS() + _, target, _ := sr.chooseLeaseToTransfer(ctx, &hottestRanges, &localDesc, storeList, storeMap) + if target.StoreID != tc.expectTarget { + t.Errorf("got target store %d for range with replicas %v and %f qps; want %d", + target.StoreID, tc.storeIDs, tc.qps, tc.expectTarget) + } + }) } } @@ -777,15 +886,16 @@ func TestNoLeaseTransferToBehindReplicas(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) - stopper, g, _, a, _ := createTestAllocator(10, false /* deterministic */) + stopper, g, _, a, _ := createTestAllocatorWithKnobs( + 10, + false, /* deterministic */ + &AllocatorTestingKnobs{AllowLeaseTransfersToReplicasNeedingSnapshots: true}, + ) defer stopper.Stop(context.Background()) gossiputil.NewStoreGossiper(g).GossipStores(noLocalityStores, t) storeList, _, _ := a.storePool.getStoreList(storeFilterThrottled) storeMap := storeListToMap(storeList) - const minQPS = 800 - const maxQPS = 1200 - localDesc := *noLocalityStores[0] cfg := TestStoreConfig(nil) cfg.Gossip = g @@ -823,8 +933,7 @@ func TestNoLeaseTransferToBehindReplicas(t *testing.T) { return status } - _, target, _ := sr.chooseLeaseToTransfer( - ctx, &hottestRanges, &localDesc, storeList, storeMap, minQPS, maxQPS) + _, target, _ := sr.chooseLeaseToTransfer(ctx, &hottestRanges, &localDesc, storeList, storeMap) expectTarget := roachpb.StoreID(4) if target.StoreID != expectTarget { t.Errorf("got target store s%d for range with RaftStatus %v; want s%d", @@ -843,7 +952,7 @@ func TestNoLeaseTransferToBehindReplicas(t *testing.T) { &hottestRanges, &localDesc, storeList, - qpsScorerOptions{deterministic: true, qpsRebalanceThreshold: 0.25}, + qpsScorerOptions{deterministic: true, qpsRebalanceThreshold: 0.05}, ) expectTargets := []roachpb.ReplicationTarget{ {NodeID: 4, StoreID: 4}, {NodeID: 3, StoreID: 3}, {NodeID: 5, StoreID: 5}, diff --git a/pkg/kv/kvserver/testing_knobs.go b/pkg/kv/kvserver/testing_knobs.go index 73443190424e..7fa2b71e6de3 100644 --- a/pkg/kv/kvserver/testing_knobs.go +++ b/pkg/kv/kvserver/testing_knobs.go @@ -37,6 +37,7 @@ type StoreTestingKnobs struct { ConsistencyTestingKnobs ConsistencyTestingKnobs TenantRateKnobs tenantrate.TestingKnobs StorageKnobs storage.TestingKnobs + AllocatorKnobs *AllocatorTestingKnobs // TestingRequestFilter is called before evaluating each request on a // replica. The filter is run before the request acquires latches, so @@ -382,6 +383,14 @@ var _ base.ModuleTestingKnobs = NodeLivenessTestingKnobs{} // ModuleTestingKnobs implements the base.ModuleTestingKnobs interface. func (NodeLivenessTestingKnobs) ModuleTestingKnobs() {} +// AllocatorTestingKnobs allows tests to override the behavior of `Allocator`. +type AllocatorTestingKnobs struct { + // AllowLeaseTransfersToReplicasNeedingSnapshots permits lease transfer + // targets produced by the Allocator to include replicas that may be waiting + // for snapshots. + AllowLeaseTransfersToReplicasNeedingSnapshots bool +} + // PinnedLeasesKnob is a testing know for controlling what store can acquire a // lease for specific ranges. type PinnedLeasesKnob struct {