diff --git a/ddl/column_modify_test.go b/ddl/column_modify_test.go index f91316d977b3a..e1a4e918153a3 100644 --- a/ddl/column_modify_test.go +++ b/ddl/column_modify_test.go @@ -885,7 +885,7 @@ func TestTransactionWithWriteOnlyColumn(t *testing.T) { dom.DDL().SetHook(hook) done := make(chan error, 1) // test transaction on add column. - go backgroundExecT(store, "alter table t1 add column c int not null", done) + go backgroundExec(store, "alter table t1 add column c int not null", done) err := <-done require.NoError(t, err) require.NoError(t, checkErr) @@ -893,7 +893,7 @@ func TestTransactionWithWriteOnlyColumn(t *testing.T) { tk.MustExec("delete from t1") // test transaction on drop column. - go backgroundExecT(store, "alter table t1 drop column c", done) + go backgroundExec(store, "alter table t1 drop column c", done) err = <-done require.NoError(t, err) require.NoError(t, checkErr) diff --git a/ddl/db_legacy_test.go b/ddl/db_legacy_test.go index dd01ef5b276d8..f487376d0b6cc 100644 --- a/ddl/db_legacy_test.go +++ b/ddl/db_legacy_test.go @@ -17,10 +17,8 @@ package ddl_test import ( "context" "fmt" - "math" "sort" "strings" - "sync" "testing" "time" @@ -35,7 +33,6 @@ import ( "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/parser/terror" parsertypes "github.com/pingcap/tidb/parser/types" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" @@ -43,7 +40,6 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/testkit" @@ -143,12 +139,7 @@ func testGetTableByName(c *C, ctx sessionctx.Context, db, table string) table.Ta return tbl } -func (s *testDBSuite) testGetTable(c *C, name string) table.Table { - ctx := s.s.(sessionctx.Context) - return testGetTableByName(c, ctx, s.schemaName, name) -} - -func backgroundExecT(s kv.Storage, sql string, done chan error) { +func backgroundExec(s kv.Storage, sql string, done chan error) { se, err := session.CreateSession4Test(s) if err != nil { done <- errors.Trace(err) @@ -178,235 +169,6 @@ func testGetIndexID(t *testing.T, ctx sessionctx.Context, dbName, tblName, idxNa return -1 } -func oldBackgroundExec(s kv.Storage, sql string, done chan error) { - se, err := session.CreateSession4Test(s) - if err != nil { - done <- errors.Trace(err) - return - } - defer se.Close() - _, err = se.Execute(context.Background(), "use test_db") - if err != nil { - done <- errors.Trace(err) - return - } - _, err = se.Execute(context.Background(), sql) - done <- errors.Trace(err) -} - -// TestCreateTableWithLike2 tests create table with like when refer table have non-public column/index. -func (s *testSerialDBSuite) TestCreateTableWithLike2(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test_db") - tk.MustExec("drop table if exists t1,t2;") - defer tk.MustExec("drop table if exists t1,t2;") - tk.MustExec("create table t1 (a int, b int, c int, index idx1(c));") - - tbl1 := testGetTableByName(c, s.s, "test_db", "t1") - doneCh := make(chan error, 2) - hook := &ddl.TestDDLCallback{Do: s.dom} - var onceChecker sync.Map - hook.OnJobRunBeforeExported = func(job *model.Job) { - if job.Type != model.ActionAddColumn && job.Type != model.ActionDropColumn && - job.Type != model.ActionAddColumns && job.Type != model.ActionDropColumns && - job.Type != model.ActionAddIndex && job.Type != model.ActionDropIndex { - return - } - if job.TableID != tbl1.Meta().ID { - return - } - - if job.SchemaState == model.StateDeleteOnly { - if _, ok := onceChecker.Load(job.ID); ok { - return - } - - onceChecker.Store(job.ID, true) - go oldBackgroundExec(s.store, "create table t2 like t1", doneCh) - } - } - originalHook := s.dom.DDL().GetHook() - defer s.dom.DDL().SetHook(originalHook) - s.dom.DDL().SetHook(hook) - - // create table when refer table add column - tk.MustExec("alter table t1 add column d int") - checkTbl2 := func() { - err := <-doneCh - c.Assert(err, IsNil) - tk.MustExec("alter table t2 add column e int") - t2Info := testGetTableByName(c, s.s, "test_db", "t2") - c.Assert(len(t2Info.Meta().Columns), Equals, len(t2Info.Cols())) - } - checkTbl2() - - // create table when refer table drop column - tk.MustExec("drop table t2;") - tk.MustExec("alter table t1 drop column b;") - checkTbl2() - - // create table when refer table add index - tk.MustExec("drop table t2;") - tk.MustExec("alter table t1 add index idx2(a);") - checkTbl2 = func() { - err := <-doneCh - c.Assert(err, IsNil) - tk.MustExec("alter table t2 add column e int") - tbl2 := testGetTableByName(c, s.s, "test_db", "t2") - c.Assert(len(tbl2.Meta().Columns), Equals, len(tbl2.Cols())) - - for i := 0; i < len(tbl2.Meta().Indices); i++ { - c.Assert(tbl2.Meta().Indices[i].State, Equals, model.StatePublic) - } - } - checkTbl2() - - // create table when refer table drop index. - tk.MustExec("drop table t2;") - tk.MustExec("alter table t1 drop index idx2;") - checkTbl2() - - // Test for table has tiflash replica. - c.Assert(failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`), IsNil) - defer func() { - err := failpoint.Disable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount") - c.Assert(err, IsNil) - }() - - s.dom.DDL().SetHook(originalHook) - tk.MustExec("drop table if exists t1,t2;") - tk.MustExec("create table t1 (a int) partition by hash(a) partitions 2;") - tk.MustExec("alter table t1 set tiflash replica 3 location labels 'a','b';") - t1 := testGetTableByName(c, s.s, "test_db", "t1") - // Mock for all partitions replica was available. - partition := t1.Meta().Partition - c.Assert(len(partition.Definitions), Equals, 2) - err := domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, partition.Definitions[0].ID, true) - c.Assert(err, IsNil) - err = domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, partition.Definitions[1].ID, true) - c.Assert(err, IsNil) - t1 = testGetTableByName(c, s.s, "test_db", "t1") - c.Assert(t1.Meta().TiFlashReplica, NotNil) - c.Assert(t1.Meta().TiFlashReplica.Available, IsTrue) - c.Assert(t1.Meta().TiFlashReplica.AvailablePartitionIDs, DeepEquals, []int64{partition.Definitions[0].ID, partition.Definitions[1].ID}) - - tk.MustExec("create table t2 like t1") - t2 := testGetTableByName(c, s.s, "test_db", "t2") - c.Assert(t2.Meta().TiFlashReplica.Count, Equals, t1.Meta().TiFlashReplica.Count) - c.Assert(t2.Meta().TiFlashReplica.LocationLabels, DeepEquals, t1.Meta().TiFlashReplica.LocationLabels) - c.Assert(t2.Meta().TiFlashReplica.Available, IsFalse) - c.Assert(t2.Meta().TiFlashReplica.AvailablePartitionIDs, HasLen, 0) - // Test for not affecting the original table. - t1 = testGetTableByName(c, s.s, "test_db", "t1") - c.Assert(t1.Meta().TiFlashReplica, NotNil) - c.Assert(t1.Meta().TiFlashReplica.Available, IsTrue) - c.Assert(t1.Meta().TiFlashReplica.AvailablePartitionIDs, DeepEquals, []int64{partition.Definitions[0].ID, partition.Definitions[1].ID}) -} - -func (s *testSerialDBSuite) TestTruncateTable(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("create table truncate_table (c1 int, c2 int)") - tk.MustExec("insert truncate_table values (1, 1), (2, 2)") - ctx := tk.Se.(sessionctx.Context) - is := domain.GetDomain(ctx).InfoSchema() - oldTblInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("truncate_table")) - c.Assert(err, IsNil) - oldTblID := oldTblInfo.Meta().ID - - tk.MustExec("truncate table truncate_table") - - tk.MustExec("insert truncate_table values (3, 3), (4, 4)") - tk.MustQuery("select * from truncate_table").Check(testkit.Rows("3 3", "4 4")) - - is = domain.GetDomain(ctx).InfoSchema() - newTblInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("truncate_table")) - c.Assert(err, IsNil) - c.Assert(newTblInfo.Meta().ID, Greater, oldTblID) - - // Verify that the old table data has been deleted by background worker. - tablePrefix := tablecodec.EncodeTablePrefix(oldTblID) - hasOldTableData := true - for i := 0; i < waitForCleanDataRound; i++ { - err = kv.RunInNewTxn(context.Background(), s.store, false, func(ctx context.Context, txn kv.Transaction) error { - it, err1 := txn.Iter(tablePrefix, nil) - if err1 != nil { - return err1 - } - if !it.Valid() { - hasOldTableData = false - } else { - hasOldTableData = it.Key().HasPrefix(tablePrefix) - } - it.Close() - return nil - }) - c.Assert(err, IsNil) - if !hasOldTableData { - break - } - time.Sleep(waitForCleanDataInterval) - } - c.Assert(hasOldTableData, IsFalse) - - // Test for truncate table should clear the tiflash available status. - c.Assert(failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`), IsNil) - defer func() { - err = failpoint.Disable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount") - c.Assert(err, IsNil) - }() - - tk.MustExec("drop table if exists t1;") - tk.MustExec("create table t1 (a int);") - tk.MustExec("alter table t1 set tiflash replica 3 location labels 'a','b';") - t1 := testGetTableByName(c, s.s, "test", "t1") - // Mock for table tiflash replica was available. - err = domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, t1.Meta().ID, true) - c.Assert(err, IsNil) - t1 = testGetTableByName(c, s.s, "test", "t1") - c.Assert(t1.Meta().TiFlashReplica, NotNil) - c.Assert(t1.Meta().TiFlashReplica.Available, IsTrue) - - tk.MustExec("truncate table t1") - t2 := testGetTableByName(c, s.s, "test", "t1") - c.Assert(t2.Meta().TiFlashReplica.Count, Equals, t1.Meta().TiFlashReplica.Count) - c.Assert(t2.Meta().TiFlashReplica.LocationLabels, DeepEquals, t1.Meta().TiFlashReplica.LocationLabels) - c.Assert(t2.Meta().TiFlashReplica.Available, IsFalse) - c.Assert(t2.Meta().TiFlashReplica.AvailablePartitionIDs, HasLen, 0) - - // Test for truncate partition should clear the tiflash available status. - tk.MustExec("drop table if exists t1;") - tk.MustExec("create table t1 (a int) partition by hash(a) partitions 2;") - tk.MustExec("alter table t1 set tiflash replica 3 location labels 'a','b';") - t1 = testGetTableByName(c, s.s, "test", "t1") - // Mock for all partitions replica was available. - partition := t1.Meta().Partition - c.Assert(len(partition.Definitions), Equals, 2) - err = domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, partition.Definitions[0].ID, true) - c.Assert(err, IsNil) - err = domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, partition.Definitions[1].ID, true) - c.Assert(err, IsNil) - t1 = testGetTableByName(c, s.s, "test", "t1") - c.Assert(t1.Meta().TiFlashReplica, NotNil) - c.Assert(t1.Meta().TiFlashReplica.Available, IsTrue) - c.Assert(t1.Meta().TiFlashReplica.AvailablePartitionIDs, DeepEquals, []int64{partition.Definitions[0].ID, partition.Definitions[1].ID}) - - tk.MustExec("alter table t1 truncate partition p0") - t2 = testGetTableByName(c, s.s, "test", "t1") - c.Assert(t2.Meta().TiFlashReplica.Count, Equals, t1.Meta().TiFlashReplica.Count) - c.Assert(t2.Meta().TiFlashReplica.LocationLabels, DeepEquals, t1.Meta().TiFlashReplica.LocationLabels) - c.Assert(t2.Meta().TiFlashReplica.Available, IsFalse) - c.Assert(t2.Meta().TiFlashReplica.AvailablePartitionIDs, DeepEquals, []int64{partition.Definitions[1].ID}) - // Test for truncate twice. - tk.MustExec("alter table t1 truncate partition p0") - t2 = testGetTableByName(c, s.s, "test", "t1") - c.Assert(t2.Meta().TiFlashReplica.Count, Equals, t1.Meta().TiFlashReplica.Count) - c.Assert(t2.Meta().TiFlashReplica.LocationLabels, DeepEquals, t1.Meta().TiFlashReplica.LocationLabels) - c.Assert(t2.Meta().TiFlashReplica.Available, IsFalse) - c.Assert(t2.Meta().TiFlashReplica.AvailablePartitionIDs, DeepEquals, []int64{partition.Definitions[1].ID}) - -} - func (s *testSerialDBSuite) TestRebaseAutoID(c *C) { c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil) defer func() { @@ -477,109 +239,6 @@ func (s *testSerialDBSuite) TestProcessColumnFlags(c *C) { }) } -func (s *testSerialDBSuite) TestSetTableFlashReplica(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`), IsNil) - - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test_db") - tk.MustExec("drop table if exists t_flash;") - tk.MustExec("create table t_flash(a int, b int)") - defer tk.MustExec("drop table t_flash;") - - t := s.testGetTable(c, "t_flash") - c.Assert(t.Meta().TiFlashReplica, IsNil) - - tk.MustExec("alter table t_flash set tiflash replica 2 location labels 'a','b';") - t = s.testGetTable(c, "t_flash") - c.Assert(t.Meta().TiFlashReplica, NotNil) - c.Assert(t.Meta().TiFlashReplica.Count, Equals, uint64(2)) - c.Assert(strings.Join(t.Meta().TiFlashReplica.LocationLabels, ","), Equals, "a,b") - - tk.MustExec("alter table t_flash set tiflash replica 0") - t = s.testGetTable(c, "t_flash") - c.Assert(t.Meta().TiFlashReplica, IsNil) - - // Test set tiflash replica for partition table. - tk.MustExec("drop table if exists t_flash;") - tk.MustExec("create table t_flash(a int, b int) partition by hash(a) partitions 3") - tk.MustExec("alter table t_flash set tiflash replica 2 location labels 'a','b';") - t = s.testGetTable(c, "t_flash") - c.Assert(t.Meta().TiFlashReplica, NotNil) - c.Assert(t.Meta().TiFlashReplica.Count, Equals, uint64(2)) - c.Assert(strings.Join(t.Meta().TiFlashReplica.LocationLabels, ","), Equals, "a,b") - - // Use table ID as physical ID, mock for partition feature was not enabled. - err := domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, t.Meta().ID, true) - c.Assert(err, IsNil) - t = s.testGetTable(c, "t_flash") - c.Assert(t.Meta().TiFlashReplica, NotNil) - c.Assert(t.Meta().TiFlashReplica.Available, Equals, true) - c.Assert(len(t.Meta().TiFlashReplica.AvailablePartitionIDs), Equals, 0) - - err = domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, t.Meta().ID, false) - c.Assert(err, IsNil) - t = s.testGetTable(c, "t_flash") - c.Assert(t.Meta().TiFlashReplica.Available, Equals, false) - - // Mock for partition 0 replica was available. - partition := t.Meta().Partition - c.Assert(len(partition.Definitions), Equals, 3) - err = domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, partition.Definitions[0].ID, true) - c.Assert(err, IsNil) - t = s.testGetTable(c, "t_flash") - c.Assert(t.Meta().TiFlashReplica.Available, Equals, false) - c.Assert(t.Meta().TiFlashReplica.AvailablePartitionIDs, DeepEquals, []int64{partition.Definitions[0].ID}) - - // Mock for partition 0 replica become unavailable. - err = domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, partition.Definitions[0].ID, false) - c.Assert(err, IsNil) - t = s.testGetTable(c, "t_flash") - c.Assert(t.Meta().TiFlashReplica.Available, Equals, false) - c.Assert(t.Meta().TiFlashReplica.AvailablePartitionIDs, HasLen, 0) - - // Mock for partition 0, 1,2 replica was available. - err = domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, partition.Definitions[0].ID, true) - c.Assert(err, IsNil) - err = domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, partition.Definitions[1].ID, true) - c.Assert(err, IsNil) - err = domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, partition.Definitions[2].ID, true) - c.Assert(err, IsNil) - t = s.testGetTable(c, "t_flash") - c.Assert(t.Meta().TiFlashReplica.Available, Equals, true) - c.Assert(t.Meta().TiFlashReplica.AvailablePartitionIDs, DeepEquals, []int64{partition.Definitions[0].ID, partition.Definitions[1].ID, partition.Definitions[2].ID}) - - // Mock for partition 1 replica was unavailable. - err = domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, partition.Definitions[1].ID, false) - c.Assert(err, IsNil) - t = s.testGetTable(c, "t_flash") - c.Assert(t.Meta().TiFlashReplica.Available, Equals, false) - c.Assert(t.Meta().TiFlashReplica.AvailablePartitionIDs, DeepEquals, []int64{partition.Definitions[0].ID, partition.Definitions[2].ID}) - - // Test for update table replica with unknown table ID. - err = domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, math.MaxInt64, false) - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[schema:1146]Table which ID = 9223372036854775807 does not exist.") - - // Test for FindTableByPartitionID. - is := domain.GetDomain(tk.Se).InfoSchema() - t, dbInfo, _ := is.FindTableByPartitionID(partition.Definitions[0].ID) - c.Assert(t, NotNil) - c.Assert(dbInfo, NotNil) - c.Assert(t.Meta().Name.L, Equals, "t_flash") - t, dbInfo, _ = is.FindTableByPartitionID(t.Meta().ID) - c.Assert(t, IsNil) - c.Assert(dbInfo, IsNil) - err = failpoint.Disable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount") - c.Assert(err, IsNil) - - // Test for set replica count more than the tiflash store count. - tk.MustExec("drop table if exists t_flash;") - tk.MustExec("create table t_flash(a int, b int)") - _, err = tk.Exec("alter table t_flash set tiflash replica 2 location labels 'a','b';") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "the tiflash replica count: 2 should be less than the total tiflash server count: 0") -} - func (s *testSerialDBSuite) TestForbitCacheTableForSystemTable(c *C) { tk := testkit.NewTestKit(c, s.store) sysTables := make([]string, 0, 24) @@ -604,59 +263,6 @@ func (s *testSerialDBSuite) TestForbitCacheTableForSystemTable(c *C) { } } -func (s *testSerialDBSuite) TestSetTableFlashReplicaForSystemTable(c *C) { - tk := testkit.NewTestKit(c, s.store) - sysTables := make([]string, 0, 24) - memOrSysDB := []string{"MySQL", "INFORMATION_SCHEMA", "PERFORMANCE_SCHEMA", "METRICS_SCHEMA"} - for _, db := range memOrSysDB { - tk.MustExec("use " + db) - tk.Se.Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil) - rows := tk.MustQuery("show tables").Rows() - for i := 0; i < len(rows); i++ { - sysTables = append(sysTables, rows[i][0].(string)) - } - for _, one := range sysTables { - _, err := tk.Exec(fmt.Sprintf("alter table `%s` set tiflash replica 1", one)) - if db == "MySQL" { - c.Assert(err.Error(), Equals, "[ddl:8200]ALTER table replica for tables in system database is currently unsupported") - } else { - c.Assert(err.Error(), Equals, fmt.Sprintf("[planner:1142]ALTER command denied to user 'root'@'%%' for table '%s'", strings.ToLower(one))) - } - - } - sysTables = sysTables[:0] - } -} - -func (s *testSerialDBSuite) TestSetTiFlashReplicaForTemporaryTable(c *C) { - // test for tiflash replica - c.Assert(failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`), IsNil) - defer func() { - err := failpoint.Disable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount") - c.Assert(err, IsNil) - }() - - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("drop table if exists temp, temp2") - tk.MustExec("drop table if exists temp") - tk.MustExec("create global temporary table temp(id int) on commit delete rows") - tk.MustExec("create temporary table temp2(id int)") - tk.MustGetErrCode("alter table temp set tiflash replica 1", errno.ErrOptOnTemporaryTable) - tk.MustGetErrCode("alter table temp2 set tiflash replica 1", errno.ErrUnsupportedDDLOperation) - tk.MustExec("drop table temp, temp2") - - tk.MustExec("drop table if exists normal") - tk.MustExec("create table normal(id int)") - defer tk.MustExec("drop table normal") - tk.MustExec("alter table normal set tiflash replica 1") - tk.MustQuery("select REPLICA_COUNT from information_schema.tiflash_replica where table_schema='test' and table_name='normal'").Check(testkit.Rows("1")) - tk.MustExec("create global temporary table temp like normal on commit delete rows") - tk.MustQuery("select REPLICA_COUNT from information_schema.tiflash_replica where table_schema='test' and table_name='temp'").Check(testkit.Rows()) - tk.MustExec("drop table temp") - tk.MustExec("create temporary table temp like normal") - tk.MustQuery("select REPLICA_COUNT from information_schema.tiflash_replica where table_schema='test' and table_name='temp'").Check(testkit.Rows()) -} - func (s *testSerialDBSuite) TestAlterShardRowIDBits(c *C) { c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil) defer func() { @@ -719,43 +325,6 @@ func (s *testSerialDBSuite) TestShardRowIDBitsOnTemporaryTable(c *C) { c.Assert(err.Error(), Equals, dbterror.ErrUnsupportedLocalTempTableDDL.GenWithStackByArgs("ALTER TABLE").Error()) } -func (s *testSerialDBSuite) TestSkipSchemaChecker(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`), IsNil) - defer func() { - err := failpoint.Disable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount") - c.Assert(err, IsNil) - }() - - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t1") - defer tk.MustExec("drop table if exists t1") - tk.MustExec("create table t1 (a int)") - tk2 := testkit.NewTestKit(c, s.store) - tk2.MustExec("use test") - - // Test skip schema checker for ActionSetTiFlashReplica. - tk.MustExec("begin") - tk.MustExec("insert into t1 set a=1;") - tk2.MustExec("alter table t1 set tiflash replica 2 location labels 'a','b';") - tk.MustExec("commit") - - // Test skip schema checker for ActionUpdateTiFlashReplicaStatus. - tk.MustExec("begin") - tk.MustExec("insert into t1 set a=1;") - tb := testGetTableByName(c, tk.Se, "test", "t1") - err := domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, tb.Meta().ID, true) - c.Assert(err, IsNil) - tk.MustExec("commit") - - // Test can't skip schema checker. - tk.MustExec("begin") - tk.MustExec("insert into t1 set a=1;") - tk2.MustExec("alter table t1 add column b int;") - _, err = tk.Exec("commit") - c.Assert(terror.ErrorEqual(domain.ErrInfoSchemaChanged, err), IsTrue) -} - func (s *testSerialDBSuite) TestDDLJobErrorCount(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index b526846db027f..51a61d3b7de0a 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -2831,7 +2831,7 @@ func testPartitionCancelAddIndex(t *testing.T, store kv.Storage, d ddl.DDL, leas jobIDExt := wrapJobIDExtCallback(hook) d.SetHook(jobIDExt) done := make(chan error, 1) - go backgroundExecT(store, addIdxSQL, done) + go backgroundExec(store, addIdxSQL, done) times := 0 ticker := time.NewTicker(lease / 2) @@ -3619,9 +3619,9 @@ func TestTruncatePartitionMultipleTimes(t *testing.T) { } } done1 := make(chan error, 1) - go backgroundExecT(store, "alter table test.t truncate partition p0;", done1) + go backgroundExec(store, "alter table test.t truncate partition p0;", done1) done2 := make(chan error, 1) - go backgroundExecT(store, "alter table test.t truncate partition p0;", done2) + go backgroundExec(store, "alter table test.t truncate partition p0;", done2) <-done1 <-done2 require.LessOrEqual(t, errCount, int32(1)) diff --git a/ddl/db_table_test.go b/ddl/db_table_test.go index 78cb31ce778f1..13a6113ad6830 100644 --- a/ddl/db_table_test.go +++ b/ddl/db_table_test.go @@ -308,7 +308,7 @@ func TestTransactionOnAddDropColumn(t *testing.T) { dom.DDL().SetHook(hook) done := make(chan error, 1) // test transaction on add column. - go backgroundExecT(store, "alter table t1 add column c int not null after a", done) + go backgroundExec(store, "alter table t1 add column c int not null after a", done) err := <-done require.NoError(t, err) require.Nil(t, checkErr) @@ -316,7 +316,7 @@ func TestTransactionOnAddDropColumn(t *testing.T) { tk.MustExec("delete from t1") // test transaction on drop column. - go backgroundExecT(store, "alter table t1 drop column c", done) + go backgroundExec(store, "alter table t1 drop column c", done) err = <-done require.NoError(t, err) require.Nil(t, checkErr) @@ -1052,7 +1052,7 @@ func TestAddColumn2(t *testing.T) { dom.DDL().SetHook(hook) done := make(chan error, 1) // test transaction on add column. - go backgroundExecT(store, "alter table t1 add column c int not null", done) + go backgroundExec(store, "alter table t1 add column c int not null", done) err := <-done require.NoError(t, err) @@ -1094,7 +1094,7 @@ func TestAddColumn2(t *testing.T) { } dom.DDL().SetHook(hook) - go backgroundExecT(store, "alter table t2 add column b int not null default 3", done) + go backgroundExec(store, "alter table t2 add column b int not null default 3", done) err = <-done require.NoError(t, err) re.Check(testkit.Rows("1 2")) diff --git a/ddl/db_test.go b/ddl/db_test.go index df2a296ee969c..dece466a9b12b 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -264,7 +264,7 @@ func TestIssue22307(t *testing.T) { dom.DDL().SetHook(hook) done := make(chan error, 1) // test transaction on add column. - go backgroundExecT(store, "alter table t drop column b;", done) + go backgroundExec(store, "alter table t drop column b;", done) err := <-done require.NoError(t, err) require.EqualError(t, checkErr1, "[planner:1054]Unknown column 'b' in 'where clause'") diff --git a/ddl/index_modify_test.go b/ddl/index_modify_test.go index 688be10e06b2f..b322e28999de5 100644 --- a/ddl/index_modify_test.go +++ b/ddl/index_modify_test.go @@ -410,7 +410,7 @@ func testAddIndexRollback(t *testing.T, idxName, addIdxSQL, errMsg string, hasNu } done := make(chan error, 1) - go backgroundExecT(store, addIdxSQL, done) + go backgroundExec(store, addIdxSQL, done) times := 0 ticker := time.NewTicker(indexModifyLease / 2) @@ -730,7 +730,7 @@ func testCancelAddIndex(t *testing.T, store kv.Storage, dom *domain.Domain, idxN jobIDExt := wrapJobIDExtCallback(hook) d.SetHook(jobIDExt) done := make(chan error, 1) - go backgroundExecT(store, addIdxSQL, done) + go backgroundExec(store, addIdxSQL, done) times := 0 ticker := time.NewTicker(indexModifyLease / 2) diff --git a/ddl/modify_column_test.go b/ddl/modify_column_test.go index 7280321ab9b02..8c68b74b25aad 100644 --- a/ddl/modify_column_test.go +++ b/ddl/modify_column_test.go @@ -972,7 +972,7 @@ func TestModifyColumnRollBack(t *testing.T) { dom.DDL().SetHook(hook) done := make(chan error, 1) - go backgroundExecT(store, "alter table test.t1 change c2 c2 bigint not null;", done) + go backgroundExec(store, "alter table test.t1 change c2 c2 bigint not null;", done) err := <-done require.EqualError(t, err, "[ddl:8214]Cancelled DDL job") diff --git a/ddl/tiflash_replica_test.go b/ddl/tiflash_replica_test.go new file mode 100644 index 0000000000000..f0ccad6e73aa9 --- /dev/null +++ b/ddl/tiflash_replica_test.go @@ -0,0 +1,447 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package ddl_test + +import ( + "context" + "fmt" + "math" + "strings" + "sync" + "testing" + "time" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/errno" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/auth" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/terror" + "github.com/pingcap/tidb/tablecodec" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/external" + "github.com/stretchr/testify/require" +) + +const tiflashReplicaLease = 600 * time.Millisecond + +func TestSetTableFlashReplica(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`)) + store, clean := testkit.CreateMockStoreWithSchemaLease(t, tiflashReplicaLease) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t_flash(a int, b int)") + + tbl := external.GetTableByName(t, tk, "test", "t_flash") + require.Nil(t, tbl.Meta().TiFlashReplica) + + tk.MustExec("alter table t_flash set tiflash replica 2 location labels 'a','b';") + tbl = external.GetTableByName(t, tk, "test", "t_flash") + require.NotNil(t, tbl.Meta().TiFlashReplica) + require.Equal(t, uint64(2), tbl.Meta().TiFlashReplica.Count) + require.Equal(t, "a,b", strings.Join(tbl.Meta().TiFlashReplica.LocationLabels, ",")) + + tk.MustExec("alter table t_flash set tiflash replica 0") + tbl = external.GetTableByName(t, tk, "test", "t_flash") + require.Nil(t, tbl.Meta().TiFlashReplica) + + // Test set tiflash replica for partition table. + tk.MustExec("drop table if exists t_flash;") + tk.MustExec("create table t_flash(a int, b int) partition by hash(a) partitions 3") + tk.MustExec("alter table t_flash set tiflash replica 2 location labels 'a','b';") + tbl = external.GetTableByName(t, tk, "test", "t_flash") + require.NotNil(t, tbl.Meta().TiFlashReplica) + require.Equal(t, uint64(2), tbl.Meta().TiFlashReplica.Count) + require.Equal(t, "a,b", strings.Join(tbl.Meta().TiFlashReplica.LocationLabels, ",")) + + // Use table ID as physical ID, mock for partition feature was not enabled. + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tbl.Meta().ID, true) + require.NoError(t, err) + tbl = external.GetTableByName(t, tk, "test", "t_flash") + require.NotNil(t, tbl.Meta().TiFlashReplica) + require.True(t, tbl.Meta().TiFlashReplica.Available) + require.Len(t, tbl.Meta().TiFlashReplica.AvailablePartitionIDs, 0) + + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tbl.Meta().ID, false) + require.NoError(t, err) + tbl = external.GetTableByName(t, tk, "test", "t_flash") + require.False(t, tbl.Meta().TiFlashReplica.Available) + + // Mock for partition 0 replica was available. + partition := tbl.Meta().Partition + require.Len(t, partition.Definitions, 3) + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), partition.Definitions[0].ID, true) + require.NoError(t, err) + tbl = external.GetTableByName(t, tk, "test", "t_flash") + require.False(t, tbl.Meta().TiFlashReplica.Available) + require.Equal(t, []int64{partition.Definitions[0].ID}, tbl.Meta().TiFlashReplica.AvailablePartitionIDs) + + // Mock for partition 0 replica become unavailable. + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), partition.Definitions[0].ID, false) + require.NoError(t, err) + tbl = external.GetTableByName(t, tk, "test", "t_flash") + require.False(t, tbl.Meta().TiFlashReplica.Available) + require.Len(t, tbl.Meta().TiFlashReplica.AvailablePartitionIDs, 0) + + // Mock for partition 0, 1,2 replica was available. + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), partition.Definitions[0].ID, true) + require.NoError(t, err) + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), partition.Definitions[1].ID, true) + require.NoError(t, err) + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), partition.Definitions[2].ID, true) + require.NoError(t, err) + tbl = external.GetTableByName(t, tk, "test", "t_flash") + require.True(t, tbl.Meta().TiFlashReplica.Available) + require.Equal(t, []int64{partition.Definitions[0].ID, partition.Definitions[1].ID, partition.Definitions[2].ID}, tbl.Meta().TiFlashReplica.AvailablePartitionIDs) + + // Mock for partition 1 replica was unavailable. + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), partition.Definitions[1].ID, false) + require.NoError(t, err) + tbl = external.GetTableByName(t, tk, "test", "t_flash") + require.Equal(t, false, tbl.Meta().TiFlashReplica.Available) + require.Equal(t, []int64{partition.Definitions[0].ID, partition.Definitions[2].ID}, tbl.Meta().TiFlashReplica.AvailablePartitionIDs) + + // Test for update table replica with unknown table ID. + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), math.MaxInt64, false) + require.EqualError(t, err, "[schema:1146]Table which ID = 9223372036854775807 does not exist.") + + // Test for FindTableByPartitionID. + is := domain.GetDomain(tk.Session()).InfoSchema() + tbl, dbInfo, _ := is.FindTableByPartitionID(partition.Definitions[0].ID) + require.NotNil(t, tbl) + require.NotNil(t, dbInfo) + require.Equal(t, "t_flash", tbl.Meta().Name.L) + tbl, dbInfo, _ = is.FindTableByPartitionID(tbl.Meta().ID) + require.Nil(t, tbl) + require.Nil(t, dbInfo) + err = failpoint.Disable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount") + require.NoError(t, err) + + // Test for set replica count more than the tiflash store count. + tk.MustExec("drop table if exists t_flash;") + tk.MustExec("create table t_flash(a int, b int)") + tk.MustGetErrMsg("alter table t_flash set tiflash replica 2 location labels 'a','b';", "the tiflash replica count: 2 should be less than the total tiflash server count: 0") +} + +func TestSetTiFlashReplicaForTemporaryTable(t *testing.T) { + // test for tiflash replica + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount")) + }() + + store, clean := testkit.CreateMockStoreWithSchemaLease(t, tiflashReplicaLease) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create global temporary table temp(id int) on commit delete rows") + tk.MustExec("create temporary table temp2(id int)") + tk.MustGetErrCode("alter table temp set tiflash replica 1", errno.ErrOptOnTemporaryTable) + tk.MustGetErrCode("alter table temp2 set tiflash replica 1", errno.ErrUnsupportedDDLOperation) + tk.MustExec("drop table temp, temp2") + + tk.MustExec("drop table if exists normal") + tk.MustExec("create table normal(id int)") + tk.MustExec("alter table normal set tiflash replica 1") + tk.MustQuery("select REPLICA_COUNT from information_schema.tiflash_replica where table_schema='test' and table_name='normal'").Check(testkit.Rows("1")) + tk.MustExec("create global temporary table temp like normal on commit delete rows") + tk.MustQuery("select REPLICA_COUNT from information_schema.tiflash_replica where table_schema='test' and table_name='temp'").Check(testkit.Rows()) + tk.MustExec("drop table temp") + tk.MustExec("create temporary table temp like normal") + tk.MustQuery("select REPLICA_COUNT from information_schema.tiflash_replica where table_schema='test' and table_name='temp'").Check(testkit.Rows()) +} + +func TestSetTableFlashReplicaForSystemTable(t *testing.T) { + store, clean := testkit.CreateMockStoreWithSchemaLease(t, tiflashReplicaLease) + defer clean() + + tk := testkit.NewTestKit(t, store) + sysTables := make([]string, 0, 24) + memOrSysDB := []string{"MySQL", "INFORMATION_SCHEMA", "PERFORMANCE_SCHEMA", "METRICS_SCHEMA"} + for _, db := range memOrSysDB { + tk.MustExec("use " + db) + tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil) + rows := tk.MustQuery("show tables").Rows() + for i := 0; i < len(rows); i++ { + sysTables = append(sysTables, rows[i][0].(string)) + } + for _, one := range sysTables { + _, err := tk.Exec(fmt.Sprintf("alter table `%s` set tiflash replica 1", one)) + if db == "MySQL" { + require.Equal(t, "[ddl:8200]ALTER table replica for tables in system database is currently unsupported", err.Error()) + } else { + require.Equal(t, fmt.Sprintf("[planner:1142]ALTER command denied to user 'root'@'%%' for table '%s'", strings.ToLower(one)), err.Error()) + } + + } + sysTables = sysTables[:0] + } +} + +func TestSkipSchemaChecker(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`)) + defer func() { + err := failpoint.Disable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount") + require.NoError(t, err) + }() + + store, clean := testkit.CreateMockStoreWithSchemaLease(t, tiflashReplicaLease) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + 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. + tk.MustExec("begin") + tk.MustExec("insert into t1 set a=1;") + tk2.MustExec("alter table t1 set tiflash replica 2 location labels 'a','b';") + tk.MustExec("commit") + + // Test skip schema checker for ActionUpdateTiFlashReplicaStatus. + tk.MustExec("begin") + tk.MustExec("insert into t1 set a=1;") + tb := external.GetTableByName(t, tk, "test", "t1") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("commit") + + // Test can't skip schema checker. + tk.MustExec("begin") + tk.MustExec("insert into t1 set a=1;") + tk2.MustExec("alter table t1 add column b int;") + err = tk.ExecToErr("commit") + require.True(t, terror.ErrorEqual(domain.ErrInfoSchemaChanged, err)) +} + +// TestCreateTableWithLike2 tests create table with like when refer table have non-public column/index. +func TestCreateTableWithLike2(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomainWithSchemaLease(t, tiflashReplicaLease) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t1 (a int, b int, c int, index idx1(c));") + + tbl1 := external.GetTableByName(t, tk, "test", "t1") + doneCh := make(chan error, 2) + hook := &ddl.TestDDLCallback{Do: dom} + var onceChecker sync.Map + hook.OnJobRunBeforeExported = func(job *model.Job) { + if job.Type != model.ActionAddColumn && job.Type != model.ActionDropColumn && + job.Type != model.ActionAddColumns && job.Type != model.ActionDropColumns && + job.Type != model.ActionAddIndex && job.Type != model.ActionDropIndex { + return + } + if job.TableID != tbl1.Meta().ID { + return + } + + if job.SchemaState == model.StateDeleteOnly { + if _, ok := onceChecker.Load(job.ID); ok { + return + } + + onceChecker.Store(job.ID, true) + go backgroundExec(store, "create table t2 like t1", doneCh) + } + } + originalHook := dom.DDL().GetHook() + defer dom.DDL().SetHook(originalHook) + dom.DDL().SetHook(hook) + + // create table when refer table add column + tk.MustExec("alter table t1 add column d int") + checkTbl2 := func() { + err := <-doneCh + require.NoError(t, err) + tk.MustExec("alter table t2 add column e int") + t2Info := external.GetTableByName(t, tk, "test", "t2") + require.Equal(t, len(t2Info.Cols()), len(t2Info.Meta().Columns)) + } + checkTbl2() + + // create table when refer table drop column + tk.MustExec("drop table t2;") + tk.MustExec("alter table t1 drop column b;") + checkTbl2() + + // create table when refer table add index + tk.MustExec("drop table t2;") + tk.MustExec("alter table t1 add index idx2(a);") + checkTbl2 = func() { + err := <-doneCh + require.NoError(t, err) + tk.MustExec("alter table t2 add column e int") + tbl2 := external.GetTableByName(t, tk, "test", "t2") + require.Equal(t, len(tbl2.Cols()), len(tbl2.Meta().Columns)) + + for i := 0; i < len(tbl2.Meta().Indices); i++ { + require.Equal(t, model.StatePublic, tbl2.Meta().Indices[i].State) + } + } + checkTbl2() + + // create table when refer table drop index. + tk.MustExec("drop table t2;") + tk.MustExec("alter table t1 drop index idx2;") + checkTbl2() + + // Test for table has tiflash replica. + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`)) + defer func() { + err := failpoint.Disable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount") + require.NoError(t, err) + }() + + dom.DDL().SetHook(originalHook) + tk.MustExec("drop table if exists t1,t2;") + tk.MustExec("create table t1 (a int) partition by hash(a) partitions 2;") + tk.MustExec("alter table t1 set tiflash replica 3 location labels 'a','b';") + t1 := external.GetTableByName(t, tk, "test", "t1") + // Mock for all partitions replica was available. + partition := t1.Meta().Partition + require.Equal(t, 2, len(partition.Definitions)) + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), partition.Definitions[0].ID, true) + require.NoError(t, err) + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), partition.Definitions[1].ID, true) + require.NoError(t, err) + t1 = external.GetTableByName(t, tk, "test", "t1") + require.NotNil(t, t1.Meta().TiFlashReplica) + require.True(t, t1.Meta().TiFlashReplica.Available) + require.Equal(t, []int64{partition.Definitions[0].ID, partition.Definitions[1].ID}, t1.Meta().TiFlashReplica.AvailablePartitionIDs) + + tk.MustExec("create table t2 like t1") + t2 := external.GetTableByName(t, tk, "test", "t2") + require.Equal(t, t1.Meta().TiFlashReplica.Count, t2.Meta().TiFlashReplica.Count) + require.Equal(t, t1.Meta().TiFlashReplica.LocationLabels, t2.Meta().TiFlashReplica.LocationLabels) + require.False(t, t2.Meta().TiFlashReplica.Available) + require.Len(t, t2.Meta().TiFlashReplica.AvailablePartitionIDs, 0) + // Test for not affecting the original table. + t1 = external.GetTableByName(t, tk, "test", "t1") + require.NotNil(t, t1.Meta().TiFlashReplica) + require.True(t, t1.Meta().TiFlashReplica.Available) + require.Equal(t, []int64{partition.Definitions[0].ID, partition.Definitions[1].ID}, t1.Meta().TiFlashReplica.AvailablePartitionIDs) +} + +func TestTruncateTable2(t *testing.T) { + store, clean := testkit.CreateMockStoreWithSchemaLease(t, tiflashReplicaLease) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table truncate_table (c1 int, c2 int)") + tk.MustExec("insert truncate_table values (1, 1), (2, 2)") + is := domain.GetDomain(tk.Session()).InfoSchema() + oldTblInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("truncate_table")) + require.NoError(t, err) + oldTblID := oldTblInfo.Meta().ID + + tk.MustExec("truncate table truncate_table") + + tk.MustExec("insert truncate_table values (3, 3), (4, 4)") + tk.MustQuery("select * from truncate_table").Check(testkit.Rows("3 3", "4 4")) + + is = domain.GetDomain(tk.Session()).InfoSchema() + newTblInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("truncate_table")) + require.NoError(t, err) + require.Greater(t, newTblInfo.Meta().ID, oldTblID) + + // Verify that the old table data has been deleted by background worker. + tablePrefix := tablecodec.EncodeTablePrefix(oldTblID) + hasOldTableData := true + for i := 0; i < waitForCleanDataRound; i++ { + err = kv.RunInNewTxn(context.Background(), store, false, func(ctx context.Context, txn kv.Transaction) error { + it, err1 := txn.Iter(tablePrefix, nil) + if err1 != nil { + return err1 + } + if !it.Valid() { + hasOldTableData = false + } else { + hasOldTableData = it.Key().HasPrefix(tablePrefix) + } + it.Close() + return nil + }) + require.NoError(t, err) + if !hasOldTableData { + break + } + time.Sleep(waitForCleanDataInterval) + } + require.False(t, hasOldTableData) + + // Test for truncate table should clear the tiflash available status. + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount")) + }() + + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1 (a int);") + tk.MustExec("alter table t1 set tiflash replica 3 location labels 'a','b';") + t1 := external.GetTableByName(t, tk, "test", "t1") + // Mock for table tiflash replica was available. + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), t1.Meta().ID, true) + require.NoError(t, err) + t1 = external.GetTableByName(t, tk, "test", "t1") + require.NotNil(t, t1.Meta().TiFlashReplica) + require.True(t, t1.Meta().TiFlashReplica.Available) + + tk.MustExec("truncate table t1") + t2 := external.GetTableByName(t, tk, "test", "t1") + require.Equal(t, t1.Meta().TiFlashReplica.Count, t2.Meta().TiFlashReplica.Count) + require.Equal(t, t1.Meta().TiFlashReplica.LocationLabels, t2.Meta().TiFlashReplica.LocationLabels) + require.False(t, t2.Meta().TiFlashReplica.Available) + require.Len(t, t2.Meta().TiFlashReplica.AvailablePartitionIDs, 0) + + // Test for truncate partition should clear the tiflash available status. + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1 (a int) partition by hash(a) partitions 2;") + tk.MustExec("alter table t1 set tiflash replica 3 location labels 'a','b';") + t1 = external.GetTableByName(t, tk, "test", "t1") + // Mock for all partitions replica was available. + partition := t1.Meta().Partition + require.Equal(t, 2, len(partition.Definitions)) + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), partition.Definitions[0].ID, true) + require.NoError(t, err) + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), partition.Definitions[1].ID, true) + require.NoError(t, err) + t1 = external.GetTableByName(t, tk, "test", "t1") + require.NotNil(t, t1.Meta().TiFlashReplica) + require.True(t, t1.Meta().TiFlashReplica.Available) + require.Equal(t, []int64{partition.Definitions[0].ID, partition.Definitions[1].ID}, t1.Meta().TiFlashReplica.AvailablePartitionIDs) + + tk.MustExec("alter table t1 truncate partition p0") + t2 = external.GetTableByName(t, tk, "test", "t1") + require.Equal(t, t1.Meta().TiFlashReplica.Count, t2.Meta().TiFlashReplica.Count) + require.Equal(t, t1.Meta().TiFlashReplica.LocationLabels, t2.Meta().TiFlashReplica.LocationLabels) + require.False(t, t2.Meta().TiFlashReplica.Available) + require.Equal(t, []int64{partition.Definitions[1].ID}, t2.Meta().TiFlashReplica.AvailablePartitionIDs) + // Test for truncate twice. + tk.MustExec("alter table t1 truncate partition p0") + t2 = external.GetTableByName(t, tk, "test", "t1") + require.Equal(t, t1.Meta().TiFlashReplica.Count, t2.Meta().TiFlashReplica.Count) + require.Equal(t, t1.Meta().TiFlashReplica.LocationLabels, t2.Meta().TiFlashReplica.LocationLabels) + require.False(t, t2.Meta().TiFlashReplica.Available) + require.Equal(t, []int64{partition.Definitions[1].ID}, t2.Meta().TiFlashReplica.AvailablePartitionIDs) + +}