diff --git a/pkg/executor/infoschema_reader.go b/pkg/executor/infoschema_reader.go index ba7640a222..71d78eb8a0 100644 --- a/pkg/executor/infoschema_reader.go +++ b/pkg/executor/infoschema_reader.go @@ -682,12 +682,14 @@ func (e *memtableRetriever) setDataFromOneTable( var rowCount, avgRowLength, dataLength, indexLength uint64 if useStatsCache { - if table.GetPartitionInfo() == nil { - err := cache.TableRowStatsCache.UpdateByID(sctx, table.ID) - if err != nil { - return rows, err - } - } else { + // Even for partitioned tables, we must update the stats cache for the main table itself. + // This is necessary because the global index length from the table also needs to be included. + // For further details, see: https://github.com/pingcap/tidb/issues/54173 + err := cache.TableRowStatsCache.UpdateByID(sctx, table.ID) + if err != nil { + return rows, err + } + if table.GetPartitionInfo() != nil { // needs to update all partitions for partition table. for _, pi := range table.GetPartitionInfo().Definitions { err := cache.TableRowStatsCache.UpdateByID(sctx, pi.ID) diff --git a/pkg/statistics/handle/cache/stats_table_row_cache.go b/pkg/statistics/handle/cache/stats_table_row_cache.go index 2b537c49a3..505b5e50b1 100644 --- a/pkg/statistics/handle/cache/stats_table_row_cache.go +++ b/pkg/statistics/handle/cache/stats_table_row_cache.go @@ -45,22 +45,9 @@ type tableHistID struct { // StatsTableRowCache is used to cache the count of table rows. type StatsTableRowCache struct { - modifyTime time.Time - tableRows map[int64]uint64 - colLength map[tableHistID]uint64 - dirtyIDs []int64 - mu syncutil.RWMutex -} - -// Invalidate invalidates the cache of the table with id. -func (c *StatsTableRowCache) Invalidate(tblID int64) { - c.mu.Lock() - defer c.mu.Unlock() - // To prevent the cache from becoming too large, - // we only record the latest 100 dirty tables that have been modified. - if len(c.dirtyIDs) < 100 { - c.dirtyIDs = append(c.dirtyIDs, tblID) - } + tableRows map[int64]uint64 + colLength map[tableHistID]uint64 + mu syncutil.RWMutex } // GetTableRows gets the count of table rows. @@ -77,34 +64,8 @@ func (c *StatsTableRowCache) GetColLength(id tableHistID) uint64 { return c.colLength[id] } -func (c *StatsTableRowCache) updateDirtyIDs(sctx sessionctx.Context) error { - if len(c.dirtyIDs) > 0 { - tableRows, err := getRowCountTables(sctx, c.dirtyIDs...) - if err != nil { - return err - } - for id, tr := range tableRows { - c.tableRows[id] = tr - } - colLength, err := getColLengthTables(sctx, c.dirtyIDs...) - if err != nil { - return err - } - for id, cl := range colLength { - c.colLength[id] = cl - } - c.dirtyIDs = c.dirtyIDs[:0] - } - return nil -} - // UpdateByID tries to update the cache by table ID. func (c *StatsTableRowCache) UpdateByID(sctx sessionctx.Context, id int64) error { - c.mu.Lock() - defer c.mu.Unlock() - if time.Since(c.modifyTime) < tableStatsCacheExpiry { - return c.updateDirtyIDs(sctx) - } tableRows, err := getRowCountTables(sctx, id) if err != nil { return err @@ -113,11 +74,12 @@ func (c *StatsTableRowCache) UpdateByID(sctx sessionctx.Context, id int64) error if err != nil { return err } + c.mu.Lock() + defer c.mu.Unlock() c.tableRows[id] = tableRows[id] for k, v := range colLength { c.colLength[k] = v } - c.modifyTime = time.Now() return nil } diff --git a/pkg/statistics/handle/history/BUILD.bazel b/pkg/statistics/handle/history/BUILD.bazel index 95e4da6ed9..0e7955cfdd 100644 --- a/pkg/statistics/handle/history/BUILD.bazel +++ b/pkg/statistics/handle/history/BUILD.bazel @@ -8,7 +8,6 @@ go_library( deps = [ "//pkg/meta/model", "//pkg/sessionctx", - "//pkg/statistics/handle/cache", "//pkg/statistics/handle/logutil", "//pkg/statistics/handle/storage", "//pkg/statistics/handle/types", diff --git a/pkg/statistics/handle/history/history_stats.go b/pkg/statistics/handle/history/history_stats.go index a2fe2e0d71..5bc78f4201 100644 --- a/pkg/statistics/handle/history/history_stats.go +++ b/pkg/statistics/handle/history/history_stats.go @@ -21,7 +21,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/statistics/handle/cache" statslogutil "github.com/pingcap/tidb/pkg/statistics/handle/logutil" "github.com/pingcap/tidb/pkg/statistics/handle/storage" "github.com/pingcap/tidb/pkg/statistics/handle/types" @@ -172,7 +171,6 @@ func RecordHistoricalStatsMeta( ); err != nil { return errors.Trace(err) } - cache.TableRowStatsCache.Invalidate(tableID) return nil } diff --git a/pkg/statistics/handle/lockstats/BUILD.bazel b/pkg/statistics/handle/lockstats/BUILD.bazel index d0c52361a7..cec5ad9e61 100644 --- a/pkg/statistics/handle/lockstats/BUILD.bazel +++ b/pkg/statistics/handle/lockstats/BUILD.bazel @@ -11,7 +11,6 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/sessionctx", - "//pkg/statistics/handle/cache", "//pkg/statistics/handle/logutil", "//pkg/statistics/handle/types", "//pkg/statistics/handle/util", diff --git a/pkg/statistics/handle/lockstats/unlock_stats.go b/pkg/statistics/handle/lockstats/unlock_stats.go index f6d6baf7f8..8ccc848466 100644 --- a/pkg/statistics/handle/lockstats/unlock_stats.go +++ b/pkg/statistics/handle/lockstats/unlock_stats.go @@ -18,7 +18,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/statistics/handle/cache" statslogutil "github.com/pingcap/tidb/pkg/statistics/handle/logutil" "github.com/pingcap/tidb/pkg/statistics/handle/types" "github.com/pingcap/tidb/pkg/statistics/handle/util" @@ -169,7 +168,6 @@ func updateStatsAndUnlockTable(sctx sessionctx.Context, tid int64) error { if err := updateDelta(sctx, count, modifyCount, tid); err != nil { return err } - cache.TableRowStatsCache.Invalidate(tid) _, _, err = util.ExecRows( sctx, @@ -188,11 +186,9 @@ func updateStatsAndUnlockPartition(sctx sessionctx.Context, partitionID int64, t if err := updateDelta(sctx, count, modifyCount, partitionID); err != nil { return err } - cache.TableRowStatsCache.Invalidate(partitionID) if err := updateDelta(sctx, count, modifyCount, tid); err != nil { return err } - cache.TableRowStatsCache.Invalidate(tid) _, _, err = util.ExecRows( sctx, diff --git a/pkg/statistics/handle/storage/BUILD.bazel b/pkg/statistics/handle/storage/BUILD.bazel index 3beffab153..4937e76b7f 100644 --- a/pkg/statistics/handle/storage/BUILD.bazel +++ b/pkg/statistics/handle/storage/BUILD.bazel @@ -25,7 +25,6 @@ go_library( "//pkg/sessionctx/variable", "//pkg/statistics", "//pkg/statistics/asyncload", - "//pkg/statistics/handle/cache", "//pkg/statistics/handle/lockstats", "//pkg/statistics/handle/logutil", "//pkg/statistics/handle/metrics", diff --git a/pkg/statistics/handle/storage/gc.go b/pkg/statistics/handle/storage/gc.go index 4f17198a40..86823a2705 100644 --- a/pkg/statistics/handle/storage/gc.go +++ b/pkg/statistics/handle/storage/gc.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/statistics" - "github.com/pingcap/tidb/pkg/statistics/handle/cache" "github.com/pingcap/tidb/pkg/statistics/handle/lockstats" "github.com/pingcap/tidb/pkg/statistics/handle/types" "github.com/pingcap/tidb/pkg/statistics/handle/util" @@ -308,7 +307,6 @@ func gcTableStats(sctx sessionctx.Context, if err != nil { return errors.Trace(err) } - cache.TableRowStatsCache.Invalidate(physicalID) return nil } diff --git a/pkg/statistics/handle/storage/save.go b/pkg/statistics/handle/storage/save.go index 7ba613b201..f32b75502e 100644 --- a/pkg/statistics/handle/storage/save.go +++ b/pkg/statistics/handle/storage/save.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/statistics" - "github.com/pingcap/tidb/pkg/statistics/handle/cache" statslogutil "github.com/pingcap/tidb/pkg/statistics/handle/logutil" "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/types" @@ -236,7 +235,6 @@ func SaveTableStatsToStorage(sctx sessionctx.Context, } statsVer = version } - cache.TableRowStatsCache.Invalidate(tableID) // 2. Save histograms. for _, result := range results.Ars { for i, hg := range result.Hist { @@ -338,7 +336,6 @@ func SaveStatsToStorage( // If the count is less than 0, then we do not want to update the modify count and count. if count >= 0 { _, err = util.Exec(sctx, "replace into mysql.stats_meta (version, table_id, count, modify_count) values (%?, %?, %?, %?)", version, tableID, count, modifyCount) - cache.TableRowStatsCache.Invalidate(tableID) } else { _, err = util.Exec(sctx, "update mysql.stats_meta set version = %? where table_id = %?", version, tableID) } @@ -389,7 +386,6 @@ func SaveMetaToStorage( } _, err = util.Exec(sctx, "replace into mysql.stats_meta (version, table_id, count, modify_count) values (%?, %?, %?, %?)", version, tableID, count, modifyCount) statsVer = version - cache.TableRowStatsCache.Invalidate(tableID) return } diff --git a/pkg/statistics/handle/storage/update.go b/pkg/statistics/handle/storage/update.go index 0660a67123..ea7ae70991 100644 --- a/pkg/statistics/handle/storage/update.go +++ b/pkg/statistics/handle/storage/update.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" - "github.com/pingcap/tidb/pkg/statistics/handle/cache" "github.com/pingcap/tidb/pkg/statistics/handle/types" statsutil "github.com/pingcap/tidb/pkg/statistics/handle/util" ) @@ -159,11 +158,6 @@ func UpdateStatsMeta( } } - // Invalidate cache for all unlocked tables - for _, id := range cacheInvalidateIDs { - cache.TableRowStatsCache.Invalidate(id) - } - return nil }