core

package
v0.0.0-...-1338f1b Latest Latest
Warning

This package is not in the latest version of its module.

Go to latest
Published: Mar 15, 2024 License: Apache-2.0 Imports: 22 Imported by: 12

Documentation

Overview

Package core defines core characteristics of the server. This file uses the errcode packate to define PD specific error codes. Probably this should be a different package.

Index

Constants

View Source
const EmptyRegionApproximateSize = 1

EmptyRegionApproximateSize is the region approximate size of an empty region (heartbeat size <= 1MB).

Variables

View Source
var (

	// StoreBlockedCode is an error due to requesting an operation that is invalid due to a store being in a blocked state
	StoreBlockedCode = storeStateCode.Child("state.store.blocked")

	// StoreTombstonedCode is an invalid operation was attempted on a store which is in a removed state.
	StoreTombstonedCode = storeStateCode.Child("state.store.tombstoned").SetHTTP(http.StatusGone)
)

Functions

func DiffRegionKeyInfo

func DiffRegionKeyInfo(origin *RegionInfo, other *RegionInfo) string

DiffRegionKeyInfo returns the difference of key info between two RegionInfo

func DiffRegionPeersInfo

func DiffRegionPeersInfo(origin *RegionInfo, other *RegionInfo) string

DiffRegionPeersInfo returns the difference of peers info between two RegionInfo

func HexRegionKey

func HexRegionKey(key []byte) []byte

HexRegionKey converts region key to hex format. Used for formating region in logs.

func NewStoreNotFoundErr

func NewStoreNotFoundErr(storeID uint64) errcode.ErrorCode

NewStoreNotFoundErr is for log of store not found

Types

type BasicCluster

type BasicCluster struct {
	sync.RWMutex
	Stores  *StoresInfo
	Regions *RegionsInfo
}

BasicCluster provides basic data member and interface for a tikv cluster.

func NewBasicCluster

func NewBasicCluster() *BasicCluster

NewBasicCluster creates a BasicCluster.

func (*BasicCluster) AttachAvailableFunc

func (bc *BasicCluster) AttachAvailableFunc(storeID uint64, f func() bool)

AttachAvailableFunc attaches an available function to a specific store.

func (*BasicCluster) BlockStore

func (bc *BasicCluster) BlockStore(storeID uint64) error

BlockStore stops balancer from selecting the store.

func (*BasicCluster) DeleteStore

func (bc *BasicCluster) DeleteStore(store *StoreInfo)

DeleteStore deletes a store.

func (*BasicCluster) GetAverageRegionSize

func (bc *BasicCluster) GetAverageRegionSize() int64

GetAverageRegionSize returns the average region approximate size.

func (*BasicCluster) GetFollowerStores

func (bc *BasicCluster) GetFollowerStores(region *RegionInfo) []*StoreInfo

GetFollowerStores returns all Stores that contains the region's follower peer.

func (*BasicCluster) GetFollowersWithLock

func (bc *BasicCluster) GetFollowersWithLock(storeID uint64, callback func(RegionsContainer))

GetFollowersWithLock return leaders subtree by storeID

func (*BasicCluster) GetLeaderStore

func (bc *BasicCluster) GetLeaderStore(region *RegionInfo) *StoreInfo

GetLeaderStore returns all Stores that contains the region's leader peer.

func (*BasicCluster) GetLeadersWithLock

func (bc *BasicCluster) GetLeadersWithLock(storeID uint64, callback func(RegionsContainer))

GetLeadersWithLock return leaders subtree by storeID

func (*BasicCluster) GetMetaRegions

func (bc *BasicCluster) GetMetaRegions() []*metapb.Region

GetMetaRegions gets a set of metapb.Region from regionMap.

func (*BasicCluster) GetMetaStores

func (bc *BasicCluster) GetMetaStores() []*metapb.Store

GetMetaStores gets a complete set of metapb.Store.

func (*BasicCluster) GetOverlaps

func (bc *BasicCluster) GetOverlaps(region *RegionInfo) []*RegionInfo

GetOverlaps returns the regions which are overlapped with the specified region range.

func (*BasicCluster) GetPendingRegionsWithLock

func (bc *BasicCluster) GetPendingRegionsWithLock(storeID uint64, callback func(RegionsContainer))

GetPendingRegionsWithLock return pending regions subtree by storeID

func (*BasicCluster) GetRegion

func (bc *BasicCluster) GetRegion(regionID uint64) *RegionInfo

GetRegion searches for a region by ID.

func (*BasicCluster) GetRegionCount

func (bc *BasicCluster) GetRegionCount() int

GetRegionCount gets the total count of RegionInfo of regionMap.

func (*BasicCluster) GetRegionStores

func (bc *BasicCluster) GetRegionStores(region *RegionInfo) []*StoreInfo

GetRegionStores returns all Stores that contains the region's peer.

func (*BasicCluster) GetRegions

func (bc *BasicCluster) GetRegions() []*RegionInfo

GetRegions gets all RegionInfo from regionMap.

func (*BasicCluster) GetStore

func (bc *BasicCluster) GetStore(storeID uint64) *StoreInfo

GetStore searches for a store by ID.

func (*BasicCluster) GetStoreCount

func (bc *BasicCluster) GetStoreCount() int

GetStoreCount returns the total count of storeInfo.

