Skip to content

Commit b7ce927

Browse files
authored
statistics: make sure PQ can analyze all indexes with stats version 1 (#57258) (#57286)
close #57257
1 parent 02f4a28 commit b7ce927

File tree

5 files changed

+81
-10
lines changed

5 files changed

+81
-10
lines changed

pkg/statistics/handle/autoanalyze/priorityqueue/dynamic_partitioned_table_analysis_job.go

Lines changed: 10 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -244,8 +244,10 @@ func (j *DynamicPartitionedTableAnalysisJob) analyzePartitionIndexes(
244244
sysProcTracker sysproctrack.Tracker,
245245
) (success bool) {
246246
analyzePartitionBatchSize := int(variable.AutoAnalyzePartitionBatchSize.Load())
247+
// For version 2, analyze one index will analyze all other indexes and columns.
248+
// For version 1, analyze one index will only analyze the specified index.
249+
analyzeVersion := sctx.GetSessionVars().AnalyzeVersion
247250

248-
OnlyPickOneIndex:
249251
for indexName, partitionNames := range j.PartitionIndexes {
250252
needAnalyzePartitionNames := make([]any, 0, len(partitionNames))
251253
for _, partition := range partitionNames {
@@ -262,10 +264,16 @@ OnlyPickOneIndex:
262264
params := append([]any{j.TableSchema, j.GlobalTableName}, needAnalyzePartitionNames[start:end]...)
263265
params = append(params, indexName)
264266
success = exec.AutoAnalyze(sctx, statsHandle, sysProcTracker, j.TableStatsVer, sql, params...)
267+
if !success {
268+
return false
269+
}
270+
}
271+
// For version 1, we need to analyze all indexes.
272+
if analyzeVersion != 1 {
265273
// Halt execution after analyzing one index.
266274
// This is because analyzing a single index also analyzes all other indexes and columns.
267275
// Therefore, to avoid redundancy, we prevent multiple analyses of the same partition.
268-
break OnlyPickOneIndex
276+
break
269277
}
270278
}
271279
return

pkg/statistics/handle/autoanalyze/priorityqueue/non_partitioned_table_analysis_job.go

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -212,6 +212,18 @@ func (j *NonPartitionedTableAnalysisJob) analyzeIndexes(
212212
if len(j.Indexes) == 0 {
213213
return true
214214
}
215+
// For version 2, analyze one index will analyze all other indexes and columns.
216+
// For version 1, analyze one index will only analyze the specified index.
217+
analyzeVersion := sctx.GetSessionVars().AnalyzeVersion
218+
if analyzeVersion == 1 {
219+
for _, index := range j.Indexes {
220+
sql, params := j.GenSQLForAnalyzeIndex(index)
221+
if !exec.AutoAnalyze(sctx, statsHandle, sysProcTracker, j.TableStatsVer, sql, params...) {
222+
return false
223+
}
224+
}
225+
return true
226+
}
215227
// Only analyze the first index.
216228
// This is because analyzing a single index also analyzes all other indexes and columns.
217229
// Therefore, to avoid redundancy, we prevent multiple analyses of the same table.

pkg/statistics/handle/autoanalyze/priorityqueue/queue.go

Lines changed: 2 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -376,7 +376,7 @@ func (pq *AnalysisPriorityQueue) ProcessDMLChanges() {
376376

377377
// Only update if we've seen a newer version
378378
if newMaxVersion > lastFetchTimestamp {
379-
statslogutil.StatsLogger().Info("Updating last fetch timestamp", zap.Uint64("new_max_version", newMaxVersion))
379+
queueSamplerLogger().Info("Updating last fetch timestamp", zap.Uint64("new_max_version", newMaxVersion))
380380
pq.syncFields.lastDMLUpdateFetchTimestamp = newMaxVersion
381381
}
382382
return nil
@@ -404,12 +404,6 @@ func (pq *AnalysisPriorityQueue) processTableStats(
404404
return errors.Trace(err)
405405
}
406406
jobFactory := NewAnalysisJobFactory(sctx, autoAnalyzeRatio, currentTs)
407-
// Check if the table is needed to be analyzed.
408-
// Note: Unanalyzed tables will also be considered.
409-
changePercent := jobFactory.CalculateChangePercentage(stats)
410-
if changePercent == 0 {
411-
return nil
412-
}
413407
is := sctx.GetDomainInfoSchema().(infoschema.InfoSchema)
414408
pruneMode := variable.PartitionPruneMode(sctx.GetSessionVars().PartitionPruneMode.Load())
415409

@@ -455,14 +449,14 @@ func (pq *AnalysisPriorityQueue) tryCreateJob(
455449
}
456450

457451
tableInfo, ok := pq.statsHandle.TableInfoByID(is, stats.PhysicalID)
458-
tableMeta := tableInfo.Meta()
459452
if !ok {
460453
statslogutil.StatsLogger().Warn(
461454
"Table info not found for table id",
462455
zap.Int64("tableID", stats.PhysicalID),
463456
)
464457
return nil
465458
}
459+
tableMeta := tableInfo.Meta()
466460
schemaName, ok := is.SchemaNameByTableID(tableMeta.ID)
467461
if !ok {
468462
statslogutil.StatsLogger().Warn(

pkg/statistics/handle/autoanalyze/priorityqueue/queue_ddl_handler_test.go

Lines changed: 45 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -781,3 +781,48 @@ func TestVectorIndexTriggerAutoAnalyze(t *testing.T) {
781781
// No event is found
782782
require.Nil(t, addIndexEvent)
783783
}
784+
785+
func TestAddIndexTriggerAutoAnalyzeWithStatsVersion1(t *testing.T) {
786+
store, do := testkit.CreateMockStoreAndDomain(t)
787+
testKit := testkit.NewTestKit(t, store)
788+
testKit.MustExec("set @@global.tidb_analyze_version=1;")
789+
testKit.MustExec("use test")
790+
testKit.MustExec("create table t (c1 int, c2 int, index idx(c1, c2)) partition by range columns (c1) (partition p0 values less than (5), partition p1 values less than (10))")
791+
is := do.InfoSchema()
792+
tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t"))
793+
require.NoError(t, err)
794+
tableInfo := tbl.Meta()
795+
h := do.StatsHandle()
796+
// Analyze table.
797+
testKit.MustExec("analyze table t")
798+
require.NoError(t, h.Update(context.Background(), do.InfoSchema()))
799+
// Insert some data.
800+
testKit.MustExec("insert into t values (1,2),(2,2)")
801+
require.NoError(t, h.DumpStatsDeltaToKV(true))
802+
require.NoError(t, h.Update(context.Background(), do.InfoSchema()))
803+
// Add two indexes.
804+
testKit.MustExec("alter table t add index idx1(c1)")
805+
testKit.MustExec("alter table t add index idx2(c2)")
806+
807+
statistics.AutoAnalyzeMinCnt = 0
808+
defer func() {
809+
statistics.AutoAnalyzeMinCnt = 1000
810+
}()
811+
812+
pq := priorityqueue.NewAnalysisPriorityQueue(h)
813+
defer pq.Close()
814+
require.NoError(t, pq.Initialize())
815+
isEmpty, err := pq.IsEmpty()
816+
require.NoError(t, err)
817+
require.False(t, isEmpty)
818+
job, err := pq.Peek()
819+
require.NoError(t, err)
820+
require.Equal(t, tableInfo.ID, job.GetTableID())
821+
require.NoError(t, job.Analyze(h, do.SysProcTracker()))
822+
823+
// Check the stats of the indexes.
824+
tableStats := h.GetTableStats(tableInfo)
825+
require.True(t, tableStats.GetIdx(1).IsAnalyzed())
826+
require.True(t, tableStats.GetIdx(2).IsAnalyzed())
827+
require.True(t, tableStats.GetIdx(3).IsAnalyzed())
828+
}

pkg/statistics/handle/autoanalyze/priorityqueue/static_partitioned_table_analysis_job.go

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -225,6 +225,18 @@ func (j *StaticPartitionedTableAnalysisJob) analyzeStaticPartitionIndexes(
225225
if len(j.Indexes) == 0 {
226226
return true
227227
}
228+
// For version 2, analyze one index will analyze all other indexes and columns.
229+
// For version 1, analyze one index will only analyze the specified index.
230+
analyzeVersion := sctx.GetSessionVars().AnalyzeVersion
231+
if analyzeVersion == 1 {
232+
for _, index := range j.Indexes {
233+
sql, params := j.GenSQLForAnalyzeStaticPartitionIndex(index)
234+
if !exec.AutoAnalyze(sctx, statsHandle, sysProcTracker, j.TableStatsVer, sql, params...) {
235+
return false
236+
}
237+
}
238+
return true
239+
}
228240
// Only analyze the first index.
229241
// This is because analyzing a single index also analyzes all other indexes and columns.
230242
// Therefore, to avoid redundancy, we prevent multiple analyses of the same partition.

0 commit comments

Comments
 (0)