diff --git a/pkg/bindinfo/capture_test.go b/pkg/bindinfo/capture_test.go index a6771bff94971..d9ea79524a01d 100644 --- a/pkg/bindinfo/capture_test.go +++ b/pkg/bindinfo/capture_test.go @@ -15,6 +15,7 @@ package bindinfo_test import ( + "context" "fmt" "strings" "testing" @@ -288,16 +289,11 @@ func TestCapturePlanBaselineIgnoreTiFlash(t *testing.T) { // Create virtual tiflash replica info. domSession := domain.GetDomain(tk.Session()) is := domSession.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } // Here the plan is the TiFlash plan. rows := tk.MustQuery("explain select * from t").Rows() diff --git a/pkg/bindinfo/tests/bind_test.go b/pkg/bindinfo/tests/bind_test.go index f6f47763dafd2..53f6346db9367 100644 --- a/pkg/bindinfo/tests/bind_test.go +++ b/pkg/bindinfo/tests/bind_test.go @@ -568,16 +568,11 @@ func TestBindingWithIsolationRead(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } tk.MustExec("create global binding for select * from t where a >= 1 and b >= 1 using select * from t use index(idx_a) where a >= 1 and b >= 1") tk.MustExec("set @@tidb_use_plan_baselines = 1") diff --git a/pkg/ddl/tests/partition/db_partition_test.go b/pkg/ddl/tests/partition/db_partition_test.go index 2d61d046d3ca3..165cf4f29f1b8 100644 --- a/pkg/ddl/tests/partition/db_partition_test.go +++ b/pkg/ddl/tests/partition/db_partition_test.go @@ -2621,6 +2621,8 @@ func getPartitionTableRecordsNum(t *testing.T, ctx sessionctx.Context, tbl table func TestPartitionErrorCode(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) + tk1 := testkit.NewTestKit(t, store) + // add partition tk.MustExec("set @@session.tidb_enable_table_partition = 1") tk.MustExec("drop database if exists test_db_with_partition") @@ -2665,7 +2667,6 @@ func TestPartitionErrorCode(t *testing.T) { tk.MustGetErrCode("alter table t_part repair partition p1;", errno.ErrUnsupportedDDLOperation) // Reduce the impact on DML when executing partition DDL - tk1 := testkit.NewTestKit(t, store) tk1.MustExec("use test") tk1.MustExec("set global tidb_enable_metadata_lock=0") tk1.MustExec("drop table if exists t;") diff --git a/pkg/ddl/tiflash_replica_test.go b/pkg/ddl/tiflash_replica_test.go index 2856bcf32a680..418d4eb4c3cdd 100644 --- a/pkg/ddl/tiflash_replica_test.go +++ b/pkg/ddl/tiflash_replica_test.go @@ -251,11 +251,12 @@ func TestSkipSchemaChecker(t *testing.T) { store := testkit.CreateMockStoreWithSchemaLease(t, tiflashReplicaLease) tk := testkit.NewTestKit(t, store) + tk2 := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("set global tidb_enable_metadata_lock=0") tk.MustExec("drop table if exists t1") tk.MustExec("create table t1 (a int)") - tk2 := testkit.NewTestKit(t, store) tk2.MustExec("use test") // Test skip schema checker for ActionSetTiFlashReplica. diff --git a/pkg/domain/domain.go b/pkg/domain/domain.go index 1bcdc14892dbe..61420836e5c29 100644 --- a/pkg/domain/domain.go +++ b/pkg/domain/domain.go @@ -453,6 +453,10 @@ const fetchSchemaConcurrency = 1 func (*Domain) splitForConcurrentFetch(schemas []*model.DBInfo) [][]*model.DBInfo { groupSize := (len(schemas) + fetchSchemaConcurrency - 1) / fetchSchemaConcurrency + if variable.SchemaCacheSize.Load() > 0 && len(schemas) > 1000 { + // TODO: Temporary solution to speed up when too many databases, will refactor it later. + groupSize = 8 + } splitted := make([][]*model.DBInfo, 0, fetchSchemaConcurrency) schemaCnt := len(schemas) for i := 0; i < schemaCnt; i += groupSize { @@ -471,10 +475,22 @@ func (*Domain) fetchSchemasWithTables(schemas []*model.DBInfo, m *meta.Meta, don // schema is not public, can't be used outside. continue } - tables, err := m.ListTables(di.ID) - if err != nil { - done <- err - return + 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) + if err != nil { + done <- err + return + } + di.TableName2ID = name2ID + tables = specialTableInfos + } else { + tables, err = m.ListTables(di.ID) + if err != nil { + done <- err + return + } } // If TreatOldVersionUTF8AsUTF8MB4 was enable, need to convert the old version schema UTF8 charset to UTF8MB4. if config.GetGlobalConfig().TreatOldVersionUTF8AsUTF8MB4 { diff --git a/pkg/executor/BUILD.bazel b/pkg/executor/BUILD.bazel index 52262920df93a..6b0c8d2cdb564 100644 --- a/pkg/executor/BUILD.bazel +++ b/pkg/executor/BUILD.bazel @@ -406,6 +406,7 @@ go_test( "//pkg/planner/core/base", "//pkg/planner/property", "//pkg/planner/util", + "//pkg/planner/util/coretestsdk", "//pkg/server", "//pkg/session", "//pkg/session/types", diff --git a/pkg/executor/internal/querywatch/BUILD.bazel b/pkg/executor/internal/querywatch/BUILD.bazel index b94749fbdc27e..5845085acce25 100644 --- a/pkg/executor/internal/querywatch/BUILD.bazel +++ b/pkg/executor/internal/querywatch/BUILD.bazel @@ -36,6 +36,7 @@ go_test( "//pkg/errno", "//pkg/kv", "//pkg/meta/autoid", + "//pkg/sessionctx/variable", "//pkg/testkit", "//pkg/testkit/testsetup", "@com_github_pingcap_failpoint//:failpoint", diff --git a/pkg/executor/internal/querywatch/query_watch_test.go b/pkg/executor/internal/querywatch/query_watch_test.go index 4c2f0b235de0b..e8f3e0fc58881 100644 --- a/pkg/executor/internal/querywatch/query_watch_test.go +++ b/pkg/executor/internal/querywatch/query_watch_test.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/failpoint" mysql "github.com/pingcap/tidb/pkg/errno" "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" ) @@ -30,6 +31,9 @@ import ( func TestQueryWatch(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) + if variable.SchemaCacheSize.Load() != 0 { + t.Skip("skip this test because the schema cache is enabled") + } tk.MustExec("use test") tk.MustExec("create table t1(a int)") tk.MustExec("insert into t1 values(1)") diff --git a/pkg/executor/partition_table_test.go b/pkg/executor/partition_table_test.go index 266de430356a9..acbb983db23cc 100644 --- a/pkg/executor/partition_table_test.go +++ b/pkg/executor/partition_table_test.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/planner/util/coretestsdk" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/external" "github.com/pingcap/tidb/pkg/util/dbterror/exeerrors" @@ -316,18 +317,18 @@ func TestOrderByAndLimit(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test_orderby_limit")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if strings.HasPrefix(tblInfo.Name.L, "tr") || strings.HasPrefix(tblInfo.Name.L, "thash") || strings.HasPrefix(tblInfo.Name.L, "tlist") { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test_orderby_limit", "trange") + coretestsdk.SetTiFlashReplica(t, dom, "test_orderby_limit", "thash") + coretestsdk.SetTiFlashReplica(t, dom, "test_orderby_limit", "tlist") + coretestsdk.SetTiFlashReplica(t, dom, "test_orderby_limit", "tregular") + coretestsdk.SetTiFlashReplica(t, dom, "test_orderby_limit", "trange_intpk") + coretestsdk.SetTiFlashReplica(t, dom, "test_orderby_limit", "thash_intpk") + coretestsdk.SetTiFlashReplica(t, dom, "test_orderby_limit", "tlist_intpk") + coretestsdk.SetTiFlashReplica(t, dom, "test_orderby_limit", "tregular_intpk") + coretestsdk.SetTiFlashReplica(t, dom, "test_orderby_limit", "trange_clustered") + coretestsdk.SetTiFlashReplica(t, dom, "test_orderby_limit", "thash_clustered") + coretestsdk.SetTiFlashReplica(t, dom, "test_orderby_limit", "tlist_clustered") + coretestsdk.SetTiFlashReplica(t, dom, "test_orderby_limit", "tregular_clustered") tk.MustExec("set @@session.tidb_isolation_read_engines=\"tikv\"") // test indexLookUp diff --git a/pkg/executor/prepared_test.go b/pkg/executor/prepared_test.go index 0cedd75819381..78a4b70d9ffa0 100644 --- a/pkg/executor/prepared_test.go +++ b/pkg/executor/prepared_test.go @@ -15,6 +15,7 @@ package executor_test import ( + "context" "fmt" "strconv" "strings" @@ -1038,16 +1039,11 @@ func TestPrepareStmtAfterIsolationReadChange(t *testing.T) { // create virtual tiflash replica. is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } tk.MustExec("set @@session.tidb_isolation_read_engines='tikv'") diff --git a/pkg/executor/test/showtest/show_test.go b/pkg/executor/test/showtest/show_test.go index b84667a7d2f5a..33aaf72484f9a 100644 --- a/pkg/executor/test/showtest/show_test.go +++ b/pkg/executor/test/showtest/show_test.go @@ -288,16 +288,11 @@ func TestShowWarningsForExprPushdown(t *testing.T) { // create tiflash replica { is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "show_warnings_expr_pushdown" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tblInfo, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("show_warnings_expr_pushdown")) + require.NoError(t, err) + tblInfo.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } } tk.MustExec("set tidb_allow_mpp=0") diff --git a/pkg/expression/integration_test/integration_test.go b/pkg/expression/integration_test/integration_test.go index ac5c166b4a1c4..8d289da9ca611 100644 --- a/pkg/expression/integration_test/integration_test.go +++ b/pkg/expression/integration_test/integration_test.go @@ -649,16 +649,11 @@ func TestShardIndexOnTiFlash(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") tk.MustExec("set @@session.tidb_enforce_mpp = 1") @@ -672,7 +667,7 @@ func TestShardIndexOnTiFlash(t *testing.T) { tk.MustExec("set @@session.tidb_enforce_mpp = 0") tk.MustExec("set @@session.tidb_allow_mpp = 0") // when we isolated the read engine as 'tiflash' and banned TiDB opening allow-mpp, no suitable plan is generated. - _, err := tk.Exec("explain select max(b) from t") + _, err = tk.Exec("explain select max(b) from t") require.NotNil(t, err) require.Equal(t, err.Error(), "[planner:1815]Internal : Can't find a proper physical plan for this query") } @@ -690,16 +685,11 @@ func TestExprPushdownBlacklist(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } tk.MustExec("insert into mysql.expr_pushdown_blacklist " + diff --git a/pkg/infoschema/builder.go b/pkg/infoschema/builder.go index ec37a8080251e..a9e4074e9011d 100644 --- a/pkg/infoschema/builder.go +++ b/pkg/infoschema/builder.go @@ -919,11 +919,28 @@ func (b *Builder) createSchemaTablesForDB(di *model.DBInfo, tableFromMeta tableF schTbls.tables[t.Name.L] = tbl b.addTable(schemaVersion, di, t, tbl) + if len(di.TableName2ID) > 0 { + delete(di.TableName2ID, t.Name.L) + } if tblInfo := tbl.Meta(); tblInfo.TempTableType != model.TempTableNone { b.addTemporaryTable(tblInfo.ID) } } + // Add the rest name to ID mappings. + if b.enableV2 { + for name, id := range di.TableName2ID { + item := tableItem{ + dbName: di.Name.L, + dbID: di.ID, + tableName: name, + tableID: id, + schemaVersion: schemaVersion, + } + b.infoData.byID.Set(item) + b.infoData.byName.Set(item) + } + } b.addDB(schemaVersion, di, schTbls) return nil @@ -931,7 +948,7 @@ func (b *Builder) createSchemaTablesForDB(di *model.DBInfo, tableFromMeta tableF func (b *Builder) addDB(schemaVersion int64, di *model.DBInfo, schTbls *schemaTables) { if b.enableV2 { - if isSpecialDB(di.Name.L) { + if IsSpecialDB(di.Name.L) { b.infoData.addSpecialDB(di, schTbls) } else { b.infoData.addDB(schemaVersion, di) diff --git a/pkg/infoschema/infoschema_v2.go b/pkg/infoschema/infoschema_v2.go index b59dd4a3c9678..176896e3af026 100644 --- a/pkg/infoschema/infoschema_v2.go +++ b/pkg/infoschema/infoschema_v2.go @@ -437,7 +437,8 @@ func (is *infoschemaV2) tableByID(id int64, noRefill bool) (val table.Table, ok return ret, true } -func isSpecialDB(dbName string) bool { +// IsSpecialDB tells whether the database is a special database. +func IsSpecialDB(dbName string) bool { return dbName == util.InformationSchemaName.L || dbName == util.PerformanceSchemaName.L || dbName == util.MetricSchemaName.L @@ -455,7 +456,7 @@ func (is *infoschemaV2) EvictTable(schema, tbl string) { } func (is *infoschemaV2) TableByName(ctx context.Context, schema, tbl model.CIStr) (t table.Table, err error) { - if isSpecialDB(schema.L) { + if IsSpecialDB(schema.L) { if raw, ok := is.specials.Load(schema.L); ok { tbNames := raw.(*schemaTables) if t, ok = tbNames.tables[tbl.L]; ok { @@ -504,7 +505,7 @@ func (is *infoschemaV2) TableInfoByID(id int64) (*model.TableInfo, bool) { // SchemaTableInfos implements InfoSchema.FindTableInfoByPartitionID func (is *infoschemaV2) SchemaTableInfos(schema model.CIStr) []*model.TableInfo { - if isSpecialDB(schema.L) { + if IsSpecialDB(schema.L) { raw, ok := is.Data.specials.Load(schema.L) if ok { schTbls := raw.(*schemaTables) @@ -554,7 +555,7 @@ func (is *infoschemaV2) FindTableInfoByPartitionID( } func (is *infoschemaV2) SchemaByName(schema model.CIStr) (val *model.DBInfo, ok bool) { - if isSpecialDB(schema.L) { + if IsSpecialDB(schema.L) { raw, ok := is.Data.specials.Load(schema.L) if !ok { return nil, false @@ -727,7 +728,7 @@ func (is *infoschemaV2) SchemaByID(id int64) (*model.DBInfo, bool) { } func (is *infoschemaV2) SchemaTables(schema model.CIStr) (tables []table.Table) { - if isSpecialDB(schema.L) { + if IsSpecialDB(schema.L) { raw, ok := is.Data.specials.Load(schema.L) if ok { schTbls := raw.(*schemaTables) diff --git a/pkg/meta/BUILD.bazel b/pkg/meta/BUILD.bazel index 6f7b77051f643..3eb87dc9b05d4 100644 --- a/pkg/meta/BUILD.bazel +++ b/pkg/meta/BUILD.bazel @@ -17,6 +17,7 @@ go_library( "//pkg/parser/mysql", "//pkg/structure", "//pkg/util/dbterror", + "//pkg/util/hack", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_kvproto//pkg/kvrpcpb", "@com_github_pingcap_kvproto//pkg/resource_manager", @@ -32,7 +33,7 @@ go_test( ], embed = [":meta"], flaky = True, - shard_count = 12, + shard_count = 14, deps = [ "//pkg/kv", "//pkg/parser/ast", diff --git a/pkg/meta/meta.go b/pkg/meta/meta.go index 1d8e036977667..acfd756b58d5b 100644 --- a/pkg/meta/meta.go +++ b/pkg/meta/meta.go @@ -20,6 +20,7 @@ import ( "encoding/json" "fmt" "math" + "regexp" "strconv" "strings" "sync" @@ -36,6 +37,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/structure" "github.com/pingcap/tidb/pkg/util/dbterror" + "github.com/pingcap/tidb/pkg/util/hack" ) var ( @@ -1012,6 +1014,70 @@ func (m *Meta) GetMetasByDBID(dbID int64) ([]structure.HashPair, error) { return res, nil } +// CheckSpecialAttributes checks if the special attributes are in the table info. +// Make it same as hasSpecialAttributes. +// Exported for testing. +// It's the regexp version for hasSpecialAttributes(), please keep up-to-date with it. +func CheckSpecialAttributes(str string) bool { + if strings.Contains(str, "\"tiflash_replica\":null") && strings.Contains(str, "\"policy_ref_info\":null") && + strings.Contains(str, "\"partition\":null") && strings.Contains(str, "\"ttl_info\":null") { + return false + } + return true +} + +// NameExtractRegexp is exported for testing. +const NameExtractRegexp = `"L":"([^"\\]*(?:\\.[^"\\]*)*)"}` + +// Unescape is exported for testing. +func Unescape(s string) string { + s = strings.ReplaceAll(s, `\"`, `"`) + s = strings.ReplaceAll(s, `\\`, `\`) + return s +} + +// GetAllNameToIDAndSpecialAttributeInfo 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) { + dbKey := m.dbKey(dbID) + if err := m.checkDBExists(dbKey); err != nil { + return nil, nil, errors.Trace(err) + } + + res := make(map[string]int64) + idRegex := regexp.MustCompile(`"id":(\d+)`) + nameLRegex := regexp.MustCompile(NameExtractRegexp) + + tableInfos := make([]*model.TableInfo, 0) + + err := m.txn.IterateHash(dbKey, func(field []byte, value []byte) error { + if !strings.HasPrefix(string(hack.String(field)), "Table") { + return nil + } + + idMatch := idRegex.FindStringSubmatch(string(hack.String(value))) + nameLMatch := nameLRegex.FindStringSubmatch(string(hack.String(value))) + id, err := strconv.Atoi(idMatch[1]) + if err != nil { + return errors.Trace(err) + } + + res[Unescape(nameLMatch[1])] = int64(id) + if CheckSpecialAttributes(string(hack.String(value))) { + tbInfo := &model.TableInfo{} + err = json.Unmarshal(value, tbInfo) + if err != nil { + return errors.Trace(err) + } + tbInfo.DBID = dbID + tableInfos = append(tableInfos, tbInfo) + } + return nil + }) + + return res, tableInfos, errors.Trace(err) +} + // ListTables shows all tables in database. func (m *Meta) ListTables(dbID int64) ([]*model.TableInfo, error) { res, err := m.GetMetasByDBID(dbID) diff --git a/pkg/meta/meta_test.go b/pkg/meta/meta_test.go index d0ef4402378ab..073e53544acde 100644 --- a/pkg/meta/meta_test.go +++ b/pkg/meta/meta_test.go @@ -16,7 +16,9 @@ package meta_test import ( "context" + "encoding/json" "fmt" + "regexp" "strconv" "testing" "time" @@ -752,3 +754,80 @@ func TestName(t *testing.T) { err = txn.Rollback() require.NoError(t, err) } + +func TestCheckSpecialAttributes(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(string(b))) + + tableInfo = &model.TableInfo{ + TiFlashReplica: &model.TiFlashReplicaInfo{Count: 1}, + } + b, err = json.Marshal(tableInfo) + require.NoError(t, err) + require.True(t, meta.CheckSpecialAttributes(string(b))) + + tableInfo = &model.TableInfo{ + PlacementPolicyRef: &model.PolicyRefInfo{ID: 1}, + } + b, err = json.Marshal(tableInfo) + require.NoError(t, err) + require.True(t, meta.CheckSpecialAttributes(string(b))) + + tableInfo = &model.TableInfo{ + Partition: &model.PartitionInfo{Expr: "a"}, + } + b, err = json.Marshal(tableInfo) + require.NoError(t, err) + require.True(t, meta.CheckSpecialAttributes(string(b))) + + tableInfo = &model.TableInfo{ + ID: 123, + } + b, err = json.Marshal(tableInfo) + require.NoError(t, err) + require.False(t, meta.CheckSpecialAttributes(string(b))) +} + +func TestTableNameExtract(t *testing.T) { + var tbl model.TableInfo + tbl.Name = model.NewCIStr(`a`) + b, err := json.Marshal(tbl) + require.NoError(t, err) + + nameLRegex := regexp.MustCompile(meta.NameExtractRegexp) + nameLMatch := nameLRegex.FindStringSubmatch(string(b)) + require.Len(t, nameLMatch, 2) + require.Equal(t, "a", nameLMatch[1]) + + tbl.Name = model.NewCIStr(`"a"`) + b, err = json.Marshal(tbl) + require.NoError(t, err) + nameLMatch = nameLRegex.FindStringSubmatch(string(b)) + require.Len(t, nameLMatch, 2) + require.Equal(t, `"a"`, meta.Unescape(nameLMatch[1])) + + tbl.Name = model.NewCIStr(`""a"`) + b, err = json.Marshal(tbl) + require.NoError(t, err) + nameLMatch = nameLRegex.FindStringSubmatch(string(b)) + require.Len(t, nameLMatch, 2) + require.Equal(t, `""a"`, meta.Unescape(nameLMatch[1])) + + tbl.Name = model.NewCIStr(`"\"a"`) + b, err = json.Marshal(tbl) + require.NoError(t, err) + nameLMatch = nameLRegex.FindStringSubmatch(string(b)) + require.Len(t, nameLMatch, 2) + require.Equal(t, `"\"a"`, meta.Unescape(nameLMatch[1])) + + tbl.Name = model.NewCIStr(`"\"啊"`) + b, err = json.Marshal(tbl) + require.NoError(t, err) + nameLMatch = nameLRegex.FindStringSubmatch(string(b)) + require.Len(t, nameLMatch, 2) + require.Equal(t, `"\"啊"`, meta.Unescape(nameLMatch[1])) +} diff --git a/pkg/parser/model/model.go b/pkg/parser/model/model.go index 79633b267dcaf..0e8440ac65471 100644 --- a/pkg/parser/model/model.go +++ b/pkg/parser/model/model.go @@ -1687,13 +1687,14 @@ func (fk *FKInfo) Clone() *FKInfo { // DBInfo provides meta data describing a DB. type DBInfo struct { - ID int64 `json:"id"` // Database ID - Name CIStr `json:"db_name"` // DB name. - Charset string `json:"charset"` - Collate string `json:"collate"` - Tables []*TableInfo `json:"-"` // Tables in the DB. - State SchemaState `json:"state"` - PlacementPolicyRef *PolicyRefInfo `json:"policy_ref_info"` + ID int64 `json:"id"` // Database ID + Name CIStr `json:"db_name"` // DB name. + Charset string `json:"charset"` + Collate string `json:"collate"` + Tables []*TableInfo `json:"-"` // Tables in the DB. + State SchemaState `json:"state"` + PlacementPolicyRef *PolicyRefInfo `json:"policy_ref_info"` + TableName2ID map[string]int64 `json:"-"` } // Clone clones DBInfo. diff --git a/pkg/planner/core/casetest/BUILD.bazel b/pkg/planner/core/casetest/BUILD.bazel index 0346f2b59c408..e005981526031 100644 --- a/pkg/planner/core/casetest/BUILD.bazel +++ b/pkg/planner/core/casetest/BUILD.bazel @@ -21,6 +21,7 @@ go_test( "//pkg/planner/core", "//pkg/planner/core/base", "//pkg/planner/property", + "//pkg/planner/util/coretestsdk", "//pkg/testkit", "//pkg/testkit/testdata", "//pkg/testkit/testmain", diff --git a/pkg/planner/core/casetest/enforcempp/enforce_mpp_test.go b/pkg/planner/core/casetest/enforcempp/enforce_mpp_test.go index edc2656a43ac1..3c812470d9563 100644 --- a/pkg/planner/core/casetest/enforcempp/enforce_mpp_test.go +++ b/pkg/planner/core/casetest/enforcempp/enforce_mpp_test.go @@ -15,6 +15,7 @@ package enforcempp import ( + "context" "strings" "testing" "time" @@ -53,27 +54,9 @@ func TestEnforceMPP(t *testing.T) { is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - if tblInfo.Name.L == "s" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - if tblInfo.Name.L == "t3" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, db.Name.L, "t") + coretestsdk.SetTiFlashReplica(t, dom, db.Name.L, "s") + coretestsdk.SetTiFlashReplica(t, dom, db.Name.L, "t3") var input []string var output []struct { @@ -147,34 +130,18 @@ func TestEnforceMPPWarning1(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: false, - } - } + tblInfo, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblInfo.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: false, } continue } if strings.HasPrefix(tt, "cmd: enable-replica") { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "t") continue } testdata.OnRecord(func() { @@ -202,16 +169,11 @@ func TestEnforceMPPWarning2(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } var input []string @@ -255,16 +217,11 @@ func TestEnforceMPPWarning3(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } var input []string @@ -318,18 +275,8 @@ func TestEnforceMPPWarning4(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" || tblInfo.Name.L == "s" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "t") + coretestsdk.SetTiFlashReplica(t, dom, "test", "s") var input []string var output []struct { @@ -380,18 +327,9 @@ func TestMPP2PhaseAggPushDown(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "c" || tblInfo.Name.L == "o" || tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "c") + coretestsdk.SetTiFlashReplica(t, dom, "test", "o") + coretestsdk.SetTiFlashReplica(t, dom, "test", "t") var input []string var output []struct { @@ -436,16 +374,11 @@ func TestMPPSkewedGroupDistinctRewrite(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } var input []string @@ -489,16 +422,11 @@ func TestMPPSingleDistinct3Stage(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } var input []string diff --git a/pkg/planner/core/casetest/hint/hint_test.go b/pkg/planner/core/casetest/hint/hint_test.go index de9e02ce0b24f..aa671b836e4a9 100644 --- a/pkg/planner/core/casetest/hint/hint_test.go +++ b/pkg/planner/core/casetest/hint/hint_test.go @@ -15,6 +15,7 @@ package hint import ( + "context" "testing" "github.com/pingcap/tidb/pkg/domain" @@ -42,16 +43,9 @@ func TestReadFromStorageHint(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "t") + coretestsdk.SetTiFlashReplica(t, dom, "test", "tt") + coretestsdk.SetTiFlashReplica(t, dom, "test", "ttt") var input []string var output []struct { @@ -93,16 +87,11 @@ func TestAllViewHintType(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } tk.MustExec("create definer='root'@'localhost' view v as select t.a, t.b from t join t1 on t.a = t1.a;") @@ -163,19 +152,9 @@ func TestJoinHintCompatibility(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - name := tblInfo.Name.L - if name == "t4" || name == "t5" || name == "t6" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "t4") + coretestsdk.SetTiFlashReplica(t, dom, "test", "t5") + coretestsdk.SetTiFlashReplica(t, dom, "test", "t6") tk.MustExec("create definer='root'@'localhost' view v as select /*+ leading(t1), inl_join(t1) */ t.a from t join t1 join t2 where t.a = t1.a and t1.b = t2.b;") tk.MustExec("create definer='root'@'localhost' view v1 as select /*+ leading(t2), merge_join(t) */ t.a from t join t1 join t2 where t.a = t1.a and t1.b = t2.b;") @@ -213,16 +192,7 @@ func TestReadFromStorageHintAndIsolationRead(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "t") var input []string var output []struct { @@ -255,16 +225,7 @@ func TestIsolationReadTiFlashUseIndexHint(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "t") tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") var input []string @@ -309,16 +270,11 @@ func TestOptimizeHintOnPartitionTable(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) diff --git a/pkg/planner/core/casetest/integration_test.go b/pkg/planner/core/casetest/integration_test.go index e43bc8a58504b..1a455775d28e6 100644 --- a/pkg/planner/core/casetest/integration_test.go +++ b/pkg/planner/core/casetest/integration_test.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/planner/util/coretestsdk" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testdata" "github.com/stretchr/testify/require" @@ -55,18 +56,8 @@ func TestVerboseExplain(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t1" || tblInfo.Name.L == "t2" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "t1") + coretestsdk.SetTiFlashReplica(t, dom, "test", "t2") var input []string var output []struct { @@ -96,14 +87,11 @@ func TestIsolationReadTiFlashNotChoosePointGet(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } + tblInfo, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblInfo.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") @@ -155,18 +143,7 @@ func TestMergeContinuousSelections(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "ts" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "ts") tk.MustExec(" set @@tidb_allow_mpp=1;") diff --git a/pkg/planner/core/casetest/mpp/BUILD.bazel b/pkg/planner/core/casetest/mpp/BUILD.bazel index 8e5eb92d33939..052fd0d9e4eaf 100644 --- a/pkg/planner/core/casetest/mpp/BUILD.bazel +++ b/pkg/planner/core/casetest/mpp/BUILD.bazel @@ -14,6 +14,7 @@ go_test( "//pkg/config", "//pkg/domain", "//pkg/parser/model", + "//pkg/planner/util/coretestsdk", "//pkg/testkit", "//pkg/testkit/testdata", "//pkg/testkit/testmain", diff --git a/pkg/planner/core/casetest/mpp/mpp_test.go b/pkg/planner/core/casetest/mpp/mpp_test.go index dde7f9716c201..43f3c3b6320e4 100644 --- a/pkg/planner/core/casetest/mpp/mpp_test.go +++ b/pkg/planner/core/casetest/mpp/mpp_test.go @@ -15,12 +15,14 @@ package mpp import ( + "context" "strings" "testing" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/planner/util/coretestsdk" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testdata" "github.com/stretchr/testify/require" @@ -51,18 +53,10 @@ func TestMPPJoin(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "fact_t" || tblInfo.Name.L == "d1_t" || tblInfo.Name.L == "d2_t" || tblInfo.Name.L == "d3_t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "fact_t") + coretestsdk.SetTiFlashReplica(t, dom, "test", "d1_t") + coretestsdk.SetTiFlashReplica(t, dom, "test", "d2_t") + coretestsdk.SetTiFlashReplica(t, dom, "test", "d3_t") tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") tk.MustExec("set @@session.tidb_allow_mpp = 1") @@ -95,16 +89,11 @@ func TestMPPLeftSemiJoin(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } var input []string @@ -149,18 +138,9 @@ func TestMPPOuterJoinBuildSideForBroadcastJoin(t *testing.T) { tk.MustExec("analyze table b all columns") // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "a" || tblInfo.Name.L == "b" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "a") + coretestsdk.SetTiFlashReplica(t, dom, "test", "b") + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") tk.MustExec("set @@session.tidb_opt_mpp_outer_join_fixed_build_side = 0") tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 10000") @@ -197,18 +177,8 @@ func TestMPPOuterJoinBuildSideForShuffleJoinWithFixedBuildSide(t *testing.T) { tk.MustExec("analyze table b") // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "a" || tblInfo.Name.L == "b" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "a") + coretestsdk.SetTiFlashReplica(t, dom, "test", "b") tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") tk.MustExec("set @@session.tidb_opt_mpp_outer_join_fixed_build_side = 1") tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 0") @@ -245,18 +215,9 @@ func TestMPPOuterJoinBuildSideForShuffleJoin(t *testing.T) { tk.MustExec("analyze table b all columns") // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "a" || tblInfo.Name.L == "b" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "a") + coretestsdk.SetTiFlashReplica(t, dom, "test", "b") + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") tk.MustExec("set @@session.tidb_opt_mpp_outer_join_fixed_build_side = 0") tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 0") @@ -308,18 +269,10 @@ func TestMPPShuffledJoin(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "fact_t" || tblInfo.Name.L == "d1_t" || tblInfo.Name.L == "d2_t" || tblInfo.Name.L == "d3_t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "fact_t") + coretestsdk.SetTiFlashReplica(t, dom, "test", "d1_t") + coretestsdk.SetTiFlashReplica(t, dom, "test", "d2_t") + coretestsdk.SetTiFlashReplica(t, dom, "test", "d3_t") tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") tk.MustExec("set @@session.tidb_allow_mpp = 1") @@ -359,18 +312,9 @@ func TestMPPJoinWithCanNotFoundColumnInSchemaColumnsError(t *testing.T) { tk.MustExec("analyze table t3 all columns") dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t1" || tblInfo.Name.L == "t2" || tblInfo.Name.L == "t3" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "t1") + coretestsdk.SetTiFlashReplica(t, dom, "test", "t2") + coretestsdk.SetTiFlashReplica(t, dom, "test", "t3") tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") tk.MustExec("set @@session.tidb_enforce_mpp = 1") @@ -410,18 +354,8 @@ func TestMPPWithHashExchangeUnderNewCollation(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "table_1" || tblInfo.Name.L == "table_2" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "table_1") + coretestsdk.SetTiFlashReplica(t, dom, "test", "table_2") tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") tk.MustExec("set @@session.tidb_allow_mpp = 1") @@ -456,18 +390,7 @@ func TestMPPWithBroadcastExchangeUnderNewCollation(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "table_1" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "table_1") tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") tk.MustExec("set @@session.tidb_allow_mpp = 1") @@ -499,18 +422,7 @@ func TestMPPAvgRewrite(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "table_1" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "table_1") tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") tk.MustExec("set @@session.tidb_allow_mpp = 1") @@ -543,18 +455,8 @@ func TestMppUnionAll(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" || tblInfo.Name.L == "t1" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "t") + coretestsdk.SetTiFlashReplica(t, dom, "test", "t1") var input []string var output []struct { @@ -587,18 +489,8 @@ func TestMppJoinDecimal(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" || tblInfo.Name.L == "tt" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "t") + coretestsdk.SetTiFlashReplica(t, dom, "test", "tt") tk.MustExec("set @@tidb_allow_mpp=1;") tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 1") @@ -634,18 +526,8 @@ func TestMppJoinExchangeColumnPrune(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" || tblInfo.Name.L == "tt" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "t") + coretestsdk.SetTiFlashReplica(t, dom, "test", "tt") tk.MustExec("set @@tidb_allow_mpp=1;") tk.MustExec("set @@tidb_enforce_mpp=1;") @@ -694,18 +576,8 @@ func TestMppFineGrainedJoinAndAgg(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" || tblInfo.Name.L == "tt" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "t") + coretestsdk.SetTiFlashReplica(t, dom, "test", "tt") tk.MustExec("set @@tidb_allow_mpp=1;") tk.MustExec("set @@tidb_enforce_mpp=1;") @@ -741,16 +613,11 @@ func TestMppAggTopNWithJoin(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } tk.MustExec(" set @@tidb_allow_mpp=1;") @@ -784,16 +651,11 @@ func TestRejectSortForMPP(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") @@ -827,16 +689,11 @@ func TestPushDownSelectionForMPP(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") @@ -871,16 +728,11 @@ func TestPushDownProjectionForMPP(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") @@ -914,16 +766,11 @@ func TestPushDownAggForMPP(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } tk.MustExec(" set @@tidb_allow_mpp=1; set @@tidb_broadcast_join_threshold_count = 1; set @@tidb_broadcast_join_threshold_size=1;") @@ -956,16 +803,11 @@ func TestMppVersion(t *testing.T) { // Create virtual tiflash replica info. is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } var input []string @@ -1012,18 +854,9 @@ func TestIssue52828(t *testing.T) { tk.MustExec("analyze table dd") // Create virtual tiflash replica info. - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "b" || tblInfo.Name.L == "c" || tblInfo.Name.L == "dd" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "b") + coretestsdk.SetTiFlashReplica(t, dom, "test", "c") + coretestsdk.SetTiFlashReplica(t, dom, "test", "dd") tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1") tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") diff --git a/pkg/planner/core/casetest/physicalplantest/physical_plan_test.go b/pkg/planner/core/casetest/physicalplantest/physical_plan_test.go index fb235302badf6..cde71695119e6 100644 --- a/pkg/planner/core/casetest/physicalplantest/physical_plan_test.go +++ b/pkg/planner/core/casetest/physicalplantest/physical_plan_test.go @@ -1238,19 +1238,9 @@ func TestHJBuildAndProbeHint4TiFlash(t *testing.T) { tk.MustExec("insert into t3 values(1,1),(2,1)") // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - table := tbl.Meta() - tableName := table.Name.L - if tableName == "t1" || tableName == "t2" || tableName == "t3" { - table.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "t1") + coretestsdk.SetTiFlashReplica(t, dom, "test", "t2") + coretestsdk.SetTiFlashReplica(t, dom, "test", "t3") tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") for i, ts := range input { @@ -1281,18 +1271,7 @@ func TestMPPSinglePartitionType(t *testing.T) { tk.MustExec("create table employee(empid int, deptid int, salary decimal(10,2))") tk.MustExec("set tidb_enforce_mpp=0") - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "employee" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "employee") for i, ts := range input { testdata.OnRecord(func() { @@ -1331,19 +1310,8 @@ func TestCountStarForTiFlash(t *testing.T) { // tiflash dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - tableName := tblInfo.Name.L - if tableName == "t" || tableName == "t_pick_row_id" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "t") + coretestsdk.SetTiFlashReplica(t, dom, "test", "t_pick_row_id") tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") for i, ts := range input { @@ -1427,19 +1395,8 @@ func TestHashAggPushdownToTiFlashCompute(t *testing.T) { }) dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - tableName := tblInfo.Name.L - if tableName == "tbl_15" || tableName == "tbl_16" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "tbl_15") + coretestsdk.SetTiFlashReplica(t, dom, "test", "tbl_16") tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") tk.MustExec("set @@tidb_partition_prune_mode = 'static';") diff --git a/pkg/planner/core/casetest/pushdown/BUILD.bazel b/pkg/planner/core/casetest/pushdown/BUILD.bazel index 55145a587d7f8..a9bfc80ead97c 100644 --- a/pkg/planner/core/casetest/pushdown/BUILD.bazel +++ b/pkg/planner/core/casetest/pushdown/BUILD.bazel @@ -13,6 +13,7 @@ go_test( deps = [ "//pkg/domain", "//pkg/parser/model", + "//pkg/planner/util/coretestsdk", "//pkg/testkit", "//pkg/testkit/testdata", "//pkg/testkit/testmain", diff --git a/pkg/planner/core/casetest/pushdown/push_down_test.go b/pkg/planner/core/casetest/pushdown/push_down_test.go index 2be7148a2a6da..646c7bbcc1fd0 100644 --- a/pkg/planner/core/casetest/pushdown/push_down_test.go +++ b/pkg/planner/core/casetest/pushdown/push_down_test.go @@ -15,10 +15,12 @@ package pushdown import ( + "context" "testing" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/planner/util/coretestsdk" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testdata" "github.com/stretchr/testify/require" @@ -36,16 +38,11 @@ func TestPushDownToTiFlashWithKeepOrder(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } tk.MustExec("set tidb_cost_model_version=2") @@ -81,16 +78,11 @@ func TestPushDownToTiFlashWithKeepOrderInFastMode(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } tk.MustExec("set tidb_cost_model_version=2") @@ -127,19 +119,7 @@ func TestPushDownProjectionForTiFlash(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "t") var input []string var output []struct { @@ -171,16 +151,11 @@ func TestPushDownProjectionForTiFlashCoprocessor(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } var input []string @@ -212,16 +187,11 @@ func TestSelPushDownTiFlash(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") @@ -260,18 +230,7 @@ func TestJoinNotSupportedByTiFlash(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "table_1" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "table_1") tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") tk.MustExec("set @@session.tidb_allow_mpp = 1") diff --git a/pkg/planner/core/casetest/rule/BUILD.bazel b/pkg/planner/core/casetest/rule/BUILD.bazel index 3fc20aaa907a9..f00aaebfc37d6 100644 --- a/pkg/planner/core/casetest/rule/BUILD.bazel +++ b/pkg/planner/core/casetest/rule/BUILD.bazel @@ -20,7 +20,6 @@ go_test( "//pkg/expression/aggregation", "//pkg/expression/contextstatic", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/planner/util/coretestsdk", "//pkg/planner/util/coreusage", diff --git a/pkg/planner/core/casetest/rule/rule_join_reorder_test.go b/pkg/planner/core/casetest/rule/rule_join_reorder_test.go index 68bdcbfe315cb..d01c5da52dfda 100644 --- a/pkg/planner/core/casetest/rule/rule_join_reorder_test.go +++ b/pkg/planner/core/casetest/rule/rule_join_reorder_test.go @@ -19,7 +19,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/domain" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/planner/util/coretestsdk" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testdata" "github.com/stretchr/testify/require" @@ -75,19 +75,13 @@ func TestJoinOrderHint4TiFlash(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - tableName := tblInfo.Name.L - if tableName == "t" || tableName == "t1" || tableName == "t2" || tableName == "t3" || tableName == "t4" || tableName == "t5" || tableName == "t6" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "t") + coretestsdk.SetTiFlashReplica(t, dom, "test", "t1") + coretestsdk.SetTiFlashReplica(t, dom, "test", "t2") + coretestsdk.SetTiFlashReplica(t, dom, "test", "t3") + coretestsdk.SetTiFlashReplica(t, dom, "test", "t4") + coretestsdk.SetTiFlashReplica(t, dom, "test", "t5") + coretestsdk.SetTiFlashReplica(t, dom, "test", "t6") tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") runJoinReorderTestData(t, tk, "TestJoinOrderHint4TiFlash") diff --git a/pkg/planner/core/casetest/tiflash_selection_late_materialization_test.go b/pkg/planner/core/casetest/tiflash_selection_late_materialization_test.go index 552cf7593706f..040994f44c454 100644 --- a/pkg/planner/core/casetest/tiflash_selection_late_materialization_test.go +++ b/pkg/planner/core/casetest/tiflash_selection_late_materialization_test.go @@ -17,9 +17,9 @@ package casetest import ( "testing" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/util/coretestsdk" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testdata" "github.com/pingcap/tidb/pkg/util/plancodec" @@ -40,18 +40,7 @@ func TestTiFlashLateMaterialization(t *testing.T) { tk.MustExec("set @@session.tidb_allow_tiflash_cop=ON") // Create virtual `tiflash` replica info. - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t1" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "t1") // Enable late materialization. tk.MustExec("set @@tidb_opt_enable_late_materialization = on") tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") diff --git a/pkg/planner/core/integration_test.go b/pkg/planner/core/integration_test.go index 72d16f3b46b2c..4ae879b2d8876 100644 --- a/pkg/planner/core/integration_test.go +++ b/pkg/planner/core/integration_test.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/util/coretestsdk" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/testkit" @@ -82,16 +83,11 @@ func TestAggPushDownEngine(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tblInfo, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblInfo.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") @@ -133,14 +129,11 @@ func TestIssue15110And49616(t *testing.T) { is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "crm_rd_150m" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tblInfo, err := is.TableByName(context.Background(), db.Name, model.NewCIStr("crm_rd_150m")) + require.NoError(t, err) + tblInfo.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") @@ -192,18 +185,13 @@ func TestNotReadOnlySQLOnTiFlash(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tblInfo, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblInfo.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } - err := tk.ExecToErr("select * from t for update") + err = tk.ExecToErr("select * from t for update") require.EqualError(t, err, `[planner:1815]Internal : No access path for table 't' is found with 'tidb_isolation_read_engines' = 'tiflash', valid values can be 'tiflash, tikv'. Please check tiflash replica or check if the query is not readonly and sql mode is strict.`) err = tk.ExecToErr("insert into t select * from t") @@ -229,16 +217,11 @@ func TestTimeToSecPushDownToTiFlash(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } rows := [][]any{ @@ -264,16 +247,11 @@ func TestRightShiftPushDownToTiFlash(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } rows := [][]any{ @@ -705,16 +683,11 @@ func TestReverseUTF8PushDownToTiFlash(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } rows := [][]any{ @@ -740,16 +713,11 @@ func TestReversePushDownToTiFlash(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } rows := [][]any{ @@ -775,16 +743,11 @@ func TestSpacePushDownToTiFlash(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } rows := [][]any{ @@ -916,16 +879,11 @@ func TestConflictReadFromStorage(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } tk.MustQuery(`explain select /*+ read_from_storage(tikv[t partition(p0)], tiflash[t partition(p1, p2)]) */ * from t`) tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 Storage hints are conflict, you can only specify one storage type of table test.t")) @@ -1229,16 +1187,11 @@ func TestRepeatPushDownToTiFlash(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } rows := [][]any{ @@ -1268,16 +1221,12 @@ func TestIssue36194(t *testing.T) { tk.MustExec("create table t(a int)") // create virtual tiflash replica. is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } tk.MustQuery("explain format = 'brief' select /*+ read_from_storage(tiflash[t]) */ * from t where a + 1 > 20 limit 100;;").Check(testkit.Rows( "Limit 100.00 root offset:0, count:100", @@ -1325,16 +1274,11 @@ func TestAggWithJsonPushDownToTiFlash(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } rows := [][]any{ @@ -1375,16 +1319,11 @@ func TestLeftShiftPushDownToTiFlash(t *testing.T) { // Create virtual tiflash replica info. is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } rows := [][]any{ @@ -1468,16 +1407,11 @@ func TestEltPushDownToTiFlash(t *testing.T) { // Create virtual tiflash replica info. is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } rows := [][]any{ @@ -1502,16 +1436,11 @@ func TestRegexpInstrPushDownToTiFlash(t *testing.T) { // Create virtual tiflash replica info. is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } rows := [][]any{ @@ -1536,16 +1465,11 @@ func TestRegexpSubstrPushDownToTiFlash(t *testing.T) { // Create virtual tiflash replica info. is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } rows := [][]any{ @@ -1570,16 +1494,11 @@ func TestRegexpReplacePushDownToTiFlash(t *testing.T) { // Create virtual tiflash replica info. is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } rows := [][]any{ @@ -1608,16 +1527,11 @@ func TestCastTimeAsDurationToTiFlash(t *testing.T) { // Create virtual tiflash replica info. is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } rows := [][]any{ @@ -1991,16 +1905,11 @@ func TestIsIPv4ToTiFlash(t *testing.T) { // Create virtual tiflash replica info. is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } rows := [][]any{ @@ -2027,16 +1936,11 @@ func TestIsIPv6ToTiFlash(t *testing.T) { // Create virtual tiflash replica info. is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } rows := [][]any{ @@ -2090,16 +1994,11 @@ func TestVirtualExprPushDown(t *testing.T) { tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1") tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } // TopN to tiflash. @@ -2146,18 +2045,8 @@ func TestWindowRangeFramePushDownTiflash(t *testing.T) { tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") // Create virtual tiflash replica info. - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "first_range" || tblInfo.Name.L == "first_range_d64" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + coretestsdk.SetTiFlashReplica(t, dom, "test", "first_range") + coretestsdk.SetTiFlashReplica(t, dom, "test", "first_range_d64") tk.MustExec(`set @@tidb_max_tiflash_threads=20`) diff --git a/pkg/planner/core/plan_test.go b/pkg/planner/core/plan_test.go index 3c875784a1576..2e855457bb8d6 100644 --- a/pkg/planner/core/plan_test.go +++ b/pkg/planner/core/plan_test.go @@ -16,6 +16,7 @@ package core_test import ( "bytes" + "context" "fmt" "testing" "time" @@ -369,16 +370,11 @@ func TestExplainFormatHintRecoverableForTiFlashReplica(t *testing.T) { tk.MustExec("create table t(a int)") // Create virtual `tiflash` replica info. is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tblInfo, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblInfo.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } rows := tk.MustQuery("explain select * from t").Rows() diff --git a/pkg/planner/core/runtime_filter_generator_test.go b/pkg/planner/core/runtime_filter_generator_test.go index e6f2c931f8554..96734c3345ec7 100644 --- a/pkg/planner/core/runtime_filter_generator_test.go +++ b/pkg/planner/core/runtime_filter_generator_test.go @@ -15,6 +15,7 @@ package core_test import ( + "context" "fmt" "testing" @@ -51,17 +52,17 @@ func TestRuntimeFilterGenerator(t *testing.T) { // set tiflash replica dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - tableName := tblInfo.Name.L - if tableName == "t1" || tableName == "t2" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tblInfo, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + require.NoError(t, err) + tblInfo.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + tblInfo, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t2")) + require.NoError(t, err) + tblInfo.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } // runtime filter test case diff --git a/pkg/planner/util/coretestsdk/testkit.go b/pkg/planner/util/coretestsdk/testkit.go index ac06e52eb461f..c770ca86d8f93 100644 --- a/pkg/planner/util/coretestsdk/testkit.go +++ b/pkg/planner/util/coretestsdk/testkit.go @@ -15,6 +15,7 @@ package coretestsdk import ( + "context" "fmt" "strings" "testing" @@ -31,16 +32,11 @@ import ( // SetTiFlashReplica is to set TiFlash replica func SetTiFlashReplica(t *testing.T, dom *domain.Domain, dbName, tableName string) { is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr(dbName)) - require.True(t, exists) - for _, tbl := range is.SchemaTables(db.Name) { - tblInfo := tbl.Meta() - if tblInfo.Name.L == tableName { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } + tblInfo, err := is.TableByName(context.Background(), model.NewCIStr(dbName), model.NewCIStr(tableName)) + require.NoError(t, err) + tblInfo.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, } } diff --git a/pkg/structure/hash.go b/pkg/structure/hash.go index cc988645636fd..3ad1e3a5c8342 100644 --- a/pkg/structure/hash.go +++ b/pkg/structure/hash.go @@ -143,7 +143,7 @@ func (t *TxStructure) HDel(key []byte, fields ...[]byte) error { // HKeys gets all the fields in a hash. func (t *TxStructure) HKeys(key []byte) ([][]byte, error) { var keys [][]byte - err := t.iterateHash(key, func(field []byte, _ []byte) error { + err := t.IterateHash(key, func(field []byte, _ []byte) error { keys = append(keys, append([]byte{}, field...)) return nil }) @@ -154,7 +154,7 @@ func (t *TxStructure) HKeys(key []byte) ([][]byte, error) { // HGetAll gets all the fields and values in a hash. func (t *TxStructure) HGetAll(key []byte) ([]HashPair, error) { var res []HashPair - err := t.iterateHash(key, func(field []byte, value []byte) error { + err := t.IterateHash(key, func(field []byte, value []byte) error { pair := HashPair{ Field: append([]byte{}, field...), Value: append([]byte{}, value...), @@ -168,7 +168,7 @@ func (t *TxStructure) HGetAll(key []byte) ([]HashPair, error) { // HGetIter iterates all the fields and values in hash. func (t *TxStructure) HGetIter(key []byte, fn func(pair HashPair) error) error { - return t.iterateHash(key, func(field []byte, value []byte) error { + return t.IterateHash(key, func(field []byte, value []byte) error { pair := HashPair{ Field: append([]byte{}, field...), Value: append([]byte{}, value...), @@ -181,7 +181,7 @@ func (t *TxStructure) HGetIter(key []byte, fn func(pair HashPair) error) error { // HGetLen gets the length of hash. func (t *TxStructure) HGetLen(key []byte) (uint64, error) { hashLen := 0 - err := t.iterateHash(key, func(_ []byte, _ []byte) error { + err := t.IterateHash(key, func(_ []byte, _ []byte) error { hashLen++ return nil }) @@ -208,7 +208,7 @@ func (t *TxStructure) HGetLastN(key []byte, num int) ([]HashPair, error) { // HClear removes the hash value of the key. func (t *TxStructure) HClear(key []byte) error { - err := t.iterateHash(key, func(field []byte, _ []byte) error { + err := t.IterateHash(key, func(field []byte, _ []byte) error { k := t.encodeHashDataKey(key, field) return errors.Trace(t.readWriter.Delete(k)) }) @@ -220,7 +220,8 @@ func (t *TxStructure) HClear(key []byte) error { return nil } -func (t *TxStructure) iterateHash(key []byte, fn func(k []byte, v []byte) error) error { +// IterateHash iterates all the fields and values in hash. +func (t *TxStructure) IterateHash(key []byte, fn func(k []byte, v []byte) error) error { dataPrefix := t.hashDataKeyPrefix(key) it, err := t.reader.Iter(dataPrefix, dataPrefix.PrefixNext()) if err != nil { diff --git a/pkg/table/tables/cache_test.go b/pkg/table/tables/cache_test.go index 82995b0c893d2..ab4d6396da813 100644 --- a/pkg/table/tables/cache_test.go +++ b/pkg/table/tables/cache_test.go @@ -334,6 +334,8 @@ func TestRenewLease(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk1 := testkit.NewTestKit(t, store) + se := tk.Session() tk.MustExec("create table cache_renew_t (id int)") tk.MustExec("alter table cache_renew_t cache") @@ -342,7 +344,6 @@ func TestRenewLease(t *testing.T) { var i int tk.MustExec("select * from cache_renew_t") - tk1 := testkit.NewTestKit(t, store) remote := tables.NewStateRemote(tk1.Session()) var leaseBefore uint64 for i = 0; i < 20; i++ {