From 7e73ddc91b5f9f089e84f0dc645cf71a27b2ad50 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Fri, 26 Jul 2024 09:36:06 +0800 Subject: [PATCH] statistics: add metrics for unneeded analyze table (#54822) close pingcap/tidb#54823 --- pkg/executor/test/analyzetest/BUILD.bazel | 1 - pkg/executor/test/analyzetest/analyze_test.go | 27 +++++------ .../analyzetest/memorycontrol/BUILD.bazel | 2 +- .../memorycontrol/memory_control_test.go | 8 ++-- pkg/statistics/BUILD.bazel | 1 - pkg/statistics/handle/autoanalyze/BUILD.bazel | 1 - .../handle/autoanalyze/autoanalyze.go | 4 +- .../handle/autoanalyze/autoanalyze_test.go | 31 ++++++------ .../handle/autoanalyze/exec/exec.go | 4 -- .../handle/autoanalyze/refresher/BUILD.bazel | 1 - .../handle/autoanalyze/refresher/refresher.go | 22 ++------- .../autoanalyze/refresher/refresher_test.go | 47 +++++++++---------- pkg/statistics/handle/cache/statscache.go | 9 +++- pkg/statistics/handle/metrics/metrics.go | 1 + pkg/statistics/handle/updatetest/BUILD.bazel | 1 - .../handle/updatetest/update_test.go | 29 ++++++------ pkg/statistics/integration_test.go | 5 +- pkg/statistics/table.go | 17 +++++++ pkg/ttl/ttlworker/BUILD.bazel | 2 +- .../ttlworker/job_manager_integration_test.go | 14 +++--- 20 files changed, 112 insertions(+), 115 deletions(-) diff --git a/pkg/executor/test/analyzetest/BUILD.bazel b/pkg/executor/test/analyzetest/BUILD.bazel index ea5f921883358..40d1a1d73ec94 100644 --- a/pkg/executor/test/analyzetest/BUILD.bazel +++ b/pkg/executor/test/analyzetest/BUILD.bazel @@ -26,7 +26,6 @@ go_test( "//pkg/sessionctx", "//pkg/sessionctx/variable", "//pkg/statistics", - "//pkg/statistics/handle/autoanalyze/exec", "//pkg/testkit", "//pkg/testkit/analyzehelper", "//pkg/util/dbterror/exeerrors", diff --git a/pkg/executor/test/analyzetest/analyze_test.go b/pkg/executor/test/analyzetest/analyze_test.go index 87bd60a792983..2add844138bd3 100644 --- a/pkg/executor/test/analyzetest/analyze_test.go +++ b/pkg/executor/test/analyzetest/analyze_test.go @@ -39,7 +39,6 @@ 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/exec" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/analyzehelper" "github.com/pingcap/tidb/pkg/util/dbterror/exeerrors" @@ -703,11 +702,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 := exec.AutoAnalyzeMinCnt + originalVal3 := statistics.AutoAnalyzeMinCnt defer func() { - exec.AutoAnalyzeMinCnt = originalVal3 + statistics.AutoAnalyzeMinCnt = originalVal3 }() - exec.AutoAnalyzeMinCnt = 0 + statistics.AutoAnalyzeMinCnt = 0 tk.MustExec("use test") tk.MustExec("set @@session.tidb_analyze_version = 2") @@ -1046,11 +1045,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 := exec.AutoAnalyzeMinCnt + originalVal3 := statistics.AutoAnalyzeMinCnt defer func() { - exec.AutoAnalyzeMinCnt = originalVal3 + statistics.AutoAnalyzeMinCnt = originalVal3 }() - exec.AutoAnalyzeMinCnt = 0 + statistics.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)) @@ -1888,9 +1887,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) - exec.AutoAnalyzeMinCnt = 0 + statistics.AutoAnalyzeMinCnt = 0 defer func() { - exec.AutoAnalyzeMinCnt = 1000 + statistics.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)) }() @@ -1972,9 +1971,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) - exec.AutoAnalyzeMinCnt = 0 + statistics.AutoAnalyzeMinCnt = 0 defer func() { - exec.AutoAnalyzeMinCnt = 1000 + statistics.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)) }() @@ -2733,12 +2732,12 @@ func TestAutoAnalyzeAwareGlobalVariableChange(t *testing.T) { "3 0", )) - originalVal1 := exec.AutoAnalyzeMinCnt + originalVal1 := statistics.AutoAnalyzeMinCnt originalVal2 := tk.MustQuery("select @@global.tidb_auto_analyze_ratio").Rows()[0][0].(string) - exec.AutoAnalyzeMinCnt = 0 + statistics.AutoAnalyzeMinCnt = 0 tk.MustExec("set global tidb_auto_analyze_ratio = 0.001") defer func() { - exec.AutoAnalyzeMinCnt = originalVal1 + statistics.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 793de08dcf3ad..b4f9ed544df6c 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/exec", + "//pkg/statistics", "//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 3687531bb6b38..1982e8a974871 100644 --- a/pkg/executor/test/analyzetest/memorycontrol/memory_control_test.go +++ b/pkg/executor/test/analyzetest/memorycontrol/memory_control_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/executor" - "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/exec" + "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util" "github.com/stretchr/testify/require" @@ -144,12 +144,12 @@ func TestGlobalMemoryControlForAutoAnalyze(t *testing.T) { require.Len(t, rs0.Rows(), 0) h := dom.StatsHandle() - originalVal4 := exec.AutoAnalyzeMinCnt + originalVal4 := statistics.AutoAnalyzeMinCnt originalVal5 := tk.MustQuery("select @@global.tidb_auto_analyze_ratio").Rows()[0][0].(string) - exec.AutoAnalyzeMinCnt = 0 + statistics.AutoAnalyzeMinCnt = 0 tk.MustExec("set global tidb_auto_analyze_ratio = 0.001") defer func() { - exec.AutoAnalyzeMinCnt = originalVal4 + statistics.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 4e0941622d532..dc4fda20fe81f 100644 --- a/pkg/statistics/BUILD.bazel +++ b/pkg/statistics/BUILD.bazel @@ -89,7 +89,6 @@ go_test( "//pkg/parser/mysql", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", - "//pkg/statistics/handle/autoanalyze/exec", "//pkg/testkit", "//pkg/testkit/analyzehelper", "//pkg/testkit/testdata", diff --git a/pkg/statistics/handle/autoanalyze/BUILD.bazel b/pkg/statistics/handle/autoanalyze/BUILD.bazel index 34dabaa1ed867..9e508d61f20fa 100644 --- a/pkg/statistics/handle/autoanalyze/BUILD.bazel +++ b/pkg/statistics/handle/autoanalyze/BUILD.bazel @@ -46,7 +46,6 @@ 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 f4f835bffabcf..7b8e6603005bf 100644 --- a/pkg/statistics/handle/autoanalyze/autoanalyze.go +++ b/pkg/statistics/handle/autoanalyze/autoanalyze.go @@ -461,7 +461,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 < exec.AutoAnalyzeMinCnt { + if statsTbl == nil || statsTbl.Pseudo || statsTbl.RealtimeCount < statistics.AutoAnalyzeMinCnt { return false } @@ -558,7 +558,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 < exec.AutoAnalyzeMinCnt { + if partitionStats == nil || partitionStats.Pseudo || partitionStats.RealtimeCount < statistics.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 2ab081bbce604..efe5a3731b1b0 100644 --- a/pkg/statistics/handle/autoanalyze/autoanalyze_test.go +++ b/pkg/statistics/handle/autoanalyze/autoanalyze_test.go @@ -31,7 +31,6 @@ 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" @@ -58,9 +57,9 @@ func TestEnableAutoAnalyzePriorityQueue(t *testing.T) { require.NoError(t, h.DumpStatsDeltaToKV(true)) is := dom.InfoSchema() require.NoError(t, h.Update(context.Background(), is)) - exec.AutoAnalyzeMinCnt = 0 + statistics.AutoAnalyzeMinCnt = 0 defer func() { - exec.AutoAnalyzeMinCnt = 1000 + statistics.AutoAnalyzeMinCnt = 1000 }() require.True(t, dom.StatsHandle().HandleAutoAnalyze()) } @@ -79,9 +78,9 @@ func TestAutoAnalyzeLockedTable(t *testing.T) { tk.MustExec("lock stats t") is := dom.InfoSchema() require.NoError(t, h.Update(context.Background(), is)) - exec.AutoAnalyzeMinCnt = 0 + statistics.AutoAnalyzeMinCnt = 0 defer func() { - exec.AutoAnalyzeMinCnt = 1000 + statistics.AutoAnalyzeMinCnt = 1000 }() // Try to analyze the locked table, it should not analyze the table. require.False(t, dom.StatsHandle().HandleAutoAnalyze()) @@ -107,9 +106,9 @@ func TestAutoAnalyzeWithPredicateColumns(t *testing.T) { require.NoError(t, h.DumpStatsDeltaToKV(true)) is := dom.InfoSchema() require.NoError(t, h.Update(context.Background(), is)) - exec.AutoAnalyzeMinCnt = 0 + statistics.AutoAnalyzeMinCnt = 0 defer func() { - exec.AutoAnalyzeMinCnt = 1000 + statistics.AutoAnalyzeMinCnt = 1000 }() // Check column_stats_usage. @@ -157,9 +156,9 @@ func disableAutoAnalyzeCase(t *testing.T, tk *testkit.TestKit, dom *domain.Domai require.NoError(t, h.Update(context.Background(), is)) tk.MustExec("set @@global.tidb_enable_auto_analyze = 0") - exec.AutoAnalyzeMinCnt = 0 + statistics.AutoAnalyzeMinCnt = 0 defer func() { - exec.AutoAnalyzeMinCnt = 1000 + statistics.AutoAnalyzeMinCnt = 1000 }() // Even auto analyze ratio is set to 0, we still need to analyze the unanalyzed tables. require.True(t, dom.StatsHandle().HandleAutoAnalyze()) @@ -182,9 +181,9 @@ func TestAutoAnalyzeOnChangeAnalyzeVer(t *testing.T) { tk.MustExec("insert into t values(1)") tk.MustExec("set @@global.tidb_analyze_version = 1") do := dom - exec.AutoAnalyzeMinCnt = 0 + statistics.AutoAnalyzeMinCnt = 0 defer func() { - exec.AutoAnalyzeMinCnt = 1000 + statistics.AutoAnalyzeMinCnt = 1000 }() h := do.StatsHandle() err := h.HandleDDLEvent(<-h.DDLEventCh()) @@ -352,10 +351,10 @@ func TestAutoAnalyzeSkipColumnTypes(t *testing.T) { require.NoError(t, h.DumpColStatsUsageToKV()) tk.MustExec("set @@global.tidb_analyze_skip_column_types = 'json,blob,mediumblob,text,mediumtext'") - originalVal := exec.AutoAnalyzeMinCnt - exec.AutoAnalyzeMinCnt = 0 + originalVal := statistics.AutoAnalyzeMinCnt + statistics.AutoAnalyzeMinCnt = 0 defer func() { - exec.AutoAnalyzeMinCnt = originalVal + statistics.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 all indexes, columns a, b, d with 256 buckets, 100 topn, 1 samplerate")) @@ -384,7 +383,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(exec.AutoAnalyzeMinCnt))) + tk.MustExec("insert into t values (1)" + strings.Repeat(", (1)", int(statistics.AutoAnalyzeMinCnt))) require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(true)) require.NoError(t, dom.StatsHandle().Update(context.Background(), dom.InfoSchema())) @@ -419,7 +418,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(exec.AutoAnalyzeMinCnt))) + tk.MustExec("insert into t values (1)" + strings.Repeat(", (1)", int(statistics.AutoAnalyzeMinCnt))) require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(true)) require.NoError(t, dom.StatsHandle().Update(context.Background(), dom.InfoSchema())) diff --git a/pkg/statistics/handle/autoanalyze/exec/exec.go b/pkg/statistics/handle/autoanalyze/exec/exec.go index a69d5c11ee0b6..8fa9999d525b7 100644 --- a/pkg/statistics/handle/autoanalyze/exec/exec.go +++ b/pkg/statistics/handle/autoanalyze/exec/exec.go @@ -35,10 +35,6 @@ 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, diff --git a/pkg/statistics/handle/autoanalyze/refresher/BUILD.bazel b/pkg/statistics/handle/autoanalyze/refresher/BUILD.bazel index 9b3388602fe81..a26587cd67d9b 100644 --- a/pkg/statistics/handle/autoanalyze/refresher/BUILD.bazel +++ b/pkg/statistics/handle/autoanalyze/refresher/BUILD.bazel @@ -35,7 +35,6 @@ go_test( ":refresher", "//pkg/parser/model", "//pkg/statistics", - "//pkg/statistics/handle/autoanalyze/exec", "//pkg/statistics/handle/autoanalyze/priorityqueue", "//pkg/testkit", "@com_github_stretchr_testify//require", diff --git a/pkg/statistics/handle/autoanalyze/refresher/refresher.go b/pkg/statistics/handle/autoanalyze/refresher/refresher.go index c563bf692d72e..aca694e3fa322 100644 --- a/pkg/statistics/handle/autoanalyze/refresher/refresher.go +++ b/pkg/statistics/handle/autoanalyze/refresher/refresher.go @@ -287,7 +287,7 @@ func CreateTableAnalysisJob( autoAnalyzeRatio float64, currentTs uint64, ) priorityqueue.AnalysisJob { - if !isEligibleForAnalysis(tblStats) { + if !tblStats.IsEligibleForAnalysis() { return nil } @@ -331,7 +331,7 @@ func CreateStaticPartitionAnalysisJob( autoAnalyzeRatio float64, currentTs uint64, ) priorityqueue.AnalysisJob { - if !isEligibleForAnalysis(partitionStats) { + if !partitionStats.IsEligibleForAnalysis() { return nil } @@ -465,7 +465,7 @@ func createTableAnalysisJobForPartitions( autoAnalyzeRatio float64, currentTs uint64, ) priorityqueue.AnalysisJob { - if !isEligibleForAnalysis(tblStats) { + if !tblStats.IsEligibleForAnalysis() { return nil } @@ -611,7 +611,7 @@ func getPartitionStats( for _, def := range defs { stats := statsHandle.GetPartitionStatsForAutoAnalyze(tblInfo, def.ID) // Ignore the partition if it's not ready to analyze. - if !isEligibleForAnalysis(stats) { + if !stats.IsEligibleForAnalysis() { continue } d := PartitionIDAndName{ @@ -624,20 +624,6 @@ func getPartitionStats( return partitionStats } -func isEligibleForAnalysis( - tblStats *statistics.Table, -) bool { - // 1. If the statistics are either not loaded or are classified as pseudo, there is no need for analyze. - // 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 tblStats == nil || tblStats.Pseudo || tblStats.RealtimeCount < exec.AutoAnalyzeMinCnt { - return false - } - - 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 diff --git a/pkg/statistics/handle/autoanalyze/refresher/refresher_test.go b/pkg/statistics/handle/autoanalyze/refresher/refresher_test.go index 76ad5d278a9bb..121de14df00c0 100644 --- a/pkg/statistics/handle/autoanalyze/refresher/refresher_test.go +++ b/pkg/statistics/handle/autoanalyze/refresher/refresher_test.go @@ -23,7 +23,6 @@ import ( "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/statistics/handle/autoanalyze/refresher" "github.com/pingcap/tidb/pkg/testkit" @@ -32,9 +31,9 @@ import ( ) func TestSkipAnalyzeTableWhenAutoAnalyzeRatioIsZero(t *testing.T) { - exec.AutoAnalyzeMinCnt = 0 + statistics.AutoAnalyzeMinCnt = 0 defer func() { - exec.AutoAnalyzeMinCnt = 1000 + statistics.AutoAnalyzeMinCnt = 1000 }() store, dom := testkit.CreateMockStoreAndDomain(t) @@ -84,9 +83,9 @@ func TestSkipAnalyzeTableWhenAutoAnalyzeRatioIsZero(t *testing.T) { } func TestIgnoreNilOrPseudoStatsOfPartitionedTable(t *testing.T) { - exec.AutoAnalyzeMinCnt = 0 + statistics.AutoAnalyzeMinCnt = 0 defer func() { - exec.AutoAnalyzeMinCnt = 1000 + statistics.AutoAnalyzeMinCnt = 1000 }() store, dom := testkit.CreateMockStoreAndDomain(t) @@ -104,9 +103,9 @@ func TestIgnoreNilOrPseudoStatsOfPartitionedTable(t *testing.T) { } func TestIgnoreNilOrPseudoStatsOfNonPartitionedTable(t *testing.T) { - exec.AutoAnalyzeMinCnt = 0 + statistics.AutoAnalyzeMinCnt = 0 defer func() { - exec.AutoAnalyzeMinCnt = 1000 + statistics.AutoAnalyzeMinCnt = 1000 }() store, dom := testkit.CreateMockStoreAndDomain(t) @@ -124,9 +123,9 @@ func TestIgnoreNilOrPseudoStatsOfNonPartitionedTable(t *testing.T) { } func TestIgnoreTinyTable(t *testing.T) { - exec.AutoAnalyzeMinCnt = 10 + statistics.AutoAnalyzeMinCnt = 10 defer func() { - exec.AutoAnalyzeMinCnt = 1000 + statistics.AutoAnalyzeMinCnt = 1000 }() store, dom := testkit.CreateMockStoreAndDomain(t) @@ -168,9 +167,9 @@ func TestIgnoreTinyTable(t *testing.T) { } func TestPickOneTableAndAnalyzeByPriority(t *testing.T) { - exec.AutoAnalyzeMinCnt = 0 + statistics.AutoAnalyzeMinCnt = 0 defer func() { - exec.AutoAnalyzeMinCnt = 1000 + statistics.AutoAnalyzeMinCnt = 1000 }() store, dom := testkit.CreateMockStoreAndDomain(t) @@ -322,11 +321,11 @@ func insertFailedJobForPartitionWithStartTime( } func TestRebuildTableAnalysisJobQueue(t *testing.T) { - old := exec.AutoAnalyzeMinCnt + old := statistics.AutoAnalyzeMinCnt defer func() { - exec.AutoAnalyzeMinCnt = old + statistics.AutoAnalyzeMinCnt = old }() - exec.AutoAnalyzeMinCnt = 0 + statistics.AutoAnalyzeMinCnt = 0 store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") @@ -399,7 +398,7 @@ func TestCalculateChangePercentage(t *testing.T) { { name: "Test Table not analyzed", tblStats: &statistics.Table{ - HistColl: *statistics.NewHistCollWithColsAndIdxs(0, false, exec.AutoAnalyzeMinCnt+1, 0, unanalyzedColumns, unanalyzedIndices), + HistColl: *statistics.NewHistCollWithColsAndIdxs(0, false, statistics.AutoAnalyzeMinCnt+1, 0, unanalyzedColumns, unanalyzedIndices), ColAndIdxExistenceMap: bothUnanalyzedMap, }, autoAnalyzeRatio: 0.5, @@ -408,7 +407,7 @@ func TestCalculateChangePercentage(t *testing.T) { { name: "Based on change percentage", tblStats: &statistics.Table{ - HistColl: *statistics.NewHistCollWithColsAndIdxs(0, false, exec.AutoAnalyzeMinCnt+1, (exec.AutoAnalyzeMinCnt+1)*2, analyzedColumns, analyzedIndices), + HistColl: *statistics.NewHistCollWithColsAndIdxs(0, false, statistics.AutoAnalyzeMinCnt+1, (statistics.AutoAnalyzeMinCnt+1)*2, analyzedColumns, analyzedIndices), ColAndIdxExistenceMap: bothAnalyzedMap, LastAnalyzeVersion: 1, }, @@ -568,7 +567,7 @@ func TestCalculateIndicatorsForPartitions(t *testing.T) { }: { HistColl: statistics.HistColl{ Pseudo: false, - RealtimeCount: exec.AutoAnalyzeMinCnt + 1, + RealtimeCount: statistics.AutoAnalyzeMinCnt + 1, }, ColAndIdxExistenceMap: unanalyzedMap, }, @@ -578,7 +577,7 @@ func TestCalculateIndicatorsForPartitions(t *testing.T) { }: { HistColl: statistics.HistColl{ Pseudo: false, - RealtimeCount: exec.AutoAnalyzeMinCnt + 1, + RealtimeCount: statistics.AutoAnalyzeMinCnt + 1, }, ColAndIdxExistenceMap: unanalyzedMap, }, @@ -624,7 +623,7 @@ func TestCalculateIndicatorsForPartitions(t *testing.T) { ID: 1, Name: "p0", }: { - HistColl: *statistics.NewHistCollWithColsAndIdxs(0, false, exec.AutoAnalyzeMinCnt+1, (exec.AutoAnalyzeMinCnt+1)*2, map[int64]*statistics.Column{ + HistColl: *statistics.NewHistCollWithColsAndIdxs(0, false, statistics.AutoAnalyzeMinCnt+1, (statistics.AutoAnalyzeMinCnt+1)*2, map[int64]*statistics.Column{ 1: { StatsVer: 2, Histogram: statistics.Histogram{ @@ -646,7 +645,7 @@ func TestCalculateIndicatorsForPartitions(t *testing.T) { ID: 2, Name: "p1", }: { - HistColl: *statistics.NewHistCollWithColsAndIdxs(0, false, exec.AutoAnalyzeMinCnt+1, 0, map[int64]*statistics.Column{ + HistColl: *statistics.NewHistCollWithColsAndIdxs(0, false, statistics.AutoAnalyzeMinCnt+1, 0, map[int64]*statistics.Column{ 1: { StatsVer: 2, Histogram: statistics.Histogram{ @@ -706,7 +705,7 @@ func TestCalculateIndicatorsForPartitions(t *testing.T) { ID: 1, Name: "p0", }: { - HistColl: *statistics.NewHistCollWithColsAndIdxs(0, false, exec.AutoAnalyzeMinCnt+1, 0, map[int64]*statistics.Column{ + HistColl: *statistics.NewHistCollWithColsAndIdxs(0, false, statistics.AutoAnalyzeMinCnt+1, 0, map[int64]*statistics.Column{ 1: { StatsVer: 2, Histogram: statistics.Histogram{ @@ -728,7 +727,7 @@ func TestCalculateIndicatorsForPartitions(t *testing.T) { ID: 2, Name: "p1", }: { - HistColl: *statistics.NewHistCollWithColsAndIdxs(0, false, exec.AutoAnalyzeMinCnt+1, 0, map[int64]*statistics.Column{ + HistColl: *statistics.NewHistCollWithColsAndIdxs(0, false, statistics.AutoAnalyzeMinCnt+1, 0, map[int64]*statistics.Column{ 1: { StatsVer: 2, Histogram: statistics.Histogram{ @@ -817,14 +816,14 @@ func TestCheckNewlyAddedIndexesNeedAnalyzeForPartitionedTable(t *testing.T) { ID: 1, Name: "p0", }: { - HistColl: *statistics.NewHistCollWithColsAndIdxs(0, false, exec.AutoAnalyzeMinCnt+1, 0, nil, map[int64]*statistics.Index{}), + HistColl: *statistics.NewHistCollWithColsAndIdxs(0, false, statistics.AutoAnalyzeMinCnt+1, 0, nil, map[int64]*statistics.Index{}), ColAndIdxExistenceMap: statistics.NewColAndIndexExistenceMap(0, 0), }, { ID: 2, Name: "p1", }: { - HistColl: *statistics.NewHistCollWithColsAndIdxs(0, false, exec.AutoAnalyzeMinCnt+1, 0, nil, map[int64]*statistics.Index{ + HistColl: *statistics.NewHistCollWithColsAndIdxs(0, false, statistics.AutoAnalyzeMinCnt+1, 0, nil, map[int64]*statistics.Index{ 2: { StatsVer: 2, }, diff --git a/pkg/statistics/handle/cache/statscache.go b/pkg/statistics/handle/cache/statscache.go index c42f2f02e6928..66444375ea6e9 100644 --- a/pkg/statistics/handle/cache/statscache.go +++ b/pkg/statistics/handle/cache/statscache.go @@ -258,7 +258,14 @@ func (s *StatsCacheImpl) SetStatsCacheCapacity(c int64) { // UpdateStatsHealthyMetrics updates stats healthy distribution metrics according to stats cache. func (s *StatsCacheImpl) UpdateStatsHealthyMetrics() { distribution := make([]int64, 5) + uneligibleAnalyze := 0 for _, tbl := range s.Values() { + distribution[4]++ // total table count + isEligibleForAnalysis := tbl.IsEligibleForAnalysis() + if !isEligibleForAnalysis { + uneligibleAnalyze++ + continue + } healthy, ok := tbl.GetStatsHealthy() if !ok { continue @@ -272,9 +279,9 @@ func (s *StatsCacheImpl) UpdateStatsHealthyMetrics() { } else { distribution[3]++ } - distribution[4]++ } for i, val := range distribution { handle_metrics.StatsHealthyGauges[i].Set(float64(val)) } + handle_metrics.StatsHealthyGauges[5].Set(float64(uneligibleAnalyze)) } diff --git a/pkg/statistics/handle/metrics/metrics.go b/pkg/statistics/handle/metrics/metrics.go index 175fef6359df4..c4fd3a644546d 100644 --- a/pkg/statistics/handle/metrics/metrics.go +++ b/pkg/statistics/handle/metrics/metrics.go @@ -40,6 +40,7 @@ func InitMetricsVars() { metrics.StatsHealthyGauge.WithLabelValues("[100,100]"), // [0,100] should always be the last metrics.StatsHealthyGauge.WithLabelValues("[0,100]"), + metrics.StatsHealthyGauge.WithLabelValues("unneeded analyze"), } DumpHistoricalStatsSuccessCounter = metrics.HistoricalStatsCounter.WithLabelValues("dump", "success") diff --git a/pkg/statistics/handle/updatetest/BUILD.bazel b/pkg/statistics/handle/updatetest/BUILD.bazel index 6dfe9f61ede3a..b204b0035b10b 100644 --- a/pkg/statistics/handle/updatetest/BUILD.bazel +++ b/pkg/statistics/handle/updatetest/BUILD.bazel @@ -17,7 +17,6 @@ go_test( "//pkg/sessionctx/stmtctx", "//pkg/sessionctx/variable", "//pkg/statistics", - "//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 218abfde66572..0e4ac3a82c7d8 100644 --- a/pkg/statistics/handle/updatetest/update_test.go +++ b/pkg/statistics/handle/updatetest/update_test.go @@ -30,7 +30,6 @@ 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/exec" "github.com/pingcap/tidb/pkg/statistics/handle/usage" "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/testkit" @@ -367,10 +366,10 @@ func TestAutoUpdate(t *testing.T) { testKit.MustExec("create table t (a varchar(20))") analyzehelper.TriggerPredicateColumnsCollection(t, testKit, store, "t", "a") - exec.AutoAnalyzeMinCnt = 0 + statistics.AutoAnalyzeMinCnt = 0 testKit.MustExec("set global tidb_auto_analyze_ratio = 0.2") defer func() { - exec.AutoAnalyzeMinCnt = 1000 + statistics.AutoAnalyzeMinCnt = 1000 testKit.MustExec("set global tidb_auto_analyze_ratio = 0.5") }() @@ -472,10 +471,10 @@ func TestAutoUpdatePartition(t *testing.T) { testKit.MustExec("create table t (a int, index idx(a)) PARTITION BY RANGE (a) (PARTITION p0 VALUES LESS THAN (6))") testKit.MustExec("analyze table t") - exec.AutoAnalyzeMinCnt = 0 + statistics.AutoAnalyzeMinCnt = 0 testKit.MustExec("set global tidb_auto_analyze_ratio = 0.6") defer func() { - exec.AutoAnalyzeMinCnt = 1000 + statistics.AutoAnalyzeMinCnt = 1000 testKit.MustExec("set global tidb_auto_analyze_ratio = 0.5") }() @@ -517,7 +516,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(exec.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(statistics.AutoAnalyzeMinCnt))) require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(true)) require.NoError(t, dom.StatsHandle().Update(context.Background(), dom.InfoSchema())) @@ -805,10 +804,10 @@ func TestAutoUpdatePartitionInDynamicOnlyMode(t *testing.T) { testKit.MustExec("set @@tidb_analyze_version = 2") testKit.MustExec("analyze table t") - exec.AutoAnalyzeMinCnt = 0 + statistics.AutoAnalyzeMinCnt = 0 testKit.MustExec("set global tidb_auto_analyze_ratio = 0.1") defer func() { - exec.AutoAnalyzeMinCnt = 1000 + statistics.AutoAnalyzeMinCnt = 1000 testKit.MustExec("set global tidb_auto_analyze_ratio = 0.5") }() @@ -851,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) - exec.AutoAnalyzeMinCnt = 0 + statistics.AutoAnalyzeMinCnt = 0 defer func() { - exec.AutoAnalyzeMinCnt = 1000 + statistics.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)) }() @@ -1053,9 +1052,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) - exec.AutoAnalyzeMinCnt = 0 + statistics.AutoAnalyzeMinCnt = 0 defer func() { - exec.AutoAnalyzeMinCnt = 1000 + statistics.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)) }() @@ -1254,15 +1253,15 @@ func TestNotDumpSysTable(t *testing.T) { func TestAutoAnalyzePartitionTableAfterAddingIndex(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) - oriMinCnt := exec.AutoAnalyzeMinCnt + oriMinCnt := statistics.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() { - exec.AutoAnalyzeMinCnt = oriMinCnt + statistics.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)) }() - exec.AutoAnalyzeMinCnt = 0 + statistics.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 9b2cdb535887b..40be8945c9dc1 100644 --- a/pkg/statistics/integration_test.go +++ b/pkg/statistics/integration_test.go @@ -26,7 +26,6 @@ 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/exec" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/analyzehelper" "github.com/pingcap/tidb/pkg/testkit/testdata" @@ -331,9 +330,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) - exec.AutoAnalyzeMinCnt = 0 + statistics.AutoAnalyzeMinCnt = 0 defer func() { - exec.AutoAnalyzeMinCnt = 1000 + statistics.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/statistics/table.go b/pkg/statistics/table.go index 45188c5ac81ab..41292fbd5689e 100644 --- a/pkg/statistics/table.go +++ b/pkg/statistics/table.go @@ -41,6 +41,10 @@ const ( PseudoRowCount = 10000 ) +// 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 ( // Below functions are used to solve cycle import problem. // Note: all functions below will be removed after finishing moving all estimation functions into the cardinality package. @@ -709,6 +713,19 @@ func (t *Table) IsAnalyzed() bool { return t.LastAnalyzeVersion > 0 } +// IsEligibleForAnalysis checks whether the table is eligible for analysis. +func (t *Table) IsEligibleForAnalysis() bool { + // 1. If the statistics are either not loaded or are classified as pseudo, there is no need for analyze. + // 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 t == nil || t.Pseudo || t.RealtimeCount < AutoAnalyzeMinCnt { + return false + } + + return true +} + // GetAnalyzeRowCount tries to get the row count of a column or an index if possible. // This method is useful because this row count doesn't consider the modify count. func (coll *HistColl) GetAnalyzeRowCount() float64 { diff --git a/pkg/ttl/ttlworker/BUILD.bazel b/pkg/ttl/ttlworker/BUILD.bazel index 5d36b6a909a56..00bac1c43717a 100644 --- a/pkg/ttl/ttlworker/BUILD.bazel +++ b/pkg/ttl/ttlworker/BUILD.bazel @@ -83,7 +83,7 @@ go_test( "//pkg/session", "//pkg/sessionctx", "//pkg/sessionctx/variable", - "//pkg/statistics/handle/autoanalyze/exec", + "//pkg/statistics", "//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 1cfae74ba04b6..85981859b340a 100644 --- a/pkg/ttl/ttlworker/job_manager_integration_test.go +++ b/pkg/ttl/ttlworker/job_manager_integration_test.go @@ -33,7 +33,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/exec" + "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/testkit" timerapi "github.com/pingcap/tidb/pkg/timer/api" timertable "github.com/pingcap/tidb/pkg/timer/tablestore" @@ -218,10 +218,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 := exec.AutoAnalyzeMinCnt - exec.AutoAnalyzeMinCnt = 0 + originAutoAnalyzeMinCnt := statistics.AutoAnalyzeMinCnt + statistics.AutoAnalyzeMinCnt = 0 defer func() { - exec.AutoAnalyzeMinCnt = originAutoAnalyzeMinCnt + statistics.AutoAnalyzeMinCnt = originAutoAnalyzeMinCnt }() store, dom := testkit.CreateMockStoreAndDomain(t) @@ -404,10 +404,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 := exec.AutoAnalyzeMinCnt - exec.AutoAnalyzeMinCnt = 0 + originAutoAnalyzeMinCnt := statistics.AutoAnalyzeMinCnt + statistics.AutoAnalyzeMinCnt = 0 defer func() { - exec.AutoAnalyzeMinCnt = originAutoAnalyzeMinCnt + statistics.AutoAnalyzeMinCnt = originAutoAnalyzeMinCnt }() store, dom := testkit.CreateMockStoreAndDomain(t)