Skip to content

Commit

Permalink
statistics: add metrics for unneeded analyze table (#54822) (#55577)
Browse files Browse the repository at this point in the history
close #54823
  • Loading branch information
ti-chi-bot authored Aug 25, 2024
1 parent 576749e commit 47279f5
Show file tree
Hide file tree
Showing 15 changed files with 85 additions and 70 deletions.
1 change: 0 additions & 1 deletion pkg/executor/test/analyzetest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@ go_test(
"//pkg/sessionctx",
"//pkg/sessionctx/variable",
"//pkg/statistics",
"//pkg/statistics/handle/autoanalyze",
"//pkg/testkit",
"//pkg/util/dbterror/exeerrors",
"@com_github_pingcap_errors//:errors",
Expand Down
27 changes: 13 additions & 14 deletions pkg/executor/test/analyzetest/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/util/dbterror/exeerrors"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -743,11 +742,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 := statistics.AutoAnalyzeMinCnt
defer func() {
autoanalyze.AutoAnalyzeMinCnt = originalVal3
statistics.AutoAnalyzeMinCnt = originalVal3
}()
autoanalyze.AutoAnalyzeMinCnt = 0
statistics.AutoAnalyzeMinCnt = 0

tk.MustExec("use test")
tk.MustExec("set @@session.tidb_analyze_version = 2")
Expand Down Expand Up @@ -1085,11 +1084,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 := statistics.AutoAnalyzeMinCnt
defer func() {
autoanalyze.AutoAnalyzeMinCnt = originalVal3
statistics.AutoAnalyzeMinCnt = originalVal3
}()
autoanalyze.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))
Expand Down Expand Up @@ -1966,9 +1965,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
statistics.AutoAnalyzeMinCnt = 0
defer func() {
autoanalyze.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))
}()
Expand Down Expand Up @@ -2050,9 +2049,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
statistics.AutoAnalyzeMinCnt = 0
defer func() {
autoanalyze.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))
}()
Expand Down Expand Up @@ -2805,12 +2804,12 @@ func TestAutoAnalyzeAwareGlobalVariableChange(t *testing.T) {
"3 0",
))

originalVal1 := autoanalyze.AutoAnalyzeMinCnt
originalVal1 := statistics.AutoAnalyzeMinCnt
originalVal2 := tk.MustQuery("select @@global.tidb_auto_analyze_ratio").Rows()[0][0].(string)
autoanalyze.AutoAnalyzeMinCnt = 0
statistics.AutoAnalyzeMinCnt = 0
tk.MustExec("set global tidb_auto_analyze_ratio = 0.001")
defer func() {
autoanalyze.AutoAnalyzeMinCnt = originalVal1
statistics.AutoAnalyzeMinCnt = originalVal1
tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_ratio = %v", originalVal2))
}()

