Documentation ¶
Index ¶
- Constants
- func IsEndStatus(s OpStatus) bool
- func OpStatusToPDPB(s OpStatus) pdpb.OperatorStatus
- func OpStatusToString(s OpStatus) string
- func SetOperatorStatusReachTime(op *Operator, st OpStatus, t time.Time)
- type AddLearner
- type AddLightLearner
- type AddLightPeer
- type AddPeer
- type Builder
- func (b *Builder) AddPeer(p *metapb.Peer) *Builder
- func (b *Builder) Build(kind OpKind) (*Operator, error)
- func (b *Builder) PromoteLearner(storeID uint64) *Builder
- func (b *Builder) RemovePeer(storeID uint64) *Builder
- func (b *Builder) SetLeader(storeID uint64) *Builder
- func (b *Builder) SetLightWeight() *Builder
- func (b *Builder) SetPeers(peers map[uint64]*metapb.Peer) *Builder
- type Cluster
- type MergeRegion
- type OpHistory
- type OpInfluence
- type OpKind
- type OpStatus
- type OpStatusTracker
- func (trk *OpStatusTracker) CheckExpired(exp time.Duration) bool
- func (trk *OpStatusTracker) CheckTimeout(wait time.Duration) bool
- func (trk *OpStatusTracker) IsEnd() bool
- func (trk *OpStatusTracker) ReachTime() time.Time
- func (trk *OpStatusTracker) ReachTimeOf(s OpStatus) time.Time
- func (trk *OpStatusTracker) Status() OpStatus
- func (trk *OpStatusTracker) String() string
- func (trk *OpStatusTracker) To(dst OpStatus) bool
- type OpStep
- type Operator
- func CreateAddPeerOperator(desc string, cluster Cluster, region *core.RegionInfo, peer *metapb.Peer, ...) (*Operator, error)
- func CreateMergeRegionOperator(desc string, cluster Cluster, source *core.RegionInfo, target *core.RegionInfo, ...) ([]*Operator, error)
- func CreateMoveLeaderOperator(desc string, cluster Cluster, region *core.RegionInfo, kind OpKind, ...) (*Operator, error)
- func CreateMovePeerOperator(desc string, cluster Cluster, region *core.RegionInfo, kind OpKind, ...) (*Operator, error)
- func CreateMoveRegionOperator(desc string, cluster Cluster, region *core.RegionInfo, kind OpKind, ...) (*Operator, error)
- func CreatePromoteLearnerOperator(desc string, cluster Cluster, region *core.RegionInfo, peer *metapb.Peer) (*Operator, error)
- func CreateRemovePeerOperator(desc string, cluster Cluster, kind OpKind, region *core.RegionInfo, ...) (*Operator, error)
- func CreateScatterRegionOperator(desc string, cluster Cluster, origin *core.RegionInfo, ...) (*Operator, error)
- func CreateSplitRegionOperator(desc string, region *core.RegionInfo, kind OpKind, policy pdpb.CheckPolicy, ...) *Operator
- func CreateTransferLeaderOperator(desc string, cluster Cluster, region *core.RegionInfo, sourceStoreID uint64, ...) (*Operator, error)
- func NewOperator(desc, brief string, regionID uint64, regionEpoch *metapb.RegionEpoch, ...) *Operator
- func (o *Operator) AttachKind(kind OpKind)
- func (o *Operator) Cancel() bool
- func (o *Operator) Check(region *core.RegionInfo) OpStep
- func (o *Operator) CheckExpired() bool
- func (o *Operator) CheckSuccess() bool
- func (o *Operator) CheckTimeout() bool
- func (o *Operator) ConfVerChanged(region *core.RegionInfo) int
- func (o *Operator) Desc() string
- func (o *Operator) ElapsedTime() time.Duration
- func (o *Operator) GetCreateTime() time.Time
- func (o *Operator) GetPriorityLevel() core.PriorityLevel
- func (o *Operator) GetReachTimeOf(st OpStatus) time.Time
- func (o *Operator) GetStartTime() time.Time
- func (o *Operator) HasStarted() bool
- func (o *Operator) History() []OpHistory
- func (o *Operator) IsEnd() bool
- func (o *Operator) Kind() OpKind
- func (o *Operator) Len() int
- func (o *Operator) MarshalJSON() ([]byte, error)
- func (o *Operator) RegionEpoch() *metapb.RegionEpoch
- func (o *Operator) RegionID() uint64
- func (o *Operator) Replace() bool
- func (o *Operator) RunningTime() time.Duration
- func (o *Operator) SetDesc(desc string)
- func (o *Operator) SetPriorityLevel(level core.PriorityLevel)
- func (o *Operator) Start() bool
- func (o *Operator) Status() OpStatus
- func (o *Operator) Step(i int) OpStep
- func (o *Operator) String() string
- func (o *Operator) TotalInfluence(opInfluence OpInfluence, region *core.RegionInfo)
- func (o *Operator) UnfinishedInfluence(opInfluence OpInfluence, region *core.RegionInfo)
- type PromoteLearner
- type RemovePeer
- type SplitRegion
- type StoreInfluence
- type TransferLeader
Constants ¶
const ( // OperatorExpireTime is the duration that when an operator is not started // after it, the operator will be considered expired. OperatorExpireTime = 3 * time.Second // LeaderOperatorWaitTime is the duration that when a leader operator runs // longer than it, the operator will be considered timeout. LeaderOperatorWaitTime = 10 * time.Second // RegionOperatorWaitTime is the duration that when a region operator runs // longer than it, the operator will be considered timeout. RegionOperatorWaitTime = 10 * time.Minute )
Variables ¶
This section is empty.
Functions ¶
func OpStatusToPDPB ¶
func OpStatusToPDPB(s OpStatus) pdpb.OperatorStatus
OpStatusToPDPB converts OpStatus to pdpb.OperatorStatus.
func OpStatusToString ¶
OpStatusToString converts Status to string.
Types ¶
type AddLearner ¶
type AddLearner struct {
ToStore, PeerID uint64
}
AddLearner is an OpStep that adds a region learner peer.
func (AddLearner) ConfVerChanged ¶
func (al AddLearner) ConfVerChanged(region *core.RegionInfo) bool
ConfVerChanged returns true if the conf version has been changed by this step
func (AddLearner) Influence ¶
func (al AddLearner) Influence(opInfluence OpInfluence, region *core.RegionInfo)
Influence calculates the store difference that current step makes.
func (AddLearner) IsFinish ¶
func (al AddLearner) IsFinish(region *core.RegionInfo) bool
IsFinish checks if current step is finished.
func (AddLearner) String ¶
func (al AddLearner) String() string
type AddLightLearner ¶
type AddLightLearner struct {
ToStore, PeerID uint64
}
AddLightLearner is an OpStep that adds a region learner peer without considering the influence.
func (AddLightLearner) ConfVerChanged ¶
func (al AddLightLearner) ConfVerChanged(region *core.RegionInfo) bool
ConfVerChanged returns true if the conf version has been changed by this step
func (AddLightLearner) Influence ¶
func (al AddLightLearner) Influence(opInfluence OpInfluence, region *core.RegionInfo)
Influence calculates the store difference that current step makes.
func (AddLightLearner) IsFinish ¶
func (al AddLightLearner) IsFinish(region *core.RegionInfo) bool
IsFinish checks if current step is finished.
func (AddLightLearner) String ¶
func (al AddLightLearner) String() string
type AddLightPeer ¶
type AddLightPeer struct {
ToStore, PeerID uint64
}
AddLightPeer is an OpStep that adds a region peer without considering the influence.
func (AddLightPeer) ConfVerChanged ¶
func (ap AddLightPeer) ConfVerChanged(region *core.RegionInfo) bool
ConfVerChanged returns true if the conf version has been changed by this step
func (AddLightPeer) Influence ¶
func (ap AddLightPeer) Influence(opInfluence OpInfluence, region *core.RegionInfo)
Influence calculates the store difference that current step makes.
func (AddLightPeer) IsFinish ¶
func (ap AddLightPeer) IsFinish(region *core.RegionInfo) bool
IsFinish checks if current step is finished.
func (AddLightPeer) String ¶
func (ap AddLightPeer) String() string
type AddPeer ¶
type AddPeer struct {
ToStore, PeerID uint64
}
AddPeer is an OpStep that adds a region peer.
func (AddPeer) ConfVerChanged ¶
func (ap AddPeer) ConfVerChanged(region *core.RegionInfo) bool
ConfVerChanged returns true if the conf version has been changed by this step
func (AddPeer) Influence ¶
func (ap AddPeer) Influence(opInfluence OpInfluence, region *core.RegionInfo)
Influence calculates the store difference that current step makes.
type Builder ¶
type Builder struct {
// contains filtered or unexported fields
}
Builder is used to create operators. Usage:
op, err := NewBuilder(desc, cluster, region). RemovePeer(store1). AddPeer(peer1). SetLeader(store2). Build(kind)
The generated Operator will choose the most appropriate execution order according to various constraints.
func NewBuilder ¶
func NewBuilder(desc string, cluster Cluster, region *core.RegionInfo) *Builder
NewBuilder creates a Builder.
func (*Builder) AddPeer ¶
AddPeer records an add Peer operation in Builder. If p.Id is 0, the builder will allocate a new peer ID later.
func (*Builder) PromoteLearner ¶
PromoteLearner records a promote learner operation in Builder.
func (*Builder) RemovePeer ¶
RemovePeer records a remove peer operation in Builder.
func (*Builder) SetLightWeight ¶
SetLightWeight marks the region as light weight. It is used for scatter regions.
type Cluster ¶
type Cluster interface { opt.Options GetStore(id uint64) *core.StoreInfo AllocID() (uint64, error) FitRegion(region *core.RegionInfo) *placement.RegionFit }
Cluster provides an overview of a cluster's regions distribution.
type MergeRegion ¶
type MergeRegion struct { FromRegion *metapb.Region ToRegion *metapb.Region // there are two regions involved in merge process, // so to keep them from other scheduler, // both of them should add MerRegion operatorStep. // But actually, TiKV just needs the region want to be merged to get the merge request, // thus use a IsPassive mark to indicate that // this region doesn't need to send merge request to TiKV. IsPassive bool }
MergeRegion is an OpStep that merge two regions.
func (MergeRegion) ConfVerChanged ¶
func (mr MergeRegion) ConfVerChanged(region *core.RegionInfo) bool
ConfVerChanged returns true if the conf version has been changed by this step
func (MergeRegion) Influence ¶
func (mr MergeRegion) Influence(opInfluence OpInfluence, region *core.RegionInfo)
Influence calculates the store difference that current step makes.
func (MergeRegion) IsFinish ¶
func (mr MergeRegion) IsFinish(region *core.RegionInfo) bool
IsFinish checks if current step is finished.
func (MergeRegion) String ¶
func (mr MergeRegion) String() string
type OpHistory ¶
type OpHistory struct { FinishTime time.Time From, To uint64 Kind core.ResourceKind }
OpHistory is used to log and visualize completed operators.
type OpInfluence ¶
type OpInfluence struct {
StoresInfluence map[uint64]*StoreInfluence
}
OpInfluence records the influence of the cluster.
func (OpInfluence) GetStoreInfluence ¶
func (m OpInfluence) GetStoreInfluence(id uint64) *StoreInfluence
GetStoreInfluence get storeInfluence of specific store.
type OpKind ¶
type OpKind uint32
OpKind is a bit field to identify operator types.
const ( OpLeader OpKind = 1 << iota // Include leader transfer. OpRegion // Include peer movement. OpSplit // Include region split. OpAdmin // Initiated by admin. OpHotRegion // Initiated by hot region scheduler. OpAdjacent // Initiated by adjacent region scheduler. OpReplica // Initiated by replica checkers. OpBalance // Initiated by balancers. OpMerge // Initiated by merge checkers or merge schedulers. OpRange // Initiated by range scheduler. )
Flags for operators.
func ParseOperatorKind ¶
ParseOperatorKind converts string (flag name list concat by ',') to OpKind.
type OpStatus ¶
type OpStatus = uint32
OpStatus represents the status of an Operator.
const ( // Status list // Just created. Next status: {RUNNING, CANCELED, EXPIRED}. CREATED OpStatus = iota // Started and not finished. Next status: {SUCCESS, CANCELED, REPLACED, TIMEOUT}. STARTED // Followings are end status, i.e. no next status. SUCCESS // Finished successfully CANCELED // Canceled due to some reason REPLACED // Replaced by an higher priority operator EXPIRED // Didn't start to run for too long TIMEOUT // Running for too long )
Status list
type OpStatusTracker ¶
type OpStatusTracker struct {
// contains filtered or unexported fields
}
OpStatusTracker represents the status of an operator.
func NewOpStatusTracker ¶
func NewOpStatusTracker() OpStatusTracker
NewOpStatusTracker creates an OpStatus.
func (*OpStatusTracker) CheckExpired ¶
func (trk *OpStatusTracker) CheckExpired(exp time.Duration) bool
CheckExpired checks if expired, and update the current status.
func (*OpStatusTracker) CheckTimeout ¶
func (trk *OpStatusTracker) CheckTimeout(wait time.Duration) bool
CheckTimeout checks if timeout, and update the current status.
func (*OpStatusTracker) IsEnd ¶
func (trk *OpStatusTracker) IsEnd() bool
IsEnd checks whether the current status is an end status.
func (*OpStatusTracker) ReachTime ¶
func (trk *OpStatusTracker) ReachTime() time.Time
ReachTime returns the reach time of current status.
func (*OpStatusTracker) ReachTimeOf ¶
func (trk *OpStatusTracker) ReachTimeOf(s OpStatus) time.Time
ReachTimeOf returns the time when reached given status. If didn't reatched the given status, return zero.
func (*OpStatusTracker) Status ¶
func (trk *OpStatusTracker) Status() OpStatus
Status returns current status.
func (*OpStatusTracker) String ¶
func (trk *OpStatusTracker) String() string
String implements fmt.Stringer.
func (*OpStatusTracker) To ¶
func (trk *OpStatusTracker) To(dst OpStatus) bool
To transfer the current status to dst if this transition is valid, returns whether transferred.
type OpStep ¶
type OpStep interface { fmt.Stringer ConfVerChanged(region *core.RegionInfo) bool IsFinish(region *core.RegionInfo) bool Influence(opInfluence OpInfluence, region *core.RegionInfo) }
OpStep describes the basic scheduling steps that can not be subdivided.
type Operator ¶
type Operator struct { Counters []prometheus.Counter // contains filtered or unexported fields }
Operator contains execution steps generated by scheduler.
func CreateAddPeerOperator ¶
func CreateAddPeerOperator(desc string, cluster Cluster, region *core.RegionInfo, peer *metapb.Peer, kind OpKind) (*Operator, error)
CreateAddPeerOperator creates an operator that adds a new peer.
func CreateMergeRegionOperator ¶
func CreateMergeRegionOperator(desc string, cluster Cluster, source *core.RegionInfo, target *core.RegionInfo, kind OpKind) ([]*Operator, error)
CreateMergeRegionOperator creates an operator that merge two region into one.
func CreateMoveLeaderOperator ¶
func CreateMoveLeaderOperator(desc string, cluster Cluster, region *core.RegionInfo, kind OpKind, oldStore uint64, peer *metapb.Peer) (*Operator, error)
CreateMoveLeaderOperator creates an operator that replaces an old leader with a new leader.
func CreateMovePeerOperator ¶
func CreateMovePeerOperator(desc string, cluster Cluster, region *core.RegionInfo, kind OpKind, oldStore uint64, peer *metapb.Peer) (*Operator, error)
CreateMovePeerOperator creates an operator that replaces an old peer with a new peer.
func CreateMoveRegionOperator ¶
func CreateMoveRegionOperator(desc string, cluster Cluster, region *core.RegionInfo, kind OpKind, peers map[uint64]*metapb.Peer) (*Operator, error)
CreateMoveRegionOperator creates an operator that moves a region to specified stores.
func CreatePromoteLearnerOperator ¶
func CreatePromoteLearnerOperator(desc string, cluster Cluster, region *core.RegionInfo, peer *metapb.Peer) (*Operator, error)
CreatePromoteLearnerOperator creates an operator that promotes a learner.
func CreateRemovePeerOperator ¶
func CreateRemovePeerOperator(desc string, cluster Cluster, kind OpKind, region *core.RegionInfo, storeID uint64) (*Operator, error)
CreateRemovePeerOperator creates an operator that removes a peer from region.
func CreateScatterRegionOperator ¶
func CreateScatterRegionOperator(desc string, cluster Cluster, origin *core.RegionInfo, targetPeers map[uint64]*metapb.Peer) (*Operator, error)
CreateScatterRegionOperator creates an operator that scatters the specified region.
func CreateSplitRegionOperator ¶
func CreateSplitRegionOperator(desc string, region *core.RegionInfo, kind OpKind, policy pdpb.CheckPolicy, keys [][]byte) *Operator
CreateSplitRegionOperator creates an operator that splits a region.
func CreateTransferLeaderOperator ¶
func CreateTransferLeaderOperator(desc string, cluster Cluster, region *core.RegionInfo, sourceStoreID uint64, targetStoreID uint64, kind OpKind) (*Operator, error)
CreateTransferLeaderOperator creates an operator that transfers the leader from a source store to a target store.
func NewOperator ¶
func NewOperator(desc, brief string, regionID uint64, regionEpoch *metapb.RegionEpoch, kind OpKind, steps ...OpStep) *Operator
NewOperator creates a new operator.
func (*Operator) AttachKind ¶
AttachKind attaches an operator kind for the operator.
func (*Operator) Check ¶
func (o *Operator) Check(region *core.RegionInfo) OpStep
Check checks if current step is finished, returns next step to take action. If operator is at an end status, check returns nil. It's safe to be called by multiple goroutine concurrently. FIXME: metrics is not thread-safe
func (*Operator) CheckExpired ¶
CheckExpired checks if the operator is expired, and update the status.
func (*Operator) CheckSuccess ¶
CheckSuccess checks if all steps are finished, and update the status.
func (*Operator) CheckTimeout ¶
CheckTimeout checks if the operator is timeout, and update the status.
func (*Operator) ConfVerChanged ¶
func (o *Operator) ConfVerChanged(region *core.RegionInfo) int
ConfVerChanged returns the number of confver has consumed by steps
func (*Operator) ElapsedTime ¶
ElapsedTime returns duration since it was created.
func (*Operator) GetCreateTime ¶
GetCreateTime gets the create time of operator.
func (*Operator) GetPriorityLevel ¶
func (o *Operator) GetPriorityLevel() core.PriorityLevel
GetPriorityLevel gets the priority level.
func (*Operator) GetReachTimeOf ¶
GetReachTimeOf returns the time when operator reaches the given status.
func (*Operator) GetStartTime ¶
GetStartTime gets the start time of operator.
func (*Operator) HasStarted ¶
HasStarted returns whether operator has started.
func (*Operator) MarshalJSON ¶
MarshalJSON serializes custom types to JSON.
func (*Operator) RegionEpoch ¶
func (o *Operator) RegionEpoch() *metapb.RegionEpoch
RegionEpoch returns the region's epoch that is attached to the operator.
func (*Operator) RunningTime ¶
RunningTime returns duration since it started.
func (*Operator) SetPriorityLevel ¶
func (o *Operator) SetPriorityLevel(level core.PriorityLevel)
SetPriorityLevel sets the priority level for operator.
func (*Operator) TotalInfluence ¶
func (o *Operator) TotalInfluence(opInfluence OpInfluence, region *core.RegionInfo)
TotalInfluence calculates the store difference which whole operator steps make.
func (*Operator) UnfinishedInfluence ¶
func (o *Operator) UnfinishedInfluence(opInfluence OpInfluence, region *core.RegionInfo)
UnfinishedInfluence calculates the store difference which unfinished operator steps make.
type PromoteLearner ¶
type PromoteLearner struct {
ToStore, PeerID uint64
}
PromoteLearner is an OpStep that promotes a region learner peer to normal voter.
func (PromoteLearner) ConfVerChanged ¶
func (pl PromoteLearner) ConfVerChanged(region *core.RegionInfo) bool
ConfVerChanged returns true if the conf version has been changed by this step
func (PromoteLearner) Influence ¶
func (pl PromoteLearner) Influence(opInfluence OpInfluence, region *core.RegionInfo)
Influence calculates the store difference that current step makes.
func (PromoteLearner) IsFinish ¶
func (pl PromoteLearner) IsFinish(region *core.RegionInfo) bool
IsFinish checks if current step is finished.
func (PromoteLearner) String ¶
func (pl PromoteLearner) String() string
type RemovePeer ¶
type RemovePeer struct {
FromStore uint64
}
RemovePeer is an OpStep that removes a region peer.
func (RemovePeer) ConfVerChanged ¶
func (rp RemovePeer) ConfVerChanged(region *core.RegionInfo) bool
ConfVerChanged returns true if the conf version has been changed by this step
func (RemovePeer) Influence ¶
func (rp RemovePeer) Influence(opInfluence OpInfluence, region *core.RegionInfo)
Influence calculates the store difference that current step makes.
func (RemovePeer) IsFinish ¶
func (rp RemovePeer) IsFinish(region *core.RegionInfo) bool
IsFinish checks if current step is finished.
func (RemovePeer) String ¶
func (rp RemovePeer) String() string
type SplitRegion ¶
type SplitRegion struct {
StartKey, EndKey []byte
Policy pdpb.CheckPolicy
SplitKeys [][]byte
}
SplitRegion is an OpStep that splits a region.
func (SplitRegion) ConfVerChanged ¶
func (sr SplitRegion) ConfVerChanged(region *core.RegionInfo) bool
ConfVerChanged returns true if the conf version has been changed by this step
func (SplitRegion) Influence ¶
func (sr SplitRegion) Influence(opInfluence OpInfluence, region *core.RegionInfo)
Influence calculates the store difference that current step makes.
func (SplitRegion) IsFinish ¶
func (sr SplitRegion) IsFinish(region *core.RegionInfo) bool
IsFinish checks if current step is finished.
func (SplitRegion) String ¶
func (sr SplitRegion) String() string
type StoreInfluence ¶
type StoreInfluence struct { RegionSize int64 RegionCount int64 LeaderSize int64 LeaderCount int64 StepCost map[storelimit.Type]int64 }
StoreInfluence records influences that pending operators will make.
func (*StoreInfluence) AdjustStepCost ¶ added in v3.1.1
func (s *StoreInfluence) AdjustStepCost(limitType storelimit.Type, regionSize int64)
AdjustStepCost adjusts the step cost of specific type store limit according to region size
func (StoreInfluence) GetStepCost ¶ added in v3.1.1
func (s StoreInfluence) GetStepCost(limitType storelimit.Type) int64
GetStepCost returns the specific type step cost
func (StoreInfluence) ResourceProperty ¶
func (s StoreInfluence) ResourceProperty(kind core.ScheduleKind) int64
ResourceProperty returns delta size of leader/region by influence.
type TransferLeader ¶
type TransferLeader struct {
FromStore, ToStore uint64
}
TransferLeader is an OpStep that transfers a region's leader.
func (TransferLeader) ConfVerChanged ¶
func (tl TransferLeader) ConfVerChanged(region *core.RegionInfo) bool
ConfVerChanged returns true if the conf version has been changed by this step
func (TransferLeader) Influence ¶
func (tl TransferLeader) Influence(opInfluence OpInfluence, region *core.RegionInfo)
Influence calculates the store difference that current step makes.
func (TransferLeader) IsFinish ¶
func (tl TransferLeader) IsFinish(region *core.RegionInfo) bool
IsFinish checks if current step is finished.
func (TransferLeader) String ¶
func (tl TransferLeader) String() string