func (*BasicCluster) GetStoreFollowerCount

func (bc *BasicCluster) GetStoreFollowerCount(storeID uint64) int

GetStoreFollowerCount get the total count of a store's follower RegionInfo.

func (*BasicCluster) GetStoreLeaderCount

func (bc *BasicCluster) GetStoreLeaderCount(storeID uint64) int

GetStoreLeaderCount get the total count of a store's leader RegionInfo.

func (*BasicCluster) GetStoreLeaderRegionSize

func (bc *BasicCluster) GetStoreLeaderRegionSize(storeID uint64) int64

GetStoreLeaderRegionSize get total size of store's leader regions.

func (*BasicCluster) GetStorePendingPeerCount

func (bc *BasicCluster) GetStorePendingPeerCount(storeID uint64) int

GetStorePendingPeerCount gets the total count of a store's region that includes pending peer.

func (*BasicCluster) GetStoreRegionCount

func (bc *BasicCluster) GetStoreRegionCount(storeID uint64) int

GetStoreRegionCount gets the total count of a store's leader and follower RegionInfo by storeID.

func (*BasicCluster) GetStoreRegionSize

func (bc *BasicCluster) GetStoreRegionSize(storeID uint64) int64

GetStoreRegionSize get total size of store's regions.

func (*BasicCluster) GetStoreRegions

func (bc *BasicCluster) GetStoreRegions(storeID uint64) []*RegionInfo

GetStoreRegions gets all RegionInfo with a given storeID.

func (*BasicCluster) GetStores

func (bc *BasicCluster) GetStores() []*StoreInfo

GetStores returns all Stores in the cluster.

func (*BasicCluster) Length

func (bc *BasicCluster) Length() int

Length returns the RegionsInfo length.

func (*BasicCluster) PutRegion

func (bc *BasicCluster) PutRegion(region *RegionInfo) []*RegionInfo

PutRegion put a region.

func (*BasicCluster) PutStore

func (bc *BasicCluster) PutStore(store *StoreInfo)

PutStore put a store.

func (*BasicCluster) RandFollowerRegion

func (bc *BasicCluster) RandFollowerRegion(storeID uint64, opts ...RegionOption) *RegionInfo

RandFollowerRegion returns a random region that has a follower on the store.

func (*BasicCluster) RandLeaderRegion

func (bc *BasicCluster) RandLeaderRegion(storeID uint64, opts ...RegionOption) *RegionInfo

RandLeaderRegion returns a random region that has leader on the store.

func (*BasicCluster) RandPendingRegion

func (bc *BasicCluster) RandPendingRegion(storeID uint64, opts ...RegionOption) *RegionInfo

RandPendingRegion returns a random region that has a pending peer on the store.

func (*BasicCluster) RemoveRegion

func (bc *BasicCluster) RemoveRegion(region *RegionInfo)

RemoveRegion removes RegionInfo from regionTree and regionMap.

func (*BasicCluster) ScanRange

func (bc *BasicCluster) ScanRange(startKey, endKey []byte, limit int) []*RegionInfo

ScanRange scans regions intersecting [start key, end key), returns at most `limit` regions. limit <= 0 means no limit.

func (*BasicCluster) SearchPrevRegion

func (bc *BasicCluster) SearchPrevRegion(regionKey []byte) *RegionInfo

SearchPrevRegion searches previous RegionInfo from regionTree.

func (*BasicCluster) SearchRegion

func (bc *BasicCluster) SearchRegion(regionKey []byte) *RegionInfo

SearchRegion searches RegionInfo from regionTree.

func (*BasicCluster) TakeStore

func (bc *BasicCluster) TakeStore(storeID uint64) *StoreInfo

TakeStore returns the point of the origin StoreInfo with the specified storeID.

func (*BasicCluster) UnblockStore

func (bc *BasicCluster) UnblockStore(storeID uint64)

UnblockStore allows balancer to select the store.

func (*BasicCluster) UpdateStoreStatus

func (bc *BasicCluster) UpdateStoreStatus(storeID uint64, leaderCount int, regionCount int, pendingPeerCount int, leaderSize int64, regionSize int64)

UpdateStoreStatus updates the information of the store.

type HexRegionMeta

type HexRegionMeta struct {
	*metapb.Region
}

HexRegionMeta is a region meta in the hex format. Used for formating region in logs.

func RegionToHexMeta

func RegionToHexMeta(meta *metapb.Region) HexRegionMeta

RegionToHexMeta converts a region meta's keys to hex format. Used for formating region in logs.

func (HexRegionMeta) String

func (h HexRegionMeta) String() string

type HexRegionsMeta

type HexRegionsMeta []*metapb.Region

HexRegionsMeta is a slice of regions' meta in the hex format. Used for formating region in logs.

func RegionsToHexMeta

func RegionsToHexMeta(regions []*metapb.Region) HexRegionsMeta

RegionsToHexMeta converts regions' meta keys to hex format. Used for formating region in logs.

func (HexRegionsMeta) String

func (h HexRegionsMeta) String() string

type PriorityLevel

type PriorityLevel int

PriorityLevel lower level means higher priority

const (
	LowPriority PriorityLevel = iota
	NormalPriority
	HighPriority
)

Built-in priority level

type RegionCreateOption

type RegionCreateOption func(region *RegionInfo)

RegionCreateOption used to create region.

