Skip to content

Commit

Permalink
meta,infoschema: fix a bug that after HasTemporaryTable() wrong infos…
Browse files Browse the repository at this point in the history
…chema v2 full load (#54879)

ref #50959
  • Loading branch information
tiancaiamao authored Jul 29, 2024
1 parent 108e98a commit 247b06e
Show file tree
Hide file tree
Showing 5 changed files with 55 additions and 23 deletions.
2 changes: 1 addition & 1 deletion pkg/domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -489,7 +489,7 @@ func (*Domain) fetchSchemasWithTables(schemas []*model.DBInfo, m *meta.Meta, don
var tables []*model.TableInfo
var err error
if variable.SchemaCacheSize.Load() > 0 && !infoschema.IsSpecialDB(di.Name.L) {
name2ID, specialTableInfos, err := meta.GetAllNameToIDAndSpecialAttributeInfo(m, di.ID)
name2ID, specialTableInfos, err := meta.GetAllNameToIDAndTheMustLoadedTableInfo(m, di.ID)
if err != nil {
done <- err
return
Expand Down
12 changes: 11 additions & 1 deletion pkg/infoschema/test/infoschemav2test/v2_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -331,7 +331,7 @@ func BenchmarkTableByName(t *testing.B) {
}

func TestFullLoadAndSnapshot(t *testing.T) {
store := testkit.CreateMockStore(t)
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("set @@global.tidb_schema_cache_size = 512 * 1024 * 1024")

Expand All @@ -342,6 +342,9 @@ func TestFullLoadAndSnapshot(t *testing.T) {
UPDATE variable_value = '%[1]s'`
tk.MustExec(fmt.Sprintf(safePointSQL, timeSafe))

tk.MustExec("use test")
tk.MustExec("create global temporary table tmp (id int) on commit delete rows")

tk.MustExec("create database db1")
tk.MustExec("create database db2")
tk.MustExec("use db1")
Expand All @@ -360,6 +363,13 @@ func TestFullLoadAndSnapshot(t *testing.T) {
tk.MustExec("use db1")
tk.MustQuery("show tables").Check(testkit.Rows())

// Cover a bug that after full load using infoschema v2, the temporary table is gone.
// Check global temporary table not dispear after full load.
require.True(t, dom.InfoSchema().HasTemporaryTable())
tk.MustExec("begin")
tk.MustExec("insert into test.tmp values (1)")
tk.MustExec("commit")

testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/domain/MockTryLoadDiffError", `return("dropdatabase")`)
tk.MustExec("drop database db1")
tk.MustExecToErr("use db1")
Expand Down
2 changes: 1 addition & 1 deletion pkg/meta/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ go_test(
],
embed = [":meta"],
flaky = True,
shard_count = 14,
shard_count = 15,
deps = [
"//pkg/ddl",
"//pkg/kv",
Expand Down
19 changes: 11 additions & 8 deletions pkg/meta/meta.go
Original file line number Diff line number Diff line change
Expand Up @@ -1038,20 +1038,23 @@ func (m *Meta) GetMetasByDBID(dbID int64) ([]structure.HashPair, error) {
return res, nil
}

var checkSubstringsInOrder = [6]string{
var checkSubstringsInOrder = [7]string{
`"fk_info":null`,
`"partition":null`,
`"Lock":null`,
`"tiflash_replica":null`,
`"temp_table_type":0`,
`"policy_ref_info":null`,
`"ttl_info":null`,
}

// CheckSpecialAttributes checks if the special attributes are in the table info.
// Make it same as hasSpecialAttributes.
// IsTableInfoMustLoad checks the above substrings in a table info's json representing.
// When a table contains one of them, tidb must load the table info during schema full load.
// hasSpecialAttributes() is a subset of it, the difference is that:
// If a table need to be resident in-memory, its table info MUST be loaded.
// If a table info is loaded, it's NOT NECESSARILY to be keep in-memory.
// Exported for testing.
// It's the regexp version for hasSpecialAttributes(), please keep up-to-date with it.
func CheckSpecialAttributes(json []byte) bool {
func IsTableInfoMustLoad(json []byte) bool {
idx := 0
for _, substr := range checkSubstringsInOrder {
idx = bytes.Index(json, hack.Slice(substr))
Expand All @@ -1073,9 +1076,9 @@ func Unescape(s string) string {
return s
}

// GetAllNameToIDAndSpecialAttributeInfo gets all the fields and values and table info for special attributes in a hash.
// GetAllNameToIDAndTheMustLoadedTableInfo gets all the fields and values and table info for special attributes in a hash.
// It's used to get some infos for information schema cache in a faster way.
func GetAllNameToIDAndSpecialAttributeInfo(m *Meta, dbID int64) (map[string]int64, []*model.TableInfo, error) {
func GetAllNameToIDAndTheMustLoadedTableInfo(m *Meta, dbID int64) (map[string]int64, []*model.TableInfo, error) {
dbKey := m.dbKey(dbID)
if err := m.checkDBExists(dbKey); err != nil {
return nil, nil, errors.Trace(err)
Expand All @@ -1101,7 +1104,7 @@ func GetAllNameToIDAndSpecialAttributeInfo(m *Meta, dbID int64) (map[string]int6

key := Unescape(nameLMatch[1])
res[strings.Clone(key)] = int64(id)
if CheckSpecialAttributes(value) {
if IsTableInfoMustLoad(value) {
tbInfo := &model.TableInfo{}
err = json.Unmarshal(value, tbInfo)
if err != nil {
Expand Down
43 changes: 31 additions & 12 deletions pkg/meta/meta_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -760,55 +760,74 @@ func TestName(t *testing.T) {
require.NoError(t, err)
}

func TestCheckSpecialAttributes(t *testing.T) {
func TestIsTableInfoMustLoad(t *testing.T) {
tableInfo := &model.TableInfo{
TTLInfo: &model.TTLInfo{IntervalExprStr: "1", IntervalTimeUnit: int(ast.TimeUnitDay), JobInterval: "1h"},
}
b, err := json.Marshal(tableInfo)
require.NoError(t, err)
require.True(t, meta.CheckSpecialAttributes(b))
require.True(t, meta.IsTableInfoMustLoad(b))

tableInfo = &model.TableInfo{
TiFlashReplica: &model.TiFlashReplicaInfo{Count: 1},
}
b, err = json.Marshal(tableInfo)
require.NoError(t, err)
require.True(t, meta.CheckSpecialAttributes(b))
require.True(t, meta.IsTableInfoMustLoad(b))

tableInfo = &model.TableInfo{
PlacementPolicyRef: &model.PolicyRefInfo{ID: 1},
}
b, err = json.Marshal(tableInfo)
require.NoError(t, err)
require.True(t, meta.CheckSpecialAttributes(b))
require.True(t, meta.IsTableInfoMustLoad(b))

tableInfo = &model.TableInfo{
Partition: &model.PartitionInfo{Expr: "a"},
}
b, err = json.Marshal(tableInfo)
require.NoError(t, err)
require.True(t, meta.CheckSpecialAttributes(b))
require.True(t, meta.IsTableInfoMustLoad(b))

tableInfo = &model.TableInfo{
Lock: &model.TableLockInfo{State: model.TableLockStatePreLock},
}
b, err = json.Marshal(tableInfo)
require.NoError(t, err)
require.True(t, meta.CheckSpecialAttributes(b))
require.True(t, meta.IsTableInfoMustLoad(b))

tableInfo = &model.TableInfo{
ForeignKeys: []*model.FKInfo{{ID: 1}},
}
b, err = json.Marshal(tableInfo)
require.NoError(t, err)
require.True(t, meta.CheckSpecialAttributes(b))
require.True(t, meta.IsTableInfoMustLoad(b))

tableInfo = &model.TableInfo{
TempTableType: model.TempTableGlobal,
}
b, err = json.Marshal(tableInfo)
require.NoError(t, err)
require.True(t, meta.IsTableInfoMustLoad(b))

tableInfo = &model.TableInfo{
ID: 123,
}
b, err = json.Marshal(tableInfo)
require.NoError(t, err)
require.False(t, meta.CheckSpecialAttributes(b))
require.False(t, meta.IsTableInfoMustLoad(b))
}

func TestIsTableInfoMustLoadSubStringsOrder(t *testing.T) {
// The order matter!
// IsTableInfoMustLoad relies on the order of the json marshal result,
// or the internal of the json marshal in other words.
// This test cover the invariance, if Go std library changes, we can catch it.
tableInfo := &model.TableInfo{}
b, err := json.Marshal(tableInfo)
require.NoError(t, err)
expect := `{"id":0,"name":{"O":"","L":""},"charset":"","collate":"","cols":null,"index_info":null,"constraint_info":null,"fk_info":null,"state":0,"pk_is_handle":false,"is_common_handle":false,"common_handle_version":0,"comment":"","auto_inc_id":0,"auto_id_cache":0,"auto_rand_id":0,"max_col_id":0,"max_idx_id":0,"max_fk_id":0,"max_cst_id":0,"update_timestamp":0,"ShardRowIDBits":0,"max_shard_row_id_bits":0,"auto_random_bits":0,"auto_random_range_bits":0,"pre_split_regions":0,"partition":null,"compression":"","view":null,"sequence":null,"Lock":null,"version":0,"tiflash_replica":null,"is_columnar":false,"temp_table_type":0,"cache_table_status":0,"policy_ref_info":null,"stats_options":null,"exchange_partition_info":null,"ttl_info":null,"revision":0}`
require.Equal(t, string(b), expect)
}

func TestTableNameExtract(t *testing.T) {
Expand Down Expand Up @@ -851,7 +870,7 @@ func TestTableNameExtract(t *testing.T) {
require.Equal(t, `"\"啊"`, meta.Unescape(nameLMatch[1]))
}

func BenchmarkCheckSpecialAttributes(b *testing.B) {
func BenchmarkIsTableInfoMustLoad(b *testing.B) {
benchCases := [][2]string{
{"narrow", `CREATE TABLE t (c INT PRIMARY KEY);`},
{"wide", `
Expand Down Expand Up @@ -879,12 +898,12 @@ CREATE TABLE t (

for _, benchCase := range benchCases {
b.Run(benchCase[0], func(b *testing.B) {
benchCheckSpecialAttributes(b, benchCase[1])
benchIsTableInfoMustLoad(b, benchCase[1])
})
}
}

func benchCheckSpecialAttributes(b *testing.B, sql string) {
func benchIsTableInfoMustLoad(b *testing.B, sql string) {
p := parser.New()
stmt, err := p.ParseOneStmt(sql, "", "")
require.NoError(b, err)
Expand All @@ -896,7 +915,7 @@ func benchCheckSpecialAttributes(b *testing.B, sql string) {

b.ResetTimer()
for i := 0; i < b.N; i++ {
got := meta.CheckSpecialAttributes(data)
got := meta.IsTableInfoMustLoad(data)
intest.Assert(!got)
}
}

0 comments on commit 247b06e

Please sign in to comment.