Skip to content

Commit

Permalink
statistics: do not use the infoSchema from domain directly (#49822)
Browse files Browse the repository at this point in the history
  • Loading branch information
Rustin170506 authored Dec 28, 2023
1 parent 87f8355 commit 3b41717
Show file tree
Hide file tree
Showing 8 changed files with 50 additions and 52 deletions.
2 changes: 1 addition & 1 deletion pkg/domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -2472,7 +2472,7 @@ func (do *Domain) autoAnalyzeWorker(owner owner.Manager) {
select {
case <-analyzeTicker.C:
if variable.RunAutoAnalyze.Load() && !do.stopAutoAnalyze.Load() && owner.IsOwner() {
statsHandle.HandleAutoAnalyze(do.InfoSchema())
statsHandle.HandleAutoAnalyze()
}
case <-do.exit:
return
Expand Down
14 changes: 7 additions & 7 deletions pkg/executor/test/analyzetest/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -732,7 +732,7 @@ func TestSavedAnalyzeOptions(t *testing.T) {
tk.MustExec("insert into t values (10,10,10)")
require.Nil(t, h.DumpStatsDeltaToKV(true))
require.Nil(t, h.Update(is))
h.HandleAutoAnalyze(is)
h.HandleAutoAnalyze()
tbl = h.GetTableStats(tableInfo)
require.Greater(t, tbl.Version, lastVersion)
lastVersion = tbl.Version
Expand Down Expand Up @@ -1086,7 +1086,7 @@ func TestSavedAnalyzeColumnOptions(t *testing.T) {
require.Nil(t, h.DumpStatsDeltaToKV(true))
require.Nil(t, h.Update(is))
// auto analyze uses the saved option(predicate columns).
h.HandleAutoAnalyze(is)
h.HandleAutoAnalyze()
tblStats = h.GetTableStats(tblInfo)
require.Less(t, lastVersion, tblStats.Version)
lastVersion = tblStats.Version
Expand Down Expand Up @@ -1939,7 +1939,7 @@ func testKillAutoAnalyze(t *testing.T, ver int) {
require.NoError(t, failpoint.Disable(mockSlowAnalyze))
}()
}
require.True(t, h.HandleAutoAnalyze(is), comment)
require.True(t, h.HandleAutoAnalyze(), comment)
currentVersion := h.GetTableStats(tableInfo).Version
if status == "finished" {
// If we kill a finished job, after kill command the status is still finished and the table stats are updated.
Expand Down Expand Up @@ -2011,7 +2011,7 @@ func TestKillAutoAnalyzeIndex(t *testing.T) {
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/executor/mockSlowAnalyzeIndex"))
}()
}
require.True(t, h.HandleAutoAnalyze(dom.InfoSchema()), comment)
require.True(t, h.HandleAutoAnalyze(), comment)
currentVersion := h.GetTableStats(tblInfo).Version
if status == "finished" {
// If we kill a finished job, after kill command the status is still finished and the index stats are updated.
Expand Down Expand Up @@ -2616,7 +2616,7 @@ PARTITION BY RANGE ( id ) (
h.SetLease(oriLease)
}()
is := dom.InfoSchema()
h.HandleAutoAnalyze(is)
h.HandleAutoAnalyze()
tk.MustExec("create index idxa on t (a)")
tk.MustExec("create index idxb on t (b)")
table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
Expand Down Expand Up @@ -2651,7 +2651,7 @@ PARTITION BY RANGE ( id ) (
h.SetLease(oriLease)
}()
is := dom.InfoSchema()
h.HandleAutoAnalyze(is)
h.HandleAutoAnalyze()
tk.MustExec("alter table t add column a int")
tk.MustExec("alter table t add column b int")
table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
Expand Down Expand Up @@ -2746,7 +2746,7 @@ func TestAutoAnalyzeAwareGlobalVariableChange(t *testing.T) {
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/injectAnalyzeSnapshot", fmt.Sprintf("return(%d)", startTS)))
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/injectBaseCount", "return(3)"))
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/injectBaseModifyCount", "return(0)"))
require.True(t, h.HandleAutoAnalyze(dom.InfoSchema()))
require.True(t, h.HandleAutoAnalyze())
// Check the count / modify_count changes during the analyze are not lost.
tk.MustQuery(fmt.Sprintf("select count, modify_count from mysql.stats_meta where table_id = %d", tid)).Check(testkit.Rows(
"6 3",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -174,7 +174,7 @@ func TestGlobalMemoryControlForAutoAnalyze(t *testing.T) {
childTrackers = executor.GlobalAnalyzeMemoryTracker.GetChildrenForTest()
require.Len(t, childTrackers, 0)

h.HandleAutoAnalyze(dom.InfoSchema())
h.HandleAutoAnalyze()
rs := tk.MustQuery("select fail_reason from mysql.analyze_jobs where table_name=? and state=? limit 1", "t", "failed")
failReason := rs.Rows()[0][0].(string)
require.True(t, strings.Contains(failReason, "Your query has been cancelled due to exceeding the allowed memory limit for the tidb-server instance and this query is currently using the most memory. Please try narrowing your query scope or increase the tidb_server_memory_limit and try again."))
Expand Down
11 changes: 5 additions & 6 deletions pkg/statistics/handle/autoanalyze/autoanalyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -236,10 +236,11 @@ func CleanupCorruptedAnalyzeJobsOnDeadInstances(
return nil
}

// HandleAutoAnalyze analyzes the newly created table or index.
func (sa *statsAnalyze) HandleAutoAnalyze(is infoschema.InfoSchema) (analyzed bool) {
// HandleAutoAnalyze analyzes the outdated tables. (The change percent of the table exceeds the threshold)
// It also analyzes newly created tables and newly added indexes.
func (sa *statsAnalyze) HandleAutoAnalyze() (analyzed bool) {
_ = statsutil.CallWithSCtx(sa.statsHandle.SPool(), func(sctx sessionctx.Context) error {
analyzed = HandleAutoAnalyze(sctx, sa.statsHandle, sa.sysProcTracker, is)
analyzed = HandleAutoAnalyze(sctx, sa.statsHandle, sa.sysProcTracker)
return nil
})
return
Expand Down Expand Up @@ -304,7 +305,6 @@ func HandleAutoAnalyze(
sctx sessionctx.Context,
statsHandle statstypes.StatsHandle,
sysProcTracker sessionctx.SysProcTracker,
is infoschema.InfoSchema,
) (analyzed bool) {
defer func() {
if r := recover(); r != nil {
Expand Down Expand Up @@ -339,7 +339,6 @@ func HandleAutoAnalyze(
sctx,
statsHandle,
sysProcTracker,
is,
autoAnalyzeRatio,
pruneMode,
start,
Expand All @@ -357,11 +356,11 @@ func RandomPickOneTableAndTryAutoAnalyze(
sctx sessionctx.Context,
statsHandle statstypes.StatsHandle,
sysProcTracker sessionctx.SysProcTracker,
is infoschema.InfoSchema,
autoAnalyzeRatio float64,
pruneMode variable.PartitionPruneMode,
start, end time.Time,
) bool {
is := sctx.GetDomainInfoSchema().(infoschema.InfoSchema)
dbs := is.AllSchemaNames()
// Shuffle the database and table slice to randomize the order of analyzing tables.
rd := rand.New(rand.NewSource(time.Now().UnixNano())) // #nosec G404
Expand Down
33 changes: 16 additions & 17 deletions pkg/statistics/handle/autoanalyze/autoanalyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -59,12 +59,12 @@ func TestAutoAnalyzeLockedTable(t *testing.T) {
autoanalyze.AutoAnalyzeMinCnt = 1000
}()
// Try to analyze the locked table, it should not analyze the table.
require.False(t, dom.StatsHandle().HandleAutoAnalyze(dom.InfoSchema()))
require.False(t, dom.StatsHandle().HandleAutoAnalyze())

// Unlock the table.
tk.MustExec("unlock stats t")
// Try again, it should analyze the table.
require.True(t, dom.StatsHandle().HandleAutoAnalyze(dom.InfoSchema()))
require.True(t, dom.StatsHandle().HandleAutoAnalyze())
}

func TestDisableAutoAnalyze(t *testing.T) {
Expand All @@ -87,16 +87,16 @@ func TestDisableAutoAnalyze(t *testing.T) {
autoanalyze.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()))
require.True(t, dom.StatsHandle().HandleAutoAnalyze())
require.NoError(t, h.Update(is))

// Try again, it should not analyze the table because it's already analyzed and auto analyze ratio is 0.
require.False(t, dom.StatsHandle().HandleAutoAnalyze(dom.InfoSchema()))
require.False(t, dom.StatsHandle().HandleAutoAnalyze())

// Index analyze doesn't depend on auto analyze ratio. Only control by tidb_enable_auto_analyze.
// Even auto analyze ratio is set to 0, we still need to analyze the newly created index.
tk.MustExec("alter table t add index ia(a)")
require.True(t, dom.StatsHandle().HandleAutoAnalyze(dom.InfoSchema()))
require.True(t, dom.StatsHandle().HandleAutoAnalyze())
}

func TestAutoAnalyzeOnChangeAnalyzeVer(t *testing.T) {
Expand All @@ -118,7 +118,7 @@ func TestAutoAnalyzeOnChangeAnalyzeVer(t *testing.T) {
is := do.InfoSchema()
require.NoError(t, h.Update(is))
// Auto analyze when global ver is 1.
h.HandleAutoAnalyze(is)
h.HandleAutoAnalyze()
require.NoError(t, h.Update(is))
tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
require.NoError(t, err)
Expand All @@ -135,7 +135,7 @@ func TestAutoAnalyzeOnChangeAnalyzeVer(t *testing.T) {
require.NoError(t, h.DumpStatsDeltaToKV(true))
require.NoError(t, h.Update(is))
// Auto analyze t whose version is 1 after setting global ver to 2.
h.HandleAutoAnalyze(is)
h.HandleAutoAnalyze()
require.NoError(t, h.Update(is))
statsTbl1 = h.GetTableStats(tbl.Meta())
require.Equal(t, int64(5), statsTbl1.RealtimeCount)
Expand All @@ -156,7 +156,7 @@ func TestAutoAnalyzeOnChangeAnalyzeVer(t *testing.T) {
tbl2, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("tt"))
require.NoError(t, err)
require.NoError(t, h.Update(is))
h.HandleAutoAnalyze(is)
h.HandleAutoAnalyze()
require.NoError(t, h.Update(is))
statsTbl2 := h.GetTableStats(tbl2.Meta())
// Since it's a newly created table. Auto analyze should analyze it's statistics to version2.
Expand Down Expand Up @@ -292,7 +292,7 @@ func TestAutoAnalyzeSkipColumnTypes(t *testing.T) {
defer func() {
autoanalyze.AutoAnalyzeMinCnt = originalVal
}()
require.True(t, h.HandleAutoAnalyze(dom.InfoSchema()))
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"))
}

Expand All @@ -312,7 +312,7 @@ func TestAutoAnalyzeOnEmptyTable(t *testing.T) {
start, end := fmt.Sprintf("%02d:%02d +0000", h, m), fmt.Sprintf("%02d:%02d +0000", h, m)
tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_start_time='%v'", start))
tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_end_time='%v'", end))
dom.StatsHandle().HandleAutoAnalyze(dom.InfoSchema())
dom.StatsHandle().HandleAutoAnalyze()

tk.MustExec("use test")
tk.MustExec("create table t (a int, index idx(a))")
Expand All @@ -324,11 +324,11 @@ func TestAutoAnalyzeOnEmptyTable(t *testing.T) {
require.NoError(t, dom.StatsHandle().Update(dom.InfoSchema()))

// test if it will be limited by the time range
require.False(t, dom.StatsHandle().HandleAutoAnalyze(dom.InfoSchema()))
require.False(t, dom.StatsHandle().HandleAutoAnalyze())

tk.MustExec("set global tidb_auto_analyze_start_time='00:00 +0000'")
tk.MustExec("set global tidb_auto_analyze_end_time='23:59 +0000'")
require.True(t, dom.StatsHandle().HandleAutoAnalyze(dom.InfoSchema()))
require.True(t, dom.StatsHandle().HandleAutoAnalyze())
}

func TestAutoAnalyzeOutOfSpecifiedTime(t *testing.T) {
Expand All @@ -347,7 +347,7 @@ func TestAutoAnalyzeOutOfSpecifiedTime(t *testing.T) {
start, end := fmt.Sprintf("%02d:%02d +0000", h, m), fmt.Sprintf("%02d:%02d +0000", h, m)
tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_start_time='%v'", start))
tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_end_time='%v'", end))
dom.StatsHandle().HandleAutoAnalyze(dom.InfoSchema())
dom.StatsHandle().HandleAutoAnalyze()

tk.MustExec("use test")
tk.MustExec("create table t (a int)")
Expand All @@ -358,15 +358,15 @@ func TestAutoAnalyzeOutOfSpecifiedTime(t *testing.T) {
require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(true))
require.NoError(t, dom.StatsHandle().Update(dom.InfoSchema()))

require.False(t, dom.StatsHandle().HandleAutoAnalyze(dom.InfoSchema()))
require.False(t, dom.StatsHandle().HandleAutoAnalyze())
tk.MustExec("analyze table t")

tk.MustExec("alter table t add index ia(a)")
require.False(t, dom.StatsHandle().HandleAutoAnalyze(dom.InfoSchema()))
require.False(t, dom.StatsHandle().HandleAutoAnalyze())

tk.MustExec("set global tidb_auto_analyze_start_time='00:00 +0000'")
tk.MustExec("set global tidb_auto_analyze_end_time='23:59 +0000'")
require.True(t, dom.StatsHandle().HandleAutoAnalyze(dom.InfoSchema()))
require.True(t, dom.StatsHandle().HandleAutoAnalyze())
}

func makeFailpointRes(t *testing.T, v interface{}) string {
Expand Down Expand Up @@ -548,7 +548,6 @@ func TestSkipAutoAnalyzeOutsideTheAvailableTime(t *testing.T) {
se.(sessionctx.Context),
dom.StatsHandle(),
dom.SysProcTracker(),
dom.InfoSchema(),
0.6,
variable.Dynamic,
ttStart,
Expand Down
5 changes: 3 additions & 2 deletions pkg/statistics/handle/types/interfaces.go
Original file line number Diff line number Diff line change
Expand Up @@ -126,8 +126,9 @@ type StatsAnalyze interface {
// their associated instances being removed from the current cluster.
CleanupCorruptedAnalyzeJobsOnDeadInstances() error

// HandleAutoAnalyze analyzes the newly created table or index.
HandleAutoAnalyze(is infoschema.InfoSchema) (analyzed bool)
// HandleAutoAnalyze analyzes the outdated tables. (The change percent of the table exceeds the threshold)
// It also analyzes newly created tables and newly added indexes.
HandleAutoAnalyze() (analyzed bool)

// CheckAnalyzeVersion checks whether all the statistics versions of this table's columns and indexes are the same.
CheckAnalyzeVersion(tblInfo *model.TableInfo, physicalIDs []int64, version *int) bool
Expand Down
Loading

0 comments on commit 3b41717

Please sign in to comment.