diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index b5121c5853aef..a299b4fd48e46 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -24,7 +24,6 @@ import ( "testing" "time" - . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/config" @@ -45,37 +44,49 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" - ntestkit "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/admin" - "github.com/pingcap/tidb/util/israce" + "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/mock" - "github.com/pingcap/tidb/util/testkit" + "github.com/stretchr/testify/require" "go.uber.org/zap" ) -var _ = Suite(&testIntegrationSuite1{&testIntegrationSuite{}}) -var _ = Suite(&testIntegrationSuite2{&testIntegrationSuite{}}) -var _ = Suite(&testIntegrationSuite3{&testIntegrationSuite{}}) -var _ = Suite(&testIntegrationSuite4{&testIntegrationSuite{}}) -var _ = Suite(&testIntegrationSuite5{&testIntegrationSuite{}}) -var _ = Suite(&testIntegrationSuite6{&testIntegrationSuite{}}) - -type testIntegrationSuite1 struct{ *testIntegrationSuite } -type testIntegrationSuite2 struct{ *testIntegrationSuite } +func checkGlobalIndexCleanUpDone(t *testing.T, ctx sessionctx.Context, tblInfo *model.TableInfo, idxInfo *model.IndexInfo, pid int64) int { + require.NoError(t, ctx.NewTxn(context.Background())) + txn, err := ctx.Txn(true) + require.NoError(t, err) + defer func() { + err := txn.Rollback() + require.NoError(t, err) + }() -func (s *testIntegrationSuite2) TearDownTest(c *C) { - tearDownIntegrationSuiteTest(s.testIntegrationSuite, c) + cnt := 0 + prefix := tablecodec.EncodeTableIndexPrefix(tblInfo.ID, idxInfo.ID) + it, err := txn.Iter(prefix, nil) + require.NoError(t, err) + for it.Valid() { + if !it.Key().HasPrefix(prefix) { + break + } + segs := tablecodec.SplitIndexValue(it.Value()) + require.NotNil(t, segs.PartitionID) + _, pi, err := codec.DecodeInt(segs.PartitionID) + require.NoError(t, err) + require.NotEqual(t, pid, pi) + cnt++ + err = it.Next() + require.NoError(t, err) + } + return cnt } -type testIntegrationSuite3 struct{ *testIntegrationSuite } -type testIntegrationSuite4 struct{ *testIntegrationSuite } -type testIntegrationSuite5 struct{ *testIntegrationSuite } -type testIntegrationSuite6 struct{ *testIntegrationSuite } - -func (s *testIntegrationSuite3) TestCreateTableWithPartition(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCreateTableWithPartition(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists tp;") tk.MustExec(`CREATE TABLE tp (a int) PARTITION BY RANGE(a) ( @@ -83,24 +94,24 @@ func (s *testIntegrationSuite3) TestCreateTableWithPartition(c *C) { PARTITION p1 VALUES LESS THAN (20), PARTITION p2 VALUES LESS THAN (MAXVALUE) );`) - ctx := tk.Se.(sessionctx.Context) + ctx := tk.Session() is := domain.GetDomain(ctx).InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("tp")) - c.Assert(err, IsNil) - c.Assert(tbl.Meta().Partition, NotNil) + require.NoError(t, err) + require.NotNil(t, tbl.Meta().Partition) part := tbl.Meta().Partition - c.Assert(part.Type, Equals, model.PartitionTypeRange) - c.Assert(part.Expr, Equals, "`a`") + require.Equal(t, model.PartitionTypeRange, part.Type) + require.Equal(t, "`a`", part.Expr) for _, pdef := range part.Definitions { - c.Assert(pdef.ID, Greater, int64(0)) + require.Greater(t, pdef.ID, int64(0)) } - c.Assert(part.Definitions, HasLen, 3) - c.Assert(part.Definitions[0].LessThan[0], Equals, "10") - c.Assert(part.Definitions[0].Name.L, Equals, "p0") - c.Assert(part.Definitions[1].LessThan[0], Equals, "20") - c.Assert(part.Definitions[1].Name.L, Equals, "p1") - c.Assert(part.Definitions[2].LessThan[0], Equals, "MAXVALUE") - c.Assert(part.Definitions[2].Name.L, Equals, "p2") + require.Len(t, part.Definitions, 3) + require.Equal(t, "10", part.Definitions[0].LessThan[0]) + require.Equal(t, "p0", part.Definitions[0].Name.L) + require.Equal(t, "20", part.Definitions[1].LessThan[0]) + require.Equal(t, "p1", part.Definitions[1].Name.L) + require.Equal(t, "MAXVALUE", part.Definitions[2].LessThan[0]) + require.Equal(t, "p2", part.Definitions[2].Name.L) tk.MustExec("drop table if exists employees;") sql1 := `create table employees ( @@ -158,7 +169,7 @@ func (s *testIntegrationSuite3) TestCreateTableWithPartition(c *C) { partition p3 values less than (65,30,13), partition p4 values less than (maxvalue,30,40) );`) - c.Assert(err, IsNil) + require.NoError(t, err) sql6 := `create table employees ( id int not null, @@ -211,7 +222,7 @@ func (s *testIntegrationSuite3) TestCreateTableWithPartition(c *C) { partition p2 values less than (20), partition p3 values less than (20) );`) - c.Assert(ddl.ErrRangeNotIncreasing.Equal(err), IsTrue) + require.True(t, ddl.ErrRangeNotIncreasing.Equal(err)) tk.MustGetErrCode(`create TABLE t10 (c1 int,c2 int) partition by range(c1 / c2 ) (partition p0 values less than (2));`, tmysql.ErrPartitionFunctionIsNotAllowed) @@ -327,8 +338,10 @@ partition by range (a) partition p2 values less than maxvalue)`) } -func (s *testIntegrationSuite2) TestCreateTableWithHashPartition(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCreateTableWithHashPartition(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists employees;") tk.MustExec("set @@session.tidb_enable_table_partition = 1") @@ -384,8 +397,10 @@ func (s *testIntegrationSuite2) TestCreateTableWithHashPartition(c *C) { tk.MustExec("create table t4 (a int, b int) partition by hash(floor(a-b)) partitions 10") } -func (s *testSerialDBSuite1) TestCreateTableWithRangeColumnPartition(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCreateTableWithRangeColumnPartition(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists log_message_1;") tk.MustExec("set @@session.tidb_enable_list_partition = ON") @@ -577,12 +592,12 @@ create table log_message_1 ( ddl.ErrWrongExprInPartitionFunc, }, } - for i, t := range cases { - _, err := tk.Exec(t.sql) - c.Assert(t.err.Equal(err), IsTrue, Commentf( + for i, tt := range cases { + _, err := tk.Exec(tt.sql) + require.Truef(t, tt.err.Equal(err), "case %d fail, sql = `%s`\nexpected error = `%v`\n actual error = `%v`", - i, t.sql, t.err, err, - )) + i, tt.sql, tt.err, err, + ) } tk.MustExec("drop table if exists t1;") @@ -615,11 +630,10 @@ create table log_message_1 ( tk.MustQuery(`select * from t where a < X'0D' order by a`).Check(testkit.Rows("\x0B", "\x0C")) } -func (s *testIntegrationSuite1) TestDisableTablePartition(c *C) { - if israce.RaceEnabled { - c.Skip("skip race test") - } - tk := testkit.NewTestKit(c, s.store) +func TestDisableTablePartition(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") for _, v := range []string{"'AUTO'", "'OFF'", "0", "'ON'"} { tk.MustExec("set @@session.tidb_enable_table_partition = " + v) @@ -627,17 +641,17 @@ func (s *testIntegrationSuite1) TestDisableTablePartition(c *C) { tk.MustExec("drop table if exists t") tk.MustExec(`create table t (id int) partition by list (id) ( partition p0 values in (1,2),partition p1 values in (3,4));`) - tbl := testGetTableByName(c, tk.Se, "test", "t") - c.Assert(tbl.Meta().Partition, IsNil) + tbl := tk.GetTableByName("test", "t") + require.Nil(t, tbl.Meta().Partition) _, err := tk.Exec(`alter table t add partition ( partition p4 values in (7), partition p5 values in (8,9));`) - c.Assert(ddl.ErrPartitionMgmtOnNonpartitioned.Equal(err), IsTrue) + require.True(t, ddl.ErrPartitionMgmtOnNonpartitioned.Equal(err)) tk.MustExec("insert into t values (1),(3),(5),(100),(null)") } } -func (s *testIntegrationSuite1) generatePartitionTableByNum(num int) string { +func generatePartitionTableByNum(num int) string { buf := bytes.NewBuffer(make([]byte, 0, 1024*1024)) buf.WriteString("create table gen_t (id int) partition by list (id) (") for i := 0; i < num; i++ { @@ -650,8 +664,10 @@ func (s *testIntegrationSuite1) generatePartitionTableByNum(num int) string { return buf.String() } -func (s *testIntegrationSuite1) TestCreateTableWithListPartition(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCreateTableWithListPartition(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("set @@session.tidb_enable_list_partition = ON") tk.MustExec("drop table if exists t") @@ -746,16 +762,16 @@ func (s *testIntegrationSuite1) TestCreateTableWithListPartition(c *C) { ddl.ErrUniqueKeyNeedAllFieldsInPf, }, { - s.generatePartitionTableByNum(ddl.PartitionCountLimit + 1), + generatePartitionTableByNum(ddl.PartitionCountLimit + 1), ddl.ErrTooManyPartitions, }, } - for i, t := range cases { - _, err := tk.Exec(t.sql) - c.Assert(t.err.Equal(err), IsTrue, Commentf( + for i, tt := range cases { + _, err := tk.Exec(tt.sql) + require.Truef(t, tt.err.Equal(err), "case %d fail, sql = `%s`\nexpected error = `%v`\n actual error = `%v`", - i, t.sql, t.err, err, - )) + i, tt.sql, tt.err, err, + ) } validCases := []string{ @@ -777,7 +793,7 @@ func (s *testIntegrationSuite1) TestCreateTableWithListPartition(c *C) { "create table t (a datetime) partition by list (to_seconds(a)) (partition p0 values in (to_seconds('2020-09-28 17:03:38'),to_seconds('2020-09-28 17:03:39')));", "create table t (a int, b int generated always as (a+1) virtual) partition by list (b + 1) (partition p0 values in (1));", "create table t(a binary) partition by list columns (a) (partition p0 values in (X'0C'));", - s.generatePartitionTableByNum(ddl.PartitionCountLimit), + generatePartitionTableByNum(ddl.PartitionCountLimit), } for id, sql := range validCases { @@ -787,16 +803,18 @@ func (s *testIntegrationSuite1) TestCreateTableWithListPartition(c *C) { if id == len(validCases)-1 { tblName = "gen_t" } - tbl := testGetTableByName(c, s.ctx, "test", tblName) + tbl := tk.GetTableByName("test", tblName) tblInfo := tbl.Meta() - c.Assert(tblInfo.Partition, NotNil) - c.Assert(tblInfo.Partition.Enable, Equals, true) - c.Assert(tblInfo.Partition.Type == model.PartitionTypeList, IsTrue) + require.NotNil(t, tblInfo.Partition) + require.True(t, tblInfo.Partition.Enable) + require.Equal(t, model.PartitionTypeList, tblInfo.Partition.Type) } } -func (s *testIntegrationSuite1) TestCreateTableWithListColumnsPartition(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCreateTableWithListColumnsPartition(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("set @@session.tidb_enable_list_partition = ON") tk.MustExec("drop table if exists t") @@ -955,12 +973,12 @@ func (s *testIntegrationSuite1) TestCreateTableWithListColumnsPartition(c *C) { ddl.ErrSameNamePartition, }, } - for i, t := range cases { - _, err := tk.Exec(t.sql) - c.Assert(t.err.Equal(err), IsTrue, Commentf( + for i, tt := range cases { + _, err := tk.Exec(tt.sql) + require.Truef(t, tt.err.Equal(err), "case %d fail, sql = `%s`\nexpected error = `%v`\n actual error = `%v`", - i, t.sql, t.err, err, - )) + i, tt.sql, tt.err, err, + ) } validCases := []string{ @@ -992,16 +1010,18 @@ func (s *testIntegrationSuite1) TestCreateTableWithListColumnsPartition(c *C) { for _, sql := range validCases { tk.MustExec("drop table if exists t") tk.MustExec(sql) - tbl := testGetTableByName(c, s.ctx, "test", "t") + tbl := tk.GetTableByName("test", "t") tblInfo := tbl.Meta() - c.Assert(tblInfo.Partition, NotNil) - c.Assert(tblInfo.Partition.Enable, Equals, true) - c.Assert(tblInfo.Partition.Type == model.PartitionTypeList, IsTrue) + require.NotNil(t, tblInfo.Partition) + require.Equal(t, true, tblInfo.Partition.Enable) + require.True(t, tblInfo.Partition.Type == model.PartitionTypeList) } } -func (s *testIntegrationSuite5) TestAlterTableAddPartitionByList(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAlterTableAddPartitionByList(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t;") tk.MustExec("set @@session.tidb_enable_list_partition = ON") @@ -1014,26 +1034,26 @@ func (s *testIntegrationSuite5) TestAlterTableAddPartitionByList(c *C) { partition p4 values in (7), partition p5 values in (8,9));`) - ctx := tk.Se.(sessionctx.Context) + ctx := tk.Session() is := domain.GetDomain(ctx).InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) - c.Assert(tbl.Meta().Partition, NotNil) + require.NoError(t, err) + require.NotNil(t, tbl.Meta().Partition) part := tbl.Meta().Partition - c.Assert(part.Type == model.PartitionTypeList, IsTrue) - - c.Assert(part.Expr, Equals, "`id`") - c.Assert(part.Definitions, HasLen, 5) - c.Assert(part.Definitions[0].InValues, DeepEquals, [][]string{{"1"}, {"2"}}) - c.Assert(part.Definitions[0].Name, Equals, model.NewCIStr("p0")) - c.Assert(part.Definitions[1].InValues, DeepEquals, [][]string{{"3"}, {"4"}}) - c.Assert(part.Definitions[1].Name, Equals, model.NewCIStr("p1")) - c.Assert(part.Definitions[2].InValues, DeepEquals, [][]string{{"5"}, {"NULL"}}) - c.Assert(part.Definitions[2].Name, Equals, model.NewCIStr("p3")) - c.Assert(part.Definitions[3].InValues, DeepEquals, [][]string{{"7"}}) - c.Assert(part.Definitions[3].Name, Equals, model.NewCIStr("p4")) - c.Assert(part.Definitions[4].InValues, DeepEquals, [][]string{{"8"}, {"9"}}) - c.Assert(part.Definitions[4].Name, Equals, model.NewCIStr("p5")) + require.True(t, part.Type == model.PartitionTypeList) + + require.Equal(t, "`id`", part.Expr) + require.Len(t, part.Definitions, 5) + require.Equal(t, [][]string{{"1"}, {"2"}}, part.Definitions[0].InValues) + require.Equal(t, model.NewCIStr("p0"), part.Definitions[0].Name) + require.Equal(t, [][]string{{"3"}, {"4"}}, part.Definitions[1].InValues) + require.Equal(t, model.NewCIStr("p1"), part.Definitions[1].Name) + require.Equal(t, [][]string{{"5"}, {"NULL"}}, part.Definitions[2].InValues) + require.Equal(t, model.NewCIStr("p3"), part.Definitions[2].Name) + require.Equal(t, [][]string{{"7"}}, part.Definitions[3].InValues) + require.Equal(t, model.NewCIStr("p4"), part.Definitions[3].Name) + require.Equal(t, [][]string{{"8"}, {"9"}}, part.Definitions[4].InValues) + require.Equal(t, model.NewCIStr("p5"), part.Definitions[4].Name) errorCases := []struct { sql string @@ -1059,12 +1079,12 @@ func (s *testIntegrationSuite5) TestAlterTableAddPartitionByList(c *C) { }, } - for i, t := range errorCases { - _, err := tk.Exec(t.sql) - c.Assert(t.err.Equal(err), IsTrue, Commentf( + for i, tt := range errorCases { + _, err := tk.Exec(tt.sql) + require.Truef(t, tt.err.Equal(err), "case %d fail, sql = `%s`\nexpected error = `%v`\n actual error = `%v`", - i, t.sql, t.err, err, - )) + i, tt.sql, tt.err, err, + ) } errorCases2 := []struct { @@ -1104,19 +1124,21 @@ func (s *testIntegrationSuite5) TestAlterTableAddPartitionByList(c *C) { }, } - for i, t := range errorCases2 { + for i, tt := range errorCases2 { tk.MustExec("drop table if exists t;") - tk.MustExec(t.create) - _, err := tk.Exec(t.alter) - c.Assert(t.err.Equal(err), IsTrue, Commentf( + tk.MustExec(tt.create) + _, err := tk.Exec(tt.alter) + require.Truef(t, tt.err.Equal(err), "case %d fail, sql = `%s`\nexpected error = `%v`\n actual error = `%v`", - i, t.alter, t.err, err, - )) + i, tt.alter, tt.err, err, + ) } } -func (s *testIntegrationSuite5) TestAlterTableAddPartitionByListColumns(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAlterTableAddPartitionByListColumns(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t;") tk.MustExec("set @@session.tidb_enable_list_partition = ON") @@ -1129,28 +1151,28 @@ func (s *testIntegrationSuite5) TestAlterTableAddPartitionByListColumns(c *C) { partition p4 values in ((7,'a')), partition p5 values in ((8,'a')));`) - ctx := tk.Se.(sessionctx.Context) + ctx := tk.Session() is := domain.GetDomain(ctx).InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) - c.Assert(tbl.Meta().Partition, NotNil) + require.NoError(t, err) + require.NotNil(t, tbl.Meta().Partition) part := tbl.Meta().Partition - c.Assert(part.Type == model.PartitionTypeList, IsTrue) - - c.Assert(part.Expr, Equals, "") - c.Assert(part.Columns[0].O, Equals, "id") - c.Assert(part.Columns[1].O, Equals, "name") - c.Assert(part.Definitions, HasLen, 5) - c.Assert(part.Definitions[0].InValues, DeepEquals, [][]string{{"1", `"a"`}, {"2", `"b"`}}) - c.Assert(part.Definitions[0].Name, Equals, model.NewCIStr("p0")) - c.Assert(part.Definitions[1].InValues, DeepEquals, [][]string{{"3", `"a"`}, {"4", `"b"`}}) - c.Assert(part.Definitions[1].Name, Equals, model.NewCIStr("p1")) - c.Assert(part.Definitions[2].InValues, DeepEquals, [][]string{{"5", `NULL`}}) - c.Assert(part.Definitions[2].Name, Equals, model.NewCIStr("p3")) - c.Assert(part.Definitions[3].InValues, DeepEquals, [][]string{{"7", `"a"`}}) - c.Assert(part.Definitions[3].Name, Equals, model.NewCIStr("p4")) - c.Assert(part.Definitions[4].InValues, DeepEquals, [][]string{{"8", `"a"`}}) - c.Assert(part.Definitions[4].Name, Equals, model.NewCIStr("p5")) + require.True(t, part.Type == model.PartitionTypeList) + + require.Equal(t, "", part.Expr) + require.Equal(t, "id", part.Columns[0].O) + require.Equal(t, "name", part.Columns[1].O) + require.Len(t, part.Definitions, 5) + require.Equal(t, [][]string{{"1", `"a"`}, {"2", `"b"`}}, part.Definitions[0].InValues) + require.Equal(t, model.NewCIStr("p0"), part.Definitions[0].Name) + require.Equal(t, [][]string{{"3", `"a"`}, {"4", `"b"`}}, part.Definitions[1].InValues) + require.Equal(t, model.NewCIStr("p1"), part.Definitions[1].Name) + require.Equal(t, [][]string{{"5", `NULL`}}, part.Definitions[2].InValues) + require.Equal(t, model.NewCIStr("p3"), part.Definitions[2].Name) + require.Equal(t, [][]string{{"7", `"a"`}}, part.Definitions[3].InValues) + require.Equal(t, model.NewCIStr("p4"), part.Definitions[3].Name) + require.Equal(t, [][]string{{"8", `"a"`}}, part.Definitions[4].InValues) + require.Equal(t, model.NewCIStr("p5"), part.Definitions[4].Name) errorCases := []struct { sql string @@ -1173,17 +1195,19 @@ func (s *testIntegrationSuite5) TestAlterTableAddPartitionByListColumns(c *C) { }, } - for i, t := range errorCases { - _, err := tk.Exec(t.sql) - c.Assert(t.err.Equal(err), IsTrue, Commentf( + for i, tt := range errorCases { + _, err := tk.Exec(tt.sql) + require.Truef(t, tt.err.Equal(err), "case %d fail, sql = `%s`\nexpected error = `%v`\n actual error = `%v`", - i, t.sql, t.err, err, - )) + i, tt.sql, tt.err, err, + ) } } -func (s *testIntegrationSuite5) TestAlterTableDropPartitionByList(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAlterTableDropPartitionByList(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t;") tk.MustExec("set @@session.tidb_enable_list_partition = ON") @@ -1195,19 +1219,19 @@ func (s *testIntegrationSuite5) TestAlterTableDropPartitionByList(c *C) { tk.MustExec(`insert into t values (1),(3),(5),(null)`) tk.MustExec(`alter table t drop partition p1`) tk.MustQuery("select * from t").Check(testkit.Rows("1", "5", "")) - ctx := tk.Se.(sessionctx.Context) + ctx := tk.Session() is := domain.GetDomain(ctx).InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) - c.Assert(tbl.Meta().Partition, NotNil) + require.NoError(t, err) + require.NotNil(t, tbl.Meta().Partition) part := tbl.Meta().Partition - c.Assert(part.Type == model.PartitionTypeList, IsTrue) - c.Assert(part.Expr, Equals, "`id`") - c.Assert(part.Definitions, HasLen, 2) - c.Assert(part.Definitions[0].InValues, DeepEquals, [][]string{{"1"}, {"2"}}) - c.Assert(part.Definitions[0].Name, Equals, model.NewCIStr("p0")) - c.Assert(part.Definitions[1].InValues, DeepEquals, [][]string{{"5"}, {"NULL"}}) - c.Assert(part.Definitions[1].Name, Equals, model.NewCIStr("p3")) + require.True(t, part.Type == model.PartitionTypeList) + require.Equal(t, "`id`", part.Expr) + require.Len(t, part.Definitions, 2) + require.Equal(t, [][]string{{"1"}, {"2"}}, part.Definitions[0].InValues) + require.Equal(t, model.NewCIStr("p0"), part.Definitions[0].Name) + require.Equal(t, [][]string{{"5"}, {"NULL"}}, part.Definitions[1].InValues) + require.Equal(t, model.NewCIStr("p3"), part.Definitions[1].Name) sql := "alter table t drop partition p10;" tk.MustGetErrCode(sql, tmysql.ErrDropPartitionNonExistent) @@ -1217,8 +1241,10 @@ func (s *testIntegrationSuite5) TestAlterTableDropPartitionByList(c *C) { tk.MustGetErrCode(sql, tmysql.ErrDropLastPartition) } -func (s *testIntegrationSuite5) TestAlterTableDropPartitionByListColumns(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAlterTableDropPartitionByListColumns(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t;") tk.MustExec("set @@session.tidb_enable_list_partition = ON") @@ -1230,21 +1256,21 @@ func (s *testIntegrationSuite5) TestAlterTableDropPartitionByListColumns(c *C) { tk.MustExec(`insert into t values (1,'a'),(3,'a'),(5,'a'),(null,null)`) tk.MustExec(`alter table t drop partition p1`) tk.MustQuery("select * from t").Sort().Check(testkit.Rows("1 a", "5 a", " ")) - ctx := tk.Se.(sessionctx.Context) + ctx := tk.Session() is := domain.GetDomain(ctx).InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) - c.Assert(tbl.Meta().Partition, NotNil) + require.NoError(t, err) + require.NotNil(t, tbl.Meta().Partition) part := tbl.Meta().Partition - c.Assert(part.Type == model.PartitionTypeList, IsTrue) - c.Assert(part.Expr, Equals, "") - c.Assert(part.Columns[0].O, Equals, "id") - c.Assert(part.Columns[1].O, Equals, "name") - c.Assert(part.Definitions, HasLen, 2) - c.Assert(part.Definitions[0].InValues, DeepEquals, [][]string{{"1", `"a"`}, {"2", `"b"`}}) - c.Assert(part.Definitions[0].Name, Equals, model.NewCIStr("p0")) - c.Assert(part.Definitions[1].InValues, DeepEquals, [][]string{{"5", `"a"`}, {"NULL", "NULL"}}) - c.Assert(part.Definitions[1].Name, Equals, model.NewCIStr("p3")) + require.True(t, part.Type == model.PartitionTypeList) + require.Equal(t, "", part.Expr) + require.Equal(t, "id", part.Columns[0].O) + require.Equal(t, "name", part.Columns[1].O) + require.Len(t, part.Definitions, 2) + require.Equal(t, [][]string{{"1", `"a"`}, {"2", `"b"`}}, part.Definitions[0].InValues) + require.Equal(t, model.NewCIStr("p0"), part.Definitions[0].Name) + require.Equal(t, [][]string{{"5", `"a"`}, {"NULL", "NULL"}}, part.Definitions[1].InValues) + require.Equal(t, model.NewCIStr("p3"), part.Definitions[1].Name) sql := "alter table t drop partition p10;" tk.MustGetErrCode(sql, tmysql.ErrDropPartitionNonExistent) @@ -1254,8 +1280,10 @@ func (s *testIntegrationSuite5) TestAlterTableDropPartitionByListColumns(c *C) { tk.MustGetErrCode(sql, tmysql.ErrDropLastPartition) } -func (s *testIntegrationSuite5) TestAlterTableTruncatePartitionByList(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAlterTableTruncatePartitionByList(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t;") tk.MustExec("set @@session.tidb_enable_list_partition = ON") @@ -1265,17 +1293,17 @@ func (s *testIntegrationSuite5) TestAlterTableTruncatePartitionByList(c *C) { partition p3 values in (5,null) );`) tk.MustExec(`insert into t values (1),(3),(5),(null)`) - oldTbl := testGetTableByName(c, tk.Se, "test", "t") + oldTbl := tk.GetTableByName("test", "t") tk.MustExec(`alter table t truncate partition p1`) tk.MustQuery("select * from t").Check(testkit.Rows("1", "5", "")) - tbl := testGetTableByName(c, tk.Se, "test", "t") - c.Assert(tbl.Meta().Partition, NotNil) + tbl := tk.GetTableByName("test", "t") + require.NotNil(t, tbl.Meta().Partition) part := tbl.Meta().Partition - c.Assert(part.Type == model.PartitionTypeList, IsTrue) - c.Assert(part.Definitions, HasLen, 3) - c.Assert(part.Definitions[1].InValues, DeepEquals, [][]string{{"3"}, {"4"}}) - c.Assert(part.Definitions[1].Name, Equals, model.NewCIStr("p1")) - c.Assert(part.Definitions[1].ID == oldTbl.Meta().Partition.Definitions[1].ID, IsFalse) + require.True(t, part.Type == model.PartitionTypeList) + require.Len(t, part.Definitions, 3) + require.Equal(t, [][]string{{"3"}, {"4"}}, part.Definitions[1].InValues) + require.Equal(t, model.NewCIStr("p1"), part.Definitions[1].Name) + require.False(t, part.Definitions[1].ID == oldTbl.Meta().Partition.Definitions[1].ID) sql := "alter table t truncate partition p10;" tk.MustGetErrCode(sql, tmysql.ErrUnknownPartition) @@ -1285,8 +1313,10 @@ func (s *testIntegrationSuite5) TestAlterTableTruncatePartitionByList(c *C) { tk.MustQuery("select * from t").Check(testkit.Rows()) } -func (s *testIntegrationSuite5) TestAlterTableTruncatePartitionByListColumns(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAlterTableTruncatePartitionByListColumns(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t;") tk.MustExec("set @@session.tidb_enable_list_partition = ON") @@ -1296,17 +1326,17 @@ func (s *testIntegrationSuite5) TestAlterTableTruncatePartitionByListColumns(c * partition p3 values in ((5,'a'),(null,null)) );`) tk.MustExec(`insert into t values (1,'a'),(3,'a'),(5,'a'),(null,null)`) - oldTbl := testGetTableByName(c, tk.Se, "test", "t") + oldTbl := tk.GetTableByName("test", "t") tk.MustExec(`alter table t truncate partition p1`) tk.MustQuery("select * from t").Check(testkit.Rows("1 a", "5 a", " ")) - tbl := testGetTableByName(c, tk.Se, "test", "t") - c.Assert(tbl.Meta().Partition, NotNil) + tbl := tk.GetTableByName("test", "t") + require.NotNil(t, tbl.Meta().Partition) part := tbl.Meta().Partition - c.Assert(part.Type == model.PartitionTypeList, IsTrue) - c.Assert(part.Definitions, HasLen, 3) - c.Assert(part.Definitions[1].InValues, DeepEquals, [][]string{{"3", `"a"`}, {"4", `"b"`}}) - c.Assert(part.Definitions[1].Name, Equals, model.NewCIStr("p1")) - c.Assert(part.Definitions[1].ID == oldTbl.Meta().Partition.Definitions[1].ID, IsFalse) + require.True(t, part.Type == model.PartitionTypeList) + require.Len(t, part.Definitions, 3) + require.Equal(t, [][]string{{"3", `"a"`}, {"4", `"b"`}}, part.Definitions[1].InValues) + require.Equal(t, model.NewCIStr("p1"), part.Definitions[1].Name) + require.False(t, part.Definitions[1].ID == oldTbl.Meta().Partition.Definitions[1].ID) sql := "alter table t truncate partition p10;" tk.MustGetErrCode(sql, tmysql.ErrUnknownPartition) @@ -1316,8 +1346,10 @@ func (s *testIntegrationSuite5) TestAlterTableTruncatePartitionByListColumns(c * tk.MustQuery("select * from t").Check(testkit.Rows()) } -func (s *testIntegrationSuite3) TestCreateTableWithKeyPartition(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCreateTableWithKeyPartition(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists tm1;") tk.MustExec(`create table tm1 @@ -1330,8 +1362,10 @@ func (s *testIntegrationSuite3) TestCreateTableWithKeyPartition(c *C) { tk.MustExec(`create table tm2 (a char(5), unique key(a(5))) partition by key() partitions 5;`) } -func (s *testIntegrationSuite5) TestAlterTableAddPartition(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAlterTableAddPartition(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists employees;") tk.MustExec(`create table employees ( @@ -1348,26 +1382,26 @@ func (s *testIntegrationSuite5) TestAlterTableAddPartition(c *C) { partition p5 values less than MAXVALUE );`) - ctx := tk.Se.(sessionctx.Context) + ctx := tk.Session() is := domain.GetDomain(ctx).InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("employees")) - c.Assert(err, IsNil) - c.Assert(tbl.Meta().Partition, NotNil) + require.NoError(t, err) + require.NotNil(t, tbl.Meta().Partition) part := tbl.Meta().Partition - c.Assert(part.Type, Equals, model.PartitionTypeRange) - - c.Assert(part.Expr, Equals, "YEAR(`hired`)") - c.Assert(part.Definitions, HasLen, 5) - c.Assert(part.Definitions[0].LessThan[0], Equals, "1991") - c.Assert(part.Definitions[0].Name, Equals, model.NewCIStr("p1")) - c.Assert(part.Definitions[1].LessThan[0], Equals, "1996") - c.Assert(part.Definitions[1].Name, Equals, model.NewCIStr("p2")) - c.Assert(part.Definitions[2].LessThan[0], Equals, "2001") - c.Assert(part.Definitions[2].Name, Equals, model.NewCIStr("p3")) - c.Assert(part.Definitions[3].LessThan[0], Equals, "2010") - c.Assert(part.Definitions[3].Name, Equals, model.NewCIStr("p4")) - c.Assert(part.Definitions[4].LessThan[0], Equals, "MAXVALUE") - c.Assert(part.Definitions[4].Name, Equals, model.NewCIStr("p5")) + require.Equal(t, model.PartitionTypeRange, part.Type) + + require.Equal(t, "YEAR(`hired`)", part.Expr) + require.Len(t, part.Definitions, 5) + require.Equal(t, "1991", part.Definitions[0].LessThan[0]) + require.Equal(t, model.NewCIStr("p1"), part.Definitions[0].Name) + require.Equal(t, "1996", part.Definitions[1].LessThan[0]) + require.Equal(t, model.NewCIStr("p2"), part.Definitions[1].Name) + require.Equal(t, "2001", part.Definitions[2].LessThan[0]) + require.Equal(t, model.NewCIStr("p3"), part.Definitions[2].Name) + require.Equal(t, "2010", part.Definitions[3].LessThan[0]) + require.Equal(t, model.NewCIStr("p4"), part.Definitions[3].Name) + require.Equal(t, "MAXVALUE", part.Definitions[4].LessThan[0]) + require.Equal(t, model.NewCIStr("p5"), part.Definitions[4].Name) tk.MustExec("drop table if exists table1;") tk.MustExec("create table table1(a int)") @@ -1476,8 +1510,10 @@ func (s *testIntegrationSuite5) TestAlterTableAddPartition(c *C) { tk.MustGetErrCode(sql, tmysql.ErrWrongTypeColumnValue) } -func (s *testIntegrationSuite5) TestAlterTableDropPartition(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAlterTableDropPartition(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists employees") tk.MustExec(`create table employees ( @@ -1491,19 +1527,19 @@ func (s *testIntegrationSuite5) TestAlterTableDropPartition(c *C) { );`) tk.MustExec("alter table employees drop partition p3;") - ctx := tk.Se.(sessionctx.Context) + ctx := tk.Session() is := domain.GetDomain(ctx).InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("employees")) - c.Assert(err, IsNil) - c.Assert(tbl.Meta().GetPartitionInfo(), NotNil) + require.NoError(t, err) + require.NotNil(t, tbl.Meta().GetPartitionInfo()) part := tbl.Meta().Partition - c.Assert(part.Type, Equals, model.PartitionTypeRange) - c.Assert(part.Expr, Equals, "`hired`") - c.Assert(part.Definitions, HasLen, 2) - c.Assert(part.Definitions[0].LessThan[0], Equals, "1991") - c.Assert(part.Definitions[0].Name, Equals, model.NewCIStr("p1")) - c.Assert(part.Definitions[1].LessThan[0], Equals, "1996") - c.Assert(part.Definitions[1].Name, Equals, model.NewCIStr("p2")) + require.Equal(t, model.PartitionTypeRange, part.Type) + require.Equal(t, "`hired`", part.Expr) + require.Len(t, part.Definitions, 2) + require.Equal(t, "1991", part.Definitions[0].LessThan[0]) + require.Equal(t, model.NewCIStr("p1"), part.Definitions[0].Name) + require.Equal(t, "1996", part.Definitions[1].LessThan[0]) + require.Equal(t, model.NewCIStr("p2"), part.Definitions[1].Name) tk.MustExec("drop table if exists table1;") tk.MustExec("create table table1 (a int);") @@ -1546,16 +1582,16 @@ func (s *testIntegrationSuite5) TestAlterTableDropPartition(c *C) { tk.MustExec("alter table table4 drop partition p2;") is = domain.GetDomain(ctx).InfoSchema() tbl, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("table4")) - c.Assert(err, IsNil) - c.Assert(tbl.Meta().GetPartitionInfo(), NotNil) + require.NoError(t, err) + require.NotNil(t, tbl.Meta().GetPartitionInfo()) part = tbl.Meta().Partition - c.Assert(part.Type, Equals, model.PartitionTypeRange) - c.Assert(part.Expr, Equals, "`id`") - c.Assert(part.Definitions, HasLen, 2) - c.Assert(part.Definitions[0].LessThan[0], Equals, "10") - c.Assert(part.Definitions[0].Name, Equals, model.NewCIStr("p1")) - c.Assert(part.Definitions[1].LessThan[0], Equals, "MAXVALUE") - c.Assert(part.Definitions[1].Name, Equals, model.NewCIStr("p3")) + require.Equal(t, model.PartitionTypeRange, part.Type) + require.Equal(t, "`id`", part.Expr) + require.Len(t, part.Definitions, 2) + require.Equal(t, "10", part.Definitions[0].LessThan[0]) + require.Equal(t, model.NewCIStr("p1"), part.Definitions[0].Name) + require.Equal(t, "MAXVALUE", part.Definitions[1].LessThan[0]) + require.Equal(t, model.NewCIStr("p3"), part.Definitions[1].Name) tk.MustExec("drop table if exists tr;") tk.MustExec(` create table tr( @@ -1616,8 +1652,10 @@ func (s *testIntegrationSuite5) TestAlterTableDropPartition(c *C) { tk.MustGetErrCode("alter table t1 drop partition p2", tmysql.ErrOnlyOnRangeListPartition) } -func (s *testIntegrationSuite5) TestMultiPartitionDropAndTruncate(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestMultiPartitionDropAndTruncate(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists employees") tk.MustExec(`create table employees ( @@ -1641,11 +1679,15 @@ func (s *testIntegrationSuite5) TestMultiPartitionDropAndTruncate(c *C) { result.Check(testkit.Rows(`2010`)) } -func (s *testSerialDBSuite1) TestDropPartitionWithGlobalIndex(c *C) { +func TestDropPartitionWithGlobalIndex(t *testing.T) { + restore := config.RestoreFunc() + defer restore() + store, clean := testkit.CreateMockStore(t) + defer clean() config.UpdateGlobal(func(conf *config.Config) { conf.EnableGlobalIndex = true }) - tk := testkit.NewTestKit(c, s.store) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists test_global") tk.MustExec(`create table test_global ( a int, b int, c int) @@ -1653,8 +1695,8 @@ func (s *testSerialDBSuite1) TestDropPartitionWithGlobalIndex(c *C) { partition p1 values less than (10), partition p2 values less than (20) );`) - t := testGetTableByName(c, s.ctx, "test", "test_global") - pid := t.Meta().Partition.Definitions[1].ID + tt := tk.GetTableByName("test", "test_global") + pid := tt.Meta().Partition.Definitions[1].ID tk.MustExec("Alter Table test_global Add Unique Index idx_b (b);") tk.MustExec("Alter Table test_global Add Unique Index idx_c (c);") @@ -1664,23 +1706,25 @@ func (s *testSerialDBSuite1) TestDropPartitionWithGlobalIndex(c *C) { result := tk.MustQuery("select * from test_global;") result.Sort().Check(testkit.Rows(`1 1 1`, `2 2 2`)) - t = testGetTableByName(c, s.ctx, "test", "test_global") - idxInfo := t.Meta().FindIndexByName("idx_b") - c.Assert(idxInfo, NotNil) - cnt := checkGlobalIndexCleanUpDone(c, s.ctx, t.Meta(), idxInfo, pid) - c.Assert(cnt, Equals, 2) + tt = tk.GetTableByName("test", "test_global") + idxInfo := tt.Meta().FindIndexByName("idx_b") + require.NotNil(t, idxInfo) + cnt := checkGlobalIndexCleanUpDone(t, tk.Session(), tt.Meta(), idxInfo, pid) + require.Equal(t, 2, cnt) - idxInfo = t.Meta().FindIndexByName("idx_c") - c.Assert(idxInfo, NotNil) - cnt = checkGlobalIndexCleanUpDone(c, s.ctx, t.Meta(), idxInfo, pid) - c.Assert(cnt, Equals, 2) + idxInfo = tt.Meta().FindIndexByName("idx_c") + require.NotNil(t, idxInfo) + cnt = checkGlobalIndexCleanUpDone(t, tk.Session(), tt.Meta(), idxInfo, pid) + require.Equal(t, 2, cnt) config.UpdateGlobal(func(conf *config.Config) { conf.EnableGlobalIndex = false }) } -func (s *testSerialDBSuite1) TestAlterTableExchangePartition(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAlterTableExchangePartition(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists e") tk.MustExec("drop table if exists e2") @@ -1834,10 +1878,10 @@ func (s *testSerialDBSuite1) TestAlterTableExchangePartition(c *C) { tk.MustGetErrCode("alter table e12 exchange partition p0 with table e14", tmysql.ErrPartitionExchangeDifferentOption) // test for tiflash replica - c.Assert(failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`), IsNil) + require.Nil(t, failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`)) defer func() { err := failpoint.Disable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount") - c.Assert(err, IsNil) + require.NoError(t, err) }() tk.MustExec("create table e15 (a int) partition by hash(a) partitions 1;") @@ -1845,15 +1889,15 @@ func (s *testSerialDBSuite1) TestAlterTableExchangePartition(c *C) { tk.MustExec("alter table e15 set tiflash replica 1;") tk.MustExec("alter table e16 set tiflash replica 2;") - e15 := testGetTableByName(c, s.ctx, "test", "e15") + e15 := tk.GetTableByName("test", "e15") partition := e15.Meta().Partition - err := domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, partition.Definitions[0].ID, true) - c.Assert(err, IsNil) + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), partition.Definitions[0].ID, true) + require.NoError(t, err) - e16 := testGetTableByName(c, s.ctx, "test", "e16") - err = domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, e16.Meta().ID, true) - c.Assert(err, IsNil) + e16 := tk.GetTableByName("test", "e16") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), e16.Meta().ID, true) + require.NoError(t, err) tk.MustGetErrCode("alter table e15 exchange partition p0 with table e16", tmysql.ErrTablesDifferentMetadata) tk.MustExec("drop table e15, e16") @@ -1863,29 +1907,29 @@ func (s *testSerialDBSuite1) TestAlterTableExchangePartition(c *C) { tk.MustExec("alter table e15 set tiflash replica 1;") tk.MustExec("alter table e16 set tiflash replica 1;") - e15 = testGetTableByName(c, s.ctx, "test", "e15") + e15 = tk.GetTableByName("test", "e15") partition = e15.Meta().Partition - err = domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, partition.Definitions[0].ID, true) - c.Assert(err, IsNil) + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), partition.Definitions[0].ID, true) + require.NoError(t, err) - e16 = testGetTableByName(c, s.ctx, "test", "e16") - err = domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, e16.Meta().ID, true) - c.Assert(err, IsNil) + e16 = tk.GetTableByName("test", "e16") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), e16.Meta().ID, true) + require.NoError(t, err) tk.MustExec("alter table e15 exchange partition p0 with table e16") - e15 = testGetTableByName(c, s.ctx, "test", "e15") + e15 = tk.GetTableByName("test", "e15") partition = e15.Meta().Partition - c.Assert(e15.Meta().TiFlashReplica, NotNil) - c.Assert(e15.Meta().TiFlashReplica.Available, IsTrue) - c.Assert(e15.Meta().TiFlashReplica.AvailablePartitionIDs, DeepEquals, []int64{partition.Definitions[0].ID}) + require.NotNil(t, e15.Meta().TiFlashReplica) + require.True(t, e15.Meta().TiFlashReplica.Available) + require.Equal(t, []int64{partition.Definitions[0].ID}, e15.Meta().TiFlashReplica.AvailablePartitionIDs) - e16 = testGetTableByName(c, s.ctx, "test", "e16") - c.Assert(e16.Meta().TiFlashReplica, NotNil) - c.Assert(e16.Meta().TiFlashReplica.Available, IsTrue) + e16 = tk.GetTableByName("test", "e16") + require.NotNil(t, e16.Meta().TiFlashReplica) + require.True(t, e16.Meta().TiFlashReplica.Available) tk.MustExec("drop table e15, e16") tk.MustExec("create table e15 (a int) partition by hash(a) partitions 1;") @@ -1898,20 +1942,22 @@ func (s *testSerialDBSuite1) TestAlterTableExchangePartition(c *C) { tk.MustExec("alter table e16 set tiflash replica 1 location labels 'a', 'b';") - e15 = testGetTableByName(c, s.ctx, "test", "e15") + e15 = tk.GetTableByName("test", "e15") partition = e15.Meta().Partition - err = domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, partition.Definitions[0].ID, true) - c.Assert(err, IsNil) + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), partition.Definitions[0].ID, true) + require.NoError(t, err) - e16 = testGetTableByName(c, s.ctx, "test", "e16") - err = domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, e16.Meta().ID, true) - c.Assert(err, IsNil) + e16 = tk.GetTableByName("test", "e16") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), e16.Meta().ID, true) + require.NoError(t, err) tk.MustExec("alter table e15 exchange partition p0 with table e16") } -func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { +func TestExchangePartitionTableCompatiable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() type testCase struct { ptSQL string ntSQL string @@ -2093,29 +2139,41 @@ func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { }, } - tk := testkit.NewTestKit(c, s.store) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - err := tk.Se.GetSessionVars().SetSystemVar("tidb_enable_exchange_partition", "1") - c.Assert(err, IsNil) - for i, t := range cases { - tk.MustExec(t.ptSQL) - tk.MustExec(t.ntSQL) - if t.err != nil { - _, err := tk.Exec(t.exchangeSQL) - c.Assert(terror.ErrorEqual(err, t.err), IsTrue, Commentf( + err := tk.Session().GetSessionVars().SetSystemVar("tidb_enable_exchange_partition", "1") + require.NoError(t, err) + for i, tt := range cases { + tk.MustExec(tt.ptSQL) + tk.MustExec(tt.ntSQL) + if tt.err != nil { + _, err := tk.Exec(tt.exchangeSQL) + require.Truef(t, terror.ErrorEqual(err, tt.err), "case %d fail, sql = `%s`\nexpected error = `%v`\n actual error = `%v`", - i, t.exchangeSQL, t.err, err, - )) + i, tt.exchangeSQL, tt.err, err, + ) } else { - tk.MustExec(t.exchangeSQL) + tk.MustExec(tt.exchangeSQL) } } - err = tk.Se.GetSessionVars().SetSystemVar("tidb_enable_exchange_partition", "0") - c.Assert(err, IsNil) + err = tk.Session().GetSessionVars().SetSystemVar("tidb_enable_exchange_partition", "0") + require.NoError(t, err) } -func (s *testSerialDBSuite1) TestExchangePartitionExpressIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestExchangePartitionExpressIndex(t *testing.T) { + restore := config.RestoreFunc() + defer restore() + config.UpdateGlobal(func(conf *config.Config) { + // Test for table lock. + conf.EnableTableLock = true + conf.Log.SlowThreshold = 10000 + conf.TiKVClient.AsyncCommit.SafeWindow = 0 + conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0 + conf.Experimental.AllowsExpressionIndex = true + }) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@tidb_enable_exchange_partition=1") defer tk.MustExec("set @@tidb_enable_exchange_partition=0") @@ -2147,8 +2205,10 @@ func (s *testSerialDBSuite1) TestExchangePartitionExpressIndex(c *C) { } -func (s *testIntegrationSuite4) TestAddPartitionTooManyPartitions(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAddPartitionTooManyPartitions(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") count := ddl.PartitionCountLimit tk.MustExec("drop table if exists p1;") @@ -2179,13 +2239,13 @@ func (s *testIntegrationSuite4) TestAddPartitionTooManyPartitions(c *C) { tk.MustGetErrCode(sql3, tmysql.ErrTooManyPartitions) } -func waitGCDeleteRangeDone(c *C, tk *testkit.TestKit, physicalID int64) bool { +func waitGCDeleteRangeDone(t *testing.T, tk *testkit.TestKit, physicalID int64) bool { var i int for i = 0; i < waitForCleanDataRound; i++ { rs, err := tk.Exec("select count(1) from mysql.gc_delete_range_done where element_id = ?", physicalID) - c.Assert(err, IsNil) - rows, err := session.ResultSetToStringSlice(context.Background(), tk.Se, rs) - c.Assert(err, IsNil) + require.NoError(t, err) + rows, err := session.ResultSetToStringSlice(context.Background(), tk.Session(), rs) + require.NoError(t, err) val := rows[0][0] if val != "0" { return true @@ -2196,11 +2256,11 @@ func waitGCDeleteRangeDone(c *C, tk *testkit.TestKit, physicalID int64) bool { return false } -func checkPartitionDelRangeDone(c *C, tk *testkit.TestKit, s *testIntegrationSuite, oldPID int64) { +func checkPartitionDelRangeDone(t *testing.T, tk *testkit.TestKit, store kv.Storage, oldPID int64) { startTime := time.Now() partitionPrefix := tablecodec.EncodeTablePrefix(oldPID) - done := waitGCDeleteRangeDone(c, tk, oldPID) + done := waitGCDeleteRangeDone(t, tk, oldPID) if !done { // Takes too long, give up the check. logutil.BgLogger().Info("truncate partition table", @@ -2211,7 +2271,7 @@ func checkPartitionDelRangeDone(c *C, tk *testkit.TestKit, s *testIntegrationSui } hasOldPartitionData := true - err := kv.RunInNewTxn(context.Background(), s.store, false, func(ctx context.Context, txn kv.Transaction) error { + err := kv.RunInNewTxn(context.Background(), store, false, func(ctx context.Context, txn kv.Transaction) error { it, err := txn.Iter(partitionPrefix, nil) if err != nil { return err @@ -2224,12 +2284,14 @@ func checkPartitionDelRangeDone(c *C, tk *testkit.TestKit, s *testIntegrationSui it.Close() return nil }) - c.Assert(err, IsNil) - c.Assert(hasOldPartitionData, IsFalse) + require.NoError(t, err) + require.False(t, hasOldPartitionData) } -func (s *testIntegrationSuite5) TestTruncatePartitionAndDropTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestTruncatePartitionAndDropTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") // Test truncate common table. tk.MustExec("drop table if exists t1;") @@ -2281,14 +2343,14 @@ func (s *testIntegrationSuite5) TestTruncatePartitionAndDropTable(c *C) { (10, 'lava lamp', '1998-12-25');`) result = tk.MustQuery("select count(*) from t3;") result.Check(testkit.Rows("10")) - ctx := tk.Se.(sessionctx.Context) + ctx := tk.Session() is := domain.GetDomain(ctx).InfoSchema() oldTblInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t3")) - c.Assert(err, IsNil) + require.NoError(t, err) // Only one partition id test is taken here. tk.MustExec("truncate table t3;") oldPID := oldTblInfo.Meta().Partition.Definitions[0].ID - checkPartitionDelRangeDone(c, tk, s.testIntegrationSuite, oldPID) + checkPartitionDelRangeDone(t, tk, store, oldPID) // Test drop table partition. tk.MustExec("drop table if exists t4;") @@ -2319,11 +2381,11 @@ func (s *testIntegrationSuite5) TestTruncatePartitionAndDropTable(c *C) { result.Check(testkit.Rows("10")) is = domain.GetDomain(ctx).InfoSchema() oldTblInfo, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t4")) - c.Assert(err, IsNil) + require.NoError(t, err) // Only one partition id test is taken here. oldPID = oldTblInfo.Meta().Partition.Definitions[1].ID tk.MustExec("drop table t4;") - checkPartitionDelRangeDone(c, tk, s.testIntegrationSuite, oldPID) + checkPartitionDelRangeDone(t, tk, store, oldPID) tk.MustGetErrCode("select * from t4;", tmysql.ErrNoSuchTable) // Test truncate table partition reassigns new partitionIDs. @@ -2343,15 +2405,15 @@ func (s *testIntegrationSuite5) TestTruncatePartitionAndDropTable(c *C) { );`) is = domain.GetDomain(ctx).InfoSchema() oldTblInfo, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t5")) - c.Assert(err, IsNil) + require.NoError(t, err) oldPID = oldTblInfo.Meta().Partition.Definitions[0].ID tk.MustExec("truncate table t5;") is = domain.GetDomain(ctx).InfoSchema() newTblInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t5")) - c.Assert(err, IsNil) + require.NoError(t, err) newPID := newTblInfo.Meta().Partition.Definitions[0].ID - c.Assert(oldPID != newPID, IsTrue) + require.True(t, oldPID != newPID) tk.MustExec("set @@session.tidb_enable_table_partition = 1;") tk.MustExec("drop table if exists clients;") @@ -2365,22 +2427,24 @@ func (s *testIntegrationSuite5) TestTruncatePartitionAndDropTable(c *C) { partitions 12;`) is = domain.GetDomain(ctx).InfoSchema() oldTblInfo, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("clients")) - c.Assert(err, IsNil) + require.NoError(t, err) oldDefs := oldTblInfo.Meta().Partition.Definitions // Test truncate `hash partitioned table` reassigns new partitionIDs. tk.MustExec("truncate table clients;") is = domain.GetDomain(ctx).InfoSchema() newTblInfo, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("clients")) - c.Assert(err, IsNil) + require.NoError(t, err) newDefs := newTblInfo.Meta().Partition.Definitions for i := 0; i < len(oldDefs); i++ { - c.Assert(oldDefs[i].ID != newDefs[i].ID, IsTrue) + require.True(t, oldDefs[i].ID != newDefs[i].ID) } } -func (s *testIntegrationSuite5) TestPartitionUniqueKeyNeedAllFieldsInPf(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPartitionUniqueKeyNeedAllFieldsInPf(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists part1;") tk.MustExec(`create table part1 ( @@ -2631,24 +2695,28 @@ func (s *testIntegrationSuite5) TestPartitionUniqueKeyNeedAllFieldsInPf(c *C) { tk.MustExec(sql13) } -func (s *testIntegrationSuite2) TestPartitionDropPrimaryKey(c *C) { +func TestPartitionDropPrimaryKey(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() idxName := "primary" addIdxSQL := "alter table partition_drop_idx add primary key idx1 (c1);" dropIdxSQL := "alter table partition_drop_idx drop primary key;" - testPartitionDropIndex(c, s.store, s.lease, idxName, addIdxSQL, dropIdxSQL) + testPartitionDropIndex(t, store, 50*time.Millisecond, idxName, addIdxSQL, dropIdxSQL) } -func (s *testIntegrationSuite3) TestPartitionDropIndex(c *C) { +func TestPartitionDropIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() idxName := "idx1" addIdxSQL := "alter table partition_drop_idx add index idx1 (c1);" dropIdxSQL := "alter table partition_drop_idx drop index idx1;" - testPartitionDropIndex(c, s.store, s.lease, idxName, addIdxSQL, dropIdxSQL) + testPartitionDropIndex(t, store, 50*time.Millisecond, idxName, addIdxSQL, dropIdxSQL) } -func testPartitionDropIndex(c *C, store kv.Storage, lease time.Duration, idxName, addIdxSQL, dropIdxSQL string) { - tk := testkit.NewTestKit(c, store) +func testPartitionDropIndex(t *testing.T, store kv.Storage, lease time.Duration, idxName, addIdxSQL, dropIdxSQL string) { + tk := testkit.NewTestKit(t, store) done := make(chan error, 1) - tk.MustExec("use test_db") + tk.MustExec("use test") tk.MustExec("drop table if exists partition_drop_idx;") tk.MustExec(`create table partition_drop_idx ( c1 int, c2 int, c3 int @@ -2669,12 +2737,12 @@ func testPartitionDropIndex(c *C, store kv.Storage, lease time.Duration, idxName } tk.MustExec(addIdxSQL) - ctx := tk.Se.(sessionctx.Context) - indexID := testGetIndexID(c, ctx, "test_db", "partition_drop_idx", idxName) + ctx := tk.Session() + indexID := testGetIndexIDT(t, ctx, "test", "partition_drop_idx", idxName) jobIDExt, reset := setupJobIDExtCallback(ctx) defer reset() - testutil.SessionExecInGoroutine(store, dropIdxSQL, done) + testutil.ExecMultiSQLInGoroutine(store, "test", []string{dropIdxSQL}, done) ticker := time.NewTicker(lease / 2) defer ticker.Stop() LOOP: @@ -2684,7 +2752,7 @@ LOOP: if err == nil { break LOOP } - c.Assert(err, IsNil, Commentf("err:%v", errors.ErrorStack(err))) + require.NoError(t, err) case <-ticker.C: step := 10 rand.Seed(time.Now().Unix()) @@ -2696,26 +2764,41 @@ LOOP: num += step } } - checkDelRangeAdded(tk, jobIDExt.jobID, indexID) + checkDelRangeAddedN(tk, jobIDExt.jobID, indexID) tk.MustExec("drop table partition_drop_idx;") } -func (s *testIntegrationSuite2) TestPartitionCancelAddPrimaryKey(c *C) { +func TestPartitionCancelAddPrimaryKey(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() idxName := "primary" addIdxSQL := "alter table t1 add primary key c3_index (c1);" - testPartitionCancelAddIndex(c, s.store, s.dom.DDL(), s.lease, idxName, addIdxSQL) + testPartitionCancelAddIndex(t, store, dom.DDL(), 50*time.Millisecond, idxName, addIdxSQL) } -func (s *testIntegrationSuite4) TestPartitionCancelAddIndex(c *C) { +func TestPartitionCancelAddIndex(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() idxName := "c3_index" addIdxSQL := "create unique index c3_index on t1 (c1)" - testPartitionCancelAddIndex(c, s.store, s.dom.DDL(), s.lease, idxName, addIdxSQL) + testPartitionCancelAddIndex(t, store, dom.DDL(), 50*time.Millisecond, idxName, addIdxSQL) +} + +func batchInsertT(tk *testkit.TestKit, tbl string, start, end int) { + dml := fmt.Sprintf("insert into %s values", tbl) + for i := start; i < end; i++ { + dml += fmt.Sprintf("(%d, %d, %d)", i, i, i) + if i != end-1 { + dml += "," + } + } + tk.MustExec(dml) } -func testPartitionCancelAddIndex(c *C, store kv.Storage, d ddl.DDL, lease time.Duration, idxName, addIdxSQL string) { - tk := testkit.NewTestKit(c, store) +func testPartitionCancelAddIndex(t *testing.T, store kv.Storage, d ddl.DDL, lease time.Duration, idxName, addIdxSQL string) { + tk := testkit.NewTestKit(t, store) - tk.MustExec("use test_db") + tk.MustExec("use test") tk.MustExec("drop table if exists t1;") tk.MustExec(`create table t1 ( c1 int, c2 int, c3 int @@ -2730,7 +2813,7 @@ func testPartitionCancelAddIndex(c *C, store kv.Storage, d ddl.DDL, lease time.D count := defaultBatchSize * 32 // add some rows for i := 0; i < count; i += defaultBatchSize { - batchInsert(tk, "t1", i, i+defaultBatchSize) + batchInsertT(tk, "t1", i, i+defaultBatchSize) } var checkErr error @@ -2739,15 +2822,14 @@ func testPartitionCancelAddIndex(c *C, store kv.Storage, d ddl.DDL, lease time.D originBatchSize := tk.MustQuery("select @@global.tidb_ddl_reorg_batch_size") // Set batch size to lower try to slow down add-index reorganization, This if for hook to cancel this ddl job. tk.MustExec("set @@global.tidb_ddl_reorg_batch_size = 32") - ctx := tk.Se.(sessionctx.Context) defer tk.MustExec(fmt.Sprintf("set @@global.tidb_ddl_reorg_batch_size = %v", originBatchSize.Rows()[0][0])) - hook.OnJobUpdatedExported, c3IdxInfo, checkErr = backgroundExecOnJobUpdatedExported(c, store, ctx, hook, idxName) + hook.OnJobUpdatedExported, c3IdxInfo, checkErr = backgroundExecOnJobUpdatedExportedT(tk, store, hook, idxName) originHook := d.GetHook() defer d.(ddl.DDLForTest).SetHook(originHook) jobIDExt := wrapJobIDExtCallback(hook) d.(ddl.DDLForTest).SetHook(jobIDExt) done := make(chan error, 1) - go backgroundExec(store, addIdxSQL, done) + go backgroundExecT(store, addIdxSQL, done) times := 0 ticker := time.NewTicker(lease / 2) @@ -2756,9 +2838,8 @@ LOOP: for { select { case err := <-done: - c.Assert(checkErr, IsNil) - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:8214]Cancelled DDL job") + require.Nil(t, checkErr) + require.EqualError(t, err, "[ddl:8214]Cancelled DDL job") break LOOP case <-ticker.C: if times >= 10 { @@ -2776,11 +2857,11 @@ LOOP: times++ } } - checkDelRangeAdded(tk, jobIDExt.jobID, c3IdxInfo.ID) + checkDelRangeAddedN(tk, jobIDExt.jobID, c3IdxInfo.ID) tk.MustExec("drop table t1") } -func backgroundExecOnJobUpdatedExported(c *C, store kv.Storage, ctx sessionctx.Context, hook *ddl.TestDDLCallback, idxName string) ( +func backgroundExecOnJobUpdatedExportedT(tk *testkit.TestKit, store kv.Storage, hook *ddl.TestDDLCallback, idxName string) ( func(*model.Job), *model.IndexInfo, error) { var checkErr error first := true @@ -2795,7 +2876,7 @@ func backgroundExecOnJobUpdatedExported(c *C, store kv.Storage, ctx sessionctx.C if c3IdxInfo.ID != 0 { return } - t := testGetTableByName(c, ctx, "test_db", "t1") + t := tk.GetTableByName("test", "t1") for _, index := range t.Indices() { if !tables.IsIndexWritable(index) { continue @@ -2850,17 +2931,21 @@ func backgroundExecOnJobUpdatedExported(c *C, store kv.Storage, ctx sessionctx.C return hook.OnJobUpdatedExported, c3IdxInfo, checkErr } -func (s *testIntegrationSuite5) TestPartitionAddPrimaryKey(c *C) { - tk := testkit.NewTestKit(c, s.store) - testPartitionAddIndexOrPK(c, tk, "primary key") +func TestPartitionAddPrimaryKey(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + testPartitionAddIndexOrPK(t, tk, "primary key") } -func (s *testIntegrationSuite1) TestPartitionAddIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) - testPartitionAddIndexOrPK(c, tk, "index") +func TestPartitionAddIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + testPartitionAddIndexOrPK(t, tk, "index") } -func testPartitionAddIndexOrPK(c *C, tk *testkit.TestKit, key string) { +func testPartitionAddIndexOrPK(t *testing.T, tk *testkit.TestKit, key string) { tk.MustExec("use test") tk.MustExec(`create table partition_add_idx ( id int not null, @@ -2874,7 +2959,7 @@ func testPartitionAddIndexOrPK(c *C, tk *testkit.TestKit, key string) { partition p6 values less than (2012), partition p7 values less than (2018) );`) - testPartitionAddIndex(tk, c, key) + testPartitionAddIndex(tk, t, key) // test hash partition table. tk.MustExec("set @@session.tidb_enable_table_partition = '1';") @@ -2883,7 +2968,7 @@ func testPartitionAddIndexOrPK(c *C, tk *testkit.TestKit, key string) { id int not null, hired date not null ) partition by hash( year(hired) ) partitions 4;`) - testPartitionAddIndex(tk, c, key) + testPartitionAddIndex(tk, t, key) // Test hash partition for pr 10475. tk.MustExec("drop table if exists t1") @@ -2902,7 +2987,7 @@ func testPartitionAddIndexOrPK(c *C, tk *testkit.TestKit, key string) { tk.MustExec("admin check table t1;") } -func testPartitionAddIndex(tk *testkit.TestKit, c *C, key string) { +func testPartitionAddIndex(tk *testkit.TestKit, t *testing.T, key string) { idxName1 := "idx1" f := func(end int, isPK bool) string { @@ -2931,18 +3016,18 @@ func testPartitionAddIndex(tk *testkit.TestKit, c *C, key string) { tk.MustExec(fmt.Sprintf("alter table partition_add_idx add %s idx1 (hired)", key)) tk.MustExec("alter table partition_add_idx add index idx2 (id, hired)") - ctx := tk.Se.(sessionctx.Context) + ctx := tk.Session() is := domain.GetDomain(ctx).InfoSchema() - t, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("partition_add_idx")) - c.Assert(err, IsNil) + tt, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("partition_add_idx")) + require.NoError(t, err) var idx1 table.Index - for _, idx := range t.Indices() { + for _, idx := range tt.Indices() { if idx.Meta().Name.L == idxName1 { idx1 = idx break } } - c.Assert(idx1, NotNil) + require.NotNil(t, idx1) tk.MustQuery(fmt.Sprintf("select count(hired) from partition_add_idx use index(%s)", idxName1)).Check(testkit.Rows("500")) tk.MustQuery("select count(id) from partition_add_idx use index(idx2)").Check(testkit.Rows("500")) @@ -2951,8 +3036,10 @@ func testPartitionAddIndex(tk *testkit.TestKit, c *C, key string) { tk.MustExec("drop table partition_add_idx") } -func (s *testIntegrationSuite5) TestDropSchemaWithPartitionTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestDropSchemaWithPartitionTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("drop database if exists test_db_with_partition") tk.MustExec("create database test_db_with_partition") tk.MustExec("use test_db_with_partition") @@ -2962,39 +3049,39 @@ func (s *testIntegrationSuite5) TestDropSchemaWithPartitionTable(c *C) { partition p1 values less than (20) );`) tk.MustExec("insert into t_part values (1),(2),(11),(12);") - ctx := s.ctx - tbl := testGetTableByName(c, ctx, "test_db_with_partition", "t_part") + ctx := tk.Session() + tbl := tk.GetTableByName("test_db_with_partition", "t_part") // check records num before drop database. - recordsNum := getPartitionTableRecordsNum(c, ctx, tbl.(table.PartitionedTable)) - c.Assert(recordsNum, Equals, 4) + recordsNum := getPartitionTableRecordsNum(t, ctx, tbl.(table.PartitionedTable)) + require.Equal(t, 4, recordsNum) tk.MustExec("drop database if exists test_db_with_partition") // check job args. rs, err := tk.Exec("admin show ddl jobs") - c.Assert(err, IsNil) - rows, err := session.GetRows4Test(context.Background(), tk.Se, rs) - c.Assert(err, IsNil) + require.NoError(t, err) + rows, err := session.GetRows4Test(context.Background(), tk.Session(), rs) + require.NoError(t, err) row := rows[0] - c.Assert(row.GetString(3), Equals, "drop schema") + require.Equal(t, "drop schema", row.GetString(3)) jobID := row.GetInt64(0) var tableIDs []int64 - err = kv.RunInNewTxn(context.Background(), s.store, false, func(ctx context.Context, txn kv.Transaction) error { - t := meta.NewMeta(txn) - historyJob, err := t.GetHistoryDDLJob(jobID) - c.Assert(err, IsNil) + err = kv.RunInNewTxn(context.Background(), store, false, func(ctx context.Context, txn kv.Transaction) error { + tt := meta.NewMeta(txn) + historyJob, err := tt.GetHistoryDDLJob(jobID) + require.NoError(t, err) err = historyJob.DecodeArgs(&tableIDs) - c.Assert(err, IsNil) + require.NoError(t, err) // There is 2 partitions. - c.Assert(len(tableIDs), Equals, 3) + require.Equal(t, 3, len(tableIDs)) return nil }) - c.Assert(err, IsNil) + require.NoError(t, err) startTime := time.Now() - done := waitGCDeleteRangeDone(c, tk, tableIDs[2]) + done := waitGCDeleteRangeDone(t, tk, tableIDs[2]) if !done { // Takes too long, give up the check. logutil.BgLogger().Info("drop schema", @@ -3006,35 +3093,37 @@ func (s *testIntegrationSuite5) TestDropSchemaWithPartitionTable(c *C) { // check records num after drop database. for i := 0; i < waitForCleanDataRound; i++ { - recordsNum = getPartitionTableRecordsNum(c, ctx, tbl.(table.PartitionedTable)) + recordsNum = getPartitionTableRecordsNum(t, ctx, tbl.(table.PartitionedTable)) if recordsNum != 0 { time.Sleep(waitForCleanDataInterval) } else { break } } - c.Assert(recordsNum, Equals, 0) + require.Equal(t, 0, recordsNum) } -func getPartitionTableRecordsNum(c *C, ctx sessionctx.Context, tbl table.PartitionedTable) int { +func getPartitionTableRecordsNum(t *testing.T, ctx sessionctx.Context, tbl table.PartitionedTable) int { num := 0 info := tbl.Meta().GetPartitionInfo() for _, def := range info.Definitions { pid := def.ID partition := tbl.GetPartition(pid) - c.Assert(ctx.NewTxn(context.Background()), IsNil) + require.Nil(t, ctx.NewTxn(context.Background())) err := tables.IterRecords(partition, ctx, partition.Cols(), func(_ kv.Handle, data []types.Datum, cols []*table.Column) (bool, error) { num++ return true, nil }) - c.Assert(err, IsNil) + require.NoError(t, err) } return num } -func (s *testIntegrationSuite3) TestPartitionErrorCode(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPartitionErrorCode(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := 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") @@ -3052,10 +3141,10 @@ func (s *testIntegrationSuite3) TestPartitionErrorCode(c *C) { partition by hash(store_id) partitions 4;`) _, err := tk.Exec("alter table employees add partition partitions 8;") - c.Assert(ddl.ErrUnsupportedAddPartition.Equal(err), IsTrue) + require.True(t, ddl.ErrUnsupportedAddPartition.Equal(err)) _, err = tk.Exec("alter table employees add partition (partition p5 values less than (42));") - c.Assert(ddl.ErrUnsupportedAddPartition.Equal(err), IsTrue) + require.True(t, ddl.ErrUnsupportedAddPartition.Equal(err)) // coalesce partition tk.MustExec(`create table clients ( @@ -3067,7 +3156,7 @@ func (s *testIntegrationSuite3) TestPartitionErrorCode(c *C) { partition by hash( month(signed) ) partitions 12;`) _, err = tk.Exec("alter table clients coalesce partition 4;") - c.Assert(ddl.ErrUnsupportedCoalescePartition.Equal(err), IsTrue) + require.True(t, ddl.ErrUnsupportedCoalescePartition.Equal(err)) tk.MustExec(`create table t_part (a int key) partition by range(a) ( @@ -3075,7 +3164,7 @@ func (s *testIntegrationSuite3) TestPartitionErrorCode(c *C) { partition p1 values less than (20) );`) _, err = tk.Exec("alter table t_part coalesce partition 4;") - c.Assert(ddl.ErrCoalesceOnlyOnHashPartition.Equal(err), IsTrue) + require.True(t, ddl.ErrCoalesceOnlyOnHashPartition.Equal(err)) tk.MustGetErrCode(`alter table t_part reorganize partition p0, p1 into ( partition p0 values less than (1980));`, tmysql.ErrUnsupportedDDLOperation) @@ -3087,7 +3176,7 @@ func (s *testIntegrationSuite3) TestPartitionErrorCode(c *C) { tk.MustGetErrCode("alter table t_part repair partition p1;", tmysql.ErrUnsupportedDDLOperation) // Reduce the impact on DML when executing partition DDL - tk1 := testkit.NewTestKit(c, s.store) + tk1 := testkit.NewTestKit(t, store) tk1.MustExec("use test") tk1.MustExec("drop table if exists t;") tk1.MustExec(`create table t(id int primary key) @@ -3095,16 +3184,18 @@ func (s *testIntegrationSuite3) TestPartitionErrorCode(c *C) { tk1.MustExec("begin") tk1.MustExec("insert into t values(1);") - tk2 := testkit.NewTestKit(c, s.store) + tk2 := testkit.NewTestKit(t, store) tk2.MustExec("use test") tk2.MustExec("alter table t truncate partition p0;") _, err = tk1.Exec("commit") - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *testIntegrationSuite5) TestConstAndTimezoneDepent(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestConstAndTimezoneDepent(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := 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_const") @@ -3191,8 +3282,10 @@ func (s *testIntegrationSuite5) TestConstAndTimezoneDepent(c *C) { tk.MustGetErrCode(sql14, tmysql.ErrWrongExprInPartitionFunc) } -func (s *testIntegrationSuite5) TestConstAndTimezoneDepent2(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestConstAndTimezoneDepent2(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := 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_const") @@ -3221,8 +3314,10 @@ func (s *testIntegrationSuite5) TestConstAndTimezoneDepent2(c *C) { );`) } -func (s *testIntegrationSuite3) TestUnsupportedPartitionManagementDDLs(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestUnsupportedPartitionManagementDDLs(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists test_1465;") tk.MustExec(` @@ -3235,11 +3330,22 @@ func (s *testIntegrationSuite3) TestUnsupportedPartitionManagementDDLs(c *C) { `) _, err := tk.Exec("alter table test_1465 partition by hash(a)") - c.Assert(err, ErrorMatches, ".*alter table partition is unsupported") + require.Regexp(t, ".*alter table partition is unsupported", err.Error()) } -func (s *testSerialDBSuite1) TestCommitWhenSchemaChange(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCommitWhenSchemaChange(t *testing.T) { + restore := config.RestoreFunc() + defer restore() + config.UpdateGlobal(func(conf *config.Config) { + // Test for table lock. + conf.EnableTableLock = true + conf.Log.SlowThreshold = 10000 + conf.Experimental.AllowsExpressionIndex = true + }) + store, clean := testkit.CreateMockStoreWithSchemaLease(t, time.Second) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@global.tidb_max_delta_schema_count= 4096") tk.MustExec("use test") tk.MustExec(`create table schema_change (a int, b timestamp) partition by range(a) ( @@ -3247,7 +3353,7 @@ func (s *testSerialDBSuite1) TestCommitWhenSchemaChange(c *C) { partition p1 values less than (7), partition p2 values less than (11) )`) - tk2 := testkit.NewTestKit(c, s.store) + tk2 := testkit.NewTestKit(t, store) tk2.MustExec("use test") tk2.MustExec("set @@tidb_enable_exchange_partition=1") defer tk2.MustExec("set @@tidb_enable_exchange_partition=0") @@ -3264,8 +3370,9 @@ func (s *testSerialDBSuite1) TestCommitWhenSchemaChange(c *C) { defer func() { atomic.StoreUint32(&session.SchemaChangedWithoutRetry, 0) }() - _, err := tk.Se.Execute(context.Background(), "commit") - c.Assert(domain.ErrInfoSchemaChanged.Equal(err), IsTrue) + _, err := tk.Exec("commit") + require.Error(t, err) + require.Truef(t, domain.ErrInfoSchemaChanged.Equal(err), err.Error()) // Cover a bug that schema validator does not prevent transaction commit when // the schema has changed on the partitioned table. @@ -3282,8 +3389,8 @@ func (s *testSerialDBSuite1) TestCommitWhenSchemaChange(c *C) { tk.MustExec("insert into nt values (1), (3), (5);") tk2.MustExec("alter table pt exchange partition p1 with table nt;") tk.MustExec("insert into nt values (7), (9);") - _, err = tk.Se.Execute(context.Background(), "commit") - c.Assert(domain.ErrInfoSchemaChanged.Equal(err), IsTrue) + _, err = tk.Session().Execute(context.Background(), "commit") + require.True(t, domain.ErrInfoSchemaChanged.Equal(err)) tk.MustExec("admin check table pt") tk.MustQuery("select * from pt").Check(testkit.Rows()) @@ -3294,8 +3401,8 @@ func (s *testSerialDBSuite1) TestCommitWhenSchemaChange(c *C) { tk.MustExec("insert into pt values (1), (3), (5);") tk2.MustExec("alter table pt exchange partition p1 with table nt;") tk.MustExec("insert into pt values (7), (9);") - _, err = tk.Se.Execute(context.Background(), "commit") - c.Assert(domain.ErrInfoSchemaChanged.Equal(err), IsTrue) + _, err = tk.Session().Execute(context.Background(), "commit") + require.True(t, domain.ErrInfoSchemaChanged.Equal(err)) tk.MustExec("admin check table pt") tk.MustQuery("select * from pt").Check(testkit.Rows()) @@ -3303,8 +3410,10 @@ func (s *testSerialDBSuite1) TestCommitWhenSchemaChange(c *C) { tk.MustQuery("select * from nt").Check(testkit.Rows()) } -func (s *testSerialDBSuite1) TestCreatePartitionTableWithWrongType(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCreatePartitionTableWithWrongType(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") var err error @@ -3314,8 +3423,8 @@ func (s *testSerialDBSuite1) TestCreatePartitionTableWithWrongType(c *C) { partition p0 values less than (0x10), partition p3 values less than (0x20) )`) - c.Assert(err, NotNil) - c.Assert(ddl.ErrWrongTypeColumnValue.Equal(err), IsTrue) + require.Error(t, err) + require.True(t, ddl.ErrWrongTypeColumnValue.Equal(err)) _, err = tk.Exec(`create table t( b int(10) @@ -3323,8 +3432,8 @@ func (s *testSerialDBSuite1) TestCreatePartitionTableWithWrongType(c *C) { partition p0 values less than ('g'), partition p3 values less than ('k') )`) - c.Assert(err, NotNil) - c.Assert(ddl.ErrWrongTypeColumnValue.Equal(err), IsTrue) + require.Error(t, err) + require.True(t, ddl.ErrWrongTypeColumnValue.Equal(err)) _, err = tk.Exec(`create table t( b char(10) @@ -3332,8 +3441,8 @@ func (s *testSerialDBSuite1) TestCreatePartitionTableWithWrongType(c *C) { partition p0 values less than (30), partition p3 values less than (60) )`) - c.Assert(err, NotNil) - c.Assert(ddl.ErrWrongTypeColumnValue.Equal(err), IsTrue) + require.Error(t, err) + require.True(t, ddl.ErrWrongTypeColumnValue.Equal(err)) _, err = tk.Exec(`create table t( b datetime @@ -3341,11 +3450,13 @@ func (s *testSerialDBSuite1) TestCreatePartitionTableWithWrongType(c *C) { partition p0 values less than ('g'), partition p3 values less than ('m') )`) - c.Assert(err, NotNil) + require.Error(t, err) } -func (s *testSerialDBSuite1) TestAddPartitionForTableWithWrongType(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAddPartitionForTableWithWrongType(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop tables if exists t_int, t_char, t_date") tk.MustExec(`create table t_int(b int(10)) @@ -3366,36 +3477,38 @@ func (s *testSerialDBSuite1) TestAddPartitionForTableWithWrongType(c *C) { var err error _, err = tk.Exec("alter table t_int add partition (partition p1 values less than ('g'))") - c.Assert(err, NotNil) - c.Assert(ddl.ErrWrongTypeColumnValue.Equal(err), IsTrue) + require.Error(t, err) + require.True(t, ddl.ErrWrongTypeColumnValue.Equal(err)) _, err = tk.Exec("alter table t_int add partition (partition p1 values less than (0x20))") - c.Assert(err, NotNil) - c.Assert(ddl.ErrWrongTypeColumnValue.Equal(err), IsTrue) + require.Error(t, err) + require.True(t, ddl.ErrWrongTypeColumnValue.Equal(err)) _, err = tk.Exec("alter table t_char add partition (partition p1 values less than (0x20))") - c.Assert(err, NotNil) - c.Assert(ddl.ErrRangeNotIncreasing.Equal(err), IsTrue) + require.Error(t, err) + require.True(t, ddl.ErrRangeNotIncreasing.Equal(err)) _, err = tk.Exec("alter table t_char add partition (partition p1 values less than (10))") - c.Assert(err, NotNil) - c.Assert(ddl.ErrWrongTypeColumnValue.Equal(err), IsTrue) + require.Error(t, err) + require.True(t, ddl.ErrWrongTypeColumnValue.Equal(err)) _, err = tk.Exec("alter table t_date add partition (partition p1 values less than ('m'))") - c.Assert(err, NotNil) - c.Assert(ddl.ErrWrongTypeColumnValue.Equal(err), IsTrue) + require.Error(t, err) + require.True(t, ddl.ErrWrongTypeColumnValue.Equal(err)) _, err = tk.Exec("alter table t_date add partition (partition p1 values less than (0x20))") - c.Assert(err, NotNil) - c.Assert(ddl.ErrWrongTypeColumnValue.Equal(err), IsTrue) + require.Error(t, err) + require.True(t, ddl.ErrWrongTypeColumnValue.Equal(err)) _, err = tk.Exec("alter table t_date add partition (partition p1 values less than (20))") - c.Assert(err, NotNil) - c.Assert(ddl.ErrWrongTypeColumnValue.Equal(err), IsTrue) + require.Error(t, err) + require.True(t, ddl.ErrWrongTypeColumnValue.Equal(err)) } -func (s *testSerialDBSuite1) TestPartitionListWithTimeType(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestPartitionListWithTimeType(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("set @@session.tidb_enable_list_partition = ON") tk.MustExec("create table t_list1(a date) partition by list columns (a) (partition p0 values in ('2010-02-02', '20180203'), partition p1 values in ('20200202'));") @@ -3403,8 +3516,10 @@ func (s *testSerialDBSuite1) TestPartitionListWithTimeType(c *C) { tk.MustQuery(`select * from t_list1 partition (p0);`).Check(testkit.Rows("2018-02-03")) } -func (s *testSerialDBSuite1) TestPartitionListWithNewCollation(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestPartitionListWithNewCollation(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t;") tk.MustExec("set @@session.tidb_enable_list_partition = ON") @@ -3415,19 +3530,21 @@ func (s *testSerialDBSuite1) TestPartitionListWithNewCollation(c *C) { tk.MustQuery(`select * from t11 partition (p0);`).Check(testkit.Rows("A", "B")) tk.MustQuery(`select * from t11 partition (p1);`).Check(testkit.Rows("c", "C", "d")) str := tk.MustQuery(`desc select * from t11 where a = 'b';`).Rows()[0][3].(string) - c.Assert(strings.Contains(str, "partition:p0"), IsTrue) + require.True(t, strings.Contains(str, "partition:p0")) } -func (s *testSerialDBSuite1) TestAddTableWithPartition(c *C) { +func TestAddTableWithPartition(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() // for global temporary table - tk := testkit.NewTestKitWithInit(c, s.store) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists global_partition_table;") tk.MustGetErrCode("create global temporary table global_partition_table (a int, b int) partition by hash(a) partitions 3 ON COMMIT DELETE ROWS;", errno.ErrPartitionNoTemporary) tk.MustExec("drop table if exists global_partition_table;") tk.MustExec("drop table if exists partition_table;") _, err := tk.Exec("create table partition_table (a int, b int) partition by hash(a) partitions 3;") - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustExec("drop table if exists partition_table;") tk.MustExec("drop table if exists partition_range_table;") tk.MustGetErrCode(`create global temporary table partition_range_table (c1 smallint(6) not null, c2 char(5) default null) partition by range ( c1 ) ( @@ -3453,7 +3570,7 @@ func (s *testSerialDBSuite1) TestAddTableWithPartition(c *C) { tk.MustExec("drop table if exists local_partition_table;") tk.MustExec("drop table if exists partition_table;") _, err = tk.Exec("create table partition_table (a int, b int) partition by hash(a) partitions 3;") - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustExec("drop table if exists partition_table;") tk.MustExec("drop table if exists local_partition_range_table;") tk.MustGetErrCode(`create temporary table local_partition_range_table (c1 smallint(6) not null, c2 char(5) default null) partition by range ( c1 ) ( @@ -3473,13 +3590,15 @@ func (s *testSerialDBSuite1) TestAddTableWithPartition(c *C) { tk.MustExec("drop table if exists local_partition_list_table;") } -func (s *testSerialDBSuite1) TestTruncatePartitionMultipleTimes(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestTruncatePartitionMultipleTimes(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("drop table if exists test.t;") tk.MustExec(`create table test.t (a int primary key) partition by range (a) ( partition p0 values less than (10), partition p1 values less than (maxvalue));`) - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) originHook := dom.DDL().GetHook() defer dom.DDL().SetHook(originHook) hook := &ddl.TestDDLCallback{} @@ -3498,60 +3617,62 @@ func (s *testSerialDBSuite1) TestTruncatePartitionMultipleTimes(c *C) { } } done1 := make(chan error, 1) - go backgroundExec(s.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 backgroundExec(s.store, "alter table test.t truncate partition p0;", done2) + go backgroundExec(store, "alter table test.t truncate partition p0;", done2) <-done1 <-done2 - c.Assert(errCount, LessEqual, int32(1)) + require.LessOrEqual(t, errCount, int32(1)) } -func (s *testSerialDBSuite1) TestAddPartitionReplicaBiggerThanTiFlashStores(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAddPartitionReplicaBiggerThanTiFlashStores(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create database if not exists test_partition2") tk.MustExec("use test_partition2") tk.MustExec("drop table if exists t1") // Build a tableInfo with replica count = 1 while there is no real tiFlash store. - c.Assert(failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`)) tk.MustExec(`create table t1 (c int) partition by range(c) ( partition p0 values less than (100), partition p1 values less than (200))`) tk.MustExec("alter table t1 set tiflash replica 1") - c.Assert(failpoint.Disable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount"), IsNil) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount")) // Mock partitions replica as available. - t1 := testGetTableByName(c, s.s, "test_partition2", "t1") + t1 := tk.GetTableByName("test_partition2", "t1") 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_partition2", "t1") - c.Assert(t1.Meta().TiFlashReplica.Available, IsTrue) + 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 = tk.GetTableByName("test_partition2", "t1") + require.True(t, t1.Meta().TiFlashReplica.Available) // Since there is no real TiFlash store (less than replica count), adding a partition will error here. err = tk.ExecToErr("alter table t1 add partition (partition p2 values less than (300));") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:-1][ddl] the tiflash replica count: 1 should be less than the total tiflash server count: 0") + require.Error(t, err) + require.EqualError(t, err, "[ddl:-1][ddl] the tiflash replica count: 1 should be less than the total tiflash server count: 0") // Test `add partition` waiting TiFlash replica can exit when its retry count is beyond the limitation. originErrCountLimit := variable.GetDDLErrorCountLimit() tk.MustExec("set @@global.tidb_ddl_error_count_limit = 3") defer func() { tk.MustExec(fmt.Sprintf("set @@global.tidb_ddl_error_count_limit = %v", originErrCountLimit)) }() - c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/mockWaitTiFlashReplica", `return(true)`), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/ddl/mockWaitTiFlashReplica", `return(true)`)) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/mockWaitTiFlashReplica"), IsNil) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/ddl/mockWaitTiFlashReplica")) }() - c.Assert(t1.Meta().TiFlashReplica.Available, IsTrue) + require.True(t, t1.Meta().TiFlashReplica.Available) err = tk.ExecToErr("alter table t1 add partition (partition p3 values less than (300));") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:-1]DDL job rollback, error msg: [ddl] add partition wait for tiflash replica to complete") + require.Error(t, err) + require.Equal(t, "[ddl:-1]DDL job rollback, error msg: [ddl] add partition wait for tiflash replica to complete", err.Error()) } func TestDropAndTruncatePartition(t *testing.T) { // Useless, but is required to initialize the global infoSync // Otherwise this test throw a "infoSyncer is not initialized" error - _, clean := ntestkit.CreateMockStore(t) + _, clean := testkit.CreateMockStore(t) defer clean() ddl.ExportTestDropAndTruncatePartition(t) @@ -3560,7 +3681,7 @@ func TestDropAndTruncatePartition(t *testing.T) { func TestTable(t *testing.T) { // Useless, but is required to initialize the global infoSync // Otherwise this test throw a "infoSyncer is not initialized" error - _, clean := ntestkit.CreateMockStore(t) + _, clean := testkit.CreateMockStore(t) defer clean() ddl.ExportTestTable(t) @@ -3569,21 +3690,23 @@ func TestTable(t *testing.T) { func TestRenameTables(t *testing.T) { // Useless, but is required to initialize the global infoSync // Otherwise this test throw a "infoSyncer is not initialized" error - _, clean := ntestkit.CreateMockStore(t) + _, clean := testkit.CreateMockStore(t) defer clean() ddl.ExportTestRenameTables(t) } func TestCreateTables(t *testing.T) { - _, clean := ntestkit.CreateMockStore(t) + _, clean := testkit.CreateMockStore(t) defer clean() ddl.ExportTestRenameTables(t) } -func (s *testIntegrationSuite1) TestDuplicatePartitionNames(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestDuplicatePartitionNames(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create database DuplicatePartitionNames") defer tk.MustExec("drop database DuplicatePartitionNames") @@ -3596,17 +3719,17 @@ func (s *testIntegrationSuite1) TestDuplicatePartitionNames(c *C) { tk.MustQuery("select * from t1").Sort().Check(testkit.Rows("2", "3")) tk.MustExec("insert into t1 values (1)") err := tk.ExecToErr("alter table t1 drop partition p1,p1") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:1507]Error in list of partitions to DROP") + require.Error(t, err) + require.Equal(t, "[ddl:1507]Error in list of partitions to DROP", err.Error()) err = tk.ExecToErr("alter table t1 drop partition p1,p9") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:1507]Error in list of partitions to DROP") + require.Error(t, err) + require.Equal(t, "[ddl:1507]Error in list of partitions to DROP", err.Error()) err = tk.ExecToErr("alter table t1 drop partition p1,p1,p1") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:1508]Cannot remove all partitions, use DROP TABLE instead") + require.Error(t, err) + require.Equal(t, "[ddl:1508]Cannot remove all partitions, use DROP TABLE instead", err.Error()) err = tk.ExecToErr("alter table t1 drop partition p1,p9,p1") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:1508]Cannot remove all partitions, use DROP TABLE instead") + require.Error(t, err) + require.Equal(t, "[ddl:1508]Cannot remove all partitions, use DROP TABLE instead", err.Error()) tk.MustQuery("select * from t1").Sort().Check(testkit.Rows("1", "2", "3")) tk.MustExec("alter table t1 drop partition p1") tk.MustQuery("select * from t1").Sort().Check(testkit.Rows("2", "3")) diff --git a/ddl/db_test.go b/ddl/db_test.go index f7c4346ca074f..f8ec6bdf9532e 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -25,6 +25,7 @@ import ( "strings" "sync" "sync/atomic" + "testing" "time" . "github.com/pingcap/check" @@ -55,6 +56,7 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" + ntestkit "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/admin" @@ -65,6 +67,7 @@ import ( "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testutil" + "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/testutils" ) @@ -296,6 +299,22 @@ func backgroundExec(s kv.Storage, sql string, done chan error) { done <- errors.Trace(err) } +func backgroundExecT(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") + if err != nil { + done <- errors.Trace(err) + return + } + _, err = se.Execute(context.Background(), sql) + done <- errors.Trace(err) +} + // TestAddPrimaryKeyRollback1 is used to test scenarios that will roll back when a duplicate primary key is encountered. func (s *testDBSuite8) TestAddPrimaryKeyRollback1(c *C) { hasNullValsInKey := false @@ -582,6 +601,76 @@ func (s *testDBSuite7) TestCancelAddIndex(c *C) { tk.MustExec("drop table t1") } +func backgroundExecOnJobUpdatedExported(c *C, store kv.Storage, ctx sessionctx.Context, hook *ddl.TestDDLCallback, idxName string) ( + func(*model.Job), *model.IndexInfo, error) { + var checkErr error + first := true + c3IdxInfo := &model.IndexInfo{} + hook.OnJobUpdatedExported = func(job *model.Job) { + addIndexNotFirstReorg := (job.Type == model.ActionAddIndex || job.Type == model.ActionAddPrimaryKey) && + job.SchemaState == model.StateWriteReorganization && job.SnapshotVer != 0 + // If the action is adding index and the state is writing reorganization, it want to test the case of cancelling the job when backfilling indexes. + // When the job satisfies this case of addIndexNotFirstReorg, the worker will start to backfill indexes. + if !addIndexNotFirstReorg { + // Get the index's meta. + if c3IdxInfo.ID != 0 { + return + } + t := testGetTableByName(c, ctx, "test_db", "t1") + for _, index := range t.Indices() { + if !tables.IsIndexWritable(index) { + continue + } + if index.Meta().Name.L == idxName { + *c3IdxInfo = *index.Meta() + } + } + return + } + // The job satisfies the case of addIndexNotFirst for the first time, the worker hasn't finished a batch of backfill indexes. + if first { + first = false + return + } + if checkErr != nil { + return + } + hookCtx := mock.NewContext() + hookCtx.Store = store + err := hookCtx.NewTxn(context.Background()) + if err != nil { + checkErr = errors.Trace(err) + return + } + jobIDs := []int64{job.ID} + txn, err := hookCtx.Txn(true) + if err != nil { + checkErr = errors.Trace(err) + return + } + errs, err := admin.CancelJobs(txn, jobIDs) + if err != nil { + checkErr = errors.Trace(err) + return + } + // It only tests cancel one DDL job. + if errs[0] != nil { + checkErr = errors.Trace(errs[0]) + return + } + txn, err = hookCtx.Txn(true) + if err != nil { + checkErr = errors.Trace(err) + return + } + err = txn.Commit(context.Background()) + if err != nil { + checkErr = errors.Trace(err) + } + } + return hook.OnJobUpdatedExported, c3IdxInfo, checkErr +} + func testCancelAddIndex(c *C, store kv.Storage, d ddl.DDL, lease time.Duration, idxName, addIdxSQL, sqlModeSQL string, dom *domain.Domain) { tk := testkit.NewTestKit(c, store) tk.MustExec("use test_db") @@ -1944,6 +2033,20 @@ func testGetIndexID(c *C, ctx sessionctx.Context, dbName, tblName, idxName strin return -1 } +func testGetIndexIDT(t *testing.T, ctx sessionctx.Context, dbName, tblName, idxName string) int64 { + is := domain.GetDomain(ctx).InfoSchema() + tt, err := is.TableByName(model.NewCIStr(dbName), model.NewCIStr(tblName)) + require.NoError(t, err) + + for _, idx := range tt.Indices() { + if idx.Meta().Name.L == idxName { + return idx.Meta().ID + } + } + t.Fatalf("index %s not found(db: %s, tbl: %s)", idxName, dbName, tblName) + return -1 +} + type testDDLJobIDCallback struct { ddl.Callback jobID int64 @@ -1982,33 +2085,11 @@ func checkDelRangeAdded(tk *testkit.TestKit, jobID int64, elemID int64) { tk.MustQuery(query, jobID, elemID, jobID, elemID).Check(testkit.Rows("1")) } -func checkGlobalIndexCleanUpDone(c *C, ctx sessionctx.Context, tblInfo *model.TableInfo, idxInfo *model.IndexInfo, pid int64) int { - c.Assert(ctx.NewTxn(context.Background()), IsNil) - txn, err := ctx.Txn(true) - c.Assert(err, IsNil) - defer func() { - err := txn.Rollback() - c.Assert(err, IsNil) - }() - - cnt := 0 - prefix := tablecodec.EncodeTableIndexPrefix(tblInfo.ID, idxInfo.ID) - it, err := txn.Iter(prefix, nil) - c.Assert(err, IsNil) - for it.Valid() { - if !it.Key().HasPrefix(prefix) { - break - } - segs := tablecodec.SplitIndexValue(it.Value()) - c.Assert(segs.PartitionID, NotNil) - _, pi, err := codec.DecodeInt(segs.PartitionID) - c.Assert(err, IsNil) - c.Assert(pi, Not(Equals), pid) - cnt++ - err = it.Next() - c.Assert(err, IsNil) - } - return cnt +func checkDelRangeAddedN(tk *ntestkit.TestKit, jobID int64, elemID int64) { + query := `select sum(cnt) from + (select count(1) cnt from mysql.gc_delete_range where job_id = ? and element_id = ? union + select count(1) cnt from mysql.gc_delete_range_done where job_id = ? and element_id = ?) as gdr;` + tk.MustQuery(query, jobID, elemID, jobID, elemID).Check(testkit.Rows("1")) } func (s *testDBSuite5) TestAlterPrimaryKey(c *C) { diff --git a/ddl/primary_key_handle_test.go b/ddl/primary_key_handle_test.go index 317bf9bf87483..12389b46b90d4 100644 --- a/ddl/primary_key_handle_test.go +++ b/ddl/primary_key_handle_test.go @@ -84,16 +84,6 @@ type testIntegrationSuite struct { ctx sessionctx.Context } -func tearDownIntegrationSuiteTest(s *testIntegrationSuite, c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - r := tk.MustQuery("show tables") - for _, tb := range r.Rows() { - tableName := tb[0] - tk.MustExec(fmt.Sprintf("drop table %v", tableName)) - } -} - func (s *testIntegrationSuite) SetUpSuite(c *C) { var err error s.lease = 50 * time.Millisecond diff --git a/testkit/mockstore.go b/testkit/mockstore.go index 181d1c609e364..afa9658b22ae5 100644 --- a/testkit/mockstore.go +++ b/testkit/mockstore.go @@ -19,6 +19,7 @@ package testkit import ( "testing" + "time" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" @@ -39,12 +40,12 @@ func CreateMockStore(t testing.TB, opts ...mockstore.MockTiKVStoreOption) (store func CreateMockStoreAndDomain(t testing.TB, opts ...mockstore.MockTiKVStoreOption) (kv.Storage, *domain.Domain, func()) { store, err := mockstore.NewMockStore(opts...) require.NoError(t, err) - dom, clean := bootstrap(t, store) + dom, clean := bootstrap(t, store, 0) return store, dom, clean } -func bootstrap(t testing.TB, store kv.Storage) (*domain.Domain, func()) { - session.SetSchemaLease(0) +func bootstrap(t testing.TB, store kv.Storage, lease time.Duration) (*domain.Domain, func()) { + session.SetSchemaLease(lease) session.DisableStats4Test() dom, err := session.BootstrapSession(store) require.NoError(t, err) @@ -59,12 +60,26 @@ func bootstrap(t testing.TB, store kv.Storage) (*domain.Domain, func()) { return dom, clean } +// CreateMockStoreWithSchemaLease return a new mock kv.Storage. +func CreateMockStoreWithSchemaLease(t testing.TB, lease time.Duration, opts ...mockstore.MockTiKVStoreOption) (store kv.Storage, clean func()) { + store, _, clean = CreateMockStoreAndDomainWithSchemaLease(t, lease, opts...) + return +} + +// CreateMockStoreAndDomainWithSchemaLease return a new mock kv.Storage and *domain.Domain. +func CreateMockStoreAndDomainWithSchemaLease(t testing.TB, lease time.Duration, opts ...mockstore.MockTiKVStoreOption) (kv.Storage, *domain.Domain, func()) { + store, err := mockstore.NewMockStore(opts...) + require.NoError(t, err) + dom, clean := bootstrap(t, store, lease) + return store, dom, clean +} + // CreateMockStoreWithOracle returns a new mock kv.Storage and *domain.Domain, providing the oracle for the store. func CreateMockStoreWithOracle(t testing.TB, oracle oracle.Oracle, opts ...mockstore.MockTiKVStoreOption) (kv.Storage, *domain.Domain, func()) { store, err := mockstore.NewMockStore(opts...) require.NoError(t, err) store.GetOracle().Close() store.(tikv.Storage).SetOracle(oracle) - dom, clean := bootstrap(t, store) + dom, clean := bootstrap(t, store, 0) return store, dom, clean }