Documentation
¶
Index ¶
- Constants
- Variables
- func ConfigWithTTLSeconds(ttlSeconds float64) string
- func GetCancelSchedulerJobURIByNameAndJobID(name string, jobID uint64) string
- func GetDeleteSafePointURI(serviceID string) string
- func GetKeyspaceMetaByIDURL(id uint32) string
- func GetKeyspaceMetaByNameURL(keyspaceName string) string
- func GetSchedulerConfigURIByName(name string) string
- func GetUpdateKeyspaceConfigURL(keyspaceName string) string
- func LabelByStoreID(storeID int64) string
- func MicroserviceMembers(service string) string
- func MicroservicePrimary(service string) string
- func NewHTTPClientWithRequestChecker(checker func(req *http.Request) error) *http.Client
- func PProfGoroutineWithDebugLevel(level int) string
- func PProfProfileAPIWithInterval(interval time.Duration) string
- func PlacementRuleBundleByGroup(group string) string
- func PlacementRuleBundleWithPartialParameter(partial bool) string
- func PlacementRuleByGroupAndID(group, id string) string
- func PlacementRuleGroupByID(id string) string
- func PlacementRulesByGroup(group string) string
- func RegionByID(regionID uint64) string
- func RegionByKey(key []byte) string
- func RegionDistributionsByKeyRange(keyRange *KeyRange, engine string) string
- func RegionSiblingsByID(regionID uint64) string
- func RegionStatsByKeyRange(keyRange *KeyRange, onlyCount bool) string
- func RegionsByKeyRange(keyRange *KeyRange, limit int) string
- func RegionsByStoreID(storeID uint64) string
- func RegionsReplicatedByKeyRange(keyRange *KeyRange) string
- func ScatterRangeSchedulerWithName(name string) string
- func SchedulerByName(name string) string
- func StoreByID(id uint64) string
- func StoreLabelByID(id uint64) string
- func TransferLeaderByID(leaderID string) string
- type AffinityGroup
- type AffinityGroupKeyRange
- type AffinityGroupState
- type AffinityGroupsResponse
- type BatchDeleteAffinityGroupsRequest
- type BatchModifyAffinityGroupsRequest
- type Client
- type ClientOption
- type ClusterState
- type CreateAffinityGroupInput
- type CreateAffinityGroupsRequest
- type GroupBundle
- type GroupRangesModification
- type HeaderOption
- type Health
- type HistoryHotRegion
- type HistoryHotRegions
- type HistoryHotRegionsRequest
- type HotPeerStatShow
- type HotPeersStat
- type KeyRange
- type KeyspaceGCManagementType
- type KeyspaceGCManagementTypeConfig
- type LabelConstraint
- type LabelConstraintOp
- type LabelRule
- type LabelRulePatch
- type ListServiceGCSafepoint
- type MembersInfo
- type MetaStore
- type MicroserviceMember
- type PeerRoleType
- type RegionDistribution
- type RegionDistributions
- type RegionEpoch
- type RegionInfo
- type RegionLabel
- type RegionPeer
- type RegionPeerStat
- type RegionStats
- type RegionsInfo
- type ReplicationStatus
- type Rule
- type RuleGroup
- type RuleOp
- type RuleOpType
- type ServiceSafePoint
- type State
- type StoreHotPeersInfos
- type StoreHotPeersStat
- type StoreInfo
- type StoreLabel
- type StoreStatus
- type StoresInfo
- type UpdateAffinityGroupPeersRequest
Constants ¶
const ( // Metadata HotRead = "/pd/api/v1/hotspot/regions/read" HotWrite = "/pd/api/v1/hotspot/regions/write" HotHistory = "/pd/api/v1/hotspot/regions/history" RegionByIDPrefix = "/pd/api/v1/region/id" Regions = "/pd/api/v1/regions" RegionsByStoreIDPrefix = "/pd/api/v1/regions/store" EmptyRegions = "/pd/api/v1/regions/check/empty-region" AccelerateSchedule = "/pd/api/v1/regions/accelerate-schedule" AccelerateScheduleInBatch = "/pd/api/v1/regions/accelerate-schedule/batch" Stores = "/pd/api/v1/stores" StatsRegion = "/pd/api/v1/stats/region" // Config Config = "/pd/api/v1/config" ClusterVersion = "/pd/api/v1/config/cluster-version" ScheduleConfig = "/pd/api/v1/config/schedule" ReplicateConfig = "/pd/api/v1/config/replicate" // Rule PlacementRule = "/pd/api/v1/config/rule" PlacementRules = "/pd/api/v1/config/rules" PlacementRulesInBatch = "/pd/api/v1/config/rules/batch" PlacementRuleBundle = "/pd/api/v1/config/placement-rule" RegionLabelRule = "/pd/api/v1/config/region-label/rule" RegionLabelRules = "/pd/api/v1/config/region-label/rules" RegionLabelRulesByIDs = "/pd/api/v1/config/region-label/rules/ids" // Scheduler Schedulers = "/pd/api/v1/schedulers" SchedulerConfig = "/pd/api/v1/scheduler-config" // Admin ResetTS = "/pd/api/v1/admin/reset-ts" BaseAllocID = "/pd/api/v1/admin/base-alloc-id" SnapshotRecoveringMark = "/pd/api/v1/admin/cluster/markers/snapshot-recovering" PitrRestoreModeMark = "/pd/api/v1/admin/cluster/markers/pitr-restore-mode" // Debug PProfProfile = "/pd/api/v1/debug/pprof/profile" PProfHeap = "/pd/api/v1/debug/pprof/heap" PProfMutex = "/pd/api/v1/debug/pprof/mutex" PProfAllocs = "/pd/api/v1/debug/pprof/allocs" PProfBlock = "/pd/api/v1/debug/pprof/block" PProfGoroutine = "/pd/api/v1/debug/pprof/goroutine" // Others MinResolvedTSPrefix = "/pd/api/v1/min-resolved-ts" Cluster = "/pd/api/v1/cluster" ClusterStatus = "/pd/api/v1/cluster/status" Status = "/pd/api/v1/status" Version = "/pd/api/v1/version" // Keyspace KeyspaceConfig = "/pd/api/v2/keyspaces/%s/config" GetKeyspaceMetaByName = "/pd/api/v2/keyspaces/%s" GetKeyspaceMetaByID = "/pd/api/v2/keyspaces/id/%d" // Affinity AffinityGroups = "/pd/api/v2/affinity-groups" AffinityGroupByID = "/pd/api/v2/affinity-groups/%s" )
The following constants are the paths of PD HTTP APIs.
const ( UpdateKeyspaceGCManagementTypeName = "UpdateKeyspaceGCManagementType" GetKeyspaceMetaByNameName = "GetKeyspaceMetaByName" GetKeyspaceMetaByIDName = "GetKeyspaceMetaByID" GetGCSafePointName = "GetGCSafePoint" DeleteGCSafePointName = "DeleteGCSafePoint" )
The following constants are the names of the requests.
Variables ¶
var NewKeyRange = pd.NewKeyRange
NewKeyRange alias pd.NewKeyRange to avoid break client compatibility.
Functions ¶
func ConfigWithTTLSeconds ¶
ConfigWithTTLSeconds returns the config API with the TTL seconds parameter.
func GetCancelSchedulerJobURIByNameAndJobID ¶
GetCancelSchedulerJobURIByNameAndJobID returns the path of PD HTTP API to cancel the job of the given scheduler
func GetDeleteSafePointURI ¶
GetDeleteSafePointURI returns the URI for delete safepoint service
func GetKeyspaceMetaByIDURL ¶
GetKeyspaceMetaByIDURL returns the path of PD HTTP API to get keyspace meta by keyspace id.
func GetKeyspaceMetaByNameURL ¶
GetKeyspaceMetaByNameURL returns the path of PD HTTP API to get keyspace meta by keyspace name.
func GetSchedulerConfigURIByName ¶
GetSchedulerConfigURIByName returns the path of PD HTTP API to get configuration of the given scheduler
func GetUpdateKeyspaceConfigURL ¶
GetUpdateKeyspaceConfigURL returns the path of PD HTTP API to update keyspace config.
func LabelByStoreID ¶
LabelByStoreID returns the path of PD HTTP API to set store label.
func MicroserviceMembers ¶
MicroserviceMembers returns the path of PD HTTP API to get the members of microservice.
func MicroservicePrimary ¶
MicroservicePrimary returns the path of PD HTTP API to get the primary of microservice.
func NewHTTPClientWithRequestChecker ¶
NewHTTPClientWithRequestChecker returns a http client with checker.
func PProfGoroutineWithDebugLevel ¶
PProfGoroutineWithDebugLevel returns the pprof goroutine API with debug level parameter.
func PProfProfileAPIWithInterval ¶
PProfProfileAPIWithInterval returns the pprof profile API with interval parameter.
func PlacementRuleBundleByGroup ¶
PlacementRuleBundleByGroup returns the path of PD HTTP API to get placement rule bundle by group.
func PlacementRuleBundleWithPartialParameter ¶
PlacementRuleBundleWithPartialParameter returns the path of PD HTTP API to get placement rule bundle with partial parameter.
func PlacementRuleByGroupAndID ¶
PlacementRuleByGroupAndID returns the path of PD HTTP API to get placement rule by group and ID.
func PlacementRuleGroupByID ¶
PlacementRuleGroupByID returns the path of PD HTTP API to get placement rule group by ID.
func PlacementRulesByGroup ¶
PlacementRulesByGroup returns the path of PD HTTP API to get placement rules by group.
func RegionByID ¶
RegionByID returns the path of PD HTTP API to get region by ID.
func RegionByKey ¶
RegionByKey returns the path of PD HTTP API to get region by key.
func RegionDistributionsByKeyRange ¶
RegionDistributionsByKeyRange returns the path of PD HTTP API to get region distribution by start key and end key.
func RegionSiblingsByID ¶
RegionSiblingsByID returns the path of PD HTTP API to get sibling regions by ID.
func RegionStatsByKeyRange ¶
RegionStatsByKeyRange returns the path of PD HTTP API to get region stats by start key and end key.
func RegionsByKeyRange ¶
RegionsByKeyRange returns the path of PD HTTP API to scan regions with given start key, end key and limit parameters.
func RegionsByStoreID ¶
RegionsByStoreID returns the path of PD HTTP API to get regions by store ID.
func RegionsReplicatedByKeyRange ¶
RegionsReplicatedByKeyRange returns the path of PD HTTP API to get replicated regions with given start key and end key.
func ScatterRangeSchedulerWithName ¶
ScatterRangeSchedulerWithName returns the scatter range scheduler API with name parameter. It is used in https://github.com/pingcap/tidb/blob/2a3352c45dd0f8dd5102adb92879bbfa964e7f5f/pkg/server/handler/tikvhandler/tikv_handler.go#L1252.
func SchedulerByName ¶
SchedulerByName returns the scheduler API with the given scheduler name.
func StoreLabelByID ¶
StoreLabelByID returns the store label API with store ID parameter.
func TransferLeaderByID ¶
TransferLeaderByID returns the path of PD HTTP API to transfer leader by ID.
Types ¶
type AffinityGroup ¶
type AffinityGroup struct {
ID string `json:"id"`
CreateTimestamp uint64 `json:"create_timestamp"`
LeaderStoreID uint64 `json:"leader_store_id,omitempty"`
VoterStoreIDs []uint64 `json:"voter_store_ids,omitempty"`
}
AffinityGroup defines an affinity group.
type AffinityGroupKeyRange ¶
type AffinityGroupKeyRange struct {
StartKey []byte `json:"start_key"`
EndKey []byte `json:"end_key"`
}
AffinityGroupKeyRange represents a key range for affinity group operations.
type AffinityGroupState ¶
type AffinityGroupState struct {
AffinityGroup
Phase string `json:"phase"`
RangeCount int `json:"range_count"`
RegionCount int `json:"region_count"`
AffinityRegionCount int `json:"affinity_region_count"`
}
AffinityGroupState defines the runtime state of an affinity group.
func (*AffinityGroupState) IsPending ¶
func (s *AffinityGroupState) IsPending() bool
IsPending indicates that the Group is still determining the StoreIDs. If the Group has no KeyRanges, it remains in pending forever.
func (*AffinityGroupState) IsPreparing ¶
func (s *AffinityGroupState) IsPreparing() bool
IsPreparing indicates that the Group is scheduling Regions according to the required Peers.
func (*AffinityGroupState) IsStable ¶
func (s *AffinityGroupState) IsStable() bool
IsStable indicates that the Group has completed the required scheduling and is currently in a stable state.
type AffinityGroupsResponse ¶
type AffinityGroupsResponse struct {
AffinityGroups map[string]*AffinityGroupState `json:"affinity_groups"`
}
AffinityGroupsResponse defines the success response for affinity group operations.
type BatchDeleteAffinityGroupsRequest ¶
type BatchDeleteAffinityGroupsRequest struct {
IDs []string `json:"ids"`
Force bool `json:"force,omitempty"`
}
BatchDeleteAffinityGroupsRequest defines the body for batch delete request.
type BatchModifyAffinityGroupsRequest ¶
type BatchModifyAffinityGroupsRequest struct {
Add []GroupRangesModification `json:"add,omitempty"`
Remove []GroupRangesModification `json:"remove,omitempty"`
}
BatchModifyAffinityGroupsRequest defines the body for batch modify request.
type Client ¶
type Client interface {
/* Member-related interfaces */
GetMembers(context.Context) (*MembersInfo, error)
GetLeader(context.Context) (*pdpb.Member, error)
TransferLeader(context.Context, string) error
/* Meta-related interfaces */
GetRegionByID(context.Context, uint64) (*RegionInfo, error)
GetRegionByKey(context.Context, []byte) (*RegionInfo, error)
GetRegions(context.Context) (*RegionsInfo, error)
GetRegionsByKeyRange(context.Context, *KeyRange, int) (*RegionsInfo, error)
GetRegionsByStoreID(context.Context, uint64) (*RegionsInfo, error)
GetEmptyRegions(context.Context) (*RegionsInfo, error)
GetRegionsReplicatedStateByKeyRange(context.Context, *KeyRange) (string, error)
GetRegionSiblingsByID(context.Context, uint64) (*RegionsInfo, error)
GetHotReadRegions(context.Context) (*StoreHotPeersInfos, error)
GetHotWriteRegions(context.Context) (*StoreHotPeersInfos, error)
GetHistoryHotRegions(context.Context, *HistoryHotRegionsRequest) (*HistoryHotRegions, error)
GetRegionStatusByKeyRange(context.Context, *KeyRange, bool) (*RegionStats, error)
GetRegionDistributionByKeyRange(ctx context.Context, keyRange *KeyRange, engine string) (*RegionDistributions, error)
GetStores(context.Context) (*StoresInfo, error)
GetStore(context.Context, uint64) (*StoreInfo, error)
DeleteStore(context.Context, uint64) error
SetStoreLabels(context.Context, int64, map[string]string) error
DeleteStoreLabel(ctx context.Context, storeID int64, labelKey string) error
GetHealthStatus(context.Context) ([]Health, error)
/* Config-related interfaces */
GetConfig(context.Context) (map[string]any, error)
SetConfig(context.Context, map[string]any, ...float64) error
GetScheduleConfig(context.Context) (map[string]any, error)
SetScheduleConfig(context.Context, map[string]any) error
GetClusterVersion(context.Context) (string, error)
GetCluster(context.Context) (*metapb.Cluster, error)
GetClusterStatus(context.Context) (*ClusterState, error)
GetStatus(context.Context) (*State, error)
GetReplicateConfig(context.Context) (map[string]any, error)
/* Scheduler-related interfaces */
GetSchedulers(context.Context) ([]string, error)
CreateScheduler(ctx context.Context, name string, storeID uint64) error
CreateSchedulerWithInput(ctx context.Context, name string, input map[string]any) error
CancelSchedulerJob(ctx context.Context, name string, jobID uint64) error
DeleteScheduler(ctx context.Context, name string) error
SetSchedulerDelay(context.Context, string, int64) error
GetSchedulerConfig(ctx context.Context, name string) (any, error)
/* Rule-related interfaces */
GetAllPlacementRuleBundles(context.Context) ([]*GroupBundle, error)
GetPlacementRuleBundleByGroup(context.Context, string) (*GroupBundle, error)
GetPlacementRulesByGroup(context.Context, string) ([]*Rule, error)
GetPlacementRule(context.Context, string, string) (*Rule, error)
SetPlacementRule(context.Context, *Rule) error
SetPlacementRuleInBatch(context.Context, []*RuleOp) error
SetPlacementRuleBundles(context.Context, []*GroupBundle, bool) error
DeletePlacementRule(context.Context, string, string) error
GetAllPlacementRuleGroups(context.Context) ([]*RuleGroup, error)
GetPlacementRuleGroupByID(context.Context, string) (*RuleGroup, error)
SetPlacementRuleGroup(context.Context, *RuleGroup) error
DeletePlacementRuleGroupByID(context.Context, string) error
GetAllRegionLabelRules(context.Context) ([]*LabelRule, error)
GetRegionLabelRulesByIDs(context.Context, []string) ([]*LabelRule, error)
// `SetRegionLabelRule` sets the label rule for a region.
// When a label rule (deny scheduler) is set,
// 1. All schedulers will be disabled except for the evict-leader-scheduler.
// 2. The merge-checker will be disabled, preventing these regions from being merged.
SetRegionLabelRule(context.Context, *LabelRule) error
PatchRegionLabelRules(context.Context, *LabelRulePatch) error
/* Scheduling-related interfaces */
AccelerateSchedule(context.Context, *KeyRange) error
AccelerateScheduleInBatch(context.Context, []*KeyRange) error
/* Admin-related interfaces */
ResetTS(context.Context, uint64, bool) error
ResetBaseAllocID(context.Context, uint64) error
SetSnapshotRecoveringMark(context.Context) error
DeleteSnapshotRecoveringMark(context.Context) error
SetPitrRestoreModeMark(context.Context) error
DeletePitrRestoreModeMark(context.Context) error
/* Other interfaces */
GetMinResolvedTSByStoresIDs(context.Context, []uint64) (uint64, map[uint64]uint64, error)
GetPDVersion(context.Context) (string, error)
GetGCSafePoint(context.Context) (ListServiceGCSafepoint, error)
DeleteGCSafePoint(context.Context, string) (string, error)
/* Microservice interfaces */
GetMicroserviceMembers(context.Context, string) ([]MicroserviceMember, error)
GetMicroservicePrimary(context.Context, string) (string, error)
CreateOperators(context.Context, map[string]any) error
DeleteOperators(context.Context) error
// UpdateKeyspaceGCManagementType update the `gc_management_type` in keyspace meta config.
// If `gc_management_type` is `global_gc`, it means the current keyspace requires a tidb without 'keyspace-name'
// configured to run a global gc worker to calculate a global gc safe point.
// If `gc_management_type` is `keyspace_level_gc` it means the current keyspace can calculate gc safe point by its own.
UpdateKeyspaceGCManagementType(ctx context.Context, keyspaceName string, keyspaceGCManagementType *KeyspaceGCManagementTypeConfig) error
GetKeyspaceMetaByName(ctx context.Context, keyspaceName string) (*keyspacepb.KeyspaceMeta, error)
GetKeyspaceMetaByID(ctx context.Context, keyspaceID uint32) (*keyspacepb.KeyspaceMeta, error)
// CreateAffinityGroups creates one or more affinity groups with key ranges.
// The affinityGroups parameter is a map from group ID to a list of key ranges.
CreateAffinityGroups(ctx context.Context, affinityGroups map[string][]AffinityGroupKeyRange) (map[string]*AffinityGroupState, error)
// GetAffinityGroup gets an affinity group by group ID.
GetAffinityGroup(ctx context.Context, groupID string) (*AffinityGroupState, error)
// GetAllAffinityGroups gets all affinity groups.
GetAllAffinityGroups(ctx context.Context) (map[string]*AffinityGroupState, error)
// UpdateAffinityGroupPeers updates the leader and voter stores of an affinity group.
UpdateAffinityGroupPeers(ctx context.Context, groupID string, leaderStoreID uint64, voterStoreIDs []uint64) (*AffinityGroupState, error)
// DeleteAffinityGroup deletes an affinity group by group ID.
DeleteAffinityGroup(ctx context.Context, groupID string, force bool) error
// BatchDeleteAffinityGroups deletes multiple affinity groups in batch.
BatchDeleteAffinityGroups(ctx context.Context, groupIDs []string, force bool) error
// AddAffinityGroupKeyRanges adds key ranges to affinity groups.
AddAffinityGroupKeyRanges(ctx context.Context, groupKeyRanges map[string][]AffinityGroupKeyRange) (map[string]*AffinityGroupState, error)
// RemoveAffinityGroupKeyRanges removes key ranges from affinity groups.
RemoveAffinityGroupKeyRanges(ctx context.Context, groupKeyRanges map[string][]AffinityGroupKeyRange) (map[string]*AffinityGroupState, error)
/* Client-related methods */
// WithCallerID sets and returns a new client with the given caller ID.
WithCallerID(string) Client
// WithRespHandler sets and returns a new client with the given HTTP response handler.
// This allows the caller to customize how the response is handled, including error handling logic.
// Additionally, it is important for the caller to handle the content of the response body properly
// in order to ensure that it can be read and marshaled correctly into `res`.
WithRespHandler(func(resp *http.Response, res any) error) Client
// WithBackoffer sets and returns a new client with the given backoffer.
WithBackoffer(*retry.Backoffer) Client
// WithTargetURL sets and returns a new client with the given target URL.
WithTargetURL(string) Client
// Close gracefully closes the HTTP client.
Close()
}
Client is a PD (Placement Driver) HTTP client.
func NewClient ¶
func NewClient( source string, pdAddrs []string, opts ...ClientOption, ) Client
NewClient creates a PD HTTP client with the given PD addresses and TLS config.
func NewClientWithServiceDiscovery ¶
func NewClientWithServiceDiscovery( source string, sd sd.ServiceDiscovery, opts ...ClientOption, ) Client
NewClientWithServiceDiscovery creates a PD HTTP client with the given service discovery.
type ClientOption ¶
type ClientOption func(c *client)
ClientOption configures the HTTP client.
func WithHTTPClient ¶
func WithHTTPClient(cli *http.Client) ClientOption
WithHTTPClient configures the client with the given initialized HTTP client.
func WithMetrics ¶
func WithMetrics( requestCounter *prometheus.CounterVec, executionDuration *prometheus.HistogramVec, ) ClientOption
WithMetrics configures the client with metrics.
func WithTLSConfig ¶
func WithTLSConfig(tlsConf *tls.Config) ClientOption
WithTLSConfig configures the client with the given TLS config. This option won't work if the client is configured with WithHTTPClient.
type ClusterState ¶
type ClusterState struct {
RaftBootstrapTime time.Time `json:"raft_bootstrap_time,omitempty"`
IsInitialized bool `json:"is_initialized"`
ReplicationStatus string `json:"replication_status"`
}
ClusterState saves some cluster state information. NOTE: This type sync with https://github.com/tikv/pd/blob/5eae459c01a797cbd0c416054c6f0cad16b8740a/server/cluster/cluster.go#L173
type CreateAffinityGroupInput ¶
type CreateAffinityGroupInput struct {
Ranges []AffinityGroupKeyRange `json:"ranges"`
}
CreateAffinityGroupInput defines the input for a single group in the creation request.
type CreateAffinityGroupsRequest ¶
type CreateAffinityGroupsRequest struct {
AffinityGroups map[string]CreateAffinityGroupInput `json:"affinity_groups"`
}
CreateAffinityGroupsRequest defines the body for the POST request to create affinity groups.
type GroupBundle ¶
type GroupBundle struct {
ID string `json:"group_id"`
Index int `json:"group_index"`
Override bool `json:"group_override"`
Rules []*Rule `json:"rules"`
}
GroupBundle represents a rule group and all rules belong to the group.
type GroupRangesModification ¶
type GroupRangesModification struct {
ID string `json:"id"`
Ranges []AffinityGroupKeyRange `json:"ranges"`
}
GroupRangesModification defines add or remove operations for a specific group.
type HeaderOption ¶
HeaderOption configures the HTTP header.
func WithAllowFollowerHandle ¶
func WithAllowFollowerHandle() HeaderOption
WithAllowFollowerHandle sets the header field to allow a PD follower to handle this request.
type Health ¶
type Health struct {
Name string `json:"name"`
MemberID uint64 `json:"member_id"`
ClientUrls []string `json:"client_urls"`
Health bool `json:"health"`
}
Health reflects the cluster's health. NOTE: This type is moved from `server/api/health.go`, maybe move them to the same place later.
type HistoryHotRegion ¶
type HistoryHotRegion struct {
UpdateTime int64 `json:"update_time"`
RegionID uint64 `json:"region_id"`
PeerID uint64 `json:"peer_id"`
StoreID uint64 `json:"store_id"`
IsLeader bool `json:"is_leader"`
IsLearner bool `json:"is_learner"`
HotRegionType string `json:"hot_region_type"`
HotDegree int64 `json:"hot_degree"`
FlowBytes float64 `json:"flow_bytes"`
KeyRate float64 `json:"key_rate"`
QueryRate float64 `json:"query_rate"`
StartKey string `json:"start_key"`
EndKey string `json:"end_key"`
// Encryption metadata for start_key and end_key. encryption_meta.iv is IV for start_key.
// IV for end_key is calculated from (encryption_meta.iv + len(start_key)).
// The field is only used by PD and should be ignored otherwise.
// If encryption_meta is empty (i.e. nil), it means start_key and end_key are unencrypted.
EncryptionMeta *encryptionpb.EncryptionMeta `json:"encryption_meta,omitempty"`
}
HistoryHotRegion wraps hot region info it is storage format of hot_region_storage
type HistoryHotRegions ¶
type HistoryHotRegions struct {
HistoryHotRegion []*HistoryHotRegion `json:"history_hot_region"`
}
HistoryHotRegions wraps historyHotRegion
type HistoryHotRegionsRequest ¶
type HistoryHotRegionsRequest struct {
StartTime int64 `json:"start_time,omitempty"`
EndTime int64 `json:"end_time,omitempty"`
RegionIDs []uint64 `json:"region_ids,omitempty"`
StoreIDs []uint64 `json:"store_ids,omitempty"`
PeerIDs []uint64 `json:"peer_ids,omitempty"`
IsLearners []bool `json:"is_learners,omitempty"`
IsLeaders []bool `json:"is_leaders,omitempty"`
HotRegionTypes []string `json:"hot_region_type,omitempty"`
}
HistoryHotRegionsRequest wrap the request conditions.
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 KeyspaceGCManagementType ¶
type KeyspaceGCManagementType struct {
GCManagementType string `json:"gc_management_type,omitempty"`
}
KeyspaceGCManagementType represents parameters needed to modify the gc management type. If `gc_management_type` is `global_gc`, it means the current keyspace requires a tidb without 'keyspace-name' configured to run a global gc worker to calculate a global gc safe point. If `gc_management_type` is `keyspace_level_gc` it means the current keyspace can calculate gc safe point by its own.
type KeyspaceGCManagementTypeConfig ¶
type KeyspaceGCManagementTypeConfig struct {
Config KeyspaceGCManagementType `json:"config"`
}
KeyspaceGCManagementTypeConfig represents parameters needed to modify target keyspace's configs.
type LabelConstraint ¶
type LabelConstraint struct {
Key string `json:"key,omitempty"`
Op LabelConstraintOp `json:"op,omitempty"`
Values []string `json:"values,omitempty"`
}
LabelConstraint is used to filter store when trying to place peer of a region.
type LabelConstraintOp ¶
type LabelConstraintOp string
LabelConstraintOp defines how a LabelConstraint matches a store. It can be one of 'in', 'notIn', 'exists', or 'notExists'.
const ( // In restricts the store label value should in the value list. // If label does not exist, `in` is always false. In LabelConstraintOp = "in" // NotIn restricts the store label value should not in the value list. // If label does not exist, `notIn` is always true. NotIn LabelConstraintOp = "notIn" // Exists restricts the store should have the label. Exists LabelConstraintOp = "exists" // NotExists restricts the store should not have the label. NotExists LabelConstraintOp = "notExists" )
type LabelRule ¶
type LabelRule struct {
ID string `json:"id"`
Index int `json:"index"`
Labels []RegionLabel `json:"labels"`
RuleType string `json:"rule_type"`
Data any `json:"data"`
}
LabelRule is the rule to assign labels to a region.
type LabelRulePatch ¶
type LabelRulePatch struct {
SetRules []*LabelRule `json:"sets"`
DeleteRules []string `json:"deletes"`
}
LabelRulePatch is the patch to update the label rules.
type ListServiceGCSafepoint ¶
type ListServiceGCSafepoint struct {
ServiceGCSafepoints []*ServiceSafePoint `json:"service_gc_safe_points"`
MinServiceGcSafepoint uint64 `json:"min_service_gc_safe_point,omitempty"`
GCSafePoint uint64 `json:"gc_safe_point"`
}
ListServiceGCSafepoint is the response for list service GC safepoint. NOTE: This type is in sync with pd/server/api/service_gc_safepoint.go
type MembersInfo ¶
type MembersInfo struct {
Header *pdpb.ResponseHeader `json:"header,omitempty"`
Members []*pdpb.Member `json:"members,omitempty"`
Leader *pdpb.Member `json:"leader,omitempty"`
EtcdLeader *pdpb.Member `json:"etcd_leader,omitempty"`
}
MembersInfo is PD members info returned from PD RESTful interface type Members map[string][]*pdpb.Member
type MetaStore ¶
type MetaStore struct {
ID int64 `json:"id"`
Address string `json:"address"`
State int64 `json:"state"`
StateName string `json:"state_name"`
Version string `json:"version"`
Labels []StoreLabel `json:"labels"`
StatusAddress string `json:"status_address"`
GitHash string `json:"git_hash"`
StartTimestamp int64 `json:"start_timestamp"`
}
MetaStore represents the meta information of one store.
type MicroserviceMember ¶
type MicroserviceMember struct {
ServiceAddr string `json:"service-addr"`
Version string `json:"version"`
GitHash string `json:"git-hash"`
DeployPath string `json:"deploy-path"`
StartTimestamp int64 `json:"start-timestamp"`
}
MicroserviceMember is the member info of a microservice.
type PeerRoleType ¶
type PeerRoleType string
PeerRoleType is the expected peer type of the placement rule.
const ( // Voter can either match a leader peer or follower peer Voter PeerRoleType = "voter" // Leader matches a leader. Leader PeerRoleType = "leader" // Follower matches a follower. Follower PeerRoleType = "follower" // Learner matches a learner. Learner PeerRoleType = "learner" )
type RegionDistribution ¶
type RegionDistribution struct {
StoreID uint64 `json:"store_id"`
EngineType string `json:"engine_type"`
RegionLeaderCount int `json:"region_leader_count"`
RegionPeerCount int `json:"region_peer_count"`
ApproximateSize int64 `json:"approximate_size"`
ApproximateKeys int64 `json:"approximate_keys"`
// write
RegionWriteBytes uint64 `json:"region_write_bytes"`
RegionWriteKeys uint64 `json:"region_write_keys"`
RegionWriteQuery uint64 `json:"region_write_query"`
// leader read
RegionLeaderReadBytes uint64 `json:"region_leader_read_bytes"`
RegionLeaderReadKeys uint64 `json:"region_leader_read_keys"`
RegionLeaderReadQuery uint64 `json:"region_leader_read_query"`
// peer read
RegionPeerReadBytes uint64 `json:"region_peer_read_bytes"`
RegionPeerReadKeys uint64 `json:"region_peer_read_keys"`
RegionPeerReadQuery uint64 `json:"region_peer_read_query"`
}
RegionDistribution wraps region distribution info it is storage format of region_distribution_storage
type RegionDistributions ¶
type RegionDistributions struct {
RegionDistributions []*RegionDistribution `json:"region_distribution"`
}
RegionDistributions wraps region distribution info
type RegionEpoch ¶
RegionEpoch stores the information about its epoch.
type RegionInfo ¶
type RegionInfo struct {
ID int64 `json:"id"`
StartKey string `json:"start_key"`
EndKey string `json:"end_key"`
Epoch RegionEpoch `json:"epoch"`
Peers []RegionPeer `json:"peers"`
Leader RegionPeer `json:"leader"`
DownPeers []RegionPeerStat `json:"down_peers"`
PendingPeers []RegionPeer `json:"pending_peers"`
WrittenBytes uint64 `json:"written_bytes"`
ReadBytes uint64 `json:"read_bytes"`
WrittenKeys uint64 `json:"written_keys"`
ReadKeys uint64 `json:"read_keys"`
ApproximateSize int64 `json:"approximate_size"`
ApproximateKeys int64 `json:"approximate_keys"`
ReplicationStatus *ReplicationStatus `json:"replication_status,omitempty"`
}
RegionInfo stores the information of one region.
func (*RegionInfo) GetEndKey ¶
func (r *RegionInfo) GetEndKey() string
GetEndKey gets the end key of the region.
func (*RegionInfo) GetStartKey ¶
func (r *RegionInfo) GetStartKey() string
GetStartKey gets the start key of the region.
type RegionLabel ¶
type RegionLabel struct {
Key string `json:"key"`
Value string `json:"value"`
TTL string `json:"ttl,omitempty"`
StartAt string `json:"start_at,omitempty"`
}
RegionLabel is the label of a region.
type RegionPeer ¶
type RegionPeer struct {
ID int64 `json:"id"`
StoreID int64 `json:"store_id"`
IsLearner bool `json:"is_learner"`
}
RegionPeer stores information of one peer.
type RegionPeerStat ¶
type RegionPeerStat struct {
Peer RegionPeer `json:"peer"`
DownSec int64 `json:"down_seconds"`
}
RegionPeerStat stores one field `DownSec` which indicates how long it's down than `RegionPeer`.
type RegionStats ¶
type RegionStats struct {
Count int `json:"count"`
EmptyCount int `json:"empty_count"`
StorageSize int64 `json:"storage_size"`
StorageKeys int64 `json:"storage_keys"`
StoreLeaderCount map[uint64]int `json:"store_leader_count"`
StorePeerCount map[uint64]int `json:"store_peer_count"`
StorePeerSize map[uint64]int64 `json:"store_peer_size,omitempty"`
StorePeerKeys map[uint64]int64 `json:"store_peer_keys,omitempty"`
StoreWriteBytes map[uint64]uint64 `json:"store_write_bytes,omitempty"`
StoreWriteKeys map[uint64]uint64 `json:"store_write_keys,omitempty"`
StoreWriteQuery map[uint64]uint64 `json:"store_write_query,omitempty"`
StoreLeaderReadBytes map[uint64]uint64 `json:"store_leader_read_bytes,omitempty"`
StoreLeaderReadKeys map[uint64]uint64 `json:"store_leader_read_keys,omitempty"`
StoreLeaderReadQuery map[uint64]uint64 `json:"store_leader_read_query,omitempty"`
StorePeerReadBytes map[uint64]uint64 `json:"store_peer_read_bytes,omitempty"`
StorePeerReadKeys map[uint64]uint64 `json:"store_peer_read_keys,omitempty"`
StorePeerReadQuery map[uint64]uint64 `json:"store_peer_read_query,omitempty"`
StoreEngine map[uint64]string `json:"store_engine,omitempty"`
}
RegionStats stores the statistics of regions.
type RegionsInfo ¶
type RegionsInfo struct {
Count int64 `json:"count"`
Regions []RegionInfo `json:"regions"`
}
RegionsInfo stores the information of regions.
func (*RegionsInfo) Merge ¶
func (ri *RegionsInfo) Merge(other *RegionsInfo) *RegionsInfo
Merge merges two RegionsInfo together and returns a new one.
type ReplicationStatus ¶
ReplicationStatus represents the replication mode status of the region.
type Rule ¶
type Rule struct {
GroupID string `json:"group_id"` // mark the source that add the rule
ID string `json:"id"` // unique ID within a group
Index int `json:"index,omitempty"` // rule apply order in a group, rule with less ID is applied first when indexes are equal
Override bool `json:"override,omitempty"` // when it is true, all rules with less indexes are disabled
StartKey []byte `json:"-"` // range start key
StartKeyHex string `json:"start_key"` // hex format start key, for marshal/unmarshal
EndKey []byte `json:"-"` // range end key
EndKeyHex string `json:"end_key"` // hex format end key, for marshal/unmarshal
Role PeerRoleType `json:"role"` // expected role of the peers
IsWitness bool `json:"is_witness"` // when it is true, it means the role is also a witness
Count int `json:"count"` // expected count of the peers
LabelConstraints []LabelConstraint `json:"label_constraints,omitempty"` // used to select stores to place peers
LocationLabels []string `json:"location_labels,omitempty"` // used to make peers isolated physically
IsolationLevel string `json:"isolation_level,omitempty"` // used to isolate replicas explicitly and forcibly
Version uint64 `json:"version,omitempty"` // only set at runtime, add 1 each time rules updated, begin from 0.
CreateTimestamp uint64 `json:"create_timestamp,omitempty"` // only set at runtime, recorded rule create timestamp
}
Rule is the placement rule that can be checked against a region. When applying rules (apply means schedule regions to match selected rules), the apply order is defined by the tuple [GroupIndex, GroupID, Index, ID].
func (*Rule) MarshalJSON ¶
MarshalJSON implements `json.Marshaler` interface to make sure we could set the correct start/end key.
func (*Rule) UnmarshalJSON ¶
UnmarshalJSON implements `json.Unmarshaler` interface to make sure we could get the correct start/end key.
type RuleGroup ¶
type RuleGroup struct {
ID string `json:"id,omitempty"`
Index int `json:"index,omitempty"`
Override bool `json:"override,omitempty"`
}
RuleGroup defines properties of a rule group.
type RuleOp ¶
type RuleOp struct {
*Rule // information of the placement rule to add/delete the operation type
Action RuleOpType `json:"action"`
DeleteByIDPrefix bool `json:"delete_by_id_prefix"` // if action == delete, delete by the prefix of id
}
RuleOp is for batching placement rule actions. The action type is distinguished by the field `Action`.
func (*RuleOp) MarshalJSON ¶
MarshalJSON implements `json.Marshaler` interface to make sure we could set the correct start/end key.
func (*RuleOp) UnmarshalJSON ¶
UnmarshalJSON implements `json.Unmarshaler` interface to make sure we could get the correct start/end key.
type RuleOpType ¶
type RuleOpType string
RuleOpType indicates the operation type
const ( // RuleOpAdd a placement rule, only need to specify the field *Rule RuleOpAdd RuleOpType = "add" // RuleOpDel a placement rule, only need to specify the field `GroupID`, `ID`, `MatchID` RuleOpDel RuleOpType = "del" )
type ServiceSafePoint ¶
type ServiceSafePoint struct {
ServiceID string `json:"service_id"`
ExpiredAt int64 `json:"expired_at"`
SafePoint uint64 `json:"safe_point"`
}
ServiceSafePoint is the safepoint for a specific service NOTE: This type is in sync with pd/pkg/storage/endpoint/gc_safe_point.go
type State ¶
type State struct {
BuildTS string `json:"build_ts"`
Version string `json:"version"`
GitHash string `json:"git_hash"`
StartTimestamp int64 `json:"start_timestamp"`
KernelType string `json:"kernel_type"`
}
State is the status of PD server. NOTE: This type sync with https://github.com/tikv/pd/blob/1d77b25656bc18e1f5aa82337d4ab62a34b10087/pkg/versioninfo/versioninfo.go#L29
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.
type StoreHotPeersStat ¶
type StoreHotPeersStat map[uint64]*HotPeersStat
StoreHotPeersStat is used to record the hot region statistics group by store.
type StoreInfo ¶
type StoreInfo struct {
Store MetaStore `json:"store"`
Status StoreStatus `json:"status"`
}
StoreInfo represents the information of one TiKV/TiFlash store.
type StoreLabel ¶
StoreLabel stores the information of one store label.
type StoreStatus ¶
type StoreStatus struct {
Capacity string `json:"capacity"`
Available string `json:"available"`
LeaderCount int64 `json:"leader_count"`
LeaderWeight float64 `json:"leader_weight"`
LeaderScore float64 `json:"leader_score"`
LeaderSize int64 `json:"leader_size"`
RegionCount int64 `json:"region_count"`
RegionWeight float64 `json:"region_weight"`
RegionScore float64 `json:"region_score"`
RegionSize int64 `json:"region_size"`
StartTS time.Time `json:"start_ts"`
LastHeartbeatTS time.Time `json:"last_heartbeat_ts"`
Uptime string `json:"uptime"`
}
StoreStatus stores the detail information of one store.
type StoresInfo ¶
StoresInfo represents the information of all TiKV/TiFlash stores.
type UpdateAffinityGroupPeersRequest ¶
type UpdateAffinityGroupPeersRequest struct {
LeaderStoreID uint64 `json:"leader_store_id"`
VoterStoreIDs []uint64 `json:"voter_store_ids"`
}
UpdateAffinityGroupPeersRequest defines the body for updating peer distribution of an affinity group.