statistics: Remove the ineffective dirty IDs from the row count cache (#56287)
close pingcap/tidb#55803
This commit is contained in:
@ -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)
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
|
||||
@ -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",
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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",
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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",
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user