Skip to content

Commit

Permalink
statistics: enable the auto-analysis priority queue (#51537)
Browse files Browse the repository at this point in the history
ref #50132
  • Loading branch information
Rustin170506 authored Mar 7, 2024
1 parent c128a2d commit a8503e8
Show file tree
Hide file tree
Showing 7 changed files with 72 additions and 20 deletions.
2 changes: 1 addition & 1 deletion pkg/sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -1361,7 +1361,7 @@ const (
DefTiDBBatchDMLIgnoreError = false
DefTiDBMemQuotaAnalyze = -1
DefTiDBEnableAutoAnalyze = true
DefTiDBEnableAutoAnalyzePriorityQueue = false
DefTiDBEnableAutoAnalyzePriorityQueue = true
DefTiDBMemOOMAction = "CANCEL"
DefTiDBMaxAutoAnalyzeTime = 12 * 60 * 60
DefTiDBEnablePrepPlanCache = true
Expand Down
21 changes: 2 additions & 19 deletions pkg/statistics/handle/autoanalyze/autoanalyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -259,23 +259,6 @@ func (sa *statsAnalyze) CheckAnalyzeVersion(tblInfo *model.TableInfo, physicalID
return statistics.CheckAnalyzeVerOnTable(tbl, version)
}

// parseAnalyzePeriod parses the start and end time for auto analyze.
// It parses the times in UTC location.
func parseAnalyzePeriod(start, end string) (time.Time, time.Time, error) {
if start == "" {
start = variable.DefAutoAnalyzeStartTime
}
if end == "" {
end = variable.DefAutoAnalyzeEndTime
}
s, err := time.ParseInLocation(variable.FullDayTimeFormat, start, time.UTC)
if err != nil {
return s, s, errors.Trace(err)
}
e, err := time.ParseInLocation(variable.FullDayTimeFormat, end, time.UTC)
return s, e, err
}

// HandleAutoAnalyze analyzes the newly created table or index.
func HandleAutoAnalyze(
sctx sessionctx.Context,
Expand Down Expand Up @@ -303,8 +286,8 @@ func HandleAutoAnalyze(

parameters := exec.GetAutoAnalyzeParameters(sctx)
autoAnalyzeRatio := exec.ParseAutoAnalyzeRatio(parameters[variable.TiDBAutoAnalyzeRatio])
// Get the available time period for auto analyze and check if the current time is in the period.
start, end, err := parseAnalyzePeriod(
// Determine the time window for auto-analysis and verify if the current time falls within this range.
start, end, err := exec.ParseAutoAnalysisWindow(
parameters[variable.TiDBAutoAnalyzeStartTime],
parameters[variable.TiDBAutoAnalyzeEndTime],
)
Expand Down
1 change: 1 addition & 0 deletions pkg/statistics/handle/autoanalyze/exec/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ go_library(
"//pkg/util/chunk",
"//pkg/util/sqlescape",
"//pkg/util/sqlexec",
"@com_github_pingcap_errors//:errors",
"@org_uber_go_zap//:zap",
],
)
Expand Down
18 changes: 18 additions & 0 deletions pkg/statistics/handle/autoanalyze/exec/exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"strconv"
"time"

"github.com/pingcap/errors"
"github.com/pingcap/tidb/pkg/metrics"
"github.com/pingcap/tidb/pkg/parser/ast"
"github.com/pingcap/tidb/pkg/sessionctx"
Expand Down Expand Up @@ -130,3 +131,20 @@ func ParseAutoAnalyzeRatio(ratio string) float64 {
}
return math.Max(autoAnalyzeRatio, 0)
}

// ParseAutoAnalysisWindow parses the time window for auto analysis.
// It parses the times in UTC location.
func ParseAutoAnalysisWindow(start, end string) (time.Time, time.Time, error) {
if start == "" {
start = variable.DefAutoAnalyzeStartTime
}
if end == "" {
end = variable.DefAutoAnalyzeEndTime
}
s, err := time.ParseInLocation(variable.FullDayTimeFormat, start, time.UTC)
if err != nil {
return s, s, errors.Trace(err)
}
e, err := time.ParseInLocation(variable.FullDayTimeFormat, end, time.UTC)
return s, e, err
}
1 change: 1 addition & 0 deletions pkg/statistics/handle/autoanalyze/refresher/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ go_library(
"//pkg/statistics/handle/types",
"//pkg/statistics/handle/util",
"//pkg/util",
"//pkg/util/timeutil",
"@com_github_tikv_client_go_v2//oracle",
"@org_uber_go_zap//:zap",
],
Expand Down
48 changes: 48 additions & 0 deletions pkg/statistics/handle/autoanalyze/refresher/refresher.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
statstypes "github.com/pingcap/tidb/pkg/statistics/handle/types"
statsutil "github.com/pingcap/tidb/pkg/statistics/handle/util"
"github.com/pingcap/tidb/pkg/util"
"github.com/pingcap/tidb/pkg/util/timeutil"
"github.com/tikv/client-go/v2/oracle"
"go.uber.org/zap"
)
Expand All @@ -46,6 +47,8 @@ const (
type Refresher struct {
statsHandle statstypes.StatsHandle
sysProcTracker sessionctx.SysProcTracker
// This will be refreshed every time we rebuild the priority queue.
autoAnalysisTimeWindow

// Jobs is the priority queue of analysis jobs.
// Exported for testing purposes.
Expand All @@ -68,6 +71,10 @@ func NewRefresher(

// PickOneTableAndAnalyzeByPriority picks one table and analyzes it by priority.
func (r *Refresher) PickOneTableAndAnalyzeByPriority() bool {
if !r.autoAnalysisTimeWindow.isWithinTimeWindow(time.Now()) {
return false
}

se, err := r.statsHandle.SPool().Get()
if err != nil {
statslogutil.StatsLogger().Error(
Expand Down Expand Up @@ -125,6 +132,27 @@ func (r *Refresher) RebuildTableAnalysisJobQueue() error {
func(sctx sessionctx.Context) error {
parameters := exec.GetAutoAnalyzeParameters(sctx)
autoAnalyzeRatio := exec.ParseAutoAnalyzeRatio(parameters[variable.TiDBAutoAnalyzeRatio])
// Get the available time period for auto analyze and check if the current time is in the period.
start, end, err := exec.ParseAutoAnalysisWindow(
parameters[variable.TiDBAutoAnalyzeStartTime],
parameters[variable.TiDBAutoAnalyzeEndTime],
)
if err != nil {
statslogutil.StatsLogger().Error(
"parse auto analyze period failed",
zap.Error(err),
)
return err
}
// We will check it again when we try to execute the job.
// So store the time window for later use.
r.autoAnalysisTimeWindow = autoAnalysisTimeWindow{
start: start,
end: end,
}
if !r.autoAnalysisTimeWindow.isWithinTimeWindow(time.Now()) {
return nil
}
calculator := priorityqueue.NewPriorityCalculator()
pruneMode := variable.PartitionPruneMode(sctx.GetSessionVars().PartitionPruneMode.Load())
is := sctx.GetDomainInfoSchema().(infoschema.InfoSchema)
Expand All @@ -142,6 +170,11 @@ func (r *Refresher) RebuildTableAnalysisJobQueue() error {

dbs := infoschema.AllSchemaNames(is)
for _, db := range dbs {
// Sometimes the tables are too many. Auto-analyze will take too much time on it.
// so we need to check the available time.
if !r.autoAnalysisTimeWindow.isWithinTimeWindow(time.Now()) {
return nil
}
// Ignore the memory and system database.
if util.IsMemOrSysDB(strings.ToLower(db)) {
continue
Expand Down Expand Up @@ -613,3 +646,18 @@ func isEligibleForAnalysis(

return true
}

// autoAnalysisTimeWindow is a struct that contains the start and end time of the auto analyze time window.
type autoAnalysisTimeWindow struct {
start time.Time
end time.Time
}

// 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.
func (a autoAnalysisTimeWindow) isWithinTimeWindow(currentTime time.Time) bool {
if a.start == (time.Time{}) || a.end == (time.Time{}) {
return false
}
return timeutil.WithinDayTimePeriod(a.start, a.end, currentTime)
}
Original file line number Diff line number Diff line change
Expand Up @@ -233,6 +233,7 @@ func TestPickOneTableAndAnalyzeByPriorityWithFailedAnalysis(t *testing.T) {
handle := dom.StatsHandle()
sysProcTracker := dom.SysProcTracker()
r := refresher.NewRefresher(handle, sysProcTracker)
r.RebuildTableAnalysisJobQueue()
// No jobs in the queue.
r.PickOneTableAndAnalyzeByPriority()
// The table is not analyzed.
Expand Down

0 comments on commit a8503e8

Please sign in to comment.