From f693bae514387d1561bcc6a0c32154c547698724 Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Mon, 7 Aug 2023 11:31:10 +0800 Subject: [PATCH] *: merge hot stats (#6891) ref tikv/pd#5839 Signed-off-by: Ryan Leung Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- pkg/mock/mockcluster/mockcluster.go | 29 +- pkg/schedule/coordinator.go | 33 +- pkg/schedule/schedulers/grant_hot_region.go | 5 +- pkg/schedule/schedulers/hot_region.go | 127 +-- pkg/schedule/schedulers/hot_region_config.go | 32 +- pkg/schedule/schedulers/hot_region_test.go | 740 +++++++++--------- pkg/schedule/schedulers/hot_region_v2.go | 12 +- pkg/schedule/schedulers/hot_region_v2_test.go | 99 +-- pkg/schedule/schedulers/scheduler_test.go | 48 +- .../buckets/bucket_stat_informer.go | 8 +- pkg/statistics/buckets/hot_bucket_cache.go | 9 + pkg/statistics/collector.go | 43 +- pkg/statistics/hot_cache.go | 33 +- pkg/statistics/hot_peer.go | 17 +- pkg/statistics/hot_peer_cache.go | 116 ++- pkg/statistics/hot_peer_cache_test.go | 151 ++-- pkg/statistics/hot_stat.go | 8 +- pkg/statistics/region_stat_informer.go | 7 +- pkg/statistics/store.go | 87 +- pkg/statistics/store_collection.go | 39 +- pkg/statistics/store_collection_test.go | 5 +- pkg/statistics/store_hot_peers_infos.go | 41 +- pkg/statistics/store_load.go | 43 +- pkg/statistics/store_load_test.go | 13 +- pkg/statistics/{util.go => utils/constant.go} | 26 +- pkg/statistics/{ => utils}/kind.go | 21 +- pkg/statistics/{ => utils}/kind_test.go | 2 +- pkg/statistics/{ => utils}/topn.go | 2 +- pkg/statistics/{ => utils}/topn_test.go | 2 +- server/api/hot_status.go | 18 +- server/cluster/cluster.go | 41 +- server/cluster/cluster_test.go | 35 +- server/cluster/cluster_worker.go | 2 +- tests/pdctl/hot/hot_test.go | 83 +- tests/pdctl/store/store_test.go | 6 +- .../server/storage/hot_region_storage_test.go | 21 +- tools/pd-ctl/pdctl/command/scheduler.go | 12 +- 37 files changed, 1001 insertions(+), 1015 deletions(-) rename pkg/statistics/{util.go => utils/constant.go} (54%) rename pkg/statistics/{ => utils}/kind.go (93%) rename pkg/statistics/{ => utils}/kind_test.go (99%) rename pkg/statistics/{ => utils}/topn.go (99%) rename pkg/statistics/{ => utils}/topn_test.go (99%) diff --git a/pkg/mock/mockcluster/mockcluster.go b/pkg/mock/mockcluster/mockcluster.go index 4b98e83b75b..60ab3a47976 100644 --- a/pkg/mock/mockcluster/mockcluster.go +++ b/pkg/mock/mockcluster/mockcluster.go @@ -34,6 +34,7 @@ import ( "github.com/tikv/pd/pkg/schedule/placement" "github.com/tikv/pd/pkg/statistics" "github.com/tikv/pd/pkg/statistics/buckets" + "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/storage" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/pkg/versioninfo" @@ -145,7 +146,7 @@ func (mc *Cluster) IsRegionHot(region *core.RegionInfo) bool { } // GetHotPeerStat returns hot peer stat with specified regionID and storeID. -func (mc *Cluster) GetHotPeerStat(rw statistics.RWType, regionID, storeID uint64) *statistics.HotPeerStat { +func (mc *Cluster) GetHotPeerStat(rw utils.RWType, regionID, storeID uint64) *statistics.HotPeerStat { return mc.HotCache.GetHotPeerStat(rw, regionID, storeID) } @@ -153,7 +154,7 @@ func (mc *Cluster) GetHotPeerStat(rw statistics.RWType, regionID, storeID uint64 // The result only includes peers that are hot enough. func (mc *Cluster) RegionReadStats() map[uint64][]*statistics.HotPeerStat { // We directly use threshold for read stats for mockCluster - return mc.HotCache.RegionStats(statistics.Read, mc.GetHotRegionCacheHitsThreshold()) + return mc.HotCache.RegionStats(utils.Read, mc.GetHotRegionCacheHitsThreshold()) } // BucketsStats returns hot region's buckets stats. @@ -168,11 +169,11 @@ func (mc *Cluster) BucketsStats(degree int, regions ...uint64) map[uint64][]*buc // RegionWriteStats returns hot region's write stats. // The result only includes peers that are hot enough. func (mc *Cluster) RegionWriteStats() map[uint64][]*statistics.HotPeerStat { - return mc.HotCache.RegionStats(statistics.Write, mc.GetHotRegionCacheHitsThreshold()) + return mc.HotCache.RegionStats(utils.Write, mc.GetHotRegionCacheHitsThreshold()) } // HotRegionsFromStore picks hot regions in specify store. -func (mc *Cluster) HotRegionsFromStore(store uint64, kind statistics.RWType) []*core.RegionInfo { +func (mc *Cluster) HotRegionsFromStore(store uint64, kind utils.RWType) []*core.RegionInfo { stats := hotRegionsFromStore(mc.HotCache, store, kind, mc.GetHotRegionCacheHitsThreshold()) regions := make([]*core.RegionInfo, 0, len(stats)) for _, stat := range stats { @@ -185,7 +186,7 @@ func (mc *Cluster) HotRegionsFromStore(store uint64, kind statistics.RWType) []* } // hotRegionsFromStore picks hot region in specify store. -func hotRegionsFromStore(w *statistics.HotCache, storeID uint64, kind statistics.RWType, minHotDegree int) []*statistics.HotPeerStat { +func hotRegionsFromStore(w *statistics.HotCache, storeID uint64, kind utils.RWType, minHotDegree int) []*statistics.HotPeerStat { if stats, ok := w.RegionStats(kind, minHotDegree)[storeID]; ok && len(stats) > 0 { return stats } @@ -464,7 +465,7 @@ func (mc *Cluster) AddRegionWithReadInfo( r = r.Clone(core.SetReadKeys(readKeys)) r = r.Clone(core.SetReportInterval(0, reportInterval)) r = r.Clone(core.SetReadQuery(readQuery)) - filledNum := statistics.DefaultAotSize + filledNum := utils.DefaultAotSize if len(filledNums) > 0 { filledNum = filledNums[0] } @@ -473,7 +474,7 @@ func (mc *Cluster) AddRegionWithReadInfo( for i := 0; i < filledNum; i++ { items = mc.CheckRegionRead(r) for _, item := range items { - mc.HotCache.Update(item, statistics.Read) + mc.HotCache.Update(item, utils.Read) } } mc.PutRegion(r) @@ -485,7 +486,7 @@ func (mc *Cluster) AddRegionWithPeerReadInfo(regionID, leaderStoreID, targetStor otherPeerStoreIDs []uint64, filledNums ...int) []*statistics.HotPeerStat { r := mc.newMockRegionInfo(regionID, leaderStoreID, otherPeerStoreIDs...) r = r.Clone(core.SetReadBytes(readBytes), core.SetReadKeys(readKeys), core.SetReportInterval(0, reportInterval)) - filledNum := statistics.DefaultAotSize + filledNum := utils.DefaultAotSize if len(filledNums) > 0 { filledNum = filledNums[0] } @@ -494,7 +495,7 @@ func (mc *Cluster) AddRegionWithPeerReadInfo(regionID, leaderStoreID, targetStor items = mc.CheckRegionRead(r) for _, item := range items { if item.StoreID == targetStoreID { - mc.HotCache.Update(item, statistics.Read) + mc.HotCache.Update(item, utils.Read) } } } @@ -513,7 +514,7 @@ func (mc *Cluster) AddRegionLeaderWithReadInfo( r = r.Clone(core.SetReadKeys(readKeys)) r = r.Clone(core.SetReadQuery(readQuery)) r = r.Clone(core.SetReportInterval(0, reportInterval)) - filledNum := statistics.DefaultAotSize + filledNum := utils.DefaultAotSize if len(filledNums) > 0 { filledNum = filledNums[0] } @@ -522,7 +523,7 @@ func (mc *Cluster) AddRegionLeaderWithReadInfo( for i := 0; i < filledNum; i++ { items = mc.CheckRegionLeaderRead(r) for _, item := range items { - mc.HotCache.Update(item, statistics.Read) + mc.HotCache.Update(item, utils.Read) } } mc.PutRegion(r) @@ -541,7 +542,7 @@ func (mc *Cluster) AddLeaderRegionWithWriteInfo( r = r.Clone(core.SetReportInterval(0, reportInterval)) r = r.Clone(core.SetWrittenQuery(writtenQuery)) - filledNum := statistics.DefaultAotSize + filledNum := utils.DefaultAotSize if len(filledNums) > 0 { filledNum = filledNums[0] } @@ -550,7 +551,7 @@ func (mc *Cluster) AddLeaderRegionWithWriteInfo( for i := 0; i < filledNum; i++ { items = mc.CheckRegionWrite(r) for _, item := range items { - mc.HotCache.Update(item, statistics.Write) + mc.HotCache.Update(item, utils.Write) } } mc.PutRegion(r) @@ -734,7 +735,7 @@ func (mc *Cluster) updateStorageStatistics(storeID uint64, update func(*pdpb.Sto newStats := typeutil.DeepClone(store.GetStoreStats(), core.StoreStatsFactory) update(newStats) now := time.Now().Unix() - interval := &pdpb.TimeInterval{StartTimestamp: uint64(now - statistics.StoreHeartBeatReportInterval), EndTimestamp: uint64(now)} + interval := &pdpb.TimeInterval{StartTimestamp: uint64(now - utils.StoreHeartBeatReportInterval), EndTimestamp: uint64(now)} newStats.Interval = interval newStore := store.Clone(core.SetStoreStats(newStats)) mc.Set(storeID, newStats) diff --git a/pkg/schedule/coordinator.go b/pkg/schedule/coordinator.go index 16d72ff913b..76334dc02ba 100644 --- a/pkg/schedule/coordinator.go +++ b/pkg/schedule/coordinator.go @@ -37,6 +37,7 @@ import ( "github.com/tikv/pd/pkg/schedule/schedulers" "github.com/tikv/pd/pkg/schedule/splitter" "github.com/tikv/pd/pkg/statistics" + "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" "go.uber.org/zap" @@ -503,18 +504,18 @@ func (c *Coordinator) Stop() { } // GetHotRegionsByType gets hot regions' statistics by RWType. -func (c *Coordinator) GetHotRegionsByType(typ statistics.RWType) *statistics.StoreHotPeersInfos { +func (c *Coordinator) GetHotRegionsByType(typ utils.RWType) *statistics.StoreHotPeersInfos { isTraceFlow := c.cluster.GetSchedulerConfig().IsTraceRegionFlow() storeLoads := c.cluster.GetStoresLoads() stores := c.cluster.GetStores() var infos *statistics.StoreHotPeersInfos switch typ { - case statistics.Write: + case utils.Write: regionStats := c.cluster.RegionWriteStats() - infos = statistics.GetHotStatus(stores, storeLoads, regionStats, statistics.Write, isTraceFlow) - case statistics.Read: + infos = statistics.GetHotStatus(stores, storeLoads, regionStats, utils.Write, isTraceFlow) + case utils.Read: regionStats := c.cluster.RegionReadStats() - infos = statistics.GetHotStatus(stores, storeLoads, regionStats, statistics.Read, isTraceFlow) + infos = statistics.GetHotStatus(stores, storeLoads, regionStats, utils.Read, isTraceFlow) default: } // update params `IsLearner` and `LastUpdateTime` @@ -528,11 +529,11 @@ func (c *Coordinator) GetHotRegionsByType(typ statistics.RWType) *statistics.Sto h.IsLearner = core.IsLearner(region.GetPeer(h.StoreID)) } switch typ { - case statistics.Write: + case utils.Write: if region != nil { h.LastUpdateTime = time.Unix(int64(region.GetInterval().GetEndTimestamp()), 0) } - case statistics.Read: + case utils.Read: store := c.cluster.GetStore(h.StoreID) if store != nil { ts := store.GetMeta().GetLastHeartbeat() @@ -555,24 +556,24 @@ func (c *Coordinator) GetWaitGroup() *sync.WaitGroup { func (c *Coordinator) CollectHotSpotMetrics() { stores := c.cluster.GetStores() // Collects hot write region metrics. - collectHotMetrics(c.cluster, stores, statistics.Write) + collectHotMetrics(c.cluster, stores, utils.Write) // Collects hot read region metrics. - collectHotMetrics(c.cluster, stores, statistics.Read) + collectHotMetrics(c.cluster, stores, utils.Read) } -func collectHotMetrics(cluster sche.ClusterInformer, stores []*core.StoreInfo, typ statistics.RWType) { +func collectHotMetrics(cluster sche.ClusterInformer, stores []*core.StoreInfo, typ utils.RWType) { var ( kind string regionStats map[uint64][]*statistics.HotPeerStat ) switch typ { - case statistics.Read: + case utils.Read: regionStats = cluster.RegionReadStats() - kind = statistics.Read.String() - case statistics.Write: + kind = utils.Read.String() + case utils.Write: regionStats = cluster.RegionWriteStats() - kind = statistics.Write.String() + kind = utils.Write.String() } status := statistics.CollectHotPeerInfos(stores, regionStats) // only returns TotalBytesRate,TotalKeysRate,TotalQueryRate,Count @@ -608,8 +609,8 @@ func collectHotMetrics(cluster sche.ClusterInformer, stores []*core.StoreInfo, t } if !hasHotLeader && !hasHotPeer { - statistics.ForeachRegionStats(func(rwTy statistics.RWType, dim int, _ statistics.RegionStatKind) { - schedulers.HotPendingSum.DeleteLabelValues(storeLabel, rwTy.String(), statistics.DimToString(dim)) + utils.ForeachRegionStats(func(rwTy utils.RWType, dim int, _ utils.RegionStatKind) { + schedulers.HotPendingSum.DeleteLabelValues(storeLabel, rwTy.String(), utils.DimToString(dim)) }) } } diff --git a/pkg/schedule/schedulers/grant_hot_region.go b/pkg/schedule/schedulers/grant_hot_region.go index d0c0cc9a1be..4c8051de677 100644 --- a/pkg/schedule/schedulers/grant_hot_region.go +++ b/pkg/schedule/schedulers/grant_hot_region.go @@ -32,6 +32,7 @@ import ( "github.com/tikv/pd/pkg/schedule/plan" "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/statistics" + "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/syncutil" @@ -234,7 +235,7 @@ func (s *grantHotRegionScheduler) Schedule(cluster sche.SchedulerCluster, dryRun return s.dispatch(rw, cluster), nil } -func (s *grantHotRegionScheduler) dispatch(typ statistics.RWType, cluster sche.SchedulerCluster) []*operator.Operator { +func (s *grantHotRegionScheduler) dispatch(typ utils.RWType, cluster sche.SchedulerCluster) []*operator.Operator { stLoadInfos := s.stLoadInfos[buildResourceType(typ, constant.RegionKind)] infos := make([]*statistics.StoreLoadDetail, len(stLoadInfos)) index := 0 @@ -243,7 +244,7 @@ func (s *grantHotRegionScheduler) dispatch(typ statistics.RWType, cluster sche.S index++ } sort.Slice(infos, func(i, j int) bool { - return infos[i].LoadPred.Current.Loads[statistics.ByteDim] > infos[j].LoadPred.Current.Loads[statistics.ByteDim] + return infos[i].LoadPred.Current.Loads[utils.ByteDim] > infos[j].LoadPred.Current.Loads[utils.ByteDim] }) return s.randomSchedule(cluster, infos) } diff --git a/pkg/schedule/schedulers/hot_region.go b/pkg/schedule/schedulers/hot_region.go index 626396b2a43..27c89bf011f 100644 --- a/pkg/schedule/schedulers/hot_region.go +++ b/pkg/schedule/schedulers/hot_region.go @@ -38,6 +38,7 @@ import ( "github.com/tikv/pd/pkg/schedule/plan" "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/statistics" + "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/utils/keyutil" "github.com/tikv/pd/pkg/utils/syncutil" "go.uber.org/zap" @@ -95,7 +96,7 @@ type baseHotScheduler struct { // this records regionID which have pending Operator by operation type. During filterHotPeers, the hot peers won't // be selected if its owner region is tracked in this attribute. regionPendings map[uint64]*pendingInfluence - types []statistics.RWType + types []utils.RWType r *rand.Rand updateReadTime time.Time updateWriteTime time.Time @@ -105,9 +106,9 @@ func newBaseHotScheduler(opController *operator.Controller) *baseHotScheduler { base := NewBaseScheduler(opController) ret := &baseHotScheduler{ BaseScheduler: base, - types: []statistics.RWType{statistics.Write, statistics.Read}, + types: []utils.RWType{utils.Write, utils.Read}, regionPendings: make(map[uint64]*pendingInfluence), - stHistoryLoads: statistics.NewStoreHistoryLoads(statistics.DimLen), + stHistoryLoads: statistics.NewStoreHistoryLoads(utils.DimLen), r: rand.New(rand.NewSource(time.Now().UnixNano())), } for ty := resourceType(0); ty < resourceTypeLen; ty++ { @@ -118,7 +119,7 @@ func newBaseHotScheduler(opController *operator.Controller) *baseHotScheduler { // prepareForBalance calculate the summary of pending Influence for each store and prepare the load detail for // each store, only update read or write load detail -func (h *baseHotScheduler) prepareForBalance(rw statistics.RWType, cluster sche.SchedulerCluster) { +func (h *baseHotScheduler) prepareForBalance(rw utils.RWType, cluster sche.SchedulerCluster) { h.stInfos = statistics.SummaryStoreInfos(cluster.GetStores()) h.summaryPendingInfluence() h.storesLoads = cluster.GetStoresLoads() @@ -135,7 +136,7 @@ func (h *baseHotScheduler) prepareForBalance(rw statistics.RWType, cluster sche. rw, resource) } switch rw { - case statistics.Read: + case utils.Read: // update read statistics if time.Since(h.updateReadTime) >= statisticsInterval { regionRead := cluster.RegionReadStats() @@ -143,7 +144,7 @@ func (h *baseHotScheduler) prepareForBalance(rw statistics.RWType, cluster sche. prepare(regionRead, constant.RegionKind) h.updateReadTime = time.Now() } - case statistics.Write: + case utils.Write: // update write statistics if time.Since(h.updateWriteTime) >= statisticsInterval { regionWrite := cluster.RegionWriteStats() @@ -179,8 +180,8 @@ func (h *baseHotScheduler) summaryPendingInfluence() { for storeID, info := range h.stInfos { storeLabel := strconv.FormatUint(storeID, 10) if infl := info.PendingSum; infl != nil { - statistics.ForeachRegionStats(func(rwTy statistics.RWType, dim int, kind statistics.RegionStatKind) { - setHotPendingInfluenceMetrics(storeLabel, rwTy.String(), statistics.DimToString(dim), infl.Loads[kind]) + utils.ForeachRegionStats(func(rwTy utils.RWType, dim int, kind utils.RegionStatKind) { + setHotPendingInfluenceMetrics(storeLabel, rwTy.String(), utils.DimToString(dim), infl.Loads[kind]) }) } } @@ -191,7 +192,7 @@ func setHotPendingInfluenceMetrics(storeLabel, rwTy, dim string, load float64) { HotPendingSum.WithLabelValues(storeLabel, rwTy, dim).Set(load) } -func (h *baseHotScheduler) randomRWType() statistics.RWType { +func (h *baseHotScheduler) randomRWType() utils.RWType { return h.types[h.r.Int()%len(h.types)] } @@ -277,7 +278,7 @@ func (h *hotScheduler) Schedule(cluster sche.SchedulerCluster, dryRun bool) ([]* return h.dispatch(rw, cluster), nil } -func (h *hotScheduler) dispatch(typ statistics.RWType, cluster sche.SchedulerCluster) []*operator.Operator { +func (h *hotScheduler) dispatch(typ utils.RWType, cluster sche.SchedulerCluster) []*operator.Operator { h.Lock() defer h.Unlock() h.prepareForBalance(typ, cluster) @@ -286,9 +287,9 @@ func (h *hotScheduler) dispatch(typ statistics.RWType, cluster sche.SchedulerClu return nil } switch typ { - case statistics.Read: + case utils.Read: return h.balanceHotReadRegions(cluster) - case statistics.Write: + case utils.Write: return h.balanceHotWriteRegions(cluster) } return nil @@ -305,16 +306,16 @@ func (h *hotScheduler) tryAddPendingInfluence(op *operator.Operator, srcStore, d influence := newPendingInfluence(op, srcStore, dstStore, infl, maxZombieDur) h.regionPendings[regionID] = influence - statistics.ForeachRegionStats(func(rwTy statistics.RWType, dim int, kind statistics.RegionStatKind) { - hotPeerHist.WithLabelValues(h.GetName(), rwTy.String(), statistics.DimToString(dim)).Observe(infl.Loads[kind]) + utils.ForeachRegionStats(func(rwTy utils.RWType, dim int, kind utils.RegionStatKind) { + hotPeerHist.WithLabelValues(h.GetName(), rwTy.String(), utils.DimToString(dim)).Observe(infl.Loads[kind]) }) return true } func (h *hotScheduler) balanceHotReadRegions(cluster sche.SchedulerCluster) []*operator.Operator { - leaderSolver := newBalanceSolver(h, cluster, statistics.Read, transferLeader) + leaderSolver := newBalanceSolver(h, cluster, utils.Read, transferLeader) leaderOps := leaderSolver.solve() - peerSolver := newBalanceSolver(h, cluster, statistics.Read, movePeer) + peerSolver := newBalanceSolver(h, cluster, utils.Read, movePeer) peerOps := peerSolver.solve() if len(leaderOps) == 0 && len(peerOps) == 0 { hotSchedulerSkipCounter.Inc() @@ -359,7 +360,7 @@ func (h *hotScheduler) balanceHotWriteRegions(cluster sche.SchedulerCluster) []* s := h.r.Intn(100) switch { case s < int(schedulePeerPr*100): - peerSolver := newBalanceSolver(h, cluster, statistics.Write, movePeer) + peerSolver := newBalanceSolver(h, cluster, utils.Write, movePeer) ops := peerSolver.solve() if len(ops) > 0 && peerSolver.tryAddPendingInfluence() { return ops @@ -367,7 +368,7 @@ func (h *hotScheduler) balanceHotWriteRegions(cluster sche.SchedulerCluster) []* default: } - leaderSolver := newBalanceSolver(h, cluster, statistics.Write, transferLeader) + leaderSolver := newBalanceSolver(h, cluster, utils.Write, transferLeader) ops := leaderSolver.solve() if len(ops) > 0 && leaderSolver.tryAddPendingInfluence() { return ops @@ -421,7 +422,7 @@ func (s *solution) getPendingLoad(dim int) (src float64, dst float64) { // calcPeersRate precomputes the peer rate and stores it in cachedPeersRate. func (s *solution) calcPeersRate(dims ...int) { - s.cachedPeersRate = make([]float64, statistics.DimLen) + s.cachedPeersRate = make([]float64, utils.DimLen) for _, dim := range dims { peersRate := s.mainPeerStat.GetLoad(dim) if s.revertPeerStat != nil { @@ -446,7 +447,7 @@ type balanceSolver struct { sche.SchedulerCluster sche *hotScheduler stLoadDetail map[uint64]*statistics.StoreLoadDetail - rwTy statistics.RWType + rwTy utils.RWType opTy opType resourceTy resourceType @@ -491,15 +492,15 @@ func (bs *balanceSolver) init() { bs.resourceTy = toResourceType(bs.rwTy, bs.opTy) bs.stLoadDetail = bs.sche.stLoadInfos[bs.resourceTy] - bs.maxSrc = &statistics.StoreLoad{Loads: make([]float64, statistics.DimLen)} + bs.maxSrc = &statistics.StoreLoad{Loads: make([]float64, utils.DimLen)} bs.minDst = &statistics.StoreLoad{ - Loads: make([]float64, statistics.DimLen), + Loads: make([]float64, utils.DimLen), Count: math.MaxFloat64, } for i := range bs.minDst.Loads { bs.minDst.Loads[i] = math.MaxFloat64 } - maxCur := &statistics.StoreLoad{Loads: make([]float64, statistics.DimLen)} + maxCur := &statistics.StoreLoad{Loads: make([]float64, utils.DimLen)} for _, detail := range bs.stLoadDetail { bs.maxSrc = statistics.MaxLoad(bs.maxSrc, detail.LoadPred.Min()) @@ -508,10 +509,10 @@ func (bs *balanceSolver) init() { } rankStepRatios := []float64{ - statistics.ByteDim: bs.sche.conf.GetByteRankStepRatio(), - statistics.KeyDim: bs.sche.conf.GetKeyRankStepRatio(), - statistics.QueryDim: bs.sche.conf.GetQueryRateRankStepRatio()} - stepLoads := make([]float64, statistics.DimLen) + utils.ByteDim: bs.sche.conf.GetByteRankStepRatio(), + utils.KeyDim: bs.sche.conf.GetKeyRankStepRatio(), + utils.QueryDim: bs.sche.conf.GetQueryRateRankStepRatio()} + stepLoads := make([]float64, utils.DimLen) for i := range stepLoads { stepLoads[i] = maxCur.Loads[i] * rankStepRatios[i] } @@ -579,7 +580,7 @@ func (bs *balanceSolver) getPriorities() []string { return []string{} } -func newBalanceSolver(sche *hotScheduler, cluster sche.SchedulerCluster, rwTy statistics.RWType, opTy opType) *balanceSolver { +func newBalanceSolver(sche *hotScheduler, cluster sche.SchedulerCluster, rwTy utils.RWType, opTy opType) *balanceSolver { bs := &balanceSolver{ SchedulerCluster: cluster, sche: sche, @@ -695,7 +696,7 @@ func (bs *balanceSolver) solve() []*operator.Operator { } func (bs *balanceSolver) skipCounter(label string) prometheus.Counter { - if bs.rwTy == statistics.Read { + if bs.rwTy == utils.Read { switch label { case "byte": return readSkipByteDimUniformStoreCounter @@ -749,7 +750,7 @@ func (bs *balanceSolver) tryAddPendingInfluence() bool { } func (bs *balanceSolver) collectPendingInfluence(peer *statistics.HotPeerStat) statistics.Influence { - infl := statistics.Influence{Loads: make([]float64, statistics.RegionStatCount), Count: 1} + infl := statistics.Influence{Loads: make([]float64, utils.RegionStatCount), Count: 1} bs.rwTy.SetFullLoadRates(infl.Loads, peer.GetLoads()) inverse := bs.rwTy.Inverse() another := bs.GetHotPeerStat(inverse, peer.RegionID, peer.StoreID) @@ -764,7 +765,7 @@ func (bs *balanceSolver) collectPendingInfluence(peer *statistics.HotPeerStat) s func (bs *balanceSolver) calcMaxZombieDur() time.Duration { switch bs.resourceTy { case writeLeader: - if bs.firstPriority == statistics.QueryDim { + if bs.firstPriority == utils.QueryDim { // We use store query info rather than total of hot write leader to guide hot write leader scheduler // when its first priority is `QueryDim`, because `Write-peer` does not have `QueryDim`. // The reason is the same with `tikvCollector.GetLoads`. @@ -793,7 +794,7 @@ func (bs *balanceSolver) filterSrcStores() map[uint64]*statistics.StoreLoadDetai if !confEnableForTiFlash { continue } - if bs.rwTy != statistics.Write || bs.opTy != movePeer { + if bs.rwTy != utils.Write || bs.opTy != movePeer { continue } srcToleranceRatio += tiflashToleranceRatioCorrection @@ -957,7 +958,7 @@ func (bs *balanceSolver) filterDstStores() map[uint64]*statistics.StoreLoadDetai srcStore := bs.cur.srcStore.StoreInfo switch bs.opTy { case movePeer: - if bs.rwTy == statistics.Read && bs.cur.mainPeerStat.IsLeader() { // for hot-read scheduler, only move peer + if bs.rwTy == utils.Read && bs.cur.mainPeerStat.IsLeader() { // for hot-read scheduler, only move peer return nil } filters = []filter.Filter{ @@ -978,7 +979,7 @@ func (bs *balanceSolver) filterDstStores() map[uint64]*statistics.StoreLoadDetai &filter.StoreStateFilter{ActionScope: bs.sche.GetName(), TransferLeader: true, OperatorLevel: constant.High}, filter.NewSpecialUseFilter(bs.sche.GetName(), filter.SpecialUseHotRegion), } - if bs.rwTy == statistics.Read { + if bs.rwTy == utils.Read { peers := bs.cur.region.GetPeers() moveLeaderFilters := []filter.Filter{&filter.StoreStateFilter{ActionScope: bs.sche.GetName(), MoveRegion: true, OperatorLevel: constant.High}} if leaderFilter := filter.NewPlacementLeaderSafeguard(bs.sche.GetName(), bs.GetSchedulerConfig(), bs.GetBasicCluster(), bs.GetRuleManager(), bs.cur.region, srcStore, true /*allowMoveLeader*/); leaderFilter != nil { @@ -1028,7 +1029,7 @@ func (bs *balanceSolver) pickDstStores(filters []filter.Filter, candidates []*st if !confEnableForTiFlash { continue } - if bs.rwTy != statistics.Write || bs.opTy != movePeer { + if bs.rwTy != utils.Write || bs.opTy != movePeer { continue } dstToleranceRatio += tiflashToleranceRatioCorrection @@ -1231,11 +1232,11 @@ func (bs *balanceSolver) isNotWorsened(dim int) bool { func (bs *balanceSolver) getMinRate(dim int) float64 { switch dim { - case statistics.KeyDim: + case utils.KeyDim: return bs.sche.conf.GetMinHotKeyRate() - case statistics.ByteDim: + case utils.ByteDim: return bs.sche.conf.GetMinHotByteRate() - case statistics.QueryDim: + case utils.QueryDim: return bs.sche.conf.GetMinHotQueryRate() } return -1 @@ -1302,10 +1303,10 @@ func (bs *balanceSolver) betterThanV1(old *solution) bool { return false } -var dimToStep = [statistics.DimLen]float64{ - statistics.ByteDim: 100, - statistics.KeyDim: 10, - statistics.QueryDim: 10, +var dimToStep = [utils.DimLen]float64{ + utils.ByteDim: 100, + utils.KeyDim: 10, + utils.QueryDim: 10, } func (bs *balanceSolver) getRkCmpPrioritiesV1(old *solution) (firstCmp int, secondCmp int) { @@ -1446,9 +1447,9 @@ func (bs *balanceSolver) buildOperators() (ops []*operator.Operator) { var createOperator func(region *core.RegionInfo, srcStoreID, dstStoreID uint64) (op *operator.Operator, typ string, err error) switch bs.rwTy { - case statistics.Read: + case utils.Read: createOperator = bs.createReadOperator - case statistics.Write: + case utils.Write: createOperator = bs.createWriteOperator } @@ -1720,16 +1721,16 @@ const ( resourceTypeLen ) -func toResourceType(rwTy statistics.RWType, opTy opType) resourceType { +func toResourceType(rwTy utils.RWType, opTy opType) resourceType { switch rwTy { - case statistics.Write: + case utils.Write: switch opTy { case movePeer: return writePeer case transferLeader: return writeLeader } - case statistics.Read: + case utils.Read: switch opTy { case movePeer: return readPeer @@ -1740,16 +1741,16 @@ func toResourceType(rwTy statistics.RWType, opTy opType) resourceType { panic(fmt.Sprintf("invalid arguments for toResourceType: rwTy = %v, opTy = %v", rwTy, opTy)) } -func buildResourceType(rwTy statistics.RWType, ty constant.ResourceKind) resourceType { +func buildResourceType(rwTy utils.RWType, ty constant.ResourceKind) resourceType { switch rwTy { - case statistics.Write: + case utils.Write: switch ty { case constant.RegionKind: return writePeer case constant.LeaderKind: return writeLeader } - case statistics.Read: + case utils.Read: switch ty { case constant.RegionKind: return readPeer @@ -1762,24 +1763,24 @@ func buildResourceType(rwTy statistics.RWType, ty constant.ResourceKind) resourc func stringToDim(name string) int { switch name { - case statistics.BytePriority: - return statistics.ByteDim - case statistics.KeyPriority: - return statistics.KeyDim - case statistics.QueryPriority: - return statistics.QueryDim + case utils.BytePriority: + return utils.ByteDim + case utils.KeyPriority: + return utils.KeyDim + case utils.QueryPriority: + return utils.QueryDim } - return statistics.ByteDim + return utils.ByteDim } func dimToString(dim int) string { switch dim { - case statistics.ByteDim: - return statistics.BytePriority - case statistics.KeyDim: - return statistics.KeyPriority - case statistics.QueryDim: - return statistics.QueryPriority + case utils.ByteDim: + return utils.BytePriority + case utils.KeyDim: + return utils.KeyPriority + case utils.QueryDim: + return utils.QueryPriority default: return "" } diff --git a/pkg/schedule/schedulers/hot_region_config.go b/pkg/schedule/schedulers/hot_region_config.go index 803e1cb475e..ee05523d4d5 100644 --- a/pkg/schedule/schedulers/hot_region_config.go +++ b/pkg/schedule/schedulers/hot_region_config.go @@ -26,7 +26,7 @@ import ( "github.com/tikv/pd/pkg/errs" sche "github.com/tikv/pd/pkg/schedule/core" "github.com/tikv/pd/pkg/slice" - "github.com/tikv/pd/pkg/statistics" + "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/reflectutil" "github.com/tikv/pd/pkg/utils/syncutil" @@ -43,16 +43,16 @@ const ( ) var defaultPrioritiesConfig = prioritiesConfig{ - read: []string{statistics.QueryPriority, statistics.BytePriority}, - writeLeader: []string{statistics.QueryPriority, statistics.BytePriority}, - writePeer: []string{statistics.BytePriority, statistics.KeyPriority}, + read: []string{utils.QueryPriority, utils.BytePriority}, + writeLeader: []string{utils.QueryPriority, utils.BytePriority}, + writePeer: []string{utils.BytePriority, utils.KeyPriority}, } // because tikv below 5.2.0 does not report query information, we will use byte and key as the scheduling dimensions var compatiblePrioritiesConfig = prioritiesConfig{ - read: []string{statistics.BytePriority, statistics.KeyPriority}, - writeLeader: []string{statistics.KeyPriority, statistics.BytePriority}, - writePeer: []string{statistics.BytePriority, statistics.KeyPriority}, + read: []string{utils.BytePriority, utils.KeyPriority}, + writeLeader: []string{utils.KeyPriority, utils.BytePriority}, + writePeer: []string{utils.BytePriority, utils.KeyPriority}, } // params about hot region. @@ -154,13 +154,13 @@ func (conf *hotRegionSchedulerConfig) EncodeConfig() ([]byte, error) { func (conf *hotRegionSchedulerConfig) GetStoreStatZombieDuration() time.Duration { conf.RLock() defer conf.RUnlock() - return time.Duration(conf.MaxZombieRounds*statistics.StoreHeartBeatReportInterval) * time.Second + return time.Duration(conf.MaxZombieRounds*utils.StoreHeartBeatReportInterval) * time.Second } func (conf *hotRegionSchedulerConfig) GetRegionsStatZombieDuration() time.Duration { conf.RLock() defer conf.RUnlock() - return time.Duration(conf.MaxZombieRounds*statistics.RegionHeartBeatReportInterval) * time.Second + return time.Duration(conf.MaxZombieRounds*utils.RegionHeartBeatReportInterval) * time.Second } func (conf *hotRegionSchedulerConfig) GetMaxPeerNumber() int { @@ -310,7 +310,7 @@ func (conf *hotRegionSchedulerConfig) getRankFormulaVersionLocked() string { } } -func (conf *hotRegionSchedulerConfig) IsForbidRWType(rw statistics.RWType) bool { +func (conf *hotRegionSchedulerConfig) IsForbidRWType(rw utils.RWType) bool { conf.RLock() defer conf.RUnlock() return rw.String() == conf.ForbidRWType @@ -318,7 +318,7 @@ func (conf *hotRegionSchedulerConfig) IsForbidRWType(rw statistics.RWType) bool func (conf *hotRegionSchedulerConfig) getForbidRWTypeLocked() string { switch conf.ForbidRWType { - case statistics.Read.String(), statistics.Write.String(): + case utils.Read.String(), utils.Write.String(): return conf.ForbidRWType default: return "" @@ -342,7 +342,7 @@ func (conf *hotRegionSchedulerConfig) handleGetConfig(w http.ResponseWriter, r * func isPriorityValid(priorities []string) (map[string]bool, error) { priorityMap := map[string]bool{} for _, p := range priorities { - if p != statistics.BytePriority && p != statistics.KeyPriority && p != statistics.QueryPriority { + if p != utils.BytePriority && p != utils.KeyPriority && p != utils.QueryPriority { return nil, errs.ErrSchedulerConfig.FastGenByArgs("invalid scheduling dimensions") } priorityMap[p] = true @@ -365,7 +365,7 @@ func (conf *hotRegionSchedulerConfig) valid() error { } if pm, err := isPriorityValid(conf.WritePeerPriorities); err != nil { return err - } else if pm[statistics.QueryPriority] { + } else if pm[utils.QueryPriority] { return errs.ErrSchedulerConfig.FastGenByArgs("query is not allowed to be set in priorities for write-peer-priorities") } @@ -373,7 +373,7 @@ func (conf *hotRegionSchedulerConfig) valid() error { return errs.ErrSchedulerConfig.FastGenByArgs("invalid rank-formula-version") } - if conf.ForbidRWType != statistics.Read.String() && conf.ForbidRWType != statistics.Write.String() && + if conf.ForbidRWType != utils.Read.String() && conf.ForbidRWType != utils.Write.String() && conf.ForbidRWType != "none" && conf.ForbidRWType != "" { return errs.ErrSchedulerConfig.FastGenByArgs("invalid forbid-rw-type") } @@ -477,7 +477,7 @@ func getWritePeerPriorities(c *prioritiesConfig) []string { // because tikv below 5.2.0 does not report query information, we will use byte and key as the scheduling dimensions func adjustPrioritiesConfig(querySupport bool, origins []string, getPriorities func(*prioritiesConfig) []string) []string { withQuery := slice.AnyOf(origins, func(i int) bool { - return origins[i] == statistics.QueryPriority + return origins[i] == utils.QueryPriority }) compatibles := getPriorities(&compatiblePrioritiesConfig) if !querySupport && withQuery { @@ -486,7 +486,7 @@ func adjustPrioritiesConfig(querySupport bool, origins []string, getPriorities f defaults := getPriorities(&defaultPrioritiesConfig) isLegal := slice.AllOf(origins, func(i int) bool { - return origins[i] == statistics.BytePriority || origins[i] == statistics.KeyPriority || origins[i] == statistics.QueryPriority + return origins[i] == utils.BytePriority || origins[i] == utils.KeyPriority || origins[i] == utils.QueryPriority }) if len(defaults) == len(origins) && isLegal && origins[0] != origins[1] { return origins diff --git a/pkg/schedule/schedulers/hot_region_test.go b/pkg/schedule/schedulers/hot_region_test.go index e4cf6b121f8..6b1df5f8bd9 100644 --- a/pkg/schedule/schedulers/hot_region_test.go +++ b/pkg/schedule/schedulers/hot_region_test.go @@ -22,14 +22,13 @@ import ( "github.com/docker/go-units" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/kvproto/pkg/pdpb" "github.com/stretchr/testify/require" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/schedule/placement" "github.com/tikv/pd/pkg/statistics" - "github.com/tikv/pd/pkg/statistics/buckets" + "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/storage" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/operatorutil" @@ -39,11 +38,11 @@ import ( func init() { schedulePeerPr = 1.0 - RegisterScheduler(statistics.Write.String(), func(opController *operator.Controller, storage endpoint.ConfigStorage, decoder ConfigDecoder, removeSchedulerCb ...func(string) error) (Scheduler, error) { + RegisterScheduler(utils.Write.String(), func(opController *operator.Controller, storage endpoint.ConfigStorage, decoder ConfigDecoder, removeSchedulerCb ...func(string) error) (Scheduler, error) { cfg := initHotRegionScheduleConfig() return newHotWriteScheduler(opController, cfg), nil }) - RegisterScheduler(statistics.Read.String(), func(opController *operator.Controller, storage endpoint.ConfigStorage, decoder ConfigDecoder, removeSchedulerCb ...func(string) error) (Scheduler, error) { + RegisterScheduler(utils.Read.String(), func(opController *operator.Controller, storage endpoint.ConfigStorage, decoder ConfigDecoder, removeSchedulerCb ...func(string) error) (Scheduler, error) { return newHotReadScheduler(opController, initHotRegionScheduleConfig()), nil }) } @@ -51,14 +50,14 @@ func init() { func newHotReadScheduler(opController *operator.Controller, conf *hotRegionSchedulerConfig) *hotScheduler { ret := newHotScheduler(opController, conf) ret.name = "" - ret.types = []statistics.RWType{statistics.Read} + ret.types = []utils.RWType{utils.Read} return ret } func newHotWriteScheduler(opController *operator.Controller, conf *hotRegionSchedulerConfig) *hotScheduler { ret := newHotScheduler(opController, conf) ret.name = "" - ret.types = []statistics.RWType{statistics.Write} + ret.types = []utils.RWType{utils.Write} return ret } @@ -74,35 +73,35 @@ func TestUpgrade(t *testing.T) { sche, err := CreateScheduler(HotRegionType, oc, storage.NewStorageWithMemoryBackend(), ConfigSliceDecoder(HotRegionType, nil)) re.NoError(err) hb := sche.(*hotScheduler) - re.Equal([]string{statistics.QueryPriority, statistics.BytePriority}, hb.conf.GetReadPriorities()) - re.Equal([]string{statistics.QueryPriority, statistics.BytePriority}, hb.conf.GetWriteLeaderPriorities()) - re.Equal([]string{statistics.BytePriority, statistics.KeyPriority}, hb.conf.GetWritePeerPriorities()) + re.Equal([]string{utils.QueryPriority, utils.BytePriority}, hb.conf.GetReadPriorities()) + re.Equal([]string{utils.QueryPriority, utils.BytePriority}, hb.conf.GetWriteLeaderPriorities()) + re.Equal([]string{utils.BytePriority, utils.KeyPriority}, hb.conf.GetWritePeerPriorities()) re.Equal("v2", hb.conf.GetRankFormulaVersion()) // upgrade from json(null) sche, err = CreateScheduler(HotRegionType, oc, storage.NewStorageWithMemoryBackend(), ConfigJSONDecoder([]byte("null"))) re.NoError(err) hb = sche.(*hotScheduler) - re.Equal([]string{statistics.QueryPriority, statistics.BytePriority}, hb.conf.GetReadPriorities()) - re.Equal([]string{statistics.QueryPriority, statistics.BytePriority}, hb.conf.GetWriteLeaderPriorities()) - re.Equal([]string{statistics.BytePriority, statistics.KeyPriority}, hb.conf.GetWritePeerPriorities()) + re.Equal([]string{utils.QueryPriority, utils.BytePriority}, hb.conf.GetReadPriorities()) + re.Equal([]string{utils.QueryPriority, utils.BytePriority}, hb.conf.GetWriteLeaderPriorities()) + re.Equal([]string{utils.BytePriority, utils.KeyPriority}, hb.conf.GetWritePeerPriorities()) re.Equal("v2", hb.conf.GetRankFormulaVersion()) // upgrade from < 5.2 config51 := `{"min-hot-byte-rate":100,"min-hot-key-rate":10,"min-hot-query-rate":10,"max-zombie-rounds":5,"max-peer-number":1000,"byte-rate-rank-step-ratio":0.05,"key-rate-rank-step-ratio":0.05,"query-rate-rank-step-ratio":0.05,"count-rank-step-ratio":0.01,"great-dec-ratio":0.95,"minor-dec-ratio":0.99,"src-tolerance-ratio":1.05,"dst-tolerance-ratio":1.05,"strict-picking-store":"true","enable-for-tiflash":"true"}` sche, err = CreateScheduler(HotRegionType, oc, storage.NewStorageWithMemoryBackend(), ConfigJSONDecoder([]byte(config51))) re.NoError(err) hb = sche.(*hotScheduler) - re.Equal([]string{statistics.BytePriority, statistics.KeyPriority}, hb.conf.GetReadPriorities()) - re.Equal([]string{statistics.KeyPriority, statistics.BytePriority}, hb.conf.GetWriteLeaderPriorities()) - re.Equal([]string{statistics.BytePriority, statistics.KeyPriority}, hb.conf.GetWritePeerPriorities()) + re.Equal([]string{utils.BytePriority, utils.KeyPriority}, hb.conf.GetReadPriorities()) + re.Equal([]string{utils.KeyPriority, utils.BytePriority}, hb.conf.GetWriteLeaderPriorities()) + re.Equal([]string{utils.BytePriority, utils.KeyPriority}, hb.conf.GetWritePeerPriorities()) re.Equal("v1", hb.conf.GetRankFormulaVersion()) // upgrade from < 6.4 config54 := `{"min-hot-byte-rate":100,"min-hot-key-rate":10,"min-hot-query-rate":10,"max-zombie-rounds":5,"max-peer-number":1000,"byte-rate-rank-step-ratio":0.05,"key-rate-rank-step-ratio":0.05,"query-rate-rank-step-ratio":0.05,"count-rank-step-ratio":0.01,"great-dec-ratio":0.95,"minor-dec-ratio":0.99,"src-tolerance-ratio":1.05,"dst-tolerance-ratio":1.05,"read-priorities":["query","byte"],"write-leader-priorities":["query","byte"],"write-peer-priorities":["byte","key"],"strict-picking-store":"true","enable-for-tiflash":"true","forbid-rw-type":"none"}` sche, err = CreateScheduler(HotRegionType, oc, storage.NewStorageWithMemoryBackend(), ConfigJSONDecoder([]byte(config54))) re.NoError(err) hb = sche.(*hotScheduler) - re.Equal([]string{statistics.QueryPriority, statistics.BytePriority}, hb.conf.GetReadPriorities()) - re.Equal([]string{statistics.QueryPriority, statistics.BytePriority}, hb.conf.GetWriteLeaderPriorities()) - re.Equal([]string{statistics.BytePriority, statistics.KeyPriority}, hb.conf.GetWritePeerPriorities()) + re.Equal([]string{utils.QueryPriority, utils.BytePriority}, hb.conf.GetReadPriorities()) + re.Equal([]string{utils.QueryPriority, utils.BytePriority}, hb.conf.GetWriteLeaderPriorities()) + re.Equal([]string{utils.BytePriority, utils.KeyPriority}, hb.conf.GetWritePeerPriorities()) re.Equal("v1", hb.conf.GetRankFormulaVersion()) } @@ -139,8 +138,8 @@ func checkGCPendingOpInfos(re *require.Assertions, enablePlacementRules bool) { re.NoError(err) re.NotNil(op) op.Start() - op.SetStatusReachTime(operator.CREATED, time.Now().Add(-5*statistics.StoreHeartBeatReportInterval*time.Second)) - op.SetStatusReachTime(operator.STARTED, time.Now().Add((-5*statistics.StoreHeartBeatReportInterval+1)*time.Second)) + op.SetStatusReachTime(operator.CREATED, time.Now().Add(-5*utils.StoreHeartBeatReportInterval*time.Second)) + op.SetStatusReachTime(operator.STARTED, time.Now().Add((-5*utils.StoreHeartBeatReportInterval+1)*time.Second)) return newPendingInfluence(op, 2, 4, statistics.Influence{}, hb.conf.GetStoreStatZombieDuration()) } justDoneOpInfluence := func(region *core.RegionInfo, ty opType) *pendingInfluence { @@ -150,7 +149,7 @@ func checkGCPendingOpInfos(re *require.Assertions, enablePlacementRules bool) { } shouldRemoveOpInfluence := func(region *core.RegionInfo, ty opType) *pendingInfluence { infl := justDoneOpInfluence(region, ty) - infl.op.SetStatusReachTime(operator.CANCELED, time.Now().Add(-3*statistics.StoreHeartBeatReportInterval*time.Second)) + infl.op.SetStatusReachTime(operator.CANCELED, time.Now().Add(-3*utils.StoreHeartBeatReportInterval*time.Second)) return infl } opInfluenceCreators := [3]func(region *core.RegionInfo, ty opType) *pendingInfluence{shouldRemoveOpInfluence, notDoneOpInfluence, justDoneOpInfluence} @@ -202,47 +201,6 @@ func TestHotWriteRegionScheduleByteRateOnly(t *testing.T) { checkHotWriteRegionScheduleByteRateOnly(re, true /* enable placement rules */) } -func TestSplitBuckets(t *testing.T) { - re := require.New(t) - statistics.Denoising = false - cancel, _, tc, oc := prepareSchedulersTest() - tc.SetHotRegionCacheHitsThreshold(1) - defer cancel() - hb, err := CreateScheduler(statistics.Read.String(), oc, storage.NewStorageWithMemoryBackend(), nil) - re.NoError(err) - solve := newBalanceSolver(hb.(*hotScheduler), tc, statistics.Read, transferLeader) - region := core.NewTestRegionInfo(1, 1, []byte(""), []byte("")) - - // the hot range is [a,c],[e,f] - b := &metapb.Buckets{ - RegionId: 1, - PeriodInMs: 1000, - Keys: [][]byte{[]byte("a"), []byte("b"), []byte("c"), []byte("d"), []byte("e"), []byte("f")}, - Stats: &metapb.BucketStats{ - ReadBytes: []uint64{10 * units.KiB, 10 * units.KiB, 0, 10 * units.KiB, 10 * units.KiB}, - ReadKeys: []uint64{256, 256, 0, 256, 256}, - ReadQps: []uint64{0, 0, 0, 0, 0}, - WriteBytes: []uint64{0, 0, 0, 0, 0}, - WriteQps: []uint64{0, 0, 0, 0, 0}, - WriteKeys: []uint64{0, 0, 0, 0, 0}, - }, - } - - task := buckets.NewCheckPeerTask(b) - re.True(tc.HotBucketCache.CheckAsync(task)) - time.Sleep(time.Millisecond * 10) - ops := solve.createSplitOperator([]*core.RegionInfo{region}) - re.Equal(1, len(ops)) - op := ops[0] - re.Equal(splitBucket, op.Desc()) - expectKeys := [][]byte{[]byte("a"), []byte("c"), []byte("d"), []byte("f")} - expectOp, err := operator.CreateSplitRegionOperator(splitBucket, region, operator.OpSplit, pdpb.CheckPolicy_USEKEY, expectKeys) - re.NoError(err) - expectOp.GetCreateTime() - re.Equal(expectOp.Brief(), op.Brief()) - re.Equal(expectOp.GetAdditionalInfo(), op.GetAdditionalInfo()) -} - func checkHotWriteRegionScheduleByteRateOnly(re *require.Assertions, enablePlacementRules bool) { cancel, opt, tc, oc := prepareSchedulersTest() defer cancel() @@ -250,7 +208,7 @@ func checkHotWriteRegionScheduleByteRateOnly(re *require.Assertions, enablePlace tc.SetEnablePlacementRules(enablePlacementRules) labels := []string{"zone", "host"} tc.SetMaxReplicasWithLabel(enablePlacementRules, 3, labels...) - hb, err := CreateScheduler(statistics.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) + hb, err := CreateScheduler(utils.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) re.NoError(err) tc.SetHotRegionCacheHitsThreshold(0) @@ -272,10 +230,10 @@ func checkHotWriteRegionScheduleByteRateOnly(re *require.Assertions, enablePlace // | 4 | 6MB | // | 5 | 0MB | // | 6 | 0MB | - tc.UpdateStorageWrittenBytes(1, 7.5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenBytes(2, 4.5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenBytes(3, 4.5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenBytes(4, 6*units.MiB*statistics.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenBytes(1, 7.5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenBytes(2, 4.5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenBytes(3, 4.5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenBytes(4, 6*units.MiB*utils.StoreHeartBeatReportInterval) tc.UpdateStorageWrittenBytes(5, 0) tc.UpdateStorageWrittenBytes(6, 0) @@ -285,7 +243,7 @@ func checkHotWriteRegionScheduleByteRateOnly(re *require.Assertions, enablePlace // | 2 | 1 | 3 | 4 | 512KB | // | 3 | 1 | 2 | 4 | 512KB | // Region 1, 2 and 3 are hot regions. - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + addRegionInfo(tc, utils.Write, []testRegionInfo{ {1, []uint64{1, 2, 3}, 512 * units.KiB, 0, 0}, {2, []uint64{1, 3, 4}, 512 * units.KiB, 0, 0}, {3, []uint64{1, 2, 4}, 512 * units.KiB, 0, 0}, @@ -359,12 +317,12 @@ func checkHotWriteRegionScheduleByteRateOnly(re *require.Assertions, enablePlace // | 4 | 3.1MB | // | 5 | 0MB | // | 6 | 3MB | - tc.UpdateStorageWrittenBytes(1, 6*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenBytes(2, 5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenBytes(3, 6*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenBytes(4, 3.1*units.MiB*statistics.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenBytes(1, 6*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenBytes(2, 5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenBytes(3, 6*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenBytes(4, 3.1*units.MiB*utils.StoreHeartBeatReportInterval) tc.UpdateStorageWrittenBytes(5, 0) - tc.UpdateStorageWrittenBytes(6, 3*units.MiB*statistics.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenBytes(6, 3*units.MiB*utils.StoreHeartBeatReportInterval) // | region_id | leader_store | follower_store | follower_store | written_bytes | // |-----------|--------------|----------------|----------------|---------------| @@ -373,7 +331,7 @@ func checkHotWriteRegionScheduleByteRateOnly(re *require.Assertions, enablePlace // | 3 | 6 | 1 | 4 | 512KB | // | 4 | 5 | 6 | 4 | 512KB | // | 5 | 3 | 4 | 5 | 512KB | - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + addRegionInfo(tc, utils.Write, []testRegionInfo{ {1, []uint64{1, 2, 3}, 512 * units.KiB, 0, 0}, {2, []uint64{1, 2, 3}, 512 * units.KiB, 0, 0}, {3, []uint64{6, 1, 4}, 512 * units.KiB, 0, 0}, @@ -453,7 +411,7 @@ func TestHotWriteRegionScheduleByteRateOnlyWithTiFlash(t *testing.T) { }, }, })) - sche, err := CreateScheduler(statistics.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) + sche, err := CreateScheduler(utils.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) re.NoError(err) hb := sche.(*hotScheduler) @@ -495,7 +453,7 @@ func TestHotWriteRegionScheduleByteRateOnlyWithTiFlash(t *testing.T) { {3, []uint64{1, 2, 4, 9}, 512 * units.KiB, 5 * units.KiB, 3000}, {4, []uint64{2, 10}, 100, 1, 1}, } - addRegionInfo(tc, statistics.Write, testRegions) + addRegionInfo(tc, utils.Write, testRegions) regionBytesSum := 0.0 regionKeysSum := 0.0 regionQuerySum := 0.0 @@ -551,10 +509,10 @@ func TestHotWriteRegionScheduleByteRateOnlyWithTiFlash(t *testing.T) { // | 10 | n/a | // | 11 | n/a | storesBytes := map[uint64]uint64{ - 1: 7.5 * units.MiB * statistics.StoreHeartBeatReportInterval, - 2: 4.5 * units.MiB * statistics.StoreHeartBeatReportInterval, - 3: 4.5 * units.MiB * statistics.StoreHeartBeatReportInterval, - 4: 6 * units.MiB * statistics.StoreHeartBeatReportInterval, + 1: 7.5 * units.MiB * utils.StoreHeartBeatReportInterval, + 2: 4.5 * units.MiB * utils.StoreHeartBeatReportInterval, + 3: 4.5 * units.MiB * utils.StoreHeartBeatReportInterval, + 4: 6 * units.MiB * utils.StoreHeartBeatReportInterval, } tc.SetStoreEvictLeader(5, true) tikvBytesSum, tikvKeysSum, tikvQuerySum := 0.0, 0.0, 0.0 @@ -642,22 +600,22 @@ func TestHotWriteRegionScheduleWithQuery(t *testing.T) { statistics.Denoising = false statisticsInterval = 0 - hb, err := CreateScheduler(statistics.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) + hb, err := CreateScheduler(utils.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) re.NoError(err) hb.(*hotScheduler).conf.SetSrcToleranceRatio(1) hb.(*hotScheduler).conf.SetDstToleranceRatio(1) - hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{statistics.QueryPriority, statistics.BytePriority} + hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{utils.QueryPriority, utils.BytePriority} tc.SetHotRegionCacheHitsThreshold(0) tc.AddRegionStore(1, 20) tc.AddRegionStore(2, 20) tc.AddRegionStore(3, 20) - tc.UpdateStorageWriteQuery(1, 11000*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWriteQuery(2, 10000*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWriteQuery(3, 9000*statistics.StoreHeartBeatReportInterval) + tc.UpdateStorageWriteQuery(1, 11000*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWriteQuery(2, 10000*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWriteQuery(3, 9000*utils.StoreHeartBeatReportInterval) - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + addRegionInfo(tc, utils.Write, []testRegionInfo{ {1, []uint64{1, 2, 3}, 500, 0, 500}, {2, []uint64{1, 2, 3}, 500, 0, 500}, {3, []uint64{2, 1, 3}, 500, 0, 500}, @@ -678,11 +636,11 @@ func TestHotWriteRegionScheduleWithKeyRate(t *testing.T) { cancel, _, tc, oc := prepareSchedulersTest() defer cancel() - hb, err := CreateScheduler(statistics.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) + hb, err := CreateScheduler(utils.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) re.NoError(err) hb.(*hotScheduler).conf.SetDstToleranceRatio(1) hb.(*hotScheduler).conf.SetSrcToleranceRatio(1) - hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{statistics.KeyPriority, statistics.BytePriority} + hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{utils.KeyPriority, utils.BytePriority} hb.(*hotScheduler).conf.RankFormulaVersion = "v1" tc.SetHotRegionCacheHitsThreshold(0) @@ -693,13 +651,13 @@ func TestHotWriteRegionScheduleWithKeyRate(t *testing.T) { tc.AddRegionStore(4, 20) tc.AddRegionStore(5, 20) - tc.UpdateStorageWrittenStats(1, 10.5*units.MiB*statistics.StoreHeartBeatReportInterval, 10.5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(2, 9.5*units.MiB*statistics.StoreHeartBeatReportInterval, 9.5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(3, 9.5*units.MiB*statistics.StoreHeartBeatReportInterval, 9.8*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(4, 9*units.MiB*statistics.StoreHeartBeatReportInterval, 9*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(5, 8.9*units.MiB*statistics.StoreHeartBeatReportInterval, 9.2*units.MiB*statistics.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(1, 10.5*units.MiB*utils.StoreHeartBeatReportInterval, 10.5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(2, 9.5*units.MiB*utils.StoreHeartBeatReportInterval, 9.5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(3, 9.5*units.MiB*utils.StoreHeartBeatReportInterval, 9.8*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(4, 9*units.MiB*utils.StoreHeartBeatReportInterval, 9*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(5, 8.9*units.MiB*utils.StoreHeartBeatReportInterval, 9.2*units.MiB*utils.StoreHeartBeatReportInterval) - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + addRegionInfo(tc, utils.Write, []testRegionInfo{ {1, []uint64{2, 1, 3}, 0.5 * units.MiB, 0.5 * units.MiB, 0}, {2, []uint64{2, 1, 3}, 0.5 * units.MiB, 0.5 * units.MiB, 0}, {3, []uint64{2, 4, 3}, 0.05 * units.MiB, 0.1 * units.MiB, 0}, @@ -737,7 +695,7 @@ func TestHotWriteRegionScheduleUnhealthyStore(t *testing.T) { cancel, _, tc, oc := prepareSchedulersTest() defer cancel() - hb, err := CreateScheduler(statistics.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) + hb, err := CreateScheduler(utils.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) re.NoError(err) hb.(*hotScheduler).conf.SetDstToleranceRatio(1) hb.(*hotScheduler).conf.SetSrcToleranceRatio(1) @@ -749,11 +707,11 @@ func TestHotWriteRegionScheduleUnhealthyStore(t *testing.T) { tc.AddRegionStore(3, 20) tc.AddRegionStore(4, 20) - tc.UpdateStorageWrittenStats(1, 10.5*units.MiB*statistics.StoreHeartBeatReportInterval, 10.5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(2, 10*units.MiB*statistics.StoreHeartBeatReportInterval, 10*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(3, 9.5*units.MiB*statistics.StoreHeartBeatReportInterval, 9.5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(4, 0*units.MiB*statistics.StoreHeartBeatReportInterval, 0*units.MiB*statistics.StoreHeartBeatReportInterval) - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + tc.UpdateStorageWrittenStats(1, 10.5*units.MiB*utils.StoreHeartBeatReportInterval, 10.5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(2, 10*units.MiB*utils.StoreHeartBeatReportInterval, 10*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(3, 9.5*units.MiB*utils.StoreHeartBeatReportInterval, 9.5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(4, 0*units.MiB*utils.StoreHeartBeatReportInterval, 0*units.MiB*utils.StoreHeartBeatReportInterval) + addRegionInfo(tc, utils.Write, []testRegionInfo{ {1, []uint64{1, 2, 3}, 0.5 * units.MiB, 0.5 * units.MiB, 0}, {2, []uint64{2, 1, 3}, 0.5 * units.MiB, 0.5 * units.MiB, 0}, {3, []uint64{3, 2, 1}, 0.5 * units.MiB, 0.5 * units.MiB, 0}, @@ -785,7 +743,7 @@ func TestHotWriteRegionScheduleCheckHot(t *testing.T) { cancel, _, tc, oc := prepareSchedulersTest() defer cancel() - hb, err := CreateScheduler(statistics.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) + hb, err := CreateScheduler(utils.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) re.NoError(err) hb.(*hotScheduler).conf.SetDstToleranceRatio(1) hb.(*hotScheduler).conf.SetSrcToleranceRatio(1) @@ -798,12 +756,12 @@ func TestHotWriteRegionScheduleCheckHot(t *testing.T) { tc.AddRegionStore(4, 20) tc.AddRegionStore(5, 20) - tc.UpdateStorageWrittenStats(1, 10.5*units.MiB*statistics.StoreHeartBeatReportInterval, 10.5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(2, 10.5*units.MiB*statistics.StoreHeartBeatReportInterval, 10.5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(3, 10.5*units.MiB*statistics.StoreHeartBeatReportInterval, 10.5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(4, 9.5*units.MiB*statistics.StoreHeartBeatReportInterval, 10*units.MiB*statistics.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(1, 10.5*units.MiB*utils.StoreHeartBeatReportInterval, 10.5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(2, 10.5*units.MiB*utils.StoreHeartBeatReportInterval, 10.5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(3, 10.5*units.MiB*utils.StoreHeartBeatReportInterval, 10.5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(4, 9.5*units.MiB*utils.StoreHeartBeatReportInterval, 10*units.MiB*utils.StoreHeartBeatReportInterval) - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + addRegionInfo(tc, utils.Write, []testRegionInfo{ {1, []uint64{1, 2, 3}, 90, 0.5 * units.MiB, 0}, // no hot {1, []uint64{2, 1, 3}, 90, 0.5 * units.MiB, 0}, // no hot {2, []uint64{3, 2, 1}, 0.5 * units.MiB, 0.5 * units.MiB, 0}, // byteDecRatio is greater than greatDecRatio @@ -820,8 +778,8 @@ func TestHotWriteRegionScheduleWithLeader(t *testing.T) { cancel, _, tc, oc := prepareSchedulersTest() defer cancel() - hb, err := CreateScheduler(statistics.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) - hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{statistics.KeyPriority, statistics.BytePriority} + hb, err := CreateScheduler(utils.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) + hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{utils.KeyPriority, utils.BytePriority} re.NoError(err) tc.SetHotRegionCacheHitsThreshold(0) @@ -829,19 +787,19 @@ func TestHotWriteRegionScheduleWithLeader(t *testing.T) { tc.AddRegionStore(2, 20) tc.AddRegionStore(3, 20) - tc.UpdateStorageWrittenBytes(1, 10*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenBytes(2, 10*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenBytes(3, 10*units.MiB*statistics.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenBytes(1, 10*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenBytes(2, 10*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenBytes(3, 10*units.MiB*utils.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenKeys(1, 10*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenKeys(2, 10*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenKeys(3, 10*units.MiB*statistics.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenKeys(1, 10*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenKeys(2, 10*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenKeys(3, 10*units.MiB*utils.StoreHeartBeatReportInterval) // store1 has 2 peer as leader // store2 has 3 peer as leader // store3 has 2 peer as leader // If transfer leader from store2 to store1 or store3, it will keep on looping, which introduces a lot of unnecessary scheduling - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + addRegionInfo(tc, utils.Write, []testRegionInfo{ {1, []uint64{1, 2, 3}, 0.5 * units.MiB, 1 * units.MiB, 0}, {2, []uint64{1, 2, 3}, 0.5 * units.MiB, 1 * units.MiB, 0}, {3, []uint64{2, 1, 3}, 0.5 * units.MiB, 1 * units.MiB, 0}, @@ -857,7 +815,7 @@ func TestHotWriteRegionScheduleWithLeader(t *testing.T) { re.Empty(ops) } - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + addRegionInfo(tc, utils.Write, []testRegionInfo{ {8, []uint64{2, 1, 3}, 0.5 * units.MiB, 1 * units.MiB, 0}, }) @@ -886,9 +844,9 @@ func TestHotWriteRegionScheduleWithPendingInfluence(t *testing.T) { func checkHotWriteRegionScheduleWithPendingInfluence(re *require.Assertions, dim int) { cancel, _, tc, oc := prepareSchedulersTest() defer cancel() - hb, err := CreateScheduler(statistics.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) + hb, err := CreateScheduler(utils.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) re.NoError(err) - hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{statistics.KeyPriority, statistics.BytePriority} + hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{utils.KeyPriority, utils.BytePriority} hb.(*hotScheduler).conf.RankFormulaVersion = "v1" old := pendingAmpFactor pendingAmpFactor = 0.0 @@ -907,13 +865,13 @@ func checkHotWriteRegionScheduleWithPendingInfluence(re *require.Assertions, dim if dim == 1 { // key rate updateStore = tc.UpdateStorageWrittenKeys } - updateStore(1, 8*units.MiB*statistics.StoreHeartBeatReportInterval) - updateStore(2, 6*units.MiB*statistics.StoreHeartBeatReportInterval) - updateStore(3, 6*units.MiB*statistics.StoreHeartBeatReportInterval) - updateStore(4, 4*units.MiB*statistics.StoreHeartBeatReportInterval) + updateStore(1, 8*units.MiB*utils.StoreHeartBeatReportInterval) + updateStore(2, 6*units.MiB*utils.StoreHeartBeatReportInterval) + updateStore(3, 6*units.MiB*utils.StoreHeartBeatReportInterval) + updateStore(4, 4*units.MiB*utils.StoreHeartBeatReportInterval) if dim == 0 { // byte rate - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + addRegionInfo(tc, utils.Write, []testRegionInfo{ {1, []uint64{1, 2, 3}, 512 * units.KiB, 0, 0}, {2, []uint64{1, 2, 3}, 512 * units.KiB, 0, 0}, {3, []uint64{1, 2, 3}, 512 * units.KiB, 0, 0}, @@ -922,7 +880,7 @@ func checkHotWriteRegionScheduleWithPendingInfluence(re *require.Assertions, dim {6, []uint64{1, 2, 3}, 512 * units.KiB, 0, 0}, }) } else if dim == 1 { // key rate - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + addRegionInfo(tc, utils.Write, []testRegionInfo{ {1, []uint64{1, 2, 3}, 0, 512 * units.KiB, 0}, {2, []uint64{1, 2, 3}, 0, 512 * units.KiB, 0}, {3, []uint64{1, 2, 3}, 0, 512 * units.KiB, 0}, @@ -974,9 +932,9 @@ func TestHotWriteRegionScheduleWithRuleEnabled(t *testing.T) { cancel, _, tc, oc := prepareSchedulersTest() defer cancel() tc.SetEnablePlacementRules(true) - hb, err := CreateScheduler(statistics.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) + hb, err := CreateScheduler(utils.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) re.NoError(err) - hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{statistics.KeyPriority, statistics.BytePriority} + hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{utils.KeyPriority, utils.BytePriority} tc.SetHotRegionCacheHitsThreshold(0) key, err := hex.DecodeString("") @@ -1022,15 +980,15 @@ func TestHotWriteRegionScheduleWithRuleEnabled(t *testing.T) { }) re.NoError(err) - tc.UpdateStorageWrittenBytes(1, 10*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenBytes(2, 10*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenBytes(3, 10*units.MiB*statistics.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenBytes(1, 10*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenBytes(2, 10*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenBytes(3, 10*units.MiB*utils.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenKeys(1, 10*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenKeys(2, 10*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenKeys(3, 10*units.MiB*statistics.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenKeys(1, 10*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenKeys(2, 10*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenKeys(3, 10*units.MiB*utils.StoreHeartBeatReportInterval) - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + addRegionInfo(tc, utils.Write, []testRegionInfo{ {1, []uint64{1, 2, 3}, 0.5 * units.MiB, 1 * units.MiB, 0}, {2, []uint64{1, 2, 3}, 0.5 * units.MiB, 1 * units.MiB, 0}, {3, []uint64{2, 1, 3}, 0.5 * units.MiB, 1 * units.MiB, 0}, @@ -1056,10 +1014,10 @@ func TestHotReadRegionScheduleByteRateOnly(t *testing.T) { cancel, _, tc, oc := prepareSchedulersTest() defer cancel() tc.SetClusterVersion(versioninfo.MinSupportedVersion(versioninfo.Version4_0)) - scheduler, err := CreateScheduler(statistics.Read.String(), oc, storage.NewStorageWithMemoryBackend(), nil) + scheduler, err := CreateScheduler(utils.Read.String(), oc, storage.NewStorageWithMemoryBackend(), nil) re.NoError(err) hb := scheduler.(*hotScheduler) - hb.conf.ReadPriorities = []string{statistics.BytePriority, statistics.KeyPriority} + hb.conf.ReadPriorities = []string{utils.BytePriority, utils.KeyPriority} tc.SetHotRegionCacheHitsThreshold(0) // Add stores 1, 2, 3, 4, 5 with region counts 3, 2, 2, 2, 0. @@ -1076,10 +1034,10 @@ func TestHotReadRegionScheduleByteRateOnly(t *testing.T) { // | 3 | 3.7MB | // | 4 | 6MB | // | 5 | 0MB | - tc.UpdateStorageReadBytes(1, 7.5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadBytes(2, 4.9*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadBytes(3, 3.7*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadBytes(4, 6*units.MiB*statistics.StoreHeartBeatReportInterval) + tc.UpdateStorageReadBytes(1, 7.5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadBytes(2, 4.9*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadBytes(3, 3.7*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadBytes(4, 6*units.MiB*utils.StoreHeartBeatReportInterval) tc.UpdateStorageReadBytes(5, 0) // | region_id | leader_store | follower_store | follower_store | read_bytes_rate | @@ -1089,7 +1047,7 @@ func TestHotReadRegionScheduleByteRateOnly(t *testing.T) { // | 3 | 1 | 2 | 3 | 510KB | // | 11 | 1 | 2 | 3 | 7KB | // Region 1, 2 and 3 are hot regions. - addRegionInfo(tc, statistics.Read, []testRegionInfo{ + addRegionInfo(tc, utils.Read, []testRegionInfo{ {1, []uint64{1, 2, 3}, 512 * units.KiB, 0, 0}, {2, []uint64{2, 1, 3}, 511 * units.KiB, 0, 0}, {3, []uint64{1, 2, 3}, 510 * units.KiB, 0, 0}, @@ -1099,14 +1057,14 @@ func TestHotReadRegionScheduleByteRateOnly(t *testing.T) { re.True(tc.IsRegionHot(tc.GetRegion(1))) re.False(tc.IsRegionHot(tc.GetRegion(11))) // check randomly pick hot region - r := tc.HotRegionsFromStore(2, statistics.Read) + r := tc.HotRegionsFromStore(2, utils.Read) re.Len(r, 3) // check hot items - stats := tc.HotCache.RegionStats(statistics.Read, 0) + stats := tc.HotCache.RegionStats(utils.Read, 0) re.Len(stats, 3) for _, ss := range stats { for _, s := range ss { - re.Less(500.0*units.KiB, s.GetLoad(statistics.ByteDim)) + re.Less(500.0*units.KiB, s.GetLoad(utils.ByteDim)) } } @@ -1117,11 +1075,11 @@ func TestHotReadRegionScheduleByteRateOnly(t *testing.T) { // it is better than transfer leader from store 1 to store 3 operatorutil.CheckTransferPeerWithLeaderTransfer(re, op, operator.OpHotRegion, 1, 5) re.Contains(hb.regionPendings, uint64(1)) - re.True(typeutil.Float64Equal(512.0*units.KiB, hb.regionPendings[1].origin.Loads[statistics.RegionReadBytes])) + re.True(typeutil.Float64Equal(512.0*units.KiB, hb.regionPendings[1].origin.Loads[utils.RegionReadBytes])) clearPendingInfluence(hb) // assume handle the transfer leader operator rather than move leader - tc.AddRegionWithReadInfo(3, 3, 512*units.KiB*statistics.ReadReportInterval, 0, 0, statistics.ReadReportInterval, []uint64{1, 2}) + tc.AddRegionWithReadInfo(3, 3, 512*units.KiB*utils.StoreHeartBeatReportInterval, 0, 0, utils.StoreHeartBeatReportInterval, []uint64{1, 2}) // After transfer a hot region leader from store 1 to store 3 // the three region leader will be evenly distributed in three stores @@ -1132,11 +1090,11 @@ func TestHotReadRegionScheduleByteRateOnly(t *testing.T) { // | 3 | 5.5MB | // | 4 | 3.4MB | // | 5 | 3MB | - tc.UpdateStorageReadBytes(1, 6*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadBytes(2, 5.5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadBytes(3, 5.5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadBytes(4, 3.4*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadBytes(5, 3*units.MiB*statistics.StoreHeartBeatReportInterval) + tc.UpdateStorageReadBytes(1, 6*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadBytes(2, 5.5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadBytes(3, 5.5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadBytes(4, 3.4*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadBytes(5, 3*units.MiB*utils.StoreHeartBeatReportInterval) // | region_id | leader_store | follower_store | follower_store | read_bytes_rate | // |-----------|--------------|----------------|----------------|--------------------| @@ -1146,7 +1104,7 @@ func TestHotReadRegionScheduleByteRateOnly(t *testing.T) { // | 4 | 1 | 2 | 3 | 509KB | // | 5 | 4 | 2 | 5 | 508KB | // | 11 | 1 | 2 | 3 | 7KB | - addRegionInfo(tc, statistics.Read, []testRegionInfo{ + addRegionInfo(tc, utils.Read, []testRegionInfo{ {4, []uint64{1, 2, 3}, 509 * units.KiB, 0, 0}, {5, []uint64{4, 2, 5}, 508 * units.KiB, 0, 0}, }) @@ -1156,7 +1114,7 @@ func TestHotReadRegionScheduleByteRateOnly(t *testing.T) { op = ops[0] operatorutil.CheckTransferPeerWithLeaderTransfer(re, op, operator.OpHotRegion|operator.OpLeader, 1, 5) re.Contains(hb.regionPendings, uint64(1)) - re.True(typeutil.Float64Equal(512.0*units.KiB, hb.regionPendings[1].origin.Loads[statistics.RegionReadBytes])) + re.True(typeutil.Float64Equal(512.0*units.KiB, hb.regionPendings[1].origin.Loads[utils.RegionReadBytes])) clearPendingInfluence(hb) // Should not panic if region not found. @@ -1168,7 +1126,7 @@ func TestHotReadRegionScheduleByteRateOnly(t *testing.T) { hb.updateReadTime = time.Now().Add(-time.Second) hb.Schedule(tc, false) re.Contains(hb.regionPendings, uint64(4)) - re.True(typeutil.Float64Equal(509.0*units.KiB, hb.regionPendings[4].origin.Loads[statistics.RegionReadBytes])) + re.True(typeutil.Float64Equal(509.0*units.KiB, hb.regionPendings[4].origin.Loads[utils.RegionReadBytes])) clearPendingInfluence(hb) } @@ -1179,7 +1137,7 @@ func TestHotReadRegionScheduleWithQuery(t *testing.T) { cancel, _, tc, oc := prepareSchedulersTest() defer cancel() - hb, err := CreateScheduler(statistics.Read.String(), oc, storage.NewStorageWithMemoryBackend(), nil) + hb, err := CreateScheduler(utils.Read.String(), oc, storage.NewStorageWithMemoryBackend(), nil) re.NoError(err) hb.(*hotScheduler).conf.SetSrcToleranceRatio(1) hb.(*hotScheduler).conf.SetDstToleranceRatio(1) @@ -1190,11 +1148,11 @@ func TestHotReadRegionScheduleWithQuery(t *testing.T) { tc.AddRegionStore(2, 20) tc.AddRegionStore(3, 20) - tc.UpdateStorageReadQuery(1, 10500*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadQuery(2, 10000*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadQuery(3, 9000*statistics.StoreHeartBeatReportInterval) + tc.UpdateStorageReadQuery(1, 10500*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadQuery(2, 10000*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadQuery(3, 9000*utils.StoreHeartBeatReportInterval) - addRegionInfo(tc, statistics.Read, []testRegionInfo{ + addRegionInfo(tc, utils.Read, []testRegionInfo{ {1, []uint64{1, 2, 3}, 0, 0, 500}, {2, []uint64{2, 1, 3}, 0, 0, 500}, }) @@ -1214,12 +1172,12 @@ func TestHotReadRegionScheduleWithKeyRate(t *testing.T) { cancel, _, tc, oc := prepareSchedulersTest() defer cancel() - hb, err := CreateScheduler(statistics.Read.String(), oc, storage.NewStorageWithMemoryBackend(), nil) + hb, err := CreateScheduler(utils.Read.String(), oc, storage.NewStorageWithMemoryBackend(), nil) re.NoError(err) hb.(*hotScheduler).conf.RankFormulaVersion = "v1" hb.(*hotScheduler).conf.SetSrcToleranceRatio(1) hb.(*hotScheduler).conf.SetDstToleranceRatio(1) - hb.(*hotScheduler).conf.ReadPriorities = []string{statistics.BytePriority, statistics.KeyPriority} + hb.(*hotScheduler).conf.ReadPriorities = []string{utils.BytePriority, utils.KeyPriority} tc.SetHotRegionCacheHitsThreshold(0) tc.AddRegionStore(1, 20) @@ -1228,13 +1186,13 @@ func TestHotReadRegionScheduleWithKeyRate(t *testing.T) { tc.AddRegionStore(4, 20) tc.AddRegionStore(5, 20) - tc.UpdateStorageReadStats(1, 10.5*units.MiB*statistics.StoreHeartBeatReportInterval, 10.5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadStats(2, 9.5*units.MiB*statistics.StoreHeartBeatReportInterval, 9.5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadStats(3, 9.5*units.MiB*statistics.StoreHeartBeatReportInterval, 9.8*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadStats(4, 9*units.MiB*statistics.StoreHeartBeatReportInterval, 9*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadStats(5, 8.9*units.MiB*statistics.StoreHeartBeatReportInterval, 9.2*units.MiB*statistics.StoreHeartBeatReportInterval) + tc.UpdateStorageReadStats(1, 10.5*units.MiB*utils.StoreHeartBeatReportInterval, 10.5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadStats(2, 9.5*units.MiB*utils.StoreHeartBeatReportInterval, 9.5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadStats(3, 9.5*units.MiB*utils.StoreHeartBeatReportInterval, 9.8*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadStats(4, 9*units.MiB*utils.StoreHeartBeatReportInterval, 9*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadStats(5, 8.9*units.MiB*utils.StoreHeartBeatReportInterval, 9.2*units.MiB*utils.StoreHeartBeatReportInterval) - addRegionInfo(tc, statistics.Read, []testRegionInfo{ + addRegionInfo(tc, utils.Read, []testRegionInfo{ {1, []uint64{1, 2, 4}, 0.5 * units.MiB, 0.5 * units.MiB, 0}, {2, []uint64{1, 2, 4}, 0.5 * units.MiB, 0.5 * units.MiB, 0}, {3, []uint64{3, 4, 5}, 0.05 * units.MiB, 0.1 * units.MiB, 0}, @@ -1276,14 +1234,14 @@ func TestHotReadRegionScheduleWithPendingInfluence(t *testing.T) { func checkHotReadRegionScheduleWithPendingInfluence(re *require.Assertions, dim int) { cancel, _, tc, oc := prepareSchedulersTest() defer cancel() - hb, err := CreateScheduler(statistics.Read.String(), oc, storage.NewStorageWithMemoryBackend(), nil) + hb, err := CreateScheduler(utils.Read.String(), oc, storage.NewStorageWithMemoryBackend(), nil) re.NoError(err) // For test hb.(*hotScheduler).conf.RankFormulaVersion = "v1" hb.(*hotScheduler).conf.GreatDecRatio = 0.99 hb.(*hotScheduler).conf.MinorDecRatio = 1 hb.(*hotScheduler).conf.DstToleranceRatio = 1 - hb.(*hotScheduler).conf.ReadPriorities = []string{statistics.BytePriority, statistics.KeyPriority} + hb.(*hotScheduler).conf.ReadPriorities = []string{utils.BytePriority, utils.KeyPriority} old := pendingAmpFactor pendingAmpFactor = 0.0 defer func() { @@ -1301,13 +1259,13 @@ func checkHotReadRegionScheduleWithPendingInfluence(re *require.Assertions, dim if dim == 1 { // key rate updateStore = tc.UpdateStorageReadKeys } - updateStore(1, 7.1*units.MiB*statistics.StoreHeartBeatReportInterval) - updateStore(2, 6.1*units.MiB*statistics.StoreHeartBeatReportInterval) - updateStore(3, 6*units.MiB*statistics.StoreHeartBeatReportInterval) - updateStore(4, 5*units.MiB*statistics.StoreHeartBeatReportInterval) + updateStore(1, 7.1*units.MiB*utils.StoreHeartBeatReportInterval) + updateStore(2, 6.1*units.MiB*utils.StoreHeartBeatReportInterval) + updateStore(3, 6*units.MiB*utils.StoreHeartBeatReportInterval) + updateStore(4, 5*units.MiB*utils.StoreHeartBeatReportInterval) if dim == 0 { // byte rate - addRegionInfo(tc, statistics.Read, []testRegionInfo{ + addRegionInfo(tc, utils.Read, []testRegionInfo{ {1, []uint64{1, 2, 3}, 512 * units.KiB, 0, 0}, {2, []uint64{1, 2, 3}, 512 * units.KiB, 0, 0}, {3, []uint64{1, 2, 3}, 512 * units.KiB, 0, 0}, @@ -1318,7 +1276,7 @@ func checkHotReadRegionScheduleWithPendingInfluence(re *require.Assertions, dim {8, []uint64{3, 2, 1}, 512 * units.KiB, 0, 0}, }) } else if dim == 1 { // key rate - addRegionInfo(tc, statistics.Read, []testRegionInfo{ + addRegionInfo(tc, utils.Read, []testRegionInfo{ {1, []uint64{1, 2, 3}, 0, 512 * units.KiB, 0}, {2, []uint64{1, 2, 3}, 0, 512 * units.KiB, 0}, {3, []uint64{1, 2, 3}, 0, 512 * units.KiB, 0}, @@ -1394,12 +1352,12 @@ func TestHotReadWithEvictLeaderScheduler(t *testing.T) { cancel, _, tc, oc := prepareSchedulersTest() defer cancel() - hb, err := CreateScheduler(statistics.Read.String(), oc, storage.NewStorageWithMemoryBackend(), nil) + hb, err := CreateScheduler(utils.Read.String(), oc, storage.NewStorageWithMemoryBackend(), nil) re.NoError(err) hb.(*hotScheduler).conf.SetSrcToleranceRatio(1) hb.(*hotScheduler).conf.SetDstToleranceRatio(1) hb.(*hotScheduler).conf.SetStrictPickingStore(false) - hb.(*hotScheduler).conf.ReadPriorities = []string{statistics.BytePriority, statistics.KeyPriority} + hb.(*hotScheduler).conf.ReadPriorities = []string{utils.BytePriority, utils.KeyPriority} tc.SetHotRegionCacheHitsThreshold(0) tc.SetClusterVersion(versioninfo.MinSupportedVersion(versioninfo.Version4_0)) tc.AddRegionStore(1, 20) @@ -1410,11 +1368,11 @@ func TestHotReadWithEvictLeaderScheduler(t *testing.T) { tc.AddRegionStore(6, 20) // no uniform among four stores - tc.UpdateStorageReadStats(1, 10.05*units.MB*statistics.StoreHeartBeatReportInterval, 10.05*units.MB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadStats(2, 10.05*units.MB*statistics.StoreHeartBeatReportInterval, 10.05*units.MB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadStats(3, 10.05*units.MB*statistics.StoreHeartBeatReportInterval, 10.05*units.MB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadStats(4, 0.0*units.MB*statistics.StoreHeartBeatReportInterval, 0.0*units.MB*statistics.StoreHeartBeatReportInterval) - addRegionInfo(tc, statistics.Read, []testRegionInfo{ + tc.UpdateStorageReadStats(1, 10.05*units.MB*utils.StoreHeartBeatReportInterval, 10.05*units.MB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadStats(2, 10.05*units.MB*utils.StoreHeartBeatReportInterval, 10.05*units.MB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadStats(3, 10.05*units.MB*utils.StoreHeartBeatReportInterval, 10.05*units.MB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadStats(4, 0.0*units.MB*utils.StoreHeartBeatReportInterval, 0.0*units.MB*utils.StoreHeartBeatReportInterval) + addRegionInfo(tc, utils.Read, []testRegionInfo{ {1, []uint64{1, 5, 6}, 0.05 * units.MB, 0.05 * units.MB, 0}, }) ops, _ := hb.Schedule(tc, false) @@ -1435,41 +1393,41 @@ func TestHotCacheUpdateCache(t *testing.T) { tc.SetHotRegionCacheHitsThreshold(0) // For read flow - addRegionInfo(tc, statistics.Read, []testRegionInfo{ + addRegionInfo(tc, utils.Read, []testRegionInfo{ {1, []uint64{1, 2, 3}, 512 * units.KiB, 0, 0}, {2, []uint64{2, 1, 3}, 512 * units.KiB, 0, 0}, {3, []uint64{1, 2, 3}, 20 * units.KiB, 0, 0}, // lower than hot read flow rate, but higher than write flow rate {11, []uint64{1, 2, 3}, 7 * units.KiB, 0, 0}, }) - stats := tc.RegionStats(statistics.Read, 0) + stats := tc.RegionStats(utils.Read, 0) re.Len(stats[1], 3) re.Len(stats[2], 3) re.Len(stats[3], 3) - addRegionInfo(tc, statistics.Read, []testRegionInfo{ + addRegionInfo(tc, utils.Read, []testRegionInfo{ {3, []uint64{2, 1, 3}, 20 * units.KiB, 0, 0}, {11, []uint64{1, 2, 3}, 7 * units.KiB, 0, 0}, }) - stats = tc.RegionStats(statistics.Read, 0) + stats = tc.RegionStats(utils.Read, 0) re.Len(stats[1], 3) re.Len(stats[2], 3) re.Len(stats[3], 3) - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + addRegionInfo(tc, utils.Write, []testRegionInfo{ {4, []uint64{1, 2, 3}, 512 * units.KiB, 0, 0}, {5, []uint64{1, 2, 3}, 20 * units.KiB, 0, 0}, {6, []uint64{1, 2, 3}, 0.8 * units.KiB, 0, 0}, }) - stats = tc.RegionStats(statistics.Write, 0) + stats = tc.RegionStats(utils.Write, 0) re.Len(stats[1], 2) re.Len(stats[2], 2) re.Len(stats[3], 2) - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + addRegionInfo(tc, utils.Write, []testRegionInfo{ {5, []uint64{1, 2, 5}, 20 * units.KiB, 0, 0}, }) - stats = tc.RegionStats(statistics.Write, 0) + stats = tc.RegionStats(utils.Write, 0) re.Len(stats[1], 2) re.Len(stats[2], 2) @@ -1484,7 +1442,7 @@ func TestHotCacheUpdateCache(t *testing.T) { // lower than hot read flow rate, but higher than write flow rate {31, []uint64{4, 5, 6}, 7 * units.KiB, 0, 0}, }) - stats = tc.RegionStats(statistics.Read, 0) + stats = tc.RegionStats(utils.Read, 0) re.Len(stats[4], 2) re.Len(stats[5], 1) re.Empty(stats[6]) @@ -1500,17 +1458,17 @@ func TestHotCacheKeyThresholds(t *testing.T) { cancel, _, tc, _ := prepareSchedulersTest() defer cancel() tc.SetHotRegionCacheHitsThreshold(0) - addRegionInfo(tc, statistics.Read, []testRegionInfo{ + addRegionInfo(tc, utils.Read, []testRegionInfo{ {1, []uint64{1, 2, 3}, 0, 1, 0}, {2, []uint64{1, 2, 3}, 0, 1 * units.KiB, 0}, }) - stats := tc.RegionStats(statistics.Read, 0) + stats := tc.RegionStats(utils.Read, 0) re.Len(stats[1], 1) - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + addRegionInfo(tc, utils.Write, []testRegionInfo{ {3, []uint64{4, 5, 6}, 0, 1, 0}, {4, []uint64{4, 5, 6}, 0, 1 * units.KiB, 0}, }) - stats = tc.RegionStats(statistics.Write, 0) + stats = tc.RegionStats(utils.Write, 0) re.Len(stats[4], 1) re.Len(stats[5], 1) re.Len(stats[6], 1) @@ -1535,31 +1493,31 @@ func TestHotCacheKeyThresholds(t *testing.T) { } { // read - addRegionInfo(tc, statistics.Read, regions) - stats := tc.RegionStats(statistics.Read, 0) + addRegionInfo(tc, utils.Read, regions) + stats := tc.RegionStats(utils.Read, 0) re.Greater(len(stats[1]), 500) // for AntiCount - addRegionInfo(tc, statistics.Read, regions) - addRegionInfo(tc, statistics.Read, regions) - addRegionInfo(tc, statistics.Read, regions) - addRegionInfo(tc, statistics.Read, regions) - stats = tc.RegionStats(statistics.Read, 0) + addRegionInfo(tc, utils.Read, regions) + addRegionInfo(tc, utils.Read, regions) + addRegionInfo(tc, utils.Read, regions) + addRegionInfo(tc, utils.Read, regions) + stats = tc.RegionStats(utils.Read, 0) re.Len(stats[1], 500) } { // write - addRegionInfo(tc, statistics.Write, regions) - stats := tc.RegionStats(statistics.Write, 0) + addRegionInfo(tc, utils.Write, regions) + stats := tc.RegionStats(utils.Write, 0) re.Greater(len(stats[1]), 500) re.Greater(len(stats[2]), 500) re.Greater(len(stats[3]), 500) // for AntiCount - addRegionInfo(tc, statistics.Write, regions) - addRegionInfo(tc, statistics.Write, regions) - addRegionInfo(tc, statistics.Write, regions) - addRegionInfo(tc, statistics.Write, regions) - stats = tc.RegionStats(statistics.Write, 0) + addRegionInfo(tc, utils.Write, regions) + addRegionInfo(tc, utils.Write, regions) + addRegionInfo(tc, utils.Write, regions) + addRegionInfo(tc, utils.Write, regions) + stats = tc.RegionStats(utils.Write, 0) re.Len(stats[1], 500) re.Len(stats[2], 500) re.Len(stats[3], 500) @@ -1586,32 +1544,32 @@ func TestHotCacheByteAndKey(t *testing.T) { }) } { // read - addRegionInfo(tc, statistics.Read, regions) - stats := tc.RegionStats(statistics.Read, 0) + addRegionInfo(tc, utils.Read, regions) + stats := tc.RegionStats(utils.Read, 0) re.Len(stats[1], 500) - addRegionInfo(tc, statistics.Read, []testRegionInfo{ + addRegionInfo(tc, utils.Read, []testRegionInfo{ {10001, []uint64{1, 2, 3}, 10 * units.KiB, 10 * units.KiB, 0}, {10002, []uint64{1, 2, 3}, 500 * units.KiB, 10 * units.KiB, 0}, {10003, []uint64{1, 2, 3}, 10 * units.KiB, 500 * units.KiB, 0}, {10004, []uint64{1, 2, 3}, 500 * units.KiB, 500 * units.KiB, 0}, }) - stats = tc.RegionStats(statistics.Read, 0) + stats = tc.RegionStats(utils.Read, 0) re.Len(stats[1], 503) } { // write - addRegionInfo(tc, statistics.Write, regions) - stats := tc.RegionStats(statistics.Write, 0) + addRegionInfo(tc, utils.Write, regions) + stats := tc.RegionStats(utils.Write, 0) re.Len(stats[1], 500) re.Len(stats[2], 500) re.Len(stats[3], 500) - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + addRegionInfo(tc, utils.Write, []testRegionInfo{ {10001, []uint64{1, 2, 3}, 10 * units.KiB, 10 * units.KiB, 0}, {10002, []uint64{1, 2, 3}, 500 * units.KiB, 10 * units.KiB, 0}, {10003, []uint64{1, 2, 3}, 10 * units.KiB, 500 * units.KiB, 0}, {10004, []uint64{1, 2, 3}, 500 * units.KiB, 500 * units.KiB, 0}, }) - stats = tc.RegionStats(statistics.Write, 0) + stats = tc.RegionStats(utils.Write, 0) re.Len(stats[1], 503) re.Len(stats[2], 503) re.Len(stats[3], 503) @@ -1627,14 +1585,14 @@ type testRegionInfo struct { queryRate float64 } -func addRegionInfo(tc *mockcluster.Cluster, rwTy statistics.RWType, regions []testRegionInfo) { +func addRegionInfo(tc *mockcluster.Cluster, rwTy utils.RWType, regions []testRegionInfo) { addFunc := tc.AddRegionWithReadInfo - if rwTy == statistics.Write { + if rwTy == utils.Write { addFunc = tc.AddLeaderRegionWithWriteInfo } - reportIntervalSecs := statistics.WriteReportInterval - if rwTy == statistics.Read { - reportIntervalSecs = statistics.ReadReportInterval + reportIntervalSecs := utils.RegionHeartBeatReportInterval + if rwTy == utils.Read { + reportIntervalSecs = utils.StoreHeartBeatReportInterval } for _, r := range regions { addFunc( @@ -1650,7 +1608,7 @@ func addRegionInfo(tc *mockcluster.Cluster, rwTy statistics.RWType, regions []te func addRegionLeaderReadInfo(tc *mockcluster.Cluster, regions []testRegionInfo) { addFunc := tc.AddRegionLeaderWithReadInfo - reportIntervalSecs := statistics.ReadReportInterval + reportIntervalSecs := utils.StoreHeartBeatReportInterval for _, r := range regions { addFunc( r.id, r.peers[0], @@ -1664,7 +1622,7 @@ func addRegionLeaderReadInfo(tc *mockcluster.Cluster, regions []testRegionInfo) } type testHotCacheCheckRegionFlowCase struct { - kind statistics.RWType + kind utils.RWType onlyLeader bool DegreeAfterTransferLeader int } @@ -1673,17 +1631,17 @@ func TestHotCacheCheckRegionFlow(t *testing.T) { re := require.New(t) testCases := []testHotCacheCheckRegionFlowCase{ { - kind: statistics.Write, + kind: utils.Write, onlyLeader: false, DegreeAfterTransferLeader: 3, }, { - kind: statistics.Read, + kind: utils.Read, onlyLeader: false, DegreeAfterTransferLeader: 4, }, { - kind: statistics.Read, + kind: utils.Read, onlyLeader: true, DegreeAfterTransferLeader: 1, }, @@ -1706,7 +1664,7 @@ func checkHotCacheCheckRegionFlow(re *require.Assertions, testCase testHotCacheC re.NoError(err) hb := sche.(*hotScheduler) heartbeat := tc.AddLeaderRegionWithWriteInfo - if testCase.kind == statistics.Read { + if testCase.kind == utils.Read { if testCase.onlyLeader { heartbeat = tc.AddRegionLeaderWithReadInfo } else { @@ -1714,10 +1672,10 @@ func checkHotCacheCheckRegionFlow(re *require.Assertions, testCase testHotCacheC } } tc.AddRegionStore(2, 20) - tc.UpdateStorageReadStats(2, 9.5*units.MiB*statistics.StoreHeartBeatReportInterval, 9.5*units.MiB*statistics.StoreHeartBeatReportInterval) - reportInterval := uint64(statistics.WriteReportInterval) - if testCase.kind == statistics.Read { - reportInterval = uint64(statistics.ReadReportInterval) + tc.UpdateStorageReadStats(2, 9.5*units.MiB*utils.StoreHeartBeatReportInterval, 9.5*units.MiB*utils.StoreHeartBeatReportInterval) + reportInterval := uint64(utils.RegionHeartBeatReportInterval) + if testCase.kind == utils.Read { + reportInterval = uint64(utils.StoreHeartBeatReportInterval) } // hot degree increase heartbeat(1, 1, 512*units.KiB*reportInterval, 0, 0, reportInterval, []uint64{2, 3}, 1) @@ -1757,7 +1715,7 @@ func checkHotCacheCheckRegionFlow(re *require.Assertions, testCase testHotCacheC re.NotEmpty(items) for _, item := range items { if item.StoreID == 3 { - re.Equal(statistics.Remove, item.GetActionType()) + re.Equal(utils.Remove, item.GetActionType()) continue } re.Equal(testCase.DegreeAfterTransferLeader+2, item.HotDegree) @@ -1779,28 +1737,28 @@ func checkHotCacheCheckRegionFlowWithDifferentThreshold(re *require.Assertions, tc.SetMaxReplicasWithLabel(enablePlacementRules, 3, labels...) statistics.ThresholdsUpdateInterval = 0 defer func() { - statistics.ThresholdsUpdateInterval = statistics.StoreHeartBeatReportInterval + statistics.ThresholdsUpdateInterval = utils.StoreHeartBeatReportInterval }() // some peers are hot, and some are cold #3198 rate := uint64(512 * units.KiB) for i := 0; i < statistics.TopNN; i++ { - for j := 0; j < statistics.DefaultAotSize; j++ { - tc.AddLeaderRegionWithWriteInfo(uint64(i+100), 1, rate*statistics.WriteReportInterval, 0, 0, statistics.WriteReportInterval, []uint64{2, 3}, 1) + for j := 0; j < utils.DefaultAotSize; j++ { + tc.AddLeaderRegionWithWriteInfo(uint64(i+100), 1, rate*utils.RegionHeartBeatReportInterval, 0, 0, utils.RegionHeartBeatReportInterval, []uint64{2, 3}, 1) } } - items := tc.AddLeaderRegionWithWriteInfo(201, 1, rate*statistics.WriteReportInterval, 0, 0, statistics.WriteReportInterval, []uint64{2, 3}, 1) - re.Equal(float64(rate)*statistics.HotThresholdRatio, tc.HotCache.GetThresholds(statistics.Write, items[0].StoreID)[0]) + items := tc.AddLeaderRegionWithWriteInfo(201, 1, rate*utils.RegionHeartBeatReportInterval, 0, 0, utils.RegionHeartBeatReportInterval, []uint64{2, 3}, 1) + re.Equal(float64(rate)*statistics.HotThresholdRatio, tc.HotCache.GetThresholds(utils.Write, items[0].StoreID)[0]) // Threshold of store 1,2,3 is 409.6 units.KiB and others are 1 units.KiB // Make the hot threshold of some store is high and the others are low rate = 10 * units.KiB - tc.AddLeaderRegionWithWriteInfo(201, 1, rate*statistics.WriteReportInterval, 0, 0, statistics.WriteReportInterval, []uint64{2, 3, 4}, 1) - items = tc.AddLeaderRegionWithWriteInfo(201, 1, rate*statistics.WriteReportInterval, 0, 0, statistics.WriteReportInterval, []uint64{3, 4}, 1) + tc.AddLeaderRegionWithWriteInfo(201, 1, rate*utils.RegionHeartBeatReportInterval, 0, 0, utils.RegionHeartBeatReportInterval, []uint64{2, 3, 4}, 1) + items = tc.AddLeaderRegionWithWriteInfo(201, 1, rate*utils.RegionHeartBeatReportInterval, 0, 0, utils.RegionHeartBeatReportInterval, []uint64{3, 4}, 1) for _, item := range items { if item.StoreID < 4 { - re.Equal(statistics.Remove, item.GetActionType()) + re.Equal(utils.Remove, item.GetActionType()) } else { - re.Equal(statistics.Update, item.GetActionType()) + re.Equal(utils.Update, item.GetActionType()) } } } @@ -1812,24 +1770,24 @@ func TestHotCacheSortHotPeer(t *testing.T) { sche, err := CreateScheduler(HotRegionType, oc, storage.NewStorageWithMemoryBackend(), ConfigJSONDecoder([]byte("null"))) re.NoError(err) hb := sche.(*hotScheduler) - leaderSolver := newBalanceSolver(hb, tc, statistics.Read, transferLeader) + leaderSolver := newBalanceSolver(hb, tc, utils.Read, transferLeader) hotPeers := []*statistics.HotPeerStat{{ RegionID: 1, Loads: []float64{ - statistics.QueryDim: 10, - statistics.ByteDim: 1, + utils.QueryDim: 10, + utils.ByteDim: 1, }, }, { RegionID: 2, Loads: []float64{ - statistics.QueryDim: 1, - statistics.ByteDim: 10, + utils.QueryDim: 1, + utils.ByteDim: 10, }, }, { RegionID: 3, Loads: []float64{ - statistics.QueryDim: 5, - statistics.ByteDim: 6, + utils.QueryDim: 5, + utils.ByteDim: 6, }, }} @@ -1868,7 +1826,7 @@ func TestInfluenceByRWType(t *testing.T) { cancel, _, tc, oc := prepareSchedulersTest() defer cancel() - hb, err := CreateScheduler(statistics.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) + hb, err := CreateScheduler(utils.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) re.NoError(err) hb.(*hotScheduler).conf.SetDstToleranceRatio(1) hb.(*hotScheduler).conf.SetSrcToleranceRatio(1) @@ -1878,14 +1836,14 @@ func TestInfluenceByRWType(t *testing.T) { tc.AddRegionStore(2, 20) tc.AddRegionStore(3, 20) tc.AddRegionStore(4, 20) - tc.UpdateStorageWrittenStats(1, 99*units.MiB*statistics.StoreHeartBeatReportInterval, 99*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(2, 50*units.MiB*statistics.StoreHeartBeatReportInterval, 98*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(3, 2*units.MiB*statistics.StoreHeartBeatReportInterval, 2*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(4, 1*units.MiB*statistics.StoreHeartBeatReportInterval, 1*units.MiB*statistics.StoreHeartBeatReportInterval) - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + tc.UpdateStorageWrittenStats(1, 99*units.MiB*utils.StoreHeartBeatReportInterval, 99*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(2, 50*units.MiB*utils.StoreHeartBeatReportInterval, 98*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(3, 2*units.MiB*utils.StoreHeartBeatReportInterval, 2*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(4, 1*units.MiB*utils.StoreHeartBeatReportInterval, 1*units.MiB*utils.StoreHeartBeatReportInterval) + addRegionInfo(tc, utils.Write, []testRegionInfo{ {1, []uint64{2, 1, 3}, 0.5 * units.MiB, 0.5 * units.MiB, 0}, }) - addRegionInfo(tc, statistics.Read, []testRegionInfo{ + addRegionInfo(tc, utils.Read, []testRegionInfo{ {1, []uint64{2, 1, 3}, 0.5 * units.MiB, 0.5 * units.MiB, 0}, }) // must move peer @@ -1897,18 +1855,18 @@ func TestInfluenceByRWType(t *testing.T) { hb.(*hotScheduler).summaryPendingInfluence() stInfos := hb.(*hotScheduler).stInfos - re.True(nearlyAbout(stInfos[1].PendingSum.Loads[statistics.RegionWriteKeys], -0.5*units.MiB)) - re.True(nearlyAbout(stInfos[1].PendingSum.Loads[statistics.RegionWriteBytes], -0.5*units.MiB)) - re.True(nearlyAbout(stInfos[4].PendingSum.Loads[statistics.RegionWriteKeys], 0.5*units.MiB)) - re.True(nearlyAbout(stInfos[4].PendingSum.Loads[statistics.RegionWriteBytes], 0.5*units.MiB)) - re.True(nearlyAbout(stInfos[1].PendingSum.Loads[statistics.RegionReadKeys], -0.5*units.MiB)) - re.True(nearlyAbout(stInfos[1].PendingSum.Loads[statistics.RegionReadBytes], -0.5*units.MiB)) - re.True(nearlyAbout(stInfos[4].PendingSum.Loads[statistics.RegionReadKeys], 0.5*units.MiB)) - re.True(nearlyAbout(stInfos[4].PendingSum.Loads[statistics.RegionReadBytes], 0.5*units.MiB)) + re.True(nearlyAbout(stInfos[1].PendingSum.Loads[utils.RegionWriteKeys], -0.5*units.MiB)) + re.True(nearlyAbout(stInfos[1].PendingSum.Loads[utils.RegionWriteBytes], -0.5*units.MiB)) + re.True(nearlyAbout(stInfos[4].PendingSum.Loads[utils.RegionWriteKeys], 0.5*units.MiB)) + re.True(nearlyAbout(stInfos[4].PendingSum.Loads[utils.RegionWriteBytes], 0.5*units.MiB)) + re.True(nearlyAbout(stInfos[1].PendingSum.Loads[utils.RegionReadKeys], -0.5*units.MiB)) + re.True(nearlyAbout(stInfos[1].PendingSum.Loads[utils.RegionReadBytes], -0.5*units.MiB)) + re.True(nearlyAbout(stInfos[4].PendingSum.Loads[utils.RegionReadKeys], 0.5*units.MiB)) + re.True(nearlyAbout(stInfos[4].PendingSum.Loads[utils.RegionReadBytes], 0.5*units.MiB)) // consider pending amp, there are nine regions or more. for i := 2; i < 13; i++ { - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + addRegionInfo(tc, utils.Write, []testRegionInfo{ {uint64(i), []uint64{1, 2, 3}, 0.7 * units.MiB, 0.7 * units.MiB, 0}, }) } @@ -1923,14 +1881,14 @@ func TestInfluenceByRWType(t *testing.T) { hb.(*hotScheduler).summaryPendingInfluence() stInfos = hb.(*hotScheduler).stInfos // assert read/write influence is the sum of write peer and write leader - re.True(nearlyAbout(stInfos[1].PendingSum.Loads[statistics.RegionWriteKeys], -1.2*units.MiB)) - re.True(nearlyAbout(stInfos[1].PendingSum.Loads[statistics.RegionWriteBytes], -1.2*units.MiB)) - re.True(nearlyAbout(stInfos[3].PendingSum.Loads[statistics.RegionWriteKeys], 0.7*units.MiB)) - re.True(nearlyAbout(stInfos[3].PendingSum.Loads[statistics.RegionWriteBytes], 0.7*units.MiB)) - re.True(nearlyAbout(stInfos[1].PendingSum.Loads[statistics.RegionReadKeys], -1.2*units.MiB)) - re.True(nearlyAbout(stInfos[1].PendingSum.Loads[statistics.RegionReadBytes], -1.2*units.MiB)) - re.True(nearlyAbout(stInfos[3].PendingSum.Loads[statistics.RegionReadKeys], 0.7*units.MiB)) - re.True(nearlyAbout(stInfos[3].PendingSum.Loads[statistics.RegionReadBytes], 0.7*units.MiB)) + re.True(nearlyAbout(stInfos[1].PendingSum.Loads[utils.RegionWriteKeys], -1.2*units.MiB)) + re.True(nearlyAbout(stInfos[1].PendingSum.Loads[utils.RegionWriteBytes], -1.2*units.MiB)) + re.True(nearlyAbout(stInfos[3].PendingSum.Loads[utils.RegionWriteKeys], 0.7*units.MiB)) + re.True(nearlyAbout(stInfos[3].PendingSum.Loads[utils.RegionWriteBytes], 0.7*units.MiB)) + re.True(nearlyAbout(stInfos[1].PendingSum.Loads[utils.RegionReadKeys], -1.2*units.MiB)) + re.True(nearlyAbout(stInfos[1].PendingSum.Loads[utils.RegionReadBytes], -1.2*units.MiB)) + re.True(nearlyAbout(stInfos[3].PendingSum.Loads[utils.RegionReadKeys], 0.7*units.MiB)) + re.True(nearlyAbout(stInfos[3].PendingSum.Loads[utils.RegionReadBytes], 0.7*units.MiB)) } func nearlyAbout(f1, f2 float64) bool { @@ -1941,7 +1899,7 @@ func nearlyAbout(f1, f2 float64) bool { } func loadsEqual(loads1, loads2 []float64) bool { - if len(loads1) != statistics.DimLen || len(loads2) != statistics.DimLen { + if len(loads1) != utils.DimLen || len(loads2) != utils.DimLen { return false } for i, load := range loads1 { @@ -1969,10 +1927,10 @@ func checkHotReadPeerSchedule(re *require.Assertions, enablePlacementRules bool) tc.PutStoreWithLabels(id) } - sche, err := CreateScheduler(statistics.Read.String(), oc, storage.NewStorageWithMemoryBackend(), ConfigJSONDecoder([]byte("null"))) + sche, err := CreateScheduler(utils.Read.String(), oc, storage.NewStorageWithMemoryBackend(), ConfigJSONDecoder([]byte("null"))) re.NoError(err) hb := sche.(*hotScheduler) - hb.conf.ReadPriorities = []string{statistics.BytePriority, statistics.KeyPriority} + hb.conf.ReadPriorities = []string{utils.BytePriority, utils.KeyPriority} tc.UpdateStorageReadStats(1, 20*units.MiB, 20*units.MiB) tc.UpdateStorageReadStats(2, 19*units.MiB, 19*units.MiB) @@ -1991,7 +1949,7 @@ func TestHotScheduleWithPriority(t *testing.T) { cancel, _, tc, oc := prepareSchedulersTest() defer cancel() - hb, err := CreateScheduler(statistics.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) + hb, err := CreateScheduler(utils.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) re.NoError(err) hb.(*hotScheduler).conf.SetDstToleranceRatio(1.05) hb.(*hotScheduler).conf.SetSrcToleranceRatio(1.05) @@ -2010,61 +1968,61 @@ func TestHotScheduleWithPriority(t *testing.T) { tc.AddRegionStore(4, 20) tc.AddRegionStore(5, 20) - tc.UpdateStorageWrittenStats(1, 10*units.MiB*statistics.StoreHeartBeatReportInterval, 9*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(2, 6*units.MiB*statistics.StoreHeartBeatReportInterval, 6*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(3, 6*units.MiB*statistics.StoreHeartBeatReportInterval, 6*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(4, 9*units.MiB*statistics.StoreHeartBeatReportInterval, 10*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(5, 1*units.MiB*statistics.StoreHeartBeatReportInterval, 1*units.MiB*statistics.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(1, 10*units.MiB*utils.StoreHeartBeatReportInterval, 9*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(2, 6*units.MiB*utils.StoreHeartBeatReportInterval, 6*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(3, 6*units.MiB*utils.StoreHeartBeatReportInterval, 6*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(4, 9*units.MiB*utils.StoreHeartBeatReportInterval, 10*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(5, 1*units.MiB*utils.StoreHeartBeatReportInterval, 1*units.MiB*utils.StoreHeartBeatReportInterval) // must transfer peer schedulePeerPr = 1.0 - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + addRegionInfo(tc, utils.Write, []testRegionInfo{ {1, []uint64{1, 2, 3}, 2 * units.MiB, 1 * units.MiB, 0}, {6, []uint64{4, 2, 3}, 1 * units.MiB, 2 * units.MiB, 0}, }) - hb.(*hotScheduler).conf.WritePeerPriorities = []string{statistics.BytePriority, statistics.KeyPriority} + hb.(*hotScheduler).conf.WritePeerPriorities = []string{utils.BytePriority, utils.KeyPriority} ops, _ := hb.Schedule(tc, false) re.Len(ops, 1) operatorutil.CheckTransferPeer(re, ops[0], operator.OpHotRegion, 1, 5) clearPendingInfluence(hb.(*hotScheduler)) - hb.(*hotScheduler).conf.WritePeerPriorities = []string{statistics.KeyPriority, statistics.BytePriority} + hb.(*hotScheduler).conf.WritePeerPriorities = []string{utils.KeyPriority, utils.BytePriority} ops, _ = hb.Schedule(tc, false) re.Len(ops, 1) operatorutil.CheckTransferPeer(re, ops[0], operator.OpHotRegion, 4, 5) clearPendingInfluence(hb.(*hotScheduler)) // assert read priority schedule - hb, err = CreateScheduler(statistics.Read.String(), oc, storage.NewStorageWithMemoryBackend(), nil) + hb, err = CreateScheduler(utils.Read.String(), oc, storage.NewStorageWithMemoryBackend(), nil) re.NoError(err) - tc.UpdateStorageReadStats(5, 10*units.MiB*statistics.StoreHeartBeatReportInterval, 10*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadStats(4, 10*units.MiB*statistics.StoreHeartBeatReportInterval, 10*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadStats(1, 10*units.MiB*statistics.StoreHeartBeatReportInterval, 10*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadStats(2, 1*units.MiB*statistics.StoreHeartBeatReportInterval, 7*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadStats(3, 7*units.MiB*statistics.StoreHeartBeatReportInterval, 1*units.MiB*statistics.StoreHeartBeatReportInterval) - addRegionInfo(tc, statistics.Read, []testRegionInfo{ + tc.UpdateStorageReadStats(5, 10*units.MiB*utils.StoreHeartBeatReportInterval, 10*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadStats(4, 10*units.MiB*utils.StoreHeartBeatReportInterval, 10*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadStats(1, 10*units.MiB*utils.StoreHeartBeatReportInterval, 10*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadStats(2, 1*units.MiB*utils.StoreHeartBeatReportInterval, 7*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadStats(3, 7*units.MiB*utils.StoreHeartBeatReportInterval, 1*units.MiB*utils.StoreHeartBeatReportInterval) + addRegionInfo(tc, utils.Read, []testRegionInfo{ {1, []uint64{1, 2, 3}, 2 * units.MiB, 2 * units.MiB, 0}, }) - hb.(*hotScheduler).conf.ReadPriorities = []string{statistics.BytePriority, statistics.KeyPriority} + hb.(*hotScheduler).conf.ReadPriorities = []string{utils.BytePriority, utils.KeyPriority} ops, _ = hb.Schedule(tc, false) re.Len(ops, 1) operatorutil.CheckTransferLeader(re, ops[0], operator.OpHotRegion, 1, 2) clearPendingInfluence(hb.(*hotScheduler)) - hb.(*hotScheduler).conf.ReadPriorities = []string{statistics.KeyPriority, statistics.BytePriority} + hb.(*hotScheduler).conf.ReadPriorities = []string{utils.KeyPriority, utils.BytePriority} ops, _ = hb.Schedule(tc, false) re.Len(ops, 1) operatorutil.CheckTransferLeader(re, ops[0], operator.OpHotRegion, 1, 3) - hb, err = CreateScheduler(statistics.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) - hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{statistics.KeyPriority, statistics.BytePriority} + hb, err = CreateScheduler(utils.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) + hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{utils.KeyPriority, utils.BytePriority} hb.(*hotScheduler).conf.RankFormulaVersion = "v1" re.NoError(err) // assert loose store picking - tc.UpdateStorageWrittenStats(1, 10*units.MiB*statistics.StoreHeartBeatReportInterval, 1*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(2, 6.1*units.MiB*statistics.StoreHeartBeatReportInterval, 6.1*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(3, 6*units.MiB*statistics.StoreHeartBeatReportInterval, 6*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(4, 6*units.MiB*statistics.StoreHeartBeatReportInterval, 6*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(5, 1*units.MiB*statistics.StoreHeartBeatReportInterval, 1*units.MiB*statistics.StoreHeartBeatReportInterval) - hb.(*hotScheduler).conf.WritePeerPriorities = []string{statistics.BytePriority, statistics.KeyPriority} + tc.UpdateStorageWrittenStats(1, 10*units.MiB*utils.StoreHeartBeatReportInterval, 1*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(2, 6.1*units.MiB*utils.StoreHeartBeatReportInterval, 6.1*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(3, 6*units.MiB*utils.StoreHeartBeatReportInterval, 6*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(4, 6*units.MiB*utils.StoreHeartBeatReportInterval, 6*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(5, 1*units.MiB*utils.StoreHeartBeatReportInterval, 1*units.MiB*utils.StoreHeartBeatReportInterval) + hb.(*hotScheduler).conf.WritePeerPriorities = []string{utils.BytePriority, utils.KeyPriority} hb.(*hotScheduler).conf.StrictPickingStore = true ops, _ = hb.Schedule(tc, false) re.Empty(ops) @@ -2074,12 +2032,12 @@ func TestHotScheduleWithPriority(t *testing.T) { operatorutil.CheckTransferPeer(re, ops[0], operator.OpHotRegion, 1, 5) clearPendingInfluence(hb.(*hotScheduler)) - tc.UpdateStorageWrittenStats(1, 6*units.MiB*statistics.StoreHeartBeatReportInterval, 6*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(2, 6.1*units.MiB*statistics.StoreHeartBeatReportInterval, 6.1*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(3, 6*units.MiB*statistics.StoreHeartBeatReportInterval, 6*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(4, 1*units.MiB*statistics.StoreHeartBeatReportInterval, 10*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(5, 1*units.MiB*statistics.StoreHeartBeatReportInterval, 1*units.MiB*statistics.StoreHeartBeatReportInterval) - hb.(*hotScheduler).conf.WritePeerPriorities = []string{statistics.KeyPriority, statistics.BytePriority} + tc.UpdateStorageWrittenStats(1, 6*units.MiB*utils.StoreHeartBeatReportInterval, 6*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(2, 6.1*units.MiB*utils.StoreHeartBeatReportInterval, 6.1*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(3, 6*units.MiB*utils.StoreHeartBeatReportInterval, 6*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(4, 1*units.MiB*utils.StoreHeartBeatReportInterval, 10*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(5, 1*units.MiB*utils.StoreHeartBeatReportInterval, 1*units.MiB*utils.StoreHeartBeatReportInterval) + hb.(*hotScheduler).conf.WritePeerPriorities = []string{utils.KeyPriority, utils.BytePriority} hb.(*hotScheduler).conf.StrictPickingStore = true ops, _ = hb.Schedule(tc, false) re.Empty(ops) @@ -2097,7 +2055,7 @@ func TestHotScheduleWithStddev(t *testing.T) { cancel, _, tc, oc := prepareSchedulersTest() defer cancel() - hb, err := CreateScheduler(statistics.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) + hb, err := CreateScheduler(utils.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) re.NoError(err) hb.(*hotScheduler).conf.SetDstToleranceRatio(1.0) hb.(*hotScheduler).conf.SetSrcToleranceRatio(1.0) @@ -2112,15 +2070,15 @@ func TestHotScheduleWithStddev(t *testing.T) { hb.(*hotScheduler).conf.StrictPickingStore = false // skip uniform cluster - tc.UpdateStorageWrittenStats(1, 5*units.MiB*statistics.StoreHeartBeatReportInterval, 5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(2, 5.3*units.MiB*statistics.StoreHeartBeatReportInterval, 5.3*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(3, 5*units.MiB*statistics.StoreHeartBeatReportInterval, 5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(4, 5*units.MiB*statistics.StoreHeartBeatReportInterval, 5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(5, 4.8*units.MiB*statistics.StoreHeartBeatReportInterval, 4.8*units.MiB*statistics.StoreHeartBeatReportInterval) - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + tc.UpdateStorageWrittenStats(1, 5*units.MiB*utils.StoreHeartBeatReportInterval, 5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(2, 5.3*units.MiB*utils.StoreHeartBeatReportInterval, 5.3*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(3, 5*units.MiB*utils.StoreHeartBeatReportInterval, 5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(4, 5*units.MiB*utils.StoreHeartBeatReportInterval, 5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(5, 4.8*units.MiB*utils.StoreHeartBeatReportInterval, 4.8*units.MiB*utils.StoreHeartBeatReportInterval) + addRegionInfo(tc, utils.Write, []testRegionInfo{ {6, []uint64{3, 4, 2}, 0.1 * units.MiB, 0.1 * units.MiB, 0}, }) - hb.(*hotScheduler).conf.WritePeerPriorities = []string{statistics.BytePriority, statistics.KeyPriority} + hb.(*hotScheduler).conf.WritePeerPriorities = []string{utils.BytePriority, utils.KeyPriority} stddevThreshold = 0.1 ops, _ := hb.Schedule(tc, false) re.Empty(ops) @@ -2131,15 +2089,15 @@ func TestHotScheduleWithStddev(t *testing.T) { clearPendingInfluence(hb.(*hotScheduler)) // skip -1 case (uniform cluster) - tc.UpdateStorageWrittenStats(1, 5*units.MiB*statistics.StoreHeartBeatReportInterval, 100*units.MiB*statistics.StoreHeartBeatReportInterval) // two dims are not uniform. - tc.UpdateStorageWrittenStats(2, 5.3*units.MiB*statistics.StoreHeartBeatReportInterval, 4.8*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(3, 5*units.MiB*statistics.StoreHeartBeatReportInterval, 5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(4, 5*units.MiB*statistics.StoreHeartBeatReportInterval, 5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(5, 4.8*units.MiB*statistics.StoreHeartBeatReportInterval, 5*units.MiB*statistics.StoreHeartBeatReportInterval) - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + tc.UpdateStorageWrittenStats(1, 5*units.MiB*utils.StoreHeartBeatReportInterval, 100*units.MiB*utils.StoreHeartBeatReportInterval) // two dims are not uniform. + tc.UpdateStorageWrittenStats(2, 5.3*units.MiB*utils.StoreHeartBeatReportInterval, 4.8*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(3, 5*units.MiB*utils.StoreHeartBeatReportInterval, 5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(4, 5*units.MiB*utils.StoreHeartBeatReportInterval, 5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(5, 4.8*units.MiB*utils.StoreHeartBeatReportInterval, 5*units.MiB*utils.StoreHeartBeatReportInterval) + addRegionInfo(tc, utils.Write, []testRegionInfo{ {6, []uint64{3, 4, 2}, 0.1 * units.MiB, 0.1 * units.MiB, 0}, }) - hb.(*hotScheduler).conf.WritePeerPriorities = []string{statistics.BytePriority, statistics.KeyPriority} + hb.(*hotScheduler).conf.WritePeerPriorities = []string{utils.BytePriority, utils.KeyPriority} stddevThreshold = 0.1 ops, _ = hb.Schedule(tc, false) re.Empty(ops) @@ -2157,7 +2115,7 @@ func TestHotWriteLeaderScheduleWithPriority(t *testing.T) { cancel, _, tc, oc := prepareSchedulersTest() defer cancel() - hb, err := CreateScheduler(statistics.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) + hb, err := CreateScheduler(utils.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) re.NoError(err) hb.(*hotScheduler).conf.SetDstToleranceRatio(1) hb.(*hotScheduler).conf.SetSrcToleranceRatio(1) @@ -2167,11 +2125,11 @@ func TestHotWriteLeaderScheduleWithPriority(t *testing.T) { tc.AddRegionStore(1, 20) tc.AddRegionStore(2, 20) tc.AddRegionStore(3, 20) - tc.UpdateStorageWrittenStats(1, 31*units.MiB*statistics.StoreHeartBeatReportInterval, 31*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(2, 10*units.MiB*statistics.StoreHeartBeatReportInterval, 1*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(3, 1*units.MiB*statistics.StoreHeartBeatReportInterval, 10*units.MiB*statistics.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(1, 31*units.MiB*utils.StoreHeartBeatReportInterval, 31*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(2, 10*units.MiB*utils.StoreHeartBeatReportInterval, 1*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(3, 1*units.MiB*utils.StoreHeartBeatReportInterval, 10*units.MiB*utils.StoreHeartBeatReportInterval) - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + addRegionInfo(tc, utils.Write, []testRegionInfo{ {1, []uint64{1, 2, 3}, 10 * units.MiB, 10 * units.MiB, 0}, {2, []uint64{1, 2, 3}, 10 * units.MiB, 10 * units.MiB, 0}, {3, []uint64{1, 2, 3}, 10 * units.MiB, 10 * units.MiB, 0}, @@ -2183,11 +2141,11 @@ func TestHotWriteLeaderScheduleWithPriority(t *testing.T) { defer func() { schedulePeerPr, pendingAmpFactor = old1, old2 }() - hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{statistics.KeyPriority, statistics.BytePriority} + hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{utils.KeyPriority, utils.BytePriority} ops, _ := hb.Schedule(tc, false) re.Len(ops, 1) operatorutil.CheckTransferLeader(re, ops[0], operator.OpHotRegion, 1, 2) - hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{statistics.BytePriority, statistics.KeyPriority} + hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{utils.BytePriority, utils.KeyPriority} ops, _ = hb.Schedule(tc, false) re.Len(ops, 1) operatorutil.CheckTransferLeader(re, ops[0], operator.OpHotRegion, 1, 3) @@ -2200,64 +2158,64 @@ func TestCompatibility(t *testing.T) { cancel, _, tc, oc := prepareSchedulersTest() defer cancel() - hb, err := CreateScheduler(statistics.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) + hb, err := CreateScheduler(utils.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) re.NoError(err) // default checkPriority(re, hb.(*hotScheduler), tc, [3][2]int{ - {statistics.QueryDim, statistics.ByteDim}, - {statistics.QueryDim, statistics.ByteDim}, - {statistics.ByteDim, statistics.KeyDim}, + {utils.QueryDim, utils.ByteDim}, + {utils.QueryDim, utils.ByteDim}, + {utils.ByteDim, utils.KeyDim}, }) // config error value hb.(*hotScheduler).conf.ReadPriorities = []string{"error"} - hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{"error", statistics.BytePriority} - hb.(*hotScheduler).conf.WritePeerPriorities = []string{statistics.QueryPriority, statistics.BytePriority, statistics.KeyPriority} + hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{"error", utils.BytePriority} + hb.(*hotScheduler).conf.WritePeerPriorities = []string{utils.QueryPriority, utils.BytePriority, utils.KeyPriority} checkPriority(re, hb.(*hotScheduler), tc, [3][2]int{ - {statistics.QueryDim, statistics.ByteDim}, - {statistics.QueryDim, statistics.ByteDim}, - {statistics.ByteDim, statistics.KeyDim}, + {utils.QueryDim, utils.ByteDim}, + {utils.QueryDim, utils.ByteDim}, + {utils.ByteDim, utils.KeyDim}, }) // low version tc.SetClusterVersion(versioninfo.MinSupportedVersion(versioninfo.Version5_0)) checkPriority(re, hb.(*hotScheduler), tc, [3][2]int{ - {statistics.ByteDim, statistics.KeyDim}, - {statistics.KeyDim, statistics.ByteDim}, - {statistics.ByteDim, statistics.KeyDim}, + {utils.ByteDim, utils.KeyDim}, + {utils.KeyDim, utils.ByteDim}, + {utils.ByteDim, utils.KeyDim}, }) // config byte and key - hb.(*hotScheduler).conf.ReadPriorities = []string{statistics.KeyPriority, statistics.BytePriority} - hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{statistics.BytePriority, statistics.KeyPriority} - hb.(*hotScheduler).conf.WritePeerPriorities = []string{statistics.KeyPriority, statistics.BytePriority} + hb.(*hotScheduler).conf.ReadPriorities = []string{utils.KeyPriority, utils.BytePriority} + hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{utils.BytePriority, utils.KeyPriority} + hb.(*hotScheduler).conf.WritePeerPriorities = []string{utils.KeyPriority, utils.BytePriority} checkPriority(re, hb.(*hotScheduler), tc, [3][2]int{ - {statistics.KeyDim, statistics.ByteDim}, - {statistics.ByteDim, statistics.KeyDim}, - {statistics.KeyDim, statistics.ByteDim}, + {utils.KeyDim, utils.ByteDim}, + {utils.ByteDim, utils.KeyDim}, + {utils.KeyDim, utils.ByteDim}, }) // config query in low version - hb.(*hotScheduler).conf.ReadPriorities = []string{statistics.QueryPriority, statistics.BytePriority} - hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{statistics.QueryPriority, statistics.BytePriority} - hb.(*hotScheduler).conf.WritePeerPriorities = []string{statistics.QueryPriority, statistics.BytePriority} + hb.(*hotScheduler).conf.ReadPriorities = []string{utils.QueryPriority, utils.BytePriority} + hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{utils.QueryPriority, utils.BytePriority} + hb.(*hotScheduler).conf.WritePeerPriorities = []string{utils.QueryPriority, utils.BytePriority} checkPriority(re, hb.(*hotScheduler), tc, [3][2]int{ - {statistics.ByteDim, statistics.KeyDim}, - {statistics.KeyDim, statistics.ByteDim}, - {statistics.ByteDim, statistics.KeyDim}, + {utils.ByteDim, utils.KeyDim}, + {utils.KeyDim, utils.ByteDim}, + {utils.ByteDim, utils.KeyDim}, }) // config error value hb.(*hotScheduler).conf.ReadPriorities = []string{"error", "error"} hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{} - hb.(*hotScheduler).conf.WritePeerPriorities = []string{statistics.QueryPriority, statistics.BytePriority, statistics.KeyPriority} + hb.(*hotScheduler).conf.WritePeerPriorities = []string{utils.QueryPriority, utils.BytePriority, utils.KeyPriority} checkPriority(re, hb.(*hotScheduler), tc, [3][2]int{ - {statistics.ByteDim, statistics.KeyDim}, - {statistics.KeyDim, statistics.ByteDim}, - {statistics.ByteDim, statistics.KeyDim}, + {utils.ByteDim, utils.KeyDim}, + {utils.KeyDim, utils.ByteDim}, + {utils.ByteDim, utils.KeyDim}, }) // test version change tc.SetClusterVersion(versioninfo.MinSupportedVersion(versioninfo.HotScheduleWithQuery)) re.False(hb.(*hotScheduler).conf.lastQuerySupported) // it will updated after scheduling checkPriority(re, hb.(*hotScheduler), tc, [3][2]int{ - {statistics.QueryDim, statistics.ByteDim}, - {statistics.QueryDim, statistics.ByteDim}, - {statistics.ByteDim, statistics.KeyDim}, + {utils.QueryDim, utils.ByteDim}, + {utils.QueryDim, utils.ByteDim}, + {utils.ByteDim, utils.KeyDim}, }) re.True(hb.(*hotScheduler).conf.lastQuerySupported) } @@ -2271,9 +2229,9 @@ func TestCompatibilityConfig(t *testing.T) { hb, err := CreateScheduler(HotRegionType, oc, storage.NewStorageWithMemoryBackend(), ConfigSliceDecoder("hot-region", nil)) re.NoError(err) checkPriority(re, hb.(*hotScheduler), tc, [3][2]int{ - {statistics.QueryDim, statistics.ByteDim}, - {statistics.QueryDim, statistics.ByteDim}, - {statistics.ByteDim, statistics.KeyDim}, + {utils.QueryDim, utils.ByteDim}, + {utils.QueryDim, utils.ByteDim}, + {utils.ByteDim, utils.KeyDim}, }) // Config file is not currently supported @@ -2281,9 +2239,9 @@ func TestCompatibilityConfig(t *testing.T) { ConfigSliceDecoder("hot-region", []string{"read-priorities=byte,query"})) re.NoError(err) checkPriority(re, hb.(*hotScheduler), tc, [3][2]int{ - {statistics.QueryDim, statistics.ByteDim}, - {statistics.QueryDim, statistics.ByteDim}, - {statistics.ByteDim, statistics.KeyDim}, + {utils.QueryDim, utils.ByteDim}, + {utils.QueryDim, utils.ByteDim}, + {utils.ByteDim, utils.KeyDim}, }) // from 4.0 or 5.0 or 5.1 cluster @@ -2308,9 +2266,9 @@ func TestCompatibilityConfig(t *testing.T) { hb, err = CreateScheduler(HotRegionType, oc, storage, ConfigJSONDecoder(data)) re.NoError(err) checkPriority(re, hb.(*hotScheduler), tc, [3][2]int{ - {statistics.ByteDim, statistics.KeyDim}, - {statistics.KeyDim, statistics.ByteDim}, - {statistics.ByteDim, statistics.KeyDim}, + {utils.ByteDim, utils.KeyDim}, + {utils.KeyDim, utils.ByteDim}, + {utils.ByteDim, utils.KeyDim}, }) // From configured cluster @@ -2324,16 +2282,16 @@ func TestCompatibilityConfig(t *testing.T) { hb, err = CreateScheduler(HotRegionType, oc, storage, ConfigJSONDecoder(data)) re.NoError(err) checkPriority(re, hb.(*hotScheduler), tc, [3][2]int{ - {statistics.KeyDim, statistics.QueryDim}, - {statistics.QueryDim, statistics.KeyDim}, - {statistics.ByteDim, statistics.KeyDim}, + {utils.KeyDim, utils.QueryDim}, + {utils.QueryDim, utils.KeyDim}, + {utils.ByteDim, utils.KeyDim}, }) } func checkPriority(re *require.Assertions, hb *hotScheduler, tc *mockcluster.Cluster, dims [3][2]int) { - readSolver := newBalanceSolver(hb, tc, statistics.Read, transferLeader) - writeLeaderSolver := newBalanceSolver(hb, tc, statistics.Write, transferLeader) - writePeerSolver := newBalanceSolver(hb, tc, statistics.Write, movePeer) + readSolver := newBalanceSolver(hb, tc, utils.Read, transferLeader) + writeLeaderSolver := newBalanceSolver(hb, tc, utils.Write, transferLeader) + writePeerSolver := newBalanceSolver(hb, tc, utils.Write, movePeer) re.Equal(dims[0][0], readSolver.firstPriority) re.Equal(dims[0][1], readSolver.secondPriority) re.Equal(dims[1][0], writeLeaderSolver.firstPriority) @@ -2396,20 +2354,20 @@ func TestConfigValidation(t *testing.T) { // forbid-rw-type // default hc = initHotRegionScheduleConfig() - re.False(hc.IsForbidRWType(statistics.Read)) - re.False(hc.IsForbidRWType(statistics.Write)) + re.False(hc.IsForbidRWType(utils.Read)) + re.False(hc.IsForbidRWType(utils.Write)) // read hc.ForbidRWType = "read" err = hc.valid() re.NoError(err) - re.True(hc.IsForbidRWType(statistics.Read)) - re.False(hc.IsForbidRWType(statistics.Write)) + re.True(hc.IsForbidRWType(utils.Read)) + re.False(hc.IsForbidRWType(utils.Write)) // write hc.ForbidRWType = "write" err = hc.valid() re.NoError(err) - re.False(hc.IsForbidRWType(statistics.Read)) - re.True(hc.IsForbidRWType(statistics.Write)) + re.False(hc.IsForbidRWType(utils.Read)) + re.True(hc.IsForbidRWType(utils.Write)) // illegal hc.ForbidRWType = "test" err = hc.valid() @@ -2433,30 +2391,30 @@ func TestMaxZombieDuration(t *testing.T) { testCases := []maxZombieDurTestCase{ { typ: readPeer, - maxZombieDur: maxZombieDur * statistics.StoreHeartBeatReportInterval, + maxZombieDur: maxZombieDur * utils.StoreHeartBeatReportInterval, }, { typ: readLeader, - maxZombieDur: maxZombieDur * statistics.StoreHeartBeatReportInterval, + maxZombieDur: maxZombieDur * utils.StoreHeartBeatReportInterval, }, { typ: writePeer, - maxZombieDur: maxZombieDur * statistics.StoreHeartBeatReportInterval, + maxZombieDur: maxZombieDur * utils.StoreHeartBeatReportInterval, }, { typ: writePeer, isTiFlash: true, - maxZombieDur: maxZombieDur * statistics.RegionHeartBeatReportInterval, + maxZombieDur: maxZombieDur * utils.RegionHeartBeatReportInterval, }, { typ: writeLeader, - firstPriority: statistics.KeyDim, - maxZombieDur: maxZombieDur * statistics.RegionHeartBeatReportInterval, + firstPriority: utils.KeyDim, + maxZombieDur: maxZombieDur * utils.RegionHeartBeatReportInterval, }, { typ: writeLeader, - firstPriority: statistics.QueryDim, - maxZombieDur: maxZombieDur * statistics.StoreHeartBeatReportInterval, + firstPriority: utils.QueryDim, + maxZombieDur: maxZombieDur * utils.StoreHeartBeatReportInterval, }, } for _, testCase := range testCases { @@ -2765,8 +2723,8 @@ func TestExpect(t *testing.T) { } bs := &balanceSolver{ sche: hb.(*hotScheduler), - firstPriority: statistics.KeyDim, - secondPriority: statistics.ByteDim, + firstPriority: utils.KeyDim, + secondPriority: utils.ByteDim, resourceTy: testCase.rs, } bs.sche.conf.StrictPickingStore = testCase.strict diff --git a/pkg/schedule/schedulers/hot_region_v2.go b/pkg/schedule/schedulers/hot_region_v2.go index 2b7e96af0fb..b32840e90e4 100644 --- a/pkg/schedule/schedulers/hot_region_v2.go +++ b/pkg/schedule/schedulers/hot_region_v2.go @@ -20,7 +20,7 @@ import ( "fmt" "math" - "github.com/tikv/pd/pkg/statistics" + "github.com/tikv/pd/pkg/statistics/utils" ) const ( @@ -105,11 +105,11 @@ func (bs *balanceSolver) filterUniformStoreV2() (string, bool) { } if isUniformFirstPriority && (bs.cur.progressiveRank == -2 || bs.cur.progressiveRank == -3) { // If first priority dim is enough uniform, -2 is unnecessary and maybe lead to worse balance for second priority dim - return statistics.DimToString(bs.firstPriority), true + return utils.DimToString(bs.firstPriority), true } if isUniformSecondPriority && bs.cur.progressiveRank == -1 { // If second priority dim is enough uniform, -1 is unnecessary and maybe lead to worse balance for first priority dim - return statistics.DimToString(bs.secondPriority), true + return utils.DimToString(bs.secondPriority), true } return "", false } @@ -367,11 +367,11 @@ func (bs *balanceSolver) rankToDimStringV2() string { case -4: return "all" case -3: - return statistics.DimToString(bs.firstPriority) + return utils.DimToString(bs.firstPriority) case -2: - return statistics.DimToString(bs.firstPriority) + "-only" + return utils.DimToString(bs.firstPriority) + "-only" case -1: - return statistics.DimToString(bs.secondPriority) + return utils.DimToString(bs.secondPriority) default: return "none" } diff --git a/pkg/schedule/schedulers/hot_region_v2_test.go b/pkg/schedule/schedulers/hot_region_v2_test.go index 3dd40fcb984..6930723fdda 100644 --- a/pkg/schedule/schedulers/hot_region_v2_test.go +++ b/pkg/schedule/schedulers/hot_region_v2_test.go @@ -21,6 +21,7 @@ import ( "github.com/stretchr/testify/require" "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/statistics" + "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/storage" "github.com/tikv/pd/pkg/utils/operatorutil" "github.com/tikv/pd/pkg/versioninfo" @@ -33,7 +34,7 @@ func TestHotWriteRegionScheduleWithRevertRegionsDimSecond(t *testing.T) { defer cancel() statistics.Denoising = false - sche, err := CreateScheduler(statistics.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil, nil) + sche, err := CreateScheduler(utils.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil, nil) re.NoError(err) hb := sche.(*hotScheduler) hb.conf.SetDstToleranceRatio(0.0) @@ -46,14 +47,14 @@ func TestHotWriteRegionScheduleWithRevertRegionsDimSecond(t *testing.T) { tc.AddRegionStore(3, 20) tc.AddRegionStore(4, 20) tc.AddRegionStore(5, 20) - hb.conf.WritePeerPriorities = []string{statistics.BytePriority, statistics.KeyPriority} + hb.conf.WritePeerPriorities = []string{utils.BytePriority, utils.KeyPriority} - tc.UpdateStorageWrittenStats(1, 15*units.MiB*statistics.StoreHeartBeatReportInterval, 15*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(2, 16*units.MiB*statistics.StoreHeartBeatReportInterval, 20*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(3, 15*units.MiB*statistics.StoreHeartBeatReportInterval, 15*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(4, 15*units.MiB*statistics.StoreHeartBeatReportInterval, 15*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(5, 14*units.MiB*statistics.StoreHeartBeatReportInterval, 10*units.MiB*statistics.StoreHeartBeatReportInterval) - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + tc.UpdateStorageWrittenStats(1, 15*units.MiB*utils.StoreHeartBeatReportInterval, 15*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(2, 16*units.MiB*utils.StoreHeartBeatReportInterval, 20*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(3, 15*units.MiB*utils.StoreHeartBeatReportInterval, 15*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(4, 15*units.MiB*utils.StoreHeartBeatReportInterval, 15*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(5, 14*units.MiB*utils.StoreHeartBeatReportInterval, 10*units.MiB*utils.StoreHeartBeatReportInterval) + addRegionInfo(tc, utils.Write, []testRegionInfo{ {6, []uint64{3, 2, 4}, 2 * units.MiB, 3 * units.MiB, 0}, {7, []uint64{1, 4, 5}, 2 * units.MiB, 0.1 * units.MiB, 0}, }) @@ -78,7 +79,7 @@ func TestHotWriteRegionScheduleWithRevertRegionsDimSecond(t *testing.T) { re.True(hb.searchRevertRegions[writePeer]) clearPendingInfluence(hb) // When there is a better solution, there will only be one operator. - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + addRegionInfo(tc, utils.Write, []testRegionInfo{ {8, []uint64{3, 2, 4}, 0.5 * units.MiB, 3 * units.MiB, 0}, }) ops, _ = hb.Schedule(tc, false) @@ -95,7 +96,7 @@ func TestHotWriteRegionScheduleWithRevertRegionsDimFirst(t *testing.T) { defer cancel() statistics.Denoising = false - sche, err := CreateScheduler(statistics.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil, nil) + sche, err := CreateScheduler(utils.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil, nil) re.NoError(err) hb := sche.(*hotScheduler) hb.conf.SetDstToleranceRatio(0.0) @@ -108,14 +109,14 @@ func TestHotWriteRegionScheduleWithRevertRegionsDimFirst(t *testing.T) { tc.AddRegionStore(3, 20) tc.AddRegionStore(4, 20) tc.AddRegionStore(5, 20) - hb.conf.WritePeerPriorities = []string{statistics.BytePriority, statistics.KeyPriority} + hb.conf.WritePeerPriorities = []string{utils.BytePriority, utils.KeyPriority} - tc.UpdateStorageWrittenStats(1, 15*units.MiB*statistics.StoreHeartBeatReportInterval, 15*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(2, 20*units.MiB*statistics.StoreHeartBeatReportInterval, 14*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(3, 15*units.MiB*statistics.StoreHeartBeatReportInterval, 15*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(4, 15*units.MiB*statistics.StoreHeartBeatReportInterval, 15*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(5, 10*units.MiB*statistics.StoreHeartBeatReportInterval, 16*units.MiB*statistics.StoreHeartBeatReportInterval) - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + tc.UpdateStorageWrittenStats(1, 15*units.MiB*utils.StoreHeartBeatReportInterval, 15*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(2, 20*units.MiB*utils.StoreHeartBeatReportInterval, 14*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(3, 15*units.MiB*utils.StoreHeartBeatReportInterval, 15*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(4, 15*units.MiB*utils.StoreHeartBeatReportInterval, 15*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(5, 10*units.MiB*utils.StoreHeartBeatReportInterval, 16*units.MiB*utils.StoreHeartBeatReportInterval) + addRegionInfo(tc, utils.Write, []testRegionInfo{ {6, []uint64{3, 2, 4}, 3 * units.MiB, 1.8 * units.MiB, 0}, {7, []uint64{1, 4, 5}, 0.1 * units.MiB, 2 * units.MiB, 0}, }) @@ -148,7 +149,7 @@ func TestHotWriteRegionScheduleWithRevertRegionsDimFirstOnly(t *testing.T) { cancel, _, tc, oc := prepareSchedulersTest() defer cancel() - sche, err := CreateScheduler(statistics.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil, nil) + sche, err := CreateScheduler(utils.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil, nil) re.NoError(err) hb := sche.(*hotScheduler) hb.conf.SetDstToleranceRatio(0.0) @@ -161,14 +162,14 @@ func TestHotWriteRegionScheduleWithRevertRegionsDimFirstOnly(t *testing.T) { tc.AddRegionStore(3, 20) tc.AddRegionStore(4, 20) tc.AddRegionStore(5, 20) - hb.conf.WritePeerPriorities = []string{statistics.BytePriority, statistics.KeyPriority} + hb.conf.WritePeerPriorities = []string{utils.BytePriority, utils.KeyPriority} - tc.UpdateStorageWrittenStats(1, 15*units.MiB*statistics.StoreHeartBeatReportInterval, 15*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(2, 20*units.MiB*statistics.StoreHeartBeatReportInterval, 14*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(3, 15*units.MiB*statistics.StoreHeartBeatReportInterval, 15*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(4, 15*units.MiB*statistics.StoreHeartBeatReportInterval, 16*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenStats(5, 10*units.MiB*statistics.StoreHeartBeatReportInterval, 18*units.MiB*statistics.StoreHeartBeatReportInterval) - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + tc.UpdateStorageWrittenStats(1, 15*units.MiB*utils.StoreHeartBeatReportInterval, 15*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(2, 20*units.MiB*utils.StoreHeartBeatReportInterval, 14*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(3, 15*units.MiB*utils.StoreHeartBeatReportInterval, 15*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(4, 15*units.MiB*utils.StoreHeartBeatReportInterval, 16*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenStats(5, 10*units.MiB*utils.StoreHeartBeatReportInterval, 18*units.MiB*utils.StoreHeartBeatReportInterval) + addRegionInfo(tc, utils.Write, []testRegionInfo{ {6, []uint64{3, 2, 4}, 3 * units.MiB, 3 * units.MiB, 0}, {7, []uint64{1, 4, 5}, 0.1 * units.MiB, 0.1 * units.MiB, 0}, }) @@ -192,7 +193,7 @@ func TestHotWriteRegionScheduleWithRevertRegionsDimFirstOnly(t *testing.T) { re.True(hb.searchRevertRegions[writePeer]) clearPendingInfluence(hb) // Two operators can be generated when there is a better solution - addRegionInfo(tc, statistics.Write, []testRegionInfo{ + addRegionInfo(tc, utils.Write, []testRegionInfo{ {8, []uint64{1, 4, 5}, 0.1 * units.MiB, 3 * units.MiB, 0}, }) ops, _ = hb.Schedule(tc, false) @@ -210,7 +211,7 @@ func TestHotReadRegionScheduleWithRevertRegionsDimSecond(t *testing.T) { cancel, _, tc, oc := prepareSchedulersTest() defer cancel() - sche, err := CreateScheduler(statistics.Read.String(), oc, storage.NewStorageWithMemoryBackend(), nil, nil) + sche, err := CreateScheduler(utils.Read.String(), oc, storage.NewStorageWithMemoryBackend(), nil, nil) re.NoError(err) hb := sche.(*hotScheduler) hb.conf.SetDstToleranceRatio(0.0) @@ -223,14 +224,14 @@ func TestHotReadRegionScheduleWithRevertRegionsDimSecond(t *testing.T) { tc.AddRegionStore(3, 20) tc.AddRegionStore(4, 20) tc.AddRegionStore(5, 20) - hb.conf.ReadPriorities = []string{statistics.BytePriority, statistics.KeyPriority} + hb.conf.ReadPriorities = []string{utils.BytePriority, utils.KeyPriority} - tc.UpdateStorageReadStats(1, 15*units.MiB*statistics.StoreHeartBeatReportInterval, 15*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadStats(2, 16*units.MiB*statistics.StoreHeartBeatReportInterval, 20*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadStats(3, 15*units.MiB*statistics.StoreHeartBeatReportInterval, 15*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadStats(4, 15*units.MiB*statistics.StoreHeartBeatReportInterval, 15*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadStats(5, 14*units.MiB*statistics.StoreHeartBeatReportInterval, 10*units.MiB*statistics.StoreHeartBeatReportInterval) - addRegionInfo(tc, statistics.Read, []testRegionInfo{ + tc.UpdateStorageReadStats(1, 15*units.MiB*utils.StoreHeartBeatReportInterval, 15*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadStats(2, 16*units.MiB*utils.StoreHeartBeatReportInterval, 20*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadStats(3, 15*units.MiB*utils.StoreHeartBeatReportInterval, 15*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadStats(4, 15*units.MiB*utils.StoreHeartBeatReportInterval, 15*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadStats(5, 14*units.MiB*utils.StoreHeartBeatReportInterval, 10*units.MiB*utils.StoreHeartBeatReportInterval) + addRegionInfo(tc, utils.Read, []testRegionInfo{ {6, []uint64{2, 1, 5}, 2 * units.MiB, 3 * units.MiB, 0}, {7, []uint64{5, 4, 2}, 2 * units.MiB, 0.1 * units.MiB, 0}, }) @@ -255,7 +256,7 @@ func TestHotReadRegionScheduleWithRevertRegionsDimSecond(t *testing.T) { re.True(hb.searchRevertRegions[readLeader]) clearPendingInfluence(hb) // When there is a better solution, there will only be one operator. - addRegionInfo(tc, statistics.Read, []testRegionInfo{ + addRegionInfo(tc, utils.Read, []testRegionInfo{ {8, []uint64{2, 1, 5}, 0.5 * units.MiB, 3 * units.MiB, 0}, }) ops, _ = hb.Schedule(tc, false) @@ -271,12 +272,12 @@ func TestSkipUniformStore(t *testing.T) { cancel, _, tc, oc := prepareSchedulersTest() defer cancel() - hb, err := CreateScheduler(statistics.Read.String(), oc, storage.NewStorageWithMemoryBackend(), nil, nil) + hb, err := CreateScheduler(utils.Read.String(), oc, storage.NewStorageWithMemoryBackend(), nil, nil) re.NoError(err) hb.(*hotScheduler).conf.SetSrcToleranceRatio(1) hb.(*hotScheduler).conf.SetDstToleranceRatio(1) hb.(*hotScheduler).conf.SetRankFormulaVersion("v2") - hb.(*hotScheduler).conf.ReadPriorities = []string{statistics.BytePriority, statistics.KeyPriority} + hb.(*hotScheduler).conf.ReadPriorities = []string{utils.BytePriority, utils.KeyPriority} tc.SetHotRegionCacheHitsThreshold(0) tc.AddRegionStore(1, 20) tc.AddRegionStore(2, 20) @@ -285,10 +286,10 @@ func TestSkipUniformStore(t *testing.T) { tc.AddRegionStore(5, 20) // Case1: two dim are both enough uniform - tc.UpdateStorageReadStats(1, 10.05*units.MB*statistics.StoreHeartBeatReportInterval, 10.05*units.MB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadStats(2, 9.15*units.MB*statistics.StoreHeartBeatReportInterval, 9.15*units.MB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadStats(3, 10.0*units.MB*statistics.StoreHeartBeatReportInterval, 10.0*units.MB*statistics.StoreHeartBeatReportInterval) - addRegionInfo(tc, statistics.Read, []testRegionInfo{ + tc.UpdateStorageReadStats(1, 10.05*units.MB*utils.StoreHeartBeatReportInterval, 10.05*units.MB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadStats(2, 9.15*units.MB*utils.StoreHeartBeatReportInterval, 9.15*units.MB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadStats(3, 10.0*units.MB*utils.StoreHeartBeatReportInterval, 10.0*units.MB*utils.StoreHeartBeatReportInterval) + addRegionInfo(tc, utils.Read, []testRegionInfo{ {1, []uint64{1, 2, 3}, 0.3 * units.MB, 0.3 * units.MB, 0}, }) // when there is no uniform store filter, still schedule although the cluster is enough uniform @@ -304,10 +305,10 @@ func TestSkipUniformStore(t *testing.T) { clearPendingInfluence(hb.(*hotScheduler)) // Case2: the first dim is enough uniform, we should schedule the second dim - tc.UpdateStorageReadStats(1, 10.15*units.MB*statistics.StoreHeartBeatReportInterval, 10.05*units.MB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadStats(2, 9.25*units.MB*statistics.StoreHeartBeatReportInterval, 9.85*units.MB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadStats(3, 9.85*units.MB*statistics.StoreHeartBeatReportInterval, 16.0*units.MB*statistics.StoreHeartBeatReportInterval) - addRegionInfo(tc, statistics.Read, []testRegionInfo{ + tc.UpdateStorageReadStats(1, 10.15*units.MB*utils.StoreHeartBeatReportInterval, 10.05*units.MB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadStats(2, 9.25*units.MB*utils.StoreHeartBeatReportInterval, 9.85*units.MB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadStats(3, 9.85*units.MB*utils.StoreHeartBeatReportInterval, 16.0*units.MB*utils.StoreHeartBeatReportInterval) + addRegionInfo(tc, utils.Read, []testRegionInfo{ {1, []uint64{1, 2, 3}, 0.3 * units.MB, 0.3 * units.MB, 0}, {2, []uint64{3, 2, 1}, 0.3 * units.MB, 2 * units.MB, 0}, }) @@ -325,10 +326,10 @@ func TestSkipUniformStore(t *testing.T) { clearPendingInfluence(hb.(*hotScheduler)) // Case3: the second dim is enough uniform, we should schedule the first dim, although its rank is higher than the second dim - tc.UpdateStorageReadStats(1, 10.05*units.MB*statistics.StoreHeartBeatReportInterval, 10.05*units.MB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadStats(2, 9.85*units.MB*statistics.StoreHeartBeatReportInterval, 9.45*units.MB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadStats(3, 16*units.MB*statistics.StoreHeartBeatReportInterval, 9.85*units.MB*statistics.StoreHeartBeatReportInterval) - addRegionInfo(tc, statistics.Read, []testRegionInfo{ + tc.UpdateStorageReadStats(1, 10.05*units.MB*utils.StoreHeartBeatReportInterval, 10.05*units.MB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadStats(2, 9.85*units.MB*utils.StoreHeartBeatReportInterval, 9.45*units.MB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadStats(3, 16*units.MB*utils.StoreHeartBeatReportInterval, 9.85*units.MB*utils.StoreHeartBeatReportInterval) + addRegionInfo(tc, utils.Read, []testRegionInfo{ {1, []uint64{1, 2, 3}, 0.3 * units.MB, 0.3 * units.MB, 0}, {2, []uint64{3, 2, 1}, 2 * units.MB, 0.3 * units.MB, 0}, }) diff --git a/pkg/schedule/schedulers/scheduler_test.go b/pkg/schedule/schedulers/scheduler_test.go index 54ee9366f45..12ab9f8aa2f 100644 --- a/pkg/schedule/schedulers/scheduler_test.go +++ b/pkg/schedule/schedulers/scheduler_test.go @@ -28,7 +28,7 @@ import ( "github.com/tikv/pd/pkg/schedule/hbstream" "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/schedule/placement" - "github.com/tikv/pd/pkg/statistics" + "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/storage" "github.com/tikv/pd/pkg/utils/operatorutil" "github.com/tikv/pd/pkg/versioninfo" @@ -167,10 +167,10 @@ func checkBalance(re *require.Assertions, enablePlacementRules bool) { tc.AddLabelsStore(6, 0, map[string]string{"zone": "z4", "host": "h6"}) // Report store written bytes. - tc.UpdateStorageWrittenBytes(1, 7.5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenBytes(2, 4.5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenBytes(3, 4.5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenBytes(4, 6*units.MiB*statistics.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenBytes(1, 7.5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenBytes(2, 4.5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenBytes(3, 4.5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenBytes(4, 6*units.MiB*utils.StoreHeartBeatReportInterval) tc.UpdateStorageWrittenBytes(5, 0) tc.UpdateStorageWrittenBytes(6, 0) @@ -180,9 +180,9 @@ func checkBalance(re *require.Assertions, enablePlacementRules bool) { // | 1 | 1 | 2 | 3 | 512KB | // | 2 | 1 | 3 | 4 | 512KB | // | 3 | 1 | 2 | 4 | 512KB | - tc.AddLeaderRegionWithWriteInfo(1, 1, 512*units.KiB*statistics.WriteReportInterval, 0, 0, statistics.WriteReportInterval, []uint64{2, 3}) - tc.AddLeaderRegionWithWriteInfo(2, 1, 512*units.KiB*statistics.WriteReportInterval, 0, 0, statistics.WriteReportInterval, []uint64{3, 4}) - tc.AddLeaderRegionWithWriteInfo(3, 1, 512*units.KiB*statistics.WriteReportInterval, 0, 0, statistics.WriteReportInterval, []uint64{2, 4}) + tc.AddLeaderRegionWithWriteInfo(1, 1, 512*units.KiB*utils.RegionHeartBeatReportInterval, 0, 0, utils.RegionHeartBeatReportInterval, []uint64{2, 3}) + tc.AddLeaderRegionWithWriteInfo(2, 1, 512*units.KiB*utils.RegionHeartBeatReportInterval, 0, 0, utils.RegionHeartBeatReportInterval, []uint64{3, 4}) + tc.AddLeaderRegionWithWriteInfo(3, 1, 512*units.KiB*utils.RegionHeartBeatReportInterval, 0, 0, utils.RegionHeartBeatReportInterval, []uint64{2, 4}) tc.SetHotRegionCacheHitsThreshold(0) // try to get an operator @@ -203,7 +203,7 @@ func TestHotRegionScheduleAbnormalReplica(t *testing.T) { cancel, _, tc, oc := prepareSchedulersTest() defer cancel() tc.SetHotRegionScheduleLimit(0) - hb, err := CreateScheduler(statistics.Read.String(), oc, storage.NewStorageWithMemoryBackend(), nil) + hb, err := CreateScheduler(utils.Read.String(), oc, storage.NewStorageWithMemoryBackend(), nil) re.NoError(err) tc.AddRegionStore(1, 3) @@ -211,13 +211,13 @@ func TestHotRegionScheduleAbnormalReplica(t *testing.T) { tc.AddRegionStore(3, 2) // Report store read bytes. - tc.UpdateStorageReadBytes(1, 7.5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadBytes(2, 4.5*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageReadBytes(3, 4.5*units.MiB*statistics.StoreHeartBeatReportInterval) + tc.UpdateStorageReadBytes(1, 7.5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadBytes(2, 4.5*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageReadBytes(3, 4.5*units.MiB*utils.StoreHeartBeatReportInterval) - tc.AddRegionWithReadInfo(1, 1, 512*units.KiB*statistics.ReadReportInterval, 0, 0, statistics.ReadReportInterval, []uint64{2}) - tc.AddRegionWithReadInfo(2, 2, 512*units.KiB*statistics.ReadReportInterval, 0, 0, statistics.ReadReportInterval, []uint64{1, 3}) - tc.AddRegionWithReadInfo(3, 1, 512*units.KiB*statistics.ReadReportInterval, 0, 0, statistics.ReadReportInterval, []uint64{2, 3}) + tc.AddRegionWithReadInfo(1, 1, 512*units.KiB*utils.StoreHeartBeatReportInterval, 0, 0, utils.StoreHeartBeatReportInterval, []uint64{2}) + tc.AddRegionWithReadInfo(2, 2, 512*units.KiB*utils.StoreHeartBeatReportInterval, 0, 0, utils.StoreHeartBeatReportInterval, []uint64{1, 3}) + tc.AddRegionWithReadInfo(3, 1, 512*units.KiB*utils.StoreHeartBeatReportInterval, 0, 0, utils.StoreHeartBeatReportInterval, []uint64{2, 3}) tc.SetHotRegionCacheHitsThreshold(0) re.True(tc.IsRegionHot(tc.GetRegion(1))) re.False(hb.IsScheduleAllowed(tc)) @@ -315,7 +315,7 @@ func TestSpecialUseHotRegion(t *testing.T) { cd := ConfigSliceDecoder(BalanceRegionType, []string{"", ""}) bs, err := CreateScheduler(BalanceRegionType, oc, storage, cd) re.NoError(err) - hs, err := CreateScheduler(statistics.Write.String(), oc, storage, cd) + hs, err := CreateScheduler(utils.Write.String(), oc, storage, cd) re.NoError(err) tc.SetHotRegionCacheHitsThreshold(0) @@ -343,16 +343,16 @@ func TestSpecialUseHotRegion(t *testing.T) { re.Empty(ops) // can only move peer to 4 - tc.UpdateStorageWrittenBytes(1, 60*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenBytes(2, 6*units.MiB*statistics.StoreHeartBeatReportInterval) - tc.UpdateStorageWrittenBytes(3, 6*units.MiB*statistics.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenBytes(1, 60*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenBytes(2, 6*units.MiB*utils.StoreHeartBeatReportInterval) + tc.UpdateStorageWrittenBytes(3, 6*units.MiB*utils.StoreHeartBeatReportInterval) tc.UpdateStorageWrittenBytes(4, 0) tc.UpdateStorageWrittenBytes(5, 0) - tc.AddLeaderRegionWithWriteInfo(1, 1, 512*units.KiB*statistics.WriteReportInterval, 0, 0, statistics.WriteReportInterval, []uint64{2, 3}) - tc.AddLeaderRegionWithWriteInfo(2, 1, 512*units.KiB*statistics.WriteReportInterval, 0, 0, statistics.WriteReportInterval, []uint64{2, 3}) - tc.AddLeaderRegionWithWriteInfo(3, 1, 512*units.KiB*statistics.WriteReportInterval, 0, 0, statistics.WriteReportInterval, []uint64{2, 3}) - tc.AddLeaderRegionWithWriteInfo(4, 2, 512*units.KiB*statistics.WriteReportInterval, 0, 0, statistics.WriteReportInterval, []uint64{1, 3}) - tc.AddLeaderRegionWithWriteInfo(5, 3, 512*units.KiB*statistics.WriteReportInterval, 0, 0, statistics.WriteReportInterval, []uint64{1, 2}) + tc.AddLeaderRegionWithWriteInfo(1, 1, 512*units.KiB*utils.RegionHeartBeatReportInterval, 0, 0, utils.RegionHeartBeatReportInterval, []uint64{2, 3}) + tc.AddLeaderRegionWithWriteInfo(2, 1, 512*units.KiB*utils.RegionHeartBeatReportInterval, 0, 0, utils.RegionHeartBeatReportInterval, []uint64{2, 3}) + tc.AddLeaderRegionWithWriteInfo(3, 1, 512*units.KiB*utils.RegionHeartBeatReportInterval, 0, 0, utils.RegionHeartBeatReportInterval, []uint64{2, 3}) + tc.AddLeaderRegionWithWriteInfo(4, 2, 512*units.KiB*utils.RegionHeartBeatReportInterval, 0, 0, utils.RegionHeartBeatReportInterval, []uint64{1, 3}) + tc.AddLeaderRegionWithWriteInfo(5, 3, 512*units.KiB*utils.RegionHeartBeatReportInterval, 0, 0, utils.RegionHeartBeatReportInterval, []uint64{1, 2}) ops, _ = hs.Schedule(tc, false) re.Len(ops, 1) operatorutil.CheckTransferPeer(re, ops[0], operator.OpHotRegion, 1, 4) diff --git a/pkg/statistics/buckets/bucket_stat_informer.go b/pkg/statistics/buckets/bucket_stat_informer.go index ba917a864c7..888027f1f5f 100644 --- a/pkg/statistics/buckets/bucket_stat_informer.go +++ b/pkg/statistics/buckets/bucket_stat_informer.go @@ -21,15 +21,15 @@ import ( "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/rangetree" "github.com/tikv/pd/pkg/slice" - "github.com/tikv/pd/pkg/statistics" + "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/utils/keyutil" ) -var minHotThresholds [statistics.RegionStatCount]uint64 +var minHotThresholds [utils.RegionStatCount]uint64 func init() { for i := range minHotThresholds { - minHotThresholds[i] = uint64(statistics.MinHotThresholds[i]) + minHotThresholds[i] = uint64(utils.MinHotThresholds[i]) } } @@ -45,7 +45,7 @@ type BucketStat struct { EndKey []byte HotDegree int Interval uint64 - // the order should see statistics.RegionStatKind + // the order should see utils.RegionStatKind Loads []uint64 } diff --git a/pkg/statistics/buckets/hot_bucket_cache.go b/pkg/statistics/buckets/hot_bucket_cache.go index 1be3033f314..8f24f4186c6 100644 --- a/pkg/statistics/buckets/hot_bucket_cache.go +++ b/pkg/statistics/buckets/hot_bucket_cache.go @@ -161,6 +161,15 @@ func (h *HotBucketCache) CheckAsync(task flowBucketsItemTask) bool { } } +// BucketsStats returns hot region's buckets stats. +func (h *HotBucketCache) BucketsStats(degree int, regionIDs ...uint64) map[uint64][]*BucketStat { + task := NewCollectBucketStatsTask(degree, regionIDs...) + if !h.CheckAsync(task) { + return nil + } + return task.WaitRet(h.ctx) +} + func (h *HotBucketCache) schedule() { defer logutil.LogPanic() diff --git a/pkg/statistics/collector.go b/pkg/statistics/collector.go index f4785103c89..e64b673803d 100644 --- a/pkg/statistics/collector.go +++ b/pkg/statistics/collector.go @@ -17,6 +17,7 @@ package statistics import ( "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" + "github.com/tikv/pd/pkg/statistics/utils" ) // storeCollector define the behavior of different engines of stores. @@ -26,7 +27,7 @@ type storeCollector interface { // Filter determines whether the Store needs to be handled by itself. Filter(info *StoreSummaryInfo, kind constant.ResourceKind) bool // GetLoads obtains available loads from storeLoads and peerLoadSum according to rwTy and kind. - GetLoads(storeLoads, peerLoadSum []float64, rwTy RWType, kind constant.ResourceKind) (loads []float64) + GetLoads(storeLoads, peerLoadSum []float64, rwTy utils.RWType, kind constant.ResourceKind) (loads []float64) } type tikvCollector struct{} @@ -52,26 +53,26 @@ func (c tikvCollector) Filter(info *StoreSummaryInfo, kind constant.ResourceKind return false } -func (c tikvCollector) GetLoads(storeLoads, peerLoadSum []float64, rwTy RWType, kind constant.ResourceKind) (loads []float64) { - loads = make([]float64, DimLen) +func (c tikvCollector) GetLoads(storeLoads, peerLoadSum []float64, rwTy utils.RWType, kind constant.ResourceKind) (loads []float64) { + loads = make([]float64, utils.DimLen) switch rwTy { - case Read: - loads[ByteDim] = storeLoads[StoreReadBytes] - loads[KeyDim] = storeLoads[StoreReadKeys] - loads[QueryDim] = storeLoads[StoreReadQuery] - case Write: + case utils.Read: + loads[utils.ByteDim] = storeLoads[utils.StoreReadBytes] + loads[utils.KeyDim] = storeLoads[utils.StoreReadKeys] + loads[utils.QueryDim] = storeLoads[utils.StoreReadQuery] + case utils.Write: switch kind { case constant.LeaderKind: // Use sum of hot peers to estimate leader-only byte rate. // For Write requests, Write{Bytes, Keys} is applied to all Peers at the same time, // while the Leader and Follower are under different loads (usually the Leader consumes more CPU). // Write{Query} does not require such processing. - loads[ByteDim] = peerLoadSum[ByteDim] - loads[KeyDim] = peerLoadSum[KeyDim] - loads[QueryDim] = storeLoads[StoreWriteQuery] + loads[utils.ByteDim] = peerLoadSum[utils.ByteDim] + loads[utils.KeyDim] = peerLoadSum[utils.KeyDim] + loads[utils.QueryDim] = storeLoads[utils.StoreWriteQuery] case constant.RegionKind: - loads[ByteDim] = storeLoads[StoreWriteBytes] - loads[KeyDim] = storeLoads[StoreWriteKeys] + loads[utils.ByteDim] = storeLoads[utils.StoreWriteBytes] + loads[utils.KeyDim] = storeLoads[utils.StoreWriteKeys] // The `Write-peer` does not have `QueryDim` } } @@ -100,12 +101,12 @@ func (c tiflashCollector) Filter(info *StoreSummaryInfo, kind constant.ResourceK return false } -func (c tiflashCollector) GetLoads(storeLoads, peerLoadSum []float64, rwTy RWType, kind constant.ResourceKind) (loads []float64) { - loads = make([]float64, DimLen) +func (c tiflashCollector) GetLoads(storeLoads, peerLoadSum []float64, rwTy utils.RWType, kind constant.ResourceKind) (loads []float64) { + loads = make([]float64, utils.DimLen) switch rwTy { - case Read: + case utils.Read: // TODO: Need TiFlash StoreHeartbeat support - case Write: + case utils.Write: switch kind { case constant.LeaderKind: // There is no Leader on TiFlash @@ -113,11 +114,11 @@ func (c tiflashCollector) GetLoads(storeLoads, peerLoadSum []float64, rwTy RWTyp // TiFlash is currently unable to report statistics in the same unit as Region, // so it uses the sum of Regions. If it is not accurate enough, use sum of hot peer. if c.isTraceRegionFlow { - loads[ByteDim] = storeLoads[StoreRegionsWriteBytes] - loads[KeyDim] = storeLoads[StoreRegionsWriteKeys] + loads[utils.ByteDim] = storeLoads[utils.StoreRegionsWriteBytes] + loads[utils.KeyDim] = storeLoads[utils.StoreRegionsWriteKeys] } else { - loads[ByteDim] = peerLoadSum[ByteDim] - loads[KeyDim] = peerLoadSum[KeyDim] + loads[utils.ByteDim] = peerLoadSum[utils.ByteDim] + loads[utils.KeyDim] = peerLoadSum[utils.KeyDim] } // The `Write-peer` does not have `QueryDim` } diff --git a/pkg/statistics/hot_cache.go b/pkg/statistics/hot_cache.go index 42c1e6c49a7..de7189a1332 100644 --- a/pkg/statistics/hot_cache.go +++ b/pkg/statistics/hot_cache.go @@ -19,14 +19,15 @@ import ( "github.com/smallnest/chanx" "github.com/tikv/pd/pkg/core" + "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/utils/logutil" ) const chanMaxLength = 6000000 var ( - readTaskMetrics = hotCacheFlowQueueStatusGauge.WithLabelValues(Read.String()) - writeTaskMetrics = hotCacheFlowQueueStatusGauge.WithLabelValues(Write.String()) + readTaskMetrics = hotCacheFlowQueueStatusGauge.WithLabelValues(utils.Read.String()) + writeTaskMetrics = hotCacheFlowQueueStatusGauge.WithLabelValues(utils.Write.String()) ) // HotCache is a cache hold hot regions. @@ -40,8 +41,8 @@ type HotCache struct { func NewHotCache(ctx context.Context) *HotCache { w := &HotCache{ ctx: ctx, - writeCache: NewHotPeerCache(ctx, Write), - readCache: NewHotPeerCache(ctx, Read), + writeCache: NewHotPeerCache(ctx, utils.Write), + readCache: NewHotPeerCache(ctx, utils.Read), } go w.updateItems(w.readCache.taskQueue, w.runReadTask) go w.updateItems(w.writeCache.taskQueue, w.runWriteTask) @@ -75,13 +76,13 @@ func (w *HotCache) CheckReadAsync(task FlowItemTask) bool { } // RegionStats returns hot items according to kind -func (w *HotCache) RegionStats(kind RWType, minHotDegree int) map[uint64][]*HotPeerStat { +func (w *HotCache) RegionStats(kind utils.RWType, minHotDegree int) map[uint64][]*HotPeerStat { task := newCollectRegionStatsTask(minHotDegree) var succ bool switch kind { - case Write: + case utils.Write: succ = w.CheckWriteAsync(task) - case Read: + case utils.Read: succ = w.CheckReadAsync(task) } if !succ { @@ -103,13 +104,13 @@ func (w *HotCache) IsRegionHot(region *core.RegionInfo, minHotDegree int) bool { } // GetHotPeerStat returns hot peer stat with specified regionID and storeID. -func (w *HotCache) GetHotPeerStat(kind RWType, regionID, storeID uint64) *HotPeerStat { +func (w *HotCache) GetHotPeerStat(kind utils.RWType, regionID, storeID uint64) *HotPeerStat { task := newGetHotPeerStatTask(regionID, storeID) var succ bool switch kind { - case Read: + case utils.Read: succ = w.CheckReadAsync(task) - case Write: + case utils.Write: succ = w.CheckWriteAsync(task) } if !succ { @@ -160,11 +161,11 @@ func (w *HotCache) runWriteTask(task FlowItemTask) { // Update updates the cache. // This is used for mockcluster, for test purpose. -func (w *HotCache) Update(item *HotPeerStat, kind RWType) { +func (w *HotCache) Update(item *HotPeerStat, kind utils.RWType) { switch kind { - case Write: + case utils.Write: w.writeCache.updateStat(item) - case Read: + case utils.Read: w.readCache.updateStat(item) } } @@ -195,11 +196,11 @@ func (w *HotCache) ExpiredWriteItems(region *core.RegionInfo) []*HotPeerStat { // GetThresholds returns thresholds. // This is used for test purpose. -func (w *HotCache) GetThresholds(kind RWType, storeID uint64) []float64 { +func (w *HotCache) GetThresholds(kind utils.RWType, storeID uint64) []float64 { switch kind { - case Write: + case utils.Write: return w.writeCache.calcHotThresholds(storeID) - case Read: + case utils.Read: return w.readCache.calcHotThresholds(storeID) } return nil diff --git a/pkg/statistics/hot_peer.go b/pkg/statistics/hot_peer.go index 02c50697358..79757d6e27f 100644 --- a/pkg/statistics/hot_peer.go +++ b/pkg/statistics/hot_peer.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/log" "github.com/tikv/pd/pkg/movingaverage" "github.com/tikv/pd/pkg/slice" + "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/utils/syncutil" "go.uber.org/zap" ) @@ -34,7 +35,7 @@ type dimStat struct { func newDimStat(reportInterval time.Duration) *dimStat { return &dimStat{ - rolling: movingaverage.NewTimeMedian(DefaultAotSize, rollingWindowsSize, reportInterval), + rolling: movingaverage.NewTimeMedian(utils.DefaultAotSize, rollingWindowsSize, reportInterval), lastIntervalSum: 0, lastDelta: 0, } @@ -103,7 +104,7 @@ type HotPeerStat struct { // stores contains the all peer's storeID in this region. stores []uint64 // actionType is the action type of the region, add, update or remove. - actionType ActionType + actionType utils.ActionType // isLeader is true means that the region has a leader on this store. isLeader bool // lastTransferLeaderTime is used to cool down frequent transfer leader. @@ -121,8 +122,8 @@ func (stat *HotPeerStat) ID() uint64 { return stat.RegionID } -// Less compares two HotPeerStat.Implementing TopNItem. -func (stat *HotPeerStat) Less(dim int, than TopNItem) bool { +// Less compares two HotPeerStat. Implementing TopNItem. +func (stat *HotPeerStat) Less(dim int, than utils.TopNItem) bool { return stat.GetLoad(dim) < than.(*HotPeerStat).GetLoad(dim) } @@ -144,7 +145,7 @@ func (stat *HotPeerStat) Log(str string) { } // IsNeedCoolDownTransferLeader use cooldown time after transfer leader to avoid unnecessary schedule -func (stat *HotPeerStat) IsNeedCoolDownTransferLeader(minHotDegree int, rwTy RWType) bool { +func (stat *HotPeerStat) IsNeedCoolDownTransferLeader(minHotDegree int, rwTy utils.RWType) bool { return time.Since(stat.lastTransferLeaderTime).Seconds() < float64(minHotDegree*rwTy.ReportInterval()) } @@ -154,7 +155,7 @@ func (stat *HotPeerStat) IsLeader() bool { } // GetActionType returns the item action type. -func (stat *HotPeerStat) GetActionType() ActionType { +func (stat *HotPeerStat) GetActionType() utils.ActionType { return stat.actionType } @@ -181,8 +182,8 @@ func (stat *HotPeerStat) GetLoads() []float64 { // Clone clones the HotPeerStat. func (stat *HotPeerStat) Clone() *HotPeerStat { ret := *stat - ret.Loads = make([]float64, DimLen) - for i := 0; i < DimLen; i++ { + ret.Loads = make([]float64, utils.DimLen) + for i := 0; i < utils.DimLen; i++ { ret.Loads[i] = stat.GetLoad(i) // replace with denoising loads } ret.rollingLoads = nil diff --git a/pkg/statistics/hot_peer_cache.go b/pkg/statistics/hot_peer_cache.go index 16c64b752e0..1ac07289a3c 100644 --- a/pkg/statistics/hot_peer_cache.go +++ b/pkg/statistics/hot_peer_cache.go @@ -16,15 +16,16 @@ package statistics import ( "context" + "fmt" "math" "time" - "github.com/docker/go-units" "github.com/pingcap/kvproto/pkg/metapb" "github.com/prometheus/client_golang/prometheus" "github.com/smallnest/chanx" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/slice" + "github.com/tikv/pd/pkg/statistics/utils" ) const ( @@ -32,19 +33,12 @@ const ( TopNN = 60 // HotThresholdRatio is used to calculate hot thresholds HotThresholdRatio = 0.8 - // WriteReportInterval indicates the interval between write interval - WriteReportInterval = RegionHeartBeatReportInterval - // ReadReportInterval indicates the interval between read stats report - ReadReportInterval = StoreHeartBeatReportInterval rollingWindowsSize = 5 // HotRegionReportMinInterval is used for the simulator and test HotRegionReportMinInterval = 3 - // HotRegionAntiCount is default value for antiCount - HotRegionAntiCount = 2 - queueCap = 20000 ) @@ -56,47 +50,37 @@ var ThresholdsUpdateInterval = 8 * time.Second // only turn off by the simulator and the test. var Denoising = true -// MinHotThresholds is the threshold at which this dimension is recorded as a hot spot. -var MinHotThresholds = [RegionStatCount]float64{ - RegionReadBytes: 8 * units.KiB, - RegionReadKeys: 128, - RegionReadQueryNum: 128, - RegionWriteBytes: 1 * units.KiB, - RegionWriteKeys: 32, - RegionWriteQueryNum: 32, -} - type thresholds struct { updatedTime time.Time rates []float64 topNLen int - metrics [DimLen + 1]prometheus.Gauge // 0 is for byte, 1 is for key, 2 is for query, 3 is for total length. + metrics [utils.DimLen + 1]prometheus.Gauge // 0 is for byte, 1 is for key, 2 is for query, 3 is for total length. } // hotPeerCache saves the hot peer's statistics. type hotPeerCache struct { - kind RWType - peersOfStore map[uint64]*TopN // storeID -> hot peers + kind utils.RWType + peersOfStore map[uint64]*utils.TopN // storeID -> hot peers storesOfRegion map[uint64]map[uint64]struct{} // regionID -> storeIDs regionsOfStore map[uint64]map[uint64]struct{} // storeID -> regionIDs topNTTL time.Duration taskQueue *chanx.UnboundedChan[FlowItemTask] - thresholdsOfStore map[uint64]*thresholds // storeID -> thresholds - metrics map[uint64][ActionTypeLen]prometheus.Gauge // storeID -> metrics + thresholdsOfStore map[uint64]*thresholds // storeID -> thresholds + metrics map[uint64][utils.ActionTypeLen]prometheus.Gauge // storeID -> metrics // TODO: consider to remove store info when store is offline. } // NewHotPeerCache creates a hotPeerCache -func NewHotPeerCache(ctx context.Context, kind RWType) *hotPeerCache { +func NewHotPeerCache(ctx context.Context, kind utils.RWType) *hotPeerCache { return &hotPeerCache{ kind: kind, - peersOfStore: make(map[uint64]*TopN), + peersOfStore: make(map[uint64]*utils.TopN), storesOfRegion: make(map[uint64]map[uint64]struct{}), regionsOfStore: make(map[uint64]map[uint64]struct{}), taskQueue: chanx.NewUnboundedChan[FlowItemTask](ctx, queueCap), thresholdsOfStore: make(map[uint64]*thresholds), topNTTL: time.Duration(3*kind.ReportInterval()) * time.Second, - metrics: make(map[uint64][ActionTypeLen]prometheus.Gauge), + metrics: make(map[uint64][utils.ActionTypeLen]prometheus.Gauge), } } @@ -120,10 +104,10 @@ func (f *hotPeerCache) RegionStats(minHotDegree int) map[uint64][]*HotPeerStat { func (f *hotPeerCache) updateStat(item *HotPeerStat) { switch item.actionType { - case Remove: + case utils.Remove: f.removeItem(item) item.Log("region heartbeat remove from cache") - case Add, Update: + case utils.Add, utils.Update: f.putItem(item) item.Log("region heartbeat update") default: @@ -132,14 +116,14 @@ func (f *hotPeerCache) updateStat(item *HotPeerStat) { f.incMetrics(item.actionType, item.StoreID) } -func (f *hotPeerCache) incMetrics(action ActionType, storeID uint64) { +func (f *hotPeerCache) incMetrics(action utils.ActionType, storeID uint64) { if _, ok := f.metrics[storeID]; !ok { store := storeTag(storeID) kind := f.kind.String() - f.metrics[storeID] = [ActionTypeLen]prometheus.Gauge{ - Add: hotCacheStatusGauge.WithLabelValues("add_item", store, kind), - Remove: hotCacheStatusGauge.WithLabelValues("remove_item", store, kind), - Update: hotCacheStatusGauge.WithLabelValues("update_item", store, kind), + f.metrics[storeID] = [utils.ActionTypeLen]prometheus.Gauge{ + utils.Add: hotCacheStatusGauge.WithLabelValues("add_item", store, kind), + utils.Remove: hotCacheStatusGauge.WithLabelValues("remove_item", store, kind), + utils.Update: hotCacheStatusGauge.WithLabelValues("update_item", store, kind), } } f.metrics[storeID][action].Inc() @@ -153,17 +137,17 @@ func (f *hotPeerCache) collectPeerMetrics(loads []float64, interval uint64) { // TODO: use unified metrics. (keep backward compatibility at the same time) for _, k := range f.kind.RegionStats() { switch k { - case RegionReadBytes: + case utils.RegionReadBytes: readByteHist.Observe(loads[int(k)]) - case RegionReadKeys: + case utils.RegionReadKeys: readKeyHist.Observe(loads[int(k)]) - case RegionWriteBytes: + case utils.RegionWriteBytes: writeByteHist.Observe(loads[int(k)]) - case RegionWriteKeys: + case utils.RegionWriteKeys: writeKeyHist.Observe(loads[int(k)]) - case RegionWriteQueryNum: + case utils.RegionWriteQueryNum: writeQueryHist.Observe(loads[int(k)]) - case RegionReadQueryNum: + case utils.RegionReadQueryNum: readQueryHist.Observe(loads[int(k)]) } } @@ -178,7 +162,7 @@ func (f *hotPeerCache) collectExpiredItems(region *core.RegionInfo) []*HotPeerSt if region.GetStorePeer(storeID) == nil { item := f.getOldHotPeerStat(regionID, storeID) if item != nil { - item.actionType = Remove + item.actionType = utils.Remove items = append(items, item) } } @@ -202,12 +186,12 @@ func (f *hotPeerCache) checkPeerFlow(peer *core.PeerInfo, region *core.RegionInf oldItem := f.getOldHotPeerStat(regionID, storeID) // check whether the peer is allowed to be inherited - source := direct + source := utils.Direct if oldItem == nil { for _, storeID := range f.getAllStoreIDs(region) { oldItem = f.getOldHotPeerStat(regionID, storeID) if oldItem != nil && oldItem.allowInherited { - source = inherit + source = utils.Inherit break } } @@ -231,7 +215,7 @@ func (f *hotPeerCache) checkPeerFlow(peer *core.PeerInfo, region *core.RegionInf RegionID: regionID, Loads: f.kind.GetLoadRatesFromPeer(peer), isLeader: region.GetLeader().GetStoreId() == storeID, - actionType: Update, + actionType: utils.Update, stores: make([]uint64, len(peers)), } for i, peer := range peers { @@ -272,13 +256,13 @@ func (f *hotPeerCache) checkColdPeer(storeID uint64, reportRegions map[uint64]*c // use 0 to make the cold newItem won't affect the loads. Loads: make([]float64, len(oldItem.Loads)), isLeader: oldItem.isLeader, - actionType: Update, + actionType: utils.Update, inCold: true, stores: oldItem.stores, } - deltaLoads := make([]float64, RegionStatCount) + deltaLoads := make([]float64, utils.RegionStatCount) thresholds := f.calcHotThresholds(storeID) - source := direct + source := utils.Direct for i, loads := range thresholds { deltaLoads[i] = loads * float64(interval) } @@ -293,10 +277,10 @@ func (f *hotPeerCache) checkColdPeer(storeID uint64, reportRegions map[uint64]*c func (f *hotPeerCache) collectMetrics() { for _, thresholds := range f.thresholdsOfStore { - thresholds.metrics[ByteDim].Set(thresholds.rates[ByteDim]) - thresholds.metrics[KeyDim].Set(thresholds.rates[KeyDim]) - thresholds.metrics[QueryDim].Set(thresholds.rates[QueryDim]) - thresholds.metrics[DimLen].Set(float64(thresholds.topNLen)) + thresholds.metrics[utils.ByteDim].Set(thresholds.rates[utils.ByteDim]) + thresholds.metrics[utils.KeyDim].Set(thresholds.rates[utils.KeyDim]) + thresholds.metrics[utils.QueryDim].Set(thresholds.rates[utils.QueryDim]) + thresholds.metrics[utils.DimLen].Set(float64(thresholds.topNLen)) } } @@ -320,12 +304,12 @@ func (f *hotPeerCache) calcHotThresholds(storeID uint64) []float64 { store := storeTag(storeID) kind := f.kind.String() t = &thresholds{ - rates: make([]float64, DimLen), - metrics: [DimLen + 1]prometheus.Gauge{ - ByteDim: hotCacheStatusGauge.WithLabelValues("byte-rate-threshold", store, kind), - KeyDim: hotCacheStatusGauge.WithLabelValues("key-rate-threshold", store, kind), - QueryDim: hotCacheStatusGauge.WithLabelValues("query-rate-threshold", store, kind), - DimLen: hotCacheStatusGauge.WithLabelValues("total_length", store, kind), + rates: make([]float64, utils.DimLen), + metrics: [utils.DimLen + 1]prometheus.Gauge{ + utils.ByteDim: hotCacheStatusGauge.WithLabelValues("byte-rate-threshold", store, kind), + utils.KeyDim: hotCacheStatusGauge.WithLabelValues("key-rate-threshold", store, kind), + utils.QueryDim: hotCacheStatusGauge.WithLabelValues("query-rate-threshold", store, kind), + utils.DimLen: hotCacheStatusGauge.WithLabelValues("total_length", store, kind), }, } } @@ -334,7 +318,7 @@ func (f *hotPeerCache) calcHotThresholds(storeID uint64) []float64 { t.updatedTime = time.Now() statKinds := f.kind.RegionStats() for dim, kind := range statKinds { - t.rates[dim] = MinHotThresholds[kind] + t.rates[dim] = utils.MinHotThresholds[kind] } if tn, ok := f.peersOfStore[storeID]; ok { t.topNLen = tn.Len() @@ -447,10 +431,10 @@ func (f *hotPeerCache) getHotPeerStat(regionID, storeID uint64) *HotPeerStat { return nil } -func (f *hotPeerCache) updateHotPeerStat(region *core.RegionInfo, newItem, oldItem *HotPeerStat, deltaLoads []float64, interval time.Duration, source sourceKind) *HotPeerStat { +func (f *hotPeerCache) updateHotPeerStat(region *core.RegionInfo, newItem, oldItem *HotPeerStat, deltaLoads []float64, interval time.Duration, source utils.SourceKind) *HotPeerStat { regionStats := f.kind.RegionStats() - if source == inherit { + if source == utils.Inherit { for _, dim := range oldItem.rollingLoads { newItem.rollingLoads = append(newItem.rollingLoads, dim.Clone()) } @@ -466,7 +450,7 @@ func (f *hotPeerCache) updateHotPeerStat(region *core.RegionInfo, newItem, oldIt // maintain anticount and hotdegree to avoid store threshold and hot peer are unstable. // For write stat, as the stat is send by region heartbeat, the first heartbeat will be skipped. // For read stat, as the stat is send by store heartbeat, the first heartbeat won't be skipped. - if f.kind == Write { + if f.kind == utils.Write { f.inheritItem(newItem, oldItem) return newItem } @@ -493,7 +477,7 @@ func (f *hotPeerCache) updateHotPeerStat(region *core.RegionInfo, newItem, oldIt if newItem.isHot(thresholds) { f.initItem(newItem) } else { - newItem.actionType = Remove + newItem.actionType = utils.Remove } } else { if newItem.isHot(thresholds) { @@ -514,7 +498,7 @@ func (f *hotPeerCache) updateNewHotPeerStat(newItem *HotPeerStat, deltaLoads []f if interval.Seconds() >= float64(f.kind.ReportInterval()) { f.initItem(newItem) } - newItem.actionType = Add + newItem.actionType = utils.Add newItem.rollingLoads = make([]*dimStat, len(regionStats)) for i, k := range regionStats { ds := newDimStat(f.interval()) @@ -530,7 +514,7 @@ func (f *hotPeerCache) updateNewHotPeerStat(newItem *HotPeerStat, deltaLoads []f func (f *hotPeerCache) putItem(item *HotPeerStat) { peers, ok := f.peersOfStore[item.StoreID] if !ok { - peers = NewTopN(DimLen, TopNN, f.topNTTL) + peers = utils.NewTopN(utils.DimLen, TopNN, f.topNTTL) f.peersOfStore[item.StoreID] = peers } peers.Put(item) @@ -564,7 +548,7 @@ func (f *hotPeerCache) coldItem(newItem, oldItem *HotPeerStat) { newItem.HotDegree = oldItem.HotDegree - 1 newItem.AntiCount = oldItem.AntiCount - 1 if newItem.AntiCount <= 0 { - newItem.actionType = Remove + newItem.actionType = utils.Remove } else { newItem.allowInherited = true } @@ -594,3 +578,7 @@ func (f *hotPeerCache) inheritItem(newItem, oldItem *HotPeerStat) { func (f *hotPeerCache) interval() time.Duration { return time.Duration(f.kind.ReportInterval()) * time.Second } + +func storeTag(id uint64) string { + return fmt.Sprintf("store-%d", id) +} diff --git a/pkg/statistics/hot_peer_cache_test.go b/pkg/statistics/hot_peer_cache_test.go index 5e0a7f64141..36f922d3830 100644 --- a/pkg/statistics/hot_peer_cache_test.go +++ b/pkg/statistics/hot_peer_cache_test.go @@ -27,15 +27,16 @@ import ( "github.com/stretchr/testify/require" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/movingaverage" + "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/utils/typeutil" ) func TestStoreTimeUnsync(t *testing.T) { re := require.New(t) - cache := NewHotPeerCache(context.Background(), Write) + cache := NewHotPeerCache(context.Background(), utils.Write) intervals := []uint64{120, 60} for _, interval := range intervals { - region := buildRegion(Write, 3, interval) + region := buildRegion(utils.Write, 3, interval) checkAndUpdate(re, cache, region, 3) { stats := cache.RegionStats(0) @@ -57,35 +58,35 @@ const ( ) type testCacheCase struct { - kind RWType + kind utils.RWType operator operator expect int - actionType ActionType + actionType utils.ActionType } func TestCache(t *testing.T) { re := require.New(t) tests := []*testCacheCase{ - {Read, transferLeader, 3, Update}, - {Read, movePeer, 4, Remove}, - {Read, addReplica, 4, Update}, - {Write, transferLeader, 3, Remove}, - {Write, movePeer, 4, Remove}, - {Write, addReplica, 4, Remove}, + {utils.Read, transferLeader, 3, utils.Update}, + {utils.Read, movePeer, 4, utils.Remove}, + {utils.Read, addReplica, 4, utils.Update}, + {utils.Write, transferLeader, 3, utils.Remove}, + {utils.Write, movePeer, 4, utils.Remove}, + {utils.Write, addReplica, 4, utils.Remove}, } for _, test := range tests { - defaultSize := map[RWType]int{ - Read: 3, // all peers - Write: 3, // all peers + defaultSize := map[utils.RWType]int{ + utils.Read: 3, // all peers + utils.Write: 3, // all peers } cache := NewHotPeerCache(context.Background(), test.kind) region := buildRegion(test.kind, 3, 60) checkAndUpdate(re, cache, region, defaultSize[test.kind]) - checkHit(re, cache, region, test.kind, Add) // all peers are new + checkHit(re, cache, region, test.kind, utils.Add) // all peers are new srcStore, region := schedule(re, test.operator, region, 10) res := checkAndUpdate(re, cache, region, test.expect) - checkHit(re, cache, region, test.kind, Update) // hit cache + checkHit(re, cache, region, test.kind, utils.Update) // hit cache if test.expect != defaultSize[test.kind] { checkOp(re, res, srcStore, test.actionType) } @@ -153,9 +154,9 @@ func checkAndUpdateSkipOne(re *require.Assertions, cache *hotPeerCache, region * return updateFlow(cache, res) } -func checkHit(re *require.Assertions, cache *hotPeerCache, region *core.RegionInfo, kind RWType, actionType ActionType) { +func checkHit(re *require.Assertions, cache *hotPeerCache, region *core.RegionInfo, kind utils.RWType, actionType utils.ActionType) { var peers []*metapb.Peer - if kind == Read { + if kind == utils.Read { peers = []*metapb.Peer{region.GetLeader()} } else { peers = region.GetPeers() @@ -167,7 +168,7 @@ func checkHit(re *require.Assertions, cache *hotPeerCache, region *core.RegionIn } } -func checkOp(re *require.Assertions, ret []*HotPeerStat, storeID uint64, actionType ActionType) { +func checkOp(re *require.Assertions, ret []*HotPeerStat, storeID uint64, actionType utils.ActionType) { for _, item := range ret { if item.StoreID == storeID { re.Equal(actionType, item.actionType) @@ -192,13 +193,13 @@ func checkIntervalSum(cache *hotPeerCache, region *core.RegionInfo) bool { // checkIntervalSumContinuous checks whether the interval sum of the peer is continuous. func checkIntervalSumContinuous(re *require.Assertions, intervalSums map[uint64]int, rets []*HotPeerStat, interval uint64) { for _, ret := range rets { - if ret.actionType == Remove { + if ret.actionType == utils.Remove { delete(intervalSums, ret.StoreID) continue } new := int(ret.getIntervalSum() / 1000000000) if old, ok := intervalSums[ret.StoreID]; ok { - re.Equal((old+int(interval))%RegionHeartBeatReportInterval, new) + re.Equal((old+int(interval))%utils.RegionHeartBeatReportInterval, new) } intervalSums[ret.StoreID] = new } @@ -251,7 +252,7 @@ func pickFollower(region *core.RegionInfo) (index int, peer *metapb.Peer) { return dst, meta.Peers[dst] } -func buildRegion(kind RWType, peerCount int, interval uint64) *core.RegionInfo { +func buildRegion(kind utils.RWType, peerCount int, interval uint64) *core.RegionInfo { peers := newPeers(peerCount, func(i int) uint64 { return uint64(10000 + i) }, func(i int) uint64 { return uint64(i) }) @@ -266,7 +267,7 @@ func buildRegion(kind RWType, peerCount int, interval uint64) *core.RegionInfo { leader := meta.Peers[rand.Intn(3)] switch kind { - case Read: + case utils.Read: return core.NewRegionInfo( meta, leader, @@ -275,7 +276,7 @@ func buildRegion(kind RWType, peerCount int, interval uint64) *core.RegionInfo { core.SetReadKeys(10*units.MiB*interval), core.SetReadQuery(1024*interval), ) - case Write: + case utils.Write: return core.NewRegionInfo( meta, leader, @@ -305,12 +306,12 @@ func newPeers(n int, pid genID, sid genID) []*metapb.Peer { func TestUpdateHotPeerStat(t *testing.T) { re := require.New(t) - cache := NewHotPeerCache(context.Background(), Read) + cache := NewHotPeerCache(context.Background(), utils.Read) storeID, regionID := uint64(1), uint64(2) peer := &metapb.Peer{StoreId: storeID} region := core.NewRegionInfo(&metapb.Region{Id: regionID, Peers: []*metapb.Peer{peer}}, peer) // we statistic read peer info from store heartbeat rather than region heartbeat - m := RegionHeartBeatReportInterval / StoreHeartBeatReportInterval + m := utils.RegionHeartBeatReportInterval / utils.StoreHeartBeatReportInterval ThresholdsUpdateInterval = 0 defer func() { ThresholdsUpdateInterval = 8 * time.Second @@ -319,9 +320,9 @@ func TestUpdateHotPeerStat(t *testing.T) { // skip interval=0 interval := 0 deltaLoads := []float64{0.0, 0.0, 0.0} - MinHotThresholds[RegionReadBytes] = 0.0 - MinHotThresholds[RegionReadKeys] = 0.0 - MinHotThresholds[RegionReadQueryNum] = 0.0 + utils.MinHotThresholds[utils.RegionReadBytes] = 0.0 + utils.MinHotThresholds[utils.RegionReadKeys] = 0.0 + utils.MinHotThresholds[utils.RegionReadQueryNum] = 0.0 newItem := cache.checkPeerFlow(core.NewPeerInfo(peer, deltaLoads, uint64(interval)), region) re.Nil(newItem) @@ -329,18 +330,18 @@ func TestUpdateHotPeerStat(t *testing.T) { // new peer, interval is larger than report interval, but no hot interval = 10 deltaLoads = []float64{0.0, 0.0, 0.0} - MinHotThresholds[RegionReadBytes] = 1.0 - MinHotThresholds[RegionReadKeys] = 1.0 - MinHotThresholds[RegionReadQueryNum] = 1.0 + utils.MinHotThresholds[utils.RegionReadBytes] = 1.0 + utils.MinHotThresholds[utils.RegionReadKeys] = 1.0 + utils.MinHotThresholds[utils.RegionReadQueryNum] = 1.0 newItem = cache.checkPeerFlow(core.NewPeerInfo(peer, deltaLoads, uint64(interval)), region) re.Nil(newItem) // new peer, interval is less than report interval interval = 4 deltaLoads = []float64{60.0, 60.0, 60.0} - MinHotThresholds[RegionReadBytes] = 0.0 - MinHotThresholds[RegionReadKeys] = 0.0 - MinHotThresholds[RegionReadQueryNum] = 0.0 + utils.MinHotThresholds[utils.RegionReadBytes] = 0.0 + utils.MinHotThresholds[utils.RegionReadKeys] = 0.0 + utils.MinHotThresholds[utils.RegionReadQueryNum] = 0.0 newItem = cache.checkPeerFlow(core.NewPeerInfo(peer, deltaLoads, uint64(interval)), region) re.NotNil(newItem) re.Equal(0, newItem.HotDegree) @@ -353,7 +354,7 @@ func TestUpdateHotPeerStat(t *testing.T) { re.Equal(0, newItem.HotDegree) re.Equal(0, newItem.AntiCount) // sum of interval is larger than report interval, and hot - newItem.AntiCount = Read.DefaultAntiCount() + newItem.AntiCount = utils.Read.DefaultAntiCount() cache.updateStat(newItem) newItem = cache.checkPeerFlow(core.NewPeerInfo(peer, deltaLoads, uint64(interval)), region) re.Equal(1, newItem.HotDegree) @@ -370,9 +371,9 @@ func TestUpdateHotPeerStat(t *testing.T) { re.Equal(2, newItem.HotDegree) re.Equal(2*m, newItem.AntiCount) // sum of interval is larger than report interval, and cold - MinHotThresholds[RegionReadBytes] = 10.0 - MinHotThresholds[RegionReadKeys] = 10.0 - MinHotThresholds[RegionReadQueryNum] = 10.0 + utils.MinHotThresholds[utils.RegionReadBytes] = 10.0 + utils.MinHotThresholds[utils.RegionReadKeys] = 10.0 + utils.MinHotThresholds[utils.RegionReadQueryNum] = 10.0 cache.updateStat(newItem) newItem = cache.checkPeerFlow(core.NewPeerInfo(peer, deltaLoads, uint64(interval)), region) re.Equal(1, newItem.HotDegree) @@ -384,12 +385,12 @@ func TestUpdateHotPeerStat(t *testing.T) { } re.Less(newItem.HotDegree, 0) re.Equal(0, newItem.AntiCount) - re.Equal(Remove, newItem.actionType) + re.Equal(utils.Remove, newItem.actionType) } func TestThresholdWithUpdateHotPeerStat(t *testing.T) { re := require.New(t) - byteRate := MinHotThresholds[RegionReadBytes] * 2 + byteRate := utils.MinHotThresholds[utils.RegionReadBytes] * 2 expectThreshold := byteRate * HotThresholdRatio testMetrics(re, 120., byteRate, expectThreshold) testMetrics(re, 60., byteRate, expectThreshold) @@ -399,9 +400,9 @@ func TestThresholdWithUpdateHotPeerStat(t *testing.T) { } func testMetrics(re *require.Assertions, interval, byteRate, expectThreshold float64) { - cache := NewHotPeerCache(context.Background(), Read) + cache := NewHotPeerCache(context.Background(), utils.Read) storeID := uint64(1) - re.GreaterOrEqual(byteRate, MinHotThresholds[RegionReadBytes]) + re.GreaterOrEqual(byteRate, utils.MinHotThresholds[utils.RegionReadBytes]) ThresholdsUpdateInterval = 0 defer func() { ThresholdsUpdateInterval = 8 * time.Second @@ -414,28 +415,28 @@ func testMetrics(re *require.Assertions, interval, byteRate, expectThreshold flo newItem := &HotPeerStat{ StoreID: storeID, RegionID: i, - actionType: Update, - Loads: make([]float64, DimLen), + actionType: utils.Update, + Loads: make([]float64, utils.DimLen), } - newItem.Loads[ByteDim] = byteRate - newItem.Loads[KeyDim] = 0 + newItem.Loads[utils.ByteDim] = byteRate + newItem.Loads[utils.KeyDim] = 0 oldItem = cache.getOldHotPeerStat(i, storeID) - if oldItem != nil && oldItem.rollingLoads[ByteDim].isHot(thresholds[ByteDim]) == true { + if oldItem != nil && oldItem.rollingLoads[utils.ByteDim].isHot(thresholds[utils.ByteDim]) == true { break } loads := []float64{byteRate * interval, 0.0, 0.0} if oldItem == nil { item = cache.updateNewHotPeerStat(newItem, loads, time.Duration(interval)*time.Second) } else { - item = cache.updateHotPeerStat(nil, newItem, oldItem, loads, time.Duration(interval)*time.Second, direct) + item = cache.updateHotPeerStat(nil, newItem, oldItem, loads, time.Duration(interval)*time.Second, utils.Direct) } cache.updateStat(item) } thresholds := cache.calcHotThresholds(storeID) if i < TopNN { - re.Equal(MinHotThresholds[RegionReadBytes], thresholds[ByteDim]) + re.Equal(utils.MinHotThresholds[utils.RegionReadBytes], thresholds[utils.ByteDim]) } else { - re.Equal(expectThreshold, thresholds[ByteDim]) + re.Equal(expectThreshold, thresholds[utils.ByteDim]) } } } @@ -446,8 +447,8 @@ func TestRemoveFromCache(t *testing.T) { interval := uint64(5) checkers := []check{checkAndUpdate, checkAndUpdateWithOrdering} for _, checker := range checkers { - cache := NewHotPeerCache(context.Background(), Write) - region := buildRegion(Write, peerCount, interval) + cache := NewHotPeerCache(context.Background(), utils.Write) + region := buildRegion(utils.Write, peerCount, interval) // prepare intervalSums := make(map[uint64]int) for i := 1; i <= 200; i++ { @@ -481,8 +482,8 @@ func TestRemoveFromCacheRandom(t *testing.T) { for _, peerCount := range peerCounts { for _, interval := range intervals { for _, checker := range checkers { - cache := NewHotPeerCache(context.Background(), Write) - region := buildRegion(Write, peerCount, interval) + cache := NewHotPeerCache(context.Background(), utils.Write) + region := buildRegion(utils.Write, peerCount, interval) target := uint64(10) intervalSums := make(map[uint64]int) @@ -506,7 +507,7 @@ func TestRemoveFromCacheRandom(t *testing.T) { break } } - if interval < RegionHeartBeatReportInterval { + if interval < utils.RegionHeartBeatReportInterval { re.True(checkIntervalSum(cache, region)) } re.Len(cache.storesOfRegion[region.GetID()], peerCount) @@ -535,8 +536,8 @@ func checkCoolDown(re *require.Assertions, cache *hotPeerCache, region *core.Reg func TestCoolDownTransferLeader(t *testing.T) { re := require.New(t) - cache := NewHotPeerCache(context.Background(), Read) - region := buildRegion(Read, 3, 60) + cache := NewHotPeerCache(context.Background(), utils.Read) + region := buildRegion(utils.Read, 3, 60) moveLeader := func() { _, region = schedule(re, movePeer, region, 10) @@ -568,8 +569,8 @@ func TestCoolDownTransferLeader(t *testing.T) { } testCases := []func(){moveLeader, transferLeader, movePeer, addReplica, removeReplica} for _, testCase := range testCases { - cache = NewHotPeerCache(context.Background(), Read) - region = buildRegion(Read, 3, 60) + cache = NewHotPeerCache(context.Background(), utils.Read) + region = buildRegion(utils.Read, 3, 60) for i := 1; i <= 200; i++ { checkAndUpdate(re, cache, region) } @@ -581,8 +582,8 @@ func TestCoolDownTransferLeader(t *testing.T) { // See issue #4510 func TestCacheInherit(t *testing.T) { re := require.New(t) - cache := NewHotPeerCache(context.Background(), Read) - region := buildRegion(Read, 3, 10) + cache := NewHotPeerCache(context.Background(), utils.Read) + region := buildRegion(utils.Read, 3, 10) // prepare for i := 1; i <= 200; i++ { checkAndUpdate(re, cache, region) @@ -594,9 +595,9 @@ func TestCacheInherit(t *testing.T) { newStoreID, region = schedule(re, removeReplica, region) rets := checkAndUpdate(re, cache, region) for _, ret := range rets { - if ret.actionType != Remove { - flow := ret.Loads[ByteDim] - re.Equal(float64(region.GetBytesRead()/ReadReportInterval), flow) + if ret.actionType != utils.Remove { + flow := ret.Loads[utils.ByteDim] + re.Equal(float64(region.GetBytesRead()/utils.StoreHeartBeatReportInterval), flow) } } // new flow @@ -611,9 +612,9 @@ func TestCacheInherit(t *testing.T) { _, region = schedule(re, removeReplica, region) rets = checkAndUpdate(re, cache, region) for _, ret := range rets { - if ret.actionType != Remove { - flow := ret.Loads[ByteDim] - re.Equal(float64(newFlow/ReadReportInterval), flow) + if ret.actionType != utils.Remove { + flow := ret.Loads[utils.ByteDim] + re.Equal(float64(newFlow/utils.StoreHeartBeatReportInterval), flow) } } } @@ -625,7 +626,7 @@ type testMovingAverageCase struct { func checkMovingAverage(re *require.Assertions, testCase *testMovingAverageCase) { interval := time.Second - tm := movingaverage.NewTimeMedian(DefaultAotSize, DefaultWriteMfSize, interval) + tm := movingaverage.NewTimeMedian(utils.DefaultAotSize, utils.DefaultWriteMfSize, interval) var results []float64 for _, data := range testCase.report { tm.Add(data, interval) @@ -672,7 +673,7 @@ func TestHotPeerCacheTopNThreshold(t *testing.T) { re := require.New(t) testWithUpdateInterval := func(interval time.Duration) { ThresholdsUpdateInterval = interval - cache := NewHotPeerCache(context.Background(), Write) + cache := NewHotPeerCache(context.Background(), utils.Write) now := time.Now() for id := uint64(0); id < 100; id++ { meta := &metapb.Region{ @@ -695,17 +696,17 @@ func TestHotPeerCacheTopNThreshold(t *testing.T) { } if ThresholdsUpdateInterval == 0 { if id < 60 { - re.Equal(MinHotThresholds[RegionWriteKeys], cache.calcHotThresholds(1)[KeyDim]) // num write stat of peers on this store. // The result only includes peers that are hot enough. diff --git a/pkg/statistics/store.go b/pkg/statistics/store.go index d8288a9b9cf..baeef0ad417 100644 --- a/pkg/statistics/store.go +++ b/pkg/statistics/store.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/log" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/movingaverage" + "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/utils/syncutil" "go.uber.org/zap" ) @@ -107,7 +108,7 @@ func (s *StoresStats) GetStoresLoads() map[uint64][]float64 { defer s.RUnlock() res := make(map[uint64][]float64, len(s.rollingStoresStats)) for storeID, stats := range s.rollingStoresStats { - for i := StoreStatKind(0); i < StoreStatCount; i++ { + for i := utils.StoreStatKind(0); i < utils.StoreStatCount; i++ { res[storeID] = append(res[storeID], stats.GetLoad(i)) } } @@ -140,25 +141,25 @@ type RollingStoreStats struct { // NewRollingStoreStats creates a RollingStoreStats. func newRollingStoreStats() *RollingStoreStats { - timeMedians := make([]*movingaverage.TimeMedian, StoreStatCount) - movingAvgs := make([]movingaverage.MovingAvg, StoreStatCount) + timeMedians := make([]*movingaverage.TimeMedian, utils.StoreStatCount) + movingAvgs := make([]movingaverage.MovingAvg, utils.StoreStatCount) // from StoreHeartbeat - interval := StoreHeartBeatReportInterval * time.Second - timeMedians[StoreReadBytes] = movingaverage.NewTimeMedian(DefaultAotSize, DefaultReadMfSize, interval) - timeMedians[StoreReadKeys] = movingaverage.NewTimeMedian(DefaultAotSize, DefaultReadMfSize, interval) - timeMedians[StoreReadQuery] = movingaverage.NewTimeMedian(DefaultAotSize, DefaultReadMfSize, interval) - timeMedians[StoreWriteBytes] = movingaverage.NewTimeMedian(DefaultAotSize, DefaultWriteMfSize, interval) - timeMedians[StoreWriteKeys] = movingaverage.NewTimeMedian(DefaultAotSize, DefaultWriteMfSize, interval) - timeMedians[StoreWriteQuery] = movingaverage.NewTimeMedian(DefaultAotSize, DefaultWriteMfSize, interval) - movingAvgs[StoreCPUUsage] = movingaverage.NewMedianFilter(storeStatsRollingWindowsSize) - movingAvgs[StoreDiskReadRate] = movingaverage.NewMedianFilter(storeStatsRollingWindowsSize) - movingAvgs[StoreDiskWriteRate] = movingaverage.NewMedianFilter(storeStatsRollingWindowsSize) + interval := utils.StoreHeartBeatReportInterval * time.Second + timeMedians[utils.StoreReadBytes] = movingaverage.NewTimeMedian(utils.DefaultAotSize, utils.DefaultReadMfSize, interval) + timeMedians[utils.StoreReadKeys] = movingaverage.NewTimeMedian(utils.DefaultAotSize, utils.DefaultReadMfSize, interval) + timeMedians[utils.StoreReadQuery] = movingaverage.NewTimeMedian(utils.DefaultAotSize, utils.DefaultReadMfSize, interval) + timeMedians[utils.StoreWriteBytes] = movingaverage.NewTimeMedian(utils.DefaultAotSize, utils.DefaultWriteMfSize, interval) + timeMedians[utils.StoreWriteKeys] = movingaverage.NewTimeMedian(utils.DefaultAotSize, utils.DefaultWriteMfSize, interval) + timeMedians[utils.StoreWriteQuery] = movingaverage.NewTimeMedian(utils.DefaultAotSize, utils.DefaultWriteMfSize, interval) + movingAvgs[utils.StoreCPUUsage] = movingaverage.NewMedianFilter(storeStatsRollingWindowsSize) + movingAvgs[utils.StoreDiskReadRate] = movingaverage.NewMedianFilter(storeStatsRollingWindowsSize) + movingAvgs[utils.StoreDiskWriteRate] = movingaverage.NewMedianFilter(storeStatsRollingWindowsSize) // from RegionHeartbeat // The data from regionStats is used in TiFlash, so higher tolerance is required - movingAvgs[StoreRegionsWriteBytes] = movingaverage.NewMedianFilter(RegionsStatsRollingWindowsSize) - movingAvgs[StoreRegionsWriteKeys] = movingaverage.NewMedianFilter(RegionsStatsRollingWindowsSize) + movingAvgs[utils.StoreRegionsWriteBytes] = movingaverage.NewMedianFilter(RegionsStatsRollingWindowsSize) + movingAvgs[utils.StoreRegionsWriteKeys] = movingaverage.NewMedianFilter(RegionsStatsRollingWindowsSize) return &RollingStoreStats{ timeMedians: timeMedians, @@ -190,25 +191,25 @@ func (r *RollingStoreStats) Observe(stats *pdpb.StoreStats) { r.Lock() defer r.Unlock() readQueryNum, writeQueryNum := core.GetReadQueryNum(stats.QueryStats), core.GetWriteQueryNum(stats.QueryStats) - r.timeMedians[StoreWriteBytes].Add(float64(stats.BytesWritten), interval) - r.timeMedians[StoreWriteKeys].Add(float64(stats.KeysWritten), interval) - r.timeMedians[StoreWriteQuery].Add(float64(writeQueryNum), interval) - r.timeMedians[StoreReadBytes].Add(float64(stats.BytesRead), interval) - r.timeMedians[StoreReadKeys].Add(float64(stats.KeysRead), interval) - r.timeMedians[StoreReadQuery].Add(float64(readQueryNum), interval) + r.timeMedians[utils.StoreWriteBytes].Add(float64(stats.BytesWritten), interval) + r.timeMedians[utils.StoreWriteKeys].Add(float64(stats.KeysWritten), interval) + r.timeMedians[utils.StoreWriteQuery].Add(float64(writeQueryNum), interval) + r.timeMedians[utils.StoreReadBytes].Add(float64(stats.BytesRead), interval) + r.timeMedians[utils.StoreReadKeys].Add(float64(stats.KeysRead), interval) + r.timeMedians[utils.StoreReadQuery].Add(float64(readQueryNum), interval) // Updates the cpu usages and disk rw rates of store. - r.movingAvgs[StoreCPUUsage].Add(collect(stats.GetCpuUsages())) - r.movingAvgs[StoreDiskReadRate].Add(collect(stats.GetReadIoRates())) - r.movingAvgs[StoreDiskWriteRate].Add(collect(stats.GetWriteIoRates())) + r.movingAvgs[utils.StoreCPUUsage].Add(collect(stats.GetCpuUsages())) + r.movingAvgs[utils.StoreDiskReadRate].Add(collect(stats.GetReadIoRates())) + r.movingAvgs[utils.StoreDiskWriteRate].Add(collect(stats.GetWriteIoRates())) } // ObserveRegionsStats records current statistics from region stats. func (r *RollingStoreStats) ObserveRegionsStats(writeBytesRate, writeKeysRate float64) { r.Lock() defer r.Unlock() - r.movingAvgs[StoreRegionsWriteBytes].Add(writeBytesRate) - r.movingAvgs[StoreRegionsWriteKeys].Add(writeKeysRate) + r.movingAvgs[utils.StoreRegionsWriteBytes].Add(writeBytesRate) + r.movingAvgs[utils.StoreRegionsWriteKeys].Add(writeKeysRate) } // Set sets the statistics (for test). @@ -221,46 +222,46 @@ func (r *RollingStoreStats) Set(stats *pdpb.StoreStats) { r.Lock() defer r.Unlock() readQueryNum, writeQueryNum := core.GetReadQueryNum(stats.QueryStats), core.GetWriteQueryNum(stats.QueryStats) - r.timeMedians[StoreWriteBytes].Set(float64(stats.BytesWritten) / interval) - r.timeMedians[StoreReadBytes].Set(float64(stats.BytesRead) / interval) - r.timeMedians[StoreWriteKeys].Set(float64(stats.KeysWritten) / interval) - r.timeMedians[StoreReadKeys].Set(float64(stats.KeysRead) / interval) - r.timeMedians[StoreReadQuery].Set(float64(readQueryNum) / interval) - r.timeMedians[StoreWriteQuery].Set(float64(writeQueryNum) / interval) - r.movingAvgs[StoreCPUUsage].Set(collect(stats.GetCpuUsages())) - r.movingAvgs[StoreDiskReadRate].Set(collect(stats.GetReadIoRates())) - r.movingAvgs[StoreDiskWriteRate].Set(collect(stats.GetWriteIoRates())) + r.timeMedians[utils.StoreWriteBytes].Set(float64(stats.BytesWritten) / interval) + r.timeMedians[utils.StoreReadBytes].Set(float64(stats.BytesRead) / interval) + r.timeMedians[utils.StoreWriteKeys].Set(float64(stats.KeysWritten) / interval) + r.timeMedians[utils.StoreReadKeys].Set(float64(stats.KeysRead) / interval) + r.timeMedians[utils.StoreReadQuery].Set(float64(readQueryNum) / interval) + r.timeMedians[utils.StoreWriteQuery].Set(float64(writeQueryNum) / interval) + r.movingAvgs[utils.StoreCPUUsage].Set(collect(stats.GetCpuUsages())) + r.movingAvgs[utils.StoreDiskReadRate].Set(collect(stats.GetReadIoRates())) + r.movingAvgs[utils.StoreDiskWriteRate].Set(collect(stats.GetWriteIoRates())) } // SetRegionsStats sets the statistics from region stats (for test). func (r *RollingStoreStats) SetRegionsStats(writeBytesRate, writeKeysRate float64) { r.Lock() defer r.Unlock() - r.movingAvgs[StoreRegionsWriteBytes].Set(writeBytesRate) - r.movingAvgs[StoreRegionsWriteKeys].Set(writeKeysRate) + r.movingAvgs[utils.StoreRegionsWriteBytes].Set(writeBytesRate) + r.movingAvgs[utils.StoreRegionsWriteKeys].Set(writeKeysRate) } // GetLoad returns store's load. -func (r *RollingStoreStats) GetLoad(k StoreStatKind) float64 { +func (r *RollingStoreStats) GetLoad(k utils.StoreStatKind) float64 { r.RLock() defer r.RUnlock() switch k { - case StoreReadBytes, StoreReadKeys, StoreReadQuery, StoreWriteBytes, StoreWriteKeys, StoreWriteQuery: + case utils.StoreReadBytes, utils.StoreReadKeys, utils.StoreReadQuery, utils.StoreWriteBytes, utils.StoreWriteKeys, utils.StoreWriteQuery: return r.timeMedians[k].Get() - case StoreCPUUsage, StoreDiskReadRate, StoreDiskWriteRate, StoreRegionsWriteBytes, StoreRegionsWriteKeys: + case utils.StoreCPUUsage, utils.StoreDiskReadRate, utils.StoreDiskWriteRate, utils.StoreRegionsWriteBytes, utils.StoreRegionsWriteKeys: return r.movingAvgs[k].Get() } return 0 } // GetInstantLoad returns store's instant load. -func (r *RollingStoreStats) GetInstantLoad(k StoreStatKind) float64 { +func (r *RollingStoreStats) GetInstantLoad(k utils.StoreStatKind) float64 { r.RLock() defer r.RUnlock() switch k { - case StoreReadBytes, StoreReadKeys, StoreReadQuery, StoreWriteBytes, StoreWriteKeys, StoreWriteQuery: + case utils.StoreReadBytes, utils.StoreReadKeys, utils.StoreReadQuery, utils.StoreWriteBytes, utils.StoreWriteKeys, utils.StoreWriteQuery: return r.timeMedians[k].GetInstantaneous() - case StoreCPUUsage, StoreDiskReadRate, StoreDiskWriteRate, StoreRegionsWriteBytes, StoreRegionsWriteKeys: + case utils.StoreCPUUsage, utils.StoreDiskReadRate, utils.StoreDiskWriteRate, utils.StoreRegionsWriteBytes, utils.StoreRegionsWriteKeys: return r.movingAvgs[k].GetInstantaneous() } return 0 diff --git a/pkg/statistics/store_collection.go b/pkg/statistics/store_collection.go index c362041ca00..dbf1bdc6be0 100644 --- a/pkg/statistics/store_collection.go +++ b/pkg/statistics/store_collection.go @@ -22,6 +22,7 @@ import ( "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/core/storelimit" + "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/server/config" ) @@ -154,26 +155,26 @@ func (s *storeStatistics) Observe(store *core.StoreInfo, stats *StoresStats) { return } - storeStatusGauge.WithLabelValues(storeAddress, id, "store_write_rate_bytes").Set(storeFlowStats.GetLoad(StoreWriteBytes)) - storeStatusGauge.WithLabelValues(storeAddress, id, "store_read_rate_bytes").Set(storeFlowStats.GetLoad(StoreReadBytes)) - storeStatusGauge.WithLabelValues(storeAddress, id, "store_write_rate_keys").Set(storeFlowStats.GetLoad(StoreWriteKeys)) - storeStatusGauge.WithLabelValues(storeAddress, id, "store_read_rate_keys").Set(storeFlowStats.GetLoad(StoreReadKeys)) - storeStatusGauge.WithLabelValues(storeAddress, id, "store_write_query_rate").Set(storeFlowStats.GetLoad(StoreWriteQuery)) - storeStatusGauge.WithLabelValues(storeAddress, id, "store_read_query_rate").Set(storeFlowStats.GetLoad(StoreReadQuery)) - storeStatusGauge.WithLabelValues(storeAddress, id, "store_cpu_usage").Set(storeFlowStats.GetLoad(StoreCPUUsage)) - storeStatusGauge.WithLabelValues(storeAddress, id, "store_disk_read_rate").Set(storeFlowStats.GetLoad(StoreDiskReadRate)) - storeStatusGauge.WithLabelValues(storeAddress, id, "store_disk_write_rate").Set(storeFlowStats.GetLoad(StoreDiskWriteRate)) - storeStatusGauge.WithLabelValues(storeAddress, id, "store_regions_write_rate_bytes").Set(storeFlowStats.GetLoad(StoreRegionsWriteBytes)) - storeStatusGauge.WithLabelValues(storeAddress, id, "store_regions_write_rate_keys").Set(storeFlowStats.GetLoad(StoreRegionsWriteKeys)) + storeStatusGauge.WithLabelValues(storeAddress, id, "store_write_rate_bytes").Set(storeFlowStats.GetLoad(utils.StoreWriteBytes)) + storeStatusGauge.WithLabelValues(storeAddress, id, "store_read_rate_bytes").Set(storeFlowStats.GetLoad(utils.StoreReadBytes)) + storeStatusGauge.WithLabelValues(storeAddress, id, "store_write_rate_keys").Set(storeFlowStats.GetLoad(utils.StoreWriteKeys)) + storeStatusGauge.WithLabelValues(storeAddress, id, "store_read_rate_keys").Set(storeFlowStats.GetLoad(utils.StoreReadKeys)) + storeStatusGauge.WithLabelValues(storeAddress, id, "store_write_query_rate").Set(storeFlowStats.GetLoad(utils.StoreWriteQuery)) + storeStatusGauge.WithLabelValues(storeAddress, id, "store_read_query_rate").Set(storeFlowStats.GetLoad(utils.StoreReadQuery)) + storeStatusGauge.WithLabelValues(storeAddress, id, "store_cpu_usage").Set(storeFlowStats.GetLoad(utils.StoreCPUUsage)) + storeStatusGauge.WithLabelValues(storeAddress, id, "store_disk_read_rate").Set(storeFlowStats.GetLoad(utils.StoreDiskReadRate)) + storeStatusGauge.WithLabelValues(storeAddress, id, "store_disk_write_rate").Set(storeFlowStats.GetLoad(utils.StoreDiskWriteRate)) + storeStatusGauge.WithLabelValues(storeAddress, id, "store_regions_write_rate_bytes").Set(storeFlowStats.GetLoad(utils.StoreRegionsWriteBytes)) + storeStatusGauge.WithLabelValues(storeAddress, id, "store_regions_write_rate_keys").Set(storeFlowStats.GetLoad(utils.StoreRegionsWriteKeys)) - storeStatusGauge.WithLabelValues(storeAddress, id, "store_write_rate_bytes_instant").Set(storeFlowStats.GetInstantLoad(StoreWriteBytes)) - storeStatusGauge.WithLabelValues(storeAddress, id, "store_read_rate_bytes_instant").Set(storeFlowStats.GetInstantLoad(StoreReadBytes)) - storeStatusGauge.WithLabelValues(storeAddress, id, "store_write_rate_keys_instant").Set(storeFlowStats.GetInstantLoad(StoreWriteKeys)) - storeStatusGauge.WithLabelValues(storeAddress, id, "store_read_rate_keys_instant").Set(storeFlowStats.GetInstantLoad(StoreReadKeys)) - storeStatusGauge.WithLabelValues(storeAddress, id, "store_write_query_rate_instant").Set(storeFlowStats.GetInstantLoad(StoreWriteQuery)) - storeStatusGauge.WithLabelValues(storeAddress, id, "store_read_query_rate_instant").Set(storeFlowStats.GetInstantLoad(StoreReadQuery)) - storeStatusGauge.WithLabelValues(storeAddress, id, "store_regions_write_rate_bytes_instant").Set(storeFlowStats.GetInstantLoad(StoreRegionsWriteBytes)) - storeStatusGauge.WithLabelValues(storeAddress, id, "store_regions_write_rate_keys_instant").Set(storeFlowStats.GetInstantLoad(StoreRegionsWriteKeys)) + storeStatusGauge.WithLabelValues(storeAddress, id, "store_write_rate_bytes_instant").Set(storeFlowStats.GetInstantLoad(utils.StoreWriteBytes)) + storeStatusGauge.WithLabelValues(storeAddress, id, "store_read_rate_bytes_instant").Set(storeFlowStats.GetInstantLoad(utils.StoreReadBytes)) + storeStatusGauge.WithLabelValues(storeAddress, id, "store_write_rate_keys_instant").Set(storeFlowStats.GetInstantLoad(utils.StoreWriteKeys)) + storeStatusGauge.WithLabelValues(storeAddress, id, "store_read_rate_keys_instant").Set(storeFlowStats.GetInstantLoad(utils.StoreReadKeys)) + storeStatusGauge.WithLabelValues(storeAddress, id, "store_write_query_rate_instant").Set(storeFlowStats.GetInstantLoad(utils.StoreWriteQuery)) + storeStatusGauge.WithLabelValues(storeAddress, id, "store_read_query_rate_instant").Set(storeFlowStats.GetInstantLoad(utils.StoreReadQuery)) + storeStatusGauge.WithLabelValues(storeAddress, id, "store_regions_write_rate_bytes_instant").Set(storeFlowStats.GetInstantLoad(utils.StoreRegionsWriteBytes)) + storeStatusGauge.WithLabelValues(storeAddress, id, "store_regions_write_rate_keys_instant").Set(storeFlowStats.GetInstantLoad(utils.StoreRegionsWriteKeys)) } func (s *storeStatistics) Collect() { diff --git a/pkg/statistics/store_collection_test.go b/pkg/statistics/store_collection_test.go index 76a0dbd1b32..e1847af8b1f 100644 --- a/pkg/statistics/store_collection_test.go +++ b/pkg/statistics/store_collection_test.go @@ -26,6 +26,7 @@ import ( "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/mock/mockconfig" + "github.com/tikv/pd/pkg/statistics/utils" ) func TestStoreStatistics(t *testing.T) { @@ -93,10 +94,10 @@ func TestStoreStatistics(t *testing.T) { func TestSummaryStoreInfos(t *testing.T) { re := require.New(t) - rw := Read + rw := utils.Read kind := constant.LeaderKind collector := newTikvCollector() - storeHistoryLoad := NewStoreHistoryLoads(DimLen) + storeHistoryLoad := NewStoreHistoryLoads(utils.DimLen) storeInfos := make(map[uint64]*StoreSummaryInfo) storeLoads := make(map[uint64][]float64) for _, storeID := range []int{1, 3} { diff --git a/pkg/statistics/store_hot_peers_infos.go b/pkg/statistics/store_hot_peers_infos.go index 9480c0ed879..59ee3c20b6f 100644 --- a/pkg/statistics/store_hot_peers_infos.go +++ b/pkg/statistics/store_hot_peers_infos.go @@ -20,6 +20,7 @@ import ( "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" + "github.com/tikv/pd/pkg/statistics/utils" ) // StoreHotPeersInfos is used to get human-readable description for hot regions. @@ -35,7 +36,7 @@ type StoreHotPeersStat map[uint64]*HotPeersStat // CollectHotPeerInfos only returns TotalBytesRate,TotalKeysRate,TotalQueryRate,Count func CollectHotPeerInfos(stores []*core.StoreInfo, regionStats map[uint64][]*HotPeerStat) *StoreHotPeersInfos { - peerLoadSum := make([]float64, DimLen) + peerLoadSum := make([]float64, utils.DimLen) collect := func(kind constant.ResourceKind) StoreHotPeersStat { ret := make(StoreHotPeersStat, len(stores)) for _, store := range stores { @@ -54,9 +55,9 @@ func CollectHotPeerInfos(stores []*core.StoreInfo, regionStats map[uint64][]*Hot } } ret[id] = &HotPeersStat{ - TotalBytesRate: peerLoadSum[ByteDim], - TotalKeysRate: peerLoadSum[KeyDim], - TotalQueryRate: peerLoadSum[QueryDim], + TotalBytesRate: peerLoadSum[utils.ByteDim], + TotalKeysRate: peerLoadSum[utils.KeyDim], + TotalQueryRate: peerLoadSum[utils.QueryDim], Count: len(peers), } } @@ -70,7 +71,7 @@ func CollectHotPeerInfos(stores []*core.StoreInfo, regionStats map[uint64][]*Hot // GetHotStatus returns the hot status for a given type. // NOTE: This function is exported by HTTP API. It does not contain `isLearner` and `LastUpdateTime` field. If need, please call `updateRegionInfo`. -func GetHotStatus(stores []*core.StoreInfo, storesLoads map[uint64][]float64, regionStats map[uint64][]*HotPeerStat, typ RWType, isTraceRegionFlow bool) *StoreHotPeersInfos { +func GetHotStatus(stores []*core.StoreInfo, storesLoads map[uint64][]float64, regionStats map[uint64][]*HotPeerStat, typ utils.RWType, isTraceRegionFlow bool) *StoreHotPeersInfos { stInfos := SummaryStoreInfos(stores) stLoadInfosAsLeader := SummaryStoresLoad( stInfos, @@ -110,7 +111,7 @@ func SummaryStoresLoad( storesHistoryLoads *StoreHistoryLoads, storeHotPeers map[uint64][]*HotPeerStat, isTraceRegionFlow bool, - rwTy RWType, + rwTy utils.RWType, kind constant.ResourceKind, ) map[uint64]*StoreLoadDetail { // loadDetail stores the storeID -> hotPeers stat and its current and future stat(rate,count) @@ -144,13 +145,13 @@ func summaryStoresLoadByEngine( storesLoads map[uint64][]float64, storesHistoryLoads *StoreHistoryLoads, storeHotPeers map[uint64][]*HotPeerStat, - rwTy RWType, + rwTy utils.RWType, kind constant.ResourceKind, collector storeCollector, ) []*StoreLoadDetail { loadDetail := make([]*StoreLoadDetail, 0, len(storeInfos)) - allStoreLoadSum := make([]float64, DimLen) - allStoreHistoryLoadSum := make([][]float64, DimLen) + allStoreLoadSum := make([]float64, utils.DimLen) + allStoreHistoryLoadSum := make([][]float64, utils.DimLen) allStoreCount := 0 allHotPeersCount := 0 @@ -164,7 +165,7 @@ func summaryStoresLoadByEngine( // Find all hot peers first var hotPeers []*HotPeerStat - peerLoadSum := make([]float64, DimLen) + peerLoadSum := make([]float64, utils.DimLen) // TODO: To remove `filterHotPeers`, we need to: // HotLeaders consider `Write{Bytes,Keys}`, so when we schedule `writeLeader`, all peers are leader. for _, peer := range filterHotPeers(kind, storeHotPeers[id]) { @@ -177,11 +178,11 @@ func summaryStoresLoadByEngine( // Metric for debug. // TODO: pre-allocate gauge metrics ty := "byte-rate-" + rwTy.String() + "-" + kind.String() - hotPeerSummary.WithLabelValues(ty, fmt.Sprintf("%v", id)).Set(peerLoadSum[ByteDim]) + hotPeerSummary.WithLabelValues(ty, fmt.Sprintf("%v", id)).Set(peerLoadSum[utils.ByteDim]) ty = "key-rate-" + rwTy.String() + "-" + kind.String() - hotPeerSummary.WithLabelValues(ty, fmt.Sprintf("%v", id)).Set(peerLoadSum[KeyDim]) + hotPeerSummary.WithLabelValues(ty, fmt.Sprintf("%v", id)).Set(peerLoadSum[utils.KeyDim]) ty = "query-rate-" + rwTy.String() + "-" + kind.String() - hotPeerSummary.WithLabelValues(ty, fmt.Sprintf("%v", id)).Set(peerLoadSum[QueryDim]) + hotPeerSummary.WithLabelValues(ty, fmt.Sprintf("%v", id)).Set(peerLoadSum[utils.QueryDim]) } loads := collector.GetLoads(storeLoads, peerLoadSum, rwTy, kind) @@ -231,7 +232,7 @@ func summaryStoresLoadByEngine( } // todo: remove some the max value or min value to avoid the effect of extreme value. - expectHistoryLoads := make([][]float64, DimLen) + expectHistoryLoads := make([][]float64, utils.DimLen) for i := range allStoreHistoryLoadSum { expectHistoryLoads[i] = make([]float64, len(allStoreHistoryLoadSum[i])) for j := range allStoreHistoryLoadSum[i] { @@ -254,19 +255,19 @@ func summaryStoresLoadByEngine( // Metric for debug. engine := collector.Engine() ty := "exp-byte-rate-" + rwTy.String() + "-" + kind.String() - hotPeerSummary.WithLabelValues(ty, engine).Set(expectLoads[ByteDim]) + hotPeerSummary.WithLabelValues(ty, engine).Set(expectLoads[utils.ByteDim]) ty = "exp-key-rate-" + rwTy.String() + "-" + kind.String() - hotPeerSummary.WithLabelValues(ty, engine).Set(expectLoads[KeyDim]) + hotPeerSummary.WithLabelValues(ty, engine).Set(expectLoads[utils.KeyDim]) ty = "exp-query-rate-" + rwTy.String() + "-" + kind.String() - hotPeerSummary.WithLabelValues(ty, engine).Set(expectLoads[QueryDim]) + hotPeerSummary.WithLabelValues(ty, engine).Set(expectLoads[utils.QueryDim]) ty = "exp-count-rate-" + rwTy.String() + "-" + kind.String() hotPeerSummary.WithLabelValues(ty, engine).Set(expectCount) ty = "stddev-byte-rate-" + rwTy.String() + "-" + kind.String() - hotPeerSummary.WithLabelValues(ty, engine).Set(stddevLoads[ByteDim]) + hotPeerSummary.WithLabelValues(ty, engine).Set(stddevLoads[utils.ByteDim]) ty = "stddev-key-rate-" + rwTy.String() + "-" + kind.String() - hotPeerSummary.WithLabelValues(ty, engine).Set(stddevLoads[KeyDim]) + hotPeerSummary.WithLabelValues(ty, engine).Set(stddevLoads[utils.KeyDim]) ty = "stddev-query-rate-" + rwTy.String() + "-" + kind.String() - hotPeerSummary.WithLabelValues(ty, engine).Set(stddevLoads[QueryDim]) + hotPeerSummary.WithLabelValues(ty, engine).Set(stddevLoads[utils.QueryDim]) } expect := StoreLoad{ Loads: expectLoads, diff --git a/pkg/statistics/store_load.go b/pkg/statistics/store_load.go index d110ec4a7f3..79417b65b7e 100644 --- a/pkg/statistics/store_load.go +++ b/pkg/statistics/store_load.go @@ -20,6 +20,7 @@ import ( "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" + "github.com/tikv/pd/pkg/statistics/utils" ) // StoreLoadDetail records store load information. @@ -31,8 +32,8 @@ type StoreLoadDetail struct { // ToHotPeersStat abstracts load information to HotPeersStat. func (li *StoreLoadDetail) ToHotPeersStat() *HotPeersStat { - storeByteRate, storeKeyRate, storeQueryRate := li.LoadPred.Current.Loads[ByteDim], - li.LoadPred.Current.Loads[KeyDim], li.LoadPred.Current.Loads[QueryDim] + storeByteRate, storeKeyRate, storeQueryRate := li.LoadPred.Current.Loads[utils.ByteDim], + li.LoadPred.Current.Loads[utils.KeyDim], li.LoadPred.Current.Loads[utils.QueryDim] if len(li.HotPeers) == 0 { return &HotPeersStat{ StoreByteRate: storeByteRate, @@ -50,9 +51,9 @@ func (li *StoreLoadDetail) ToHotPeersStat() *HotPeersStat { for _, peer := range li.HotPeers { if peer.HotDegree > 0 { peers = append(peers, toHotPeerStatShow(peer)) - byteRate += peer.GetLoad(ByteDim) - keyRate += peer.GetLoad(KeyDim) - queryRate += peer.GetLoad(QueryDim) + byteRate += peer.GetLoad(utils.ByteDim) + keyRate += peer.GetLoad(utils.KeyDim) + queryRate += peer.GetLoad(utils.QueryDim) } } @@ -74,9 +75,9 @@ func (li *StoreLoadDetail) IsUniform(dim int, threshold float64) bool { } func toHotPeerStatShow(p *HotPeerStat) HotPeerStatShow { - byteRate := p.GetLoad(ByteDim) - keyRate := p.GetLoad(KeyDim) - queryRate := p.GetLoad(QueryDim) + byteRate := p.GetLoad(utils.ByteDim) + keyRate := p.GetLoad(utils.KeyDim) + queryRate := p.GetLoad(utils.QueryDim) return HotPeerStatShow{ StoreID: p.StoreID, Stores: p.GetStores(), @@ -152,21 +153,21 @@ type StoreLoad struct { } // ToLoadPred returns the current load and future predictive load. -func (load StoreLoad) ToLoadPred(rwTy RWType, infl *Influence) *StoreLoadPred { +func (load StoreLoad) ToLoadPred(rwTy utils.RWType, infl *Influence) *StoreLoadPred { future := StoreLoad{ Loads: append(load.Loads[:0:0], load.Loads...), Count: load.Count, } if infl != nil { switch rwTy { - case Read: - future.Loads[ByteDim] += infl.Loads[RegionReadBytes] - future.Loads[KeyDim] += infl.Loads[RegionReadKeys] - future.Loads[QueryDim] += infl.Loads[RegionReadQueryNum] - case Write: - future.Loads[ByteDim] += infl.Loads[RegionWriteBytes] - future.Loads[KeyDim] += infl.Loads[RegionWriteKeys] - future.Loads[QueryDim] += infl.Loads[RegionWriteQueryNum] + case utils.Read: + future.Loads[utils.ByteDim] += infl.Loads[utils.RegionReadBytes] + future.Loads[utils.KeyDim] += infl.Loads[utils.RegionReadKeys] + future.Loads[utils.QueryDim] += infl.Loads[utils.RegionReadQueryNum] + case utils.Write: + future.Loads[utils.ByteDim] += infl.Loads[utils.RegionWriteBytes] + future.Loads[utils.KeyDim] += infl.Loads[utils.RegionWriteKeys] + future.Loads[utils.QueryDim] += infl.Loads[utils.RegionWriteQueryNum] } future.Count += infl.Count } @@ -255,14 +256,14 @@ var ( // StoreHistoryLoads records the history load of a store. type StoreHistoryLoads struct { // loads[read/write][leader/follower]-->[store id]-->history load - loads [RWTypeLen][constant.ResourceKindLen]map[uint64]*storeHistoryLoad + loads [utils.RWTypeLen][constant.ResourceKindLen]map[uint64]*storeHistoryLoad dim int } // NewStoreHistoryLoads creates a StoreHistoryLoads. func NewStoreHistoryLoads(dim int) *StoreHistoryLoads { st := StoreHistoryLoads{dim: dim} - for i := RWType(0); i < RWTypeLen; i++ { + for i := utils.RWType(0); i < utils.RWTypeLen; i++ { for j := constant.ResourceKind(0); j < constant.ResourceKindLen; j++ { st.loads[i][j] = make(map[uint64]*storeHistoryLoad) } @@ -271,7 +272,7 @@ func NewStoreHistoryLoads(dim int) *StoreHistoryLoads { } // Add adds the store load to the history. -func (s *StoreHistoryLoads) Add(storeID uint64, rwTp RWType, kind constant.ResourceKind, loads []float64) { +func (s *StoreHistoryLoads) Add(storeID uint64, rwTp utils.RWType, kind constant.ResourceKind, loads []float64) { load, ok := s.loads[rwTp][kind][storeID] if !ok { size := defaultSize @@ -285,7 +286,7 @@ func (s *StoreHistoryLoads) Add(storeID uint64, rwTp RWType, kind constant.Resou } // Get returns the store loads from the history, not one time point. -func (s *StoreHistoryLoads) Get(storeID uint64, rwTp RWType, kind constant.ResourceKind) [][]float64 { +func (s *StoreHistoryLoads) Get(storeID uint64, rwTp utils.RWType, kind constant.ResourceKind) [][]float64 { load, ok := s.loads[rwTp][kind][storeID] if !ok { return [][]float64{} diff --git a/pkg/statistics/store_load_test.go b/pkg/statistics/store_load_test.go index 18441f00dbc..67f2dff9cf9 100644 --- a/pkg/statistics/store_load_test.go +++ b/pkg/statistics/store_load_test.go @@ -19,27 +19,28 @@ import ( "github.com/stretchr/testify/require" "github.com/tikv/pd/pkg/core/constant" + "github.com/tikv/pd/pkg/statistics/utils" ) func TestHistoryLoads(t *testing.T) { re := require.New(t) historySampleInterval = 0 - historyLoads := NewStoreHistoryLoads(DimLen) + historyLoads := NewStoreHistoryLoads(utils.DimLen) loads := []float64{1.0, 2.0, 3.0} - rwTp := Read + rwTp := utils.Read kind := constant.LeaderKind historyLoads.Add(1, rwTp, kind, loads) re.Len(historyLoads.Get(1, rwTp, kind)[0], 10) - expectLoads := make([][]float64, DimLen) + expectLoads := make([][]float64, utils.DimLen) for i := 0; i < len(loads); i++ { expectLoads[i] = make([]float64, 10) } for i := 0; i < 10; i++ { historyLoads.Add(1, rwTp, kind, loads) - expectLoads[ByteDim][i] = 1.0 - expectLoads[KeyDim][i] = 2.0 - expectLoads[QueryDim][i] = 3.0 + expectLoads[utils.ByteDim][i] = 1.0 + expectLoads[utils.KeyDim][i] = 2.0 + expectLoads[utils.QueryDim][i] = 3.0 } re.EqualValues(expectLoads, historyLoads.Get(1, rwTp, kind)) } diff --git a/pkg/statistics/util.go b/pkg/statistics/utils/constant.go similarity index 54% rename from pkg/statistics/util.go rename to pkg/statistics/utils/constant.go index 8c3a89d9c8e..14bee23778a 100644 --- a/pkg/statistics/util.go +++ b/pkg/statistics/utils/constant.go @@ -1,4 +1,4 @@ -// Copyright 2018 TiKV Project Authors. +// Copyright 2023 TiKV Project Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -12,17 +12,21 @@ // See the License for the specific language governing permissions and // limitations under the License. -package statistics +package utils import ( - "fmt" + "github.com/docker/go-units" ) const ( - // StoreHeartBeatReportInterval is the heartbeat report interval of a store. - StoreHeartBeatReportInterval = 10 - // RegionHeartBeatReportInterval is the heartbeat report interval of a region. + // RegionHeartBeatReportInterval indicates the interval between write interval, the value is the heartbeat report interval of a region. RegionHeartBeatReportInterval = 60 + // StoreHeartBeatReportInterval indicates the interval between read stats report, the value is the heartbeat report interval of a store. + StoreHeartBeatReportInterval = 10 + + // HotRegionAntiCount is default value for antiCount + HotRegionAntiCount = 2 + // DefaultAotSize is default size of average over time. DefaultAotSize = 1 // DefaultWriteMfSize is default size of write median filter. @@ -31,6 +35,12 @@ const ( DefaultReadMfSize = 5 ) -func storeTag(id uint64) string { - return fmt.Sprintf("store-%d", id) +// MinHotThresholds is the threshold at which this dimension is recorded as a hot spot. +var MinHotThresholds = [RegionStatCount]float64{ + RegionReadBytes: 8 * units.KiB, + RegionReadKeys: 128, + RegionReadQueryNum: 128, + RegionWriteBytes: 1 * units.KiB, + RegionWriteKeys: 32, + RegionWriteQueryNum: 32, } diff --git a/pkg/statistics/kind.go b/pkg/statistics/utils/kind.go similarity index 93% rename from pkg/statistics/kind.go rename to pkg/statistics/utils/kind.go index 6a942e3feac..4d44b8d57e1 100644 --- a/pkg/statistics/kind.go +++ b/pkg/statistics/utils/kind.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package statistics +package utils import ( "github.com/tikv/pd/pkg/core" @@ -145,19 +145,20 @@ func (k StoreStatKind) String() string { return "unknown StoreStatKind" } -// sourceKind represents the statistics item source. -type sourceKind int +// SourceKind represents the statistics item source. +type SourceKind int +// Different statistics item sources. const ( - direct sourceKind = iota // there is a corresponding peer in this store. - inherit // there is no corresponding peer in this store and we need to copy from other stores. + Direct SourceKind = iota // there is a corresponding peer in this store. + Inherit // there is no corresponding peer in this store and we need to copy from other stores. ) -func (k sourceKind) String() string { +func (k SourceKind) String() string { switch k { - case direct: + case Direct: return "direct" - case inherit: + case Inherit: return "inherit" } return "unknown" @@ -213,9 +214,9 @@ func (rw RWType) Inverse() RWType { func (rw RWType) ReportInterval() int { switch rw { case Write: - return WriteReportInterval + return RegionHeartBeatReportInterval default: // Case Read - return ReadReportInterval + return StoreHeartBeatReportInterval } } diff --git a/pkg/statistics/kind_test.go b/pkg/statistics/utils/kind_test.go similarity index 99% rename from pkg/statistics/kind_test.go rename to pkg/statistics/utils/kind_test.go index 9928c7851a4..0a02ffa00c1 100644 --- a/pkg/statistics/kind_test.go +++ b/pkg/statistics/utils/kind_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package statistics +package utils import ( "testing" diff --git a/pkg/statistics/topn.go b/pkg/statistics/utils/topn.go similarity index 99% rename from pkg/statistics/topn.go rename to pkg/statistics/utils/topn.go index f5b71db66d5..916bbb82f92 100644 --- a/pkg/statistics/topn.go +++ b/pkg/statistics/utils/topn.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package statistics +package utils import ( "container/heap" diff --git a/pkg/statistics/topn_test.go b/pkg/statistics/utils/topn_test.go similarity index 99% rename from pkg/statistics/topn_test.go rename to pkg/statistics/utils/topn_test.go index 6aac24103aa..f92d5a61f34 100644 --- a/pkg/statistics/topn_test.go +++ b/pkg/statistics/utils/topn_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package statistics +package utils import ( "math/rand" diff --git a/server/api/hot_status.go b/server/api/hot_status.go index 23540f9499f..7cc4cd9e9ca 100644 --- a/server/api/hot_status.go +++ b/server/api/hot_status.go @@ -21,7 +21,7 @@ import ( "net/http" "strconv" - "github.com/tikv/pd/pkg/statistics" + "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/storage" "github.com/tikv/pd/server" "github.com/unrolled/render" @@ -154,16 +154,16 @@ func (h *hotStatusHandler) GetHotStores(w http.ResponseWriter, r *http.Request) id := store.GetID() if loads, ok := storesLoads[id]; ok { if store.IsTiFlash() { - stats.BytesWriteStats[id] = loads[statistics.StoreRegionsWriteBytes] - stats.KeysWriteStats[id] = loads[statistics.StoreRegionsWriteKeys] + stats.BytesWriteStats[id] = loads[utils.StoreRegionsWriteBytes] + stats.KeysWriteStats[id] = loads[utils.StoreRegionsWriteKeys] } else { - stats.BytesWriteStats[id] = loads[statistics.StoreWriteBytes] - stats.KeysWriteStats[id] = loads[statistics.StoreWriteKeys] + stats.BytesWriteStats[id] = loads[utils.StoreWriteBytes] + stats.KeysWriteStats[id] = loads[utils.StoreWriteKeys] } - stats.BytesReadStats[id] = loads[statistics.StoreReadBytes] - stats.KeysReadStats[id] = loads[statistics.StoreReadKeys] - stats.QueryWriteStats[id] = loads[statistics.StoreWriteQuery] - stats.QueryReadStats[id] = loads[statistics.StoreReadQuery] + stats.BytesReadStats[id] = loads[utils.StoreReadBytes] + stats.KeysReadStats[id] = loads[utils.StoreReadKeys] + stats.QueryWriteStats[id] = loads[utils.StoreWriteQuery] + stats.QueryReadStats[id] = loads[utils.StoreReadQuery] } } h.rd.JSON(w, http.StatusOK, stats) diff --git a/server/cluster/cluster.go b/server/cluster/cluster.go index ac1c6952b22..c39dddd22f7 100644 --- a/server/cluster/cluster.go +++ b/server/cluster/cluster.go @@ -53,6 +53,7 @@ import ( "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/statistics" "github.com/tikv/pd/pkg/statistics/buckets" + "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/storage" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/syncer" @@ -159,7 +160,6 @@ type RaftCluster struct { labelLevelStats *statistics.LabelStatistics regionStats *statistics.RegionStatistics hotStat *statistics.HotStat - hotBuckets *buckets.HotBucketCache slowStat *statistics.SlowStat ruleManager *placement.RuleManager regionLabeler *labeler.RegionLabeler @@ -267,7 +267,6 @@ func (c *RaftCluster) InitCluster( c.ctx, c.cancel = context.WithCancel(c.serverCtx) c.labelLevelStats = statistics.NewLabelStatistics() c.hotStat = statistics.NewHotStat(c.ctx) - c.hotBuckets = buckets.NewBucketsCache(c.ctx) c.slowStat = statistics.NewSlowStat(c.ctx) c.progressManager = progress.NewManager() c.changedRegions = make(chan *core.RegionInfo, defaultChangedRegionsLimit) @@ -919,12 +918,12 @@ func (c *RaftCluster) HandleStoreHeartbeat(heartbeat *pdpb.StoreHeartbeatRequest } readQueryNum := core.GetReadQueryNum(peerStat.GetQueryStats()) loads := []float64{ - statistics.RegionReadBytes: float64(peerStat.GetReadBytes()), - statistics.RegionReadKeys: float64(peerStat.GetReadKeys()), - statistics.RegionReadQueryNum: float64(readQueryNum), - statistics.RegionWriteBytes: 0, - statistics.RegionWriteKeys: 0, - statistics.RegionWriteQueryNum: 0, + utils.RegionReadBytes: float64(peerStat.GetReadBytes()), + utils.RegionReadKeys: float64(peerStat.GetReadKeys()), + utils.RegionReadQueryNum: float64(readQueryNum), + utils.RegionWriteBytes: 0, + utils.RegionWriteKeys: 0, + utils.RegionWriteQueryNum: 0, } peerInfo := core.NewPeerInfo(peer, loads, interval) c.hotStat.CheckReadAsync(statistics.NewCheckPeerTask(peerInfo, region)) @@ -2250,7 +2249,7 @@ func (c *RaftCluster) IsRegionHot(region *core.RegionInfo) bool { } // GetHotPeerStat returns hot peer stat with specified regionID and storeID. -func (c *RaftCluster) GetHotPeerStat(rw statistics.RWType, regionID, storeID uint64) *statistics.HotPeerStat { +func (c *RaftCluster) GetHotPeerStat(rw utils.RWType, regionID, storeID uint64) *statistics.HotPeerStat { return c.hotStat.GetHotPeerStat(rw, regionID, storeID) } @@ -2260,24 +2259,20 @@ func (c *RaftCluster) GetHotPeerStat(rw statistics.RWType, regionID, storeID uin func (c *RaftCluster) RegionReadStats() map[uint64][]*statistics.HotPeerStat { // As read stats are reported by store heartbeat, the threshold needs to be adjusted. threshold := c.GetOpts().GetHotRegionCacheHitsThreshold() * - (statistics.RegionHeartBeatReportInterval / statistics.StoreHeartBeatReportInterval) - return c.hotStat.RegionStats(statistics.Read, threshold) -} - -// BucketsStats returns hot region's buckets stats. -func (c *RaftCluster) BucketsStats(degree int, regions ...uint64) map[uint64][]*buckets.BucketStat { - task := buckets.NewCollectBucketStatsTask(degree, regions...) - if !c.hotBuckets.CheckAsync(task) { - return nil - } - return task.WaitRet(c.ctx) + (utils.RegionHeartBeatReportInterval / utils.StoreHeartBeatReportInterval) + return c.hotStat.RegionStats(utils.Read, threshold) } // RegionWriteStats returns hot region's write stats. // The result only includes peers that are hot enough. func (c *RaftCluster) RegionWriteStats() map[uint64][]*statistics.HotPeerStat { // RegionStats is a thread-safe method - return c.hotStat.RegionStats(statistics.Write, c.GetOpts().GetHotRegionCacheHitsThreshold()) + return c.hotStat.RegionStats(utils.Write, c.GetOpts().GetHotRegionCacheHitsThreshold()) +} + +// BucketsStats returns hot region's buckets stats. +func (c *RaftCluster) BucketsStats(degree int, regionIDs ...uint64) map[uint64][]*buckets.BucketStat { + return c.hotStat.BucketsStats(degree, regionIDs...) } // TODO: remove me. @@ -2294,7 +2289,7 @@ func (c *RaftCluster) putRegion(region *core.RegionInfo) error { // GetHotWriteRegions gets hot write regions' info. func (c *RaftCluster) GetHotWriteRegions(storeIDs ...uint64) *statistics.StoreHotPeersInfos { - hotWriteRegions := c.coordinator.GetHotRegionsByType(statistics.Write) + hotWriteRegions := c.coordinator.GetHotRegionsByType(utils.Write) if len(storeIDs) > 0 && hotWriteRegions != nil { hotWriteRegions = getHotRegionsByStoreIDs(hotWriteRegions, storeIDs...) } @@ -2303,7 +2298,7 @@ func (c *RaftCluster) GetHotWriteRegions(storeIDs ...uint64) *statistics.StoreHo // GetHotReadRegions gets hot read regions' info. func (c *RaftCluster) GetHotReadRegions(storeIDs ...uint64) *statistics.StoreHotPeersInfos { - hotReadRegions := c.coordinator.GetHotRegionsByType(statistics.Read) + hotReadRegions := c.coordinator.GetHotRegionsByType(utils.Read) if len(storeIDs) > 0 && hotReadRegions != nil { hotReadRegions = getHotRegionsByStoreIDs(hotReadRegions, storeIDs...) } diff --git a/server/cluster/cluster_test.go b/server/cluster/cluster_test.go index 19584eb1087..34cfe702f0a 100644 --- a/server/cluster/cluster_test.go +++ b/server/cluster/cluster_test.go @@ -49,6 +49,7 @@ import ( "github.com/tikv/pd/pkg/schedule/placement" "github.com/tikv/pd/pkg/schedule/schedulers" "github.com/tikv/pd/pkg/statistics" + "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/storage" "github.com/tikv/pd/pkg/utils/operatorutil" "github.com/tikv/pd/pkg/utils/testutil" @@ -150,23 +151,23 @@ func TestStoreHeartbeat(t *testing.T) { re.NoError(cluster.HandleStoreHeartbeat(hotReq, hotResp)) re.Equal("v1", cluster.GetStore(1).GetStoreLimit().Version()) time.Sleep(20 * time.Millisecond) - storeStats := cluster.hotStat.RegionStats(statistics.Read, 3) + storeStats := cluster.hotStat.RegionStats(utils.Read, 3) re.Len(storeStats[1], 1) re.Equal(uint64(1), storeStats[1][0].RegionID) interval := float64(hotHeartBeat.Interval.EndTimestamp - hotHeartBeat.Interval.StartTimestamp) - re.Len(storeStats[1][0].Loads, statistics.DimLen) - re.Equal(float64(hotHeartBeat.PeerStats[0].ReadBytes)/interval, storeStats[1][0].Loads[statistics.ByteDim]) - re.Equal(float64(hotHeartBeat.PeerStats[0].ReadKeys)/interval, storeStats[1][0].Loads[statistics.KeyDim]) - re.Equal(float64(hotHeartBeat.PeerStats[0].QueryStats.Get)/interval, storeStats[1][0].Loads[statistics.QueryDim]) + re.Len(storeStats[1][0].Loads, utils.DimLen) + re.Equal(float64(hotHeartBeat.PeerStats[0].ReadBytes)/interval, storeStats[1][0].Loads[utils.ByteDim]) + re.Equal(float64(hotHeartBeat.PeerStats[0].ReadKeys)/interval, storeStats[1][0].Loads[utils.KeyDim]) + re.Equal(float64(hotHeartBeat.PeerStats[0].QueryStats.Get)/interval, storeStats[1][0].Loads[utils.QueryDim]) // After cold heartbeat, we won't find region 1 peer in regionStats re.NoError(cluster.HandleStoreHeartbeat(coldReq, coldResp)) time.Sleep(20 * time.Millisecond) - storeStats = cluster.hotStat.RegionStats(statistics.Read, 1) + storeStats = cluster.hotStat.RegionStats(utils.Read, 1) re.Empty(storeStats[1]) // After hot heartbeat, we can find region 1 peer again re.NoError(cluster.HandleStoreHeartbeat(hotReq, hotResp)) time.Sleep(20 * time.Millisecond) - storeStats = cluster.hotStat.RegionStats(statistics.Read, 3) + storeStats = cluster.hotStat.RegionStats(utils.Read, 3) re.Len(storeStats[1], 1) re.Equal(uint64(1), storeStats[1][0].RegionID) // after several cold heartbeats, and one hot heartbeat, we also can't find region 1 peer @@ -174,19 +175,19 @@ func TestStoreHeartbeat(t *testing.T) { re.NoError(cluster.HandleStoreHeartbeat(coldReq, coldResp)) re.NoError(cluster.HandleStoreHeartbeat(coldReq, coldResp)) time.Sleep(20 * time.Millisecond) - storeStats = cluster.hotStat.RegionStats(statistics.Read, 0) + storeStats = cluster.hotStat.RegionStats(utils.Read, 0) re.Empty(storeStats[1]) re.Nil(cluster.HandleStoreHeartbeat(hotReq, hotResp)) time.Sleep(20 * time.Millisecond) - storeStats = cluster.hotStat.RegionStats(statistics.Read, 1) + storeStats = cluster.hotStat.RegionStats(utils.Read, 1) re.Len(storeStats[1], 0) - storeStats = cluster.hotStat.RegionStats(statistics.Read, 3) + storeStats = cluster.hotStat.RegionStats(utils.Read, 3) re.Empty(storeStats[1]) // after 2 hot heartbeats, wo can find region 1 peer again re.NoError(cluster.HandleStoreHeartbeat(hotReq, hotResp)) re.NoError(cluster.HandleStoreHeartbeat(hotReq, hotResp)) time.Sleep(20 * time.Millisecond) - storeStats = cluster.hotStat.RegionStats(statistics.Read, 3) + storeStats = cluster.hotStat.RegionStats(utils.Read, 3) re.Len(storeStats[1], 1) re.Equal(uint64(1), storeStats[1][0].RegionID) } @@ -624,14 +625,14 @@ func TestRegionHeartbeatHotStat(t *testing.T) { EndKey: []byte{byte(1 + 1)}, RegionEpoch: &metapb.RegionEpoch{ConfVer: 2, Version: 2}, } - region := core.NewRegionInfo(regionMeta, leader, core.WithInterval(&pdpb.TimeInterval{StartTimestamp: 0, EndTimestamp: statistics.RegionHeartBeatReportInterval}), + region := core.NewRegionInfo(regionMeta, leader, core.WithInterval(&pdpb.TimeInterval{StartTimestamp: 0, EndTimestamp: utils.RegionHeartBeatReportInterval}), core.SetWrittenBytes(30000*10), core.SetWrittenKeys(300000*10)) err = cluster.processRegionHeartbeat(region) re.NoError(err) // wait HotStat to update items time.Sleep(time.Second) - stats := cluster.hotStat.RegionStats(statistics.Write, 0) + stats := cluster.hotStat.RegionStats(utils.Write, 0) re.Len(stats[1], 1) re.Len(stats[2], 1) re.Len(stats[3], 1) @@ -644,7 +645,7 @@ func TestRegionHeartbeatHotStat(t *testing.T) { re.NoError(err) // wait HotStat to update items time.Sleep(time.Second) - stats = cluster.hotStat.RegionStats(statistics.Write, 0) + stats = cluster.hotStat.RegionStats(utils.Write, 0) re.Len(stats[1], 1) re.Empty(stats[2]) re.Len(stats[3], 1) @@ -2420,9 +2421,9 @@ func TestCollectMetrics(t *testing.T) { RegionID: uint64(i*1000 + k), Loads: []float64{10, 20, 30}, HotDegree: 10, - AntiCount: statistics.HotRegionAntiCount, // for write + AntiCount: utils.HotRegionAntiCount, // for write } - tc.hotStat.HotCache.Update(item, statistics.Write) + tc.hotStat.HotCache.Update(item, utils.Write) } } controller := co.GetSchedulersController() @@ -2434,7 +2435,7 @@ func TestCollectMetrics(t *testing.T) { stores := co.GetCluster().GetStores() regionStats := co.GetCluster().RegionWriteStats() status1 := statistics.CollectHotPeerInfos(stores, regionStats) - status2 := statistics.GetHotStatus(stores, co.GetCluster().GetStoresLoads(), regionStats, statistics.Write, co.GetCluster().GetSchedulerConfig().IsTraceRegionFlow()) + status2 := statistics.GetHotStatus(stores, co.GetCluster().GetStoresLoads(), regionStats, utils.Write, co.GetCluster().GetSchedulerConfig().IsTraceRegionFlow()) for _, s := range status2.AsLeader { s.Stats = nil } diff --git a/server/cluster/cluster_worker.go b/server/cluster/cluster_worker.go index 51781bde7f6..82113bc1656 100644 --- a/server/cluster/cluster_worker.go +++ b/server/cluster/cluster_worker.go @@ -250,6 +250,6 @@ func (c *RaftCluster) HandleReportBuckets(b *metapb.Buckets) error { if err := c.processReportBuckets(b); err != nil { return err } - c.hotBuckets.CheckAsync(buckets.NewCheckPeerTask(b)) + c.hotStat.CheckAsync(buckets.NewCheckPeerTask(b)) return nil } diff --git a/tests/pdctl/hot/hot_test.go b/tests/pdctl/hot/hot_test.go index 594c14f707f..4b4ff3f0f9e 100644 --- a/tests/pdctl/hot/hot_test.go +++ b/tests/pdctl/hot/hot_test.go @@ -27,6 +27,7 @@ import ( "github.com/stretchr/testify/require" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/statistics" + "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/storage" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/utils/typeutil" @@ -82,9 +83,9 @@ func TestHot(t *testing.T) { newStats.KeysWritten = keysWritten newStats.KeysRead = keysRead rc := leaderServer.GetRaftCluster() - for i := statistics.DefaultWriteMfSize; i > 0; i-- { - start := uint64(now - statistics.StoreHeartBeatReportInterval*int64(i)) - end := start + statistics.StoreHeartBeatReportInterval + for i := utils.DefaultWriteMfSize; i > 0; i-- { + start := uint64(now - utils.StoreHeartBeatReportInterval*int64(i)) + end := start + utils.StoreHeartBeatReportInterval newStats.Interval = &pdpb.TimeInterval{StartTimestamp: start, EndTimestamp: end} rc.GetStoresStats().Observe(ss.GetID(), newStats) } @@ -98,10 +99,10 @@ func TestHot(t *testing.T) { re.NoError(err) hotStores := api.HotStoreStats{} re.NoError(json.Unmarshal(output, &hotStores)) - re.Equal(float64(bytesWritten)/statistics.StoreHeartBeatReportInterval, hotStores.BytesWriteStats[1]) - re.Equal(float64(bytesRead)/statistics.StoreHeartBeatReportInterval, hotStores.BytesReadStats[1]) - re.Equal(float64(keysWritten)/statistics.StoreHeartBeatReportInterval, hotStores.KeysWriteStats[1]) - re.Equal(float64(keysRead)/statistics.StoreHeartBeatReportInterval, hotStores.KeysReadStats[1]) + re.Equal(float64(bytesWritten)/utils.StoreHeartBeatReportInterval, hotStores.BytesWriteStats[1]) + re.Equal(float64(bytesRead)/utils.StoreHeartBeatReportInterval, hotStores.BytesReadStats[1]) + re.Equal(float64(keysWritten)/utils.StoreHeartBeatReportInterval, hotStores.KeysWriteStats[1]) + re.Equal(float64(keysRead)/utils.StoreHeartBeatReportInterval, hotStores.KeysReadStats[1]) re.Equal(float64(bytesWritten), hotStores.BytesWriteStats[2]) re.Equal(float64(keysWritten), hotStores.KeysWriteStats[2]) @@ -133,12 +134,12 @@ func TestHot(t *testing.T) { switch hotType { case "read": loads := []float64{ - statistics.RegionReadBytes: float64(1000000000 * reportInterval), - statistics.RegionReadKeys: float64(1000000000 * reportInterval), - statistics.RegionReadQueryNum: float64(1000000000 * reportInterval), - statistics.RegionWriteBytes: 0, - statistics.RegionWriteKeys: 0, - statistics.RegionWriteQueryNum: 0, + utils.RegionReadBytes: float64(1000000000 * reportInterval), + utils.RegionReadKeys: float64(1000000000 * reportInterval), + utils.RegionReadQueryNum: float64(1000000000 * reportInterval), + utils.RegionWriteBytes: 0, + utils.RegionWriteKeys: 0, + utils.RegionWriteQueryNum: 0, } leader := &metapb.Peer{ Id: 100 + regionIDCounter, @@ -150,10 +151,10 @@ func TestHot(t *testing.T) { }, leader) rc.GetHotStat().CheckReadAsync(statistics.NewCheckPeerTask(peerInfo, region)) testutil.Eventually(re, func() bool { - hotPeerStat := rc.GetHotPeerStat(statistics.Read, hotRegionID, hotStoreID) + hotPeerStat := rc.GetHotPeerStat(utils.Read, hotRegionID, hotStoreID) return hotPeerStat != nil }) - if reportInterval >= statistics.ReadReportInterval { + if reportInterval >= utils.StoreHeartBeatReportInterval { count++ } testHot(hotRegionID, hotStoreID, "read") @@ -164,10 +165,10 @@ func TestHot(t *testing.T) { []byte("c"), []byte("d"), core.SetWrittenBytes(1000000000*reportInterval), core.SetReportInterval(0, reportInterval)) testutil.Eventually(re, func() bool { - hotPeerStat := rc.GetHotPeerStat(statistics.Write, hotRegionID, hotStoreID) + hotPeerStat := rc.GetHotPeerStat(utils.Write, hotRegionID, hotStoreID) return hotPeerStat != nil }) - if reportInterval >= statistics.WriteReportInterval { + if reportInterval >= utils.RegionHeartBeatReportInterval { count++ } testHot(hotRegionID, hotStoreID, "write") @@ -177,20 +178,20 @@ func TestHot(t *testing.T) { reportIntervals := []uint64{ statistics.HotRegionReportMinInterval, statistics.HotRegionReportMinInterval + 1, - statistics.WriteReportInterval, - statistics.WriteReportInterval + 1, - statistics.WriteReportInterval * 2, - statistics.WriteReportInterval*2 + 1, + utils.RegionHeartBeatReportInterval, + utils.RegionHeartBeatReportInterval + 1, + utils.RegionHeartBeatReportInterval * 2, + utils.RegionHeartBeatReportInterval*2 + 1, } testCommand(reportIntervals, "write") count = 0 reportIntervals = []uint64{ statistics.HotRegionReportMinInterval, statistics.HotRegionReportMinInterval + 1, - statistics.ReadReportInterval, - statistics.ReadReportInterval + 1, - statistics.ReadReportInterval * 2, - statistics.ReadReportInterval*2 + 1, + utils.StoreHeartBeatReportInterval, + utils.StoreHeartBeatReportInterval + 1, + utils.StoreHeartBeatReportInterval * 2, + utils.StoreHeartBeatReportInterval*2 + 1, } testCommand(reportIntervals, "read") } @@ -228,15 +229,15 @@ func TestHotWithStoreID(t *testing.T) { } defer cluster.Destroy() - pdctl.MustPutRegion(re, cluster, 1, 1, []byte("a"), []byte("b"), core.SetWrittenBytes(3000000000), core.SetReportInterval(0, statistics.WriteReportInterval)) - pdctl.MustPutRegion(re, cluster, 2, 2, []byte("c"), []byte("d"), core.SetWrittenBytes(6000000000), core.SetReportInterval(0, statistics.WriteReportInterval)) - pdctl.MustPutRegion(re, cluster, 3, 1, []byte("e"), []byte("f"), core.SetWrittenBytes(9000000000), core.SetReportInterval(0, statistics.WriteReportInterval)) + pdctl.MustPutRegion(re, cluster, 1, 1, []byte("a"), []byte("b"), core.SetWrittenBytes(3000000000), core.SetReportInterval(0, utils.RegionHeartBeatReportInterval)) + pdctl.MustPutRegion(re, cluster, 2, 2, []byte("c"), []byte("d"), core.SetWrittenBytes(6000000000), core.SetReportInterval(0, utils.RegionHeartBeatReportInterval)) + pdctl.MustPutRegion(re, cluster, 3, 1, []byte("e"), []byte("f"), core.SetWrittenBytes(9000000000), core.SetReportInterval(0, utils.RegionHeartBeatReportInterval)) // wait hot scheduler starts rc := leaderServer.GetRaftCluster() testutil.Eventually(re, func() bool { - return rc.GetHotPeerStat(statistics.Write, 1, 1) != nil && - rc.GetHotPeerStat(statistics.Write, 2, 2) != nil && - rc.GetHotPeerStat(statistics.Write, 3, 1) != nil + return rc.GetHotPeerStat(utils.Write, 1, 1) != nil && + rc.GetHotPeerStat(utils.Write, 2, 2) != nil && + rc.GetHotPeerStat(utils.Write, 3, 1) != nil }) args := []string{"-u", pdAddr, "hot", "write", "1"} output, err := pdctl.ExecuteCommand(cmd, args...) @@ -304,13 +305,13 @@ func TestHistoryHotRegions(t *testing.T) { defer cluster.Destroy() startTime := time.Now().Unix() pdctl.MustPutRegion(re, cluster, 1, 1, []byte("a"), []byte("b"), core.SetWrittenBytes(3000000000), - core.SetReportInterval(uint64(startTime-statistics.RegionHeartBeatReportInterval), uint64(startTime))) + core.SetReportInterval(uint64(startTime-utils.RegionHeartBeatReportInterval), uint64(startTime))) pdctl.MustPutRegion(re, cluster, 2, 2, []byte("c"), []byte("d"), core.SetWrittenBytes(6000000000), - core.SetReportInterval(uint64(startTime-statistics.RegionHeartBeatReportInterval), uint64(startTime))) + core.SetReportInterval(uint64(startTime-utils.RegionHeartBeatReportInterval), uint64(startTime))) pdctl.MustPutRegion(re, cluster, 3, 1, []byte("e"), []byte("f"), core.SetWrittenBytes(9000000000), - core.SetReportInterval(uint64(startTime-statistics.RegionHeartBeatReportInterval), uint64(startTime))) + core.SetReportInterval(uint64(startTime-utils.RegionHeartBeatReportInterval), uint64(startTime))) pdctl.MustPutRegion(re, cluster, 4, 3, []byte("g"), []byte("h"), core.SetWrittenBytes(9000000000), - core.SetReportInterval(uint64(startTime-statistics.RegionHeartBeatReportInterval), uint64(startTime))) + core.SetReportInterval(uint64(startTime-utils.RegionHeartBeatReportInterval), uint64(startTime))) // wait hot scheduler starts testutil.Eventually(re, func() bool { hotRegionStorage := leaderServer.GetServer().GetHistoryHotRegionStorage() @@ -418,15 +419,15 @@ func TestHotWithoutHotPeer(t *testing.T) { err := leaderServer.GetServer().GetRaftCluster().HandleStoreHeartbeat(&pdpb.StoreHeartbeatRequest{ Stats: &pdpb.StoreStats{ StoreId: store.Id, - BytesRead: uint64(load * statistics.StoreHeartBeatReportInterval), - KeysRead: uint64(load * statistics.StoreHeartBeatReportInterval), - BytesWritten: uint64(load * statistics.StoreHeartBeatReportInterval), - KeysWritten: uint64(load * statistics.StoreHeartBeatReportInterval), + BytesRead: uint64(load * utils.StoreHeartBeatReportInterval), + KeysRead: uint64(load * utils.StoreHeartBeatReportInterval), + BytesWritten: uint64(load * utils.StoreHeartBeatReportInterval), + KeysWritten: uint64(load * utils.StoreHeartBeatReportInterval), Capacity: 1000 * units.MiB, Available: 1000 * units.MiB, Interval: &pdpb.TimeInterval{ - StartTimestamp: timestamp + uint64(i*statistics.StoreHeartBeatReportInterval), - EndTimestamp: timestamp + uint64((i+1)*statistics.StoreHeartBeatReportInterval)}, + StartTimestamp: timestamp + uint64(i*utils.StoreHeartBeatReportInterval), + EndTimestamp: timestamp + uint64((i+1)*utils.StoreHeartBeatReportInterval)}, }, }, &pdpb.StoreHeartbeatResponse{}) re.NoError(err) diff --git a/tests/pdctl/store/store_test.go b/tests/pdctl/store/store_test.go index 42ceaede428..0ac68e35d98 100644 --- a/tests/pdctl/store/store_test.go +++ b/tests/pdctl/store/store_test.go @@ -24,7 +24,7 @@ import ( "github.com/stretchr/testify/require" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/storelimit" - "github.com/tikv/pd/pkg/statistics" + "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/server/api" "github.com/tikv/pd/tests" "github.com/tikv/pd/tests/pdctl" @@ -513,8 +513,8 @@ func TestTombstoneStore(t *testing.T) { pdctl.MustPutStore(re, leaderServer.GetServer(), store.Store.Store) } defer cluster.Destroy() - pdctl.MustPutRegion(re, cluster, 1, 2, []byte("a"), []byte("b"), core.SetWrittenBytes(3000000000), core.SetReportInterval(0, statistics.WriteReportInterval)) - pdctl.MustPutRegion(re, cluster, 2, 3, []byte("b"), []byte("c"), core.SetWrittenBytes(3000000000), core.SetReportInterval(0, statistics.WriteReportInterval)) + pdctl.MustPutRegion(re, cluster, 1, 2, []byte("a"), []byte("b"), core.SetWrittenBytes(3000000000), core.SetReportInterval(0, utils.RegionHeartBeatReportInterval)) + pdctl.MustPutRegion(re, cluster, 2, 3, []byte("b"), []byte("c"), core.SetWrittenBytes(3000000000), core.SetReportInterval(0, utils.RegionHeartBeatReportInterval)) // store remove-tombstone args := []string{"-u", pdAddr, "store", "remove-tombstone"} output, err := pdctl.ExecuteCommand(cmd, args...) diff --git a/tests/server/storage/hot_region_storage_test.go b/tests/server/storage/hot_region_storage_test.go index 44b1cfd274a..3ae5db729c8 100644 --- a/tests/server/storage/hot_region_storage_test.go +++ b/tests/server/storage/hot_region_storage_test.go @@ -24,6 +24,7 @@ import ( "github.com/stretchr/testify/require" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/statistics" + "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/storage" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server/config" @@ -68,17 +69,17 @@ func TestHotRegionStorage(t *testing.T) { defer cluster.Destroy() startTime := time.Now().Unix() pdctl.MustPutRegion(re, cluster, 1, 1, []byte("a"), []byte("b"), core.SetWrittenBytes(3000000000), - core.SetReportInterval(uint64(startTime-statistics.RegionHeartBeatReportInterval), uint64(startTime))) + core.SetReportInterval(uint64(startTime-utils.RegionHeartBeatReportInterval), uint64(startTime))) pdctl.MustPutRegion(re, cluster, 2, 2, []byte("c"), []byte("d"), core.SetWrittenBytes(6000000000), - core.SetReportInterval(uint64(startTime-statistics.RegionHeartBeatReportInterval), uint64(startTime))) + core.SetReportInterval(uint64(startTime-utils.RegionHeartBeatReportInterval), uint64(startTime))) pdctl.MustPutRegion(re, cluster, 3, 1, []byte("e"), []byte("f"), - core.SetReportInterval(uint64(startTime-statistics.RegionHeartBeatReportInterval), uint64(startTime))) + core.SetReportInterval(uint64(startTime-utils.RegionHeartBeatReportInterval), uint64(startTime))) pdctl.MustPutRegion(re, cluster, 4, 2, []byte("g"), []byte("h"), - core.SetReportInterval(uint64(startTime-statistics.RegionHeartBeatReportInterval), uint64(startTime))) + core.SetReportInterval(uint64(startTime-utils.RegionHeartBeatReportInterval), uint64(startTime))) storeStats := []*pdpb.StoreStats{ { StoreId: 1, - Interval: &pdpb.TimeInterval{StartTimestamp: uint64(startTime - statistics.StoreHeartBeatReportInterval), EndTimestamp: uint64(startTime)}, + Interval: &pdpb.TimeInterval{StartTimestamp: uint64(startTime - utils.StoreHeartBeatReportInterval), EndTimestamp: uint64(startTime)}, PeerStats: []*pdpb.PeerStat{ { RegionId: 3, @@ -88,7 +89,7 @@ func TestHotRegionStorage(t *testing.T) { }, { StoreId: 2, - Interval: &pdpb.TimeInterval{StartTimestamp: uint64(startTime - statistics.StoreHeartBeatReportInterval), EndTimestamp: uint64(startTime)}, + Interval: &pdpb.TimeInterval{StartTimestamp: uint64(startTime - utils.StoreHeartBeatReportInterval), EndTimestamp: uint64(startTime)}, PeerStats: []*pdpb.PeerStat{ { RegionId: 4, @@ -176,7 +177,7 @@ func TestHotRegionStorageReservedDayConfigChange(t *testing.T) { defer cluster.Destroy() startTime := time.Now().Unix() pdctl.MustPutRegion(re, cluster, 1, 1, []byte("a"), []byte("b"), core.SetWrittenBytes(3000000000), - core.SetReportInterval(uint64(startTime-statistics.RegionHeartBeatReportInterval), uint64(startTime))) + core.SetReportInterval(uint64(startTime-utils.RegionHeartBeatReportInterval), uint64(startTime))) var iter storage.HotRegionStorageIterator var next *storage.HistoryHotRegion testutil.Eventually(re, func() bool { // wait for the history hot region to be written to the storage @@ -197,7 +198,7 @@ func TestHotRegionStorageReservedDayConfigChange(t *testing.T) { leaderServer.GetServer().SetScheduleConfig(schedule) time.Sleep(3 * interval) pdctl.MustPutRegion(re, cluster, 2, 2, []byte("c"), []byte("d"), core.SetWrittenBytes(6000000000), - core.SetReportInterval(uint64(time.Now().Unix()-statistics.WriteReportInterval), uint64(time.Now().Unix()))) + core.SetReportInterval(uint64(time.Now().Unix()-utils.RegionHeartBeatReportInterval), uint64(time.Now().Unix()))) time.Sleep(10 * interval) hotRegionStorage := leaderServer.GetServer().GetHistoryHotRegionStorage() iter = hotRegionStorage.NewIterator([]string{storage.WriteType.String()}, startTime*1000, time.Now().UnixMilli()) @@ -269,7 +270,7 @@ func TestHotRegionStorageWriteIntervalConfigChange(t *testing.T) { startTime := time.Now().Unix() pdctl.MustPutRegion(re, cluster, 1, 1, []byte("a"), []byte("b"), core.SetWrittenBytes(3000000000), - core.SetReportInterval(uint64(startTime-statistics.WriteReportInterval), uint64(startTime))) + core.SetReportInterval(uint64(startTime-utils.RegionHeartBeatReportInterval), uint64(startTime))) var iter storage.HotRegionStorageIterator var next *storage.HistoryHotRegion testutil.Eventually(re, func() bool { // wait for the history hot region to be written to the storage @@ -290,7 +291,7 @@ func TestHotRegionStorageWriteIntervalConfigChange(t *testing.T) { leaderServer.GetServer().SetScheduleConfig(schedule) time.Sleep(3 * interval) pdctl.MustPutRegion(re, cluster, 2, 2, []byte("c"), []byte("d"), core.SetWrittenBytes(6000000000), - core.SetReportInterval(uint64(time.Now().Unix()-statistics.WriteReportInterval), uint64(time.Now().Unix()))) + core.SetReportInterval(uint64(time.Now().Unix()-utils.RegionHeartBeatReportInterval), uint64(time.Now().Unix()))) time.Sleep(10 * interval) // it cant get new hot region because wait time smaller than hot region write interval hotRegionStorage := leaderServer.GetServer().GetHistoryHotRegionStorage() diff --git a/tools/pd-ctl/pdctl/command/scheduler.go b/tools/pd-ctl/pdctl/command/scheduler.go index 93335723641..57658022858 100644 --- a/tools/pd-ctl/pdctl/command/scheduler.go +++ b/tools/pd-ctl/pdctl/command/scheduler.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/errors" "github.com/spf13/cobra" - "github.com/tikv/pd/pkg/statistics" + "github.com/tikv/pd/pkg/statistics/utils" ) var ( @@ -717,14 +717,14 @@ func postSchedulerConfigCommandFunc(cmd *cobra.Command, schedulerName string, ar priorities := make([]string, 0) prioritiesMap := make(map[string]struct{}) for _, priority := range strings.Split(value, ",") { - if priority != statistics.BytePriority && priority != statistics.KeyPriority && priority != statistics.QueryPriority { + if priority != utils.BytePriority && priority != utils.KeyPriority && priority != utils.QueryPriority { cmd.Println(fmt.Sprintf("priority should be one of [%s, %s, %s]", - statistics.BytePriority, - statistics.QueryPriority, - statistics.KeyPriority)) + utils.BytePriority, + utils.QueryPriority, + utils.KeyPriority)) return } - if priority == statistics.QueryPriority && key == "write-peer-priorities" { + if priority == utils.QueryPriority && key == "write-peer-priorities" { cmd.Println("query is not allowed to be set in priorities for write-peer-priorities") return }