From a8503e820cf2a66cb59bd549cd8a45b59381757c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BA=8C=E6=89=8B=E6=8E=89=E5=8C=85=E5=B7=A5=E7=A8=8B?= =?UTF-8?q?=E5=B8=88?= Date: Thu, 7 Mar 2024 16:53:07 +0800 Subject: [PATCH] statistics: enable the auto-analysis priority queue (#51537) ref pingcap/tidb#50132 --- pkg/sessionctx/variable/tidb_vars.go | 2 +- .../handle/autoanalyze/autoanalyze.go | 21 +------- .../handle/autoanalyze/exec/BUILD.bazel | 1 + .../handle/autoanalyze/exec/exec.go | 18 +++++++ .../handle/autoanalyze/refresher/BUILD.bazel | 1 + .../handle/autoanalyze/refresher/refresher.go | 48 +++++++++++++++++++ .../autoanalyze/refresher/refresher_test.go | 1 + 7 files changed, 72 insertions(+), 20 deletions(-) diff --git a/pkg/sessionctx/variable/tidb_vars.go b/pkg/sessionctx/variable/tidb_vars.go index d06e00bcaf10c..6b707dedc7106 100644 --- a/pkg/sessionctx/variable/tidb_vars.go +++ b/pkg/sessionctx/variable/tidb_vars.go @@ -1361,7 +1361,7 @@ const ( DefTiDBBatchDMLIgnoreError = false DefTiDBMemQuotaAnalyze = -1 DefTiDBEnableAutoAnalyze = true - DefTiDBEnableAutoAnalyzePriorityQueue = false + DefTiDBEnableAutoAnalyzePriorityQueue = true DefTiDBMemOOMAction = "CANCEL" DefTiDBMaxAutoAnalyzeTime = 12 * 60 * 60 DefTiDBEnablePrepPlanCache = true diff --git a/pkg/statistics/handle/autoanalyze/autoanalyze.go b/pkg/statistics/handle/autoanalyze/autoanalyze.go index a5422881939c2..eb50ac8000538 100644 --- a/pkg/statistics/handle/autoanalyze/autoanalyze.go +++ b/pkg/statistics/handle/autoanalyze/autoanalyze.go @@ -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, @@ -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], ) diff --git a/pkg/statistics/handle/autoanalyze/exec/BUILD.bazel b/pkg/statistics/handle/autoanalyze/exec/BUILD.bazel index 3deb2e9049c3d..9175318fb68ce 100644 --- a/pkg/statistics/handle/autoanalyze/exec/BUILD.bazel +++ b/pkg/statistics/handle/autoanalyze/exec/BUILD.bazel @@ -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", ], ) diff --git a/pkg/statistics/handle/autoanalyze/exec/exec.go b/pkg/statistics/handle/autoanalyze/exec/exec.go index 6ccf2584b010e..36547ae3d4700 100644 --- a/pkg/statistics/handle/autoanalyze/exec/exec.go +++ b/pkg/statistics/handle/autoanalyze/exec/exec.go @@ -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" @@ -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 +} diff --git a/pkg/statistics/handle/autoanalyze/refresher/BUILD.bazel b/pkg/statistics/handle/autoanalyze/refresher/BUILD.bazel index 826e9f94ec860..a72102f06f6f7 100644 --- a/pkg/statistics/handle/autoanalyze/refresher/BUILD.bazel +++ b/pkg/statistics/handle/autoanalyze/refresher/BUILD.bazel @@ -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", ], diff --git a/pkg/statistics/handle/autoanalyze/refresher/refresher.go b/pkg/statistics/handle/autoanalyze/refresher/refresher.go index 6f31e5c268a70..9ec802296eaa8 100644 --- a/pkg/statistics/handle/autoanalyze/refresher/refresher.go +++ b/pkg/statistics/handle/autoanalyze/refresher/refresher.go @@ -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" ) @@ -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. @@ -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( @@ -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) @@ -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 @@ -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) +} diff --git a/pkg/statistics/handle/autoanalyze/refresher/refresher_test.go b/pkg/statistics/handle/autoanalyze/refresher/refresher_test.go index 4df61c81a4811..4235a076eebc6 100644 --- a/pkg/statistics/handle/autoanalyze/refresher/refresher_test.go +++ b/pkg/statistics/handle/autoanalyze/refresher/refresher_test.go @@ -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.