Skip to content

Commit 2d24604

Browse files
committed
statistics: speed up the backgroud stats update worker
1 parent 70caa9e commit 2d24604

File tree

8 files changed

+91
-54
lines changed

8 files changed

+91
-54
lines changed

pkg/executor/show_stats.go

Lines changed: 15 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -146,27 +146,24 @@ func (e *ShowExec) appendTableForStatsMeta(dbName, tblName, partitionName string
146146
if statsTbl.Pseudo {
147147
return
148148
}
149+
row := make([]any, 8)
150+
row[0] = dbName
151+
row[1] = tblName
152+
row[2] = partitionName
153+
row[3] = e.versionToTime(statsTbl.Version)
154+
row[4] = statsTbl.ModifyCount
155+
row[5] = statsTbl.RealtimeCount
149156
if !statsTbl.IsAnalyzed() {
150-
e.appendRow([]any{
151-
dbName,
152-
tblName,
153-
partitionName,
154-
e.versionToTime(statsTbl.Version),
155-
statsTbl.ModifyCount,
156-
statsTbl.RealtimeCount,
157-
nil,
158-
})
157+
row[6] = nil
159158
} else {
160-
e.appendRow([]any{
161-
dbName,
162-
tblName,
163-
partitionName,
164-
e.versionToTime(statsTbl.Version),
165-
statsTbl.ModifyCount,
166-
statsTbl.RealtimeCount,
167-
e.versionToTime(statsTbl.LastAnalyzeVersion),
168-
})
159+
row[6] = e.versionToTime(statsTbl.LastAnalyzeVersion)
160+
}
161+
if statsTbl.LastStatsFullUpdateVersion == 0 {
162+
row[7] = nil
163+
} else {
164+
row[7] = e.versionToTime(statsTbl.LastStatsFullUpdateVersion)
169165
}
166+
e.appendRow(row)
170167
}
171168

172169
func (e *ShowExec) appendTableForStatsLocked(dbName, tblName, partitionName string) {

pkg/planner/core/planbuilder.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -5646,8 +5646,8 @@ func buildShowSchema(s *ast.ShowStmt, isView bool, isSequence bool) (schema *exp
56465646
ftypes = []byte{mysql.TypeLonglong, mysql.TypeVarchar, mysql.TypeVarchar,
56475647
mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeLong, mysql.TypeVarchar, mysql.TypeString}
56485648
case ast.ShowStatsMeta:
5649-
names = []string{"Db_name", "Table_name", "Partition_name", "Update_time", "Modify_count", "Row_count", "Last_analyze_time"}
5650-
ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeDatetime, mysql.TypeLonglong, mysql.TypeLonglong, mysql.TypeDatetime}
5649+
names = []string{"Db_name", "Table_name", "Partition_name", "Update_time", "Modify_count", "Row_count", "Last_analyze_time", "Last_stats_full_update_time"}
5650+
ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeDatetime, mysql.TypeLonglong, mysql.TypeLonglong, mysql.TypeDatetime, mysql.TypeDatetime}
56515651
case ast.ShowStatsExtended:
56525652
names = []string{"Db_name", "Table_name", "Stats_name", "Column_names", "Stats_type", "Stats_val", "Last_update_version"}
56535653
ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeLonglong}

pkg/session/bootstrap.go

