Skip to content

Commit

Permalink
statistics: add last_analyze_version for mysql.stats_meta (#51489)
Browse files Browse the repository at this point in the history
ref #49594
  • Loading branch information
winoros authored Mar 11, 2024
1 parent 8b02143 commit caf83ed
Show file tree
Hide file tree
Showing 18 changed files with 159 additions and 76 deletions.
29 changes: 21 additions & 8 deletions pkg/executor/show_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -141,14 +141,27 @@ func (e *ShowExec) appendTableForStatsMeta(dbName, tblName, partitionName string
if statsTbl.Pseudo {
return
}
e.appendRow([]any{
dbName,
tblName,
partitionName,
e.versionToTime(statsTbl.Version),
statsTbl.ModifyCount,
statsTbl.RealtimeCount,
})
if !statsTbl.IsAnalyzed() {
e.appendRow([]any{
dbName,
tblName,
partitionName,
e.versionToTime(statsTbl.Version),
statsTbl.ModifyCount,
statsTbl.RealtimeCount,
nil,
})
} else {
e.appendRow([]any{
dbName,
tblName,
partitionName,
e.versionToTime(statsTbl.Version),
statsTbl.ModifyCount,
statsTbl.RealtimeCount,
e.versionToTime(statsTbl.LastAnalyzeVersion),
})
}
}

func (e *ShowExec) appendTableForStatsLocked(dbName, tblName, partitionName string) {
Expand Down
3 changes: 3 additions & 0 deletions pkg/executor/show_stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,9 +36,12 @@ func TestShowStatsMeta(t *testing.T) {
tk.MustExec("create table t1 (a int, b int)")
tk.MustExec("analyze table t, t1")
result := tk.MustQuery("show stats_meta")
result = result.Sort()
require.Len(t, result.Rows(), 2)
require.Equal(t, "t", result.Rows()[0][1])
require.Equal(t, "t1", result.Rows()[1][1])
require.NotEqual(t, "<nil>", result.Rows()[0][6])
require.NotEqual(t, "<nil>", result.Rows()[1][6])
result = tk.MustQuery("show stats_meta where table_name = 't'")
require.Len(t, result.Rows(), 1)
require.Equal(t, "t", result.Rows()[0][1])
Expand Down
10 changes: 5 additions & 5 deletions pkg/executor/test/analyzetest/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1394,9 +1394,9 @@ func TestAnalyzeColumnsWithDynamicPartitionTable(t *testing.T) {

rows = tk.MustQuery("show stats_meta where db_name = 'test' and table_name = 't'").Sort().Rows()
require.Equal(t, 3, len(rows))
require.Equal(t, []any{"test", "t", "global", "0", "20"}, append(rows[0][:3], rows[0][4:]...))
require.Equal(t, []any{"test", "t", "p0", "0", "9"}, append(rows[1][:3], rows[1][4:]...))
require.Equal(t, []any{"test", "t", "p1", "0", "11"}, append(rows[2][:3], rows[2][4:]...))
require.Equal(t, []any{"test", "t", "global", "0", "20"}, append(rows[0][:3], rows[0][4:6]...))
require.Equal(t, []any{"test", "t", "p0", "0", "9"}, append(rows[1][:3], rows[1][4:6]...))
require.Equal(t, []any{"test", "t", "p1", "0", "11"}, append(rows[2][:3], rows[2][4:6]...))

tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't' and is_index = 0").Sort().Check(
// db, tbl, part, col, is_idx, value, count
Expand Down Expand Up @@ -1516,8 +1516,8 @@ func TestAnalyzeColumnsWithStaticPartitionTable(t *testing.T) {

rows = tk.MustQuery("show stats_meta where db_name = 'test' and table_name = 't'").Sort().Rows()
require.Equal(t, 2, len(rows))
require.Equal(t, []any{"test", "t", "p0", "0", "9"}, append(rows[0][:3], rows[0][4:]...))
require.Equal(t, []any{"test", "t", "p1", "0", "11"}, append(rows[1][:3], rows[1][4:]...))
require.Equal(t, []any{"test", "t", "p0", "0", "9"}, append(rows[0][:3], rows[0][4:6]...))
require.Equal(t, []any{"test", "t", "p1", "0", "11"}, append(rows[1][:3], rows[1][4:6]...))

tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't' and is_index = 0").Sort().Check(
// db, tbl, part, col, is_idx, value, count
Expand Down
4 changes: 2 additions & 2 deletions pkg/planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -5215,8 +5215,8 @@ func buildShowSchema(s *ast.ShowStmt, isView bool, isSequence bool) (schema *exp
names = []string{"NodeID", "Address", "State", "Max_Commit_Ts", "Update_Time"}
ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeLonglong, mysql.TypeVarchar}
case ast.ShowStatsMeta:
names = []string{"Db_name", "Table_name", "Partition_name", "Update_time", "Modify_count", "Row_count"}
ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeDatetime, mysql.TypeLonglong, mysql.TypeLonglong}
names = []string{"Db_name", "Table_name", "Partition_name", "Update_time", "Modify_count", "Row_count", "Last_analyze_time"}
ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeDatetime, mysql.TypeLonglong, mysql.TypeLonglong, mysql.TypeDatetime}
case ast.ShowStatsExtended:
names = []string{"Db_name", "Table_name", "Stats_name", "Column_names", "Stats_type", "Stats_val", "Last_update_version"}
ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeLonglong}
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/handler/optimizor/plan_replayer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -190,7 +190,7 @@ func TestDumpPlanReplayerAPI(t *testing.T) {
var dbName, tableName string
var modifyCount, count int64
var other any
err = rows.Scan(&dbName, &tableName, &other, &other, &modifyCount, &count)
err = rows.Scan(&dbName, &tableName, &other, &other, &modifyCount, &count, &other)
require.NoError(t, err)
require.Equal(t, "planReplayer", dbName)
require.Equal(t, "t", tableName)
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/handler/optimizor/statistics_handler_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -280,7 +280,7 @@ func checkData(t *testing.T, path string, client *testserverclient.TestServerCli
var dbName, tableName string
var modifyCount, count int64
var other any
err = rows.Scan(&dbName, &tableName, &other, &other, &modifyCount, &count)
err = rows.Scan(&dbName, &tableName, &other, &other, &modifyCount, &count, &other)
require.NoError(t, err)
require.Equal(t, "tidb", dbName)
require.Equal(t, "test", tableName)
Expand Down
2 changes: 2 additions & 0 deletions pkg/session/bootstraptest/main_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,8 @@ func TestMain(m *testing.M) {
goleak.IgnoreTopFunction("net/http.(*persistConn).writeLoop"),
goleak.IgnoreTopFunction("github.com/pingcap/tidb/pkg/ttl/ttlworker.(*ttlDeleteWorker).loop"),
goleak.IgnoreTopFunction("github.com/pingcap/tidb/pkg/ttl/ttlworker.(*ttlScanWorker).loop"),
goleak.IgnoreTopFunction("github.com/dgraph-io/ristretto.(*defaultPolicy).processItems"),
goleak.IgnoreTopFunction("github.com/dgraph-io/ristretto.(*Cache).processItems"),
}
callback := func(i int) int {
// wait for MVCCLevelDB to close, MVCCLevelDB will be closed in one second
Expand Down
2 changes: 2 additions & 0 deletions pkg/session/main_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,8 @@ func TestMain(m *testing.M) {
goleak.IgnoreTopFunction("google.golang.org/grpc/internal/transport.(*http2Client).keepalive"),
goleak.IgnoreTopFunction("internal/poll.runtime_pollWait"),
goleak.IgnoreTopFunction("net/http.(*persistConn).writeLoop"),
goleak.IgnoreTopFunction("github.com/dgraph-io/ristretto.(*defaultPolicy).processItems"),
goleak.IgnoreTopFunction("github.com/dgraph-io/ristretto.(*Cache).processItems"),
goleak.IgnoreTopFunction("github.com/tikv/client-go/v2/txnkv/transaction.keepAlive"),
}
callback := func(i int) int {
Expand Down
2 changes: 1 addition & 1 deletion pkg/statistics/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,7 @@ go_test(
data = glob(["testdata/**"]),
embed = [":statistics"],
flaky = True,
shard_count = 35,
shard_count = 36,
deps = [
"//pkg/config",
"//pkg/parser/ast",
Expand Down
2 changes: 1 addition & 1 deletion pkg/statistics/handle/autoanalyze/autoanalyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -516,7 +516,7 @@ func tryAutoAnalyzeTable(
//
// Exposed for test.
func NeedAnalyzeTable(tbl *statistics.Table, autoAnalyzeRatio float64) (bool, string) {
analyzed := exec.TableAnalyzed(tbl)
analyzed := tbl.IsAnalyzed()
if !analyzed {
return true, "table unanalyzed"
}
Expand Down
18 changes: 9 additions & 9 deletions pkg/statistics/handle/autoanalyze/autoanalyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -207,12 +207,12 @@ func TestTableAnalyzed(t *testing.T) {

require.NoError(t, h.Update(is))
statsTbl := h.GetTableStats(tableInfo)
require.False(t, exec.TableAnalyzed(statsTbl))
require.False(t, statsTbl.LastAnalyzeVersion > 0)

testKit.MustExec("analyze table t")
require.NoError(t, h.Update(is))
statsTbl = h.GetTableStats(tableInfo)
require.True(t, exec.TableAnalyzed(statsTbl))
require.True(t, statsTbl.LastAnalyzeVersion > 0)

h.Clear()
oriLease := h.Lease()
Expand All @@ -223,7 +223,7 @@ func TestTableAnalyzed(t *testing.T) {
}()
require.NoError(t, h.Update(is))
statsTbl = h.GetTableStats(tableInfo)
require.True(t, exec.TableAnalyzed(statsTbl))
require.True(t, statsTbl.LastAnalyzeVersion > 0)
}

func TestNeedAnalyzeTable(t *testing.T) {
Expand Down Expand Up @@ -251,42 +251,42 @@ func TestNeedAnalyzeTable(t *testing.T) {
},
// table was already analyzed but auto analyze is disabled
{
tbl: &statistics.Table{HistColl: statistics.HistColl{Columns: columns, ModifyCount: 1, RealtimeCount: 1}},
tbl: &statistics.Table{HistColl: statistics.HistColl{Columns: columns, ModifyCount: 1, RealtimeCount: 1}, LastAnalyzeVersion: 1},
ratio: 0,
result: false,
reason: "",
},
// table was already analyzed but modify count is small
{
tbl: &statistics.Table{HistColl: statistics.HistColl{Columns: columns, ModifyCount: 0, RealtimeCount: 1}},
tbl: &statistics.Table{HistColl: statistics.HistColl{Columns: columns, ModifyCount: 0, RealtimeCount: 1}, LastAnalyzeVersion: 1},
ratio: 0.3,
result: false,
reason: "",
},
// table was already analyzed
{
tbl: &statistics.Table{HistColl: statistics.HistColl{Columns: columns, ModifyCount: 1, RealtimeCount: 1}},
tbl: &statistics.Table{HistColl: statistics.HistColl{Columns: columns, ModifyCount: 1, RealtimeCount: 1}, LastAnalyzeVersion: 1},
ratio: 0.3,
result: true,
reason: "too many modifications",
},
// table was already analyzed
{
tbl: &statistics.Table{HistColl: statistics.HistColl{Columns: columns, ModifyCount: 1, RealtimeCount: 1}},
tbl: &statistics.Table{HistColl: statistics.HistColl{Columns: columns, ModifyCount: 1, RealtimeCount: 1}, LastAnalyzeVersion: 1},
ratio: 0.3,
result: true,
reason: "too many modifications",
},
// table was already analyzed
{
tbl: &statistics.Table{HistColl: statistics.HistColl{Columns: columns, ModifyCount: 1, RealtimeCount: 1}},
tbl: &statistics.Table{HistColl: statistics.HistColl{Columns: columns, ModifyCount: 1, RealtimeCount: 1}, LastAnalyzeVersion: 1},
ratio: 0.3,
result: true,
reason: "too many modifications",
},
// table was already analyzed
{
tbl: &statistics.Table{HistColl: statistics.HistColl{Columns: columns, ModifyCount: 1, RealtimeCount: 1}},
tbl: &statistics.Table{HistColl: statistics.HistColl{Columns: columns, ModifyCount: 1, RealtimeCount: 1}, LastAnalyzeVersion: 1},
ratio: 0.3,
result: true,
reason: "too many modifications",
Expand Down
15 changes: 0 additions & 15 deletions pkg/statistics/handle/autoanalyze/exec/exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -94,21 +94,6 @@ 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 (%?, %?, %?)"
Expand Down
19 changes: 4 additions & 15 deletions pkg/statistics/handle/autoanalyze/refresher/refresher.go
Original file line number Diff line number Diff line change
Expand Up @@ -370,7 +370,7 @@ func CalculateChangePercentage(
tblStats *statistics.Table,
autoAnalyzeRatio float64,
) float64 {
if !exec.TableAnalyzed(tblStats) {
if !tblStats.IsAnalyzed() {
return unanalyzedTableDefaultChangePercentage
}

Expand Down Expand Up @@ -424,23 +424,12 @@ func findLastAnalyzeTime(
tblStats *statistics.Table,
currentTs uint64,
) time.Time {
maxVersion := uint64(0)
for _, idx := range tblStats.Indices {
if idx.IsAnalyzed() {
maxVersion = max(maxVersion, idx.LastUpdateVersion)
}
}
for _, col := range tblStats.Columns {
if col.IsAnalyzed() {
maxVersion = max(maxVersion, col.LastUpdateVersion)
}
}
// Table is not analyzed, compose a fake version.
if maxVersion == 0 {
if !tblStats.IsAnalyzed() {
phy := oracle.GetTimeFromTS(currentTs)
return phy.Add(unanalyzedTableDefaultLastUpdateDuration)
}
return oracle.GetTimeFromTS(maxVersion)
return oracle.GetTimeFromTS(tblStats.LastAnalyzeVersion)
}

// CheckIndexesNeedAnalyze checks if the indexes of the table need to be analyzed.
Expand All @@ -450,7 +439,7 @@ func CheckIndexesNeedAnalyze(
) []string {
// If table is not analyzed, we need to analyze whole table.
// So we don't need to check indexes.
if !exec.TableAnalyzed(tblStats) {
if !tblStats.IsAnalyzed() {
return nil
}

Expand Down
15 changes: 11 additions & 4 deletions pkg/statistics/handle/autoanalyze/refresher/refresher_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -410,6 +410,7 @@ func TestCalculateChangePercentage(t *testing.T) {
Indices: analyzedIndices,
ModifyCount: (exec.AutoAnalyzeMinCnt + 1) * 2,
},
LastAnalyzeVersion: 1,
},
autoAnalyzeRatio: 0.5,
want: 2,
Expand Down Expand Up @@ -440,6 +441,7 @@ func TestGetTableLastAnalyzeDuration(t *testing.T) {
},
},
},
LastAnalyzeVersion: lastUpdateTs,
}
// 2024-01-01 10:00:00
currentTime := time.Date(2024, 1, 1, 10, 0, 0, 0, time.UTC)
Expand Down Expand Up @@ -505,6 +507,7 @@ func TestCheckIndexesNeedAnalyze(t *testing.T) {
},
},
},
LastAnalyzeVersion: 1,
},
want: []string{"index1"},
},
Expand Down Expand Up @@ -636,7 +639,8 @@ func TestCalculateIndicatorsForPartitions(t *testing.T) {
},
},
},
Version: currentTs,
Version: currentTs,
LastAnalyzeVersion: lastUpdateTs,
},
{
ID: 2,
Expand All @@ -661,7 +665,8 @@ func TestCalculateIndicatorsForPartitions(t *testing.T) {
},
},
},
Version: currentTs,
Version: currentTs,
LastAnalyzeVersion: lastUpdateTs,
},
},
defs: []model.PartitionDefinition{
Expand Down Expand Up @@ -724,7 +729,8 @@ func TestCalculateIndicatorsForPartitions(t *testing.T) {
},
},
},
Version: currentTs,
Version: currentTs,
LastAnalyzeVersion: lastUpdateTs,
},
{
ID: 2,
Expand All @@ -749,7 +755,8 @@ func TestCalculateIndicatorsForPartitions(t *testing.T) {
},
},
},
Version: currentTs,
Version: currentTs,
LastAnalyzeVersion: lastUpdateTs,
},
},
defs: []model.PartitionDefinition{
Expand Down
10 changes: 10 additions & 0 deletions pkg/statistics/handle/bootstrap.go
Original file line number Diff line number Diff line change
Expand Up @@ -132,6 +132,8 @@ func (h *Handle) initStatsHistograms4ChunkLite(is infoschema.InfoSchema, cache s
lastAnalyzePos.Copy(&index.LastAnalyzePos)
if index.IsAnalyzed() {
index.StatsLoadedStatus = statistics.NewStatsAllEvictedStatus()
// The LastAnalyzeVersion is added by ALTER table so its value might be 0.
table.LastAnalyzeVersion = max(table.LastAnalyzeVersion, version)
}
table.Indices[hist.ID] = index
} else {
Expand All @@ -158,6 +160,8 @@ func (h *Handle) initStatsHistograms4ChunkLite(is infoschema.InfoSchema, cache s
lastAnalyzePos.Copy(&col.LastAnalyzePos)
if col.StatsAvailable() {
col.StatsLoadedStatus = statistics.NewStatsAllEvictedStatus()
// The LastAnalyzeVersion is added by ALTER table so its value might be 0.
table.LastAnalyzeVersion = max(table.LastAnalyzeVersion, version)
}
table.Columns[hist.ID] = col
}
Expand Down Expand Up @@ -208,6 +212,8 @@ func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, cache stats
}
if statsVer != statistics.Version0 {
index.StatsLoadedStatus = statistics.NewStatsFullLoadStatus()
// The LastAnalyzeVersion is added by ALTER table so its value might be 0.
table.LastAnalyzeVersion = max(table.LastAnalyzeVersion, version)
}
lastAnalyzePos.Copy(&index.LastAnalyzePos)
table.Indices[hist.ID] = index
Expand All @@ -234,6 +240,10 @@ func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, cache stats
}
lastAnalyzePos.Copy(&col.LastAnalyzePos)
table.Columns[hist.ID] = col
if statsVer != statistics.Version0 {
// The LastAnalyzeVersion is added by ALTER table so its value might be 0.
table.LastAnalyzeVersion = max(table.LastAnalyzeVersion, version)
}
}
cache.Put(tblID, table) // put this table again since it is updated
}
Expand Down
Loading

0 comments on commit caf83ed

Please sign in to comment.