Documentation ¶
Index ¶
- Constants
- Variables
- func GetRegionLabelIsolation(stores []*core.StoreInfo, labels []string) string
- func NewStoreStatisticsMap(opt config.ConfProvider) *storeStatisticsMap
- func ObserveHotStat(store *core.StoreInfo, stats *StoresStats)
- func Reset()
- func ResetHotCacheStatusMetrics()
- func ResetLabelStatsMetrics()
- func ResetRegionStatsMetrics()
- func ResetStoreStatistics(storeAddress string, id string)
- func SummaryStoreInfos(stores []*core.StoreInfo) map[uint64]*StoreSummaryInfo
- func SummaryStoresLoad(storeInfos map[uint64]*StoreSummaryInfo, storesLoads map[uint64][]float64, ...) map[uint64]*StoreLoadDetail
- type HotCache
- func (w *HotCache) CheckReadAsync(task func(cache *HotPeerCache)) bool
- func (w *HotCache) CheckReadPeerSync(region *core.RegionInfo, peers []*metapb.Peer, loads []float64, ...) []*HotPeerStat
- func (w *HotCache) CheckWriteAsync(task func(cache *HotPeerCache)) bool
- func (w *HotCache) CheckWritePeerSync(region *core.RegionInfo, peers []*metapb.Peer, loads []float64, ...) []*HotPeerStat
- func (w *HotCache) CleanCache()
- func (w *HotCache) CollectMetrics()
- func (w *HotCache) ExpiredReadItems(region *core.RegionInfo) []*HotPeerStat
- func (w *HotCache) ExpiredWriteItems(region *core.RegionInfo) []*HotPeerStat
- func (w *HotCache) GetHotPeerStat(kind utils.RWType, regionID, storeID uint64) *HotPeerStat
- func (w *HotCache) GetHotPeerStats(kind utils.RWType, minHotDegree int) map[uint64][]*HotPeerStat
- func (w *HotCache) GetThresholds(kind utils.RWType, storeID uint64) []float64
- func (w *HotCache) IsRegionHot(region *core.RegionInfo, minHotDegree int) bool
- func (w *HotCache) Update(item *HotPeerStat, kind utils.RWType)
- type HotPeerCache
- func (f *HotPeerCache) CheckColdPeer(storeID uint64, reportRegions map[uint64]*core.RegionInfo, interval uint64) (ret []*HotPeerStat)
- func (f *HotPeerCache) CheckPeerFlow(region *core.RegionInfo, peers []*metapb.Peer, deltaLoads []float64, ...) []*HotPeerStat
- func (f *HotPeerCache) CollectExpiredItems(region *core.RegionInfo) []*HotPeerStat
- func (f *HotPeerCache) GetHotPeerStats(minHotDegree int) map[uint64][]*HotPeerStat
- func (f *HotPeerCache) UpdateStat(item *HotPeerStat)
- type HotPeerStat
- func (stat *HotPeerStat) Clone() *HotPeerStat
- func (stat *HotPeerStat) GetActionType() utils.ActionType
- func (stat *HotPeerStat) GetLoad(dim int) float64
- func (stat *HotPeerStat) GetLoads() []float64
- func (stat *HotPeerStat) GetStores() []uint64
- func (stat *HotPeerStat) ID() uint64
- func (stat *HotPeerStat) IsLeader() bool
- func (stat *HotPeerStat) IsNeedCoolDownTransferLeader(minHotDegree int, rwTy utils.RWType) bool
- func (stat *HotPeerStat) Less(dim int, than utils.TopNItem) bool
- func (stat *HotPeerStat) Log(str string)
- type HotPeerStatShow
- type HotPeersStat
- type HotStat
- type Influence
- type LabelStatistics
- func (l *LabelStatistics) ClearDefunctRegions()
- func (l *LabelStatistics) Collect()
- func (l *LabelStatistics) GetLabelCounter() map[string]int
- func (l *LabelStatistics) MarkDefunctRegion(regionID uint64)
- func (l *LabelStatistics) Observe(region *core.RegionInfo, stores []*core.StoreInfo, labels []string)
- type RegionInfoProvider
- type RegionInfoWithTS
- type RegionStatInformer
- type RegionStatisticType
- type RegionStatistics
- func (r *RegionStatistics) ClearDefunctRegion(regionID uint64)
- func (r *RegionStatistics) Collect()
- func (r *RegionStatistics) GetRegionStatsByType(typ RegionStatisticType) []*core.RegionInfo
- func (r *RegionStatistics) IsRegionStatsType(regionID uint64, typ RegionStatisticType) bool
- func (r *RegionStatistics) Observe(region *core.RegionInfo, stores []*core.StoreInfo)
- func (r *RegionStatistics) RegionStatsNeedUpdate(region *core.RegionInfo) bool
- type RegionStats
- type RollingStoreStats
- func (r *RollingStoreStats) GetInstantLoad(k utils.StoreStatKind) float64
- func (r *RollingStoreStats) GetLoad(k utils.StoreStatKind) float64
- func (r *RollingStoreStats) Observe(stats *pdpb.StoreStats)
- func (r *RollingStoreStats) ObserveRegionsStats(writeBytesRate, writeKeysRate float64)
- func (r *RollingStoreStats) Set(stats *pdpb.StoreStats)
- func (r *RollingStoreStats) SetRegionsStats(writeBytesRate, writeKeysRate float64)
- type SlowStat
- type SlowStoresStats
- type StoreHistoryLoads
- func (s *StoreHistoryLoads) Add(storeID uint64, rwTp utils.RWType, kind constant.ResourceKind, ...)
- func (s *StoreHistoryLoads) Get(storeID uint64, rwTp utils.RWType, kind constant.ResourceKind) [][]float64
- func (s *StoreHistoryLoads) UpdateConfig(sampleDuration time.Duration, sampleInterval time.Duration) *StoreHistoryLoads
- type StoreHotPeersInfos
- type StoreHotPeersStat
- type StoreLoad
- type StoreLoadDetail
- type StoreLoadPred
- type StoreStatInformer
- type StoreSummaryInfo
- type StoresStats
- func (s *StoresStats) FilterUnhealthyStore(cluster core.StoreSetInformer)
- func (s *StoresStats) GetOrCreateRollingStoreStats(storeID uint64) *RollingStoreStats
- func (s *StoresStats) GetRollingStoreStats(storeID uint64) *RollingStoreStats
- func (s *StoresStats) GetStoresLoads() map[uint64][]float64
- func (s *StoresStats) Observe(storeID uint64, stats *pdpb.StoreStats)
- func (s *StoresStats) ObserveRegionsStats(storeIDs []uint64, writeBytesRates, writeKeysRates []float64)
- func (s *StoresStats) RemoveRollingStoreStats(storeID uint64)
- func (s *StoresStats) Set(storeID uint64, stats *pdpb.StoreStats)
- func (s *StoresStats) SetRegionsStats(storeIDs []uint64, writeBytesRates, writeKeysRates []float64)
Constants ¶
const ( // TopNN is the threshold which means we can get hot threshold from store. TopNN = 60 // HotThresholdRatio is used to calculate hot thresholds HotThresholdRatio = 0.8 // HotRegionReportMinInterval is used for the simulator and test HotRegionReportMinInterval = 3 )
const ( // RegionsStatsObserveInterval is the interval for obtaining statistics from RegionTree RegionsStatsObserveInterval = 30 * time.Second // RegionsStatsRollingWindowsSize is default size of median filter for data from regionStats RegionsStatsRollingWindowsSize = 9 )
const ( // DefaultHistorySampleInterval is the sampling interval for history load. DefaultHistorySampleInterval = 30 * time.Second // DefaultHistorySampleDuration is the duration for saving history load. DefaultHistorySampleDuration = 5 * time.Minute )
Variables ¶
var Denoising = true
Denoising is an option to calculate flow base on the real heartbeats. Should only turn off by the simulator and the test.
var ( // StoreLimitGauge is used to record the current store limit. StoreLimitGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: "pd", Subsystem: "cluster", Name: "store_limit", Help: "Status of the store limit.", }, []string{"store", "type"}) )
var ThresholdsUpdateInterval = 8 * time.Second
ThresholdsUpdateInterval is the default interval to update thresholds. the refresh interval should be less than store heartbeat interval to keep the next calculate must use the latest threshold.
Functions ¶
func GetRegionLabelIsolation ¶
GetRegionLabelIsolation returns the isolation level of the region.
func NewStoreStatisticsMap ¶
func NewStoreStatisticsMap(opt config.ConfProvider) *storeStatisticsMap
NewStoreStatisticsMap creates a new storeStatisticsMap.
func ObserveHotStat ¶
func ObserveHotStat(store *core.StoreInfo, stats *StoresStats)
ObserveHotStat records the hot region metrics for the store.
func ResetHotCacheStatusMetrics ¶
func ResetHotCacheStatusMetrics()
ResetHotCacheStatusMetrics resets the hot cache metrics.
func ResetLabelStatsMetrics ¶
func ResetLabelStatsMetrics()
ResetLabelStatsMetrics resets the metrics of the label status.
func ResetRegionStatsMetrics ¶
func ResetRegionStatsMetrics()
ResetRegionStatsMetrics resets the metrics of the regions' status.
func ResetStoreStatistics ¶
ResetStoreStatistics resets the metrics of store.
func SummaryStoreInfos ¶
func SummaryStoreInfos(stores []*core.StoreInfo) map[uint64]*StoreSummaryInfo
SummaryStoreInfos return a mapping from store to summary information.
func SummaryStoresLoad ¶
func SummaryStoresLoad( storeInfos map[uint64]*StoreSummaryInfo, storesLoads map[uint64][]float64, storesHistoryLoads *StoreHistoryLoads, storeHotPeers map[uint64][]*HotPeerStat, isTraceRegionFlow bool, rwTy utils.RWType, kind constant.ResourceKind, ) map[uint64]*StoreLoadDetail
SummaryStoresLoad Load information of all available stores. it will filter the hot peer and calculate the current and future stat(rate,count) for each store
Types ¶
type HotCache ¶
type HotCache struct {
// contains filtered or unexported fields
}
HotCache is a cache hold hot regions.
func NewHotCache ¶
func NewHotCache(ctx context.Context, cluster *core.BasicCluster) *HotCache
NewHotCache creates a new hot spot cache.
func (*HotCache) CheckReadAsync ¶
func (w *HotCache) CheckReadAsync(task func(cache *HotPeerCache)) bool
CheckReadAsync puts the flowItem into queue, and check it asynchronously
func (*HotCache) CheckReadPeerSync ¶
func (w *HotCache) CheckReadPeerSync(region *core.RegionInfo, peers []*metapb.Peer, loads []float64, interval uint64) []*HotPeerStat
CheckReadPeerSync checks the read status, returns update items. This is used for mockcluster, for test purpose.
func (*HotCache) CheckWriteAsync ¶
func (w *HotCache) CheckWriteAsync(task func(cache *HotPeerCache)) bool
CheckWriteAsync puts the flowItem into queue, and check it asynchronously
func (*HotCache) CheckWritePeerSync ¶
func (w *HotCache) CheckWritePeerSync(region *core.RegionInfo, peers []*metapb.Peer, loads []float64, interval uint64) []*HotPeerStat
CheckWritePeerSync checks the write status, returns update items. This is used for mockcluster, for test purpose.
func (*HotCache) CleanCache ¶
func (w *HotCache) CleanCache()
CleanCache cleans the cache. This is used for test purpose.
func (*HotCache) CollectMetrics ¶
func (w *HotCache) CollectMetrics()
CollectMetrics collects the hot cache metrics.
func (*HotCache) ExpiredReadItems ¶
func (w *HotCache) ExpiredReadItems(region *core.RegionInfo) []*HotPeerStat
ExpiredReadItems returns the read items which are already expired. This is used for mockcluster, for test purpose.
func (*HotCache) ExpiredWriteItems ¶
func (w *HotCache) ExpiredWriteItems(region *core.RegionInfo) []*HotPeerStat
ExpiredWriteItems returns the write items which are already expired. This is used for mockcluster, for test purpose.
func (*HotCache) GetHotPeerStat ¶
func (w *HotCache) GetHotPeerStat(kind utils.RWType, regionID, storeID uint64) *HotPeerStat
GetHotPeerStat returns hot peer stat with specified regionID and storeID.
func (*HotCache) GetHotPeerStats ¶
RegionStats returns the read or write statistics for hot regions. It returns a map where the keys are store IDs and the values are slices of HotPeerStat.
func (*HotCache) GetThresholds ¶
GetThresholds returns thresholds. This is used for test purpose.
func (*HotCache) IsRegionHot ¶
func (w *HotCache) IsRegionHot(region *core.RegionInfo, minHotDegree int) bool
IsRegionHot checks if the region is hot.
type HotPeerCache ¶
type HotPeerCache struct {
// contains filtered or unexported fields
}
HotPeerCache saves the hot peer's statistics.
func NewHotPeerCache ¶
func NewHotPeerCache(ctx context.Context, cluster *core.BasicCluster, kind utils.RWType) *HotPeerCache
NewHotPeerCache creates a HotPeerCache
func (*HotPeerCache) CheckColdPeer ¶
func (f *HotPeerCache) CheckColdPeer(storeID uint64, reportRegions map[uint64]*core.RegionInfo, interval uint64) (ret []*HotPeerStat)
CheckColdPeer checks the collect the un-heartbeat peer and maintain it.
func (*HotPeerCache) CheckPeerFlow ¶
func (f *HotPeerCache) CheckPeerFlow(region *core.RegionInfo, peers []*metapb.Peer, deltaLoads []float64, interval uint64) []*HotPeerStat
CheckPeerFlow checks the flow information of a peer. Notice: CheckPeerFlow couldn't be used concurrently. CheckPeerFlow will update oldItem's rollingLoads into newItem, thus we should use write lock here.
func (*HotPeerCache) CollectExpiredItems ¶
func (f *HotPeerCache) CollectExpiredItems(region *core.RegionInfo) []*HotPeerStat
CollectExpiredItems collects expired items, mark them as needDelete and puts them into inherit items
func (*HotPeerCache) GetHotPeerStats ¶
func (f *HotPeerCache) GetHotPeerStats(minHotDegree int) map[uint64][]*HotPeerStat
GetHotPeerStats returns the read or write statistics for hot regions. It returns a map where the keys are store IDs and the values are slices of HotPeerStat.
func (*HotPeerCache) UpdateStat ¶
func (f *HotPeerCache) UpdateStat(item *HotPeerStat)
UpdateStat updates the stat cache.
type HotPeerStat ¶
type HotPeerStat struct { StoreID uint64 `json:"store_id"` RegionID uint64 `json:"region_id"` // HotDegree records the times for the region considered as hot spot during each report. HotDegree int `json:"hot_degree"` // AntiCount used to eliminate some noise when remove region in cache. AntiCount int `json:"anti_count"` // Loads contains only Kind-related statistics and is DimLen in length. Loads []float64 `json:"loads"` // contains filtered or unexported fields }
HotPeerStat records each hot peer's statistics
func (*HotPeerStat) Clone ¶
func (stat *HotPeerStat) Clone() *HotPeerStat
Clone clones the HotPeerStat.
func (*HotPeerStat) GetActionType ¶
func (stat *HotPeerStat) GetActionType() utils.ActionType
GetActionType returns the item action type.
func (*HotPeerStat) GetLoad ¶
func (stat *HotPeerStat) GetLoad(dim int) float64
GetLoad returns denoising load if possible.
func (*HotPeerStat) GetLoads ¶
func (stat *HotPeerStat) GetLoads() []float64
GetLoads returns denoising loads if possible.
func (*HotPeerStat) GetStores ¶
func (stat *HotPeerStat) GetStores() []uint64
GetStores returns the stores of all peers in the region.
func (*HotPeerStat) ID ¶
func (stat *HotPeerStat) ID() uint64
ID returns region ID. Implementing TopNItem.
func (*HotPeerStat) IsLeader ¶
func (stat *HotPeerStat) IsLeader() bool
IsLeader indicates the item belong to the leader.
func (*HotPeerStat) IsNeedCoolDownTransferLeader ¶
func (stat *HotPeerStat) IsNeedCoolDownTransferLeader(minHotDegree int, rwTy utils.RWType) bool
IsNeedCoolDownTransferLeader use cooldown time after transfer leader to avoid unnecessary schedule
type HotPeerStatShow ¶
type HotPeerStatShow struct { StoreID uint64 `json:"store_id"` Stores []uint64 `json:"stores"` IsLeader bool `json:"is_leader"` IsLearner bool `json:"is_learner"` RegionID uint64 `json:"region_id"` HotDegree int `json:"hot_degree"` ByteRate float64 `json:"flow_bytes"` KeyRate float64 `json:"flow_keys"` QueryRate float64 `json:"flow_query"` AntiCount int `json:"anti_count"` LastUpdateTime time.Time `json:"last_update_time,omitempty"` }
HotPeerStatShow records the hot region statistics for output
type HotPeersStat ¶
type HotPeersStat struct { StoreByteRate float64 `json:"store_bytes"` StoreKeyRate float64 `json:"store_keys"` StoreQueryRate float64 `json:"store_query"` TotalBytesRate float64 `json:"total_flow_bytes"` TotalKeysRate float64 `json:"total_flow_keys"` TotalQueryRate float64 `json:"total_flow_query"` Count int `json:"regions_count"` Stats []HotPeerStatShow `json:"statistics"` }
HotPeersStat records all hot regions statistics
type HotStat ¶
type HotStat struct { *HotCache *StoresStats *buckets.HotBucketCache }
HotStat contains cluster's hotspot statistics.
func NewHotStat ¶
func NewHotStat(ctx context.Context, cluster *core.BasicCluster) *HotStat
NewHotStat creates the container to hold cluster's hotspot statistics.
type LabelStatistics ¶
LabelStatistics is the statistics of the level of labels.
func NewLabelStatistics ¶
func NewLabelStatistics() *LabelStatistics
NewLabelStatistics creates a new LabelStatistics.
func (*LabelStatistics) ClearDefunctRegions ¶
func (l *LabelStatistics) ClearDefunctRegions()
ClearDefunctRegions is used to handle the overlap region. It is used to remove the defunct regions from the label statistics.
func (*LabelStatistics) Collect ¶
func (l *LabelStatistics) Collect()
Collect collects the metrics of the label status.
func (*LabelStatistics) GetLabelCounter ¶
func (l *LabelStatistics) GetLabelCounter() map[string]int
GetLabelCounter is only used for tests.
func (*LabelStatistics) MarkDefunctRegion ¶
func (l *LabelStatistics) MarkDefunctRegion(regionID uint64)
MarkDefunctRegion is used to handle the overlap region. It is used to mark the region as defunct and remove it from the label statistics later.
func (*LabelStatistics) Observe ¶
func (l *LabelStatistics) Observe(region *core.RegionInfo, stores []*core.StoreInfo, labels []string)
Observe records the current label status.
type RegionInfoProvider ¶
type RegionInfoProvider interface { // GetRegion returns the region information according to the given region ID. GetRegion(regionID uint64) *core.RegionInfo }
RegionInfoProvider is an interface to provide the region information.
type RegionInfoWithTS ¶
type RegionInfoWithTS struct {
// contains filtered or unexported fields
}
RegionInfoWithTS is used to record the extra timestamp status of a region.
type RegionStatInformer ¶
type RegionStatInformer interface { GetHotPeerStat(rw utils.RWType, regionID, storeID uint64) *HotPeerStat IsRegionHot(region *core.RegionInfo) bool // GetHotPeerStats return the read or write statistics for hot regions. // It returns a map where the keys are store IDs and the values are slices of HotPeerStat. // The result only includes peers that are hot enough. GetHotPeerStats(rw utils.RWType) map[uint64][]*HotPeerStat }
RegionStatInformer provides access to a shared informer of statistics.
type RegionStatisticType ¶
type RegionStatisticType uint16
RegionStatisticType represents the type of the region's status.
const ( MissPeer RegionStatisticType = 1 << iota ExtraPeer DownPeer PendingPeer OfflinePeer LearnerPeer EmptyRegion OversizedRegion UndersizedRegion WitnessLeader )
region status type
type RegionStatistics ¶
RegionStatistics is used to record the status of regions.
func NewRegionStatistics ¶
func NewRegionStatistics( rip RegionInfoProvider, conf sc.CheckerConfigProvider, ruleManager *placement.RuleManager, ) *RegionStatistics
NewRegionStatistics creates a new RegionStatistics.
func (*RegionStatistics) ClearDefunctRegion ¶
func (r *RegionStatistics) ClearDefunctRegion(regionID uint64)
ClearDefunctRegion is used to handle the overlap region.
func (*RegionStatistics) Collect ¶
func (r *RegionStatistics) Collect()
Collect collects the metrics of the regions' status.
func (*RegionStatistics) GetRegionStatsByType ¶
func (r *RegionStatistics) GetRegionStatsByType(typ RegionStatisticType) []*core.RegionInfo
GetRegionStatsByType gets the status of the region by types. The regions here need to be cloned, otherwise, it may cause data race problems.
func (*RegionStatistics) IsRegionStatsType ¶
func (r *RegionStatistics) IsRegionStatsType(regionID uint64, typ RegionStatisticType) bool
IsRegionStatsType returns whether the status of the region is the given type.
func (*RegionStatistics) Observe ¶
func (r *RegionStatistics) Observe(region *core.RegionInfo, stores []*core.StoreInfo)
Observe records the current regions' status.
func (*RegionStatistics) RegionStatsNeedUpdate ¶
func (r *RegionStatistics) RegionStatsNeedUpdate(region *core.RegionInfo) bool
RegionStatsNeedUpdate checks whether the region's status need to be updated due to some special state types.
type RegionStats ¶
type RegionStats struct { Count int `json:"count"` EmptyCount int `json:"empty_count"` StorageSize int64 `json:"storage_size"` UserStorageSize int64 `json:"user_storage_size"` StorageKeys int64 `json:"storage_keys"` StoreLeaderCount map[uint64]int `json:"store_leader_count"` StorePeerCount map[uint64]int `json:"store_peer_count"` StoreLeaderSize map[uint64]int64 `json:"store_leader_size"` StoreLeaderKeys map[uint64]int64 `json:"store_leader_keys"` StorePeerSize map[uint64]int64 `json:"store_peer_size"` StorePeerKeys map[uint64]int64 `json:"store_peer_keys"` }
RegionStats records a list of regions' statistics and distribution status.
func GetRegionStats ¶
func GetRegionStats(regions []*core.RegionInfo) *RegionStats
GetRegionStats sums regions' statistics.
func (*RegionStats) Observe ¶
func (s *RegionStats) Observe(r *core.RegionInfo)
Observe adds a region's statistics into RegionStats.
type RollingStoreStats ¶
RollingStoreStats are multiple sets of recent historical records with specified windows size.
func (*RollingStoreStats) GetInstantLoad ¶
func (r *RollingStoreStats) GetInstantLoad(k utils.StoreStatKind) float64
GetInstantLoad returns store's instant load.
func (*RollingStoreStats) GetLoad ¶
func (r *RollingStoreStats) GetLoad(k utils.StoreStatKind) float64
GetLoad returns store's load.
func (*RollingStoreStats) Observe ¶
func (r *RollingStoreStats) Observe(stats *pdpb.StoreStats)
Observe records current statistics.
func (*RollingStoreStats) ObserveRegionsStats ¶
func (r *RollingStoreStats) ObserveRegionsStats(writeBytesRate, writeKeysRate float64)
ObserveRegionsStats records current statistics from region stats.
func (*RollingStoreStats) Set ¶
func (r *RollingStoreStats) Set(stats *pdpb.StoreStats)
Set sets the statistics (for test).
func (*RollingStoreStats) SetRegionsStats ¶
func (r *RollingStoreStats) SetRegionsStats(writeBytesRate, writeKeysRate float64)
SetRegionsStats sets the statistics from region stats (for test).
type SlowStat ¶
type SlowStat struct {
*SlowStoresStats
}
SlowStat contains cluster's slow nodes' statistics.
func NewSlowStat ¶
func NewSlowStat() *SlowStat
NewSlowStat creates the container to hold slow nodes' statistics.
type SlowStoresStats ¶
SlowStoresStats is a cached statistics for the slow store.
func NewSlowStoresStats ¶
func NewSlowStoresStats() *SlowStoresStats
NewSlowStoresStats creates a new slowStoresStats cache.
func (*SlowStoresStats) ExistsSlowStores ¶
func (s *SlowStoresStats) ExistsSlowStores() bool
ExistsSlowStores returns whether there exists slow stores in this cluster.
func (*SlowStoresStats) ObserveSlowStoreStatus ¶
func (s *SlowStoresStats) ObserveSlowStoreStatus(storeID uint64, isSlow bool)
ObserveSlowStoreStatus updates SlowStoreStats with a given store ID.
func (*SlowStoresStats) RemoveSlowStoreStatus ¶
func (s *SlowStoresStats) RemoveSlowStoreStatus(storeID uint64)
RemoveSlowStoreStatus removes SlowStoreStats with a given store ID.
type StoreHistoryLoads ¶
type StoreHistoryLoads struct {
// contains filtered or unexported fields
}
StoreHistoryLoads records the history load of a store.
func NewStoreHistoryLoads ¶
func NewStoreHistoryLoads(dim int, sampleDuration time.Duration, sampleInterval time.Duration) *StoreHistoryLoads
NewStoreHistoryLoads creates a StoreHistoryLoads.
func (*StoreHistoryLoads) Add ¶
func (s *StoreHistoryLoads) Add(storeID uint64, rwTp utils.RWType, kind constant.ResourceKind, pointLoad []float64)
Add adds the store load to the history.
func (*StoreHistoryLoads) Get ¶
func (s *StoreHistoryLoads) Get(storeID uint64, rwTp utils.RWType, kind constant.ResourceKind) [][]float64
Get returns the store loads from the history, not one time point. In another word, the result is [dim]time.
func (*StoreHistoryLoads) UpdateConfig ¶
func (s *StoreHistoryLoads) UpdateConfig(sampleDuration time.Duration, sampleInterval time.Duration) *StoreHistoryLoads
UpdateConfig updates the sample duration and interval.
type StoreHotPeersInfos ¶
type StoreHotPeersInfos struct { AsPeer StoreHotPeersStat `json:"as_peer"` AsLeader StoreHotPeersStat `json:"as_leader"` }
StoreHotPeersInfos is used to get human-readable description for hot regions. NOTE: This type is exported by HTTP API. Please pay more attention when modifying it.
func CollectHotPeerInfos ¶
func CollectHotPeerInfos(stores []*core.StoreInfo, regionStats map[uint64][]*HotPeerStat) *StoreHotPeersInfos
CollectHotPeerInfos only returns TotalBytesRate,TotalKeysRate,TotalQueryRate,Count
func GetHotStatus ¶
func GetHotStatus(stores []*core.StoreInfo, storesLoads map[uint64][]float64, regionStats map[uint64][]*HotPeerStat, typ utils.RWType, isTraceRegionFlow bool) *StoreHotPeersInfos
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`.
type StoreHotPeersStat ¶
type StoreHotPeersStat map[uint64]*HotPeersStat
StoreHotPeersStat is used to record the hot region statistics group by store. NOTE: This type is exported by HTTP API. Please pay more attention when modifying it.
type StoreLoad ¶
StoreLoad records the current load.
func (StoreLoad) ToLoadPred ¶
func (load StoreLoad) ToLoadPred(rwTy utils.RWType, infl *Influence) *StoreLoadPred
ToLoadPred returns the current load and future predictive load.
type StoreLoadDetail ¶
type StoreLoadDetail struct { *StoreSummaryInfo LoadPred *StoreLoadPred HotPeers []*HotPeerStat }
StoreLoadDetail records store load information.
func (*StoreLoadDetail) IsUniform ¶
func (li *StoreLoadDetail) IsUniform(dim int, threshold float64) bool
IsUniform returns true if the stores are uniform.
func (*StoreLoadDetail) ToHotPeersStat ¶
func (li *StoreLoadDetail) ToHotPeersStat() *HotPeersStat
ToHotPeersStat abstracts load information to HotPeersStat.
type StoreLoadPred ¶
StoreLoadPred is a prediction of a store.
func (*StoreLoadPred) Diff ¶
func (lp *StoreLoadPred) Diff() *StoreLoad
Diff return the difference between min and max.
func (*StoreLoadPred) Max ¶
func (lp *StoreLoadPred) Max() *StoreLoad
Max returns the max load between current and future.
func (*StoreLoadPred) Min ¶
func (lp *StoreLoadPred) Min() *StoreLoad
Min returns the min load between current and future.
func (*StoreLoadPred) Pending ¶
func (lp *StoreLoadPred) Pending() *StoreLoad
Pending returns the pending load.
type StoreStatInformer ¶
StoreStatInformer provides access to a shared informer of statistics.
type StoreSummaryInfo ¶
type StoreSummaryInfo struct { *core.StoreInfo PendingSum *Influence // contains filtered or unexported fields }
StoreSummaryInfo records the summary information of store.
func (*StoreSummaryInfo) AddInfluence ¶
func (s *StoreSummaryInfo) AddInfluence(infl *Influence, w float64)
AddInfluence adds influence to pending sum.
func (*StoreSummaryInfo) IsTiFlash ¶
func (s *StoreSummaryInfo) IsTiFlash() bool
IsTiFlash returns true if the store is TiFlash.
func (*StoreSummaryInfo) SetEngineAsTiFlash ¶
func (s *StoreSummaryInfo) SetEngineAsTiFlash()
SetEngineAsTiFlash set whether store is TiFlash, it is only used in tests.
type StoresStats ¶
StoresStats is a cache hold hot regions.
func NewStoresStats ¶
func NewStoresStats() *StoresStats
NewStoresStats creates a new hot spot cache.
func (*StoresStats) FilterUnhealthyStore ¶
func (s *StoresStats) FilterUnhealthyStore(cluster core.StoreSetInformer)
FilterUnhealthyStore filter unhealthy store
func (*StoresStats) GetOrCreateRollingStoreStats ¶
func (s *StoresStats) GetOrCreateRollingStoreStats(storeID uint64) *RollingStoreStats
GetOrCreateRollingStoreStats gets or creates RollingStoreStats with a given store ID.
func (*StoresStats) GetRollingStoreStats ¶
func (s *StoresStats) GetRollingStoreStats(storeID uint64) *RollingStoreStats
GetRollingStoreStats gets RollingStoreStats with a given store ID.
func (*StoresStats) GetStoresLoads ¶
func (s *StoresStats) GetStoresLoads() map[uint64][]float64
GetStoresLoads returns all stores loads.
func (*StoresStats) Observe ¶
func (s *StoresStats) Observe(storeID uint64, stats *pdpb.StoreStats)
Observe records the current store status with a given store.
func (*StoresStats) ObserveRegionsStats ¶
func (s *StoresStats) ObserveRegionsStats(storeIDs []uint64, writeBytesRates, writeKeysRates []float64)
ObserveRegionsStats records the current stores status from region stats.
func (*StoresStats) RemoveRollingStoreStats ¶
func (s *StoresStats) RemoveRollingStoreStats(storeID uint64)
RemoveRollingStoreStats removes RollingStoreStats with a given store ID.
func (*StoresStats) Set ¶
func (s *StoresStats) Set(storeID uint64, stats *pdpb.StoreStats)
Set sets the store statistics (for test).
func (*StoresStats) SetRegionsStats ¶
func (s *StoresStats) SetRegionsStats(storeIDs []uint64, writeBytesRates, writeKeysRates []float64)
SetRegionsStats sets the store statistics from region stats (for test).