Skip to content

Commit

Permalink
storage: rewrite TestStoreRangeRebalance
Browse files Browse the repository at this point in the history
Rather than the somewhat complicated rebalancing scenario, use a simple
scenario that we perform up-replication of range 1 from 1 to 3 nodes. We
check that this up-replication is performed using preemptive
snapshots. The more complicated scenario was very fragile, frequently
being broken by innocuous changes.

Fixes cockroachdb#10497
Fixes cockroachdb#10193
Fixes cockroachdb#10156
Fixes cockroachdb#9395
  • Loading branch information
petermattis committed Nov 7, 2016
1 parent 375eee3 commit 0e1e31c
Show file tree
Hide file tree
Showing 3 changed files with 13 additions and 90 deletions.
4 changes: 2 additions & 2 deletions pkg/storage/allocator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -518,7 +518,7 @@ func TestAllocatorRebalanceThrashing(t *testing.T) {
for i := range stores {
stores[i].rangeCount = mean
}
surplus := int32(math.Ceil(float64(mean)*RebalanceThreshold + 1))
surplus := int32(math.Ceil(float64(mean)*rebalanceThreshold + 1))
stores[0].rangeCount += surplus
stores[0].shouldRebalanceFrom = true
for i := 1; i < len(stores); i++ {
Expand All @@ -537,7 +537,7 @@ func TestAllocatorRebalanceThrashing(t *testing.T) {
// Subtract enough ranges from the first store to make it a suitable
// rebalance target. To maintain the specified mean, we then add that delta
// back to the rest of the replicas.
deficit := int32(math.Ceil(float64(mean)*RebalanceThreshold + 1))
deficit := int32(math.Ceil(float64(mean)*rebalanceThreshold + 1))
stores[0].rangeCount -= deficit
for i := 1; i < len(stores); i++ {
stores[i].rangeCount += int32(math.Ceil(float64(deficit) / float64(len(stores)-1)))
Expand Down
8 changes: 4 additions & 4 deletions pkg/storage/balancer.go
Original file line number Diff line number Diff line change
Expand Up @@ -152,9 +152,9 @@ func (rcb rangeCountBalancer) improve(sl StoreList, excluded nodeIDSet) *roachpb
return candidate
}

// RebalanceThreshold is the minimum ratio of a store's range surplus to the
// rebalanceThreshold is the minimum ratio of a store's range surplus to the
// mean range count that permits rebalances away from that store.
var RebalanceThreshold = envutil.EnvOrDefaultFloat("COCKROACH_REBALANCE_THRESHOLD", 0.05)
var rebalanceThreshold = envutil.EnvOrDefaultFloat("COCKROACH_REBALANCE_THRESHOLD", 0.05)

func (rangeCountBalancer) shouldRebalance(store roachpb.StoreDescriptor, sl StoreList) bool {
// TODO(peter,bram,cuong): The FractionUsed check seems suspicious. When a
Expand All @@ -164,15 +164,15 @@ func (rangeCountBalancer) shouldRebalance(store roachpb.StoreDescriptor, sl Stor

// Rebalance if we're above the rebalance target, which is
// mean*(1+RebalanceThreshold).
target := int32(math.Ceil(sl.candidateCount.mean * (1 + RebalanceThreshold)))
target := int32(math.Ceil(sl.candidateCount.mean * (1 + rebalanceThreshold)))
rangeCountAboveTarget := store.Capacity.RangeCount > target

// Rebalance if the candidate store has a range count above the mean, and
// there exists another store that is underfull: its range count is smaller
// than mean*(1-RebalanceThreshold).
var rebalanceToUnderfullStore bool
if float64(store.Capacity.RangeCount) > sl.candidateCount.mean {
underfullThreshold := int32(math.Floor(sl.candidateCount.mean * (1 - RebalanceThreshold)))
underfullThreshold := int32(math.Floor(sl.candidateCount.mean * (1 - rebalanceThreshold)))
for _, desc := range sl.stores {
if desc.Capacity.RangeCount < underfullThreshold {
rebalanceToUnderfullStore = true
Expand Down
91 changes: 7 additions & 84 deletions pkg/storage/client_raft_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,6 @@ import (

"github.com/coreos/etcd/raft"
"github.com/coreos/etcd/raft/raftpb"
"github.com/kr/pretty"
"github.com/pkg/errors"
"golang.org/x/net/context"

Expand Down Expand Up @@ -2039,103 +2038,27 @@ func TestStoreRangeRemoveDead(t *testing.T) {
}

// TestStoreRangeRebalance verifies that the replication queue will take
// rebalancing opportunities and add a new replica on another store.
// up-replicate a range when possible.
func TestStoreRangeRebalance(t *testing.T) {
defer leaktest.AfterTest(t)()

// TODO(peter,bram): clean up this test so this isn't required and unexport
// storage.RebalanceThreshold.
defer func(threshold float64) {
storage.RebalanceThreshold = threshold
}(storage.RebalanceThreshold)
storage.RebalanceThreshold = 0

// Start multiTestContext with replica rebalancing enabled.
sc := storage.TestStoreConfig(nil)
sc.AllocatorOptions = storage.AllocatorOptions{AllowRebalance: true}
mtc := &multiTestContext{storeConfig: &sc}

mtc.Start(t, 6)
mtc.Start(t, 3)
defer mtc.Stop()

splitKey := roachpb.Key("split")
splitArgs := adminSplitArgs(roachpb.KeyMin, splitKey)
if _, err := client.SendWrapped(context.Background(), rg1(mtc.stores[0]), &splitArgs); err != nil {
t.Fatal(err)
}

// The setup for this test is to have two ranges like so:
// s1:r1, s2:r1r2, s3:r1, s4:r2, s5:r2, s6:-
// and to rebalance range 1 away from store 2 to store 6:
// s1:r1, s2:r2, s3:r1, s4:r2, s5:r2, s6:r1

replica1 := mtc.stores[0].LookupReplica(roachpb.RKeyMin, nil)
mtc.replicateRange(replica1.Desc().RangeID, 1, 2)

replica2Key := roachpb.RKey(splitKey)
replica2 := mtc.stores[0].LookupReplica(replica2Key, nil)
mtc.replicateRange(replica2.Desc().RangeID, 1, 3, 4)
mtc.unreplicateRange(replica2.Desc().RangeID, 0)

countReplicas := func() map[roachpb.StoreID]int {
counts := make(map[roachpb.StoreID]int)
rangeDescA := getRangeMetadata(roachpb.RKeyMin, mtc, t)
for _, repl := range rangeDescA.Replicas {
counts[repl.StoreID]++
}
rangeDescB := getRangeMetadata(replica2Key, mtc, t)
for _, repl := range rangeDescB.Replicas {
counts[repl.StoreID]++
}
return counts
}

// Check the initial conditions.
expectedStart := map[roachpb.StoreID]int{
roachpb.StoreID(1): 1,
roachpb.StoreID(2): 2,
roachpb.StoreID(3): 1,
roachpb.StoreID(4): 1,
roachpb.StoreID(5): 1,
}
actualStart := countReplicas()
if !reflect.DeepEqual(expectedStart, actualStart) {
t.Fatalf("replicas are not distributed as expected %s", pretty.Diff(expectedStart, actualStart))
}

expected := map[roachpb.StoreID]int{
roachpb.StoreID(1): 1,
roachpb.StoreID(2): 1,
roachpb.StoreID(3): 1,
roachpb.StoreID(4): 1,
roachpb.StoreID(5): 1,
roachpb.StoreID(6): 1,
}

mtc.initGossipNetwork()
util.SucceedsSoon(t, func() error {
// As of this writing, replicas which hold their range's lease cannot
// be removed; forcefully transfer the lease for range 1 to another
// store to allow store 2's replica to be removed.
if err := mtc.transferLease(replica1.RangeID, mtc.stores[0]); err != nil {
t.Fatal(err)
}

// It takes at least two passes to achieve the final result. In the
// first pass, we add the replica to store 6. In the second pass, we
// remove the replica from store 2. Note that it can also take some time
// for the snapshot to arrive.
mtc.stores[0].ForceReplicationScanAndProcess()

// Gossip the stores so that the store pools are up to date. Note that
// there might be a delay between the call below and the asynchronous
// update of the store pools.
mtc.gossipStores()
mtc.stores[0].ForceReplicationScanAndProcess()

// Exit when all stores have a single replica.
actual := countReplicas()
if !reflect.DeepEqual(expected, actual) {
return errors.Errorf("replicas are not distributed as expected %s", pretty.Diff(expected, actual))
const expected = 3
desc := getRangeMetadata(roachpb.RKeyMin, mtc, t)
if actual := len(desc.Replicas); actual != expected {
return errors.Errorf("expected %d replicas, but found %d", expected, actual)
}
return nil
})
Expand Down

0 comments on commit 0e1e31c

Please sign in to comment.