Skip to content

Commit

Permalink
statistics: avoid using infoschema when to init stats (#54514)
Browse files Browse the repository at this point in the history
close #55851
  • Loading branch information
hawkingrei committed Sep 30, 2024
1 parent 5ad55c2 commit bb9f4d1
Show file tree
Hide file tree
Showing 22 changed files with 223 additions and 242 deletions.
8 changes: 4 additions & 4 deletions pkg/domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -2266,7 +2266,7 @@ func (do *Domain) StatsHandle() *handle.Handle {

// CreateStatsHandle is used only for test.
func (do *Domain) CreateStatsHandle(ctx, initStatsCtx sessionctx.Context) error {
h, err := handle.NewHandle(ctx, initStatsCtx, do.statsLease, do.sysSessionPool, &do.sysProcesses, do.NextConnID, do.ReleaseConnID)
h, err := handle.NewHandle(ctx, initStatsCtx, do.statsLease, do.InfoSchema(), do.sysSessionPool, &do.sysProcesses, do.NextConnID, do.ReleaseConnID)
if err != nil {
return err
}
Expand Down Expand Up @@ -2303,7 +2303,7 @@ func (do *Domain) LoadAndUpdateStatsLoop(ctxs []sessionctx.Context, initStatsCtx
// It should be called only once in BootstrapSession.
func (do *Domain) UpdateTableStatsLoop(ctx, initStatsCtx sessionctx.Context) error {
ctx.GetSessionVars().InRestrictedSQL = true
statsHandle, err := handle.NewHandle(ctx, initStatsCtx, do.statsLease, do.sysSessionPool, &do.sysProcesses, do.NextConnID, do.ReleaseConnID)
statsHandle, err := handle.NewHandle(ctx, initStatsCtx, do.statsLease, do.InfoSchema(), do.sysSessionPool, &do.sysProcesses, do.NextConnID, do.ReleaseConnID)
if err != nil {
return err
}
Expand Down Expand Up @@ -2449,7 +2449,7 @@ func (do *Domain) initStats(ctx context.Context) {
initstats.InitStatsPercentage.Store(0)
var err error
if liteInitStats {
err = statsHandle.InitStatsLite(ctx, do.InfoSchema())
err = statsHandle.InitStatsLite(ctx)
} else {
err = statsHandle.InitStats(ctx, do.InfoSchema())
}
Expand Down Expand Up @@ -2488,7 +2488,7 @@ func (do *Domain) loadStatsWorker() {
if err != nil {
logutil.BgLogger().Debug("update stats info failed", zap.Error(err))
}
err = statsHandle.LoadNeededHistograms()
err = statsHandle.LoadNeededHistograms(do.InfoSchema())
if err != nil {
logutil.BgLogger().Debug("load histograms failed", zap.Error(err))
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/executor/infoschema_reader_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -331,7 +331,7 @@ func TestForAnalyzeStatus(t *testing.T) {
tk.MustExec("insert into t1 values (1,2),(3,4)")
tk.MustExec("analyze table t1 all columns")
tk.MustQuery("show warnings").Check(testkit.Rows("Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t1, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"")) // 1 note.
require.NoError(t, dom.StatsHandle().LoadNeededHistograms())
require.NoError(t, dom.StatsHandle().LoadNeededHistograms(dom.InfoSchema()))
tk.MustExec("CREATE ROLE r_t1 ;")
tk.MustExec("GRANT ALL PRIVILEGES ON test.t1 TO r_t1;")
tk.MustExec("GRANT r_t1 TO analyze_tester;")
Expand Down
38 changes: 19 additions & 19 deletions pkg/executor/test/analyzetest/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -724,7 +724,7 @@ func TestSavedAnalyzeOptions(t *testing.T) {
tk.MustExec("analyze table t with 1 topn, 2 buckets")
is := dom.InfoSchema()
tk.MustQuery("select * from t where b > 1 and c > 1")
require.NoError(t, h.LoadNeededHistograms())
require.NoError(t, h.LoadNeededHistograms(dom.InfoSchema()))
table, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t"))
require.NoError(t, err)
tableInfo := table.Meta()
Expand Down Expand Up @@ -761,7 +761,7 @@ func TestSavedAnalyzeOptions(t *testing.T) {
col0 = tbl.GetCol(tableInfo.Columns[0].ID)
require.Equal(t, 3, len(col0.Buckets))
tk.MustQuery("select * from t where b > 1 and c > 1")
require.NoError(t, h.LoadNeededHistograms())
require.NoError(t, h.LoadNeededHistograms(dom.InfoSchema()))
col1 = tbl.GetCol(tableInfo.Columns[1].ID)
require.Equal(t, 1, len(col1.TopN.TopN))
col2 = tbl.GetCol(tableInfo.Columns[2].ID)
Expand Down Expand Up @@ -1073,7 +1073,7 @@ func TestSavedAnalyzeColumnOptions(t *testing.T) {
tk.MustExec("select * from t where b > 1")
require.NoError(t, h.DumpColStatsUsageToKV())
tk.MustExec("analyze table t predicate columns")
require.NoError(t, h.LoadNeededHistograms())
require.NoError(t, h.LoadNeededHistograms(dom.InfoSchema()))
tblStats := h.GetTableStats(tblInfo)
lastVersion := tblStats.Version
// column b is analyzed
Expand All @@ -1086,7 +1086,7 @@ func TestSavedAnalyzeColumnOptions(t *testing.T) {
require.NoError(t, h.DumpColStatsUsageToKV())
// manually analyze uses the saved option(predicate columns).
tk.MustExec("analyze table t")
require.NoError(t, h.LoadNeededHistograms())
require.NoError(t, h.LoadNeededHistograms(dom.InfoSchema()))
tblStats = h.GetTableStats(tblInfo)
require.Less(t, lastVersion, tblStats.Version)
lastVersion = tblStats.Version
Expand Down Expand Up @@ -2218,7 +2218,7 @@ PARTITION BY RANGE ( a ) (
// analyze table only sets table options and gen globalStats
tk.MustExec("analyze table t columns a,c with 1 topn, 3 buckets")
tk.MustQuery("select * from t where b > 1 and c > 1")
require.NoError(t, h.LoadNeededHistograms())
require.NoError(t, h.LoadNeededHistograms(dom.InfoSchema()))
tbl := h.GetTableStats(tableInfo)
lastVersion := tbl.Version
// both globalStats and partition stats generated and options saved for column a,c
Expand All @@ -2238,7 +2238,7 @@ PARTITION BY RANGE ( a ) (
// analyze table with persisted table-level options
tk.MustExec("analyze table t")
tk.MustQuery("select * from t where b > 1 and c > 1")
require.NoError(t, h.LoadNeededHistograms())
require.NoError(t, h.LoadNeededHistograms(dom.InfoSchema()))
tbl = h.GetTableStats(tableInfo)
require.Greater(t, tbl.Version, lastVersion)
lastVersion = tbl.Version
Expand All @@ -2258,7 +2258,7 @@ PARTITION BY RANGE ( a ) (
// analyze table with merged table-level options
tk.MustExec("analyze table t with 2 topn, 2 buckets")
tk.MustQuery("select * from t where b > 1 and c > 1")
require.NoError(t, h.LoadNeededHistograms())
require.NoError(t, h.LoadNeededHistograms(dom.InfoSchema()))
tbl = h.GetTableStats(tableInfo)
require.Greater(t, tbl.Version, lastVersion)
require.Equal(t, 2, len(tbl.GetCol(tableInfo.Columns[0].ID).Buckets))
Expand Down Expand Up @@ -2312,7 +2312,7 @@ PARTITION BY RANGE ( a ) (
// analyze partition under static mode with options
tk.MustExec("analyze table t partition p0 columns a,c with 1 topn, 3 buckets")
tk.MustQuery("select * from t where b > 1 and c > 1")
require.NoError(t, h.LoadNeededHistograms())
require.NoError(t, h.LoadNeededHistograms(dom.InfoSchema()))
tbl := h.GetTableStats(tableInfo)
p0 := h.GetPartitionStats(tableInfo, pi.Definitions[0].ID)
p1 := h.GetPartitionStats(tableInfo, pi.Definitions[1].ID)
Expand All @@ -2337,7 +2337,7 @@ PARTITION BY RANGE ( a ) (
// analyze table in dynamic mode will ignore partition-level options and use default
tk.MustExec("analyze table t")
tk.MustQuery("select * from t where b > 1 and c > 1")
require.NoError(t, h.LoadNeededHistograms())
require.NoError(t, h.LoadNeededHistograms(dom.InfoSchema()))
tbl = h.GetTableStats(tableInfo)
require.Greater(t, tbl.Version, lastVersion)
lastVersion = tbl.Version
Expand All @@ -2361,7 +2361,7 @@ PARTITION BY RANGE ( a ) (
// analyze table under dynamic mode with specified options with old partition-level options
tk.MustExec("analyze table t columns b,d with 2 topn, 2 buckets")
tk.MustQuery("select * from t where b > 1 and d > 1")
require.NoError(t, h.LoadNeededHistograms())
require.NoError(t, h.LoadNeededHistograms(dom.InfoSchema()))
tbl = h.GetTableStats(tableInfo)
require.Greater(t, tbl.Version, lastVersion)
lastVersion = tbl.Version
Expand All @@ -2381,7 +2381,7 @@ PARTITION BY RANGE ( a ) (
// analyze table under dynamic mode without options with old table-level & partition-level options
tk.MustExec("analyze table t")
tk.MustQuery("select * from t where b > 1 and d > 1")
require.NoError(t, h.LoadNeededHistograms())
require.NoError(t, h.LoadNeededHistograms(dom.InfoSchema()))
tbl = h.GetTableStats(tableInfo)
require.Greater(t, tbl.Version, lastVersion)
lastVersion = tbl.Version
Expand All @@ -2391,7 +2391,7 @@ PARTITION BY RANGE ( a ) (
// analyze table under dynamic mode with specified options with old table-level & partition-level options
tk.MustExec("analyze table t with 1 topn")
tk.MustQuery("select * from t where b > 1 and d > 1")
require.NoError(t, h.LoadNeededHistograms())
require.NoError(t, h.LoadNeededHistograms(dom.InfoSchema()))
tbl = h.GetTableStats(tableInfo)
require.Greater(t, tbl.Version, lastVersion)
require.Equal(t, 2, len(tbl.GetCol(tableInfo.Columns[1].ID).Buckets))
Expand Down Expand Up @@ -2451,7 +2451,7 @@ PARTITION BY RANGE ( a ) (
"Warning 1105 Ignore columns and options when analyze partition in dynamic mode",
))
tk.MustQuery("select * from t where a > 1 and b > 1 and c > 1 and d > 1")
require.NoError(t, h.LoadNeededHistograms())
require.NoError(t, h.LoadNeededHistograms(dom.InfoSchema()))
tbl := h.GetTableStats(tableInfo)
lastVersion := tbl.Version
require.NotEqual(t, 3, len(tbl.GetCol(tableInfo.Columns[2].ID).Buckets))
Expand Down Expand Up @@ -2506,7 +2506,7 @@ PARTITION BY RANGE ( a ) (
tk.MustExec("set @@session.tidb_partition_prune_mode = 'static'")
tk.MustExec("analyze table t partition p0 columns a,c with 1 topn, 3 buckets")
tk.MustQuery("select * from t where a > 1 and b > 1 and c > 1 and d > 1")
require.NoError(t, h.LoadNeededHistograms())
require.NoError(t, h.LoadNeededHistograms(dom.InfoSchema()))
p0 := h.GetPartitionStats(tableInfo, pi.Definitions[0].ID)
require.Equal(t, 3, len(p0.GetCol(tableInfo.Columns[2].ID).Buckets))

Expand Down Expand Up @@ -2538,14 +2538,14 @@ PARTITION BY RANGE ( a ) (
))
// flaky test, fix it later
//tk.MustQuery("select * from t where a > 1 and b > 1 and c > 1 and d > 1")
//require.NoError(t, h.LoadNeededHistograms())
//require.NoError(t, h.LoadNeededHistograms(dom.InfoSchema()))
//tbl := h.GetTableStats(tableInfo)
//require.Equal(t, 0, len(tbl.Columns))

// ignore both p0's 3 buckets, persisted-partition-options' 1 bucket, just use table-level 2 buckets
tk.MustExec("analyze table t partition p0")
tk.MustQuery("select * from t where a > 1 and b > 1 and c > 1 and d > 1")
require.NoError(t, h.LoadNeededHistograms())
require.NoError(t, h.LoadNeededHistograms(dom.InfoSchema()))
tbl := h.GetTableStats(tableInfo)
require.Equal(t, 2, len(tbl.GetCol(tableInfo.Columns[2].ID).Buckets))
}
Expand Down Expand Up @@ -2590,7 +2590,7 @@ PARTITION BY RANGE ( a ) (
tk.MustExec("analyze table t partition p1 with 1 topn, 3 buckets")
tk.MustQuery("show warnings").Sort().Check(testkit.Rows())
tk.MustQuery("select * from t where a > 1 and b > 1 and c > 1 and d > 1")
require.NoError(t, h.LoadNeededHistograms())
require.NoError(t, h.LoadNeededHistograms(dom.InfoSchema()))
tbl := h.GetTableStats(tableInfo)
lastVersion := tbl.Version
require.Equal(t, 3, len(tbl.GetCol(tableInfo.Columns[2].ID).Buckets))
Expand Down Expand Up @@ -2962,7 +2962,7 @@ func TestAnalyzeMVIndex(t *testing.T) {
"└─TableRowIDScan(Probe) 0.03 cop[tikv] table:t keep order:false, stats:partial[ia:allEvicted, ij_char:allEvicted, j:unInitialized]",
))
// 3.2. emulate the background async loading
require.NoError(t, h.LoadNeededHistograms())
require.NoError(t, h.LoadNeededHistograms(dom.InfoSchema()))
// 3.3. now, stats on all indexes should be loaded
tk.MustQuery("explain format = brief select /*+ use_index_merge(t, ij_signed) */ * from t where 1 member of (j->'$.signed')").Check(testkit.Rows(
"IndexMerge 27.00 root type: union",
Expand Down Expand Up @@ -3017,7 +3017,7 @@ func TestAnalyzeMVIndex(t *testing.T) {
))

// 4. check stats content in the memory
require.NoError(t, h.LoadNeededHistograms())
require.NoError(t, h.LoadNeededHistograms(dom.InfoSchema()))
tk.MustQuery("show stats_meta").CheckAt([]int{0, 1, 4, 5}, testkit.Rows("test t 0 27"))
tk.MustQuery("show stats_histograms").Sort().CheckAt([]int{0, 1, 3, 4, 6, 7, 8, 9, 10}, testkit.Rows(
// db_name, table_name, column_name, is_index, distinct_count, null_count, avg_col_size, correlation, load_status
Expand Down
20 changes: 20 additions & 0 deletions pkg/meta/model/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -322,6 +322,26 @@ func (t *TableInfo) FindIndexByName(idxName string) *IndexInfo {
return nil
}

// FindColumnByID finds ColumnInfo by id.
func (t *TableInfo) FindColumnByID(id int64) *ColumnInfo {
for _, col := range t.Columns {
if col.ID == id {
return col
}
}
return nil
}

// FindIndexByID finds index by id.
func (t *TableInfo) FindIndexByID(id int64) *IndexInfo {
for _, idx := range t.Indices {
if idx.ID == id {
return idx
}
}
return nil
}

// FindPublicColumnByName finds the public column by name.
func (t *TableInfo) FindPublicColumnByName(colNameL string) *ColumnInfo {
for _, col := range t.Cols() {
Expand Down
9 changes: 4 additions & 5 deletions pkg/planner/cardinality/selectivity_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,7 @@ func TestCollationColumnEstimate(t *testing.T) {
require.Nil(t, h.DumpStatsDeltaToKV(true))
tk.MustExec("analyze table t all columns")
tk.MustExec("explain select * from t where a = 'aaa'")
require.Nil(t, h.LoadNeededHistograms())
require.Nil(t, h.LoadNeededHistograms(dom.InfoSchema()))
var (
input []string
output [][]string
Expand Down Expand Up @@ -345,7 +345,7 @@ func TestColumnIndexNullEstimation(t *testing.T) {
}
// Make sure column stats has been loaded.
testKit.MustExec(`explain select * from t where a is null`)
require.Nil(t, h.LoadNeededHistograms())
require.Nil(t, h.LoadNeededHistograms(dom.InfoSchema()))
for i := 5; i < len(input); i++ {
testdata.OnRecord(func() {
output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows())
Expand Down Expand Up @@ -582,7 +582,7 @@ func TestRangeStepOverflow(t *testing.T) {
tk.MustExec("analyze table t")
// Trigger the loading of column stats.
tk.MustQuery("select * from t where col between '8499-1-23 2:14:38' and '9961-7-23 18:35:26'").Check(testkit.Rows())
require.Nil(t, h.LoadNeededHistograms())
require.Nil(t, h.LoadNeededHistograms(dom.InfoSchema()))
// Must execute successfully after loading the column stats.
tk.MustQuery("select * from t where col between '8499-1-23 2:14:38' and '9961-7-23 18:35:26'").Check(testkit.Rows())
}
Expand Down Expand Up @@ -1341,7 +1341,7 @@ func TestBuiltinInEstWithoutStats(t *testing.T) {
tk.MustQuery("explain format='brief' select * from t where b in (1, 2, 3, 4, 5, 6, 7, 8)").Check(expectedB)

h.Clear()
require.NoError(t, h.InitStatsLite(context.Background(), is))
require.NoError(t, h.InitStatsLite(context.Background()))
tk.MustQuery("explain format='brief' select * from t where a in (1, 2, 3, 4, 5, 6, 7, 8)").Check(expectedA)
tk.MustQuery("explain format='brief' select * from t where b in (1, 2, 3, 4, 5, 6, 7, 8)").Check(expectedB)

Expand All @@ -1356,7 +1356,6 @@ func TestBuiltinInEstWithoutStats(t *testing.T) {
require.True(t, found)
require.False(t, statsTbl.ColAndIdxExistenceMap.IsEmpty())
for _, col := range tbl.Cols() {
require.True(t, statsTbl.ColAndIdxExistenceMap.Has(col.ID, false))
require.False(t, statsTbl.ColAndIdxExistenceMap.HasAnalyzed(col.ID, false))
}
}
4 changes: 2 additions & 2 deletions pkg/planner/cardinality/trace_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,7 @@ func TestTraceCE(t *testing.T) {
tk.MustExec(sql)
}
statsHandle := dom.StatsHandle()
err := statsHandle.LoadNeededHistograms()
err := statsHandle.LoadNeededHistograms(dom.InfoSchema())
require.NoError(t, err)

sctx := tk.Session().(sessionctx.Context)
Expand Down Expand Up @@ -188,7 +188,7 @@ func TestTraceDebugSelectivity(t *testing.T) {
sql := "explain " + tt
tk.MustExec(sql)
}
err := statsHandle.LoadNeededHistograms()
err := statsHandle.LoadNeededHistograms(dom.InfoSchema())
require.NoError(t, err)

sctx := tk.Session().(sessionctx.Context)
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2211,7 +2211,7 @@ func TestIssue48257(t *testing.T) {
"TableReader 10000.00 root data:TableFullScan",
"└─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo",
))
require.NoError(t, h.LoadNeededHistograms())
require.NoError(t, h.LoadNeededHistograms(dom.InfoSchema()))
tk.MustQuery("explain format = brief select * from t1").Check(testkit.Rows(
"TableReader 1.00 root data:TableFullScan",
"└─TableFullScan 1.00 cop[tikv] table:t1 keep order:false",
Expand Down
2 changes: 1 addition & 1 deletion pkg/statistics/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,7 @@ go_test(
data = glob(["testdata/**"]),
embed = [":statistics"],
flaky = True,
shard_count = 38,
shard_count = 37,
deps = [
"//pkg/config",
"//pkg/meta/model",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -120,8 +120,8 @@ func TestGetTableLastAnalyzeDuration(t *testing.T) {

func TestCheckIndexesNeedAnalyze(t *testing.T) {
analyzedMap := statistics.NewColAndIndexExistenceMap(1, 0)
analyzedMap.InsertCol(1, nil, true)
analyzedMap.InsertIndex(1, nil, false)
analyzedMap.InsertCol(1, true)
analyzedMap.InsertIndex(1, false)
tests := []struct {
name string
tblInfo *model.TableInfo
Expand Down Expand Up @@ -184,9 +184,9 @@ func TestCalculateIndicatorsForPartitions(t *testing.T) {
lastUpdateTs := oracle.GoTimeToTS(lastUpdateTime)
unanalyzedMap := statistics.NewColAndIndexExistenceMap(0, 0)
analyzedMap := statistics.NewColAndIndexExistenceMap(2, 1)
analyzedMap.InsertCol(1, nil, true)
analyzedMap.InsertCol(2, nil, true)
analyzedMap.InsertIndex(1, nil, true)
analyzedMap.InsertCol(1, true)
analyzedMap.InsertCol(2, true)
analyzedMap.InsertIndex(1, true)
tests := []struct {
name string
globalStats *statistics.Table
Expand Down
Loading

0 comments on commit bb9f4d1

Please sign in to comment.