func SetApproximateSize

func SetApproximateSize(v int64) RegionCreateOption

SetApproximateSize sets the approximate size for the region.

func SetPeers

func SetPeers(peers []*metapb.Peer) RegionCreateOption

SetPeers sets the peers for the region.

func WithAddPeer

func WithAddPeer(peer *metapb.Peer) RegionCreateOption

WithAddPeer adds a peer for the region.

func WithEndKey

func WithEndKey(key []byte) RegionCreateOption

WithEndKey sets the end key for the region.

func WithIncConfVer

func WithIncConfVer() RegionCreateOption

WithIncConfVer increases the config version of the region.

func WithIncVersion

func WithIncVersion() RegionCreateOption

WithIncVersion increases the version of the region.

func WithLeader

func WithLeader(leader *metapb.Peer) RegionCreateOption

WithLeader sets the leader for the region.

func WithLearners

func WithLearners(learner []*metapb.Peer) RegionCreateOption

WithLearners adds learner to the region

func WithPendingPeers

func WithPendingPeers(pengdingPeers []*metapb.Peer) RegionCreateOption

WithPendingPeers sets the pending peers for the region.

func WithRemoveStorePeer

func WithRemoveStorePeer(storeID uint64) RegionCreateOption

WithRemoveStorePeer removes the specified peer for the region.

func WithStartKey

func WithStartKey(key []byte) RegionCreateOption

WithStartKey sets the start key for the region.

type RegionInfo

type RegionInfo struct {
	// contains filtered or unexported fields
}

RegionInfo records detail region info. Read-Only once created.

func MergeRegions

func MergeRegions(regions []*RegionInfo) []*RegionInfo

MergeRegions merge a set of RegionInfo by regionKey

func NewRegionInfo

func NewRegionInfo(region *metapb.Region, leader *metapb.Peer, opts ...RegionCreateOption) *RegionInfo

NewRegionInfo creates RegionInfo with region's meta and leader peer.

func NewTestRegionInfo

func NewTestRegionInfo(start, end []byte) *RegionInfo

NewTestRegionInfo creates a RegionInfo for test.

func RegionFromHeartbeat

func RegionFromHeartbeat(heartbeat *schedulerpb.RegionHeartbeatRequest) *RegionInfo

RegionFromHeartbeat constructs a Region from region heartbeat.

func SplitRegions

func SplitRegions(regions []*RegionInfo) []*RegionInfo

SplitRegions split a set of RegionInfo by the middle of regionKey

func (*RegionInfo) Clone

func (r *RegionInfo) Clone(opts ...RegionCreateOption) *RegionInfo

Clone returns a copy of current regionInfo.

func (*RegionInfo) GetApproximateSize

func (r *RegionInfo) GetApproximateSize() int64

GetApproximateSize returns the approximate size of the region.

func (*RegionInfo) GetDiffFollowers

func (r *RegionInfo) GetDiffFollowers(other *RegionInfo) []*metapb.Peer

GetDiffFollowers returns the followers which is not located in the same store as any other followers of the another specified region.

func (*RegionInfo) GetDownLearner

func (r *RegionInfo) GetDownLearner(peerID uint64) *metapb.Peer

GetDownLearner returns the down learner with soecified peer id.

func (*RegionInfo) GetEndKey

func (r *RegionInfo) GetEndKey() []byte

GetEndKey returns the end key of the region.

func (*RegionInfo) GetFollower

func (r *RegionInfo) GetFollower() *metapb.Peer

GetFollower randomly returns a follow peer.

func (*RegionInfo) GetFollowers

func (r *RegionInfo) GetFollowers() map[uint64]*metapb.Peer

GetFollowers returns a map indicate the follow peers distributed.

func (*RegionInfo) GetID

func (r *RegionInfo) GetID() uint64

GetID returns the ID of the region.

func (*RegionInfo) GetLeader

func (r *RegionInfo) GetLeader() *metapb.Peer

GetLeader returns the leader of the region.

func (*RegionInfo) GetLearners

func (r *RegionInfo) GetLearners() []*metapb.Peer

GetLearners returns the learners.

func (*RegionInfo) GetMeta

func (r *RegionInfo) GetMeta() *metapb.Region

GetMeta returns the meta information of the region.

func (*RegionInfo) GetPeer

func (r *RegionInfo) GetPeer(peerID uint64) *metapb.Peer

GetPeer returns the peer with specified peer id.

func (*RegionInfo) GetPeers

func (r *RegionInfo) GetPeers() []*metapb.Peer

GetPeers returns the peers of the region.

func (*RegionInfo) GetPendingLearner

func (r *RegionInfo) GetPendingLearner(peerID uint64) *metapb.Peer

GetPendingLearner returns the pending learner peer with specified peer id.

func (*RegionInfo) GetPendingPeer

func (r *RegionInfo) GetPendingPeer(peerID uint64) *metapb.Peer

GetPendingPeer returns the pending peer with specified peer id.

func (*RegionInfo) GetPendingPeers

func (r *RegionInfo) GetPendingPeers() []*metapb.Peer

GetPendingPeers returns the pending peers of the region.

func (*RegionInfo) GetPendingVoter

func (r *RegionInfo) GetPendingVoter(peerID uint64) *metapb.Peer

GetPendingVoter returns the pending voter with specified peer id.

