Skip to content

Commit

Permalink
infoschema: don't load table info to get auto_increment value (#57296)
Browse files Browse the repository at this point in the history
close #57295
  • Loading branch information
tangenta authored Nov 13, 2024
1 parent 5129bb3 commit eea72ec
Show file tree
Hide file tree
Showing 6 changed files with 78 additions and 14 deletions.
17 changes: 16 additions & 1 deletion pkg/executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -242,11 +242,26 @@ func getAutoIncrementID(
sctx sessionctx.Context,
tblInfo *model.TableInfo,
) int64 {
if raw, ok := is.(*infoschema.SessionExtendedInfoSchema); ok {
if ok, v2 := infoschema.IsV2(raw.InfoSchema); ok {
isCached := v2.TableIsCached(tblInfo.ID)
if !isCached {
// Loading table info from kv storage invalidates the cached auto_increment id.
return 0
}
}
}
tbl, ok := is.TableByID(context.Background(), tblInfo.ID)
if !ok {
return 0
}
return tbl.Allocators(sctx.GetTableCtx()).Get(autoid.AutoIncrementType).Base() + 1
alloc := tbl.Allocators(sctx.GetTableCtx()).Get(autoid.AutoIncrementType)
if alloc == nil || alloc.Base() == 0 {
// It may not be loaded yet.
// To show global next autoID, one should use `show table x next_row_id`.
return 0
}
return alloc.Base() + 1
}

func hasPriv(ctx sessionctx.Context, priv mysql.PrivilegeType) bool {
Expand Down
25 changes: 25 additions & 0 deletions pkg/infoschema/infoschema_v2.go
Original file line number Diff line number Diff line change
Expand Up @@ -691,6 +691,31 @@ func (is *infoschemaV2) IterateAllTableItems(visit func(TableItem) bool) {
})
}

// TableIsCached checks whether the table is cached.
func (is *infoschemaV2) TableIsCached(id int64) (ok bool) {
if !tableIDIsValid(id) {
return false
}

itm, ok := is.searchTableItemByID(id)
if !ok {
return false
}

if isTableVirtual(id) {
if raw, exist := is.Data.specials.Load(itm.dbName.L); exist {
schTbls := raw.(*schemaTables)
_, ok = schTbls.tables[itm.tableName.L]
return ok
}
return false
}

key := tableCacheKey{itm.tableID, itm.schemaVersion}
tbl, found := is.tableCache.Get(key)
return found && tbl != nil
}

// IsSpecialDB tells whether the database is a special database.
func IsSpecialDB(dbName string) bool {
return dbName == util.InformationSchemaName.L ||
Expand Down
2 changes: 1 addition & 1 deletion pkg/infoschema/test/clustertablestest/tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -93,7 +93,7 @@ func TestInfoSchemaFieldValue(t *testing.T) {
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (c int auto_increment primary key, d int)")
tk.MustQuery("select auto_increment from information_schema.tables where table_name='t'").Check(
testkit.Rows("1"))
testkit.Rows("0"))
tk.MustExec("insert into t(c, d) values(1, 1)")
tk.MustQuery("select auto_increment from information_schema.tables where table_name='t'").Check(
testkit.Rows("2"))
Expand Down
3 changes: 2 additions & 1 deletion pkg/infoschema/test/infoschemav2test/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -8,12 +8,13 @@ go_test(
"v2_test.go",
],
flaky = True,
shard_count = 11,
shard_count = 12,
deps = [
"//pkg/domain",
"//pkg/domain/infosync",
"//pkg/infoschema",
"//pkg/infoschema/context",
"//pkg/meta/autoid",
"//pkg/parser/auth",
"//pkg/parser/model",
"//pkg/sessionctx/variable",
Expand Down
24 changes: 24 additions & 0 deletions pkg/infoschema/test/infoschemav2test/v2_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ import (
"github.com/pingcap/tidb/pkg/domain/infosync"
"github.com/pingcap/tidb/pkg/infoschema"
infoschemacontext "github.com/pingcap/tidb/pkg/infoschema/context"
"github.com/pingcap/tidb/pkg/meta/autoid"
"github.com/pingcap/tidb/pkg/parser/auth"
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
Expand Down Expand Up @@ -534,3 +535,26 @@ func TestSnapshotInfoschemaReader(t *testing.T) {
sql = fmt.Sprintf("select * from INFORMATION_SCHEMA.TABLES as of timestamp '%s' where table_schema = 'issue55827'", timeStr)
tk.MustQuery(sql).Check(testkit.Rows())
}

func TestInfoSchemaCachedAutoIncrement(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
autoid.SetStep(1)
tk.MustExec("set @@global.tidb_schema_cache_size = 0;")
tk.MustExec("create table t (a int primary key auto_increment);")
autoIncQuery := "select auto_increment from information_schema.tables where table_name = 't' and table_schema = 'test';"

tk.MustQuery(autoIncQuery).Check(testkit.Rows("0"))
tk.MustExec("insert into t values (),(),();")
tk.MustQuery(autoIncQuery).Check(testkit.Rows("4"))

tk.MustExec("set @@global.tidb_schema_cache_size = 1024 * 1024 * 1024;")
tk.MustExec("create table t1 (a int);") // trigger infoschema cache reload
tk.MustQuery(autoIncQuery).Check(testkit.Rows("0"))
tk.MustExec("insert into t values ();")
tk.MustQuery(autoIncQuery).Check(testkit.Rows("5"))
tk.MustExec("set @@global.tidb_schema_cache_size = 0;")
tk.MustExec("drop table t1;") // trigger infoschema cache reload
tk.MustQuery(autoIncQuery).Check(testkit.Rows("0"))
}
21 changes: 10 additions & 11 deletions pkg/statistics/handle/cache/stats_table_row_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -239,28 +239,27 @@ func getColLengthTables(sctx sessionctx.Context, tableIDs ...int64) (map[tableHi

// GetDataAndIndexLength gets the data and index length of the table.
func (c *StatsTableRowCache) GetDataAndIndexLength(info *model.TableInfo, physicalID int64, rowCount uint64) (dataLength, indexLength uint64) {
columnLength := make(map[string]uint64, len(info.Columns))
for _, col := range info.Columns {
columnLength := make([]uint64, len(info.Columns))
for i, col := range info.Columns {
if col.State != model.StatePublic {
continue
}
length := col.FieldType.StorageLength()
if length != types.VarStorageLen {
columnLength[col.Name.L] = rowCount * uint64(length)
var length uint64
if storageLen := col.FieldType.StorageLength(); storageLen != types.VarStorageLen {
length = rowCount * uint64(storageLen)
} else {
length := c.GetColLength(tableHistID{tableID: physicalID, histID: col.ID})
columnLength[col.Name.L] = length
length = c.GetColLength(tableHistID{tableID: physicalID, histID: col.ID})
}
}
for _, length := range columnLength {
dataLength += length
columnLength[i] = length
}

for _, idx := range info.Indices {
if idx.State != model.StatePublic {
continue
}
if info.GetPartitionInfo() != nil {
// Global indexes calcuated in table level.
// Global indexes calculated in table level.
if idx.Global && info.ID != physicalID {
continue
}
Expand All @@ -271,7 +270,7 @@ func (c *StatsTableRowCache) GetDataAndIndexLength(info *model.TableInfo, physic
}
for _, col := range idx.Columns {
if col.Length == types.UnspecifiedLength {
indexLength += columnLength[col.Name.L]
indexLength += columnLength[col.Offset]
} else {
indexLength += rowCount * uint64(col.Length)
}
Expand Down

0 comments on commit eea72ec

Please # to comment.