Skip to content

Commit

Permalink
storage: collection of fixes and updates for the rules solver
Browse files Browse the repository at this point in the history
- 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 cockroachdb#10275
  • Loading branch information
BramGruneir committed Dec 12, 2016
1 parent b1357d0 commit 5a4a926
Show file tree
Hide file tree
Showing 3 changed files with 248 additions and 173 deletions.
107 changes: 31 additions & 76 deletions pkg/storage/allocator.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down Expand Up @@ -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 {
Expand All @@ -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")
}

Expand Down Expand Up @@ -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
}

Expand Down
144 changes: 90 additions & 54 deletions pkg/storage/rule_solver.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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
Expand All @@ -56,66 +78,76 @@ 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,
existing: existing,
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
Expand All @@ -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] }
Loading

0 comments on commit 5a4a926

Please sign in to comment.