func (*RegionInfo) GetRegionEpoch

func (r *RegionInfo) GetRegionEpoch() *metapb.RegionEpoch

GetRegionEpoch returns the region epoch of the region.

func (*RegionInfo) GetStartKey

func (r *RegionInfo) GetStartKey() []byte

GetStartKey returns the start key of the region.

func (*RegionInfo) GetStoreIds

func (r *RegionInfo) GetStoreIds() map[uint64]struct{}

GetStoreIds returns a map indicate the region distributed.

func (*RegionInfo) GetStoreLearner

func (r *RegionInfo) GetStoreLearner(storeID uint64) *metapb.Peer

GetStoreLearner returns the learner peer in specified store.

func (*RegionInfo) GetStorePeer

func (r *RegionInfo) GetStorePeer(storeID uint64) *metapb.Peer

GetStorePeer returns the peer in specified store.

func (*RegionInfo) GetStoreVoter

func (r *RegionInfo) GetStoreVoter(storeID uint64) *metapb.Peer

GetStoreVoter returns the voter in specified store.

func (*RegionInfo) GetVoters

func (r *RegionInfo) GetVoters() []*metapb.Peer

GetVoters returns the voters.

type RegionOption

type RegionOption func(region *RegionInfo) bool

RegionOption is used to select region.

func HealthRegion

func HealthRegion() RegionOption

HealthRegion checks if the region is healthy.

func HealthRegionAllowPending

func HealthRegionAllowPending() RegionOption

HealthRegionAllowPending checks if the region is healthy with allowing the pending peer.

type RegionSetInformer

type RegionSetInformer interface {
	RandFollowerRegion(storeID uint64, opts ...RegionOption) *RegionInfo
	RandLeaderRegion(storeID uint64, opts ...RegionOption) *RegionInfo
	RandPendingRegion(storeID uint64, opts ...RegionOption) *RegionInfo
	GetPendingRegionsWithLock(storeID uint64, callback func(RegionsContainer))
	GetLeadersWithLock(storeID uint64, callback func(RegionsContainer))
	GetFollowersWithLock(storeID uint64, callback func(RegionsContainer))
	GetAverageRegionSize() int64
	GetStoreRegionCount(storeID uint64) int
	GetRegion(id uint64) *RegionInfo
	ScanRegions(startKey, endKey []byte, limit int) []*RegionInfo
}

RegionSetInformer provides access to a shared informer of regions.

type RegionsContainer

type RegionsContainer interface {
	RandomRegion(startKey, endKey []byte) *RegionInfo
}

RegionsContainer is a container to store regions.

type RegionsInfo

type RegionsInfo struct {
	// contains filtered or unexported fields
}

RegionsInfo for export

func NewRegionsInfo

func NewRegionsInfo() *RegionsInfo

NewRegionsInfo creates RegionsInfo with tree, regions, leaders and followers

func (*RegionsInfo) AddRegion

func (r *RegionsInfo) AddRegion(region *RegionInfo) []*RegionInfo

AddRegion adds RegionInfo to regionTree and regionMap, also update leaders and followers by region peers

func (*RegionsInfo) GetAverageRegionSize

func (r *RegionsInfo) GetAverageRegionSize() int64

GetAverageRegionSize returns the average region approximate size.

func (*RegionsInfo) GetFollower

func (r *RegionsInfo) GetFollower(storeID uint64, region *RegionInfo) *RegionInfo

GetFollower returns follower RegionInfo by storeID and regionID(now only used in test)

func (*RegionsInfo) GetFollowersWithLock

func (r *RegionsInfo) GetFollowersWithLock(storeID uint64, callback func(RegionsContainer))

GetFollowersWithLock returns followers subtree by storeID

func (*RegionsInfo) GetLeader

func (r *RegionsInfo) GetLeader(storeID uint64, region *RegionInfo) *RegionInfo

GetLeader returns leader RegionInfo by storeID and regionID(now only used in test)

func (*RegionsInfo) GetLeadersWithLock

func (r *RegionsInfo) GetLeadersWithLock(storeID uint64, callback func(RegionsContainer))

GetLeadersWithLock returns leaders subtree by storeID

func (*RegionsInfo) GetMetaRegions

func (r *RegionsInfo) GetMetaRegions() []*metapb.Region

GetMetaRegions gets a set of metapb.Region from regionMap

func (*RegionsInfo) GetOverlaps

func (r *RegionsInfo) GetOverlaps(region *RegionInfo) []*RegionInfo

GetOverlaps returns the regions which are overlapped with the specified region range.

func (*RegionsInfo) GetPendingRegionsWithLock

func (r *RegionsInfo) GetPendingRegionsWithLock(storeID uint64, callback func(RegionsContainer))

GetPendingRegionsWithLock returns pending regions subtree by storeID

func (*RegionsInfo) GetRegion

func (r *RegionsInfo) GetRegion(regionID uint64) *RegionInfo

GetRegion returns the RegionInfo with regionID

func (*RegionsInfo) GetRegionCount

func (r *RegionsInfo) GetRegionCount() int

GetRegionCount gets the total count of RegionInfo of regionMap

func (*RegionsInfo) GetRegions

func (r *RegionsInfo) GetRegions() []*RegionInfo

GetRegions gets all RegionInfo from regionMap

