Skip to content

Commit

Permalink
stats, infoschema: avoid some network cost
Browse files Browse the repository at this point in the history
  • Loading branch information
winoros committed Jan 21, 2025
1 parent 32d11a8 commit f5285d8
Show file tree
Hide file tree
Showing 7 changed files with 107 additions and 7 deletions.
22 changes: 22 additions & 0 deletions pkg/infoschema/infoschema.go
Original file line number Diff line number Diff line change
Expand Up @@ -315,6 +315,19 @@ func (is *infoSchema) TableByID(_ stdctx.Context, id int64) (val table.Table, ok
return slice[idx], true
}

// SchemaNameAndTableNameByID implements InfoSchema.SchemaNameAndTableNameByID.
func (is *infoSchema) SchemaNameAndTableNameByID(tableID int64) (schemaName, tableName ast.CIStr, ok bool) {
tbl, ok := is.TableByID(stdctx.Background(), tableID)
if !ok {
return
}
db, ok := is.SchemaByID(tbl.Meta().DBID)
if !ok {
return
}
return db.Name, tbl.Meta().Name, true
}

func (is *infoSchema) SchemaNameByTableID(tableID int64) (schemaName ast.CIStr, ok bool) {
tbl, ok := is.TableByID(stdctx.Background(), tableID)
if !ok {
Expand Down Expand Up @@ -412,6 +425,15 @@ func (is *infoSchema) AllSchemaNames() (schemas []ast.CIStr) {
return rs
}

// SchemaNameAndTableNameByPartitionID implements InfoSchema.SchemaNameAndTableNameByPartitionID.
func (is *infoSchema) SchemaNameAndTableNameByPartitionID(partitionID int64) (schemaName, tableName ast.CIStr, ok bool) {
tbl, db, _ := is.FindTableByPartitionID(partitionID)
if tbl == nil {
return
}
return db.Name, tbl.Meta().Name, true
}

// FindTableByPartitionID finds the partition-table info by the partitionID.
// FindTableByPartitionID will traverse all the tables to find the partitionID partition in which partition-table.
func (is *infoSchema) FindTableByPartitionID(partitionID int64) (table.Table, *model.DBInfo, *model.PartitionDefinition) {
Expand Down
34 changes: 34 additions & 0 deletions pkg/infoschema/infoschema_v2.go
Original file line number Diff line number Diff line change
Expand Up @@ -734,6 +734,15 @@ func (is *infoschemaV2) TableByID(ctx context.Context, id int64) (val table.Tabl
return ret, true
}

// SchemaNameAndTableNameByID implements the InfoSchema interface.
func (is *infoschemaV2) SchemaNameAndTableNameByID(tableID int64) (schemaName, tableName ast.CIStr, ok bool) {
itm, ok := is.searchTableItemByID(tableID)
if !ok {
return
}
return itm.dbName, itm.tableName, true
}

func (is *infoschemaV2) SchemaNameByTableID(tableID int64) (schemaName ast.CIStr, ok bool) {
if !tableIDIsValid(tableID) {
return
Expand Down Expand Up @@ -1106,6 +1115,31 @@ func (is *infoschemaV2) SchemaExists(schema ast.CIStr) bool {
return ok
}

// SchemaNameAndTableNameByPartitionID implements InfoSchema.SchemaNameAndTableNameByPartitionID.
func (is *infoschemaV2) SchemaNameAndTableNameByPartitionID(partitionID int64) (schemaName, tableName ast.CIStr, ok bool) {
var pi partitionItem
is.pid2tid.Load().DescendLessOrEqual(partitionItem{partitionID: partitionID, schemaVersion: math.MaxInt64},
func(item partitionItem) bool {
if item.partitionID != partitionID {
return false
}
if item.schemaVersion > is.infoSchema.schemaMetaVersion {
// Skip the record.
return true
}
if item.schemaVersion <= is.infoSchema.schemaMetaVersion {
ok = !item.tomb
pi = item
return false
}
return true
})
if !ok {
return
}
return is.SchemaNameAndTableNameByID(pi.tableID)
}

func (is *infoschemaV2) FindTableByPartitionID(partitionID int64) (table.Table, *model.DBInfo, *model.PartitionDefinition) {
var ok bool
var pi partitionItem
Expand Down
2 changes: 2 additions & 0 deletions pkg/infoschema/interface.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,9 @@ type InfoSchema interface {
context.MetaOnlyInfoSchema
TableByName(ctx stdctx.Context, schema, table ast.CIStr) (table.Table, error)
TableByID(ctx stdctx.Context, id int64) (table.Table, bool)
SchemaNameAndTableNameByID(tableID int64) (schemaName, tableName ast.CIStr, ok bool)
SchemaNameByTableID(tableID int64) (ast.CIStr, bool)
FindTableByPartitionID(partitionID int64) (table.Table, *model.DBInfo, *model.PartitionDefinition)
SchemaNameAndTableNameByPartitionID(partitionID int64) (schemaName, tableName ast.CIStr, ok bool)
base() *infoSchema
}
33 changes: 33 additions & 0 deletions pkg/statistics/handle/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -204,6 +204,39 @@ func (h *Handle) getPartitionStats(tblInfo *model.TableInfo, pid int64, returnPs
return tbl
}

func (h *Handle) GetPartitionStatsByID(is infoschema.InfoSchema, pid int64) *statistics.Table {
return h.getPartitionStatsByID(is, pid, true)
}

func (h *Handle) getPartitionStatsByID(is infoschema.InfoSchema, pid int64, returnPseudo bool) *statistics.Table {
var statsTbl *statistics.Table
if h == nil {
tbl, ok := h.TableInfoByID(is, pid)
if !ok {
return nil
}
statsTbl = statistics.PseudoTable(tbl.Meta(), false, false)
}
tbl, ok := h.Get(pid)
if !ok {
if returnPseudo {
tbl, ok := h.TableInfoByID(is, pid)
if !ok {
return nil
}
statsTbl = statistics.PseudoTable(tbl.Meta(), false, true)
statsTbl.PhysicalID = pid
if tbl.Meta().GetPartitionInfo() == nil || h.Len() < 64 {
h.UpdateStatsCache(types.CacheUpdate{
Updated: []*statistics.Table{statsTbl},
})
}
}
return nil
}
return tbl
}

// FlushStats flushes the cached stats update into store.
func (h *Handle) FlushStats() {
if err := h.DumpStatsDeltaToKV(true); err != nil {
Expand Down
3 changes: 3 additions & 0 deletions pkg/statistics/handle/types/interfaces.go
Original file line number Diff line number Diff line change
Expand Up @@ -547,6 +547,9 @@ type StatsHandle interface {
// GetPartitionStats retrieves the partition stats from cache.
GetPartitionStats(tblInfo *model.TableInfo, pid int64) *statistics.Table

// GetPartitionStatsByID retrieves the partition stats from cache by partition ID.
GetPartitionStatsByID(is infoschema.InfoSchema, pid int64) *statistics.Table

// GetPartitionStatsForAutoAnalyze retrieves the partition stats from cache, but it will not return pseudo.
GetPartitionStatsForAutoAnalyze(tblInfo *model.TableInfo, pid int64) *statistics.Table

Expand Down
10 changes: 3 additions & 7 deletions pkg/statistics/handle/usage/session_stats_collect.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,15 +55,11 @@ var (
// 3. If the stats delta haven't been dumped in the past hour, then return true.
// 4. If the table stats is pseudo or empty or `Modify Count / Table Count` exceeds the threshold.
func (s *statsUsageImpl) needDumpStatsDelta(is infoschema.InfoSchema, dumpAll bool, id int64, item variable.TableDelta, currentTime time.Time) bool {
tbl, ok := s.statsHandle.TableInfoByID(is, id)
dbName, _, ok := s.statsHandle.SchemaNameAndTableNameByID(is, id)
if !ok {
return false
}
dbInfo, ok := infoschema.SchemaByTable(is, tbl.Meta())
if !ok {
return false
}
if util.IsMemOrSysDB(dbInfo.Name.L) {
if util.IsMemOrSysDB(dbName.L) {
return false
}
if dumpAll {
Expand All @@ -76,7 +72,7 @@ func (s *statsUsageImpl) needDumpStatsDelta(is infoschema.InfoSchema, dumpAll bo
// Dump the stats to kv at least once 5 minutes.
return true
}
statsTbl := s.statsHandle.GetPartitionStats(tbl.Meta(), id)
statsTbl := s.statsHandle.GetPartitionStatsByID(is, id)
if statsTbl.Pseudo || statsTbl.RealtimeCount == 0 || float64(item.Count)/float64(statsTbl.RealtimeCount) > DumpStatsDeltaRatio {
// Dump the stats when there are many modifications.
return true
Expand Down
10 changes: 10 additions & 0 deletions pkg/statistics/handle/util/table_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"context"

"github.com/pingcap/tidb/pkg/infoschema"
"github.com/pingcap/tidb/pkg/parser/ast"
"github.com/pingcap/tidb/pkg/table"
)

Expand All @@ -26,6 +27,9 @@ type TableInfoGetter interface {
// TableInfoByID returns the table info specified by the physicalID.
// If the physicalID is corresponding to a partition, return its parent table.
TableInfoByID(is infoschema.InfoSchema, physicalID int64) (table.Table, bool)
// SchemaNameAndTableNameByID returns the schema name and table name specified by the physicalID.
// This is pure memory operation.
SchemaNameAndTableNameByID(is infoschema.InfoSchema, physicalID int64) (schemaName, tableName ast.CIStr, ok bool)
}

// tableInfoGetterImpl is used to get table meta info.
Expand All @@ -47,3 +51,9 @@ func (*tableInfoGetterImpl) TableInfoByID(is infoschema.InfoSchema, physicalID i
tbl, _, _ = is.FindTableByPartitionID(physicalID)
return tbl, tbl != nil
}

// SchemaNameAndTableNameByID returns the schema name and table name specified by the physicalID.
func (*tableInfoGetterImpl) SchemaNameAndTableNameByID(is infoschema.InfoSchema, physicalID int64) (schemaName, tableName ast.CIStr, ok bool) {
schemaName, tableName, ok = is.SchemaNameAndTableNameByID(physicalID)
return schemaName, tableName, ok
}

0 comments on commit f5285d8

Please sign in to comment.