From d38383afea1b59cde17526e4349cce83f5045363 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 23 Dec 2024 17:08:39 +0800 Subject: [PATCH] statistics: remove dead code (#58412) (#58442) ref pingcap/tidb#57804 --- pkg/statistics/handle/bootstrap.go | 88 +++++++----------------------- 1 file changed, 21 insertions(+), 67 deletions(-) diff --git a/pkg/statistics/handle/bootstrap.go b/pkg/statistics/handle/bootstrap.go index 0cc355385565c..41f1253da6a47 100644 --- a/pkg/statistics/handle/bootstrap.go +++ b/pkg/statistics/handle/bootstrap.go @@ -271,9 +271,9 @@ func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, cache stats } } -// initStatsHistogramsSQLGen generates the SQL to load all stats_histograms records. +// genInitStatsHistogramsSQL generates the SQL to load all stats_histograms records. // We need to read all the records since we need to do initialization of table.ColAndIdxExistenceMap. -func initStatsHistogramsSQLGen(isPaging bool) string { +func genInitStatsHistogramsSQL(isPaging bool) string { selectPrefix := "select /*+ ORDER_INDEX(mysql.stats_histograms,tbl) */ HIGH_PRIORITY table_id, is_index, hist_id, distinct_count, version, null_count, cm_sketch, tot_col_size, stats_ver, correlation, flag, last_analyze_pos from mysql.stats_histograms" orderSuffix := " order by table_id" if !isPaging { @@ -283,7 +283,7 @@ func initStatsHistogramsSQLGen(isPaging bool) string { } func (h *Handle) initStatsHistogramsLite(ctx context.Context, cache statstypes.StatsCache) error { - sql := initStatsHistogramsSQLGen(false) + sql := genInitStatsHistogramsSQL(false) rc, err := util.Exec(h.initStatsCtx, sql) if err != nil { return errors.Trace(err) @@ -306,7 +306,7 @@ func (h *Handle) initStatsHistogramsLite(ctx context.Context, cache statstypes.S } func (h *Handle) initStatsHistograms(is infoschema.InfoSchema, cache statstypes.StatsCache) error { - sql := initStatsHistogramsSQLGen(false) + sql := genInitStatsHistogramsSQL(false) rc, err := util.Exec(h.initStatsCtx, sql) if err != nil { return errors.Trace(err) @@ -340,7 +340,7 @@ func (h *Handle) initStatsHistogramsByPaging(is infoschema.InfoSchema, cache sta }() sctx := se.(sessionctx.Context) - sql := initStatsHistogramsSQLGen(true) + sql := genInitStatsHistogramsSQL(true) rc, err := util.Exec(sctx, sql, task.StartTid, task.EndTid) if err != nil { return errors.Trace(err) @@ -419,10 +419,10 @@ func (*Handle) initStatsTopN4Chunk(cache statstypes.StatsCache, iter *chunk.Iter } } -// initStatsTopNSQLGen generates the SQL to load all stats_top_n records. +// genInitStatsTopNSQLForIndexes generates the SQL to load all stats_top_n records for indexes. // We only need to load the indexes' since we only record the existence of columns in ColAndIdxExistenceMap. // The stats of the column is not loaded during the bootstrap process. -func initStatsTopNSQLGen(isPaging bool) string { +func genInitStatsTopNSQLForIndexes(isPaging bool) string { selectPrefix := "select /*+ ORDER_INDEX(mysql.stats_top_n,tbl) */ HIGH_PRIORITY table_id, hist_id, value, count from mysql.stats_top_n where is_index = 1" orderSuffix := " order by table_id" if !isPaging { @@ -432,7 +432,7 @@ func initStatsTopNSQLGen(isPaging bool) string { } func (h *Handle) initStatsTopN(cache statstypes.StatsCache, totalMemory uint64) error { - sql := initStatsTopNSQLGen(false) + sql := genInitStatsTopNSQLForIndexes(false) rc, err := util.Exec(h.initStatsCtx, sql) if err != nil { return errors.Trace(err) @@ -465,7 +465,7 @@ func (h *Handle) initStatsTopNByPaging(cache statstypes.StatsCache, task initsta } }() sctx := se.(sessionctx.Context) - sql := initStatsTopNSQLGen(true) + sql := genInitStatsTopNSQLForIndexes(true) rc, err := util.Exec(sctx, sql, task.StartTid, task.EndTid) if err != nil { return errors.Trace(err) @@ -566,14 +566,13 @@ func (h *Handle) initStatsFMSketch(cache statstypes.StatsCache) error { func (*Handle) initStatsBuckets4Chunk(cache statstypes.StatsCache, iter *chunk.Iterator4Chunk) { var table *statistics.Table - unspecifiedLengthTp := types.NewFieldType(mysql.TypeBlob) var ( hasErr bool failedTableID int64 failedHistID int64 ) for row := iter.Begin(); row != iter.End(); row = iter.Next() { - tableID, isIndex, histID := row.GetInt64(0), row.GetInt64(1), row.GetInt64(2) + tableID, histID := row.GetInt64(0), row.GetInt64(1) if table == nil || table.PhysicalID != tableID { if table != nil { table.SetAllIndexFullLoadForBootstrap() @@ -588,58 +587,13 @@ func (*Handle) initStatsBuckets4Chunk(cache statstypes.StatsCache, iter *chunk.I } var lower, upper types.Datum var hist *statistics.Histogram - if isIndex > 0 { - index := table.GetIdx(histID) - if index == nil { - continue - } - hist = &index.Histogram - lower, upper = types.NewBytesDatum(row.GetBytes(5)), types.NewBytesDatum(row.GetBytes(6)) - } else { - column := table.GetCol(histID) - if column == nil { - continue - } - if !mysql.HasPriKeyFlag(column.Info.GetFlag()) { - continue - } - hist = &column.Histogram - d := types.NewBytesDatum(row.GetBytes(5)) - var err error - if column.Info.FieldType.EvalType() == types.ETString && column.Info.FieldType.GetType() != mysql.TypeEnum && column.Info.FieldType.GetType() != mysql.TypeSet { - // For new collation data, when storing the bounds of the histogram, we store the collate key instead of the - // original value. - // But there's additional conversion logic for new collation data, and the collate key might be longer than - // the FieldType.flen. - // If we use the original FieldType here, there might be errors like "Invalid utf8mb4 character string" - // or "Data too long". - // So we change it to TypeBlob to bypass those logics here. - lower, err = d.ConvertTo(statistics.UTCWithAllowInvalidDateCtx, unspecifiedLengthTp) - } else { - lower, err = d.ConvertTo(statistics.UTCWithAllowInvalidDateCtx, &column.Info.FieldType) - } - if err != nil { - hasErr = true - failedTableID = tableID - failedHistID = histID - table.DelCol(histID) - continue - } - d = types.NewBytesDatum(row.GetBytes(6)) - if column.Info.FieldType.EvalType() == types.ETString && column.Info.FieldType.GetType() != mysql.TypeEnum && column.Info.FieldType.GetType() != mysql.TypeSet { - upper, err = d.ConvertTo(statistics.UTCWithAllowInvalidDateCtx, unspecifiedLengthTp) - } else { - upper, err = d.ConvertTo(statistics.UTCWithAllowInvalidDateCtx, &column.Info.FieldType) - } - if err != nil { - hasErr = true - failedTableID = tableID - failedHistID = histID - table.DelCol(histID) - continue - } + index := table.GetIdx(histID) + if index == nil { + continue } - hist.AppendBucketWithNDV(&lower, &upper, row.GetInt64(3), row.GetInt64(4), row.GetInt64(7)) + hist = &index.Histogram + lower, upper = types.NewBytesDatum(row.GetBytes(4) /*lower_bound*/), types.NewBytesDatum(row.GetBytes(5) /*upper_bound*/) + hist.AppendBucketWithNDV(&lower, &upper, row.GetInt64(2) /*count*/, row.GetInt64(3) /*repeats*/, row.GetInt64(6) /*ndv*/) } if table != nil { cache.Put(table.PhysicalID, table) // put this table in the cache because all statstics of the table have been read. @@ -649,11 +603,11 @@ func (*Handle) initStatsBuckets4Chunk(cache statstypes.StatsCache, iter *chunk.I } } -// initStatsBucketsSQLGen generates the SQL to load all stats_top_n records. +// genInitStatsBucketsSQLForIndexes generates the SQL to load all stats_buckets records for indexes. // We only need to load the indexes' since we only record the existence of columns in ColAndIdxExistenceMap. // The stats of the column is not loaded during the bootstrap process. -func initStatsBucketsSQLGen(isPaging bool) string { - selectPrefix := "select /*+ ORDER_INDEX(mysql.stats_buckets,tbl) */ HIGH_PRIORITY table_id, is_index, hist_id, count, repeats, lower_bound, upper_bound, ndv from mysql.stats_buckets where is_index=1" +func genInitStatsBucketsSQLForIndexes(isPaging bool) string { + selectPrefix := "select /*+ ORDER_INDEX(mysql.stats_buckets,tbl) */ HIGH_PRIORITY table_id, hist_id, count, repeats, lower_bound, upper_bound, ndv from mysql.stats_buckets where is_index=1" orderSuffix := " order by table_id" if !isPaging { return selectPrefix + orderSuffix @@ -671,7 +625,7 @@ func (h *Handle) initStatsBuckets(cache statstypes.StatsCache, totalMemory uint6 return errors.Trace(err) } } else { - sql := initStatsBucketsSQLGen(false) + sql := genInitStatsBucketsSQLForIndexes(false) rc, err := util.Exec(h.initStatsCtx, sql) if err != nil { return errors.Trace(err) @@ -710,7 +664,7 @@ func (h *Handle) initStatsBucketsByPaging(cache statstypes.StatsCache, task init } }() sctx := se.(sessionctx.Context) - sql := initStatsBucketsSQLGen(true) + sql := genInitStatsBucketsSQLForIndexes(true) rc, err := util.Exec(sctx, sql, task.StartTid, task.EndTid) if err != nil { return errors.Trace(err)