From 2d6785d723b8411ee949a62dd0e98ce094c90e4c Mon Sep 17 00:00:00 2001 From: Rustin170506 Date: Thu, 19 Dec 2024 16:14:01 +0800 Subject: [PATCH 1/4] statistics: remove dead code Signed-off-by: Rustin170506 --- pkg/statistics/handle/bootstrap.go | 60 ++++-------------------------- 1 file changed, 7 insertions(+), 53 deletions(-) diff --git a/pkg/statistics/handle/bootstrap.go b/pkg/statistics/handle/bootstrap.go index 41e1abd97a5a1..ca55af3046b51 100644 --- a/pkg/statistics/handle/bootstrap.go +++ b/pkg/statistics/handle/bootstrap.go @@ -577,14 +577,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() @@ -599,57 +598,12 @@ 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 = &index.Histogram + lower, upper = types.NewBytesDatum(row.GetBytes(5)), types.NewBytesDatum(row.GetBytes(6)) hist.AppendBucketWithNDV(&lower, &upper, row.GetInt64(3), row.GetInt64(4), row.GetInt64(7)) } if table != nil { @@ -664,7 +618,7 @@ func (*Handle) initStatsBuckets4Chunk(cache statstypes.StatsCache, iter *chunk.I // 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" + 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 From 57f5dd6fdb9027c3d5d0e27182d440577d6e8c52 Mon Sep 17 00:00:00 2001 From: Rustin170506 Date: Fri, 20 Dec 2024 12:02:42 +0800 Subject: [PATCH 2/4] refactor: rename and update comments Signed-off-by: Rustin170506 --- pkg/statistics/handle/bootstrap.go | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/pkg/statistics/handle/bootstrap.go b/pkg/statistics/handle/bootstrap.go index ca55af3046b51..8db2556c2ef9a 100644 --- a/pkg/statistics/handle/bootstrap.go +++ b/pkg/statistics/handle/bootstrap.go @@ -282,9 +282,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 { @@ -294,7 +294,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) @@ -317,7 +317,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) @@ -351,7 +351,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) @@ -430,10 +430,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 { @@ -443,7 +443,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) @@ -476,7 +476,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) @@ -614,10 +614,10 @@ 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 { +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 { @@ -636,7 +636,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) @@ -675,7 +675,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) From 2449003a89c3b1efcf4d4abe460526b0c43aa7e8 Mon Sep 17 00:00:00 2001 From: Rustin170506 Date: Fri, 20 Dec 2024 15:30:13 +0800 Subject: [PATCH 3/4] fix: use correct index Signed-off-by: Rustin170506 --- pkg/statistics/handle/bootstrap.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/statistics/handle/bootstrap.go b/pkg/statistics/handle/bootstrap.go index 8db2556c2ef9a..453f5e544eea7 100644 --- a/pkg/statistics/handle/bootstrap.go +++ b/pkg/statistics/handle/bootstrap.go @@ -603,8 +603,8 @@ func (*Handle) initStatsBuckets4Chunk(cache statstypes.StatsCache, iter *chunk.I continue } hist = &index.Histogram - lower, upper = types.NewBytesDatum(row.GetBytes(5)), types.NewBytesDatum(row.GetBytes(6)) - hist.AppendBucketWithNDV(&lower, &upper, row.GetInt64(3), row.GetInt64(4), row.GetInt64(7)) + lower, upper = types.NewBytesDatum(row.GetBytes(4)), types.NewBytesDatum(row.GetBytes(5)) + hist.AppendBucketWithNDV(&lower, &upper, row.GetInt64(2), row.GetInt64(3), row.GetInt64(6)) } if table != nil { cache.Put(table.PhysicalID, table) // put this table in the cache because all statstics of the table have been read. From cca5abc151ee47726d9e4a4225208550ddb56a8c Mon Sep 17 00:00:00 2001 From: Rustin170506 Date: Fri, 20 Dec 2024 15:38:36 +0800 Subject: [PATCH 4/4] docs: better comments Signed-off-by: Rustin170506 --- pkg/statistics/handle/bootstrap.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/statistics/handle/bootstrap.go b/pkg/statistics/handle/bootstrap.go index 453f5e544eea7..a97d6b3e3c3f9 100644 --- a/pkg/statistics/handle/bootstrap.go +++ b/pkg/statistics/handle/bootstrap.go @@ -603,8 +603,8 @@ func (*Handle) initStatsBuckets4Chunk(cache statstypes.StatsCache, iter *chunk.I continue } hist = &index.Histogram - lower, upper = types.NewBytesDatum(row.GetBytes(4)), types.NewBytesDatum(row.GetBytes(5)) - hist.AppendBucketWithNDV(&lower, &upper, row.GetInt64(2), row.GetInt64(3), row.GetInt64(6)) + 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.