diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index b90dc7d28172..82e7af98e5f0 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -136,6 +136,7 @@ ALL_TESTS = [ "//pkg/kv/kvnemesis:kvnemesis_test", "//pkg/kv/kvprober:kvprober_test", "//pkg/kv/kvserver/abortspan:abortspan_test", + "//pkg/kv/kvserver/allocator/storepool:storepool_test", "//pkg/kv/kvserver/apply:apply_test", "//pkg/kv/kvserver/asim:asim_test", "//pkg/kv/kvserver/batcheval/result:result_test", diff --git a/pkg/cli/BUILD.bazel b/pkg/cli/BUILD.bazel index f3f7ec6274ef..83ebab069d9c 100644 --- a/pkg/cli/BUILD.bazel +++ b/pkg/cli/BUILD.bazel @@ -340,6 +340,7 @@ go_test( "//pkg/keys", "//pkg/kv", "//pkg/kv/kvserver", + "//pkg/kv/kvserver/allocator/storepool", "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/kv/kvserver/loqrecovery/loqrecoverypb", "//pkg/kv/kvserver/stateloader", diff --git a/pkg/cli/zip_test.go b/pkg/cli/zip_test.go index a8b635519cba..c22a3e528535 100644 --- a/pkg/cli/zip_test.go +++ b/pkg/cli/zip_test.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server" @@ -439,7 +440,7 @@ func TestPartialZip(t *testing.T) { // is no risk to see the override bumped due to a gossip update // because this setting is not otherwise set in the test cluster. s := tc.Server(0) - kvserver.TimeUntilStoreDead.Override(ctx, &s.ClusterSettings().SV, kvserver.TestTimeUntilStoreDead) + storepool.TimeUntilStoreDead.Override(ctx, &s.ClusterSettings().SV, storepool.TestTimeUntilStoreDead) // This last case may take a little while to converge. To make this work with datadriven and at the same // time retain the ability to use the `-rewrite` flag, we use a retry loop within that already checks the diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index f6883663b108..f183ab296fea 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -83,7 +83,6 @@ go_library( "store_create_replica.go", "store_init.go", "store_merge.go", - "store_pool.go", "store_raft.go", "store_rebalancer.go", "store_remove_replica.go", @@ -117,6 +116,8 @@ go_library( "//pkg/kv/kvbase", "//pkg/kv/kvclient/rangecache", "//pkg/kv/kvserver/abortspan", + "//pkg/kv/kvserver/allocator", + "//pkg/kv/kvserver/allocator/storepool", "//pkg/kv/kvserver/apply", "//pkg/kv/kvserver/batcheval", "//pkg/kv/kvserver/batcheval/result", @@ -234,6 +235,7 @@ go_test( "client_split_burst_test.go", "client_split_test.go", "client_status_test.go", + "client_store_pool_test.go", "client_store_test.go", "client_tenant_test.go", "client_test.go", @@ -296,7 +298,6 @@ go_test( "split_queue_test.go", "split_trigger_helper_test.go", "stats_test.go", - "store_pool_test.go", "store_rebalancer_test.go", "store_replica_btree_test.go", "store_test.go", @@ -322,6 +323,8 @@ go_test( "//pkg/kv/kvclient/kvcoord", "//pkg/kv/kvclient/rangefeed/rangefeedcache", "//pkg/kv/kvserver/abortspan", + "//pkg/kv/kvserver/allocator", + "//pkg/kv/kvserver/allocator/storepool", "//pkg/kv/kvserver/apply", "//pkg/kv/kvserver/batcheval", "//pkg/kv/kvserver/batcheval/result", diff --git a/pkg/kv/kvserver/allocator.go b/pkg/kv/kvserver/allocator.go index c333b2f29147..5cccb567b182 100644 --- a/pkg/kv/kvserver/allocator.go +++ b/pkg/kv/kvserver/allocator.go @@ -20,6 +20,8 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/constraint" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -342,16 +344,8 @@ func makeAllocatorRand(source rand.Source) allocatorRand { } } -// RangeUsageInfo contains usage information (sizes and traffic) needed by the -// allocator to make rebalancing decisions for a given range. -type RangeUsageInfo struct { - LogicalBytes int64 - QueriesPerSecond float64 - WritesPerSecond float64 -} - -func rangeUsageInfoForRepl(repl *Replica) RangeUsageInfo { - info := RangeUsageInfo{ +func rangeUsageInfoForRepl(repl *Replica) allocator.RangeUsageInfo { + info := allocator.RangeUsageInfo{ LogicalBytes: repl.GetMVCCStats().Total(), } if queriesPerSecond, dur := repl.leaseholderStats.averageRatePerSecond(); dur >= MinStatsDuration { @@ -478,7 +472,7 @@ type AllocatorMetrics struct { // Allocator tries to spread replicas as evenly as possible across the stores // in the cluster. type Allocator struct { - storePool *StorePool + storePool *storepool.StorePool nodeLatencyFn func(addr string) (time.Duration, bool) // TODO(aayush): Let's replace this with a *rand.Rand that has a rand.Source // wrapped inside a mutex, to avoid misuse. @@ -511,7 +505,7 @@ func makeAllocatorMetrics() AllocatorMetrics { // MakeAllocator creates a new allocator using the specified StorePool. func MakeAllocator( - storePool *StorePool, + storePool *storepool.StorePool, nodeLatencyFn func(addr string) (time.Duration, bool), knobs *AllocatorTestingKnobs, storeMetrics *StoreMetrics, @@ -520,7 +514,7 @@ func MakeAllocator( // There are number of test cases that make a test store but don't add // gossip or a store pool. So we can't rely on the existence of the // store pool in those cases. - if storePool != nil && storePool.deterministic { + if storePool != nil && storePool.Deterministic { randSource = rand.NewSource(777) } else { randSource = rand.NewSource(rand.Int63()) @@ -684,7 +678,7 @@ func (a *Allocator) computeAction( // After that we handle rebalancing related actions, followed by removal // actions. haveVoters := len(voterReplicas) - decommissioningVoters := a.storePool.decommissioningReplicas(voterReplicas) + decommissioningVoters := a.storePool.DecommissioningReplicas(voterReplicas) // Node count including dead nodes but excluding // decommissioning/decommissioned nodes. clusterNodes := a.storePool.ClusterNodeCount() @@ -713,7 +707,7 @@ func (a *Allocator) computeAction( // heartbeat in the recent past. This means we won't move those replicas // elsewhere (for a regular rebalance or for decommissioning). const includeSuspectAndDrainingStores = true - liveVoters, deadVoters := a.storePool.liveAndDeadReplicas(voterReplicas, includeSuspectAndDrainingStores) + liveVoters, deadVoters := a.storePool.LiveAndDeadReplicas(voterReplicas, includeSuspectAndDrainingStores) if len(liveVoters) < quorum { // Do not take any replacement/removal action if we do not have a quorum of @@ -792,7 +786,7 @@ func (a *Allocator) computeAction( return action, action.Priority() } - liveNonVoters, deadNonVoters := a.storePool.liveAndDeadReplicas( + liveNonVoters, deadNonVoters := a.storePool.LiveAndDeadReplicas( nonVoterReplicas, includeSuspectAndDrainingStores, ) if haveNonVoters == neededNonVoters && len(deadNonVoters) > 0 { @@ -803,7 +797,7 @@ func (a *Allocator) computeAction( return action, action.Priority() } - decommissioningNonVoters := a.storePool.decommissioningReplicas(nonVoterReplicas) + decommissioningNonVoters := a.storePool.DecommissioningReplicas(nonVoterReplicas) if haveNonVoters == neededNonVoters && len(decommissioningNonVoters) > 0 { // The range has non-voter(s) on a decommissioning node that we should // replace. @@ -892,7 +886,7 @@ func (a *Allocator) allocateTarget( existingVoters, existingNonVoters []roachpb.ReplicaDescriptor, targetType targetReplicaType, ) (roachpb.ReplicationTarget, string, error) { - candidateStoreList, aliveStoreCount, throttled := a.storePool.getStoreList(storeFilterThrottled) + candidateStoreList, aliveStoreCount, throttled := a.storePool.GetStoreList(storepool.StoreFilterThrottled) target, details := a.allocateTargetFromList( ctx, @@ -955,7 +949,7 @@ func (a *Allocator) AllocateNonVoter( func (a *Allocator) allocateTargetFromList( ctx context.Context, - candidateStores StoreList, + candidateStores storepool.StoreList, conf roachpb.SpanConfig, existingVoters, existingNonVoters []roachpb.ReplicaDescriptor, options *rangeCountScorerOptions, @@ -963,9 +957,9 @@ func (a *Allocator) allocateTargetFromList( targetType targetReplicaType, ) (roachpb.ReplicationTarget, string) { existingReplicas := append(existingVoters, existingNonVoters...) - analyzedOverallConstraints := constraint.AnalyzeConstraints(ctx, a.storePool.getStoreDescriptor, + analyzedOverallConstraints := constraint.AnalyzeConstraints(ctx, a.storePool.GetStoreDescriptor, existingReplicas, conf.NumReplicas, conf.Constraints) - analyzedVoterConstraints := constraint.AnalyzeConstraints(ctx, a.storePool.getStoreDescriptor, + analyzedVoterConstraints := constraint.AnalyzeConstraints(ctx, a.storePool.GetStoreDescriptor, existingVoters, conf.GetNumVoters(), conf.VoterConstraints) var constraintsChecker constraintsCheckFn @@ -993,8 +987,8 @@ func (a *Allocator) allocateTargetFromList( candidateStores, constraintsChecker, existingReplicaSet, - a.storePool.getLocalitiesByStore(existingReplicaSet), - a.storePool.isStoreReadyForRoutineReplicaTransfer, + a.storePool.GetLocalitiesByStore(existingReplicaSet), + a.storePool.IsStoreReadyForRoutineReplicaTransfer, allowMultipleReplsPerNode, options, ) @@ -1022,14 +1016,14 @@ func (a Allocator) simulateRemoveTarget( candidates []roachpb.ReplicaDescriptor, existingVoters []roachpb.ReplicaDescriptor, existingNonVoters []roachpb.ReplicaDescriptor, - sl StoreList, - rangeUsageInfo RangeUsageInfo, + sl storepool.StoreList, + rangeUsageInfo allocator.RangeUsageInfo, targetType targetReplicaType, options scorerOptions, ) (roachpb.ReplicationTarget, string, error) { candidateStores := make([]roachpb.StoreDescriptor, 0, len(candidates)) for _, cand := range candidates { - for _, store := range sl.stores { + for _, store := range sl.Stores { if cand.StoreID == store.StoreID { candidateStores = append(candidateStores, store) } @@ -1039,8 +1033,8 @@ func (a Allocator) simulateRemoveTarget( // Update statistics first switch t := targetType; t { case voterTarget: - a.storePool.updateLocalStoreAfterRebalance(targetStore, rangeUsageInfo, roachpb.ADD_VOTER) - defer a.storePool.updateLocalStoreAfterRebalance( + a.storePool.UpdateLocalStoreAfterRebalance(targetStore, rangeUsageInfo, roachpb.ADD_VOTER) + defer a.storePool.UpdateLocalStoreAfterRebalance( targetStore, rangeUsageInfo, roachpb.REMOVE_VOTER, @@ -1049,12 +1043,12 @@ func (a Allocator) simulateRemoveTarget( targetStore) return a.removeTarget( - ctx, conf, makeStoreList(candidateStores), + ctx, conf, storepool.MakeStoreList(candidateStores), existingVoters, existingNonVoters, voterTarget, options, ) case nonVoterTarget: - a.storePool.updateLocalStoreAfterRebalance(targetStore, rangeUsageInfo, roachpb.ADD_NON_VOTER) - defer a.storePool.updateLocalStoreAfterRebalance( + a.storePool.UpdateLocalStoreAfterRebalance(targetStore, rangeUsageInfo, roachpb.ADD_NON_VOTER) + defer a.storePool.UpdateLocalStoreAfterRebalance( targetStore, rangeUsageInfo, roachpb.REMOVE_NON_VOTER, @@ -1062,7 +1056,7 @@ func (a Allocator) simulateRemoveTarget( log.VEventf(ctx, 3, "simulating which non-voter would be removed after adding s%d", targetStore) return a.removeTarget( - ctx, conf, makeStoreList(candidateStores), + ctx, conf, storepool.MakeStoreList(candidateStores), existingVoters, existingNonVoters, nonVoterTarget, options, ) default: @@ -1070,29 +1064,29 @@ func (a Allocator) simulateRemoveTarget( } } -func (a Allocator) storeListForTargets(candidates []roachpb.ReplicationTarget) StoreList { +func (a Allocator) storeListForTargets(candidates []roachpb.ReplicationTarget) storepool.StoreList { result := make([]roachpb.StoreDescriptor, 0, len(candidates)) - sl, _, _ := a.storePool.getStoreList(storeFilterNone) + sl, _, _ := a.storePool.GetStoreList(storepool.StoreFilterNone) for _, cand := range candidates { - for _, store := range sl.stores { + for _, store := range sl.Stores { if cand.StoreID == store.StoreID { result = append(result, store) } } } - return makeStoreList(result) + return storepool.MakeStoreList(result) } func (a Allocator) removeTarget( ctx context.Context, conf roachpb.SpanConfig, - candidateStoreList StoreList, + candidateStoreList storepool.StoreList, existingVoters []roachpb.ReplicaDescriptor, existingNonVoters []roachpb.ReplicaDescriptor, targetType targetReplicaType, options scorerOptions, ) (roachpb.ReplicationTarget, string, error) { - if len(candidateStoreList.stores) == 0 { + if len(candidateStoreList.Stores) == 0 { return roachpb.ReplicationTarget{}, "", errors.Errorf( "must supply at least one" + " candidate replica to allocator.removeTarget()", @@ -1100,9 +1094,9 @@ func (a Allocator) removeTarget( } existingReplicas := append(existingVoters, existingNonVoters...) - analyzedOverallConstraints := constraint.AnalyzeConstraints(ctx, a.storePool.getStoreDescriptor, + analyzedOverallConstraints := constraint.AnalyzeConstraints(ctx, a.storePool.GetStoreDescriptor, existingReplicas, conf.NumReplicas, conf.Constraints) - analyzedVoterConstraints := constraint.AnalyzeConstraints(ctx, a.storePool.getStoreDescriptor, + analyzedVoterConstraints := constraint.AnalyzeConstraints(ctx, a.storePool.GetStoreDescriptor, existingVoters, conf.GetNumVoters(), conf.VoterConstraints) var constraintsChecker constraintsCheckFn @@ -1126,7 +1120,7 @@ func (a Allocator) removeTarget( ctx, candidateStoreList, constraintsChecker, - a.storePool.getLocalitiesByStore(replicaSetForDiversityCalc), + a.storePool.GetLocalitiesByStore(replicaSetForDiversityCalc), options, ) @@ -1167,7 +1161,7 @@ func (a Allocator) RemoveVoter( for i, exist := range voterCandidates { candidateStoreIDs[i] = exist.StoreID } - candidateStoreList, _, _ := a.storePool.getStoreListFromIDs(candidateStoreIDs, storeFilterNone) + candidateStoreList, _, _ := a.storePool.GetStoreListFromIDs(candidateStoreIDs, storepool.StoreFilterNone) return a.removeTarget( ctx, @@ -1198,7 +1192,7 @@ func (a Allocator) RemoveNonVoter( for i, exist := range nonVoterCandidates { candidateStoreIDs[i] = exist.StoreID } - candidateStoreList, _, _ := a.storePool.getStoreListFromIDs(candidateStoreIDs, storeFilterNone) + candidateStoreList, _, _ := a.storePool.GetStoreListFromIDs(candidateStoreIDs, storepool.StoreFilterNone) return a.removeTarget( ctx, @@ -1216,12 +1210,12 @@ func (a Allocator) rebalanceTarget( conf roachpb.SpanConfig, raftStatus *raft.Status, existingVoters, existingNonVoters []roachpb.ReplicaDescriptor, - rangeUsageInfo RangeUsageInfo, - filter storeFilter, + rangeUsageInfo allocator.RangeUsageInfo, + filter storepool.StoreFilter, targetType targetReplicaType, options scorerOptions, ) (add, remove roachpb.ReplicationTarget, details string, ok bool) { - sl, _, _ := a.storePool.getStoreList(filter) + sl, _, _ := a.storePool.GetStoreList(filter) // If we're considering a rebalance due to an `AdminScatterRequest`, we'd like // to ensure that we're returning a random rebalance target to a new store @@ -1233,9 +1227,9 @@ func (a Allocator) rebalanceTarget( zero := roachpb.ReplicationTarget{} analyzedOverallConstraints := constraint.AnalyzeConstraints( - ctx, a.storePool.getStoreDescriptor, existingReplicas, conf.NumReplicas, conf.Constraints) + ctx, a.storePool.GetStoreDescriptor, existingReplicas, conf.NumReplicas, conf.Constraints) analyzedVoterConstraints := constraint.AnalyzeConstraints( - ctx, a.storePool.getStoreDescriptor, existingVoters, conf.GetNumVoters(), conf.VoterConstraints) + ctx, a.storePool.GetStoreDescriptor, existingVoters, conf.GetNumVoters(), conf.VoterConstraints) var removalConstraintsChecker constraintsCheckFn var rebalanceConstraintsChecker rebalanceConstraintsCheckFn var replicaSetToRebalance, replicasWithExcludedStores []roachpb.ReplicaDescriptor @@ -1275,8 +1269,8 @@ func (a Allocator) rebalanceTarget( rebalanceConstraintsChecker, replicaSetToRebalance, replicasWithExcludedStores, - a.storePool.getLocalitiesByStore(replicaSetForDiversityCalc), - a.storePool.isStoreReadyForRoutineReplicaTransfer, + a.storePool.GetLocalitiesByStore(replicaSetForDiversityCalc), + a.storePool.IsStoreReadyForRoutineReplicaTransfer, options, a.metrics, ) @@ -1402,8 +1396,8 @@ func (a Allocator) RebalanceVoter( conf roachpb.SpanConfig, raftStatus *raft.Status, existingVoters, existingNonVoters []roachpb.ReplicaDescriptor, - rangeUsageInfo RangeUsageInfo, - filter storeFilter, + rangeUsageInfo allocator.RangeUsageInfo, + filter storepool.StoreFilter, options scorerOptions, ) (add, remove roachpb.ReplicationTarget, details string, ok bool) { return a.rebalanceTarget( @@ -1436,8 +1430,8 @@ func (a Allocator) RebalanceNonVoter( conf roachpb.SpanConfig, raftStatus *raft.Status, existingVoters, existingNonVoters []roachpb.ReplicaDescriptor, - rangeUsageInfo RangeUsageInfo, - filter storeFilter, + rangeUsageInfo allocator.RangeUsageInfo, + filter storepool.StoreFilter, options scorerOptions, ) (add, remove roachpb.ReplicationTarget, details string, ok bool) { return a.rebalanceTarget( @@ -1456,8 +1450,8 @@ func (a Allocator) RebalanceNonVoter( func (a *Allocator) scorerOptions(ctx context.Context) *rangeCountScorerOptions { return &rangeCountScorerOptions{ storeHealthOptions: a.storeHealthOptions(ctx), - deterministic: a.storePool.deterministic, - rangeRebalanceThreshold: rangeRebalanceThreshold.Get(&a.storePool.st.SV), + deterministic: a.storePool.Deterministic, + rangeRebalanceThreshold: rangeRebalanceThreshold.Get(&a.storePool.St.SV), } } @@ -1465,7 +1459,7 @@ func (a *Allocator) scorerOptionsForScatter(ctx context.Context) *scatterScorerO return &scatterScorerOptions{ rangeCountScorerOptions: rangeCountScorerOptions{ storeHealthOptions: a.storeHealthOptions(ctx), - deterministic: a.storePool.deterministic, + deterministic: a.storePool.Deterministic, rangeRebalanceThreshold: 0, }, // We set jitter to be equal to the padding around replica-count rebalancing @@ -1474,7 +1468,7 @@ func (a *Allocator) scorerOptionsForScatter(ctx context.Context) *scatterScorerO // made by the replicateQueue during normal course of operations. In other // words, we don't want stores that are too far away from the mean to be // affected by the jitter. - jitter: rangeRebalanceThreshold.Get(&a.storePool.st.SV), + jitter: rangeRebalanceThreshold.Get(&a.storePool.St.SV), } } @@ -1512,7 +1506,7 @@ func (a *Allocator) ValidLeaseTargets( } candidates = append(candidates, existing[i]) } - candidates, _ = a.storePool.liveAndDeadReplicas( + candidates, _ = a.storePool.LiveAndDeadReplicas( candidates, false, /* includeSuspectAndDrainingStores */ ) @@ -1575,7 +1569,7 @@ func (a *Allocator) leaseholderShouldMoveDueToPreferences( } // Exclude suspect/draining/dead stores. - candidates, _ := a.storePool.liveAndDeadReplicas( + candidates, _ := a.storePool.LiveAndDeadReplicas( allExistingReplicas, false, /* includeSuspectAndDrainingStores */ ) // If there are any replicas that do match lease preferences, then we check if @@ -1600,13 +1594,13 @@ func (a *Allocator) leaseholderShouldMoveDueToPreferences( // there is a mixed version cluster, storeHealthNoAction is set instead. func (a *Allocator) storeHealthOptions(ctx context.Context) storeHealthOptions { enforcementLevel := storeHealthNoAction - if a.storePool.st.Version.IsActive(ctx, clusterversion.AutoStatsTableSettings) { - enforcementLevel = storeHealthEnforcement(l0SublevelsThresholdEnforce.Get(&a.storePool.st.SV)) + if a.storePool.St.Version.IsActive(ctx, clusterversion.AutoStatsTableSettings) { + enforcementLevel = storeHealthEnforcement(l0SublevelsThresholdEnforce.Get(&a.storePool.St.SV)) } return storeHealthOptions{ enforcementLevel: enforcementLevel, - l0SublevelThreshold: l0SublevelsThreshold.Get(&a.storePool.st.SV), + l0SublevelThreshold: l0SublevelsThreshold.Get(&a.storePool.St.SV), } } @@ -1645,14 +1639,14 @@ func (a *Allocator) TransferLeaseTarget( excludeLeaseRepl = true } - allStoresList, _, _ := a.storePool.getStoreList(storeFilterNone) + allStoresList, _, _ := a.storePool.GetStoreList(storepool.StoreFilterNone) storeDescMap := storeListToMap(allStoresList) - sl, _, _ := a.storePool.getStoreList(storeFilterSuspect) - sl = sl.excludeInvalid(conf.Constraints) - sl = sl.excludeInvalid(conf.VoterConstraints) - candidateLeasesMean := sl.candidateLeases.mean + sl, _, _ := a.storePool.GetStoreList(storepool.StoreFilterSuspect) + sl = sl.ExcludeInvalid(conf.Constraints) + sl = sl.ExcludeInvalid(conf.VoterConstraints) + candidateLeasesMean := sl.CandidateLeases.Mean - source, ok := a.storePool.getStoreDescriptor(leaseRepl.StoreID()) + source, ok := a.storePool.GetStoreDescriptor(leaseRepl.StoreID()) if !ok { return roachpb.ReplicaDescriptor{} } @@ -1711,7 +1705,7 @@ func (a *Allocator) TransferLeaseTarget( if leaseRepl.StoreID() == repl.StoreID { continue } - storeDesc, ok := a.storePool.getStoreDescriptor(repl.StoreID) + storeDesc, ok := a.storePool.GetStoreDescriptor(repl.StoreID) if !ok { continue } @@ -1762,8 +1756,8 @@ func (a *Allocator) TransferLeaseTarget( storeDescMap, &qpsScorerOptions{ storeHealthOptions: a.storeHealthOptions(ctx), - qpsRebalanceThreshold: qpsRebalanceThreshold.Get(&a.storePool.st.SV), - minRequiredQPSDiff: minQPSDifferenceForTransfers.Get(&a.storePool.st.SV), + qpsRebalanceThreshold: qpsRebalanceThreshold.Get(&a.storePool.St.SV), + minRequiredQPSDiff: minQPSDifferenceForTransfers.Get(&a.storePool.St.SV), }, ) @@ -1891,14 +1885,14 @@ func (a *Allocator) ShouldTransferLease( if len(existing) == 0 || (len(existing) == 1 && existing[0].StoreID == leaseRepl.StoreID()) { return false } - source, ok := a.storePool.getStoreDescriptor(leaseRepl.StoreID()) + source, ok := a.storePool.GetStoreDescriptor(leaseRepl.StoreID()) if !ok { return false } - sl, _, _ := a.storePool.getStoreList(storeFilterSuspect) - sl = sl.excludeInvalid(conf.Constraints) - sl = sl.excludeInvalid(conf.VoterConstraints) + sl, _, _ := a.storePool.GetStoreList(storepool.StoreFilterSuspect) + sl = sl.ExcludeInvalid(conf.Constraints) + sl = sl.ExcludeInvalid(conf.VoterConstraints) log.VEventf(ctx, 3, "ShouldTransferLease (lease-holder=s%d):\n%s", leaseRepl.StoreID(), sl) transferDec, _ := a.shouldTransferLeaseForAccessLocality( @@ -1907,7 +1901,7 @@ func (a *Allocator) ShouldTransferLease( existing, stats, nil, - sl.candidateLeases.mean, + sl.CandidateLeases.Mean, ) var result bool switch transferDec { @@ -1929,14 +1923,14 @@ func (a *Allocator) ShouldTransferLease( func (a Allocator) followTheWorkloadPrefersLocal( ctx context.Context, - sl StoreList, + sl storepool.StoreList, source roachpb.StoreDescriptor, candidate roachpb.StoreID, existing []roachpb.ReplicaDescriptor, stats *replicaStats, ) bool { adjustments := make(map[roachpb.StoreID]float64) - decision, _ := a.shouldTransferLeaseForAccessLocality(ctx, source, existing, stats, adjustments, sl.candidateLeases.mean) + decision, _ := a.shouldTransferLeaseForAccessLocality(ctx, source, existing, stats, adjustments, sl.CandidateLeases.Mean) if decision == decideWithoutStats { return false } @@ -1960,10 +1954,10 @@ func (a Allocator) shouldTransferLeaseForAccessLocality( ) (transferDecision, roachpb.ReplicaDescriptor) { // Only use load-based rebalancing if it's enabled and we have both // stats and locality information to base our decision on. - if stats == nil || !enableLoadBasedLeaseRebalancing.Get(&a.storePool.st.SV) { + if stats == nil || !enableLoadBasedLeaseRebalancing.Get(&a.storePool.St.SV) { return decideWithoutStats, roachpb.ReplicaDescriptor{} } - replicaLocalities := a.storePool.getLocalitiesByNode(existing) + replicaLocalities := a.storePool.GetLocalitiesByNode(existing) for _, locality := range replicaLocalities { if len(locality.Tiers) == 0 { return decideWithoutStats, roachpb.ReplicaDescriptor{} @@ -2017,11 +2011,11 @@ func (a Allocator) shouldTransferLeaseForAccessLocality( if repl.NodeID == source.Node.NodeID { continue } - storeDesc, ok := a.storePool.getStoreDescriptor(repl.StoreID) + storeDesc, ok := a.storePool.GetStoreDescriptor(repl.StoreID) if !ok { continue } - addr, err := a.storePool.gossip.GetNodeIDAddress(repl.NodeID) + addr, err := a.storePool.Gossip.GetNodeIDAddress(repl.NodeID) if err != nil { log.Errorf(ctx, "missing address for n%d: %+v", repl.NodeID, err) continue @@ -2033,7 +2027,7 @@ func (a Allocator) shouldTransferLeaseForAccessLocality( remoteWeight := math.Max(minReplicaWeight, replicaWeights[repl.NodeID]) replScore, rebalanceAdjustment := loadBasedLeaseRebalanceScore( - ctx, a.storePool.st, remoteWeight, remoteLatency, storeDesc, sourceWeight, source, candidateLeasesMean) + ctx, a.storePool.St, remoteWeight, remoteLatency, storeDesc, sourceWeight, source, candidateLeasesMean) if replScore > bestReplScore { bestReplScore = replScore bestRepl = repl @@ -2136,7 +2130,7 @@ func loadBasedLeaseRebalanceScore( func (a Allocator) shouldTransferLeaseForLeaseCountConvergence( ctx context.Context, - sl StoreList, + sl storepool.StoreList, source roachpb.StoreDescriptor, existing []roachpb.ReplicaDescriptor, ) bool { @@ -2147,8 +2141,8 @@ func (a Allocator) shouldTransferLeaseForLeaseCountConvergence( // Allow lease transfer if we're above the overfull threshold, which is // mean*(1+leaseRebalanceThreshold). - overfullLeaseThreshold := int32(math.Ceil(sl.candidateLeases.mean * (1 + leaseRebalanceThreshold))) - minOverfullThreshold := int32(math.Ceil(sl.candidateLeases.mean + 5)) + overfullLeaseThreshold := int32(math.Ceil(sl.CandidateLeases.Mean * (1 + leaseRebalanceThreshold))) + minOverfullThreshold := int32(math.Ceil(sl.CandidateLeases.Mean + 5)) if overfullLeaseThreshold < minOverfullThreshold { overfullLeaseThreshold = minOverfullThreshold } @@ -2156,15 +2150,15 @@ func (a Allocator) shouldTransferLeaseForLeaseCountConvergence( return true } - if float64(source.Capacity.LeaseCount) > sl.candidateLeases.mean { - underfullLeaseThreshold := int32(math.Ceil(sl.candidateLeases.mean * (1 - leaseRebalanceThreshold))) - minUnderfullThreshold := int32(math.Ceil(sl.candidateLeases.mean - 5)) + if float64(source.Capacity.LeaseCount) > sl.CandidateLeases.Mean { + underfullLeaseThreshold := int32(math.Ceil(sl.CandidateLeases.Mean * (1 - leaseRebalanceThreshold))) + minUnderfullThreshold := int32(math.Ceil(sl.CandidateLeases.Mean - 5)) if underfullLeaseThreshold > minUnderfullThreshold { underfullLeaseThreshold = minUnderfullThreshold } for _, repl := range existing { - storeDesc, ok := a.storePool.getStoreDescriptor(repl.StoreID) + storeDesc, ok := a.storePool.GetStoreDescriptor(repl.StoreID) if !ok { continue } @@ -2188,7 +2182,7 @@ func (a Allocator) preferredLeaseholders( // TODO(a-robinson): Do all these lookups at once, up front? We could // easily be passing a slice of StoreDescriptors around all the Allocator // functions instead of ReplicaDescriptors. - storeDesc, ok := a.storePool.getStoreDescriptor(repl.StoreID) + storeDesc, ok := a.storePool.GetStoreDescriptor(repl.StoreID) if !ok { continue } diff --git a/pkg/kv/kvserver/allocator/BUILD.bazel b/pkg/kv/kvserver/allocator/BUILD.bazel new file mode 100644 index 000000000000..02ba3d000be4 --- /dev/null +++ b/pkg/kv/kvserver/allocator/BUILD.bazel @@ -0,0 +1,15 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "allocator", + srcs = [ + "base.go", + "range_usage_info.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator", + visibility = ["//visibility:public"], + deps = [ + "//pkg/kv/kvserver/constraint", + "//pkg/roachpb", + ], +) diff --git a/pkg/kv/kvserver/allocator/base.go b/pkg/kv/kvserver/allocator/base.go new file mode 100644 index 000000000000..e65fee196f05 --- /dev/null +++ b/pkg/kv/kvserver/allocator/base.go @@ -0,0 +1,48 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package allocator + +import ( + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/constraint" + "github.com/cockroachdb/cockroach/pkg/roachpb" +) + +// MaxCapacityCheck returns true if the store has room for a new replica. +func MaxCapacityCheck(store roachpb.StoreDescriptor) bool { + return store.Capacity.FractionUsed() < MaxFractionUsedThreshold +} + +const ( + // MaxFractionUsedThreshold controls the point at which the store cedes having + // room for new replicas. If the fraction used of a store descriptor capacity + // is greater than this value, it will never be used as a rebalance or + // allocate target and we will actively try to move replicas off of it. + MaxFractionUsedThreshold = 0.95 +) + +// IsStoreValid returns true iff the provided store would be a valid in a +// range with the provided constraints. +func IsStoreValid( + store roachpb.StoreDescriptor, constraints []roachpb.ConstraintsConjunction, +) bool { + if len(constraints) == 0 { + return true + } + + for _, subConstraints := range constraints { + if constraintsOK := constraint.ConjunctionsCheck( + store, subConstraints.Constraints, + ); constraintsOK { + return true + } + } + return false +} diff --git a/pkg/kv/kvserver/allocator/range_usage_info.go b/pkg/kv/kvserver/allocator/range_usage_info.go new file mode 100644 index 000000000000..b05b6a562ef6 --- /dev/null +++ b/pkg/kv/kvserver/allocator/range_usage_info.go @@ -0,0 +1,19 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package allocator + +// RangeUsageInfo contains usage information (sizes and traffic) needed by the +// allocator to make rebalancing decisions for a given range. +type RangeUsageInfo struct { + LogicalBytes int64 + QueriesPerSecond float64 + WritesPerSecond float64 +} diff --git a/pkg/kv/kvserver/allocator/storepool/BUILD.bazel b/pkg/kv/kvserver/allocator/storepool/BUILD.bazel new file mode 100644 index 000000000000..aa71534bb3a3 --- /dev/null +++ b/pkg/kv/kvserver/allocator/storepool/BUILD.bazel @@ -0,0 +1,49 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "storepool", + srcs = [ + "store_pool.go", + "test_helpers.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool", + visibility = ["//visibility:public"], + deps = [ + "//pkg/base", + "//pkg/config/zonepb", + "//pkg/gossip", + "//pkg/kv/kvserver/allocator", + "//pkg/kv/kvserver/liveness", + "//pkg/kv/kvserver/liveness/livenesspb", + "//pkg/roachpb", + "//pkg/rpc", + "//pkg/settings", + "//pkg/settings/cluster", + "//pkg/util/hlc", + "//pkg/util/humanizeutil", + "//pkg/util/log", + "//pkg/util/metric", + "//pkg/util/shuffle", + "//pkg/util/stop", + "//pkg/util/syncutil", + "@com_github_cockroachdb_errors//:errors", + ], +) + +go_test( + name = "storepool_test", + srcs = ["store_pool_test.go"], + embed = [":storepool"], + deps = [ + "//pkg/kv/kvserver/liveness/livenesspb", + "//pkg/roachpb", + "//pkg/testutils/gossiputil", + "//pkg/util/hlc", + "//pkg/util/leaktest", + "//pkg/util/log", + "//pkg/util/timeutil", + "@com_github_cockroachdb_errors//:errors", + "@com_github_kr_pretty//:pretty", + "@com_github_stretchr_testify//require", + ], +) diff --git a/pkg/kv/kvserver/store_pool.go b/pkg/kv/kvserver/allocator/storepool/store_pool.go similarity index 71% rename from pkg/kv/kvserver/store_pool.go rename to pkg/kv/kvserver/allocator/storepool/store_pool.go index 4ee6e95fd8c2..d0d94cbf1113 100644 --- a/pkg/kv/kvserver/store_pool.go +++ b/pkg/kv/kvserver/allocator/storepool/store_pool.go @@ -1,4 +1,4 @@ -// Copyright 2015 The Cockroach Authors. +// Copyright 2022 The Cockroach Authors. // // Use of this software is governed by the Business Source License // included in the file licenses/BSL.txt. @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package kvserver +package storepool import ( "bytes" @@ -18,6 +18,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/gossip" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -178,35 +179,36 @@ func LivenessStatus( return livenesspb.NodeLivenessStatus_UNAVAILABLE } -type storeDetail struct { - desc *roachpb.StoreDescriptor - // throttledUntil is when a throttled store can be considered available again +// StoreDetail groups together store-relevant details. +type StoreDetail struct { + Desc *roachpb.StoreDescriptor + // ThrottledUntil is when a throttled store can be considered available again // due to a failed or declined snapshot. - throttledUntil time.Time + ThrottledUntil time.Time // throttledBecause is set to the most recent reason for which a store was // marked as throttled. throttledBecause string // lastUpdatedTime is set when a store is first consulted and every time // gossip arrives for a store. lastUpdatedTime time.Time - // lastUnavailable is set when it's detected that a store was unavailable, + // LastUnavailable is set when it's detected that a store was unavailable, // i.e. failed liveness. - lastUnavailable time.Time - // lastAvailable is set when it's detected that a store was available, + LastUnavailable time.Time + // LastAvailable is set when it's detected that a store was available, // i.e. we got a liveness heartbeat. - lastAvailable time.Time + LastAvailable time.Time } // isThrottled returns whether the store is currently throttled. -func (sd storeDetail) isThrottled(now time.Time) bool { - return sd.throttledUntil.After(now) +func (sd StoreDetail) isThrottled(now time.Time) bool { + return sd.ThrottledUntil.After(now) } // isSuspect returns whether the store is currently suspect. We measure that by // looking at the time it was last unavailable making sure we have not seen any // failures for a period of time defined by StoreSuspectDuration. -func (sd storeDetail) isSuspect(now time.Time, suspectDuration time.Duration) bool { - return sd.lastUnavailable.Add(suspectDuration).After(now) +func (sd StoreDetail) isSuspect(now time.Time, suspectDuration time.Duration) bool { + return sd.LastUnavailable.Add(suspectDuration).After(now) } // storeStatus is the current status of a store. @@ -239,7 +241,7 @@ const ( storeStatusDraining ) -func (sd *storeDetail) status( +func (sd *StoreDetail) status( now time.Time, threshold time.Duration, nl NodeLivenessFunc, suspectDuration time.Duration, ) storeStatus { // During normal operation, we expect the state transitions for stores to look like the following: @@ -270,12 +272,12 @@ func (sd *storeDetail) status( if now.After(deadAsOf) { // Wipe out the lastAvailable timestamp, so that once a node comes back // from the dead we dont consider it suspect. - sd.lastAvailable = time.Time{} + sd.LastAvailable = time.Time{} return storeStatusDead } // If there's no descriptor (meaning no gossip ever arrived for this // store), return unavailable. - if sd.desc == nil { + if sd.Desc == nil { return storeStatusUnknown } @@ -284,7 +286,7 @@ func (sd *storeDetail) status( // // Store statuses checked in the following order: // dead -> decommissioning -> unknown -> draining -> suspect -> available. - switch nl(sd.desc.Node.NodeID, now, threshold) { + switch nl(sd.Desc.Node.NodeID, now, threshold) { case livenesspb.NodeLivenessStatus_DEAD, livenesspb.NodeLivenessStatus_DECOMMISSIONED: return storeStatusDead case livenesspb.NodeLivenessStatus_DECOMMISSIONING: @@ -292,8 +294,8 @@ func (sd *storeDetail) status( case livenesspb.NodeLivenessStatus_UNAVAILABLE: // We don't want to suspect a node on startup or when it's first added to a // cluster, because we dont know its liveness yet. - if !sd.lastAvailable.IsZero() { - sd.lastUnavailable = now + if !sd.LastAvailable.IsZero() { + sd.LastUnavailable = now } return storeStatusUnknown case livenesspb.NodeLivenessStatus_UNKNOWN: @@ -303,7 +305,7 @@ func (sd *storeDetail) status( // graceful restart it will not be considered as suspect. This is best effort // and we may not see a store in this state. To help with that we perform // a similar clear of lastAvailable on a DEAD store. - sd.lastAvailable = time.Time{} + sd.LastAvailable = time.Time{} return storeStatusDraining } @@ -314,7 +316,7 @@ func (sd *storeDetail) status( if sd.isSuspect(now, suspectDuration) { return storeStatusSuspect } - sd.lastAvailable = now + sd.LastAvailable = now return storeStatusAvailable } @@ -330,30 +332,32 @@ type localityWithString struct { // StorePool maintains a list of all known stores in the cluster and // information on their health. +// +// TODO(irfansharif): Mediate access through a thin interface. type StorePool struct { log.AmbientContext - st *cluster.Settings + St *cluster.Settings // TODO(irfansharif): Shouldn't need to be exported. - clock *hlc.Clock - gossip *gossip.Gossip + Clock *hlc.Clock + Gossip *gossip.Gossip // TODO(irfansharif): Shouldn't need to be exported. nodeCountFn NodeCountFunc - nodeLivenessFn NodeLivenessFunc + NodeLivenessFn NodeLivenessFunc startTime time.Time - deterministic bool + Deterministic bool // We use separate mutexes for storeDetails and nodeLocalities because the // nodeLocalities map is used in the critical code path of Replica.Send() // and we'd rather not block that on something less important accessing // storeDetails. - detailsMu struct { + DetailsMu struct { syncutil.RWMutex - storeDetails map[roachpb.StoreID]*storeDetail + StoreDetails map[roachpb.StoreID]*StoreDetail } localitiesMu struct { syncutil.RWMutex nodeLocalities map[roachpb.NodeID]localityWithString } - // isStoreReadyForRoutineReplicaTransfer returns true iff the store's node is + // IsStoreReadyForRoutineReplicaTransfer returns true iff the store's node is // live (as indicated by its `NodeLivenessStatus`) and thus a legal candidate // to receive a replica. This is defined as a closure reference here instead // of a regular method so it can be overridden in tests. @@ -366,7 +370,7 @@ type StorePool struct { // to "non-ready" nodes (i.e. nodes that _currently_ have a non-live // `NodeLivenessStatus`), whereas we _actively move replicas off of "dead" // nodes_. - isStoreReadyForRoutineReplicaTransfer func(context.Context, roachpb.StoreID) bool + IsStoreReadyForRoutineReplicaTransfer func(context.Context, roachpb.StoreID) bool } // NewStorePool creates a StorePool and registers the store updating callback @@ -382,16 +386,16 @@ func NewStorePool( ) *StorePool { sp := &StorePool{ AmbientContext: ambient, - st: st, - clock: clock, - gossip: g, + St: st, + Clock: clock, + Gossip: g, nodeCountFn: nodeCountFn, - nodeLivenessFn: nodeLivenessFn, + NodeLivenessFn: nodeLivenessFn, startTime: clock.PhysicalTime(), - deterministic: deterministic, + Deterministic: deterministic, } - sp.isStoreReadyForRoutineReplicaTransfer = sp.isStoreReadyForRoutineReplicaTransferInternal - sp.detailsMu.storeDetails = make(map[roachpb.StoreID]*storeDetail) + sp.IsStoreReadyForRoutineReplicaTransfer = sp.isStoreReadyForRoutineReplicaTransferInternal + sp.DetailsMu.StoreDetails = make(map[roachpb.StoreID]*StoreDetail) sp.localitiesMu.nodeLocalities = make(map[roachpb.NodeID]localityWithString) // Enable redundant callbacks for the store keys because we use these @@ -404,32 +408,32 @@ func NewStorePool( } func (sp *StorePool) String() string { - sp.detailsMu.RLock() - defer sp.detailsMu.RUnlock() + sp.DetailsMu.RLock() + defer sp.DetailsMu.RUnlock() - ids := make(roachpb.StoreIDSlice, 0, len(sp.detailsMu.storeDetails)) - for id := range sp.detailsMu.storeDetails { + ids := make(roachpb.StoreIDSlice, 0, len(sp.DetailsMu.StoreDetails)) + for id := range sp.DetailsMu.StoreDetails { ids = append(ids, id) } sort.Sort(ids) var buf bytes.Buffer - now := sp.clock.Now().GoTime() - timeUntilStoreDead := TimeUntilStoreDead.Get(&sp.st.SV) - timeAfterStoreSuspect := TimeAfterStoreSuspect.Get(&sp.st.SV) + now := sp.Clock.Now().GoTime() + timeUntilStoreDead := TimeUntilStoreDead.Get(&sp.St.SV) + timeAfterStoreSuspect := TimeAfterStoreSuspect.Get(&sp.St.SV) for _, id := range ids { - detail := sp.detailsMu.storeDetails[id] + detail := sp.DetailsMu.StoreDetails[id] fmt.Fprintf(&buf, "%d", id) - status := detail.status(now, timeUntilStoreDead, sp.nodeLivenessFn, timeAfterStoreSuspect) + status := detail.status(now, timeUntilStoreDead, sp.NodeLivenessFn, timeAfterStoreSuspect) if status != storeStatusAvailable { fmt.Fprintf(&buf, " (status=%d)", status) } - if detail.desc != nil { + if detail.Desc != nil { fmt.Fprintf(&buf, ": range-count=%d fraction-used=%.2f", - detail.desc.Capacity.RangeCount, detail.desc.Capacity.FractionUsed()) + detail.Desc.Capacity.RangeCount, detail.Desc.Capacity.FractionUsed()) } - throttled := detail.throttledUntil.Sub(now) + throttled := detail.ThrottledUntil.Sub(now) if throttled > 0 { fmt.Fprintf(&buf, " [throttled=%.1fs]", throttled.Seconds()) } @@ -438,7 +442,7 @@ func (sp *StorePool) String() string { return buf.String() } -// storeGossipUpdate is the gossip callback used to keep the StorePool up to date. +// storeGossipUpdate is the Gossip callback used to keep the StorePool up to date. func (sp *StorePool) storeGossipUpdate(_ string, content roachpb.Value) { var storeDesc roachpb.StoreDescriptor if err := content.GetProto(&storeDesc); err != nil { @@ -447,11 +451,11 @@ func (sp *StorePool) storeGossipUpdate(_ string, content roachpb.Value) { return } - sp.detailsMu.Lock() - detail := sp.getStoreDetailLocked(storeDesc.StoreID) - detail.desc = &storeDesc - detail.lastUpdatedTime = sp.clock.PhysicalTime() - sp.detailsMu.Unlock() + sp.DetailsMu.Lock() + detail := sp.GetStoreDetailLocked(storeDesc.StoreID) + detail.Desc = &storeDesc + detail.lastUpdatedTime = sp.Clock.PhysicalTime() + sp.DetailsMu.Unlock() sp.localitiesMu.Lock() sp.localitiesMu.nodeLocalities[storeDesc.Node.NodeID] = @@ -459,15 +463,17 @@ func (sp *StorePool) storeGossipUpdate(_ string, content roachpb.Value) { sp.localitiesMu.Unlock() } -// updateLocalStoreAfterRebalance is used to update the local copy of the +// UpdateLocalStoreAfterRebalance is used to update the local copy of the // target store immediately after a replica addition or removal. -func (sp *StorePool) updateLocalStoreAfterRebalance( - storeID roachpb.StoreID, rangeUsageInfo RangeUsageInfo, changeType roachpb.ReplicaChangeType, +func (sp *StorePool) UpdateLocalStoreAfterRebalance( + storeID roachpb.StoreID, + rangeUsageInfo allocator.RangeUsageInfo, + changeType roachpb.ReplicaChangeType, ) { - sp.detailsMu.Lock() - defer sp.detailsMu.Unlock() - detail := *sp.getStoreDetailLocked(storeID) - if detail.desc == nil { + sp.DetailsMu.Lock() + defer sp.DetailsMu.Unlock() + detail := *sp.GetStoreDetailLocked(storeID) + if detail.Desc == nil { // We don't have this store yet (this is normal when we're // starting up and don't have full information from the gossip // network). We can't update the local store at this time. @@ -475,80 +481,80 @@ func (sp *StorePool) updateLocalStoreAfterRebalance( } switch changeType { case roachpb.ADD_VOTER, roachpb.ADD_NON_VOTER: - detail.desc.Capacity.RangeCount++ - detail.desc.Capacity.LogicalBytes += rangeUsageInfo.LogicalBytes - detail.desc.Capacity.WritesPerSecond += rangeUsageInfo.WritesPerSecond + detail.Desc.Capacity.RangeCount++ + detail.Desc.Capacity.LogicalBytes += rangeUsageInfo.LogicalBytes + detail.Desc.Capacity.WritesPerSecond += rangeUsageInfo.WritesPerSecond case roachpb.REMOVE_VOTER, roachpb.REMOVE_NON_VOTER: - detail.desc.Capacity.RangeCount-- - if detail.desc.Capacity.LogicalBytes <= rangeUsageInfo.LogicalBytes { - detail.desc.Capacity.LogicalBytes = 0 + detail.Desc.Capacity.RangeCount-- + if detail.Desc.Capacity.LogicalBytes <= rangeUsageInfo.LogicalBytes { + detail.Desc.Capacity.LogicalBytes = 0 } else { - detail.desc.Capacity.LogicalBytes -= rangeUsageInfo.LogicalBytes + detail.Desc.Capacity.LogicalBytes -= rangeUsageInfo.LogicalBytes } - if detail.desc.Capacity.WritesPerSecond <= rangeUsageInfo.WritesPerSecond { - detail.desc.Capacity.WritesPerSecond = 0 + if detail.Desc.Capacity.WritesPerSecond <= rangeUsageInfo.WritesPerSecond { + detail.Desc.Capacity.WritesPerSecond = 0 } else { - detail.desc.Capacity.WritesPerSecond -= rangeUsageInfo.WritesPerSecond + detail.Desc.Capacity.WritesPerSecond -= rangeUsageInfo.WritesPerSecond } default: return } - sp.detailsMu.storeDetails[storeID] = &detail + sp.DetailsMu.StoreDetails[storeID] = &detail } -// updateLocalStoresAfterLeaseTransfer is used to update the local copies of the +// UpdateLocalStoresAfterLeaseTransfer is used to update the local copies of the // involved store descriptors immediately after a lease transfer. -func (sp *StorePool) updateLocalStoresAfterLeaseTransfer( +func (sp *StorePool) UpdateLocalStoresAfterLeaseTransfer( from roachpb.StoreID, to roachpb.StoreID, rangeQPS float64, ) { - sp.detailsMu.Lock() - defer sp.detailsMu.Unlock() - - fromDetail := *sp.getStoreDetailLocked(from) - if fromDetail.desc != nil { - fromDetail.desc.Capacity.LeaseCount-- - if fromDetail.desc.Capacity.QueriesPerSecond < rangeQPS { - fromDetail.desc.Capacity.QueriesPerSecond = 0 + sp.DetailsMu.Lock() + defer sp.DetailsMu.Unlock() + + fromDetail := *sp.GetStoreDetailLocked(from) + if fromDetail.Desc != nil { + fromDetail.Desc.Capacity.LeaseCount-- + if fromDetail.Desc.Capacity.QueriesPerSecond < rangeQPS { + fromDetail.Desc.Capacity.QueriesPerSecond = 0 } else { - fromDetail.desc.Capacity.QueriesPerSecond -= rangeQPS + fromDetail.Desc.Capacity.QueriesPerSecond -= rangeQPS } - sp.detailsMu.storeDetails[from] = &fromDetail + sp.DetailsMu.StoreDetails[from] = &fromDetail } - toDetail := *sp.getStoreDetailLocked(to) - if toDetail.desc != nil { - toDetail.desc.Capacity.LeaseCount++ - toDetail.desc.Capacity.QueriesPerSecond += rangeQPS - sp.detailsMu.storeDetails[to] = &toDetail + toDetail := *sp.GetStoreDetailLocked(to) + if toDetail.Desc != nil { + toDetail.Desc.Capacity.LeaseCount++ + toDetail.Desc.Capacity.QueriesPerSecond += rangeQPS + sp.DetailsMu.StoreDetails[to] = &toDetail } } -// newStoreDetail makes a new storeDetail struct. It sets index to be -1 to +// newStoreDetail makes a new StoreDetail struct. It sets index to be -1 to // ensure that it will be processed by a queue immediately. -func newStoreDetail() *storeDetail { - return &storeDetail{} +func newStoreDetail() *StoreDetail { + return &StoreDetail{} } // GetStores returns information on all the stores with descriptor in the pool. // Stores without descriptor (a node that didn't come up yet after a cluster // restart) will not be part of the returned set. func (sp *StorePool) GetStores() map[roachpb.StoreID]roachpb.StoreDescriptor { - sp.detailsMu.RLock() - defer sp.detailsMu.RUnlock() - stores := make(map[roachpb.StoreID]roachpb.StoreDescriptor, len(sp.detailsMu.storeDetails)) - for _, s := range sp.detailsMu.storeDetails { - if s.desc != nil { - stores[s.desc.StoreID] = *s.desc + sp.DetailsMu.RLock() + defer sp.DetailsMu.RUnlock() + stores := make(map[roachpb.StoreID]roachpb.StoreDescriptor, len(sp.DetailsMu.StoreDetails)) + for _, s := range sp.DetailsMu.StoreDetails { + if s.Desc != nil { + stores[s.Desc.StoreID] = *s.Desc } } return stores } -// getStoreDetailLocked returns the store detail for the given storeID. +// GetStoreDetailLocked returns the store detail for the given storeID. // The lock must be held *in write mode* even though this looks like a // read-only method. -func (sp *StorePool) getStoreDetailLocked(storeID roachpb.StoreID) *storeDetail { - detail, ok := sp.detailsMu.storeDetails[storeID] +func (sp *StorePool) GetStoreDetailLocked(storeID roachpb.StoreID) *StoreDetail { + detail, ok := sp.DetailsMu.StoreDetails[storeID] if !ok { // We don't have this store yet (this is normal when we're // starting up and don't have full information from the gossip @@ -557,40 +563,40 @@ func (sp *StorePool) getStoreDetailLocked(storeID roachpb.StoreID) *storeDetail // time passes without updates from gossip. detail = newStoreDetail() detail.lastUpdatedTime = sp.startTime - sp.detailsMu.storeDetails[storeID] = detail + sp.DetailsMu.StoreDetails[storeID] = detail } return detail } -// getStoreDescriptor returns the latest store descriptor for the given +// GetStoreDescriptor returns the latest store descriptor for the given // storeID. -func (sp *StorePool) getStoreDescriptor(storeID roachpb.StoreID) (roachpb.StoreDescriptor, bool) { - sp.detailsMu.RLock() - defer sp.detailsMu.RUnlock() +func (sp *StorePool) GetStoreDescriptor(storeID roachpb.StoreID) (roachpb.StoreDescriptor, bool) { + sp.DetailsMu.RLock() + defer sp.DetailsMu.RUnlock() - if detail, ok := sp.detailsMu.storeDetails[storeID]; ok && detail.desc != nil { - return *detail.desc, true + if detail, ok := sp.DetailsMu.StoreDetails[storeID]; ok && detail.Desc != nil { + return *detail.Desc, true } return roachpb.StoreDescriptor{}, false } -// decommissioningReplicas filters out replicas on decommissioning node/store +// DecommissioningReplicas filters out replicas on decommissioning node/store // from the provided repls and returns them in a slice. -func (sp *StorePool) decommissioningReplicas( +func (sp *StorePool) DecommissioningReplicas( repls []roachpb.ReplicaDescriptor, ) (decommissioningReplicas []roachpb.ReplicaDescriptor) { - sp.detailsMu.Lock() - defer sp.detailsMu.Unlock() + sp.DetailsMu.Lock() + defer sp.DetailsMu.Unlock() // NB: We use clock.Now().GoTime() instead of clock.PhysicalTime() is order to // take clock signals from remote nodes into consideration. - now := sp.clock.Now().GoTime() - timeUntilStoreDead := TimeUntilStoreDead.Get(&sp.st.SV) - timeAfterStoreSuspect := TimeAfterStoreSuspect.Get(&sp.st.SV) + now := sp.Clock.Now().GoTime() + timeUntilStoreDead := TimeUntilStoreDead.Get(&sp.St.SV) + timeAfterStoreSuspect := TimeAfterStoreSuspect.Get(&sp.St.SV) for _, repl := range repls { - detail := sp.getStoreDetailLocked(repl.StoreID) - switch detail.status(now, timeUntilStoreDead, sp.nodeLivenessFn, timeAfterStoreSuspect) { + detail := sp.GetStoreDetailLocked(repl.StoreID) + switch detail.status(now, timeUntilStoreDead, sp.NodeLivenessFn, timeAfterStoreSuspect) { case storeStatusDecommissioning: decommissioningReplicas = append(decommissioningReplicas, repl) } @@ -609,17 +615,17 @@ func (sp *StorePool) ClusterNodeCount() int { // not found in the store pool or the status is unknown. If the store is not dead, // it returns the time to death. func (sp *StorePool) IsDead(storeID roachpb.StoreID) (bool, time.Duration, error) { - sp.detailsMu.Lock() - defer sp.detailsMu.Unlock() + sp.DetailsMu.Lock() + defer sp.DetailsMu.Unlock() - sd, ok := sp.detailsMu.storeDetails[storeID] + sd, ok := sp.DetailsMu.StoreDetails[storeID] if !ok { return false, 0, errors.Errorf("store %d was not found", storeID) } // NB: We use clock.Now().GoTime() instead of clock.PhysicalTime() is order to // take clock signals from remote nodes into consideration. - now := sp.clock.Now().GoTime() - timeUntilStoreDead := TimeUntilStoreDead.Get(&sp.st.SV) + now := sp.Clock.Now().GoTime() + timeUntilStoreDead := TimeUntilStoreDead.Get(&sp.St.SV) deadAsOf := sd.lastUpdatedTime.Add(timeUntilStoreDead) if now.After(deadAsOf) { @@ -627,7 +633,7 @@ func (sp *StorePool) IsDead(storeID roachpb.StoreID) (bool, time.Duration, error } // If there's no descriptor (meaning no gossip ever arrived for this // store), return unavailable. - if sd.desc == nil { + if sd.Desc == nil { return false, 0, errors.Errorf("store %d status unknown, cant tell if it's dead or alive", storeID) } return false, deadAsOf.Sub(now), nil @@ -666,22 +672,22 @@ func (sp *StorePool) IsLive(storeID roachpb.StoreID) (bool, error) { } func (sp *StorePool) storeStatus(storeID roachpb.StoreID) (storeStatus, error) { - sp.detailsMu.Lock() - defer sp.detailsMu.Unlock() + sp.DetailsMu.Lock() + defer sp.DetailsMu.Unlock() - sd, ok := sp.detailsMu.storeDetails[storeID] + sd, ok := sp.DetailsMu.StoreDetails[storeID] if !ok { return storeStatusUnknown, errors.Errorf("store %d was not found", storeID) } // NB: We use clock.Now().GoTime() instead of clock.PhysicalTime() is order to // take clock signals from remote nodes into consideration. - now := sp.clock.Now().GoTime() - timeUntilStoreDead := TimeUntilStoreDead.Get(&sp.st.SV) - timeAfterStoreSuspect := TimeAfterStoreSuspect.Get(&sp.st.SV) - return sd.status(now, timeUntilStoreDead, sp.nodeLivenessFn, timeAfterStoreSuspect), nil + now := sp.Clock.Now().GoTime() + timeUntilStoreDead := TimeUntilStoreDead.Get(&sp.St.SV) + timeAfterStoreSuspect := TimeAfterStoreSuspect.Get(&sp.St.SV) + return sd.status(now, timeUntilStoreDead, sp.NodeLivenessFn, timeAfterStoreSuspect), nil } -// liveAndDeadReplicas divides the provided repls slice into two slices: the +// LiveAndDeadReplicas divides the provided repls slice into two slices: the // first for live replicas, and the second for dead replicas. // // - Replicas for which liveness or deadness cannot be ascertained @@ -693,20 +699,20 @@ func (sp *StorePool) storeStatus(storeID roachpb.StoreID) (storeStatus, error) { // suspect (i.e. stores that have failed a liveness heartbeat in the recent // past), and stores that are marked as draining are considered live. Otherwise, // they are excluded from the returned slices. -func (sp *StorePool) liveAndDeadReplicas( +func (sp *StorePool) LiveAndDeadReplicas( repls []roachpb.ReplicaDescriptor, includeSuspectAndDrainingStores bool, ) (liveReplicas, deadReplicas []roachpb.ReplicaDescriptor) { - sp.detailsMu.Lock() - defer sp.detailsMu.Unlock() + sp.DetailsMu.Lock() + defer sp.DetailsMu.Unlock() - now := sp.clock.Now().GoTime() - timeUntilStoreDead := TimeUntilStoreDead.Get(&sp.st.SV) - timeAfterStoreSuspect := TimeAfterStoreSuspect.Get(&sp.st.SV) + now := sp.Clock.Now().GoTime() + timeUntilStoreDead := TimeUntilStoreDead.Get(&sp.St.SV) + timeAfterStoreSuspect := TimeAfterStoreSuspect.Get(&sp.St.SV) for _, repl := range repls { - detail := sp.getStoreDetailLocked(repl.StoreID) + detail := sp.GetStoreDetailLocked(repl.StoreID) // Mark replica as dead if store is dead. - status := detail.status(now, timeUntilStoreDead, sp.nodeLivenessFn, timeAfterStoreSuspect) + status := detail.status(now, timeUntilStoreDead, sp.NodeLivenessFn, timeAfterStoreSuspect) switch status { case storeStatusDead: deadReplicas = append(deadReplicas, repl) @@ -729,61 +735,61 @@ func (sp *StorePool) liveAndDeadReplicas( return } -// stat provides a running sample size and running stats. -type stat struct { - n, mean float64 +// Stat provides a running sample size and running stats. +type Stat struct { + n, Mean float64 } -// Update adds the specified value to the stat, augmenting the running stats. -func (s *stat) update(x float64) { +// Update adds the specified value to the Stat, augmenting the running stats. +func (s *Stat) update(x float64) { s.n++ - s.mean += (x - s.mean) / s.n + s.Mean += (x - s.Mean) / s.n } // StoreList holds a list of store descriptors and associated count and used // stats for those stores. type StoreList struct { - stores []roachpb.StoreDescriptor + Stores []roachpb.StoreDescriptor - // candidateRanges tracks range count stats for stores that are eligible to + // CandidateRanges tracks range count stats for Stores that are eligible to // be rebalance targets (their used capacity percentage must be lower than // maxFractionUsedThreshold). - candidateRanges stat + CandidateRanges Stat - // candidateLeases tracks range lease stats for stores that are eligible to + // CandidateLeases tracks range lease stats for Stores that are eligible to // be rebalance targets. - candidateLeases stat + CandidateLeases Stat - // candidateLogicalBytes tracks disk usage stats for stores that are eligible + // candidateLogicalBytes tracks disk usage stats for Stores that are eligible // to be rebalance targets. - candidateLogicalBytes stat + candidateLogicalBytes Stat - // candidateQueriesPerSecond tracks queries-per-second stats for stores that + // CandidateQueriesPerSecond tracks queries-per-second stats for Stores that // are eligible to be rebalance targets. - candidateQueriesPerSecond stat + CandidateQueriesPerSecond Stat - // candidateWritesPerSecond tracks writes-per-second stats for stores that are + // candidateWritesPerSecond tracks writes-per-second stats for Stores that are // eligible to be rebalance targets. - candidateWritesPerSecond stat + candidateWritesPerSecond Stat - // candidateWritesPerSecond tracks L0 sub-level stats for stores that are + // candidateWritesPerSecond tracks L0 sub-level stats for Stores that are // eligible to be rebalance targets. - candidateL0Sublevels stat + CandidateL0Sublevels Stat } -// Generates a new store list based on the passed in descriptors. It will -// maintain the order of those descriptors. -func makeStoreList(descriptors []roachpb.StoreDescriptor) StoreList { - sl := StoreList{stores: descriptors} +// MakeStoreList constructs a new store list based on the passed in descriptors. +// It will maintain the order of those descriptors. +func MakeStoreList(descriptors []roachpb.StoreDescriptor) StoreList { + sl := StoreList{Stores: descriptors} for _, desc := range descriptors { - if maxCapacityCheck(desc) { - sl.candidateRanges.update(float64(desc.Capacity.RangeCount)) + if allocator.MaxCapacityCheck(desc) { + sl.CandidateRanges.update(float64(desc.Capacity.RangeCount)) } - sl.candidateLeases.update(float64(desc.Capacity.LeaseCount)) + sl.CandidateLeases.update(float64(desc.Capacity.LeaseCount)) sl.candidateLogicalBytes.update(float64(desc.Capacity.LogicalBytes)) - sl.candidateQueriesPerSecond.update(desc.Capacity.QueriesPerSecond) + sl.CandidateQueriesPerSecond.update(desc.Capacity.QueriesPerSecond) sl.candidateWritesPerSecond.update(desc.Capacity.WritesPerSecond) - sl.candidateL0Sublevels.update(float64(desc.Capacity.L0Sublevels)) + sl.CandidateL0Sublevels.update(float64(desc.Capacity.L0Sublevels)) } return sl } @@ -792,17 +798,17 @@ func (sl StoreList) String() string { var buf bytes.Buffer fmt.Fprintf(&buf, " candidate: avg-ranges=%v avg-leases=%v avg-disk-usage=%v avg-queries-per-second=%v", - sl.candidateRanges.mean, - sl.candidateLeases.mean, - humanizeutil.IBytes(int64(sl.candidateLogicalBytes.mean)), - sl.candidateQueriesPerSecond.mean, + sl.CandidateRanges.Mean, + sl.CandidateLeases.Mean, + humanizeutil.IBytes(int64(sl.candidateLogicalBytes.Mean)), + sl.CandidateQueriesPerSecond.Mean, ) - if len(sl.stores) > 0 { + if len(sl.Stores) > 0 { fmt.Fprintf(&buf, "\n") } else { fmt.Fprintf(&buf, " ") } - for _, desc := range sl.stores { + for _, desc := range sl.Stores { fmt.Fprintf(&buf, " %d: ranges=%d leases=%d disk-usage=%s queries-per-second=%.2f l0-sublevels=%d\n", desc.StoreID, desc.Capacity.RangeCount, desc.Capacity.LeaseCount, humanizeutil.IBytes(desc.Capacity.LogicalBytes), @@ -813,113 +819,116 @@ func (sl StoreList) String() string { return buf.String() } -// excludeInvalid takes a store list and removes stores that would be explicitly invalid +// ExcludeInvalid takes a store list and removes Stores that would be explicitly invalid // under the given set of constraints. It maintains the original order of the // passed in store list. -func (sl StoreList) excludeInvalid(constraints []roachpb.ConstraintsConjunction) StoreList { +func (sl StoreList) ExcludeInvalid(constraints []roachpb.ConstraintsConjunction) StoreList { if len(constraints) == 0 { return sl } var filteredDescs []roachpb.StoreDescriptor - for _, store := range sl.stores { - if ok := isStoreValid(store, constraints); ok { + for _, store := range sl.Stores { + if ok := allocator.IsStoreValid(store, constraints); ok { filteredDescs = append(filteredDescs, store) } } - return makeStoreList(filteredDescs) + return MakeStoreList(filteredDescs) } -type storeFilter int +// StoreFilter is one of StoreFilter{None,Throttled,Suspect}, controlling what +// stores are excluded from the storeList. +type StoreFilter int const ( - _ storeFilter = iota - // storeFilterNone requests that the storeList include all live stores. Dead, + _ StoreFilter = iota + // StoreFilterNone requests that the storeList include all live stores. Dead, // unknown, and corrupted stores are always excluded from the storeList. - storeFilterNone - // storeFilterThrottled requests that the returned store list additionally + StoreFilterNone + // StoreFilterThrottled requests that the returned store list additionally // exclude stores that have been throttled for declining a snapshot. (See // storePool.throttle for details.) Throttled stores should not be considered // for replica rebalancing, for example, but can still be considered for lease // rebalancing. - storeFilterThrottled - // storeFilterSuspect requests that the returned store list additionally + StoreFilterThrottled + // StoreFilterSuspect requests that the returned store list additionally // exclude stores that have been suspected as unhealthy. We dont want unhealthy // stores to be considered for rebalancing or for lease transfers. i.e. we dont // actively shift leases or replicas away from them, but we dont allow them to // get any new ones until they get better. - storeFilterSuspect + StoreFilterSuspect ) -type throttledStoreReasons []string +// ThrottledStoreReasons is the set of reasons why stores have been throttled. +type ThrottledStoreReasons []string -// getStoreList returns a storeList that contains all active stores that contain +// GetStoreList returns a storeList that contains all active stores that contain // the required attributes and their associated stats. The storeList is filtered // according to the provided storeFilter. It also returns the total number of // alive stores and a list of throttled stores with a reason for why they're // throttled. -func (sp *StorePool) getStoreList(filter storeFilter) (StoreList, int, throttledStoreReasons) { - sp.detailsMu.Lock() - defer sp.detailsMu.Unlock() +func (sp *StorePool) GetStoreList(filter StoreFilter) (StoreList, int, ThrottledStoreReasons) { + sp.DetailsMu.Lock() + defer sp.DetailsMu.Unlock() var storeIDs roachpb.StoreIDSlice - for storeID := range sp.detailsMu.storeDetails { + for storeID := range sp.DetailsMu.StoreDetails { storeIDs = append(storeIDs, storeID) } return sp.getStoreListFromIDsLocked(storeIDs, filter) } -// getStoreListFromIDs is the same function as getStoreList but only returns stores +// GetStoreListFromIDs is the same function as GetStoreList but only returns stores // from the subset of passed in store IDs. -func (sp *StorePool) getStoreListFromIDs( - storeIDs roachpb.StoreIDSlice, filter storeFilter, -) (StoreList, int, throttledStoreReasons) { - sp.detailsMu.Lock() - defer sp.detailsMu.Unlock() +func (sp *StorePool) GetStoreListFromIDs( + storeIDs roachpb.StoreIDSlice, filter StoreFilter, +) (StoreList, int, ThrottledStoreReasons) { + sp.DetailsMu.Lock() + defer sp.DetailsMu.Unlock() return sp.getStoreListFromIDsLocked(storeIDs, filter) } -// getStoreListFromIDsRLocked is the same function as getStoreList but requires +// getStoreListFromIDsRLocked is the same function as GetStoreList but requires // that the detailsMU read lock is held. func (sp *StorePool) getStoreListFromIDsLocked( - storeIDs roachpb.StoreIDSlice, filter storeFilter, -) (StoreList, int, throttledStoreReasons) { - if sp.deterministic { + storeIDs roachpb.StoreIDSlice, filter StoreFilter, +) (StoreList, int, ThrottledStoreReasons) { + if sp.Deterministic { sort.Sort(storeIDs) } else { shuffle.Shuffle(storeIDs) } var aliveStoreCount int - var throttled throttledStoreReasons + var throttled ThrottledStoreReasons var storeDescriptors []roachpb.StoreDescriptor - now := sp.clock.Now().GoTime() - timeUntilStoreDead := TimeUntilStoreDead.Get(&sp.st.SV) - timeAfterStoreSuspect := TimeAfterStoreSuspect.Get(&sp.st.SV) + now := sp.Clock.Now().GoTime() + timeUntilStoreDead := TimeUntilStoreDead.Get(&sp.St.SV) + timeAfterStoreSuspect := TimeAfterStoreSuspect.Get(&sp.St.SV) for _, storeID := range storeIDs { - detail, ok := sp.detailsMu.storeDetails[storeID] + detail, ok := sp.DetailsMu.StoreDetails[storeID] if !ok { // Do nothing; this store is not in the StorePool. continue } - switch s := detail.status(now, timeUntilStoreDead, sp.nodeLivenessFn, timeAfterStoreSuspect); s { + switch s := detail.status(now, timeUntilStoreDead, sp.NodeLivenessFn, timeAfterStoreSuspect); s { case storeStatusThrottled: aliveStoreCount++ throttled = append(throttled, detail.throttledBecause) - if filter != storeFilterThrottled { - storeDescriptors = append(storeDescriptors, *detail.desc) + if filter != StoreFilterThrottled { + storeDescriptors = append(storeDescriptors, *detail.Desc) } case storeStatusAvailable: aliveStoreCount++ - storeDescriptors = append(storeDescriptors, *detail.desc) + storeDescriptors = append(storeDescriptors, *detail.Desc) case storeStatusDraining: throttled = append(throttled, fmt.Sprintf("s%d: draining", storeID)) case storeStatusSuspect: aliveStoreCount++ throttled = append(throttled, fmt.Sprintf("s%d: suspect", storeID)) - if filter != storeFilterThrottled && filter != storeFilterSuspect { - storeDescriptors = append(storeDescriptors, *detail.desc) + if filter != StoreFilterThrottled && filter != StoreFilterSuspect { + storeDescriptors = append(storeDescriptors, *detail.Desc) } case storeStatusDead, storeStatusUnknown, storeStatusDecommissioning: // Do nothing; this store cannot be used. @@ -927,48 +936,51 @@ func (sp *StorePool) getStoreListFromIDsLocked( panic(fmt.Sprintf("unknown store status: %d", s)) } } - return makeStoreList(storeDescriptors), aliveStoreCount, throttled + return MakeStoreList(storeDescriptors), aliveStoreCount, throttled } -type throttleReason int +// ThrottleReason encodes the reason for throttling a given store. +type ThrottleReason int const ( - _ throttleReason = iota - throttleFailed + _ ThrottleReason = iota + // ThrottleFailed is used when we're throttling as a result of a failed + // operation. + ThrottleFailed ) -// throttle informs the store pool that the given remote store declined a +// Throttle informs the store pool that the given remote store declined a // snapshot or failed to apply one, ensuring that it will not be considered // for up-replication or rebalancing until after the configured timeout period // has elapsed. Declined being true indicates that the remote store explicitly // declined a snapshot. -func (sp *StorePool) throttle(reason throttleReason, why string, storeID roachpb.StoreID) { - sp.detailsMu.Lock() - defer sp.detailsMu.Unlock() - detail := sp.getStoreDetailLocked(storeID) +func (sp *StorePool) Throttle(reason ThrottleReason, why string, storeID roachpb.StoreID) { + sp.DetailsMu.Lock() + defer sp.DetailsMu.Unlock() + detail := sp.GetStoreDetailLocked(storeID) detail.throttledBecause = why // If a snapshot is declined, we mark the store detail as having been declined // so it won't be considered as a candidate for new replicas until after the // configured timeout period has passed. switch reason { - case throttleFailed: - timeout := FailedReservationsTimeout.Get(&sp.st.SV) - detail.throttledUntil = sp.clock.PhysicalTime().Add(timeout) + case ThrottleFailed: + timeout := FailedReservationsTimeout.Get(&sp.St.SV) + detail.ThrottledUntil = sp.Clock.PhysicalTime().Add(timeout) if log.V(2) { ctx := sp.AnnotateCtx(context.TODO()) log.Infof(ctx, "snapshot failed (%s), s%d will be throttled for %s until %s", - why, storeID, timeout, detail.throttledUntil) + why, storeID, timeout, detail.ThrottledUntil) } default: log.Warningf(sp.AnnotateCtx(context.TODO()), "unknown throttle reason %v", reason) } } -// getLocalitiesByStore returns the localities for the provided replicas. In +// GetLocalitiesByStore returns the localities for the provided replicas. In // this case we consider the node part of the failure domain and add it to // the locality data. -func (sp *StorePool) getLocalitiesByStore( +func (sp *StorePool) GetLocalitiesByStore( replicas []roachpb.ReplicaDescriptor, ) map[roachpb.StoreID]roachpb.Locality { sp.localitiesMu.RLock() @@ -987,12 +999,12 @@ func (sp *StorePool) getLocalitiesByStore( return localities } -// getLocalitiesByNode returns the localities for the provided replicas. In this +// GetLocalitiesByNode returns the localities for the provided replicas. In this // case we only consider the locality by node, where the node itself is not // part of the failure domain. // TODO(bram): consider storing a full list of all node to node diversity // scores for faster lookups. -func (sp *StorePool) getLocalitiesByNode( +func (sp *StorePool) GetLocalitiesByNode( replicas []roachpb.ReplicaDescriptor, ) map[roachpb.NodeID]roachpb.Locality { sp.localitiesMu.RLock() @@ -1008,9 +1020,9 @@ func (sp *StorePool) getLocalitiesByNode( return localities } -// getNodeLocalityString returns the locality information for the given node +// GetNodeLocalityString returns the locality information for the given node // in its string format. -func (sp *StorePool) getNodeLocalityString(nodeID roachpb.NodeID) string { +func (sp *StorePool) GetNodeLocalityString(nodeID roachpb.NodeID) string { sp.localitiesMu.RLock() defer sp.localitiesMu.RUnlock() locality, ok := sp.localitiesMu.nodeLocalities[nodeID] @@ -1040,3 +1052,17 @@ func (sp *StorePool) isStoreReadyForRoutineReplicaTransferInternal( panic(fmt.Sprintf("unknown store status: %d", status)) } } + +// TestingGetStoreList exposes getStoreList for testing only, but with a +// hardcoded storeFilter of storeFilterNone. +func (sp *StorePool) TestingGetStoreList() (StoreList, int, int) { + list, available, throttled := sp.GetStoreList(StoreFilterNone) + return list, available, len(throttled) +} + +// TestingStores returns a copy of sl.stores. +func (sl *StoreList) TestingStores() []roachpb.StoreDescriptor { + stores := make([]roachpb.StoreDescriptor, len(sl.Stores)) + copy(stores, sl.Stores) + return stores +} diff --git a/pkg/kv/kvserver/store_pool_test.go b/pkg/kv/kvserver/allocator/storepool/store_pool_test.go similarity index 61% rename from pkg/kv/kvserver/store_pool_test.go rename to pkg/kv/kvserver/allocator/storepool/store_pool_test.go index 888b37423716..2dababee9d9f 100644 --- a/pkg/kv/kvserver/store_pool_test.go +++ b/pkg/kv/kvserver/allocator/storepool/store_pool_test.go @@ -1,4 +1,4 @@ -// Copyright 2015 The Cockroach Authors. +// Copyright 2022 The Cockroach Authors. // // Use of this software is governed by the Business Source License // included in the file licenses/BSL.txt. @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package kvserver +package storepool import ( "context" @@ -19,24 +19,13 @@ import ( "testing" "time" - "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" - "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/rpc" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/storage" - "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils/gossiputil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/metric" - "github.com/cockroachdb/cockroach/pkg/util/stop" - "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" - "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "github.com/kr/pretty" "github.com/stretchr/testify/require" @@ -57,103 +46,32 @@ var uniqueStore = []*roachpb.StoreDescriptor{ }, } -type mockNodeLiveness struct { - syncutil.Mutex - defaultNodeStatus livenesspb.NodeLivenessStatus - nodes map[roachpb.NodeID]livenesspb.NodeLivenessStatus -} - -func newMockNodeLiveness(defaultNodeStatus livenesspb.NodeLivenessStatus) *mockNodeLiveness { - return &mockNodeLiveness{ - defaultNodeStatus: defaultNodeStatus, - nodes: map[roachpb.NodeID]livenesspb.NodeLivenessStatus{}, - } -} - -func (m *mockNodeLiveness) setNodeStatus( - nodeID roachpb.NodeID, status livenesspb.NodeLivenessStatus, -) { - m.Lock() - defer m.Unlock() - m.nodes[nodeID] = status -} - -func (m *mockNodeLiveness) nodeLivenessFunc( - nodeID roachpb.NodeID, now time.Time, threshold time.Duration, -) livenesspb.NodeLivenessStatus { - m.Lock() - defer m.Unlock() - if status, ok := m.nodes[nodeID]; ok { - return status - } - return m.defaultNodeStatus -} - -// createTestStorePool creates a stopper, gossip and storePool for use in -// tests. Stopper must be stopped by the caller. -func createTestStorePool( - ctx context.Context, - timeUntilStoreDeadValue time.Duration, - deterministic bool, - nodeCount NodeCountFunc, - defaultNodeStatus livenesspb.NodeLivenessStatus, -) (*stop.Stopper, *gossip.Gossip, *hlc.ManualClock, *StorePool, *mockNodeLiveness) { - stopper := stop.NewStopper() - mc := hlc.NewManualClock(123) - clock := hlc.NewClock(mc.UnixNano, time.Nanosecond) - st := cluster.MakeTestingClusterSettings() - ambientCtx := log.MakeTestingAmbientContext(stopper.Tracer()) - rpcContext := rpc.NewContext(ctx, - rpc.ContextOptions{ - TenantID: roachpb.SystemTenantID, - Config: &base.Config{Insecure: true}, - Clock: clock, - Stopper: stopper, - Settings: st, - }) - server := rpc.NewServer(rpcContext) // never started - g := gossip.NewTest(1, rpcContext, server, stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) - mnl := newMockNodeLiveness(defaultNodeStatus) - - TimeUntilStoreDead.Override(ctx, &st.SV, timeUntilStoreDeadValue) - storePool := NewStorePool( - ambientCtx, - st, - g, - clock, - nodeCount, - mnl.nodeLivenessFunc, - deterministic, - ) - return stopper, g, mc, storePool, mnl -} - // TestStorePoolGossipUpdate ensures that the gossip callback in StorePool // correctly updates a store's details. func TestStorePoolGossipUpdate(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) ctx := context.Background() - stopper, g, _, sp, _ := createTestStorePool(ctx, + stopper, g, _, sp, _ := CreateTestStorePool(ctx, TestTimeUntilStoreDead, false, /* deterministic */ func() int { return 0 }, /* NodeCount */ livenesspb.NodeLivenessStatus_DEAD) defer stopper.Stop(ctx) sg := gossiputil.NewStoreGossiper(g) - sp.detailsMu.RLock() - if _, ok := sp.detailsMu.storeDetails[2]; ok { + sp.DetailsMu.RLock() + if _, ok := sp.DetailsMu.StoreDetails[2]; ok { t.Fatalf("store 2 is already in the pool's store list") } - sp.detailsMu.RUnlock() + sp.DetailsMu.RUnlock() sg.GossipStores(uniqueStore, t) - sp.detailsMu.RLock() - if _, ok := sp.detailsMu.storeDetails[2]; !ok { + sp.DetailsMu.RLock() + if _, ok := sp.DetailsMu.StoreDetails[2]; !ok { t.Fatalf("store 2 isn't in the pool's store list") } - sp.detailsMu.RUnlock() + sp.DetailsMu.RUnlock() } // verifyStoreList ensures that the returned list of stores is correct. @@ -161,21 +79,21 @@ func verifyStoreList( sp *StorePool, constraints []roachpb.ConstraintsConjunction, storeIDs roachpb.StoreIDSlice, // optional - filter storeFilter, + filter StoreFilter, expected []int, expectedAliveStoreCount int, expectedThrottledStoreCount int, ) error { var sl StoreList var aliveStoreCount int - var throttled throttledStoreReasons + var throttled ThrottledStoreReasons if storeIDs == nil { - sl, aliveStoreCount, throttled = sp.getStoreList(filter) + sl, aliveStoreCount, throttled = sp.GetStoreList(filter) } else { - sl, aliveStoreCount, throttled = sp.getStoreListFromIDs(storeIDs, filter) + sl, aliveStoreCount, throttled = sp.GetStoreListFromIDs(storeIDs, filter) } throttledStoreCount := len(throttled) - sl = sl.excludeInvalid(constraints) + sl = sl.ExcludeInvalid(constraints) if aliveStoreCount != expectedAliveStoreCount { return errors.Errorf("expected AliveStoreCount %d does not match actual %d", expectedAliveStoreCount, aliveStoreCount) @@ -185,7 +103,7 @@ func verifyStoreList( expectedThrottledStoreCount, throttledStoreCount) } var actual []int - for _, store := range sl.stores { + for _, store := range sl.Stores { actual = append(actual, int(store.StoreID)) } sort.Ints(expected) @@ -203,7 +121,7 @@ func TestStorePoolGetStoreList(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() // We're going to manually mark stores dead in this test. - stopper, g, _, sp, mnl := createTestStorePool(ctx, + stopper, g, _, sp, mnl := CreateTestStorePool(ctx, TestTimeUntilStoreDead, false, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_DEAD) @@ -219,10 +137,10 @@ func TestStorePoolGetStoreList(t *testing.T) { } required := []string{"ssd", "dc"} // Nothing yet. - sl, _, _ := sp.getStoreList(storeFilterNone) - sl = sl.excludeInvalid(constraints) - if len(sl.stores) != 0 { - t.Errorf("expected no stores, instead %+v", sl.stores) + sl, _, _ := sp.GetStoreList(StoreFilterNone) + sl = sl.ExcludeInvalid(constraints) + if len(sl.Stores) != 0 { + t.Errorf("expected no stores, instead %+v", sl.Stores) } matchingStore := roachpb.StoreDescriptor{ @@ -278,25 +196,25 @@ func TestStorePoolGetStoreList(t *testing.T) { // absentStore is purposefully not gossiped. }, t) for i := 1; i <= 8; i++ { - mnl.setNodeStatus(roachpb.NodeID(i), livenesspb.NodeLivenessStatus_LIVE) + mnl.SetNodeStatus(roachpb.NodeID(i), livenesspb.NodeLivenessStatus_LIVE) } // Set deadStore as dead. - mnl.setNodeStatus(deadStore.Node.NodeID, livenesspb.NodeLivenessStatus_DEAD) - sp.detailsMu.Lock() + mnl.SetNodeStatus(deadStore.Node.NodeID, livenesspb.NodeLivenessStatus_DEAD) + sp.DetailsMu.Lock() // Set declinedStore as throttled. - sp.detailsMu.storeDetails[declinedStore.StoreID].throttledUntil = sp.clock.Now().GoTime().Add(time.Hour) + sp.DetailsMu.StoreDetails[declinedStore.StoreID].ThrottledUntil = sp.Clock.Now().GoTime().Add(time.Hour) // Set suspectedStore as suspected. - sp.detailsMu.storeDetails[suspectedStore.StoreID].lastAvailable = sp.clock.Now().GoTime() - sp.detailsMu.storeDetails[suspectedStore.StoreID].lastUnavailable = sp.clock.Now().GoTime() - sp.detailsMu.Unlock() + sp.DetailsMu.StoreDetails[suspectedStore.StoreID].LastAvailable = sp.Clock.Now().GoTime() + sp.DetailsMu.StoreDetails[suspectedStore.StoreID].LastUnavailable = sp.Clock.Now().GoTime() + sp.DetailsMu.Unlock() // No filter or limited set of store IDs. if err := verifyStoreList( sp, constraints, nil, /* storeIDs */ - storeFilterNone, + StoreFilterNone, []int{ int(matchingStore.StoreID), int(supersetStore.StoreID), @@ -314,7 +232,7 @@ func TestStorePoolGetStoreList(t *testing.T) { sp, constraints, nil, /* storeIDs */ - storeFilterThrottled, + StoreFilterThrottled, []int{ int(matchingStore.StoreID), int(supersetStore.StoreID), @@ -330,7 +248,7 @@ func TestStorePoolGetStoreList(t *testing.T) { sp, constraints, nil, /* storeIDs */ - storeFilterSuspect, + StoreFilterSuspect, []int{ int(matchingStore.StoreID), int(supersetStore.StoreID), @@ -355,7 +273,7 @@ func TestStorePoolGetStoreList(t *testing.T) { sp, constraints, limitToStoreIDs, - storeFilterNone, + StoreFilterNone, []int{ int(matchingStore.StoreID), int(declinedStore.StoreID), @@ -373,7 +291,7 @@ func TestStorePoolGetStoreList(t *testing.T) { sp, constraints, limitToStoreIDs, - storeFilterThrottled, + StoreFilterThrottled, []int{ int(matchingStore.StoreID), }, @@ -389,7 +307,7 @@ func TestStorePoolGetStoreList(t *testing.T) { sp, constraints, limitToStoreIDs, - storeFilterSuspect, + StoreFilterSuspect, []int{ int(matchingStore.StoreID), int(declinedStore.StoreID), @@ -483,200 +401,15 @@ func TestStoreListFilter(t *testing.T) { } else { storeDesc.Node.Attrs.Attrs = s.attributes } - sl.stores = append(sl.stores, storeDesc) + sl.Stores = append(sl.Stores, storeDesc) if s.expected { expected = append(expected, storeDesc) } } - filtered := sl.excludeInvalid(constraints) - if !reflect.DeepEqual(expected, filtered.stores) { - t.Errorf("did not get expected stores %s", pretty.Diff(expected, filtered.stores)) - } -} - -func TestStorePoolUpdateLocalStore(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - manual := hlc.NewManualClock(123) - clock := hlc.NewClock(manual.UnixNano, time.Nanosecond) - ctx := context.Background() - // We're going to manually mark stores dead in this test. - stopper, g, _, sp, _ := createTestStorePool(ctx, - TestTimeUntilStoreDead, false, /* deterministic */ - func() int { return 10 }, /* nodeCount */ - livenesspb.NodeLivenessStatus_DEAD) - defer stopper.Stop(ctx) - sg := gossiputil.NewStoreGossiper(g) - stores := []*roachpb.StoreDescriptor{ - { - StoreID: 1, - Node: roachpb.NodeDescriptor{NodeID: 1}, - Capacity: roachpb.StoreCapacity{ - Capacity: 100, - Available: 50, - RangeCount: 5, - LeaseCount: 1, - LogicalBytes: 30, - QueriesPerSecond: 100, - WritesPerSecond: 30, - L0Sublevels: 4, - }, - }, - { - StoreID: 2, - Node: roachpb.NodeDescriptor{NodeID: 2}, - Capacity: roachpb.StoreCapacity{ - Capacity: 100, - Available: 55, - RangeCount: 4, - LeaseCount: 2, - LogicalBytes: 25, - QueriesPerSecond: 50, - WritesPerSecond: 25, - L0Sublevels: 8, - }, - }, - } - sg.GossipStores(stores, t) - - replica := &Replica{RangeID: 1} - replica.mu.Lock() - replica.mu.state.Stats = &enginepb.MVCCStats{ - KeyBytes: 2, - ValBytes: 4, - } - replica.mu.Unlock() - rs := newReplicaStats(clock, nil) - for _, store := range stores { - rs.recordCount(1, store.Node.NodeID) - } - manual.Increment(int64(MinStatsDuration + time.Second)) - replica.leaseholderStats = rs - replica.writeStats = rs - - rangeUsageInfo := rangeUsageInfoForRepl(replica) - - sp.updateLocalStoreAfterRebalance(roachpb.StoreID(1), rangeUsageInfo, roachpb.ADD_VOTER) - desc, ok := sp.getStoreDescriptor(roachpb.StoreID(1)) - if !ok { - t.Fatalf("couldn't find StoreDescriptor for Store ID %d", 1) - } - QPS, _ := replica.leaseholderStats.averageRatePerSecond() - WPS, _ := replica.writeStats.averageRatePerSecond() - if expectedRangeCount := int32(6); desc.Capacity.RangeCount != expectedRangeCount { - t.Errorf("expected RangeCount %d, but got %d", expectedRangeCount, desc.Capacity.RangeCount) - } - if expectedBytes := int64(36); desc.Capacity.LogicalBytes != expectedBytes { - t.Errorf("expected logical bytes %d, but got %d", expectedBytes, desc.Capacity.LogicalBytes) - } - if expectedQPS := float64(100); desc.Capacity.QueriesPerSecond != expectedQPS { - t.Errorf("expected QueriesPerSecond %f, but got %f", expectedQPS, desc.Capacity.QueriesPerSecond) - } - if expectedWPS := 30 + WPS; desc.Capacity.WritesPerSecond != expectedWPS { - t.Errorf("expected WritesPerSecond %f, but got %f", expectedWPS, desc.Capacity.WritesPerSecond) - } - if expectedL0Sublevels := int64(4); desc.Capacity.L0Sublevels != expectedL0Sublevels { - t.Errorf("expected L0 Sub-Levels %d, but got %d", expectedL0Sublevels, desc.Capacity.L0Sublevels) - } - - sp.updateLocalStoreAfterRebalance(roachpb.StoreID(2), rangeUsageInfo, roachpb.REMOVE_VOTER) - desc, ok = sp.getStoreDescriptor(roachpb.StoreID(2)) - if !ok { - t.Fatalf("couldn't find StoreDescriptor for Store ID %d", 2) - } - if expectedRangeCount := int32(3); desc.Capacity.RangeCount != expectedRangeCount { - t.Errorf("expected RangeCount %d, but got %d", expectedRangeCount, desc.Capacity.RangeCount) - } - if expectedBytes := int64(19); desc.Capacity.LogicalBytes != expectedBytes { - t.Errorf("expected logical bytes %d, but got %d", expectedBytes, desc.Capacity.LogicalBytes) - } - if expectedQPS := float64(50); desc.Capacity.QueriesPerSecond != expectedQPS { - t.Errorf("expected QueriesPerSecond %f, but got %f", expectedQPS, desc.Capacity.QueriesPerSecond) - } - if expectedWPS := 25 - WPS; desc.Capacity.WritesPerSecond != expectedWPS { - t.Errorf("expected WritesPerSecond %f, but got %f", expectedWPS, desc.Capacity.WritesPerSecond) - } - if expectedL0Sublevels := int64(8); desc.Capacity.L0Sublevels != expectedL0Sublevels { - t.Errorf("expected L0 Sub-Levels %d, but got %d", expectedL0Sublevels, desc.Capacity.L0Sublevels) - } - - sp.updateLocalStoresAfterLeaseTransfer(roachpb.StoreID(1), roachpb.StoreID(2), rangeUsageInfo.QueriesPerSecond) - desc, ok = sp.getStoreDescriptor(roachpb.StoreID(1)) - if !ok { - t.Fatalf("couldn't find StoreDescriptor for Store ID %d", 1) - } - if expectedLeaseCount := int32(0); desc.Capacity.LeaseCount != expectedLeaseCount { - t.Errorf("expected LeaseCount %d, but got %d", expectedLeaseCount, desc.Capacity.LeaseCount) - } - if expectedQPS := 100 - QPS; desc.Capacity.QueriesPerSecond != expectedQPS { - t.Errorf("expected QueriesPerSecond %f, but got %f", expectedQPS, desc.Capacity.QueriesPerSecond) - } - desc, ok = sp.getStoreDescriptor(roachpb.StoreID(2)) - if !ok { - t.Fatalf("couldn't find StoreDescriptor for Store ID %d", 2) - } - if expectedLeaseCount := int32(3); desc.Capacity.LeaseCount != expectedLeaseCount { - t.Errorf("expected LeaseCount %d, but got %d", expectedLeaseCount, desc.Capacity.LeaseCount) - } - if expectedQPS := 50 + QPS; desc.Capacity.QueriesPerSecond != expectedQPS { - t.Errorf("expected QueriesPerSecond %f, but got %f", expectedQPS, desc.Capacity.QueriesPerSecond) - } -} - -// TestStorePoolUpdateLocalStoreBeforeGossip verifies that an attempt to update -// the local copy of store before that store has been gossiped will be a no-op. -func TestStorePoolUpdateLocalStoreBeforeGossip(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - ctx := context.Background() - manual := hlc.NewManualClock(123) - clock := hlc.NewClock(manual.UnixNano, time.Nanosecond) - stopper, _, _, sp, _ := createTestStorePool(ctx, - TestTimeUntilStoreDead, false, /* deterministic */ - func() int { return 10 }, /* nodeCount */ - livenesspb.NodeLivenessStatus_DEAD) - defer stopper.Stop(ctx) - - // Create store. - node := roachpb.NodeDescriptor{NodeID: roachpb.NodeID(1)} - eng := storage.NewDefaultInMemForTesting() - stopper.AddCloser(eng) - cfg := TestStoreConfig(clock) - cfg.Transport = NewDummyRaftTransport(cfg.Settings, cfg.AmbientCtx.Tracer) - store := NewStore(ctx, cfg, eng, &node) - // Fake an ident because this test doesn't want to start the store - // but without an Ident there will be NPEs. - store.Ident = &roachpb.StoreIdent{ - ClusterID: uuid.Nil, - StoreID: 1, - NodeID: 1, - } - - // Create replica. - rg := roachpb.RangeDescriptor{ - RangeID: 1, - StartKey: roachpb.RKey([]byte("a")), - EndKey: roachpb.RKey([]byte("b")), - NextReplicaID: 1, - } - rg.AddReplica(1, 1, roachpb.VOTER_FULL) - replica, err := newReplica(ctx, &rg, store, 1) - if err != nil { - t.Fatalf("make replica error : %+v", err) - } - replica.leaseholderStats = newReplicaStats(store.Clock(), nil) - - rangeUsageInfo := rangeUsageInfoForRepl(replica) - - // Update StorePool, which should be a no-op. - storeID := roachpb.StoreID(1) - if _, ok := sp.getStoreDescriptor(storeID); ok { - t.Fatalf("StoreDescriptor not gossiped, should not be found") - } - sp.updateLocalStoreAfterRebalance(storeID, rangeUsageInfo, roachpb.ADD_VOTER) - if _, ok := sp.getStoreDescriptor(storeID); ok { - t.Fatalf("StoreDescriptor still not gossiped, should not be found") + filtered := sl.ExcludeInvalid(constraints) + if !reflect.DeepEqual(expected, filtered.Stores) { + t.Errorf("did not get expected stores %s", pretty.Diff(expected, filtered.Stores)) } } @@ -684,7 +417,7 @@ func TestStorePoolGetStoreDetails(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) ctx := context.Background() - stopper, g, _, sp, _ := createTestStorePool(ctx, + stopper, g, _, sp, _ := CreateTestStorePool(ctx, TestTimeUntilStoreDead, false, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_DEAD) @@ -692,12 +425,12 @@ func TestStorePoolGetStoreDetails(t *testing.T) { sg := gossiputil.NewStoreGossiper(g) sg.GossipStores(uniqueStore, t) - sp.detailsMu.Lock() - defer sp.detailsMu.Unlock() - if detail := sp.getStoreDetailLocked(roachpb.StoreID(1)); detail.desc != nil { - t.Errorf("unexpected fetched store ID 1: %+v", detail.desc) + sp.DetailsMu.Lock() + defer sp.DetailsMu.Unlock() + if detail := sp.GetStoreDetailLocked(roachpb.StoreID(1)); detail.Desc != nil { + t.Errorf("unexpected fetched store ID 1: %+v", detail.Desc) } - if detail := sp.getStoreDetailLocked(roachpb.StoreID(2)); detail.desc == nil { + if detail := sp.GetStoreDetailLocked(roachpb.StoreID(2)); detail.Desc == nil { t.Errorf("failed to fetch store ID 2") } } @@ -706,7 +439,7 @@ func TestStorePoolFindDeadReplicas(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) ctx := context.Background() - stopper, g, _, sp, mnl := createTestStorePool(ctx, + stopper, g, _, sp, mnl := CreateTestStorePool(ctx, TestTimeUntilStoreDead, false, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_DEAD) @@ -766,10 +499,10 @@ func TestStorePoolFindDeadReplicas(t *testing.T) { sg.GossipStores(stores, t) for i := 1; i <= 5; i++ { - mnl.setNodeStatus(roachpb.NodeID(i), livenesspb.NodeLivenessStatus_LIVE) + mnl.SetNodeStatus(roachpb.NodeID(i), livenesspb.NodeLivenessStatus_LIVE) } - liveReplicas, deadReplicas := sp.liveAndDeadReplicas(replicas, false /* includeSuspectAndDrainingStores */) + liveReplicas, deadReplicas := sp.LiveAndDeadReplicas(replicas, false /* includeSuspectAndDrainingStores */) if len(liveReplicas) != 5 { t.Fatalf("expected five live replicas, found %d (%v)", len(liveReplicas), liveReplicas) } @@ -777,10 +510,10 @@ func TestStorePoolFindDeadReplicas(t *testing.T) { t.Fatalf("expected no dead replicas initially, found %d (%v)", len(deadReplicas), deadReplicas) } // Mark nodes 4 & 5 as dead. - mnl.setNodeStatus(4, livenesspb.NodeLivenessStatus_DEAD) - mnl.setNodeStatus(5, livenesspb.NodeLivenessStatus_DEAD) + mnl.SetNodeStatus(4, livenesspb.NodeLivenessStatus_DEAD) + mnl.SetNodeStatus(5, livenesspb.NodeLivenessStatus_DEAD) - liveReplicas, deadReplicas = sp.liveAndDeadReplicas(replicas, false /* includeSuspectNodes */) + liveReplicas, deadReplicas = sp.LiveAndDeadReplicas(replicas, false /* includeSuspectNodes */) if a, e := liveReplicas, replicas[:3]; !reflect.DeepEqual(a, e) { t.Fatalf("expected live replicas %+v; got %+v", e, a) } @@ -789,9 +522,9 @@ func TestStorePoolFindDeadReplicas(t *testing.T) { } // Mark node 4 as merely unavailable. - mnl.setNodeStatus(4, livenesspb.NodeLivenessStatus_UNAVAILABLE) + mnl.SetNodeStatus(4, livenesspb.NodeLivenessStatus_UNAVAILABLE) - liveReplicas, deadReplicas = sp.liveAndDeadReplicas(replicas, false /* includeSuspectAndDrainingStores */) + liveReplicas, deadReplicas = sp.LiveAndDeadReplicas(replicas, false /* includeSuspectAndDrainingStores */) if a, e := liveReplicas, replicas[:3]; !reflect.DeepEqual(a, e) { t.Fatalf("expected live replicas %+v; got %+v", e, a) } @@ -811,13 +544,13 @@ func TestStorePoolDefaultState(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) ctx := context.Background() - stopper, _, _, sp, _ := createTestStorePool(ctx, + stopper, _, _, sp, _ := CreateTestStorePool(ctx, TestTimeUntilStoreDead, false, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_DEAD) defer stopper.Stop(ctx) - liveReplicas, deadReplicas := sp.liveAndDeadReplicas( + liveReplicas, deadReplicas := sp.LiveAndDeadReplicas( []roachpb.ReplicaDescriptor{{StoreID: 1}}, false, /* includeSuspectAndDrainingStores */ ) @@ -825,8 +558,8 @@ func TestStorePoolDefaultState(t *testing.T) { t.Errorf("expected 0 live and 0 dead replicas; got %v and %v", liveReplicas, deadReplicas) } - sl, alive, throttled := sp.getStoreList(storeFilterNone) - if len(sl.stores) > 0 { + sl, alive, throttled := sp.GetStoreList(StoreFilterNone) + if len(sl.Stores) > 0 { t.Errorf("expected no live stores; got list of %v", sl) } if alive != 0 { @@ -841,7 +574,7 @@ func TestStorePoolThrottle(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) ctx := context.Background() - stopper, g, _, sp, _ := createTestStorePool(ctx, + stopper, g, _, sp, _ := CreateTestStorePool(ctx, TestTimeUntilStoreDead, false, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_DEAD) @@ -850,15 +583,15 @@ func TestStorePoolThrottle(t *testing.T) { sg := gossiputil.NewStoreGossiper(g) sg.GossipStores(uniqueStore, t) - expected := sp.clock.Now().GoTime().Add(FailedReservationsTimeout.Get(&sp.st.SV)) - sp.throttle(throttleFailed, "", 1) + expected := sp.Clock.Now().GoTime().Add(FailedReservationsTimeout.Get(&sp.St.SV)) + sp.Throttle(ThrottleFailed, "", 1) - sp.detailsMu.Lock() - detail := sp.getStoreDetailLocked(1) - sp.detailsMu.Unlock() - if !detail.throttledUntil.Equal(expected) { + sp.DetailsMu.Lock() + detail := sp.GetStoreDetailLocked(1) + sp.DetailsMu.Unlock() + if !detail.ThrottledUntil.Equal(expected) { t.Errorf("expected store to have been throttled to %v, found %v", - expected, detail.throttledUntil) + expected, detail.ThrottledUntil) } } @@ -866,7 +599,7 @@ func TestStorePoolSuspected(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) ctx := context.Background() - stopper, g, _, sp, mnl := createTestStorePool(ctx, + stopper, g, _, sp, mnl := CreateTestStorePool(ctx, TestTimeUntilStoreDeadOff, false, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_DEAD) @@ -876,63 +609,63 @@ func TestStorePoolSuspected(t *testing.T) { sg.GossipStores(uniqueStore, t) store := uniqueStore[0] - now := sp.clock.Now().GoTime() - timeUntilStoreDead := TimeUntilStoreDead.Get(&sp.st.SV) - timeAfterStoreSuspect := TimeAfterStoreSuspect.Get(&sp.st.SV) + now := sp.Clock.Now().GoTime() + timeUntilStoreDead := TimeUntilStoreDead.Get(&sp.St.SV) + timeAfterStoreSuspect := TimeAfterStoreSuspect.Get(&sp.St.SV) // See state transition diagram in storeDetail.status() for a visual // representation of what this test asserts. - mnl.setNodeStatus(store.Node.NodeID, livenesspb.NodeLivenessStatus_LIVE) - sp.detailsMu.Lock() - detail := sp.getStoreDetailLocked(store.StoreID) - s := detail.status(now, timeUntilStoreDead, sp.nodeLivenessFn, timeAfterStoreSuspect) - sp.detailsMu.Unlock() + mnl.SetNodeStatus(store.Node.NodeID, livenesspb.NodeLivenessStatus_LIVE) + sp.DetailsMu.Lock() + detail := sp.GetStoreDetailLocked(store.StoreID) + s := detail.status(now, timeUntilStoreDead, sp.NodeLivenessFn, timeAfterStoreSuspect) + sp.DetailsMu.Unlock() require.Equal(t, s, storeStatusAvailable) - require.False(t, detail.lastAvailable.IsZero()) - require.True(t, detail.lastUnavailable.IsZero()) + require.False(t, detail.LastAvailable.IsZero()) + require.True(t, detail.LastUnavailable.IsZero()) - mnl.setNodeStatus(store.Node.NodeID, livenesspb.NodeLivenessStatus_UNAVAILABLE) - sp.detailsMu.Lock() - s = detail.status(now, timeUntilStoreDead, sp.nodeLivenessFn, timeAfterStoreSuspect) - sp.detailsMu.Unlock() + mnl.SetNodeStatus(store.Node.NodeID, livenesspb.NodeLivenessStatus_UNAVAILABLE) + sp.DetailsMu.Lock() + s = detail.status(now, timeUntilStoreDead, sp.NodeLivenessFn, timeAfterStoreSuspect) + sp.DetailsMu.Unlock() require.Equal(t, s, storeStatusUnknown) - require.False(t, detail.lastAvailable.IsZero()) - require.False(t, detail.lastUnavailable.IsZero()) + require.False(t, detail.LastAvailable.IsZero()) + require.False(t, detail.LastUnavailable.IsZero()) - mnl.setNodeStatus(store.Node.NodeID, livenesspb.NodeLivenessStatus_LIVE) - sp.detailsMu.Lock() - s = detail.status(now, timeUntilStoreDead, sp.nodeLivenessFn, timeAfterStoreSuspect) - sp.detailsMu.Unlock() + mnl.SetNodeStatus(store.Node.NodeID, livenesspb.NodeLivenessStatus_LIVE) + sp.DetailsMu.Lock() + s = detail.status(now, timeUntilStoreDead, sp.NodeLivenessFn, timeAfterStoreSuspect) + sp.DetailsMu.Unlock() require.Equal(t, s, storeStatusSuspect) - sp.detailsMu.Lock() + sp.DetailsMu.Lock() s = detail.status(now.Add(timeAfterStoreSuspect).Add(time.Millisecond), - timeUntilStoreDead, sp.nodeLivenessFn, timeAfterStoreSuspect) - sp.detailsMu.Unlock() + timeUntilStoreDead, sp.NodeLivenessFn, timeAfterStoreSuspect) + sp.DetailsMu.Unlock() require.Equal(t, s, storeStatusAvailable) - mnl.setNodeStatus(store.Node.NodeID, livenesspb.NodeLivenessStatus_DRAINING) - sp.detailsMu.Lock() + mnl.SetNodeStatus(store.Node.NodeID, livenesspb.NodeLivenessStatus_DRAINING) + sp.DetailsMu.Lock() s = detail.status(now, - timeUntilStoreDead, sp.nodeLivenessFn, timeAfterStoreSuspect) - sp.detailsMu.Unlock() + timeUntilStoreDead, sp.NodeLivenessFn, timeAfterStoreSuspect) + sp.DetailsMu.Unlock() require.Equal(t, s, storeStatusDraining) - require.True(t, detail.lastAvailable.IsZero()) + require.True(t, detail.LastAvailable.IsZero()) - mnl.setNodeStatus(store.Node.NodeID, livenesspb.NodeLivenessStatus_LIVE) - sp.detailsMu.Lock() + mnl.SetNodeStatus(store.Node.NodeID, livenesspb.NodeLivenessStatus_LIVE) + sp.DetailsMu.Lock() s = detail.status(now.Add(timeAfterStoreSuspect).Add(time.Millisecond), - timeUntilStoreDead, sp.nodeLivenessFn, timeAfterStoreSuspect) - sp.detailsMu.Unlock() + timeUntilStoreDead, sp.NodeLivenessFn, timeAfterStoreSuspect) + sp.DetailsMu.Unlock() require.Equal(t, s, storeStatusAvailable) - require.False(t, detail.lastAvailable.IsZero()) + require.False(t, detail.LastAvailable.IsZero()) } func TestGetLocalities(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) ctx := context.Background() - stopper, g, _, sp, _ := createTestStorePool(ctx, + stopper, g, _, sp, _ := CreateTestStorePool(ctx, TestTimeUntilStoreDead, false, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_DEAD) @@ -990,8 +723,8 @@ func TestGetLocalities(t *testing.T) { ) } - localitiesByStore := sp.getLocalitiesByStore(existingReplicas) - localitiesByNode := sp.getLocalitiesByNode(existingReplicas) + localitiesByStore := sp.GetLocalitiesByStore(existingReplicas) + localitiesByNode := sp.GetLocalitiesByNode(existingReplicas) for _, store := range stores { storeID := store.StoreID nodeID := store.Node.NodeID @@ -1005,7 +738,7 @@ func TestGetLocalities(t *testing.T) { require.Equal(t, localityByStore.Tiers[len(localityByStore.Tiers)-1], roachpb.Tier{Key: "node", Value: nodeID.String()}) require.Equal(t, int(nodeID)-1, len(localityByNode.Tiers)) - require.Equal(t, createLocality(int(nodeID)-1).String(), sp.getNodeLocalityString(nodeID)) + require.Equal(t, createLocality(int(nodeID)-1).String(), sp.GetNodeLocalityString(nodeID)) } } @@ -1013,7 +746,7 @@ func TestStorePoolDecommissioningReplicas(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) ctx := context.Background() - stopper, g, _, sp, mnl := createTestStorePool(ctx, + stopper, g, _, sp, mnl := CreateTestStorePool(ctx, TestTimeUntilStoreDead, false, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_DEAD) @@ -1073,10 +806,10 @@ func TestStorePoolDecommissioningReplicas(t *testing.T) { sg.GossipStores(stores, t) for i := 1; i <= 5; i++ { - mnl.setNodeStatus(roachpb.NodeID(i), livenesspb.NodeLivenessStatus_LIVE) + mnl.SetNodeStatus(roachpb.NodeID(i), livenesspb.NodeLivenessStatus_LIVE) } - liveReplicas, deadReplicas := sp.liveAndDeadReplicas(replicas, false /* includeSuspectAndDrainingStores */) + liveReplicas, deadReplicas := sp.LiveAndDeadReplicas(replicas, false /* includeSuspectAndDrainingStores */) if len(liveReplicas) != 5 { t.Fatalf("expected five live replicas, found %d (%v)", len(liveReplicas), liveReplicas) } @@ -1084,11 +817,11 @@ func TestStorePoolDecommissioningReplicas(t *testing.T) { t.Fatalf("expected no dead replicas initially, found %d (%v)", len(deadReplicas), deadReplicas) } // Mark node 4 as decommissioning. - mnl.setNodeStatus(4, livenesspb.NodeLivenessStatus_DECOMMISSIONING) + mnl.SetNodeStatus(4, livenesspb.NodeLivenessStatus_DECOMMISSIONING) // Mark node 5 as dead. - mnl.setNodeStatus(5, livenesspb.NodeLivenessStatus_DEAD) + mnl.SetNodeStatus(5, livenesspb.NodeLivenessStatus_DEAD) - liveReplicas, deadReplicas = sp.liveAndDeadReplicas(replicas, false /* includeSuspectAndDrainingStores */) + liveReplicas, deadReplicas = sp.LiveAndDeadReplicas(replicas, false /* includeSuspectAndDrainingStores */) // Decommissioning replicas are considered live. if a, e := liveReplicas, replicas[:4]; !reflect.DeepEqual(a, e) { t.Fatalf("expected live replicas %+v; got %+v", e, a) @@ -1097,7 +830,7 @@ func TestStorePoolDecommissioningReplicas(t *testing.T) { t.Fatalf("expected dead replicas %+v; got %+v", e, a) } - decommissioningReplicas := sp.decommissioningReplicas(replicas) + decommissioningReplicas := sp.DecommissioningReplicas(replicas) if a, e := decommissioningReplicas, replicas[3:4]; !reflect.DeepEqual(a, e) { t.Fatalf("expected decommissioning replicas %+v; got %+v", e, a) } diff --git a/pkg/kv/kvserver/allocator/storepool/test_helpers.go b/pkg/kv/kvserver/allocator/storepool/test_helpers.go new file mode 100644 index 000000000000..6d5555b484d5 --- /dev/null +++ b/pkg/kv/kvserver/allocator/storepool/test_helpers.go @@ -0,0 +1,106 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package storepool + +import ( + "context" + "time" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/gossip" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/rpc" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/metric" + "github.com/cockroachdb/cockroach/pkg/util/stop" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" +) + +// MockNodeLiveness is a testing construct to mock what node liveness status a +// store pool observes for a given node. +type MockNodeLiveness struct { + syncutil.Mutex + defaultNodeStatus livenesspb.NodeLivenessStatus + nodes map[roachpb.NodeID]livenesspb.NodeLivenessStatus +} + +// NewMockNodeLiveness constructs a MockNodeLiveness, for testing purposes. +func NewMockNodeLiveness(defaultNodeStatus livenesspb.NodeLivenessStatus) *MockNodeLiveness { + return &MockNodeLiveness{ + defaultNodeStatus: defaultNodeStatus, + nodes: map[roachpb.NodeID]livenesspb.NodeLivenessStatus{}, + } +} + +// SetNodeStatus sets the node liveness status for the given node ID. +func (m *MockNodeLiveness) SetNodeStatus( + nodeID roachpb.NodeID, status livenesspb.NodeLivenessStatus, +) { + m.Lock() + defer m.Unlock() + m.nodes[nodeID] = status +} + +// NodeLivenessFunc is the method that can be injected as part of store pool +// construction to mock out node liveness, in tests. +func (m *MockNodeLiveness) NodeLivenessFunc( + nodeID roachpb.NodeID, now time.Time, threshold time.Duration, +) livenesspb.NodeLivenessStatus { + m.Lock() + defer m.Unlock() + if status, ok := m.nodes[nodeID]; ok { + return status + } + return m.defaultNodeStatus +} + +// CreateTestStorePool creates a stopper, gossip and storePool for use in +// tests. Stopper must be stopped by the caller. +func CreateTestStorePool( + ctx context.Context, + timeUntilStoreDeadValue time.Duration, + deterministic bool, + nodeCount NodeCountFunc, + defaultNodeStatus livenesspb.NodeLivenessStatus, +) (*stop.Stopper, *gossip.Gossip, *hlc.ManualClock, *StorePool, *MockNodeLiveness) { + stopper := stop.NewStopper() + mc := hlc.NewManualClock(123) + clock := hlc.NewClock(mc.UnixNano, time.Nanosecond) + st := cluster.MakeTestingClusterSettings() + ambientCtx := log.MakeTestingAmbientContext(stopper.Tracer()) + rpcContext := rpc.NewContext(ctx, + rpc.ContextOptions{ + TenantID: roachpb.SystemTenantID, + Config: &base.Config{Insecure: true}, + Clock: clock, + Stopper: stopper, + Settings: st, + }) + server := rpc.NewServer(rpcContext) // never started + g := gossip.NewTest(1, rpcContext, server, stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) + mnl := NewMockNodeLiveness(defaultNodeStatus) + + TimeUntilStoreDead.Override(ctx, &st.SV, timeUntilStoreDeadValue) + storePool := NewStorePool( + ambientCtx, + st, + g, + clock, + nodeCount, + mnl.NodeLivenessFunc, + deterministic, + ) + return stopper, g, mc, storePool, mnl +} diff --git a/pkg/kv/kvserver/allocator_scorer.go b/pkg/kv/kvserver/allocator_scorer.go index 623735ecc9c3..d5256218844d 100644 --- a/pkg/kv/kvserver/allocator_scorer.go +++ b/pkg/kv/kvserver/allocator_scorer.go @@ -18,6 +18,8 @@ import ( "sort" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/constraint" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -46,16 +48,11 @@ const ( // https://www.eecs.harvard.edu/~michaelm/postscripts/mythesis.pdf. allocatorRandomCount = 2 - // maxFractionUsedThreshold: if the fraction used of a store descriptor - // capacity is greater than this value, it will never be used as a rebalance - // or allocate target and we will actively try to move replicas off of it. - maxFractionUsedThreshold = 0.95 - // rebalanceToMaxFractionUsedThreshold: if the fraction used of a store // descriptor capacity is greater than this value, it will never be used as a // rebalance target. This is important for providing a buffer between fully // healthy stores and full stores (as determined by - // maxFractionUsedThreshold). Without such a buffer, replicas could + // allocator.MaxFractionUsedThreshold). Without such a buffer, replicas could // hypothetically ping pong back and forth between two nodes, making one full // and then the other. rebalanceToMaxFractionUsedThreshold = 0.925 @@ -190,7 +187,7 @@ type scorerOptions interface { // // This is to ensure that, when scattering via `AdminScatterRequest`, we will // be more likely to find a rebalance opportunity. - maybeJitterStoreStats(sl StoreList, allocRand allocatorRand) StoreList + maybeJitterStoreStats(sl storepool.StoreList, allocRand allocatorRand) storepool.StoreList // deterministic is set by tests to have the allocator methods sort their // results by constraints score as well as by store IDs, as opposed to just // the score. @@ -212,7 +209,7 @@ type scorerOptions interface { // balanceScore returns a discrete score (`balanceStatus`) based on whether // the store represented by `sc` classifies as underfull, aroundTheMean, or // overfull relative to all the stores in `sl`. - balanceScore(sl StoreList, sc roachpb.StoreCapacity) balanceStatus + balanceScore(sl storepool.StoreList, sc roachpb.StoreCapacity) balanceStatus // rebalanceFromConvergenceScore assigns a convergence score to the store // referred to by `eqClass.existing` based on whether moving a replica away // from this store would converge its stats towards the mean (relative to the @@ -233,7 +230,7 @@ type scorerOptions interface { // score to the existing store (or multiple replicas, if there are multiple // with the same QPS) that would converge the range's existing stores' QPS the // most. - removalMaximallyConvergesScore(removalCandStoreList StoreList, existing roachpb.StoreDescriptor) int + removalMaximallyConvergesScore(removalCandStoreList storepool.StoreList, existing roachpb.StoreDescriptor) int // getStoreHealthOptions returns the scorer options for store health. It is // used to inform scoring based on the health of a store. getStoreHealthOptions() storeHealthOptions @@ -260,17 +257,17 @@ type scatterScorerOptions struct { } func (o *scatterScorerOptions) maybeJitterStoreStats( - sl StoreList, allocRand allocatorRand, -) (perturbedSL StoreList) { - perturbedStoreDescs := make([]roachpb.StoreDescriptor, 0, len(sl.stores)) - for _, store := range sl.stores { + sl storepool.StoreList, allocRand allocatorRand, +) (perturbedSL storepool.StoreList) { + perturbedStoreDescs := make([]roachpb.StoreDescriptor, 0, len(sl.Stores)) + for _, store := range sl.Stores { store.Capacity.RangeCount += int32(jittered( float64(store.Capacity.RangeCount), o.jitter, allocRand, )) perturbedStoreDescs = append(perturbedStoreDescs, store) } - return makeStoreList(perturbedStoreDescs) + return storepool.MakeStoreList(perturbedStoreDescs) } // rangeCountScorerOptions is used by the replicateQueue to tell the Allocator's @@ -284,8 +281,8 @@ type rangeCountScorerOptions struct { } func (o *rangeCountScorerOptions) maybeJitterStoreStats( - sl StoreList, _ allocatorRand, -) (perturbedSL StoreList) { + sl storepool.StoreList, _ allocatorRand, +) (perturbedSL storepool.StoreList) { return sl } @@ -298,31 +295,31 @@ func (o rangeCountScorerOptions) shouldRebalanceBasedOnThresholds( ) bool { store := eqClass.existing sl := eqClass.candidateSL - if len(sl.stores) == 0 { + if len(sl.Stores) == 0 { return false } - overfullThreshold := int32(math.Ceil(overfullRangeThreshold(&o, sl.candidateRanges.mean))) + overfullThreshold := int32(math.Ceil(overfullRangeThreshold(&o, sl.CandidateRanges.Mean))) // 1. We rebalance if `store` is too far above the mean (i.e. stores // that are overfull). if store.Capacity.RangeCount > overfullThreshold { log.VEventf(ctx, 2, "s%d: should-rebalance(ranges-overfull): rangeCount=%d, mean=%.2f, overfull-threshold=%d", - store.StoreID, store.Capacity.RangeCount, sl.candidateRanges.mean, overfullThreshold) + store.StoreID, store.Capacity.RangeCount, sl.CandidateRanges.Mean, overfullThreshold) return true } // 2. We rebalance if `store` isn't overfull, but it is above the mean and // there is at least one other store that is "underfull" (i.e. too far below // the mean). - if float64(store.Capacity.RangeCount) > sl.candidateRanges.mean { - underfullThreshold := int32(math.Floor(underfullRangeThreshold(&o, sl.candidateRanges.mean))) - for _, desc := range sl.stores { + if float64(store.Capacity.RangeCount) > sl.CandidateRanges.Mean { + underfullThreshold := int32(math.Floor(underfullRangeThreshold(&o, sl.CandidateRanges.Mean))) + for _, desc := range sl.Stores { if desc.Capacity.RangeCount < underfullThreshold { log.VEventf(ctx, 2, "s%d: should-rebalance(better-fit-ranges=s%d): rangeCount=%d, otherRangeCount=%d, "+ "mean=%.2f, underfull-threshold=%d", store.StoreID, desc.StoreID, store.Capacity.RangeCount, desc.Capacity.RangeCount, - sl.candidateRanges.mean, underfullThreshold) + sl.CandidateRanges.Mean, underfullThreshold) return true } } @@ -332,10 +329,10 @@ func (o rangeCountScorerOptions) shouldRebalanceBasedOnThresholds( } func (o *rangeCountScorerOptions) balanceScore( - sl StoreList, sc roachpb.StoreCapacity, + sl storepool.StoreList, sc roachpb.StoreCapacity, ) balanceStatus { - maxRangeCount := overfullRangeThreshold(o, sl.candidateRanges.mean) - minRangeCount := underfullRangeThreshold(o, sl.candidateRanges.mean) + maxRangeCount := overfullRangeThreshold(o, sl.CandidateRanges.Mean) + minRangeCount := underfullRangeThreshold(o, sl.CandidateRanges.Mean) curRangeCount := float64(sc.RangeCount) if curRangeCount < minRangeCount { return underfull @@ -378,7 +375,7 @@ func (o *rangeCountScorerOptions) rebalanceToConvergesScore( // Otherwise, a high convergesScore is assigned (which would make this store // less likely to be picked for removal). func (o *rangeCountScorerOptions) removalMaximallyConvergesScore( - removalCandStoreList StoreList, existing roachpb.StoreDescriptor, + removalCandStoreList storepool.StoreList, existing roachpb.StoreDescriptor, ) int { if !rebalanceConvergesRangeCountOnMean( removalCandStoreList, existing.Capacity, existing.Capacity.RangeCount-1, @@ -414,7 +411,9 @@ type qpsScorerOptions struct { qpsPerReplica float64 } -func (o *qpsScorerOptions) maybeJitterStoreStats(sl StoreList, _ allocatorRand) StoreList { +func (o *qpsScorerOptions) maybeJitterStoreStats( + sl storepool.StoreList, _ allocatorRand, +) storepool.StoreList { return sl } @@ -429,7 +428,7 @@ func (o *qpsScorerOptions) deterministicForTesting() bool { func (o qpsScorerOptions) shouldRebalanceBasedOnThresholds( ctx context.Context, eqClass equivalenceClass, metrics AllocatorMetrics, ) bool { - if len(eqClass.candidateSL.stores) == 0 { + if len(eqClass.candidateSL.Stores) == 0 { return false } @@ -463,7 +462,7 @@ func (o qpsScorerOptions) shouldRebalanceBasedOnThresholds( case shouldRebalance: metrics.loadBasedReplicaRebalanceMetrics.ShouldRebalance.Inc(1) var bestStoreQPS float64 - for _, store := range eqClass.candidateSL.stores { + for _, store := range eqClass.candidateSL.Stores { if bestStore == store.StoreID { bestStoreQPS = store.Capacity.QueriesPerSecond } @@ -482,9 +481,11 @@ func (o qpsScorerOptions) shouldRebalanceBasedOnThresholds( return declineReason == shouldRebalance } -func (o *qpsScorerOptions) balanceScore(sl StoreList, sc roachpb.StoreCapacity) balanceStatus { - maxQPS := overfullQPSThreshold(o, sl.candidateQueriesPerSecond.mean) - minQPS := underfullQPSThreshold(o, sl.candidateQueriesPerSecond.mean) +func (o *qpsScorerOptions) balanceScore( + sl storepool.StoreList, sc roachpb.StoreCapacity, +) balanceStatus { + maxQPS := overfullQPSThreshold(o, sl.CandidateQueriesPerSecond.Mean) + minQPS := underfullQPSThreshold(o, sl.CandidateQueriesPerSecond.Mean) curQPS := sc.QueriesPerSecond if curQPS < minQPS { return underfull @@ -525,10 +526,10 @@ func (o *qpsScorerOptions) rebalanceToConvergesScore( // hottest store (based on QPS) among the stores inside // `removalCandidateStores`. func (o *qpsScorerOptions) removalMaximallyConvergesScore( - removalCandStoreList StoreList, existing roachpb.StoreDescriptor, + removalCandStoreList storepool.StoreList, existing roachpb.StoreDescriptor, ) int { maxQPS := float64(-1) - for _, store := range removalCandStoreList.stores { + for _, store := range removalCandStoreList.Stores { if store.Capacity.QueriesPerSecond > maxQPS { maxQPS = store.Capacity.QueriesPerSecond } @@ -870,7 +871,7 @@ func (cl candidateList) removeCandidate(c candidate) candidateList { // by callers performing lateral relocation of replicas within the same node. func rankedCandidateListForAllocation( ctx context.Context, - candidateStores StoreList, + candidateStores storepool.StoreList, constraintsCheck constraintsCheckFn, existingReplicas []roachpb.ReplicaDescriptor, existingStoreLocalities map[roachpb.StoreID]roachpb.Locality, @@ -880,7 +881,7 @@ func rankedCandidateListForAllocation( ) candidateList { var candidates candidateList existingReplTargets := roachpb.MakeReplicaSet(existingReplicas).ReplicationTargets() - for _, s := range candidateStores.stores { + for _, s := range candidateStores.Stores { // Disregard all the stores that already have replicas. if storeHasReplica(s.StoreID, existingReplTargets) { continue @@ -905,10 +906,10 @@ func rankedCandidateListForAllocation( continue } - if !maxCapacityCheck(s) || !options.storeHealthOptions.readAmpIsHealthy( + if !allocator.MaxCapacityCheck(s) || !options.storeHealthOptions.readAmpIsHealthy( ctx, s, - candidateStores.candidateL0Sublevels.mean, + candidateStores.CandidateL0Sublevels.Mean, ) { continue } @@ -944,13 +945,13 @@ func rankedCandidateListForAllocation( // Stores that are marked as not valid, are in violation of a required criteria. func candidateListForRemoval( ctx context.Context, - existingReplsStoreList StoreList, + existingReplsStoreList storepool.StoreList, constraintsCheck constraintsCheckFn, existingStoreLocalities map[roachpb.StoreID]roachpb.Locality, options scorerOptions, ) candidateList { var candidates candidateList - for _, s := range existingReplsStoreList.stores { + for _, s := range existingReplsStoreList.Stores { constraintsOK, necessary := constraintsCheck(s) if !constraintsOK { candidates = append(candidates, candidate{ @@ -966,7 +967,7 @@ func candidateListForRemoval( store: s, valid: constraintsOK, necessary: necessary, - fullDisk: !maxCapacityCheck(s), + fullDisk: !allocator.MaxCapacityCheck(s), // When removing a replica from a store, we do not want to include // high amplification in ranking stores. This would submit already // high read amplification stores to additional load of moving a @@ -1015,7 +1016,7 @@ func candidateListForRemoval( for _, cand := range candidates { removalCandidateStores = append(removalCandidateStores, cand.store) } - removalCandidateStoreList := makeStoreList(removalCandidateStores) + removalCandidateStoreList := storepool.MakeStoreList(removalCandidateStores) for i := range candidates { // If removing this candidate replica does not converge the store // stats to their mean, we make it less attractive for removal by @@ -1069,7 +1070,7 @@ type equivalenceClass struct { // `candidateSl` is the `StoreList` representation of `candidates` (maintained // separately to avoid converting the latter into the former for all the // `scorerOptions` methods). - candidateSL StoreList + candidateSL storepool.StoreList candidates candidateList } @@ -1146,7 +1147,7 @@ func bestStoreToMinimizeQPSDelta( for _, desc := range storeDescMap { storeDescs = append(storeDescs, *desc) } - domainStoreList := makeStoreList(storeDescs) + domainStoreList := storepool.MakeStoreList(storeDescs) bestCandidate = getCandidateWithMinQPS(storeQPSMap, candidates) if bestCandidate == 0 { @@ -1175,7 +1176,7 @@ func bestStoreToMinimizeQPSDelta( // Only proceed with rebalancing iff `existingStore` is overfull relative to // the equivalence class. - mean := domainStoreList.candidateQueriesPerSecond.mean + mean := domainStoreList.CandidateQueriesPerSecond.Mean overfullThreshold := overfullQPSThreshold( options, mean, @@ -1228,9 +1229,9 @@ func bestStoreToMinimizeQPSDelta( func (o *qpsScorerOptions) getRebalanceTargetToMinimizeDelta( eqClass equivalenceClass, ) (bestStore roachpb.StoreID, declineReason declineReason) { - domainStoreList := makeStoreList(append(eqClass.candidateSL.stores, eqClass.existing)) - candidates := make([]roachpb.StoreID, 0, len(eqClass.candidateSL.stores)) - for _, store := range eqClass.candidateSL.stores { + domainStoreList := storepool.MakeStoreList(append(eqClass.candidateSL.Stores, eqClass.existing)) + candidates := make([]roachpb.StoreID, 0, len(eqClass.candidateSL.Stores)) + for _, store := range eqClass.candidateSL.Stores { candidates = append(candidates, store.StoreID) } return bestStoreToMinimizeQPSDelta( @@ -1248,7 +1249,7 @@ func (o *qpsScorerOptions) getRebalanceTargetToMinimizeDelta( // details. func rankedCandidateListForRebalancing( ctx context.Context, - allStores StoreList, + allStores storepool.StoreList, removalConstraintsChecker constraintsCheckFn, rebalanceConstraintsChecker rebalanceConstraintsCheckFn, existingReplicasForType, replicasOnExemptedStores []roachpb.ReplicaDescriptor, @@ -1261,13 +1262,13 @@ func rankedCandidateListForRebalancing( existingStores := make(map[roachpb.StoreID]candidate) var needRebalanceFrom bool curDiversityScore := rangeDiversityScore(existingStoreLocalities) - for _, store := range allStores.stores { + for _, store := range allStores.Stores { for _, repl := range existingReplicasForType { if store.StoreID != repl.StoreID { continue } valid, necessary := removalConstraintsChecker(store) - fullDisk := !maxCapacityCheck(store) + fullDisk := !allocator.MaxCapacityCheck(store) if !valid { if !needRebalanceFrom { @@ -1321,7 +1322,7 @@ func rankedCandidateListForRebalancing( var needRebalanceTo bool for _, existing := range existingStores { var comparableCands candidateList - for _, store := range allStores.stores { + for _, store := range allStores.Stores { // Only process replacement candidates, not existing stores. if store.StoreID == existing.store.StoreID { continue @@ -1374,7 +1375,7 @@ func rankedCandidateListForRebalancing( store: store, valid: constraintsOK, necessary: necessary, - fullDisk: !maxCapacityCheck(store), + fullDisk: !allocator.MaxCapacityCheck(store), diversityScore: diversityScore, } if !cand.less(existing) { @@ -1408,7 +1409,7 @@ func rankedCandidateListForRebalancing( } eqClass := equivalenceClass{ existing: existing.store, - candidateSL: makeStoreList(bestStores), + candidateSL: storepool.MakeStoreList(bestStores), candidates: bestCands, } equivalenceClasses = append(equivalenceClasses, eqClass) @@ -1479,7 +1480,7 @@ func rankedCandidateListForRebalancing( s, // We only wish to compare the read amplification to the // comparable stores average and not the cluster. - comparable.candidateSL.candidateL0Sublevels.mean, + comparable.candidateSL.CandidateL0Sublevels.Mean, ) cand.balanceScore = options.balanceScore(comparable.candidateSL, s.Capacity) cand.convergesScore = options.rebalanceToConvergesScore(comparable, s) @@ -1826,25 +1827,6 @@ func containsStore(stores []roachpb.StoreID, target roachpb.StoreID) bool { return false } -// isStoreValid returns true iff the provided store would be a valid in a -// range with the provided constraints. -func isStoreValid( - store roachpb.StoreDescriptor, constraints []roachpb.ConstraintsConjunction, -) bool { - if len(constraints) == 0 { - return true - } - - for _, subConstraints := range constraints { - if constraintsOK := constraint.ConjunctionsCheck( - store, subConstraints.Constraints, - ); constraintsOK { - return true - } - } - return false -} - // rangeDiversityScore returns a value between 0 and 1 based on how diverse the // given range is. A higher score means the range is more diverse. // All below diversity-scoring methods should in theory be implemented by @@ -2006,9 +1988,9 @@ func underfullQPSThreshold(options *qpsScorerOptions, mean float64) float64 { } func rebalanceConvergesRangeCountOnMean( - sl StoreList, sc roachpb.StoreCapacity, newRangeCount int32, + sl storepool.StoreList, sc roachpb.StoreCapacity, newRangeCount int32, ) bool { - return convergesOnMean(float64(sc.RangeCount), float64(newRangeCount), sl.candidateRanges.mean) + return convergesOnMean(float64(sc.RangeCount), float64(newRangeCount), sl.CandidateRanges.Mean) } func convergesOnMean(oldVal, newVal, mean float64) bool { @@ -2082,11 +2064,6 @@ func (o storeHealthOptions) rebalanceToReadAmpIsHealthy( return o.enforcementLevel < storeHealthBlockRebalanceTo } -// maxCapacityCheck returns true if the store has room for a new replica. -func maxCapacityCheck(store roachpb.StoreDescriptor) bool { - return store.Capacity.FractionUsed() < maxFractionUsedThreshold -} - // rebalanceToMaxCapacityCheck returns true if the store has enough room to // accept a rebalance. The bar for this is stricter than for whether a store // has enough room to accept a necessary replica (i.e. via AllocateCandidates). diff --git a/pkg/kv/kvserver/allocator_scorer_test.go b/pkg/kv/kvserver/allocator_scorer_test.go index 17817259b0c5..76bb38aa6014 100644 --- a/pkg/kv/kvserver/allocator_scorer_test.go +++ b/pkg/kv/kvserver/allocator_scorer_test.go @@ -20,6 +20,8 @@ import ( "sort" "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/constraint" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -675,7 +677,7 @@ func TestConstraintsCheck(t *testing.T) { for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { for _, s := range testStores { - valid := isStoreValid(s, tc.constraints) + valid := allocator.IsStoreValid(s, tc.constraints) ok := tc.expected[s.StoreID] if valid != ok { t.Errorf("expected store %d to be %t, but got %t", s.StoreID, ok, valid) @@ -1071,8 +1073,8 @@ func TestShouldRebalanceDiversity(t *testing.T) { }, } } - localityForStoreID := func(sl StoreList, id roachpb.StoreID) roachpb.Locality { - for _, store := range sl.stores { + localityForStoreID := func(sl storepool.StoreList, id roachpb.StoreID) roachpb.Locality { + for _, store := range sl.Stores { if store.StoreID == id { return store.Locality() } @@ -1092,15 +1094,15 @@ func TestShouldRebalanceDiversity(t *testing.T) { locAU := roachpb.Locality{ Tiers: testStoreTierSetup("au", "", "", ""), } - sl3by3 := StoreList{ - stores: []roachpb.StoreDescriptor{ + sl3by3 := storepool.StoreList{ + Stores: []roachpb.StoreDescriptor{ newStore(1, locUS), newStore(2, locUS), newStore(3, locUS), newStore(4, locEU), newStore(5, locEU), newStore(6, locEU), newStore(7, locAS), newStore(8, locAS), newStore(9, locAS), }, } - sl4by3 := StoreList{ - stores: []roachpb.StoreDescriptor{ + sl4by3 := storepool.StoreList{ + Stores: []roachpb.StoreDescriptor{ newStore(1, locUS), newStore(2, locUS), newStore(3, locUS), newStore(4, locEU), newStore(5, locEU), newStore(6, locEU), newStore(7, locAS), newStore(8, locAS), newStore(9, locAS), @@ -1110,7 +1112,7 @@ func TestShouldRebalanceDiversity(t *testing.T) { testCases := []struct { s roachpb.StoreDescriptor - sl StoreList + sl storepool.StoreList existingNodeIDs []roachpb.NodeID expected bool }{ @@ -1170,16 +1172,16 @@ func TestShouldRebalanceDiversity(t *testing.T) { }, } for i, tc := range testCases { - removeStore := func(sl StoreList, nodeID roachpb.NodeID) StoreList { - for i, s := range sl.stores { + removeStore := func(sl storepool.StoreList, nodeID roachpb.NodeID) storepool.StoreList { + for i, s := range sl.Stores { if s.Node.NodeID == nodeID { - return makeStoreList(append(sl.stores[:i], sl.stores[i+1:]...)) + return storepool.MakeStoreList(append(sl.Stores[:i], sl.Stores[i+1:]...)) } } return sl } filteredSL := tc.sl - filteredSL.stores = append([]roachpb.StoreDescriptor(nil), filteredSL.stores...) + filteredSL.Stores = append([]roachpb.StoreDescriptor(nil), filteredSL.Stores...) existingStoreLocalities := make(map[roachpb.StoreID]roachpb.Locality) var replicas []roachpb.ReplicaDescriptor for _, nodeID := range tc.existingNodeIDs { @@ -1495,8 +1497,8 @@ func TestBalanceScoreByRangeCount(t *testing.T) { options := rangeCountScorerOptions{ rangeRebalanceThreshold: 0.1, } - storeList := StoreList{ - candidateRanges: stat{mean: 1000}, + storeList := storepool.StoreList{ + CandidateRanges: storepool.Stat{Mean: 1000}, } testCases := []struct { @@ -1524,8 +1526,8 @@ func TestRebalanceBalanceScoreOnQPS(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - storeList := StoreList{ - candidateQueriesPerSecond: stat{mean: 1000}, + storeList := storepool.StoreList{ + CandidateQueriesPerSecond: storepool.Stat{Mean: 1000}, } options := qpsScorerOptions{ qpsRebalanceThreshold: 0.1, @@ -1562,8 +1564,8 @@ func TestRebalanceConvergesRangeCountOnMean(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - storeList := StoreList{ - candidateRanges: stat{mean: 1000}, + storeList := storepool.StoreList{ + CandidateRanges: storepool.Stat{Mean: 1000}, } testCases := []struct { @@ -1614,7 +1616,7 @@ func TestMaxCapacity(t *testing.T) { } for _, s := range testStores { - if e, a := expectedCheck[s.StoreID], maxCapacityCheck(s); e != a { + if e, a := expectedCheck[s.StoreID], allocator.MaxCapacityCheck(s); e != a { t.Errorf("store %d expected max capacity check: %t, actual %t", s.StoreID, e, a) } } diff --git a/pkg/kv/kvserver/allocator_test.go b/pkg/kv/kvserver/allocator_test.go index 395c97328c1d..47f1c040016d 100644 --- a/pkg/kv/kvserver/allocator_test.go +++ b/pkg/kv/kvserver/allocator_test.go @@ -26,6 +26,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/constraint" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -461,7 +463,7 @@ func replicas(storeIDs ...roachpb.StoreID) []roachpb.ReplicaDescriptor { // use in tests. Stopper must be stopped by the caller. func createTestAllocator( ctx context.Context, numNodes int, deterministic bool, -) (*stop.Stopper, *gossip.Gossip, *StorePool, Allocator, *hlc.ManualClock) { +) (*stop.Stopper, *gossip.Gossip, *storepool.StorePool, Allocator, *hlc.ManualClock) { return createTestAllocatorWithKnobs(ctx, numNodes, deterministic, nil /* knobs */) } @@ -470,9 +472,9 @@ func createTestAllocator( // the caller. func createTestAllocatorWithKnobs( ctx context.Context, numNodes int, deterministic bool, knobs *AllocatorTestingKnobs, -) (*stop.Stopper, *gossip.Gossip, *StorePool, Allocator, *hlc.ManualClock) { - stopper, g, manual, storePool, _ := createTestStorePool(ctx, - TestTimeUntilStoreDeadOff, deterministic, +) (*stop.Stopper, *gossip.Gossip, *storepool.StorePool, Allocator, *hlc.ManualClock) { + stopper, g, manual, storePool, _ := storepool.CreateTestStorePool(ctx, + storepool.TestTimeUntilStoreDeadOff, deterministic, func() int { return numNodes }, livenesspb.NodeLivenessStatus_LIVE) a := MakeAllocator( @@ -498,7 +500,7 @@ func checkReplExists(repl roachpb.ReplicationTarget, stores []roachpb.StoreID) ( // mockStorePool sets up a collection of a alive and dead stores in the store // pool for testing purposes. func mockStorePool( - storePool *StorePool, + storePool *storepool.StorePool, aliveStoreIDs []roachpb.StoreID, unavailableStoreIDs []roachpb.StoreID, deadStoreIDs []roachpb.StoreID, @@ -506,47 +508,47 @@ func mockStorePool( decommissionedStoreIDs []roachpb.StoreID, suspectedStoreIDs []roachpb.StoreID, ) { - storePool.detailsMu.Lock() - defer storePool.detailsMu.Unlock() + storePool.DetailsMu.Lock() + defer storePool.DetailsMu.Unlock() liveNodeSet := map[roachpb.NodeID]livenesspb.NodeLivenessStatus{} - storePool.detailsMu.storeDetails = map[roachpb.StoreID]*storeDetail{} + storePool.DetailsMu.StoreDetails = map[roachpb.StoreID]*storepool.StoreDetail{} for _, storeID := range aliveStoreIDs { liveNodeSet[roachpb.NodeID(storeID)] = livenesspb.NodeLivenessStatus_LIVE - detail := storePool.getStoreDetailLocked(storeID) - detail.desc = &roachpb.StoreDescriptor{ + detail := storePool.GetStoreDetailLocked(storeID) + detail.Desc = &roachpb.StoreDescriptor{ StoreID: storeID, Node: roachpb.NodeDescriptor{NodeID: roachpb.NodeID(storeID)}, } } for _, storeID := range unavailableStoreIDs { liveNodeSet[roachpb.NodeID(storeID)] = livenesspb.NodeLivenessStatus_UNAVAILABLE - detail := storePool.getStoreDetailLocked(storeID) - detail.desc = &roachpb.StoreDescriptor{ + detail := storePool.GetStoreDetailLocked(storeID) + detail.Desc = &roachpb.StoreDescriptor{ StoreID: storeID, Node: roachpb.NodeDescriptor{NodeID: roachpb.NodeID(storeID)}, } } for _, storeID := range deadStoreIDs { liveNodeSet[roachpb.NodeID(storeID)] = livenesspb.NodeLivenessStatus_DEAD - detail := storePool.getStoreDetailLocked(storeID) - detail.desc = &roachpb.StoreDescriptor{ + detail := storePool.GetStoreDetailLocked(storeID) + detail.Desc = &roachpb.StoreDescriptor{ StoreID: storeID, Node: roachpb.NodeDescriptor{NodeID: roachpb.NodeID(storeID)}, } } for _, storeID := range decommissioningStoreIDs { liveNodeSet[roachpb.NodeID(storeID)] = livenesspb.NodeLivenessStatus_DECOMMISSIONING - detail := storePool.getStoreDetailLocked(storeID) - detail.desc = &roachpb.StoreDescriptor{ + detail := storePool.GetStoreDetailLocked(storeID) + detail.Desc = &roachpb.StoreDescriptor{ StoreID: storeID, Node: roachpb.NodeDescriptor{NodeID: roachpb.NodeID(storeID)}, } } for _, storeID := range decommissionedStoreIDs { liveNodeSet[roachpb.NodeID(storeID)] = livenesspb.NodeLivenessStatus_DECOMMISSIONED - detail := storePool.getStoreDetailLocked(storeID) - detail.desc = &roachpb.StoreDescriptor{ + detail := storePool.GetStoreDetailLocked(storeID) + detail.Desc = &roachpb.StoreDescriptor{ StoreID: storeID, Node: roachpb.NodeDescriptor{NodeID: roachpb.NodeID(storeID)}, } @@ -554,17 +556,17 @@ func mockStorePool( for _, storeID := range suspectedStoreIDs { liveNodeSet[roachpb.NodeID(storeID)] = livenesspb.NodeLivenessStatus_LIVE - detail := storePool.getStoreDetailLocked(storeID) - detail.lastAvailable = storePool.clock.Now().GoTime() - detail.lastUnavailable = storePool.clock.Now().GoTime() - detail.desc = &roachpb.StoreDescriptor{ + detail := storePool.GetStoreDetailLocked(storeID) + detail.LastAvailable = storePool.Clock.Now().GoTime() + detail.LastUnavailable = storePool.Clock.Now().GoTime() + detail.Desc = &roachpb.StoreDescriptor{ StoreID: storeID, Node: roachpb.NodeDescriptor{NodeID: roachpb.NodeID(storeID)}, } } // Set the node liveness function using the set we constructed. - storePool.nodeLivenessFn = + storePool.NodeLivenessFn = func(nodeID roachpb.NodeID, now time.Time, threshold time.Duration) livenesspb.NodeLivenessStatus { if status, ok := liveNodeSet[nodeID]; ok { return status @@ -693,7 +695,7 @@ func TestAllocatorReadAmpCheck(t *testing.T) { sg.GossipStores(test.stores, t) // Enable read disk health checking in candidate exclusion. - l0SublevelsThresholdEnforce.Override(ctx, &a.storePool.st.SV, int64(test.enforcement)) + l0SublevelsThresholdEnforce.Override(ctx, &a.storePool.St.SV, int64(test.enforcement)) add, _, err := a.AllocateVoter( ctx, test.conf, @@ -901,7 +903,7 @@ func TestAllocatorMultipleStoresPerNode(t *testing.T) { } { - var rangeUsageInfo RangeUsageInfo + var rangeUsageInfo allocator.RangeUsageInfo target, _, details, ok := a.RebalanceVoter( ctx, emptySpanConfig(), @@ -909,7 +911,7 @@ func TestAllocatorMultipleStoresPerNode(t *testing.T) { tc.existing, nil, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, a.scorerOptions(ctx), ) if e, a := tc.expectTargetRebalance, ok; e != a { @@ -974,7 +976,7 @@ func TestAllocatorMultipleStoresPerNodeLopsided(t *testing.T) { // We run through all the ranges once to get the cluster to balance. // After that we should not be seeing replicas move. - var rangeUsageInfo RangeUsageInfo + var rangeUsageInfo allocator.RangeUsageInfo for i := 1; i < 40; i++ { add, remove, _, ok := a.RebalanceVoter( ctx, @@ -983,7 +985,7 @@ func TestAllocatorMultipleStoresPerNodeLopsided(t *testing.T) { ranges[i].InternalReplicas, nil, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, a.scorerOptions(ctx), ) if ok { @@ -1024,7 +1026,7 @@ func TestAllocatorMultipleStoresPerNodeLopsided(t *testing.T) { ranges[i].InternalReplicas, nil, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, a.scorerOptions(ctx), ) require.False(t, ok) @@ -1032,7 +1034,7 @@ func TestAllocatorMultipleStoresPerNodeLopsided(t *testing.T) { } // TestAllocatorRebalance verifies that rebalance targets are chosen -// randomly from amongst stores under the maxFractionUsedThreshold. +// randomly from amongst stores under the MaxFractionUsedThreshold. func TestAllocatorRebalanceBasedOnRangeCount(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -1053,7 +1055,7 @@ func TestAllocatorRebalanceBasedOnRangeCount(t *testing.T) { Node: roachpb.NodeDescriptor{NodeID: 3}, Capacity: roachpb.StoreCapacity{ Capacity: 100, - Available: 100 - int64(100*maxFractionUsedThreshold), + Available: 100 - int64(100*allocator.MaxFractionUsedThreshold), RangeCount: 5, }, }, @@ -1063,7 +1065,7 @@ func TestAllocatorRebalanceBasedOnRangeCount(t *testing.T) { Node: roachpb.NodeDescriptor{NodeID: 4}, Capacity: roachpb.StoreCapacity{ Capacity: 100, - Available: (100 - int64(100*maxFractionUsedThreshold)) / 2, + Available: (100 - int64(100*allocator.MaxFractionUsedThreshold)) / 2, RangeCount: 10, }, }, @@ -1088,7 +1090,7 @@ func TestAllocatorRebalanceBasedOnRangeCount(t *testing.T) { // Every rebalance target must be either store 1 or 2. for i := 0; i < 10; i++ { - var rangeUsageInfo RangeUsageInfo + var rangeUsageInfo allocator.RangeUsageInfo target, _, _, ok := a.RebalanceVoter( ctx, emptySpanConfig(), @@ -1096,7 +1098,7 @@ func TestAllocatorRebalanceBasedOnRangeCount(t *testing.T) { []roachpb.ReplicaDescriptor{{NodeID: 3, StoreID: 3}}, nil, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, a.scorerOptions(ctx), ) if !ok { @@ -1110,13 +1112,13 @@ func TestAllocatorRebalanceBasedOnRangeCount(t *testing.T) { } } - sl, _, _ := a.storePool.getStoreList(storeFilterThrottled) + sl, _, _ := a.storePool.GetStoreList(storepool.StoreFilterThrottled) eqClass := equivalenceClass{ candidateSL: sl, } // Verify shouldRebalanceBasedOnThresholds results. for i, store := range stores { - desc, ok := a.storePool.getStoreDescriptor(store.StoreID) + desc, ok := a.storePool.GetStoreDescriptor(store.StoreID) if !ok { t.Fatalf("%d: unable to get store %d descriptor", i, store.StoreID) } @@ -1238,7 +1240,7 @@ func TestAllocatorRebalanceTarget(t *testing.T) { repl.leaseholderStats = newReplicaStats(clock, nil) repl.writeStats = newReplicaStats(clock, nil) - var rangeUsageInfo RangeUsageInfo + var rangeUsageInfo allocator.RangeUsageInfo status := &raft.Status{ Progress: make(map[uint64]tracker.Progress), @@ -1258,7 +1260,7 @@ func TestAllocatorRebalanceTarget(t *testing.T) { replicas, nil, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, a.scorerOptions(ctx), ) if ok { @@ -1283,7 +1285,7 @@ func TestAllocatorRebalanceTarget(t *testing.T) { replicas, nil, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, a.scorerOptions(ctx), ) if ok { @@ -1302,7 +1304,7 @@ func TestAllocatorRebalanceTarget(t *testing.T) { replicas, nil, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, a.scorerOptions(ctx), ) expTo := stores[1].StoreID @@ -1342,16 +1344,16 @@ func TestAllocatorRebalanceDeadNodes(t *testing.T) { } // Initialize 8 stores: where store 6 is the target for rebalancing. - sp.detailsMu.Lock() - sp.getStoreDetailLocked(1).desc.Capacity = ranges(100) - sp.getStoreDetailLocked(2).desc.Capacity = ranges(100) - sp.getStoreDetailLocked(3).desc.Capacity = ranges(100) - sp.getStoreDetailLocked(4).desc.Capacity = ranges(100) - sp.getStoreDetailLocked(5).desc.Capacity = ranges(100) - sp.getStoreDetailLocked(6).desc.Capacity = ranges(0) - sp.getStoreDetailLocked(7).desc.Capacity = ranges(100) - sp.getStoreDetailLocked(8).desc.Capacity = ranges(100) - sp.detailsMu.Unlock() + sp.DetailsMu.Lock() + sp.GetStoreDetailLocked(1).Desc.Capacity = ranges(100) + sp.GetStoreDetailLocked(2).Desc.Capacity = ranges(100) + sp.GetStoreDetailLocked(3).Desc.Capacity = ranges(100) + sp.GetStoreDetailLocked(4).Desc.Capacity = ranges(100) + sp.GetStoreDetailLocked(5).Desc.Capacity = ranges(100) + sp.GetStoreDetailLocked(6).Desc.Capacity = ranges(0) + sp.GetStoreDetailLocked(7).Desc.Capacity = ranges(100) + sp.GetStoreDetailLocked(8).Desc.Capacity = ranges(100) + sp.DetailsMu.Unlock() // Each test case should describe a repair situation which has a lower // priority than the previous test case. @@ -1373,7 +1375,7 @@ func TestAllocatorRebalanceDeadNodes(t *testing.T) { for _, c := range testCases { t.Run("", func(t *testing.T) { - var rangeUsageInfo RangeUsageInfo + var rangeUsageInfo allocator.RangeUsageInfo target, _, _, ok := a.RebalanceVoter( ctx, emptySpanConfig(), @@ -1381,7 +1383,7 @@ func TestAllocatorRebalanceDeadNodes(t *testing.T) { c.existing, nil, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, a.scorerOptions(ctx), ) if c.expected > 0 { @@ -1484,7 +1486,7 @@ func TestAllocatorRebalanceThrashing(t *testing.T) { stopper, g, _, a, _ := createTestAllocator(ctx, 1, true /* deterministic */) defer stopper.Stop(ctx) - st := a.storePool.st + st := a.storePool.St cluster := tc.cluster(st) // It doesn't make sense to test sets of stores containing fewer than 4 @@ -1510,21 +1512,21 @@ func TestAllocatorRebalanceThrashing(t *testing.T) { // Ensure gossiped store descriptor changes have propagated. testutils.SucceedsSoon(t, func() error { - sl, _, _ := a.storePool.getStoreList(storeFilterThrottled) - for j, s := range sl.stores { + sl, _, _ := a.storePool.GetStoreList(storepool.StoreFilterThrottled) + for j, s := range sl.Stores { if a, e := s.Capacity.RangeCount, cluster[j].rangeCount; a != e { return errors.Errorf("range count for %d = %d != expected %d", j, a, e) } } return nil }) - sl, _, _ := a.storePool.getStoreList(storeFilterThrottled) + sl, _, _ := a.storePool.GetStoreList(storepool.StoreFilterThrottled) eqClass := equivalenceClass{ candidateSL: sl, } // Verify shouldRebalanceBasedOnThresholds returns the expected value. for j, store := range stores { - desc, ok := a.storePool.getStoreDescriptor(store.StoreID) + desc, ok := a.storePool.GetStoreDescriptor(store.StoreID) if !ok { t.Fatalf("[store %d]: unable to get store %d descriptor", j, store.StoreID) } @@ -1651,7 +1653,7 @@ func TestAllocatorRebalanceByQPS(t *testing.T) { stopper, g, _, a, _ := createTestAllocator(ctx, 10, false /* deterministic */) defer stopper.Stop(ctx) gossiputil.NewStoreGossiper(g).GossipStores(subtest.testStores, t) - var rangeUsageInfo RangeUsageInfo + var rangeUsageInfo allocator.RangeUsageInfo options := &qpsScorerOptions{ storeHealthOptions: storeHealthOptions{enforcementLevel: storeHealthNoAction}, qpsPerReplica: 100, @@ -1664,7 +1666,7 @@ func TestAllocatorRebalanceByQPS(t *testing.T) { []roachpb.ReplicaDescriptor{{StoreID: subtest.testStores[0].StoreID}}, nil, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, options, ) if subtest.expectRebalance { @@ -1672,8 +1674,8 @@ func TestAllocatorRebalanceByQPS(t *testing.T) { require.Equal(t, subtest.expectedAddTarget, add.StoreID) require.Equal(t, subtest.expectedRemoveTarget, remove.StoreID) // Verify shouldRebalanceBasedOnThresholds results. - if desc, descOk := a.storePool.getStoreDescriptor(remove.StoreID); descOk { - sl, _, _ := a.storePool.getStoreList(storeFilterThrottled) + if desc, descOk := a.storePool.GetStoreDescriptor(remove.StoreID); descOk { + sl, _, _ := a.storePool.GetStoreList(storepool.StoreFilterThrottled) eqClass := equivalenceClass{ existing: desc, candidateSL: sl, @@ -1822,7 +1824,7 @@ func TestAllocatorRebalanceByCount(t *testing.T) { // Every rebalance target must be store 4 (or nil for case of missing the only option). for i := 0; i < 10; i++ { - var rangeUsageInfo RangeUsageInfo + var rangeUsageInfo allocator.RangeUsageInfo result, _, _, ok := a.RebalanceVoter( ctx, emptySpanConfig(), @@ -1830,7 +1832,7 @@ func TestAllocatorRebalanceByCount(t *testing.T) { []roachpb.ReplicaDescriptor{{StoreID: stores[0].StoreID}}, nil, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, a.scorerOptions(ctx), ) if ok && result.StoreID != 4 { @@ -1840,11 +1842,11 @@ func TestAllocatorRebalanceByCount(t *testing.T) { // Verify shouldRebalanceBasedOnThresholds results. for i, store := range stores { - desc, ok := a.storePool.getStoreDescriptor(store.StoreID) + desc, ok := a.storePool.GetStoreDescriptor(store.StoreID) if !ok { t.Fatalf("%d: unable to get store %d descriptor", i, store.StoreID) } - sl, _, _ := a.storePool.getStoreList(storeFilterThrottled) + sl, _, _ := a.storePool.GetStoreList(storepool.StoreFilterThrottled) eqClass := equivalenceClass{ existing: desc, candidateSL: sl, @@ -2179,8 +2181,8 @@ func TestAllocatorTransferLeaseTargetDraining(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) ctx := context.Background() - stopper, g, _, storePool, nl := createTestStorePool(ctx, - TestTimeUntilStoreDeadOff, true, /* deterministic */ + stopper, g, _, storePool, nl := storepool.CreateTestStorePool(ctx, + storepool.TestTimeUntilStoreDeadOff, true, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_LIVE) a := MakeAllocator( @@ -2213,7 +2215,7 @@ func TestAllocatorTransferLeaseTargetDraining(t *testing.T) { sg := gossiputil.NewStoreGossiper(g) sg.GossipStores(stores, t) - nl.setNodeStatus(1, livenesspb.NodeLivenessStatus_DRAINING) + nl.SetNodeStatus(1, livenesspb.NodeLivenessStatus_DRAINING) preferDC1 := []roachpb.LeasePreference{ {Constraints: []roachpb.Constraint{{Key: "dc", Value: "1", Type: roachpb.Constraint_REQUIRED}}}, } @@ -2402,7 +2404,7 @@ func TestAllocatorRebalanceDifferentLocalitySizes(t *testing.T) { } for i, tc := range testCases { - var rangeUsageInfo RangeUsageInfo + var rangeUsageInfo allocator.RangeUsageInfo result, _, details, ok := a.RebalanceVoter( ctx, emptySpanConfig(), @@ -2410,7 +2412,7 @@ func TestAllocatorRebalanceDifferentLocalitySizes(t *testing.T) { tc.existing, nil, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, a.scorerOptions(ctx), ) var resultID roachpb.StoreID @@ -2474,7 +2476,7 @@ func TestAllocatorRebalanceDifferentLocalitySizes(t *testing.T) { for i, tc := range testCases2 { log.Infof(ctx, "case #%d", i) - var rangeUsageInfo RangeUsageInfo + var rangeUsageInfo allocator.RangeUsageInfo result, _, details, ok := a.RebalanceVoter( ctx, emptySpanConfig(), @@ -2482,7 +2484,7 @@ func TestAllocatorRebalanceDifferentLocalitySizes(t *testing.T) { tc.existing, nil, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, a.scorerOptions(ctx), ) var gotExpected bool @@ -2565,8 +2567,8 @@ func TestAllocatorShouldTransferLeaseDraining(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) ctx := context.Background() - stopper, g, _, storePool, nl := createTestStorePool(ctx, - TestTimeUntilStoreDeadOff, true, /* deterministic */ + stopper, g, _, storePool, nl := storepool.CreateTestStorePool(ctx, + storepool.TestTimeUntilStoreDeadOff, true, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_LIVE) a := MakeAllocator( @@ -2590,7 +2592,7 @@ func TestAllocatorShouldTransferLeaseDraining(t *testing.T) { sg.GossipStores(stores, t) // UNAVAILABLE is the node liveness status used for a node that's draining. - nl.setNodeStatus(1, livenesspb.NodeLivenessStatus_UNAVAILABLE) + nl.SetNodeStatus(1, livenesspb.NodeLivenessStatus_UNAVAILABLE) testCases := []struct { leaseholder roachpb.StoreID @@ -2633,8 +2635,8 @@ func TestAllocatorShouldTransferSuspected(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) ctx := context.Background() - stopper, g, clock, storePool, nl := createTestStorePool(ctx, - TestTimeUntilStoreDeadOff, true, /* deterministic */ + stopper, g, clock, storePool, nl := storepool.CreateTestStorePool(ctx, + storepool.TestTimeUntilStoreDeadOff, true, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_LIVE) a := MakeAllocator( @@ -2668,14 +2670,14 @@ func TestAllocatorShouldTransferSuspected(t *testing.T) { ) require.Equal(t, expected, result) } - timeAfterStoreSuspect := TimeAfterStoreSuspect.Get(&storePool.st.SV) + timeAfterStoreSuspect := storepool.TimeAfterStoreSuspect.Get(&storePool.St.SV) // Based on capacity node 1 is desirable. assertShouldTransferLease(true) // Flip node 1 to unavailable, there should be no lease transfer now. - nl.setNodeStatus(1, livenesspb.NodeLivenessStatus_UNAVAILABLE) + nl.SetNodeStatus(1, livenesspb.NodeLivenessStatus_UNAVAILABLE) assertShouldTransferLease(false) // Set node back to live, but it's still suspected so not lease transfer expected. - nl.setNodeStatus(1, livenesspb.NodeLivenessStatus_LIVE) + nl.SetNodeStatus(1, livenesspb.NodeLivenessStatus_LIVE) assertShouldTransferLease(false) // Wait out the suspected store timeout, verify that lease transfers are back. clock.Increment(timeAfterStoreSuspect.Nanoseconds() + time.Millisecond.Nanoseconds()) @@ -3327,7 +3329,7 @@ func TestAllocatorRebalanceTargetLocality(t *testing.T) { StoreID: storeID, } } - var rangeUsageInfo RangeUsageInfo + var rangeUsageInfo allocator.RangeUsageInfo target, _, details, ok := a.RebalanceVoter( ctx, emptySpanConfig(), @@ -3335,7 +3337,7 @@ func TestAllocatorRebalanceTargetLocality(t *testing.T) { existingRepls, nil, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, a.scorerOptions(ctx), ) if !ok { @@ -3508,7 +3510,7 @@ func TestAllocateCandidatesExcludeNonReadyNodes(t *testing.T) { defer stopper.Stop(ctx) sg := gossiputil.NewStoreGossiper(g) sg.GossipStores(stores, t) - sl, _, _ := a.storePool.getStoreList(storeFilterThrottled) + sl, _, _ := a.storePool.GetStoreList(storepool.StoreFilterThrottled) testCases := []struct { existing roachpb.StoreID @@ -3544,13 +3546,13 @@ func TestAllocateCandidatesExcludeNonReadyNodes(t *testing.T) { // No constraints. conf := roachpb.SpanConfig{} analyzed := constraint.AnalyzeConstraints( - ctx, a.storePool.getStoreDescriptor, existingRepls, conf.NumReplicas, + ctx, a.storePool.GetStoreDescriptor, existingRepls, conf.NumReplicas, conf.Constraints) allocationConstraintsChecker := voterConstraintsCheckerForAllocation(analyzed, constraint.EmptyAnalyzedConstraints) removalConstraintsChecker := voterConstraintsCheckerForRemoval(analyzed, constraint.EmptyAnalyzedConstraints) rebalanceConstraintsChecker := voterConstraintsCheckerForRebalance(analyzed, constraint.EmptyAnalyzedConstraints) - a.storePool.isStoreReadyForRoutineReplicaTransfer = func(_ context.Context, storeID roachpb.StoreID) bool { + a.storePool.IsStoreReadyForRoutineReplicaTransfer = func(_ context.Context, storeID roachpb.StoreID) bool { for _, s := range tc.excluded { if s == storeID { return false @@ -3565,8 +3567,8 @@ func TestAllocateCandidatesExcludeNonReadyNodes(t *testing.T) { sl, allocationConstraintsChecker, existingRepls, - a.storePool.getLocalitiesByStore(existingRepls), - a.storePool.isStoreReadyForRoutineReplicaTransfer, + a.storePool.GetLocalitiesByStore(existingRepls), + a.storePool.IsStoreReadyForRoutineReplicaTransfer, false, /* allowMultipleReplsPerNode */ a.scorerOptions(ctx), ) @@ -3585,8 +3587,8 @@ func TestAllocateCandidatesExcludeNonReadyNodes(t *testing.T) { rebalanceConstraintsChecker, existingRepls, nil, - a.storePool.getLocalitiesByStore(existingRepls), - a.storePool.isStoreReadyForRoutineReplicaTransfer, + a.storePool.GetLocalitiesByStore(existingRepls), + a.storePool.IsStoreReadyForRoutineReplicaTransfer, a.scorerOptions(ctx), a.metrics, ) @@ -3696,7 +3698,7 @@ func TestAllocateCandidatesNumReplicasConstraints(t *testing.T) { defer stopper.Stop(ctx) sg := gossiputil.NewStoreGossiper(g) sg.GossipStores(multiDiversityDCStores, t) - sl, _, _ := a.storePool.getStoreList(storeFilterThrottled) + sl, _, _ := a.storePool.GetStoreList(storepool.StoreFilterThrottled) // Given a set of existing replicas for a range, rank which of the remaining // stores from multiDiversityDCStores would be the best addition to the range @@ -3898,7 +3900,7 @@ func TestAllocateCandidatesNumReplicasConstraints(t *testing.T) { } conf := roachpb.SpanConfig{Constraints: tc.constraints} analyzed := constraint.AnalyzeConstraints( - ctx, a.storePool.getStoreDescriptor, existingRepls, conf.NumReplicas, + ctx, a.storePool.GetStoreDescriptor, existingRepls, conf.NumReplicas, conf.Constraints) checkFn := voterConstraintsCheckerForAllocation(analyzed, constraint.EmptyAnalyzedConstraints) @@ -3907,7 +3909,7 @@ func TestAllocateCandidatesNumReplicasConstraints(t *testing.T) { sl, checkFn, existingRepls, - a.storePool.getLocalitiesByStore(existingRepls), + a.storePool.GetLocalitiesByStore(existingRepls), func(context.Context, roachpb.StoreID) bool { return true }, false, /* allowMultipleReplsPerNode */ a.scorerOptions(ctx), @@ -4119,7 +4121,7 @@ func TestRemoveCandidatesNumReplicasConstraints(t *testing.T) { } for testIdx, tc := range testCases { - sl, _, _ := a.storePool.getStoreListFromIDs(tc.existing, storeFilterNone) + sl, _, _ := a.storePool.GetStoreListFromIDs(tc.existing, storepool.StoreFilterNone) existingRepls := make([]roachpb.ReplicaDescriptor, len(tc.existing)) for i, storeID := range tc.existing { existingRepls[i] = roachpb.ReplicaDescriptor{ @@ -4127,7 +4129,7 @@ func TestRemoveCandidatesNumReplicasConstraints(t *testing.T) { StoreID: storeID, } } - analyzed := constraint.AnalyzeConstraints(ctx, a.storePool.getStoreDescriptor, existingRepls, + analyzed := constraint.AnalyzeConstraints(ctx, a.storePool.GetStoreDescriptor, existingRepls, 0 /* numReplicas */, tc.constraints) // Check behavior in a span config where `voter_constraints` are empty. @@ -4135,7 +4137,7 @@ func TestRemoveCandidatesNumReplicasConstraints(t *testing.T) { candidates := candidateListForRemoval(ctx, sl, checkFn, - a.storePool.getLocalitiesByStore(existingRepls), + a.storePool.GetLocalitiesByStore(existingRepls), a.scorerOptions(ctx)) if !expectedStoreIDsMatch(tc.expected, candidates.worst()) { t.Errorf("%d (with `constraints`): expected candidateListForRemoval(%v)"+ @@ -4149,7 +4151,7 @@ func TestRemoveCandidatesNumReplicasConstraints(t *testing.T) { candidates = candidateListForRemoval(ctx, sl, checkFn, - a.storePool.getLocalitiesByStore(existingRepls), + a.storePool.GetLocalitiesByStore(existingRepls), a.scorerOptions(ctx)) if !expectedStoreIDsMatch(tc.expected, candidates.worst()) { t.Errorf("%d (with `voter_constraints`): expected candidateListForRemoval(%v)"+ @@ -4278,7 +4280,7 @@ func TestAllocatorRebalanceNonVoters(t *testing.T) { }, } - var rangeUsageInfo RangeUsageInfo + var rangeUsageInfo allocator.RangeUsageInfo chk := func(target roachpb.ReplicationTarget, expectedCandidates []roachpb.StoreID) bool { for _, candidate := range expectedCandidates { if target.StoreID == candidate { @@ -4302,7 +4304,7 @@ func TestAllocatorRebalanceNonVoters(t *testing.T) { test.existingVoters, test.existingNonVoters, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, a.scorerOptions(ctx), ) if test.expectNoAction { @@ -4404,7 +4406,7 @@ func TestAllocatorRebalanceReadAmpCheck(t *testing.T) { }, } - var rangeUsageInfo RangeUsageInfo + var rangeUsageInfo allocator.RangeUsageInfo chk := func(target roachpb.ReplicationTarget, expectedCandidates []roachpb.StoreID) bool { for _, candidate := range expectedCandidates { if target.StoreID == candidate { @@ -4430,7 +4432,7 @@ func TestAllocatorRebalanceReadAmpCheck(t *testing.T) { test.existingVoters, []roachpb.ReplicaDescriptor{}, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, options, ) if test.expectNoAction { @@ -4477,7 +4479,7 @@ func TestVotersCanRebalanceToNonVoterStores(t *testing.T) { }, } - var rangeUsageInfo RangeUsageInfo + var rangeUsageInfo allocator.RangeUsageInfo existingNonVoters := replicas(1, 2) existingVoters := replicas(3, 4) add, remove, _, ok := a.RebalanceVoter( @@ -4487,7 +4489,7 @@ func TestVotersCanRebalanceToNonVoterStores(t *testing.T) { existingVoters, existingNonVoters, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, a.scorerOptions(ctx), ) @@ -4538,7 +4540,7 @@ func TestNonVotersCannotRebalanceToVoterStores(t *testing.T) { existingVoters := replicas(2) sg.GossipStores(stores, t) - var rangeUsageInfo RangeUsageInfo + var rangeUsageInfo allocator.RangeUsageInfo add, remove, _, ok := a.RebalanceNonVoter( ctx, emptySpanConfig(), @@ -4546,7 +4548,7 @@ func TestNonVotersCannotRebalanceToVoterStores(t *testing.T) { existingVoters, existingNonVoters, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, a.scorerOptions(ctx), ) @@ -4572,7 +4574,7 @@ func TestRebalanceCandidatesNumReplicasConstraints(t *testing.T) { defer stopper.Stop(ctx) sg := gossiputil.NewStoreGossiper(g) sg.GossipStores(multiDiversityDCStores, t) - sl, _, _ := a.storePool.getStoreList(storeFilterThrottled) + sl, _, _ := a.storePool.GetStoreList(storepool.StoreFilterThrottled) // Given a set of existing replicas for a range, rank which of the remaining // stores would be best to remove if we had to remove one purely on the basis @@ -5324,13 +5326,13 @@ func TestRebalanceCandidatesNumReplicasConstraints(t *testing.T) { StoreID: storeID, } } - var rangeUsageInfo RangeUsageInfo + var rangeUsageInfo allocator.RangeUsageInfo conf := roachpb.SpanConfig{ Constraints: tc.constraints, NumReplicas: tc.numReplicas, } analyzed := constraint.AnalyzeConstraints( - ctx, a.storePool.getStoreDescriptor, existingRepls, + ctx, a.storePool.GetStoreDescriptor, existingRepls, conf.NumReplicas, conf.Constraints) removalConstraintsChecker := voterConstraintsCheckerForRemoval( analyzed, @@ -5348,7 +5350,7 @@ func TestRebalanceCandidatesNumReplicasConstraints(t *testing.T) { rebalanceConstraintsChecker, existingRepls, nil, - a.storePool.getLocalitiesByStore(existingRepls), + a.storePool.GetLocalitiesByStore(existingRepls), func(context.Context, roachpb.StoreID) bool { return true }, a.scorerOptions(ctx), a.metrics, @@ -5381,7 +5383,7 @@ func TestRebalanceCandidatesNumReplicasConstraints(t *testing.T) { existingRepls, nil, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, a.scorerOptions(ctx), ) var found bool @@ -5409,8 +5411,8 @@ func TestAllocatorTransferLeaseTargetLoadBased(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - stopper, g, _, storePool, _ := createTestStorePool(ctx, - TestTimeUntilStoreDeadOff, true, /* deterministic */ + stopper, g, _, storePool, _ := storepool.CreateTestStorePool(ctx, + storepool.TestTimeUntilStoreDeadOff, true, /* deterministic */ func() int { return 10 }, /* nodeCount */ livenesspb.NodeLivenessStatus_LIVE) defer stopper.Stop(ctx) @@ -7130,8 +7132,8 @@ func TestAllocatorComputeActionDynamicNumReplicas(t *testing.T) { var numNodes int ctx := context.Background() - stopper, _, _, sp, _ := createTestStorePool(ctx, - TestTimeUntilStoreDeadOff, false, /* deterministic */ + stopper, _, _, sp, _ := storepool.CreateTestStorePool(ctx, + storepool.TestTimeUntilStoreDeadOff, false, /* deterministic */ func() int { return numNodes }, livenesspb.NodeLivenessStatus_LIVE) a := MakeAllocator( @@ -7343,13 +7345,13 @@ func TestAllocatorThrottled(t *testing.T) { // Finally, set that store to be throttled and ensure we don't send the // replica to purgatory. - a.storePool.detailsMu.Lock() - storeDetail, ok := a.storePool.detailsMu.storeDetails[singleStore[0].StoreID] + a.storePool.DetailsMu.Lock() + storeDetail, ok := a.storePool.DetailsMu.StoreDetails[singleStore[0].StoreID] if !ok { t.Fatalf("store:%d was not found in the store pool", singleStore[0].StoreID) } - storeDetail.throttledUntil = timeutil.Now().Add(24 * time.Hour) - a.storePool.detailsMu.Unlock() + storeDetail.ThrottledUntil = timeutil.Now().Add(24 * time.Hour) + a.storePool.DetailsMu.Unlock() _, _, err = a.AllocateVoter(ctx, simpleSpanConfig, []roachpb.ReplicaDescriptor{}, nil) if errors.HasInterface(err, (*purgatoryError)(nil)) { t.Fatalf("expected a non purgatory error, got: %+v", err) @@ -7599,7 +7601,7 @@ func TestAllocatorRebalanceWithScatter(t *testing.T) { gossiputil.NewStoreGossiper(g).GossipStores(stores, t) - var rangeUsageInfo RangeUsageInfo + var rangeUsageInfo allocator.RangeUsageInfo // Ensure that we wouldn't normally rebalance when all stores have the same // replica count. @@ -7610,7 +7612,7 @@ func TestAllocatorRebalanceWithScatter(t *testing.T) { replicas(1), nil, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, a.scorerOptions(ctx), ) require.False(t, ok) @@ -7623,7 +7625,7 @@ func TestAllocatorRebalanceWithScatter(t *testing.T) { replicas(1), nil, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, a.scorerOptionsForScatter(ctx), ) require.True(t, ok) @@ -7729,7 +7731,7 @@ func TestAllocatorRebalanceAway(t *testing.T) { }, } - var rangeUsageInfo RangeUsageInfo + var rangeUsageInfo allocator.RangeUsageInfo actual, _, _, ok := a.RebalanceVoter( ctx, roachpb.SpanConfig{Constraints: []roachpb.ConstraintsConjunction{constraints}}, @@ -7737,7 +7739,7 @@ func TestAllocatorRebalanceAway(t *testing.T) { existingReplicas, nil, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, a.scorerOptions(ctx), ) @@ -7772,7 +7774,7 @@ func (ts *testStore) rebalance(ots *testStore, bytes int64, qps float64) { // Mimic a real Store's behavior of not considering target stores that are // almost out of disk. (In a real allocator this is, for example, in // rankedCandidateListFor{Allocation,Rebalancing}). - if !maxCapacityCheck(ots.StoreDescriptor) { + if !allocator.MaxCapacityCheck(ots.StoreDescriptor) { log.Infof( context.Background(), "s%d too full to accept snapshot from s%d: %v", ots.StoreID, ts.StoreID, ots.Capacity, @@ -7823,15 +7825,15 @@ func TestAllocatorFullDisks(t *testing.T) { server := rpc.NewServer(rpcContext) // never started g := gossip.NewTest(1, rpcContext, server, stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) - TimeUntilStoreDead.Override(ctx, &st.SV, TestTimeUntilStoreDeadOff) + storepool.TimeUntilStoreDead.Override(ctx, &st.SV, storepool.TestTimeUntilStoreDeadOff) const generations = 100 const nodes = 20 const capacity = (1 << 30) + 1 const rangeSize = 16 << 20 - mockNodeLiveness := newMockNodeLiveness(livenesspb.NodeLivenessStatus_LIVE) - sp := NewStorePool( + mockNodeLiveness := storepool.NewMockNodeLiveness(livenesspb.NodeLivenessStatus_LIVE) + sp := storepool.NewStorePool( log.MakeTestingAmbientContext(tr), st, g, @@ -7839,7 +7841,7 @@ func TestAllocatorFullDisks(t *testing.T) { func() int { return nodes }, - mockNodeLiveness.nodeLivenessFunc, + mockNodeLiveness.NodeLivenessFunc, false, /* deterministic */ ) alloc := MakeAllocator( @@ -7874,7 +7876,7 @@ func TestAllocatorFullDisks(t *testing.T) { for i := 0; i < generations; i++ { // First loop through test stores and randomly add data. for j := 0; j < len(testStores); j++ { - if mockNodeLiveness.nodeLivenessFunc(roachpb.NodeID(j), time.Time{}, 0) == livenesspb.NodeLivenessStatus_DEAD { + if mockNodeLiveness.NodeLivenessFunc(roachpb.NodeID(j), time.Time{}, 0) == livenesspb.NodeLivenessStatus_DEAD { continue } ts := &testStores[j] @@ -7889,7 +7891,7 @@ func TestAllocatorFullDisks(t *testing.T) { if ts.Capacity.Available <= 0 { t.Errorf("testStore %d ran out of space during generation %d (rangesAdded=%d/%d): %+v", j, i, rangesAdded, rangesToAdd, ts.Capacity) - mockNodeLiveness.setNodeStatus(roachpb.NodeID(j), livenesspb.NodeLivenessStatus_DEAD) + mockNodeLiveness.SetNodeStatus(roachpb.NodeID(j), livenesspb.NodeLivenessStatus_DEAD) } wg.Add(1) if err := g.AddInfoProto(gossip.MakeStoreKey(roachpb.StoreID(j)), &ts.StoreDescriptor, 0); err != nil { @@ -7901,13 +7903,13 @@ func TestAllocatorFullDisks(t *testing.T) { // Loop through each store a number of times and maybe rebalance. for j := 0; j < 10; j++ { for k := 0; k < len(testStores); k++ { - if mockNodeLiveness.nodeLivenessFunc(roachpb.NodeID(k), time.Time{}, 0) == livenesspb.NodeLivenessStatus_DEAD { + if mockNodeLiveness.NodeLivenessFunc(roachpb.NodeID(k), time.Time{}, 0) == livenesspb.NodeLivenessStatus_DEAD { continue } ts := &testStores[k] // Rebalance until there's no more rebalancing to do. if ts.Capacity.RangeCount > 0 { - var rangeUsageInfo RangeUsageInfo + var rangeUsageInfo allocator.RangeUsageInfo target, _, details, ok := alloc.RebalanceVoter( ctx, emptySpanConfig(), @@ -7915,7 +7917,7 @@ func TestAllocatorFullDisks(t *testing.T) { []roachpb.ReplicaDescriptor{{NodeID: ts.Node.NodeID, StoreID: ts.StoreID}}, nil, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, alloc.scorerOptions(ctx), ) if ok { @@ -7937,11 +7939,11 @@ func TestAllocatorFullDisks(t *testing.T) { // Simulate rocksdb compactions freeing up disk space. for j := 0; j < len(testStores); j++ { - if mockNodeLiveness.nodeLivenessFunc(roachpb.NodeID(j), time.Time{}, 0) != livenesspb.NodeLivenessStatus_DEAD { + if mockNodeLiveness.NodeLivenessFunc(roachpb.NodeID(j), time.Time{}, 0) != livenesspb.NodeLivenessStatus_DEAD { ts := &testStores[j] if ts.Capacity.Available <= 0 { t.Errorf("testStore %d ran out of space during generation %d: %+v", j, i, ts.Capacity) - mockNodeLiveness.setNodeStatus(roachpb.NodeID(j), livenesspb.NodeLivenessStatus_DEAD) + mockNodeLiveness.SetNodeStatus(roachpb.NodeID(j), livenesspb.NodeLivenessStatus_DEAD) } else { ts.compact() } @@ -7953,7 +7955,7 @@ func TestAllocatorFullDisks(t *testing.T) { func Example_rangeCountRebalancing() { testStores := make([]testStore, 20) rebalanceFn := func(ctx context.Context, ts *testStore, testStores []testStore, alloc *Allocator) { - var rangeUsageInfo RangeUsageInfo + var rangeUsageInfo allocator.RangeUsageInfo target, _, details, ok := alloc.RebalanceVoter( ctx, emptySpanConfig(), @@ -7961,7 +7963,7 @@ func Example_rangeCountRebalancing() { []roachpb.ReplicaDescriptor{{NodeID: ts.Node.NodeID, StoreID: ts.StoreID}}, nil, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, alloc.scorerOptions(ctx), ) if ok { @@ -8060,7 +8062,7 @@ func qpsBasedRebalanceFn( qpsPerReplica: jitteredQPS, qpsRebalanceThreshold: 0.2, } - var rangeUsageInfo RangeUsageInfo + var rangeUsageInfo allocator.RangeUsageInfo add, remove, details, ok := alloc.RebalanceVoter( ctx, emptySpanConfig(), @@ -8068,7 +8070,7 @@ func qpsBasedRebalanceFn( []roachpb.ReplicaDescriptor{{NodeID: candidate.Node.NodeID, StoreID: candidate.StoreID}}, nil, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, opts, ) if ok { @@ -8272,13 +8274,13 @@ func exampleRebalancing( server := rpc.NewServer(rpcContext) // never started g := gossip.NewTest(1, rpcContext, server, stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) - TimeUntilStoreDead.Override(ctx, &st.SV, TestTimeUntilStoreDeadOff) + storepool.TimeUntilStoreDead.Override(ctx, &st.SV, storepool.TestTimeUntilStoreDeadOff) const nodes = 20 // Deterministic must be set as this test is comparing the exact output // after each rebalance. - sp := NewStorePool( + sp := storepool.NewStorePool( ambientCtx, st, g, @@ -8286,7 +8288,7 @@ func exampleRebalancing( func() int { return nodes }, - newMockNodeLiveness(livenesspb.NodeLivenessStatus_LIVE).nodeLivenessFunc, + storepool.NewMockNodeLiveness(livenesspb.NodeLivenessStatus_LIVE).NodeLivenessFunc, /* deterministic */ true, ) alloc := MakeAllocator( diff --git a/pkg/kv/kvserver/client_lease_test.go b/pkg/kv/kvserver/client_lease_test.go index b179f0e0c3cb..415565d46d01 100644 --- a/pkg/kv/kvserver/client_lease_test.go +++ b/pkg/kv/kvserver/client_lease_test.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -802,7 +803,7 @@ func TestLeasePreferencesDuringOutage(t *testing.T) { } } // We need to wait until 2 and 3 are considered to be dead. - timeUntilStoreDead := kvserver.TimeUntilStoreDead.Get(&tc.GetFirstStoreFromServer(t, 0).GetStoreConfig().Settings.SV) + timeUntilStoreDead := storepool.TimeUntilStoreDead.Get(&tc.GetFirstStoreFromServer(t, 0).GetStoreConfig().Settings.SV) wait(timeUntilStoreDead.Nanoseconds()) checkDead := func(store *kvserver.Store, storeIdx int) error { @@ -822,10 +823,10 @@ func TestLeasePreferencesDuringOutage(t *testing.T) { testutils.SucceedsSoon(t, func() error { store := tc.GetFirstStoreFromServer(t, 0) - sl, _, _ := store.GetStoreConfig().StorePool.GetStoreList() - if len(sl.Stores()) != 3 { + sl, _, _ := store.GetStoreConfig().StorePool.TestingGetStoreList() + if len(sl.TestingStores()) != 3 { return errors.Errorf("expected all 3 remaining stores to be live, but only got %v", - sl.Stores()) + sl.TestingStores()) } if err := checkDead(store, 1); err != nil { return err @@ -1058,7 +1059,7 @@ func TestLeasesDontThrashWhenNodeBecomesSuspect(t *testing.T) { } testutils.SucceedsSoon(t, allLeasesOnNonSuspectStores) // Wait out the suspect time. - suspectDuration := kvserver.TimeAfterStoreSuspect.Get(&tc.GetFirstStoreFromServer(t, 0).GetStoreConfig().Settings.SV) + suspectDuration := storepool.TimeAfterStoreSuspect.Get(&tc.GetFirstStoreFromServer(t, 0).GetStoreConfig().Settings.SV) for i := 0; i < int(math.Ceil(suspectDuration.Seconds())); i++ { manualClock.Increment(time.Second.Nanoseconds()) heartbeat(0, 1, 2, 3) diff --git a/pkg/kv/kvserver/client_store_pool_test.go b/pkg/kv/kvserver/client_store_pool_test.go new file mode 100644 index 000000000000..aa39d3a8cfe7 --- /dev/null +++ b/pkg/kv/kvserver/client_store_pool_test.go @@ -0,0 +1,213 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package kvserver + +import ( + "context" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/testutils/gossiputil" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/uuid" +) + +func TestStorePoolUpdateLocalStore(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + manual := hlc.NewManualClock(123) + clock := hlc.NewClock(manual.UnixNano, time.Nanosecond) + ctx := context.Background() + // We're going to manually mark stores dead in this test. + stopper, g, _, sp, _ := storepool.CreateTestStorePool(ctx, + storepool.TestTimeUntilStoreDead, false, /* deterministic */ + func() int { return 10 }, /* nodeCount */ + livenesspb.NodeLivenessStatus_DEAD) + defer stopper.Stop(ctx) + sg := gossiputil.NewStoreGossiper(g) + stores := []*roachpb.StoreDescriptor{ + { + StoreID: 1, + Node: roachpb.NodeDescriptor{NodeID: 1}, + Capacity: roachpb.StoreCapacity{ + Capacity: 100, + Available: 50, + RangeCount: 5, + LeaseCount: 1, + LogicalBytes: 30, + QueriesPerSecond: 100, + WritesPerSecond: 30, + L0Sublevels: 4, + }, + }, + { + StoreID: 2, + Node: roachpb.NodeDescriptor{NodeID: 2}, + Capacity: roachpb.StoreCapacity{ + Capacity: 100, + Available: 55, + RangeCount: 4, + LeaseCount: 2, + LogicalBytes: 25, + QueriesPerSecond: 50, + WritesPerSecond: 25, + L0Sublevels: 8, + }, + }, + } + sg.GossipStores(stores, t) + + replica := Replica{RangeID: 1} + replica.mu.Lock() + replica.mu.state.Stats = &enginepb.MVCCStats{ + KeyBytes: 2, + ValBytes: 4, + } + replica.mu.Unlock() + rs := newReplicaStats(clock, nil) + for _, store := range stores { + rs.recordCount(1, store.Node.NodeID) + } + manual.Increment(int64(MinStatsDuration + time.Second)) + replica.leaseholderStats = rs + replica.writeStats = rs + + rangeUsageInfo := rangeUsageInfoForRepl(&replica) + QPS, _ := replica.leaseholderStats.averageRatePerSecond() + WPS, _ := replica.writeStats.averageRatePerSecond() + + sp.UpdateLocalStoreAfterRebalance(roachpb.StoreID(1), rangeUsageInfo, roachpb.ADD_VOTER) + desc, ok := sp.GetStoreDescriptor(roachpb.StoreID(1)) + if !ok { + t.Fatalf("couldn't find StoreDescriptor for Store ID %d", 1) + } + if expectedRangeCount := int32(6); desc.Capacity.RangeCount != expectedRangeCount { + t.Errorf("expected RangeCount %d, but got %d", expectedRangeCount, desc.Capacity.RangeCount) + } + if expectedBytes := int64(36); desc.Capacity.LogicalBytes != expectedBytes { + t.Errorf("expected logical bytes %d, but got %d", expectedBytes, desc.Capacity.LogicalBytes) + } + if expectedQPS := float64(100); desc.Capacity.QueriesPerSecond != expectedQPS { + t.Errorf("expected QueriesPerSecond %f, but got %f", expectedQPS, desc.Capacity.QueriesPerSecond) + } + if expectedWPS := 30 + WPS; desc.Capacity.WritesPerSecond != expectedWPS { + t.Errorf("expected WritesPerSecond %f, but got %f", expectedWPS, desc.Capacity.WritesPerSecond) + } + if expectedL0Sublevels := int64(4); desc.Capacity.L0Sublevels != expectedL0Sublevels { + t.Errorf("expected L0 Sub-Levels %d, but got %d", expectedL0Sublevels, desc.Capacity.L0Sublevels) + } + + sp.UpdateLocalStoreAfterRebalance(roachpb.StoreID(2), rangeUsageInfo, roachpb.REMOVE_VOTER) + desc, ok = sp.GetStoreDescriptor(roachpb.StoreID(2)) + if !ok { + t.Fatalf("couldn't find StoreDescriptor for Store ID %d", 2) + } + if expectedRangeCount := int32(3); desc.Capacity.RangeCount != expectedRangeCount { + t.Errorf("expected RangeCount %d, but got %d", expectedRangeCount, desc.Capacity.RangeCount) + } + if expectedBytes := int64(19); desc.Capacity.LogicalBytes != expectedBytes { + t.Errorf("expected logical bytes %d, but got %d", expectedBytes, desc.Capacity.LogicalBytes) + } + if expectedQPS := float64(50); desc.Capacity.QueriesPerSecond != expectedQPS { + t.Errorf("expected QueriesPerSecond %f, but got %f", expectedQPS, desc.Capacity.QueriesPerSecond) + } + if expectedWPS := 25 - WPS; desc.Capacity.WritesPerSecond != expectedWPS { + t.Errorf("expected WritesPerSecond %f, but got %f", expectedWPS, desc.Capacity.WritesPerSecond) + } + if expectedL0Sublevels := int64(8); desc.Capacity.L0Sublevels != expectedL0Sublevels { + t.Errorf("expected L0 Sub-Levels %d, but got %d", expectedL0Sublevels, desc.Capacity.L0Sublevels) + } + + sp.UpdateLocalStoresAfterLeaseTransfer(roachpb.StoreID(1), roachpb.StoreID(2), rangeUsageInfo.QueriesPerSecond) + desc, ok = sp.GetStoreDescriptor(roachpb.StoreID(1)) + if !ok { + t.Fatalf("couldn't find StoreDescriptor for Store ID %d", 1) + } + if expectedLeaseCount := int32(0); desc.Capacity.LeaseCount != expectedLeaseCount { + t.Errorf("expected LeaseCount %d, but got %d", expectedLeaseCount, desc.Capacity.LeaseCount) + } + if expectedQPS := 100 - QPS; desc.Capacity.QueriesPerSecond != expectedQPS { + t.Errorf("expected QueriesPerSecond %f, but got %f", expectedQPS, desc.Capacity.QueriesPerSecond) + } + desc, ok = sp.GetStoreDescriptor(roachpb.StoreID(2)) + if !ok { + t.Fatalf("couldn't find StoreDescriptor for Store ID %d", 2) + } + if expectedLeaseCount := int32(3); desc.Capacity.LeaseCount != expectedLeaseCount { + t.Errorf("expected LeaseCount %d, but got %d", expectedLeaseCount, desc.Capacity.LeaseCount) + } + if expectedQPS := 50 + QPS; desc.Capacity.QueriesPerSecond != expectedQPS { + t.Errorf("expected QueriesPerSecond %f, but got %f", expectedQPS, desc.Capacity.QueriesPerSecond) + } +} + +// TestStorePoolUpdateLocalStoreBeforeGossip verifies that an attempt to update +// the local copy of store before that store has been gossiped will be a no-op. +func TestStorePoolUpdateLocalStoreBeforeGossip(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + ctx := context.Background() + manual := hlc.NewManualClock(123) + clock := hlc.NewClock(manual.UnixNano, time.Nanosecond) + stopper, _, _, sp, _ := storepool.CreateTestStorePool(ctx, + storepool.TestTimeUntilStoreDead, false, /* deterministic */ + func() int { return 10 }, /* nodeCount */ + livenesspb.NodeLivenessStatus_DEAD) + defer stopper.Stop(ctx) + + // Create store. + node := roachpb.NodeDescriptor{NodeID: roachpb.NodeID(1)} + eng := storage.NewDefaultInMemForTesting() + stopper.AddCloser(eng) + cfg := TestStoreConfig(clock) + cfg.Transport = NewDummyRaftTransport(cfg.Settings, cfg.AmbientCtx.Tracer) + store := NewStore(ctx, cfg, eng, &node) + // Fake an ident because this test doesn't want to start the store + // but without an Ident there will be NPEs. + store.Ident = &roachpb.StoreIdent{ + ClusterID: uuid.Nil, + StoreID: 1, + NodeID: 1, + } + + // Create replica. + rg := roachpb.RangeDescriptor{ + RangeID: 1, + StartKey: roachpb.RKey([]byte("a")), + EndKey: roachpb.RKey([]byte("b")), + NextReplicaID: 1, + } + rg.AddReplica(1, 1, roachpb.VOTER_FULL) + replica, err := newReplica(ctx, &rg, store, 1) + if err != nil { + t.Fatalf("make replica error : %+v", err) + } + replica.leaseholderStats = newReplicaStats(store.Clock(), nil) + + rangeUsageInfo := rangeUsageInfoForRepl(replica) + + // Update StorePool, which should be a no-op. + storeID := roachpb.StoreID(1) + if _, ok := sp.GetStoreDescriptor(storeID); ok { + t.Fatalf("StoreDescriptor not gossiped, should not be found") + } + sp.UpdateLocalStoreAfterRebalance(storeID, rangeUsageInfo, roachpb.ADD_VOTER) + if _, ok := sp.GetStoreDescriptor(storeID); ok { + t.Fatalf("StoreDescriptor still not gossiped, should not be found") + } +} diff --git a/pkg/kv/kvserver/helpers_test.go b/pkg/kv/kvserver/helpers_test.go index ad4272f2e14e..8d4904dd86ef 100644 --- a/pkg/kv/kvserver/helpers_test.go +++ b/pkg/kv/kvserver/helpers_test.go @@ -24,6 +24,7 @@ import ( circuit "github.com/cockroachdb/circuitbreaker" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" @@ -205,9 +206,9 @@ func (s *Store) RaftSchedulerPriorityID() roachpb.RangeID { return s.scheduler.PriorityID() } -func NewTestStorePool(cfg StoreConfig) *StorePool { - TimeUntilStoreDead.Override(context.Background(), &cfg.Settings.SV, TestTimeUntilStoreDeadOff) - return NewStorePool( +func NewTestStorePool(cfg StoreConfig) *storepool.StorePool { + storepool.TimeUntilStoreDead.Override(context.Background(), &cfg.Settings.SV, storepool.TestTimeUntilStoreDeadOff) + return storepool.NewStorePool( cfg.AmbientCtx, cfg.Settings, cfg.Gossip, @@ -357,20 +358,6 @@ func (r *Replica) IsRaftGroupInitialized() bool { return r.mu.internalRaftGroup != nil } -// GetStoreList exposes getStoreList for testing only, but with a hardcoded -// storeFilter of storeFilterNone. -func (sp *StorePool) GetStoreList() (StoreList, int, int) { - list, available, throttled := sp.getStoreList(storeFilterNone) - return list, available, len(throttled) -} - -// Stores returns a copy of sl.stores. -func (sl *StoreList) Stores() []roachpb.StoreDescriptor { - stores := make([]roachpb.StoreDescriptor, len(sl.stores)) - copy(stores, sl.stores) - return stores -} - // SideloadedRaftMuLocked returns r.raftMu.sideloaded. Requires a previous call // to RaftLock() or some other guarantee that r.raftMu is held. func (r *Replica) SideloadedRaftMuLocked() SideloadStorage { diff --git a/pkg/kv/kvserver/liveness/BUILD.bazel b/pkg/kv/kvserver/liveness/BUILD.bazel index 0cd8a129614b..9ed7d1e34ccb 100644 --- a/pkg/kv/kvserver/liveness/BUILD.bazel +++ b/pkg/kv/kvserver/liveness/BUILD.bazel @@ -42,6 +42,7 @@ go_test( deps = [ "//pkg/base", "//pkg/kv/kvserver", + "//pkg/kv/kvserver/allocator/storepool", "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/roachpb", "//pkg/security", diff --git a/pkg/kv/kvserver/liveness/client_test.go b/pkg/kv/kvserver/liveness/client_test.go index 178369def9f0..83101a967956 100644 --- a/pkg/kv/kvserver/liveness/client_test.go +++ b/pkg/kv/kvserver/liveness/client_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -242,7 +243,7 @@ func TestNodeLivenessStatusMap(t *testing.T) { // doesn't allow durations below 1m15s, which is much too long // for a test. // We do this in every SucceedsSoon attempt, so we'll be good. - kvserver.TimeUntilStoreDead.Override(ctx, &firstServer.ClusterSettings().SV, kvserver.TestTimeUntilStoreDead) + storepool.TimeUntilStoreDead.Override(ctx, &firstServer.ClusterSettings().SV, storepool.TestTimeUntilStoreDead) log.Infof(ctx, "checking expected status (%s) for node %d", expectedStatus, nodeID) resp, err := admin.Liveness(ctx, &serverpb.LivenessRequest{}) diff --git a/pkg/kv/kvserver/node_liveness_test.go b/pkg/kv/kvserver/node_liveness_test.go index f4c04407b392..83c096b6fd95 100644 --- a/pkg/kv/kvserver/node_liveness_test.go +++ b/pkg/kv/kvserver/node_liveness_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" @@ -917,8 +918,8 @@ func TestNodeLivenessSetDraining(t *testing.T) { drainingNodeIdx := 0 drainingNodeID := tc.Servers[0].Gossip().NodeID.Get() - nodeIDAppearsInStoreList := func(id roachpb.NodeID, sl kvserver.StoreList) bool { - for _, store := range sl.Stores() { + nodeIDAppearsInStoreList := func(id roachpb.NodeID, sl storepool.StoreList) bool { + for _, store := range sl.TestingStores() { if store.Node.NodeID == id { return true } @@ -948,7 +949,7 @@ func TestNodeLivenessSetDraining(t *testing.T) { testutils.SucceedsSoon(t, func() error { for i, s := range tc.Servers { curNodeID := s.Gossip().NodeID.Get() - sl, alive, _ := tc.GetFirstStoreFromServer(t, i).GetStoreConfig().StorePool.GetStoreList() + sl, alive, _ := tc.GetFirstStoreFromServer(t, i).GetStoreConfig().StorePool.TestingGetStoreList() if alive != expectedLive { return errors.Errorf( "expected %d live stores but got %d from node %d", @@ -982,7 +983,7 @@ func TestNodeLivenessSetDraining(t *testing.T) { testutils.SucceedsSoon(t, func() error { for i, s := range tc.Servers { curNodeID := s.Gossip().NodeID.Get() - sl, alive, _ := tc.GetFirstStoreFromServer(t, i).GetStoreConfig().StorePool.GetStoreList() + sl, alive, _ := tc.GetFirstStoreFromServer(t, i).GetStoreConfig().StorePool.TestingGetStoreList() if alive != expectedLive { return errors.Errorf( "expected %d live stores but got %d from node %d", @@ -996,7 +997,7 @@ func TestNodeLivenessSetDraining(t *testing.T) { "expected node %d to appear in node %d's store list: %+v", drainingNodeID, curNodeID, - sl.Stores(), + sl.TestingStores(), ) } } diff --git a/pkg/kv/kvserver/raft_transport.go b/pkg/kv/kvserver/raft_transport.go index d0a5ab57a79a..3979ec97baeb 100644 --- a/pkg/kv/kvserver/raft_transport.go +++ b/pkg/kv/kvserver/raft_transport.go @@ -20,6 +20,7 @@ import ( "time" "unsafe" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" @@ -683,7 +684,7 @@ func (t *RaftTransport) startProcessNewQueue( // for closing the OutgoingSnapshot. func (t *RaftTransport) SendSnapshot( ctx context.Context, - storePool *StorePool, + storePool *storepool.StorePool, header kvserverpb.SnapshotRequest_Header, snap *OutgoingSnapshot, newBatch func() storage.Batch, diff --git a/pkg/kv/kvserver/replica_command.go b/pkg/kv/kvserver/replica_command.go index 305928ebfffd..b1dbbcde9484 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -1048,7 +1049,7 @@ func (r *Replica) changeReplicasImpl( if len(swaps) > 0 { desc, err = execChangeReplicasTxn(ctx, desc, reason, details, swaps, changeReplicasTxnArgs{ db: r.store.DB(), - liveAndDeadReplicas: r.store.allocator.storePool.liveAndDeadReplicas, + liveAndDeadReplicas: r.store.cfg.StorePool.LiveAndDeadReplicas, logChange: r.store.logChange, testForceJointConfig: r.store.TestingKnobs().ReplicationAlwaysUseJointConfig, testAllowDangerousReplicationChanges: r.store.TestingKnobs().AllowDangerousReplicationChanges, @@ -1129,7 +1130,7 @@ func (r *Replica) changeReplicasImpl( desc, err = execChangeReplicasTxn(ctx, desc, reason, details, iChgs, changeReplicasTxnArgs{ db: r.store.DB(), - liveAndDeadReplicas: r.store.allocator.storePool.liveAndDeadReplicas, + liveAndDeadReplicas: r.store.cfg.StorePool.LiveAndDeadReplicas, logChange: r.store.logChange, testForceJointConfig: r.store.TestingKnobs().ReplicationAlwaysUseJointConfig, testAllowDangerousReplicationChanges: r.store.TestingKnobs().AllowDangerousReplicationChanges, @@ -1271,7 +1272,7 @@ func (r *Replica) maybeLeaveAtomicChangeReplicas( ctx, desc, kvserverpb.ReasonUnknown /* unused */, "", nil, /* iChgs */ changeReplicasTxnArgs{ db: s.DB(), - liveAndDeadReplicas: s.allocator.storePool.liveAndDeadReplicas, + liveAndDeadReplicas: s.cfg.StorePool.LiveAndDeadReplicas, logChange: s.logChange, testForceJointConfig: s.TestingKnobs().ReplicationAlwaysUseJointConfig, testAllowDangerousReplicationChanges: s.TestingKnobs().AllowDangerousReplicationChanges, @@ -1287,7 +1288,7 @@ func (r *Replica) TestingRemoveLearner( []internalReplicationChange{{target: target, typ: internalChangeTypeRemoveLearner}}, changeReplicasTxnArgs{ db: r.store.DB(), - liveAndDeadReplicas: r.store.allocator.storePool.liveAndDeadReplicas, + liveAndDeadReplicas: r.store.cfg.StorePool.LiveAndDeadReplicas, logChange: r.store.logChange, testForceJointConfig: r.store.TestingKnobs().ReplicationAlwaysUseJointConfig, testAllowDangerousReplicationChanges: r.store.TestingKnobs().AllowDangerousReplicationChanges, @@ -1337,7 +1338,7 @@ func (r *Replica) maybeLeaveAtomicChangeReplicasAndRemoveLearners( ctx, desc, kvserverpb.ReasonAbandonedLearner, "", []internalReplicationChange{{target: target, typ: internalChangeTypeRemoveLearner}}, changeReplicasTxnArgs{db: store.DB(), - liveAndDeadReplicas: store.allocator.storePool.liveAndDeadReplicas, + liveAndDeadReplicas: store.cfg.StorePool.LiveAndDeadReplicas, logChange: store.logChange, testForceJointConfig: store.TestingKnobs().ReplicationAlwaysUseJointConfig, testAllowDangerousReplicationChanges: store.TestingKnobs().AllowDangerousReplicationChanges, @@ -1710,7 +1711,7 @@ func (r *Replica) initializeRaftLearners( desc, err = execChangeReplicasTxn( ctx, desc, reason, details, iChgs, changeReplicasTxnArgs{ db: r.store.DB(), - liveAndDeadReplicas: r.store.allocator.storePool.liveAndDeadReplicas, + liveAndDeadReplicas: r.store.cfg.StorePool.LiveAndDeadReplicas, logChange: r.store.logChange, testForceJointConfig: r.store.TestingKnobs().ReplicationAlwaysUseJointConfig, testAllowDangerousReplicationChanges: r.store.TestingKnobs().AllowDangerousReplicationChanges, @@ -1864,7 +1865,7 @@ func (r *Replica) execReplicationChangesForVoters( desc, err = execChangeReplicasTxn(ctx, desc, reason, details, iChgs, changeReplicasTxnArgs{ db: r.store.DB(), - liveAndDeadReplicas: r.store.allocator.storePool.liveAndDeadReplicas, + liveAndDeadReplicas: r.store.cfg.StorePool.LiveAndDeadReplicas, logChange: r.store.logChange, testForceJointConfig: r.store.TestingKnobs().ReplicationAlwaysUseJointConfig, testAllowDangerousReplicationChanges: r.store.TestingKnobs().AllowDangerousReplicationChanges, @@ -1922,7 +1923,7 @@ func (r *Replica) tryRollbackRaftLearner( []internalReplicationChange{{target: target, typ: removeChgType}}, changeReplicasTxnArgs{ db: r.store.DB(), - liveAndDeadReplicas: r.store.allocator.storePool.liveAndDeadReplicas, + liveAndDeadReplicas: r.store.cfg.StorePool.LiveAndDeadReplicas, logChange: r.store.logChange, testForceJointConfig: r.store.TestingKnobs().ReplicationAlwaysUseJointConfig, testAllowDangerousReplicationChanges: r.store.TestingKnobs().AllowDangerousReplicationChanges, @@ -2643,7 +2644,7 @@ func (r *Replica) sendSnapshot( ctx, "send-snapshot", sendSnapshotTimeout, func(ctx context.Context) error { return r.store.cfg.Transport.SendSnapshot( ctx, - r.store.allocator.storePool, + r.store.cfg.StorePool, req, snap, newBatchFn, @@ -3013,7 +3014,7 @@ func (r *Replica) relocateOne( return nil, nil, err } - storeList, _, _ := r.store.allocator.storePool.getStoreList(storeFilterNone) + storeList, _, _ := r.store.cfg.StorePool.GetStoreList(storepool.StoreFilterNone) storeMap := storeListToMap(storeList) // Compute which replica to add and/or remove, respectively. We then ask the @@ -3052,7 +3053,7 @@ func (r *Replica) relocateOne( } candidateDescs = append(candidateDescs, *store) } - candidateStoreList := makeStoreList(candidateDescs) + candidateStoreList := storepool.MakeStoreList(candidateDescs) additionTarget, _ = r.store.allocator.allocateTargetFromList( ctx, diff --git a/pkg/kv/kvserver/replica_init.go b/pkg/kv/kvserver/replica_init.go index 82c3f20dff94..2717407b3bfe 100644 --- a/pkg/kv/kvserver/replica_init.go +++ b/pkg/kv/kvserver/replica_init.go @@ -110,8 +110,8 @@ func newUnloadedReplica( r.leaseHistory = newLeaseHistory() } if store.cfg.StorePool != nil { - r.leaseholderStats = newReplicaStats(store.Clock(), store.cfg.StorePool.getNodeLocalityString) - r.loadStats = newReplicaLoad(store.Clock(), store.cfg.StorePool.getNodeLocalityString) + r.leaseholderStats = newReplicaStats(store.Clock(), store.cfg.StorePool.GetNodeLocalityString) + r.loadStats = newReplicaLoad(store.Clock(), store.cfg.StorePool.GetNodeLocalityString) } // Pass nil for the localityOracle because we intentionally don't track the // origin locality of write load. diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index 3177f5080b9a..7ac448b40ed0 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/gossip" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -430,7 +431,7 @@ func (rq *replicateQueue) shouldQueue( voterReplicas, nonVoterReplicas, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, rq.allocator.scorerOptions(ctx), ) if ok { @@ -444,7 +445,7 @@ func (rq *replicateQueue) shouldQueue( voterReplicas, nonVoterReplicas, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, rq.allocator.scorerOptions(ctx), ) if ok { @@ -544,10 +545,10 @@ func (rq *replicateQueue) processOneChange( voterReplicas := desc.Replicas().VoterDescriptors() nonVoterReplicas := desc.Replicas().NonVoterDescriptors() - liveVoterReplicas, deadVoterReplicas := rq.allocator.storePool.liveAndDeadReplicas( + liveVoterReplicas, deadVoterReplicas := rq.store.cfg.StorePool.LiveAndDeadReplicas( voterReplicas, true, /* includeSuspectAndDrainingStores */ ) - liveNonVoterReplicas, deadNonVoterReplicas := rq.allocator.storePool.liveAndDeadReplicas( + liveNonVoterReplicas, deadNonVoterReplicas := rq.store.cfg.StorePool.LiveAndDeadReplicas( nonVoterReplicas, true, /* includeSuspectAndDrainingStores */ ) @@ -618,7 +619,7 @@ func (rq *replicateQueue) processOneChange( // Replace decommissioning replicas. case AllocatorReplaceDecommissioningVoter: - decommissioningVoterReplicas := rq.allocator.storePool.decommissioningReplicas(voterReplicas) + decommissioningVoterReplicas := rq.store.cfg.StorePool.DecommissioningReplicas(voterReplicas) if len(decommissioningVoterReplicas) == 0 { // Nothing to do. return false, nil @@ -632,7 +633,7 @@ func (rq *replicateQueue) processOneChange( return rq.addOrReplaceVoters( ctx, repl, liveVoterReplicas, liveNonVoterReplicas, removeIdx, decommissioning, dryRun) case AllocatorReplaceDecommissioningNonVoter: - decommissioningNonVoterReplicas := rq.allocator.storePool.decommissioningReplicas(nonVoterReplicas) + decommissioningNonVoterReplicas := rq.store.cfg.StorePool.DecommissioningReplicas(nonVoterReplicas) if len(decommissioningNonVoterReplicas) == 0 { return false, nil } @@ -769,7 +770,7 @@ func (rq *replicateQueue) addOrReplaceVoters( return false, errors.AssertionFailedf("allocator suggested to replace replica on s%d with itself", newVoter.StoreID) } - clusterNodes := rq.allocator.storePool.ClusterNodeCount() + clusterNodes := rq.store.cfg.StorePool.ClusterNodeCount() neededVoters := GetNeededVoters(conf.GetNumVoters(), clusterNodes) // Only up-replicate if there are suitable allocation targets such that, @@ -1156,11 +1157,11 @@ func (rq *replicateQueue) removeDecommissioning( var decommissioningReplicas []roachpb.ReplicaDescriptor switch targetType { case voterTarget: - decommissioningReplicas = rq.allocator.storePool.decommissioningReplicas( + decommissioningReplicas = rq.store.cfg.StorePool.DecommissioningReplicas( desc.Replicas().VoterDescriptors(), ) case nonVoterTarget: - decommissioningReplicas = rq.allocator.storePool.decommissioningReplicas( + decommissioningReplicas = rq.store.cfg.StorePool.DecommissioningReplicas( desc.Replicas().NonVoterDescriptors(), ) default: @@ -1314,7 +1315,7 @@ func (rq *replicateQueue) considerRebalance( existingVoters, existingNonVoters, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, scorerOpts, ) if !ok { @@ -1328,7 +1329,7 @@ func (rq *replicateQueue) considerRebalance( existingVoters, existingNonVoters, rangeUsageInfo, - storeFilterThrottled, + storepool.StoreFilterThrottled, scorerOpts, ) rebalanceTargetType = nonVoterTarget @@ -1614,7 +1615,7 @@ func (rq *replicateQueue) transferLease( return errors.Wrapf(err, "%s: unable to transfer lease to s%d", repl, target.StoreID) } rq.lastLeaseTransfer.Store(timeutil.Now()) - rq.allocator.storePool.updateLocalStoresAfterLeaseTransfer( + rq.store.cfg.StorePool.UpdateLocalStoresAfterLeaseTransfer( repl.store.StoreID(), target.StoreID, rangeQPS) return nil } @@ -1640,7 +1641,7 @@ func (rq *replicateQueue) changeReplicas( } rangeUsageInfo := rangeUsageInfoForRepl(repl) for _, chg := range chgs { - rq.allocator.storePool.updateLocalStoreAfterRebalance( + rq.store.cfg.StorePool.UpdateLocalStoreAfterRebalance( chg.Target.StoreID, rangeUsageInfo, chg.ChangeType) } return nil diff --git a/pkg/kv/kvserver/reports/BUILD.bazel b/pkg/kv/kvserver/reports/BUILD.bazel index d959e20ecfa1..dfbaafc7d3d4 100644 --- a/pkg/kv/kvserver/reports/BUILD.bazel +++ b/pkg/kv/kvserver/reports/BUILD.bazel @@ -18,6 +18,7 @@ go_library( "//pkg/keys", "//pkg/kv", "//pkg/kv/kvserver", + "//pkg/kv/kvserver/allocator/storepool", "//pkg/kv/kvserver/liveness", "//pkg/roachpb", "//pkg/security", diff --git a/pkg/kv/kvserver/reports/reporter.go b/pkg/kv/kvserver/reports/reporter.go index eeafc4887b1a..22af19f74384 100644 --- a/pkg/kv/kvserver/reports/reporter.go +++ b/pkg/kv/kvserver/reports/reporter.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" @@ -71,7 +72,7 @@ type Reporter struct { db *kv.DB liveness *liveness.NodeLiveness settings *cluster.Settings - storePool *kvserver.StorePool + storePool *storepool.StorePool executor sqlutil.InternalExecutor cfgs config.SystemConfigProvider @@ -86,7 +87,7 @@ type Reporter struct { func NewReporter( db *kv.DB, localStores *kvserver.Stores, - storePool *kvserver.StorePool, + storePool *storepool.StorePool, st *cluster.Settings, liveness *liveness.NodeLiveness, executor sqlutil.InternalExecutor, diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index d5d92a30970d..e70f943dcd7e 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -33,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/sidetransport" @@ -967,7 +968,7 @@ type StoreConfig struct { DB *kv.DB Gossip *gossip.Gossip NodeLiveness *liveness.NodeLiveness - StorePool *StorePool + StorePool *storepool.StorePool Transport *RaftTransport NodeDialer *nodedialer.Dialer RPCContext *rpc.Context diff --git a/pkg/kv/kvserver/store_rebalancer.go b/pkg/kv/kvserver/store_rebalancer.go index 2079287cb4a2..6b9ffffc92f4 100644 --- a/pkg/kv/kvserver/store_rebalancer.go +++ b/pkg/kv/kvserver/store_rebalancer.go @@ -16,6 +16,7 @@ import ( "math/rand" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -238,7 +239,7 @@ func (sr *StoreRebalancer) Start(ctx context.Context, stopper *stop.Stopper) { continue } - storeList, _, _ := sr.rq.allocator.storePool.getStoreList(storeFilterSuspect) + storeList, _, _ := sr.rq.store.cfg.StorePool.GetStoreList(storepool.StoreFilterSuspect) sr.rebalanceStore(ctx, mode, storeList) } }) @@ -252,7 +253,7 @@ func (sr *StoreRebalancer) Start(ctx context.Context, stopper *stop.Stopper) { func (sr *StoreRebalancer) scorerOptions(ctx context.Context) *qpsScorerOptions { return &qpsScorerOptions{ storeHealthOptions: sr.rq.allocator.storeHealthOptions(ctx), - deterministic: sr.rq.allocator.storePool.deterministic, + deterministic: sr.rq.store.cfg.StorePool.Deterministic, qpsRebalanceThreshold: qpsRebalanceThreshold.Get(&sr.st.SV), minRequiredQPSDiff: minQPSDifferenceForTransfers.Get(&sr.st.SV), } @@ -269,13 +270,13 @@ func (sr *StoreRebalancer) scorerOptions(ctx context.Context) *qpsScorerOptions // of all the stores in the cluster. Is this desirable? Should we be more // aggressive? func (sr *StoreRebalancer) rebalanceStore( - ctx context.Context, mode LBRebalancingMode, allStoresList StoreList, + ctx context.Context, mode LBRebalancingMode, allStoresList storepool.StoreList, ) { options := sr.scorerOptions(ctx) var localDesc *roachpb.StoreDescriptor - for i := range allStoresList.stores { - if allStoresList.stores[i].StoreID == sr.rq.store.StoreID() { - localDesc = &allStoresList.stores[i] + for i := range allStoresList.Stores { + if allStoresList.Stores[i].StoreID == sr.rq.store.StoreID() { + localDesc = &allStoresList.Stores[i] break } } @@ -286,10 +287,10 @@ func (sr *StoreRebalancer) rebalanceStore( // We only bother rebalancing stores that are fielding more than the // cluster-level overfull threshold of QPS. - qpsMaxThreshold := overfullQPSThreshold(options, allStoresList.candidateQueriesPerSecond.mean) + qpsMaxThreshold := overfullQPSThreshold(options, allStoresList.CandidateQueriesPerSecond.Mean) if !(localDesc.Capacity.QueriesPerSecond > qpsMaxThreshold) { log.Infof(ctx, "local QPS %.2f is below max threshold %.2f (mean=%.2f); no rebalancing needed", - localDesc.Capacity.QueriesPerSecond, qpsMaxThreshold, allStoresList.candidateQueriesPerSecond.mean) + localDesc.Capacity.QueriesPerSecond, qpsMaxThreshold, allStoresList.CandidateQueriesPerSecond.Mean) return } @@ -299,7 +300,7 @@ func (sr *StoreRebalancer) rebalanceStore( // First check if we should transfer leases away to better balance QPS. log.Infof(ctx, "considering load-based lease transfers for s%d with %.2f qps (mean=%.2f, upperThreshold=%.2f)", - localDesc.StoreID, localDesc.Capacity.QueriesPerSecond, allStoresList.candidateQueriesPerSecond.mean, qpsMaxThreshold) + localDesc.StoreID, localDesc.Capacity.QueriesPerSecond, allStoresList.CandidateQueriesPerSecond.Mean, qpsMaxThreshold) hottestRanges := sr.replRankings.topQPS() for localDesc.Capacity.QueriesPerSecond > qpsMaxThreshold { replWithStats, target, considerForRebalance := sr.chooseLeaseToTransfer( @@ -337,7 +338,7 @@ func (sr *StoreRebalancer) rebalanceStore( if !(localDesc.Capacity.QueriesPerSecond > qpsMaxThreshold) { log.Infof(ctx, "load-based lease transfers successfully brought s%d down to %.2f qps (mean=%.2f, upperThreshold=%.2f)", - localDesc.StoreID, localDesc.Capacity.QueriesPerSecond, allStoresList.candidateQueriesPerSecond.mean, qpsMaxThreshold) + localDesc.StoreID, localDesc.Capacity.QueriesPerSecond, allStoresList.CandidateQueriesPerSecond.Mean, qpsMaxThreshold) return } @@ -426,14 +427,14 @@ func (sr *StoreRebalancer) rebalanceStore( log.Infof(ctx, "load-based replica transfers successfully brought s%d down to %.2f qps (mean=%.2f, upperThreshold=%.2f)", - localDesc.StoreID, localDesc.Capacity.QueriesPerSecond, allStoresList.candidateQueriesPerSecond.mean, qpsMaxThreshold) + localDesc.StoreID, localDesc.Capacity.QueriesPerSecond, allStoresList.CandidateQueriesPerSecond.Mean, qpsMaxThreshold) } func (sr *StoreRebalancer) chooseLeaseToTransfer( ctx context.Context, hottestRanges *[]replicaWithStats, localDesc *roachpb.StoreDescriptor, - storeList StoreList, + storeList storepool.StoreList, storeMap map[roachpb.StoreID]*roachpb.StoreDescriptor, ) (replicaWithStats, roachpb.ReplicaDescriptor, []replicaWithStats) { var considerForRebalance []replicaWithStats @@ -503,8 +504,8 @@ func (sr *StoreRebalancer) chooseLeaseToTransfer( continue } - filteredStoreList := storeList.excludeInvalid(conf.Constraints) - filteredStoreList = storeList.excludeInvalid(conf.VoterConstraints) + filteredStoreList := storeList.ExcludeInvalid(conf.Constraints) + filteredStoreList = storeList.ExcludeInvalid(conf.VoterConstraints) if sr.rq.allocator.followTheWorkloadPrefersLocal( ctx, filteredStoreList, @@ -550,7 +551,7 @@ func (sr *StoreRebalancer) chooseRangeToRebalance( ctx context.Context, hottestRanges *[]replicaWithStats, localDesc *roachpb.StoreDescriptor, - allStoresList StoreList, + allStoresList storepool.StoreList, options *qpsScorerOptions, ) (replWithStats replicaWithStats, voterTargets, nonVoterTargets []roachpb.ReplicationTarget) { now := sr.rq.store.Clock().NowAsClockTimestamp() @@ -713,7 +714,7 @@ func (sr *StoreRebalancer) getRebalanceTargetsBasedOnQPS( finalVoterTargets, finalNonVoterTargets, rangeUsageInfoForRepl(rbCtx.replWithStats.repl), - storeFilterSuspect, + storepool.StoreFilterSuspect, voterTarget, options, ) @@ -777,7 +778,7 @@ func (sr *StoreRebalancer) getRebalanceTargetsBasedOnQPS( finalVoterTargets, finalNonVoterTargets, rangeUsageInfoForRepl(rbCtx.replWithStats.repl), - storeFilterSuspect, + storepool.StoreFilterSuspect, nonVoterTarget, options, ) @@ -820,10 +821,10 @@ func (sr *StoreRebalancer) getRebalanceTargetsBasedOnQPS( return finalVoterTargets, finalNonVoterTargets, foundRebalance } -func storeListToMap(sl StoreList) map[roachpb.StoreID]*roachpb.StoreDescriptor { +func storeListToMap(sl storepool.StoreList) map[roachpb.StoreID]*roachpb.StoreDescriptor { storeMap := make(map[roachpb.StoreID]*roachpb.StoreDescriptor) - for i := range sl.stores { - storeMap[sl.stores[i].StoreID] = &sl.stores[i] + for i := range sl.Stores { + storeMap[sl.Stores[i].StoreID] = &sl.Stores[i] } return storeMap } diff --git a/pkg/kv/kvserver/store_rebalancer_test.go b/pkg/kv/kvserver/store_rebalancer_test.go index 8f43daa5fa40..6e962c30ea1e 100644 --- a/pkg/kv/kvserver/store_rebalancer_test.go +++ b/pkg/kv/kvserver/store_rebalancer_test.go @@ -18,6 +18,7 @@ import ( "strings" "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils/gossiputil" @@ -493,7 +494,7 @@ func TestChooseLeaseToTransfer(t *testing.T) { ) defer stopper.Stop(context.Background()) gossiputil.NewStoreGossiper(g).GossipStores(noLocalityStores, t) - storeList, _, _ := a.storePool.getStoreList(storeFilterThrottled) + storeList, _, _ := a.storePool.GetStoreList(storepool.StoreFilterThrottled) storeMap := storeListToMap(storeList) localDesc := *noLocalityStores[0] cfg := TestStoreConfig(nil) @@ -766,7 +767,7 @@ func TestChooseRangeToRebalanceRandom(t *testing.T) { // Test setup boilerplate. gossiputil.NewStoreGossiper(g).GossipStores(stores, t) - storeList, _, _ := a.storePool.getStoreList(storeFilterThrottled) + storeList, _, _ := a.storePool.GetStoreList(storepool.StoreFilterThrottled) localDesc := *stores[0] cfg := TestStoreConfig(nil) s := createTestStoreWithoutStart(ctx, t, stopper, testStoreOpts{createSystemRanges: true}, &cfg) @@ -791,7 +792,7 @@ func TestChooseRangeToRebalanceRandom(t *testing.T) { } return status } - a.storePool.isStoreReadyForRoutineReplicaTransfer = func(_ context.Context, this roachpb.StoreID) bool { + a.storePool.IsStoreReadyForRoutineReplicaTransfer = func(_ context.Context, this roachpb.StoreID) bool { for _, deadStore := range deadStores { // NodeID match StoreIDs here, so this comparison is valid. if deadStore.StoreID == this { @@ -1036,7 +1037,7 @@ func TestChooseRangeToRebalanceAcrossHeterogeneousZones(t *testing.T) { stopper, g, _, a, _ := createTestAllocator(ctx, 10, false /* deterministic */) defer stopper.Stop(context.Background()) gossiputil.NewStoreGossiper(g).GossipStores(multiRegionStores, t) - storeList, _, _ := a.storePool.getStoreList(storeFilterThrottled) + storeList, _, _ := a.storePool.GetStoreList(storepool.StoreFilterThrottled) var localDesc roachpb.StoreDescriptor for _, store := range multiRegionStores { @@ -1140,7 +1141,7 @@ func TestChooseRangeToRebalanceIgnoresRangeOnBestStores(t *testing.T) { &AllocatorTestingKnobs{AllowLeaseTransfersToReplicasNeedingSnapshots: true}, ) defer stopper.Stop(context.Background()) - storeList, _, _ := a.storePool.getStoreList(storeFilterThrottled) + storeList, _, _ := a.storePool.GetStoreList(storepool.StoreFilterThrottled) localDesc := *noLocalityStores[len(noLocalityStores)-1] cfg := TestStoreConfig(nil) @@ -1283,7 +1284,7 @@ func TestChooseRangeToRebalanceOffHotNodes(t *testing.T) { stopper, g, _, a, _ := createTestAllocator(ctx, 10, false /* deterministic */) defer stopper.Stop(context.Background()) gossiputil.NewStoreGossiper(g).GossipStores(imbalancedStores, t) - storeList, _, _ := a.storePool.getStoreList(storeFilterThrottled) + storeList, _, _ := a.storePool.GetStoreList(storepool.StoreFilterThrottled) var localDesc roachpb.StoreDescriptor for _, store := range imbalancedStores { @@ -1363,7 +1364,7 @@ func TestNoLeaseTransferToBehindReplicas(t *testing.T) { &AllocatorTestingKnobs{AllowLeaseTransfersToReplicasNeedingSnapshots: true}, ) defer stopper.Stop(context.Background()) - storeList, _, _ := a.storePool.getStoreList(storeFilterThrottled) + storeList, _, _ := a.storePool.GetStoreList(storepool.StoreFilterThrottled) storeMap := storeListToMap(storeList) localDesc := *noLocalityStores[0] @@ -1568,7 +1569,7 @@ func TestStoreRebalancerReadAmpCheck(t *testing.T) { t.Run(fmt.Sprintf("%d_%s", i+1, test.name), func(t *testing.T) { stopper, g, _, a, _ := createTestAllocator(ctx, 10, false /* deterministic */) defer stopper.Stop(ctx) - storeList, _, _ := a.storePool.getStoreList(storeFilterThrottled) + storeList, _, _ := a.storePool.GetStoreList(storepool.StoreFilterThrottled) localDesc := *noLocalityStores[0] cfg := TestStoreConfig(nil) diff --git a/pkg/kv/kvserver/store_snapshot.go b/pkg/kv/kvserver/store_snapshot.go index 1d7fba36525e..b43153e9e711 100644 --- a/pkg/kv/kvserver/store_snapshot.go +++ b/pkg/kv/kvserver/store_snapshot.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftentry" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" @@ -751,7 +752,7 @@ func sendSnapshotError(stream incomingSnapshotStream, err error) error { // SnapshotStorePool narrows StorePool to make sendSnapshot easier to test. type SnapshotStorePool interface { - throttle(reason throttleReason, why string, toStoreID roachpb.StoreID) + Throttle(reason storepool.ThrottleReason, why string, toStoreID roachpb.StoreID) } // minSnapshotRate defines the minimum value that the rate limit for rebalance @@ -1099,7 +1100,7 @@ func SendEmptySnapshot( // noopStorePool is a hollowed out StorePool that does not throttle. It's used in recovery scenarios. type noopStorePool struct{} -func (n noopStorePool) throttle(throttleReason, string, roachpb.StoreID) {} +func (n noopStorePool) Throttle(storepool.ThrottleReason, string, roachpb.StoreID) {} // sendSnapshot sends an outgoing snapshot via a pre-opened GRPC stream. func sendSnapshot( @@ -1130,12 +1131,12 @@ func sendSnapshot( // the actual snapshot (if not rejected). resp, err := stream.Recv() if err != nil { - storePool.throttle(throttleFailed, err.Error(), to.StoreID) + storePool.Throttle(storepool.ThrottleFailed, err.Error(), to.StoreID) return err } switch resp.Status { case kvserverpb.SnapshotResponse_ERROR: - storePool.throttle(throttleFailed, resp.Message, to.StoreID) + storePool.Throttle(storepool.ThrottleFailed, resp.Message, to.StoreID) return errors.Errorf("%s: remote couldn't accept %s with error: %s", to, snap, resp.Message) case kvserverpb.SnapshotResponse_ACCEPTED: @@ -1143,7 +1144,7 @@ func sendSnapshot( default: err := errors.Errorf("%s: server sent an invalid status while negotiating %s: %s", to, snap, resp.Status) - storePool.throttle(throttleFailed, err.Error(), to.StoreID) + storePool.Throttle(storepool.ThrottleFailed, err.Error(), to.StoreID) return err } diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index 5fc02a640d0a..aa345f03be90 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -32,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" @@ -2776,9 +2777,11 @@ type fakeStorePool struct { failedThrottles int } -func (sp *fakeStorePool) throttle(reason throttleReason, why string, toStoreID roachpb.StoreID) { +func (sp *fakeStorePool) Throttle( + reason storepool.ThrottleReason, why string, toStoreID roachpb.StoreID, +) { switch reason { - case throttleFailed: + case storepool.ThrottleFailed: sp.failedThrottles++ default: panic("unknown reason") @@ -2921,9 +2924,9 @@ func TestReserveSnapshotFullnessLimit(t *testing.T) { desc.Capacity.Available = 1 desc.Capacity.Used = desc.Capacity.Capacity - desc.Capacity.Available - s.cfg.StorePool.detailsMu.Lock() - s.cfg.StorePool.getStoreDetailLocked(desc.StoreID).desc = desc - s.cfg.StorePool.detailsMu.Unlock() + s.cfg.StorePool.DetailsMu.Lock() + s.cfg.StorePool.GetStoreDetailLocked(desc.StoreID).Desc = desc + s.cfg.StorePool.DetailsMu.Unlock() if n := s.ReservationCount(); n != 0 { t.Fatalf("expected 0 reservations, but found %d", n) @@ -2945,9 +2948,9 @@ func TestReserveSnapshotFullnessLimit(t *testing.T) { // available disk space should be rejected. desc.Capacity.Available = desc.Capacity.Capacity / 2 desc.Capacity.Used = desc.Capacity.Capacity - desc.Capacity.Available - s.cfg.StorePool.detailsMu.Lock() - s.cfg.StorePool.getStoreDetailLocked(desc.StoreID).desc = desc - s.cfg.StorePool.detailsMu.Unlock() + s.cfg.StorePool.DetailsMu.Lock() + s.cfg.StorePool.GetStoreDetailLocked(desc.StoreID).Desc = desc + s.cfg.StorePool.DetailsMu.Unlock() if n := s.ReservationCount(); n != 0 { t.Fatalf("expected 0 reservations, but found %d", n) diff --git a/pkg/kv/kvserver/testing_knobs.go b/pkg/kv/kvserver/testing_knobs.go index 70380c39e26d..84baba3353c1 100644 --- a/pkg/kv/kvserver/testing_knobs.go +++ b/pkg/kv/kvserver/testing_knobs.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/tenantrate" @@ -409,7 +410,7 @@ type NodeLivenessTestingKnobs struct { RenewalDuration time.Duration // StorePoolNodeLivenessFn is the function used by the StorePool to determine // whether a node is live or not. - StorePoolNodeLivenessFn NodeLivenessFunc + StorePoolNodeLivenessFn storepool.NodeLivenessFunc } var _ base.ModuleTestingKnobs = NodeLivenessTestingKnobs{} diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 1278d83c9c53..749d4d0975b9 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -95,6 +95,7 @@ go_library( "//pkg/kv/kvclient/rangefeed", "//pkg/kv/kvprober", "//pkg/kv/kvserver", + "//pkg/kv/kvserver/allocator/storepool", "//pkg/kv/kvserver/closedts/ctpb", "//pkg/kv/kvserver/closedts/sidetransport", "//pkg/kv/kvserver/kvserverbase", diff --git a/pkg/server/admin.go b/pkg/server/admin.go index 0b0a1188c983..39872eb9861d 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -31,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" @@ -1936,11 +1937,11 @@ func getLivenessStatusMap( nl *liveness.NodeLiveness, now time.Time, st *cluster.Settings, ) map[roachpb.NodeID]livenesspb.NodeLivenessStatus { livenesses := nl.GetLivenesses() - threshold := kvserver.TimeUntilStoreDead.Get(&st.SV) + threshold := storepool.TimeUntilStoreDead.Get(&st.SV) statusMap := make(map[roachpb.NodeID]livenesspb.NodeLivenessStatus, len(livenesses)) for _, liveness := range livenesses { - status := kvserver.LivenessStatus(liveness, now, threshold) + status := storepool.LivenessStatus(liveness, now, threshold) statusMap[liveness.NodeID] = status } return statusMap diff --git a/pkg/server/server.go b/pkg/server/server.go index 7a310c1be064..7eb86757669e 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -32,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" "github.com/cockroachdb/cockroach/pkg/kv/kvprober" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/sidetransport" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" @@ -103,7 +104,7 @@ type Server struct { gossip *gossip.Gossip nodeDialer *nodedialer.Dialer nodeLiveness *liveness.NodeLiveness - storePool *kvserver.StorePool + storePool *storepool.StorePool tcsFactory *kvcoord.TxnCoordSenderFactory distSender *kvcoord.DistSender db *kv.DB @@ -416,12 +417,12 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { }) registry.AddMetricStruct(nodeLiveness.Metrics()) - nodeLivenessFn := kvserver.MakeStorePoolNodeLivenessFunc(nodeLiveness) + nodeLivenessFn := storepool.MakeStorePoolNodeLivenessFunc(nodeLiveness) if nodeLivenessKnobs, ok := cfg.TestingKnobs.NodeLiveness.(kvserver.NodeLivenessTestingKnobs); ok && nodeLivenessKnobs.StorePoolNodeLivenessFn != nil { nodeLivenessFn = nodeLivenessKnobs.StorePoolNodeLivenessFn } - storePool := kvserver.NewStorePool( + storePool := storepool.NewStorePool( cfg.AmbientCtx, st, g, diff --git a/pkg/server/status.go b/pkg/server/status.go index 0550a00d3bd9..1cfc128f07c8 100644 --- a/pkg/server/status.go +++ b/pkg/server/status.go @@ -39,6 +39,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -452,7 +453,7 @@ type statusServer struct { gossip *gossip.Gossip metricSource metricMarshaler nodeLiveness *liveness.NodeLiveness - storePool *kvserver.StorePool + storePool *storepool.StorePool stores *kvserver.Stores si systemInfoOnce stmtDiagnosticsRequester StmtDiagnosticsRequester @@ -495,7 +496,7 @@ func newStatusServer( gossip *gossip.Gossip, metricSource metricMarshaler, nodeLiveness *liveness.NodeLiveness, - storePool *kvserver.StorePool, + storePool *storepool.StorePool, rpcCtx *rpc.Context, stores *kvserver.Stores, stopper *stop.Stopper, diff --git a/pkg/testutils/localtestcluster/BUILD.bazel b/pkg/testutils/localtestcluster/BUILD.bazel index 1d9fd8487f76..bfb56f78fc65 100644 --- a/pkg/testutils/localtestcluster/BUILD.bazel +++ b/pkg/testutils/localtestcluster/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "//pkg/kv", "//pkg/kv/kvclient/rangefeed", "//pkg/kv/kvserver", + "//pkg/kv/kvserver/allocator/storepool", "//pkg/kv/kvserver/closedts/sidetransport", "//pkg/kv/kvserver/liveness", "//pkg/roachpb", diff --git a/pkg/testutils/localtestcluster/local_test_cluster.go b/pkg/testutils/localtestcluster/local_test_cluster.go index ff81edf609da..58cf0d645f5f 100644 --- a/pkg/testutils/localtestcluster/local_test_cluster.go +++ b/pkg/testutils/localtestcluster/local_test_cluster.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/sidetransport" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -191,14 +192,14 @@ func (ltc *LocalTestCluster) Start(t testing.TB, baseCtx *base.Config, initFacto Settings: cfg.Settings, HistogramWindowInterval: cfg.HistogramWindowInterval, }) - kvserver.TimeUntilStoreDead.Override(ctx, &cfg.Settings.SV, kvserver.TestTimeUntilStoreDead) - cfg.StorePool = kvserver.NewStorePool( + storepool.TimeUntilStoreDead.Override(ctx, &cfg.Settings.SV, storepool.TestTimeUntilStoreDead) + cfg.StorePool = storepool.NewStorePool( cfg.AmbientCtx, cfg.Settings, cfg.Gossip, cfg.Clock, cfg.NodeLiveness.GetNodeCount, - kvserver.MakeStorePoolNodeLivenessFunc(cfg.NodeLiveness), + storepool.MakeStorePoolNodeLivenessFunc(cfg.NodeLiveness), /* deterministic */ false, ) cfg.Transport = transport diff --git a/pkg/ts/BUILD.bazel b/pkg/ts/BUILD.bazel index d7f9ab17f7f2..7ebb945b8384 100644 --- a/pkg/ts/BUILD.bazel +++ b/pkg/ts/BUILD.bazel @@ -23,6 +23,7 @@ go_library( "//pkg/keys", "//pkg/kv", "//pkg/kv/kvserver", + "//pkg/kv/kvserver/allocator/storepool", "//pkg/roachpb", "//pkg/settings", "//pkg/settings/cluster", diff --git a/pkg/ts/server.go b/pkg/ts/server.go index 2e76d50f42a3..a4c79f592552 100644 --- a/pkg/ts/server.go +++ b/pkg/ts/server.go @@ -15,7 +15,7 @@ import ( "math" "github.com/cockroachdb/cockroach/pkg/kv" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/ts/catalog" "github.com/cockroachdb/cockroach/pkg/ts/tspb" @@ -187,7 +187,7 @@ func (s *Server) Query( // dead. This is a conservatively long span, but gives us a good indication of // when a gap likely indicates an outage (and thus missing values should not // be interpolated). - interpolationLimit := kvserver.TimeUntilStoreDead.Get(&s.db.st.SV).Nanoseconds() + interpolationLimit := storepool.TimeUntilStoreDead.Get(&s.db.st.SV).Nanoseconds() // Get the estimated number of nodes on the cluster, used to compute more // accurate memory usage estimates. Set a minimum of 1 in order to avoid