From 2c4e8dc9bb9270daff956e23420745938f26e4e0 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 13 Sep 2023 19:15:09 +0800 Subject: [PATCH] *: unite stats cache's GET (#46939) ref pingcap/tidb#46905 --- executor/analyze.go | 8 +--- planner/core/BUILD.bazel | 1 - planner/core/logical_plan_builder.go | 9 ++-- statistics/handle/bootstrap.go | 10 ++-- statistics/handle/cache/bench_test.go | 4 +- statistics/handle/cache/internal/inner.go | 2 +- .../handle/cache/internal/lfu/lfu_cache.go | 2 +- .../cache/internal/lfu/lfu_cache_test.go | 14 +++--- .../cache/internal/mapcache/map_cache.go | 2 +- statistics/handle/cache/statscache.go | 6 +-- statistics/handle/cache/statscacheinner.go | 47 ++----------------- statistics/handle/handle.go | 44 ++++++----------- statistics/handle/handle_hist.go | 9 ++-- statistics/handle/handletest/BUILD.bazel | 1 - statistics/handle/handletest/handle_test.go | 3 +- statistics/handle/historical_stats_handler.go | 2 +- statistics/handle/updatetest/BUILD.bazel | 1 - statistics/handle/updatetest/update_test.go | 3 +- 18 files changed, 52 insertions(+), 116 deletions(-) diff --git a/executor/analyze.go b/executor/analyze.go index 85f15a5df471d..a3ac2bc8f7a8c 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -39,7 +39,6 @@ import ( "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" - "github.com/pingcap/tidb/statistics/handle/cache" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" @@ -164,12 +163,7 @@ func (e *AnalyzeExec) Next(ctx context.Context, _ *chunk.Chunk) error { if err != nil { sessionVars.StmtCtx.AppendWarning(err) } - - if sessionVars.InRestrictedSQL { - return statsHandle.Update(infoSchema) - } - - return statsHandle.Update(infoSchema, cache.WithTableStatsByQuery()) + return statsHandle.Update(infoSchema) } // filterAndCollectTasks filters the tasks that are not locked and collects the table IDs. diff --git a/planner/core/BUILD.bazel b/planner/core/BUILD.bazel index 1dfd3bd328de8..19a6d7e27abd4 100644 --- a/planner/core/BUILD.bazel +++ b/planner/core/BUILD.bazel @@ -123,7 +123,6 @@ go_library( "//sessiontxn", "//sessiontxn/staleread", "//statistics", - "//statistics/handle/cache", "//table", "//table/tables", "//table/temptable", diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 102c3e84a31dc..8d11e3f5b2c03 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -50,7 +50,6 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" - "github.com/pingcap/tidb/statistics/handle/cache" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/table/temptable" @@ -4723,10 +4722,10 @@ func getStatsTable(ctx sessionctx.Context, tblInfo *model.TableInfo, pid int64) } if pid == tblInfo.ID || ctx.GetSessionVars().StmtCtx.UseDynamicPartitionPrune() { - statsTbl = statsHandle.GetTableStats(tblInfo, cache.WithTableStatsByQuery()) + statsTbl = statsHandle.GetTableStats(tblInfo) } else { usePartitionStats = true - statsTbl = statsHandle.GetPartitionStats(tblInfo, pid, cache.WithTableStatsByQuery()) + statsTbl = statsHandle.GetPartitionStats(tblInfo, pid) } allowPseudoTblTriggerLoading := false @@ -4791,9 +4790,9 @@ func getLatestVersionFromStatsTable(ctx sessionctx.Context, tblInfo *model.Table var statsTbl *statistics.Table if pid == tblInfo.ID || ctx.GetSessionVars().StmtCtx.UseDynamicPartitionPrune() { - statsTbl = statsHandle.GetTableStats(tblInfo, cache.WithTableStatsByQuery()) + statsTbl = statsHandle.GetTableStats(tblInfo) } else { - statsTbl = statsHandle.GetPartitionStats(tblInfo, pid, cache.WithTableStatsByQuery()) + statsTbl = statsHandle.GetPartitionStats(tblInfo, pid) } // 2. Table row count from statistics is zero. Pseudo stats table. diff --git a/statistics/handle/bootstrap.go b/statistics/handle/bootstrap.go index 400cf9ca68708..952a38f7b47d0 100644 --- a/statistics/handle/bootstrap.go +++ b/statistics/handle/bootstrap.go @@ -93,7 +93,7 @@ func (h *Handle) initStatsMeta(is infoschema.InfoSchema) (*cache.StatsCache, err func (h *Handle) initStatsHistograms4ChunkLite(is infoschema.InfoSchema, cache *cache.StatsCache, iter *chunk.Iterator4Chunk) { for row := iter.Begin(); row != iter.End(); row = iter.Next() { tblID := row.GetInt64(0) - table, ok := cache.GetFromInternal(tblID) + table, ok := cache.Get(tblID) if !ok { continue } @@ -164,7 +164,7 @@ func (h *Handle) initStatsHistograms4ChunkLite(is infoschema.InfoSchema, cache * func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, cache *cache.StatsCache, iter *chunk.Iterator4Chunk) { for row := iter.Begin(); row != iter.End(); row = iter.Next() { tblID, statsVer := row.GetInt64(0), row.GetInt64(8) - table, ok := cache.GetFromInternal(tblID) + table, ok := cache.Get(tblID) table = table.Copy() if !ok { continue @@ -280,7 +280,7 @@ func (h *Handle) initStatsHistograms(is infoschema.InfoSchema, cache *cache.Stat func (*Handle) initStatsTopN4Chunk(cache *cache.StatsCache, iter *chunk.Iterator4Chunk) { affectedIndexes := make(map[*statistics.Index]struct{}) for row := iter.Begin(); row != iter.End(); row = iter.Next() { - table, ok := cache.GetFromInternal(row.GetInt64(0)) + table, ok := cache.Get(row.GetInt64(0)) if !ok { continue } @@ -328,7 +328,7 @@ func (h *Handle) initStatsTopN(cache *cache.StatsCache) error { func (*Handle) initStatsFMSketch4Chunk(cache *cache.StatsCache, iter *chunk.Iterator4Chunk) { for row := iter.Begin(); row != iter.End(); row = iter.Next() { - table, ok := cache.GetFromInternal(row.GetInt64(0)) + table, ok := cache.Get(row.GetInt64(0)) if !ok { continue } @@ -379,7 +379,7 @@ func (h *Handle) initStatsFMSketch(cache *cache.StatsCache) error { func (*Handle) initStatsBuckets4Chunk(cache *cache.StatsCache, iter *chunk.Iterator4Chunk) { for row := iter.Begin(); row != iter.End(); row = iter.Next() { tableID, isIndex, histID := row.GetInt64(0), row.GetInt64(1), row.GetInt64(2) - table, ok := cache.GetFromInternal(tableID) + table, ok := cache.Get(tableID) if !ok { continue } diff --git a/statistics/handle/cache/bench_test.go b/statistics/handle/cache/bench_test.go index d178ba47e801b..ce0dac1af5b87 100644 --- a/statistics/handle/cache/bench_test.go +++ b/statistics/handle/cache/bench_test.go @@ -57,7 +57,7 @@ func benchPutGet(b *testing.B, c *StatsCachePointer) { wg.Add(1) go func(i int) { defer wg.Done() - c.Load().GetFromUser(int64(i)) + c.Load().Get(int64(i)) }(i) } wg.Wait() @@ -82,7 +82,7 @@ func benchGet(b *testing.B, c *StatsCachePointer) { wg.Add(1) go func(i int) { defer wg.Done() - c.Load().GetFromUser(int64(i)) + c.Load().Get(int64(i)) }(i) } wg.Wait() diff --git a/statistics/handle/cache/internal/inner.go b/statistics/handle/cache/internal/inner.go index eb934d5be4f40..c493aed6384c3 100644 --- a/statistics/handle/cache/internal/inner.go +++ b/statistics/handle/cache/internal/inner.go @@ -24,7 +24,7 @@ import ( // 2. remove remove the Version method. type StatsCacheInner interface { // Get gets the cache. - Get(tid int64, moveFront bool) (*statistics.Table, bool) + Get(tid int64) (*statistics.Table, bool) // Put puts a cache. Put(tid int64, tbl *statistics.Table) bool // Del deletes a cache. diff --git a/statistics/handle/cache/internal/lfu/lfu_cache.go b/statistics/handle/cache/internal/lfu/lfu_cache.go index e4c9426124232..804cfb8091715 100644 --- a/statistics/handle/cache/internal/lfu/lfu_cache.go +++ b/statistics/handle/cache/internal/lfu/lfu_cache.go @@ -87,7 +87,7 @@ func NewLFU(totalMemCost int64) (*LFU, error) { } // Get implements statsCacheInner -func (s *LFU) Get(tid int64, _ bool) (*statistics.Table, bool) { +func (s *LFU) Get(tid int64) (*statistics.Table, bool) { result, ok := s.cache.Get(tid) if !ok { return s.resultKeySet.Get(tid) diff --git a/statistics/handle/cache/internal/lfu/lfu_cache_test.go b/statistics/handle/cache/internal/lfu/lfu_cache_test.go index 102418555c347..9a0a7e46228dc 100644 --- a/statistics/handle/cache/internal/lfu/lfu_cache_test.go +++ b/statistics/handle/cache/internal/lfu/lfu_cache_test.go @@ -41,7 +41,7 @@ func TestLFUPutGetDel(t *testing.T) { lfu.Put(mockTableID, mockTable) lfu.wait() lfu.Del(mockTableID) - v, ok := lfu.Get(mockTableID, false) + v, ok := lfu.Get(mockTableID) require.False(t, ok) require.Nil(t, v) lfu.wait() @@ -91,7 +91,7 @@ func TestLFUPutTooBig(t *testing.T) { mockTable := testutil.NewMockStatisticsTable(1, 1, true, false, false) // put mockTable, the index should be evicted but the table still exists in the list. lfu.Put(int64(1), mockTable) - _, ok := lfu.Get(int64(1), false) + _, ok := lfu.Get(int64(1)) require.True(t, ok) lfu.wait() require.Equal(t, uint64(lfu.Cost()), lfu.metrics().CostAdded()-lfu.metrics().CostEvicted()) @@ -136,7 +136,7 @@ func TestLFUCachePutGetWithManyConcurrency(t *testing.T) { }(i) go func(i int) { defer wg.Done() - lfu.Get(int64(i), true) + lfu.Get(int64(i)) }(i) } wg.Wait() @@ -167,7 +167,7 @@ func TestLFUCachePutGetWithManyConcurrency2(t *testing.T) { go func() { defer wg.Done() for n := 0; n < 1000; n++ { - lfu.Get(int64(n), true) + lfu.Get(int64(n)) } }() } @@ -203,7 +203,7 @@ func TestLFUCachePutGetWithManyConcurrencyAndSmallConcurrency(t *testing.T) { defer wg.Done() for c := 0; c < 1000; c++ { for n := 0; n < 50; n++ { - tbl, ok := lfu.Get(int64(n), true) + tbl, ok := lfu.Get(int64(n)) require.True(t, ok) checkTable(t, tbl) } @@ -212,7 +212,7 @@ func TestLFUCachePutGetWithManyConcurrencyAndSmallConcurrency(t *testing.T) { } wg.Wait() lfu.wait() - v, ok := lfu.Get(rand.Int63n(50), false) + v, ok := lfu.Get(rand.Int63n(50)) require.True(t, ok) for _, c := range v.Columns { require.Equal(t, c.GetEvictedStatus(), statistics.AllEvicted) @@ -262,7 +262,7 @@ func TestLFUReject(t *testing.T) { time.Sleep(3 * time.Second) require.Equal(t, int64(0), lfu.Cost()) require.Len(t, lfu.Values(), 2) - v, ok := lfu.Get(2, false) + v, ok := lfu.Get(2) require.True(t, ok) for _, c := range v.Columns { require.Equal(t, statistics.AllEvicted, c.GetEvictedStatus()) diff --git a/statistics/handle/cache/internal/mapcache/map_cache.go b/statistics/handle/cache/internal/mapcache/map_cache.go index 8e9f9a2532120..7a7285483610e 100644 --- a/statistics/handle/cache/internal/mapcache/map_cache.go +++ b/statistics/handle/cache/internal/mapcache/map_cache.go @@ -48,7 +48,7 @@ func NewMapCache() *MapCache { } // Get implements StatsCacheInner -func (m *MapCache) Get(k int64, _ bool) (*statistics.Table, bool) { +func (m *MapCache) Get(k int64) (*statistics.Table, bool) { v, ok := m.tables[k] return v.value, ok } diff --git a/statistics/handle/cache/statscache.go b/statistics/handle/cache/statscache.go index d82c24d658aef..c8dba4015b730 100644 --- a/statistics/handle/cache/statscache.go +++ b/statistics/handle/cache/statscache.go @@ -53,10 +53,10 @@ func (s *StatsCachePointer) Replace(newCache *StatsCache) { } // UpdateStatsCache updates the cache with the new cache. -func (s *StatsCachePointer) UpdateStatsCache(newCache *StatsCache, tables []*statistics.Table, deletedIDs []int64, opts ...TableStatsOpt) { +func (s *StatsCachePointer) UpdateStatsCache(newCache *StatsCache, tables []*statistics.Table, deletedIDs []int64) { if enableQuota := config.GetGlobalConfig().Performance.EnableStatsCacheMemQuota; enableQuota { - s.Load().Update(tables, deletedIDs, opts...) + s.Load().Update(tables, deletedIDs) } else { - s.Replace(newCache.CopyAndUpdate(tables, deletedIDs, opts...)) + s.Replace(newCache.CopyAndUpdate(tables, deletedIDs)) } } diff --git a/statistics/handle/cache/statscacheinner.go b/statistics/handle/cache/statscacheinner.go index 09fbde7436ad2..d1ca09475cc69 100644 --- a/statistics/handle/cache/statscacheinner.go +++ b/statistics/handle/cache/statscacheinner.go @@ -39,26 +39,6 @@ import ( "go.uber.org/zap" ) -// TableStatsOption used to indicate the way to get table stats -type TableStatsOption struct { - byQuery bool -} - -// ByQuery indicates whether the stats is got by query -func (t *TableStatsOption) ByQuery() bool { - return t.byQuery -} - -// TableStatsOpt used to edit getTableStatsOption -type TableStatsOpt func(*TableStatsOption) - -// WithTableStatsByQuery indicates user needed -func WithTableStatsByQuery() TableStatsOpt { - return func(option *TableStatsOption) { - option.byQuery = true - } -} - // NewStatsCache creates a new StatsCacheWrapper. func NewStatsCache() (*StatsCache, error) { enableQuota := config.GetGlobalConfig().Performance.EnableStatsCacheMemQuota @@ -89,16 +69,12 @@ func (sc *StatsCache) Len() int { return sc.c.Len() } -// GetFromUser returns the statistics of the specified Table ID. +// Get returns the statistics of the specified Table ID. // The returned value should be read-only, if you update it, don't forget to use Put to put it back again, otherwise the memory trace can be inaccurate. // // e.g. v := sc.Get(id); /* update the value */ v.Version = 123; sc.Put(id, v); -func (sc *StatsCache) GetFromUser(id int64) (*statistics.Table, bool) { - return sc.getCache(id, true) -} - -func (sc *StatsCache) getCache(id int64, moveFront bool) (*statistics.Table, bool) { - result, ok := sc.c.Get(id, moveFront) +func (sc *StatsCache) Get(id int64) (*statistics.Table, bool) { + result, ok := sc.c.Get(id) if ok { metrics.HitCounter.Add(1) } else { @@ -107,11 +83,6 @@ func (sc *StatsCache) getCache(id int64, moveFront bool) (*statistics.Table, boo return result, ok } -// GetFromInternal returns the statistics of the specified Table ID. -func (sc *StatsCache) GetFromInternal(id int64) (*statistics.Table, bool) { - return sc.getCache(id, false) -} - // Put puts the table statistics to the cache from query. func (sc *StatsCache) Put(id int64, t *statistics.Table) { sc.put(id, t) @@ -171,11 +142,7 @@ func (sc *StatsCache) Version() uint64 { } // CopyAndUpdate copies a new cache and updates the new statistics table cache. It is only used in the COW mode. -func (sc *StatsCache) CopyAndUpdate(tables []*statistics.Table, deletedIDs []int64, opts ...TableStatsOpt) *StatsCache { - option := &TableStatsOption{} - for _, opt := range opts { - opt(option) - } +func (sc *StatsCache) CopyAndUpdate(tables []*statistics.Table, deletedIDs []int64) *StatsCache { newCache := &StatsCache{c: sc.c.Copy()} newCache.maxTblStatsVer.Store(sc.maxTblStatsVer.Load()) for _, tbl := range tables { @@ -196,11 +163,7 @@ func (sc *StatsCache) CopyAndUpdate(tables []*statistics.Table, deletedIDs []int } // Update updates the new statistics table cache. -func (sc *StatsCache) Update(tables []*statistics.Table, deletedIDs []int64, opts ...TableStatsOpt) { - option := &TableStatsOption{} - for _, opt := range opts { - opt(option) - } +func (sc *StatsCache) Update(tables []*statistics.Table, deletedIDs []int64) { for _, tbl := range tables { id := tbl.PhysicalID metrics.UpdateCounter.Inc() diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index f1c434eaff3fa..3096d6e9a28de 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -278,7 +278,7 @@ func (h *Handle) UpdateStatsHealthyMetrics() { } // Update reads stats meta from store and updates the stats map. -func (h *Handle) Update(is infoschema.InfoSchema, opts ...cache.TableStatsOpt) error { +func (h *Handle) Update(is infoschema.InfoSchema) error { oldCache := h.statsCache.Load() lastVersion := oldCache.Version() // We need this because for two tables, the smaller version may write later than the one with larger version. @@ -297,10 +297,6 @@ func (h *Handle) Update(is infoschema.InfoSchema, opts ...cache.TableStatsOpt) e if err != nil { return errors.Trace(err) } - option := &cache.TableStatsOption{} - for _, opt := range opts { - opt(option) - } tables := make([]*statistics.Table, 0, len(rows)) deletedTableIDs := make([]int64, 0, len(rows)) for _, row := range rows { @@ -315,7 +311,7 @@ func (h *Handle) Update(is infoschema.InfoSchema, opts ...cache.TableStatsOpt) e continue } tableInfo := table.Meta() - if oldTbl, ok := oldCache.GetFromInternal(physicalID); ok && oldTbl.Version >= version && tableInfo.UpdateTS == oldTbl.TblInfoUpdateTS { + if oldTbl, ok := oldCache.Get(physicalID); ok && oldTbl.Version >= version && tableInfo.UpdateTS == oldTbl.TblInfoUpdateTS { continue } tbl, err := h.TableStatsFromStorage(tableInfo, physicalID, false, 0) @@ -335,7 +331,7 @@ func (h *Handle) Update(is infoschema.InfoSchema, opts ...cache.TableStatsOpt) e tbl.TblInfoUpdateTS = tableInfo.UpdateTS tables = append(tables, tbl) } - h.updateStatsCache(oldCache, tables, deletedTableIDs, opts...) + h.updateStatsCache(oldCache, tables, deletedTableIDs) return nil } @@ -724,12 +720,12 @@ func (h *Handle) GetMemConsumed() (size int64) { } // GetTableStats retrieves the statistics table from cache, and the cache will be updated by a goroutine. -func (h *Handle) GetTableStats(tblInfo *model.TableInfo, opts ...cache.TableStatsOpt) *statistics.Table { - return h.GetPartitionStats(tblInfo, tblInfo.ID, opts...) +func (h *Handle) GetTableStats(tblInfo *model.TableInfo) *statistics.Table { + return h.GetPartitionStats(tblInfo, tblInfo.ID) } // GetPartitionStats retrieves the partition stats from cache. -func (h *Handle) GetPartitionStats(tblInfo *model.TableInfo, pid int64, opts ...cache.TableStatsOpt) *statistics.Table { +func (h *Handle) GetPartitionStats(tblInfo *model.TableInfo, pid int64) *statistics.Table { var tbl *statistics.Table if h == nil { tbl = statistics.PseudoTable(tblInfo, false) @@ -737,16 +733,7 @@ func (h *Handle) GetPartitionStats(tblInfo *model.TableInfo, pid int64, opts ... return tbl } statsCache := h.statsCache.Load() - var ok bool - option := &cache.TableStatsOption{} - for _, opt := range opts { - opt(option) - } - if option.ByQuery() { - tbl, ok = statsCache.GetFromUser(pid) - } else { - tbl, ok = statsCache.GetFromInternal(pid) - } + tbl, ok := statsCache.Get(pid) if !ok { tbl = statistics.PseudoTable(tblInfo, false) tbl.PhysicalID = pid @@ -770,9 +757,8 @@ func (h *Handle) initStatsCache(newCache *cache.StatsCache) { // If it is in the COW mode. it overrides the global statsCache with a new one, it may fail // if the global statsCache has been modified by others already. // Callers should add retry loop if necessary. -func (h *Handle) updateStatsCache(newCache *cache.StatsCache, tables []*statistics.Table, deletedIDs []int64, - opts ...cache.TableStatsOpt) (updated bool) { - h.statsCache.UpdateStatsCache(newCache, tables, deletedIDs, opts...) +func (h *Handle) updateStatsCache(newCache *cache.StatsCache, tables []*statistics.Table, deletedIDs []int64) (updated bool) { + h.statsCache.UpdateStatsCache(newCache, tables, deletedIDs) return true } @@ -807,7 +793,7 @@ func (h *Handle) LoadNeededHistograms() (err error) { func (h *Handle) loadNeededColumnHistograms(reader *statistics.StatsReader, col model.TableItemID, loadFMSketch bool) (err error) { oldCache := h.statsCache.Load() - tbl, ok := oldCache.GetFromInternal(col.TableID) + tbl, ok := oldCache.Get(col.TableID) if !ok { return nil } @@ -856,7 +842,7 @@ func (h *Handle) loadNeededColumnHistograms(reader *statistics.StatsReader, col // Reload the latest stats cache, otherwise the `updateStatsCache` may fail with high probability, because functions // like `GetPartitionStats` called in `fmSketchFromStorage` would have modified the stats cache already. oldCache = h.statsCache.Load() - tbl, ok = oldCache.GetFromInternal(col.TableID) + tbl, ok = oldCache.Get(col.TableID) if !ok { return nil } @@ -870,7 +856,7 @@ func (h *Handle) loadNeededColumnHistograms(reader *statistics.StatsReader, col func (h *Handle) loadNeededIndexHistograms(reader *statistics.StatsReader, idx model.TableItemID, loadFMSketch bool) (err error) { oldCache := h.statsCache.Load() - tbl, ok := oldCache.GetFromInternal(idx.TableID) + tbl, ok := oldCache.Get(idx.TableID) if !ok { return nil } @@ -909,7 +895,7 @@ func (h *Handle) loadNeededIndexHistograms(reader *statistics.StatsReader, idx m index.LastAnalyzePos.Copy(&idxHist.LastAnalyzePos) oldCache = h.statsCache.Load() - tbl, ok = oldCache.GetFromInternal(idx.TableID) + tbl, ok = oldCache.Get(idx.TableID) if !ok { return nil } @@ -951,7 +937,7 @@ func (h *Handle) TableStatsFromStorage(tableInfo *model.TableInfo, physicalID in err = err1 } }() - statsTbl, ok := h.statsCache.Load().GetFromInternal(physicalID) + statsTbl, ok := h.statsCache.Load().Get(physicalID) if !ok { statsTbl = nil } @@ -1539,7 +1525,7 @@ const updateStatsCacheRetryCnt = 5 func (h *Handle) removeExtendedStatsItem(tableID int64, statsName string) { for retry := updateStatsCacheRetryCnt; retry > 0; retry-- { oldCache := h.statsCache.Load() - tbl, ok := oldCache.GetFromInternal(tableID) + tbl, ok := oldCache.Get(tableID) if !ok || tbl.ExtendedStats == nil || len(tbl.ExtendedStats.Stats) == 0 { return } diff --git a/statistics/handle/handle_hist.go b/statistics/handle/handle_hist.go index c901530990ca0..4e0da9f21a6c7 100644 --- a/statistics/handle/handle_hist.go +++ b/statistics/handle/handle_hist.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" - "github.com/pingcap/tidb/statistics/handle/cache" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" @@ -145,7 +144,7 @@ func (h *Handle) removeHistLoadedColumns(neededItems []model.TableItemID) []mode statsCache := h.statsCache.Load() remainedItems := make([]model.TableItemID, 0, len(neededItems)) for _, item := range neededItems { - tbl, ok := statsCache.GetFromInternal(item.TableID) + tbl, ok := statsCache.Get(item.TableID) if !ok { continue } @@ -238,7 +237,7 @@ func (h *Handle) handleOneItemTask(task *NeededItemTask, readerCtx *StatsReaderC result := stmtctx.StatsLoadResult{Item: task.TableItemID} item := result.Item oldCache := h.statsCache.Load() - tbl, ok := oldCache.GetFromInternal(item.TableID) + tbl, ok := oldCache.Get(item.TableID) if !ok { h.writeToResultChan(task.ResultCh, result) return nil, nil @@ -480,7 +479,7 @@ func (h *Handle) updateCachedItem(item model.TableItemID, colHist *statistics.Co // Reload the latest stats cache, otherwise the `updateStatsCache` may fail with high probability, because functions // like `GetPartitionStats` called in `fmSketchFromStorage` would have modified the stats cache already. oldCache := h.statsCache.Load() - tbl, ok := oldCache.GetFromInternal(item.TableID) + tbl, ok := oldCache.Get(item.TableID) if !ok { return true } @@ -499,7 +498,7 @@ func (h *Handle) updateCachedItem(item model.TableItemID, colHist *statistics.Co tbl = tbl.Copy() tbl.Indices[item.ID] = idxHist } - return h.updateStatsCache(oldCache, []*statistics.Table{tbl}, nil, cache.WithTableStatsByQuery()) + return h.updateStatsCache(oldCache, []*statistics.Table{tbl}, nil) } func (h *Handle) setWorking(item model.TableItemID, resultCh chan stmtctx.StatsLoadResult) bool { diff --git a/statistics/handle/handletest/BUILD.bazel b/statistics/handle/handletest/BUILD.bazel index abe9e20cfe030..9074f6b07f931 100644 --- a/statistics/handle/handletest/BUILD.bazel +++ b/statistics/handle/handletest/BUILD.bazel @@ -19,7 +19,6 @@ go_test( "//sessionctx/variable", "//statistics", "//statistics/handle", - "//statistics/handle/cache", "//statistics/handle/internal", "//testkit", "//testkit/testsetup", diff --git a/statistics/handle/handletest/handle_test.go b/statistics/handle/handletest/handle_test.go index 9a36c1bbfe1cc..e766cddfdfb59 100644 --- a/statistics/handle/handletest/handle_test.go +++ b/statistics/handle/handletest/handle_test.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" - "github.com/pingcap/tidb/statistics/handle/cache" "github.com/pingcap/tidb/statistics/handle/internal" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" @@ -187,7 +186,7 @@ func TestVersion(t *testing.T) { // We can read it without analyze again! Thanks for PrevLastVersion. require.NotNil(t, statsTbl2.Columns[int64(3)]) // assert WithGetTableStatsByQuery get the same result - statsTbl2 = h.GetTableStats(tableInfo2, cache.WithTableStatsByQuery()) + statsTbl2 = h.GetTableStats(tableInfo2) require.False(t, statsTbl2.Pseudo) require.NotNil(t, statsTbl2.Columns[int64(3)]) } diff --git a/statistics/handle/historical_stats_handler.go b/statistics/handle/historical_stats_handler.go index 7829e048b80d9..4fe0042f955e1 100644 --- a/statistics/handle/historical_stats_handler.go +++ b/statistics/handle/historical_stats_handler.go @@ -84,7 +84,7 @@ func (h *Handle) recordHistoricalStatsMeta(tableID int64, version uint64, source return } sc := v - tbl, ok := sc.GetFromInternal(tableID) + tbl, ok := sc.Get(tableID) if !ok { return } diff --git a/statistics/handle/updatetest/BUILD.bazel b/statistics/handle/updatetest/BUILD.bazel index ec974a7203738..78bcb956ae574 100644 --- a/statistics/handle/updatetest/BUILD.bazel +++ b/statistics/handle/updatetest/BUILD.bazel @@ -16,7 +16,6 @@ go_test( "//sessionctx/variable", "//statistics", "//statistics/handle", - "//statistics/handle/cache", "//testkit", "//testkit/testsetup", "//types", diff --git a/statistics/handle/updatetest/update_test.go b/statistics/handle/updatetest/update_test.go index f487f89aaef59..072a48af28cd9 100644 --- a/statistics/handle/updatetest/update_test.go +++ b/statistics/handle/updatetest/update_test.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" - "github.com/pingcap/tidb/statistics/handle/cache" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/collate" @@ -346,7 +345,7 @@ func TestUpdatePartition(t *testing.T) { } // assert WithGetTableStatsByQuery get the same result for _, def := range pi.Definitions { - statsTbl := h.GetPartitionStats(tableInfo, def.ID, cache.WithTableStatsByQuery()) + statsTbl := h.GetPartitionStats(tableInfo, def.ID) require.Equal(t, int64(3), statsTbl.ModifyCount) require.Equal(t, int64(0), statsTbl.RealtimeCount) require.Equal(t, int64(0), statsTbl.Columns[bColID].TotColSize)