Lines changed: 23 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -217,12 +217,16 @@ const (
217217

218218
// CreateStatsMetaTable stores the meta of table statistics.
219219
CreateStatsMetaTable = `CREATE TABLE IF NOT EXISTS mysql.stats_meta (
220-
version BIGINT(64) UNSIGNED NOT NULL,
221-
table_id BIGINT(64) NOT NULL,
222-
modify_count BIGINT(64) NOT NULL DEFAULT 0,
223-
count BIGINT(64) UNSIGNED NOT NULL DEFAULT 0,
224-
snapshot BIGINT(64) UNSIGNED NOT NULL DEFAULT 0,
220+
version BIGINT(64) UNSIGNED NOT NULL,
221+
table_id BIGINT(64) NOT NULL,
222+
modify_count BIGINT(64) NOT NULL DEFAULT 0,
223+
count BIGINT(64) UNSIGNED NOT NULL DEFAULT 0,
224+
snapshot BIGINT(64) UNSIGNED NOT NULL DEFAULT 0,
225+
last_analyze_version BIGINT(64) UNSIGNED DEFAULT NULL,
226+
last_affected_ddl_version BIGINT(64) UNSIGNED DEFAULT NULL,
225227
INDEX idx_ver(version),
228+
INDEX idx_analyze_version(last_analyze_version),
229+
INDEX idx_last_affected_ddl_version(last_affected_ddl_version),
226230
UNIQUE INDEX tbl(table_id)
227231
);`
228232

@@ -1234,11 +1238,15 @@ const (
12341238
// version 240
12351239
// Add indexes to mysql.analyze_jobs to speed up the query.
12361240
version240 = 240
1241+
1242+
// version 241
1243+
// Add last_affected_ddl_version to mysql.stats_meta.
1244+
version241 = 241
12371245
)
12381246

12391247
// currentBootstrapVersion is defined as a variable, so we can modify its value for testing.
12401248
// please make sure this is the largest version
1241-
var currentBootstrapVersion int64 = version240
1249+
var currentBootstrapVersion int64 = version241
12421250

12431251
// DDL owner key's expired time is ManagerSessionTTL seconds, we should wait the time and give more time to have a chance to finish it.
12441252
var internalSQLTimeout = owner.ManagerSessionTTL + 15
@@ -1414,6 +1422,7 @@ var (
14141422
upgradeToVer218,
14151423
upgradeToVer239,
14161424
upgradeToVer240,
1425+
upgradeToVer241,
14171426
}
14181427
)
14191428

@@ -3343,6 +3352,14 @@ func upgradeToVer240(s sessiontypes.Session, ver int64) {
33433352
doReentrantDDL(s, addAnalyzeJobsSchemaTablePartitionStateIndex, dbterror.ErrDupKeyName)
33443353
}
33453354

3355+
func upgradeToVer241(s sessiontypes.Session, ver int64) {
3356+
if ver >= version241 {
3357+
return
3358+
}
3359+
doReentrantDDL(s, "ALTER TABLE mysql.stats_meta ADD COLUMN last_analyze_version bigint(20) unsigned DEFAULT NULL")
3360+
doReentrantDDL(s, "ALTER TABLE mysql.stats_meta ADD COLUMN last_affected_ddl_version bigint(20) unsigned DEFAULT NULL")
3361+
}
3362+
33463363
// initGlobalVariableIfNotExists initialize a global variable with specific val if it does not exist.
33473364
func initGlobalVariableIfNotExists(s sessiontypes.Session, name string, val any) {
33483365
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap)

pkg/statistics/handle/bootstrap.go