func (*RegionsInfo) GetStoreFollowerCount

func (r *RegionsInfo) GetStoreFollowerCount(storeID uint64) int

GetStoreFollowerCount gets the total count of a store's follower RegionInfo

func (*RegionsInfo) GetStoreFollowerRegionSize

func (r *RegionsInfo) GetStoreFollowerRegionSize(storeID uint64) int64

GetStoreFollowerRegionSize gets total size of store's follower regions

func (*RegionsInfo) GetStoreLeaderCount

func (r *RegionsInfo) GetStoreLeaderCount(storeID uint64) int

GetStoreLeaderCount gets the total count of a store's leader RegionInfo

func (*RegionsInfo) GetStoreLeaderRegionSize

func (r *RegionsInfo) GetStoreLeaderRegionSize(storeID uint64) int64

GetStoreLeaderRegionSize gets total size of store's leader regions

func (*RegionsInfo) GetStoreLearnerCount

func (r *RegionsInfo) GetStoreLearnerCount(storeID uint64) int

GetStoreLearnerCount gets the total count of a store's learner RegionInfo

func (*RegionsInfo) GetStoreLearnerRegionSize

func (r *RegionsInfo) GetStoreLearnerRegionSize(storeID uint64) int64

GetStoreLearnerRegionSize gets total size of store's learner regions

func (*RegionsInfo) GetStorePendingPeerCount

func (r *RegionsInfo) GetStorePendingPeerCount(storeID uint64) int

GetStorePendingPeerCount gets the total count of a store's region that includes pending peer

func (*RegionsInfo) GetStoreRegionCount

func (r *RegionsInfo) GetStoreRegionCount(storeID uint64) int

GetStoreRegionCount gets the total count of a store's leader and follower RegionInfo by storeID

func (*RegionsInfo) GetStoreRegionSize

func (r *RegionsInfo) GetStoreRegionSize(storeID uint64) int64

GetStoreRegionSize gets total size of store's regions

func (*RegionsInfo) GetStoreRegions

func (r *RegionsInfo) GetStoreRegions(storeID uint64) []*RegionInfo

GetStoreRegions gets all RegionInfo with a given storeID

func (*RegionsInfo) Length

func (r *RegionsInfo) Length() int

Length returns the RegionsInfo length

func (*RegionsInfo) RandFollowerRegion

func (r *RegionsInfo) RandFollowerRegion(storeID uint64, opts ...RegionOption) *RegionInfo

RandFollowerRegion randomly gets a store's follower region.

func (*RegionsInfo) RandLeaderRegion

func (r *RegionsInfo) RandLeaderRegion(storeID uint64, opts ...RegionOption) *RegionInfo

RandLeaderRegion randomly gets a store's leader region.

func (*RegionsInfo) RandPendingRegion

func (r *RegionsInfo) RandPendingRegion(storeID uint64, opts ...RegionOption) *RegionInfo

RandPendingRegion randomly gets a store's region with a pending peer.

func (*RegionsInfo) RandRegion

func (r *RegionsInfo) RandRegion(opts ...RegionOption) *RegionInfo

RandRegion gets a region by random

func (*RegionsInfo) RemoveRegion

func (r *RegionsInfo) RemoveRegion(region *RegionInfo)

RemoveRegion removes RegionInfo from regionTree and regionMap

func (*RegionsInfo) ScanRange

func (r *RegionsInfo) ScanRange(startKey, endKey []byte, limit int) []*RegionInfo

ScanRange scans regions intersecting [start key, end key), returns at most `limit` regions. limit <= 0 means no limit.

func (*RegionsInfo) SearchPrevRegion

func (r *RegionsInfo) SearchPrevRegion(regionKey []byte) *RegionInfo

SearchPrevRegion searches previous RegionInfo from regionTree

func (*RegionsInfo) SearchRegion

func (r *RegionsInfo) SearchRegion(regionKey []byte) *RegionInfo

SearchRegion searches RegionInfo from regionTree

func (*RegionsInfo) SetRegion

func (r *RegionsInfo) SetRegion(region *RegionInfo) []*RegionInfo

SetRegion sets the RegionInfo with regionID

func (*RegionsInfo) TreeLength

func (r *RegionsInfo) TreeLength() int

TreeLength returns the RegionsInfo tree length(now only used in test)

type ResourceKind

type ResourceKind int

ResourceKind distinguishes different kinds of resources.

const (
	// LeaderKind indicates the leader kind resource
	LeaderKind ResourceKind = iota
	// RegionKind indicates the region kind resource
	RegionKind
)

func (ResourceKind) String

func (k ResourceKind) String() string

type ScheduleKind

type ScheduleKind struct {
	Resource ResourceKind
}

ScheduleKind distinguishes resources and schedule strategy.

func NewScheduleKind

func NewScheduleKind(Resource ResourceKind) ScheduleKind

NewScheduleKind creates a schedule kind with resource kind and schedule strategy.

type Storage

type Storage struct {
	kv.Base
}

Storage wraps all kv operations, keep it stateless.

func NewStorage

func NewStorage(base kv.Base) *Storage

NewStorage creates Storage instance with Base.

func (*Storage) Close

func (s *Storage) Close() error

Close closes the s.

func (*Storage) ClusterStatePath

func (s *Storage) ClusterStatePath(option string) string

ClusterStatePath returns the path to save an option.

