Documentation ¶
Index ¶
- Constants
- Variables
- func GetAverageAnalysisDuration(sctx sessionctx.Context, schema, tableName string, partitionNames ...string) (time.Duration, error)
- func GetLastFailedAnalysisDuration(sctx sessionctx.Context, schema, tableName string, partitionNames ...string) (time.Duration, error)
- func GetPartitionStats(statsHandle statstypes.StatsHandle, tblInfo *model.TableInfo, ...) map[PartitionIDAndName]*statistics.Table
- func IsDynamicPartitionedTableAnalysisJob(job AnalysisJob) bool
- type AnalysisJob
- type AnalysisJobFactory
- func (f *AnalysisJobFactory) CalculateChangePercentage(tblStats *statistics.Table) float64
- func (f *AnalysisJobFactory) CalculateIndicatorsForPartitions(globalStats *statistics.Table, ...) (avgChange float64, avgSize float64, avgLastAnalyzeDuration time.Duration, ...)
- func (*AnalysisJobFactory) CalculateTableSize(tblStats *statistics.Table) float64
- func (*AnalysisJobFactory) CheckIndexesNeedAnalyze(tblInfo *model.TableInfo, tblStats *statistics.Table) []string
- func (*AnalysisJobFactory) CheckNewlyAddedIndexesNeedAnalyzeForPartitionedTable(tblInfo *model.TableInfo, ...) map[string][]string
- func (f *AnalysisJobFactory) CreateDynamicPartitionedTableAnalysisJob(tableSchema string, globalTblInfo *model.TableInfo, ...) AnalysisJob
- func (f *AnalysisJobFactory) CreateNonPartitionedTableAnalysisJob(tableSchema string, tblInfo *model.TableInfo, tblStats *statistics.Table) AnalysisJob
- func (f *AnalysisJobFactory) CreateStaticPartitionAnalysisJob(tableSchema string, globalTblInfo *model.TableInfo, partitionID int64, ...) AnalysisJob
- func (f *AnalysisJobFactory) FindLastAnalyzeTime(tblStats *statistics.Table) time.Time
- func (f *AnalysisJobFactory) GetTableLastAnalyzeDuration(tblStats *statistics.Table) time.Duration
- type AnalysisPriorityQueue
- func (pq *AnalysisPriorityQueue) Close()
- func (pq *AnalysisPriorityQueue) GetLastFetchTimestamp() uint64
- func (pq *AnalysisPriorityQueue) GetRunningJobs() map[int64]struct{}
- func (pq *AnalysisPriorityQueue) HandleDDLEvent(_ context.Context, sctx sessionctx.Context, event *notifier.SchemaChangeEvent) (err error)
- func (pq *AnalysisPriorityQueue) Initialize() error
- func (pq *AnalysisPriorityQueue) IsEmpty() (bool, error)
- func (pq *AnalysisPriorityQueue) IsInitialized() bool
- func (pq *AnalysisPriorityQueue) Len() (int, error)
- func (pq *AnalysisPriorityQueue) Peek() (AnalysisJob, error)
- func (pq *AnalysisPriorityQueue) Pop() (AnalysisJob, error)
- func (pq *AnalysisPriorityQueue) ProcessDMLChanges()
- func (pq *AnalysisPriorityQueue) Push(job AnalysisJob) error
- func (pq *AnalysisPriorityQueue) Rebuild() error
- func (pq *AnalysisPriorityQueue) RefreshLastAnalysisDuration()
- func (pq *AnalysisPriorityQueue) RequeueMustRetryJobs()
- type AutoAnalysisTimeWindow
- type DynamicPartitionedTableAnalysisJob
- func (j *DynamicPartitionedTableAnalysisJob) Analyze(statsHandle statstypes.StatsHandle, sysProcTracker sysproctrack.Tracker) error
- func (j *DynamicPartitionedTableAnalysisJob) GetIndicators() Indicators
- func (j *DynamicPartitionedTableAnalysisJob) GetTableID() int64
- func (j *DynamicPartitionedTableAnalysisJob) GetWeight() float64
- func (j *DynamicPartitionedTableAnalysisJob) HasNewlyAddedIndex() bool
- func (j *DynamicPartitionedTableAnalysisJob) IsValidToAnalyze(sctx sessionctx.Context) (bool, string)
- func (j *DynamicPartitionedTableAnalysisJob) RegisterFailureHook(hook JobHook)
- func (j *DynamicPartitionedTableAnalysisJob) RegisterSuccessHook(hook JobHook)
- func (j *DynamicPartitionedTableAnalysisJob) SetIndicators(indicators Indicators)
- func (j *DynamicPartitionedTableAnalysisJob) SetWeight(weight float64)
- func (j *DynamicPartitionedTableAnalysisJob) String() string
- type Indicators
- type JobHook
- type NonPartitionedTableAnalysisJob
- func (j *NonPartitionedTableAnalysisJob) Analyze(statsHandle statstypes.StatsHandle, sysProcTracker sysproctrack.Tracker) error
- func (j *NonPartitionedTableAnalysisJob) GenSQLForAnalyzeIndex(index string) (string, []any)
- func (j *NonPartitionedTableAnalysisJob) GenSQLForAnalyzeTable() (string, []any)
- func (j *NonPartitionedTableAnalysisJob) GetIndicators() Indicators
- func (j *NonPartitionedTableAnalysisJob) GetTableID() int64
- func (j *NonPartitionedTableAnalysisJob) GetWeight() float64
- func (j *NonPartitionedTableAnalysisJob) HasNewlyAddedIndex() bool
- func (j *NonPartitionedTableAnalysisJob) IsValidToAnalyze(sctx sessionctx.Context) (bool, string)
- func (j *NonPartitionedTableAnalysisJob) RegisterFailureHook(hook JobHook)
- func (j *NonPartitionedTableAnalysisJob) RegisterSuccessHook(hook JobHook)
- func (j *NonPartitionedTableAnalysisJob) SetIndicators(indicators Indicators)
- func (j *NonPartitionedTableAnalysisJob) SetWeight(weight float64)
- func (j *NonPartitionedTableAnalysisJob) String() string
- type PartitionIDAndName
- type PriorityCalculator
- type StaticPartitionedTableAnalysisJob
- func (j *StaticPartitionedTableAnalysisJob) Analyze(statsHandle statstypes.StatsHandle, sysProcTracker sysproctrack.Tracker) error
- func (j *StaticPartitionedTableAnalysisJob) GenSQLForAnalyzeStaticPartition() (string, []any)
- func (j *StaticPartitionedTableAnalysisJob) GenSQLForAnalyzeStaticPartitionIndex(index string) (string, []any)
- func (j *StaticPartitionedTableAnalysisJob) GetIndicators() Indicators
- func (j *StaticPartitionedTableAnalysisJob) GetTableID() int64
- func (j *StaticPartitionedTableAnalysisJob) GetWeight() float64
- func (j *StaticPartitionedTableAnalysisJob) HasNewlyAddedIndex() bool
- func (j *StaticPartitionedTableAnalysisJob) IsValidToAnalyze(sctx sessionctx.Context) (bool, string)
- func (j *StaticPartitionedTableAnalysisJob) RegisterFailureHook(hook JobHook)
- func (j *StaticPartitionedTableAnalysisJob) RegisterSuccessHook(hook JobHook)
- func (j *StaticPartitionedTableAnalysisJob) SetIndicators(indicators Indicators)
- func (j *StaticPartitionedTableAnalysisJob) SetWeight(weight float64)
- func (j *StaticPartitionedTableAnalysisJob) String() string
Constants ¶
const ( // EventNone represents no special event. EventNone = 0.0 // EventNewIndex represents a special event for newly added indexes. EventNewIndex = 2.0 )
const NoRecord = -1
NoRecord is used to indicate that there is no related record in mysql.analyze_jobs.
Variables ¶
var ( // ErrHeapIsEmpty is returned when the heap is empty. ErrHeapIsEmpty = errors.New("heap is empty") )
Functions ¶
func GetAverageAnalysisDuration ¶
func GetAverageAnalysisDuration( sctx sessionctx.Context, schema, tableName string, partitionNames ...string, ) (time.Duration, error)
GetAverageAnalysisDuration returns the average duration of the last 5 successful analyses for each specified partition. If there are no successful analyses, it returns 0.
func GetLastFailedAnalysisDuration ¶
func GetLastFailedAnalysisDuration( sctx sessionctx.Context, schema, tableName string, partitionNames ...string, ) (time.Duration, error)
GetLastFailedAnalysisDuration returns the duration since the last failed analysis. If there is no failed analysis, it returns 0.
func GetPartitionStats ¶
func GetPartitionStats( statsHandle statstypes.StatsHandle, tblInfo *model.TableInfo, defs []model.PartitionDefinition, ) map[PartitionIDAndName]*statistics.Table
GetPartitionStats gets the partition stats.
func IsDynamicPartitionedTableAnalysisJob ¶
func IsDynamicPartitionedTableAnalysisJob(job AnalysisJob) bool
IsDynamicPartitionedTableAnalysisJob checks whether the job is a dynamic partitioned table analysis job.
Types ¶
type AnalysisJob ¶
type AnalysisJob interface { // IsValidToAnalyze checks whether the table is valid to analyze. // It checks the last failed analysis duration and the average analysis duration. // If the last failed analysis duration is less than 2 times the average analysis duration, // we skip this table to avoid too much failed analysis. IsValidToAnalyze( sctx sessionctx.Context, ) (bool, string) // Analyze executes the analyze statement within a transaction. Analyze( statsHandle statstypes.StatsHandle, sysProcTracker sysproctrack.Tracker, ) error // SetWeight sets the weight of the job. SetWeight(weight float64) // GetWeight gets the weight of the job. GetWeight() float64 // HasNewlyAddedIndex checks whether the job has newly added index. HasNewlyAddedIndex() bool // GetIndicators gets the indicators of the job. GetIndicators() Indicators // SetIndicators sets the indicators of the job. SetIndicators(indicators Indicators) // GetTableID gets the table ID of the job. GetTableID() int64 // RegisterSuccessHook registers a successHook function that will be called after the job can be marked as successful. RegisterSuccessHook(hook JobHook) // RegisterFailureHook registers a successHook function that will be called after the job is marked as failed. RegisterFailureHook(hook JobHook) fmt.Stringer }
AnalysisJob is the interface for the analysis job.
type AnalysisJobFactory ¶
type AnalysisJobFactory struct {
// contains filtered or unexported fields
}
AnalysisJobFactory is responsible for creating different types of analysis jobs. NOTE: This struct is not thread-safe.
func NewAnalysisJobFactory ¶
func NewAnalysisJobFactory(sctx sessionctx.Context, autoAnalyzeRatio float64, currentTs uint64) *AnalysisJobFactory
NewAnalysisJobFactory creates a new AnalysisJobFactory.
func (*AnalysisJobFactory) CalculateChangePercentage ¶
func (f *AnalysisJobFactory) CalculateChangePercentage(tblStats *statistics.Table) float64
CalculateChangePercentage calculates the change percentage of the table based on the change count and the analysis count.
func (*AnalysisJobFactory) CalculateIndicatorsForPartitions ¶
func (f *AnalysisJobFactory) CalculateIndicatorsForPartitions( globalStats *statistics.Table, partitionStats map[PartitionIDAndName]*statistics.Table, ) ( avgChange float64, avgSize float64, avgLastAnalyzeDuration time.Duration, partitionNames []string, )
CalculateIndicatorsForPartitions calculates the average change percentage, average size and average last analyze duration for the partitions that meet the threshold. Change percentage is the ratio of the number of modified rows to the total number of rows. Size is the product of the number of rows and the number of columns. Last analyze duration is the duration since the last analyze.
func (*AnalysisJobFactory) CalculateTableSize ¶
func (*AnalysisJobFactory) CalculateTableSize(tblStats *statistics.Table) float64
CalculateTableSize calculates the size of the table.
func (*AnalysisJobFactory) CheckIndexesNeedAnalyze ¶
func (*AnalysisJobFactory) CheckIndexesNeedAnalyze(tblInfo *model.TableInfo, tblStats *statistics.Table) []string
CheckIndexesNeedAnalyze checks if the indexes need to be analyzed.
func (*AnalysisJobFactory) CheckNewlyAddedIndexesNeedAnalyzeForPartitionedTable ¶
func (*AnalysisJobFactory) CheckNewlyAddedIndexesNeedAnalyzeForPartitionedTable( tblInfo *model.TableInfo, partitionStats map[PartitionIDAndName]*statistics.Table, ) map[string][]string
CheckNewlyAddedIndexesNeedAnalyzeForPartitionedTable checks if the indexes of the partitioned table need to be analyzed. It returns a map from index name to the names of the partitions that need to be analyzed. NOTE: This is only for newly added indexes.
func (*AnalysisJobFactory) CreateDynamicPartitionedTableAnalysisJob ¶
func (f *AnalysisJobFactory) CreateDynamicPartitionedTableAnalysisJob( tableSchema string, globalTblInfo *model.TableInfo, globalTblStats *statistics.Table, partitionStats map[PartitionIDAndName]*statistics.Table, ) AnalysisJob
CreateDynamicPartitionedTableAnalysisJob creates a job for dynamic partitioned tables.
func (*AnalysisJobFactory) CreateNonPartitionedTableAnalysisJob ¶
func (f *AnalysisJobFactory) CreateNonPartitionedTableAnalysisJob( tableSchema string, tblInfo *model.TableInfo, tblStats *statistics.Table, ) AnalysisJob
CreateNonPartitionedTableAnalysisJob creates a job for non-partitioned tables.
func (*AnalysisJobFactory) CreateStaticPartitionAnalysisJob ¶
func (f *AnalysisJobFactory) CreateStaticPartitionAnalysisJob( tableSchema string, globalTblInfo *model.TableInfo, partitionID int64, partitionName string, partitionStats *statistics.Table, ) AnalysisJob
CreateStaticPartitionAnalysisJob creates a job for static partitions.
func (*AnalysisJobFactory) FindLastAnalyzeTime ¶
func (f *AnalysisJobFactory) FindLastAnalyzeTime(tblStats *statistics.Table) time.Time
FindLastAnalyzeTime finds the last analyze time of the table. It uses `LastUpdateVersion` to find the last analyze time. The `LastUpdateVersion` is the version of the transaction that updates the statistics. It always not null(default 0), so we can use it to find the last analyze time.
func (*AnalysisJobFactory) GetTableLastAnalyzeDuration ¶
func (f *AnalysisJobFactory) GetTableLastAnalyzeDuration(tblStats *statistics.Table) time.Duration
GetTableLastAnalyzeDuration gets the last analyze duration of the table.
type AnalysisPriorityQueue ¶
type AnalysisPriorityQueue struct {
// contains filtered or unexported fields
}
AnalysisPriorityQueue is a priority queue for TableAnalysisJobs. Testing shows that keeping all jobs in memory is feasible. Memory usage for one million tables is approximately 300 to 500 MiB, which is acceptable. Typically, not many tables need to be analyzed simultaneously.
func NewAnalysisPriorityQueue ¶
func NewAnalysisPriorityQueue(handle statstypes.StatsHandle) *AnalysisPriorityQueue
NewAnalysisPriorityQueue creates a new AnalysisPriorityQueue2.
func (*AnalysisPriorityQueue) Close ¶
func (pq *AnalysisPriorityQueue) Close()
Close closes the priority queue. Note: This function is thread-safe.
func (*AnalysisPriorityQueue) GetLastFetchTimestamp ¶
func (pq *AnalysisPriorityQueue) GetLastFetchTimestamp() uint64
GetLastFetchTimestamp returns the last fetch timestamp of DML updates. Note: This function is thread-safe. Exported for testing.
func (*AnalysisPriorityQueue) GetRunningJobs ¶
func (pq *AnalysisPriorityQueue) GetRunningJobs() map[int64]struct{}
GetRunningJobs returns the running jobs. Note: This function is thread-safe. Exported for testing.
func (*AnalysisPriorityQueue) HandleDDLEvent ¶
func (pq *AnalysisPriorityQueue) HandleDDLEvent(_ context.Context, sctx sessionctx.Context, event *notifier.SchemaChangeEvent) (err error)
HandleDDLEvent handles DDL events for the priority queue.
func (*AnalysisPriorityQueue) Initialize ¶
func (pq *AnalysisPriorityQueue) Initialize() error
Initialize initializes the priority queue. Note: This function is thread-safe.
func (*AnalysisPriorityQueue) IsEmpty ¶
func (pq *AnalysisPriorityQueue) IsEmpty() (bool, error)
IsEmpty checks whether the priority queue is empty. Note: This function is thread-safe.
func (*AnalysisPriorityQueue) IsInitialized ¶
func (pq *AnalysisPriorityQueue) IsInitialized() bool
IsInitialized checks if the priority queue is initialized.
func (*AnalysisPriorityQueue) Len ¶
func (pq *AnalysisPriorityQueue) Len() (int, error)
Len returns the number of jobs in the priority queue. Note: This function is thread-safe.
func (*AnalysisPriorityQueue) Peek ¶
func (pq *AnalysisPriorityQueue) Peek() (AnalysisJob, error)
Peek peeks the top job from the priority queue.
func (*AnalysisPriorityQueue) Pop ¶
func (pq *AnalysisPriorityQueue) Pop() (AnalysisJob, error)
Pop pops a job from the priority queue and marks it as running. Note: This function is thread-safe.
func (*AnalysisPriorityQueue) ProcessDMLChanges ¶
func (pq *AnalysisPriorityQueue) ProcessDMLChanges()
ProcessDMLChanges processes DML changes. Note: This function is thread-safe. Performance: To scan all table stats and process the DML changes, it takes about less than 100ms for 1m tables.
func (*AnalysisPriorityQueue) Push ¶
func (pq *AnalysisPriorityQueue) Push(job AnalysisJob) error
Push pushes a job into the priority queue. Note: This function is thread-safe.
func (*AnalysisPriorityQueue) Rebuild ¶
func (pq *AnalysisPriorityQueue) Rebuild() error
Rebuild rebuilds the priority queue. Note: This function is thread-safe.
func (*AnalysisPriorityQueue) RefreshLastAnalysisDuration ¶
func (pq *AnalysisPriorityQueue) RefreshLastAnalysisDuration()
RefreshLastAnalysisDuration refreshes the last analysis duration of all jobs in the priority queue. Note: This function is thread-safe.
func (*AnalysisPriorityQueue) RequeueMustRetryJobs ¶
func (pq *AnalysisPriorityQueue) RequeueMustRetryJobs()
RequeueMustRetryJobs requeues the must retry jobs.
type AutoAnalysisTimeWindow ¶
type AutoAnalysisTimeWindow struct {
// contains filtered or unexported fields
}
AutoAnalysisTimeWindow is a struct that contains the start and end time of the auto analyze time window.
func NewAutoAnalysisTimeWindow ¶
func NewAutoAnalysisTimeWindow(start, end time.Time) AutoAnalysisTimeWindow
NewAutoAnalysisTimeWindow creates a new AutoAnalysisTimeWindow.
func (AutoAnalysisTimeWindow) IsWithinTimeWindow ¶
func (a AutoAnalysisTimeWindow) IsWithinTimeWindow(currentTime time.Time) bool
IsWithinTimeWindow checks if the current time is within the time window. If the auto analyze time window is not set or the current time is not in the window, return false.
type DynamicPartitionedTableAnalysisJob ¶
type DynamicPartitionedTableAnalysisJob struct { // Only set when partitions's indexes need to be analyzed. // It looks like: {"indexName": ["partitionName1", "partitionName2"]} // This is only for newly added indexes. // The reason why we need to record the partition names is that we need to analyze partitions in batch mode // and we don't want to analyze the same partition multiple times. // For example, the user may analyze some partitions manually, and we don't want to analyze them again. PartitionIndexes map[string][]string TableSchema string GlobalTableName string // This will analyze all indexes and columns of the specified partitions. Partitions []string // Some indicators to help us decide whether we need to analyze this table. Indicators GlobalTableID int64 // Analyze table with this version of statistics. TableStatsVer int // Weight is used to calculate the priority of the job. Weight float64 // contains filtered or unexported fields }
DynamicPartitionedTableAnalysisJob is a TableAnalysisJob for analyzing dynamic pruned partitioned table.
func NewDynamicPartitionedTableAnalysisJob ¶
func NewDynamicPartitionedTableAnalysisJob( schema, tableName string, tableID int64, partitions []string, partitionIndexes map[string][]string, tableStatsVer int, changePercentage float64, tableSize float64, lastAnalysisDuration time.Duration, ) *DynamicPartitionedTableAnalysisJob
NewDynamicPartitionedTableAnalysisJob creates a new job for analyzing a dynamic partitioned table's partitions.
func (*DynamicPartitionedTableAnalysisJob) Analyze ¶
func (j *DynamicPartitionedTableAnalysisJob) Analyze( statsHandle statstypes.StatsHandle, sysProcTracker sysproctrack.Tracker, ) error
Analyze analyzes the partitions or partition indexes.
func (*DynamicPartitionedTableAnalysisJob) GetIndicators ¶
func (j *DynamicPartitionedTableAnalysisJob) GetIndicators() Indicators
GetIndicators returns the indicators of the table.
func (*DynamicPartitionedTableAnalysisJob) GetTableID ¶
func (j *DynamicPartitionedTableAnalysisJob) GetTableID() int64
GetTableID gets the table ID of the job.
func (*DynamicPartitionedTableAnalysisJob) GetWeight ¶
func (j *DynamicPartitionedTableAnalysisJob) GetWeight() float64
GetWeight gets the weight of the job.
func (*DynamicPartitionedTableAnalysisJob) HasNewlyAddedIndex ¶
func (j *DynamicPartitionedTableAnalysisJob) HasNewlyAddedIndex() bool
HasNewlyAddedIndex checks whether the job has newly added index.
func (*DynamicPartitionedTableAnalysisJob) IsValidToAnalyze ¶
func (j *DynamicPartitionedTableAnalysisJob) IsValidToAnalyze( sctx sessionctx.Context, ) (bool, string)
IsValidToAnalyze checks whether the table or partition is valid to analyze. We need to check each partition to determine whether the table is valid to analyze.
func (*DynamicPartitionedTableAnalysisJob) RegisterFailureHook ¶
func (j *DynamicPartitionedTableAnalysisJob) RegisterFailureHook(hook JobHook)
RegisterFailureHook registers a successHook function that will be called after the job can be marked as failed.
func (*DynamicPartitionedTableAnalysisJob) RegisterSuccessHook ¶
func (j *DynamicPartitionedTableAnalysisJob) RegisterSuccessHook(hook JobHook)
RegisterSuccessHook registers a successHook function that will be called after the job can be marked as successful.
func (*DynamicPartitionedTableAnalysisJob) SetIndicators ¶
func (j *DynamicPartitionedTableAnalysisJob) SetIndicators(indicators Indicators)
SetIndicators sets the indicators of the table.
func (*DynamicPartitionedTableAnalysisJob) SetWeight ¶
func (j *DynamicPartitionedTableAnalysisJob) SetWeight(weight float64)
SetWeight sets the weight of the job.
func (*DynamicPartitionedTableAnalysisJob) String ¶
func (j *DynamicPartitionedTableAnalysisJob) String() string
String implements fmt.Stringer interface.
type Indicators ¶
type Indicators struct { // ChangePercentage is the percentage of the changed rows. // Usually, the more the changed rows, the higher the priority. // It is calculated by modifiedCount / last time analysis count. ChangePercentage float64 // TableSize is the table size in rows * len(columns). TableSize float64 // LastAnalysisDuration is the duration from the last analysis to now. LastAnalysisDuration time.Duration }
Indicators contains some indicators to evaluate the table priority.
type JobHook ¶
type JobHook func(job AnalysisJob)
JobHook is the successHook function that will be called after the job is completed.
type NonPartitionedTableAnalysisJob ¶
type NonPartitionedTableAnalysisJob struct { TableSchema string TableName string // This is only for newly added indexes. Indexes []string Indicators TableID int64 TableStatsVer int Weight float64 // contains filtered or unexported fields }
NonPartitionedTableAnalysisJob is a TableAnalysisJob for analyzing the physical table.
func NewNonPartitionedTableAnalysisJob ¶
func NewNonPartitionedTableAnalysisJob( schema, tableName string, tableID int64, indexes []string, tableStatsVer int, changePercentage float64, tableSize float64, lastAnalysisDuration time.Duration, ) *NonPartitionedTableAnalysisJob
NewNonPartitionedTableAnalysisJob creates a new TableAnalysisJob for analyzing the physical table.
func (*NonPartitionedTableAnalysisJob) Analyze ¶
func (j *NonPartitionedTableAnalysisJob) Analyze( statsHandle statstypes.StatsHandle, sysProcTracker sysproctrack.Tracker, ) error
Analyze analyzes the table or indexes.
func (*NonPartitionedTableAnalysisJob) GenSQLForAnalyzeIndex ¶
func (j *NonPartitionedTableAnalysisJob) GenSQLForAnalyzeIndex(index string) (string, []any)
GenSQLForAnalyzeIndex generates the SQL for analyzing the specified index.
func (*NonPartitionedTableAnalysisJob) GenSQLForAnalyzeTable ¶
func (j *NonPartitionedTableAnalysisJob) GenSQLForAnalyzeTable() (string, []any)
GenSQLForAnalyzeTable generates the SQL for analyzing the specified table.
func (*NonPartitionedTableAnalysisJob) GetIndicators ¶
func (j *NonPartitionedTableAnalysisJob) GetIndicators() Indicators
GetIndicators returns the indicators of the table.
func (*NonPartitionedTableAnalysisJob) GetTableID ¶
func (j *NonPartitionedTableAnalysisJob) GetTableID() int64
GetTableID gets the table ID of the job.
func (*NonPartitionedTableAnalysisJob) GetWeight ¶
func (j *NonPartitionedTableAnalysisJob) GetWeight() float64
GetWeight gets the weight of the job.
func (*NonPartitionedTableAnalysisJob) HasNewlyAddedIndex ¶
func (j *NonPartitionedTableAnalysisJob) HasNewlyAddedIndex() bool
HasNewlyAddedIndex checks whether the table has newly added indexes.
func (*NonPartitionedTableAnalysisJob) IsValidToAnalyze ¶
func (j *NonPartitionedTableAnalysisJob) IsValidToAnalyze( sctx sessionctx.Context, ) (bool, string)
IsValidToAnalyze checks whether the table is valid to analyze. We will check the last failed job and average analyze duration to determine whether the table is valid to analyze.
func (*NonPartitionedTableAnalysisJob) RegisterFailureHook ¶
func (j *NonPartitionedTableAnalysisJob) RegisterFailureHook(hook JobHook)
RegisterFailureHook registers a failureHook function that will be called after the job can be marked as failed.
func (*NonPartitionedTableAnalysisJob) RegisterSuccessHook ¶
func (j *NonPartitionedTableAnalysisJob) RegisterSuccessHook(hook JobHook)
RegisterSuccessHook registers a successHook function that will be called after the job can be marked as successful.
func (*NonPartitionedTableAnalysisJob) SetIndicators ¶
func (j *NonPartitionedTableAnalysisJob) SetIndicators(indicators Indicators)
SetIndicators sets the indicators of the table.
func (*NonPartitionedTableAnalysisJob) SetWeight ¶
func (j *NonPartitionedTableAnalysisJob) SetWeight(weight float64)
SetWeight sets the weight of the job.
func (*NonPartitionedTableAnalysisJob) String ¶
func (j *NonPartitionedTableAnalysisJob) String() string
String implements fmt.Stringer interface.
type PartitionIDAndName ¶
PartitionIDAndName is a struct that contains the ID and name of a partition. Exported for testing purposes. Do not use it in other packages.
func NewPartitionIDAndName ¶
func NewPartitionIDAndName(name string, id int64) PartitionIDAndName
NewPartitionIDAndName creates a new PartitionIDAndName.
type PriorityCalculator ¶
type PriorityCalculator struct{}
PriorityCalculator implements the WeightCalculator interface.
func NewPriorityCalculator ¶
func NewPriorityCalculator() *PriorityCalculator
NewPriorityCalculator creates a new PriorityCalculator.
For more information, please visit: https://github.com/pingcap/tidb/blob/master/docs/design/2023-11-29-priority-queue-for-auto-analyze.md
func (*PriorityCalculator) CalculateWeight ¶
func (pc *PriorityCalculator) CalculateWeight(job AnalysisJob) float64
CalculateWeight calculates the weight based on the given rules. - Table Change Ratio (Change Ratio): Accounts for 60% - Table Size (Size): Accounts for 10% - Analysis Interval (Analysis Interval): Accounts for 30% priority_score calculates the priority score based on the following formula:
priority_score = (0.6 * math.Log10(1 + ChangeRatio) + 0.1 * (1 - math.Log10(1 + TableSize)) + 0.3 * math.Log10(1 + math.Sqrt(AnalysisInterval)) + special_event[event])
func (*PriorityCalculator) GetSpecialEvent ¶
func (*PriorityCalculator) GetSpecialEvent(job AnalysisJob) float64
GetSpecialEvent returns the special event weight. Exported for testing purposes.
type StaticPartitionedTableAnalysisJob ¶
type StaticPartitionedTableAnalysisJob struct { TableSchema string GlobalTableName string StaticPartitionName string // This is only for newly added indexes. Indexes []string Indicators GlobalTableID int64 StaticPartitionID int64 TableStatsVer int Weight float64 // contains filtered or unexported fields }
StaticPartitionedTableAnalysisJob is a job for analyzing a static partitioned table.
func NewStaticPartitionTableAnalysisJob ¶
func NewStaticPartitionTableAnalysisJob( schema, globalTableName string, globalTableID int64, partitionName string, partitionID int64, indexes []string, tableStatsVer int, changePercentage float64, tableSize float64, lastAnalysisDuration time.Duration, ) *StaticPartitionedTableAnalysisJob
NewStaticPartitionTableAnalysisJob creates a job for analyzing a static partitioned table.
func (*StaticPartitionedTableAnalysisJob) Analyze ¶
func (j *StaticPartitionedTableAnalysisJob) Analyze( statsHandle statstypes.StatsHandle, sysProcTracker sysproctrack.Tracker, ) error
Analyze analyzes the specified static partition or indexes.
func (*StaticPartitionedTableAnalysisJob) GenSQLForAnalyzeStaticPartition ¶
func (j *StaticPartitionedTableAnalysisJob) GenSQLForAnalyzeStaticPartition() (string, []any)
GenSQLForAnalyzeStaticPartition generates the SQL for analyzing the specified static partition.
func (*StaticPartitionedTableAnalysisJob) GenSQLForAnalyzeStaticPartitionIndex ¶
func (j *StaticPartitionedTableAnalysisJob) GenSQLForAnalyzeStaticPartitionIndex(index string) (string, []any)
GenSQLForAnalyzeStaticPartitionIndex generates the SQL for analyzing the specified static partition index.
func (*StaticPartitionedTableAnalysisJob) GetIndicators ¶
func (j *StaticPartitionedTableAnalysisJob) GetIndicators() Indicators
GetIndicators implements AnalysisJob.
func (*StaticPartitionedTableAnalysisJob) GetTableID ¶
func (j *StaticPartitionedTableAnalysisJob) GetTableID() int64
GetTableID gets the table ID of the job.
func (*StaticPartitionedTableAnalysisJob) GetWeight ¶
func (j *StaticPartitionedTableAnalysisJob) GetWeight() float64
GetWeight implements AnalysisJob.
func (*StaticPartitionedTableAnalysisJob) HasNewlyAddedIndex ¶
func (j *StaticPartitionedTableAnalysisJob) HasNewlyAddedIndex() bool
HasNewlyAddedIndex implements AnalysisJob.
func (*StaticPartitionedTableAnalysisJob) IsValidToAnalyze ¶
func (j *StaticPartitionedTableAnalysisJob) IsValidToAnalyze( sctx sessionctx.Context, ) (bool, string)
IsValidToAnalyze checks whether the partition is valid to analyze. Only the specified static partition is checked.
func (*StaticPartitionedTableAnalysisJob) RegisterFailureHook ¶
func (j *StaticPartitionedTableAnalysisJob) RegisterFailureHook(hook JobHook)
RegisterFailureHook registers a failureHook function that will be called after the job can be marked as failed.
func (*StaticPartitionedTableAnalysisJob) RegisterSuccessHook ¶
func (j *StaticPartitionedTableAnalysisJob) RegisterSuccessHook(hook JobHook)
RegisterSuccessHook registers a successHook function that will be called after the job can be marked as successful.
func (*StaticPartitionedTableAnalysisJob) SetIndicators ¶
func (j *StaticPartitionedTableAnalysisJob) SetIndicators(indicators Indicators)
SetIndicators implements AnalysisJob.
func (*StaticPartitionedTableAnalysisJob) SetWeight ¶
func (j *StaticPartitionedTableAnalysisJob) SetWeight(weight float64)
SetWeight implements AnalysisJob.
func (*StaticPartitionedTableAnalysisJob) String ¶
func (j *StaticPartitionedTableAnalysisJob) String() string
String implements fmt.Stringer interface.