statistics: Remove the ineffective dirty IDs from the row count cache (#56287)

close pingcap/tidb#55803
This commit is contained in:
Rustin
2025-03-02 16:37:51 +08:00
committed by GitHub
parent ec18512047
commit 2214bd07fc
10 changed files with 13 additions and 70 deletions

View File

@ -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)

View File

@ -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
}

View File

@ -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",

View File

@ -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
}

View File

@ -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",

View File

@ -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,

View File

@ -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",

View File

@ -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
}

View File

@ -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
}

View File

@ -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
}