func (*Storage) DeleteStore

func (s *Storage) DeleteStore(store *metapb.Store) error

DeleteStore deletes one store from storage.

func (*Storage) Flush

func (s *Storage) Flush() error

Flush flushes the dirty region to storage.

func (*Storage) LoadAllScheduleConfig

func (s *Storage) LoadAllScheduleConfig() ([]string, []string, error)

LoadAllScheduleConfig loads all schedulers' config.

func (*Storage) LoadGCSafePoint

func (s *Storage) LoadGCSafePoint() (uint64, error)

LoadGCSafePoint loads current GC safe point from storage.

func (*Storage) LoadMeta

func (s *Storage) LoadMeta(meta *metapb.Cluster) (bool, error)

LoadMeta loads cluster meta from storage.

func (*Storage) LoadScheduleConfig

func (s *Storage) LoadScheduleConfig(scheduleName string) (string, error)

LoadScheduleConfig loads the config of scheduler.

func (*Storage) LoadStore

func (s *Storage) LoadStore(storeID uint64, store *metapb.Store) (bool, error)

LoadStore loads one store from storage.

func (*Storage) LoadStores

func (s *Storage) LoadStores(f func(store *StoreInfo)) error

LoadStores loads all stores from storage to StoresInfo.

func (*Storage) RemoveScheduleConfig

func (s *Storage) RemoveScheduleConfig(scheduleName string) error

RemoveScheduleConfig remvoes the config of scheduler.

func (*Storage) SaveGCSafePoint

func (s *Storage) SaveGCSafePoint(safePoint uint64) error

SaveGCSafePoint saves new GC safe point to storage.

func (*Storage) SaveMeta

func (s *Storage) SaveMeta(meta *metapb.Cluster) error

SaveMeta save cluster meta to storage.

func (*Storage) SaveScheduleConfig

func (s *Storage) SaveScheduleConfig(scheduleName string, data []byte) error

SaveScheduleConfig saves the config of scheduler.

func (*Storage) SaveStore

func (s *Storage) SaveStore(store *metapb.Store) error

SaveStore saves one store to storage.

func (*Storage) SaveStoreWeight

func (s *Storage) SaveStoreWeight(storeID uint64, leader, region float64) error

SaveStoreWeight saves a store's leader and region weight to storage.

type StoreBlockedErr

type StoreBlockedErr StoreErr

StoreBlockedErr has a Code() of StoreBlockedCode

func (StoreBlockedErr) Code

func (e StoreBlockedErr) Code() errcode.Code

Code returns StoreBlockedCode

func (StoreBlockedErr) Error

func (e StoreBlockedErr) Error() string

type StoreCreateOption

type StoreCreateOption func(region *StoreInfo)

StoreCreateOption is used to create store.

func SetAvailableFunc

func SetAvailableFunc(f func() bool) StoreCreateOption

SetAvailableFunc sets a customize function for the store. The function f returns true if the store limit is not exceeded.

func SetLastHeartbeatTS

func SetLastHeartbeatTS(lastHeartbeatTS time.Time) StoreCreateOption

SetLastHeartbeatTS sets the time of last heartbeat for the store.

func SetLeaderCount

func SetLeaderCount(leaderCount int) StoreCreateOption

SetLeaderCount sets the leader count for the store.

func SetLeaderSize

func SetLeaderSize(leaderSize int64) StoreCreateOption

SetLeaderSize sets the leader size for the store.

func SetLeaderWeight

func SetLeaderWeight(leaderWeight float64) StoreCreateOption

SetLeaderWeight sets the leader weight for the store.

func SetPendingPeerCount

func SetPendingPeerCount(pendingPeerCount int) StoreCreateOption

SetPendingPeerCount sets the pending peer count for the store.

func SetRegionCount

func SetRegionCount(regionCount int) StoreCreateOption

SetRegionCount sets the Region count for the store.

func SetRegionSize

func SetRegionSize(regionSize int64) StoreCreateOption

SetRegionSize sets the Region size for the store.

func SetRegionWeight

func SetRegionWeight(regionWeight float64) StoreCreateOption

SetRegionWeight sets the Region weight for the store.

func SetStoreAddress

func SetStoreAddress(address string) StoreCreateOption

SetStoreAddress sets the address for the store.

func SetStoreBlock

func SetStoreBlock() StoreCreateOption

SetStoreBlock stops balancer from selecting the store.

func SetStoreState

func SetStoreState(state metapb.StoreState) StoreCreateOption

SetStoreState sets the state for the store.

func SetStoreStats

func SetStoreStats(stats *schedulerpb.StoreStats) StoreCreateOption

SetStoreStats sets the statistics information for the store.

func SetStoreUnBlock

func SetStoreUnBlock() StoreCreateOption

SetStoreUnBlock allows balancer to select the store.

type StoreErr

type StoreErr struct {
	StoreID uint64 `json:"storeId"`
}

StoreErr can be newtyped or embedded in your own error

type StoreInfo

type StoreInfo struct {
	// contains filtered or unexported fields
}

StoreInfo contains information about a store.

func NewStoreInfo

func NewStoreInfo(store *metapb.Store, opts ...StoreCreateOption) *StoreInfo

NewStoreInfo creates StoreInfo with meta data.

func NewStoreInfoWithIdAndCount