Expand Down
2 changes: 1 addition & 1 deletion pkg/executor/test/analyzetest/memorycontrol/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,7 @@ go_test(
"//pkg/config",
"//pkg/executor",
"//pkg/sessionctx/variable",
"//pkg/statistics/handle/autoanalyze",
"//pkg/statistics",
"//pkg/testkit",
"//pkg/util",
"//pkg/util/memory",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/util"
"github.com/pingcap/tidb/pkg/util/memory"
Expand Down Expand Up @@ -144,12 +144,12 @@ func TestGlobalMemoryControlForAutoAnalyze(t *testing.T) {
require.Len(t, rs0.Rows(), 0)

h := dom.StatsHandle()
originalVal4 := autoanalyze.AutoAnalyzeMinCnt
originalVal4 := statistics.AutoAnalyzeMinCnt
originalVal5 := tk.MustQuery("select @@global.tidb_auto_analyze_ratio").Rows()[0][0].(string)
autoanalyze.AutoAnalyzeMinCnt = 0
statistics.AutoAnalyzeMinCnt = 0
tk.MustExec("set global tidb_auto_analyze_ratio = 0.001")
defer func() {
autoanalyze.AutoAnalyzeMinCnt = originalVal4
statistics.AutoAnalyzeMinCnt = originalVal4
tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_ratio = %v", originalVal5))
}()

Expand Down
1 change: 0 additions & 1 deletion pkg/statistics/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,6 @@ go_test(
"//pkg/parser/mysql",
"//pkg/sessionctx",
"//pkg/sessionctx/stmtctx",
"//pkg/statistics/handle/autoanalyze",
"//pkg/testkit",
"//pkg/testkit/testdata",
"//pkg/testkit/testmain",
Expand Down
7 changes: 2 additions & 5 deletions pkg/statistics/handle/autoanalyze/autoanalyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -274,14 +274,11 @@ func RandomPickOneTableAndTryAutoAnalyze(
return false
}

// AutoAnalyzeMinCnt means if the count of table is less than this value, we needn't do auto analyze.
var AutoAnalyzeMinCnt int64 = 1000

func autoAnalyzeTable(sctx sessionctx.Context,
statsHandle statsutil.StatsHandle,
tblInfo *model.TableInfo, statsTbl *statistics.Table,
ratio float64, sql string, params ...interface{}) bool {
if statsTbl.Pseudo || statsTbl.RealtimeCount < AutoAnalyzeMinCnt {
if statsTbl.Pseudo || statsTbl.RealtimeCount < statistics.AutoAnalyzeMinCnt {
return false
}
if needAnalyze, reason := NeedAnalyzeTable(statsTbl, 20*statsHandle.Lease(), ratio); needAnalyze {
Expand Down Expand Up @@ -375,7 +372,7 @@ func autoAnalyzePartitionTableInDynamicMode(sctx sessionctx.Context,
partitionNames := make([]interface{}, 0, len(partitionDefs))
for _, def := range partitionDefs {
partitionStatsTbl := statsHandle.GetPartitionStats(tblInfo, def.ID)
if partitionStatsTbl.Pseudo || partitionStatsTbl.RealtimeCount < AutoAnalyzeMinCnt {
if partitionStatsTbl.Pseudo || partitionStatsTbl.RealtimeCount < statistics.AutoAnalyzeMinCnt {
continue
}
if needAnalyze, reason := NeedAnalyzeTable(partitionStatsTbl, 20*statsHandle.Lease(), ratio); needAnalyze {
Expand Down
25 changes: 12 additions & 13 deletions pkg/statistics/handle/autoanalyze/autoanalyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,9 +44,9 @@ func TestAutoAnalyzeLockedTable(t *testing.T) {
tk.MustExec("lock stats t")
is := dom.InfoSchema()
require.NoError(t, h.Update(is))
autoanalyze.AutoAnalyzeMinCnt = 0
statistics.AutoAnalyzeMinCnt = 0
defer func() {
autoanalyze.AutoAnalyzeMinCnt = 1000
statistics.AutoAnalyzeMinCnt = 1000
}()
// Try to analyze the locked table, it should not analyze the table.
require.False(t, dom.StatsHandle().HandleAutoAnalyze(dom.InfoSchema()))
Expand All @@ -70,11 +70,10 @@ func TestDisableAutoAnalyze(t *testing.T) {
is := dom.InfoSchema()
require.NoError(t, h.Update(is))

// Set auto analyze ratio to 0.
tk.MustExec("set @@global.tidb_auto_analyze_ratio = 0")
autoanalyze.AutoAnalyzeMinCnt = 0
tk.MustExec("set @@global.tidb_enable_auto_analyze = 0")
statistics.AutoAnalyzeMinCnt = 0
defer func() {
autoanalyze.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(dom.InfoSchema()))
Expand All @@ -97,9 +96,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
statistics.AutoAnalyzeMinCnt = 0
defer func() {
autoanalyze.AutoAnalyzeMinCnt = 1000
statistics.AutoAnalyzeMinCnt = 1000
}()
h := do.StatsHandle()
err := h.HandleDDLEvent(<-h.DDLEventCh())
Expand Down Expand Up @@ -286,10 +285,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 := statistics.AutoAnalyzeMinCnt
statistics.AutoAnalyzeMinCnt = 0
defer func() {
autoanalyze.AutoAnalyzeMinCnt = originalVal
statistics.AutoAnalyzeMinCnt = originalVal
}()
require.True(t, h.HandleAutoAnalyze(dom.InfoSchema()))
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"))
Expand Down Expand Up @@ -318,7 +317,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(statistics.AutoAnalyzeMinCnt)))
require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(true))
require.NoError(t, dom.StatsHandle().Update(dom.InfoSchema()))

Expand Down Expand Up @@ -353,7 +352,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(statistics.AutoAnalyzeMinCnt)))
require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(true))
require.NoError(t, dom.StatsHandle().Update(dom.InfoSchema()))

Expand Down
9 changes: 8 additions & 1 deletion pkg/statistics/handle/cache/statscache.go
Original file line number Diff line number Diff line change
Expand Up @@ -200,7 +200,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
Expand All @@ -214,9 +221,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))
}
1 change: 1 addition & 0 deletions pkg/statistics/handle/metrics/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -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")
Expand Down
1 change: 0 additions & 1 deletion pkg/statistics/handle/updatetest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ go_test(
"//pkg/sessionctx",
"//pkg/sessionctx/variable",
"//pkg/statistics",
"//pkg/statistics/handle/autoanalyze",
"//pkg/statistics/handle/usage",
"//pkg/statistics/handle/util",
"//pkg/testkit",
Expand Down
35 changes: 17 additions & 18 deletions pkg/statistics/handle/updatetest/update_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,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"
"github.com/pingcap/tidb/pkg/statistics/handle/usage"
"github.com/pingcap/tidb/pkg/statistics/handle/util"
"github.com/pingcap/tidb/pkg/testkit"
Expand Down Expand Up @@ -363,11 +362,11 @@ func TestAutoUpdate(t *testing.T) {
testKit.MustExec("use test")
testKit.MustExec("create table t (a varchar(20))")

autoanalyze.AutoAnalyzeMinCnt = 0
statistics.AutoAnalyzeMinCnt = 0
testKit.MustExec("set global tidb_auto_analyze_ratio = 0.2")
defer func() {
autoanalyze.AutoAnalyzeMinCnt = 1000
testKit.MustExec("set global tidb_auto_analyze_ratio = 0.0")
statistics.AutoAnalyzeMinCnt = 1000
testKit.MustExec("set global tidb_auto_analyze_ratio = 0.5")
}()

do := dom
Expand Down Expand Up @@ -468,11 +467,11 @@ 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
statistics.AutoAnalyzeMinCnt = 0
testKit.MustExec("set global tidb_auto_analyze_ratio = 0.6")
defer func() {
autoanalyze.AutoAnalyzeMinCnt = 1000
testKit.MustExec("set global tidb_auto_analyze_ratio = 0.0")
statistics.AutoAnalyzeMinCnt = 1000
testKit.MustExec("set global tidb_auto_analyze_ratio = 0.5")
}()

do := dom
Expand Down Expand Up @@ -513,7 +512,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(statistics.AutoAnalyzeMinCnt)))
require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(true))
require.NoError(t, dom.StatsHandle().Update(dom.InfoSchema()))

Expand Down Expand Up @@ -803,11 +802,11 @@ func TestAutoUpdatePartitionInDynamicOnlyMode(t *testing.T) {
testKit.MustExec("set @@tidb_analyze_version = 2")
testKit.MustExec("analyze table t")

autoanalyze.AutoAnalyzeMinCnt = 0
statistics.AutoAnalyzeMinCnt = 0
testKit.MustExec("set global tidb_auto_analyze_ratio = 0.1")
defer func() {
autoanalyze.AutoAnalyzeMinCnt = 1000
testKit.MustExec("set global tidb_auto_analyze_ratio = 0.0")
statistics.AutoAnalyzeMinCnt = 1000
testKit.MustExec("set global tidb_auto_analyze_ratio = 0.5")
}()

require.NoError(t, h.Update(is))
Expand Down Expand Up @@ -849,9 +848,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
statistics.AutoAnalyzeMinCnt = 0
defer func() {
autoanalyze.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))
}()
Expand Down Expand Up @@ -1077,9 +1076,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
statistics.AutoAnalyzeMinCnt = 0
defer func() {
autoanalyze.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))
}()
Expand Down Expand Up @@ -1277,15 +1276,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 := 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() {
autoanalyze.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))
}()
autoanalyze.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")
Expand Down
5 changes: 2 additions & 3 deletions pkg/statistics/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@ import (
"github.com/pingcap/tidb/pkg/config"
"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/testkit"
"github.com/pingcap/tidb/pkg/testkit/testdata"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -307,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
statistics.AutoAnalyzeMinCnt = 0
defer func() {
autoanalyze.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))
}()
Expand Down
Loading

0 comments on commit 47279f5

Please sign in to comment.