Documentation
¶
Index ¶
- func IsDefaultScheduler(typ string) bool
- func IsDeprecated(config string) bool
- func NormalizeReplicationMode(m string) string
- func ParseUrls(s string) ([]url.URL, error)
- func ValidateLabels(labels []*metapb.StoreLabel) error
- type Config
- func (c *Config) Adjust(meta *toml.MetaData) error
- func (c *Config) Clone() *Config
- func (c *Config) GenEmbedEtcdConfig() (*embed.Config, error)
- func (c *Config) GetConfigFile() string
- func (c *Config) GetZapLogProperties() *log.ZapProperties
- func (c *Config) GetZapLogger() *zap.Logger
- func (c *Config) Parse(arguments []string) error
- func (c *Config) RewriteFile(new *Config) error
- func (c *Config) SetupLogger() error
- func (c *Config) String() string
- func (c *Config) Validate() error
- type DRAutoSyncReplicationConfig
- type DashboardConfig
- type LabelPropertyConfig
- type PDServerConfig
- type PersistOptions
- func (o *PersistOptions) AddSchedulerCfg(tp string, args []string)
- func (o *PersistOptions) CASClusterVersion(old, new *semver.Version) bool
- func (o *PersistOptions) CheckLabelProperty(typ string, labels []*metapb.StoreLabel) bool
- func (o *PersistOptions) DeleteLabelProperty(typ, labelKey, labelValue string)
- func (o *PersistOptions) GetClusterVersion() *semver.Version
- func (o *PersistOptions) GetDashboardAddress() string
- func (o *PersistOptions) GetHighSpaceRatio() float64
- func (o *PersistOptions) GetHotRegionCacheHitsThreshold() int
- func (o *PersistOptions) GetHotRegionScheduleLimit() uint64
- func (o *PersistOptions) GetKeyType() core.KeyType
- func (o *PersistOptions) GetLabelPropertyConfig() LabelPropertyConfig
- func (o *PersistOptions) GetLeaderScheduleLimit() uint64
- func (o *PersistOptions) GetLeaderSchedulePolicy() core.SchedulePolicy
- func (o *PersistOptions) GetLocationLabels() []string
- func (o *PersistOptions) GetLowSpaceRatio() float64
- func (o *PersistOptions) GetMaxMergeRegionKeys() uint64
- func (o *PersistOptions) GetMaxMergeRegionSize() uint64
- func (o *PersistOptions) GetMaxPendingPeerCount() uint64
- func (o *PersistOptions) GetMaxReplicas() int
- func (o *PersistOptions) GetMaxResetTSGap() time.Duration
- func (o *PersistOptions) GetMaxSnapshotCount() uint64
- func (o *PersistOptions) GetMaxStoreDownTime() time.Duration
- func (o *PersistOptions) GetMergeScheduleLimit() uint64
- func (o *PersistOptions) GetPDServerConfig() *PDServerConfig
- func (o *PersistOptions) GetPatrolRegionInterval() time.Duration
- func (o *PersistOptions) GetRegionScheduleLimit() uint64
- func (o *PersistOptions) GetReplicaScheduleLimit() uint64
- func (o *PersistOptions) GetReplicationConfig() *ReplicationConfig
- func (o *PersistOptions) GetReplicationModeConfig() *ReplicationModeConfig
- func (o *PersistOptions) GetScheduleConfig() *ScheduleConfig
- func (o *PersistOptions) GetSchedulerMaxWaitingOperator() uint64
- func (o *PersistOptions) GetSchedulers() SchedulerConfigs
- func (o *PersistOptions) GetSplitMergeInterval() time.Duration
- func (o *PersistOptions) GetStoreBalanceRate() float64
- func (o *PersistOptions) GetStoreLimitMode() string
- func (o *PersistOptions) GetStrictlyMatchLabel() bool
- func (o *PersistOptions) GetTolerantSizeRatio() float64
- func (o *PersistOptions) IsCrossTableMergeEnabled() bool
- func (o *PersistOptions) IsDebugMetricsEnabled() bool
- func (o *PersistOptions) IsLocationReplacementEnabled() bool
- func (o *PersistOptions) IsMakeUpReplicaEnabled() bool
- func (o *PersistOptions) IsOneWayMergeEnabled() bool
- func (o *PersistOptions) IsPlacementRulesEnabled() bool
- func (o *PersistOptions) IsRemoveDownReplicaEnabled() bool
- func (o *PersistOptions) IsRemoveExtraReplicaEnabled() bool
- func (o *PersistOptions) IsReplaceOfflineReplicaEnabled() bool
- func (o *PersistOptions) IsUseRegionStorage() bool
- func (o *PersistOptions) Persist(storage *core.Storage) error
- func (o *PersistOptions) Reload(storage *core.Storage) error
- func (o *PersistOptions) RemoveSchedulerCfg(ctx context.Context, name string) error
- func (o *PersistOptions) SetClusterVersion(v *semver.Version)
- func (o *PersistOptions) SetLabelProperty(typ, labelKey, labelValue string)
- func (o *PersistOptions) SetLabelPropertyConfig(cfg LabelPropertyConfig)
- func (o *PersistOptions) SetMaxReplicas(replicas int)
- func (o *PersistOptions) SetPDServerConfig(cfg *PDServerConfig)
- func (o *PersistOptions) SetReplicationConfig(cfg *ReplicationConfig)
- func (o *PersistOptions) SetReplicationModeConfig(cfg *ReplicationModeConfig)
- func (o *PersistOptions) SetScheduleConfig(cfg *ScheduleConfig)
- func (o *PersistOptions) SetSplitMergeInterval(splitMergeInterval time.Duration)
- type ReplicationConfig
- type ReplicationModeConfig
- type ScheduleConfig
- type SchedulerConfig
- type SchedulerConfigs
- type StoreLabel
Constants ¶
This section is empty.
Variables ¶
This section is empty.
Functions ¶
func IsDefaultScheduler ¶
IsDefaultScheduler checks whether the scheduler is enable by default.
func IsDeprecated ¶
IsDeprecated returns if a config is deprecated.
func NormalizeReplicationMode ¶
NormalizeReplicationMode converts user's input mode to internal use. It returns "" if failed to convert.
func ValidateLabels ¶
func ValidateLabels(labels []*metapb.StoreLabel) error
ValidateLabels checks the legality of the labels.
Types ¶
type Config ¶
type Config struct {
Version bool `json:"-"`
ConfigCheck bool `json:"-"`
ClientUrls string `toml:"client-urls" json:"client-urls"`
PeerUrls string `toml:"peer-urls" json:"peer-urls"`
AdvertiseClientUrls string `toml:"advertise-client-urls" json:"advertise-client-urls"`
AdvertisePeerUrls string `toml:"advertise-peer-urls" json:"advertise-peer-urls"`
Name string `toml:"name" json:"name"`
DataDir string `toml:"data-dir" json:"data-dir"`
ForceNewCluster bool `json:"force-new-cluster"`
EnableGRPCGateway bool `json:"enable-grpc-gateway"`
InitialCluster string `toml:"initial-cluster" json:"initial-cluster"`
InitialClusterState string `toml:"initial-cluster-state" json:"initial-cluster-state"`
// Join to an existing pd cluster, a string of endpoints.
Join string `toml:"join" json:"join"`
// LeaderLease time, if leader doesn't update its TTL
// in etcd after lease time, etcd will expire the leader key
// and other servers can campaign the leader again.
// Etcd only supports seconds TTL, so here is second too.
LeaderLease int64 `toml:"lease" json:"lease"`
// Log related config.
Log log.Config `toml:"log" json:"log"`
// Backward compatibility.
LogFileDeprecated string `toml:"log-file" json:"log-file,omitempty"`
LogLevelDeprecated string `toml:"log-level" json:"log-level,omitempty"`
// TsoSaveInterval is the interval to save timestamp.
TsoSaveInterval typeutil.Duration `toml:"tso-save-interval" json:"tso-save-interval"`
Metric metricutil.MetricConfig `toml:"metric" json:"metric"`
Schedule ScheduleConfig `toml:"schedule" json:"schedule"`
Replication ReplicationConfig `toml:"replication" json:"replication"`
PDServerCfg PDServerConfig `toml:"pd-server" json:"pd-server"`
ClusterVersion semver.Version `toml:"cluster-version" json:"cluster-version"`
// QuotaBackendBytes Raise alarms when backend size exceeds the given quota. 0 means use the default quota.
// the default size is 2GB, the maximum is 8GB.
QuotaBackendBytes typeutil.ByteSize `toml:"quota-backend-bytes" json:"quota-backend-bytes"`
// AutoCompactionMode is either 'periodic' or 'revision'. The default value is 'periodic'.
AutoCompactionMode string `toml:"auto-compaction-mode" json:"auto-compaction-mode"`
// AutoCompactionRetention is either duration string with time unit
// (e.g. '5m' for 5-minute), or revision unit (e.g. '5000').
// If no time unit is provided and compaction mode is 'periodic',
// the unit defaults to hour. For example, '5' translates into 5-hour.
// The default retention is 1 hour.
// Before etcd v3.3.x, the type of retention is int. We add 'v2' suffix to make it backward compatible.
AutoCompactionRetention string `toml:"auto-compaction-retention" json:"auto-compaction-retention-v2"`
// TickInterval is the interval for etcd Raft tick.
TickInterval typeutil.Duration `toml:"tick-interval"`
// ElectionInterval is the interval for etcd Raft election.
ElectionInterval typeutil.Duration `toml:"election-interval"`
// Prevote is true to enable Raft Pre-Vote.
// If enabled, Raft runs an additional election phase
// to check whether it would get enough votes to win
// an election, thus minimizing disruptions.
PreVote bool `toml:"enable-prevote"`
Security grpcutil.SecurityConfig `toml:"security" json:"security"`
LabelProperty LabelPropertyConfig `toml:"label-property" json:"label-property"`
// For all warnings during parsing.
WarningMsgs []string
DisableStrictReconfigCheck bool
HeartbeatStreamBindInterval typeutil.Duration
LeaderPriorityCheckInterval typeutil.Duration
Dashboard DashboardConfig `toml:"dashboard" json:"dashboard"`
ReplicationMode ReplicationModeConfig `toml:"replication-mode" json:"replication-mode"`
// contains filtered or unexported fields
}
Config is the pd server configuration.
func (*Config) GenEmbedEtcdConfig ¶
GenEmbedEtcdConfig generates a configuration for embedded etcd.
func (*Config) GetConfigFile ¶
GetConfigFile gets the config file.
func (*Config) GetZapLogProperties ¶
func (c *Config) GetZapLogProperties() *log.ZapProperties
GetZapLogProperties gets properties of the zap logger.
func (*Config) GetZapLogger ¶
GetZapLogger gets the created zap logger.
func (*Config) RewriteFile ¶
RewriteFile rewrites the config file after updating the config.
type DRAutoSyncReplicationConfig ¶
type DRAutoSyncReplicationConfig struct {
LabelKey string `toml:"label-key" json:"label-key"`
Primary string `toml:"primary" json:"primary"`
DR string `toml:"dr" json:"dr"`
PrimaryReplicas int `toml:"primary-replicas" json:"primary-replicas"`
DRReplicas int `toml:"dr-replicas" json:"dr-replicas"`
WaitStoreTimeout typeutil.Duration `toml:"wait-store-timeout" json:"wait-store-timeout"`
WaitSyncTimeout typeutil.Duration `toml:"wait-sync-timeout" json:"wait-sync-timeout"`
}
DRAutoSyncReplicationConfig is the configuration for auto sync mode between 2 data centers.
type DashboardConfig ¶
type DashboardConfig struct {
TiDBCAPath string `toml:"tidb-cacert-path" json:"tidb_cacert_path"`
TiDBCertPath string `toml:"tidb-cert-path" json:"tidb_cert_path"`
TiDBKeyPath string `toml:"tidb-key-path" json:"tidb_key_path"`
PublicPathPrefix string `toml:"public-path-prefix" json:"public_path_prefix"`
}
DashboardConfig is the configuration for tidb-dashboard.
func (DashboardConfig) ToTiDBTLSConfig ¶
func (c DashboardConfig) ToTiDBTLSConfig() (*tls.Config, error)
ToTiDBTLSConfig generates tls config for connecting to TiDB, used by tidb-dashboard.
type LabelPropertyConfig ¶
type LabelPropertyConfig map[string][]StoreLabel
LabelPropertyConfig is the config section to set properties to store labels.
func (LabelPropertyConfig) Clone ¶
func (c LabelPropertyConfig) Clone() LabelPropertyConfig
Clone returns a cloned label property configuration.
type PDServerConfig ¶
type PDServerConfig struct {
// UseRegionStorage enables the independent region storage.
UseRegionStorage bool `toml:"use-region-storage" json:"use-region-storage,string"`
// MaxResetTSGap is the max gap to reset the tso.
MaxResetTSGap typeutil.Duration `toml:"max-gap-reset-ts" json:"max-gap-reset-ts"`
// KeyType is option to specify the type of keys.
// There are some types supported: ["table", "raw", "txn"], default: "table"
KeyType string `toml:"key-type" json:"key-type"`
// RuntimeServices is the running the running extension services.
RuntimeServices typeutil.StringSlice `toml:"runtime-services" json:"runtime-services"`
// MetricStorage is the cluster metric storage.
// Currently we use prometheus as metric storage, we may use PD/TiKV as metric storage later.
MetricStorage string `toml:"metric-storage" json:"metric-storage"`
// There are some values supported: "auto", "none", or a specific address, default: "auto"
DashboardAddress string `toml:"dashboard-address" json:"dashboard-address"`
}
PDServerConfig is the configuration for pd server.
func (*PDServerConfig) Clone ¶
func (c *PDServerConfig) Clone() *PDServerConfig
Clone retruns a cloned PD server config.
type PersistOptions ¶
type PersistOptions struct {
// contains filtered or unexported fields
}
PersistOptions wraps all configurations that need to persist to storage and allows to access them safely.
func NewPersistOptions ¶
func NewPersistOptions(cfg *Config) *PersistOptions
NewPersistOptions creates a new PersistOptions instance.
func (*PersistOptions) AddSchedulerCfg ¶
func (o *PersistOptions) AddSchedulerCfg(tp string, args []string)
AddSchedulerCfg adds the scheduler configurations.
func (*PersistOptions) CASClusterVersion ¶
func (o *PersistOptions) CASClusterVersion(old, new *semver.Version) bool
CASClusterVersion sets the cluster version.
func (*PersistOptions) CheckLabelProperty ¶
func (o *PersistOptions) CheckLabelProperty(typ string, labels []*metapb.StoreLabel) bool
CheckLabelProperty checks the label property.
func (*PersistOptions) DeleteLabelProperty ¶
func (o *PersistOptions) DeleteLabelProperty(typ, labelKey, labelValue string)
DeleteLabelProperty deletes the label property.
func (*PersistOptions) GetClusterVersion ¶
func (o *PersistOptions) GetClusterVersion() *semver.Version
GetClusterVersion returns the cluster version.
func (*PersistOptions) GetDashboardAddress ¶
func (o *PersistOptions) GetDashboardAddress() string
GetDashboardAddress gets dashboard address.
func (*PersistOptions) GetHighSpaceRatio ¶
func (o *PersistOptions) GetHighSpaceRatio() float64
GetHighSpaceRatio returns the high space ratio.
func (*PersistOptions) GetHotRegionCacheHitsThreshold ¶
func (o *PersistOptions) GetHotRegionCacheHitsThreshold() int
GetHotRegionCacheHitsThreshold is a threshold to decide if a region is hot.
func (*PersistOptions) GetHotRegionScheduleLimit ¶
func (o *PersistOptions) GetHotRegionScheduleLimit() uint64
GetHotRegionScheduleLimit returns the limit for hot region schedule.
func (*PersistOptions) GetKeyType ¶
func (o *PersistOptions) GetKeyType() core.KeyType
GetKeyType is to get key type.
func (*PersistOptions) GetLabelPropertyConfig ¶
func (o *PersistOptions) GetLabelPropertyConfig() LabelPropertyConfig
GetLabelPropertyConfig returns the label property.
func (*PersistOptions) GetLeaderScheduleLimit ¶
func (o *PersistOptions) GetLeaderScheduleLimit() uint64
GetLeaderScheduleLimit returns the limit for leader schedule.
func (*PersistOptions) GetLeaderSchedulePolicy ¶
func (o *PersistOptions) GetLeaderSchedulePolicy() core.SchedulePolicy
GetLeaderSchedulePolicy is to get leader schedule policy.
func (*PersistOptions) GetLocationLabels ¶
func (o *PersistOptions) GetLocationLabels() []string
GetLocationLabels returns the location labels for each region.
func (*PersistOptions) GetLowSpaceRatio ¶
func (o *PersistOptions) GetLowSpaceRatio() float64
GetLowSpaceRatio returns the low space ratio.
func (*PersistOptions) GetMaxMergeRegionKeys ¶
func (o *PersistOptions) GetMaxMergeRegionKeys() uint64
GetMaxMergeRegionKeys returns the max number of keys.
func (*PersistOptions) GetMaxMergeRegionSize ¶
func (o *PersistOptions) GetMaxMergeRegionSize() uint64
GetMaxMergeRegionSize returns the max region size.
func (*PersistOptions) GetMaxPendingPeerCount ¶
func (o *PersistOptions) GetMaxPendingPeerCount() uint64
GetMaxPendingPeerCount returns the number of the max pending peers.
func (*PersistOptions) GetMaxReplicas ¶
func (o *PersistOptions) GetMaxReplicas() int
GetMaxReplicas returns the number of replicas for each region.
func (*PersistOptions) GetMaxResetTSGap ¶
func (o *PersistOptions) GetMaxResetTSGap() time.Duration
GetMaxResetTSGap gets the max gap to reset the tso.
func (*PersistOptions) GetMaxSnapshotCount ¶
func (o *PersistOptions) GetMaxSnapshotCount() uint64
GetMaxSnapshotCount returns the number of the max snapshot which is allowed to send.
func (*PersistOptions) GetMaxStoreDownTime ¶
func (o *PersistOptions) GetMaxStoreDownTime() time.Duration
GetMaxStoreDownTime returns the max down time of a store.
func (*PersistOptions) GetMergeScheduleLimit ¶
func (o *PersistOptions) GetMergeScheduleLimit() uint64
GetMergeScheduleLimit returns the limit for merge schedule.
func (*PersistOptions) GetPDServerConfig ¶
func (o *PersistOptions) GetPDServerConfig() *PDServerConfig
GetPDServerConfig returns pd server configurations.
func (*PersistOptions) GetPatrolRegionInterval ¶
func (o *PersistOptions) GetPatrolRegionInterval() time.Duration
GetPatrolRegionInterval returns the interval of patroling region.
func (*PersistOptions) GetRegionScheduleLimit ¶
func (o *PersistOptions) GetRegionScheduleLimit() uint64
GetRegionScheduleLimit returns the limit for region schedule.
func (*PersistOptions) GetReplicaScheduleLimit ¶
func (o *PersistOptions) GetReplicaScheduleLimit() uint64
GetReplicaScheduleLimit returns the limit for replica schedule.
func (*PersistOptions) GetReplicationConfig ¶
func (o *PersistOptions) GetReplicationConfig() *ReplicationConfig
GetReplicationConfig returns replication configurations.
func (*PersistOptions) GetReplicationModeConfig ¶
func (o *PersistOptions) GetReplicationModeConfig() *ReplicationModeConfig
GetReplicationModeConfig returns the replication mode config.
func (*PersistOptions) GetScheduleConfig ¶
func (o *PersistOptions) GetScheduleConfig() *ScheduleConfig
GetScheduleConfig returns scheduling configurations.
func (*PersistOptions) GetSchedulerMaxWaitingOperator ¶
func (o *PersistOptions) GetSchedulerMaxWaitingOperator() uint64
GetSchedulerMaxWaitingOperator returns the number of the max waiting operators.
func (*PersistOptions) GetSchedulers ¶
func (o *PersistOptions) GetSchedulers() SchedulerConfigs
GetSchedulers gets the scheduler configurations.
func (*PersistOptions) GetSplitMergeInterval ¶
func (o *PersistOptions) GetSplitMergeInterval() time.Duration
GetSplitMergeInterval returns the interval between finishing split and starting to merge.
func (*PersistOptions) GetStoreBalanceRate ¶
func (o *PersistOptions) GetStoreBalanceRate() float64
GetStoreBalanceRate returns the balance rate of a store.
func (*PersistOptions) GetStoreLimitMode ¶
func (o *PersistOptions) GetStoreLimitMode() string
GetStoreLimitMode returns the limit mode of store.
func (*PersistOptions) GetStrictlyMatchLabel ¶
func (o *PersistOptions) GetStrictlyMatchLabel() bool
GetStrictlyMatchLabel returns whether check label strict.
func (*PersistOptions) GetTolerantSizeRatio ¶
func (o *PersistOptions) GetTolerantSizeRatio() float64
GetTolerantSizeRatio gets the tolerant size ratio.
func (*PersistOptions) IsCrossTableMergeEnabled ¶
func (o *PersistOptions) IsCrossTableMergeEnabled() bool
IsCrossTableMergeEnabled returns if across table merge is enabled.
func (*PersistOptions) IsDebugMetricsEnabled ¶
func (o *PersistOptions) IsDebugMetricsEnabled() bool
IsDebugMetricsEnabled mocks method
func (*PersistOptions) IsLocationReplacementEnabled ¶
func (o *PersistOptions) IsLocationReplacementEnabled() bool
IsLocationReplacementEnabled returns if location replace is enabled.
func (*PersistOptions) IsMakeUpReplicaEnabled ¶
func (o *PersistOptions) IsMakeUpReplicaEnabled() bool
IsMakeUpReplicaEnabled returns if make up replica is enabled.
func (*PersistOptions) IsOneWayMergeEnabled ¶
func (o *PersistOptions) IsOneWayMergeEnabled() bool
IsOneWayMergeEnabled returns if a region can only be merged into the next region of it.
func (*PersistOptions) IsPlacementRulesEnabled ¶
func (o *PersistOptions) IsPlacementRulesEnabled() bool
IsPlacementRulesEnabled returns if the placement rules is enabled.
func (*PersistOptions) IsRemoveDownReplicaEnabled ¶
func (o *PersistOptions) IsRemoveDownReplicaEnabled() bool
IsRemoveDownReplicaEnabled returns if remove down replica is enabled.
func (*PersistOptions) IsRemoveExtraReplicaEnabled ¶
func (o *PersistOptions) IsRemoveExtraReplicaEnabled() bool
IsRemoveExtraReplicaEnabled returns if remove extra replica is enabled.
func (*PersistOptions) IsReplaceOfflineReplicaEnabled ¶
func (o *PersistOptions) IsReplaceOfflineReplicaEnabled() bool
IsReplaceOfflineReplicaEnabled returns if replace offline replica is enabled.
func (*PersistOptions) IsUseRegionStorage ¶
func (o *PersistOptions) IsUseRegionStorage() bool
IsUseRegionStorage returns if the independent region storage is enabled.
func (*PersistOptions) Persist ¶
func (o *PersistOptions) Persist(storage *core.Storage) error
Persist saves the configuration to the storage.
func (*PersistOptions) Reload ¶
func (o *PersistOptions) Reload(storage *core.Storage) error
Reload reloads the configuration from the storage.
func (*PersistOptions) RemoveSchedulerCfg ¶
func (o *PersistOptions) RemoveSchedulerCfg(ctx context.Context, name string) error
RemoveSchedulerCfg removes the scheduler configurations.
func (*PersistOptions) SetClusterVersion ¶
func (o *PersistOptions) SetClusterVersion(v *semver.Version)
SetClusterVersion sets the cluster version.
func (*PersistOptions) SetLabelProperty ¶
func (o *PersistOptions) SetLabelProperty(typ, labelKey, labelValue string)
SetLabelProperty sets the label property.
func (*PersistOptions) SetLabelPropertyConfig ¶
func (o *PersistOptions) SetLabelPropertyConfig(cfg LabelPropertyConfig)
SetLabelPropertyConfig sets the label property configuration.
func (*PersistOptions) SetMaxReplicas ¶
func (o *PersistOptions) SetMaxReplicas(replicas int)
SetMaxReplicas sets the number of replicas for each region.
func (*PersistOptions) SetPDServerConfig ¶
func (o *PersistOptions) SetPDServerConfig(cfg *PDServerConfig)
SetPDServerConfig sets the PD configuration.
func (*PersistOptions) SetReplicationConfig ¶
func (o *PersistOptions) SetReplicationConfig(cfg *ReplicationConfig)
SetReplicationConfig sets the PD replication configuration.
func (*PersistOptions) SetReplicationModeConfig ¶
func (o *PersistOptions) SetReplicationModeConfig(cfg *ReplicationModeConfig)
SetReplicationModeConfig sets the replication mode config.
func (*PersistOptions) SetScheduleConfig ¶
func (o *PersistOptions) SetScheduleConfig(cfg *ScheduleConfig)
SetScheduleConfig sets the PD scheduling configuration.
func (*PersistOptions) SetSplitMergeInterval ¶
func (o *PersistOptions) SetSplitMergeInterval(splitMergeInterval time.Duration)
SetSplitMergeInterval to set the interval between finishing split and starting to merge. It's only used to test.
type ReplicationConfig ¶
type ReplicationConfig struct {
// MaxReplicas is the number of replicas for each region.
MaxReplicas uint64 `toml:"max-replicas" json:"max-replicas"`
// The label keys specified the location of a store.
// The placement priorities is implied by the order of label keys.
// For example, ["zone", "rack"] means that we should place replicas to
// different zones first, then to different racks if we don't have enough zones.
LocationLabels typeutil.StringSlice `toml:"location-labels" json:"location-labels"`
// StrictlyMatchLabel strictly checks if the label of TiKV is matched with LocationLabels.
StrictlyMatchLabel bool `toml:"strictly-match-label" json:"strictly-match-label,string"`
// When PlacementRules feature is enabled. MaxReplicas and LocationLabels are not uesd any more.
EnablePlacementRules bool `toml:"enable-placement-rules" json:"enable-placement-rules,string"`
}
ReplicationConfig is the replication configuration.
func (*ReplicationConfig) Validate ¶
func (c *ReplicationConfig) Validate() error
Validate is used to validate if some replication configurations are right.
type ReplicationModeConfig ¶
type ReplicationModeConfig struct {
ReplicationMode string `toml:"replication-mode" json:"replication-mode"` // can be 'dr-auto-sync' or 'majority', default value is 'majority'
DRAutoSync DRAutoSyncReplicationConfig `toml:"dr-auto-sync" json:"dr-auto-sync"` // used when ReplicationMode is 'dr-auto-sync'
}
ReplicationModeConfig is the configuration for the replication policy.
func (*ReplicationModeConfig) Clone ¶
func (c *ReplicationModeConfig) Clone() *ReplicationModeConfig
Clone returns a copy of replication mode config.
type ScheduleConfig ¶
type ScheduleConfig struct {
// If the snapshot count of one store is greater than this value,
// it will never be used as a source or target store.
MaxSnapshotCount uint64 `toml:"max-snapshot-count" json:"max-snapshot-count"`
MaxPendingPeerCount uint64 `toml:"max-pending-peer-count" json:"max-pending-peer-count"`
// If both the size of region is smaller than MaxMergeRegionSize
// and the number of rows in region is smaller than MaxMergeRegionKeys,
// it will try to merge with adjacent regions.
MaxMergeRegionSize uint64 `toml:"max-merge-region-size" json:"max-merge-region-size"`
MaxMergeRegionKeys uint64 `toml:"max-merge-region-keys" json:"max-merge-region-keys"`
// SplitMergeInterval is the minimum interval time to permit merge after split.
SplitMergeInterval typeutil.Duration `toml:"split-merge-interval" json:"split-merge-interval"`
// EnableOneWayMerge is the option to enable one way merge. This means a Region can only be merged into the next region of it.
EnableOneWayMerge bool `toml:"enable-one-way-merge" json:"enable-one-way-merge,string"`
// EnableCrossTableMerge is the option to enable cross table merge. This means two Regions can be merged with different table IDs.
// This option only works when key type is "table".
EnableCrossTableMerge bool `toml:"enable-cross-table-merge" json:"enable-cross-table-merge,string"`
// PatrolRegionInterval is the interval for scanning region during patrol.
PatrolRegionInterval typeutil.Duration `toml:"patrol-region-interval" json:"patrol-region-interval"`
// MaxStoreDownTime is the max duration after which
// a store will be considered to be down if it hasn't reported heartbeats.
MaxStoreDownTime typeutil.Duration `toml:"max-store-down-time" json:"max-store-down-time"`
// LeaderScheduleLimit is the max coexist leader schedules.
LeaderScheduleLimit uint64 `toml:"leader-schedule-limit" json:"leader-schedule-limit"`
// LeaderSchedulePolicy is the option to balance leader, there are some policies supported: ["count", "size"], default: "count"
LeaderSchedulePolicy string `toml:"leader-schedule-policy" json:"leader-schedule-policy"`
// RegionScheduleLimit is the max coexist region schedules.
RegionScheduleLimit uint64 `toml:"region-schedule-limit" json:"region-schedule-limit"`
// ReplicaScheduleLimit is the max coexist replica schedules.
ReplicaScheduleLimit uint64 `toml:"replica-schedule-limit" json:"replica-schedule-limit"`
// MergeScheduleLimit is the max coexist merge schedules.
MergeScheduleLimit uint64 `toml:"merge-schedule-limit" json:"merge-schedule-limit"`
// HotRegionScheduleLimit is the max coexist hot region schedules.
HotRegionScheduleLimit uint64 `toml:"hot-region-schedule-limit" json:"hot-region-schedule-limit"`
// HotRegionCacheHitThreshold is the cache hits threshold of the hot region.
// If the number of times a region hits the hot cache is greater than this
// threshold, it is considered a hot region.
HotRegionCacheHitsThreshold uint64 `toml:"hot-region-cache-hits-threshold" json:"hot-region-cache-hits-threshold"`
// StoreBalanceRate is the maximum of balance rate for each store.
StoreBalanceRate float64 `toml:"store-balance-rate" json:"store-balance-rate"`
// TolerantSizeRatio is the ratio of buffer size for balance scheduler.
TolerantSizeRatio float64 `toml:"tolerant-size-ratio" json:"tolerant-size-ratio"`
//
// high space stage transition stage low space stage
// |--------------------|-----------------------------|-------------------------|
// ^ ^ ^ ^
// 0 HighSpaceRatio * capacity LowSpaceRatio * capacity capacity
//
// LowSpaceRatio is the lowest usage ratio of store which regraded as low space.
// When in low space, store region score increases to very large and varies inversely with available size.
LowSpaceRatio float64 `toml:"low-space-ratio" json:"low-space-ratio"`
// HighSpaceRatio is the highest usage ratio of store which regraded as high space.
// High space means there is a lot of spare capacity, and store region score varies directly with used size.
HighSpaceRatio float64 `toml:"high-space-ratio" json:"high-space-ratio"`
// SchedulerMaxWaitingOperator is the max coexist operators for each scheduler.
SchedulerMaxWaitingOperator uint64 `toml:"scheduler-max-waiting-operator" json:"scheduler-max-waiting-operator"`
// WARN: DisableLearner is deprecated.
// DisableLearner is the option to disable using AddLearnerNode instead of AddNode.
DisableLearner bool `toml:"disable-raft-learner" json:"disable-raft-learner,string,omitempty"`
// DisableRemoveDownReplica is the option to prevent replica checker from
// removing down replicas.
// WARN: DisableRemoveDownReplica is deprecated.
DisableRemoveDownReplica bool `toml:"disable-remove-down-replica" json:"disable-remove-down-replica,string,omitempty"`
// DisableReplaceOfflineReplica is the option to prevent replica checker from
// replacing offline replicas.
// WARN: DisableReplaceOfflineReplica is deprecated.
DisableReplaceOfflineReplica bool `toml:"disable-replace-offline-replica" json:"disable-replace-offline-replica,string,omitempty"`
// DisableMakeUpReplica is the option to prevent replica checker from making up
// replicas when replica count is less than expected.
// WARN: DisableMakeUpReplica is deprecated.
DisableMakeUpReplica bool `toml:"disable-make-up-replica" json:"disable-make-up-replica,string,omitempty"`
// DisableRemoveExtraReplica is the option to prevent replica checker from
// removing extra replicas.
// WARN: DisableRemoveExtraReplica is deprecated.
DisableRemoveExtraReplica bool `toml:"disable-remove-extra-replica" json:"disable-remove-extra-replica,string,omitempty"`
// DisableLocationReplacement is the option to prevent replica checker from
// moving replica to a better location.
// WARN: DisableLocationReplacement is deprecated.
DisableLocationReplacement bool `toml:"disable-location-replacement" json:"disable-location-replacement,string,omitempty"`
// EnableRemoveDownReplica is the option to enable replica checker to remove down replica.
EnableRemoveDownReplica bool `toml:"enable-remove-down-replica" json:"enable-remove-down-replica,string"`
// EnableReplaceOfflineReplica is the option to enable replica checker to replace offline replica.
EnableReplaceOfflineReplica bool `toml:"enable-replace-offline-replica" json:"enable-replace-offline-replica,string"`
// EnableMakeUpReplica is the option to enable replica checker to make up replica.
EnableMakeUpReplica bool `toml:"enable-make-up-replica" json:"enable-make-up-replica,string"`
// EnableRemoveExtraReplica is the option to enable replica checker to remove extra replica.
EnableRemoveExtraReplica bool `toml:"enable-remove-extra-replica" json:"enable-remove-extra-replica,string"`
// EnableLocationReplacement is the option to enable replica checker to move replica to a better location.
EnableLocationReplacement bool `toml:"enable-location-replacement" json:"enable-location-replacement,string"`
// EnableDebugMetrics is the option to enable debug metrics.
EnableDebugMetrics bool `toml:"enable-debug-metrics" json:"enable-debug-metrics,string"`
// Schedulers support for loading customized schedulers
Schedulers SchedulerConfigs `toml:"schedulers" json:"schedulers-v2"` // json v2 is for the sake of compatible upgrade
// Only used to display
SchedulersPayload map[string]string `toml:"schedulers-payload" json:"schedulers-payload"`
// StoreLimitMode can be auto or manual, when set to auto,
// PD tries to change the store limit values according to
// the load state of the cluster dynamically. User can
// overwrite the auto-tuned value by pd-ctl, when the value
// is overwritten, the value is fixed until it is deleted.
// Default: manual
StoreLimitMode string `toml:"store-limit-mode" json:"store-limit-mode"`
}
ScheduleConfig is the schedule configuration.
func (*ScheduleConfig) Clone ¶
func (c *ScheduleConfig) Clone() *ScheduleConfig
Clone returns a cloned scheduling configuration.
func (*ScheduleConfig) Deprecated ¶
func (c *ScheduleConfig) Deprecated() error
Deprecated is used to find if there is an option has been deprecated.
func (*ScheduleConfig) MigrateDeprecatedFlags ¶
func (c *ScheduleConfig) MigrateDeprecatedFlags()
MigrateDeprecatedFlags updates new flags according to deprecated flags.
func (*ScheduleConfig) Validate ¶
func (c *ScheduleConfig) Validate() error
Validate is used to validate if some scheduling configurations are right.
type SchedulerConfig ¶
type SchedulerConfig struct {
Type string `toml:"type" json:"type"`
Args []string `toml:"args" json:"args"`
Disable bool `toml:"disable" json:"disable"`
ArgsPayload string `toml:"args-payload" json:"args-payload"`
}
SchedulerConfig is customized scheduler configuration
type SchedulerConfigs ¶
type SchedulerConfigs []SchedulerConfig
SchedulerConfigs is a slice of customized scheduler configuration.
type StoreLabel ¶
type StoreLabel struct {
Key string `toml:"key" json:"key"`
Value string `toml:"value" json:"value"`
}
StoreLabel is the config item of LabelPropertyConfig.