From b9726304a5f86e2770458dd694f3f33ffa217a04 Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Sun, 29 Jan 2023 15:48:55 +0800 Subject: [PATCH 1/4] move TableStatsFromStorage out of handle --- statistics/handle/gc.go | 7 +- statistics/handle/handle.go | 255 +++--------------- statistics/handle/handle_hist.go | 8 +- statistics/interact_with_storage.go | 384 ++++++++++++++++++++++++++++ 4 files changed, 428 insertions(+), 226 deletions(-) diff --git a/statistics/handle/gc.go b/statistics/handle/gc.go index f16e2c9719088..9483819c8c3ce 100644 --- a/statistics/handle/gc.go +++ b/statistics/handle/gc.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/mathutil" @@ -112,7 +113,7 @@ func (h *Handle) gcTableStats(is infoschema.InfoSchema, physicalID int64) error } } // Mark records in mysql.stats_extended as `deleted`. - rows, _, err = h.execRestrictedSQL(ctx, "select name, column_ids from mysql.stats_extended where table_id = %? and status in (%?, %?)", physicalID, StatsStatusAnalyzed, StatsStatusInited) + rows, _, err = h.execRestrictedSQL(ctx, "select name, column_ids from mysql.stats_extended where table_id = %? and status in (%?, %?)", physicalID, statistics.ExtendedStatsAnalyzed, statistics.ExtendedStatsInited) if err != nil { return errors.Trace(err) } @@ -289,7 +290,7 @@ func (h *Handle) DeleteTableStatsFromKV(statsIDs []int64) (err error) { if _, err = exec.ExecuteInternal(ctx, "delete from mysql.stats_feedback where table_id = %?", statsID); err != nil { return err } - if _, err = exec.ExecuteInternal(ctx, "update mysql.stats_extended set version = %?, status = %? where table_id = %? and status in (%?, %?)", startTS, StatsStatusDeleted, statsID, StatsStatusAnalyzed, StatsStatusInited); err != nil { + if _, err = exec.ExecuteInternal(ctx, "update mysql.stats_extended set version = %?, status = %? where table_id = %? and status in (%?, %?)", startTS, statistics.ExtendedStatsDeleted, statsID, statistics.ExtendedStatsAnalyzed, statistics.ExtendedStatsInited); err != nil { return err } if _, err = exec.ExecuteInternal(ctx, "delete from mysql.stats_fm_sketch where table_id = %?", statsID); err != nil { @@ -318,6 +319,6 @@ func (h *Handle) removeDeletedExtendedStats(version uint64) (err error) { err = finishTransaction(ctx, exec, err) }() const sql = "delete from mysql.stats_extended where status = %? and version < %?" - _, err = exec.ExecuteInternal(ctx, sql, StatsStatusDeleted, version) + _, err = exec.ExecuteInternal(ctx, sql, statistics.ExtendedStatsDeleted, version) return } diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 0f46a1f74f395..5c77180f59a28 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -27,7 +27,6 @@ import ( "github.com/ngaut/pools" "github.com/pingcap/errors" - "github.com/pingcap/failpoint" "github.com/pingcap/tidb/config" ddlUtil "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/infoschema" @@ -1078,17 +1077,17 @@ func (h *Handle) loadNeededColumnHistograms(reader *statistics.StatsReader, col statistics.HistogramNeededItems.Delete(col) return nil } - hg, err := h.histogramFromStorage(reader, col.TableID, c.ID, &c.Info.FieldType, c.Histogram.NDV, 0, c.LastUpdateVersion, c.NullCount, c.TotColSize, c.Correlation) + hg, err := statistics.HistogramFromStorage(reader, col.TableID, c.ID, &c.Info.FieldType, c.Histogram.NDV, 0, c.LastUpdateVersion, c.NullCount, c.TotColSize, c.Correlation) if err != nil { return errors.Trace(err) } - cms, topN, err := h.cmSketchAndTopNFromStorage(reader, col.TableID, 0, col.ID) + cms, topN, err := statistics.CMSketchAndTopNFromStorage(reader, col.TableID, 0, col.ID) if err != nil { return errors.Trace(err) } var fms *statistics.FMSketch if loadFMSketch { - fms, err = h.fmSketchFromStorage(reader, col.TableID, 0, col.ID) + fms, err = statistics.FMSketchFromStorage(reader, col.TableID, 0, col.ID) if err != nil { return errors.Trace(err) } @@ -1145,17 +1144,17 @@ func (h *Handle) loadNeededIndexHistograms(reader *statistics.StatsReader, idx m statistics.HistogramNeededItems.Delete(idx) return nil } - hg, err := h.histogramFromStorage(reader, idx.TableID, index.ID, types.NewFieldType(mysql.TypeBlob), index.Histogram.NDV, 1, index.LastUpdateVersion, index.NullCount, index.TotColSize, index.Correlation) + hg, err := statistics.HistogramFromStorage(reader, idx.TableID, index.ID, types.NewFieldType(mysql.TypeBlob), index.Histogram.NDV, 1, index.LastUpdateVersion, index.NullCount, index.TotColSize, index.Correlation) if err != nil { return errors.Trace(err) } - cms, topN, err := h.cmSketchAndTopNFromStorage(reader, idx.TableID, 1, idx.ID) + cms, topN, err := statistics.CMSketchAndTopNFromStorage(reader, idx.TableID, 1, idx.ID) if err != nil { return errors.Trace(err) } var fms *statistics.FMSketch if loadFMSketch { - fms, err = h.fmSketchFromStorage(reader, idx.TableID, 1, idx.ID) + fms, err = statistics.FMSketchFromStorage(reader, idx.TableID, 1, idx.ID) if err != nil { return errors.Trace(err) } @@ -1214,29 +1213,6 @@ func (h *Handle) FlushStats() { } } -func (h *Handle) cmSketchAndTopNFromStorage(reader *statistics.StatsReader, tblID int64, isIndex, histID int64) (_ *statistics.CMSketch, _ *statistics.TopN, err error) { - topNRows, _, err := reader.Read("select HIGH_PRIORITY value, count from mysql.stats_top_n where table_id = %? and is_index = %? and hist_id = %?", tblID, isIndex, histID) - if err != nil { - return nil, nil, err - } - rows, _, err := reader.Read("select cm_sketch from mysql.stats_histograms where table_id = %? and is_index = %? and hist_id = %?", tblID, isIndex, histID) - if err != nil { - return nil, nil, err - } - if len(rows) == 0 { - return statistics.DecodeCMSketchAndTopN(nil, topNRows) - } - return statistics.DecodeCMSketchAndTopN(rows[0].GetBytes(0), topNRows) -} - -func (h *Handle) fmSketchFromStorage(reader *statistics.StatsReader, tblID int64, isIndex, histID int64) (_ *statistics.FMSketch, err error) { - rows, _, err := reader.Read("select value from mysql.stats_fm_sketch where table_id = %? and is_index = %? and hist_id = %?", tblID, isIndex, histID) - if err != nil || len(rows) == 0 { - return nil, err - } - return statistics.DecodeFMSketch(rows[0].GetBytes(0)) -} - func (h *Handle) indexStatsFromStorage(reader *statistics.StatsReader, row chunk.Row, table *statistics.Table, tableInfo *model.TableInfo) error { histID := row.GetInt64(2) distinct := row.GetInt64(3) @@ -1257,15 +1233,15 @@ func (h *Handle) indexStatsFromStorage(reader *statistics.StatsReader, row chunk continue } if idx == nil || idx.LastUpdateVersion < histVer { - hg, err := h.histogramFromStorage(reader, table.PhysicalID, histID, types.NewFieldType(mysql.TypeBlob), distinct, 1, histVer, nullCount, 0, 0) + hg, err := statistics.HistogramFromStorage(reader, table.PhysicalID, histID, types.NewFieldType(mysql.TypeBlob), distinct, 1, histVer, nullCount, 0, 0) if err != nil { return errors.Trace(err) } - cms, topN, err := h.cmSketchAndTopNFromStorage(reader, table.PhysicalID, 1, idxInfo.ID) + cms, topN, err := statistics.CMSketchAndTopNFromStorage(reader, table.PhysicalID, 1, idxInfo.ID) if err != nil { return errors.Trace(err) } - fmSketch, err := h.fmSketchFromStorage(reader, table.PhysicalID, 1, histID) + fmSketch, err := statistics.FMSketchFromStorage(reader, table.PhysicalID, 1, histID) if err != nil { return errors.Trace(err) } @@ -1327,7 +1303,7 @@ func (h *Handle) columnStatsFromStorage(reader *statistics.StatsReader, row chun (col == nil || !col.IsStatsInitialized() && col.LastUpdateVersion < histVer) && !loadAll if notNeedLoad { - count, err := h.columnCountFromStorage(reader, table.PhysicalID, histID, statsVer) + count, err := statistics.ColumnCountFromStorage(reader, table.PhysicalID, histID, statsVer) if err != nil { return errors.Trace(err) } @@ -1351,11 +1327,11 @@ func (h *Handle) columnStatsFromStorage(reader *statistics.StatsReader, row chun break } if col == nil || col.LastUpdateVersion < histVer || loadAll { - hg, err := h.histogramFromStorage(reader, table.PhysicalID, histID, &colInfo.FieldType, distinct, 0, histVer, nullCount, totColSize, correlation) + hg, err := statistics.HistogramFromStorage(reader, table.PhysicalID, histID, &colInfo.FieldType, distinct, 0, histVer, nullCount, totColSize, correlation) if err != nil { return errors.Trace(err) } - cms, topN, err := h.cmSketchAndTopNFromStorage(reader, table.PhysicalID, 0, colInfo.ID) + cms, topN, err := statistics.CMSketchAndTopNFromStorage(reader, table.PhysicalID, 0, colInfo.ID) if err != nil { return errors.Trace(err) } @@ -1363,7 +1339,7 @@ func (h *Handle) columnStatsFromStorage(reader *statistics.StatsReader, row chun if loadAll { // FMSketch is only used when merging partition stats into global stats. When merging partition stats into global stats, // we load all the statistics, i.e., loadAll is true. - fmSketch, err = h.fmSketchFromStorage(reader, table.PhysicalID, 0, histID) + fmSketch, err = statistics.FMSketchFromStorage(reader, table.PhysicalID, 0, histID) if err != nil { return errors.Trace(err) } @@ -1420,97 +1396,28 @@ func (h *Handle) TableStatsFromStorage(tableInfo *model.TableInfo, physicalID in err = err1 } }() - table, ok := h.statsCache.Load().(statsCache).Get(physicalID) - // If table stats is pseudo, we also need to copy it, since we will use the column stats when - // the average error rate of it is small. - if !ok || snapshot > 0 { - histColl := statistics.HistColl{ - PhysicalID: physicalID, - HavePhysicalID: true, - Columns: make(map[int64]*statistics.Column, len(tableInfo.Columns)), - Indices: make(map[int64]*statistics.Index, len(tableInfo.Indices)), - } - table = &statistics.Table{ - HistColl: histColl, - } - } else { - // We copy it before writing to avoid race. - table = table.Copy() + statsTbl, ok := h.statsCache.Load().(statsCache).Get(physicalID) + if !ok { + statsTbl = nil } - table.Pseudo = false - - rows, _, err := reader.Read("select modify_count, count from mysql.stats_meta where table_id = %?", physicalID) - if err != nil || len(rows) == 0 { + statsTbl, err = statistics.TableStatsFromStorage(reader, tableInfo, physicalID, loadAll, h.Lease(), statsTbl) + if err != nil { return nil, err } - table.ModifyCount = rows[0].GetInt64(0) - table.Count = rows[0].GetInt64(1) - - rows, _, err = reader.Read("select table_id, is_index, hist_id, distinct_count, version, null_count, tot_col_size, stats_ver, flag, correlation, last_analyze_pos from mysql.stats_histograms where table_id = %?", physicalID) - // Check deleted table. - if err != nil || len(rows) == 0 { - return nil, nil + if reader.IsHistory() { + return statsTbl, err } - for _, row := range rows { - if row.GetInt64(1) > 0 { - err = h.indexStatsFromStorage(reader, row, table, tableInfo) - } else { - err = h.columnStatsFromStorage(reader, row, table, tableInfo, loadAll) - } - if err != nil { - return nil, err + for histID, idx := range statsTbl.Indices { + if statistics.IsAnalyzed(idx.Flag) { + h.mu.rateMap.clear(physicalID, histID, true) } } - return h.extendedStatsFromStorage(reader, table, physicalID, loadAll) -} - -func (h *Handle) extendedStatsFromStorage(reader *statistics.StatsReader, table *statistics.Table, physicalID int64, loadAll bool) (*statistics.Table, error) { - failpoint.Inject("injectExtStatsLoadErr", func() { - failpoint.Return(nil, errors.New("gofail extendedStatsFromStorage error")) - }) - lastVersion := uint64(0) - if table.ExtendedStats != nil && !loadAll { - lastVersion = table.ExtendedStats.LastUpdateVersion - } else { - table.ExtendedStats = statistics.NewExtendedStatsColl() - } - rows, _, err := reader.Read("select name, status, type, column_ids, stats, version from mysql.stats_extended where table_id = %? and status in (%?, %?, %?) and version > %?", physicalID, StatsStatusInited, StatsStatusAnalyzed, StatsStatusDeleted, lastVersion) - if err != nil || len(rows) == 0 { - return table, nil - } - for _, row := range rows { - lastVersion = mathutil.Max(lastVersion, row.GetUint64(5)) - name := row.GetString(0) - status := uint8(row.GetInt64(1)) - if status == StatsStatusDeleted || status == StatsStatusInited { - delete(table.ExtendedStats.Stats, name) - } else { - item := &statistics.ExtendedStatsItem{ - Tp: uint8(row.GetInt64(2)), - } - colIDs := row.GetString(3) - err := json.Unmarshal([]byte(colIDs), &item.ColIDs) - if err != nil { - logutil.BgLogger().Error("[stats] decode column IDs failed", zap.String("column_ids", colIDs), zap.Error(err)) - return nil, err - } - statsStr := row.GetString(4) - if item.Tp == ast.StatsTypeCardinality || item.Tp == ast.StatsTypeCorrelation { - if statsStr != "" { - item.ScalarVals, err = strconv.ParseFloat(statsStr, 64) - if err != nil { - logutil.BgLogger().Error("[stats] parse scalar stats failed", zap.String("stats", statsStr), zap.Error(err)) - return nil, err - } - } - } else { - item.StringVals = statsStr - } - table.ExtendedStats.Stats[name] = item + for histID, col := range statsTbl.Columns { + if statistics.IsAnalyzed(col.Flag) { + h.mu.rateMap.clear(physicalID, histID, false) } } - table.ExtendedStats.LastUpdateVersion = lastVersion - return table, nil + return statsTbl, nil } // StatsMetaCountAndModifyCount reads count and modify_count for the given table from mysql.stats_meta. @@ -1794,7 +1701,7 @@ func SaveTableStatsToStorage(sctx sessionctx.Context, results *statistics.Analyz case ast.StatsTypeDependency: statsStr = item.StringVals } - if _, err = exec.ExecuteInternal(ctx, "replace into mysql.stats_extended values (%?, %?, %?, %?, %?, %?, %?)", name, item.Tp, tableID, strColIDs, statsStr, version, StatsStatusAnalyzed); err != nil { + if _, err = exec.ExecuteInternal(ctx, "replace into mysql.stats_extended values (%?, %?, %?, %?, %?, %?, %?)", name, item.Tp, tableID, strColIDs, statsStr, version, statistics.ExtendedStatsAnalyzed); err != nil { return err } } @@ -1913,87 +1820,6 @@ func (h *Handle) SaveMetaToStorage(tableID, count, modifyCount int64, source str return err } -func (h *Handle) histogramFromStorage(reader *statistics.StatsReader, tableID int64, colID int64, tp *types.FieldType, distinct int64, isIndex int, ver uint64, nullCount int64, totColSize int64, corr float64) (_ *statistics.Histogram, err error) { - rows, fields, err := reader.Read("select count, repeats, lower_bound, upper_bound, ndv from mysql.stats_buckets where table_id = %? and is_index = %? and hist_id = %? order by bucket_id", tableID, isIndex, colID) - if err != nil { - return nil, errors.Trace(err) - } - bucketSize := len(rows) - hg := statistics.NewHistogram(colID, distinct, nullCount, ver, tp, bucketSize, totColSize) - hg.Correlation = corr - totalCount := int64(0) - for i := 0; i < bucketSize; i++ { - count := rows[i].GetInt64(0) - repeats := rows[i].GetInt64(1) - var upperBound, lowerBound types.Datum - if isIndex == 1 { - lowerBound = rows[i].GetDatum(2, &fields[2].Column.FieldType) - upperBound = rows[i].GetDatum(3, &fields[3].Column.FieldType) - } else { - // Invalid date values may be inserted into table under some relaxed sql mode. Those values may exist in statistics. - // Hence, when reading statistics, we should skip invalid date check. See #39336. - sc := &stmtctx.StatementContext{TimeZone: time.UTC, AllowInvalidDate: true, IgnoreZeroInDate: true} - d := rows[i].GetDatum(2, &fields[2].Column.FieldType) - // 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. - if tp.EvalType() == types.ETString && tp.GetType() != mysql.TypeEnum && tp.GetType() != mysql.TypeSet { - tp = types.NewFieldType(mysql.TypeBlob) - } - lowerBound, err = d.ConvertTo(sc, tp) - if err != nil { - return nil, errors.Trace(err) - } - d = rows[i].GetDatum(3, &fields[3].Column.FieldType) - upperBound, err = d.ConvertTo(sc, tp) - if err != nil { - return nil, errors.Trace(err) - } - } - totalCount += count - hg.AppendBucketWithNDV(&lowerBound, &upperBound, totalCount, repeats, rows[i].GetInt64(4)) - } - hg.PreCalculateScalar() - return hg, nil -} - -func (h *Handle) columnCountFromStorage(reader *statistics.StatsReader, tableID, colID, statsVer int64) (int64, error) { - count := int64(0) - rows, _, err := reader.Read("select sum(count) from mysql.stats_buckets where table_id = %? and is_index = 0 and hist_id = %?", tableID, colID) - if err != nil { - return 0, errors.Trace(err) - } - // If there doesn't exist any buckets, the SQL will return NULL. So we only use the result if it's not NULL. - if !rows[0].IsNull(0) { - count, err = rows[0].GetMyDecimal(0).ToInt() - if err != nil { - return 0, errors.Trace(err) - } - } - - if statsVer >= statistics.Version2 { - // Before stats ver 2, histogram represents all data in this column. - // In stats ver 2, histogram + TopN represent all data in this column. - // So we need to add TopN total count here. - rows, _, err = reader.Read("select sum(count) from mysql.stats_top_n where table_id = %? and is_index = 0 and hist_id = %?", tableID, colID) - if err != nil { - return 0, errors.Trace(err) - } - if !rows[0].IsNull(0) { - topNCount, err := rows[0].GetMyDecimal(0).ToInt() - if err != nil { - return 0, errors.Trace(err) - } - count += topNCount - } - } - return count, err -} - func (h *Handle) statsMetaByTableIDFromStorage(tableID int64, snapshot uint64) (version uint64, modifyCount, count int64, err error) { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) var rows []chunk.Row @@ -2032,15 +1858,6 @@ func (h *Handle) releaseGlobalStatsReader(reader *statistics.StatsReader) error return reader.Close() } -const ( - // StatsStatusInited is the status for extended stats which are just registered but have not been analyzed yet. - StatsStatusInited uint8 = iota - // StatsStatusAnalyzed is the status for extended stats which have been collected in analyze. - StatsStatusAnalyzed - // StatsStatusDeleted is the status for extended stats which were dropped. These "deleted" records would be removed from storage by GCStats(). - StatsStatusDeleted -) - // InsertExtendedStats inserts a record into mysql.stats_extended and update version in mysql.stats_meta. func (h *Handle) InsertExtendedStats(statsName string, colIDs []int64, tp int, tableID int64, ifNotExists bool) (err error) { statsVer := uint64(0) @@ -2067,7 +1884,7 @@ func (h *Handle) InsertExtendedStats(statsName string, colIDs []int64, tp int, t err = finishTransaction(ctx, exec, err) }() // No need to use `exec.ExecuteInternal` since we have acquired the lock. - rows, _, err := h.execRestrictedSQL(ctx, "SELECT name, type, column_ids FROM mysql.stats_extended WHERE table_id = %? and status in (%?, %?)", tableID, StatsStatusInited, StatsStatusAnalyzed) + rows, _, err := h.execRestrictedSQL(ctx, "SELECT name, type, column_ids FROM mysql.stats_extended WHERE table_id = %? and status in (%?, %?)", tableID, statistics.ExtendedStatsInited, statistics.ExtendedStatsAnalyzed) if err != nil { return errors.Trace(err) } @@ -2106,7 +1923,7 @@ func (h *Handle) InsertExtendedStats(statsName string, colIDs []int64, tp int, t // next `Update()` to remove the cached item then. h.removeExtendedStatsItem(tableID, statsName) const sql = "INSERT INTO mysql.stats_extended(name, type, table_id, column_ids, version, status) VALUES (%?, %?, %?, %?, %?, %?)" - if _, err = exec.ExecuteInternal(ctx, sql, statsName, tp, tableID, strColIDs, version, StatsStatusInited); err != nil { + if _, err = exec.ExecuteInternal(ctx, sql, statsName, tp, tableID, strColIDs, version, statistics.ExtendedStatsInited); err != nil { return err } return @@ -2121,7 +1938,7 @@ func (h *Handle) MarkExtendedStatsDeleted(statsName string, tableID int64, ifExi } }() ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) - rows, _, err := h.execRestrictedSQL(ctx, "SELECT name FROM mysql.stats_extended WHERE name = %? and table_id = %? and status in (%?, %?)", statsName, tableID, StatsStatusInited, StatsStatusAnalyzed) + rows, _, err := h.execRestrictedSQL(ctx, "SELECT name FROM mysql.stats_extended WHERE name = %? and table_id = %? and status in (%?, %?)", statsName, tableID, statistics.ExtendedStatsInited, statistics.ExtendedStatsAnalyzed) if err != nil { return errors.Trace(err) } @@ -2158,7 +1975,7 @@ func (h *Handle) MarkExtendedStatsDeleted(statsName string, tableID int64, ifExi return err } statsVer = version - if _, err = exec.ExecuteInternal(ctx, "UPDATE mysql.stats_extended SET version = %?, status = %? WHERE name = %? and table_id = %?", version, StatsStatusDeleted, statsName, tableID); err != nil { + if _, err = exec.ExecuteInternal(ctx, "UPDATE mysql.stats_extended SET version = %?, status = %? WHERE name = %? and table_id = %?", version, statistics.ExtendedStatsDeleted, statsName, tableID); err != nil { return err } return nil @@ -2202,7 +2019,7 @@ func (h *Handle) ReloadExtendedStatistics() error { oldCache := h.statsCache.Load().(statsCache) tables := make([]*statistics.Table, 0, oldCache.Len()) for physicalID, tbl := range oldCache.Map() { - t, err := h.extendedStatsFromStorage(reader, tbl.Copy(), physicalID, true) + t, err := statistics.ExtendedStatsFromStorage(reader, tbl.Copy(), physicalID, true) if err != nil { return err } @@ -2219,7 +2036,7 @@ func (h *Handle) ReloadExtendedStatistics() error { func (h *Handle) BuildExtendedStats(tableID int64, cols []*model.ColumnInfo, collectors []*statistics.SampleCollector) (*statistics.ExtendedStatsColl, error) { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) const sql = "SELECT name, type, column_ids FROM mysql.stats_extended WHERE table_id = %? and status in (%?, %?)" - rows, _, err := h.execRestrictedSQL(ctx, sql, tableID, StatsStatusAnalyzed, StatsStatusInited) + rows, _, err := h.execRestrictedSQL(ctx, sql, tableID, statistics.ExtendedStatsAnalyzed, statistics.ExtendedStatsInited) if err != nil { return nil, errors.Trace(err) } @@ -2366,7 +2183,7 @@ func (h *Handle) SaveExtendedStatsToStorage(tableID int64, extStats *statistics. statsStr = item.StringVals } // If isLoad is true, it's INSERT; otherwise, it's UPDATE. - if _, err := exec.ExecuteInternal(ctx, "replace into mysql.stats_extended values (%?, %?, %?, %?, %?, %?, %?)", name, item.Tp, tableID, strColIDs, statsStr, version, StatsStatusAnalyzed); err != nil { + if _, err := exec.ExecuteInternal(ctx, "replace into mysql.stats_extended values (%?, %?, %?, %?, %?, %?, %?)", name, item.Tp, tableID, strColIDs, statsStr, version, statistics.ExtendedStatsAnalyzed); err != nil { return err } } @@ -2483,7 +2300,7 @@ func (h *Handle) LoadColumnStatsUsage(loc *time.Location) (map[model.TableItemID func (h *Handle) CollectColumnsInExtendedStats(tableID int64) ([]int64, error) { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) const sql = "SELECT name, type, column_ids FROM mysql.stats_extended WHERE table_id = %? and status in (%?, %?)" - rows, _, err := h.execRestrictedSQL(ctx, sql, tableID, StatsStatusAnalyzed, StatsStatusInited) + rows, _, err := h.execRestrictedSQL(ctx, sql, tableID, statistics.ExtendedStatsAnalyzed, statistics.ExtendedStatsInited) if err != nil { return nil, errors.Trace(err) } diff --git a/statistics/handle/handle_hist.go b/statistics/handle/handle_hist.go index 1d41e14791446..6c49afe736afe 100644 --- a/statistics/handle/handle_hist.go +++ b/statistics/handle/handle_hist.go @@ -334,25 +334,25 @@ func (h *Handle) readStatsForOneItem(item model.TableItemID, w *statsWrapper, re isIndexFlag = 1 } if item.IsIndex { - hg, err = h.histogramFromStorage(reader, item.TableID, item.ID, types.NewFieldType(mysql.TypeBlob), index.Histogram.NDV, int(isIndexFlag), index.LastUpdateVersion, index.NullCount, index.TotColSize, index.Correlation) + hg, err = statistics.HistogramFromStorage(reader, item.TableID, item.ID, types.NewFieldType(mysql.TypeBlob), index.Histogram.NDV, int(isIndexFlag), index.LastUpdateVersion, index.NullCount, index.TotColSize, index.Correlation) if err != nil { return nil, errors.Trace(err) } } else { - hg, err = h.histogramFromStorage(reader, item.TableID, item.ID, &c.Info.FieldType, c.Histogram.NDV, int(isIndexFlag), c.LastUpdateVersion, c.NullCount, c.TotColSize, c.Correlation) + hg, err = statistics.HistogramFromStorage(reader, item.TableID, item.ID, &c.Info.FieldType, c.Histogram.NDV, int(isIndexFlag), c.LastUpdateVersion, c.NullCount, c.TotColSize, c.Correlation) if err != nil { return nil, errors.Trace(err) } } var cms *statistics.CMSketch var topN *statistics.TopN - cms, topN, err = h.cmSketchAndTopNFromStorage(reader, item.TableID, isIndexFlag, item.ID) + cms, topN, err = statistics.CMSketchAndTopNFromStorage(reader, item.TableID, isIndexFlag, item.ID) if err != nil { return nil, errors.Trace(err) } var fms *statistics.FMSketch if loadFMSketch { - fms, err = h.fmSketchFromStorage(reader, item.TableID, isIndexFlag, item.ID) + fms, err = statistics.FMSketchFromStorage(reader, item.TableID, isIndexFlag, item.ID) if err != nil { return nil, errors.Trace(err) } diff --git a/statistics/interact_with_storage.go b/statistics/interact_with_storage.go index 478b845937067..dc8ea0c948d85 100644 --- a/statistics/interact_with_storage.go +++ b/statistics/interact_with_storage.go @@ -16,13 +16,23 @@ package statistics import ( "context" + "encoding/json" "fmt" + "go.uber.org/zap" + "strconv" + "time" "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/ast" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/mathutil" "github.com/pingcap/tidb/util/sqlexec" ) @@ -84,3 +94,377 @@ func (sr *StatsReader) Close() error { _, err := sr.ctx.(sqlexec.SQLExecutor).ExecuteInternal(ctx, "commit") return err } + +func HistogramFromStorage(reader *StatsReader, tableID int64, colID int64, tp *types.FieldType, distinct int64, isIndex int, ver uint64, nullCount int64, totColSize int64, corr float64) (_ *Histogram, err error) { + rows, fields, err := reader.Read("select count, repeats, lower_bound, upper_bound, ndv from mysql.stats_buckets where table_id = %? and is_index = %? and hist_id = %? order by bucket_id", tableID, isIndex, colID) + if err != nil { + return nil, errors.Trace(err) + } + bucketSize := len(rows) + hg := NewHistogram(colID, distinct, nullCount, ver, tp, bucketSize, totColSize) + hg.Correlation = corr + totalCount := int64(0) + for i := 0; i < bucketSize; i++ { + count := rows[i].GetInt64(0) + repeats := rows[i].GetInt64(1) + var upperBound, lowerBound types.Datum + if isIndex == 1 { + lowerBound = rows[i].GetDatum(2, &fields[2].Column.FieldType) + upperBound = rows[i].GetDatum(3, &fields[3].Column.FieldType) + } else { + // Invalid date values may be inserted into table under some relaxed sql mode. Those values may exist in statistics. + // Hence, when reading statistics, we should skip invalid date check. See #39336. + sc := &stmtctx.StatementContext{TimeZone: time.UTC, AllowInvalidDate: true, IgnoreZeroInDate: true} + d := rows[i].GetDatum(2, &fields[2].Column.FieldType) + // 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. + if tp.EvalType() == types.ETString && tp.GetType() != mysql.TypeEnum && tp.GetType() != mysql.TypeSet { + tp = types.NewFieldType(mysql.TypeBlob) + } + lowerBound, err = d.ConvertTo(sc, tp) + if err != nil { + return nil, errors.Trace(err) + } + d = rows[i].GetDatum(3, &fields[3].Column.FieldType) + upperBound, err = d.ConvertTo(sc, tp) + if err != nil { + return nil, errors.Trace(err) + } + } + totalCount += count + hg.AppendBucketWithNDV(&lowerBound, &upperBound, totalCount, repeats, rows[i].GetInt64(4)) + } + hg.PreCalculateScalar() + return hg, nil +} + +func CMSketchAndTopNFromStorage(reader *StatsReader, tblID int64, isIndex, histID int64) (_ *CMSketch, _ *TopN, err error) { + topNRows, _, err := reader.Read("select HIGH_PRIORITY value, count from mysql.stats_top_n where table_id = %? and is_index = %? and hist_id = %?", tblID, isIndex, histID) + if err != nil { + return nil, nil, err + } + rows, _, err := reader.Read("select cm_sketch from mysql.stats_histograms where table_id = %? and is_index = %? and hist_id = %?", tblID, isIndex, histID) + if err != nil { + return nil, nil, err + } + if len(rows) == 0 { + return DecodeCMSketchAndTopN(nil, topNRows) + } + return DecodeCMSketchAndTopN(rows[0].GetBytes(0), topNRows) +} + +func FMSketchFromStorage(reader *StatsReader, tblID int64, isIndex, histID int64) (_ *FMSketch, err error) { + rows, _, err := reader.Read("select value from mysql.stats_fm_sketch where table_id = %? and is_index = %? and hist_id = %?", tblID, isIndex, histID) + if err != nil || len(rows) == 0 { + return nil, err + } + return DecodeFMSketch(rows[0].GetBytes(0)) +} + +func ColumnCountFromStorage(reader *StatsReader, tableID, colID, statsVer int64) (int64, error) { + count := int64(0) + rows, _, err := reader.Read("select sum(count) from mysql.stats_buckets where table_id = %? and is_index = 0 and hist_id = %?", tableID, colID) + if err != nil { + return 0, errors.Trace(err) + } + // If there doesn't exist any buckets, the SQL will return NULL. So we only use the result if it's not NULL. + if !rows[0].IsNull(0) { + count, err = rows[0].GetMyDecimal(0).ToInt() + if err != nil { + return 0, errors.Trace(err) + } + } + + if statsVer >= Version2 { + // Before stats ver 2, histogram represents all data in this column. + // In stats ver 2, histogram + TopN represent all data in this column. + // So we need to add TopN total count here. + rows, _, err = reader.Read("select sum(count) from mysql.stats_top_n where table_id = %? and is_index = 0 and hist_id = %?", tableID, colID) + if err != nil { + return 0, errors.Trace(err) + } + if !rows[0].IsNull(0) { + topNCount, err := rows[0].GetMyDecimal(0).ToInt() + if err != nil { + return 0, errors.Trace(err) + } + count += topNCount + } + } + return count, err +} + +const ( + // ExtendedStatsInited is the status for extended stats which are just registered but have not been analyzed yet. + ExtendedStatsInited uint8 = iota + // ExtendedStatsAnalyzed is the status for extended stats which have been collected in analyze. + ExtendedStatsAnalyzed + // ExtendedStatsDeleted is the status for extended stats which were dropped. These "deleted" records would be removed from storage by GCStats(). + ExtendedStatsDeleted +) + +func ExtendedStatsFromStorage(reader *StatsReader, table *Table, physicalID int64, loadAll bool) (*Table, error) { + failpoint.Inject("injectExtStatsLoadErr", func() { + failpoint.Return(nil, errors.New("gofail extendedStatsFromStorage error")) + }) + lastVersion := uint64(0) + if table.ExtendedStats != nil && !loadAll { + lastVersion = table.ExtendedStats.LastUpdateVersion + } else { + table.ExtendedStats = NewExtendedStatsColl() + } + rows, _, err := reader.Read("select name, status, type, column_ids, stats, version from mysql.stats_extended where table_id = %? and status in (%?, %?, %?) and version > %?", physicalID, ExtendedStatsInited, ExtendedStatsAnalyzed, ExtendedStatsDeleted, lastVersion) + if err != nil || len(rows) == 0 { + return table, nil + } + for _, row := range rows { + lastVersion = mathutil.Max(lastVersion, row.GetUint64(5)) + name := row.GetString(0) + status := uint8(row.GetInt64(1)) + if status == ExtendedStatsDeleted || status == ExtendedStatsInited { + delete(table.ExtendedStats.Stats, name) + } else { + item := &ExtendedStatsItem{ + Tp: uint8(row.GetInt64(2)), + } + colIDs := row.GetString(3) + err := json.Unmarshal([]byte(colIDs), &item.ColIDs) + if err != nil { + logutil.BgLogger().Error("[stats] decode column IDs failed", zap.String("column_ids", colIDs), zap.Error(err)) + return nil, err + } + statsStr := row.GetString(4) + if item.Tp == ast.StatsTypeCardinality || item.Tp == ast.StatsTypeCorrelation { + if statsStr != "" { + item.ScalarVals, err = strconv.ParseFloat(statsStr, 64) + if err != nil { + logutil.BgLogger().Error("[stats] parse scalar stats failed", zap.String("stats", statsStr), zap.Error(err)) + return nil, err + } + } + } else { + item.StringVals = statsStr + } + table.ExtendedStats.Stats[name] = item + } + } + table.ExtendedStats.LastUpdateVersion = lastVersion + return table, nil +} + +func indexStatsFromStorage(reader *StatsReader, row chunk.Row, table *Table, tableInfo *model.TableInfo) error { + histID := row.GetInt64(2) + distinct := row.GetInt64(3) + histVer := row.GetUint64(4) + nullCount := row.GetInt64(5) + statsVer := row.GetInt64(7) + idx := table.Indices[histID] + errorRate := ErrorRate{} + flag := row.GetInt64(8) + lastAnalyzePos := row.GetDatum(10, types.NewFieldType(mysql.TypeBlob)) + if (!IsAnalyzed(flag) || reader.IsHistory()) && idx != nil { + errorRate = idx.ErrorRate + } + for _, idxInfo := range tableInfo.Indices { + if histID != idxInfo.ID { + continue + } + if idx == nil || idx.LastUpdateVersion < histVer { + hg, err := HistogramFromStorage(reader, table.PhysicalID, histID, types.NewFieldType(mysql.TypeBlob), distinct, 1, histVer, nullCount, 0, 0) + if err != nil { + return errors.Trace(err) + } + cms, topN, err := CMSketchAndTopNFromStorage(reader, table.PhysicalID, 1, idxInfo.ID) + if err != nil { + return errors.Trace(err) + } + fmSketch, err := FMSketchFromStorage(reader, table.PhysicalID, 1, histID) + if err != nil { + return errors.Trace(err) + } + idx = &Index{ + Histogram: *hg, + CMSketch: cms, + TopN: topN, + FMSketch: fmSketch, + Info: idxInfo, + ErrorRate: errorRate, + StatsVer: statsVer, + Flag: flag, + PhysicalID: table.PhysicalID, + } + if statsVer != Version0 { + idx.StatsLoadedStatus = NewStatsFullLoadStatus() + } + lastAnalyzePos.Copy(&idx.LastAnalyzePos) + } + break + } + if idx != nil { + table.Indices[histID] = idx + } else { + logutil.BgLogger().Debug("we cannot find index id in table info. It may be deleted.", zap.Int64("indexID", histID), zap.String("table", tableInfo.Name.O)) + } + return nil +} + +func columnStatsFromStorage(reader *StatsReader, row chunk.Row, table *Table, tableInfo *model.TableInfo, loadAll bool, lease time.Duration) error { + histID := row.GetInt64(2) + distinct := row.GetInt64(3) + histVer := row.GetUint64(4) + nullCount := row.GetInt64(5) + totColSize := row.GetInt64(6) + statsVer := row.GetInt64(7) + correlation := row.GetFloat64(9) + lastAnalyzePos := row.GetDatum(10, types.NewFieldType(mysql.TypeBlob)) + col := table.Columns[histID] + errorRate := ErrorRate{} + flag := row.GetInt64(8) + if (!IsAnalyzed(flag) || reader.IsHistory()) && col != nil { + errorRate = col.ErrorRate + } + for _, colInfo := range tableInfo.Columns { + if histID != colInfo.ID { + continue + } + isHandle := tableInfo.PKIsHandle && mysql.HasPriKeyFlag(colInfo.GetFlag()) + // We will not load buckets if: + // 1. lease > 0, and: + // 2. this column is not handle, and: + // 3. the column doesn't has any statistics before, and: + // 4. loadAll is false. + notNeedLoad := lease > 0 && + !isHandle && + (col == nil || !col.IsStatsInitialized() && col.LastUpdateVersion < histVer) && + !loadAll + if notNeedLoad { + count, err := ColumnCountFromStorage(reader, table.PhysicalID, histID, statsVer) + if err != nil { + return errors.Trace(err) + } + col = &Column{ + PhysicalID: table.PhysicalID, + Histogram: *NewHistogram(histID, distinct, nullCount, histVer, &colInfo.FieldType, 0, totColSize), + Info: colInfo, + Count: count + nullCount, + ErrorRate: errorRate, + IsHandle: tableInfo.PKIsHandle && mysql.HasPriKeyFlag(colInfo.GetFlag()), + Flag: flag, + StatsVer: statsVer, + } + // When adding/modifying a column, we create its stats(all values are default values) without setting stats_ver. + // So we need add col.Count > 0 here. + if statsVer != Version0 || col.Count > 0 { + col.StatsLoadedStatus = NewStatsAllEvictedStatus() + } + lastAnalyzePos.Copy(&col.LastAnalyzePos) + col.Histogram.Correlation = correlation + break + } + if col == nil || col.LastUpdateVersion < histVer || loadAll { + hg, err := HistogramFromStorage(reader, table.PhysicalID, histID, &colInfo.FieldType, distinct, 0, histVer, nullCount, totColSize, correlation) + if err != nil { + return errors.Trace(err) + } + cms, topN, err := CMSketchAndTopNFromStorage(reader, table.PhysicalID, 0, colInfo.ID) + if err != nil { + return errors.Trace(err) + } + var fmSketch *FMSketch + if loadAll { + // FMSketch is only used when merging partition stats into global stats. When merging partition stats into global stats, + // we load all the statistics, i.e., loadAll is true. + fmSketch, err = FMSketchFromStorage(reader, table.PhysicalID, 0, histID) + if err != nil { + return errors.Trace(err) + } + } + col = &Column{ + PhysicalID: table.PhysicalID, + Histogram: *hg, + Info: colInfo, + CMSketch: cms, + TopN: topN, + FMSketch: fmSketch, + ErrorRate: errorRate, + IsHandle: tableInfo.PKIsHandle && mysql.HasPriKeyFlag(colInfo.GetFlag()), + Flag: flag, + StatsVer: statsVer, + } + // Column.Count is calculated by Column.TotalRowCount(). Hence we don't set Column.Count when initializing col. + col.Count = int64(col.TotalRowCount()) + // When adding/modifying a column, we create its stats(all values are default values) without setting stats_ver. + // So we need add colHist.Count > 0 here. + if statsVer != Version0 || col.Count > 0 { + col.StatsLoadedStatus = NewStatsFullLoadStatus() + } + lastAnalyzePos.Copy(&col.LastAnalyzePos) + break + } + if col.TotColSize != totColSize { + newCol := *col + newCol.TotColSize = totColSize + col = &newCol + } + break + } + if col != nil { + table.Columns[col.ID] = col + } else { + // If we didn't find a Column or Index in tableInfo, we won't load the histogram for it. + // But don't worry, next lease the ddl will be updated, and we will load a same table for two times to + // avoid error. + logutil.BgLogger().Debug("we cannot find column in table info now. It may be deleted", zap.Int64("colID", histID), zap.String("table", tableInfo.Name.O)) + } + return nil +} + +// TableStatsFromStorage loads table stats info from storage. +func TableStatsFromStorage(reader *StatsReader, tableInfo *model.TableInfo, physicalID int64, loadAll bool, lease time.Duration, table *Table) (_ *Table, err error) { + // If table stats is pseudo, we also need to copy it, since we will use the column stats when + // the average error rate of it is small. + if table == nil || reader.IsHistory() { + histColl := HistColl{ + PhysicalID: physicalID, + HavePhysicalID: true, + Columns: make(map[int64]*Column, len(tableInfo.Columns)), + Indices: make(map[int64]*Index, len(tableInfo.Indices)), + } + table = &Table{ + HistColl: histColl, + } + } else { + // We copy it before writing to avoid race. + table = table.Copy() + } + table.Pseudo = false + + rows, _, err := reader.Read("select modify_count, count from mysql.stats_meta where table_id = %?", physicalID) + if err != nil || len(rows) == 0 { + return nil, err + } + table.ModifyCount = rows[0].GetInt64(0) + table.Count = rows[0].GetInt64(1) + + rows, _, err = reader.Read("select table_id, is_index, hist_id, distinct_count, version, null_count, tot_col_size, stats_ver, flag, correlation, last_analyze_pos from mysql.stats_histograms where table_id = %?", physicalID) + // Check deleted table. + if err != nil || len(rows) == 0 { + return nil, nil + } + for _, row := range rows { + if row.GetInt64(1) > 0 { + err = indexStatsFromStorage(reader, row, table, tableInfo) + } else { + err = columnStatsFromStorage(reader, row, table, tableInfo, loadAll, lease) + } + if err != nil { + return nil, err + } + } + return ExtendedStatsFromStorage(reader, table, physicalID, loadAll) +} From 0ad0baf7684c7e0b24ce3d2f445fc860fac1e696 Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Thu, 2 Feb 2023 14:38:14 +0800 Subject: [PATCH 2/4] add comment and fix fmt --- statistics/interact_with_storage.go | 16 ++++++---------- statistics/table.go | 9 +++++++++ 2 files changed, 15 insertions(+), 10 deletions(-) diff --git a/statistics/interact_with_storage.go b/statistics/interact_with_storage.go index dc8ea0c948d85..c0acce31b77b7 100644 --- a/statistics/interact_with_storage.go +++ b/statistics/interact_with_storage.go @@ -18,7 +18,6 @@ import ( "context" "encoding/json" "fmt" - "go.uber.org/zap" "strconv" "time" @@ -34,6 +33,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/mathutil" "github.com/pingcap/tidb/util/sqlexec" + "go.uber.org/zap" ) // StatsReader is used for simplifying code that needs to read statistics from system tables(mysql.stats_xxx) in different sqls @@ -95,6 +95,7 @@ func (sr *StatsReader) Close() error { return err } +// HistogramFromStorage reads histogram from storage. func HistogramFromStorage(reader *StatsReader, tableID int64, colID int64, tp *types.FieldType, distinct int64, isIndex int, ver uint64, nullCount int64, totColSize int64, corr float64) (_ *Histogram, err error) { rows, fields, err := reader.Read("select count, repeats, lower_bound, upper_bound, ndv from mysql.stats_buckets where table_id = %? and is_index = %? and hist_id = %? order by bucket_id", tableID, isIndex, colID) if err != nil { @@ -143,6 +144,7 @@ func HistogramFromStorage(reader *StatsReader, tableID int64, colID int64, tp *t return hg, nil } +// CMSketchAndTopNFromStorage reads CMSketch and TopN from storage. func CMSketchAndTopNFromStorage(reader *StatsReader, tblID int64, isIndex, histID int64) (_ *CMSketch, _ *TopN, err error) { topNRows, _, err := reader.Read("select HIGH_PRIORITY value, count from mysql.stats_top_n where table_id = %? and is_index = %? and hist_id = %?", tblID, isIndex, histID) if err != nil { @@ -158,6 +160,7 @@ func CMSketchAndTopNFromStorage(reader *StatsReader, tblID int64, isIndex, histI return DecodeCMSketchAndTopN(rows[0].GetBytes(0), topNRows) } +// FMSketchFromStorage reads FMSketch from storage func FMSketchFromStorage(reader *StatsReader, tblID int64, isIndex, histID int64) (_ *FMSketch, err error) { rows, _, err := reader.Read("select value from mysql.stats_fm_sketch where table_id = %? and is_index = %? and hist_id = %?", tblID, isIndex, histID) if err != nil || len(rows) == 0 { @@ -166,6 +169,7 @@ func FMSketchFromStorage(reader *StatsReader, tblID int64, isIndex, histID int64 return DecodeFMSketch(rows[0].GetBytes(0)) } +// ColumnCountFromStorage reads column count from storage func ColumnCountFromStorage(reader *StatsReader, tableID, colID, statsVer int64) (int64, error) { count := int64(0) rows, _, err := reader.Read("select sum(count) from mysql.stats_buckets where table_id = %? and is_index = 0 and hist_id = %?", tableID, colID) @@ -199,15 +203,7 @@ func ColumnCountFromStorage(reader *StatsReader, tableID, colID, statsVer int64) return count, err } -const ( - // ExtendedStatsInited is the status for extended stats which are just registered but have not been analyzed yet. - ExtendedStatsInited uint8 = iota - // ExtendedStatsAnalyzed is the status for extended stats which have been collected in analyze. - ExtendedStatsAnalyzed - // ExtendedStatsDeleted is the status for extended stats which were dropped. These "deleted" records would be removed from storage by GCStats(). - ExtendedStatsDeleted -) - +// ExtendedStatsFromStorage reads extended stats from storage. func ExtendedStatsFromStorage(reader *StatsReader, table *Table, physicalID int64, loadAll bool) (*Table, error) { failpoint.Inject("injectExtStatsLoadErr", func() { failpoint.Return(nil, errors.New("gofail extendedStatsFromStorage error")) diff --git a/statistics/table.go b/statistics/table.go index f22699cfdb95b..6759020dec4f1 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -94,6 +94,15 @@ func NewExtendedStatsColl() *ExtendedStatsColl { return &ExtendedStatsColl{Stats: make(map[string]*ExtendedStatsItem)} } +const ( + // ExtendedStatsInited is the status for extended stats which are just registered but have not been analyzed yet. + ExtendedStatsInited uint8 = iota + // ExtendedStatsAnalyzed is the status for extended stats which have been collected in analyze. + ExtendedStatsAnalyzed + // ExtendedStatsDeleted is the status for extended stats which were dropped. These "deleted" records would be removed from storage by GCStats(). + ExtendedStatsDeleted +) + // HistColl is a collection of histogram. It collects enough information for plan to calculate the selectivity. type HistColl struct { PhysicalID int64 From 9a34556e0e28d72eb3e07949f61ffd39203a4abf Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Thu, 2 Feb 2023 16:26:59 +0800 Subject: [PATCH 3/4] fix panic --- statistics/handle/handle.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 5c77180f59a28..8cea080fd99d5 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -1404,8 +1404,8 @@ func (h *Handle) TableStatsFromStorage(tableInfo *model.TableInfo, physicalID in if err != nil { return nil, err } - if reader.IsHistory() { - return statsTbl, err + if reader.IsHistory() || statsTbl == nil { + return statsTbl, nil } for histID, idx := range statsTbl.Indices { if statistics.IsAnalyzed(idx.Flag) { From 047e03d7cb9acadb861bfb4194c8997cb6bd313b Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Thu, 2 Feb 2023 17:52:49 +0800 Subject: [PATCH 4/4] fix ut --- statistics/handle/handle_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index dc399a87fcad3..607c00931e493 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -553,10 +553,10 @@ func TestReloadExtStatsLockRelease(t *testing.T) { tk.MustExec("insert into t values(1,1),(2,2),(3,3)") tk.MustExec("alter table t add stats_extended s1 correlation(a,b)") tk.MustExec("analyze table t") - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/statistics/handle/injectExtStatsLoadErr", `return("")`)) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/statistics/injectExtStatsLoadErr", `return("")`)) err := tk.ExecToErr("admin reload stats_extended") require.Equal(t, "gofail extendedStatsFromStorage error", err.Error()) - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/statistics/handle/injectExtStatsLoadErr")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/statistics/injectExtStatsLoadErr")) // Check the lock is released by `admin reload stats_extended` if error happens. tk.MustExec("analyze table t") }