func NewStoreInfoWithIdAndCount(id uint64, regionCount int) *StoreInfo

NewStoreInfoWithIdAndCount is create a store with specified id and regionCount.

func NewStoreInfoWithSizeCount

func NewStoreInfoWithSizeCount(id uint64, regionCount, leaderCount int, regionSize, leaderSize int64) *StoreInfo

NewStoreInfoWithSizeCount is create a store with size and count.

func (*StoreInfo) AvailableRatio

func (s *StoreInfo) AvailableRatio() float64

AvailableRatio is store's freeSpace/capacity.

func (*StoreInfo) Clone

func (s *StoreInfo) Clone(opts ...StoreCreateOption) *StoreInfo

Clone creates a copy of current StoreInfo.

func (*StoreInfo) DownTime

func (s *StoreInfo) DownTime() time.Duration

DownTime returns the time elapsed since last heartbeat.

func (*StoreInfo) GetAddress

func (s *StoreInfo) GetAddress() string

GetAddress returns the address of the store.

func (*StoreInfo) GetApplyingSnapCount

func (s *StoreInfo) GetApplyingSnapCount() uint32

GetApplyingSnapCount returns the current applying snapshot count of the store.

func (*StoreInfo) GetAvailable

func (s *StoreInfo) GetAvailable() uint64

GetAvailable returns the available size of the store.

func (*StoreInfo) GetCapacity

func (s *StoreInfo) GetCapacity() uint64

GetCapacity returns the capacity size of the store.

func (*StoreInfo) GetID

func (s *StoreInfo) GetID() uint64

GetID returns the ID of the store.

func (*StoreInfo) GetLastHeartbeatTS

func (s *StoreInfo) GetLastHeartbeatTS() time.Time

GetLastHeartbeatTS returns the last heartbeat timestamp of the store.

func (*StoreInfo) GetLeaderCount

func (s *StoreInfo) GetLeaderCount() int

GetLeaderCount returns the leader count of the store.

func (*StoreInfo) GetLeaderSize

func (s *StoreInfo) GetLeaderSize() int64

GetLeaderSize returns the leader size of the store.

func (*StoreInfo) GetLeaderWeight

func (s *StoreInfo) GetLeaderWeight() float64

GetLeaderWeight returns the leader weight of the store.

func (*StoreInfo) GetMeta

func (s *StoreInfo) GetMeta() *metapb.Store

GetMeta returns the meta information of the store.

func (*StoreInfo) GetPendingPeerCount

func (s *StoreInfo) GetPendingPeerCount() int

GetPendingPeerCount returns the pending peer count of the store.

func (*StoreInfo) GetReceivingSnapCount

func (s *StoreInfo) GetReceivingSnapCount() uint32

GetReceivingSnapCount returns the current receiving snapshot count of the store.

func (*StoreInfo) GetRegionCount

func (s *StoreInfo) GetRegionCount() int

GetRegionCount returns the Region count of the store.

func (*StoreInfo) GetRegionSize

func (s *StoreInfo) GetRegionSize() int64

GetRegionSize returns the Region size of the store.

func (*StoreInfo) GetRegionWeight

func (s *StoreInfo) GetRegionWeight() float64

GetRegionWeight returns the Region weight of the store.

func (*StoreInfo) GetSendingSnapCount

func (s *StoreInfo) GetSendingSnapCount() uint32

GetSendingSnapCount returns the current sending snapshot count of the store.

func (*StoreInfo) GetStartTS

func (s *StoreInfo) GetStartTS() time.Time

GetStartTS returns the start timestamp.

func (*StoreInfo) GetStartTime

func (s *StoreInfo) GetStartTime() uint32

GetStartTime returns the start time of the store.

func (*StoreInfo) GetState

func (s *StoreInfo) GetState() metapb.StoreState

GetState returns the state of the store.

func (*StoreInfo) GetStoreStats

func (s *StoreInfo) GetStoreStats() *schedulerpb.StoreStats

GetStoreStats returns the statistics information of the store.

func (*StoreInfo) GetUptime

func (s *StoreInfo) GetUptime() time.Duration

GetUptime returns the uptime.

func (*StoreInfo) GetUsedSize

func (s *StoreInfo) GetUsedSize() uint64

GetUsedSize returns the used size of the store.

func (*StoreInfo) IsAvailable

func (s *StoreInfo) IsAvailable() bool

IsAvailable returns if the store bucket of limitation is available

func (*StoreInfo) IsBlocked

func (s *StoreInfo) IsBlocked() bool

IsBlocked returns if the store is blocked.

func (*StoreInfo) IsBusy

func (s *StoreInfo) IsBusy() bool

IsBusy returns if the store is busy.

func (*StoreInfo) IsDisconnected

func (s *StoreInfo) IsDisconnected() bool

IsDisconnected checks if a store is disconnected, which means PD misses tikv's store heartbeat for a short time, maybe caused by process restart or temporary network failure.

func (*StoreInfo) IsLowSpace

func (s *StoreInfo) IsLowSpace(lowSpaceRatio float64) bool

IsLowSpace checks if the store is lack of space.

func (*StoreInfo) IsOffline

func (s *StoreInfo) IsOffline() bool

IsOffline checks if the store's state is Offline.

func (*StoreInfo) IsTombstone

func (s *StoreInfo) IsTombstone() bool

