diff --git a/pkg/executor/test/analyzetest/BUILD.bazel b/pkg/executor/test/analyzetest/BUILD.bazel index 5f37de422e445..58ef0f964e7b3 100644 --- a/pkg/executor/test/analyzetest/BUILD.bazel +++ b/pkg/executor/test/analyzetest/BUILD.bazel @@ -26,7 +26,7 @@ go_test( "//pkg/sessionctx", "//pkg/sessionctx/variable", "//pkg/statistics", - "//pkg/statistics/handle/autoanalyze", + "//pkg/statistics/handle/autoanalyze/exec", "//pkg/testkit", "//pkg/util/dbterror/exeerrors", "@com_github_pingcap_errors//:errors", diff --git a/pkg/executor/test/analyzetest/analyze_test.go b/pkg/executor/test/analyzetest/analyze_test.go index bc4ca62ffd042..1947235540333 100644 --- a/pkg/executor/test/analyzetest/analyze_test.go +++ b/pkg/executor/test/analyzetest/analyze_test.go @@ -39,7 +39,7 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" - "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze" + "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/exec" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util/dbterror/exeerrors" "github.com/stretchr/testify/require" @@ -689,11 +689,11 @@ func TestSavedAnalyzeOptions(t *testing.T) { tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_ratio = %v", originalVal2)) }() tk.MustExec("set global tidb_auto_analyze_ratio = 0.01") - originalVal3 := autoanalyze.AutoAnalyzeMinCnt + originalVal3 := exec.AutoAnalyzeMinCnt defer func() { - autoanalyze.AutoAnalyzeMinCnt = originalVal3 + exec.AutoAnalyzeMinCnt = originalVal3 }() - autoanalyze.AutoAnalyzeMinCnt = 0 + exec.AutoAnalyzeMinCnt = 0 tk.MustExec("use test") tk.MustExec("set @@session.tidb_analyze_version = 2") @@ -1031,11 +1031,11 @@ func TestSavedAnalyzeColumnOptions(t *testing.T) { tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_ratio = %v", originalVal2)) }() tk.MustExec("set global tidb_auto_analyze_ratio = 0.01") - originalVal3 := autoanalyze.AutoAnalyzeMinCnt + originalVal3 := exec.AutoAnalyzeMinCnt defer func() { - autoanalyze.AutoAnalyzeMinCnt = originalVal3 + exec.AutoAnalyzeMinCnt = originalVal3 }() - autoanalyze.AutoAnalyzeMinCnt = 0 + exec.AutoAnalyzeMinCnt = 0 originalVal4 := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) defer func() { tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal4)) @@ -1884,9 +1884,9 @@ func testKillAutoAnalyze(t *testing.T, ver int) { tk := testkit.NewTestKit(t, store) oriStart := tk.MustQuery("select @@tidb_auto_analyze_start_time").Rows()[0][0].(string) oriEnd := tk.MustQuery("select @@tidb_auto_analyze_end_time").Rows()[0][0].(string) - autoanalyze.AutoAnalyzeMinCnt = 0 + exec.AutoAnalyzeMinCnt = 0 defer func() { - autoanalyze.AutoAnalyzeMinCnt = 1000 + exec.AutoAnalyzeMinCnt = 1000 tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_start_time='%v'", oriStart)) tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_end_time='%v'", oriEnd)) }() @@ -1967,9 +1967,9 @@ func TestKillAutoAnalyzeIndex(t *testing.T) { tk := testkit.NewTestKit(t, store) oriStart := tk.MustQuery("select @@tidb_auto_analyze_start_time").Rows()[0][0].(string) oriEnd := tk.MustQuery("select @@tidb_auto_analyze_end_time").Rows()[0][0].(string) - autoanalyze.AutoAnalyzeMinCnt = 0 + exec.AutoAnalyzeMinCnt = 0 defer func() { - autoanalyze.AutoAnalyzeMinCnt = 1000 + exec.AutoAnalyzeMinCnt = 1000 tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_start_time='%v'", oriStart)) tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_end_time='%v'", oriEnd)) }() @@ -2722,12 +2722,12 @@ func TestAutoAnalyzeAwareGlobalVariableChange(t *testing.T) { "3 0", )) - originalVal1 := autoanalyze.AutoAnalyzeMinCnt + originalVal1 := exec.AutoAnalyzeMinCnt originalVal2 := tk.MustQuery("select @@global.tidb_auto_analyze_ratio").Rows()[0][0].(string) - autoanalyze.AutoAnalyzeMinCnt = 0 + exec.AutoAnalyzeMinCnt = 0 tk.MustExec("set global tidb_auto_analyze_ratio = 0.001") defer func() { - autoanalyze.AutoAnalyzeMinCnt = originalVal1 + exec.AutoAnalyzeMinCnt = originalVal1 tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_ratio = %v", originalVal2)) }() diff --git a/pkg/executor/test/analyzetest/memorycontrol/BUILD.bazel b/pkg/executor/test/analyzetest/memorycontrol/BUILD.bazel index d2990c150d057..793de08dcf3ad 100644 --- a/pkg/executor/test/analyzetest/memorycontrol/BUILD.bazel +++ b/pkg/executor/test/analyzetest/memorycontrol/BUILD.bazel @@ -13,7 +13,7 @@ go_test( "//pkg/config", "//pkg/executor", "//pkg/sessionctx/variable", - "//pkg/statistics/handle/autoanalyze", + "//pkg/statistics/handle/autoanalyze/exec", "//pkg/testkit", "//pkg/util", "@com_github_pingcap_failpoint//:failpoint", diff --git a/pkg/executor/test/analyzetest/memorycontrol/memory_control_test.go b/pkg/executor/test/analyzetest/memorycontrol/memory_control_test.go index e2358304e6fe8..b50ecced58529 100644 --- a/pkg/executor/test/analyzetest/memorycontrol/memory_control_test.go +++ b/pkg/executor/test/analyzetest/memorycontrol/memory_control_test.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/executor" - "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze" + "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/exec" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util" "github.com/stretchr/testify/require" @@ -143,12 +143,12 @@ func TestGlobalMemoryControlForAutoAnalyze(t *testing.T) { require.Len(t, rs0.Rows(), 0) h := dom.StatsHandle() - originalVal4 := autoanalyze.AutoAnalyzeMinCnt + originalVal4 := exec.AutoAnalyzeMinCnt originalVal5 := tk.MustQuery("select @@global.tidb_auto_analyze_ratio").Rows()[0][0].(string) - autoanalyze.AutoAnalyzeMinCnt = 0 + exec.AutoAnalyzeMinCnt = 0 tk.MustExec("set global tidb_auto_analyze_ratio = 0.001") defer func() { - autoanalyze.AutoAnalyzeMinCnt = originalVal4 + exec.AutoAnalyzeMinCnt = originalVal4 tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_ratio = %v", originalVal5)) }() diff --git a/pkg/statistics/BUILD.bazel b/pkg/statistics/BUILD.bazel index 7b8a5054788c8..8371bf0948a77 100644 --- a/pkg/statistics/BUILD.bazel +++ b/pkg/statistics/BUILD.bazel @@ -87,7 +87,7 @@ go_test( "//pkg/parser/mysql", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", - "//pkg/statistics/handle/autoanalyze", + "//pkg/statistics/handle/autoanalyze/exec", "//pkg/testkit", "//pkg/testkit/testdata", "//pkg/testkit/testmain", diff --git a/pkg/statistics/handle/autoanalyze/BUILD.bazel b/pkg/statistics/handle/autoanalyze/BUILD.bazel index af5cfac537d75..1b67c6cd08df5 100644 --- a/pkg/statistics/handle/autoanalyze/BUILD.bazel +++ b/pkg/statistics/handle/autoanalyze/BUILD.bazel @@ -42,6 +42,7 @@ go_test( "//pkg/sessionctx", "//pkg/sessionctx/variable", "//pkg/statistics", + "//pkg/statistics/handle/autoanalyze/exec", "//pkg/statistics/handle/util", "//pkg/statistics/handle/util/test", "//pkg/testkit", diff --git a/pkg/statistics/handle/autoanalyze/autoanalyze.go b/pkg/statistics/handle/autoanalyze/autoanalyze.go index 925d10922e722..c7c49ba801897 100644 --- a/pkg/statistics/handle/autoanalyze/autoanalyze.go +++ b/pkg/statistics/handle/autoanalyze/autoanalyze.go @@ -17,7 +17,6 @@ package autoanalyze import ( "context" "fmt" - "math" "math/rand" "net" "strconv" @@ -259,14 +258,6 @@ func (sa *statsAnalyze) CheckAnalyzeVersion(tblInfo *model.TableInfo, physicalID return statistics.CheckAnalyzeVerOnTable(tbl, version) } -func parseAutoAnalyzeRatio(ratio string) float64 { - autoAnalyzeRatio, err := strconv.ParseFloat(ratio, 64) - if err != nil { - return variable.DefAutoAnalyzeRatio - } - return math.Max(autoAnalyzeRatio, 0) -} - // 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) { @@ -284,19 +275,6 @@ func parseAnalyzePeriod(start, end string) (time.Time, time.Time, error) { return s, e, err } -func getAutoAnalyzeParameters(sctx sessionctx.Context) map[string]string { - sql := "select variable_name, variable_value from mysql.global_variables where variable_name in (%?, %?, %?)" - rows, _, err := statsutil.ExecWithOpts(sctx, nil, sql, variable.TiDBAutoAnalyzeRatio, variable.TiDBAutoAnalyzeStartTime, variable.TiDBAutoAnalyzeEndTime) - if err != nil { - return map[string]string{} - } - parameters := make(map[string]string, len(rows)) - for _, row := range rows { - parameters[row.GetString(0)] = row.GetString(1) - } - return parameters -} - // HandleAutoAnalyze analyzes the newly created table or index. func HandleAutoAnalyze( sctx sessionctx.Context, @@ -313,8 +291,8 @@ func HandleAutoAnalyze( } }() - parameters := getAutoAnalyzeParameters(sctx) - autoAnalyzeRatio := parseAutoAnalyzeRatio(parameters[variable.TiDBAutoAnalyzeRatio]) + 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( parameters[variable.TiDBAutoAnalyzeStartTime], @@ -473,10 +451,6 @@ func getPartitionStats( return partitionStats } -// AutoAnalyzeMinCnt means if the count of table is less than this value, we don't need to do auto analyze. -// Exported for testing. -var AutoAnalyzeMinCnt int64 = 1000 - // Determine whether the table and index require analysis. func tryAutoAnalyzeTable( sctx sessionctx.Context, @@ -492,7 +466,7 @@ func tryAutoAnalyzeTable( // Pseudo statistics can be created by the optimizer, so we need to double check it. // 2. If the table is too small, we don't want to waste time to analyze it. // Leave the opportunity to other bigger tables. - if statsTbl == nil || statsTbl.Pseudo || statsTbl.RealtimeCount < AutoAnalyzeMinCnt { + if statsTbl == nil || statsTbl.Pseudo || statsTbl.RealtimeCount < exec.AutoAnalyzeMinCnt { return false } @@ -549,7 +523,7 @@ func tryAutoAnalyzeTable( // // Exposed for test. func NeedAnalyzeTable(tbl *statistics.Table, autoAnalyzeRatio float64) (bool, string) { - analyzed := TableAnalyzed(tbl) + analyzed := exec.TableAnalyzed(tbl) if !analyzed { return true, "table unanalyzed" } @@ -568,21 +542,6 @@ func NeedAnalyzeTable(tbl *statistics.Table, autoAnalyzeRatio float64) (bool, st return true, fmt.Sprintf("too many modifications(%v/%v>%v)", tbl.ModifyCount, tblCnt, autoAnalyzeRatio) } -// TableAnalyzed checks if any column or index of the table has been analyzed. -func TableAnalyzed(tbl *statistics.Table) bool { - for _, col := range tbl.Columns { - if col.IsAnalyzed() { - return true - } - } - for _, idx := range tbl.Indices { - if idx.IsAnalyzed() { - return true - } - } - return false -} - // It is very similar to tryAutoAnalyzeTable, but it commits the analyze job in batch for partitions. func tryAutoAnalyzePartitionTableInDynamicMode( sctx sessionctx.Context, @@ -604,7 +563,7 @@ func tryAutoAnalyzePartitionTableInDynamicMode( // Pseudo statistics can be created by the optimizer, so we need to double check it. // 2. If the table is too small, we don't want to waste time to analyze it. // Leave the opportunity to other bigger tables. - if partitionStats == nil || partitionStats.Pseudo || partitionStats.RealtimeCount < AutoAnalyzeMinCnt { + if partitionStats == nil || partitionStats.Pseudo || partitionStats.RealtimeCount < exec.AutoAnalyzeMinCnt { continue } if needAnalyze, reason := NeedAnalyzeTable( diff --git a/pkg/statistics/handle/autoanalyze/autoanalyze_test.go b/pkg/statistics/handle/autoanalyze/autoanalyze_test.go index ea3ee1baa272f..cb799c0aaf947 100644 --- a/pkg/statistics/handle/autoanalyze/autoanalyze_test.go +++ b/pkg/statistics/handle/autoanalyze/autoanalyze_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze" + "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/exec" statsutil "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/statistics/handle/util/test" "github.com/pingcap/tidb/pkg/testkit" @@ -54,9 +55,9 @@ func TestAutoAnalyzeLockedTable(t *testing.T) { tk.MustExec("lock stats t") is := dom.InfoSchema() require.NoError(t, h.Update(is)) - autoanalyze.AutoAnalyzeMinCnt = 0 + exec.AutoAnalyzeMinCnt = 0 defer func() { - autoanalyze.AutoAnalyzeMinCnt = 1000 + exec.AutoAnalyzeMinCnt = 1000 }() // Try to analyze the locked table, it should not analyze the table. require.False(t, dom.StatsHandle().HandleAutoAnalyze()) @@ -82,9 +83,9 @@ func TestDisableAutoAnalyze(t *testing.T) { // Set auto analyze ratio to 0. tk.MustExec("set @@global.tidb_auto_analyze_ratio = 0") - autoanalyze.AutoAnalyzeMinCnt = 0 + exec.AutoAnalyzeMinCnt = 0 defer func() { - autoanalyze.AutoAnalyzeMinCnt = 1000 + exec.AutoAnalyzeMinCnt = 1000 }() // Even auto analyze ratio is set to 0, we still need to analyze the unanalyzed tables. require.True(t, dom.StatsHandle().HandleAutoAnalyze()) @@ -107,9 +108,9 @@ func TestAutoAnalyzeOnChangeAnalyzeVer(t *testing.T) { tk.MustExec("insert into t values(1)") tk.MustExec("set @@global.tidb_analyze_version = 1") do := dom - autoanalyze.AutoAnalyzeMinCnt = 0 + exec.AutoAnalyzeMinCnt = 0 defer func() { - autoanalyze.AutoAnalyzeMinCnt = 1000 + exec.AutoAnalyzeMinCnt = 1000 }() h := do.StatsHandle() err := h.HandleDDLEvent(<-h.DDLEventCh()) @@ -184,12 +185,12 @@ func TestTableAnalyzed(t *testing.T) { require.NoError(t, h.Update(is)) statsTbl := h.GetTableStats(tableInfo) - require.False(t, autoanalyze.TableAnalyzed(statsTbl)) + require.False(t, exec.TableAnalyzed(statsTbl)) testKit.MustExec("analyze table t") require.NoError(t, h.Update(is)) statsTbl = h.GetTableStats(tableInfo) - require.True(t, autoanalyze.TableAnalyzed(statsTbl)) + require.True(t, exec.TableAnalyzed(statsTbl)) h.Clear() oriLease := h.Lease() @@ -200,7 +201,7 @@ func TestTableAnalyzed(t *testing.T) { }() require.NoError(t, h.Update(is)) statsTbl = h.GetTableStats(tableInfo) - require.True(t, autoanalyze.TableAnalyzed(statsTbl)) + require.True(t, exec.TableAnalyzed(statsTbl)) } func TestNeedAnalyzeTable(t *testing.T) { @@ -287,10 +288,10 @@ func TestAutoAnalyzeSkipColumnTypes(t *testing.T) { require.NoError(t, h.Update(dom.InfoSchema())) tk.MustExec("set @@global.tidb_analyze_skip_column_types = 'json,blob,mediumblob,text,mediumtext'") - originalVal := autoanalyze.AutoAnalyzeMinCnt - autoanalyze.AutoAnalyzeMinCnt = 0 + originalVal := exec.AutoAnalyzeMinCnt + exec.AutoAnalyzeMinCnt = 0 defer func() { - autoanalyze.AutoAnalyzeMinCnt = originalVal + exec.AutoAnalyzeMinCnt = originalVal }() require.True(t, h.HandleAutoAnalyze()) tk.MustQuery("select job_info from mysql.analyze_jobs where job_info like '%auto analyze table%'").Check(testkit.Rows("auto analyze table columns a, b, d with 256 buckets, 500 topn, 1 samplerate")) @@ -319,7 +320,7 @@ func TestAutoAnalyzeOnEmptyTable(t *testing.T) { // to pass the stats.Pseudo check in autoAnalyzeTable tk.MustExec("analyze table t") // to pass the AutoAnalyzeMinCnt check in autoAnalyzeTable - tk.MustExec("insert into t values (1)" + strings.Repeat(", (1)", int(autoanalyze.AutoAnalyzeMinCnt))) + tk.MustExec("insert into t values (1)" + strings.Repeat(", (1)", int(exec.AutoAnalyzeMinCnt))) require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(true)) require.NoError(t, dom.StatsHandle().Update(dom.InfoSchema())) @@ -354,7 +355,7 @@ func TestAutoAnalyzeOutOfSpecifiedTime(t *testing.T) { // to pass the stats.Pseudo check in autoAnalyzeTable tk.MustExec("analyze table t") // to pass the AutoAnalyzeMinCnt check in autoAnalyzeTable - tk.MustExec("insert into t values (1)" + strings.Repeat(", (1)", int(autoanalyze.AutoAnalyzeMinCnt))) + tk.MustExec("insert into t values (1)" + strings.Repeat(", (1)", int(exec.AutoAnalyzeMinCnt))) require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(true)) require.NoError(t, dom.StatsHandle().Update(dom.InfoSchema())) diff --git a/pkg/statistics/handle/autoanalyze/exec/BUILD.bazel b/pkg/statistics/handle/autoanalyze/exec/BUILD.bazel index 02bb9a2300e51..3deb2e9049c3d 100644 --- a/pkg/statistics/handle/autoanalyze/exec/BUILD.bazel +++ b/pkg/statistics/handle/autoanalyze/exec/BUILD.bazel @@ -9,6 +9,7 @@ go_library( "//pkg/metrics", "//pkg/parser/ast", "//pkg/sessionctx", + "//pkg/sessionctx/variable", "//pkg/statistics", "//pkg/statistics/handle/logutil", "//pkg/statistics/handle/types", diff --git a/pkg/statistics/handle/autoanalyze/exec/exec.go b/pkg/statistics/handle/autoanalyze/exec/exec.go index cd8220ad87e9c..6ccf2584b010e 100644 --- a/pkg/statistics/handle/autoanalyze/exec/exec.go +++ b/pkg/statistics/handle/autoanalyze/exec/exec.go @@ -15,11 +15,14 @@ package exec import ( + "math" + "strconv" "time" "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" statslogutil "github.com/pingcap/tidb/pkg/statistics/handle/logutil" statstypes "github.com/pingcap/tidb/pkg/statistics/handle/types" @@ -30,6 +33,10 @@ import ( "go.uber.org/zap" ) +// AutoAnalyzeMinCnt means if the count of table is less than this value, we don't need to do auto analyze. +// Exported for testing. +var AutoAnalyzeMinCnt int64 = 1000 + var execOptionForAnalyze = map[int]sqlexec.OptionFuncAlias{ statistics.Version0: sqlexec.ExecOptionAnalyzeVer1, statistics.Version1: sqlexec.ExecOptionAnalyzeVer1, @@ -85,3 +92,41 @@ func execAnalyzeStmt( } return statsutil.ExecWithOpts(sctx, optFuncs, sql, params...) } + +// TableAnalyzed checks if any column or index of the table has been analyzed. +func TableAnalyzed(tbl *statistics.Table) bool { + for _, col := range tbl.Columns { + if col.IsAnalyzed() { + return true + } + } + for _, idx := range tbl.Indices { + if idx.IsAnalyzed() { + return true + } + } + return false +} + +// GetAutoAnalyzeParameters gets the auto analyze parameters from mysql.global_variables. +func GetAutoAnalyzeParameters(sctx sessionctx.Context) map[string]string { + sql := "select variable_name, variable_value from mysql.global_variables where variable_name in (%?, %?, %?)" + rows, _, err := statsutil.ExecWithOpts(sctx, nil, sql, variable.TiDBAutoAnalyzeRatio, variable.TiDBAutoAnalyzeStartTime, variable.TiDBAutoAnalyzeEndTime) + if err != nil { + return map[string]string{} + } + parameters := make(map[string]string, len(rows)) + for _, row := range rows { + parameters[row.GetString(0)] = row.GetString(1) + } + return parameters +} + +// ParseAutoAnalyzeRatio parses the auto analyze ratio from the string. +func ParseAutoAnalyzeRatio(ratio string) float64 { + autoAnalyzeRatio, err := strconv.ParseFloat(ratio, 64) + if err != nil { + return variable.DefAutoAnalyzeRatio + } + return math.Max(autoAnalyzeRatio, 0) +} diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/BUILD.bazel b/pkg/statistics/handle/autoanalyze/priorityqueue/BUILD.bazel index 80700586b6a1c..287a6a6d32b9b 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/BUILD.bazel +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/BUILD.bazel @@ -3,6 +3,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "priorityqueue", srcs = [ + "calculator.go", "interval.go", "job.go", "queue.go", diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/calculator.go b/pkg/statistics/handle/autoanalyze/priorityqueue/calculator.go new file mode 100644 index 0000000000000..dcf23844d5184 --- /dev/null +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/calculator.go @@ -0,0 +1,36 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package priorityqueue + +// WeightCalculator is an interface for calculating weights of analysis jobs. +type WeightCalculator interface { + CalculateWeight(job *TableAnalysisJob) float64 +} + +// PriorityCalculator implements the WeightCalculator interface. +type PriorityCalculator struct { + threshold float64 +} + +// NewPriorityCalculator creates a new PriorityCalculator with the given threshold. +func NewPriorityCalculator(threshold float64) *PriorityCalculator { + return &PriorityCalculator{threshold: threshold} +} + +// CalculateWeight calculates the weight based on the given rules. +func (*PriorityCalculator) CalculateWeight(_ *TableAnalysisJob) float64 { + // TODO: implement the weight calculation + return 1 +} diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/job.go b/pkg/statistics/handle/autoanalyze/priorityqueue/job.go index de8d35be6b7aa..31e3a2f93823b 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/job.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/job.go @@ -50,11 +50,13 @@ type TableAnalysisJob struct { // Only set when table's indexes need to be analyzed. Indexes []string // Only set when table's partitions need to be analyzed. - Partitions []string - TableID int64 - TableStatsVer int - ChangePercentage float64 - Weight float64 + Partitions []string + TableID int64 + TableStatsVer int + ChangePercentage float64 + TableSize float64 + LastAnalysisDuration time.Duration + Weight float64 } // IsValidToAnalyze checks whether the table is valid to analyze. diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/queue.go b/pkg/statistics/handle/autoanalyze/priorityqueue/queue.go index e902c6a9c3be8..380f2c0b37c1a 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/queue.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/queue.go @@ -23,9 +23,11 @@ type AnalysisPriorityQueue struct { // NewAnalysisPriorityQueue creates a new AnalysisPriorityQueue. func NewAnalysisPriorityQueue() *AnalysisPriorityQueue { - return &AnalysisPriorityQueue{ + q := &AnalysisPriorityQueue{ inner: &analysisInnerQueue{}, } + heap.Init(q.inner) + return q } // Push adds a job to the priority queue with the given weight. diff --git a/pkg/statistics/handle/autoanalyze/refresher/BUILD.bazel b/pkg/statistics/handle/autoanalyze/refresher/BUILD.bazel index 787a5a943b06a..e1112d37bdf80 100644 --- a/pkg/statistics/handle/autoanalyze/refresher/BUILD.bazel +++ b/pkg/statistics/handle/autoanalyze/refresher/BUILD.bazel @@ -6,10 +6,19 @@ go_library( importpath = "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/refresher", visibility = ["//visibility:public"], deps = [ + "//pkg/infoschema", + "//pkg/parser/model", "//pkg/sessionctx", + "//pkg/sessionctx/variable", + "//pkg/statistics", + "//pkg/statistics/handle/autoanalyze/exec", "//pkg/statistics/handle/autoanalyze/priorityqueue", + "//pkg/statistics/handle/lockstats", "//pkg/statistics/handle/logutil", "//pkg/statistics/handle/types", + "//pkg/statistics/handle/util", + "//pkg/util", + "@com_github_tikv_client_go_v2//oracle", "@org_uber_go_zap//:zap", ], ) @@ -20,10 +29,14 @@ go_test( srcs = ["refresher_test.go"], embed = [":refresher"], flaky = True, + shard_count = 6, deps = [ "//pkg/parser/model", + "//pkg/statistics", + "//pkg/statistics/handle/autoanalyze/exec", "//pkg/statistics/handle/autoanalyze/priorityqueue", "//pkg/testkit", "@com_github_stretchr_testify//require", + "@com_github_tikv_client_go_v2//oracle", ], ) diff --git a/pkg/statistics/handle/autoanalyze/refresher/refresher.go b/pkg/statistics/handle/autoanalyze/refresher/refresher.go index 1081985ccf1ae..32b9fb3b02cb1 100644 --- a/pkg/statistics/handle/autoanalyze/refresher/refresher.go +++ b/pkg/statistics/handle/autoanalyze/refresher/refresher.go @@ -15,10 +15,22 @@ package refresher import ( + "strings" + "time" + + "github.com/pingcap/tidb/pkg/infoschema" + "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/statistics" + "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/exec" "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/priorityqueue" + "github.com/pingcap/tidb/pkg/statistics/handle/lockstats" statslogutil "github.com/pingcap/tidb/pkg/statistics/handle/logutil" 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/tikv/client-go/v2/oracle" "go.uber.org/zap" ) @@ -87,3 +99,192 @@ func (r *Refresher) pickOneTableAndAnalyzeByPriority() { return } } + +func (r *Refresher) rebuildTableAnalysisJobQueue() error { + // Reset the priority queue. + r.jobs = priorityqueue.NewAnalysisPriorityQueue() + + if err := statsutil.CallWithSCtx( + r.statsHandle.SPool(), + func(sctx sessionctx.Context) error { + parameters := exec.GetAutoAnalyzeParameters(sctx) + autoAnalyzeRatio := exec.ParseAutoAnalyzeRatio(parameters[variable.TiDBAutoAnalyzeRatio]) + calculator := priorityqueue.NewPriorityCalculator(autoAnalyzeRatio) + is := sctx.GetDomainInfoSchema().(infoschema.InfoSchema) + // Query locked tables once to minimize overhead. + // Outdated lock info is acceptable as we verify table lock status pre-analysis. + lockedTables, err := lockstats.QueryLockedTables(sctx) + if err != nil { + return err + } + // Get current timestamp from the session context. + currentTs, err := getStartTs(sctx) + if err != nil { + return err + } + + dbs := infoschema.AllSchemaNames(is) + for _, db := range dbs { + // Ignore the memory and system database. + if util.IsMemOrSysDB(strings.ToLower(db)) { + continue + } + + tbls := is.SchemaTables(model.NewCIStr(db)) + // We need to check every partition of every table to see if it needs to be analyzed. + for _, tbl := range tbls { + // If table locked, skip analyze all partitions of the table. + if _, ok := lockedTables[tbl.Meta().ID]; ok { + continue + } + + tblInfo := tbl.Meta() + if tblInfo.IsView() { + continue + } + pi := tblInfo.GetPartitionInfo() + pushJobFunc := func(job *priorityqueue.TableAnalysisJob) { + if job == nil { + return + } + // Calculate the weight of the job. + job.Weight = calculator.CalculateWeight(job) + if job.Weight == 0 { + return + } + // Push the job onto the queue. + r.jobs.Push(job) + } + // No partitions or prune mode is static, analyze the whole table. + if pi == nil { + job := createTableAnalysisJob( + sctx, + db, + tblInfo, + r.statsHandle.GetPartitionStats(tblInfo, tblInfo.ID), + autoAnalyzeRatio, + currentTs, + ) + pushJobFunc(job) + } + } + } + + return nil + }, + statsutil.FlagWrapTxn, + ); err != nil { + return err + } + + return nil +} + +func createTableAnalysisJob( + sctx sessionctx.Context, + tableSchema string, + tblInfo *model.TableInfo, + tblStats *statistics.Table, + autoAnalyzeRatio float64, + currentTs uint64, +) *priorityqueue.TableAnalysisJob { + tableStatsVer := sctx.GetSessionVars().AnalyzeVersion + statistics.CheckAnalyzeVerOnTable(tblStats, &tableStatsVer) + + changePercentage := calculateChangePercentage(tblStats, autoAnalyzeRatio) + tableSize := calculateTableSize(tblInfo, tblStats) + lastAnalysisDuration := getTableLastAnalyzeDuration(tblStats, currentTs) + indexes := checkIndexesNeedAnalyze(tblInfo, tblStats) + + job := &priorityqueue.TableAnalysisJob{ + TableID: tblInfo.ID, + TableSchema: tableSchema, + TableName: tblInfo.Name.O, + TableStatsVer: tableStatsVer, + ChangePercentage: changePercentage, + TableSize: tableSize, + LastAnalysisDuration: lastAnalysisDuration, + Indexes: indexes, + } + + return job +} + +func calculateChangePercentage( + tblStats *statistics.Table, + autoAnalyzeRatio float64, +) float64 { + // If the stats are not loaded, we don't need to analyze it. + // If the table is too small, we don't want to waste time to analyze it. + // Leave the opportunity to other bigger tables. + if tblStats.Pseudo || tblStats.RealtimeCount < exec.AutoAnalyzeMinCnt { + return 0 + } + + if !exec.TableAnalyzed(tblStats) { + return 1 + } + + tblCnt := float64(tblStats.RealtimeCount) + if histCnt := tblStats.GetAnalyzeRowCount(); histCnt > 0 { + tblCnt = histCnt + } + res := float64(tblStats.ModifyCount) / tblCnt + if res > autoAnalyzeRatio { + return res + } + + return 0 +} + +func calculateTableSize( + tblInfo *model.TableInfo, + tblStats *statistics.Table, +) float64 { + tblCnt := float64(tblStats.RealtimeCount) + // TODO: Ignore unanalyzable columns. + colCnt := float64(len(tblInfo.Columns)) + + return tblCnt * colCnt +} + +func getTableLastAnalyzeDuration( + tblStats *statistics.Table, + currentTs uint64, +) time.Duration { + // Calculate the duration since last analyze. + versionTs := tblStats.Version + currentTime := oracle.GetTimeFromTS(currentTs) + versionTime := oracle.GetTimeFromTS(versionTs) + + return time.Duration(currentTime.Sub(versionTime).Seconds()) +} + +func checkIndexesNeedAnalyze( + tblInfo *model.TableInfo, + tblStats *statistics.Table, +) []string { + // If table is not analyzed, we need to analyze whole table. + // So we don't need to check indexes. + if !exec.TableAnalyzed(tblStats) { + return nil + } + + indexes := make([]string, 0, len(tblInfo.Indices)) + // Check if missing index stats. + for _, idx := range tblInfo.Indices { + if _, ok := tblStats.Indices[idx.ID]; !ok && idx.State == model.StatePublic { + indexes = append(indexes, idx.Name.O) + } + } + + return indexes +} + +func getStartTs(sctx sessionctx.Context) (uint64, error) { + txn, err := sctx.Txn(true) + if err != nil { + return 0, err + } + return txn.StartTS(), nil +} diff --git a/pkg/statistics/handle/autoanalyze/refresher/refresher_test.go b/pkg/statistics/handle/autoanalyze/refresher/refresher_test.go index 8a69864312268..ba3a9f11a7911 100644 --- a/pkg/statistics/handle/autoanalyze/refresher/refresher_test.go +++ b/pkg/statistics/handle/autoanalyze/refresher/refresher_test.go @@ -16,11 +16,15 @@ package refresher import ( "testing" + "time" "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/statistics" + "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/exec" "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/priorityqueue" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" + "github.com/tikv/client-go/v2/oracle" ) func TestPickOneTableAndAnalyzeByPriority(t *testing.T) { @@ -175,3 +179,200 @@ func insertFailedJobForPartitionWithStartTime( startTime, ) } + +func TestRebuildTableAnalysisJobQueue(t *testing.T) { + old := exec.AutoAnalyzeMinCnt + defer func() { + exec.AutoAnalyzeMinCnt = old + }() + exec.AutoAnalyzeMinCnt = 0 + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t1 (a int, b int, index idx(a))") + tk.MustExec("insert into t1 values (1, 1), (2, 2), (3, 3)") + handle := dom.StatsHandle() + require.Nil(t, handle.DumpStatsDeltaToKV(true)) + tk.MustExec("analyze table t1") + require.Nil(t, handle.Update(dom.InfoSchema())) + + sysProcTracker := dom.SysProcTracker() + r, err := NewRefresher(handle, sysProcTracker) + require.NoError(t, err) + + // Rebuild the job queue. + err = r.rebuildTableAnalysisJobQueue() + require.NoError(t, err) + require.Equal(t, 1, r.jobs.Len()) + job1 := r.jobs.Pop() + require.Equal(t, float64(1), job1.Weight) + require.Equal(t, float64(0), job1.ChangePercentage) + require.Equal(t, float64(3*2), job1.TableSize) + require.GreaterOrEqual(t, job1.LastAnalysisDuration, time.Duration(0)) + // Insert more data into t1. + tk.MustExec("insert into t1 values (4, 4), (5, 5), (6, 6)") + require.Nil(t, handle.DumpStatsDeltaToKV(true)) + require.Nil(t, handle.Update(dom.InfoSchema())) + err = r.rebuildTableAnalysisJobQueue() + require.NoError(t, err) + require.Equal(t, 1, r.jobs.Len()) + job1 = r.jobs.Pop() + require.Equal(t, float64(1), job1.Weight) + require.Equal(t, float64(1), job1.ChangePercentage) + require.Equal(t, float64(6*2), job1.TableSize) + require.GreaterOrEqual(t, job1.LastAnalysisDuration, time.Duration(0)) +} + +func TestCalculateChangePercentage(t *testing.T) { + unanalyzedColumns := map[int64]*statistics.Column{ + 1: {}, + 2: {}, + } + unanalyzedIndices := map[int64]*statistics.Index{ + 1: {}, + 2: {}, + } + analyzedColumns := map[int64]*statistics.Column{ + 1: { + StatsVer: 2, + }, + 2: { + StatsVer: 2, + }, + } + analyzedIndices := map[int64]*statistics.Index{ + 1: { + StatsVer: 2, + }, + 2: { + StatsVer: 2, + }, + } + tests := []struct { + name string + tblStats *statistics.Table + autoAnalyzeRatio float64 + want float64 + }{ + { + name: "Test Pseudo", + tblStats: &statistics.Table{ + HistColl: statistics.HistColl{ + Pseudo: true, + }, + }, + autoAnalyzeRatio: 0.5, + want: 0, + }, + { + name: "Test RealtimeCount less than AutoAnalyzeMinCnt", + tblStats: &statistics.Table{ + HistColl: statistics.HistColl{ + RealtimeCount: exec.AutoAnalyzeMinCnt - 1, + }, + }, + autoAnalyzeRatio: 0.5, + want: 0, + }, + { + name: "Test Table not analyzed", + tblStats: &statistics.Table{ + HistColl: statistics.HistColl{ + Pseudo: false, + RealtimeCount: exec.AutoAnalyzeMinCnt + 1, + Columns: unanalyzedColumns, + Indices: unanalyzedIndices, + }, + }, + autoAnalyzeRatio: 0.5, + want: 1, + }, + { + name: "Based on change percentage", + tblStats: &statistics.Table{ + HistColl: statistics.HistColl{ + Pseudo: false, + RealtimeCount: exec.AutoAnalyzeMinCnt + 1, + Columns: analyzedColumns, + Indices: analyzedIndices, + ModifyCount: (exec.AutoAnalyzeMinCnt + 1) * 2, + }, + }, + autoAnalyzeRatio: 0.5, + want: 2, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + got := calculateChangePercentage(tt.tblStats, tt.autoAnalyzeRatio) + require.Equal(t, tt.want, got) + }) + } +} + +func TestGetTableLastAnalyzeDuration(t *testing.T) { + tblStats := &statistics.Table{ + Version: oracle.ComposeTS(time.Hour.Nanoseconds()*1000, 0), + } + currentTs := oracle.ComposeTS((time.Hour.Nanoseconds()+time.Second.Nanoseconds())*1000, 0) + want := time.Second + + got := getTableLastAnalyzeDuration(tblStats, currentTs) + require.Equal(t, want, got) +} + +func TestCheckIndexesNeedAnalyze(t *testing.T) { + tests := []struct { + name string + tblInfo *model.TableInfo + tblStats *statistics.Table + want []string + }{ + { + name: "Test Table not analyzed", + tblInfo: &model.TableInfo{ + Indices: []*model.IndexInfo{ + { + ID: 1, + Name: model.NewCIStr("index1"), + State: model.StatePublic, + }, + }, + }, + tblStats: &statistics.Table{}, + want: nil, + }, + { + name: "Test Index not analyzed", + tblInfo: &model.TableInfo{ + Indices: []*model.IndexInfo{ + { + ID: 1, + Name: model.NewCIStr("index1"), + State: model.StatePublic, + }, + }, + }, + tblStats: &statistics.Table{ + HistColl: statistics.HistColl{ + Pseudo: false, + Indices: map[int64]*statistics.Index{}, + Columns: map[int64]*statistics.Column{ + 1: { + StatsVer: 2, + }, + }, + }, + }, + want: []string{"index1"}, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + got := checkIndexesNeedAnalyze(tt.tblInfo, tt.tblStats) + require.Equal(t, tt.want, got) + }) + } +} diff --git a/pkg/statistics/handle/updatetest/BUILD.bazel b/pkg/statistics/handle/updatetest/BUILD.bazel index 339906f411962..7f50f975a2b73 100644 --- a/pkg/statistics/handle/updatetest/BUILD.bazel +++ b/pkg/statistics/handle/updatetest/BUILD.bazel @@ -17,7 +17,7 @@ go_test( "//pkg/sessionctx/stmtctx", "//pkg/sessionctx/variable", "//pkg/statistics", - "//pkg/statistics/handle/autoanalyze", + "//pkg/statistics/handle/autoanalyze/exec", "//pkg/statistics/handle/usage", "//pkg/statistics/handle/util", "//pkg/testkit", diff --git a/pkg/statistics/handle/updatetest/update_test.go b/pkg/statistics/handle/updatetest/update_test.go index e73ed97a7fd2b..c087d01304fce 100644 --- a/pkg/statistics/handle/updatetest/update_test.go +++ b/pkg/statistics/handle/updatetest/update_test.go @@ -29,7 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" - "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze" + "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/exec" "github.com/pingcap/tidb/pkg/statistics/handle/usage" "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/testkit" @@ -364,10 +364,10 @@ func TestAutoUpdate(t *testing.T) { testKit.MustExec("use test") testKit.MustExec("create table t (a varchar(20))") - autoanalyze.AutoAnalyzeMinCnt = 0 + exec.AutoAnalyzeMinCnt = 0 testKit.MustExec("set global tidb_auto_analyze_ratio = 0.2") defer func() { - autoanalyze.AutoAnalyzeMinCnt = 1000 + exec.AutoAnalyzeMinCnt = 1000 testKit.MustExec("set global tidb_auto_analyze_ratio = 0.0") }() @@ -469,10 +469,10 @@ func TestAutoUpdatePartition(t *testing.T) { testKit.MustExec("create table t (a int) PARTITION BY RANGE (a) (PARTITION p0 VALUES LESS THAN (6))") testKit.MustExec("analyze table t") - autoanalyze.AutoAnalyzeMinCnt = 0 + exec.AutoAnalyzeMinCnt = 0 testKit.MustExec("set global tidb_auto_analyze_ratio = 0.6") defer func() { - autoanalyze.AutoAnalyzeMinCnt = 1000 + exec.AutoAnalyzeMinCnt = 1000 testKit.MustExec("set global tidb_auto_analyze_ratio = 0.0") }() @@ -514,7 +514,7 @@ func TestIssue25700(t *testing.T) { tk.MustExec("drop table if exists t") tk.MustExec("CREATE TABLE `t` ( `ldecimal` decimal(32,4) DEFAULT NULL, `rdecimal` decimal(32,4) DEFAULT NULL, `gen_col` decimal(36,4) GENERATED ALWAYS AS (`ldecimal` + `rdecimal`) VIRTUAL, `col_timestamp` timestamp(3) NULL DEFAULT NULL ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;") tk.MustExec("analyze table t") - tk.MustExec("INSERT INTO `t` (`ldecimal`, `rdecimal`, `col_timestamp`) VALUES (2265.2200, 9843.4100, '1999-12-31 16:00:00')" + strings.Repeat(", (2265.2200, 9843.4100, '1999-12-31 16:00:00')", int(autoanalyze.AutoAnalyzeMinCnt))) + tk.MustExec("INSERT INTO `t` (`ldecimal`, `rdecimal`, `col_timestamp`) VALUES (2265.2200, 9843.4100, '1999-12-31 16:00:00')" + strings.Repeat(", (2265.2200, 9843.4100, '1999-12-31 16:00:00')", int(exec.AutoAnalyzeMinCnt))) require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(true)) require.NoError(t, dom.StatsHandle().Update(dom.InfoSchema())) @@ -804,10 +804,10 @@ func TestAutoUpdatePartitionInDynamicOnlyMode(t *testing.T) { testKit.MustExec("set @@tidb_analyze_version = 2") testKit.MustExec("analyze table t") - autoanalyze.AutoAnalyzeMinCnt = 0 + exec.AutoAnalyzeMinCnt = 0 testKit.MustExec("set global tidb_auto_analyze_ratio = 0.1") defer func() { - autoanalyze.AutoAnalyzeMinCnt = 1000 + exec.AutoAnalyzeMinCnt = 1000 testKit.MustExec("set global tidb_auto_analyze_ratio = 0.0") }() @@ -850,9 +850,9 @@ func TestAutoAnalyzeRatio(t *testing.T) { oriStart := tk.MustQuery("select @@tidb_auto_analyze_start_time").Rows()[0][0].(string) oriEnd := tk.MustQuery("select @@tidb_auto_analyze_end_time").Rows()[0][0].(string) - autoanalyze.AutoAnalyzeMinCnt = 0 + exec.AutoAnalyzeMinCnt = 0 defer func() { - autoanalyze.AutoAnalyzeMinCnt = 1000 + exec.AutoAnalyzeMinCnt = 1000 tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_start_time='%v'", oriStart)) tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_end_time='%v'", oriEnd)) }() @@ -1078,9 +1078,9 @@ func TestStatsLockUnlockForAutoAnalyze(t *testing.T) { oriStart := tk.MustQuery("select @@tidb_auto_analyze_start_time").Rows()[0][0].(string) oriEnd := tk.MustQuery("select @@tidb_auto_analyze_end_time").Rows()[0][0].(string) - autoanalyze.AutoAnalyzeMinCnt = 0 + exec.AutoAnalyzeMinCnt = 0 defer func() { - autoanalyze.AutoAnalyzeMinCnt = 1000 + exec.AutoAnalyzeMinCnt = 1000 tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_start_time='%v'", oriStart)) tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_end_time='%v'", oriEnd)) }() @@ -1278,15 +1278,15 @@ func TestNotDumpSysTable(t *testing.T) { func TestAutoAnalyzePartitionTableAfterAddingIndex(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) - oriMinCnt := autoanalyze.AutoAnalyzeMinCnt + oriMinCnt := exec.AutoAnalyzeMinCnt oriStart := tk.MustQuery("select @@tidb_auto_analyze_start_time").Rows()[0][0].(string) oriEnd := tk.MustQuery("select @@tidb_auto_analyze_end_time").Rows()[0][0].(string) defer func() { - autoanalyze.AutoAnalyzeMinCnt = oriMinCnt + exec.AutoAnalyzeMinCnt = oriMinCnt tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_start_time='%v'", oriStart)) tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_end_time='%v'", oriEnd)) }() - autoanalyze.AutoAnalyzeMinCnt = 0 + exec.AutoAnalyzeMinCnt = 0 tk.MustExec("set global tidb_auto_analyze_start_time='00:00 +0000'") tk.MustExec("set global tidb_auto_analyze_end_time='23:59 +0000'") tk.MustExec("set global tidb_analyze_version = 2") diff --git a/pkg/statistics/integration_test.go b/pkg/statistics/integration_test.go index 59cc2ac9ff050..700fefa2e24ad 100644 --- a/pkg/statistics/integration_test.go +++ b/pkg/statistics/integration_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/statistics" - "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze" + "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/exec" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testdata" "github.com/stretchr/testify/require" @@ -306,9 +306,9 @@ func TestOutdatedStatsCheck(t *testing.T) { oriStart := tk.MustQuery("select @@tidb_auto_analyze_start_time").Rows()[0][0].(string) oriEnd := tk.MustQuery("select @@tidb_auto_analyze_end_time").Rows()[0][0].(string) - autoanalyze.AutoAnalyzeMinCnt = 0 + exec.AutoAnalyzeMinCnt = 0 defer func() { - autoanalyze.AutoAnalyzeMinCnt = 1000 + exec.AutoAnalyzeMinCnt = 1000 tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_start_time='%v'", oriStart)) tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_end_time='%v'", oriEnd)) }() diff --git a/pkg/ttl/ttlworker/BUILD.bazel b/pkg/ttl/ttlworker/BUILD.bazel index 19c1d0636af7c..29a8505d6c5a2 100644 --- a/pkg/ttl/ttlworker/BUILD.bazel +++ b/pkg/ttl/ttlworker/BUILD.bazel @@ -82,7 +82,7 @@ go_test( "//pkg/session", "//pkg/sessionctx", "//pkg/sessionctx/variable", - "//pkg/statistics/handle/autoanalyze", + "//pkg/statistics/handle/autoanalyze/exec", "//pkg/store/mockstore", "//pkg/testkit", "//pkg/timer/api", diff --git a/pkg/ttl/ttlworker/job_manager_integration_test.go b/pkg/ttl/ttlworker/job_manager_integration_test.go index 5e76506a1c103..ef2fe34692ac4 100644 --- a/pkg/ttl/ttlworker/job_manager_integration_test.go +++ b/pkg/ttl/ttlworker/job_manager_integration_test.go @@ -32,7 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" dbsession "github.com/pingcap/tidb/pkg/session" - "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze" + "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/exec" "github.com/pingcap/tidb/pkg/testkit" timerapi "github.com/pingcap/tidb/pkg/timer/api" timertable "github.com/pingcap/tidb/pkg/timer/tablestore" @@ -178,10 +178,10 @@ func TestTTLAutoAnalyze(t *testing.T) { failpoint.Enable("github.com/pingcap/tidb/pkg/ttl/ttlworker/task-manager-loop-interval", fmt.Sprintf("return(%d)", time.Second)) defer failpoint.Disable("github.com/pingcap/tidb/pkg/ttl/ttlworker/task-manager-loop-interval") - originAutoAnalyzeMinCnt := autoanalyze.AutoAnalyzeMinCnt - autoanalyze.AutoAnalyzeMinCnt = 0 + originAutoAnalyzeMinCnt := exec.AutoAnalyzeMinCnt + exec.AutoAnalyzeMinCnt = 0 defer func() { - autoanalyze.AutoAnalyzeMinCnt = originAutoAnalyzeMinCnt + exec.AutoAnalyzeMinCnt = originAutoAnalyzeMinCnt }() store, dom := testkit.CreateMockStoreAndDomain(t) @@ -364,10 +364,10 @@ func TestTTLJobDisable(t *testing.T) { failpoint.Enable("github.com/pingcap/tidb/pkg/ttl/ttlworker/resize-workers-interval", fmt.Sprintf("return(%d)", time.Second)) defer failpoint.Disable("github.com/pingcap/tidb/pkg/ttl/ttlworker/resize-workers-interval") - originAutoAnalyzeMinCnt := autoanalyze.AutoAnalyzeMinCnt - autoanalyze.AutoAnalyzeMinCnt = 0 + originAutoAnalyzeMinCnt := exec.AutoAnalyzeMinCnt + exec.AutoAnalyzeMinCnt = 0 defer func() { - autoanalyze.AutoAnalyzeMinCnt = originAutoAnalyzeMinCnt + exec.AutoAnalyzeMinCnt = originAutoAnalyzeMinCnt }() store, dom := testkit.CreateMockStoreAndDomain(t)