Lines changed: 9 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -82,7 +82,8 @@ func (*Handle) initStatsMeta4Chunk(cache statstypes.StatsCache, iter *chunk.Iter
8282
// it will stay at 0 and auto-analyze won't be able to detect that the table has been analyzed.
8383
// But in the future, we maybe will create some records for _row_id, see:
8484
// https://github.com/pingcap/tidb/issues/51098
85-
LastAnalyzeVersion: snapshot,
85+
LastAnalyzeVersion: snapshot,
86+
LastStatsFullUpdateVersion: snapshot,
8687
}
8788
cache.Put(physicalID, tbl) // put this table again since it is updated
8889
}
@@ -149,17 +150,14 @@ func (*Handle) initStatsHistograms4ChunkLite(cache statstypes.StatsCache, iter *
149150
}
150151
if isIndex > 0 {
151152
table.ColAndIdxExistenceMap.InsertIndex(id, statsVer != statistics.Version0)
152-
if statsVer != statistics.Version0 {
153-
// The LastAnalyzeVersion is added by ALTER table so its value might be 0.
154-
table.LastAnalyzeVersion = max(table.LastAnalyzeVersion, row.GetUint64(4))
155-
}
156153
} else {
157154
table.ColAndIdxExistenceMap.InsertCol(id, statsVer != statistics.Version0 || ndv > 0 || nullCount > 0)
158-
if statsVer != statistics.Version0 {
159-
// The LastAnalyzeVersion is added by ALTER table so its value might be 0.
160-
table.LastAnalyzeVersion = max(table.LastAnalyzeVersion, row.GetUint64(4))
161-
}
162155
}
156+
// The LastXXXVersion can be added by ALTER table so its value might be 0.
157+
if statsVer != statistics.Version0 {
158+
table.LastAnalyzeVersion = max(table.LastAnalyzeVersion, row.GetUint64(4))
159+
}
160+
table.LastStatsFullUpdateVersion = max(table.LastStatsFullUpdateVersion, row.GetUint64(4))
163161
}
164162
if table != nil {
165163
cache.Put(table.PhysicalID, table) // put this table in the cache because all statstics of the table have been read.
@@ -239,6 +237,7 @@ func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, cache stats
239237
// The LastAnalyzeVersion is added by ALTER table so its value might be 0.
240238
table.LastAnalyzeVersion = max(table.LastAnalyzeVersion, version)
241239
}
240+
table.LastStatsFullUpdateVersion = max(table.LastStatsFullUpdateVersion, version)
242241
lastAnalyzePos.Copy(&index.LastAnalyzePos)
243242
table.SetIdx(idxInfo.ID, index)
244243
table.ColAndIdxExistenceMap.InsertIndex(idxInfo.ID, statsVer != statistics.Version0)
@@ -277,6 +276,7 @@ func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, cache stats
277276
col.StatsLoadedStatus = statistics.NewStatsAllEvictedStatus()
278277
}
279278
// Otherwise the column's stats is not initialized.
279+
table.LastStatsFullUpdateVersion = max(table.LastStatsFullUpdateVersion, version)
280280
}
281281
}
282282
if table != nil {

pkg/statistics/handle/cache/statscache.go

Lines changed: 10 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -123,7 +123,7 @@ func (s *StatsCacheImpl) Update(ctx context.Context, is infoschema.InfoSchema, t
123123
err error
124124
)
125125
if err := util.CallWithSCtx(s.statsHandle.SPool(), func(sctx sessionctx.Context) error {
126-
query := "SELECT version, table_id, modify_count, count, snapshot from mysql.stats_meta where version > %? "
126+
query := "SELECT version, table_id, modify_count, count, snapshot, last_analyze_version, last_affected_ddl_version from mysql.stats_meta where version > %? "
127127
args := []any{lastVersion}
128128

129129
if len(tableAndPartitionIDs) > 0 {
@@ -168,6 +168,13 @@ func (s *StatsCacheImpl) Update(ctx context.Context, is infoschema.InfoSchema, t
168168
modifyCount := row.GetInt64(2)
169169
count := row.GetInt64(3)
170170
snapshot := row.GetUint64(4)
171+
var latestHistUpdateVersion uint64 = 0
172+
if !row.IsNull(5) {
173+
latestHistUpdateVersion = row.GetUint64(5)
174+
}
175+
if !row.IsNull(6) {
176+
latestHistUpdateVersion = max(latestHistUpdateVersion, row.GetUint64(6))
177+
}
171178

172179
// Detect the context cancel signal, since it may take a long time for the loop.
173180
// TODO: add context to TableInfoByID and remove this code block?
@@ -192,7 +199,8 @@ func (s *StatsCacheImpl) Update(ctx context.Context, is infoschema.InfoSchema, t
192199
oldTbl, ok := s.Get(physicalID)
193200
if ok &&
194201
oldTbl.Version >= version &&
195-
tableInfo.UpdateTS == oldTbl.TblInfoUpdateTS {
202+
tableInfo.UpdateTS == oldTbl.TblInfoUpdateTS &&
203+
oldTbl.LastStatsFullUpdateVersion >= latestHistUpdateVersion {
196204
continue
197205
}
198206
if !ok {

pkg/statistics/handle/storage/read.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -319,6 +319,7 @@ func indexStatsFromStorage(sctx sessionctx.Context, row chunk.Row, table *statis
319319
table.StatsVer = int(statsVer)
320320
table.LastAnalyzeVersion = max(table.LastAnalyzeVersion, histVer)
321321
}
322+
table.LastStatsFullUpdateVersion = max(table.LastStatsFullUpdateVersion, histVer)
322323
// We will not load buckets, topn and cmsketch if:
323324
// 1. lease > 0, and:
324325
// 2. the index doesn't have any of buckets, topn, cmsketch in memory before, and:
@@ -415,6 +416,7 @@ func columnStatsFromStorage(sctx sessionctx.Context, row chunk.Row, table *stati
415416
table.StatsVer = int(statsVer)
416417
table.LastAnalyzeVersion = max(table.LastAnalyzeVersion, histVer)
417418
}
419+
table.LastStatsFullUpdateVersion = max(table.LastStatsFullUpdateVersion, histVer)
418420
isHandle := tableInfo.PKIsHandle && mysql.HasPriKeyFlag(colInfo.GetFlag())
419421
// We will not load buckets, topn and cmsketch if:
420422
// 1. lease > 0, and:

pkg/statistics/handle/storage/save.go

Lines changed: 10 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -175,11 +175,12 @@ func SaveTableStatsToStorage(sctx sessionctx.Context,
175175
count = 0
176176
}
177177
if _, err = util.Exec(sctx,
178-
"replace into mysql.stats_meta (version, table_id, count, snapshot) values (%?, %?, %?, %?)",
178+
"replace into mysql.stats_meta (version, table_id, count, snapshot, last_analyze_version) values (%?, %?, %?, %?, %?)",
179179
version,
180180
tableID,
181181
count,
182182
snapShot,
183+
version,
183184
); err != nil {
184185
return 0, err
185186
}
@@ -188,7 +189,8 @@ func SaveTableStatsToStorage(sctx sessionctx.Context,
188189
// 1-2. There's already an existing record for this table, and we are handling stats for mv index now.
189190
// In this case, we only update the version. See comments for AnalyzeResults.ForMVIndex for more details.
190191
if _, err = util.Exec(sctx,
191-
"update mysql.stats_meta set version=%? where table_id=%?",
192+
"update mysql.stats_meta set version=%?, last_analyze_version=%? where table_id=%?",
193+
version,
192194
version,
193195
tableID,
194196
); err != nil {
@@ -228,11 +230,12 @@ func SaveTableStatsToStorage(sctx sessionctx.Context,
228230
zap.Int64("count", cnt))
229231
}
230232
if _, err = util.Exec(sctx,
231-
"update mysql.stats_meta set version=%?, modify_count=%?, count=%?, snapshot=%? where table_id=%?",
233+
"update mysql.stats_meta set version=%?, modify_count=%?, count=%?, snapshot=%?, last_analyze_version=%? where table_id=%?",
232234
version,
233235
modifyCnt,
234236
cnt,
235237
results.Snapshot,
238+
version,
236239
tableID,
237240
); err != nil {
238241
return 0, err
@@ -430,8 +433,8 @@ func InsertColStats2KV(
430433
// First of all, we update the version.
431434
_, err = util.ExecWithCtx(
432435
ctx, sctx,
433-
"update mysql.stats_meta set version = %? where table_id = %?",
434-
startTS, physicalID,
436+
"update mysql.stats_meta set version = %?, last_affected_ddl_version = %? where table_id = %?",
437+
startTS, startTS, physicalID,
435438
)
436439
if err != nil {
437440
return 0, errors.Trace(err)
@@ -521,8 +524,8 @@ func InsertTableStats2KV(
521524
}
522525
if _, err = util.ExecWithCtx(
523526
ctx, sctx,
524-
"insert into mysql.stats_meta (version, table_id) values(%?, %?)",
525-
startTS, physicalID,
527+
"insert into mysql.stats_meta (version, table_id, last_affected_ddl_version) values(%?, %?, %?)",
528+
startTS, physicalID, startTS,
526529
); err != nil {
527530
return 0, errors.Trace(err)
528531
}

pkg/statistics/table.go

Lines changed: 20 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -72,6 +72,14 @@ type Table struct {
7272
// 1. Initialized by snapshot when loading stats_meta.
7373
// 2. Updated by the analysis time of a specific column or index when loading the histogram of the column or index.
7474
LastAnalyzeVersion uint64
75+
// LastStatsFullUpdateVersion is the mvcc version of the last full update of histograms.
76+
// It differs from LastAnalyzeVersion because it can be influenced by some DDL.
77+
// e.g. When we execute ALTER TABLE ADD COLUMN, there'll be new record inserted into mysql.stats_histograms.
78+
// We need to load the corresponding one into memory too.
79+
// It's used to skip redundant loading of stats, i.e, if the cached stats is already update-to-date with mysql.stats_xxx tables,
80+
// and the schema of the table does not change, we don't need to load the stats for this table again.
81+
// Stats' sync load/async load should not change this field since they are not table-level update.
82+
LastStatsFullUpdateVersion uint64
7583
// TblInfoUpdateTS is the UpdateTS of the TableInfo used when filling this struct.
7684
// It is the schema version of the corresponding table. It is used to skip redundant
7785
// loading of stats, i.e, if the cached stats is already update-to-date with mysql.stats_xxx tables,
@@ -607,10 +615,11 @@ func (t *Table) Copy() *Table {
607615
newHistColl.indices[id] = idx.Copy()
608616
}
609617
nt := &Table{
610-
HistColl: newHistColl,
611-
Version: t.Version,
612-
TblInfoUpdateTS: t.TblInfoUpdateTS,
613-
LastAnalyzeVersion: t.LastAnalyzeVersion,
618+
HistColl: newHistColl,
619+
Version: t.Version,
620+
TblInfoUpdateTS: t.TblInfoUpdateTS,
621+
LastAnalyzeVersion: t.LastAnalyzeVersion,
622+
LastStatsFullUpdateVersion: t.LastStatsFullUpdateVersion,
614623
}
615624
if t.ExtendedStats != nil {
616625
newExtStatsColl := &ExtendedStatsColl{
@@ -643,12 +652,13 @@ func (t *Table) ShallowCopy() *Table {
643652
StatsVer: t.StatsVer,
644653
}
645654
nt := &Table{
646-
HistColl: newHistColl,
647-
Version: t.Version,
648-
TblInfoUpdateTS: t.TblInfoUpdateTS,
649-
ExtendedStats: t.ExtendedStats,
650-
ColAndIdxExistenceMap: t.ColAndIdxExistenceMap,
651-
LastAnalyzeVersion: t.LastAnalyzeVersion,
655+
HistColl: newHistColl,
656+
Version: t.Version,
657+
TblInfoUpdateTS: t.TblInfoUpdateTS,
658+
ExtendedStats: t.ExtendedStats,
659+
ColAndIdxExistenceMap: t.ColAndIdxExistenceMap,
660+
LastAnalyzeVersion: t.LastAnalyzeVersion,
661+
LastStatsFullUpdateVersion: t.LastStatsFullUpdateVersion,
652662
}
653663
return nt
654664
}

0 commit comments

Comments
 (0)