diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/dynamic_partitioned_table_analysis_job.go b/pkg/statistics/handle/autoanalyze/priorityqueue/dynamic_partitioned_table_analysis_job.go index 070c19a515707..fab2050552776 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/dynamic_partitioned_table_analysis_job.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/dynamic_partitioned_table_analysis_job.go @@ -244,8 +244,10 @@ func (j *DynamicPartitionedTableAnalysisJob) analyzePartitionIndexes( sysProcTracker sysproctrack.Tracker, ) (success bool) { analyzePartitionBatchSize := int(variable.AutoAnalyzePartitionBatchSize.Load()) + // For version 2, analyze one index will analyze all other indexes and columns. + // For version 1, analyze one index will only analyze the specified index. + analyzeVersion := sctx.GetSessionVars().AnalyzeVersion -OnlyPickOneIndex: for indexName, partitionNames := range j.PartitionIndexes { needAnalyzePartitionNames := make([]any, 0, len(partitionNames)) for _, partition := range partitionNames { @@ -262,10 +264,16 @@ OnlyPickOneIndex: params := append([]any{j.TableSchema, j.GlobalTableName}, needAnalyzePartitionNames[start:end]...) params = append(params, indexName) success = exec.AutoAnalyze(sctx, statsHandle, sysProcTracker, j.TableStatsVer, sql, params...) + if !success { + return false + } + } + // For version 1, we need to analyze all indexes. + if analyzeVersion != 1 { // Halt execution after analyzing one index. // This is because analyzing a single index also analyzes all other indexes and columns. // Therefore, to avoid redundancy, we prevent multiple analyses of the same partition. - break OnlyPickOneIndex + break } } return diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/non_partitioned_table_analysis_job.go b/pkg/statistics/handle/autoanalyze/priorityqueue/non_partitioned_table_analysis_job.go index 5404c2359dfba..64da57a8c0116 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/non_partitioned_table_analysis_job.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/non_partitioned_table_analysis_job.go @@ -212,6 +212,18 @@ func (j *NonPartitionedTableAnalysisJob) analyzeIndexes( if len(j.Indexes) == 0 { return true } + // For version 2, analyze one index will analyze all other indexes and columns. + // For version 1, analyze one index will only analyze the specified index. + analyzeVersion := sctx.GetSessionVars().AnalyzeVersion + if analyzeVersion == 1 { + for _, index := range j.Indexes { + sql, params := j.GenSQLForAnalyzeIndex(index) + if !exec.AutoAnalyze(sctx, statsHandle, sysProcTracker, j.TableStatsVer, sql, params...) { + return false + } + } + return true + } // Only analyze the first index. // This is because analyzing a single index also analyzes all other indexes and columns. // Therefore, to avoid redundancy, we prevent multiple analyses of the same table. diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/queue.go b/pkg/statistics/handle/autoanalyze/priorityqueue/queue.go index a65d16c1f42c4..c5b214b3b517f 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/queue.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/queue.go @@ -376,7 +376,7 @@ func (pq *AnalysisPriorityQueue) ProcessDMLChanges() { // Only update if we've seen a newer version if newMaxVersion > lastFetchTimestamp { - statslogutil.StatsLogger().Info("Updating last fetch timestamp", zap.Uint64("new_max_version", newMaxVersion)) + queueSamplerLogger().Info("Updating last fetch timestamp", zap.Uint64("new_max_version", newMaxVersion)) pq.syncFields.lastDMLUpdateFetchTimestamp = newMaxVersion } return nil @@ -404,12 +404,6 @@ func (pq *AnalysisPriorityQueue) processTableStats( return errors.Trace(err) } jobFactory := NewAnalysisJobFactory(sctx, autoAnalyzeRatio, currentTs) - // Check if the table is needed to be analyzed. - // Note: Unanalyzed tables will also be considered. - changePercent := jobFactory.CalculateChangePercentage(stats) - if changePercent == 0 { - return nil - } is := sctx.GetDomainInfoSchema().(infoschema.InfoSchema) pruneMode := variable.PartitionPruneMode(sctx.GetSessionVars().PartitionPruneMode.Load()) @@ -455,7 +449,6 @@ func (pq *AnalysisPriorityQueue) tryCreateJob( } tableInfo, ok := pq.statsHandle.TableInfoByID(is, stats.PhysicalID) - tableMeta := tableInfo.Meta() if !ok { statslogutil.StatsLogger().Warn( "Table info not found for table id", @@ -463,6 +456,7 @@ func (pq *AnalysisPriorityQueue) tryCreateJob( ) return nil } + tableMeta := tableInfo.Meta() schemaName, ok := is.SchemaNameByTableID(tableMeta.ID) if !ok { statslogutil.StatsLogger().Warn( diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/queue_ddl_handler_test.go b/pkg/statistics/handle/autoanalyze/priorityqueue/queue_ddl_handler_test.go index 1a5a79ef2a0ad..8a0c6d1702d76 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/queue_ddl_handler_test.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/queue_ddl_handler_test.go @@ -902,3 +902,48 @@ func TestVectorIndexTriggerAutoAnalyze(t *testing.T) { // No event is found require.Nil(t, addIndexEvent) } + +func TestAddIndexTriggerAutoAnalyzeWithStatsVersion1(t *testing.T) { + store, do := testkit.CreateMockStoreAndDomain(t) + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("set @@global.tidb_analyze_version=1;") + testKit.MustExec("use test") + 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))") + is := do.InfoSchema() + tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + require.NoError(t, err) + tableInfo := tbl.Meta() + h := do.StatsHandle() + // Analyze table. + testKit.MustExec("analyze table t") + require.NoError(t, h.Update(context.Background(), do.InfoSchema())) + // Insert some data. + testKit.MustExec("insert into t values (1,2),(2,2)") + require.NoError(t, h.DumpStatsDeltaToKV(true)) + require.NoError(t, h.Update(context.Background(), do.InfoSchema())) + // Add two indexes. + testKit.MustExec("alter table t add index idx1(c1)") + testKit.MustExec("alter table t add index idx2(c2)") + + statistics.AutoAnalyzeMinCnt = 0 + defer func() { + statistics.AutoAnalyzeMinCnt = 1000 + }() + + pq := priorityqueue.NewAnalysisPriorityQueue(h) + defer pq.Close() + require.NoError(t, pq.Initialize()) + isEmpty, err := pq.IsEmpty() + require.NoError(t, err) + require.False(t, isEmpty) + job, err := pq.Peek() + require.NoError(t, err) + require.Equal(t, tableInfo.ID, job.GetTableID()) + require.NoError(t, job.Analyze(h, do.SysProcTracker())) + + // Check the stats of the indexes. + tableStats := h.GetTableStats(tableInfo) + require.True(t, tableStats.GetIdx(1).IsAnalyzed()) + require.True(t, tableStats.GetIdx(2).IsAnalyzed()) + require.True(t, tableStats.GetIdx(3).IsAnalyzed()) +} diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/static_partitioned_table_analysis_job.go b/pkg/statistics/handle/autoanalyze/priorityqueue/static_partitioned_table_analysis_job.go index f488e35398b50..4ecc21d9e4dce 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/static_partitioned_table_analysis_job.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/static_partitioned_table_analysis_job.go @@ -225,6 +225,18 @@ func (j *StaticPartitionedTableAnalysisJob) analyzeStaticPartitionIndexes( if len(j.Indexes) == 0 { return true } + // For version 2, analyze one index will analyze all other indexes and columns. + // For version 1, analyze one index will only analyze the specified index. + analyzeVersion := sctx.GetSessionVars().AnalyzeVersion + if analyzeVersion == 1 { + for _, index := range j.Indexes { + sql, params := j.GenSQLForAnalyzeStaticPartitionIndex(index) + if !exec.AutoAnalyze(sctx, statsHandle, sysProcTracker, j.TableStatsVer, sql, params...) { + return false + } + } + return true + } // Only analyze the first index. // This is because analyzing a single index also analyzes all other indexes and columns. // Therefore, to avoid redundancy, we prevent multiple analyses of the same partition.