mirror of
https://github.com/talent-plan/tinykv.git
synced 2025-01-27 12:51:05 +08:00
pick master change and add comment
Signed-off-by: Yang Keao <keao.yang@yahoo.com>
This commit is contained in:
parent
c477bc0115
commit
0eecd7db66
@ -278,6 +278,8 @@ func (c *RaftCluster) handleStoreHeartbeat(stats *schedulerpb.StoreStats) error
|
||||
|
||||
// processRegionHeartbeat updates the region information.
|
||||
func (c *RaftCluster) processRegionHeartbeat(region *core.RegionInfo) error {
|
||||
// Your Code Here(3C).
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -15,7 +15,6 @@ package core
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"math"
|
||||
"time"
|
||||
|
||||
"github.com/gogo/protobuf/proto"
|
||||
@ -221,16 +220,6 @@ func (s *StoreInfo) GetLastHeartbeatTS() time.Time {
|
||||
|
||||
const minWeight = 1e-6
|
||||
|
||||
// LeaderScore returns the store's leader score.
|
||||
func (s *StoreInfo) LeaderScore(delta int64) float64 {
|
||||
return float64(int64(s.GetLeaderCount())+delta) / math.Max(s.GetLeaderWeight(), minWeight)
|
||||
}
|
||||
|
||||
// RegionScore returns the store's region score.
|
||||
func (s *StoreInfo) RegionScore() float64 {
|
||||
return float64(s.GetRegionSize()) / math.Max(s.GetRegionWeight(), minWeight)
|
||||
}
|
||||
|
||||
// StorageSize returns store's used storage size reported from tikv.
|
||||
func (s *StoreInfo) StorageSize() uint64 {
|
||||
return s.GetUsedSize()
|
||||
@ -273,18 +262,6 @@ func (s *StoreInfo) ResourceSize(kind ResourceKind) int64 {
|
||||
}
|
||||
}
|
||||
|
||||
// ResourceScore returns score of leader/region in the store.
|
||||
func (s *StoreInfo) ResourceScore(scheduleKind ScheduleKind, delta int64) float64 {
|
||||
switch scheduleKind.Resource {
|
||||
case LeaderKind:
|
||||
return s.LeaderScore(delta)
|
||||
case RegionKind:
|
||||
return s.RegionScore()
|
||||
default:
|
||||
return 0
|
||||
}
|
||||
}
|
||||
|
||||
// ResourceWeight returns weight of leader/region in the score
|
||||
func (s *StoreInfo) ResourceWeight(kind ResourceKind) float64 {
|
||||
switch kind {
|
||||
|
@ -21,57 +21,6 @@ import (
|
||||
"github.com/pingcap-incubator/tinykv/scheduler/server/schedule/opt"
|
||||
)
|
||||
|
||||
// BalanceSelector selects source/target from store candidates based on their
|
||||
// resource scores.
|
||||
type BalanceSelector struct {
|
||||
kind core.ScheduleKind
|
||||
filters []filter.Filter
|
||||
}
|
||||
|
||||
// NewBalanceSelector creates a BalanceSelector instance.
|
||||
func NewBalanceSelector(kind core.ScheduleKind, filters []filter.Filter) *BalanceSelector {
|
||||
return &BalanceSelector{
|
||||
kind: kind,
|
||||
filters: filters,
|
||||
}
|
||||
}
|
||||
|
||||
// SelectSource selects the store that can pass all filters and has the maximal
|
||||
// resource score.
|
||||
func (s *BalanceSelector) SelectSource(opt opt.Options, stores []*core.StoreInfo, filters ...filter.Filter) *core.StoreInfo {
|
||||
filters = append(filters, s.filters...)
|
||||
var result *core.StoreInfo
|
||||
for _, store := range stores {
|
||||
if filter.Source(opt, store, filters) {
|
||||
continue
|
||||
}
|
||||
if result == nil ||
|
||||
result.ResourceScore(s.kind, 0) <
|
||||
store.ResourceScore(s.kind, 0) {
|
||||
result = store
|
||||
}
|
||||
}
|
||||
return result
|
||||
}
|
||||
|
||||
// SelectTarget selects the store that can pass all filters and has the minimal
|
||||
// resource score.
|
||||
func (s *BalanceSelector) SelectTarget(opt opt.Options, stores []*core.StoreInfo, filters ...filter.Filter) *core.StoreInfo {
|
||||
filters = append(filters, s.filters...)
|
||||
var result *core.StoreInfo
|
||||
for _, store := range stores {
|
||||
if filter.Target(opt, store, filters) {
|
||||
continue
|
||||
}
|
||||
if result == nil ||
|
||||
result.ResourceScore(s.kind, 0) >
|
||||
store.ResourceScore(s.kind, 0) {
|
||||
result = store
|
||||
}
|
||||
}
|
||||
return result
|
||||
}
|
||||
|
||||
// ReplicaSelector selects source/target store candidates based on their
|
||||
// distinct scores based on a region's peer stores.
|
||||
type ReplicaSelector struct {
|
||||
@ -130,12 +79,12 @@ func (s *ReplicaSelector) SelectTarget(opt opt.Options, stores []*core.StoreInfo
|
||||
// Returns -1 if store B is better than store A.
|
||||
func compareStoreScore(storeA *core.StoreInfo, storeB *core.StoreInfo) int {
|
||||
// The store with lower region score is better.
|
||||
if storeA.RegionScore() <
|
||||
storeB.RegionScore() {
|
||||
if storeA.GetRegionSize() <
|
||||
storeB.GetRegionSize() {
|
||||
return 1
|
||||
}
|
||||
if storeA.RegionScore() >
|
||||
storeB.RegionScore() {
|
||||
if storeA.GetRegionSize() >
|
||||
storeB.GetRegionSize() {
|
||||
return -1
|
||||
}
|
||||
return 0
|
||||
|
@ -19,7 +19,6 @@ import (
|
||||
"github.com/pingcap-incubator/tinykv/scheduler/pkg/mock/mockcluster"
|
||||
"github.com/pingcap-incubator/tinykv/scheduler/pkg/mock/mockoption"
|
||||
"github.com/pingcap-incubator/tinykv/scheduler/server/core"
|
||||
"github.com/pingcap-incubator/tinykv/scheduler/server/schedule/filter"
|
||||
. "github.com/pingcap/check"
|
||||
)
|
||||
|
||||
@ -47,37 +46,3 @@ func (s *testSelectorSuite) TestCompareStoreScore(c *C) {
|
||||
|
||||
c.Assert(compareStoreScore(store1, store3), Equals, 1)
|
||||
}
|
||||
|
||||
func (s *testSelectorSuite) TestScheduleConfig(c *C) {
|
||||
filters := make([]filter.Filter, 0)
|
||||
testScheduleConfig := func(selector *BalanceSelector, stores []*core.StoreInfo, expectSourceID, expectTargetID uint64) {
|
||||
c.Assert(selector.SelectSource(s.tc, stores).GetID(), Equals, expectSourceID)
|
||||
c.Assert(selector.SelectTarget(s.tc, stores).GetID(), Equals, expectTargetID)
|
||||
}
|
||||
|
||||
kinds := []core.ScheduleKind{{
|
||||
Resource: core.RegionKind,
|
||||
}}
|
||||
|
||||
for _, kind := range kinds {
|
||||
selector := NewBalanceSelector(kind, filters)
|
||||
stores := []*core.StoreInfo{
|
||||
core.NewStoreInfoWithSizeCount(1, 2, 3, 10, 5),
|
||||
core.NewStoreInfoWithSizeCount(2, 2, 3, 4, 5),
|
||||
core.NewStoreInfoWithSizeCount(3, 2, 3, 4, 5),
|
||||
core.NewStoreInfoWithSizeCount(4, 2, 3, 2, 5),
|
||||
}
|
||||
testScheduleConfig(selector, stores, 1, 4)
|
||||
}
|
||||
|
||||
selector := NewBalanceSelector(core.ScheduleKind{
|
||||
Resource: core.LeaderKind,
|
||||
}, filters)
|
||||
stores := []*core.StoreInfo{
|
||||
core.NewStoreInfoWithSizeCount(1, 2, 20, 10, 25),
|
||||
core.NewStoreInfoWithSizeCount(2, 2, 66, 10, 5),
|
||||
core.NewStoreInfoWithSizeCount(3, 2, 6, 10, 5),
|
||||
core.NewStoreInfoWithSizeCount(4, 2, 20, 10, 1),
|
||||
}
|
||||
testScheduleConfig(selector, stores, 2, 3)
|
||||
}
|
||||
|
@ -86,10 +86,10 @@ func (l *balanceLeaderScheduler) Schedule(cluster opt.Cluster) *operator.Operato
|
||||
sources := filter.SelectSourceStores(stores, l.filters, cluster)
|
||||
targets := filter.SelectTargetStores(stores, l.filters, cluster)
|
||||
sort.Slice(sources, func(i, j int) bool {
|
||||
return sources[i].LeaderScore(0) > sources[j].LeaderScore(0)
|
||||
return sources[i].GetLeaderCount() > sources[j].GetLeaderCount()
|
||||
})
|
||||
sort.Slice(targets, func(i, j int) bool {
|
||||
return targets[i].LeaderScore(0) < targets[j].LeaderScore(0)
|
||||
return targets[i].GetLeaderCount() < targets[j].GetLeaderCount()
|
||||
})
|
||||
|
||||
for i := 0; i < len(sources) || i < len(targets); i++ {
|
||||
@ -133,7 +133,7 @@ func (l *balanceLeaderScheduler) transferLeaderOut(cluster opt.Cluster, source *
|
||||
targets := cluster.GetFollowerStores(region)
|
||||
targets = filter.SelectTargetStores(targets, l.filters, cluster)
|
||||
sort.Slice(targets, func(i, j int) bool {
|
||||
return targets[i].LeaderScore(0) < targets[j].LeaderScore(0)
|
||||
return targets[i].GetLeaderCount() < targets[j].GetLeaderCount()
|
||||
})
|
||||
for _, target := range targets {
|
||||
if op := l.createOperator(cluster, region, source, target); op != nil {
|
||||
@ -174,8 +174,7 @@ func (l *balanceLeaderScheduler) transferLeaderIn(cluster opt.Cluster, target *c
|
||||
func (l *balanceLeaderScheduler) createOperator(cluster opt.Cluster, region *core.RegionInfo, source, target *core.StoreInfo) *operator.Operator {
|
||||
targetID := target.GetID()
|
||||
|
||||
kind := core.NewScheduleKind(core.LeaderKind)
|
||||
if !shouldBalance(cluster, source, target, region, kind, l.GetName()) {
|
||||
if source.GetLeaderCount()-target.GetLeaderCount() < 2*int(1.0*leaderTolerantSizeRatio) {
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -76,5 +76,7 @@ func (s *balanceRegionScheduler) IsScheduleAllowed(cluster opt.Cluster) bool {
|
||||
}
|
||||
|
||||
func (s *balanceRegionScheduler) Schedule(cluster opt.Cluster) *operator.Operator {
|
||||
// Your Code Here(3C).
|
||||
|
||||
return nil
|
||||
}
|
||||
|
@ -162,31 +162,6 @@ func (s *testBalanceRegionSchedulerSuite) TestReplicas53C(c *C) {
|
||||
testutil.CheckTransferPeer(c, sb.Schedule(tc), operator.OpBalance, 11, 6)
|
||||
}
|
||||
|
||||
func (s *testBalanceRegionSchedulerSuite) TestStoreWeight3C(c *C) {
|
||||
opt := mockoption.NewScheduleOptions()
|
||||
tc := mockcluster.NewCluster(opt)
|
||||
oc := schedule.NewOperatorController(s.ctx, nil, nil)
|
||||
|
||||
sb, err := schedule.CreateScheduler("balance-region", oc, core.NewStorage(kv.NewMemoryKV()), nil)
|
||||
c.Assert(err, IsNil)
|
||||
opt.SetMaxReplicas(1)
|
||||
|
||||
tc.AddRegionStore(1, 10)
|
||||
tc.AddRegionStore(2, 10)
|
||||
tc.AddRegionStore(3, 10)
|
||||
tc.AddRegionStore(4, 10)
|
||||
tc.UpdateStoreRegionWeight(1, 0.5)
|
||||
tc.UpdateStoreRegionWeight(2, 0.9)
|
||||
tc.UpdateStoreRegionWeight(3, 1.0)
|
||||
tc.UpdateStoreRegionWeight(4, 2.0)
|
||||
|
||||
tc.AddLeaderRegion(1, 1)
|
||||
testutil.CheckTransferPeer(c, sb.Schedule(tc), operator.OpBalance, 1, 4)
|
||||
|
||||
tc.UpdateRegionCount(4, 30)
|
||||
testutil.CheckTransferPeer(c, sb.Schedule(tc), operator.OpBalance, 1, 3)
|
||||
}
|
||||
|
||||
func (s *testBalanceRegionSchedulerSuite) TestReplacePendingRegion3C(c *C) {
|
||||
opt := mockoption.NewScheduleOptions()
|
||||
tc := mockcluster.NewCluster(opt)
|
||||
@ -216,75 +191,6 @@ func (s *testBalanceRegionSchedulerSuite) TestReplacePendingRegion3C(c *C) {
|
||||
testutil.CheckTransferPeer(c, sb.Schedule(tc), operator.OpBalance, 1, 4)
|
||||
}
|
||||
|
||||
var _ = Suite(&testBalanceSpeedSuite{})
|
||||
|
||||
type testBalanceSpeedSuite struct{}
|
||||
|
||||
type testBalanceSpeedCase struct {
|
||||
sourceCount uint64
|
||||
targetCount uint64
|
||||
regionSize int64
|
||||
expectedResult bool
|
||||
}
|
||||
|
||||
func (s *testBalanceSpeedSuite) TestShouldBalance3C(c *C) {
|
||||
tests := []testBalanceSpeedCase{
|
||||
// all store capacity is 1024MB
|
||||
// size = count * 10
|
||||
|
||||
// target size is zero
|
||||
{2, 0, 1, true},
|
||||
{2, 0, 10, true},
|
||||
// all in high space stage
|
||||
{10, 5, 1, true},
|
||||
{10, 5, 20, true},
|
||||
{10, 10, 1, false},
|
||||
{10, 10, 20, false},
|
||||
// all in transition stage
|
||||
{70, 50, 1, true},
|
||||
{70, 50, 50, true},
|
||||
{70, 70, 1, false},
|
||||
// all in low space stage
|
||||
{90, 80, 1, true},
|
||||
{90, 80, 50, true},
|
||||
{90, 90, 1, false},
|
||||
// one in high space stage, other in transition stage
|
||||
{65, 55, 5, true},
|
||||
{65, 50, 50, true},
|
||||
// one in transition space stage, other in low space stage
|
||||
{80, 70, 5, true},
|
||||
{80, 70, 50, true},
|
||||
}
|
||||
|
||||
opt := mockoption.NewScheduleOptions()
|
||||
tc := mockcluster.NewCluster(opt)
|
||||
// create a region to control average region size.
|
||||
tc.AddLeaderRegion(1, 1, 2)
|
||||
|
||||
for _, t := range tests {
|
||||
tc.AddRegionStore(1, int(t.sourceCount))
|
||||
tc.AddRegionStore(2, int(t.targetCount))
|
||||
source := tc.GetStore(1)
|
||||
target := tc.GetStore(2)
|
||||
region := tc.GetRegion(1).Clone(core.SetApproximateSize(t.regionSize))
|
||||
tc.PutRegion(region)
|
||||
kind := core.NewScheduleKind(core.RegionKind)
|
||||
c.Assert(shouldBalance(tc, source, target, region, kind, ""), Equals, t.expectedResult)
|
||||
}
|
||||
}
|
||||
|
||||
func (s *testBalanceSpeedSuite) TestTolerantRatio3C(c *C) {
|
||||
opt := mockoption.NewScheduleOptions()
|
||||
tc := mockcluster.NewCluster(opt)
|
||||
// create a region to control average region size.
|
||||
tc.AddLeaderRegion(1, 1, 2)
|
||||
regionSize := int64(96 * 1024)
|
||||
region := tc.GetRegion(1).Clone(core.SetApproximateSize(regionSize))
|
||||
|
||||
c.Assert(getTolerantResource(tc, region, core.ScheduleKind{Resource: core.LeaderKind}), Equals, int64(leaderTolerantSizeRatio))
|
||||
c.Assert(getTolerantResource(tc, region, core.ScheduleKind{Resource: core.RegionKind}), Equals, int64(getTolerantRatio(tc)*float64(regionSize)))
|
||||
}
|
||||
|
||||
var _ = Suite(&testBalanceLeaderSchedulerSuite{})
|
||||
|
||||
type testBalanceLeaderSchedulerSuite struct {
|
||||
@ -412,25 +318,6 @@ func (s *testBalanceLeaderSchedulerSuite) TestBalanceFilter(c *C) {
|
||||
c.Assert(s.schedule(), IsNil)
|
||||
}
|
||||
|
||||
func (s *testBalanceLeaderSchedulerSuite) TestLeaderWeight(c *C) {
|
||||
// Stores: 1 2 3 4
|
||||
// Leaders: 16 16 16 16->48
|
||||
// Weight: 0.5 0.9 1 2
|
||||
// Region1: L F F F
|
||||
s.tc.AddLeaderStore(1, 16)
|
||||
s.tc.AddLeaderStore(2, 16)
|
||||
s.tc.AddLeaderStore(3, 16)
|
||||
s.tc.AddLeaderStore(4, 16)
|
||||
s.tc.UpdateStoreLeaderWeight(1, 0.5)
|
||||
s.tc.UpdateStoreLeaderWeight(2, 0.9)
|
||||
s.tc.UpdateStoreLeaderWeight(3, 1)
|
||||
s.tc.UpdateStoreLeaderWeight(4, 2)
|
||||
s.tc.AddLeaderRegion(1, 1, 2, 3, 4)
|
||||
testutil.CheckTransferLeader(c, s.schedule(), operator.OpBalance, 1, 4)
|
||||
s.tc.UpdateLeaderCount(4, 48)
|
||||
testutil.CheckTransferLeader(c, s.schedule(), operator.OpBalance, 1, 3)
|
||||
}
|
||||
|
||||
func (s *testBalanceLeaderSchedulerSuite) TestBalanceSelector(c *C) {
|
||||
// Stores: 1 2 3 4
|
||||
// Leaders: 1 2 3 16
|
||||
|
@ -17,17 +17,11 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/pingcap-incubator/tinykv/scheduler/server/core"
|
||||
"github.com/pingcap-incubator/tinykv/scheduler/server/schedule/opt"
|
||||
"github.com/pingcap/log"
|
||||
"github.com/pkg/errors"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
const (
|
||||
// adjustRatio is used to adjust TolerantSizeRatio according to region count.
|
||||
adjustRatio float64 = 0.005
|
||||
leaderTolerantSizeRatio float64 = 5.0
|
||||
minTolerantSizeRatio float64 = 1.0
|
||||
)
|
||||
|
||||
// ErrScheduleConfigNotExist the config is not correct.
|
||||
@ -57,58 +51,3 @@ func minDuration(a, b time.Duration) time.Duration {
|
||||
func isRegionUnhealthy(region *core.RegionInfo) bool {
|
||||
return len(region.GetLearners()) != 0
|
||||
}
|
||||
|
||||
func shouldBalance(cluster opt.Cluster, source, target *core.StoreInfo, region *core.RegionInfo, kind core.ScheduleKind, scheduleName string) bool {
|
||||
// The reason we use max(regionSize, averageRegionSize) to check is:
|
||||
// 1. prevent moving small regions between stores with close scores, leading to unnecessary balance.
|
||||
// 2. prevent moving huge regions, leading to over balance.
|
||||
sourceID := source.GetID()
|
||||
targetID := target.GetID()
|
||||
tolerantResource := getTolerantResource(cluster, region, kind)
|
||||
sourceScore := source.ResourceScore(kind, -tolerantResource)
|
||||
targetScore := target.ResourceScore(kind, tolerantResource)
|
||||
|
||||
// Make sure after move, source score is still greater than target score.
|
||||
shouldBalance := sourceScore > targetScore
|
||||
|
||||
if !shouldBalance {
|
||||
log.Debug("skip balance "+kind.Resource.String(),
|
||||
zap.String("scheduler", scheduleName), zap.Uint64("region-id", region.GetID()), zap.Uint64("source-store", sourceID), zap.Uint64("target-store", targetID),
|
||||
zap.Int64("source-size", source.GetRegionSize()), zap.Float64("source-score", sourceScore),
|
||||
zap.Int64("target-size", target.GetRegionSize()), zap.Float64("target-score", targetScore),
|
||||
zap.Int64("average-region-size", cluster.GetAverageRegionSize()),
|
||||
zap.Int64("tolerant-resource", tolerantResource))
|
||||
}
|
||||
return shouldBalance
|
||||
}
|
||||
|
||||
func getTolerantResource(cluster opt.Cluster, region *core.RegionInfo, kind core.ScheduleKind) int64 {
|
||||
if kind.Resource == core.LeaderKind {
|
||||
leaderCount := int64(1.0 * leaderTolerantSizeRatio)
|
||||
return leaderCount
|
||||
}
|
||||
|
||||
regionSize := region.GetApproximateSize()
|
||||
if regionSize < cluster.GetAverageRegionSize() {
|
||||
regionSize = cluster.GetAverageRegionSize()
|
||||
}
|
||||
regionSize = int64(float64(regionSize) * getTolerantRatio(cluster))
|
||||
return regionSize
|
||||
}
|
||||
|
||||
func getTolerantRatio(cluster opt.Cluster) float64 {
|
||||
var maxRegionCount float64
|
||||
stores := cluster.GetStores()
|
||||
for _, store := range stores {
|
||||
regionCount := float64(cluster.GetStoreRegionCount(store.GetID()))
|
||||
if maxRegionCount < regionCount {
|
||||
maxRegionCount = regionCount
|
||||
}
|
||||
}
|
||||
tolerantSizeRatio := maxRegionCount * adjustRatio
|
||||
if tolerantSizeRatio < minTolerantSizeRatio {
|
||||
tolerantSizeRatio = minTolerantSizeRatio
|
||||
}
|
||||
|
||||
return tolerantSizeRatio
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user