From 5a4a926a6136b2c6df372b52033eb18b7554760d Mon Sep 17 00:00:00 2001 From: Bram Gruneir Date: Mon, 21 Nov 2016 13:42:45 -0500 Subject: [PATCH 1/3] storage: collection of fixes and updates for the rules solver - Splits the scores returned in the rule solver into a constraint and balance scores. - Add a valid field to constraints and add it to all rules. - Solve now returns all candidates instead of just the valid ones. To get only the valid candidates, the new function onlyValid and new type condidateList have also been added. - This allows us to use solve for removeTarget. It also cleans up the logic in removeTarget to more closely match the non-rule solver version. - Split the capcity rules into two rules. They were performing two different operations and didnt' make sense being combined. This will also ease the change of converting the rules to basic functions. Part of #10275 --- pkg/storage/allocator.go | 107 ++++++-------------- pkg/storage/rule_solver.go | 144 +++++++++++++++++---------- pkg/storage/rule_solver_test.go | 170 ++++++++++++++++++++++++-------- 3 files changed, 248 insertions(+), 173 deletions(-) diff --git a/pkg/storage/allocator.go b/pkg/storage/allocator.go index 31dcbf4ca9c7..0463013f3c97 100644 --- a/pkg/storage/allocator.go +++ b/pkg/storage/allocator.go @@ -220,16 +220,13 @@ func (a *Allocator) AllocateTarget( return nil, errors.Errorf("%d matching stores are currently throttled", throttledStoreCount) } - candidates, err := allocateRuleSolver.Solve( + candidates := allocateRuleSolver.Solve( sl, constraints, existing, a.storePool.getNodeLocalities(existing), ) - if err != nil { - return nil, err - } - + candidates = candidates.onlyValid() if len(candidates) == 0 { return nil, &allocatorError{ required: constraints.Constraints, @@ -289,52 +286,6 @@ func (a Allocator) RemoveTarget( return roachpb.ReplicaDescriptor{}, errors.Errorf("must supply at least one replica to allocator.RemoveTarget()") } - if a.options.UseRuleSolver { - // TODO(bram): #10275 Is this getStoreList call required? Compute candidate - // requires a store list, but we should be able to create one using only - // the stores that belong to the range. - // Use an invalid range ID as we don't care about a corrupt replicas since - // as we are removing a replica and not trying to add one. - sl, _, _ := a.storePool.getStoreList(roachpb.RangeID(0)) - - var worst roachpb.ReplicaDescriptor - worstScore := math.Inf(0) - for _, exist := range existing { - if exist.StoreID == leaseStoreID { - continue - } - desc, ok := a.storePool.getStoreDescriptor(exist.StoreID) - if !ok { - continue - } - - candidate, valid := removeRuleSolver.computeCandidate(solveState{ - constraints: constraints, - store: desc, - sl: sl, - existing: nil, - existingNodeLocalities: a.storePool.getNodeLocalities(existing), - }) - // When a candidate is not valid, it means that it can be - // considered the worst existing replica. - if !valid { - return exist, nil - } - - if candidate.score < worstScore { - worstScore = candidate.score - worst = exist - } - - } - - if !math.IsInf(worstScore, 0) { - return worst, nil - } - - return roachpb.ReplicaDescriptor{}, errors.New("could not select an appropriate replica to be removed") - } - // Retrieve store descriptors for the provided replicas from the StorePool. descriptors := make([]roachpb.StoreDescriptor, 0, len(existing)) for _, exist := range existing { @@ -345,15 +296,37 @@ func (a Allocator) RemoveTarget( descriptors = append(descriptors, desc) } } - sl := makeStoreList(descriptors) - if bad := a.selectBad(sl); bad != nil { + var badStoreID roachpb.StoreID + + if a.options.UseRuleSolver { + candidates := removeRuleSolver.Solve( + sl, + constraints, + existing, + a.storePool.getNodeLocalities(existing), + ) + + if len(candidates) != 0 { + // TODO(bram): There needs some randomness here and the logic from + // selectBad around rebalanceFromConvergesOnMean. + badStoreID = candidates[len(candidates)-1].store.StoreID + } + } else { + bad := a.selectBad(sl) + if bad != nil { + badStoreID = bad.StoreID + } + } + + if badStoreID != 0 { for _, exist := range existing { - if exist.StoreID == bad.StoreID { + if exist.StoreID == badStoreID { return exist, nil } } } + return roachpb.ReplicaDescriptor{}, errors.New("could not select an appropriate replica to be removed") } @@ -430,31 +403,13 @@ func (a Allocator) RebalanceTarget( existingStoreList := makeStoreList(existingDescs) candidateStoreList := makeStoreList(candidateDescs) - existingCandidates, err := removeRuleSolver.Solve(existingStoreList, constraints, nil, nil) - if err != nil { - return nil, err - } - candidates, err := allocateRuleSolver.Solve(candidateStoreList, constraints, nil, nil) - if err != nil { - return nil, err - } - - // Find all candidates that are better than the worst existing store. - var worstCandidateStore float64 - // If any store from existing is not included in existingCandidates, it - // is because it no longer meets the constraints. If so, its score is - // considered to be 0. - if len(existingCandidates) == len(existing) { - worstCandidateStore = existingCandidates[len(existingCandidates)-1].score - } + existingCandidates := removeRuleSolver.Solve(existingStoreList, constraints, nil, nil) + candidates := allocateRuleSolver.Solve(candidateStoreList, constraints, nil, nil) // TODO(bram): #10275 Need some randomness here! - for _, cand := range candidates { - if cand.score > worstCandidateStore { - return &candidates[0].store, nil - } + if existingCandidates[len(existingCandidates)-1].less(candidates[0]) { + return &candidates[0].store, nil } - return nil, nil } diff --git a/pkg/storage/rule_solver.go b/pkg/storage/rule_solver.go index 62f5a4ede9e7..747d1585ad61 100644 --- a/pkg/storage/rule_solver.go +++ b/pkg/storage/rule_solver.go @@ -23,21 +23,43 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" ) -const ( - ruleDiversityWeight = 0.1 - ruleCapacityWeight = 0.01 -) - // candidate store for allocation. type candidate struct { - store roachpb.StoreDescriptor - score float64 + store roachpb.StoreDescriptor + valid bool + constraint float64 // Score used to pick the top candidates. + balance float64 // Score used to choose between top candidates. +} + +// less first compares constraint scores, then balance scores. +func (c candidate) less(o candidate) bool { + if !o.valid { + return false + } + if !c.valid { + return true + } + if c.constraint != o.constraint { + return c.constraint < o.constraint + } + return c.balance < o.balance +} + +type candidateList []candidate + +// onlyValid returns all the elements in a sorted candidate list that are valid. +func (cl candidateList) onlyValid() candidateList { + for i := len(cl) - 1; i >= 0; i-- { + if cl[i].valid { + return cl[:i+1] + } + } + return candidateList{} } // solveState is used to pass solution state information into a rule. type solveState struct { constraints config.Constraints - store roachpb.StoreDescriptor sl StoreList existing []roachpb.ReplicaDescriptor existingNodeLocalities map[roachpb.NodeID]roachpb.Locality @@ -47,7 +69,7 @@ type solveState struct { // disqualify a store. The store in solveState can be disqualified by // returning false. Unless disqualified, the higher the returned score, the // more likely the store will be picked as a candidate. -type rule func(state solveState) (float64, bool) +type rule func(roachpb.StoreDescriptor, solveState) (bool, float64, float64) // ruleSolver is used to test a collection of rules against stores. type ruleSolver []rule @@ -56,26 +78,28 @@ type ruleSolver []rule var allocateRuleSolver = ruleSolver{ ruleReplicasUniqueNodes, ruleConstraints, - ruleCapacity, + ruleCapacityMax, ruleDiversity, + ruleCapacity, } // removeRuleSolver is the set of rules used for removing existing replicas. var removeRuleSolver = ruleSolver{ ruleConstraints, - ruleCapacity, + ruleCapacityMax, ruleDiversity, + ruleCapacity, } // Solve runs the rules against the stores in the store list and returns all -// passing stores and their scores ordered from best to worst score. +// candidate stores and their scores ordered from best to worst score. func (rs ruleSolver) Solve( sl StoreList, c config.Constraints, existing []roachpb.ReplicaDescriptor, existingNodeLocalities map[roachpb.NodeID]roachpb.Locality, -) ([]candidate, error) { - candidates := make([]candidate, 0, len(sl.stores)) +) candidateList { + candidates := make(candidateList, len(sl.stores), len(sl.stores)) state := solveState{ constraints: c, sl: sl, @@ -83,39 +107,47 @@ func (rs ruleSolver) Solve( existingNodeLocalities: existingNodeLocalities, } - for _, store := range sl.stores { - state.store = store - if cand, ok := rs.computeCandidate(state); ok { - candidates = append(candidates, cand) - } + for i, store := range sl.stores { + candidates[i] = rs.computeCandidate(store, state) } sort.Sort(sort.Reverse(byScore(candidates))) - return candidates, nil + return candidates } // computeCandidate runs the rules against a candidate store using the provided // state and returns each candidate's score and if the candidate is valid. -func (rs ruleSolver) computeCandidate(state solveState) (candidate, bool) { - var totalScore float64 +func (rs ruleSolver) computeCandidate(store roachpb.StoreDescriptor, state solveState) candidate { + var totalConstraintScore, totalBalanceScore float64 for _, rule := range rs { - score, valid := rule(state) + valid, constraintScore, balanceScore := rule(store, state) if !valid { - return candidate{}, false + return candidate{ + store: store, + valid: false, + } } - totalScore += score + totalConstraintScore += constraintScore + totalBalanceScore += balanceScore + } + return candidate{ + store: store, + valid: true, + constraint: totalConstraintScore, + balance: totalBalanceScore, } - return candidate{store: state.store, score: totalScore}, true } // ruleReplicasUniqueNodes returns true iff no existing replica is present on -// the candidate's node. -func ruleReplicasUniqueNodes(state solveState) (float64, bool) { +// the candidate's node. All other scores are always 0. +func ruleReplicasUniqueNodes( + store roachpb.StoreDescriptor, state solveState, +) (bool, float64, float64) { for _, r := range state.existing { - if r.NodeID == state.store.Node.NodeID { - return 0, false + if r.NodeID == store.Node.NodeID { + return false, 0, 0 } } - return 0, true + return true, 0, 0 } // storeHasConstraint returns whether a store descriptor attributes or locality @@ -142,56 +174,60 @@ func storeHasConstraint(store roachpb.StoreDescriptor, c config.Constraint) bool // ruleConstraints returns true iff all required and prohibited constraints are // satisfied. Stores with attributes or localities that match the most positive // constraints return higher scores. -func ruleConstraints(state solveState) (float64, bool) { +func ruleConstraints(store roachpb.StoreDescriptor, state solveState) (bool, float64, float64) { if len(state.constraints.Constraints) == 0 { - return 0, true + return true, 0, 0 } matched := 0 for _, c := range state.constraints.Constraints { - hasConstraint := storeHasConstraint(state.store, c) + hasConstraint := storeHasConstraint(store, c) switch { - case c.Type == config.Constraint_POSITIVE && hasConstraint: - matched++ case c.Type == config.Constraint_REQUIRED && !hasConstraint: - return 0, false + return false, 0, 0 case c.Type == config.Constraint_PROHIBITED && hasConstraint: - return 0, false + return false, 0, 0 + case (c.Type == config.Constraint_POSITIVE && hasConstraint) || + (c.Type == config.Constraint_REQUIRED && hasConstraint) || + (c.Type == config.Constraint_PROHIBITED && !hasConstraint): + matched++ } } - return float64(matched) / float64(len(state.constraints.Constraints)), true + return true, float64(matched) / float64(len(state.constraints.Constraints)), 0 } // ruleDiversity returns higher scores for stores with the fewest locality tiers // in common with already existing replicas. It always returns true. -func ruleDiversity(state solveState) (float64, bool) { +func ruleDiversity(store roachpb.StoreDescriptor, state solveState) (bool, float64, float64) { minScore := 1.0 for _, locality := range state.existingNodeLocalities { - if newScore := state.store.Node.Locality.DiversityScore(locality); newScore < minScore { + if newScore := store.Node.Locality.DiversityScore(locality); newScore < minScore { minScore = newScore } } - return minScore * ruleDiversityWeight, true + return true, minScore, 0 } -// ruleCapacity returns true iff a new replica won't overfill the store. The -// score returned is inversely proportional to the number of ranges on the -// candidate store, with the most empty nodes having the highest scores. -// TODO(bram): consider splitting this into two rules. -func ruleCapacity(state solveState) (float64, bool) { - // Don't overfill stores. - if state.store.Capacity.FractionUsed() > maxFractionUsedThreshold { - return 0, false - } +// ruleCapacity returns a balance score that is inversely proportional to the +// number of ranges on the candidate store such that the most empty store will +// have the highest scores. Scores are always between 0 and 1. +func ruleCapacity(store roachpb.StoreDescriptor, state solveState) (bool, float64, float64) { + return true, 0, 1 / float64(store.Capacity.RangeCount+1) +} - return ruleCapacityWeight / float64(state.store.Capacity.RangeCount+1), true +// ruleCapacityMax ensures that we don't try to overfill a store. +func ruleCapacityMax(store roachpb.StoreDescriptor, state solveState) (bool, float64, float64) { + if store.Capacity.FractionUsed() > maxFractionUsedThreshold { + return false, 0, 0 + } + return true, 0, 0 } // byScore implements sort.Interface to sort by scores. -type byScore []candidate +type byScore candidateList var _ sort.Interface = byScore(nil) func (c byScore) Len() int { return len(c) } -func (c byScore) Less(i, j int) bool { return c[i].score < c[j].score } +func (c byScore) Less(i, j int) bool { return c[i].less(c[j]) } func (c byScore) Swap(i, j int) { c[i], c[j] = c[j], c[i] } diff --git a/pkg/storage/rule_solver_test.go b/pkg/storage/rule_solver_test.go index 9f368c026400..32db4d8b5348 100644 --- a/pkg/storage/rule_solver_test.go +++ b/pkg/storage/rule_solver_test.go @@ -17,6 +17,8 @@ package storage import ( + "bytes" + "fmt" "sort" "testing" @@ -25,17 +27,37 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" ) -type byScoreAndID []candidate +type byScoreAndID candidateList func (c byScoreAndID) Len() int { return len(c) } func (c byScoreAndID) Less(i, j int) bool { - if c[i].score == c[j].score { + if c[i].constraint == c[j].constraint && + c[i].balance == c[j].balance && + c[i].valid == c[j].valid { return c[i].store.StoreID < c[j].store.StoreID } - return c[i].score > c[j].score + return c[i].less(c[j]) } func (c byScoreAndID) Swap(i, j int) { c[i], c[j] = c[j], c[i] } +func (c candidate) String() string { + return fmt.Sprintf("StoreID:%d, valid:%t, con:%.2f, bal:%.2f", + c.store.StoreID, c.valid, c.constraint, c.balance) +} + +func (cl candidateList) String() string { + var buffer bytes.Buffer + buffer.WriteRune('[') + for i, c := range cl { + if i != 0 { + buffer.WriteString("; ") + } + buffer.WriteString(c.String()) + } + buffer.WriteRune(']') + return buffer.String() +} + // TODO(bram): This test suite is not even close to exhaustive. The scores are // not checked and each rule should have many more test cases. Also add a // corrupt replica test and remove the 0 range ID used when calling @@ -111,29 +133,31 @@ func TestRuleSolver(t *testing.T) { storePool.mu.Unlock() testCases := []struct { - name string - rule rule - c config.Constraints - existing []roachpb.ReplicaDescriptor - expected []roachpb.StoreID + name string + rule rule + c config.Constraints + existing []roachpb.ReplicaDescriptor + expectedValid []roachpb.StoreID + expectedInvalid []roachpb.StoreID }{ { - name: "no constraints or rules", - expected: []roachpb.StoreID{storeUSa15, storeUSa1, storeUSb, storeEurope}, + name: "no constraints or rules", + expectedValid: []roachpb.StoreID{storeEurope, storeUSb, storeUSa1, storeUSa15}, }, { name: "white list rule", - rule: func(state solveState) (float64, bool) { - switch state.store.StoreID { + rule: func(store roachpb.StoreDescriptor, _ solveState) (bool, float64, float64) { + switch store.StoreID { case storeUSa15: - return 0, true + return true, 0, 0 case storeUSb: - return 1, true + return true, 1, 0 default: - return 0, false + return false, 0, 0 } }, - expected: []roachpb.StoreID{storeUSb, storeUSa15}, + expectedValid: []roachpb.StoreID{storeUSb, storeUSa15}, + expectedInvalid: []roachpb.StoreID{storeEurope, storeUSa1}, }, { name: "ruleReplicasUniqueNodes - 2 available nodes", @@ -142,7 +166,8 @@ func TestRuleSolver(t *testing.T) { {NodeID: roachpb.NodeID(storeUSa15)}, {NodeID: roachpb.NodeID(storeUSb)}, }, - expected: []roachpb.StoreID{storeUSa1, storeEurope}, + expectedValid: []roachpb.StoreID{storeEurope, storeUSa1}, + expectedInvalid: []roachpb.StoreID{storeUSb, storeUSa15}, }, { name: "ruleReplicasUniqueNodes - 0 available nodes", @@ -153,7 +178,8 @@ func TestRuleSolver(t *testing.T) { {NodeID: roachpb.NodeID(storeUSb)}, {NodeID: roachpb.NodeID(storeEurope)}, }, - expected: nil, + expectedValid: nil, + expectedInvalid: []roachpb.StoreID{storeEurope, storeUSb, storeUSa1, storeUSa15}, }, { name: "ruleConstraints - required constraints", @@ -163,7 +189,8 @@ func TestRuleSolver(t *testing.T) { {Value: "b", Type: config.Constraint_REQUIRED}, }, }, - expected: []roachpb.StoreID{storeUSa1, storeUSb}, + expectedValid: []roachpb.StoreID{storeUSb, storeUSa1}, + expectedInvalid: []roachpb.StoreID{storeEurope, storeUSa15}, }, { name: "ruleConstraints - required locality constraints", @@ -173,7 +200,8 @@ func TestRuleSolver(t *testing.T) { {Key: "datacenter", Value: "us", Type: config.Constraint_REQUIRED}, }, }, - expected: []roachpb.StoreID{storeUSa15, storeUSa1, storeUSb}, + expectedValid: []roachpb.StoreID{storeUSb, storeUSa1, storeUSa15}, + expectedInvalid: []roachpb.StoreID{storeEurope}, }, { name: "ruleConstraints - prohibited constraints", @@ -183,7 +211,8 @@ func TestRuleSolver(t *testing.T) { {Value: "b", Type: config.Constraint_PROHIBITED}, }, }, - expected: []roachpb.StoreID{storeUSa15, storeEurope}, + expectedValid: []roachpb.StoreID{storeEurope, storeUSa15}, + expectedInvalid: []roachpb.StoreID{storeUSb, storeUSa1}, }, { name: "ruleConstraints - prohibited locality constraints", @@ -193,7 +222,8 @@ func TestRuleSolver(t *testing.T) { {Key: "datacenter", Value: "us", Type: config.Constraint_PROHIBITED}, }, }, - expected: []roachpb.StoreID{storeEurope}, + expectedValid: []roachpb.StoreID{storeEurope}, + expectedInvalid: []roachpb.StoreID{storeUSb, storeUSa1, storeUSa15}, }, { name: "ruleConstraints - positive constraints", @@ -205,7 +235,7 @@ func TestRuleSolver(t *testing.T) { {Value: "c"}, }, }, - expected: []roachpb.StoreID{storeUSb, storeUSa1, storeUSa15, storeEurope}, + expectedValid: []roachpb.StoreID{storeUSb, storeUSa1, storeUSa15, storeEurope}, }, { name: "ruleConstraints - positive locality constraints", @@ -215,13 +245,12 @@ func TestRuleSolver(t *testing.T) { {Key: "datacenter", Value: "eur"}, }, }, - expected: []roachpb.StoreID{storeEurope, storeUSa15, storeUSa1, storeUSb}, + expectedValid: []roachpb.StoreID{storeEurope, storeUSb, storeUSa1, storeUSa15}, }, { - name: "ruleDiversity - no existing replicas", - rule: ruleDiversity, - existing: nil, - expected: []roachpb.StoreID{storeUSa15, storeUSa1, storeUSb, storeEurope}, + name: "ruleDiversity - no existing replicas", + rule: ruleDiversity, + expectedValid: []roachpb.StoreID{storeEurope, storeUSb, storeUSa1, storeUSa15}, }, { name: "ruleDiversity - one existing replicas", @@ -229,7 +258,7 @@ func TestRuleSolver(t *testing.T) { existing: []roachpb.ReplicaDescriptor{ {NodeID: roachpb.NodeID(storeUSa15)}, }, - expected: []roachpb.StoreID{storeEurope, storeUSb, storeUSa1, storeUSa15}, + expectedValid: []roachpb.StoreID{storeEurope, storeUSb, storeUSa1, storeUSa15}, }, { name: "ruleDiversity - two existing replicas", @@ -238,12 +267,18 @@ func TestRuleSolver(t *testing.T) { {NodeID: roachpb.NodeID(storeUSa15)}, {NodeID: roachpb.NodeID(storeEurope)}, }, - expected: []roachpb.StoreID{storeUSb, storeUSa1, storeUSa15, storeEurope}, + expectedValid: []roachpb.StoreID{storeUSb, storeUSa1, storeEurope, storeUSa15}, + }, + { + name: "ruleCapacityMax", + rule: ruleCapacityMax, + expectedValid: []roachpb.StoreID{storeEurope, storeUSb, storeUSa1}, + expectedInvalid: []roachpb.StoreID{storeUSa15}, }, { - name: "ruleCapacity", - rule: ruleCapacity, - expected: []roachpb.StoreID{storeUSa1, storeEurope, storeUSb}, + name: "ruleCapacity", + rule: ruleCapacity, + expectedValid: []roachpb.StoreID{storeUSa1, storeEurope, storeUSb, storeUSa15}, }, } @@ -254,25 +289,74 @@ func TestRuleSolver(t *testing.T) { solver = ruleSolver{tc.rule} } sl, _, _ := storePool.getStoreList(roachpb.RangeID(0)) - candidates, err := solver.Solve( + candidates := solver.Solve( sl, tc.c, tc.existing, storePool.getNodeLocalities(tc.existing), ) - if err != nil { - t.Fatal(err) + sort.Sort(sort.Reverse(byScoreAndID(candidates))) + valid := candidates.onlyValid() + invalid := candidates[len(valid):] + + if len(valid) != len(tc.expectedValid) { + t.Fatalf("length of valid %+v should match %+v", valid, tc.expectedValid) + } + for i, expected := range tc.expectedValid { + if actual := valid[i].store.StoreID; actual != expected { + t.Errorf("valid[%d].store.StoreID = %d; not %d; %+v", + i, actual, expected, valid) + } } - sort.Sort(byScoreAndID(candidates)) - if len(candidates) != len(tc.expected) { - t.Fatalf("length of %+v should match %+v", candidates, tc.expected) + if len(invalid) != len(tc.expectedInvalid) { + t.Fatalf("length of invalids %+v should match %+v", invalid, tc.expectedInvalid) } - for i, expected := range tc.expected { - if actual := candidates[i].store.StoreID; actual != expected { - t.Errorf("candidates[%d].store.StoreID = %d; not %d; %+v", - i, actual, expected, candidates) + for i, expected := range tc.expectedInvalid { + if actual := invalid[i].store.StoreID; actual != expected { + t.Errorf("invalid[%d].store.StoreID = %d; not %d; %+v", + i, actual, expected, invalid) } } }) } } + +func TestOnlyValid(t *testing.T) { + defer leaktest.AfterTest(t)() + + testCases := []struct { + valid, invalid int + }{ + {0, 0}, + {1, 0}, + {0, 1}, + {1, 1}, + {2, 0}, + {2, 1}, + {2, 2}, + {1, 2}, + {0, 2}, + } + + for _, tc := range testCases { + t.Run(fmt.Sprintf("%d,%d", tc.valid, tc.invalid), func(t *testing.T) { + var cl candidateList + // Order these in backward to ensure sorting works correctly. + for i := 0; i < tc.invalid; i++ { + cl = append(cl, candidate{}) + } + for i := 0; i < tc.valid; i++ { + cl = append(cl, candidate{valid: true}) + } + sort.Sort(sort.Reverse(byScore(cl))) + + valid := cl.onlyValid() + if a, e := len(valid), tc.valid; a != e { + t.Errorf("expected %d valid, actual %d", e, a) + } + if a, e := len(cl)-len(valid), tc.invalid; a != e { + t.Errorf("expected %d invalid, actual %d", e, a) + } + }) + } +} From 38d06267fbfa198a93a61b9ebafebe00257feee4 Mon Sep 17 00:00:00 2001 From: Bram Gruneir Date: Wed, 30 Nov 2016 15:28:31 -0500 Subject: [PATCH 2/3] storage: add a little randomness into the rule_solver This commit adds the equivalent of the current selectGood and selectBad so that the rule solver will also use the "power of two random choices" method that is currently used by the balancer. --- pkg/storage/allocator.go | 45 +++++-- pkg/storage/allocator_test.go | 103 +++++++------- pkg/storage/rule_solver.go | 181 ++++++++++++++++++++++++- pkg/storage/rule_solver_test.go | 230 +++++++++++++++++++++++++++----- 4 files changed, 461 insertions(+), 98 deletions(-) diff --git a/pkg/storage/allocator.go b/pkg/storage/allocator.go index 0463013f3c97..e812a524efdd 100644 --- a/pkg/storage/allocator.go +++ b/pkg/storage/allocator.go @@ -23,6 +23,8 @@ import ( "math" "math/rand" + "github.com/coreos/etcd/raft" + "github.com/pkg/errors" "golang.org/x/net/context" "github.com/cockroachdb/cockroach/pkg/config" @@ -31,8 +33,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/syncutil" - "github.com/coreos/etcd/raft" - "github.com/pkg/errors" ) const ( @@ -225,6 +225,7 @@ func (a *Allocator) AllocateTarget( constraints, existing, a.storePool.getNodeLocalities(existing), + a.storePool.deterministic, ) candidates = candidates.onlyValid() if len(candidates) == 0 { @@ -232,8 +233,8 @@ func (a *Allocator) AllocateTarget( required: constraints.Constraints, } } - // TODO(bram): #10275 Need some randomness here! - return &candidates[0].store, nil + chosenCandidate := candidates.selectGood(a.randGen).store + return &chosenCandidate, nil } existingNodes := make(nodeIDSet, len(existing)) @@ -305,12 +306,10 @@ func (a Allocator) RemoveTarget( constraints, existing, a.storePool.getNodeLocalities(existing), + a.storePool.deterministic, ) - if len(candidates) != 0 { - // TODO(bram): There needs some randomness here and the logic from - // selectBad around rebalanceFromConvergesOnMean. - badStoreID = candidates[len(candidates)-1].store.StoreID + badStoreID = candidates.selectBad(a.randGen).store.StoreID } } else { bad := a.selectBad(sl) @@ -403,12 +402,32 @@ func (a Allocator) RebalanceTarget( existingStoreList := makeStoreList(existingDescs) candidateStoreList := makeStoreList(candidateDescs) - existingCandidates := removeRuleSolver.Solve(existingStoreList, constraints, nil, nil) - candidates := allocateRuleSolver.Solve(candidateStoreList, constraints, nil, nil) + localities := a.storePool.getNodeLocalities(existing) + existingCandidates := rebalanceExisting.Solve( + existingStoreList, + constraints, + existing, + localities, + a.storePool.deterministic, + ) + candidates := rebalance.Solve( + candidateStoreList, + constraints, + existing, + localities, + a.storePool.deterministic, + ) + + /* + fmt.Printf("existing: %s\n", existingCandidates) + fmt.Printf("candidates: %s\n", candidates) + */ - // TODO(bram): #10275 Need some randomness here! - if existingCandidates[len(existingCandidates)-1].less(candidates[0]) { - return &candidates[0].store, nil + // Find all candidates that are better than the worst existing. + targets := candidates.betterThan(existingCandidates[len(existingCandidates)-1]) + if len(targets) != 0 { + target := targets.selectGood(a.randGen).store + return &target, nil } return nil, nil } diff --git a/pkg/storage/allocator_test.go b/pkg/storage/allocator_test.go index 45f8e4ae3acc..8d8bff0c477e 100644 --- a/pkg/storage/allocator_test.go +++ b/pkg/storage/allocator_test.go @@ -720,6 +720,7 @@ func TestAllocatorRebalance(t *testing.T) { if err != nil { t.Fatal(err) } + fmt.Printf("%d: %s\n", i, result) if result == nil { i-- // loop until we find 10 candidates continue @@ -1927,56 +1928,56 @@ func Example_rebalancingWithRuleSolver() { // +-----+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+ // | 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 | 1 88% | 1 11% | 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 | 1 32% | 2 24% | 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 21% | 2 20% | - // | 4 | 2 9% | 2 22% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 3 5% | 1 0% | 2 9% | 2 19% | 2 21% | 3 9% | - // | 6 | 3 5% | 2 5% | 0 0% | 0 0% | 0 0% | 0 0% | 6 15% | 1 3% | 0 0% | 0 0% | 0 0% | 0 0% | 2 4% | 7 27% | 3 1% | 2 3% | 2 6% | 2 10% | 2 9% | 3 6% | - // | 8 | 3 4% | 3 5% | 0 0% | 0 0% | 0 0% | 0 0% | 6 9% | 3 2% | 7 8% | 7 6% | 3 7% | 0 0% | 3 4% | 7 17% | 4 3% | 3 2% | 3 6% | 3 9% | 3 6% | 3 5% | - // | 10 | 5 3% | 5 4% | 0 0% | 0 0% | 5 4% | 5 6% | 6 6% | 5 4% | 7 5% | 7 5% | 5 8% | 0 0% | 5 5% | 7 11% | 5 3% | 5 4% | 5 5% | 5 9% | 5 5% | 5 5% | - // | 12 | 6 3% | 6 3% | 5 5% | 15 14% | 6 3% | 6 6% | 6 4% | 6 3% | 7 2% | 7 3% | 6 7% | 3 1% | 6 4% | 7 7% | 6 3% | 6 2% | 6 3% | 6 7% | 6 4% | 6 5% | - // | 14 | 8 4% | 8 4% | 8 6% | 15 10% | 8 2% | 8 4% | 8 5% | 8 4% | 8 3% | 8 2% | 8 6% | 9 5% | 8 3% | 8 5% | 8 4% | 8 3% | 8 4% | 8 7% | 8 4% | 8 6% | - // | 16 | 11 5% | 10 4% | 10 5% | 15 8% | 10 3% | 10 5% | 10 4% | 10 4% | 10 3% | 10 2% | 10 5% | 11 4% | 10 4% | 10 5% | 10 4% | 10 4% | 10 4% | 10 6% | 10 4% | 11 6% | - // | 18 | 13 4% | 12 4% | 12 6% | 15 7% | 12 3% | 12 5% | 12 4% | 12 4% | 12 4% | 12 2% | 12 6% | 13 5% | 12 4% | 12 5% | 12 5% | 12 4% | 13 5% | 13 6% | 12 4% | 13 6% | - // | 20 | 15 4% | 14 4% | 14 6% | 16 6% | 14 3% | 14 5% | 14 5% | 14 4% | 14 4% | 14 3% | 14 6% | 15 4% | 14 4% | 14 4% | 14 5% | 15 4% | 15 5% | 15 6% | 14 4% | 15 5% | - // | 22 | 17 4% | 16 4% | 16 6% | 18 6% | 16 3% | 16 5% | 16 5% | 16 4% | 16 4% | 16 3% | 16 6% | 17 4% | 16 4% | 16 4% | 16 5% | 17 4% | 17 5% | 17 6% | 16 5% | 17 5% | - // | 24 | 19 4% | 18 4% | 18 6% | 20 6% | 18 3% | 18 5% | 18 5% | 18 4% | 18 4% | 18 3% | 18 5% | 19 4% | 18 4% | 18 4% | 18 5% | 19 4% | 19 5% | 19 6% | 18 5% | 19 5% | - // | 26 | 21 4% | 20 4% | 20 5% | 22 5% | 20 3% | 20 5% | 20 4% | 20 4% | 20 4% | 20 4% | 20 5% | 21 5% | 20 4% | 20 4% | 20 5% | 21 3% | 21 5% | 21 6% | 20 5% | 21 5% | - // | 28 | 23 4% | 22 4% | 22 5% | 24 5% | 22 3% | 22 5% | 22 5% | 22 4% | 22 4% | 22 4% | 22 5% | 23 5% | 22 4% | 22 4% | 22 5% | 23 3% | 23 5% | 23 5% | 22 5% | 23 5% | - // | 30 | 25 4% | 24 4% | 24 5% | 26 5% | 24 3% | 24 5% | 24 4% | 24 4% | 24 4% | 24 4% | 24 5% | 25 4% | 24 4% | 24 5% | 24 5% | 25 4% | 25 5% | 25 6% | 24 5% | 25 5% | - // | 32 | 27 4% | 26 5% | 26 5% | 28 5% | 26 3% | 26 5% | 26 4% | 26 4% | 26 4% | 26 4% | 26 5% | 27 4% | 26 4% | 26 4% | 26 5% | 27 4% | 27 5% | 27 5% | 26 5% | 27 4% | - // | 34 | 29 4% | 28 4% | 28 5% | 30 5% | 28 3% | 28 5% | 28 4% | 28 4% | 28 4% | 28 4% | 28 5% | 29 5% | 28 4% | 28 4% | 28 5% | 29 4% | 29 5% | 29 5% | 28 5% | 29 5% | - // | 36 | 31 4% | 30 4% | 30 5% | 32 5% | 30 3% | 30 5% | 30 4% | 30 4% | 30 4% | 30 5% | 30 5% | 31 4% | 30 4% | 30 4% | 30 5% | 31 4% | 31 5% | 31 5% | 30 5% | 31 5% | - // | 38 | 33 4% | 32 4% | 32 5% | 34 5% | 32 4% | 32 4% | 32 4% | 32 4% | 32 4% | 32 5% | 32 5% | 33 4% | 32 5% | 32 4% | 32 5% | 33 4% | 33 6% | 33 5% | 32 5% | 33 5% | - // | 40 | 35 4% | 34 5% | 34 5% | 36 5% | 34 4% | 34 5% | 34 4% | 34 4% | 34 4% | 34 5% | 34 5% | 35 4% | 34 5% | 34 4% | 34 5% | 35 4% | 35 6% | 35 5% | 34 5% | 35 5% | - // | 42 | 37 4% | 36 4% | 36 5% | 38 5% | 36 4% | 36 5% | 36 4% | 36 4% | 36 4% | 36 4% | 36 5% | 37 5% | 36 5% | 36 4% | 36 5% | 37 4% | 37 5% | 37 5% | 36 5% | 37 5% | - // | 44 | 39 4% | 38 4% | 38 5% | 40 5% | 38 4% | 38 5% | 38 4% | 38 4% | 38 4% | 38 4% | 38 5% | 39 4% | 38 5% | 38 4% | 38 5% | 39 4% | 39 5% | 39 5% | 38 5% | 39 5% | - // | 46 | 41 4% | 40 4% | 40 5% | 42 5% | 40 4% | 40 5% | 40 4% | 40 4% | 40 4% | 40 4% | 40 5% | 41 4% | 40 5% | 40 4% | 40 5% | 41 4% | 41 5% | 41 5% | 40 5% | 41 5% | - // | 48 | 43 4% | 42 4% | 42 5% | 44 5% | 42 4% | 42 5% | 42 5% | 42 4% | 42 4% | 42 4% | 42 5% | 43 4% | 42 5% | 42 4% | 42 5% | 43 4% | 43 5% | 43 5% | 42 5% | 43 5% | - // | 50 | 45 4% | 44 4% | 44 5% | 46 5% | 44 4% | 44 5% | 44 5% | 44 4% | 44 4% | 44 4% | 44 5% | 45 4% | 44 5% | 44 4% | 44 4% | 45 4% | 45 5% | 45 5% | 44 5% | 45 5% | - // | 52 | 47 4% | 46 4% | 46 5% | 48 5% | 46 4% | 46 5% | 46 5% | 46 4% | 46 4% | 46 4% | 46 5% | 47 5% | 46 5% | 46 4% | 46 4% | 47 4% | 47 5% | 47 5% | 46 5% | 47 5% | - // | 54 | 49 4% | 48 4% | 48 5% | 50 5% | 48 4% | 48 5% | 48 5% | 48 4% | 48 4% | 48 4% | 48 5% | 49 5% | 48 5% | 48 4% | 48 4% | 49 4% | 49 5% | 49 5% | 48 5% | 49 5% | - // | 56 | 51 4% | 50 4% | 50 5% | 52 5% | 50 4% | 50 5% | 50 5% | 50 5% | 50 4% | 50 4% | 50 5% | 51 5% | 50 5% | 50 4% | 50 4% | 51 4% | 51 5% | 51 5% | 50 5% | 51 5% | - // | 58 | 53 4% | 52 4% | 52 5% | 54 5% | 52 4% | 52 5% | 52 5% | 52 5% | 52 4% | 52 4% | 52 5% | 53 5% | 52 5% | 52 4% | 52 4% | 53 4% | 53 5% | 53 5% | 52 5% | 53 5% | - // | 60 | 55 4% | 54 4% | 54 5% | 56 5% | 54 4% | 54 5% | 54 5% | 54 5% | 54 4% | 54 4% | 54 5% | 55 5% | 54 5% | 54 4% | 54 4% | 55 4% | 55 5% | 55 5% | 54 5% | 55 5% | - // | 62 | 57 4% | 56 4% | 56 5% | 58 5% | 56 4% | 56 5% | 56 5% | 56 5% | 56 4% | 56 4% | 56 5% | 57 5% | 56 5% | 56 4% | 56 5% | 57 4% | 57 5% | 57 5% | 56 5% | 57 5% | - // | 64 | 59 4% | 58 4% | 58 5% | 60 5% | 58 4% | 58 5% | 58 4% | 58 5% | 58 4% | 58 4% | 58 5% | 59 4% | 58 5% | 58 4% | 58 5% | 59 4% | 59 5% | 59 5% | 58 5% | 59 5% | - // | 66 | 61 4% | 60 4% | 60 5% | 62 5% | 60 4% | 60 5% | 60 4% | 60 5% | 60 4% | 60 4% | 60 5% | 61 4% | 60 5% | 60 4% | 60 5% | 61 4% | 61 5% | 61 5% | 60 5% | 61 5% | - // | 68 | 63 4% | 62 4% | 62 5% | 64 5% | 62 4% | 62 5% | 62 4% | 62 5% | 62 4% | 62 4% | 62 5% | 63 4% | 62 5% | 62 4% | 62 5% | 63 4% | 63 5% | 63 5% | 62 5% | 63 5% | - // | 70 | 65 4% | 64 4% | 64 5% | 66 5% | 64 4% | 64 5% | 64 4% | 64 5% | 64 5% | 64 4% | 64 5% | 65 4% | 64 5% | 64 4% | 64 5% | 65 4% | 65 5% | 65 5% | 64 5% | 65 5% | - // | 72 | 67 4% | 66 4% | 66 5% | 68 5% | 66 4% | 66 5% | 66 4% | 66 5% | 66 5% | 66 4% | 66 5% | 67 4% | 66 5% | 66 4% | 66 5% | 67 4% | 67 5% | 67 5% | 66 5% | 67 5% | - // | 74 | 69 4% | 68 4% | 68 5% | 70 5% | 68 4% | 68 5% | 68 4% | 68 5% | 68 5% | 68 4% | 68 5% | 69 4% | 68 5% | 68 4% | 68 5% | 69 4% | 69 5% | 69 5% | 68 5% | 69 5% | - // | 76 | 71 4% | 70 4% | 70 5% | 72 5% | 70 4% | 70 5% | 70 4% | 70 4% | 70 5% | 70 4% | 70 5% | 71 4% | 70 5% | 70 4% | 70 5% | 71 4% | 71 5% | 71 5% | 70 5% | 71 5% | - // | 78 | 73 4% | 72 4% | 72 5% | 74 5% | 72 4% | 72 5% | 72 4% | 72 4% | 72 5% | 72 4% | 72 5% | 73 4% | 72 5% | 72 4% | 72 5% | 73 4% | 73 5% | 73 5% | 72 5% | 73 5% | - // | 80 | 75 4% | 74 4% | 74 5% | 76 5% | 74 4% | 74 5% | 74 4% | 74 5% | 74 5% | 74 4% | 74 5% | 75 4% | 74 5% | 74 4% | 74 5% | 75 4% | 75 5% | 75 5% | 74 5% | 75 5% | - // | 82 | 77 4% | 76 4% | 76 5% | 78 5% | 76 4% | 76 5% | 76 4% | 76 5% | 76 5% | 76 4% | 76 5% | 77 4% | 76 5% | 76 4% | 76 5% | 77 4% | 77 5% | 77 5% | 76 5% | 77 5% | - // | 84 | 79 4% | 78 4% | 78 5% | 80 5% | 78 4% | 78 5% | 78 4% | 78 4% | 78 5% | 78 4% | 78 5% | 79 4% | 78 5% | 78 4% | 78 5% | 79 4% | 79 5% | 79 5% | 78 5% | 79 5% | - // | 86 | 81 4% | 80 4% | 80 5% | 82 5% | 80 4% | 80 5% | 80 4% | 80 4% | 80 4% | 80 4% | 80 5% | 81 4% | 80 5% | 80 4% | 80 5% | 81 4% | 81 5% | 81 5% | 80 5% | 81 5% | - // | 88 | 83 4% | 82 4% | 82 5% | 84 5% | 82 4% | 82 5% | 82 4% | 82 5% | 82 4% | 82 4% | 82 5% | 83 4% | 82 5% | 82 4% | 82 5% | 83 4% | 83 5% | 83 5% | 82 5% | 83 5% | - // | 90 | 85 4% | 84 4% | 84 5% | 86 5% | 84 4% | 84 5% | 84 4% | 84 4% | 84 4% | 84 4% | 84 5% | 85 4% | 84 5% | 84 4% | 84 4% | 85 4% | 85 5% | 85 5% | 84 5% | 85 5% | - // | 92 | 87 4% | 86 4% | 86 5% | 88 5% | 86 4% | 86 5% | 86 4% | 86 4% | 86 4% | 86 4% | 86 5% | 87 4% | 86 5% | 86 4% | 86 4% | 87 4% | 87 5% | 87 5% | 86 5% | 87 5% | - // | 94 | 89 4% | 88 4% | 88 5% | 90 5% | 88 4% | 88 5% | 88 4% | 88 5% | 88 4% | 88 4% | 88 5% | 89 4% | 88 5% | 88 4% | 88 4% | 89 4% | 89 5% | 89 5% | 88 5% | 89 5% | - // | 96 | 91 4% | 90 4% | 90 5% | 92 5% | 90 4% | 90 5% | 90 4% | 90 5% | 90 4% | 90 4% | 90 5% | 91 4% | 90 5% | 90 4% | 90 4% | 91 4% | 91 5% | 91 5% | 90 5% | 91 5% | - // | 98 | 93 4% | 92 4% | 92 5% | 94 5% | 92 4% | 92 5% | 92 4% | 92 4% | 92 4% | 92 4% | 92 5% | 93 4% | 92 5% | 92 4% | 92 4% | 93 4% | 93 5% | 93 5% | 92 5% | 93 5% | + // | 0 | 1 48% | 0 0% | 0 0% | 1 51% | 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 | 2 35% | 0 0% | 0 0% | 2 24% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 1 11% | 0 0% | 1 14% | 0 0% | 1 13% | 0 0% | 0 0% | 0 0% | + // | 4 | 2 19% | 0 0% | 0 0% | 2 8% | 0 0% | 0 0% | 1 7% | 2 11% | 0 0% | 1 1% | 2 9% | 0 0% | 2 3% | 1 1% | 2 9% | 1 9% | 2 4% | 0 0% | 2 13% | 0 0% | + // | 6 | 2 10% | 3 5% | 3 6% | 2 1% | 3 7% | 3 3% | 3 3% | 2 7% | 2 1% | 2 3% | 2 7% | 4 5% | 3 2% | 2 2% | 2 2% | 2 4% | 2 0% | 3 10% | 2 8% | 2 1% | + // | 8 | 4 5% | 5 6% | 5 4% | 4 2% | 5 6% | 5 5% | 5 6% | 4 7% | 4 2% | 4 4% | 5 6% | 5 2% | 5 3% | 4 4% | 4 3% | 4 4% | 4 3% | 5 8% | 4 6% | 4 3% | + // | 10 | 6 5% | 7 6% | 7 3% | 6 3% | 7 5% | 7 5% | 7 5% | 6 7% | 6 3% | 6 5% | 7 6% | 7 3% | 7 3% | 6 4% | 6 4% | 6 4% | 6 2% | 7 8% | 6 5% | 6 4% | + // | 12 | 8 5% | 9 5% | 9 5% | 8 3% | 9 5% | 9 4% | 9 5% | 8 6% | 8 4% | 8 4% | 9 7% | 9 3% | 9 3% | 8 4% | 8 4% | 8 4% | 8 3% | 9 7% | 8 5% | 8 5% | + // | 14 | 10 4% | 11 6% | 11 4% | 10 3% | 11 5% | 11 4% | 11 6% | 10 6% | 10 5% | 10 4% | 11 6% | 11 4% | 11 3% | 10 4% | 10 3% | 10 4% | 10 3% | 11 7% | 10 5% | 10 4% | + // | 16 | 12 4% | 13 6% | 13 4% | 12 3% | 13 5% | 13 3% | 13 5% | 12 6% | 12 5% | 12 5% | 13 6% | 13 4% | 13 3% | 12 4% | 12 4% | 12 4% | 12 3% | 13 6% | 12 4% | 12 5% | + // | 18 | 14 4% | 15 6% | 15 4% | 14 3% | 15 5% | 15 4% | 15 5% | 14 6% | 14 5% | 14 4% | 15 6% | 15 4% | 15 3% | 14 5% | 14 4% | 14 5% | 14 4% | 15 6% | 14 4% | 14 4% | + // | 20 | 16 4% | 17 6% | 17 4% | 16 4% | 17 5% | 17 3% | 17 5% | 16 5% | 16 6% | 16 4% | 17 5% | 17 5% | 17 3% | 16 5% | 16 4% | 16 5% | 16 4% | 17 6% | 16 4% | 16 4% | + // | 22 | 18 4% | 19 6% | 19 4% | 18 4% | 19 5% | 19 4% | 19 5% | 18 5% | 18 5% | 18 4% | 19 5% | 19 5% | 19 3% | 18 5% | 18 4% | 18 4% | 18 4% | 19 6% | 18 4% | 18 4% | + // | 24 | 20 5% | 21 5% | 21 4% | 20 4% | 21 5% | 21 3% | 21 4% | 20 5% | 20 5% | 20 4% | 21 6% | 21 5% | 21 3% | 20 4% | 20 4% | 20 4% | 20 4% | 21 6% | 20 5% | 20 4% | + // | 26 | 22 5% | 23 5% | 23 4% | 22 4% | 23 5% | 23 3% | 23 5% | 22 5% | 22 4% | 22 4% | 23 5% | 23 5% | 23 3% | 22 4% | 22 4% | 22 5% | 22 4% | 23 5% | 22 5% | 22 4% | + // | 28 | 24 5% | 25 5% | 25 4% | 24 4% | 25 5% | 25 3% | 25 4% | 24 5% | 24 5% | 24 4% | 25 5% | 25 5% | 25 3% | 24 4% | 24 4% | 24 5% | 24 4% | 25 5% | 24 5% | 24 4% | + // | 30 | 26 5% | 27 5% | 27 4% | 26 4% | 27 5% | 27 4% | 27 4% | 26 5% | 26 5% | 26 4% | 27 5% | 27 5% | 27 3% | 26 4% | 26 4% | 26 5% | 26 4% | 27 5% | 26 5% | 26 5% | + // | 32 | 28 5% | 29 5% | 29 4% | 28 4% | 29 5% | 29 4% | 29 4% | 28 5% | 28 4% | 28 4% | 29 5% | 29 5% | 29 3% | 28 4% | 28 4% | 28 5% | 28 4% | 29 5% | 28 5% | 28 5% | + // | 34 | 30 5% | 31 4% | 31 4% | 30 4% | 31 5% | 31 4% | 31 4% | 30 4% | 30 5% | 30 4% | 31 5% | 31 5% | 31 3% | 30 4% | 30 4% | 30 4% | 30 4% | 31 5% | 30 5% | 30 5% | + // | 36 | 32 5% | 33 5% | 33 4% | 32 4% | 33 5% | 33 4% | 33 4% | 32 5% | 32 4% | 32 4% | 33 5% | 33 5% | 33 3% | 32 4% | 32 4% | 32 4% | 32 4% | 33 5% | 32 5% | 32 5% | + // | 38 | 34 5% | 35 5% | 35 5% | 34 4% | 35 5% | 35 4% | 35 4% | 34 5% | 34 5% | 34 4% | 35 5% | 35 5% | 35 3% | 34 4% | 34 4% | 34 4% | 34 4% | 35 5% | 34 5% | 34 5% | + // | 40 | 36 5% | 37 5% | 37 5% | 36 4% | 37 5% | 37 4% | 37 4% | 36 4% | 36 5% | 36 4% | 37 5% | 37 5% | 37 3% | 36 4% | 36 4% | 36 5% | 36 4% | 37 5% | 36 5% | 36 5% | + // | 42 | 38 5% | 39 5% | 39 5% | 38 4% | 39 5% | 39 4% | 39 4% | 38 4% | 38 5% | 38 4% | 39 5% | 39 5% | 39 3% | 38 4% | 38 5% | 38 5% | 38 4% | 39 5% | 38 5% | 38 5% | + // | 44 | 40 5% | 41 5% | 41 5% | 40 4% | 41 5% | 41 4% | 41 5% | 40 4% | 40 5% | 40 4% | 41 5% | 41 5% | 41 3% | 40 4% | 40 5% | 40 5% | 40 4% | 41 5% | 40 5% | 40 5% | + // | 46 | 42 5% | 43 5% | 43 5% | 42 4% | 43 5% | 43 4% | 43 5% | 42 4% | 42 5% | 42 4% | 43 5% | 43 5% | 43 4% | 42 4% | 42 5% | 42 5% | 42 4% | 43 5% | 42 4% | 42 5% | + // | 48 | 44 5% | 45 5% | 45 5% | 44 4% | 45 5% | 45 4% | 45 5% | 44 4% | 44 5% | 44 4% | 45 5% | 45 5% | 45 4% | 44 4% | 44 4% | 44 5% | 44 4% | 45 5% | 44 4% | 44 5% | + // | 50 | 46 5% | 47 5% | 47 5% | 46 4% | 47 5% | 47 4% | 47 5% | 46 4% | 46 5% | 46 4% | 47 5% | 47 5% | 47 4% | 46 4% | 46 4% | 46 5% | 46 4% | 47 5% | 46 5% | 46 5% | + // | 52 | 48 5% | 49 5% | 49 5% | 48 4% | 49 5% | 49 4% | 49 5% | 48 4% | 48 5% | 48 4% | 49 5% | 49 5% | 49 4% | 48 4% | 48 5% | 48 5% | 48 4% | 49 5% | 48 5% | 48 5% | + // | 54 | 50 5% | 51 5% | 51 5% | 50 4% | 51 5% | 51 4% | 51 5% | 50 4% | 50 5% | 50 4% | 51 5% | 51 5% | 51 4% | 50 4% | 50 4% | 50 5% | 50 4% | 51 5% | 50 5% | 50 5% | + // | 56 | 52 5% | 53 4% | 53 5% | 52 4% | 53 5% | 53 4% | 53 5% | 52 4% | 52 5% | 52 4% | 53 5% | 53 5% | 53 4% | 52 4% | 52 4% | 52 5% | 52 4% | 53 5% | 52 4% | 52 5% | + // | 58 | 54 5% | 55 4% | 55 5% | 54 4% | 55 5% | 55 4% | 55 5% | 54 4% | 54 5% | 54 4% | 55 5% | 55 5% | 55 4% | 54 4% | 54 5% | 54 4% | 54 4% | 55 5% | 54 5% | 54 5% | + // | 60 | 56 5% | 57 4% | 57 5% | 56 4% | 57 5% | 57 4% | 57 5% | 56 4% | 56 5% | 56 4% | 57 5% | 57 5% | 57 4% | 56 4% | 56 5% | 56 4% | 56 4% | 57 5% | 56 4% | 56 5% | + // | 62 | 58 5% | 59 4% | 59 5% | 58 4% | 59 5% | 59 4% | 59 5% | 58 4% | 58 5% | 58 4% | 59 5% | 59 5% | 59 4% | 58 4% | 58 5% | 58 4% | 58 4% | 59 5% | 58 4% | 58 5% | + // | 64 | 60 5% | 61 4% | 61 5% | 60 4% | 61 5% | 61 4% | 61 5% | 60 4% | 60 5% | 60 4% | 61 5% | 61 5% | 61 4% | 60 4% | 60 5% | 60 4% | 60 4% | 61 5% | 60 4% | 60 5% | + // | 66 | 62 5% | 63 5% | 63 5% | 62 4% | 63 5% | 63 4% | 63 5% | 62 4% | 62 5% | 62 4% | 63 5% | 63 5% | 63 4% | 62 4% | 62 5% | 62 4% | 62 4% | 63 5% | 62 4% | 62 5% | + // | 68 | 64 5% | 65 5% | 65 5% | 64 4% | 65 5% | 65 4% | 65 4% | 64 4% | 64 5% | 64 4% | 65 5% | 65 5% | 65 4% | 64 4% | 64 5% | 64 4% | 64 4% | 65 5% | 64 5% | 64 5% | + // | 70 | 66 5% | 67 5% | 67 4% | 66 4% | 67 5% | 67 4% | 67 4% | 66 4% | 66 5% | 66 4% | 67 5% | 67 5% | 67 4% | 66 4% | 66 5% | 66 4% | 66 4% | 67 5% | 66 4% | 66 5% | + // | 72 | 68 5% | 69 4% | 69 4% | 68 4% | 69 5% | 69 4% | 69 4% | 68 4% | 68 5% | 68 4% | 69 5% | 69 5% | 69 4% | 68 4% | 68 5% | 68 4% | 68 4% | 69 5% | 68 4% | 68 5% | + // | 74 | 70 4% | 71 4% | 71 4% | 70 4% | 71 5% | 71 4% | 71 4% | 70 4% | 70 5% | 70 5% | 71 5% | 71 5% | 71 4% | 70 4% | 70 5% | 70 4% | 70 4% | 71 5% | 70 4% | 70 5% | + // | 76 | 72 4% | 73 5% | 73 4% | 72 4% | 73 5% | 73 4% | 73 4% | 72 4% | 72 5% | 72 5% | 73 5% | 73 5% | 73 4% | 72 4% | 72 5% | 72 4% | 72 4% | 73 5% | 72 4% | 72 5% | + // | 78 | 74 4% | 75 5% | 75 4% | 74 4% | 75 5% | 75 4% | 75 4% | 74 4% | 74 5% | 74 5% | 75 5% | 75 5% | 75 4% | 74 4% | 74 5% | 74 4% | 74 4% | 75 5% | 74 5% | 74 5% | + // | 80 | 76 5% | 77 5% | 77 4% | 76 4% | 77 5% | 77 4% | 77 4% | 76 4% | 76 5% | 76 5% | 77 5% | 77 5% | 77 4% | 76 4% | 76 5% | 76 4% | 76 4% | 77 5% | 76 5% | 76 5% | + // | 82 | 78 5% | 79 5% | 79 4% | 78 4% | 79 5% | 79 4% | 79 4% | 78 4% | 78 5% | 78 5% | 79 5% | 79 5% | 79 4% | 78 4% | 78 5% | 78 4% | 78 4% | 79 5% | 78 5% | 78 5% | + // | 84 | 80 5% | 81 4% | 81 4% | 80 4% | 81 5% | 81 4% | 81 4% | 80 4% | 80 5% | 80 5% | 81 5% | 81 5% | 81 4% | 80 4% | 80 5% | 80 4% | 80 4% | 81 5% | 80 5% | 80 5% | + // | 86 | 82 4% | 83 4% | 83 5% | 82 4% | 83 5% | 83 4% | 83 4% | 82 4% | 82 5% | 82 5% | 83 5% | 83 5% | 83 4% | 82 4% | 82 5% | 82 4% | 82 4% | 83 5% | 82 5% | 82 5% | + // | 88 | 84 4% | 85 4% | 85 5% | 84 4% | 85 5% | 85 4% | 85 4% | 84 4% | 84 5% | 84 5% | 85 5% | 85 5% | 85 4% | 84 4% | 84 5% | 84 4% | 84 4% | 85 5% | 84 4% | 84 5% | + // | 90 | 86 4% | 87 4% | 87 5% | 86 4% | 87 5% | 87 4% | 87 4% | 86 4% | 86 5% | 86 5% | 87 5% | 87 5% | 87 4% | 86 4% | 86 5% | 86 4% | 86 4% | 87 5% | 86 5% | 86 5% | + // | 92 | 88 4% | 89 4% | 89 5% | 88 4% | 89 5% | 89 4% | 89 4% | 88 4% | 88 5% | 88 5% | 89 5% | 89 5% | 89 4% | 88 4% | 88 5% | 88 4% | 88 4% | 89 5% | 88 5% | 88 5% | + // | 94 | 90 4% | 91 4% | 91 4% | 90 4% | 91 5% | 91 4% | 91 4% | 90 4% | 90 5% | 90 5% | 91 5% | 91 5% | 91 4% | 90 4% | 90 5% | 90 4% | 90 4% | 91 5% | 90 5% | 90 4% | + // | 96 | 92 4% | 93 5% | 93 5% | 92 4% | 93 5% | 93 4% | 93 4% | 92 4% | 92 5% | 92 5% | 93 5% | 93 5% | 93 4% | 92 4% | 92 5% | 92 4% | 92 4% | 93 5% | 92 5% | 92 4% | + // | 98 | 94 4% | 95 5% | 95 4% | 94 4% | 95 5% | 95 4% | 95 4% | 94 4% | 94 5% | 94 5% | 95 5% | 95 5% | 95 4% | 94 4% | 94 5% | 94 4% | 94 4% | 95 5% | 94 5% | 94 4% | // +-----+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+ - // Total bytes=969243192, ranges=1868 + // Total bytes=998843773, ranges=1909 } diff --git a/pkg/storage/rule_solver.go b/pkg/storage/rule_solver.go index 747d1585ad61..4e915d5ca12e 100644 --- a/pkg/storage/rule_solver.go +++ b/pkg/storage/rule_solver.go @@ -17,6 +17,8 @@ package storage import ( + "bytes" + "fmt" "sort" "github.com/cockroachdb/cockroach/pkg/config" @@ -31,6 +33,11 @@ type candidate struct { balance float64 // Score used to choose between top candidates. } +func (c candidate) String() string { + return fmt.Sprintf("StoreID:%d, valid:%t, con:%.2f, bal:%.2f", + c.store.StoreID, c.valid, c.constraint, c.balance) +} + // less first compares constraint scores, then balance scores. func (c candidate) less(o candidate) bool { if !o.valid { @@ -47,6 +54,19 @@ func (c candidate) less(o candidate) bool { type candidateList []candidate +func (cl candidateList) String() string { + var buffer bytes.Buffer + buffer.WriteRune('[') + for i, c := range cl { + if i != 0 { + buffer.WriteString("; ") + } + buffer.WriteString(c.String()) + } + buffer.WriteRune(']') + return buffer.String() +} + // onlyValid returns all the elements in a sorted candidate list that are valid. func (cl candidateList) onlyValid() candidateList { for i := len(cl) - 1; i >= 0; i-- { @@ -57,6 +77,92 @@ func (cl candidateList) onlyValid() candidateList { return candidateList{} } +// best returns all the elements in a sorted candidate list that share +// the highest constraint score and are valid. +func (cl candidateList) best() candidateList { + cl = cl.onlyValid() + if len(cl) <= 1 { + return cl + } + for i := 1; i < len(cl); i++ { + if cl[i].constraint < cl[0].constraint { + return cl[0:i] + } + } + return cl +} + +// worst returns all the elements in a sorted candidate list that +// share the lowest constraint score. +func (cl candidateList) worst() candidateList { + if len(cl) <= 1 { + return cl + } + // Are there invalid values? If so, pick those. + if !cl[len(cl)-1].valid { + for i := len(cl) - 2; i >= 0; i-- { + if cl[i].valid { + return cl[i+1:] + } + } + } + // Find the worst constraint values. + for i := len(cl) - 2; i >= 0; i-- { + if cl[i].constraint > cl[len(cl)-1].constraint { + return cl[i+1:] + } + } + return cl +} + +// betterThan returns all elements that score higher than the candidate. +func (cl candidateList) betterThan(c candidate) candidateList { + for i := 0; i < len(cl); i++ { + if !c.less(cl[i]) { + return cl[0:i] + } + } + return cl +} + +// selectGood randomly chooses a good candidate from a sorted candidate list +// using the provided random generator. +func (cl candidateList) selectGood(randGen allocatorRand) candidate { + cl = cl.best() + if len(cl) == 1 { + return cl[0] + } + randGen.Lock() + order := randGen.Perm(len(cl)) + randGen.Unlock() + best := cl[order[0]] + for i := 1; i < allocatorRandomCount; i++ { + if best.less(cl[order[i]]) { + best = cl[order[i]] + } + } + return best +} + +// selectBad randomly chooses a bad candidate from a sorted candidate list using +// the provided random generator. +func (cl candidateList) selectBad(randGen allocatorRand) candidate { + cl = cl.worst() + if len(cl) == 1 { + return cl[0] + } + randGen.Lock() + order := randGen.Perm(len(cl)) + randGen.Unlock() + worst := cl[order[0]] + for i := 1; i < allocatorRandomCount; i++ { + if cl[order[i]].less(worst) { + worst = cl[order[i]] + } + } + return worst +} + // solveState is used to pass solution state information into a rule. type solveState struct { constraints config.Constraints @@ -80,6 +186,7 @@ var allocateRuleSolver = ruleSolver{ ruleConstraints, ruleCapacityMax, ruleDiversity, + ruleCapacityToMean, ruleCapacity, } @@ -88,6 +195,23 @@ var removeRuleSolver = ruleSolver{ ruleConstraints, ruleCapacityMax, ruleDiversity, + ruleCapacityFromMean, + ruleCapacity, +} + +var rebalanceExisting = ruleSolver{ + ruleConstraints, + ruleCapacityMax, + ruleDiversity, + ruleCapacityToMean, + ruleCapacity, +} + +var rebalance = ruleSolver{ + ruleConstraints, + ruleCapacityMax, + ruleDiversity, + ruleCapacityFromMean, ruleCapacity, } @@ -98,6 +222,7 @@ func (rs ruleSolver) Solve( c config.Constraints, existing []roachpb.ReplicaDescriptor, existingNodeLocalities map[roachpb.NodeID]roachpb.Locality, + deterministic bool, ) candidateList { candidates := make(candidateList, len(sl.stores), len(sl.stores)) state := solveState{ @@ -110,7 +235,12 @@ func (rs ruleSolver) Solve( for i, store := range sl.stores { candidates[i] = rs.computeCandidate(store, state) } - sort.Sort(sort.Reverse(byScore(candidates))) + if deterministic { + sort.Sort(sort.Reverse(byScoreAndID(candidates))) + } else { + sort.Sort(sort.Reverse(byScore(candidates))) + } + return candidates } @@ -208,6 +338,20 @@ func ruleDiversity(store roachpb.StoreDescriptor, state solveState) (bool, float return true, minScore, 0 } +/* +// ruleDiversityExisting returns higher scores for stores with the fewest locality tiers +// in common with already existing replicas. It always returns true. +func ruleDiversity(store roachpb.StoreDescriptor, state solveState) (bool, float64, float64) { + minScore := 1.0 + for _, locality := range state.existingNodeLocalities { + if newScore := store.Node.Locality.DiversityScore(locality); newScore < minScore { + minScore = newScore + } + } + return true, minScore, 0 +} +*/ + // ruleCapacity returns a balance score that is inversely proportional to the // number of ranges on the candidate store such that the most empty store will // have the highest scores. Scores are always between 0 and 1. @@ -223,6 +367,26 @@ func ruleCapacityMax(store roachpb.StoreDescriptor, state solveState) (bool, flo return true, 0, 0 } +// ruleCapacityFromMean is designed for removals and yields a lower constraint +// score if the removal of this store would push the store closer away from +// the mean number of ranges. +func ruleCapacityFromMean(store roachpb.StoreDescriptor, state solveState) (bool, float64, float64) { + if rebalanceFromConvergesOnMean(state.sl, store) { + return true, 0, 0 + } + return true, 0.1, 0 +} + +// ruleCapacityToMean is designed for rebalancing and yields a higher constraint +// score if the addition of this store would push the store closer to the mean +// number of ranges. +func ruleCapacityToMean(store roachpb.StoreDescriptor, state solveState) (bool, float64, float64) { + if rebalanceToConvergesOnMean(state.sl, store) { + return true, 0, 0 + } + return true, 0.1, 0 +} + // byScore implements sort.Interface to sort by scores. type byScore candidateList @@ -231,3 +395,18 @@ var _ sort.Interface = byScore(nil) func (c byScore) Len() int { return len(c) } func (c byScore) Less(i, j int) bool { return c[i].less(c[j]) } func (c byScore) Swap(i, j int) { c[i], c[j] = c[j], c[i] } + +type byScoreAndID candidateList + +var _ sort.Interface = byScoreAndID(nil) + +func (c byScoreAndID) Len() int { return len(c) } +func (c byScoreAndID) Less(i, j int) bool { + if c[i].constraint == c[j].constraint && + c[i].balance == c[j].balance && + c[i].valid == c[j].valid { + return c[i].store.StoreID < c[j].store.StoreID + } + return c[i].less(c[j]) +} +func (c byScoreAndID) Swap(i, j int) { c[i], c[j] = c[j], c[i] } diff --git a/pkg/storage/rule_solver_test.go b/pkg/storage/rule_solver_test.go index 32db4d8b5348..ef71bf8302cd 100644 --- a/pkg/storage/rule_solver_test.go +++ b/pkg/storage/rule_solver_test.go @@ -19,45 +19,18 @@ package storage import ( "bytes" "fmt" + "math/rand" + "reflect" "sort" "testing" + "github.com/kr/pretty" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/leaktest" ) -type byScoreAndID candidateList - -func (c byScoreAndID) Len() int { return len(c) } -func (c byScoreAndID) Less(i, j int) bool { - if c[i].constraint == c[j].constraint && - c[i].balance == c[j].balance && - c[i].valid == c[j].valid { - return c[i].store.StoreID < c[j].store.StoreID - } - return c[i].less(c[j]) -} -func (c byScoreAndID) Swap(i, j int) { c[i], c[j] = c[j], c[i] } - -func (c candidate) String() string { - return fmt.Sprintf("StoreID:%d, valid:%t, con:%.2f, bal:%.2f", - c.store.StoreID, c.valid, c.constraint, c.balance) -} - -func (cl candidateList) String() string { - var buffer bytes.Buffer - buffer.WriteRune('[') - for i, c := range cl { - if i != 0 { - buffer.WriteString("; ") - } - buffer.WriteString(c.String()) - } - buffer.WriteRune(']') - return buffer.String() -} - // TODO(bram): This test suite is not even close to exhaustive. The scores are // not checked and each rule should have many more test cases. Also add a // corrupt replica test and remove the 0 range ID used when calling @@ -67,7 +40,7 @@ func TestRuleSolver(t *testing.T) { stopper, _, _, storePool, _ := createTestStorePool( TestTimeUntilStoreDeadOff, - /* deterministic */ false, + /* deterministic */ true, /* defaultNodeLiveness */ true, ) defer stopper.Stop() @@ -294,8 +267,8 @@ func TestRuleSolver(t *testing.T) { tc.c, tc.existing, storePool.getNodeLocalities(tc.existing), + storePool.deterministic, ) - sort.Sort(sort.Reverse(byScoreAndID(candidates))) valid := candidates.onlyValid() invalid := candidates[len(valid):] @@ -360,3 +333,194 @@ func TestOnlyValid(t *testing.T) { }) } } + +// TestCandidateSelection tests select{good,bad} and {best,worst}constraints. +func TestCandidateSelection(t *testing.T) { + defer leaktest.AfterTest(t)() + + type scoreTuple struct { + constraint int + balance int + } + genCandidates := func(scores []scoreTuple) candidateList { + var cl candidateList + for _, score := range scores { + cl = append(cl, candidate{ + constraint: float64(score.constraint), + balance: float64(score.balance), + valid: true, + }) + } + sort.Sort(sort.Reverse(byScore(cl))) + return cl + } + + formatter := func(cl candidateList) string { + var buffer bytes.Buffer + for i, c := range cl { + if i != 0 { + buffer.WriteRune(',') + } + buffer.WriteString(fmt.Sprintf("%d:%d", int(c.constraint), int(c.balance))) + } + return buffer.String() + } + + testCases := []struct { + candidates []scoreTuple + best []scoreTuple + worst []scoreTuple + good scoreTuple + bad scoreTuple + }{ + { + candidates: []scoreTuple{{0, 0}}, + best: []scoreTuple{{0, 0}}, + worst: []scoreTuple{{0, 0}}, + good: scoreTuple{0, 0}, + bad: scoreTuple{0, 0}, + }, + { + candidates: []scoreTuple{{0, 1}, {0, 0}}, + best: []scoreTuple{{0, 1}, {0, 0}}, + worst: []scoreTuple{{0, 1}, {0, 0}}, + good: scoreTuple{0, 1}, + bad: scoreTuple{0, 0}, + }, + { + candidates: []scoreTuple{{0, 2}, {0, 1}, {0, 0}}, + best: []scoreTuple{{0, 2}, {0, 1}, {0, 0}}, + worst: []scoreTuple{{0, 2}, {0, 1}, {0, 0}}, + good: scoreTuple{0, 1}, + bad: scoreTuple{0, 0}, + }, + { + candidates: []scoreTuple{{1, 0}, {0, 1}}, + best: []scoreTuple{{1, 0}}, + worst: []scoreTuple{{0, 1}}, + good: scoreTuple{1, 0}, + bad: scoreTuple{0, 1}, + }, + { + candidates: []scoreTuple{{1, 0}, {0, 2}, {0, 1}}, + best: []scoreTuple{{1, 0}}, + worst: []scoreTuple{{0, 2}, {0, 1}}, + good: scoreTuple{1, 0}, + bad: scoreTuple{0, 1}, + }, + { + candidates: []scoreTuple{{1, 1}, {1, 0}, {0, 2}}, + best: []scoreTuple{{1, 1}, {1, 0}}, + worst: []scoreTuple{{0, 2}}, + good: scoreTuple{1, 1}, + bad: scoreTuple{0, 2}, + }, + { + candidates: []scoreTuple{{1, 1}, {1, 0}, {0, 3}, {0, 2}}, + best: []scoreTuple{{1, 1}, {1, 0}}, + worst: []scoreTuple{{0, 3}, {0, 2}}, + good: scoreTuple{1, 1}, + bad: scoreTuple{0, 2}, + }, + } + + allocRand := makeAllocatorRand(rand.NewSource(0)) + for _, tc := range testCases { + cl := genCandidates(tc.candidates) + t.Run(fmt.Sprintf("best-%s", formatter(cl)), func(t *testing.T) { + if a, e := cl.best(), genCandidates(tc.best); !reflect.DeepEqual(a, e) { + t.Errorf("expected:%s actual:%s diff:%v", formatter(e), formatter(a), pretty.Diff(e, a)) + } + }) + t.Run(fmt.Sprintf("worst-%s", formatter(cl)), func(t *testing.T) { + if a, e := cl.worst(), genCandidates(tc.worst); !reflect.DeepEqual(a, e) { + t.Errorf("expected:%s actual:%s diff:%v", formatter(e), formatter(a), pretty.Diff(e, a)) + } + }) + t.Run(fmt.Sprintf("good-%s", formatter(cl)), func(t *testing.T) { + good := cl.selectGood(allocRand) + actual := scoreTuple{int(good.constraint), int(good.balance)} + if actual != tc.good { + t.Errorf("expected:%v actual:%v", tc.good, actual) + } + }) + t.Run(fmt.Sprintf("bad-%s", formatter(cl)), func(t *testing.T) { + bad := cl.selectBad(allocRand) + actual := scoreTuple{int(bad.constraint), int(bad.balance)} + if actual != tc.bad { + t.Errorf("expected:%v actual:%v", tc.bad, actual) + } + }) + } +} + +func TestBetterThan(t *testing.T) { + defer leaktest.AfterTest(t)() + + testCandidateList := candidateList{ + { + valid: true, + constraint: 1, + balance: 1, + }, + { + valid: true, + constraint: 1, + balance: 1, + }, + { + valid: true, + constraint: 1, + balance: 0, + }, + { + valid: true, + constraint: 1, + balance: 0, + }, + { + valid: true, + constraint: 0, + balance: 1, + }, + { + valid: true, + constraint: 0, + balance: 1, + }, + { + valid: true, + constraint: 0, + balance: 0, + }, + { + valid: true, + constraint: 0, + balance: 0, + }, + { + valid: false, + constraint: 1, + balance: 0.5, + }, + { + valid: false, + constraint: 0, + balance: 0.5, + }, + { + valid: false, + constraint: 0, + balance: 0, + }, + } + + expectedResults := []int{0, 0, 2, 2, 4, 4, 6, 6, 8, 8, 8} + + for i := 0; i < len(testCandidateList); i++ { + betterThan := testCandidateList.betterThan(testCandidateList[i]) + if e, a := expectedResults[i], len(betterThan); e != a { + t.Errorf("expected %d results, actual %d", e, a) + } + } +} From bbe17fab0636dd3356641b701ae96891378b77af Mon Sep 17 00:00:00 2001 From: Bram Gruneir Date: Wed, 7 Dec 2016 11:58:08 -0500 Subject: [PATCH 3/3] storage: replace the rule solver Remove the concept of rules and replace it with clean direct functions. --- pkg/storage/allocator.go | 125 ++---- pkg/storage/allocator_test.go | 105 +++-- pkg/storage/rule_solver.go | 465 +++++++++++---------- pkg/storage/rule_solver_test.go | 705 +++++++++++++++++++------------- 4 files changed, 772 insertions(+), 628 deletions(-) diff --git a/pkg/storage/allocator.go b/pkg/storage/allocator.go index e812a524efdd..a500bcd039d2 100644 --- a/pkg/storage/allocator.go +++ b/pkg/storage/allocator.go @@ -212,29 +212,30 @@ func (a *Allocator) AllocateTarget( rangeID roachpb.RangeID, relaxConstraints bool, ) (*roachpb.StoreDescriptor, error) { + sl, aliveStoreCount, throttledStoreCount := a.storePool.getStoreList(rangeID) if a.options.UseRuleSolver { - sl, _, throttledStoreCount := a.storePool.getStoreList(rangeID) - // When there are throttled stores that do match, we shouldn't send - // the replica to purgatory. - if throttledStoreCount > 0 { - return nil, errors.Errorf("%d matching stores are currently throttled", throttledStoreCount) - } - - candidates := allocateRuleSolver.Solve( + candidates := allocateCandidates( sl, constraints, existing, a.storePool.getNodeLocalities(existing), a.storePool.deterministic, ) - candidates = candidates.onlyValid() - if len(candidates) == 0 { - return nil, &allocatorError{ - required: constraints.Constraints, - } + if log.V(3) { + log.Infof(context.TODO(), "allocate candidates: %s", candidates) + } + if target := candidates.selectGood(a.randGen); target != nil { + return target, nil + } + + // When there are throttled stores that do match, we shouldn't send + // the replica to purgatory. + if throttledStoreCount > 0 { + return nil, errors.Errorf("%d matching stores are currently throttled", throttledStoreCount) + } + return nil, &allocatorError{ + required: constraints.Constraints, } - chosenCandidate := candidates.selectGood(a.randGen).store - return &chosenCandidate, nil } existingNodes := make(nodeIDSet, len(existing)) @@ -242,8 +243,6 @@ func (a *Allocator) AllocateTarget( existingNodes[repl.NodeID] = struct{}{} } - sl, aliveStoreCount, throttledStoreCount := a.storePool.getStoreList(rangeID) - // Because more redundancy is better than less, if relaxConstraints, the // matching here is lenient, and tries to find a target by relaxing an // attribute constraint, from last attribute to first. @@ -273,11 +272,6 @@ func (a *Allocator) AllocateTarget( // that have greater than the average number of replicas. Failing that, it // falls back to selecting a random target from any of the existing // replicas. It also will exclude any replica that lives on leaseStoreID. -// -// TODO(mrtracy): removeTarget eventually needs to accept the attributes from -// the zone config associated with the provided replicas. This will allow it to -// make correct decisions in the case of ranges with heterogeneous replica -// requirements (i.e. multiple data centers). func (a Allocator) RemoveTarget( constraints config.Constraints, existing []roachpb.ReplicaDescriptor, @@ -298,29 +292,26 @@ func (a Allocator) RemoveTarget( } } sl := makeStoreList(descriptors) - var badStoreID roachpb.StoreID + var bad *roachpb.StoreDescriptor if a.options.UseRuleSolver { - candidates := removeRuleSolver.Solve( + candidates := removeCandidates( sl, constraints, - existing, a.storePool.getNodeLocalities(existing), a.storePool.deterministic, ) - if len(candidates) != 0 { - badStoreID = candidates.selectBad(a.randGen).store.StoreID + if log.V(3) { + log.Infof(context.TODO(), "remove candidates: %s", candidates) } + bad = candidates.selectBad(a.randGen) } else { - bad := a.selectBad(sl) - if bad != nil { - badStoreID = bad.StoreID - } + bad = a.selectBad(sl) } - if badStoreID != 0 { + if bad != nil { for _, exist := range existing { - if exist.StoreID == badStoreID { + if exist.StoreID == bad.StoreID { return exist, nil } } @@ -358,13 +349,13 @@ func (a Allocator) RebalanceTarget( if !a.options.AllowRebalance { return nil, nil } + sl, _, _ := a.storePool.getStoreList(rangeID) if a.options.UseRuleSolver { - sl, _, _ := a.storePool.getStoreList(rangeID) - if log.V(3) { - log.Infof(context.TODO(), "rebalance-target (lease-holder=%d):\n%s", leaseStoreID, sl) - } - + // TODO(bram): ShouldRebalance should be part of rebalanceCandidates + // and decision made afterward, not it's own function. It is + // performing the same operations as rebalanceCandidates and any + // missing functionality can be added. var shouldRebalance bool for _, repl := range existing { if leaseStoreID == repl.StoreID { @@ -380,59 +371,29 @@ func (a Allocator) RebalanceTarget( return nil, nil } - // Load the exiting storesIDs into a map so to eliminate having to loop - // through the existing descriptors more than once. - existingStoreIDs := make(map[roachpb.StoreID]struct{}) - for _, repl := range existing { - existingStoreIDs[repl.StoreID] = struct{}{} - } - - // Split the store list into existing and candidate stores lists so that - // we can call solve independently on both store lists. - var existingDescs []roachpb.StoreDescriptor - var candidateDescs []roachpb.StoreDescriptor - for _, desc := range sl.stores { - if _, ok := existingStoreIDs[desc.StoreID]; ok { - existingDescs = append(existingDescs, desc) - } else { - candidateDescs = append(candidateDescs, desc) - } - } - - existingStoreList := makeStoreList(existingDescs) - candidateStoreList := makeStoreList(candidateDescs) - - localities := a.storePool.getNodeLocalities(existing) - existingCandidates := rebalanceExisting.Solve( - existingStoreList, - constraints, - existing, - localities, - a.storePool.deterministic, - ) - candidates := rebalance.Solve( - candidateStoreList, + existingCandidates, candidates := rebalanceCandidates( + sl, constraints, existing, - localities, + a.storePool.getNodeLocalities(existing), a.storePool.deterministic, ) - /* - fmt.Printf("existing: %s\n", existingCandidates) - fmt.Printf("candidates: %s\n", candidates) - */ + if len(existingCandidates) == 0 { + return nil, errors.Errorf( + "all existing replicas' stores are not present in the store pool: %v\n%s", existing, sl) + } + + if log.V(3) { + log.Infof(context.TODO(), "existing replicas: %s", existingCandidates) + log.Infof(context.TODO(), "candidates: %s", candidates) + } - // Find all candidates that are better than the worst existing. + // Find all candidates that are better than the worst existing replica. targets := candidates.betterThan(existingCandidates[len(existingCandidates)-1]) - if len(targets) != 0 { - target := targets.selectGood(a.randGen).store - return &target, nil - } - return nil, nil + return targets.selectGood(a.randGen), nil } - sl, _, _ := a.storePool.getStoreList(rangeID) sl = sl.filter(constraints) if log.V(3) { log.Infof(context.TODO(), "rebalance-target (lease-holder=%d):\n%s", leaseStoreID, sl) diff --git a/pkg/storage/allocator_test.go b/pkg/storage/allocator_test.go index 8d8bff0c477e..a8bec87a9d7b 100644 --- a/pkg/storage/allocator_test.go +++ b/pkg/storage/allocator_test.go @@ -720,7 +720,6 @@ func TestAllocatorRebalance(t *testing.T) { if err != nil { t.Fatal(err) } - fmt.Printf("%d: %s\n", i, result) if result == nil { i-- // loop until we find 10 candidates continue @@ -1925,59 +1924,59 @@ func Example_rebalancingWithRuleSolver() { exampleRebalancingCore(true /* useRuleSolver */) // 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 | 1 48% | 0 0% | 0 0% | 1 51% | 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 | 2 35% | 0 0% | 0 0% | 2 24% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 1 11% | 0 0% | 1 14% | 0 0% | 1 13% | 0 0% | 0 0% | 0 0% | - // | 4 | 2 19% | 0 0% | 0 0% | 2 8% | 0 0% | 0 0% | 1 7% | 2 11% | 0 0% | 1 1% | 2 9% | 0 0% | 2 3% | 1 1% | 2 9% | 1 9% | 2 4% | 0 0% | 2 13% | 0 0% | - // | 6 | 2 10% | 3 5% | 3 6% | 2 1% | 3 7% | 3 3% | 3 3% | 2 7% | 2 1% | 2 3% | 2 7% | 4 5% | 3 2% | 2 2% | 2 2% | 2 4% | 2 0% | 3 10% | 2 8% | 2 1% | - // | 8 | 4 5% | 5 6% | 5 4% | 4 2% | 5 6% | 5 5% | 5 6% | 4 7% | 4 2% | 4 4% | 5 6% | 5 2% | 5 3% | 4 4% | 4 3% | 4 4% | 4 3% | 5 8% | 4 6% | 4 3% | - // | 10 | 6 5% | 7 6% | 7 3% | 6 3% | 7 5% | 7 5% | 7 5% | 6 7% | 6 3% | 6 5% | 7 6% | 7 3% | 7 3% | 6 4% | 6 4% | 6 4% | 6 2% | 7 8% | 6 5% | 6 4% | - // | 12 | 8 5% | 9 5% | 9 5% | 8 3% | 9 5% | 9 4% | 9 5% | 8 6% | 8 4% | 8 4% | 9 7% | 9 3% | 9 3% | 8 4% | 8 4% | 8 4% | 8 3% | 9 7% | 8 5% | 8 5% | - // | 14 | 10 4% | 11 6% | 11 4% | 10 3% | 11 5% | 11 4% | 11 6% | 10 6% | 10 5% | 10 4% | 11 6% | 11 4% | 11 3% | 10 4% | 10 3% | 10 4% | 10 3% | 11 7% | 10 5% | 10 4% | - // | 16 | 12 4% | 13 6% | 13 4% | 12 3% | 13 5% | 13 3% | 13 5% | 12 6% | 12 5% | 12 5% | 13 6% | 13 4% | 13 3% | 12 4% | 12 4% | 12 4% | 12 3% | 13 6% | 12 4% | 12 5% | - // | 18 | 14 4% | 15 6% | 15 4% | 14 3% | 15 5% | 15 4% | 15 5% | 14 6% | 14 5% | 14 4% | 15 6% | 15 4% | 15 3% | 14 5% | 14 4% | 14 5% | 14 4% | 15 6% | 14 4% | 14 4% | - // | 20 | 16 4% | 17 6% | 17 4% | 16 4% | 17 5% | 17 3% | 17 5% | 16 5% | 16 6% | 16 4% | 17 5% | 17 5% | 17 3% | 16 5% | 16 4% | 16 5% | 16 4% | 17 6% | 16 4% | 16 4% | - // | 22 | 18 4% | 19 6% | 19 4% | 18 4% | 19 5% | 19 4% | 19 5% | 18 5% | 18 5% | 18 4% | 19 5% | 19 5% | 19 3% | 18 5% | 18 4% | 18 4% | 18 4% | 19 6% | 18 4% | 18 4% | - // | 24 | 20 5% | 21 5% | 21 4% | 20 4% | 21 5% | 21 3% | 21 4% | 20 5% | 20 5% | 20 4% | 21 6% | 21 5% | 21 3% | 20 4% | 20 4% | 20 4% | 20 4% | 21 6% | 20 5% | 20 4% | - // | 26 | 22 5% | 23 5% | 23 4% | 22 4% | 23 5% | 23 3% | 23 5% | 22 5% | 22 4% | 22 4% | 23 5% | 23 5% | 23 3% | 22 4% | 22 4% | 22 5% | 22 4% | 23 5% | 22 5% | 22 4% | - // | 28 | 24 5% | 25 5% | 25 4% | 24 4% | 25 5% | 25 3% | 25 4% | 24 5% | 24 5% | 24 4% | 25 5% | 25 5% | 25 3% | 24 4% | 24 4% | 24 5% | 24 4% | 25 5% | 24 5% | 24 4% | - // | 30 | 26 5% | 27 5% | 27 4% | 26 4% | 27 5% | 27 4% | 27 4% | 26 5% | 26 5% | 26 4% | 27 5% | 27 5% | 27 3% | 26 4% | 26 4% | 26 5% | 26 4% | 27 5% | 26 5% | 26 5% | - // | 32 | 28 5% | 29 5% | 29 4% | 28 4% | 29 5% | 29 4% | 29 4% | 28 5% | 28 4% | 28 4% | 29 5% | 29 5% | 29 3% | 28 4% | 28 4% | 28 5% | 28 4% | 29 5% | 28 5% | 28 5% | - // | 34 | 30 5% | 31 4% | 31 4% | 30 4% | 31 5% | 31 4% | 31 4% | 30 4% | 30 5% | 30 4% | 31 5% | 31 5% | 31 3% | 30 4% | 30 4% | 30 4% | 30 4% | 31 5% | 30 5% | 30 5% | - // | 36 | 32 5% | 33 5% | 33 4% | 32 4% | 33 5% | 33 4% | 33 4% | 32 5% | 32 4% | 32 4% | 33 5% | 33 5% | 33 3% | 32 4% | 32 4% | 32 4% | 32 4% | 33 5% | 32 5% | 32 5% | - // | 38 | 34 5% | 35 5% | 35 5% | 34 4% | 35 5% | 35 4% | 35 4% | 34 5% | 34 5% | 34 4% | 35 5% | 35 5% | 35 3% | 34 4% | 34 4% | 34 4% | 34 4% | 35 5% | 34 5% | 34 5% | - // | 40 | 36 5% | 37 5% | 37 5% | 36 4% | 37 5% | 37 4% | 37 4% | 36 4% | 36 5% | 36 4% | 37 5% | 37 5% | 37 3% | 36 4% | 36 4% | 36 5% | 36 4% | 37 5% | 36 5% | 36 5% | - // | 42 | 38 5% | 39 5% | 39 5% | 38 4% | 39 5% | 39 4% | 39 4% | 38 4% | 38 5% | 38 4% | 39 5% | 39 5% | 39 3% | 38 4% | 38 5% | 38 5% | 38 4% | 39 5% | 38 5% | 38 5% | - // | 44 | 40 5% | 41 5% | 41 5% | 40 4% | 41 5% | 41 4% | 41 5% | 40 4% | 40 5% | 40 4% | 41 5% | 41 5% | 41 3% | 40 4% | 40 5% | 40 5% | 40 4% | 41 5% | 40 5% | 40 5% | - // | 46 | 42 5% | 43 5% | 43 5% | 42 4% | 43 5% | 43 4% | 43 5% | 42 4% | 42 5% | 42 4% | 43 5% | 43 5% | 43 4% | 42 4% | 42 5% | 42 5% | 42 4% | 43 5% | 42 4% | 42 5% | - // | 48 | 44 5% | 45 5% | 45 5% | 44 4% | 45 5% | 45 4% | 45 5% | 44 4% | 44 5% | 44 4% | 45 5% | 45 5% | 45 4% | 44 4% | 44 4% | 44 5% | 44 4% | 45 5% | 44 4% | 44 5% | - // | 50 | 46 5% | 47 5% | 47 5% | 46 4% | 47 5% | 47 4% | 47 5% | 46 4% | 46 5% | 46 4% | 47 5% | 47 5% | 47 4% | 46 4% | 46 4% | 46 5% | 46 4% | 47 5% | 46 5% | 46 5% | - // | 52 | 48 5% | 49 5% | 49 5% | 48 4% | 49 5% | 49 4% | 49 5% | 48 4% | 48 5% | 48 4% | 49 5% | 49 5% | 49 4% | 48 4% | 48 5% | 48 5% | 48 4% | 49 5% | 48 5% | 48 5% | - // | 54 | 50 5% | 51 5% | 51 5% | 50 4% | 51 5% | 51 4% | 51 5% | 50 4% | 50 5% | 50 4% | 51 5% | 51 5% | 51 4% | 50 4% | 50 4% | 50 5% | 50 4% | 51 5% | 50 5% | 50 5% | - // | 56 | 52 5% | 53 4% | 53 5% | 52 4% | 53 5% | 53 4% | 53 5% | 52 4% | 52 5% | 52 4% | 53 5% | 53 5% | 53 4% | 52 4% | 52 4% | 52 5% | 52 4% | 53 5% | 52 4% | 52 5% | - // | 58 | 54 5% | 55 4% | 55 5% | 54 4% | 55 5% | 55 4% | 55 5% | 54 4% | 54 5% | 54 4% | 55 5% | 55 5% | 55 4% | 54 4% | 54 5% | 54 4% | 54 4% | 55 5% | 54 5% | 54 5% | - // | 60 | 56 5% | 57 4% | 57 5% | 56 4% | 57 5% | 57 4% | 57 5% | 56 4% | 56 5% | 56 4% | 57 5% | 57 5% | 57 4% | 56 4% | 56 5% | 56 4% | 56 4% | 57 5% | 56 4% | 56 5% | - // | 62 | 58 5% | 59 4% | 59 5% | 58 4% | 59 5% | 59 4% | 59 5% | 58 4% | 58 5% | 58 4% | 59 5% | 59 5% | 59 4% | 58 4% | 58 5% | 58 4% | 58 4% | 59 5% | 58 4% | 58 5% | - // | 64 | 60 5% | 61 4% | 61 5% | 60 4% | 61 5% | 61 4% | 61 5% | 60 4% | 60 5% | 60 4% | 61 5% | 61 5% | 61 4% | 60 4% | 60 5% | 60 4% | 60 4% | 61 5% | 60 4% | 60 5% | - // | 66 | 62 5% | 63 5% | 63 5% | 62 4% | 63 5% | 63 4% | 63 5% | 62 4% | 62 5% | 62 4% | 63 5% | 63 5% | 63 4% | 62 4% | 62 5% | 62 4% | 62 4% | 63 5% | 62 4% | 62 5% | - // | 68 | 64 5% | 65 5% | 65 5% | 64 4% | 65 5% | 65 4% | 65 4% | 64 4% | 64 5% | 64 4% | 65 5% | 65 5% | 65 4% | 64 4% | 64 5% | 64 4% | 64 4% | 65 5% | 64 5% | 64 5% | - // | 70 | 66 5% | 67 5% | 67 4% | 66 4% | 67 5% | 67 4% | 67 4% | 66 4% | 66 5% | 66 4% | 67 5% | 67 5% | 67 4% | 66 4% | 66 5% | 66 4% | 66 4% | 67 5% | 66 4% | 66 5% | - // | 72 | 68 5% | 69 4% | 69 4% | 68 4% | 69 5% | 69 4% | 69 4% | 68 4% | 68 5% | 68 4% | 69 5% | 69 5% | 69 4% | 68 4% | 68 5% | 68 4% | 68 4% | 69 5% | 68 4% | 68 5% | - // | 74 | 70 4% | 71 4% | 71 4% | 70 4% | 71 5% | 71 4% | 71 4% | 70 4% | 70 5% | 70 5% | 71 5% | 71 5% | 71 4% | 70 4% | 70 5% | 70 4% | 70 4% | 71 5% | 70 4% | 70 5% | - // | 76 | 72 4% | 73 5% | 73 4% | 72 4% | 73 5% | 73 4% | 73 4% | 72 4% | 72 5% | 72 5% | 73 5% | 73 5% | 73 4% | 72 4% | 72 5% | 72 4% | 72 4% | 73 5% | 72 4% | 72 5% | - // | 78 | 74 4% | 75 5% | 75 4% | 74 4% | 75 5% | 75 4% | 75 4% | 74 4% | 74 5% | 74 5% | 75 5% | 75 5% | 75 4% | 74 4% | 74 5% | 74 4% | 74 4% | 75 5% | 74 5% | 74 5% | - // | 80 | 76 5% | 77 5% | 77 4% | 76 4% | 77 5% | 77 4% | 77 4% | 76 4% | 76 5% | 76 5% | 77 5% | 77 5% | 77 4% | 76 4% | 76 5% | 76 4% | 76 4% | 77 5% | 76 5% | 76 5% | - // | 82 | 78 5% | 79 5% | 79 4% | 78 4% | 79 5% | 79 4% | 79 4% | 78 4% | 78 5% | 78 5% | 79 5% | 79 5% | 79 4% | 78 4% | 78 5% | 78 4% | 78 4% | 79 5% | 78 5% | 78 5% | - // | 84 | 80 5% | 81 4% | 81 4% | 80 4% | 81 5% | 81 4% | 81 4% | 80 4% | 80 5% | 80 5% | 81 5% | 81 5% | 81 4% | 80 4% | 80 5% | 80 4% | 80 4% | 81 5% | 80 5% | 80 5% | - // | 86 | 82 4% | 83 4% | 83 5% | 82 4% | 83 5% | 83 4% | 83 4% | 82 4% | 82 5% | 82 5% | 83 5% | 83 5% | 83 4% | 82 4% | 82 5% | 82 4% | 82 4% | 83 5% | 82 5% | 82 5% | - // | 88 | 84 4% | 85 4% | 85 5% | 84 4% | 85 5% | 85 4% | 85 4% | 84 4% | 84 5% | 84 5% | 85 5% | 85 5% | 85 4% | 84 4% | 84 5% | 84 4% | 84 4% | 85 5% | 84 4% | 84 5% | - // | 90 | 86 4% | 87 4% | 87 5% | 86 4% | 87 5% | 87 4% | 87 4% | 86 4% | 86 5% | 86 5% | 87 5% | 87 5% | 87 4% | 86 4% | 86 5% | 86 4% | 86 4% | 87 5% | 86 5% | 86 5% | - // | 92 | 88 4% | 89 4% | 89 5% | 88 4% | 89 5% | 89 4% | 89 4% | 88 4% | 88 5% | 88 5% | 89 5% | 89 5% | 89 4% | 88 4% | 88 5% | 88 4% | 88 4% | 89 5% | 88 5% | 88 5% | - // | 94 | 90 4% | 91 4% | 91 4% | 90 4% | 91 5% | 91 4% | 91 4% | 90 4% | 90 5% | 90 5% | 91 5% | 91 5% | 91 4% | 90 4% | 90 5% | 90 4% | 90 4% | 91 5% | 90 5% | 90 4% | - // | 96 | 92 4% | 93 5% | 93 5% | 92 4% | 93 5% | 93 4% | 93 4% | 92 4% | 92 5% | 92 5% | 93 5% | 93 5% | 93 4% | 92 4% | 92 5% | 92 4% | 92 4% | 93 5% | 92 5% | 92 4% | - // | 98 | 94 4% | 95 5% | 95 4% | 94 4% | 95 5% | 95 4% | 95 4% | 94 4% | 94 5% | 94 5% | 95 5% | 95 5% | 95 4% | 94 4% | 94 5% | 94 4% | 94 4% | 95 5% | 94 5% | 94 4% | + // | 0 | 2 100% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | + // | 2 | 4 100% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | + // | 4 | 6 100% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | + // | 6 | 8 100% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | + // | 8 | 10 100% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | + // | 10 | 10 68% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 1 2% | 0 0% | 0 0% | 1 11% | 0 0% | 1 18% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | 0 0% | + // | 12 | 10 21% | 1 10% | 0 0% | 1 1% | 1 3% | 1 5% | 2 7% | 1 9% | 1 7% | 0 0% | 0 0% | 1 7% | 1 5% | 1 10% | 0 0% | 1 2% | 1 4% | 1 4% | 0 0% | 1 0% | + // | 14 | 10 8% | 3 7% | 0 0% | 3 3% | 3 4% | 3 5% | 3 5% | 3 8% | 3 8% | 3 8% | 0 0% | 3 8% | 3 7% | 3 6% | 0 0% | 3 2% | 3 4% | 3 8% | 1 0% | 3 2% | + // | 16 | 10 5% | 5 6% | 3 1% | 5 5% | 5 5% | 5 6% | 5 4% | 5 7% | 5 6% | 5 6% | 0 0% | 5 8% | 5 9% | 5 5% | 0 0% | 5 2% | 5 5% | 5 7% | 3 1% | 5 4% | + // | 18 | 10 4% | 6 6% | 6 3% | 6 4% | 6 3% | 6 5% | 6 4% | 6 6% | 6 4% | 6 4% | 8 3% | 6 6% | 6 7% | 6 4% | 7 5% | 6 0% | 6 5% | 6 6% | 8 6% | 6 4% | + // | 20 | 12 4% | 8 6% | 8 4% | 8 4% | 8 4% | 8 5% | 8 3% | 8 5% | 8 4% | 8 4% | 10 4% | 8 6% | 8 7% | 8 4% | 9 5% | 8 2% | 8 5% | 8 5% | 10 5% | 8 4% | + // | 22 | 14 4% | 10 5% | 10 4% | 10 5% | 10 5% | 10 5% | 10 4% | 10 5% | 10 4% | 10 4% | 12 4% | 10 5% | 10 7% | 10 4% | 11 5% | 10 3% | 10 4% | 10 5% | 12 5% | 10 4% | + // | 24 | 16 4% | 12 5% | 12 4% | 12 4% | 12 5% | 12 5% | 12 4% | 12 5% | 12 4% | 12 4% | 14 4% | 12 5% | 12 6% | 12 4% | 13 5% | 12 3% | 12 5% | 12 4% | 14 5% | 12 3% | + // | 26 | 18 4% | 14 5% | 14 4% | 14 5% | 14 4% | 14 5% | 14 4% | 14 5% | 14 4% | 14 5% | 16 3% | 14 5% | 14 6% | 14 4% | 15 5% | 14 3% | 14 5% | 14 4% | 16 5% | 14 3% | + // | 28 | 20 5% | 16 5% | 16 4% | 16 5% | 16 4% | 16 5% | 16 4% | 16 6% | 16 4% | 16 5% | 18 4% | 16 5% | 16 5% | 16 4% | 17 4% | 16 3% | 16 5% | 16 4% | 18 5% | 16 4% | + // | 30 | 22 5% | 18 5% | 18 4% | 18 5% | 18 4% | 18 5% | 18 4% | 18 6% | 18 4% | 18 5% | 20 4% | 18 5% | 18 5% | 18 4% | 19 5% | 18 3% | 18 4% | 18 4% | 20 5% | 18 3% | + // | 32 | 24 5% | 20 5% | 20 4% | 20 5% | 20 4% | 20 5% | 20 4% | 20 6% | 20 4% | 20 5% | 22 4% | 20 5% | 20 6% | 20 4% | 21 5% | 20 3% | 20 4% | 20 4% | 22 5% | 20 4% | + // | 34 | 26 5% | 22 5% | 22 4% | 22 5% | 22 4% | 22 5% | 22 4% | 22 6% | 22 4% | 22 4% | 24 5% | 22 4% | 22 6% | 22 4% | 23 5% | 22 4% | 22 5% | 22 4% | 24 5% | 22 4% | + // | 36 | 28 5% | 24 5% | 24 4% | 24 5% | 24 4% | 24 5% | 24 4% | 24 6% | 24 4% | 24 5% | 26 5% | 24 4% | 24 6% | 24 4% | 25 5% | 24 4% | 24 5% | 24 4% | 26 5% | 24 4% | + // | 38 | 30 5% | 26 5% | 26 4% | 26 5% | 26 4% | 26 4% | 26 4% | 26 5% | 26 4% | 26 5% | 28 5% | 26 4% | 26 5% | 26 4% | 27 5% | 26 4% | 26 5% | 26 4% | 28 5% | 26 4% | + // | 40 | 32 5% | 28 5% | 28 4% | 28 5% | 28 4% | 28 5% | 28 4% | 28 5% | 28 4% | 28 5% | 30 5% | 28 4% | 28 5% | 28 4% | 29 5% | 28 4% | 28 4% | 28 5% | 30 5% | 28 4% | + // | 42 | 34 5% | 30 4% | 30 4% | 30 5% | 30 4% | 30 4% | 30 4% | 30 5% | 30 4% | 30 5% | 32 5% | 30 4% | 30 5% | 30 4% | 31 5% | 30 4% | 30 5% | 30 5% | 32 5% | 30 4% | + // | 44 | 36 4% | 32 5% | 32 4% | 32 5% | 32 4% | 32 4% | 32 4% | 32 5% | 32 4% | 32 5% | 34 5% | 32 4% | 32 5% | 32 4% | 33 5% | 32 4% | 32 4% | 32 5% | 34 5% | 32 4% | + // | 46 | 38 4% | 34 5% | 34 4% | 34 5% | 34 4% | 34 4% | 34 4% | 34 5% | 34 4% | 34 5% | 36 5% | 34 4% | 34 5% | 34 4% | 35 5% | 34 4% | 34 5% | 34 5% | 36 5% | 34 4% | + // | 48 | 40 4% | 36 5% | 36 4% | 36 5% | 36 4% | 36 4% | 36 4% | 36 5% | 36 4% | 36 5% | 38 5% | 36 4% | 36 5% | 36 4% | 37 5% | 36 4% | 36 5% | 36 5% | 38 5% | 36 5% | + // | 50 | 42 4% | 38 5% | 38 4% | 38 5% | 38 4% | 38 4% | 38 4% | 38 5% | 38 4% | 38 5% | 40 5% | 38 4% | 38 5% | 38 4% | 39 5% | 38 4% | 38 5% | 38 5% | 40 5% | 38 5% | + // | 52 | 44 4% | 40 5% | 40 4% | 40 5% | 40 4% | 40 4% | 40 4% | 40 5% | 40 4% | 40 5% | 42 5% | 40 4% | 40 5% | 40 4% | 41 5% | 40 4% | 40 5% | 40 5% | 42 5% | 40 4% | + // | 54 | 46 5% | 42 5% | 42 4% | 42 5% | 42 4% | 42 4% | 42 4% | 42 5% | 42 4% | 42 5% | 44 5% | 42 4% | 42 5% | 42 4% | 43 5% | 42 4% | 42 5% | 42 5% | 44 5% | 42 4% | + // | 56 | 48 5% | 44 5% | 44 4% | 44 5% | 44 4% | 44 5% | 44 4% | 44 5% | 44 4% | 44 5% | 46 5% | 44 4% | 44 5% | 44 4% | 45 5% | 44 4% | 44 5% | 44 5% | 46 5% | 44 4% | + // | 58 | 50 4% | 46 5% | 46 4% | 46 5% | 46 4% | 46 5% | 46 4% | 46 5% | 46 4% | 46 4% | 48 5% | 46 4% | 46 5% | 46 5% | 47 5% | 46 4% | 46 5% | 46 5% | 48 5% | 46 5% | + // | 60 | 52 4% | 48 5% | 48 4% | 48 5% | 48 4% | 48 5% | 48 4% | 48 5% | 48 4% | 48 4% | 50 5% | 48 4% | 48 5% | 48 5% | 49 5% | 48 4% | 48 5% | 48 4% | 50 5% | 48 4% | + // | 62 | 54 4% | 50 5% | 50 4% | 50 5% | 50 4% | 50 5% | 50 4% | 50 5% | 50 4% | 50 4% | 52 5% | 50 4% | 50 5% | 50 4% | 51 5% | 50 4% | 50 5% | 50 4% | 52 5% | 50 5% | + // | 64 | 56 4% | 52 5% | 52 4% | 52 5% | 52 4% | 52 5% | 52 4% | 52 5% | 52 4% | 52 4% | 54 5% | 52 4% | 52 5% | 52 4% | 53 5% | 52 4% | 52 5% | 52 4% | 54 5% | 52 5% | + // | 66 | 58 4% | 54 5% | 54 4% | 54 5% | 54 4% | 54 5% | 54 4% | 54 5% | 54 4% | 54 4% | 56 5% | 54 4% | 54 5% | 54 5% | 55 5% | 54 4% | 54 5% | 54 4% | 56 5% | 54 5% | + // | 68 | 60 4% | 56 5% | 56 4% | 56 5% | 56 4% | 56 5% | 56 4% | 56 5% | 56 4% | 56 4% | 58 5% | 56 4% | 56 5% | 56 4% | 57 5% | 56 4% | 56 5% | 56 4% | 58 5% | 56 5% | + // | 70 | 62 4% | 58 5% | 58 4% | 58 5% | 58 4% | 58 5% | 58 4% | 58 5% | 58 4% | 58 4% | 60 5% | 58 4% | 58 5% | 58 4% | 59 5% | 58 4% | 58 5% | 58 4% | 60 5% | 58 5% | + // | 72 | 64 4% | 60 5% | 60 4% | 60 5% | 60 4% | 60 5% | 60 4% | 60 5% | 60 4% | 60 4% | 62 5% | 60 4% | 60 5% | 60 5% | 61 5% | 60 4% | 60 5% | 60 4% | 62 5% | 60 5% | + // | 74 | 66 4% | 62 5% | 62 4% | 62 5% | 62 4% | 62 5% | 62 4% | 62 5% | 62 4% | 62 4% | 64 5% | 62 4% | 62 5% | 62 5% | 63 5% | 62 4% | 62 5% | 62 4% | 64 5% | 62 5% | + // | 76 | 68 4% | 64 5% | 64 4% | 64 5% | 64 4% | 64 5% | 64 4% | 64 5% | 64 4% | 64 4% | 66 5% | 64 4% | 64 5% | 64 5% | 65 5% | 64 4% | 64 5% | 64 4% | 66 5% | 64 5% | + // | 78 | 70 4% | 66 5% | 66 4% | 66 5% | 66 4% | 66 5% | 66 4% | 66 5% | 66 4% | 66 5% | 68 5% | 66 4% | 66 5% | 66 5% | 67 5% | 66 4% | 66 5% | 66 4% | 68 5% | 66 5% | + // | 80 | 72 4% | 68 5% | 68 4% | 68 5% | 68 4% | 68 5% | 68 4% | 68 5% | 68 4% | 68 5% | 70 5% | 68 4% | 68 5% | 68 5% | 69 4% | 68 4% | 68 5% | 68 4% | 70 5% | 68 4% | + // | 82 | 74 4% | 70 5% | 70 4% | 70 5% | 70 4% | 70 4% | 70 4% | 70 5% | 70 4% | 70 5% | 72 5% | 70 4% | 70 5% | 70 5% | 71 4% | 70 4% | 70 5% | 70 4% | 72 5% | 70 4% | + // | 84 | 76 4% | 72 5% | 72 4% | 72 5% | 72 4% | 72 4% | 72 4% | 72 5% | 72 4% | 72 5% | 74 5% | 72 4% | 72 5% | 72 5% | 73 4% | 72 4% | 72 5% | 72 4% | 74 5% | 72 4% | + // | 86 | 78 4% | 74 5% | 74 4% | 74 5% | 74 4% | 74 4% | 74 4% | 74 5% | 74 4% | 74 5% | 76 5% | 74 4% | 74 5% | 74 5% | 75 4% | 74 4% | 74 5% | 74 4% | 76 5% | 74 4% | + // | 88 | 80 4% | 76 5% | 76 4% | 76 5% | 76 4% | 76 4% | 76 4% | 76 5% | 76 4% | 76 5% | 78 5% | 76 4% | 76 5% | 76 5% | 77 4% | 76 4% | 76 5% | 76 4% | 78 5% | 76 4% | + // | 90 | 82 4% | 78 5% | 78 4% | 78 5% | 78 4% | 78 4% | 78 4% | 78 5% | 78 4% | 78 5% | 80 5% | 78 4% | 78 5% | 78 5% | 79 4% | 78 4% | 78 5% | 78 4% | 80 5% | 78 4% | + // | 92 | 84 4% | 80 5% | 80 4% | 80 5% | 80 4% | 80 4% | 80 4% | 80 5% | 80 5% | 80 5% | 82 5% | 80 4% | 80 5% | 80 5% | 81 4% | 80 4% | 80 5% | 80 5% | 82 5% | 80 4% | + // | 94 | 86 4% | 82 5% | 82 4% | 82 5% | 82 4% | 82 4% | 82 4% | 82 5% | 82 5% | 82 4% | 84 5% | 82 4% | 82 5% | 82 5% | 83 4% | 82 4% | 82 5% | 82 5% | 84 5% | 82 4% | + // | 96 | 88 4% | 84 5% | 84 4% | 84 5% | 84 4% | 84 4% | 84 4% | 84 5% | 84 5% | 84 4% | 86 5% | 84 4% | 84 5% | 84 5% | 85 5% | 84 4% | 84 5% | 84 5% | 86 5% | 84 4% | + // | 98 | 90 4% | 86 5% | 86 4% | 86 5% | 86 4% | 86 4% | 86 4% | 86 5% | 86 5% | 86 4% | 88 5% | 86 4% | 86 5% | 86 5% | 87 4% | 86 4% | 86 5% | 86 5% | 88 5% | 86 4% | // +-----+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+----------+ - // Total bytes=998843773, ranges=1909 + // Total bytes=906655663, ranges=1749 } diff --git a/pkg/storage/rule_solver.go b/pkg/storage/rule_solver.go index 4e915d5ca12e..060a9e828f8f 100644 --- a/pkg/storage/rule_solver.go +++ b/pkg/storage/rule_solver.go @@ -30,15 +30,16 @@ type candidate struct { store roachpb.StoreDescriptor valid bool constraint float64 // Score used to pick the top candidates. - balance float64 // Score used to choose between top candidates. + capacity float64 // Score used to choose between top candidates. } func (c candidate) String() string { - return fmt.Sprintf("StoreID:%d, valid:%t, con:%.2f, bal:%.2f", - c.store.StoreID, c.valid, c.constraint, c.balance) + return fmt.Sprintf("s%d, valid:%t, con:%.2f, bal:%.2f", + c.store.StoreID, c.valid, c.constraint, c.capacity) } -// less first compares constraint scores, then balance scores. +// less first compares valid, then constraint scores, then capacity +// scores. func (c candidate) less(o candidate) bool { if !o.valid { return false @@ -49,7 +50,7 @@ func (c candidate) less(o candidate) bool { if c.constraint != o.constraint { return c.constraint < o.constraint } - return c.balance < o.balance + return c.capacity < o.capacity } type candidateList []candidate @@ -67,7 +68,33 @@ func (cl candidateList) String() string { return buffer.String() } -// onlyValid returns all the elements in a sorted candidate list that are valid. +// byScore implements sort.Interface to sort by scores. +type byScore candidateList + +var _ sort.Interface = byScore(nil) + +func (c byScore) Len() int { return len(c) } +func (c byScore) Less(i, j int) bool { return c[i].less(c[j]) } +func (c byScore) Swap(i, j int) { c[i], c[j] = c[j], c[i] } + +// byScoreAndID implements sort.Interface to sort by scores and ids. +type byScoreAndID candidateList + +var _ sort.Interface = byScoreAndID(nil) + +func (c byScoreAndID) Len() int { return len(c) } +func (c byScoreAndID) Less(i, j int) bool { + if c[i].constraint == c[j].constraint && + c[i].capacity == c[j].capacity && + c[i].valid == c[j].valid { + return c[i].store.StoreID < c[j].store.StoreID + } + return c[i].less(c[j]) +} +func (c byScoreAndID) Swap(i, j int) { c[i], c[j] = c[j], c[i] } + +// onlyValid returns all the elements in a sorted (by score reversed) candidate +// list that are valid. func (cl candidateList) onlyValid() candidateList { for i := len(cl) - 1; i >= 0; i-- { if cl[i].valid { @@ -77,8 +104,8 @@ func (cl candidateList) onlyValid() candidateList { return candidateList{} } -// best returns all the elements in a sorted candidate list that share -// the highest constraint score and are valid. +// best returns all the elements in a sorted (by score reversed) candidate list +// that share the highest constraint score and are valid. func (cl candidateList) best() candidateList { cl = cl.onlyValid() if len(cl) <= 1 { @@ -86,19 +113,19 @@ func (cl candidateList) best() candidateList { } for i := 1; i < len(cl); i++ { if cl[i].constraint < cl[0].constraint { - return cl[0:i] + return cl[:i] } } return cl } -// worst returns all the elements in a sorted candidate list that -// share the lowest constraint score. +// worst returns all the elements in a sorted (by score reversed) candidate +// list that share the lowest constraint score. func (cl candidateList) worst() candidateList { if len(cl) <= 1 { return cl } - // Are there invalid values? If so, pick those. + // Are there invalid candidates? If so, pick those. if !cl[len(cl)-1].valid { for i := len(cl) - 2; i >= 0; i-- { if cl[i].valid { @@ -115,172 +142,235 @@ func (cl candidateList) worst() candidateList { return cl } -// betterThan returns all elements that score higher than the candidate. +// betterThan returns all elements from a sorted (by score reversed) candidate +// list that have a higher score than the candidate func (cl candidateList) betterThan(c candidate) candidateList { for i := 0; i < len(cl); i++ { if !c.less(cl[i]) { - return cl[0:i] + return cl[:i] } } return cl } -// selectGood randomly chooses a good candidate from a sorted candidate list -// using the provided random generator. -func (cl candidateList) selectGood(randGen allocatorRand) candidate { +// selectGood randomly chooses a good candidate store from a sorted (by score +// reserved) candidate list using the provided random generator. +func (cl candidateList) selectGood(randGen allocatorRand) *roachpb.StoreDescriptor { + if len(cl) == 0 { + return nil + } cl = cl.best() if len(cl) == 1 { - return cl[0] + return &cl[0].store } randGen.Lock() order := randGen.Perm(len(cl)) randGen.Unlock() - best := cl[order[0]] + best := &cl[order[0]] for i := 1; i < allocatorRandomCount; i++ { if best.less(cl[order[i]]) { - best = cl[order[i]] + best = &cl[order[i]] } } - return best + return &best.store } -// selectBad randomly chooses a bad candidate from a sorted candidate list using -// the provided random generator. -func (cl candidateList) selectBad(randGen allocatorRand) candidate { +// selectBad randomly chooses a bad candidate store from a sorted (by score +// reversed) candidate list using the provided random generator. +func (cl candidateList) selectBad(randGen allocatorRand) *roachpb.StoreDescriptor { + if len(cl) == 0 { + return nil + } cl = cl.worst() if len(cl) == 1 { - return cl[0] + return &cl[0].store } randGen.Lock() order := randGen.Perm(len(cl)) randGen.Unlock() - worst := cl[order[0]] + worst := &cl[order[0]] for i := 1; i < allocatorRandomCount; i++ { - if cl[order[i]].less(worst) { - worst = cl[order[i]] + if cl[order[i]].less(*worst) { + worst = &cl[order[i]] } } - return worst -} - -// solveState is used to pass solution state information into a rule. -type solveState struct { - constraints config.Constraints - sl StoreList - existing []roachpb.ReplicaDescriptor - existingNodeLocalities map[roachpb.NodeID]roachpb.Locality + return &worst.store } -// rule is a function that given a solveState will score and possibly -// disqualify a store. The store in solveState can be disqualified by -// returning false. Unless disqualified, the higher the returned score, the -// more likely the store will be picked as a candidate. -type rule func(roachpb.StoreDescriptor, solveState) (bool, float64, float64) - -// ruleSolver is used to test a collection of rules against stores. -type ruleSolver []rule - -// allocateRuleSolver is the set of rules used for adding new replicas. -var allocateRuleSolver = ruleSolver{ - ruleReplicasUniqueNodes, - ruleConstraints, - ruleCapacityMax, - ruleDiversity, - ruleCapacityToMean, - ruleCapacity, -} - -// removeRuleSolver is the set of rules used for removing existing replicas. -var removeRuleSolver = ruleSolver{ - ruleConstraints, - ruleCapacityMax, - ruleDiversity, - ruleCapacityFromMean, - ruleCapacity, -} - -var rebalanceExisting = ruleSolver{ - ruleConstraints, - ruleCapacityMax, - ruleDiversity, - ruleCapacityToMean, - ruleCapacity, -} +// allocateCandidates creates a candidate list of all stores that can used for +// allocating a new replica ordered from the best to the worst. Only stores +// that meet the criteria are included in the list. +func allocateCandidates( + sl StoreList, + constraints config.Constraints, + existing []roachpb.ReplicaDescriptor, + existingNodeLocalities map[roachpb.NodeID]roachpb.Locality, + deterministic bool, +) candidateList { + var candidates candidateList + for _, s := range sl.stores { + if !preexistingReplicaCheck(s.Node.NodeID, existing) { + continue + } + constraintsOk, preferredMatched := constraintCheck(s, constraints) + if !constraintsOk { + continue + } + if !maxCapacityCheck(s) { + continue + } -var rebalance = ruleSolver{ - ruleConstraints, - ruleCapacityMax, - ruleDiversity, - ruleCapacityFromMean, - ruleCapacity, + constraintScore := diversityScore(s, existingNodeLocalities) + float64(preferredMatched) + candidates = append(candidates, candidate{ + store: s, + valid: true, + constraint: constraintScore, + capacity: capacityScore(s), + }) + } + if deterministic { + sort.Sort(sort.Reverse(byScoreAndID(candidates))) + } else { + sort.Sort(sort.Reverse(byScore(candidates))) + } + return candidates } -// Solve runs the rules against the stores in the store list and returns all -// candidate stores and their scores ordered from best to worst score. -func (rs ruleSolver) Solve( +// removeCandidates creates a candidate list of all existing replicas' stores +// ordered from least qualified for removal to most qualified. Stores that are +// marked as not valid, are in violation of a required criteria. +func removeCandidates( sl StoreList, - c config.Constraints, - existing []roachpb.ReplicaDescriptor, + constraints config.Constraints, existingNodeLocalities map[roachpb.NodeID]roachpb.Locality, deterministic bool, ) candidateList { - candidates := make(candidateList, len(sl.stores), len(sl.stores)) - state := solveState{ - constraints: c, - sl: sl, - existing: existing, - existingNodeLocalities: existingNodeLocalities, - } + var candidates candidateList + for _, s := range sl.stores { + constraintsOk, preferredMatched := constraintCheck(s, constraints) + if !constraintsOk { + candidates = append(candidates, candidate{store: s, valid: false}) + continue + } + if !maxCapacityCheck(s) { + candidates = append(candidates, candidate{store: s, valid: false}) + continue + } + constraintScore := diversityRemovalScore(s.Node.NodeID, existingNodeLocalities) + float64(preferredMatched) + if !rebalanceFromConvergesOnMean(sl, s) { + // If removing this candidate replica does not converge the range + // counts to the mean, we make it less attractive for removal by + // adding 1 to the constraint score. Note that when selecting a + // candidate for removal the candidates with the lowest scores are + // more likely to be removed. + constraintScore++ + } - for i, store := range sl.stores { - candidates[i] = rs.computeCandidate(store, state) + candidates = append(candidates, candidate{ + store: s, + valid: true, + constraint: constraintScore, + capacity: capacityScore(s), + }) } if deterministic { sort.Sort(sort.Reverse(byScoreAndID(candidates))) } else { sort.Sort(sort.Reverse(byScore(candidates))) } - return candidates } -// computeCandidate runs the rules against a candidate store using the provided -// state and returns each candidate's score and if the candidate is valid. -func (rs ruleSolver) computeCandidate(store roachpb.StoreDescriptor, state solveState) candidate { - var totalConstraintScore, totalBalanceScore float64 - for _, rule := range rs { - valid, constraintScore, balanceScore := rule(store, state) - if !valid { - return candidate{ - store: store, - valid: false, +// rebalanceCandidates creates two candidate list. The first contains all +// existing replica's stores, order from least qualified for rebalancing to +// most qualified. The second list is of all potential stores that could be +// used as rebalancing receivers, ordered from best to worst. +func rebalanceCandidates( + sl StoreList, + constraints config.Constraints, + existing []roachpb.ReplicaDescriptor, + existingNodeLocalities map[roachpb.NodeID]roachpb.Locality, + deterministic bool, +) (candidateList, candidateList) { + // Load the exiting storesIDs into a map so to eliminate having to loop + // through the existing descriptors more than once. + existingStoreIDs := make(map[roachpb.StoreID]struct{}) + for _, repl := range existing { + existingStoreIDs[repl.StoreID] = struct{}{} + } + + var existingCandidates candidateList + var candidates candidateList + for _, s := range sl.stores { + constraintsOk, preferredMatched := constraintCheck(s, constraints) + maxCapacityOK := maxCapacityCheck(s) + if _, ok := existingStoreIDs[s.StoreID]; ok { + if !constraintsOk { + existingCandidates = append(existingCandidates, candidate{store: s, valid: false}) + continue + } + if !maxCapacityOK { + existingCandidates = append(existingCandidates, candidate{store: s, valid: false}) + continue } + constraintScore := diversityRemovalScore(s.Node.NodeID, existingNodeLocalities) + float64(preferredMatched) + if !rebalanceFromConvergesOnMean(sl, s) { + // Similarly to in removeCandidates, any replica whose removal + // would not converge the range counts to the mean is given a + // constraint score boost of 1 to make it less attractive for + // removal. + constraintScore++ + } + existingCandidates = append(existingCandidates, candidate{ + store: s, + valid: true, + constraint: constraintScore, + capacity: capacityScore(s), + }) + } else { + if !constraintsOk || !maxCapacityOK || !rebalanceToConvergesOnMean(sl, s) { + continue + } + // The 1.0 here represents that adding this candidate does indeed + // converge the range counts to the mean, which matches its + // counterpart of !rebalanceFromConvergesOnMean from the existing + // candidates. This ensures that when comparing the best candidate + // against the worst existing candidate, any existing candidate + // that does rebalanceFromConvergesOnMean will always have a lower + // score. + constraintScore := 1.0 + diversityScore(s, existingNodeLocalities) + float64(preferredMatched) + candidates = append(candidates, candidate{ + store: s, + valid: true, + constraint: constraintScore, + capacity: capacityScore(s), + }) } - totalConstraintScore += constraintScore - totalBalanceScore += balanceScore } - return candidate{ - store: store, - valid: true, - constraint: totalConstraintScore, - balance: totalBalanceScore, + + if deterministic { + sort.Sort(sort.Reverse(byScoreAndID(existingCandidates))) + sort.Sort(sort.Reverse(byScoreAndID(candidates))) + } else { + sort.Sort(sort.Reverse(byScore(existingCandidates))) + sort.Sort(sort.Reverse(byScore(candidates))) } + + return existingCandidates, candidates } -// ruleReplicasUniqueNodes returns true iff no existing replica is present on -// the candidate's node. All other scores are always 0. -func ruleReplicasUniqueNodes( - store roachpb.StoreDescriptor, state solveState, -) (bool, float64, float64) { - for _, r := range state.existing { - if r.NodeID == store.Node.NodeID { - return false, 0, 0 +// preexistingReplicaCheck returns true if no existing replica is present on +// the candidate's node. +func preexistingReplicaCheck(nodeID roachpb.NodeID, existing []roachpb.ReplicaDescriptor) bool { + for _, r := range existing { + if r.NodeID == nodeID { + return false } } - return true, 0, 0 + return true } -// storeHasConstraint returns whether a store descriptor attributes or locality +// storeHasConstraint returns whether a store's attributes or node's locality // matches the key value pair in the constraint. func storeHasConstraint(store roachpb.StoreDescriptor, c config.Constraint) bool { if c.Key == "" { @@ -301,112 +391,73 @@ func storeHasConstraint(store roachpb.StoreDescriptor, c config.Constraint) bool return false } -// ruleConstraints returns true iff all required and prohibited constraints are +// constraintCheck returns true iff all required and prohibited constraints are // satisfied. Stores with attributes or localities that match the most positive // constraints return higher scores. -func ruleConstraints(store roachpb.StoreDescriptor, state solveState) (bool, float64, float64) { - if len(state.constraints.Constraints) == 0 { - return true, 0, 0 +func constraintCheck(store roachpb.StoreDescriptor, constraints config.Constraints) (bool, int) { + if len(constraints.Constraints) == 0 { + return true, 0 } - matched := 0 - for _, c := range state.constraints.Constraints { - hasConstraint := storeHasConstraint(store, c) + positive := 0 + for _, constraint := range constraints.Constraints { + hasConstraint := storeHasConstraint(store, constraint) switch { - case c.Type == config.Constraint_REQUIRED && !hasConstraint: - return false, 0, 0 - case c.Type == config.Constraint_PROHIBITED && hasConstraint: - return false, 0, 0 - case (c.Type == config.Constraint_POSITIVE && hasConstraint) || - (c.Type == config.Constraint_REQUIRED && hasConstraint) || - (c.Type == config.Constraint_PROHIBITED && !hasConstraint): - matched++ + case constraint.Type == config.Constraint_REQUIRED && !hasConstraint: + return false, 0 + case constraint.Type == config.Constraint_PROHIBITED && hasConstraint: + return false, 0 + case (constraint.Type == config.Constraint_POSITIVE && hasConstraint): + positive++ } } - - return true, float64(matched) / float64(len(state.constraints.Constraints)), 0 + return true, positive } -// ruleDiversity returns higher scores for stores with the fewest locality tiers -// in common with already existing replicas. It always returns true. -func ruleDiversity(store roachpb.StoreDescriptor, state solveState) (bool, float64, float64) { +// diversityScore returns a score between 1 and 0 where higher scores are stores +// with the fewest locality tiers in common with already existing replicas. +func diversityScore( + store roachpb.StoreDescriptor, existingNodeLocalities map[roachpb.NodeID]roachpb.Locality, +) float64 { minScore := 1.0 - for _, locality := range state.existingNodeLocalities { + for _, locality := range existingNodeLocalities { if newScore := store.Node.Locality.DiversityScore(locality); newScore < minScore { minScore = newScore } } - return true, minScore, 0 + return minScore } -/* -// ruleDiversityExisting returns higher scores for stores with the fewest locality tiers -// in common with already existing replicas. It always returns true. -func ruleDiversity(store roachpb.StoreDescriptor, state solveState) (bool, float64, float64) { - minScore := 1.0 - for _, locality := range state.existingNodeLocalities { - if newScore := store.Node.Locality.DiversityScore(locality); newScore < minScore { - minScore = newScore +// diversityRemovalScore is similar to diversityScore but instead of calculating +// the score if a new node is added, it calculates the remaining diversity if a +// node is removed. +func diversityRemovalScore( + nodeID roachpb.NodeID, existingNodeLocalities map[roachpb.NodeID]roachpb.Locality, +) float64 { + var maxScore float64 + for nodeIDx, localityX := range existingNodeLocalities { + if nodeIDx == nodeID { + continue + } + for nodeIDy, localityY := range existingNodeLocalities { + if nodeIDy == nodeID || nodeIDx >= nodeIDy { + continue + } + if newScore := localityX.DiversityScore(localityY); newScore > maxScore { + maxScore = newScore + } } } - return true, minScore, 0 -} -*/ - -// ruleCapacity returns a balance score that is inversely proportional to the -// number of ranges on the candidate store such that the most empty store will -// have the highest scores. Scores are always between 0 and 1. -func ruleCapacity(store roachpb.StoreDescriptor, state solveState) (bool, float64, float64) { - return true, 0, 1 / float64(store.Capacity.RangeCount+1) -} - -// ruleCapacityMax ensures that we don't try to overfill a store. -func ruleCapacityMax(store roachpb.StoreDescriptor, state solveState) (bool, float64, float64) { - if store.Capacity.FractionUsed() > maxFractionUsedThreshold { - return false, 0, 0 - } - return true, 0, 0 -} - -// ruleCapacityFromMean is designed for removals and yields a lower constraint -// score if the removal of this store would push the store closer away from -// the mean number of ranges. -func ruleCapacityFromMean(store roachpb.StoreDescriptor, state solveState) (bool, float64, float64) { - if rebalanceFromConvergesOnMean(state.sl, store) { - return true, 0, 0 - } - return true, 0.1, 0 + return maxScore } -// ruleCapacityToMean is designed for rebalancing and yields a higher constraint -// score if the addition of this store would push the store closer to the mean -// number of ranges. -func ruleCapacityToMean(store roachpb.StoreDescriptor, state solveState) (bool, float64, float64) { - if rebalanceToConvergesOnMean(state.sl, store) { - return true, 0, 0 - } - return true, 0.1, 0 +// capacityScore returns a score between 0 and 1 that is inversely proportional +// to the number of ranges on the store such that the most empty store will have +// the highest scores. +func capacityScore(store roachpb.StoreDescriptor) float64 { + return 1.0 / float64(store.Capacity.RangeCount+1) } -// byScore implements sort.Interface to sort by scores. -type byScore candidateList - -var _ sort.Interface = byScore(nil) - -func (c byScore) Len() int { return len(c) } -func (c byScore) Less(i, j int) bool { return c[i].less(c[j]) } -func (c byScore) Swap(i, j int) { c[i], c[j] = c[j], c[i] } - -type byScoreAndID candidateList - -var _ sort.Interface = byScoreAndID(nil) - -func (c byScoreAndID) Len() int { return len(c) } -func (c byScoreAndID) Less(i, j int) bool { - if c[i].constraint == c[j].constraint && - c[i].balance == c[j].balance && - c[i].valid == c[j].valid { - return c[i].store.StoreID < c[j].store.StoreID - } - return c[i].less(c[j]) +// maxCapacityCheck returns true if the store has room for a new replica. +func maxCapacityCheck(store roachpb.StoreDescriptor) bool { + return store.Capacity.FractionUsed() < maxFractionUsedThreshold } -func (c byScoreAndID) Swap(i, j int) { c[i], c[j] = c[j], c[i] } diff --git a/pkg/storage/rule_solver_test.go b/pkg/storage/rule_solver_test.go index ef71bf8302cd..04563ea36822 100644 --- a/pkg/storage/rule_solver_test.go +++ b/pkg/storage/rule_solver_test.go @@ -31,269 +31,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" ) -// TODO(bram): This test suite is not even close to exhaustive. The scores are -// not checked and each rule should have many more test cases. Also add a -// corrupt replica test and remove the 0 range ID used when calling -// getStoreList. -func TestRuleSolver(t *testing.T) { - defer leaktest.AfterTest(t)() - - stopper, _, _, storePool, _ := createTestStorePool( - TestTimeUntilStoreDeadOff, - /* deterministic */ true, - /* defaultNodeLiveness */ true, - ) - defer stopper.Stop() - - storeUSa15 := roachpb.StoreID(1) // us-a-1-5 - storeUSa1 := roachpb.StoreID(2) // us-a-1 - storeUSb := roachpb.StoreID(3) // us-b - storeDead := roachpb.StoreID(4) - storeEurope := roachpb.StoreID(5) // eur-a-1-5 - - mockStorePool(storePool, []roachpb.StoreID{storeUSa15, storeUSa1, storeUSb, storeEurope}, []roachpb.StoreID{storeDead}, nil) - - // tierSetup returns a tier struct constructed using the passed in values. - // If any value is an empty string, it is not included. - tierSetup := func(datacenter, floor, rack, slot string) []roachpb.Tier { - var tiers []roachpb.Tier - if datacenter != "" { - tiers = append(tiers, roachpb.Tier{Key: "datacenter", Value: datacenter}) - } - if floor != "" { - tiers = append(tiers, roachpb.Tier{Key: "floor", Value: floor}) - } - if rack != "" { - tiers = append(tiers, roachpb.Tier{Key: "rack", Value: rack}) - } - if slot != "" { - tiers = append(tiers, roachpb.Tier{Key: "slot", Value: slot}) - } - return tiers - } - - // capacitySetup returns a store capacity in which the total capacity is - // always 100 and available and range count are passed in. - capacitySetup := func(available int64, rangeCount int32) roachpb.StoreCapacity { - return roachpb.StoreCapacity{ - Capacity: 100, - Available: available, - RangeCount: rangeCount, - } - } - - storePool.mu.Lock() - - storePool.mu.storeDetails[storeUSa15].desc.Attrs.Attrs = []string{"a"} - storePool.mu.storeDetails[storeUSa15].desc.Node.Locality.Tiers = tierSetup("us", "a", "1", "5") - storePool.mu.storeDetails[storeUSa15].desc.Capacity = capacitySetup(1, 99) - storePool.mu.nodeLocalities[roachpb.NodeID(storeUSa15)] = storePool.mu.storeDetails[storeUSa15].desc.Node.Locality - - storePool.mu.storeDetails[storeUSa1].desc.Attrs.Attrs = []string{"a", "b"} - storePool.mu.storeDetails[storeUSa1].desc.Node.Locality.Tiers = tierSetup("us", "a", "1", "") - storePool.mu.storeDetails[storeUSa1].desc.Capacity = capacitySetup(100, 0) - storePool.mu.nodeLocalities[roachpb.NodeID(storeUSa1)] = storePool.mu.storeDetails[storeUSa1].desc.Node.Locality - - storePool.mu.storeDetails[storeUSb].desc.Attrs.Attrs = []string{"a", "b", "c"} - storePool.mu.storeDetails[storeUSb].desc.Node.Locality.Tiers = tierSetup("us", "b", "", "") - storePool.mu.storeDetails[storeUSb].desc.Capacity = capacitySetup(50, 50) - storePool.mu.nodeLocalities[roachpb.NodeID(storeUSb)] = storePool.mu.storeDetails[storeUSb].desc.Node.Locality - - storePool.mu.storeDetails[storeEurope].desc.Node.Locality.Tiers = tierSetup("eur", "a", "1", "5") - storePool.mu.storeDetails[storeEurope].desc.Capacity = capacitySetup(60, 40) - storePool.mu.nodeLocalities[roachpb.NodeID(storeEurope)] = storePool.mu.storeDetails[storeEurope].desc.Node.Locality - - storePool.mu.Unlock() - - testCases := []struct { - name string - rule rule - c config.Constraints - existing []roachpb.ReplicaDescriptor - expectedValid []roachpb.StoreID - expectedInvalid []roachpb.StoreID - }{ - { - name: "no constraints or rules", - expectedValid: []roachpb.StoreID{storeEurope, storeUSb, storeUSa1, storeUSa15}, - }, - { - name: "white list rule", - rule: func(store roachpb.StoreDescriptor, _ solveState) (bool, float64, float64) { - switch store.StoreID { - case storeUSa15: - return true, 0, 0 - case storeUSb: - return true, 1, 0 - default: - return false, 0, 0 - } - }, - expectedValid: []roachpb.StoreID{storeUSb, storeUSa15}, - expectedInvalid: []roachpb.StoreID{storeEurope, storeUSa1}, - }, - { - name: "ruleReplicasUniqueNodes - 2 available nodes", - rule: ruleReplicasUniqueNodes, - existing: []roachpb.ReplicaDescriptor{ - {NodeID: roachpb.NodeID(storeUSa15)}, - {NodeID: roachpb.NodeID(storeUSb)}, - }, - expectedValid: []roachpb.StoreID{storeEurope, storeUSa1}, - expectedInvalid: []roachpb.StoreID{storeUSb, storeUSa15}, - }, - { - name: "ruleReplicasUniqueNodes - 0 available nodes", - rule: ruleReplicasUniqueNodes, - existing: []roachpb.ReplicaDescriptor{ - {NodeID: roachpb.NodeID(storeUSa15)}, - {NodeID: roachpb.NodeID(storeUSa1)}, - {NodeID: roachpb.NodeID(storeUSb)}, - {NodeID: roachpb.NodeID(storeEurope)}, - }, - expectedValid: nil, - expectedInvalid: []roachpb.StoreID{storeEurope, storeUSb, storeUSa1, storeUSa15}, - }, - { - name: "ruleConstraints - required constraints", - rule: ruleConstraints, - c: config.Constraints{ - Constraints: []config.Constraint{ - {Value: "b", Type: config.Constraint_REQUIRED}, - }, - }, - expectedValid: []roachpb.StoreID{storeUSb, storeUSa1}, - expectedInvalid: []roachpb.StoreID{storeEurope, storeUSa15}, - }, - { - name: "ruleConstraints - required locality constraints", - rule: ruleConstraints, - c: config.Constraints{ - Constraints: []config.Constraint{ - {Key: "datacenter", Value: "us", Type: config.Constraint_REQUIRED}, - }, - }, - expectedValid: []roachpb.StoreID{storeUSb, storeUSa1, storeUSa15}, - expectedInvalid: []roachpb.StoreID{storeEurope}, - }, - { - name: "ruleConstraints - prohibited constraints", - rule: ruleConstraints, - c: config.Constraints{ - Constraints: []config.Constraint{ - {Value: "b", Type: config.Constraint_PROHIBITED}, - }, - }, - expectedValid: []roachpb.StoreID{storeEurope, storeUSa15}, - expectedInvalid: []roachpb.StoreID{storeUSb, storeUSa1}, - }, - { - name: "ruleConstraints - prohibited locality constraints", - rule: ruleConstraints, - c: config.Constraints{ - Constraints: []config.Constraint{ - {Key: "datacenter", Value: "us", Type: config.Constraint_PROHIBITED}, - }, - }, - expectedValid: []roachpb.StoreID{storeEurope}, - expectedInvalid: []roachpb.StoreID{storeUSb, storeUSa1, storeUSa15}, - }, - { - name: "ruleConstraints - positive constraints", - rule: ruleConstraints, - c: config.Constraints{ - Constraints: []config.Constraint{ - {Value: "a"}, - {Value: "b"}, - {Value: "c"}, - }, - }, - expectedValid: []roachpb.StoreID{storeUSb, storeUSa1, storeUSa15, storeEurope}, - }, - { - name: "ruleConstraints - positive locality constraints", - rule: ruleConstraints, - c: config.Constraints{ - Constraints: []config.Constraint{ - {Key: "datacenter", Value: "eur"}, - }, - }, - expectedValid: []roachpb.StoreID{storeEurope, storeUSb, storeUSa1, storeUSa15}, - }, - { - name: "ruleDiversity - no existing replicas", - rule: ruleDiversity, - expectedValid: []roachpb.StoreID{storeEurope, storeUSb, storeUSa1, storeUSa15}, - }, - { - name: "ruleDiversity - one existing replicas", - rule: ruleDiversity, - existing: []roachpb.ReplicaDescriptor{ - {NodeID: roachpb.NodeID(storeUSa15)}, - }, - expectedValid: []roachpb.StoreID{storeEurope, storeUSb, storeUSa1, storeUSa15}, - }, - { - name: "ruleDiversity - two existing replicas", - rule: ruleDiversity, - existing: []roachpb.ReplicaDescriptor{ - {NodeID: roachpb.NodeID(storeUSa15)}, - {NodeID: roachpb.NodeID(storeEurope)}, - }, - expectedValid: []roachpb.StoreID{storeUSb, storeUSa1, storeEurope, storeUSa15}, - }, - { - name: "ruleCapacityMax", - rule: ruleCapacityMax, - expectedValid: []roachpb.StoreID{storeEurope, storeUSb, storeUSa1}, - expectedInvalid: []roachpb.StoreID{storeUSa15}, - }, - { - name: "ruleCapacity", - rule: ruleCapacity, - expectedValid: []roachpb.StoreID{storeUSa1, storeEurope, storeUSb, storeUSa15}, - }, - } - - for _, tc := range testCases { - t.Run(tc.name, func(t *testing.T) { - var solver ruleSolver - if tc.rule != nil { - solver = ruleSolver{tc.rule} - } - sl, _, _ := storePool.getStoreList(roachpb.RangeID(0)) - candidates := solver.Solve( - sl, - tc.c, - tc.existing, - storePool.getNodeLocalities(tc.existing), - storePool.deterministic, - ) - valid := candidates.onlyValid() - invalid := candidates[len(valid):] - - if len(valid) != len(tc.expectedValid) { - t.Fatalf("length of valid %+v should match %+v", valid, tc.expectedValid) - } - for i, expected := range tc.expectedValid { - if actual := valid[i].store.StoreID; actual != expected { - t.Errorf("valid[%d].store.StoreID = %d; not %d; %+v", - i, actual, expected, valid) - } - } - if len(invalid) != len(tc.expectedInvalid) { - t.Fatalf("length of invalids %+v should match %+v", invalid, tc.expectedInvalid) - } - for i, expected := range tc.expectedInvalid { - if actual := invalid[i].store.StoreID; actual != expected { - t.Errorf("invalid[%d].store.StoreID = %d; not %d; %+v", - i, actual, expected, invalid) - } - } - }) - } -} - func TestOnlyValid(t *testing.T) { defer leaktest.AfterTest(t)() @@ -340,14 +77,17 @@ func TestCandidateSelection(t *testing.T) { type scoreTuple struct { constraint int - balance int + capacity int } - genCandidates := func(scores []scoreTuple) candidateList { + genCandidates := func(scores []scoreTuple, idShift int) candidateList { var cl candidateList - for _, score := range scores { + for i, score := range scores { cl = append(cl, candidate{ + store: roachpb.StoreDescriptor{ + StoreID: roachpb.StoreID(i + idShift), + }, constraint: float64(score.constraint), - balance: float64(score.balance), + capacity: float64(score.capacity), valid: true, }) } @@ -361,7 +101,7 @@ func TestCandidateSelection(t *testing.T) { if i != 0 { buffer.WriteRune(',') } - buffer.WriteString(fmt.Sprintf("%d:%d", int(c.constraint), int(c.balance))) + buffer.WriteString(fmt.Sprintf("%d:%d", int(c.constraint), int(c.capacity))) } return buffer.String() } @@ -424,29 +164,56 @@ func TestCandidateSelection(t *testing.T) { }, } + pickResult := func(cl candidateList, storeID roachpb.StoreID) *candidate { + for _, c := range cl { + if c.store.StoreID == storeID { + return &c + } + } + return nil + } + allocRand := makeAllocatorRand(rand.NewSource(0)) for _, tc := range testCases { - cl := genCandidates(tc.candidates) + cl := genCandidates(tc.candidates, 1) t.Run(fmt.Sprintf("best-%s", formatter(cl)), func(t *testing.T) { - if a, e := cl.best(), genCandidates(tc.best); !reflect.DeepEqual(a, e) { + if a, e := cl.best(), genCandidates(tc.best, 1); !reflect.DeepEqual(a, e) { t.Errorf("expected:%s actual:%s diff:%v", formatter(e), formatter(a), pretty.Diff(e, a)) } }) t.Run(fmt.Sprintf("worst-%s", formatter(cl)), func(t *testing.T) { - if a, e := cl.worst(), genCandidates(tc.worst); !reflect.DeepEqual(a, e) { + // Shifting the ids is required to match the end of the list. + if a, e := cl.worst(), genCandidates( + tc.worst, + len(tc.candidates)-len(tc.worst)+1, + ); !reflect.DeepEqual(a, e) { t.Errorf("expected:%s actual:%s diff:%v", formatter(e), formatter(a), pretty.Diff(e, a)) } }) t.Run(fmt.Sprintf("good-%s", formatter(cl)), func(t *testing.T) { - good := cl.selectGood(allocRand) - actual := scoreTuple{int(good.constraint), int(good.balance)} + goodStore := cl.selectGood(allocRand) + if goodStore == nil { + t.Fatalf("no good store found") + } + good := pickResult(cl, goodStore.StoreID) + if good == nil { + t.Fatalf("candidate for store %d not found in candidate list: %s", goodStore.StoreID, cl) + } + actual := scoreTuple{int(good.constraint), int(good.capacity)} if actual != tc.good { t.Errorf("expected:%v actual:%v", tc.good, actual) } }) t.Run(fmt.Sprintf("bad-%s", formatter(cl)), func(t *testing.T) { - bad := cl.selectBad(allocRand) - actual := scoreTuple{int(bad.constraint), int(bad.balance)} + badStore := cl.selectBad(allocRand) + if badStore == nil { + t.Fatalf("no bad store found") + } + bad := pickResult(cl, badStore.StoreID) + if bad == nil { + t.Fatalf("candidate for store %d not found in candidate list: %s", badStore.StoreID, cl) + } + actual := scoreTuple{int(bad.constraint), int(bad.capacity)} if actual != tc.bad { t.Errorf("expected:%v actual:%v", tc.bad, actual) } @@ -461,57 +228,57 @@ func TestBetterThan(t *testing.T) { { valid: true, constraint: 1, - balance: 1, + capacity: 1, }, { valid: true, constraint: 1, - balance: 1, + capacity: 1, }, { valid: true, constraint: 1, - balance: 0, + capacity: 0, }, { valid: true, constraint: 1, - balance: 0, + capacity: 0, }, { valid: true, constraint: 0, - balance: 1, + capacity: 1, }, { valid: true, constraint: 0, - balance: 1, + capacity: 1, }, { valid: true, constraint: 0, - balance: 0, + capacity: 0, }, { valid: true, constraint: 0, - balance: 0, + capacity: 0, }, { valid: false, constraint: 1, - balance: 0.5, + capacity: 0.5, }, { valid: false, constraint: 0, - balance: 0.5, + capacity: 0.5, }, { valid: false, constraint: 0, - balance: 0, + capacity: 0, }, } @@ -524,3 +291,369 @@ func TestBetterThan(t *testing.T) { } } } + +func TestPreexistingReplicaCheck(t *testing.T) { + defer leaktest.AfterTest(t)() + + var existing []roachpb.ReplicaDescriptor + for i := 2; i < 10; i += 2 { + existing = append(existing, roachpb.ReplicaDescriptor{NodeID: roachpb.NodeID(i)}) + } + for i := 1; i < 10; i++ { + if e, a := i%2 != 0, preexistingReplicaCheck(roachpb.NodeID(i), existing); e != a { + t.Errorf("NodeID %d expected to be %t, got %t", i, e, a) + } + } +} + +// testStoreTierSetup returns a tier struct constructed using the passed in values. +// If any value is an empty string, it is not included. +func testStoreTierSetup(datacenter, floor, rack, slot string) []roachpb.Tier { + var tiers []roachpb.Tier + if datacenter != "" { + tiers = append(tiers, roachpb.Tier{Key: "datacenter", Value: datacenter}) + } + if floor != "" { + tiers = append(tiers, roachpb.Tier{Key: "floor", Value: floor}) + } + if rack != "" { + tiers = append(tiers, roachpb.Tier{Key: "rack", Value: rack}) + } + if slot != "" { + tiers = append(tiers, roachpb.Tier{Key: "slot", Value: slot}) + } + return tiers +} + +// testStoreCapacitySetup returns a store capacity in which the total capacity +// is always 100 and available and range count are passed in. +func testStoreCapacitySetup(available int64, rangeCount int32) roachpb.StoreCapacity { + return roachpb.StoreCapacity{ + Capacity: 100, + Available: available, + RangeCount: rangeCount, + } +} + +// This is a collection of test stores used by a suite of tests. +var ( + testStoreUSa15 = roachpb.StoreID(1) // us-a-1-5 + testStoreUSa1 = roachpb.StoreID(2) // us-a-1 + testStoreUSb = roachpb.StoreID(3) // us-b + testStoreEurope = roachpb.StoreID(4) // eur-a-1-5 + + testStores = []roachpb.StoreDescriptor{ + { + StoreID: testStoreUSa15, + Attrs: roachpb.Attributes{ + Attrs: []string{"a"}, + }, + Node: roachpb.NodeDescriptor{ + NodeID: roachpb.NodeID(testStoreUSa15), + Locality: roachpb.Locality{ + Tiers: testStoreTierSetup("us", "a", "1", "5"), + }, + }, + Capacity: testStoreCapacitySetup(1, 99), + }, + { + StoreID: testStoreUSa1, + Attrs: roachpb.Attributes{ + Attrs: []string{"a", "b"}, + }, + Node: roachpb.NodeDescriptor{ + NodeID: roachpb.NodeID(testStoreUSa1), + Locality: roachpb.Locality{ + Tiers: testStoreTierSetup("us", "a", "1", ""), + }, + }, + Capacity: testStoreCapacitySetup(100, 0), + }, + { + StoreID: testStoreUSb, + Attrs: roachpb.Attributes{ + Attrs: []string{"a", "b", "c"}, + }, + Node: roachpb.NodeDescriptor{ + NodeID: roachpb.NodeID(testStoreUSb), + Locality: roachpb.Locality{ + Tiers: testStoreTierSetup("us", "b", "", ""), + }, + }, + Capacity: testStoreCapacitySetup(50, 50), + }, + { + StoreID: testStoreEurope, + Node: roachpb.NodeDescriptor{ + NodeID: roachpb.NodeID(testStoreEurope), + Locality: roachpb.Locality{ + Tiers: testStoreTierSetup("eur", "a", "1", "5"), + }, + }, + Capacity: testStoreCapacitySetup(60, 40), + }, + } +) + +func TestConstraintCheck(t *testing.T) { + defer leaktest.AfterTest(t)() + + testCases := []struct { + name string + constraints []config.Constraint + expected map[roachpb.StoreID]int + }{ + { + name: "required constraint", + constraints: []config.Constraint{ + {Value: "b", Type: config.Constraint_REQUIRED}, + }, + expected: map[roachpb.StoreID]int{ + testStoreUSa1: 0, + testStoreUSb: 0, + }, + }, + { + name: "required locality constraints", + constraints: []config.Constraint{ + {Key: "datacenter", Value: "us", Type: config.Constraint_REQUIRED}, + }, + expected: map[roachpb.StoreID]int{ + testStoreUSa15: 0, + testStoreUSa1: 0, + testStoreUSb: 0, + }, + }, + { + name: "prohibited constraints", + constraints: []config.Constraint{ + {Value: "b", Type: config.Constraint_PROHIBITED}, + }, + expected: map[roachpb.StoreID]int{ + testStoreUSa15: 0, + testStoreEurope: 0, + }, + }, + { + name: "prohibited locality constraints", + constraints: []config.Constraint{ + {Key: "datacenter", Value: "us", Type: config.Constraint_PROHIBITED}, + }, + expected: map[roachpb.StoreID]int{ + testStoreEurope: 0, + }, + }, + { + name: "positive constraints", + constraints: []config.Constraint{ + {Value: "a"}, + {Value: "b"}, + {Value: "c"}, + }, + expected: map[roachpb.StoreID]int{ + testStoreUSa15: 1, + testStoreUSa1: 2, + testStoreUSb: 3, + testStoreEurope: 0, + }, + }, + { + name: "positive locality constraints", + constraints: []config.Constraint{ + {Key: "datacenter", Value: "eur"}, + }, + expected: map[roachpb.StoreID]int{ + testStoreUSa15: 0, + testStoreUSa1: 0, + testStoreUSb: 0, + testStoreEurope: 1, + }, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + for _, s := range testStores { + valid, positive := constraintCheck(s, config.Constraints{Constraints: tc.constraints}) + expectedPositive, ok := tc.expected[s.StoreID] + if valid != ok { + t.Errorf("expected store %d to be %t, but got %t", s.StoreID, ok, valid) + continue + } + if positive != expectedPositive { + t.Errorf("expected store %d to have %d positives, but got %d", s.StoreID, expectedPositive, positive) + } + } + }) + } +} + +func TestDiversityScore(t *testing.T) { + defer leaktest.AfterTest(t)() + + testCases := []struct { + name string + existing []roachpb.NodeID + expected map[roachpb.StoreID]float64 + }{ + { + name: "no existing replicas", + expected: map[roachpb.StoreID]float64{ + testStoreUSa15: 1, + testStoreUSa1: 1, + testStoreUSb: 1, + testStoreEurope: 1, + }, + }, + { + name: "one existing replicas", + existing: []roachpb.NodeID{ + roachpb.NodeID(testStoreUSa15), + }, + expected: map[roachpb.StoreID]float64{ + testStoreUSa15: 0, + testStoreUSa1: 1.0 / 4.0, + testStoreUSb: 1.0 / 2.0, + testStoreEurope: 1, + }, + }, + { + name: "two existing replicas", + existing: []roachpb.NodeID{ + roachpb.NodeID(testStoreUSa15), + roachpb.NodeID(testStoreEurope), + }, + expected: map[roachpb.StoreID]float64{ + testStoreUSa15: 0, + testStoreUSa1: 1.0 / 4.0, + testStoreUSb: 1.0 / 2.0, + testStoreEurope: 0, + }, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + existingNodeLocalities := make(map[roachpb.NodeID]roachpb.Locality) + for _, nodeID := range tc.existing { + for _, s := range testStores { + if s.Node.NodeID == nodeID { + existingNodeLocalities[roachpb.NodeID(s.Node.NodeID)] = s.Node.Locality + } + } + } + for _, s := range testStores { + actualScore := diversityScore(s, existingNodeLocalities) + expectedScore, ok := tc.expected[s.StoreID] + if !ok { + t.Fatalf("no expected score found for storeID %d", s.StoreID) + } + if actualScore != expectedScore { + t.Errorf("store %d expected diversity score: %.2f, actual %.2f", s.StoreID, expectedScore, actualScore) + } + } + }) + } +} + +func TestDiversityRemovalScore(t *testing.T) { + defer leaktest.AfterTest(t)() + + testCases := []struct { + name string + expected map[roachpb.StoreID]float64 + }{ + { + name: "four existing replicas", + expected: map[roachpb.StoreID]float64{ + testStoreUSa15: 1, + testStoreUSa1: 1, + testStoreUSb: 1, + testStoreEurope: 1.0 / 2.0, + }, + }, + { + name: "three existing replicas - testStoreUSa15", + expected: map[roachpb.StoreID]float64{ + testStoreUSa1: 1, + testStoreUSb: 1, + testStoreEurope: 1.0 / 2.0, + }, + }, + { + name: "three existing replicas - testStoreUSa1", + expected: map[roachpb.StoreID]float64{ + testStoreUSa15: 1, + testStoreUSb: 1, + testStoreEurope: 1.0 / 2.0, + }, + }, + { + name: "three existing replicas - testStoreUSb", + expected: map[roachpb.StoreID]float64{ + testStoreUSa15: 1, + testStoreUSa1: 1, + testStoreEurope: 1.0 / 4.0, + }, + }, + { + name: "three existing replicas - testStoreEurope", + expected: map[roachpb.StoreID]float64{ + testStoreUSa15: 1.0 / 2.0, + testStoreUSa1: 1.0 / 2.0, + testStoreUSb: 1.0 / 4.0, + }, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + existingNodeLocalities := make(map[roachpb.NodeID]roachpb.Locality) + for _, s := range testStores { + if _, ok := tc.expected[s.StoreID]; ok { + existingNodeLocalities[roachpb.NodeID(s.Node.NodeID)] = s.Node.Locality + } + } + for _, s := range testStores { + if _, ok := tc.expected[s.StoreID]; !ok { + continue + } + actualScore := diversityRemovalScore(s.Node.NodeID, existingNodeLocalities) + expectedScore, ok := tc.expected[s.StoreID] + if !ok { + t.Fatalf("no expected score found for storeID %d", s.StoreID) + } + if actualScore != expectedScore { + t.Errorf("store %d expected diversity removal score: %.2f, actual %.2f", s.StoreID, expectedScore, actualScore) + } + } + }) + } +} + +// TestCapacityScore tests both capacityScore and maxCapacityCheck. +func TestCapacityScore(t *testing.T) { + defer leaktest.AfterTest(t)() + + expectedCheck := map[roachpb.StoreID]bool{ + testStoreUSa15: false, + testStoreUSa1: true, + testStoreUSb: true, + testStoreEurope: true, + } + expectedScore := map[roachpb.StoreID]float64{ + testStoreUSa15: 1.0 / 100.0, + testStoreUSa1: 1.0, + testStoreUSb: 1.0 / 51.0, + testStoreEurope: 1.0 / 41.0, + } + + for _, s := range testStores { + if e, a := expectedScore[s.StoreID], capacityScore(s); e != a { + t.Errorf("store %d expected capacity score: %.2f, actual %.2f", s.StoreID, e, a) + } + if e, a := expectedCheck[s.StoreID], maxCapacityCheck(s); e != a { + t.Errorf("store %d expected max capacity check: %t, actual %t", s.StoreID, e, a) + } + } +}