IsTombstone checks if the store's state is Tombstone.

func (*StoreInfo) IsUnhealth

func (s *StoreInfo) IsUnhealth() bool

IsUnhealth checks if a store is unhealth.

func (*StoreInfo) IsUp

func (s *StoreInfo) IsUp() bool

IsUp checks if the store's state is Up.

func (*StoreInfo) ResourceCount

func (s *StoreInfo) ResourceCount(kind ResourceKind) uint64

ResourceCount returns count of leader/region in the store.

func (*StoreInfo) ResourceSize

func (s *StoreInfo) ResourceSize(kind ResourceKind) int64

ResourceSize returns size of leader/region in the store

func (*StoreInfo) ResourceWeight

func (s *StoreInfo) ResourceWeight(kind ResourceKind) float64

ResourceWeight returns weight of leader/region in the score

func (*StoreInfo) StorageSize

func (s *StoreInfo) StorageSize() uint64

StorageSize returns store's used storage size reported from tikv.

type StoreSetController

type StoreSetController interface {
	BlockStore(id uint64) error
	UnblockStore(id uint64)

	AttachAvailableFunc(id uint64, f func() bool)
}

StoreSetController is used to control stores' status.

type StoreSetInformer

type StoreSetInformer interface {
	GetStores() []*StoreInfo
	GetStore(id uint64) *StoreInfo

	GetRegionStores(region *RegionInfo) []*StoreInfo
	GetFollowerStores(region *RegionInfo) []*StoreInfo
	GetLeaderStore(region *RegionInfo) *StoreInfo
}

StoreSetInformer provides access to a shared informer of stores.

type StoreTombstonedErr

type StoreTombstonedErr StoreErr

StoreTombstonedErr is an invalid operation was attempted on a store which is in a removed state.

func (StoreTombstonedErr) Code

func (e StoreTombstonedErr) Code() errcode.Code

Code returns StoreTombstonedCode

func (StoreTombstonedErr) Error

func (e StoreTombstonedErr) Error() string

type StoresInfo

type StoresInfo struct {
	// contains filtered or unexported fields
}

StoresInfo contains information about all stores.

func NewStoresInfo

func NewStoresInfo() *StoresInfo

NewStoresInfo create a StoresInfo with map of storeID to StoreInfo

func (*StoresInfo) AttachAvailableFunc

func (s *StoresInfo) AttachAvailableFunc(storeID uint64, f func() bool)

AttachAvailableFunc attaches f to a specific store.

func (*StoresInfo) BlockStore

func (s *StoresInfo) BlockStore(storeID uint64) errcode.ErrorCode

BlockStore blocks a StoreInfo with storeID.

func (*StoresInfo) DeleteStore

func (s *StoresInfo) DeleteStore(store *StoreInfo)

DeleteStore deletes tombstone record form store

func (*StoresInfo) GetMetaStores

func (s *StoresInfo) GetMetaStores() []*metapb.Store

GetMetaStores gets a complete set of metapb.Store.

func (*StoresInfo) GetStore

func (s *StoresInfo) GetStore(storeID uint64) *StoreInfo

GetStore returns a copy of the StoreInfo with the specified storeID.

func (*StoresInfo) GetStoreCount

func (s *StoresInfo) GetStoreCount() int

GetStoreCount returns the total count of storeInfo.

func (*StoresInfo) GetStores

func (s *StoresInfo) GetStores() []*StoreInfo

GetStores gets a complete set of StoreInfo.

func (*StoresInfo) SetLeaderCount

func (s *StoresInfo) SetLeaderCount(storeID uint64, leaderCount int)

SetLeaderCount sets the leader count to a storeInfo.

func (*StoresInfo) SetLeaderSize

func (s *StoresInfo) SetLeaderSize(storeID uint64, leaderSize int64)

SetLeaderSize sets the leader size to a storeInfo.

func (*StoresInfo) SetPendingPeerCount

func (s *StoresInfo) SetPendingPeerCount(storeID uint64, pendingPeerCount int)

SetPendingPeerCount sets the pending count to a storeInfo.

func (*StoresInfo) SetRegionCount

func (s *StoresInfo) SetRegionCount(storeID uint64, regionCount int)

SetRegionCount sets the region count to a storeInfo.

func (*StoresInfo) SetRegionSize

func (s *StoresInfo) SetRegionSize(storeID uint64, regionSize int64)

SetRegionSize sets the region size to a storeInfo.

func (*StoresInfo) SetStore

func (s *StoresInfo) SetStore(store *StoreInfo)

SetStore sets a StoreInfo with storeID.

func (*StoresInfo) TakeStore

func (s *StoresInfo) TakeStore(storeID uint64) *StoreInfo

TakeStore returns the point of the origin StoreInfo with the specified storeID.

func (*StoresInfo) UnblockStore

func (s *StoresInfo) UnblockStore(storeID uint64)

UnblockStore unblocks a StoreInfo with storeID.

func (*StoresInfo) UpdateStoreStatus

func (s *StoresInfo) UpdateStoreStatus(storeID uint64, leaderCount int, regionCount int, pendingPeerCount int, leaderSize int64, regionSize int64)

UpdateStoreStatus updates the information of the store.

Jump to

Keyboard shortcuts

? : This menu
/ : Search site
f or F : Jump to
y or Y : Canonical URL