Skip to content

Commit

Permalink
kvserver: pass load dimension to test load ranges
Browse files Browse the repository at this point in the history
Previously `Queries` was hardcoded as the dimension to use when creating
test ranges for use in store rebalancer tests. This patch enables
passing in any `dimension`.

Release note: None
  • Loading branch information
kvoli committed Feb 7, 2023
1 parent cc99062 commit 43ab658
Showing 1 changed file with 10 additions and 10 deletions.
20 changes: 10 additions & 10 deletions pkg/kv/kvserver/store_rebalancer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -436,8 +436,8 @@ type testRange struct {
qps float64
}

func loadRanges(rr *ReplicaRankings, s *Store, ranges []testRange) {
acc := NewReplicaAccumulator(load.Queries)
func loadRanges(rr *ReplicaRankings, s *Store, ranges []testRange, loadDimension load.Dimension) {
acc := NewReplicaAccumulator(loadDimension)
for i, r := range ranges {
rangeID := roachpb.RangeID(i + 1)
repl := &Replica{store: s, RangeID: rangeID}
Expand Down Expand Up @@ -660,7 +660,7 @@ func TestChooseLeaseToTransfer(t *testing.T) {

for _, tc := range testCases {
t.Run("", func(t *testing.T) {
loadRanges(rr, s, []testRange{{voters: tc.storeIDs, qps: tc.qps}})
loadRanges(rr, s, []testRange{{voters: tc.storeIDs, qps: tc.qps}}, load.Queries)
hottestRanges := sr.replicaRankings.TopLoad()
options := sr.scorerOptions(ctx)
rctx := sr.NewRebalanceContext(ctx, options, hottestRanges, LBRebalancingMode(LoadBasedRebalancingMode.Get(&sr.st.SV)))
Expand Down Expand Up @@ -792,7 +792,7 @@ func TestChooseRangeToRebalanceRandom(t *testing.T) {
loadRanges(
rr, s, []testRange{
{voters: voterStores, nonVoters: nonVoterStores, qps: perReplicaQPS},
},
}, load.Queries,
)

hottestRanges := sr.replicaRankings.TopLoad()
Expand Down Expand Up @@ -1128,7 +1128,7 @@ func TestChooseRangeToRebalanceAcrossHeterogeneousZones(t *testing.T) {
loadRanges(
rr, s, []testRange{
{voters: tc.voters, nonVoters: tc.nonVoters, qps: testingQPS},
},
}, load.Queries,
)

hottestRanges := sr.replicaRankings.TopLoad()
Expand Down Expand Up @@ -1208,7 +1208,7 @@ func TestChooseRangeToRebalanceIgnoresRangeOnBestStores(t *testing.T) {
// Load a fake hot range that's already on the best stores. We want to ensure
// that the store rebalancer doesn't attempt to rebalance ranges that it
// cannot find better rebalance opportunities for.
loadRanges(rr, s, []testRange{{voters: []roachpb.StoreID{localDesc.StoreID}, qps: 100}})
loadRanges(rr, s, []testRange{{voters: []roachpb.StoreID{localDesc.StoreID}, qps: 100}}, load.Queries)

hottestRanges := sr.replicaRankings.TopLoad()
options := sr.scorerOptions(ctx)
Expand Down Expand Up @@ -1361,7 +1361,7 @@ func TestChooseRangeToRebalanceOffHotNodes(t *testing.T) {
}

s.cfg.DefaultSpanConfig.NumReplicas = int32(len(tc.voters))
loadRanges(rr, s, []testRange{{voters: tc.voters, qps: tc.QPS}})
loadRanges(rr, s, []testRange{{voters: tc.voters, qps: tc.QPS}}, load.Queries)

hottestRanges := sr.replicaRankings.TopLoad()
options := sr.scorerOptions(ctx)
Expand Down Expand Up @@ -1452,7 +1452,7 @@ func TestNoLeaseTransferToBehindReplicas(t *testing.T) {

// Load in a range with replicas on an overfull node, a slightly underfull
// node, and a very underfull node.
loadRanges(rr, s, []testRange{{voters: []roachpb.StoreID{1, 4, 5}, qps: 100}})
loadRanges(rr, s, []testRange{{voters: []roachpb.StoreID{1, 4, 5}, qps: 100}}, load.Queries)

hottestRanges := sr.replicaRankings.TopLoad()
options := sr.scorerOptions(ctx)
Expand All @@ -1472,7 +1472,7 @@ func TestNoLeaseTransferToBehindReplicas(t *testing.T) {
// Then do the same, but for replica rebalancing. Make s5 an existing replica
// that's behind, and see how a new replica is preferred as the leaseholder
// over it.
loadRanges(rr, s, []testRange{{voters: []roachpb.StoreID{1, 3, 5}, qps: 100}})
loadRanges(rr, s, []testRange{{voters: []roachpb.StoreID{1, 3, 5}, qps: 100}}, load.Queries)

hottestRanges = sr.replicaRankings.TopLoad()
options = sr.scorerOptions(ctx)
Expand Down Expand Up @@ -1641,7 +1641,7 @@ func TestStoreRebalancerReadAmpCheck(t *testing.T) {

// Load in a range with replicas on an overfull node, a slightly underfull
// node, and a very underfull node.
loadRanges(rr, s, []testRange{{voters: []roachpb.StoreID{1, 3, 5}, qps: 100}})
loadRanges(rr, s, []testRange{{voters: []roachpb.StoreID{1, 3, 5}, qps: 100}}, load.Queries)

hottestRanges := sr.replicaRankings.TopLoad()
options := sr.scorerOptions(ctx)
Expand Down

0 comments on commit 43ab658

Please sign in to comment.