From 1f38a116213ca93bdefe5320cb7ec13f7a79cb93 Mon Sep 17 00:00:00 2001 From: Aayush Shah Date: Tue, 11 Jan 2022 21:58:22 -0500 Subject: [PATCH] kvserver: rebalance ranges to minimize QPS delta among stores This commit fixes the regression(s) introduced by https://github.com/cockroachdb/cockroach/pull/65379 where we observed replica thrashing in various workloads (#70396 and #71244). The following is a description of the differences between the QPS based rebalancing scheme used in the previous implementation of the store rebalancer (release-21.2 and before). ** lease rebalancing ** *** release 21.2 and before *** QPS based lease rebalancing in CRDB 21.2 considers the overall cluster level average QPS and computes underfull and overfull thresholds based off of this average. For each range that the local store has a lease for, the store rebalancer goroutine checks whether transferring said range's lease away will bring the local store's QPS below the underfull threshold. If so, it ignores the range and moves on to the next one. Otherwise, it iterates through the stores of all the non-leaseholder voting replicas (in ascending order of their QPS) and checks whether it would be reasonable to transfer the lease away to such a store. It ensures that the receiving store would not become overfull after the lease transfer. It checks that the receiving store doesn't have a replica that's lagging behind the current leaseholder. It checks that the receiving store is not in violation of lease preferences. Finally, it ensures that the lease is not on the local store because of access locality considerations (i.e. because of follow-the-workload). All of this was bespoke logic that lived in the store rebalancer (using none of the Allocator's machinery). *** master and this commit *** In #65379, we moved this decision making into the Allocator by adding a new mode in `Allocator.TransferLeaseTarget` that tries to determine whether transferring the lease to another voting replica would reduce the qps delta between the hottest and the coldest stores in the replica set. This commit adds some padding to this logic by ensuring that the qps difference between the store relinquishing the lease and the store receiving the lease is at least 200qps. Furthermore, it ensures that the store receiving the lease won't become significantly hotter than the current leaseholder. ** replica rebalancing ** *** release 21.2 and before *** QPS replica rebalancing in CRDB <=21.2 works similarly to the lease rebalancing logic. We first compute a cluster level QPS average, overfull and underfull thresholds. Based on these thresholds we try to move replicas away from overfull stores and onto stores that are underfull, all while ensuring that the receiving stores would not become overfull after the rebalance. A critical assumption that the store rebalancer made (and still does, in the approach implemented by this commit) is that follower replicas serve the same traffic as the leaseholder. *** master and this commit *** The approach implemented by #65379 and refined by this commit tries to leverage machinery in the Allocator that makes rebalancing decisions that converge load based statistics per equivalence class. Previously, this machinery was only used for range count based replica rebalancing (performed by the `replicateQueue`) but not for qps-based rebalancing. This commit implements a similar approach to what we do now for lease rebalancing, which is to determine whether a rebalance action would reduce the qps delta between the hottest and the coldest store in the equivalence class. This commit adds some safeguards around this logic by ensuring that the store relinquishing the replica and the store receiving it differ by at least 200 qps. Furthermore, it ensures that the replica rebalance would not significantly switch the relative dispositions of the two stores. An important thing to note with the 21.2 implementation of the store rebalancer is that it was making all of its decisions based on cluster-level QPS averages. This behaves poorly in heterogenously sized / loaded clusters where some localities are designed to receive more traffic than others. In such clusters, heavily loaded localities can always be considered "overfull". This usually means that all stores in such localities would be above the "overfull" threshold in the cluster. The logic described above would effectively not do anything since there are no underfull stores to move replicas to. Release note (performance improvement): A set of bugs that rendered QPS-based lease and replica rebalancing in CRDB 21.2 and prior ineffective under heterogenously loaded cluster localities has been fixed. Additionally a limitation which prevent CRDB from effectively alleviating extreme QPS hotspots from nodes has also been fixed. --- pkg/cmd/roachtest/tests/follower_reads.go | 1 - pkg/kv/kvserver/allocator.go | 122 ++--- pkg/kv/kvserver/allocator_scorer.go | 505 ++++++++++++++++----- pkg/kv/kvserver/allocator_scorer_test.go | 23 +- pkg/kv/kvserver/allocator_test.go | 522 +++++++++++++++++----- pkg/kv/kvserver/store_rebalancer.go | 84 +++- pkg/kv/kvserver/store_rebalancer_test.go | 255 +++++++++-- 7 files changed, 1154 insertions(+), 358 deletions(-) diff --git a/pkg/cmd/roachtest/tests/follower_reads.go b/pkg/cmd/roachtest/tests/follower_reads.go index f0b93f64665b..f7e3700a0999 100644 --- a/pkg/cmd/roachtest/tests/follower_reads.go +++ b/pkg/cmd/roachtest/tests/follower_reads.go @@ -50,7 +50,6 @@ func registerFollowerReads(r registry.Registry) { name = name + "/insufficient-quorum" } r.Add(registry.TestSpec{ - Skip: "https://github.com/cockroachdb/cockroach/issues/69817", Name: name, Owner: registry.OwnerKV, Cluster: r.MakeClusterSpec( diff --git a/pkg/kv/kvserver/allocator.go b/pkg/kv/kvserver/allocator.go index f16e63c0d96c..dc23554226bd 100644 --- a/pkg/kv/kvserver/allocator.go +++ b/pkg/kv/kvserver/allocator.go @@ -1425,97 +1425,99 @@ func (a *Allocator) TransferLeaseTarget( return candidates[a.randGen.Intn(len(candidates))] case qpsConvergence: + leaseReplQPS, _ := stats.avgQPS() + candidates := make([]roachpb.StoreID, 0, len(existing)-1) + for _, repl := range existing { + if repl.StoreID != leaseRepl.StoreID() { + candidates = append(candidates, repl.StoreID) + } + } + // 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. + bestStore, noRebalanceReason := bestStoreToMinimizeQPSDelta( + leaseReplQPS, + qpsRebalanceThreshold.Get(&a.storePool.st.SV), + minQPSDifferenceForTransfers.Get(&a.storePool.st.SV), + leaseRepl.StoreID(), + candidates, + storeDescMap, + ) - // 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 - } - - leaseholderStoreQPS, ok := storeQPSMap[leaseRepl.StoreID()] - if !ok { + switch noRebalanceReason { + case noBetterCandidate: + log.VEventf(ctx, 5, "r%d: could not find a better target for lease", leaseRepl.GetRangeID()) + return roachpb.ReplicaDescriptor{} + case existingNotOverfull: log.VEventf( - ctx, 3, "cannot find store descriptor for leaseholder s%d;"+ - " skipping this range", leaseRepl.StoreID(), + ctx, 5, "r%d: existing leaseholder s%d is not overfull", + leaseRepl.GetRangeID(), leaseRepl.StoreID(), ) return roachpb.ReplicaDescriptor{} - } - - leaseholderReplQPS, _ := stats.avgQPS() - currentDelta := getQPSDelta(storeQPSMap, existing) - bestOption := getCandidateWithMinQPS(storeQPSMap, existing) - if bestOption != (roachpb.ReplicaDescriptor{}) && bestOption.StoreID != leaseRepl.StoreID() && - // 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) + case deltaNotSignificant: 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, + ctx, 5, + "r%d: delta between s%d and the coldest follower (ignoring r%d's lease) is not large enough", + leaseRepl.GetRangeID(), leaseRepl.StoreID(), leaseRepl.GetRangeID(), ) - return bestOption + return roachpb.ReplicaDescriptor{} + case significantlySwitchesRelativeDisposition: + log.VEventf(ctx, 5, + "r%d: lease transfer away from s%d would make it hotter than the coldest follower", + leaseRepl.GetRangeID(), leaseRepl.StoreID()) + return roachpb.ReplicaDescriptor{} + case missingStatsForExistingStore: + log.VEventf( + ctx, 5, "r%d: missing stats for leaseholder s%d", + leaseRepl.GetRangeID(), leaseRepl.StoreID(), + ) + return roachpb.ReplicaDescriptor{} + case none: + default: + log.Fatalf(ctx, "unknown declineReason: %v", noRebalanceReason) } - return roachpb.ReplicaDescriptor{} + + for _, repl := range existing { + if repl.StoreID == bestStore { + return repl + } + } + panic("unreachable") default: log.Fatalf(ctx, "unexpected lease transfer goal %d", g) } panic("unreachable") } -// getCandidateWithMinQPS returns the `ReplicaDescriptor` that belongs to the -// store serving the lowest QPS among all the `existing` replicas. +// getCandidateWithMinQPS returns the StoreID that belongs to the store serving +// the lowest QPS among all the `candidates` stores. func getCandidateWithMinQPS( - storeQPSMap map[roachpb.StoreID]float64, existing []roachpb.ReplicaDescriptor, -) roachpb.ReplicaDescriptor { + storeQPSMap map[roachpb.StoreID]float64, candidates []roachpb.StoreID, +) (bestCandidate roachpb.StoreID) { minCandidateQPS := math.MaxFloat64 - var candidateWithMin roachpb.ReplicaDescriptor - for _, repl := range existing { - candidateQPS, ok := storeQPSMap[repl.StoreID] + for _, store := range candidates { + candidateQPS, ok := storeQPSMap[store] if !ok { continue } if minCandidateQPS > candidateQPS { minCandidateQPS = candidateQPS - candidateWithMin = repl + bestCandidate = store } } - return candidateWithMin + return bestCandidate } // 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 { +// and the store serving the lowest QPS, among the set of stores in the +// `domain`. +func getQPSDelta(storeQPSMap map[roachpb.StoreID]float64, domain []roachpb.StoreID) float64 { maxCandidateQPS := float64(0) minCandidateQPS := math.MaxFloat64 - for _, repl := range existing { - candidateQPS, ok := storeQPSMap[repl.StoreID] + for _, cand := range domain { + candidateQPS, ok := storeQPSMap[cand] if !ok { continue } diff --git a/pkg/kv/kvserver/allocator_scorer.go b/pkg/kv/kvserver/allocator_scorer.go index ce5b79cfc856..f5fe211f2d7f 100644 --- a/pkg/kv/kvserver/allocator_scorer.go +++ b/pkg/kv/kvserver/allocator_scorer.go @@ -86,6 +86,9 @@ var rangeRebalanceThreshold = func() *settings.FloatSetting { return s }() +// CockroachDB's has two heuristics that trigger replica rebalancing: range +// count convergence and QPS convergence. scorerOptions defines the interface +// that both of these heuristics must implement. 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 @@ -93,30 +96,41 @@ type scorerOptions interface { 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. + // list based on either range count or QPS. + // + // CRDB's rebalancing logic first checks whether any existing replica is in + // violation of constraints or is on stores that have an almost-full disk. If + // not, it then uses `shouldRebalanceBasedOnThresholds()` to determine whether + // the `eqClass`'s current stats are divergent enough to justify rebalancing + // replicas. shouldRebalanceBasedOnThresholds( - ctx context.Context, store roachpb.StoreDescriptor, sl StoreList, + ctx context.Context, eqClass equivalenceClass, ) bool // balanceScore returns a discrete score (`balanceStatus`) based on whether // the store represented by `sc` classifies as underfull, aroundTheMean, 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 + // referred to by `eqClass.existing` based on whether moving a replica away + // from this store would converge its stats towards the mean (relative to the + // equivalence class `eqClass`). If moving the replica away from the existing + // store 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 + rebalanceFromConvergesScore(eqClass equivalenceClass) 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 + // referred to by `candidate` will converge its stats towards the mean + // (relative to the equivalence class `eqClass`). This makes it more likely + // for us to pick this store as the rebalance target. + rebalanceToConvergesScore(eqClass equivalenceClass, candidate roachpb.StoreDescriptor) int + // removalConvergesScore is similar to `rebalanceFromConvergesScore` (both + // deal with computing a converges score for existing stores that might + // relinquish a replica). removalConvergesScore assigns a negative convergence + // score to the existing store (or multiple replicas, if there are multiple + // with the same QPS) that would converge the range's existing stores' QPS the + // most. + removalConvergesScore(removalCandStoreList StoreList, existing roachpb.StoreDescriptor) int } // rangeCountScorerOptions is used by the replicateQueue to tell the Allocator's @@ -133,11 +147,14 @@ func (o rangeCountScorerOptions) deterministicForTesting() bool { } func (o rangeCountScorerOptions) shouldRebalanceBasedOnThresholds( - ctx context.Context, store roachpb.StoreDescriptor, sl StoreList, + ctx context.Context, eqClass equivalenceClass, ) bool { + store := eqClass.existing + sl := eqClass.candidateSL if len(sl.stores) == 0 { return false } + 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). @@ -182,27 +199,43 @@ func (o rangeCountScorerOptions) balanceScore( } // 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) { +// `sd` will _not_ converge its range count towards the mean of the candidate +// stores in the equivalence class `eqClass`. 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(eqClass equivalenceClass) int { + if !rebalanceConvergesRangeCountOnMean( + eqClass.candidateSL, eqClass.existing.Capacity, eqClass.existing.Capacity.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`. +// rebalanceToConvergesScore returns 1 if rebalancing a replica to `sd` will +// converge its range count towards the mean of the candidate stores inside +// `eqClass`. func (o rangeCountScorerOptions) rebalanceToConvergesScore( - sl StoreList, sc roachpb.StoreCapacity, + eqClass equivalenceClass, candidate roachpb.StoreDescriptor, ) int { - if rebalanceConvergesRangeCountOnMean(sl, sc, sc.RangeCount+1) { + if rebalanceConvergesRangeCountOnMean(eqClass.candidateSL, candidate.Capacity, candidate.Capacity.RangeCount+1) { + return 1 + } + return 0 +} + +// removalConvergesScore assigns a low convergesScore to the existing store if +// removing it would converge the range counts of the existing stores towards +// the mean (this low score makes it more likely to be picked for removal). +// Otherwise, a high convergesScore is assigned (which would make this store +// less likely to be picked for removal). +func (o rangeCountScorerOptions) removalConvergesScore( + removalCandStoreList StoreList, existing roachpb.StoreDescriptor, +) int { + if !rebalanceConvergesRangeCountOnMean( + removalCandStoreList, existing.Capacity, existing.Capacity.RangeCount-1, + ) { return 1 } return 0 @@ -213,63 +246,73 @@ func (o rangeCountScorerOptions) rebalanceToConvergesScore( // 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 + deterministic bool + qpsRebalanceThreshold, minRequiredQPSDiff float64 + + // QPS-based rebalancing assumes that: + // 1. Every replica of a range receives the same level of traffic + // 2. Transferring this replica to another store would also transfer all of + // this replica's load onto that receiving store. + // + // See comment inside `StoreRebalancer.chooseRangeToRebalance()` for why these + // assumptions are justified. + + // qpsPerReplica states the level of traffic being served by each replica in a + // range. + qpsPerReplica float64 } func (o qpsScorerOptions) deterministicForTesting() bool { return o.deterministic } +// shouldRebalanceBasedOnThresholds tries to determine if, within the given +// equivalenceClass `eqClass`, rebalancing a replica from one of the existing +// stores to one of the candidate stores will lead to QPS convergence among the +// stores in the equivalence class. func (o qpsScorerOptions) shouldRebalanceBasedOnThresholds( - ctx context.Context, store roachpb.StoreDescriptor, sl StoreList, + ctx context.Context, eqClass equivalenceClass, ) bool { - if len(sl.stores) == 0 { + if len(eqClass.candidateSL.stores) == 0 { return false } - // 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 { + + _, noRebalanceReason := o.getRebalanceTargetToMinimizeDelta(eqClass) + switch noRebalanceReason { + case noBetterCandidate: 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, + ctx, 4, "could not find a better candidate to replace s%d", eqClass.existing.StoreID, ) - 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 - } - } + return false + case existingNotOverfull: + log.VEventf(ctx, 4, "existing store s%d is not overfull", eqClass.existing.StoreID) + return false + case deltaNotSignificant: + log.VEventf( + ctx, 4, + "delta between s%d and the next best candidate is not significant enough", + eqClass.existing.StoreID, + ) + return false + case significantlySwitchesRelativeDisposition: + log.VEventf( + ctx, 4, + "rebalancing from s%[1]d to the next best candidate could make it significantly hotter than s%[1]d", + eqClass.existing.StoreID, + ) + return false + case missingStatsForExistingStore: + log.VEventf(ctx, 4, "missing QPS stats for s%d", eqClass.existing.StoreID) + return false + case none: + default: } - // If we reached this point, we're happy with the range where it is. - return false + + return true } func (o qpsScorerOptions) balanceScore(sl StoreList, sc roachpb.StoreCapacity) balanceStatus { - maxQPS := overfullQPSThreshold(o, sl.candidateQueriesPerSecond.mean) + maxQPS := overfullQPSThreshold(o.qpsRebalanceThreshold, sl.candidateQueriesPerSecond.mean) minQPS := underfullQPSThreshold(o, sl.candidateQueriesPerSecond.mean) curQPS := sc.QueriesPerSecond if curQPS < minQPS { @@ -280,21 +323,50 @@ func (o qpsScorerOptions) balanceScore(sl StoreList, sc roachpb.StoreCapacity) b return aroundTheMean } -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. +// rebalanceFromConvergesScore returns a score of -1 if the existing store in +// eqClass needs to be rebalanced away in order to minimize the QPS delta +// between the stores in the equivalence class `eqClass`. +func (o qpsScorerOptions) rebalanceFromConvergesScore(eqClass equivalenceClass) int { + _, noRebalanceReason := o.getRebalanceTargetToMinimizeDelta(eqClass) + // If there are any rebalance opportunities that minimize the QPS delta in + // this equivalence class, we return a score of -1 to make the existing store + // more likely to be picked for removal. + if noRebalanceReason == none { + return -1 + } 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. +// rebalanceToConvergesScore returns a score of 1 if `candidate` needs to be +// rebalanced to in order to minimize the QPS delta between the stores in the +// equivalence class `eqClass` +func (o qpsScorerOptions) rebalanceToConvergesScore( + eqClass equivalenceClass, candidate roachpb.StoreDescriptor, +) int { + bestTarget, noRebalanceReason := o.getRebalanceTargetToMinimizeDelta(eqClass) + if noRebalanceReason == none && bestTarget == candidate.StoreID { + return 1 + } + return 0 +} + +// removalConvergesScore returns a score of -1 `existing` is the hottest store +// (based on QPS) among the stores inside `removalCandidateStores`. +func (o qpsScorerOptions) removalConvergesScore( + removalCandStoreList StoreList, existing roachpb.StoreDescriptor, +) int { + maxQPS := float64(-1) + for _, store := range removalCandStoreList.stores { + if store.Capacity.QueriesPerSecond > maxQPS { + maxQPS = store.Capacity.QueriesPerSecond + } + } + // NB: Note that if there are multiple stores inside `removalCandStoreList` + // with the same (or similar) maxQPS, we will return a removalConvergesScore + // of -1 for all of them. + if scoresAlmostEqual(maxQPS, existing.Capacity.QueriesPerSecond) { + return -1 + } return 0 } @@ -736,8 +808,8 @@ 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 = options.rebalanceFromConvergesScore( - removalCandidateStoreList, candidates[i].store.Capacity, + candidates[i].convergesScore = options.removalConvergesScore( + removalCandidateStoreList, candidates[i].store, ) candidates[i].balanceScore = options.balanceScore( removalCandidateStoreList, candidates[i].store.Capacity, @@ -765,6 +837,171 @@ type rebalanceOptions struct { candidates candidateList } +// equivalenceClass captures the set of "equivalent" replacement candidates +// for each existing replica. "equivalent" here means the candidates that are +// just as diverse as the existing replica, conform to zone config constraints +// on the range and don't have a full disk. +// Following are a few examples: +// 1. Consider a 3 region cluster with regions A, B and C. Assume there is a +// range that has 1 replica in each of those regions. For each existing +// replica, its equivalence class would contain all the other stores in its +// own region. +// 2. Consider a cluster with 10 racks, each with 2 stores (on 2 different +// nodes). Assume that racks 1, 2 and 3 each have a replica for a range. For +// the existing replica in rack 1, its equivalence class would contain its +// neighboring store in rack 1 and all stores in racks 4...10. +type equivalenceClass struct { + existing roachpb.StoreDescriptor + // `candidateSl` is just the `StoreList` representation of `candidates` + // (maintained separately to avoid converting the latter into the former for + // all the `scorerOptions` methods). + candidateSL StoreList + candidates candidateList +} + +const ( + // We generally discard replica and lease rebalancing opportunities that would + // invert the relative dispositions of the sending and receiving stores. In + // other words, we generally disallow rebalances were qps(s1) < qps(s2) before + // the rebalance but qps(s1) > qps(s2) after the rebalance. However, an + // exception to this is that if the inversion is insignificant (less than + // minQPSTransferOvershoot qps). + maxQPSTransferOvershoot = 500 +) + +type declineReason int + +const ( + none declineReason = iota + noBetterCandidate + existingNotOverfull + deltaNotSignificant + significantlySwitchesRelativeDisposition + missingStatsForExistingStore +) + +// bestStoreToMinimizeQPSDelta computes a rebalance (or lease transfer) target +// for the existing store such that executing the rebalance (or lease transfer) +// decision would minimize the QPS range between the existing store and the +// coldest store in the equivalence class. +func bestStoreToMinimizeQPSDelta( + qps, rebalanceThreshold, minRequiredQPSDiff float64, + existing roachpb.StoreID, + candidates []roachpb.StoreID, + storeDescMap map[roachpb.StoreID]*roachpb.StoreDescriptor, +) (bestCandidate roachpb.StoreID, reason declineReason) { + storeQPSMap := make(map[roachpb.StoreID]float64, len(candidates)+1) + for _, store := range candidates { + if desc, ok := storeDescMap[store]; ok { + storeQPSMap[store] = desc.Capacity.QueriesPerSecond + } + } + if desc, ok := storeDescMap[existing]; !ok { + return 0, missingStatsForExistingStore + } else { + storeQPSMap[existing] = desc.Capacity.QueriesPerSecond + } + + // domain defines the domain over which this function tries to minimize the + // QPS delta. + domain := append(candidates, existing) + storeDescs := make([]roachpb.StoreDescriptor, 0, len(domain)) + for _, desc := range storeDescMap { + storeDescs = append(storeDescs, *desc) + } + domainStoreList := makeStoreList(storeDescs) + + bestCandidate = getCandidateWithMinQPS(storeQPSMap, candidates) + if bestCandidate == 0 { + return 0, noBetterCandidate + } + + bestCandQPS := storeQPSMap[bestCandidate] + existingQPS := storeQPSMap[existing] + if bestCandQPS > existingQPS { + return 0, noBetterCandidate + } + + // NB: The store's QPS and the replica's QPS aren't captured at the same + // time, so they may be mutually inconsistent. Thus, it is possible for + // the store's QPS captured here to be lower than the replica's QPS. So we + // defensively use the `math.Max` here. + existingQPSIgnoringRepl := math.Max(existingQPS-qps, 0) + + // Only proceed if the QPS difference between `existing` and + // `bestCandidate` (not accounting for the replica under consideration) is + // higher than `minQPSDifferenceForTransfers`. + diffIgnoringRepl := existingQPSIgnoringRepl - bestCandQPS + if diffIgnoringRepl < minRequiredQPSDiff { + return 0, deltaNotSignificant + } + + // Only proceed with rebalancing iff `existingStore` is overfull relative to + // the equivalence class. + mean := domainStoreList.candidateQueriesPerSecond.mean + overfullThreshold := overfullQPSThreshold(rebalanceThreshold, mean) + if existingQPS < overfullThreshold { + return 0, existingNotOverfull + } + + // Simulate the coldest candidate's QPS after it receives a replica for the + // range. + storeQPSMap[bestCandidate] += qps + // Simulate the hottest existing store's QPS after it sheds the replica away. + storeQPSMap[existing] = existingQPSIgnoringRepl + bestCandQPSWithRepl := storeQPSMap[bestCandidate] + + if existingQPSIgnoringRepl+maxQPSTransferOvershoot < bestCandQPSWithRepl { + return 0, significantlySwitchesRelativeDisposition + } + + // NB: We proceed with a rebalance even if `currentQPSDelta` is exactly equal + // to `newQPSDelta`. Consider the following example: + // perReplicaQPS: 10qps + // existingQPS: 100qps + // candidates: [100qps, 0qps, 0qps] + // + // In such (perhaps unrealistic) scenarios, rebalancing from the existing + // store to the coldest store is not going to reduce the delta between all + // these stores, but it is still a desirable action to take. + + currentQPSDelta := getQPSDelta(storeQPSMap, domain) + newQPSDelta := getQPSDelta(storeQPSMap, domain) + if currentQPSDelta < newQPSDelta { + panic( + fmt.Sprintf( + "programming error: projected QPS delta higher than current delta;"+ + " existing: %0.2f qps, coldest candidate: %0.2f qps, replica/lease: %0.2f qps", + existingQPS, bestCandQPS, qps, + ), + ) + } + + return bestCandidate, none +} + +// getRebalanceTargetToMinimizeDelta returns the best store (from the set of +// candidates in the equivalence class) such that rebalancing to this store +// would minimize the delta between the existing store and the coldest store in +// the equivalence class. +func (o *qpsScorerOptions) getRebalanceTargetToMinimizeDelta( + eqClass equivalenceClass, +) (bestStore roachpb.StoreID, noRebalanceReason declineReason) { + domainStoreList := makeStoreList(append(eqClass.candidateSL.stores, eqClass.existing)) + candidates := make([]roachpb.StoreID, 0, len(eqClass.candidateSL.stores)) + for _, store := range eqClass.candidateSL.stores { + candidates = append(candidates, store.StoreID) + } + return bestStoreToMinimizeQPSDelta( + o.qpsPerReplica, + o.qpsRebalanceThreshold, + o.minRequiredQPSDiff, + eqClass.existing.StoreID, + candidates, + storeListToMap(domainStoreList), + ) +} + // rankedCandidateListForRebalancing returns a list of `rebalanceOptions`, i.e. // groups of candidate stores and the existing replicas that they could legally // replace in the range. See comment above `rebalanceOptions()` for more @@ -833,24 +1070,6 @@ func rankedCandidateListForRebalancing( // stores by attributes as well, but it's simplest to just run this for each // store. - // equivalenceClass captures the set of "equivalent" replacement candidates - // for each existing replica. "equivalent" here means the candidates that are - // just as diverse as the existing replica, conform to zone config constraints - // on the range and don't have a full disk. - // Following are a few examples: - // 1. Consider a 3 region cluster with regions A, B and C. Assume there is a - // range that has 1 replica in each of those regions. For each existing - // replica, its equivalence class would contain all the other stores in its - // own region. - // 2. Consider a cluster with 10 racks, each with 2 stores (on 2 different - // nodes). Assume that racks 1, 2 and 3 each have a replica for a range. For - // the existing replica in rack 1, its equivalence class would contain its - // neighboring store in rack 1 and all stores in racks 4...10. - type equivalenceClass struct { - existing roachpb.StoreDescriptor - candidateSL StoreList - candidates candidateList - } var equivalenceClasses []equivalenceClass var needRebalanceTo bool for _, existing := range existingStores { @@ -910,6 +1129,7 @@ func rankedCandidateListForRebalancing( if !cand.less(existing) { // If `cand` is not worse than `existing`, add it to the list. comparableCands = append(comparableCands, cand) + if !needRebalanceFrom && !needRebalanceTo && existing.less(cand) { needRebalanceTo = true log.VEventf(ctx, 2, @@ -925,17 +1145,22 @@ func rankedCandidateListForRebalancing( } else { sort.Sort(sort.Reverse(byScore(comparableCands))) } + + // Filter down to the set of stores that are better than the rest based on + // diversity, disk fullness and constraints conformance. These stores are + // all in the same equivalence class with regards to the range in question. bestCands := comparableCands.best() + bestStores := make([]roachpb.StoreDescriptor, len(bestCands)) for i := range bestCands { bestStores[i] = bestCands[i].store } - equivalenceClasses = append( - equivalenceClasses, equivalenceClass{ - existing: existing.store, - candidateSL: makeStoreList(bestStores), - candidates: bestCands, - }) + eqClass := equivalenceClass{ + existing: existing.store, + candidateSL: makeStoreList(bestStores), + candidates: bestCands, + } + equivalenceClasses = append(equivalenceClasses, eqClass) } // 3. Decide whether we should try to rebalance. Note that for each existing @@ -945,24 +1170,14 @@ func rankedCandidateListForRebalancing( needRebalance := needRebalanceFrom || needRebalanceTo var shouldRebalanceCheck bool if !needRebalance { - for _, existing := range existingStores { - var candidateSL StoreList - outer: - for _, comparable := range equivalenceClasses { - if existing.store.StoreID == comparable.existing.StoreID { - candidateSL = comparable.candidateSL - break outer - } - } - // NB: If we have any candidates that are at least as good as the existing - // replicas in terms of diversity and disk fullness, check whether the - // existing replicas' stats are divergent enough to justify a rebalance. - if options.shouldRebalanceBasedOnThresholds(ctx, existing.store, candidateSL) { + for _, eqClass := range equivalenceClasses { + if options.shouldRebalanceBasedOnThresholds(ctx, eqClass) { shouldRebalanceCheck = true break } } } + if !needRebalance && !shouldRebalanceCheck { return nil } @@ -984,7 +1199,7 @@ 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 := options.rebalanceFromConvergesScore(comparable.candidateSL, existing.store.Capacity) + convergesScore := options.rebalanceFromConvergesScore(comparable) balanceScore := options.balanceScore(comparable.candidateSL, existing.store.Capacity) existing.convergesScore = convergesScore existing.balanceScore = balanceScore @@ -1004,7 +1219,7 @@ func rankedCandidateListForRebalancing( s := cand.store cand.fullDisk = !rebalanceToMaxCapacityCheck(s) cand.balanceScore = options.balanceScore(comparable.candidateSL, s.Capacity) - cand.convergesScore = options.rebalanceToConvergesScore(comparable.candidateSL, s.Capacity) + cand.convergesScore = options.rebalanceToConvergesScore(comparable, s) cand.rangeCount = int(s.Capacity.RangeCount) candidates = append(candidates, cand) } @@ -1035,6 +1250,48 @@ func rankedCandidateListForRebalancing( return results } +// getStoreWithMinQPS returns the `StoreDescriptor` that belongs to the store +// serving the lowest QPS among all the `candidate` stores. +func getStoreWithMinQPS( + storeQPSMap map[roachpb.StoreID]float64, candidates []roachpb.StoreDescriptor, +) roachpb.StoreDescriptor { + minCandidateQPS := math.MaxFloat64 + var candidateWithMin roachpb.StoreDescriptor + for _, store := range candidates { + candidateQPS, ok := storeQPSMap[store.StoreID] + if !ok { + continue + } + if minCandidateQPS > candidateQPS { + minCandidateQPS = candidateQPS + candidateWithMin = store + } + } + return candidateWithMin +} + +// getStoreQPSDelta computes the QPS delta between the coldest and the hottest +// store in `stores`. +func getStoreQPSDelta( + storeQPSMap map[roachpb.StoreID]float64, stores []roachpb.StoreDescriptor, +) float64 { + maxCandidateQPS := float64(0) + minCandidateQPS := math.MaxFloat64 + for _, repl := range stores { + candidateQPS, ok := storeQPSMap[repl.StoreID] + if !ok { + continue + } + if maxCandidateQPS < candidateQPS { + maxCandidateQPS = candidateQPS + } + if minCandidateQPS > candidateQPS { + minCandidateQPS = candidateQPS + } + } + return maxCandidateQPS - minCandidateQPS +} + // bestRebalanceTarget returns the best target to try to rebalance to out of // the provided options, and removes it from the relevant candidate list. // Also returns the existing replicas that the chosen candidate was compared to. @@ -1519,8 +1776,8 @@ func underfullRangeThreshold(options rangeCountScorerOptions, mean float64) floa return mean - math.Max(mean*options.rangeRebalanceThreshold, minRangeRebalanceThreshold) } -func overfullQPSThreshold(options qpsScorerOptions, mean float64) float64 { - return mean + math.Max(mean*options.qpsRebalanceThreshold, minQPSThresholdDifference) +func overfullQPSThreshold(rebalanceThreshold float64, mean float64) float64 { + return mean + math.Max(mean*rebalanceThreshold, minQPSThresholdDifference) } func underfullQPSThreshold(options qpsScorerOptions, mean float64) float64 { diff --git a/pkg/kv/kvserver/allocator_scorer_test.go b/pkg/kv/kvserver/allocator_scorer_test.go index 95c9d87e09d9..797d3da86180 100644 --- a/pkg/kv/kvserver/allocator_scorer_test.go +++ b/pkg/kv/kvserver/allocator_scorer_test.go @@ -1570,21 +1570,24 @@ func TestRebalanceConvergesRangeCountOnMean(t *testing.T) { } options := rangeCountScorerOptions{} + eqClass := equivalenceClass{ + candidateSL: storeList, + } for i, tc := range testCases { - sc := roachpb.StoreCapacity{ - RangeCount: tc.rangeCount, + sd := roachpb.StoreDescriptor{ + StoreID: 1, + Capacity: roachpb.StoreCapacity{ + RangeCount: tc.rangeCount, + }, } - 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) + eqClass.existing = sd + if a, e := options.rebalanceToConvergesScore(eqClass, sd) == 1, tc.toConverges; a != e { + t.Errorf("%d: rebalanceToConvergesScore(eqClass, %+v) got %t; want %t", i, sd, 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 := options.rebalanceFromConvergesScore( - storeList, sc, - ) == 0, tc.fromConverges; a != e { - t.Errorf("%d: rebalanceFromConvergesScore(storeList, %+v) got %t; want %t", i, sc, a, e) + if a, e := options.rebalanceFromConvergesScore(eqClass) == 0, tc.fromConverges; a != e { + t.Errorf("%d: rebalanceFromConvergesScore(eqClass, %+v) got %t; want %t", i, sd, a, e) } } } diff --git a/pkg/kv/kvserver/allocator_test.go b/pkg/kv/kvserver/allocator_test.go index dab243cd8ccb..6226dc209cc7 100644 --- a/pkg/kv/kvserver/allocator_test.go +++ b/pkg/kv/kvserver/allocator_test.go @@ -847,7 +847,7 @@ func TestAllocatorMultipleStoresPerNodeLopsided(t *testing.T) { // TestAllocatorRebalance verifies that rebalance targets are chosen // randomly from amongst stores under the maxFractionUsedThreshold. -func TestAllocatorRebalance(t *testing.T) { +func TestAllocatorRebalanceBasedOnRangeCount(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -924,14 +924,18 @@ func TestAllocatorRebalance(t *testing.T) { } } + sl, _, _ := a.storePool.getStoreList(storeFilterThrottled) + eqClass := equivalenceClass{ + candidateSL: sl, + } // 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 := a.scorerOptions().shouldRebalanceBasedOnThresholds(ctx, desc, sl) + eqClass.existing = desc + result := a.scorerOptions().shouldRebalanceBasedOnThresholds(ctx, eqClass) if expResult := (i >= 2); expResult != result { t.Errorf("%d: expected rebalance %t; got %t; desc %+v; sl: %+v", i, expResult, result, desc, sl) } @@ -1323,15 +1327,18 @@ func TestAllocatorRebalanceThrashing(t *testing.T) { return nil }) sl, _, _ := a.storePool.getStoreList(storeFilterThrottled) - + eqClass := equivalenceClass{ + candidateSL: sl, + } // 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) } + eqClass.existing = desc if a, e := a.scorerOptions().shouldRebalanceBasedOnThresholds( - context.Background(), desc, sl, + context.Background(), eqClass, ), cluster[j].shouldRebalanceFrom; a != e { t.Errorf( "[store %d]: shouldRebalanceBasedOnThresholds %t != expected %t", store.StoreID, a, e, @@ -1342,9 +1349,8 @@ func TestAllocatorRebalanceThrashing(t *testing.T) { } } -// TestAllocatorRebalanceByQPS verifies that rebalance targets are chosen by QPS -// in the event that some stores lie outside the [underfull, overfull] window -// for QPS. +// TestAllocatorRebalanceByQPS tests that the allocator rebalances replicas +// based on QPS if there are underfull or overfull stores in the cluster. func TestAllocatorRebalanceByQPS(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -1371,11 +1377,35 @@ func TestAllocatorRebalanceByQPS(t *testing.T) { Capacity: roachpb.StoreCapacity{RangeCount: 1000, QueriesPerSecond: 1000}, }, } + + allStoresAroundTheMean := []*roachpb.StoreDescriptor{ + { + StoreID: 1, + Node: roachpb.NodeDescriptor{NodeID: 1}, + Capacity: roachpb.StoreCapacity{RangeCount: 1000, QueriesPerSecond: 1100}, + }, + { + 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: 900}, + }, + { + 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}, + Capacity: roachpb.StoreCapacity{RangeCount: 1000, QueriesPerSecond: 1300}, }, { StoreID: 2, @@ -1390,7 +1420,7 @@ func TestAllocatorRebalanceByQPS(t *testing.T) { { StoreID: 4, Node: roachpb.NodeDescriptor{NodeID: 4}, - Capacity: roachpb.StoreCapacity{RangeCount: 1000, QueriesPerSecond: 800}, + Capacity: roachpb.StoreCapacity{RangeCount: 1000, QueriesPerSecond: 700}, }, } @@ -1406,6 +1436,12 @@ func TestAllocatorRebalanceByQPS(t *testing.T) { testStores: allStoresEqual, expectRebalance: false, }, + { + // We don't expect any QPS based rebalancing when all stores are "close + // enough" to the mean. + testStores: allStoresAroundTheMean, + 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. @@ -1423,7 +1459,8 @@ func TestAllocatorRebalanceByQPS(t *testing.T) { ctx := context.Background() var rangeUsageInfo RangeUsageInfo options := qpsScorerOptions{ - qpsRebalanceThreshold: 0.1, + qpsPerReplica: 100, + qpsRebalanceThreshold: 0.2, } add, remove, _, ok := a.RebalanceVoter( ctx, @@ -1442,7 +1479,11 @@ func TestAllocatorRebalanceByQPS(t *testing.T) { // Verify shouldRebalanceBasedOnThresholds results. if desc, descOk := a.storePool.getStoreDescriptor(remove.StoreID); descOk { sl, _, _ := a.storePool.getStoreList(storeFilterThrottled) - result := options.shouldRebalanceBasedOnThresholds(ctx, desc, sl) + eqClass := equivalenceClass{ + existing: desc, + candidateSL: sl, + } + result := options.shouldRebalanceBasedOnThresholds(ctx, eqClass) require.True(t, result) } else { t.Fatalf("unable to get store %d descriptor", remove.StoreID) @@ -1604,7 +1645,11 @@ func TestAllocatorRebalanceByCount(t *testing.T) { t.Fatalf("%d: unable to get store %d descriptor", i, store.StoreID) } sl, _, _ := a.storePool.getStoreList(storeFilterThrottled) - result := a.scorerOptions().shouldRebalanceBasedOnThresholds(ctx, desc, sl) + eqClass := equivalenceClass{ + existing: desc, + candidateSL: sl, + } + result := a.scorerOptions().shouldRebalanceBasedOnThresholds(ctx, eqClass) if expResult := (i < 3); expResult != result { t.Errorf("%d: expected rebalance %t; got %t", i, expResult, result) } @@ -7266,14 +7311,15 @@ type testStore struct { immediateCompaction bool } -func (ts *testStore) add(bytes int64) { +func (ts *testStore) add(bytes int64, qps float64) { ts.Capacity.RangeCount++ ts.Capacity.Available -= bytes ts.Capacity.Used += bytes ts.Capacity.LogicalBytes += bytes + ts.Capacity.QueriesPerSecond += qps } -func (ts *testStore) rebalance(ots *testStore, bytes int64) { +func (ts *testStore) rebalance(ots *testStore, bytes int64, qps float64) { if ts.Capacity.RangeCount == 0 || (ts.Capacity.Capacity-ts.Capacity.Available) < bytes { return } @@ -7281,18 +7327,22 @@ func (ts *testStore) rebalance(ots *testStore, bytes int64) { // almost out of disk. (In a real allocator this is, for example, in // rankedCandidateListFor{Allocation,Rebalancing}). if !maxCapacityCheck(ots.StoreDescriptor) { - log.Infof(context.Background(), - "s%d too full to accept snapshot from s%d: %v", ots.StoreID, ts.StoreID, ots.Capacity) + log.Infof( + context.Background(), + "s%d too full to accept snapshot from s%d: %v", ots.StoreID, ts.StoreID, ots.Capacity, + ) return } log.Infof(context.Background(), "s%d accepting snapshot from s%d", ots.StoreID, ts.StoreID) ts.Capacity.RangeCount-- + ts.Capacity.QueriesPerSecond -= qps if ts.immediateCompaction { ts.Capacity.Available += bytes ts.Capacity.Used -= bytes } ts.Capacity.LogicalBytes -= bytes ots.Capacity.RangeCount++ + ots.Capacity.QueriesPerSecond += qps ots.Capacity.Available -= bytes ots.Capacity.Used += bytes ots.Capacity.LogicalBytes += bytes @@ -7373,7 +7423,7 @@ func TestAllocatorFullDisks(t *testing.T) { testStores[i].Capacity = roachpb.StoreCapacity{Capacity: capacity, Available: capacity} } // Initialize the cluster with a single range. - testStores[0].add(rangeSize) + testStores[0].add(rangeSize, 0) rangesAdded := 1 for i := 0; i < generations; i++ { @@ -7387,7 +7437,7 @@ func TestAllocatorFullDisks(t *testing.T) { toAdd := alloc.randGen.Intn(3) for k := 0; k < toAdd; k++ { if rangesAdded < rangesToAdd { - ts.add(rangeSize) + ts.add(rangeSize, 0) rangesAdded++ } } @@ -7427,7 +7477,7 @@ func TestAllocatorFullDisks(t *testing.T) { if log.V(1) { log.Infof(ctx, "rebalancing to %v; details: %s", target, details) } - testStores[k].rebalance(&testStores[int(target.StoreID)], rangeSize) + testStores[k].rebalance(&testStores[int(target.StoreID)], rangeSize, 0 /* qps */) } } // Gossip occasionally, as real Stores do when replicas move around. @@ -7455,7 +7505,307 @@ func TestAllocatorFullDisks(t *testing.T) { } } -func Example_rebalancing() { +func Example_RangeCount_rebalancing() { + testStores := make([]testStore, 20) + rebalanceFn := func(ctx context.Context, ts *testStore, testStores []testStore, alloc *Allocator) { + var rangeUsageInfo RangeUsageInfo + 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) + ts.rebalance(&testStores[int(target.StoreID)], alloc.randGen.Int63n(1<<20), 0 /* qps */) + } + } + + generation := 0 + const printEvery = 2 + printFn := func(testStores []testStore, table *tablewriter.Table) { + if generation%printEvery == 0 { + var totalBytes int64 + for j := 0; j < len(testStores); j++ { + totalBytes += testStores[j].Capacity.Capacity - testStores[j].Capacity.Available + } + row := make([]string, len(testStores)+1) + row[0] = fmt.Sprintf("%d", generation) + for j := 0; j < len(testStores); j++ { + ts := testStores[j] + bytes := ts.Capacity.Capacity - ts.Capacity.Available + row[j+1] = fmt.Sprintf("%3d %3d%%", ts.Capacity.RangeCount, (100*bytes)/totalBytes) + } + table.Append(row) + } + generation++ + } + + exampleRebalancing(testStores, rebalanceFn, printFn) + + // Output: + // +-----+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+ + // | 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 | 3 75% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 2 24% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | + // | 4 | 3 18% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 1 14% | 0 0% | 0 0% | 3 35% | 2 31% | 0 0% | 1 0% | 0 0% | 0 0% | 0 0% | 0 0% | + // | 6 | 3 7% | 0 0% | 0 0% | 1 5% | 0 0% | 2 8% | 0 0% | 0 0% | 0 0% | 3 17% | 0 0% | 2 9% | 3 21% | 3 20% | 2 1% | 3 8% | 0 0% | 0 0% | 0 0% | 0 0% | + // | 8 | 4 4% | 0 0% | 1 0% | 3 6% | 1 1% | 4 9% | 0 0% | 2 4% | 0 0% | 4 14% | 0 0% | 4 8% | 4 11% | 4 14% | 4 8% | 4 10% | 0 0% | 0 0% | 0 0% | 1 4% | + // | 10 | 5 3% | 1 2% | 3 2% | 4 5% | 3 5% | 5 6% | 0 0% | 3 5% | 3 2% | 5 11% | 3 2% | 5 7% | 5 10% | 5 11% | 5 5% | 5 9% | 1 1% | 1 0% | 1 1% | 3 6% | + // | 12 | 6 2% | 4 5% | 5 2% | 5 2% | 5 4% | 6 4% | 4 4% | 5 5% | 5 4% | 6 8% | 5 3% | 6 5% | 6 8% | 6 6% | 6 4% | 6 7% | 4 2% | 6 6% | 4 3% | 5 6% | + // | 14 | 8 3% | 6 5% | 7 2% | 7 3% | 7 3% | 8 4% | 6 5% | 7 6% | 7 4% | 8 7% | 7 3% | 8 5% | 8 7% | 8 7% | 8 4% | 8 6% | 6 3% | 8 5% | 6 3% | 7 5% | + // | 16 | 10 4% | 8 5% | 9 3% | 9 3% | 9 3% | 10 4% | 8 4% | 9 5% | 9 4% | 10 7% | 9 4% | 10 5% | 10 6% | 10 6% | 10 3% | 10 6% | 8 3% | 10 5% | 8 4% | 9 5% | + // | 18 | 12 4% | 10 5% | 11 4% | 11 4% | 11 4% | 12 4% | 10 4% | 11 5% | 11 4% | 12 6% | 11 4% | 12 5% | 12 6% | 12 6% | 12 4% | 12 5% | 10 4% | 12 4% | 10 4% | 11 4% | + // | 20 | 14 4% | 12 5% | 13 4% | 13 4% | 13 4% | 14 4% | 12 4% | 13 5% | 13 4% | 14 5% | 13 4% | 14 5% | 14 6% | 14 6% | 14 4% | 14 5% | 12 4% | 14 4% | 12 4% | 13 5% | + // | 22 | 16 4% | 14 5% | 15 4% | 15 4% | 15 4% | 16 5% | 14 4% | 15 5% | 15 4% | 16 5% | 15 4% | 16 4% | 16 6% | 16 5% | 16 4% | 16 4% | 14 4% | 16 4% | 14 5% | 15 5% | + // | 24 | 18 4% | 16 5% | 17 3% | 17 4% | 17 4% | 18 5% | 16 4% | 17 5% | 17 4% | 18 5% | 17 4% | 18 4% | 18 5% | 18 5% | 18 4% | 18 5% | 16 4% | 18 4% | 16 5% | 17 5% | + // | 26 | 20 4% | 18 5% | 19 3% | 19 4% | 19 4% | 20 5% | 18 4% | 19 5% | 19 4% | 20 5% | 19 4% | 20 4% | 20 5% | 20 5% | 20 4% | 20 4% | 18 5% | 20 4% | 18 5% | 19 4% | + // | 28 | 22 4% | 20 5% | 21 3% | 21 4% | 21 4% | 22 6% | 20 4% | 21 5% | 21 4% | 22 5% | 21 4% | 22 4% | 22 5% | 22 5% | 22 4% | 22 4% | 20 4% | 22 4% | 20 5% | 21 4% | + // | 30 | 24 4% | 22 5% | 23 3% | 23 4% | 23 4% | 24 6% | 22 4% | 23 5% | 23 5% | 24 5% | 23 4% | 24 4% | 24 5% | 24 5% | 24 4% | 24 4% | 22 4% | 24 4% | 22 5% | 23 4% | + // | 32 | 26 4% | 24 5% | 25 3% | 25 4% | 25 4% | 26 5% | 24 4% | 25 5% | 25 5% | 26 5% | 25 4% | 26 4% | 26 5% | 26 5% | 26 4% | 26 5% | 24 4% | 26 5% | 24 4% | 25 5% | + // | 34 | 28 4% | 26 5% | 27 3% | 27 4% | 27 4% | 28 5% | 26 4% | 27 5% | 27 5% | 28 5% | 27 4% | 28 4% | 28 5% | 28 5% | 28 4% | 28 5% | 26 4% | 28 5% | 26 5% | 27 4% | + // | 36 | 30 4% | 28 5% | 29 3% | 29 4% | 29 4% | 30 5% | 28 4% | 29 5% | 29 5% | 30 4% | 29 4% | 30 4% | 30 5% | 30 5% | 30 4% | 30 5% | 28 4% | 30 5% | 28 4% | 29 4% | + // | 38 | 32 4% | 30 5% | 31 4% | 31 4% | 31 4% | 32 5% | 30 4% | 31 5% | 31 5% | 32 4% | 31 4% | 32 4% | 32 5% | 32 5% | 32 4% | 32 5% | 30 4% | 32 5% | 30 4% | 31 4% | + // | 40 | 34 4% | 32 5% | 33 4% | 33 4% | 33 4% | 34 5% | 32 4% | 33 5% | 33 5% | 34 4% | 33 4% | 34 5% | 34 5% | 34 5% | 34 4% | 34 5% | 32 4% | 34 5% | 32 4% | 33 4% | + // | 42 | 36 4% | 34 5% | 35 4% | 35 4% | 35 4% | 36 5% | 34 4% | 35 5% | 35 5% | 36 4% | 35 4% | 36 5% | 36 5% | 36 5% | 36 4% | 36 5% | 34 5% | 36 5% | 34 4% | 35 5% | + // | 44 | 38 4% | 36 5% | 37 4% | 37 4% | 37 4% | 38 5% | 36 4% | 37 5% | 37 5% | 38 4% | 37 4% | 38 5% | 38 5% | 38 5% | 38 4% | 38 5% | 36 5% | 38 5% | 36 4% | 37 5% | + // | 46 | 40 4% | 38 5% | 39 4% | 39 4% | 39 4% | 40 5% | 38 4% | 39 5% | 39 5% | 40 4% | 39 4% | 40 5% | 40 5% | 40 5% | 40 4% | 40 5% | 38 5% | 40 5% | 38 4% | 39 5% | + // | 48 | 42 4% | 40 5% | 41 4% | 41 4% | 41 4% | 42 5% | 40 4% | 41 5% | 41 5% | 42 4% | 41 4% | 42 5% | 42 5% | 42 5% | 42 4% | 42 5% | 40 5% | 42 5% | 40 4% | 41 5% | + // | 50 | 44 4% | 42 5% | 43 3% | 43 4% | 43 4% | 44 5% | 42 4% | 43 5% | 43 5% | 44 4% | 43 4% | 44 5% | 44 5% | 44 5% | 44 4% | 44 5% | 42 5% | 44 5% | 42 4% | 43 5% | + // | 52 | 46 4% | 44 5% | 45 4% | 45 4% | 45 4% | 46 5% | 44 4% | 45 5% | 45 5% | 46 4% | 45 4% | 46 5% | 46 5% | 46 5% | 46 4% | 46 5% | 44 5% | 46 5% | 44 4% | 45 5% | + // | 54 | 48 4% | 46 5% | 47 4% | 47 4% | 47 4% | 48 5% | 46 4% | 47 5% | 47 5% | 48 4% | 47 4% | 48 5% | 48 5% | 48 5% | 48 5% | 48 5% | 46 5% | 48 5% | 46 4% | 47 5% | + // | 56 | 50 4% | 48 5% | 49 4% | 49 4% | 49 4% | 50 5% | 48 4% | 49 5% | 49 5% | 50 4% | 49 4% | 50 5% | 50 5% | 50 5% | 50 4% | 50 4% | 48 5% | 50 5% | 48 4% | 49 5% | + // | 58 | 52 4% | 50 5% | 51 4% | 51 4% | 51 4% | 52 5% | 50 4% | 51 5% | 51 5% | 52 4% | 51 4% | 52 5% | 52 5% | 52 5% | 52 4% | 52 4% | 50 5% | 52 5% | 50 4% | 51 5% | + // | 60 | 54 4% | 52 5% | 53 4% | 53 4% | 53 4% | 54 5% | 52 4% | 53 5% | 53 5% | 54 4% | 53 4% | 54 5% | 54 5% | 54 5% | 54 4% | 54 5% | 52 5% | 54 5% | 52 4% | 53 5% | + // | 62 | 56 4% | 54 5% | 55 4% | 55 4% | 55 4% | 56 5% | 54 4% | 55 5% | 55 5% | 56 5% | 55 4% | 56 5% | 56 5% | 56 5% | 56 5% | 56 5% | 54 5% | 56 5% | 54 4% | 55 5% | + // | 64 | 58 4% | 56 5% | 57 4% | 57 4% | 57 4% | 58 5% | 56 4% | 57 4% | 57 5% | 58 5% | 57 4% | 58 5% | 58 5% | 58 5% | 58 5% | 58 5% | 56 5% | 58 5% | 56 4% | 57 5% | + // | 66 | 60 4% | 58 5% | 59 4% | 59 4% | 59 4% | 60 5% | 58 4% | 59 5% | 59 5% | 60 5% | 59 4% | 60 5% | 60 5% | 60 5% | 60 5% | 60 5% | 58 5% | 60 5% | 58 4% | 59 5% | + // | 68 | 62 4% | 60 5% | 61 4% | 61 4% | 61 4% | 62 5% | 60 4% | 61 4% | 61 5% | 62 5% | 61 4% | 62 5% | 62 5% | 62 5% | 62 5% | 62 5% | 60 5% | 62 5% | 60 4% | 61 5% | + // | 70 | 64 4% | 62 5% | 63 4% | 63 4% | 63 4% | 64 5% | 62 4% | 63 5% | 63 5% | 64 4% | 63 4% | 64 5% | 64 5% | 64 5% | 64 4% | 64 4% | 62 5% | 64 5% | 62 4% | 63 5% | + // | 72 | 66 4% | 64 5% | 65 4% | 65 4% | 65 4% | 66 5% | 64 4% | 65 5% | 65 5% | 66 5% | 65 4% | 66 5% | 66 5% | 66 5% | 66 4% | 66 5% | 64 5% | 66 5% | 64 4% | 65 5% | + // | 74 | 68 4% | 66 5% | 67 4% | 67 4% | 67 4% | 68 5% | 66 4% | 67 5% | 67 5% | 68 4% | 67 4% | 68 5% | 68 5% | 68 5% | 68 4% | 68 4% | 66 5% | 68 5% | 66 4% | 67 5% | + // | 76 | 70 4% | 68 5% | 69 4% | 69 4% | 69 4% | 70 5% | 68 4% | 69 5% | 69 5% | 70 4% | 69 4% | 70 5% | 70 5% | 70 5% | 70 4% | 70 5% | 68 5% | 70 5% | 68 4% | 69 5% | + // | 78 | 72 4% | 70 5% | 71 4% | 71 4% | 71 4% | 72 5% | 70 4% | 71 5% | 71 5% | 72 4% | 71 4% | 72 5% | 72 5% | 72 5% | 72 4% | 72 4% | 70 4% | 72 5% | 70 4% | 71 5% | + // | 80 | 74 4% | 72 5% | 73 4% | 73 4% | 73 4% | 74 5% | 72 4% | 73 5% | 73 5% | 74 4% | 73 4% | 74 5% | 74 5% | 74 5% | 74 4% | 74 5% | 72 5% | 74 5% | 72 4% | 73 4% | + // | 82 | 76 4% | 74 5% | 75 4% | 75 4% | 75 4% | 76 5% | 74 4% | 75 5% | 75 5% | 76 4% | 75 4% | 76 5% | 76 5% | 76 5% | 76 4% | 76 4% | 74 5% | 76 4% | 74 4% | 75 5% | + // | 84 | 78 4% | 76 5% | 77 4% | 77 4% | 77 4% | 78 5% | 76 4% | 77 5% | 77 5% | 78 4% | 77 4% | 78 5% | 78 5% | 78 5% | 78 5% | 78 4% | 76 5% | 78 4% | 76 4% | 77 4% | + // | 86 | 80 4% | 78 5% | 79 4% | 79 4% | 79 4% | 80 5% | 78 4% | 79 5% | 79 5% | 80 4% | 79 4% | 80 5% | 80 5% | 80 5% | 80 5% | 80 5% | 78 5% | 80 4% | 78 4% | 79 4% | + // | 88 | 82 4% | 80 5% | 81 4% | 81 4% | 81 4% | 82 5% | 80 5% | 81 5% | 81 5% | 82 4% | 81 4% | 82 5% | 82 5% | 82 5% | 82 5% | 82 5% | 80 5% | 82 5% | 80 4% | 81 4% | + // | 90 | 84 4% | 82 5% | 83 4% | 83 4% | 83 4% | 84 5% | 82 5% | 83 5% | 83 5% | 84 4% | 83 4% | 84 5% | 84 5% | 84 5% | 84 4% | 84 5% | 82 5% | 84 5% | 82 4% | 83 4% | + // | 92 | 86 4% | 84 5% | 85 4% | 85 4% | 85 4% | 86 5% | 84 5% | 85 5% | 85 5% | 86 4% | 85 4% | 86 5% | 86 5% | 86 5% | 86 4% | 86 5% | 84 5% | 86 4% | 84 4% | 85 5% | + // | 94 | 88 4% | 86 5% | 87 4% | 87 4% | 87 4% | 88 5% | 86 5% | 87 5% | 87 5% | 88 4% | 87 4% | 88 5% | 88 5% | 88 5% | 88 4% | 88 5% | 86 5% | 88 4% | 86 4% | 87 5% | + // | 96 | 90 4% | 88 5% | 89 4% | 89 4% | 89 4% | 90 5% | 88 5% | 89 5% | 89 5% | 90 5% | 89 4% | 90 5% | 90 5% | 90 5% | 90 4% | 90 5% | 88 5% | 90 4% | 88 4% | 89 5% | + // | 98 | 92 4% | 90 5% | 91 4% | 91 4% | 91 4% | 92 5% | 90 5% | 91 5% | 91 5% | 92 5% | 91 4% | 92 5% | 92 5% | 92 5% | 92 4% | 92 5% | 90 5% | 92 4% | 90 4% | 91 5% | + // +-----+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+ + // Total bytes=969478392, ranges=1845 +} + +func qpsBasedRebalanceFn( + ctx context.Context, candidate *testStore, testStores []testStore, alloc *Allocator, +) { + avgQPS := candidate.Capacity.QueriesPerSecond / float64(candidate.Capacity.RangeCount) + jitteredQPS := avgQPS * (1 + alloc.randGen.Float64()) + opts := qpsScorerOptions{ + qpsPerReplica: jitteredQPS, + qpsRebalanceThreshold: 0.2, + } + var rangeUsageInfo RangeUsageInfo + add, remove, details, ok := alloc.RebalanceVoter( + ctx, + emptySpanConfig(), + nil, + []roachpb.ReplicaDescriptor{{NodeID: candidate.Node.NodeID, StoreID: candidate.StoreID}}, + nil, + rangeUsageInfo, + storeFilterThrottled, + opts, + ) + if ok { + log.Infof(ctx, "rebalancing from %v to %v; details: %s", remove, add, details) + candidate.rebalance(&testStores[int(add.StoreID)], alloc.randGen.Int63n(1<<20), jitteredQPS) + } +} + +func Example_QPS_rebalancing_single_region() { + generation := 0 + const printEvery = 2 + printFn := func(testStores []testStore, table *tablewriter.Table) { + if generation%printEvery == 0 { + row := make([]string, len(testStores)+1) + row[0] = fmt.Sprintf("%d", generation) + for j := 0; j < len(testStores); j++ { + ts := testStores[j] + row[j+1] = fmt.Sprintf("%d %0.2f", ts.Capacity.RangeCount, ts.Capacity.QueriesPerSecond) + } + table.Append(row) + } + generation++ + } + + testStores := make([]testStore, 10) + exampleRebalancing(testStores, qpsBasedRebalanceFn, printFn) + + // Output: + // +-----+------------+------------+------------+------------+------------+------------+------------+------------+------------+------------+ + // | gen | store 0 | store 1 | store 2 | store 3 | store 4 | store 5 | store 6 | store 7 | store 8 | store 9 | + // +-----+------------+------------+------------+------------+------------+------------+------------+------------+------------+------------+ + // | 0 | 2 29943.92 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | + // | 2 | 3 17950.57 | 2 11993.35 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | + // | 4 | 3 7473.51 | 3 6220.99 | 2 7930.67 | 2 8318.76 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | + // | 6 | 3 3411.46 | 4 3478.11 | 3 4735.76 | 3 4775.10 | 2 2598.82 | 4 10944.68 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | + // | 8 | 5 3411.46 | 6 3478.11 | 4 3392.44 | 4 2972.12 | 4 2598.82 | 4 4107.39 | 3 4636.29 | 2 5347.30 | 0 0.00 | 0 0.00 | + // | 10 | 7 3411.46 | 8 3478.11 | 6 3392.44 | 6 2972.12 | 6 2598.82 | 5 2743.76 | 4 2814.28 | 3 2757.89 | 3 3205.33 | 1 2569.71 | + // | 12 | 9 3411.46 | 10 3478.11 | 8 3392.44 | 8 2972.12 | 8 2598.82 | 7 2743.76 | 6 2814.28 | 5 2757.89 | 5 3205.33 | 3 2569.71 | + // | 14 | 11 3411.46 | 12 3478.11 | 10 3392.44 | 10 2972.12 | 10 2598.82 | 9 2743.76 | 8 2814.28 | 7 2757.89 | 7 3205.33 | 5 2569.71 | + // | 16 | 13 3411.46 | 14 3478.11 | 12 3392.44 | 12 2972.12 | 12 2598.82 | 11 2743.76 | 10 2814.28 | 9 2757.89 | 9 3205.33 | 7 2569.71 | + // | 18 | 15 3411.46 | 16 3478.11 | 14 3392.44 | 14 2972.12 | 14 2598.82 | 13 2743.76 | 12 2814.28 | 11 2757.89 | 11 3205.33 | 9 2569.71 | + // | 20 | 17 3411.46 | 18 3478.11 | 16 3392.44 | 16 2972.12 | 16 2598.82 | 15 2743.76 | 14 2814.28 | 13 2757.89 | 13 3205.33 | 11 2569.71 | + // | 22 | 19 3411.46 | 20 3478.11 | 18 3392.44 | 18 2972.12 | 18 2598.82 | 17 2743.76 | 16 2814.28 | 15 2757.89 | 15 3205.33 | 13 2569.71 | + // | 24 | 21 3411.46 | 22 3478.11 | 20 3392.44 | 20 2972.12 | 20 2598.82 | 19 2743.76 | 18 2814.28 | 17 2757.89 | 17 3205.33 | 15 2569.71 | + // | 26 | 23 3411.46 | 24 3478.11 | 22 3392.44 | 22 2972.12 | 22 2598.82 | 21 2743.76 | 20 2814.28 | 19 2757.89 | 19 3205.33 | 17 2569.71 | + // | 28 | 25 3411.46 | 26 3478.11 | 24 3392.44 | 24 2972.12 | 24 2598.82 | 23 2743.76 | 22 2814.28 | 21 2757.89 | 21 3205.33 | 19 2569.71 | + // | 30 | 27 3411.46 | 28 3478.11 | 26 3392.44 | 26 2972.12 | 26 2598.82 | 25 2743.76 | 24 2814.28 | 23 2757.89 | 23 3205.33 | 21 2569.71 | + // | 32 | 29 3411.46 | 30 3478.11 | 28 3392.44 | 28 2972.12 | 28 2598.82 | 27 2743.76 | 26 2814.28 | 25 2757.89 | 25 3205.33 | 23 2569.71 | + // | 34 | 31 3411.46 | 32 3478.11 | 30 3392.44 | 30 2972.12 | 30 2598.82 | 29 2743.76 | 28 2814.28 | 27 2757.89 | 27 3205.33 | 25 2569.71 | + // | 36 | 33 3411.46 | 34 3478.11 | 32 3392.44 | 32 2972.12 | 32 2598.82 | 31 2743.76 | 30 2814.28 | 29 2757.89 | 29 3205.33 | 27 2569.71 | + // | 38 | 35 3411.46 | 36 3478.11 | 34 3392.44 | 34 2972.12 | 34 2598.82 | 33 2743.76 | 32 2814.28 | 31 2757.89 | 31 3205.33 | 29 2569.71 | + // | 40 | 37 3411.46 | 38 3478.11 | 36 3392.44 | 36 2972.12 | 36 2598.82 | 35 2743.76 | 34 2814.28 | 33 2757.89 | 33 3205.33 | 31 2569.71 | + // | 42 | 39 3411.46 | 40 3478.11 | 38 3392.44 | 38 2972.12 | 38 2598.82 | 37 2743.76 | 36 2814.28 | 35 2757.89 | 35 3205.33 | 33 2569.71 | + // | 44 | 41 3411.46 | 42 3478.11 | 40 3392.44 | 40 2972.12 | 40 2598.82 | 39 2743.76 | 38 2814.28 | 37 2757.89 | 37 3205.33 | 35 2569.71 | + // | 46 | 43 3411.46 | 44 3478.11 | 42 3392.44 | 42 2972.12 | 42 2598.82 | 41 2743.76 | 40 2814.28 | 39 2757.89 | 39 3205.33 | 37 2569.71 | + // | 48 | 45 3411.46 | 46 3478.11 | 44 3392.44 | 44 2972.12 | 44 2598.82 | 43 2743.76 | 42 2814.28 | 41 2757.89 | 41 3205.33 | 39 2569.71 | + // | 50 | 47 3411.46 | 48 3478.11 | 46 3392.44 | 46 2972.12 | 46 2598.82 | 45 2743.76 | 44 2814.28 | 43 2757.89 | 43 3205.33 | 41 2569.71 | + // | 52 | 49 3411.46 | 50 3478.11 | 48 3392.44 | 48 2972.12 | 48 2598.82 | 47 2743.76 | 46 2814.28 | 45 2757.89 | 45 3205.33 | 43 2569.71 | + // | 54 | 51 3411.46 | 52 3478.11 | 50 3392.44 | 50 2972.12 | 50 2598.82 | 49 2743.76 | 48 2814.28 | 47 2757.89 | 47 3205.33 | 45 2569.71 | + // | 56 | 53 3411.46 | 54 3478.11 | 52 3392.44 | 52 2972.12 | 52 2598.82 | 51 2743.76 | 50 2814.28 | 49 2757.89 | 49 3205.33 | 47 2569.71 | + // | 58 | 55 3411.46 | 56 3478.11 | 54 3392.44 | 54 2972.12 | 54 2598.82 | 53 2743.76 | 52 2814.28 | 51 2757.89 | 51 3205.33 | 49 2569.71 | + // | 60 | 57 3411.46 | 58 3478.11 | 56 3392.44 | 56 2972.12 | 56 2598.82 | 55 2743.76 | 54 2814.28 | 53 2757.89 | 53 3205.33 | 51 2569.71 | + // | 62 | 59 3411.46 | 60 3478.11 | 58 3392.44 | 58 2972.12 | 58 2598.82 | 57 2743.76 | 56 2814.28 | 55 2757.89 | 55 3205.33 | 53 2569.71 | + // | 64 | 61 3411.46 | 62 3478.11 | 60 3392.44 | 60 2972.12 | 60 2598.82 | 59 2743.76 | 58 2814.28 | 57 2757.89 | 57 3205.33 | 55 2569.71 | + // | 66 | 63 3411.46 | 64 3478.11 | 62 3392.44 | 62 2972.12 | 62 2598.82 | 61 2743.76 | 60 2814.28 | 59 2757.89 | 59 3205.33 | 57 2569.71 | + // | 68 | 65 3411.46 | 66 3478.11 | 64 3392.44 | 64 2972.12 | 64 2598.82 | 63 2743.76 | 62 2814.28 | 61 2757.89 | 61 3205.33 | 59 2569.71 | + // | 70 | 67 3411.46 | 68 3478.11 | 66 3392.44 | 66 2972.12 | 66 2598.82 | 65 2743.76 | 64 2814.28 | 63 2757.89 | 63 3205.33 | 61 2569.71 | + // | 72 | 69 3411.46 | 70 3478.11 | 68 3392.44 | 68 2972.12 | 68 2598.82 | 67 2743.76 | 66 2814.28 | 65 2757.89 | 65 3205.33 | 63 2569.71 | + // | 74 | 71 3411.46 | 72 3478.11 | 70 3392.44 | 70 2972.12 | 70 2598.82 | 69 2743.76 | 68 2814.28 | 67 2757.89 | 67 3205.33 | 65 2569.71 | + // | 76 | 73 3411.46 | 74 3478.11 | 72 3392.44 | 72 2972.12 | 72 2598.82 | 71 2743.76 | 70 2814.28 | 69 2757.89 | 69 3205.33 | 67 2569.71 | + // | 78 | 75 3411.46 | 76 3478.11 | 74 3392.44 | 74 2972.12 | 74 2598.82 | 73 2743.76 | 72 2814.28 | 71 2757.89 | 71 3205.33 | 69 2569.71 | + // | 80 | 77 3411.46 | 78 3478.11 | 76 3392.44 | 76 2972.12 | 76 2598.82 | 75 2743.76 | 74 2814.28 | 73 2757.89 | 73 3205.33 | 71 2569.71 | + // | 82 | 79 3411.46 | 80 3478.11 | 78 3392.44 | 78 2972.12 | 78 2598.82 | 77 2743.76 | 76 2814.28 | 75 2757.89 | 75 3205.33 | 73 2569.71 | + // | 84 | 81 3411.46 | 82 3478.11 | 80 3392.44 | 80 2972.12 | 80 2598.82 | 79 2743.76 | 78 2814.28 | 77 2757.89 | 77 3205.33 | 75 2569.71 | + // | 86 | 83 3411.46 | 84 3478.11 | 82 3392.44 | 82 2972.12 | 82 2598.82 | 81 2743.76 | 80 2814.28 | 79 2757.89 | 79 3205.33 | 77 2569.71 | + // | 88 | 85 3411.46 | 86 3478.11 | 84 3392.44 | 84 2972.12 | 84 2598.82 | 83 2743.76 | 82 2814.28 | 81 2757.89 | 81 3205.33 | 79 2569.71 | + // | 90 | 87 3411.46 | 88 3478.11 | 86 3392.44 | 86 2972.12 | 86 2598.82 | 85 2743.76 | 84 2814.28 | 83 2757.89 | 83 3205.33 | 81 2569.71 | + // | 92 | 89 3411.46 | 90 3478.11 | 88 3392.44 | 88 2972.12 | 88 2598.82 | 87 2743.76 | 86 2814.28 | 85 2757.89 | 85 3205.33 | 83 2569.71 | + // | 94 | 91 3411.46 | 92 3478.11 | 90 3392.44 | 90 2972.12 | 90 2598.82 | 89 2743.76 | 88 2814.28 | 87 2757.89 | 87 3205.33 | 85 2569.71 | + // | 96 | 93 3411.46 | 94 3478.11 | 92 3392.44 | 92 2972.12 | 92 2598.82 | 91 2743.76 | 90 2814.28 | 89 2757.89 | 89 3205.33 | 87 2569.71 | + // | 98 | 95 3411.46 | 96 3478.11 | 94 3392.44 | 94 2972.12 | 94 2598.82 | 93 2743.76 | 92 2814.28 | 91 2757.89 | 91 3205.33 | 89 2569.71 | + // +-----+------------+------------+------------+------------+------------+------------+------------+------------+------------+------------+ + // Total bytes=477376096, ranges=939 +} + +func Example_QPS_rebalancing_multi_region() { + generation := 0 + const printEvery = 2 + printFn := func(testStores []testStore, table *tablewriter.Table) { + if generation%printEvery == 0 { + row := make([]string, len(testStores)+1) + row[0] = fmt.Sprintf("%d", generation) + for j := 0; j < len(testStores); j++ { + ts := testStores[j] + row[j+1] = fmt.Sprintf("%d %0.2f", ts.Capacity.RangeCount, ts.Capacity.QueriesPerSecond) + } + table.Append(row) + } + generation++ + } + + const numStores = 15 + testStores := make([]testStore, numStores) + for i := 0; i < 8; i++ { + testStores[i].Node.Locality = roachpb.Locality{ + []roachpb.Tier{{Key: "region", Value: "A"}}, + } + } + for i := 8; i < 13; i++ { + testStores[i].Node.Locality = roachpb.Locality{ + []roachpb.Tier{{Key: "region", Value: "B"}}, + } + } + for i := 13; i < numStores; i++ { + testStores[i].Node.Locality = roachpb.Locality{ + []roachpb.Tier{{Key: "region", Value: "C"}}, + } + } + exampleRebalancing(testStores, qpsBasedRebalanceFn, printFn) + + // Output: + // +-----+------------+------------+------------+------------+------------+------------+------------+------------+------------+------------+------------+------------+------------+------------+------------+ + // | 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 | + // +-----+------------+------------+------------+------------+------------+------------+------------+------------+------------+------------+------------+------------+------------+------------+------------+ + // | 0 | 2 29943.92 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | + // | 2 | 3 17950.57 | 2 11993.35 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | + // | 4 | 3 7473.51 | 3 6220.99 | 2 7930.67 | 2 8318.76 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | + // | 6 | 3 3411.46 | 3 1788.05 | 3 5620.53 | 3 5486.53 | 3 4928.13 | 4 8709.23 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | + // | 8 | 4 1954.26 | 5 1788.05 | 3 2121.87 | 3 2138.62 | 3 1269.93 | 4 3938.09 | 5 7465.02 | 5 9268.08 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | + // | 10 | 6 1954.26 | 7 1788.05 | 5 2121.87 | 5 2138.62 | 5 1269.93 | 4 1738.27 | 5 3800.32 | 5 5472.52 | 3 3550.29 | 4 6109.78 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | 0 0.00 | + // | 12 | 8 1954.26 | 9 1788.05 | 7 2121.87 | 7 2138.62 | 7 1269.93 | 6 1738.27 | 5 2038.40 | 5 3055.35 | 4 2027.79 | 4 2656.52 | 4 3698.73 | 4 5456.13 | 0 0.00 | 0 0.00 | 0 0.00 | + // | 14 | 10 1954.26 | 11 1788.05 | 9 2121.87 | 9 2138.62 | 9 1269.93 | 8 1738.27 | 7 2038.40 | 6 2169.10 | 6 2027.79 | 5 1608.05 | 5 2380.65 | 4 3152.09 | 4 3451.23 | 2 2105.61 | 0 0.00 | + // | 16 | 12 1954.26 | 13 1788.05 | 11 2121.87 | 11 2138.62 | 11 1269.93 | 10 1738.27 | 9 2038.40 | 8 2169.10 | 8 2027.79 | 7 1608.05 | 7 2380.65 | 5 2041.69 | 5 2161.29 | 4 2105.61 | 3 2400.34 | + // | 18 | 14 1954.26 | 15 1788.05 | 13 2121.87 | 13 2138.62 | 14 1881.00 | 12 1738.27 | 11 2038.40 | 10 2169.10 | 10 2027.79 | 9 1608.05 | 9 2380.65 | 7 2041.69 | 7 2161.29 | 6 2105.61 | 4 1789.27 | + // | 20 | 16 1954.26 | 17 1788.05 | 15 2121.87 | 15 2138.62 | 16 1881.00 | 14 1738.27 | 13 2038.40 | 12 2169.10 | 12 2027.79 | 11 1608.05 | 11 2380.65 | 9 2041.69 | 9 2161.29 | 8 2105.61 | 6 1789.27 | + // | 22 | 18 1954.26 | 19 1788.05 | 17 2121.87 | 17 2138.62 | 18 1881.00 | 16 1738.27 | 15 2038.40 | 14 2169.10 | 14 2027.79 | 13 1608.05 | 13 2380.65 | 11 2041.69 | 11 2161.29 | 10 2105.61 | 8 1789.27 | + // | 24 | 20 1954.26 | 21 1788.05 | 19 2121.87 | 19 2138.62 | 20 1881.00 | 18 1738.27 | 17 2038.40 | 16 2169.10 | 16 2027.79 | 15 1608.05 | 15 2380.65 | 13 2041.69 | 13 2161.29 | 12 2105.61 | 10 1789.27 | + // | 26 | 22 1954.26 | 23 1788.05 | 21 2121.87 | 21 2138.62 | 22 1881.00 | 20 1738.27 | 19 2038.40 | 18 2169.10 | 18 2027.79 | 17 1608.05 | 17 2380.65 | 15 2041.69 | 15 2161.29 | 14 2105.61 | 12 1789.27 | + // | 28 | 24 1954.26 | 25 1788.05 | 23 2121.87 | 23 2138.62 | 24 1881.00 | 22 1738.27 | 21 2038.40 | 20 2169.10 | 20 2027.79 | 19 1608.05 | 19 2380.65 | 17 2041.69 | 17 2161.29 | 16 2105.61 | 14 1789.27 | + // | 30 | 26 1954.26 | 27 1788.05 | 25 2121.87 | 25 2138.62 | 26 1881.00 | 24 1738.27 | 23 2038.40 | 22 2169.10 | 22 2027.79 | 21 1608.05 | 21 2380.65 | 19 2041.69 | 19 2161.29 | 18 2105.61 | 16 1789.27 | + // | 32 | 28 1954.26 | 29 1788.05 | 27 2121.87 | 27 2138.62 | 28 1881.00 | 26 1738.27 | 25 2038.40 | 24 2169.10 | 24 2027.79 | 23 1608.05 | 23 2380.65 | 21 2041.69 | 21 2161.29 | 20 2105.61 | 18 1789.27 | + // | 34 | 30 1954.26 | 31 1788.05 | 29 2121.87 | 29 2138.62 | 30 1881.00 | 28 1738.27 | 27 2038.40 | 26 2169.10 | 26 2027.79 | 25 1608.05 | 25 2380.65 | 23 2041.69 | 23 2161.29 | 22 2105.61 | 20 1789.27 | + // | 36 | 32 1954.26 | 33 1788.05 | 31 2121.87 | 31 2138.62 | 32 1881.00 | 30 1738.27 | 29 2038.40 | 28 2169.10 | 28 2027.79 | 27 1608.05 | 27 2380.65 | 25 2041.69 | 25 2161.29 | 24 2105.61 | 22 1789.27 | + // | 38 | 34 1954.26 | 35 1788.05 | 33 2121.87 | 33 2138.62 | 34 1881.00 | 32 1738.27 | 31 2038.40 | 30 2169.10 | 30 2027.79 | 29 1608.05 | 29 2380.65 | 27 2041.69 | 27 2161.29 | 26 2105.61 | 24 1789.27 | + // | 40 | 36 1954.26 | 37 1788.05 | 35 2121.87 | 35 2138.62 | 36 1881.00 | 34 1738.27 | 33 2038.40 | 32 2169.10 | 32 2027.79 | 31 1608.05 | 31 2380.65 | 29 2041.69 | 29 2161.29 | 28 2105.61 | 26 1789.27 | + // | 42 | 38 1954.26 | 39 1788.05 | 37 2121.87 | 37 2138.62 | 38 1881.00 | 36 1738.27 | 35 2038.40 | 34 2169.10 | 34 2027.79 | 33 1608.05 | 33 2380.65 | 31 2041.69 | 31 2161.29 | 30 2105.61 | 28 1789.27 | + // | 44 | 40 1954.26 | 41 1788.05 | 39 2121.87 | 39 2138.62 | 40 1881.00 | 38 1738.27 | 37 2038.40 | 36 2169.10 | 36 2027.79 | 35 1608.05 | 35 2380.65 | 33 2041.69 | 33 2161.29 | 32 2105.61 | 30 1789.27 | + // | 46 | 42 1954.26 | 43 1788.05 | 41 2121.87 | 41 2138.62 | 42 1881.00 | 40 1738.27 | 39 2038.40 | 38 2169.10 | 38 2027.79 | 37 1608.05 | 37 2380.65 | 35 2041.69 | 35 2161.29 | 34 2105.61 | 32 1789.27 | + // | 48 | 44 1954.26 | 45 1788.05 | 43 2121.87 | 43 2138.62 | 44 1881.00 | 42 1738.27 | 41 2038.40 | 40 2169.10 | 40 2027.79 | 39 1608.05 | 39 2380.65 | 37 2041.69 | 37 2161.29 | 36 2105.61 | 34 1789.27 | + // | 50 | 46 1954.26 | 47 1788.05 | 45 2121.87 | 45 2138.62 | 46 1881.00 | 44 1738.27 | 43 2038.40 | 42 2169.10 | 42 2027.79 | 41 1608.05 | 41 2380.65 | 39 2041.69 | 39 2161.29 | 38 2105.61 | 36 1789.27 | + // | 52 | 48 1954.26 | 49 1788.05 | 47 2121.87 | 47 2138.62 | 48 1881.00 | 46 1738.27 | 45 2038.40 | 44 2169.10 | 44 2027.79 | 43 1608.05 | 43 2380.65 | 41 2041.69 | 41 2161.29 | 40 2105.61 | 38 1789.27 | + // | 54 | 50 1954.26 | 51 1788.05 | 49 2121.87 | 49 2138.62 | 50 1881.00 | 48 1738.27 | 47 2038.40 | 46 2169.10 | 46 2027.79 | 45 1608.05 | 45 2380.65 | 43 2041.69 | 43 2161.29 | 42 2105.61 | 40 1789.27 | + // | 56 | 52 1954.26 | 53 1788.05 | 51 2121.87 | 51 2138.62 | 52 1881.00 | 50 1738.27 | 49 2038.40 | 48 2169.10 | 48 2027.79 | 47 1608.05 | 47 2380.65 | 45 2041.69 | 45 2161.29 | 44 2105.61 | 42 1789.27 | + // | 58 | 54 1954.26 | 55 1788.05 | 53 2121.87 | 53 2138.62 | 54 1881.00 | 52 1738.27 | 51 2038.40 | 50 2169.10 | 50 2027.79 | 49 1608.05 | 49 2380.65 | 47 2041.69 | 47 2161.29 | 46 2105.61 | 44 1789.27 | + // | 60 | 56 1954.26 | 57 1788.05 | 55 2121.87 | 55 2138.62 | 56 1881.00 | 54 1738.27 | 53 2038.40 | 52 2169.10 | 52 2027.79 | 51 1608.05 | 51 2380.65 | 49 2041.69 | 49 2161.29 | 48 2105.61 | 46 1789.27 | + // | 62 | 58 1954.26 | 59 1788.05 | 57 2121.87 | 57 2138.62 | 58 1881.00 | 56 1738.27 | 55 2038.40 | 54 2169.10 | 54 2027.79 | 53 1608.05 | 53 2380.65 | 51 2041.69 | 51 2161.29 | 50 2105.61 | 48 1789.27 | + // | 64 | 60 1954.26 | 61 1788.05 | 59 2121.87 | 59 2138.62 | 60 1881.00 | 58 1738.27 | 57 2038.40 | 56 2169.10 | 56 2027.79 | 55 1608.05 | 55 2380.65 | 53 2041.69 | 53 2161.29 | 52 2105.61 | 50 1789.27 | + // | 66 | 62 1954.26 | 63 1788.05 | 61 2121.87 | 61 2138.62 | 62 1881.00 | 60 1738.27 | 59 2038.40 | 58 2169.10 | 58 2027.79 | 57 1608.05 | 57 2380.65 | 55 2041.69 | 55 2161.29 | 54 2105.61 | 52 1789.27 | + // | 68 | 64 1954.26 | 65 1788.05 | 63 2121.87 | 63 2138.62 | 64 1881.00 | 62 1738.27 | 61 2038.40 | 60 2169.10 | 60 2027.79 | 59 1608.05 | 59 2380.65 | 57 2041.69 | 57 2161.29 | 56 2105.61 | 54 1789.27 | + // | 70 | 66 1954.26 | 67 1788.05 | 65 2121.87 | 65 2138.62 | 66 1881.00 | 64 1738.27 | 63 2038.40 | 62 2169.10 | 62 2027.79 | 61 1608.05 | 61 2380.65 | 59 2041.69 | 59 2161.29 | 58 2105.61 | 56 1789.27 | + // | 72 | 68 1954.26 | 69 1788.05 | 67 2121.87 | 67 2138.62 | 68 1881.00 | 66 1738.27 | 65 2038.40 | 64 2169.10 | 64 2027.79 | 63 1608.05 | 63 2380.65 | 61 2041.69 | 61 2161.29 | 60 2105.61 | 58 1789.27 | + // | 74 | 70 1954.26 | 71 1788.05 | 69 2121.87 | 69 2138.62 | 70 1881.00 | 68 1738.27 | 67 2038.40 | 66 2169.10 | 66 2027.79 | 65 1608.05 | 65 2380.65 | 63 2041.69 | 63 2161.29 | 62 2105.61 | 60 1789.27 | + // | 76 | 72 1954.26 | 73 1788.05 | 71 2121.87 | 71 2138.62 | 72 1881.00 | 70 1738.27 | 69 2038.40 | 68 2169.10 | 68 2027.79 | 67 1608.05 | 67 2380.65 | 65 2041.69 | 65 2161.29 | 64 2105.61 | 62 1789.27 | + // | 78 | 74 1954.26 | 75 1788.05 | 73 2121.87 | 73 2138.62 | 74 1881.00 | 72 1738.27 | 71 2038.40 | 70 2169.10 | 70 2027.79 | 69 1608.05 | 69 2380.65 | 67 2041.69 | 67 2161.29 | 66 2105.61 | 64 1789.27 | + // | 80 | 76 1954.26 | 77 1788.05 | 75 2121.87 | 75 2138.62 | 76 1881.00 | 74 1738.27 | 73 2038.40 | 72 2169.10 | 72 2027.79 | 71 1608.05 | 71 2380.65 | 69 2041.69 | 69 2161.29 | 68 2105.61 | 66 1789.27 | + // | 82 | 78 1954.26 | 79 1788.05 | 77 2121.87 | 77 2138.62 | 78 1881.00 | 76 1738.27 | 75 2038.40 | 74 2169.10 | 74 2027.79 | 73 1608.05 | 73 2380.65 | 71 2041.69 | 71 2161.29 | 70 2105.61 | 68 1789.27 | + // | 84 | 80 1954.26 | 81 1788.05 | 79 2121.87 | 79 2138.62 | 80 1881.00 | 78 1738.27 | 77 2038.40 | 76 2169.10 | 76 2027.79 | 75 1608.05 | 75 2380.65 | 73 2041.69 | 73 2161.29 | 72 2105.61 | 70 1789.27 | + // | 86 | 82 1954.26 | 83 1788.05 | 81 2121.87 | 81 2138.62 | 82 1881.00 | 80 1738.27 | 79 2038.40 | 78 2169.10 | 78 2027.79 | 77 1608.05 | 77 2380.65 | 75 2041.69 | 75 2161.29 | 74 2105.61 | 72 1789.27 | + // | 88 | 84 1954.26 | 85 1788.05 | 83 2121.87 | 83 2138.62 | 84 1881.00 | 82 1738.27 | 81 2038.40 | 80 2169.10 | 80 2027.79 | 79 1608.05 | 79 2380.65 | 77 2041.69 | 77 2161.29 | 76 2105.61 | 74 1789.27 | + // | 90 | 86 1954.26 | 87 1788.05 | 85 2121.87 | 85 2138.62 | 86 1881.00 | 84 1738.27 | 83 2038.40 | 82 2169.10 | 82 2027.79 | 81 1608.05 | 81 2380.65 | 79 2041.69 | 79 2161.29 | 78 2105.61 | 76 1789.27 | + // | 92 | 88 1954.26 | 89 1788.05 | 87 2121.87 | 87 2138.62 | 88 1881.00 | 86 1738.27 | 85 2038.40 | 84 2169.10 | 84 2027.79 | 83 1608.05 | 83 2380.65 | 81 2041.69 | 81 2161.29 | 80 2105.61 | 78 1789.27 | + // | 94 | 90 1954.26 | 91 1788.05 | 89 2121.87 | 89 2138.62 | 90 1881.00 | 88 1738.27 | 87 2038.40 | 86 2169.10 | 86 2027.79 | 85 1608.05 | 85 2380.65 | 83 2041.69 | 83 2161.29 | 82 2105.61 | 80 1789.27 | + // | 96 | 92 1954.26 | 93 1788.05 | 91 2121.87 | 91 2138.62 | 92 1881.00 | 90 1738.27 | 89 2038.40 | 88 2169.10 | 88 2027.79 | 87 1608.05 | 87 2380.65 | 85 2041.69 | 85 2161.29 | 84 2105.61 | 82 1789.27 | + // | 98 | 94 1954.26 | 95 1788.05 | 93 2121.87 | 93 2138.62 | 94 1881.00 | 92 1738.27 | 91 2038.40 | 90 2169.10 | 90 2027.79 | 89 1608.05 | 89 2380.65 | 87 2041.69 | 87 2161.29 | 86 2105.61 | 84 1789.27 | + // +-----+------------+------------+------------+------------+------------+------------+------------+------------+------------+------------+------------+------------+------------+------------+------------+ + // Total bytes=709837749, ranges=1369 +} + +func exampleRebalancing( + testStores []testStore, + rebalanceFn func(context.Context, *testStore, []testStore, *Allocator), + printFn func([]testStore, *tablewriter.Table), +) { stopper := stop.NewStopper() defer stopper.Stop(context.Background()) @@ -7464,8 +7814,6 @@ func Example_rebalancing() { tr := tracing.NewTracer() clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) - // Model a set of stores in a cluster, - // adding / rebalancing ranges of random sizes. rpcContext := rpc.NewContext(rpc.ContextOptions{ TenantID: roachpb.SystemTenantID, AmbientCtx: log.AmbientContext{Tracer: tr}, @@ -7479,9 +7827,7 @@ func Example_rebalancing() { TimeUntilStoreDead.Override(ctx, &st.SV, TestTimeUntilStoreDeadOff) - const generations = 100 const nodes = 20 - const printGenerations = generations / 2 // Deterministic must be set as this test is comparing the exact output // after each rebalance. @@ -7509,15 +7855,11 @@ func Example_rebalancing() { gossip.Redundant) // Initialize testStores. - var testStores [nodes]testStore - for i := 0; i < len(testStores); i++ { - testStores[i].immediateCompaction = true - testStores[i].StoreID = roachpb.StoreID(i) - testStores[i].Node = roachpb.NodeDescriptor{NodeID: roachpb.NodeID(i)} - testStores[i].Capacity = roachpb.StoreCapacity{Capacity: 1 << 30, Available: 1 << 30} - } - // Initialize the cluster with a single range. - testStores[0].add(alloc.randGen.Int63n(1 << 20)) + initTestStores( + testStores, + alloc.randGen.Int63n(1<<20), /* firstRangeSize */ + alloc.randGen.Float64()*1e5, /* firstRangeQPS */ + ) table := tablewriter.NewWriter(os.Stdout) table.SetAutoFormatHeaders(false) @@ -7530,15 +7872,20 @@ func Example_rebalancing() { } table.SetHeader(header) + const generations = 100 for i := 0; i < generations; i++ { // First loop through test stores and add data. wg.Add(len(testStores)) for j := 0; j < len(testStores); j++ { // Add a pretend range to the testStore if there's already one. if testStores[j].Capacity.RangeCount > 0 { - testStores[j].add(alloc.randGen.Int63n(1 << 20)) + testStores[j].add(alloc.randGen.Int63n(1<<20), 0) } - if err := g.AddInfoProto(gossip.MakeStoreKey(roachpb.StoreID(j)), &testStores[j].StoreDescriptor, 0); err != nil { + if err := g.AddInfoProto( + gossip.MakeStoreKey(roachpb.StoreID(j)), + &testStores[j].StoreDescriptor, + 0, + ); err != nil { panic(err) } } @@ -7547,37 +7894,13 @@ func Example_rebalancing() { // Next loop through test stores and maybe rebalance. 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, - 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)) + if ts.Capacity.RangeCount == 0 { + continue } + rebalanceFn(ctx, ts, testStores, &alloc) } - if i%(generations/printGenerations) == 0 { - var totalBytes int64 - for j := 0; j < len(testStores); j++ { - totalBytes += testStores[j].Capacity.Capacity - testStores[j].Capacity.Available - } - row := make([]string, len(testStores)+1) - row[0] = fmt.Sprintf("%d", i) - for j := 0; j < len(testStores); j++ { - ts := testStores[j] - bytes := ts.Capacity.Capacity - ts.Capacity.Available - row[j+1] = fmt.Sprintf("%3d %3d%%", ts.Capacity.RangeCount, (100*bytes)/totalBytes) - } - table.Append(row) - } + printFn(testStores, table) } var totBytes int64 @@ -7588,60 +7911,19 @@ func Example_rebalancing() { } table.Render() fmt.Printf("Total bytes=%d, ranges=%d\n", totBytes, totRanges) - // Output: - // +-----+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+ - // | 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 | 3 54% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 2 45% | 0 0% | 0 0% | 0 0% | 0 0% | 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% | 2 20% | 1 8% | 3 20% | 0 0% | 0 0% | 0 0% | 0 0% | 1 15% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | - // | 6 | 3 9% | 0 0% | 0 0% | 1 8% | 0 0% | 1 3% | 3 9% | 3 12% | 3 12% | 0 0% | 1 7% | 0 0% | 2 5% | 3 23% | 0 0% | 0 0% | 2 6% | 0 0% | 0 0% | 0 0% | - // | 8 | 4 5% | 2 5% | 0 0% | 3 9% | 0 0% | 3 5% | 4 9% | 4 5% | 4 7% | 0 0% | 3 7% | 1 3% | 4 10% | 4 11% | 0 0% | 0 0% | 4 11% | 1 3% | 1 3% | 0 0% | - // | 10 | 5 7% | 3 5% | 1 0% | 4 7% | 3 3% | 4 3% | 5 5% | 5 6% | 5 6% | 2 1% | 4 7% | 3 5% | 5 7% | 5 7% | 2 4% | 2 0% | 5 8% | 3 5% | 3 5% | 1 0% | - // | 12 | 7 6% | 5 5% | 3 2% | 6 7% | 5 3% | 6 3% | 7 5% | 7 4% | 7 5% | 4 2% | 6 6% | 5 4% | 7 7% | 7 7% | 4 4% | 4 1% | 7 8% | 5 4% | 5 4% | 3 3% | - // | 14 | 9 5% | 7 5% | 5 2% | 8 6% | 7 4% | 8 5% | 9 5% | 9 5% | 9 5% | 6 3% | 8 5% | 7 3% | 9 6% | 9 6% | 6 4% | 6 3% | 9 7% | 7 5% | 7 4% | 5 3% | - // | 16 | 11 4% | 9 5% | 7 2% | 10 6% | 9 5% | 10 5% | 11 4% | 11 4% | 11 5% | 8 3% | 10 5% | 9 4% | 11 6% | 11 6% | 8 4% | 8 3% | 11 6% | 9 5% | 9 5% | 7 3% | - // | 18 | 13 4% | 11 5% | 9 1% | 12 6% | 11 5% | 12 5% | 13 5% | 13 5% | 13 5% | 10 4% | 12 5% | 11 4% | 13 5% | 13 5% | 10 4% | 10 3% | 13 6% | 11 5% | 11 5% | 9 4% | - // | 20 | 15 5% | 13 5% | 11 2% | 14 5% | 13 4% | 14 5% | 15 4% | 15 4% | 15 5% | 12 4% | 14 5% | 13 5% | 15 5% | 15 5% | 12 4% | 12 3% | 15 6% | 13 4% | 13 5% | 11 4% | - // | 22 | 17 5% | 15 5% | 13 3% | 16 5% | 15 4% | 16 5% | 17 4% | 17 4% | 17 5% | 14 4% | 16 5% | 15 4% | 17 5% | 17 4% | 14 5% | 14 3% | 17 6% | 15 4% | 15 4% | 13 4% | - // | 24 | 19 5% | 17 5% | 15 3% | 18 5% | 17 4% | 18 5% | 19 4% | 19 4% | 19 5% | 16 4% | 18 5% | 17 4% | 19 5% | 19 4% | 16 5% | 16 3% | 19 6% | 17 5% | 17 5% | 15 4% | - // | 26 | 21 4% | 19 5% | 17 3% | 20 6% | 19 4% | 20 5% | 21 4% | 21 4% | 21 4% | 18 4% | 20 5% | 19 4% | 21 5% | 21 4% | 18 5% | 18 3% | 21 6% | 19 4% | 19 5% | 17 4% | - // | 28 | 23 4% | 21 5% | 19 3% | 22 6% | 21 4% | 22 5% | 23 4% | 23 4% | 23 5% | 20 4% | 22 5% | 21 4% | 23 5% | 23 4% | 20 5% | 20 3% | 23 6% | 21 4% | 21 4% | 19 4% | - // | 30 | 25 4% | 23 5% | 21 3% | 24 6% | 23 4% | 24 5% | 25 5% | 25 4% | 25 5% | 22 4% | 24 5% | 23 4% | 25 5% | 25 4% | 22 5% | 22 3% | 25 6% | 23 4% | 23 4% | 21 5% | - // | 32 | 27 4% | 25 5% | 23 3% | 26 6% | 25 4% | 26 5% | 27 5% | 27 4% | 27 5% | 24 4% | 26 5% | 25 4% | 27 5% | 27 4% | 24 4% | 24 3% | 27 6% | 25 4% | 25 5% | 23 4% | - // | 34 | 29 4% | 27 5% | 25 4% | 28 6% | 27 4% | 28 5% | 29 5% | 29 4% | 29 5% | 26 4% | 28 5% | 27 5% | 29 5% | 29 4% | 26 5% | 26 4% | 29 6% | 27 4% | 27 4% | 25 5% | - // | 36 | 31 4% | 29 4% | 27 4% | 30 5% | 29 4% | 30 5% | 31 5% | 31 4% | 31 5% | 28 4% | 30 5% | 29 4% | 31 5% | 31 4% | 28 5% | 28 4% | 31 5% | 29 4% | 29 5% | 27 5% | - // | 38 | 33 4% | 31 5% | 29 3% | 32 5% | 31 4% | 32 5% | 33 5% | 33 4% | 33 5% | 30 4% | 32 5% | 31 4% | 33 5% | 33 5% | 30 4% | 30 4% | 33 5% | 31 4% | 31 4% | 29 5% | - // | 40 | 35 4% | 33 4% | 31 3% | 34 5% | 33 4% | 34 5% | 35 5% | 35 4% | 35 5% | 32 4% | 34 5% | 33 4% | 35 5% | 35 5% | 32 4% | 32 4% | 35 5% | 33 4% | 33 5% | 31 5% | - // | 42 | 37 4% | 35 4% | 33 4% | 36 5% | 35 4% | 36 5% | 37 5% | 37 4% | 37 4% | 34 4% | 36 5% | 35 5% | 37 5% | 37 5% | 34 5% | 34 4% | 37 5% | 35 4% | 35 5% | 33 5% | - // | 44 | 39 4% | 37 4% | 35 4% | 38 5% | 37 4% | 38 5% | 39 5% | 39 4% | 39 4% | 36 4% | 38 5% | 37 5% | 39 5% | 39 5% | 36 5% | 36 4% | 39 5% | 37 4% | 37 5% | 35 5% | - // | 46 | 41 4% | 39 4% | 37 3% | 40 5% | 39 4% | 40 5% | 41 5% | 41 4% | 41 4% | 38 4% | 40 5% | 39 5% | 41 5% | 41 5% | 38 5% | 38 4% | 41 5% | 39 4% | 39 5% | 37 5% | - // | 48 | 43 4% | 41 4% | 39 3% | 42 5% | 41 4% | 42 5% | 43 5% | 43 4% | 43 4% | 40 4% | 42 5% | 41 4% | 43 5% | 43 5% | 40 5% | 40 4% | 43 5% | 41 4% | 41 5% | 39 5% | - // | 50 | 45 4% | 43 4% | 41 4% | 44 5% | 43 4% | 44 5% | 45 5% | 45 4% | 45 4% | 42 4% | 44 5% | 43 4% | 45 5% | 45 5% | 42 5% | 42 4% | 45 5% | 43 5% | 43 5% | 41 5% | - // | 52 | 47 4% | 45 4% | 43 3% | 46 5% | 45 4% | 46 5% | 47 5% | 47 4% | 47 4% | 44 4% | 46 5% | 45 5% | 47 5% | 47 5% | 44 5% | 44 4% | 47 5% | 45 5% | 45 5% | 43 5% | - // | 54 | 49 4% | 47 4% | 45 3% | 48 5% | 47 4% | 48 5% | 49 5% | 49 4% | 49 4% | 46 4% | 48 5% | 47 4% | 49 5% | 49 5% | 46 5% | 46 4% | 49 5% | 47 5% | 47 5% | 45 5% | - // | 56 | 51 4% | 49 5% | 47 3% | 50 5% | 49 4% | 50 5% | 51 5% | 51 4% | 51 4% | 48 4% | 50 5% | 49 4% | 51 5% | 51 5% | 48 5% | 48 4% | 51 5% | 49 5% | 49 4% | 47 5% | - // | 58 | 53 4% | 51 5% | 49 3% | 52 5% | 51 4% | 52 5% | 53 5% | 53 4% | 53 4% | 50 4% | 52 5% | 51 4% | 53 5% | 53 4% | 50 5% | 50 4% | 53 5% | 51 5% | 51 4% | 49 5% | - // | 60 | 55 4% | 53 5% | 51 3% | 54 5% | 53 4% | 54 5% | 55 5% | 55 4% | 55 4% | 52 4% | 54 5% | 53 4% | 55 5% | 55 4% | 52 5% | 52 4% | 55 5% | 53 5% | 53 4% | 51 5% | - // | 62 | 57 4% | 55 5% | 53 3% | 56 5% | 55 4% | 56 5% | 57 5% | 57 4% | 57 4% | 54 4% | 56 5% | 55 4% | 57 5% | 57 4% | 54 5% | 54 4% | 57 5% | 55 5% | 55 4% | 53 4% | - // | 64 | 59 4% | 57 5% | 55 4% | 58 5% | 57 4% | 58 5% | 59 5% | 59 4% | 59 4% | 56 4% | 58 5% | 57 4% | 59 5% | 59 4% | 56 5% | 56 4% | 59 5% | 57 5% | 57 4% | 55 5% | - // | 66 | 61 4% | 59 5% | 57 4% | 60 5% | 59 4% | 60 5% | 61 5% | 61 4% | 61 4% | 58 4% | 60 5% | 59 4% | 61 5% | 61 4% | 58 5% | 58 4% | 61 5% | 59 5% | 59 4% | 57 5% | - // | 68 | 63 4% | 61 5% | 59 4% | 62 5% | 61 4% | 62 5% | 63 5% | 63 4% | 63 4% | 60 4% | 62 5% | 61 4% | 63 5% | 63 4% | 60 5% | 60 4% | 63 5% | 61 5% | 61 4% | 59 5% | - // | 70 | 65 4% | 63 5% | 61 4% | 64 5% | 63 4% | 64 5% | 65 5% | 65 4% | 65 5% | 62 4% | 64 5% | 63 4% | 65 5% | 65 4% | 62 5% | 62 5% | 65 5% | 63 5% | 63 4% | 61 5% | - // | 72 | 67 4% | 65 5% | 63 4% | 66 5% | 65 4% | 66 5% | 67 5% | 67 4% | 67 5% | 64 4% | 66 5% | 65 4% | 67 5% | 67 4% | 64 5% | 64 5% | 67 5% | 65 5% | 65 4% | 63 5% | - // | 74 | 69 4% | 67 5% | 65 4% | 68 5% | 67 4% | 68 5% | 69 5% | 69 4% | 69 4% | 66 4% | 68 5% | 67 5% | 69 5% | 69 4% | 66 5% | 66 5% | 69 5% | 67 5% | 67 4% | 65 5% | - // | 76 | 71 4% | 69 5% | 67 4% | 70 5% | 69 4% | 70 5% | 71 5% | 71 4% | 71 4% | 68 4% | 70 5% | 69 5% | 71 5% | 71 4% | 68 5% | 68 4% | 71 5% | 69 5% | 69 4% | 67 5% | - // | 78 | 73 4% | 71 5% | 69 4% | 72 5% | 71 4% | 72 5% | 73 5% | 73 4% | 73 4% | 70 4% | 72 5% | 71 5% | 73 5% | 73 4% | 70 5% | 70 4% | 73 5% | 71 5% | 71 4% | 69 5% | - // | 80 | 75 4% | 73 4% | 71 4% | 74 5% | 73 4% | 74 5% | 75 5% | 75 4% | 75 4% | 72 5% | 74 5% | 73 4% | 75 5% | 75 4% | 72 5% | 72 4% | 75 5% | 73 5% | 73 4% | 71 5% | - // | 82 | 77 4% | 75 4% | 73 4% | 76 5% | 75 4% | 76 5% | 77 5% | 77 4% | 77 4% | 74 5% | 76 4% | 75 5% | 77 5% | 77 4% | 74 5% | 74 4% | 77 5% | 75 4% | 75 4% | 73 5% | - // | 84 | 79 4% | 77 4% | 75 4% | 78 5% | 77 4% | 78 5% | 79 5% | 79 4% | 79 4% | 76 5% | 78 5% | 77 4% | 79 5% | 79 4% | 76 5% | 76 4% | 79 5% | 77 5% | 77 5% | 75 5% | - // | 86 | 81 4% | 79 4% | 77 4% | 80 5% | 79 4% | 80 5% | 81 5% | 81 4% | 81 4% | 78 5% | 80 5% | 79 5% | 81 5% | 81 4% | 78 5% | 78 4% | 81 5% | 79 4% | 79 4% | 77 5% | - // | 88 | 83 4% | 81 4% | 79 4% | 82 5% | 81 5% | 82 5% | 83 5% | 83 4% | 83 4% | 80 5% | 82 4% | 81 5% | 83 5% | 83 4% | 80 5% | 80 4% | 83 5% | 81 4% | 81 4% | 79 5% | - // | 90 | 85 4% | 83 4% | 81 4% | 84 5% | 83 5% | 84 5% | 85 5% | 85 4% | 85 4% | 82 5% | 84 4% | 83 5% | 85 5% | 85 5% | 82 5% | 82 4% | 85 5% | 83 4% | 83 5% | 81 5% | - // | 92 | 87 4% | 85 4% | 83 4% | 86 5% | 85 5% | 86 5% | 87 5% | 87 4% | 87 4% | 84 5% | 86 5% | 85 5% | 87 5% | 87 5% | 84 5% | 84 4% | 87 5% | 85 4% | 85 4% | 83 5% | - // | 94 | 89 4% | 87 4% | 85 4% | 88 5% | 87 5% | 88 5% | 89 5% | 89 4% | 89 4% | 86 5% | 88 5% | 87 4% | 89 5% | 89 5% | 86 5% | 86 4% | 89 5% | 87 4% | 87 4% | 85 5% | - // | 96 | 91 4% | 89 4% | 87 4% | 90 5% | 89 5% | 90 5% | 91 5% | 91 4% | 91 4% | 88 5% | 90 4% | 89 4% | 91 5% | 91 5% | 88 5% | 88 4% | 91 5% | 89 4% | 89 4% | 87 5% | - // | 98 | 93 4% | 91 4% | 89 4% | 92 5% | 91 5% | 92 5% | 93 5% | 93 4% | 93 4% | 90 5% | 92 4% | 91 4% | 93 5% | 93 5% | 90 5% | 90 4% | 93 5% | 91 5% | 91 4% | 89 5% | - // +-----+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+ - // Total bytes=968307769, ranges=1850 +} + +func initTestStores(testStores []testStore, firstRangeSize int64, firstStoreQPS float64) { + for i := 0; i < len(testStores); i++ { + testStores[i].immediateCompaction = true + testStores[i].StoreID = roachpb.StoreID(i) + testStores[i].Node = roachpb.NodeDescriptor{NodeID: roachpb.NodeID(i)} + testStores[i].Capacity = roachpb.StoreCapacity{ + Capacity: 1 << 30, + Available: 1 << 30, + } + } + + // Initialize the cluster with a single range. + testStores[0].add(firstRangeSize, firstStoreQPS) } diff --git a/pkg/kv/kvserver/store_rebalancer.go b/pkg/kv/kvserver/store_rebalancer.go index d28f86450a8a..a5e4364df24f 100644 --- a/pkg/kv/kvserver/store_rebalancer.go +++ b/pkg/kv/kvserver/store_rebalancer.go @@ -34,9 +34,10 @@ const ( // minQPSThresholdDifference is the minimum QPS difference from the cluster // mean that this system should care about. In other words, we won't worry - // about rebalancing for QPS reasons if a store's QPS differs from the mean - // by less than this amount even if the amount is greater than the percentage - // threshold. This avoids too many lease transfers in lightly loaded clusters. + // about rebalancing for QPS reasons if a store's QPS differs from the mean by + // less than this amount even if the amount is greater than the percentage + // threshold. This avoids too many lease transfers / range rebalances in + // lightly loaded clusters. minQPSThresholdDifference = 100 ) @@ -98,6 +99,28 @@ var qpsRebalanceThreshold = func() *settings.FloatSetting { return s }() +// minQPSDifferenceForTransfers is the minimum QPS difference that the store +// rebalancer would care to reconcile (via lease or replica rebalancing) between +// any two stores. +// +// NB: This value is used to compare the QPS of two stores _without accounting_ +// for the QPS of the replica or lease that is being considered for the +// transfer. This is set to be twice the minimum threshold that a store needs to +// be above or below the mean to be considered overfull or underfull +// respectively. This is to make lease and replica transfers less sensitive to +// the jitters in any given workload. +var minQPSDifferenceForTransfers = func() *settings.FloatSetting { + s := settings.RegisterFloatSetting( + "kv.allocator.min_qps_difference_for_transfers", + "the minimum qps difference that must exist between any two stores"+ + " for the allocator to allow a lease or replica transfer between them", + 2*minQPSThresholdDifference, + settings.NonNegativeFloat, + ) + s.SetVisibility(settings.Reserved) + return s +}() + // LBRebalancingMode controls if and when we do store-level rebalancing // based on load. type LBRebalancingMode int64 @@ -206,10 +229,11 @@ func (sr *StoreRebalancer) Start(ctx context.Context, stopper *stop.Stopper) { // `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 { +func (sr *StoreRebalancer) scorerOptions() qpsScorerOptions { return qpsScorerOptions{ deterministic: sr.rq.allocator.storePool.deterministic, qpsRebalanceThreshold: qpsRebalanceThreshold.Get(&sr.st.SV), + minRequiredQPSDiff: minQPSDifferenceForTransfers.Get(&sr.st.SV), } } @@ -227,13 +251,12 @@ func (sr *StoreRebalancer) rebalanceStore( ctx context.Context, mode LBRebalancingMode, allStoresList StoreList, ) { // First check if we should transfer leases away to better balance QPS. - options, ok := sr.scorerOptions().(qpsScorerOptions) - if !ok { - log.Fatalf(ctx, "expected the `StoreRebalancer` to be using a `qpsScorerOptions`") - } + options := sr.scorerOptions() // We only bother rebalancing stores that are fielding more than the // cluster-level overfull threshold of QPS. - qpsMaxThreshold := overfullQPSThreshold(options, allStoresList.candidateQueriesPerSecond.mean) + qpsMaxThreshold := overfullQPSThreshold( + options.qpsRebalanceThreshold, allStoresList.candidateQueriesPerSecond.mean, + ) var localDesc *roachpb.StoreDescriptor for i := range allStoresList.stores { @@ -508,7 +531,7 @@ func (sr *StoreRebalancer) chooseRangeToRebalance( hottestRanges *[]replicaWithStats, localDesc *roachpb.StoreDescriptor, allStoresList StoreList, - options scorerOptions, + options qpsScorerOptions, ) (replWithStats replicaWithStats, voterTargets, nonVoterTargets []roachpb.ReplicationTarget) { now := sr.rq.store.Clock().NowAsClockTimestamp() for { @@ -572,19 +595,44 @@ func (sr *StoreRebalancer) chooseRangeToRebalance( conf: conf, } + // We ascribe the leaseholder's QPS to every follower replica. The store + // rebalancer first attempts to transfer the leases of its hot ranges away + // in `chooseLeaseToTransfer`. If it cannot move enough leases away to bring + // down the store's QPS below the cluster-level overfullness threshold, it + // moves on to rebalancing replicas. In other words, for every hot range on + // the store, the StoreRebalancer first tries moving the load away to one of + // its existing replicas but then tries to reconfigure the range (i.e. move + // the range to a different set of stores) to _then_ hopefully succeed in + // moving the lease away to another replica. + // + // Thus, we ideally want to base our replica rebalancing on the assumption + // that all of the load from the leaseholder's replica is going to shift to + // the new store that we end up rebalancing to. + options.qpsPerReplica = replWithStats.qps + 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) + log.VEventf( + ctx, + 3, + "considering replica rebalance for r%d with %.2f qps", + replWithStats.repl.GetRangeID(), + replWithStats.qps, + ) - targetVoterRepls, targetNonVoterRepls := sr.getRebalanceTargetsBasedOnQPS( + targetVoterRepls, targetNonVoterRepls, ok := sr.getRebalanceTargetsBasedOnQPS( ctx, rebalanceCtx, options, ) + // Move on to another range if we could not find a better set of replication + // targets for any of the existing replicas. + if !ok { + continue + } storeDescMap := storeListToMap(allStoresList) // Pick the voter with the least QPS to be leaseholder; @@ -623,7 +671,7 @@ func (sr *StoreRebalancer) chooseRangeToRebalance( // the stores in this cluster. func (sr *StoreRebalancer) getRebalanceTargetsBasedOnQPS( ctx context.Context, rbCtx rangeRebalanceContext, options scorerOptions, -) (finalVoterTargets, finalNonVoterTargets []roachpb.ReplicaDescriptor) { +) (finalVoterTargets, finalNonVoterTargets []roachpb.ReplicaDescriptor, ok bool) { finalVoterTargets = rbCtx.rangeDesc.Replicas().VoterDescriptors() finalNonVoterTargets = rbCtx.rangeDesc.Replicas().NonVoterDescriptors() @@ -651,6 +699,9 @@ func (sr *StoreRebalancer) getRebalanceTargetsBasedOnQPS( ) break } + // Record the fact that we've found at least one rebalance opportunity that + // improves QPS balance. + ok = true log.VEventf( ctx, 3, @@ -711,6 +762,9 @@ func (sr *StoreRebalancer) getRebalanceTargetsBasedOnQPS( ) break } + // Record the fact that we've found at least one rebalance opportunity that + // improves QPS balance. + ok = true log.VEventf( ctx, 3, @@ -735,7 +789,7 @@ func (sr *StoreRebalancer) getRebalanceTargetsBasedOnQPS( // Pretend that we've executed upon this rebalancing decision. finalNonVoterTargets = newNonVoters } - return finalVoterTargets, finalNonVoterTargets + return finalVoterTargets, finalNonVoterTargets, ok } func storeListToMap(sl StoreList) map[roachpb.StoreID]*roachpb.StoreDescriptor { diff --git a/pkg/kv/kvserver/store_rebalancer_test.go b/pkg/kv/kvserver/store_rebalancer_test.go index b3de2c1fbe4d..b45b05949e0b 100644 --- a/pkg/kv/kvserver/store_rebalancer_test.go +++ b/pkg/kv/kvserver/store_rebalancer_test.go @@ -207,7 +207,7 @@ var ( StoreID: 2, Node: roachpb.NodeDescriptor{NodeID: 2}, Capacity: roachpb.StoreCapacity{ - QueriesPerSecond: 1100, + QueriesPerSecond: 1300, }, }, { @@ -340,16 +340,21 @@ func TestChooseLeaseToTransfer(t *testing.T) { qps: 100, expectTarget: 0, }, + + // NB: The two cases below expect no lease transfer because the average QPS + // (1300 for stores 1 and 2) is close enough to the current leaseholder's + // QPS (1500). { storeIDs: []roachpb.StoreID{1, 2}, qps: 100, - expectTarget: 2, + expectTarget: 0, }, { - storeIDs: []roachpb.StoreID{1, 3}, - qps: 100, - expectTarget: 3, + storeIDs: []roachpb.StoreID{1, 2}, + qps: 1000, + expectTarget: 0, }, + { storeIDs: []roachpb.StoreID{1, 4}, qps: 100, @@ -366,11 +371,8 @@ func TestChooseLeaseToTransfer(t *testing.T) { expectTarget: 0, }, { - storeIDs: []roachpb.StoreID{1, 2}, - qps: 200, - expectTarget: 2, - }, - { + // s1 is 1500qps, s3 is 1000qps. After the lease transfer, s1 and s3 would + // be projected to have 1300 and 1200 qps respectively. storeIDs: []roachpb.StoreID{1, 3}, qps: 200, expectTarget: 3, @@ -395,10 +397,12 @@ func TestChooseLeaseToTransfer(t *testing.T) { qps: 500, expectTarget: 0, }, + // s1 without the lease would be projected to have 1000 qps, which is close + // enough to s4's 900 qps. { storeIDs: []roachpb.StoreID{1, 4}, qps: 500, - expectTarget: 4, + expectTarget: 0, }, { storeIDs: []roachpb.StoreID{1, 5}, @@ -410,26 +414,41 @@ func TestChooseLeaseToTransfer(t *testing.T) { qps: 600, expectTarget: 5, }, - { - storeIDs: []roachpb.StoreID{1, 5}, - qps: 700, - expectTarget: 5, - }, + + // NB: s1 serves 1500 qps and s5 serves 500. Without the lease, s1 would + // be projected to have 700 qps, which is close enough to 500 that we + // wouldn't expect a lease transfer in this situation. { storeIDs: []roachpb.StoreID{1, 5}, qps: 800, - expectTarget: 5, + expectTarget: 0, }, { storeIDs: []roachpb.StoreID{1, 4, 5}, qps: 800, - expectTarget: 5, + expectTarget: 0, }, { storeIDs: []roachpb.StoreID{1, 3, 4, 5}, qps: 800, + expectTarget: 0, + }, + // If s1 is projected to have 701qps and s5 is projected to have 1299qps, we + // would not transfer the lease because doing so would significantly switch + // the relative dispositions of s1 and s5. + { + storeIDs: []roachpb.StoreID{1, 3, 4, 5}, + qps: 799, + expectTarget: 0, + }, + // NB: However, if s1 is projected to have 750 qps, we would expect a lease + // transfer to s5. + { + storeIDs: []roachpb.StoreID{1, 3, 4, 5}, + qps: 750, expectTarget: 5, }, + { storeIDs: []roachpb.StoreID{1, 4}, qps: 1.5, @@ -691,6 +710,9 @@ func TestChooseRangeToRebalanceAcrossHeterogeneousZones(t *testing.T) { twoReplicasInSecondHottestRegion := []roachpb.ConstraintsConjunction{ constraint("b", 2), } + oneReplicaInColdestRegion := []roachpb.ConstraintsConjunction{ + constraint("c", 1), + } testCases := []struct { name string @@ -706,7 +728,7 @@ func TestChooseRangeToRebalanceAcrossHeterogeneousZones(t *testing.T) { name: "no rebalance", voters: []roachpb.StoreID{3, 6, 9}, constraints: oneReplicaPerRegion, - expRebalancedVoters: []roachpb.StoreID{9, 6, 3}, + expRebalancedVoters: []roachpb.StoreID{}, }, // 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 @@ -738,19 +760,21 @@ func TestChooseRangeToRebalanceAcrossHeterogeneousZones(t *testing.T) { // the same region. { // Within the hottest region, expect rebalance from the hottest node (n1) - // to the coolest node (n3). Within the lease hot region, expect movement - // from n8 to n9. + // to the coolest node (n3). Within the lease hot region, we don't expect + // a rebalance from n8 to n9 because the qps difference between the two + // stores is too small. name: "QPS balance without constraints", voters: []roachpb.StoreID{1, 5, 8}, - expRebalancedVoters: []roachpb.StoreID{9, 5, 3}, + expRebalancedVoters: []roachpb.StoreID{8, 5, 3}, }, { // Within the second hottest region, expect rebalance from the hottest - // node (n4) to the coolest node (n6). Within the lease hot region, expect - // movement from n8 to n9. + // node (n4) to the coolest node (n6). Within the lease hot region, we + // don't expect a rebalance from n8 to n9 because the qps difference + // between the two stores is too small. name: "QPS balance without constraints", voters: []roachpb.StoreID{8, 4, 3}, - expRebalancedVoters: []roachpb.StoreID{9, 6, 3}, + expRebalancedVoters: []roachpb.StoreID{8, 6, 3}, }, // Multi-region database configurations. @@ -788,8 +812,17 @@ func TestChooseRangeToRebalanceAcrossHeterogeneousZones(t *testing.T) { 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}, + // the least hot region. Additionally, in region B, we've got one replica + // on store 4 (which is the hottest store in that region). We expect that + // replica to be moved to store 6. + expRebalancedVoters: []roachpb.StoreID{9, 2, 6, 8, 3}, + }, + { + name: "one voter on sub-optimal node in the coldest region", + voters: []roachpb.StoreID{5,6,7}, + constraints: append(twoReplicasInSecondHottestRegion, oneReplicaInColdestRegion...), + // NB: Expect replica from node 7 to move to node 9. + expRebalancedVoters: []roachpb.StoreID{9, 5, 6}, }, } for _, tc := range testCases { @@ -836,7 +869,7 @@ func TestChooseRangeToRebalanceAcrossHeterogeneousZones(t *testing.T) { 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) + const testingQPS = float64(60) loadRanges( rr, s, []testRange{ {voters: tc.voters, nonVoters: tc.nonVoters, qps: testingQPS}, @@ -878,6 +911,172 @@ func TestChooseRangeToRebalanceAcrossHeterogeneousZones(t *testing.T) { } } +func TestChooseRangeToRebalanceOffHotNodes(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + imbalancedStores := []*roachpb.StoreDescriptor{ + { + StoreID: 1, + Node: roachpb.NodeDescriptor{ + NodeID: 1, + }, + Capacity: roachpb.StoreCapacity{ + QueriesPerSecond: 12000, + }, + }, + { + StoreID: 2, + Node: roachpb.NodeDescriptor{ + NodeID: 2, + }, + Capacity: roachpb.StoreCapacity{ + QueriesPerSecond: 10000, + }, + }, + { + StoreID: 3, + Node: roachpb.NodeDescriptor{ + NodeID: 3, + }, + Capacity: roachpb.StoreCapacity{ + QueriesPerSecond: 8000, + }, + }, + { + StoreID: 4, + Node: roachpb.NodeDescriptor{ + NodeID: 4, + }, + Capacity: roachpb.StoreCapacity{ + QueriesPerSecond: 200, + }, + }, + { + StoreID: 5, + Node: roachpb.NodeDescriptor{ + NodeID: 5, + }, + Capacity: roachpb.StoreCapacity{ + QueriesPerSecond: 100, + }, + }, + } + for _, tc := range []struct { + voters, expRebalancedVoters []roachpb.StoreID + QPS, rebalanceThreshold float64 + shouldRebalance bool + }{ + { + voters: []roachpb.StoreID{1, 2, 3}, + expRebalancedVoters: []roachpb.StoreID{3, 4, 5}, + QPS: 5000, + rebalanceThreshold: 0.25, + shouldRebalance: true, + }, + { + voters: []roachpb.StoreID{1, 2, 3}, + expRebalancedVoters: []roachpb.StoreID{5, 2, 3}, + QPS: 5000, + rebalanceThreshold: 0.8, + shouldRebalance: true, + }, + { + voters: []roachpb.StoreID{1, 2, 3}, + expRebalancedVoters: []roachpb.StoreID{3, 4, 5}, + QPS: 1000, + rebalanceThreshold: 0.05, + shouldRebalance: true, + }, + { + voters: []roachpb.StoreID{1, 2, 3}, + QPS: 5000, + // NB: This will lead to an overfull of just above 12000. Thus, no store + // should be considered overfull and we should not rebalance at all. + rebalanceThreshold: 2, + shouldRebalance: false, + }, + { + voters: []roachpb.StoreID{4}, + QPS: 100, + rebalanceThreshold: 0.01, + // NB: We don't expect a rebalance here because the difference between s4 + // and s5 is not high enough to justify a rebalance. + shouldRebalance: false, + }, + { + voters: []roachpb.StoreID{1, 2, 3}, + QPS: 10000, + rebalanceThreshold: 0.01, + // NB: Rebalancing a replica with 10000qps from s1 to s5 would switch the + // relative dispositions of s1 and s5 significantly. We expect no + // rebalance in this situation, see maxQPSTransferOvershoot. + shouldRebalance: false, + }, + } { + t.Run("", func(t *testing.T) { + stopper, g, _, a, _ := createTestAllocator(10, false /* deterministic */) + defer stopper.Stop(context.Background()) + gossiputil.NewStoreGossiper(g).GossipStores(imbalancedStores, t) + storeList, _, _ := a.storePool.getStoreList(storeFilterThrottled) + + var localDesc roachpb.StoreDescriptor + for _, store := range imbalancedStores { + 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 status + } + + s.cfg.DefaultSpanConfig.NumReplicas = int32(len(tc.voters)) + loadRanges(rr, s, []testRange{{voters: tc.voters, qps: tc.QPS}}) + hottestRanges := rr.topQPS() + _, voterTargets, _ := sr.chooseRangeToRebalance( + ctx, + &hottestRanges, + &localDesc, + storeList, + qpsScorerOptions{deterministic: true, qpsRebalanceThreshold: tc.rebalanceThreshold}, + ) + require.Len(t, voterTargets, len(tc.expRebalancedVoters)) + + voterStoreIDs := make([]roachpb.StoreID, len(voterTargets)) + for i, target := range voterTargets { + voterStoreIDs[i] = target.StoreID + } + require.Equal(t, !tc.shouldRebalance, len(voterStoreIDs) == 0) + if tc.shouldRebalance { + require.ElementsMatch(t, voterStoreIDs, tc.expRebalancedVoters) + } + }) + } +} + func TestNoLeaseTransferToBehindReplicas(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t)