diff --git a/ddl/column_change_test.go b/ddl/column_change_test.go index 2349c7fc3d..9a144b1565 100644 --- a/ddl/column_change_test.go +++ b/ddl/column_change_test.go @@ -19,10 +19,11 @@ import ( "fmt" "sync" "sync/atomic" + "testing" "time" - . "github.com/pingcap/check" "github.com/pingcap/errors" + "github.com/pingcap/tidb/domain/infosync" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" @@ -36,64 +37,72 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testutil" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" ) -var _ = Suite(&testColumnChangeSuite{}) - -type testColumnChangeSuite struct { +type testColumnChangeSuiteToVerify struct { + suite.Suite store kv.Storage dbInfo *model.DBInfo } -func (s *testColumnChangeSuite) SetUpSuite(c *C) { +func TestColumnChangeSuite(t *testing.T) { + _, err := infosync.GlobalInfoSyncerInit(context.Background(), "t", func() uint64 { return 1 }, nil, true) + require.NoError(t, err) + + suite.Run(t, new(testColumnChangeSuiteToVerify)) +} + +func (s *testColumnChangeSuiteToVerify) SetupSuite() { SetWaitTimeWhenErrorOccurred(1 * time.Microsecond) - s.store = testCreateStore(c, "test_column_change") + s.store = testCreateStore(s.T(), "test_column_change") d, err := testNewDDLAndStart( context.Background(), WithStore(s.store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() s.dbInfo, err = testSchemaInfo(d, "test_index_change") - c.Assert(err, IsNil) - testCreateSchema(c, testNewContext(d), d, s.dbInfo) + require.NoError(s.T(), err) + testCreateSchema(s.T(), testNewContext(d), d, s.dbInfo) } -func (s *testColumnChangeSuite) TearDownSuite(c *C) { +func (s *testColumnChangeSuiteToVerify) TearDownSuite() { s.store.Close() } -func (s *testColumnChangeSuite) TestColumnChange(c *C) { +func (s *testColumnChangeSuiteToVerify) TestColumnChange() { d, err := testNewDDLAndStart( context.Background(), WithStore(s.store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() // create table t (c1 int, c2 int); tblInfo, err := testTableInfo(d, "t", 2) - c.Assert(err, IsNil) + require.NoError(s.T(), err) ctx := testNewContext(d) err = ctx.NewTxn(context.Background()) - c.Assert(err, IsNil) - testCreateTable(c, ctx, d, s.dbInfo, tblInfo) + require.NoError(s.T(), err) + testCreateTable(s.T(), ctx, d, s.dbInfo, tblInfo) // insert t values (1, 2); - originTable := testGetTable(c, d, s.dbInfo.ID, tblInfo.ID) + originTable := testGetTable(s.T(), d, s.dbInfo.ID, tblInfo.ID) row := types.MakeDatums(1, 2) h, err := originTable.AddRecord(ctx, row) - c.Assert(err, IsNil) + require.NoError(s.T(), err) txn, err := ctx.Txn(true) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) var mu sync.Mutex tc := &TestDDLCallback{} @@ -154,31 +163,31 @@ func (s *testColumnChangeSuite) TestColumnChange(c *C) { } d.SetHook(tc) defaultValue := int64(3) - job := testCreateColumn(c, ctx, d, s.dbInfo, tblInfo, "c3", &ast.ColumnPosition{Tp: ast.ColumnPositionNone}, defaultValue) - c.Assert(checkErr, IsNil) - testCheckJobDone(c, d, job, true) + job := testCreateColumn(s.T(), ctx, d, s.dbInfo, tblInfo, "c3", &ast.ColumnPosition{Tp: ast.ColumnPositionNone}, defaultValue) + require.NoError(s.T(), checkErr) + testCheckJobDone(s.T(), d, job, true) mu.Lock() tb := publicTable mu.Unlock() - s.testColumnDrop(c, ctx, d, tb) - s.testAddColumnNoDefault(c, ctx, d, tblInfo) + s.testColumnDrop(ctx, d, tb) + s.testAddColumnNoDefault(ctx, d, tblInfo) } -func (s *testColumnChangeSuite) TestModifyAutoRandColumnWithMetaKeyChanged(c *C) { +func (s *testColumnChangeSuiteToVerify) TestModifyAutoRandColumnWithMetaKeyChanged() { d, err := testNewDDLAndStart( context.Background(), WithStore(s.store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() ids, err := d.genGlobalIDs(1) tableID := ids[0] - c.Assert(err, IsNil) + require.NoError(s.T(), err) colInfo := &model.ColumnInfo{ Name: model.NewCIStr("a"), Offset: 0, @@ -193,7 +202,7 @@ func (s *testColumnChangeSuite) TestModifyAutoRandColumnWithMetaKeyChanged(c *C) } colInfo.ID = allocateColumnID(tblInfo) ctx := testNewContext(d) - testCreateTable(c, ctx, d, s.dbInfo, tblInfo) + testCreateTable(s.T(), ctx, d, s.dbInfo, tblInfo) tc := &TestDDLCallback{} var errCount int32 = 3 @@ -219,10 +228,10 @@ func (s *testColumnChangeSuite) TestModifyAutoRandColumnWithMetaKeyChanged(c *C) Args: []interface{}{colInfo, colInfo.Name, ast.ColumnPosition{}, 0, newAutoRandomBits}, } err = d.doDDLJob(ctx, job) - c.Assert(err, IsNil) - c.Assert(errCount == 0, IsTrue) - c.Assert(genAutoRandErr, IsNil) - testCheckJobDone(c, d, job, true) + require.NoError(s.T(), err) + require.True(s.T(), errCount == 0) + require.Nil(s.T(), genAutoRandErr) + testCheckJobDone(s.T(), d, job, true) var newTbInfo *model.TableInfo err = kv.RunInNewTxn(context.Background(), d.store, false, func(ctx context.Context, txn kv.Transaction) error { t := meta.NewMeta(txn) @@ -233,11 +242,11 @@ func (s *testColumnChangeSuite) TestModifyAutoRandColumnWithMetaKeyChanged(c *C) } return nil }) - c.Assert(err, IsNil) - c.Assert(newTbInfo.AutoRandomBits, Equals, newAutoRandomBits) + require.NoError(s.T(), err) + require.Equal(s.T(), newTbInfo.AutoRandomBits, newAutoRandomBits) } -func (s *testColumnChangeSuite) testAddColumnNoDefault(c *C, ctx sessionctx.Context, d *ddl, tblInfo *model.TableInfo) { +func (s *testColumnChangeSuiteToVerify) testAddColumnNoDefault(ctx sessionctx.Context, d *ddl, tblInfo *model.TableInfo) { tc := &TestDDLCallback{} // set up hook prevState := model.StateNone @@ -280,12 +289,12 @@ func (s *testColumnChangeSuite) testAddColumnNoDefault(c *C, ctx sessionctx.Cont } } d.SetHook(tc) - job := testCreateColumn(c, ctx, d, s.dbInfo, tblInfo, "c3", &ast.ColumnPosition{Tp: ast.ColumnPositionNone}, nil) - c.Assert(checkErr, IsNil) - testCheckJobDone(c, d, job, true) + job := testCreateColumn(s.T(), ctx, d, s.dbInfo, tblInfo, "c3", &ast.ColumnPosition{Tp: ast.ColumnPositionNone}, nil) + require.NoError(s.T(), checkErr) + testCheckJobDone(s.T(), d, job, true) } -func (s *testColumnChangeSuite) testColumnDrop(c *C, ctx sessionctx.Context, d *ddl, tbl table.Table) { +func (s *testColumnChangeSuiteToVerify) testColumnDrop(ctx sessionctx.Context, d *ddl, tbl table.Table) { dropCol := tbl.Cols()[2] tc := &TestDDLCallback{} // set up hook @@ -307,8 +316,8 @@ func (s *testColumnChangeSuite) testColumnDrop(c *C, ctx sessionctx.Context, d * } } d.SetHook(tc) - c.Assert(checkErr, IsNil) - testDropColumn(c, ctx, d, s.dbInfo, tbl.Meta(), dropCol.Name.L, false) + require.NoError(s.T(), checkErr) + testDropColumn(s.T(), ctx, d, s.dbInfo, tbl.Meta(), dropCol.Name.L, false) } func seek(t table.PhysicalTable, ctx sessionctx.Context, h kv.Handle) (kv.Handle, bool, error) { @@ -333,7 +342,7 @@ func seek(t table.PhysicalTable, ctx sessionctx.Context, h kv.Handle) (kv.Handle return handle, true, nil } -func (s *testColumnChangeSuite) checkAddWriteOnly(ctx sessionctx.Context, d *ddl, deleteOnlyTable, writeOnlyTable table.Table, h kv.Handle) error { +func (s *testColumnChangeSuiteToVerify) checkAddWriteOnly(ctx sessionctx.Context, d *ddl, deleteOnlyTable, writeOnlyTable table.Table, h kv.Handle) error { // WriteOnlyTable: insert t values (2, 3) err := ctx.NewTxn(context.Background()) if err != nil { @@ -407,7 +416,7 @@ func touchedSlice(t table.Table) []bool { return touched } -func (s *testColumnChangeSuite) checkAddPublic(sctx sessionctx.Context, d *ddl, writeOnlyTable, publicTable table.Table) error { +func (s *testColumnChangeSuiteToVerify) checkAddPublic(sctx sessionctx.Context, d *ddl, writeOnlyTable, publicTable table.Table) error { ctx := context.TODO() // publicTable Insert t values (4, 4, 4) err := sctx.NewTxn(ctx) diff --git a/ddl/column_test.go b/ddl/column_test.go index 1cb58c4867..fe0e58575f 100644 --- a/ddl/column_test.go +++ b/ddl/column_test.go @@ -21,8 +21,8 @@ import ( "sync" "testing" - . "github.com/pingcap/check" "github.com/pingcap/errors" + "github.com/pingcap/tidb/domain/infosync" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser" @@ -39,33 +39,40 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/collate" "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" ) -var _ = Suite(&testColumnSuite{}) - -type testColumnSuite struct { +type testColumnSuiteToVerify struct { + suite.Suite store kv.Storage dbInfo *model.DBInfo } -func (s *testColumnSuite) SetUpSuite(c *C) { - s.store = testCreateStore(c, "test_column") +func TestColumnSuite(t *testing.T) { + _, err := infosync.GlobalInfoSyncerInit(context.Background(), "t", func() uint64 { return 1 }, nil, true) + require.NoError(t, err) + + suite.Run(t, new(testColumnSuiteToVerify)) +} + +func (s *testColumnSuiteToVerify) SetupSuite() { + s.store = testCreateStore(s.T(), "test_column") d, err := testNewDDLAndStart( context.Background(), WithStore(s.store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) s.dbInfo, err = testSchemaInfo(d, "test_column") - c.Assert(err, IsNil) - testCreateSchema(c, testNewContext(d), d, s.dbInfo) - c.Assert(d.Stop(), IsNil) + require.NoError(s.T(), err) + testCreateSchema(s.T(), testNewContext(d), d, s.dbInfo) + require.Nil(s.T(), d.Stop()) } -func (s *testColumnSuite) TearDownSuite(c *C) { +func (s *testColumnSuiteToVerify) TearDownSuite() { err := s.store.Close() - c.Assert(err, IsNil) + require.NoError(s.T(), err) } func buildCreateColumnJob(dbInfo *model.DBInfo, tblInfo *model.TableInfo, colName string, @@ -89,13 +96,13 @@ func buildCreateColumnJob(dbInfo *model.DBInfo, tblInfo *model.TableInfo, colNam return job } -func testCreateColumn(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, +func testCreateColumn(t *testing.T, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, colName string, pos *ast.ColumnPosition, defaultValue interface{}) *model.Job { job := buildCreateColumnJob(dbInfo, tblInfo, colName, pos, defaultValue) err := d.doDDLJob(ctx, job) - c.Assert(err, IsNil) - v := getSchemaVer(c, ctx) - checkHistoryJobArgs(c, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) + require.NoError(t, err) + v := getSchemaVer(t, ctx) + checkHistoryJobArgs(t, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) return job } @@ -126,13 +133,13 @@ func buildCreateColumnsJob(dbInfo *model.DBInfo, tblInfo *model.TableInfo, colNa return job } -func testCreateColumns(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, +func testCreateColumns(t *testing.T, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, colNames []string, positions []*ast.ColumnPosition, defaultValue interface{}) *model.Job { job := buildCreateColumnsJob(dbInfo, tblInfo, colNames, positions, defaultValue) err := d.doDDLJob(ctx, job) - c.Assert(err, IsNil) - v := getSchemaVer(c, ctx) - checkHistoryJobArgs(c, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) + require.NoError(t, err) + v := getSchemaVer(t, ctx) + checkHistoryJobArgs(t, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) return job } @@ -147,16 +154,16 @@ func buildDropColumnJob(dbInfo *model.DBInfo, tblInfo *model.TableInfo, colName } } -func testDropColumn(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, colName string, isError bool) *model.Job { +func testDropColumn(t *testing.T, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, colName string, isError bool) *model.Job { job := buildDropColumnJob(dbInfo, tblInfo, colName) err := d.doDDLJob(ctx, job) if isError { - c.Assert(err, NotNil) + require.Error(t, err) return nil } - c.Assert(err, IsNil) - v := getSchemaVer(c, ctx) - checkHistoryJobArgs(c, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) + require.NoError(t, err) + v := getSchemaVer(t, ctx) + checkHistoryJobArgs(t, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) return job } @@ -176,179 +183,179 @@ func buildDropColumnsJob(dbInfo *model.DBInfo, tblInfo *model.TableInfo, colName return job } -func testDropColumns(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, colNames []string, isError bool) *model.Job { +func testDropColumns(t *testing.T, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, colNames []string, isError bool) *model.Job { job := buildDropColumnsJob(dbInfo, tblInfo, colNames) err := d.doDDLJob(ctx, job) if isError { - c.Assert(err, NotNil) + require.Error(t, err) return nil } - c.Assert(err, IsNil) - v := getSchemaVer(c, ctx) - checkHistoryJobArgs(c, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) + require.NoError(t, err) + v := getSchemaVer(t, ctx) + checkHistoryJobArgs(t, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) return job } -func (s *testColumnSuite) TestColumnBasic(c *C) { +func (s *testColumnSuiteToVerify) TestColumnBasic() { d, err := testNewDDLAndStart( context.Background(), WithStore(s.store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() tblInfo, err := testTableInfo(d, "t1", 3) - c.Assert(err, IsNil) + require.NoError(s.T(), err) ctx := testNewContext(d) - testCreateTable(c, ctx, d, s.dbInfo, tblInfo) - t := testGetTable(c, d, s.dbInfo.ID, tblInfo.ID) + testCreateTable(s.T(), ctx, d, s.dbInfo, tblInfo) + t := testGetTable(s.T(), d, s.dbInfo.ID, tblInfo.ID) num := 10 for i := 0; i < num; i++ { _, err := t.AddRecord(ctx, types.MakeDatums(i, 10*i, 100*i)) - c.Assert(err, IsNil) + require.NoError(s.T(), err) } err = ctx.NewTxn(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) i := int64(0) err = tables.IterRecords(t, ctx, t.Cols(), func(_ kv.Handle, data []types.Datum, cols []*table.Column) (bool, error) { - c.Assert(data, HasLen, 3) - c.Assert(data[0].GetInt64(), Equals, i) - c.Assert(data[1].GetInt64(), Equals, 10*i) - c.Assert(data[2].GetInt64(), Equals, 100*i) + require.Len(s.T(), data, 3) + require.Equal(s.T(), data[0].GetInt64(), i) + require.Equal(s.T(), data[1].GetInt64(), 10*i) + require.Equal(s.T(), data[2].GetInt64(), 100*i) i++ return true, nil }) - c.Assert(err, IsNil) - c.Assert(i, Equals, int64(num)) + require.NoError(s.T(), err) + require.Equal(s.T(), i, int64(num)) - c.Assert(table.FindCol(t.Cols(), "c4"), IsNil) + require.Nil(s.T(), table.FindCol(t.Cols(), "c4")) - job := testCreateColumn(c, ctx, d, s.dbInfo, tblInfo, "c4", &ast.ColumnPosition{Tp: ast.ColumnPositionAfter, RelativeColumn: &ast.ColumnName{Name: model.NewCIStr("c3")}}, 100) - testCheckJobDone(c, d, job, true) + job := testCreateColumn(s.T(), ctx, d, s.dbInfo, tblInfo, "c4", &ast.ColumnPosition{Tp: ast.ColumnPositionAfter, RelativeColumn: &ast.ColumnName{Name: model.NewCIStr("c3")}}, 100) + testCheckJobDone(s.T(), d, job, true) - t = testGetTable(c, d, s.dbInfo.ID, tblInfo.ID) - c.Assert(table.FindCol(t.Cols(), "c4"), NotNil) + t = testGetTable(s.T(), d, s.dbInfo.ID, tblInfo.ID) + require.NotNil(s.T(), table.FindCol(t.Cols(), "c4")) i = int64(0) err = tables.IterRecords(t, ctx, t.Cols(), func(_ kv.Handle, data []types.Datum, cols []*table.Column) (bool, error) { - c.Assert(data, HasLen, 4) - c.Assert(data[0].GetInt64(), Equals, i) - c.Assert(data[1].GetInt64(), Equals, 10*i) - c.Assert(data[2].GetInt64(), Equals, 100*i) - c.Assert(data[3].GetInt64(), Equals, int64(100)) + require.Len(s.T(), data, 4) + require.Equal(s.T(), data[0].GetInt64(), i) + require.Equal(s.T(), data[1].GetInt64(), 10*i) + require.Equal(s.T(), data[2].GetInt64(), 100*i) + require.Equal(s.T(), data[3].GetInt64(), int64(100)) i++ return true, nil }) - c.Assert(err, IsNil) - c.Assert(i, Equals, int64(num)) + require.NoError(s.T(), err) + require.Equal(s.T(), i, int64(num)) h, err := t.AddRecord(ctx, types.MakeDatums(11, 12, 13, 14)) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = ctx.NewTxn(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) values, err := tables.RowWithCols(t, ctx, h, t.Cols()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) - c.Assert(values, HasLen, 4) - c.Assert(values[3].GetInt64(), Equals, int64(14)) + require.Len(s.T(), values, 4) + require.Equal(s.T(), values[3].GetInt64(), int64(14)) - job = testDropColumn(c, ctx, d, s.dbInfo, tblInfo, "c4", false) - testCheckJobDone(c, d, job, false) + job = testDropColumn(s.T(), ctx, d, s.dbInfo, tblInfo, "c4", false) + testCheckJobDone(s.T(), d, job, false) - t = testGetTable(c, d, s.dbInfo.ID, tblInfo.ID) + t = testGetTable(s.T(), d, s.dbInfo.ID, tblInfo.ID) values, err = tables.RowWithCols(t, ctx, h, t.Cols()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) - c.Assert(values, HasLen, 3) - c.Assert(values[2].GetInt64(), Equals, int64(13)) + require.Len(s.T(), values, 3) + require.Equal(s.T(), values[2].GetInt64(), int64(13)) - job = testCreateColumn(c, ctx, d, s.dbInfo, tblInfo, "c4", &ast.ColumnPosition{Tp: ast.ColumnPositionNone}, 111) - testCheckJobDone(c, d, job, true) + job = testCreateColumn(s.T(), ctx, d, s.dbInfo, tblInfo, "c4", &ast.ColumnPosition{Tp: ast.ColumnPositionNone}, 111) + testCheckJobDone(s.T(), d, job, true) - t = testGetTable(c, d, s.dbInfo.ID, tblInfo.ID) + t = testGetTable(s.T(), d, s.dbInfo.ID, tblInfo.ID) values, err = tables.RowWithCols(t, ctx, h, t.Cols()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) - c.Assert(values, HasLen, 4) - c.Assert(values[3].GetInt64(), Equals, int64(111)) + require.Len(s.T(), values, 4) + require.Equal(s.T(), values[3].GetInt64(), int64(111)) - job = testCreateColumn(c, ctx, d, s.dbInfo, tblInfo, "c5", &ast.ColumnPosition{Tp: ast.ColumnPositionNone}, 101) - testCheckJobDone(c, d, job, true) + job = testCreateColumn(s.T(), ctx, d, s.dbInfo, tblInfo, "c5", &ast.ColumnPosition{Tp: ast.ColumnPositionNone}, 101) + testCheckJobDone(s.T(), d, job, true) - t = testGetTable(c, d, s.dbInfo.ID, tblInfo.ID) + t = testGetTable(s.T(), d, s.dbInfo.ID, tblInfo.ID) values, err = tables.RowWithCols(t, ctx, h, t.Cols()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) - c.Assert(values, HasLen, 5) - c.Assert(values[4].GetInt64(), Equals, int64(101)) + require.Len(s.T(), values, 5) + require.Equal(s.T(), values[4].GetInt64(), int64(101)) - job = testCreateColumn(c, ctx, d, s.dbInfo, tblInfo, "c6", &ast.ColumnPosition{Tp: ast.ColumnPositionFirst}, 202) - testCheckJobDone(c, d, job, true) + job = testCreateColumn(s.T(), ctx, d, s.dbInfo, tblInfo, "c6", &ast.ColumnPosition{Tp: ast.ColumnPositionFirst}, 202) + testCheckJobDone(s.T(), d, job, true) - t = testGetTable(c, d, s.dbInfo.ID, tblInfo.ID) + t = testGetTable(s.T(), d, s.dbInfo.ID, tblInfo.ID) cols := t.Cols() - c.Assert(cols, HasLen, 6) - c.Assert(cols[0].Offset, Equals, 0) - c.Assert(cols[0].Name.L, Equals, "c6") - c.Assert(cols[1].Offset, Equals, 1) - c.Assert(cols[1].Name.L, Equals, "c1") - c.Assert(cols[2].Offset, Equals, 2) - c.Assert(cols[2].Name.L, Equals, "c2") - c.Assert(cols[3].Offset, Equals, 3) - c.Assert(cols[3].Name.L, Equals, "c3") - c.Assert(cols[4].Offset, Equals, 4) - c.Assert(cols[4].Name.L, Equals, "c4") - c.Assert(cols[5].Offset, Equals, 5) - c.Assert(cols[5].Name.L, Equals, "c5") + require.Len(s.T(), cols, 6) + require.Equal(s.T(), cols[0].Offset, 0) + require.Equal(s.T(), cols[0].Name.L, "c6") + require.Equal(s.T(), cols[1].Offset, 1) + require.Equal(s.T(), cols[1].Name.L, "c1") + require.Equal(s.T(), cols[2].Offset, 2) + require.Equal(s.T(), cols[2].Name.L, "c2") + require.Equal(s.T(), cols[3].Offset, 3) + require.Equal(s.T(), cols[3].Name.L, "c3") + require.Equal(s.T(), cols[4].Offset, 4) + require.Equal(s.T(), cols[4].Name.L, "c4") + require.Equal(s.T(), cols[5].Offset, 5) + require.Equal(s.T(), cols[5].Name.L, "c5") values, err = tables.RowWithCols(t, ctx, h, cols) - c.Assert(err, IsNil) + require.NoError(s.T(), err) - c.Assert(values, HasLen, 6) - c.Assert(values[0].GetInt64(), Equals, int64(202)) - c.Assert(values[5].GetInt64(), Equals, int64(101)) + require.Len(s.T(), values, 6) + require.Equal(s.T(), values[0].GetInt64(), int64(202)) + require.Equal(s.T(), values[5].GetInt64(), int64(101)) - job = testDropColumn(c, ctx, d, s.dbInfo, tblInfo, "c2", false) - testCheckJobDone(c, d, job, false) + job = testDropColumn(s.T(), ctx, d, s.dbInfo, tblInfo, "c2", false) + testCheckJobDone(s.T(), d, job, false) - t = testGetTable(c, d, s.dbInfo.ID, tblInfo.ID) + t = testGetTable(s.T(), d, s.dbInfo.ID, tblInfo.ID) values, err = tables.RowWithCols(t, ctx, h, t.Cols()) - c.Assert(err, IsNil) - c.Assert(values, HasLen, 5) - c.Assert(values[0].GetInt64(), Equals, int64(202)) - c.Assert(values[4].GetInt64(), Equals, int64(101)) + require.NoError(s.T(), err) + require.Len(s.T(), values, 5) + require.Equal(s.T(), values[0].GetInt64(), int64(202)) + require.Equal(s.T(), values[4].GetInt64(), int64(101)) - job = testDropColumn(c, ctx, d, s.dbInfo, tblInfo, "c1", false) - testCheckJobDone(c, d, job, false) + job = testDropColumn(s.T(), ctx, d, s.dbInfo, tblInfo, "c1", false) + testCheckJobDone(s.T(), d, job, false) - job = testDropColumn(c, ctx, d, s.dbInfo, tblInfo, "c3", false) - testCheckJobDone(c, d, job, false) + job = testDropColumn(s.T(), ctx, d, s.dbInfo, tblInfo, "c3", false) + testCheckJobDone(s.T(), d, job, false) - job = testDropColumn(c, ctx, d, s.dbInfo, tblInfo, "c4", false) - testCheckJobDone(c, d, job, false) + job = testDropColumn(s.T(), ctx, d, s.dbInfo, tblInfo, "c4", false) + testCheckJobDone(s.T(), d, job, false) - job = testCreateIndex(c, ctx, d, s.dbInfo, tblInfo, false, "c5_idx", "c5") - testCheckJobDone(c, d, job, true) + job = testCreateIndex(s.T(), ctx, d, s.dbInfo, tblInfo, false, "c5_idx", "c5") + testCheckJobDone(s.T(), d, job, true) - job = testDropColumn(c, ctx, d, s.dbInfo, tblInfo, "c5", false) - testCheckJobDone(c, d, job, false) + job = testDropColumn(s.T(), ctx, d, s.dbInfo, tblInfo, "c5", false) + testCheckJobDone(s.T(), d, job, false) - testDropColumn(c, ctx, d, s.dbInfo, tblInfo, "c6", true) + testDropColumn(s.T(), ctx, d, s.dbInfo, tblInfo, "c6", true) - testDropTable(c, ctx, d, s.dbInfo, tblInfo) + testDropTable(s.T(), ctx, d, s.dbInfo, tblInfo) } -func (s *testColumnSuite) checkColumnKVExist(ctx sessionctx.Context, t table.Table, handle kv.Handle, col *table.Column, columnValue interface{}, isExist bool) error { +func (s *testColumnSuiteToVerify) checkColumnKVExist(ctx sessionctx.Context, t table.Table, handle kv.Handle, col *table.Column, columnValue interface{}, isExist bool) error { err := ctx.NewTxn(context.Background()) if err != nil { return errors.Trace(err) @@ -394,7 +401,7 @@ func (s *testColumnSuite) checkColumnKVExist(ctx sessionctx.Context, t table.Tab return nil } -func (s *testColumnSuite) checkNoneColumn(ctx sessionctx.Context, d *ddl, tblInfo *model.TableInfo, handle kv.Handle, col *table.Column, columnValue interface{}) error { +func (s *testColumnSuiteToVerify) checkNoneColumn(ctx sessionctx.Context, d *ddl, tblInfo *model.TableInfo, handle kv.Handle, col *table.Column, columnValue interface{}) error { t, err := testGetTableWithError(d, s.dbInfo.ID, tblInfo.ID) if err != nil { return errors.Trace(err) @@ -410,7 +417,7 @@ func (s *testColumnSuite) checkNoneColumn(ctx sessionctx.Context, d *ddl, tblInf return nil } -func (s *testColumnSuite) checkDeleteOnlyColumn(ctx sessionctx.Context, d *ddl, tblInfo *model.TableInfo, handle kv.Handle, col *table.Column, row []types.Datum, columnValue interface{}) error { +func (s *testColumnSuiteToVerify) checkDeleteOnlyColumn(ctx sessionctx.Context, d *ddl, tblInfo *model.TableInfo, handle kv.Handle, col *table.Column, row []types.Datum, columnValue interface{}) error { t, err := testGetTableWithError(d, s.dbInfo.ID, tblInfo.ID) if err != nil { return errors.Trace(err) @@ -511,7 +518,7 @@ func (s *testColumnSuite) checkDeleteOnlyColumn(ctx sessionctx.Context, d *ddl, return nil } -func (s *testColumnSuite) checkWriteOnlyColumn(ctx sessionctx.Context, d *ddl, tblInfo *model.TableInfo, handle kv.Handle, col *table.Column, row []types.Datum, columnValue interface{}) error { +func (s *testColumnSuiteToVerify) checkWriteOnlyColumn(ctx sessionctx.Context, d *ddl, tblInfo *model.TableInfo, handle kv.Handle, col *table.Column, row []types.Datum, columnValue interface{}) error { t, err := testGetTableWithError(d, s.dbInfo.ID, tblInfo.ID) if err != nil { return errors.Trace(err) @@ -616,7 +623,7 @@ func (s *testColumnSuite) checkWriteOnlyColumn(ctx sessionctx.Context, d *ddl, t return nil } -func (s *testColumnSuite) checkReorganizationColumn(ctx sessionctx.Context, d *ddl, tblInfo *model.TableInfo, col *table.Column, row []types.Datum, columnValue interface{}) error { +func (s *testColumnSuiteToVerify) checkReorganizationColumn(ctx sessionctx.Context, d *ddl, tblInfo *model.TableInfo, col *table.Column, row []types.Datum, columnValue interface{}) error { t, err := testGetTableWithError(d, s.dbInfo.ID, tblInfo.ID) if err != nil { return errors.Trace(err) @@ -712,7 +719,7 @@ func (s *testColumnSuite) checkReorganizationColumn(ctx sessionctx.Context, d *d return nil } -func (s *testColumnSuite) checkPublicColumn(ctx sessionctx.Context, d *ddl, tblInfo *model.TableInfo, newCol *table.Column, oldRow []types.Datum, columnValue interface{}) error { +func (s *testColumnSuiteToVerify) checkPublicColumn(ctx sessionctx.Context, d *ddl, tblInfo *model.TableInfo, newCol *table.Column, oldRow []types.Datum, columnValue interface{}) error { t, err := testGetTableWithError(d, s.dbInfo.ID, tblInfo.ID) if err != nil { return errors.Trace(err) @@ -809,7 +816,7 @@ func (s *testColumnSuite) checkPublicColumn(ctx sessionctx.Context, d *ddl, tblI return nil } -func (s *testColumnSuite) checkAddColumn(state model.SchemaState, d *ddl, tblInfo *model.TableInfo, handle kv.Handle, newCol *table.Column, oldRow []types.Datum, columnValue interface{}) error { +func (s *testColumnSuiteToVerify) checkAddColumn(state model.SchemaState, d *ddl, tblInfo *model.TableInfo, handle kv.Handle, newCol *table.Column, oldRow []types.Datum, columnValue interface{}) error { ctx := testNewContext(d) var err error switch state { @@ -827,7 +834,7 @@ func (s *testColumnSuite) checkAddColumn(state model.SchemaState, d *ddl, tblInf return err } -func (s *testColumnSuite) testGetColumn(t table.Table, name string, isExist bool) error { +func (s *testColumnSuiteToVerify) testGetColumn(t table.Table, name string, isExist bool) error { col := table.FindCol(t.Cols(), name) if isExist { if col == nil { @@ -841,31 +848,31 @@ func (s *testColumnSuite) testGetColumn(t table.Table, name string, isExist bool return nil } -func (s *testColumnSuite) TestAddColumn(c *C) { +func (s *testColumnSuiteToVerify) TestAddColumn() { d, err := testNewDDLAndStart( context.Background(), WithStore(s.store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) tblInfo, err := testTableInfo(d, "t", 3) - c.Assert(err, IsNil) + require.NoError(s.T(), err) ctx := testNewContext(d) err = ctx.NewTxn(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) - testCreateTable(c, ctx, d, s.dbInfo, tblInfo) - t := testGetTable(c, d, s.dbInfo.ID, tblInfo.ID) + testCreateTable(s.T(), ctx, d, s.dbInfo, tblInfo) + t := testGetTable(s.T(), d, s.dbInfo.ID, tblInfo.ID) oldRow := types.MakeDatums(int64(1), int64(2), int64(3)) handle, err := t.AddRecord(ctx, oldRow) - c.Assert(err, IsNil) + require.NoError(s.T(), err) txn, err := ctx.Txn(true) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) newColName := "c4" defaultColValue := int64(4) @@ -905,56 +912,56 @@ func (s *testColumnSuite) TestAddColumn(c *C) { d.SetHook(tc) - job := testCreateColumn(c, ctx, d, s.dbInfo, tblInfo, newColName, &ast.ColumnPosition{Tp: ast.ColumnPositionNone}, defaultColValue) + job := testCreateColumn(s.T(), ctx, d, s.dbInfo, tblInfo, newColName, &ast.ColumnPosition{Tp: ast.ColumnPositionNone}, defaultColValue) - testCheckJobDone(c, d, job, true) + testCheckJobDone(s.T(), d, job, true) mu.Lock() hErr := hookErr ok := checkOK mu.Unlock() - c.Assert(hErr, IsNil) - c.Assert(ok, IsTrue) + require.NoError(s.T(), hErr) + require.True(s.T(), ok) err = ctx.NewTxn(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) - job = testDropTable(c, ctx, d, s.dbInfo, tblInfo) - testCheckJobDone(c, d, job, false) + job = testDropTable(s.T(), ctx, d, s.dbInfo, tblInfo) + testCheckJobDone(s.T(), d, job, false) txn, err = ctx.Txn(true) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) } -func (s *testColumnSuite) TestAddColumns(c *C) { +func (s *testColumnSuiteToVerify) TestAddColumns() { d, err := testNewDDLAndStart( context.Background(), WithStore(s.store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) tblInfo, err := testTableInfo(d, "t", 3) - c.Assert(err, IsNil) + require.NoError(s.T(), err) ctx := testNewContext(d) err = ctx.NewTxn(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) - testCreateTable(c, ctx, d, s.dbInfo, tblInfo) - t := testGetTable(c, d, s.dbInfo.ID, tblInfo.ID) + testCreateTable(s.T(), ctx, d, s.dbInfo, tblInfo) + t := testGetTable(s.T(), d, s.dbInfo.ID, tblInfo.ID) oldRow := types.MakeDatums(int64(1), int64(2), int64(3)) handle, err := t.AddRecord(ctx, oldRow) - c.Assert(err, IsNil) + require.NoError(s.T(), err) txn, err := ctx.Txn(true) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) newColNames := []string{"c4,c5,c6"} positions := make([]*ast.ColumnPosition, 3) @@ -1000,49 +1007,49 @@ func (s *testColumnSuite) TestAddColumns(c *C) { d.SetHook(tc) - job := testCreateColumns(c, ctx, d, s.dbInfo, tblInfo, newColNames, positions, defaultColValue) + job := testCreateColumns(s.T(), ctx, d, s.dbInfo, tblInfo, newColNames, positions, defaultColValue) - testCheckJobDone(c, d, job, true) + testCheckJobDone(s.T(), d, job, true) mu.Lock() hErr := hookErr ok := checkOK mu.Unlock() - c.Assert(hErr, IsNil) - c.Assert(ok, IsTrue) + require.NoError(s.T(), hErr) + require.True(s.T(), ok) - job = testDropTable(c, ctx, d, s.dbInfo, tblInfo) - testCheckJobDone(c, d, job, false) + job = testDropTable(s.T(), ctx, d, s.dbInfo, tblInfo) + testCheckJobDone(s.T(), d, job, false) err = d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) } -func (s *testColumnSuite) TestDropColumn(c *C) { +func (s *testColumnSuiteToVerify) TestDropColumn() { d, err := testNewDDLAndStart( context.Background(), WithStore(s.store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) tblInfo, err := testTableInfo(d, "t2", 4) - c.Assert(err, IsNil) + require.NoError(s.T(), err) ctx := testNewContext(d) err = ctx.NewTxn(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) - testCreateTable(c, ctx, d, s.dbInfo, tblInfo) - t := testGetTable(c, d, s.dbInfo.ID, tblInfo.ID) + testCreateTable(s.T(), ctx, d, s.dbInfo, tblInfo) + t := testGetTable(s.T(), d, s.dbInfo.ID, tblInfo.ID) colName := "c4" defaultColValue := int64(4) row := types.MakeDatums(int64(1), int64(2), int64(3)) _, err = t.AddRecord(ctx, append(row, types.NewDatum(defaultColValue))) - c.Assert(err, IsNil) + require.NoError(s.T(), err) txn, err := ctx.Txn(true) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) checkOK := false var hookErr error @@ -1069,57 +1076,57 @@ func (s *testColumnSuite) TestDropColumn(c *C) { d.SetHook(tc) - job := testDropColumn(c, ctx, d, s.dbInfo, tblInfo, colName, false) - testCheckJobDone(c, d, job, false) + job := testDropColumn(s.T(), ctx, d, s.dbInfo, tblInfo, colName, false) + testCheckJobDone(s.T(), d, job, false) mu.Lock() hErr := hookErr ok := checkOK mu.Unlock() - c.Assert(hErr, IsNil) - c.Assert(ok, IsTrue) + require.NoError(s.T(), hErr) + require.True(s.T(), ok) err = ctx.NewTxn(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) - job = testDropTable(c, ctx, d, s.dbInfo, tblInfo) - testCheckJobDone(c, d, job, false) + job = testDropTable(s.T(), ctx, d, s.dbInfo, tblInfo) + testCheckJobDone(s.T(), d, job, false) txn, err = ctx.Txn(true) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) } -func (s *testColumnSuite) TestDropColumns(c *C) { +func (s *testColumnSuiteToVerify) TestDropColumns() { d, err := testNewDDLAndStart( context.Background(), WithStore(s.store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) tblInfo, err := testTableInfo(d, "t2", 4) - c.Assert(err, IsNil) + require.NoError(s.T(), err) ctx := testNewContext(d) err = ctx.NewTxn(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) - testCreateTable(c, ctx, d, s.dbInfo, tblInfo) - t := testGetTable(c, d, s.dbInfo.ID, tblInfo.ID) + testCreateTable(s.T(), ctx, d, s.dbInfo, tblInfo) + t := testGetTable(s.T(), d, s.dbInfo.ID, tblInfo.ID) colNames := []string{"c3", "c4"} defaultColValue := int64(4) row := types.MakeDatums(int64(1), int64(2), int64(3)) _, err = t.AddRecord(ctx, append(row, types.NewDatum(defaultColValue))) - c.Assert(err, IsNil) + require.NoError(s.T(), err) txn, err := ctx.Txn(true) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) checkOK := false var hookErr error @@ -1148,19 +1155,19 @@ func (s *testColumnSuite) TestDropColumns(c *C) { d.SetHook(tc) - job := testDropColumns(c, ctx, d, s.dbInfo, tblInfo, colNames, false) - testCheckJobDone(c, d, job, false) + job := testDropColumns(s.T(), ctx, d, s.dbInfo, tblInfo, colNames, false) + testCheckJobDone(s.T(), d, job, false) mu.Lock() hErr := hookErr ok := checkOK mu.Unlock() - c.Assert(hErr, IsNil) - c.Assert(ok, IsTrue) + require.NoError(s.T(), hErr) + require.True(s.T(), ok) - job = testDropTable(c, ctx, d, s.dbInfo, tblInfo) - testCheckJobDone(c, d, job, false) + job = testDropTable(s.T(), ctx, d, s.dbInfo, tblInfo) + testCheckJobDone(s.T(), d, job, false) err = d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) } func TestModifyColumn(t *testing.T) { @@ -1247,51 +1254,51 @@ func TestFieldCase(t *testing.T) { require.EqualError(t, err, infoschema.ErrColumnExists.GenWithStackByArgs("Field").Error()) } -func (s *testColumnSuite) TestAutoConvertBlobTypeByLength(c *C) { +func (s *testColumnSuiteToVerify) TestAutoConvertBlobTypeByLength() { d, err := testNewDDLAndStart( context.Background(), WithStore(s.store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) // Close the customized ddl(worker goroutine included) after the test is finished, otherwise, it will // cause go routine in TiDB leak test. defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() sql := fmt.Sprintf("create table t0(c0 Blob(%d), c1 Blob(%d), c2 Blob(%d), c3 Blob(%d))", tinyBlobMaxLength-1, blobMaxLength-1, mediumBlobMaxLength-1, longBlobMaxLength-1) stmt, err := parser.New().ParseOneStmt(sql, "", "") - c.Assert(err, IsNil) + require.NoError(s.T(), err) tblInfo, err := BuildTableInfoFromAST(stmt.(*ast.CreateTableStmt)) - c.Assert(err, IsNil) + require.NoError(s.T(), err) genIDs, err := d.genGlobalIDs(1) - c.Assert(err, IsNil) + require.NoError(s.T(), err) tblInfo.ID = genIDs[0] ctx := testNewContext(d) err = ctx.NewTxn(context.Background()) - c.Assert(err, IsNil) - testCreateTable(c, ctx, d, s.dbInfo, tblInfo) - t := testGetTable(c, d, s.dbInfo.ID, tblInfo.ID) + require.NoError(s.T(), err) + testCreateTable(s.T(), ctx, d, s.dbInfo, tblInfo) + t := testGetTable(s.T(), d, s.dbInfo.ID, tblInfo.ID) - c.Assert(t.Cols()[0].Tp, Equals, mysql.TypeTinyBlob) - c.Assert(t.Cols()[0].Flen, Equals, tinyBlobMaxLength) - c.Assert(t.Cols()[1].Tp, Equals, mysql.TypeBlob) - c.Assert(t.Cols()[1].Flen, Equals, blobMaxLength) - c.Assert(t.Cols()[2].Tp, Equals, mysql.TypeMediumBlob) - c.Assert(t.Cols()[2].Flen, Equals, mediumBlobMaxLength) - c.Assert(t.Cols()[3].Tp, Equals, mysql.TypeLongBlob) - c.Assert(t.Cols()[3].Flen, Equals, longBlobMaxLength) + require.Equal(s.T(), t.Cols()[0].Tp, mysql.TypeTinyBlob) + require.Equal(s.T(), t.Cols()[0].Flen, tinyBlobMaxLength) + require.Equal(s.T(), t.Cols()[1].Tp, mysql.TypeBlob) + require.Equal(s.T(), t.Cols()[1].Flen, blobMaxLength) + require.Equal(s.T(), t.Cols()[2].Tp, mysql.TypeMediumBlob) + require.Equal(s.T(), t.Cols()[2].Flen, mediumBlobMaxLength) + require.Equal(s.T(), t.Cols()[3].Tp, mysql.TypeLongBlob) + require.Equal(s.T(), t.Cols()[3].Flen, longBlobMaxLength) oldRow := types.MakeDatums([]byte("a"), []byte("a"), []byte("a"), []byte("a")) _, err = t.AddRecord(ctx, oldRow) - c.Assert(err, IsNil) + require.NoError(s.T(), err) txn, err := ctx.Txn(true) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) } diff --git a/ddl/ddl_test.go b/ddl/ddl_test.go index 25bb0fa218..ddec7f9fd0 100644 --- a/ddl/ddl_test.go +++ b/ddl/ddl_test.go @@ -107,9 +107,9 @@ func testNewDDLAndStart(ctx context.Context, options ...Option) (*ddl, error) { return d, err } -func testCreateStore(c *C, name string) kv.Storage { +func testCreateStore(t *testing.T, name string) kv.Storage { store, err := mockstore.NewMockStore() - c.Assert(err, IsNil) + require.NoError(t, err) return store } @@ -125,14 +125,14 @@ func testNewContext(d *ddl) sessionctx.Context { return ctx } -func getSchemaVer(c *C, ctx sessionctx.Context) int64 { +func getSchemaVer(t *testing.T, ctx sessionctx.Context) int64 { err := ctx.NewTxn(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) txn, err := ctx.Txn(true) - c.Assert(err, IsNil) + require.NoError(t, err) m := meta.NewMeta(txn) ver, err := m.GetSchemaVersion() - c.Assert(err, IsNil) + require.NoError(t, err) return ver } @@ -154,14 +154,14 @@ type historyJobArgs struct { tblIDs map[int64]struct{} } -func checkEqualTable(c *C, t1, t2 *model.TableInfo) { - c.Assert(t1.ID, Equals, t2.ID) - c.Assert(t1.Name, Equals, t2.Name) - c.Assert(t1.Charset, Equals, t2.Charset) - c.Assert(t1.Collate, Equals, t2.Collate) - c.Assert(t1.PKIsHandle, DeepEquals, t2.PKIsHandle) - c.Assert(t1.Comment, DeepEquals, t2.Comment) - c.Assert(t1.AutoIncID, DeepEquals, t2.AutoIncID) +func checkEqualTable(t *testing.T, t1, t2 *model.TableInfo) { + require.Equal(t, t1.ID, t2.ID) + require.Equal(t, t1.Name, t2.Name) + require.Equal(t, t1.Charset, t2.Charset) + require.Equal(t, t1.Collate, t2.Collate) + require.Equal(t, t1.PKIsHandle, t2.PKIsHandle) + require.Equal(t, t1.Comment, t2.Comment) + require.Equal(t, t1.AutoIncID, t2.AutoIncID) } func checkEqualTableT(t *testing.T, t1, t2 *model.TableInfo) { @@ -174,27 +174,27 @@ func checkEqualTableT(t *testing.T, t1, t2 *model.TableInfo) { require.EqualValues(t, t1.AutoIncID, t2.AutoIncID) } -func checkHistoryJob(c *C, job *model.Job) { - c.Assert(job.State, Equals, model.JobStateSynced) +func checkHistoryJob(t *testing.T, job *model.Job) { + require.Equal(t, job.State, model.JobStateSynced) } -func checkHistoryJobArgs(c *C, ctx sessionctx.Context, id int64, args *historyJobArgs) { +func checkHistoryJobArgs(t *testing.T, ctx sessionctx.Context, id int64, args *historyJobArgs) { txn, err := ctx.Txn(true) - c.Assert(err, IsNil) - t := meta.NewMeta(txn) - historyJob, err := t.GetHistoryDDLJob(id) - c.Assert(err, IsNil) - c.Assert(historyJob.BinlogInfo.FinishedTS, Greater, uint64(0)) + require.NoError(t, err) + tran := meta.NewMeta(txn) + historyJob, err := tran.GetHistoryDDLJob(id) + require.NoError(t, err) + require.Greater(t, historyJob.BinlogInfo.FinishedTS, uint64(0)) if args.tbl != nil { - c.Assert(historyJob.BinlogInfo.SchemaVersion, Equals, args.ver) - checkEqualTable(c, historyJob.BinlogInfo.TableInfo, args.tbl) + require.Equal(t, historyJob.BinlogInfo.SchemaVersion, args.ver) + checkEqualTable(t, historyJob.BinlogInfo.TableInfo, args.tbl) return } // for handling schema job - c.Assert(historyJob.BinlogInfo.SchemaVersion, Equals, args.ver) - c.Assert(historyJob.BinlogInfo.DBInfo, DeepEquals, args.db) + require.Equal(t, historyJob.BinlogInfo.SchemaVersion, args.ver) + require.Equal(t, historyJob.BinlogInfo.DBInfo, args.db) // only for creating schema job if args.db != nil && len(args.tblIDs) == 0 { return @@ -237,26 +237,26 @@ func buildCreateIdxJob(dbInfo *model.DBInfo, tblInfo *model.TableInfo, unique bo } } -func testCreatePrimaryKey(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, colName string) *model.Job { +func testCreatePrimaryKey(t *testing.T, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, colName string) *model.Job { job := buildCreateIdxJob(dbInfo, tblInfo, true, "primary", colName) job.Type = model.ActionAddPrimaryKey err := d.doDDLJob(ctx, job) - c.Assert(err, IsNil) - v := getSchemaVer(c, ctx) - checkHistoryJobArgs(c, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) + require.NoError(t, err) + v := getSchemaVer(t, ctx) + checkHistoryJobArgs(t, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) return job } -func testCreateIndex(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, unique bool, indexName string, colName string) *model.Job { +func testCreateIndex(t *testing.T, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, unique bool, indexName string, colName string) *model.Job { job := buildCreateIdxJob(dbInfo, tblInfo, unique, indexName, colName) err := d.doDDLJob(ctx, job) - c.Assert(err, IsNil) - v := getSchemaVer(c, ctx) - checkHistoryJobArgs(c, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) + require.NoError(t, err) + v := getSchemaVer(t, ctx) + checkHistoryJobArgs(t, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) return job } -func testAddColumn(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, args []interface{}) *model.Job { +func testAddColumn(t *testing.T, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, args []interface{}) *model.Job { job := &model.Job{ SchemaID: dbInfo.ID, TableID: tblInfo.ID, @@ -265,13 +265,13 @@ func testAddColumn(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, t BinlogInfo: &model.HistoryInfo{}, } err := d.doDDLJob(ctx, job) - c.Assert(err, IsNil) - v := getSchemaVer(c, ctx) - checkHistoryJobArgs(c, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) + require.NoError(t, err) + v := getSchemaVer(t, ctx) + checkHistoryJobArgs(t, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) return job } -func testAddColumns(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, args []interface{}) *model.Job { +func testAddColumns(t *testing.T, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, args []interface{}) *model.Job { job := &model.Job{ SchemaID: dbInfo.ID, TableID: tblInfo.ID, @@ -280,9 +280,9 @@ func testAddColumns(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, BinlogInfo: &model.HistoryInfo{}, } err := d.doDDLJob(ctx, job) - c.Assert(err, IsNil) - v := getSchemaVer(c, ctx) - checkHistoryJobArgs(c, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) + require.NoError(t, err) + v := getSchemaVer(t, ctx) + checkHistoryJobArgs(t, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) return job } @@ -300,12 +300,12 @@ func buildDropIdxJob(dbInfo *model.DBInfo, tblInfo *model.TableInfo, indexName s } } -func testDropIndex(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, indexName string) *model.Job { +func testDropIndex(t *testing.T, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, indexName string) *model.Job { job := buildDropIdxJob(dbInfo, tblInfo, indexName) err := d.doDDLJob(ctx, job) - c.Assert(err, IsNil) - v := getSchemaVer(c, ctx) - checkHistoryJobArgs(c, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) + require.NoError(t, err) + v := getSchemaVer(t, ctx) + checkHistoryJobArgs(t, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) return job } @@ -319,7 +319,7 @@ func buildRebaseAutoIDJobJob(dbInfo *model.DBInfo, tblInfo *model.TableInfo, new } } -func (s *testDDLSuite) TestGetIntervalFromPolicy(c *C) { +func (s *testDDLSuiteToVerify) TestGetIntervalFromPolicy() { policy := []time.Duration{ 1 * time.Second, 2 * time.Second, @@ -330,18 +330,18 @@ func (s *testDDLSuite) TestGetIntervalFromPolicy(c *C) { ) val, changed = getIntervalFromPolicy(policy, 0) - c.Assert(val, Equals, 1*time.Second) - c.Assert(changed, Equals, true) + require.Equal(s.T(), val, 1*time.Second) + require.Equal(s.T(), changed, true) val, changed = getIntervalFromPolicy(policy, 1) - c.Assert(val, Equals, 2*time.Second) - c.Assert(changed, Equals, true) + require.Equal(s.T(), val, 2*time.Second) + require.Equal(s.T(), changed, true) val, changed = getIntervalFromPolicy(policy, 2) - c.Assert(val, Equals, 2*time.Second) - c.Assert(changed, Equals, false) + require.Equal(s.T(), val, 2*time.Second) + require.Equal(s.T(), changed, false) val, changed = getIntervalFromPolicy(policy, 3) - c.Assert(val, Equals, 2*time.Second) - c.Assert(changed, Equals, false) + require.Equal(s.T(), val, 2*time.Second) + require.Equal(s.T(), changed, false) } diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index 56b74381f2..3811b3cc96 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -20,9 +20,9 @@ import ( "testing" "time" - . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/domain/infosync" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/parser/ast" @@ -38,30 +38,46 @@ import ( "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/testutil" "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" ) -var _ = Suite(&testDDLSuite{}) -var _ = SerialSuites(&testDDLSerialSuite{}) - -type testDDLSuite struct { +type testDDLSuiteToVerify struct { + suite.Suite testutil.CommonHandleSuite } -type testDDLSerialSuite struct{} + +func TestDDLSuite(t *testing.T) { + _, err := infosync.GlobalInfoSyncerInit(context.Background(), "t", func() uint64 { return 1 }, nil, true) + require.NoError(t, err) + + suite.Run(t, new(testDDLSuiteToVerify)) +} + +type testDDLSerialSuiteToVerify struct { + suite.Suite +} + +func TestDDLSerialSuite(t *testing.T) { + _, err := infosync.GlobalInfoSyncerInit(context.Background(), "t", func() uint64 { return 1 }, nil, true) + require.NoError(t, err) + + suite.Run(t, new(testDDLSerialSuiteToVerify)) +} const testLease = 5 * time.Millisecond -func (s *testDDLSerialSuite) SetUpSuite(c *C) { +func (s *testDDLSerialSuiteToVerify) SetupSuite() { SetWaitTimeWhenErrorOccurred(1 * time.Microsecond) // We hope that this test is serially executed. So put it here. - s.testRunWorker(c) + s.testRunWorker() } -func (s *testDDLSuite) TestCheckOwner(c *C) { - store := testCreateStore(c, "test_owner") +func (s *testDDLSuiteToVerify) TestCheckOwner() { + store := testCreateStore(s.T(), "test_owner") defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() d1, err := testNewDDLAndStart( @@ -69,19 +85,19 @@ func (s *testDDLSuite) TestCheckOwner(c *C) { WithStore(store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := d1.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() time.Sleep(testLease) - testCheckOwner(c, d1, true) + testCheckOwner(s.T(), d1, true) - c.Assert(d1.GetLease(), Equals, testLease) + require.Equal(s.T(), d1.GetLease(), testLease) } -func (s *testDDLSuite) TestNotifyDDLJob(c *C) { - store := testCreateStore(c, "test_notify_job") +func (s *testDDLSuiteToVerify) TestNotifyDDLJob() { + store := testCreateStore(s.T(), "test_notify_job") defer store.Close() getFirstNotificationAfterStartDDL := func(d *ddl) { @@ -102,10 +118,10 @@ func (s *testDDLSuite) TestNotifyDDLJob(c *C) { WithStore(store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() getFirstNotificationAfterStartDDL(d) // Ensure that the notification is not handled in workers `start` function. @@ -127,7 +143,7 @@ func (s *testDDLSuite) TestNotifyDDLJob(c *C) { select { case <-d.workers[generalWorker].ddlJobCh: default: - c.Fatal("do not get the general job notification") + require.FailNow(s.T(), "do not get the general job notification") } // Test the notification mechanism of the owner and the server receiving the DDL request on the same TiDB. // This DDL request is a add index DDL job. @@ -136,7 +152,7 @@ func (s *testDDLSuite) TestNotifyDDLJob(c *C) { select { case <-d.workers[addIdxWorker].ddlJobCh: default: - c.Fatal("do not get the add index job notification") + require.FailNow(s.T(), "do not get the add index job notification") } // Test the notification mechanism that the owner and the server receiving the DDL request are not on the same TiDB. @@ -146,10 +162,10 @@ func (s *testDDLSuite) TestNotifyDDLJob(c *C) { WithStore(store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := d1.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() getFirstNotificationAfterStartDDL(d1) // Ensure that the notification is not handled by worker's "start". @@ -161,22 +177,22 @@ func (s *testDDLSuite) TestNotifyDDLJob(c *C) { d1.asyncNotifyWorker(job) job.Type = model.ActionCreateTable d1.asyncNotifyWorker(job) - testCheckOwner(c, d1, false) + testCheckOwner(s.T(), d1, false) select { case <-d1.workers[addIdxWorker].ddlJobCh: - c.Fatal("should not get the add index job notification") + require.FailNow(s.T(), "should not get the add index job notification") case <-d1.workers[generalWorker].ddlJobCh: - c.Fatal("should not get the general job notification") + require.FailNow(s.T(), "should not get the general job notification") default: } } // testRunWorker tests no job is handled when the value of RunWorker is false. -func (s *testDDLSerialSuite) testRunWorker(c *C) { - store := testCreateStore(c, "test_run_worker") +func (s *testDDLSerialSuiteToVerify) testRunWorker() { + store := testCreateStore(s.T(), "test_run_worker") defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() RunWorker = false @@ -185,16 +201,16 @@ func (s *testDDLSerialSuite) testRunWorker(c *C) { WithStore(store), WithLease(testLease), ) - c.Assert(err, IsNil) - testCheckOwner(c, d, false) + require.NoError(s.T(), err) + testCheckOwner(s.T(), d, false) defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() // Make sure the DDL worker is nil. worker := d.generalWorker() - c.Assert(worker, IsNil) + require.Nil(s.T(), worker) // Make sure the DDL job can be done and exit that goroutine. RunWorker = true d1, err := testNewDDLAndStart( @@ -202,21 +218,21 @@ func (s *testDDLSerialSuite) testRunWorker(c *C) { WithStore(store), WithLease(testLease), ) - c.Assert(err, IsNil) - testCheckOwner(c, d1, true) + require.NoError(s.T(), err) + testCheckOwner(s.T(), d1, true) defer func() { err := d1.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() worker = d1.generalWorker() - c.Assert(worker, NotNil) + require.NotNil(s.T(), worker) } -func (s *testDDLSuite) TestSchemaError(c *C) { - store := testCreateStore(c, "test_schema_error") +func (s *testDDLSuiteToVerify) TestSchemaError() { + store := testCreateStore(s.T(), "test_schema_error") defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() d, err := testNewDDLAndStart( @@ -224,21 +240,21 @@ func (s *testDDLSuite) TestSchemaError(c *C) { WithStore(store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() ctx := testNewContext(d) - doDDLJobErr(c, 1, 0, model.ActionCreateSchema, []interface{}{1}, ctx, d) + doDDLJobErr(s.T(), 1, 0, model.ActionCreateSchema, []interface{}{1}, ctx, d) } -func (s *testDDLSuite) TestTableError(c *C) { - store := testCreateStore(c, "test_table_error") +func (s *testDDLSuiteToVerify) TestTableError() { + store := testCreateStore(s.T(), "test_table_error") defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() d, err := testNewDDLAndStart( @@ -246,53 +262,53 @@ func (s *testDDLSuite) TestTableError(c *C) { WithStore(store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() ctx := testNewContext(d) // Schema ID is wrong, so dropping table is failed. - doDDLJobErr(c, -1, 1, model.ActionDropTable, nil, ctx, d) + doDDLJobErr(s.T(), -1, 1, model.ActionDropTable, nil, ctx, d) // Table ID is wrong, so dropping table is failed. dbInfo, err := testSchemaInfo(d, "test_ddl") - c.Assert(err, IsNil) - testCreateSchema(c, testNewContext(d), d, dbInfo) - job := doDDLJobErr(c, dbInfo.ID, -1, model.ActionDropTable, nil, ctx, d) + require.NoError(s.T(), err) + testCreateSchema(s.T(), testNewContext(d), d, dbInfo) + job := doDDLJobErr(s.T(), dbInfo.ID, -1, model.ActionDropTable, nil, ctx, d) // Table ID or schema ID is wrong, so getting table is failed. tblInfo, err := testTableInfo(d, "t", 3) - c.Assert(err, IsNil) - testCreateTable(c, ctx, d, dbInfo, tblInfo) + require.NoError(s.T(), err) + testCreateTable(s.T(), ctx, d, dbInfo, tblInfo) err = kv.RunInNewTxn(context.Background(), store, false, func(ctx context.Context, txn kv.Transaction) error { job.SchemaID = -1 job.TableID = -1 t := meta.NewMeta(txn) _, err1 := getTableInfoAndCancelFaultJob(t, job, job.SchemaID) - c.Assert(err1, NotNil) + require.Error(s.T(), err1) job.SchemaID = dbInfo.ID _, err1 = getTableInfoAndCancelFaultJob(t, job, job.SchemaID) - c.Assert(err1, NotNil) + require.Error(s.T(), err1) return nil }) - c.Assert(err, IsNil) + require.NoError(s.T(), err) // Args is wrong, so creating table is failed. - doDDLJobErr(c, 1, 1, model.ActionCreateTable, []interface{}{1}, ctx, d) + doDDLJobErr(s.T(), 1, 1, model.ActionCreateTable, []interface{}{1}, ctx, d) // Schema ID is wrong, so creating table is failed. - doDDLJobErr(c, -1, tblInfo.ID, model.ActionCreateTable, []interface{}{tblInfo}, ctx, d) + doDDLJobErr(s.T(), -1, tblInfo.ID, model.ActionCreateTable, []interface{}{tblInfo}, ctx, d) // Table exists, so creating table is failed. tblInfo.ID++ - doDDLJobErr(c, dbInfo.ID, tblInfo.ID, model.ActionCreateTable, []interface{}{tblInfo}, ctx, d) + doDDLJobErr(s.T(), dbInfo.ID, tblInfo.ID, model.ActionCreateTable, []interface{}{tblInfo}, ctx, d) } -func (s *testDDLSuite) TestViewError(c *C) { - store := testCreateStore(c, "test_view_error") +func (s *testDDLSuiteToVerify) TestViewError() { + store := testCreateStore(s.T(), "test_view_error") defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() d, err := testNewDDLAndStart( @@ -300,45 +316,45 @@ func (s *testDDLSuite) TestViewError(c *C) { WithStore(store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() ctx := testNewContext(d) dbInfo, err := testSchemaInfo(d, "test_ddl") - c.Assert(err, IsNil) - testCreateSchema(c, testNewContext(d), d, dbInfo) + require.NoError(s.T(), err) + testCreateSchema(s.T(), testNewContext(d), d, dbInfo) // Table ID or schema ID is wrong, so getting table is failed. - tblInfo := testViewInfo(c, d, "t", 3) - testCreateView(c, ctx, d, dbInfo, tblInfo) + tblInfo := testViewInfo(s.T(), d, "t", 3) + testCreateView(s.T(), ctx, d, dbInfo, tblInfo) // Args is wrong, so creating view is failed. - doDDLJobErr(c, 1, 1, model.ActionCreateView, []interface{}{1}, ctx, d) + doDDLJobErr(s.T(), 1, 1, model.ActionCreateView, []interface{}{1}, ctx, d) // Schema ID is wrong and orReplace is false, so creating view is failed. - doDDLJobErr(c, -1, tblInfo.ID, model.ActionCreateView, []interface{}{tblInfo, false}, ctx, d) + doDDLJobErr(s.T(), -1, tblInfo.ID, model.ActionCreateView, []interface{}{tblInfo, false}, ctx, d) // View exists and orReplace is false, so creating view is failed. tblInfo.ID++ - doDDLJobErr(c, dbInfo.ID, tblInfo.ID, model.ActionCreateView, []interface{}{tblInfo, false}, ctx, d) + doDDLJobErr(s.T(), dbInfo.ID, tblInfo.ID, model.ActionCreateView, []interface{}{tblInfo, false}, ctx, d) } -func (s *testDDLSuite) TestInvalidDDLJob(c *C) { - store := testCreateStore(c, "test_invalid_ddl_job_type_error") +func (s *testDDLSuiteToVerify) TestInvalidDDLJob() { + store := testCreateStore(s.T(), "test_invalid_ddl_job_type_error") defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() d, err := testNewDDLAndStart( context.Background(), WithStore(store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() ctx := testNewContext(d) @@ -350,14 +366,14 @@ func (s *testDDLSuite) TestInvalidDDLJob(c *C) { Args: []interface{}{}, } err = d.doDDLJob(ctx, job) - c.Assert(err.Error(), Equals, "[ddl:8204]invalid ddl job type: none") + require.Equal(s.T(), err.Error(), "[ddl:8204]invalid ddl job type: none") } -func (s *testDDLSuite) TestForeignKeyError(c *C) { - store := testCreateStore(c, "test_foreign_key_error") +func (s *testDDLSuiteToVerify) TestForeignKeyError() { + store := testCreateStore(s.T(), "test_foreign_key_error") defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() d, err := testNewDDLAndStart( @@ -365,30 +381,30 @@ func (s *testDDLSuite) TestForeignKeyError(c *C) { WithStore(store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() ctx := testNewContext(d) - doDDLJobErr(c, -1, 1, model.ActionAddForeignKey, nil, ctx, d) - doDDLJobErr(c, -1, 1, model.ActionDropForeignKey, nil, ctx, d) + doDDLJobErr(s.T(), -1, 1, model.ActionAddForeignKey, nil, ctx, d) + doDDLJobErr(s.T(), -1, 1, model.ActionDropForeignKey, nil, ctx, d) dbInfo, err := testSchemaInfo(d, "test_ddl") - c.Assert(err, IsNil) + require.NoError(s.T(), err) tblInfo, err := testTableInfo(d, "t", 3) - c.Assert(err, IsNil) - testCreateSchema(c, ctx, d, dbInfo) - testCreateTable(c, ctx, d, dbInfo, tblInfo) - doDDLJobErr(c, dbInfo.ID, tblInfo.ID, model.ActionDropForeignKey, []interface{}{model.NewCIStr("c1_foreign_key")}, ctx, d) + require.NoError(s.T(), err) + testCreateSchema(s.T(), ctx, d, dbInfo) + testCreateTable(s.T(), ctx, d, dbInfo, tblInfo) + doDDLJobErr(s.T(), dbInfo.ID, tblInfo.ID, model.ActionDropForeignKey, []interface{}{model.NewCIStr("c1_foreign_key")}, ctx, d) } -func (s *testDDLSuite) TestIndexError(c *C) { - store := testCreateStore(c, "test_index_error") +func (s *testDDLSuiteToVerify) TestIndexError() { + store := testCreateStore(s.T(), "test_index_error") defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() d, err := testNewDDLAndStart( @@ -396,67 +412,67 @@ func (s *testDDLSuite) TestIndexError(c *C) { WithStore(store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() ctx := testNewContext(d) // Schema ID is wrong. - doDDLJobErr(c, -1, 1, model.ActionAddIndex, nil, ctx, d) - doDDLJobErr(c, -1, 1, model.ActionDropIndex, nil, ctx, d) + doDDLJobErr(s.T(), -1, 1, model.ActionAddIndex, nil, ctx, d) + doDDLJobErr(s.T(), -1, 1, model.ActionDropIndex, nil, ctx, d) dbInfo, err := testSchemaInfo(d, "test_ddl") - c.Assert(err, IsNil) + require.NoError(s.T(), err) tblInfo, err := testTableInfo(d, "t", 3) - c.Assert(err, IsNil) - testCreateSchema(c, ctx, d, dbInfo) - testCreateTable(c, ctx, d, dbInfo, tblInfo) + require.NoError(s.T(), err) + testCreateSchema(s.T(), ctx, d, dbInfo) + testCreateTable(s.T(), ctx, d, dbInfo, tblInfo) // for adding index - doDDLJobErr(c, dbInfo.ID, tblInfo.ID, model.ActionAddIndex, []interface{}{1}, ctx, d) - doDDLJobErr(c, dbInfo.ID, tblInfo.ID, model.ActionAddIndex, + doDDLJobErr(s.T(), dbInfo.ID, tblInfo.ID, model.ActionAddIndex, []interface{}{1}, ctx, d) + doDDLJobErr(s.T(), dbInfo.ID, tblInfo.ID, model.ActionAddIndex, []interface{}{false, model.NewCIStr("t"), 1, []*ast.IndexPartSpecification{{Column: &ast.ColumnName{Name: model.NewCIStr("c")}, Length: 256}}}, ctx, d) - doDDLJobErr(c, dbInfo.ID, tblInfo.ID, model.ActionAddIndex, + doDDLJobErr(s.T(), dbInfo.ID, tblInfo.ID, model.ActionAddIndex, []interface{}{false, model.NewCIStr("c1_index"), 1, []*ast.IndexPartSpecification{{Column: &ast.ColumnName{Name: model.NewCIStr("c")}, Length: 256}}}, ctx, d) - testCreateIndex(c, ctx, d, dbInfo, tblInfo, false, "c1_index", "c1") - doDDLJobErr(c, dbInfo.ID, tblInfo.ID, model.ActionAddIndex, + testCreateIndex(s.T(), ctx, d, dbInfo, tblInfo, false, "c1_index", "c1") + doDDLJobErr(s.T(), dbInfo.ID, tblInfo.ID, model.ActionAddIndex, []interface{}{false, model.NewCIStr("c1_index"), 1, []*ast.IndexPartSpecification{{Column: &ast.ColumnName{Name: model.NewCIStr("c1")}, Length: 256}}}, ctx, d) // for dropping index - doDDLJobErr(c, dbInfo.ID, tblInfo.ID, model.ActionDropIndex, []interface{}{1}, ctx, d) - testDropIndex(c, ctx, d, dbInfo, tblInfo, "c1_index") - doDDLJobErr(c, dbInfo.ID, tblInfo.ID, model.ActionDropIndex, []interface{}{model.NewCIStr("c1_index")}, ctx, d) + doDDLJobErr(s.T(), dbInfo.ID, tblInfo.ID, model.ActionDropIndex, []interface{}{1}, ctx, d) + testDropIndex(s.T(), ctx, d, dbInfo, tblInfo, "c1_index") + doDDLJobErr(s.T(), dbInfo.ID, tblInfo.ID, model.ActionDropIndex, []interface{}{model.NewCIStr("c1_index")}, ctx, d) } -func (s *testDDLSuite) TestColumnError(c *C) { - store := testCreateStore(c, "test_column_error") +func (s *testDDLSuiteToVerify) TestColumnError() { + store := testCreateStore(s.T(), "test_column_error") defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() d, err := testNewDDLAndStart( context.Background(), WithStore(store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() ctx := testNewContext(d) dbInfo, err := testSchemaInfo(d, "test_ddl") - c.Assert(err, IsNil) + require.NoError(s.T(), err) tblInfo, err := testTableInfo(d, "t", 3) - c.Assert(err, IsNil) - testCreateSchema(c, ctx, d, dbInfo) - testCreateTable(c, ctx, d, dbInfo, tblInfo) + require.NoError(s.T(), err) + testCreateSchema(s.T(), ctx, d, dbInfo) + testCreateTable(s.T(), ctx, d, dbInfo, tblInfo) col := &model.ColumnInfo{ Name: model.NewCIStr("c4"), Offset: len(tblInfo.Columns), @@ -470,75 +486,75 @@ func (s *testDDLSuite) TestColumnError(c *C) { positions := &[]*ast.ColumnPosition{pos} // for adding column - doDDLJobErr(c, -1, tblInfo.ID, model.ActionAddColumn, []interface{}{col, pos, 0}, ctx, d) - doDDLJobErr(c, dbInfo.ID, -1, model.ActionAddColumn, []interface{}{col, pos, 0}, ctx, d) - doDDLJobErr(c, dbInfo.ID, tblInfo.ID, model.ActionAddColumn, []interface{}{0}, ctx, d) - doDDLJobErr(c, dbInfo.ID, tblInfo.ID, model.ActionAddColumn, []interface{}{col, pos, 0}, ctx, d) + doDDLJobErr(s.T(), -1, tblInfo.ID, model.ActionAddColumn, []interface{}{col, pos, 0}, ctx, d) + doDDLJobErr(s.T(), dbInfo.ID, -1, model.ActionAddColumn, []interface{}{col, pos, 0}, ctx, d) + doDDLJobErr(s.T(), dbInfo.ID, tblInfo.ID, model.ActionAddColumn, []interface{}{0}, ctx, d) + doDDLJobErr(s.T(), dbInfo.ID, tblInfo.ID, model.ActionAddColumn, []interface{}{col, pos, 0}, ctx, d) // for dropping column - doDDLJobErr(c, -1, tblInfo.ID, model.ActionDropColumn, []interface{}{col, pos, 0}, ctx, d) - doDDLJobErr(c, dbInfo.ID, -1, model.ActionDropColumn, []interface{}{col, pos, 0}, ctx, d) - doDDLJobErr(c, dbInfo.ID, tblInfo.ID, model.ActionDropColumn, []interface{}{0}, ctx, d) - doDDLJobErr(c, dbInfo.ID, tblInfo.ID, model.ActionDropColumn, []interface{}{model.NewCIStr("c5")}, ctx, d) + doDDLJobErr(s.T(), -1, tblInfo.ID, model.ActionDropColumn, []interface{}{col, pos, 0}, ctx, d) + doDDLJobErr(s.T(), dbInfo.ID, -1, model.ActionDropColumn, []interface{}{col, pos, 0}, ctx, d) + doDDLJobErr(s.T(), dbInfo.ID, tblInfo.ID, model.ActionDropColumn, []interface{}{0}, ctx, d) + doDDLJobErr(s.T(), dbInfo.ID, tblInfo.ID, model.ActionDropColumn, []interface{}{model.NewCIStr("c5")}, ctx, d) // for adding columns - doDDLJobErr(c, -1, tblInfo.ID, model.ActionAddColumns, []interface{}{cols, positions, 0}, ctx, d) - doDDLJobErr(c, dbInfo.ID, -1, model.ActionAddColumns, []interface{}{cols, positions, 0}, ctx, d) - doDDLJobErr(c, dbInfo.ID, tblInfo.ID, model.ActionAddColumns, []interface{}{0}, ctx, d) - doDDLJobErr(c, dbInfo.ID, tblInfo.ID, model.ActionAddColumns, []interface{}{cols, positions, 0}, ctx, d) + doDDLJobErr(s.T(), -1, tblInfo.ID, model.ActionAddColumns, []interface{}{cols, positions, 0}, ctx, d) + doDDLJobErr(s.T(), dbInfo.ID, -1, model.ActionAddColumns, []interface{}{cols, positions, 0}, ctx, d) + doDDLJobErr(s.T(), dbInfo.ID, tblInfo.ID, model.ActionAddColumns, []interface{}{0}, ctx, d) + doDDLJobErr(s.T(), dbInfo.ID, tblInfo.ID, model.ActionAddColumns, []interface{}{cols, positions, 0}, ctx, d) // for dropping columns - doDDLJobErr(c, -1, tblInfo.ID, model.ActionDropColumns, []interface{}{col, pos, 0}, ctx, d) - doDDLJobErr(c, dbInfo.ID, -1, model.ActionDropColumns, []interface{}{col, pos, 0}, ctx, d) - doDDLJobErr(c, dbInfo.ID, tblInfo.ID, model.ActionDropColumns, []interface{}{0}, ctx, d) - doDDLJobErr(c, dbInfo.ID, tblInfo.ID, model.ActionDropColumns, []interface{}{[]model.CIStr{model.NewCIStr("c5"), model.NewCIStr("c6")}, make([]bool, 2)}, ctx, d) + doDDLJobErr(s.T(), -1, tblInfo.ID, model.ActionDropColumns, []interface{}{col, pos, 0}, ctx, d) + doDDLJobErr(s.T(), dbInfo.ID, -1, model.ActionDropColumns, []interface{}{col, pos, 0}, ctx, d) + doDDLJobErr(s.T(), dbInfo.ID, tblInfo.ID, model.ActionDropColumns, []interface{}{0}, ctx, d) + doDDLJobErr(s.T(), dbInfo.ID, tblInfo.ID, model.ActionDropColumns, []interface{}{[]model.CIStr{model.NewCIStr("c5"), model.NewCIStr("c6")}, make([]bool, 2)}, ctx, d) } -func (s *testDDLSerialSuite) TestAddBatchJobError(c *C) { - store := testCreateStore(c, "test_add_batch_job_error") +func (s *testDDLSerialSuiteToVerify) TestAddBatchJobError() { + store := testCreateStore(s.T(), "test_add_batch_job_error") defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() d, err := testNewDDLAndStart( context.Background(), WithStore(store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() ctx := testNewContext(d) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/mockAddBatchDDLJobsErr", `return(true)`), IsNil) + require.Nil(s.T(), failpoint.Enable("github.com/pingcap/tidb/ddl/mockAddBatchDDLJobsErr", `return(true)`)) // Test the job runner should not hang forever. job := &model.Job{SchemaID: 1, TableID: 1} err = d.doDDLJob(ctx, job) - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "mockAddBatchDDLJobsErr") - c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/mockAddBatchDDLJobsErr"), IsNil) + require.Error(s.T(), err) + require.Equal(s.T(), err.Error(), "mockAddBatchDDLJobsErr") + require.Nil(s.T(), failpoint.Disable("github.com/pingcap/tidb/ddl/mockAddBatchDDLJobsErr")) } -func testCheckOwner(c *C, d *ddl, expectedVal bool) { - c.Assert(d.isOwner(), Equals, expectedVal) +func testCheckOwner(t *testing.T, d *ddl, expectedVal bool) { + require.Equal(t, d.isOwner(), expectedVal) } -func testCheckJobDone(c *C, d *ddl, job *model.Job, isAdd bool) { +func testCheckJobDone(test *testing.T, d *ddl, job *model.Job, isAdd bool) { err := kv.RunInNewTxn(context.Background(), d.store, false, func(ctx context.Context, txn kv.Transaction) error { t := meta.NewMeta(txn) historyJob, err := t.GetHistoryDDLJob(job.ID) - c.Assert(err, IsNil) - checkHistoryJob(c, historyJob) + require.NoError(test, err) + checkHistoryJob(test, historyJob) if isAdd { - c.Assert(historyJob.SchemaState, Equals, model.StatePublic) + require.Equal(test, historyJob.SchemaState, model.StatePublic) } else { - c.Assert(historyJob.SchemaState, Equals, model.StateNone) + require.Equal(test, historyJob.SchemaState, model.StateNone) } return nil }) - c.Assert(err, IsNil) + require.NoError(test, err) } func testCheckJobDoneT(t *testing.T, d *ddl, job *model.Job, isAdd bool) { @@ -572,21 +588,21 @@ func testCheckJobCancelledT(t *testing.T, d *ddl, job *model.Job, state *model.S require.NoError(t, err) } -func testCheckJobCancelled(c *C, d *ddl, job *model.Job, state *model.SchemaState) { +func testCheckJobCancelled(test *testing.T, d *ddl, job *model.Job, state *model.SchemaState) { err := kv.RunInNewTxn(context.Background(), d.store, false, func(ctx context.Context, txn kv.Transaction) error { t := meta.NewMeta(txn) historyJob, err := t.GetHistoryDDLJob(job.ID) - c.Assert(err, IsNil) - c.Assert(historyJob.IsCancelled() || historyJob.IsRollbackDone(), IsTrue, Commentf("history job %s", historyJob)) + require.NoError(test, err) + require.True(test, historyJob.IsCancelled() || historyJob.IsRollbackDone(), "history job %s", historyJob) if state != nil { - c.Assert(historyJob.SchemaState, Equals, *state) + require.Equal(test, historyJob.SchemaState, *state) } return nil }) - c.Assert(err, IsNil) + require.NoError(test, err) } -func doDDLJobErrWithSchemaState(ctx sessionctx.Context, d *ddl, c *C, schemaID, tableID int64, tp model.ActionType, +func doDDLJobErrWithSchemaState(ctx sessionctx.Context, d *ddl, t *testing.T, schemaID, tableID int64, tp model.ActionType, args []interface{}, state *model.SchemaState) *model.Job { job := &model.Job{ SchemaID: schemaID, @@ -597,8 +613,8 @@ func doDDLJobErrWithSchemaState(ctx sessionctx.Context, d *ddl, c *C, schemaID, } err := d.doDDLJob(ctx, job) // TODO: Add the detail error check. - c.Assert(err, NotNil, Commentf("err:%v", err)) - testCheckJobCancelled(c, d, job, state) + require.Error(t, err, "err:%v", err) + testCheckJobCancelled(t, d, job, state) return job } @@ -620,7 +636,7 @@ func doDDLJobErrWithSchemaStateT(ctx sessionctx.Context, d *ddl, t *testing.T, s return job } -func doDDLJobSuccess(ctx sessionctx.Context, d *ddl, c *C, schemaID, tableID int64, tp model.ActionType, +func doDDLJobSuccess(ctx sessionctx.Context, d *ddl, t *testing.T, schemaID, tableID int64, tp model.ActionType, args []interface{}) { job := &model.Job{ SchemaID: schemaID, @@ -630,12 +646,12 @@ func doDDLJobSuccess(ctx sessionctx.Context, d *ddl, c *C, schemaID, tableID int BinlogInfo: &model.HistoryInfo{}, } err := d.doDDLJob(ctx, job) - c.Assert(err, IsNil) + require.Nil(t, err) } -func doDDLJobErr(c *C, schemaID, tableID int64, tp model.ActionType, args []interface{}, +func doDDLJobErr(t *testing.T, schemaID, tableID int64, tp model.ActionType, args []interface{}, ctx sessionctx.Context, d *ddl) *model.Job { - return doDDLJobErrWithSchemaState(ctx, d, c, schemaID, tableID, tp, args, nil) + return doDDLJobErrWithSchemaState(ctx, d, t, schemaID, tableID, tp, args, nil) } func doDDLJobErrT(t *testing.T, schemaID, tableID int64, tp model.ActionType, args []interface{}, @@ -760,16 +776,16 @@ func buildCancelJobTests(firstID int64) []testCancelJob { return tests } -func (s *testDDLSerialSuite) checkDropIdx(c *C, d *ddl, schemaID int64, tableID int64, idxName string, success bool) { - checkIdxExist(c, d, schemaID, tableID, idxName, !success) +func (s *testDDLSerialSuiteToVerify) checkDropIdx(t *testing.T, d *ddl, schemaID int64, tableID int64, idxName string, success bool) { + checkIdxExist(t, d, schemaID, tableID, idxName, !success) } -func (s *testDDLSerialSuite) checkAddIdx(c *C, d *ddl, schemaID int64, tableID int64, idxName string, success bool) { - checkIdxExist(c, d, schemaID, tableID, idxName, success) +func (s *testDDLSerialSuiteToVerify) checkAddIdx(t *testing.T, d *ddl, schemaID int64, tableID int64, idxName string, success bool) { + checkIdxExist(t, d, schemaID, tableID, idxName, success) } -func checkIdxExist(c *C, d *ddl, schemaID int64, tableID int64, idxName string, expectedExist bool) { - changedTable := testGetTable(c, d, schemaID, tableID) +func checkIdxExist(t *testing.T, d *ddl, schemaID int64, tableID int64, idxName string, expectedExist bool) { + changedTable := testGetTable(t, d, schemaID, tableID) var found bool for _, idxInfo := range changedTable.Meta().Indices { if idxInfo.Name.O == idxName { @@ -777,19 +793,19 @@ func checkIdxExist(c *C, d *ddl, schemaID int64, tableID int64, idxName string, break } } - c.Assert(found, Equals, expectedExist) + require.Equal(t, found, expectedExist) } -func (s *testDDLSerialSuite) checkAddColumns(c *C, d *ddl, schemaID int64, tableID int64, colNames []string, success bool) { - changedTable := testGetTable(c, d, schemaID, tableID) +func (s *testDDLSerialSuiteToVerify) checkAddColumns(d *ddl, schemaID int64, tableID int64, colNames []string, success bool) { + changedTable := testGetTable(s.T(), d, schemaID, tableID) found := !checkColumnsNotFound(changedTable, colNames) - c.Assert(found, Equals, success) + require.Equal(s.T(), found, success) } -func (s *testDDLSerialSuite) checkCancelDropColumns(c *C, d *ddl, schemaID int64, tableID int64, colNames []string, success bool) { - changedTable := testGetTable(c, d, schemaID, tableID) +func (s *testDDLSerialSuiteToVerify) checkCancelDropColumns(d *ddl, schemaID int64, tableID int64, colNames []string, success bool) { + changedTable := testGetTable(s.T(), d, schemaID, tableID) notFound := checkColumnsNotFound(changedTable, colNames) - c.Assert(notFound, Equals, success) + require.Equal(s.T(), notFound, success) } func checkColumnsNotFound(t table.Table, colNames []string) bool { @@ -813,56 +829,56 @@ func checkIdxVisibility(changedTable table.Table, idxName string, expected bool) return false } -func (s *testDDLSerialSuite) TestCancelJob(c *C) { - store := testCreateStore(c, "test_cancel_job") +func (s *testDDLSerialSuiteToVerify) TestCancelJob() { + store := testCreateStore(s.T(), "test_cancel_job") defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() d, err := testNewDDLAndStart( context.Background(), WithStore(store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() dbInfo, err := testSchemaInfo(d, "test_cancel_job") - c.Assert(err, IsNil) - testCreateSchema(c, testNewContext(d), d, dbInfo) + require.NoError(s.T(), err) + testCreateSchema(s.T(), testNewContext(d), d, dbInfo) // create a partition table. - partitionTblInfo := testTableInfoWithPartition(c, d, "t_partition", 5) + partitionTblInfo := testTableInfoWithPartition(s.T(), d, "t_partition", 5) // Skip using sessPool. Make sure adding primary key can be successful. partitionTblInfo.Columns[0].Flag |= mysql.NotNullFlag // create table t (c1 int, c2 int, c3 int, c4 int, c5 int); tblInfo, err := testTableInfo(d, "t", 5) - c.Assert(err, IsNil) + require.NoError(s.T(), err) ctx := testNewContext(d) err = ctx.NewTxn(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = ctx.GetSessionVars().SetSystemVar("tidb_enable_exchange_partition", "1") - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := ctx.GetSessionVars().SetSystemVar("tidb_enable_exchange_partition", "0") - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() - testCreateTable(c, ctx, d, dbInfo, partitionTblInfo) + testCreateTable(s.T(), ctx, d, dbInfo, partitionTblInfo) tableAutoID := int64(100) shardRowIDBits := uint64(5) tblInfo.AutoIncID = tableAutoID tblInfo.ShardRowIDBits = shardRowIDBits - job := testCreateTable(c, ctx, d, dbInfo, tblInfo) + job := testCreateTable(s.T(), ctx, d, dbInfo, tblInfo) // insert t values (1, 2, 3, 4, 5); - originTable := testGetTable(c, d, dbInfo.ID, tblInfo.ID) + originTable := testGetTable(s.T(), d, dbInfo.ID, tblInfo.ID) row := types.MakeDatums(1, 2, 3, 4, 5) _, err = originTable.AddRecord(ctx, row) - c.Assert(err, IsNil) + require.NoError(s.T(), err) txn, err := ctx.Txn(true) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) tc := &TestDDLCallback{} // set up hook @@ -931,24 +947,24 @@ func (s *testDDLSerialSuite) TestCancelJob(c *C) { // When the job satisfies this test case, the option will be rollback, so the job's schema state is none. cancelState := model.StateNone - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, model.ActionAddIndex, validArgs, &cancelState) - c.Check(checkErr, IsNil) - s.checkAddIdx(c, d, dbInfo.ID, tblInfo.ID, idxOrigName, false) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, model.ActionAddIndex, validArgs, &cancelState) + require.NoError(s.T(), checkErr) + s.checkAddIdx(s.T(), d, dbInfo.ID, tblInfo.ID, idxOrigName, false) updateTest(&tests[1]) - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, model.ActionAddIndex, validArgs, &cancelState) - c.Check(checkErr, IsNil) - s.checkAddIdx(c, d, dbInfo.ID, tblInfo.ID, idxOrigName, false) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, model.ActionAddIndex, validArgs, &cancelState) + require.NoError(s.T(), checkErr) + s.checkAddIdx(s.T(), d, dbInfo.ID, tblInfo.ID, idxOrigName, false) updateTest(&tests[2]) - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, model.ActionAddIndex, validArgs, &cancelState) - c.Check(checkErr, IsNil) - s.checkAddIdx(c, d, dbInfo.ID, tblInfo.ID, idxOrigName, false) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, model.ActionAddIndex, validArgs, &cancelState) + require.NoError(s.T(), checkErr) + s.checkAddIdx(s.T(), d, dbInfo.ID, tblInfo.ID, idxOrigName, false) updateTest(&tests[3]) - testCreateIndex(c, ctx, d, dbInfo, tblInfo, false, "idx", "c2") - c.Check(checkErr, IsNil) + testCreateIndex(s.T(), ctx, d, dbInfo, tblInfo, false, "idx", "c2") + require.NoError(s.T(), checkErr) txn, err = ctx.Txn(true) - c.Assert(err, IsNil) - c.Assert(txn.Commit(context.Background()), IsNil) - s.checkAddIdx(c, d, dbInfo.ID, tblInfo.ID, idxOrigName, true) + require.NoError(s.T(), err) + require.Nil(s.T(), txn.Commit(context.Background())) + s.checkAddIdx(s.T(), d, dbInfo.ID, tblInfo.ID, idxOrigName, true) // for add column updateTest(&tests[4]) @@ -960,91 +976,91 @@ func (s *testDDLSerialSuite) TestCancelJob(c *C) { } chs, coll := charset.GetDefaultCharsetAndCollate() col, _, err := buildColumnAndConstraint(ctx, 2, newColumnDef, nil, chs, coll) - c.Assert(err, IsNil) + require.NoError(s.T(), err) addColumnArgs := []interface{}{col, &ast.ColumnPosition{Tp: ast.ColumnPositionNone}, 0} - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, model.ActionAddColumn, addColumnArgs, &cancelState) - c.Check(checkErr, IsNil) - s.checkAddColumns(c, d, dbInfo.ID, tblInfo.ID, []string{addingColName}, false) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, model.ActionAddColumn, addColumnArgs, &cancelState) + require.NoError(s.T(), checkErr) + s.checkAddColumns(d, dbInfo.ID, tblInfo.ID, []string{addingColName}, false) updateTest(&tests[5]) - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, model.ActionAddColumn, addColumnArgs, &cancelState) - c.Check(checkErr, IsNil) - s.checkAddColumns(c, d, dbInfo.ID, tblInfo.ID, []string{addingColName}, false) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, model.ActionAddColumn, addColumnArgs, &cancelState) + require.NoError(s.T(), checkErr) + s.checkAddColumns(d, dbInfo.ID, tblInfo.ID, []string{addingColName}, false) updateTest(&tests[6]) - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, model.ActionAddColumn, addColumnArgs, &cancelState) - c.Check(checkErr, IsNil) - s.checkAddColumns(c, d, dbInfo.ID, tblInfo.ID, []string{addingColName}, false) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, model.ActionAddColumn, addColumnArgs, &cancelState) + require.NoError(s.T(), checkErr) + s.checkAddColumns(d, dbInfo.ID, tblInfo.ID, []string{addingColName}, false) updateTest(&tests[7]) - testAddColumn(c, ctx, d, dbInfo, tblInfo, addColumnArgs) - c.Check(checkErr, IsNil) - s.checkAddColumns(c, d, dbInfo.ID, tblInfo.ID, []string{addingColName}, true) + testAddColumn(s.T(), ctx, d, dbInfo, tblInfo, addColumnArgs) + require.NoError(s.T(), checkErr) + s.checkAddColumns(d, dbInfo.ID, tblInfo.ID, []string{addingColName}, true) // for create table tblInfo1, err := testTableInfo(d, "t1", 2) - c.Assert(err, IsNil) + require.NoError(s.T(), err) updateTest(&tests[8]) - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo1.ID, model.ActionCreateTable, []interface{}{tblInfo1}, &cancelState) - c.Check(checkErr, IsNil) - testCheckTableState(c, d, dbInfo, tblInfo1, model.StateNone) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, tblInfo1.ID, model.ActionCreateTable, []interface{}{tblInfo1}, &cancelState) + require.NoError(s.T(), checkErr) + testCheckTableState(s.T(), d, dbInfo, tblInfo1, model.StateNone) // for create database dbInfo1, err := testSchemaInfo(d, "test_cancel_job1") - c.Assert(err, IsNil) + require.NoError(s.T(), err) updateTest(&tests[9]) - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo1.ID, 0, model.ActionCreateSchema, []interface{}{dbInfo1}, &cancelState) - c.Check(checkErr, IsNil) - testCheckSchemaState(c, d, dbInfo1, model.StateNone) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo1.ID, 0, model.ActionCreateSchema, []interface{}{dbInfo1}, &cancelState) + require.NoError(s.T(), checkErr) + testCheckSchemaState(s.T(), d, dbInfo1, model.StateNone) // for drop column. updateTest(&tests[10]) dropColName := "c3" - s.checkCancelDropColumns(c, d, dbInfo.ID, tblInfo.ID, []string{dropColName}, false) - testDropColumn(c, ctx, d, dbInfo, tblInfo, dropColName, false) - c.Check(checkErr, IsNil) - s.checkCancelDropColumns(c, d, dbInfo.ID, tblInfo.ID, []string{dropColName}, true) + s.checkCancelDropColumns(d, dbInfo.ID, tblInfo.ID, []string{dropColName}, false) + testDropColumn(s.T(), ctx, d, dbInfo, tblInfo, dropColName, false) + require.NoError(s.T(), checkErr) + s.checkCancelDropColumns(d, dbInfo.ID, tblInfo.ID, []string{dropColName}, true) updateTest(&tests[11]) dropColName = "c4" - s.checkCancelDropColumns(c, d, dbInfo.ID, tblInfo.ID, []string{dropColName}, false) - testDropColumn(c, ctx, d, dbInfo, tblInfo, dropColName, false) - c.Check(checkErr, IsNil) - s.checkCancelDropColumns(c, d, dbInfo.ID, tblInfo.ID, []string{dropColName}, true) + s.checkCancelDropColumns(d, dbInfo.ID, tblInfo.ID, []string{dropColName}, false) + testDropColumn(s.T(), ctx, d, dbInfo, tblInfo, dropColName, false) + require.NoError(s.T(), checkErr) + s.checkCancelDropColumns(d, dbInfo.ID, tblInfo.ID, []string{dropColName}, true) updateTest(&tests[12]) dropColName = "c5" - s.checkCancelDropColumns(c, d, dbInfo.ID, tblInfo.ID, []string{dropColName}, false) - testDropColumn(c, ctx, d, dbInfo, tblInfo, dropColName, false) - c.Check(checkErr, IsNil) - s.checkCancelDropColumns(c, d, dbInfo.ID, tblInfo.ID, []string{dropColName}, true) + s.checkCancelDropColumns(d, dbInfo.ID, tblInfo.ID, []string{dropColName}, false) + testDropColumn(s.T(), ctx, d, dbInfo, tblInfo, dropColName, false) + require.NoError(s.T(), checkErr) + s.checkCancelDropColumns(d, dbInfo.ID, tblInfo.ID, []string{dropColName}, true) // cancel rebase auto id updateTest(&tests[13]) rebaseIDArgs := []interface{}{int64(200)} - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, model.ActionRebaseAutoID, rebaseIDArgs, &cancelState) - c.Check(checkErr, IsNil) - changedTable := testGetTable(c, d, dbInfo.ID, tblInfo.ID) - c.Assert(changedTable.Meta().AutoIncID, Equals, tableAutoID) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, model.ActionRebaseAutoID, rebaseIDArgs, &cancelState) + require.NoError(s.T(), checkErr) + changedTable := testGetTable(s.T(), d, dbInfo.ID, tblInfo.ID) + require.Equal(s.T(), changedTable.Meta().AutoIncID, tableAutoID) // cancel shard bits updateTest(&tests[14]) shardRowIDArgs := []interface{}{uint64(7)} - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, model.ActionShardRowID, shardRowIDArgs, &cancelState) - c.Check(checkErr, IsNil) - changedTable = testGetTable(c, d, dbInfo.ID, tblInfo.ID) - c.Assert(changedTable.Meta().ShardRowIDBits, Equals, shardRowIDBits) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, model.ActionShardRowID, shardRowIDArgs, &cancelState) + require.NoError(s.T(), checkErr) + changedTable = testGetTable(s.T(), d, dbInfo.ID, tblInfo.ID) + require.Equal(s.T(), changedTable.Meta().ShardRowIDBits, shardRowIDBits) // modify none-state column col.DefaultValue = "1" updateTest(&tests[15]) modifyColumnArgs := []interface{}{col, col.Name, &ast.ColumnPosition{}, byte(0), uint64(0)} - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, test.act, modifyColumnArgs, &test.cancelState) - c.Check(checkErr, IsNil) - changedTable = testGetTable(c, d, dbInfo.ID, tblInfo.ID) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, test.act, modifyColumnArgs, &test.cancelState) + require.NoError(s.T(), checkErr) + changedTable = testGetTable(s.T(), d, dbInfo.ID, tblInfo.ID) changedCol := model.FindColumnInfo(changedTable.Meta().Columns, col.Name.L) - c.Assert(changedCol.DefaultValue, IsNil) + require.Nil(s.T(), changedCol.DefaultValue) // modify delete-only-state column, col.FieldType.Tp = mysql.TypeTiny @@ -1052,111 +1068,111 @@ func (s *testDDLSerialSuite) TestCancelJob(c *C) { updateTest(&tests[16]) modifyColumnArgs = []interface{}{col, col.Name, &ast.ColumnPosition{}, byte(0), uint64(0)} cancelState = model.StateNone - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, test.act, modifyColumnArgs, &cancelState) - c.Check(checkErr, IsNil) - changedTable = testGetTable(c, d, dbInfo.ID, tblInfo.ID) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, test.act, modifyColumnArgs, &cancelState) + require.NoError(s.T(), checkErr) + changedTable = testGetTable(s.T(), d, dbInfo.ID, tblInfo.ID) changedCol = model.FindColumnInfo(changedTable.Meta().Columns, col.Name.L) - c.Assert(changedCol.FieldType.Tp, Equals, mysql.TypeLonglong) - c.Assert(changedCol.FieldType.Flen, Equals, col.FieldType.Flen+1) + require.Equal(s.T(), changedCol.FieldType.Tp, mysql.TypeLonglong) + require.Equal(s.T(), changedCol.FieldType.Flen, col.FieldType.Flen+1) col.FieldType.Flen++ // Test add foreign key failed cause by canceled. updateTest(&tests[17]) addForeignKeyArgs := []interface{}{model.FKInfo{Name: model.NewCIStr("fk1")}} - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, test.act, addForeignKeyArgs, &test.cancelState) - c.Check(checkErr, IsNil) - changedTable = testGetTable(c, d, dbInfo.ID, tblInfo.ID) - c.Assert(len(changedTable.Meta().ForeignKeys), Equals, 0) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, test.act, addForeignKeyArgs, &test.cancelState) + require.NoError(s.T(), checkErr) + changedTable = testGetTable(s.T(), d, dbInfo.ID, tblInfo.ID) + require.Equal(s.T(), len(changedTable.Meta().ForeignKeys), 0) // Test add foreign key successful. updateTest(&tests[18]) - doDDLJobSuccess(ctx, d, c, dbInfo.ID, tblInfo.ID, test.act, addForeignKeyArgs) - c.Check(checkErr, IsNil) - changedTable = testGetTable(c, d, dbInfo.ID, tblInfo.ID) - c.Assert(len(changedTable.Meta().ForeignKeys), Equals, 1) - c.Assert(changedTable.Meta().ForeignKeys[0].Name, Equals, addForeignKeyArgs[0].(model.FKInfo).Name) + doDDLJobSuccess(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, test.act, addForeignKeyArgs) + require.NoError(s.T(), checkErr) + changedTable = testGetTable(s.T(), d, dbInfo.ID, tblInfo.ID) + require.Equal(s.T(), len(changedTable.Meta().ForeignKeys), 1) + require.Equal(s.T(), changedTable.Meta().ForeignKeys[0].Name, addForeignKeyArgs[0].(model.FKInfo).Name) // Test drop foreign key failed cause by canceled. updateTest(&tests[19]) dropForeignKeyArgs := []interface{}{addForeignKeyArgs[0].(model.FKInfo).Name} - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, test.act, dropForeignKeyArgs, &test.cancelState) - c.Check(checkErr, IsNil) - changedTable = testGetTable(c, d, dbInfo.ID, tblInfo.ID) - c.Assert(len(changedTable.Meta().ForeignKeys), Equals, 1) - c.Assert(changedTable.Meta().ForeignKeys[0].Name, Equals, dropForeignKeyArgs[0].(model.CIStr)) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, test.act, dropForeignKeyArgs, &test.cancelState) + require.NoError(s.T(), checkErr) + changedTable = testGetTable(s.T(), d, dbInfo.ID, tblInfo.ID) + require.Equal(s.T(), len(changedTable.Meta().ForeignKeys), 1) + require.Equal(s.T(), changedTable.Meta().ForeignKeys[0].Name, dropForeignKeyArgs[0].(model.CIStr)) // Test drop foreign key successful. updateTest(&tests[20]) - doDDLJobSuccess(ctx, d, c, dbInfo.ID, tblInfo.ID, test.act, dropForeignKeyArgs) - c.Check(checkErr, IsNil) - changedTable = testGetTable(c, d, dbInfo.ID, tblInfo.ID) - c.Assert(len(changedTable.Meta().ForeignKeys), Equals, 0) + doDDLJobSuccess(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, test.act, dropForeignKeyArgs) + require.NoError(s.T(), checkErr) + changedTable = testGetTable(s.T(), d, dbInfo.ID, tblInfo.ID) + require.Equal(s.T(), len(changedTable.Meta().ForeignKeys), 0) // test rename table failed caused by canceled. test = &tests[21] renameTableArgs := []interface{}{dbInfo.ID, model.NewCIStr("t2"), dbInfo.Name} - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, test.act, renameTableArgs, &test.cancelState) - c.Check(checkErr, IsNil) - changedTable = testGetTable(c, d, dbInfo.ID, tblInfo.ID) - c.Assert(changedTable.Meta().Name.L, Equals, "t") + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, test.act, renameTableArgs, &test.cancelState) + require.NoError(s.T(), checkErr) + changedTable = testGetTable(s.T(), d, dbInfo.ID, tblInfo.ID) + require.Equal(s.T(), changedTable.Meta().Name.L, "t") // test rename table successful. test = &tests[22] - doDDLJobSuccess(ctx, d, c, dbInfo.ID, tblInfo.ID, test.act, renameTableArgs) - c.Check(checkErr, IsNil) - changedTable = testGetTable(c, d, dbInfo.ID, tblInfo.ID) - c.Assert(changedTable.Meta().Name.L, Equals, "t2") + doDDLJobSuccess(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, test.act, renameTableArgs) + require.NoError(s.T(), checkErr) + changedTable = testGetTable(s.T(), d, dbInfo.ID, tblInfo.ID) + require.Equal(s.T(), changedTable.Meta().Name.L, "t2") // test modify table charset failed caused by canceled. test = &tests[23] modifyTableCharsetArgs := []interface{}{"utf8mb4", "utf8mb4_bin"} - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, test.act, modifyTableCharsetArgs, &test.cancelState) - c.Check(checkErr, IsNil) - changedTable = testGetTable(c, d, dbInfo.ID, tblInfo.ID) - c.Assert(changedTable.Meta().Charset, Equals, "utf8") - c.Assert(changedTable.Meta().Collate, Equals, "utf8_bin") + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, test.act, modifyTableCharsetArgs, &test.cancelState) + require.NoError(s.T(), checkErr) + changedTable = testGetTable(s.T(), d, dbInfo.ID, tblInfo.ID) + require.Equal(s.T(), changedTable.Meta().Charset, "utf8") + require.Equal(s.T(), changedTable.Meta().Collate, "utf8_bin") // test modify table charset successfully. test = &tests[24] - doDDLJobSuccess(ctx, d, c, dbInfo.ID, tblInfo.ID, test.act, modifyTableCharsetArgs) - c.Check(checkErr, IsNil) - changedTable = testGetTable(c, d, dbInfo.ID, tblInfo.ID) - c.Assert(changedTable.Meta().Charset, Equals, "utf8mb4") - c.Assert(changedTable.Meta().Collate, Equals, "utf8mb4_bin") + doDDLJobSuccess(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, test.act, modifyTableCharsetArgs) + require.NoError(s.T(), checkErr) + changedTable = testGetTable(s.T(), d, dbInfo.ID, tblInfo.ID) + require.Equal(s.T(), changedTable.Meta().Charset, "utf8mb4") + require.Equal(s.T(), changedTable.Meta().Collate, "utf8mb4_bin") // test truncate table partition failed caused by canceled. test = &tests[25] truncateTblPartitionArgs := []interface{}{[]int64{partitionTblInfo.Partition.Definitions[0].ID}} - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, partitionTblInfo.ID, test.act, truncateTblPartitionArgs, &test.cancelState) - c.Check(checkErr, IsNil) - changedTable = testGetTable(c, d, dbInfo.ID, partitionTblInfo.ID) - c.Assert(changedTable.Meta().Partition.Definitions[0].ID == partitionTblInfo.Partition.Definitions[0].ID, IsTrue) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, partitionTblInfo.ID, test.act, truncateTblPartitionArgs, &test.cancelState) + require.NoError(s.T(), checkErr) + changedTable = testGetTable(s.T(), d, dbInfo.ID, partitionTblInfo.ID) + require.True(s.T(), changedTable.Meta().Partition.Definitions[0].ID == partitionTblInfo.Partition.Definitions[0].ID) // test truncate table partition charset successfully. test = &tests[26] - doDDLJobSuccess(ctx, d, c, dbInfo.ID, partitionTblInfo.ID, test.act, truncateTblPartitionArgs) - c.Check(checkErr, IsNil) - changedTable = testGetTable(c, d, dbInfo.ID, partitionTblInfo.ID) - c.Assert(changedTable.Meta().Partition.Definitions[0].ID == partitionTblInfo.Partition.Definitions[0].ID, IsFalse) + doDDLJobSuccess(ctx, d, s.T(), dbInfo.ID, partitionTblInfo.ID, test.act, truncateTblPartitionArgs) + require.NoError(s.T(), checkErr) + changedTable = testGetTable(s.T(), d, dbInfo.ID, partitionTblInfo.ID) + require.False(s.T(), changedTable.Meta().Partition.Definitions[0].ID == partitionTblInfo.Partition.Definitions[0].ID) // test modify schema charset failed caused by canceled. test = &tests[27] charsetAndCollate := []interface{}{"utf8mb4", "utf8mb4_bin"} - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, test.act, charsetAndCollate, &test.cancelState) - c.Check(checkErr, IsNil) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, test.act, charsetAndCollate, &test.cancelState) + require.NoError(s.T(), checkErr) dbInfo, err = testGetSchemaInfoWithError(d, dbInfo.ID) - c.Assert(err, IsNil) - c.Assert(dbInfo.Charset, Equals, "") - c.Assert(dbInfo.Collate, Equals, "") + require.NoError(s.T(), err) + require.Equal(s.T(), dbInfo.Charset, "") + require.Equal(s.T(), dbInfo.Collate, "") // test modify table charset successfully. test = &tests[28] - doDDLJobSuccess(ctx, d, c, dbInfo.ID, tblInfo.ID, test.act, charsetAndCollate) - c.Check(checkErr, IsNil) + doDDLJobSuccess(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, test.act, charsetAndCollate) + require.NoError(s.T(), checkErr) dbInfo, err = testGetSchemaInfoWithError(d, dbInfo.ID) - c.Assert(err, IsNil) - c.Assert(dbInfo.Charset, Equals, "utf8mb4") - c.Assert(dbInfo.Collate, Equals, "utf8mb4_bin") + require.NoError(s.T(), err) + require.Equal(s.T(), dbInfo.Charset, "utf8mb4") + require.Equal(s.T(), dbInfo.Collate, "utf8mb4_bin") // for adding primary key tblInfo = changedTable.Meta() @@ -1168,34 +1184,34 @@ func (s *testDDLSerialSuite) TestCancelJob(c *C) { Length: -1, }}, nil} cancelState = model.StateNone - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, model.ActionAddPrimaryKey, validArgs, &cancelState) - c.Check(checkErr, IsNil) - s.checkAddIdx(c, d, dbInfo.ID, tblInfo.ID, idxOrigName, false) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, model.ActionAddPrimaryKey, validArgs, &cancelState) + require.NoError(s.T(), checkErr) + s.checkAddIdx(s.T(), d, dbInfo.ID, tblInfo.ID, idxOrigName, false) updateTest(&tests[30]) - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, model.ActionAddPrimaryKey, validArgs, &cancelState) - c.Check(checkErr, IsNil) - s.checkAddIdx(c, d, dbInfo.ID, tblInfo.ID, idxOrigName, false) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, model.ActionAddPrimaryKey, validArgs, &cancelState) + require.NoError(s.T(), checkErr) + s.checkAddIdx(s.T(), d, dbInfo.ID, tblInfo.ID, idxOrigName, false) updateTest(&tests[31]) - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, model.ActionAddPrimaryKey, validArgs, &cancelState) - c.Check(checkErr, IsNil) - s.checkAddIdx(c, d, dbInfo.ID, tblInfo.ID, idxOrigName, false) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, model.ActionAddPrimaryKey, validArgs, &cancelState) + require.NoError(s.T(), checkErr) + s.checkAddIdx(s.T(), d, dbInfo.ID, tblInfo.ID, idxOrigName, false) updateTest(&tests[32]) - testCreatePrimaryKey(c, ctx, d, dbInfo, tblInfo, "c1") - c.Check(checkErr, IsNil) + testCreatePrimaryKey(s.T(), ctx, d, dbInfo, tblInfo, "c1") + require.NoError(s.T(), checkErr) txn, err = ctx.Txn(true) - c.Assert(err, IsNil) - c.Assert(txn.Commit(context.Background()), IsNil) - s.checkAddIdx(c, d, dbInfo.ID, tblInfo.ID, idxOrigName, true) + require.NoError(s.T(), err) + require.Nil(s.T(), txn.Commit(context.Background())) + s.checkAddIdx(s.T(), d, dbInfo.ID, tblInfo.ID, idxOrigName, true) // for dropping primary key updateTest(&tests[33]) - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, model.ActionDropPrimaryKey, validArgs, &cancelState) - c.Check(checkErr, IsNil) - s.checkDropIdx(c, d, dbInfo.ID, tblInfo.ID, idxOrigName, false) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, model.ActionDropPrimaryKey, validArgs, &cancelState) + require.NoError(s.T(), checkErr) + s.checkDropIdx(s.T(), d, dbInfo.ID, tblInfo.ID, idxOrigName, false) updateTest(&tests[34]) - testDropIndex(c, ctx, d, dbInfo, tblInfo, idxOrigName) - c.Check(checkErr, IsNil) - s.checkDropIdx(c, d, dbInfo.ID, tblInfo.ID, idxOrigName, true) + testDropIndex(s.T(), ctx, d, dbInfo, tblInfo, idxOrigName) + require.NoError(s.T(), checkErr) + s.checkDropIdx(s.T(), d, dbInfo.ID, tblInfo.ID, idxOrigName, true) // for add columns updateTest(&tests[35]) @@ -1208,7 +1224,7 @@ func (s *testDDLSerialSuite) TestCancelJob(c *C) { Options: []*ast.ColumnOption{}, } col, _, err := buildColumnAndConstraint(ctx, 0, newColumnDef, nil, mysql.DefaultCharset, "") - c.Assert(err, IsNil) + require.NoError(s.T(), err) cols[i] = col } offsets := make([]int, len(cols)) @@ -1219,129 +1235,129 @@ func (s *testDDLSerialSuite) TestCancelJob(c *C) { ifNotExists := make([]bool, len(cols)) addColumnArgs = []interface{}{cols, positions, offsets, ifNotExists} - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, model.ActionAddColumns, addColumnArgs, &cancelState) - c.Check(checkErr, IsNil) - s.checkAddColumns(c, d, dbInfo.ID, tblInfo.ID, addingColNames, false) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, model.ActionAddColumns, addColumnArgs, &cancelState) + require.NoError(s.T(), checkErr) + s.checkAddColumns(d, dbInfo.ID, tblInfo.ID, addingColNames, false) updateTest(&tests[36]) - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, model.ActionAddColumns, addColumnArgs, &cancelState) - c.Check(checkErr, IsNil) - s.checkAddColumns(c, d, dbInfo.ID, tblInfo.ID, addingColNames, false) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, model.ActionAddColumns, addColumnArgs, &cancelState) + require.NoError(s.T(), checkErr) + s.checkAddColumns(d, dbInfo.ID, tblInfo.ID, addingColNames, false) updateTest(&tests[37]) - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, model.ActionAddColumns, addColumnArgs, &cancelState) - c.Check(checkErr, IsNil) - s.checkAddColumns(c, d, dbInfo.ID, tblInfo.ID, addingColNames, false) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, model.ActionAddColumns, addColumnArgs, &cancelState) + require.NoError(s.T(), checkErr) + s.checkAddColumns(d, dbInfo.ID, tblInfo.ID, addingColNames, false) updateTest(&tests[38]) - testAddColumns(c, ctx, d, dbInfo, tblInfo, addColumnArgs) - c.Check(checkErr, IsNil) - s.checkAddColumns(c, d, dbInfo.ID, tblInfo.ID, addingColNames, true) + testAddColumns(s.T(), ctx, d, dbInfo, tblInfo, addColumnArgs) + require.NoError(s.T(), checkErr) + s.checkAddColumns(d, dbInfo.ID, tblInfo.ID, addingColNames, true) // for drop columns updateTest(&tests[39]) dropColNames := []string{"colA", "colB"} - s.checkCancelDropColumns(c, d, dbInfo.ID, tblInfo.ID, dropColNames, false) - testDropColumns(c, ctx, d, dbInfo, tblInfo, dropColNames, false) - c.Check(checkErr, IsNil) - s.checkCancelDropColumns(c, d, dbInfo.ID, tblInfo.ID, dropColNames, true) + s.checkCancelDropColumns(d, dbInfo.ID, tblInfo.ID, dropColNames, false) + testDropColumns(s.T(), ctx, d, dbInfo, tblInfo, dropColNames, false) + require.NoError(s.T(), checkErr) + s.checkCancelDropColumns(d, dbInfo.ID, tblInfo.ID, dropColNames, true) updateTest(&tests[40]) dropColNames = []string{"colC", "colD"} - s.checkCancelDropColumns(c, d, dbInfo.ID, tblInfo.ID, dropColNames, false) - testDropColumns(c, ctx, d, dbInfo, tblInfo, dropColNames, false) - c.Check(checkErr, IsNil) - s.checkCancelDropColumns(c, d, dbInfo.ID, tblInfo.ID, dropColNames, true) + s.checkCancelDropColumns(d, dbInfo.ID, tblInfo.ID, dropColNames, false) + testDropColumns(s.T(), ctx, d, dbInfo, tblInfo, dropColNames, false) + require.NoError(s.T(), checkErr) + s.checkCancelDropColumns(d, dbInfo.ID, tblInfo.ID, dropColNames, true) updateTest(&tests[41]) dropColNames = []string{"colE", "colF"} - s.checkCancelDropColumns(c, d, dbInfo.ID, tblInfo.ID, dropColNames, false) - testDropColumns(c, ctx, d, dbInfo, tblInfo, dropColNames, false) - c.Check(checkErr, IsNil) - s.checkCancelDropColumns(c, d, dbInfo.ID, tblInfo.ID, dropColNames, true) + s.checkCancelDropColumns(d, dbInfo.ID, tblInfo.ID, dropColNames, false) + testDropColumns(s.T(), ctx, d, dbInfo, tblInfo, dropColNames, false) + require.NoError(s.T(), checkErr) + s.checkCancelDropColumns(d, dbInfo.ID, tblInfo.ID, dropColNames, true) // test alter index visibility failed caused by canceled. indexName := "idx_c3" - testCreateIndex(c, ctx, d, dbInfo, tblInfo, false, indexName, "c3") - c.Check(checkErr, IsNil) + testCreateIndex(s.T(), ctx, d, dbInfo, tblInfo, false, indexName, "c3") + require.NoError(s.T(), checkErr) txn, err = ctx.Txn(true) - c.Assert(err, IsNil) - c.Assert(txn.Commit(context.Background()), IsNil) - s.checkAddIdx(c, d, dbInfo.ID, tblInfo.ID, indexName, true) + require.NoError(s.T(), err) + require.Nil(s.T(), txn.Commit(context.Background())) + s.checkAddIdx(s.T(), d, dbInfo.ID, tblInfo.ID, indexName, true) updateTest(&tests[42]) alterIndexVisibility := []interface{}{model.NewCIStr(indexName), true} - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, test.act, alterIndexVisibility, &test.cancelState) - c.Check(checkErr, IsNil) - changedTable = testGetTable(c, d, dbInfo.ID, tblInfo.ID) - c.Assert(checkIdxVisibility(changedTable, indexName, false), IsTrue) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, test.act, alterIndexVisibility, &test.cancelState) + require.NoError(s.T(), checkErr) + changedTable = testGetTable(s.T(), d, dbInfo.ID, tblInfo.ID) + require.True(s.T(), checkIdxVisibility(changedTable, indexName, false)) // cancel alter index visibility successfully updateTest(&tests[43]) alterIndexVisibility = []interface{}{model.NewCIStr(indexName), true} - doDDLJobSuccess(ctx, d, c, dbInfo.ID, tblInfo.ID, test.act, alterIndexVisibility) - c.Check(checkErr, IsNil) - changedTable = testGetTable(c, d, dbInfo.ID, tblInfo.ID) - c.Assert(checkIdxVisibility(changedTable, indexName, true), IsTrue) + doDDLJobSuccess(ctx, d, s.T(), dbInfo.ID, tblInfo.ID, test.act, alterIndexVisibility) + require.NoError(s.T(), checkErr) + changedTable = testGetTable(s.T(), d, dbInfo.ID, tblInfo.ID) + require.True(s.T(), checkIdxVisibility(changedTable, indexName, true)) // test exchange partition failed caused by canceled - pt := testTableInfoWithPartition(c, d, "pt", 5) + pt := testTableInfoWithPartition(s.T(), d, "pt", 5) nt, err := testTableInfo(d, "nt", 5) - c.Assert(err, IsNil) - testCreateTable(c, ctx, d, dbInfo, pt) - testCreateTable(c, ctx, d, dbInfo, nt) + require.NoError(s.T(), err) + testCreateTable(s.T(), ctx, d, dbInfo, pt) + testCreateTable(s.T(), ctx, d, dbInfo, nt) updateTest(&tests[44]) defID := pt.Partition.Definitions[0].ID exchangeTablePartition := []interface{}{defID, dbInfo.ID, pt.ID, "p0", true} - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, nt.ID, test.act, exchangeTablePartition, &test.cancelState) - c.Check(checkErr, IsNil) - changedNtTable := testGetTable(c, d, dbInfo.ID, nt.ID) - changedPtTable := testGetTable(c, d, dbInfo.ID, pt.ID) - c.Assert(changedNtTable.Meta().ID == nt.ID, IsTrue) - c.Assert(changedPtTable.Meta().Partition.Definitions[0].ID == pt.Partition.Definitions[0].ID, IsTrue) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, nt.ID, test.act, exchangeTablePartition, &test.cancelState) + require.NoError(s.T(), checkErr) + changedNtTable := testGetTable(s.T(), d, dbInfo.ID, nt.ID) + changedPtTable := testGetTable(s.T(), d, dbInfo.ID, pt.ID) + require.True(s.T(), changedNtTable.Meta().ID == nt.ID) + require.True(s.T(), changedPtTable.Meta().Partition.Definitions[0].ID == pt.Partition.Definitions[0].ID) // cancel exchange partition successfully updateTest(&tests[45]) - doDDLJobSuccess(ctx, d, c, dbInfo.ID, nt.ID, test.act, exchangeTablePartition) - c.Check(checkErr, IsNil) - changedNtTable = testGetTable(c, d, dbInfo.ID, pt.Partition.Definitions[0].ID) - changedPtTable = testGetTable(c, d, dbInfo.ID, pt.ID) - c.Assert(changedNtTable.Meta().ID == nt.ID, IsFalse) - c.Assert(changedPtTable.Meta().Partition.Definitions[0].ID == nt.ID, IsTrue) + doDDLJobSuccess(ctx, d, s.T(), dbInfo.ID, nt.ID, test.act, exchangeTablePartition) + require.NoError(s.T(), checkErr) + changedNtTable = testGetTable(s.T(), d, dbInfo.ID, pt.Partition.Definitions[0].ID) + changedPtTable = testGetTable(s.T(), d, dbInfo.ID, pt.ID) + require.False(s.T(), changedNtTable.Meta().ID == nt.ID) + require.True(s.T(), changedPtTable.Meta().Partition.Definitions[0].ID == nt.ID) // Cancel add table partition. - baseTableInfo := testTableInfoWithPartitionLessThan(c, d, "empty_table", 5, "1000") - testCreateTable(c, ctx, d, dbInfo, baseTableInfo) + baseTableInfo := testTableInfoWithPartitionLessThan(s.T(), d, "empty_table", 5, "1000") + testCreateTable(s.T(), ctx, d, dbInfo, baseTableInfo) cancelState = model.StateNone updateTest(&tests[46]) - addedPartInfo := testAddedNewTablePartitionInfo(c, d, baseTableInfo, "p1", "maxvalue") + addedPartInfo := testAddedNewTablePartitionInfo(s.T(), d, baseTableInfo, "p1", "maxvalue") addPartitionArgs := []interface{}{addedPartInfo} - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, baseTableInfo.ID, test.act, addPartitionArgs, &cancelState) - c.Check(checkErr, IsNil) - baseTable := testGetTable(c, d, dbInfo.ID, baseTableInfo.ID) - c.Assert(len(baseTable.Meta().Partition.Definitions), Equals, 1) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, baseTableInfo.ID, test.act, addPartitionArgs, &cancelState) + require.NoError(s.T(), checkErr) + baseTable := testGetTable(s.T(), d, dbInfo.ID, baseTableInfo.ID) + require.Equal(s.T(), len(baseTable.Meta().Partition.Definitions), 1) updateTest(&tests[47]) - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, baseTableInfo.ID, test.act, addPartitionArgs, &cancelState) - c.Check(checkErr, IsNil) - baseTable = testGetTable(c, d, dbInfo.ID, baseTableInfo.ID) - c.Assert(len(baseTable.Meta().Partition.Definitions), Equals, 1) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, baseTableInfo.ID, test.act, addPartitionArgs, &cancelState) + require.NoError(s.T(), checkErr) + baseTable = testGetTable(s.T(), d, dbInfo.ID, baseTableInfo.ID) + require.Equal(s.T(), len(baseTable.Meta().Partition.Definitions), 1) updateTest(&tests[48]) - doDDLJobSuccess(ctx, d, c, dbInfo.ID, baseTableInfo.ID, test.act, addPartitionArgs) - c.Check(checkErr, IsNil) - baseTable = testGetTable(c, d, dbInfo.ID, baseTableInfo.ID) - c.Assert(len(baseTable.Meta().Partition.Definitions), Equals, 2) - c.Assert(baseTable.Meta().Partition.Definitions[1].ID, Equals, addedPartInfo.Definitions[0].ID) - c.Assert(baseTable.Meta().Partition.Definitions[1].LessThan[0], Equals, addedPartInfo.Definitions[0].LessThan[0]) + doDDLJobSuccess(ctx, d, s.T(), dbInfo.ID, baseTableInfo.ID, test.act, addPartitionArgs) + require.NoError(s.T(), checkErr) + baseTable = testGetTable(s.T(), d, dbInfo.ID, baseTableInfo.ID) + require.Equal(s.T(), len(baseTable.Meta().Partition.Definitions), 2) + require.Equal(s.T(), baseTable.Meta().Partition.Definitions[1].ID, addedPartInfo.Definitions[0].ID) + require.Equal(s.T(), baseTable.Meta().Partition.Definitions[1].LessThan[0], addedPartInfo.Definitions[0].LessThan[0]) // Cancel modify column which should reorg the data. - c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/skipMockContextDoExec", `return(true)`), IsNil) - baseTableInfo = testTableInfoWith2IndexOnFirstColumn(c, d, "modify-table", 2) + require.Nil(s.T(), failpoint.Enable("github.com/pingcap/tidb/ddl/skipMockContextDoExec", `return(true)`)) + baseTableInfo = testTableInfoWith2IndexOnFirstColumn(s.T(), d, "modify-table", 2) // This will cost 2 global id, one for table id, the other for the job id. - testCreateTable(c, ctx, d, dbInfo, baseTableInfo) + testCreateTable(s.T(), ctx, d, dbInfo, baseTableInfo) cancelState = model.StateNone newCol := baseTableInfo.Columns[0].Clone() @@ -1356,64 +1372,64 @@ func (s *testDDLSerialSuite) TestCancelJob(c *C) { updateTest(&tests[49]) modifyColumnArgs = []interface{}{&newCol, originColName, pos, mysql.TypeNull, 0} - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, baseTableInfo.ID, test.act, modifyColumnArgs, &cancelState) - c.Check(checkErr, IsNil) - baseTable = testGetTable(c, d, dbInfo.ID, baseTableInfo.ID) - c.Assert(baseTable.Meta().Columns[0].FieldType.Tp, Equals, mysql.TypeLong) - c.Assert(mysql.HasNotNullFlag(baseTable.Meta().Columns[0].FieldType.Flag), Equals, false) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, baseTableInfo.ID, test.act, modifyColumnArgs, &cancelState) + require.NoError(s.T(), checkErr) + baseTable = testGetTable(s.T(), d, dbInfo.ID, baseTableInfo.ID) + require.Equal(s.T(), baseTable.Meta().Columns[0].FieldType.Tp, mysql.TypeLong) + require.Equal(s.T(), mysql.HasNotNullFlag(baseTable.Meta().Columns[0].FieldType.Flag), false) updateTest(&tests[50]) - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, baseTableInfo.ID, test.act, modifyColumnArgs, &cancelState) - c.Check(checkErr, IsNil) - baseTable = testGetTable(c, d, dbInfo.ID, baseTableInfo.ID) - c.Assert(baseTable.Meta().Columns[0].FieldType.Tp, Equals, mysql.TypeLong) - c.Assert(baseTable.Meta().Columns[0].FieldType.Flag&mysql.NotNullFlag, Equals, uint(0)) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, baseTableInfo.ID, test.act, modifyColumnArgs, &cancelState) + require.NoError(s.T(), checkErr) + baseTable = testGetTable(s.T(), d, dbInfo.ID, baseTableInfo.ID) + require.Equal(s.T(), baseTable.Meta().Columns[0].FieldType.Tp, mysql.TypeLong) + require.Equal(s.T(), baseTable.Meta().Columns[0].FieldType.Flag&mysql.NotNullFlag, uint(0)) updateTest(&tests[51]) - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, baseTableInfo.ID, test.act, modifyColumnArgs, &cancelState) - c.Check(checkErr, IsNil) - baseTable = testGetTable(c, d, dbInfo.ID, baseTableInfo.ID) - c.Assert(baseTable.Meta().Columns[0].FieldType.Tp, Equals, mysql.TypeLong) - c.Assert(baseTable.Meta().Columns[0].FieldType.Flag&mysql.NotNullFlag, Equals, uint(0)) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, baseTableInfo.ID, test.act, modifyColumnArgs, &cancelState) + require.NoError(s.T(), checkErr) + baseTable = testGetTable(s.T(), d, dbInfo.ID, baseTableInfo.ID) + require.Equal(s.T(), baseTable.Meta().Columns[0].FieldType.Tp, mysql.TypeLong) + require.Equal(s.T(), baseTable.Meta().Columns[0].FieldType.Flag&mysql.NotNullFlag, uint(0)) updateTest(&tests[52]) - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, baseTableInfo.ID, test.act, modifyColumnArgs, &cancelState) - c.Check(checkErr, IsNil) - baseTable = testGetTable(c, d, dbInfo.ID, baseTableInfo.ID) - c.Assert(baseTable.Meta().Columns[0].FieldType.Tp, Equals, mysql.TypeLong) - c.Assert(baseTable.Meta().Columns[0].FieldType.Flag&mysql.NotNullFlag, Equals, uint(0)) + doDDLJobErrWithSchemaState(ctx, d, s.T(), dbInfo.ID, baseTableInfo.ID, test.act, modifyColumnArgs, &cancelState) + require.NoError(s.T(), checkErr) + baseTable = testGetTable(s.T(), d, dbInfo.ID, baseTableInfo.ID) + require.Equal(s.T(), baseTable.Meta().Columns[0].FieldType.Tp, mysql.TypeLong) + require.Equal(s.T(), baseTable.Meta().Columns[0].FieldType.Flag&mysql.NotNullFlag, uint(0)) updateTest(&tests[53]) - doDDLJobSuccess(ctx, d, c, dbInfo.ID, baseTableInfo.ID, test.act, modifyColumnArgs) - c.Check(checkErr, IsNil) - baseTable = testGetTable(c, d, dbInfo.ID, baseTableInfo.ID) - c.Assert(baseTable.Meta().Columns[0].FieldType.Tp, Equals, mysql.TypeTiny) - c.Assert(baseTable.Meta().Columns[0].FieldType.Flag&mysql.NotNullFlag, Equals, uint(1)) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/skipMockContextDoExec"), IsNil) + doDDLJobSuccess(ctx, d, s.T(), dbInfo.ID, baseTableInfo.ID, test.act, modifyColumnArgs) + require.NoError(s.T(), checkErr) + baseTable = testGetTable(s.T(), d, dbInfo.ID, baseTableInfo.ID) + require.Equal(s.T(), baseTable.Meta().Columns[0].FieldType.Tp, mysql.TypeTiny) + require.Equal(s.T(), baseTable.Meta().Columns[0].FieldType.Flag&mysql.NotNullFlag, uint(1)) + require.Nil(s.T(), failpoint.Disable("github.com/pingcap/tidb/ddl/skipMockContextDoExec")) // for drop indexes updateTest(&tests[54]) ifExists := make([]bool, 2) idxNames := []model.CIStr{model.NewCIStr("i1"), model.NewCIStr("i2")} dropIndexesArgs := []interface{}{idxNames, ifExists} - tableInfo := createTestTableForDropIndexes(c, ctx, d, dbInfo, "test-drop-indexes", 6) - doDDLJobSuccess(ctx, d, c, dbInfo.ID, tableInfo.ID, test.act, dropIndexesArgs) - s.checkDropIndexes(c, d, dbInfo.ID, tableInfo.ID, idxNames, true) + tableInfo := createTestTableForDropIndexes(s.T(), ctx, d, dbInfo, "test-drop-indexes", 6) + doDDLJobSuccess(ctx, d, s.T(), dbInfo.ID, tableInfo.ID, test.act, dropIndexesArgs) + s.checkDropIndexes(d, dbInfo.ID, tableInfo.ID, idxNames, true) updateTest(&tests[55]) idxNames = []model.CIStr{model.NewCIStr("i3"), model.NewCIStr("i4")} dropIndexesArgs = []interface{}{idxNames, ifExists} - doDDLJobSuccess(ctx, d, c, dbInfo.ID, tableInfo.ID, test.act, dropIndexesArgs) - s.checkDropIndexes(c, d, dbInfo.ID, tableInfo.ID, idxNames, true) + doDDLJobSuccess(ctx, d, s.T(), dbInfo.ID, tableInfo.ID, test.act, dropIndexesArgs) + s.checkDropIndexes(d, dbInfo.ID, tableInfo.ID, idxNames, true) updateTest(&tests[56]) idxNames = []model.CIStr{model.NewCIStr("i5"), model.NewCIStr("i6")} dropIndexesArgs = []interface{}{idxNames, ifExists} - doDDLJobSuccess(ctx, d, c, dbInfo.ID, tableInfo.ID, test.act, dropIndexesArgs) - s.checkDropIndexes(c, d, dbInfo.ID, tableInfo.ID, idxNames, true) + doDDLJobSuccess(ctx, d, s.T(), dbInfo.ID, tableInfo.ID, test.act, dropIndexesArgs) + s.checkDropIndexes(d, dbInfo.ID, tableInfo.ID, idxNames, true) } -func (s *testDDLSuite) TestIgnorableSpec(c *C) { +func (s *testDDLSuiteToVerify) TestIgnorableSpec() { specs := []ast.AlterTableType{ ast.AlterTableOption, ast.AlterTableAddColumns, @@ -1428,7 +1444,7 @@ func (s *testDDLSuite) TestIgnorableSpec(c *C) { ast.AlterTableAlterColumn, } for _, spec := range specs { - c.Assert(isIgnorableSpec(spec), IsFalse) + require.False(s.T(), isIgnorableSpec(spec)) } ignorableSpecs := []ast.AlterTableType{ @@ -1436,15 +1452,15 @@ func (s *testDDLSuite) TestIgnorableSpec(c *C) { ast.AlterTableAlgorithm, } for _, spec := range ignorableSpecs { - c.Assert(isIgnorableSpec(spec), IsTrue) + require.True(s.T(), isIgnorableSpec(spec)) } } -func (s *testDDLSuite) TestBuildJobDependence(c *C) { - store := testCreateStore(c, "test_set_job_relation") +func (s *testDDLSuiteToVerify) TestBuildJobDependence() { + store := testCreateStore(s.T(), "test_set_job_relation") defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() // Add some non-add-index jobs. job1 := &model.Job{ID: 1, TableID: 1, Type: model.ActionAddColumn} @@ -1457,95 +1473,95 @@ func (s *testDDLSuite) TestBuildJobDependence(c *C) { err := kv.RunInNewTxn(context.Background(), store, false, func(ctx context.Context, txn kv.Transaction) error { t := meta.NewMeta(txn) err := t.EnQueueDDLJob(job1) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = t.EnQueueDDLJob(job2) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = t.EnQueueDDLJob(job3) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = t.EnQueueDDLJob(job6) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = t.EnQueueDDLJob(job7) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = t.EnQueueDDLJob(job9) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = t.EnQueueDDLJob(job11) - c.Assert(err, IsNil) + require.NoError(s.T(), err) return nil }) - c.Assert(err, IsNil) + require.NoError(s.T(), err) job4 := &model.Job{ID: 4, TableID: 1, Type: model.ActionAddIndex} err = kv.RunInNewTxn(context.Background(), store, false, func(ctx context.Context, txn kv.Transaction) error { t := meta.NewMeta(txn) err := buildJobDependence(t, job4) - c.Assert(err, IsNil) - c.Assert(job4.DependencyID, Equals, int64(2)) + require.NoError(s.T(), err) + require.Equal(s.T(), job4.DependencyID, int64(2)) return nil }) - c.Assert(err, IsNil) + require.NoError(s.T(), err) job5 := &model.Job{ID: 5, TableID: 2, Type: model.ActionAddIndex} err = kv.RunInNewTxn(context.Background(), store, false, func(ctx context.Context, txn kv.Transaction) error { t := meta.NewMeta(txn) err := buildJobDependence(t, job5) - c.Assert(err, IsNil) - c.Assert(job5.DependencyID, Equals, int64(3)) + require.NoError(s.T(), err) + require.Equal(s.T(), job5.DependencyID, int64(3)) return nil }) - c.Assert(err, IsNil) + require.NoError(s.T(), err) job8 := &model.Job{ID: 8, TableID: 3, Type: model.ActionAddIndex} err = kv.RunInNewTxn(context.Background(), store, false, func(ctx context.Context, txn kv.Transaction) error { t := meta.NewMeta(txn) err := buildJobDependence(t, job8) - c.Assert(err, IsNil) - c.Assert(job8.DependencyID, Equals, int64(0)) + require.NoError(s.T(), err) + require.Equal(s.T(), job8.DependencyID, int64(0)) return nil }) - c.Assert(err, IsNil) + require.NoError(s.T(), err) job10 := &model.Job{ID: 10, SchemaID: 111, TableID: 3, Type: model.ActionAddIndex} err = kv.RunInNewTxn(context.Background(), store, false, func(ctx context.Context, txn kv.Transaction) error { t := meta.NewMeta(txn) err := buildJobDependence(t, job10) - c.Assert(err, IsNil) - c.Assert(job10.DependencyID, Equals, int64(9)) + require.NoError(s.T(), err) + require.Equal(s.T(), job10.DependencyID, int64(9)) return nil }) - c.Assert(err, IsNil) + require.NoError(s.T(), err) job12 := &model.Job{ID: 12, SchemaID: 112, TableID: 2, Type: model.ActionAddIndex} err = kv.RunInNewTxn(context.Background(), store, false, func(ctx context.Context, txn kv.Transaction) error { t := meta.NewMeta(txn) err := buildJobDependence(t, job12) - c.Assert(err, IsNil) - c.Assert(job12.DependencyID, Equals, int64(11)) + require.NoError(s.T(), err) + require.Equal(s.T(), job12.DependencyID, int64(11)) return nil }) - c.Assert(err, IsNil) + require.NoError(s.T(), err) } -func addDDLJob(c *C, d *ddl, job *model.Job) { +func addDDLJob(t *testing.T, d *ddl, job *model.Job) { task := &limitJobTask{job, make(chan error)} d.limitJobCh <- task err := <-task.err - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *testDDLSuite) TestParallelDDL(c *C) { - store := testCreateStore(c, "test_parallel_ddl") +func (s *testDDLSuiteToVerify) TestParallelDDL() { + store := testCreateStore(s.T(), "test_parallel_ddl") defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() d, err := testNewDDLAndStart( context.Background(), WithStore(store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() ctx := testNewContext(d) err = ctx.NewTxn(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) /* build structure: DBs -> { @@ -1565,44 +1581,44 @@ func (s *testDDLSuite) TestParallelDDL(c *C) { */ // create database test_parallel_ddl_1; dbInfo1, err := testSchemaInfo(d, "test_parallel_ddl_1") - c.Assert(err, IsNil) - testCreateSchema(c, ctx, d, dbInfo1) + require.NoError(s.T(), err) + testCreateSchema(s.T(), ctx, d, dbInfo1) // create table t1 (c1 int, c2 int); tblInfo1, err := testTableInfo(d, "t1", 2) - c.Assert(err, IsNil) - testCreateTable(c, ctx, d, dbInfo1, tblInfo1) + require.NoError(s.T(), err) + testCreateTable(s.T(), ctx, d, dbInfo1, tblInfo1) // insert t1 values (10, 10), (20, 20) - tbl1 := testGetTable(c, d, dbInfo1.ID, tblInfo1.ID) + tbl1 := testGetTable(s.T(), d, dbInfo1.ID, tblInfo1.ID) _, err = tbl1.AddRecord(ctx, types.MakeDatums(1, 1)) - c.Assert(err, IsNil) + require.NoError(s.T(), err) _, err = tbl1.AddRecord(ctx, types.MakeDatums(2, 2)) - c.Assert(err, IsNil) + require.NoError(s.T(), err) // create table t2 (c1 int primary key, c2 int, c3 int); tblInfo2, err := testTableInfo(d, "t2", 3) - c.Assert(err, IsNil) + require.NoError(s.T(), err) tblInfo2.Columns[0].Flag = mysql.PriKeyFlag | mysql.NotNullFlag tblInfo2.PKIsHandle = true - testCreateTable(c, ctx, d, dbInfo1, tblInfo2) + testCreateTable(s.T(), ctx, d, dbInfo1, tblInfo2) // insert t2 values (1, 1), (2, 2), (3, 3) - tbl2 := testGetTable(c, d, dbInfo1.ID, tblInfo2.ID) + tbl2 := testGetTable(s.T(), d, dbInfo1.ID, tblInfo2.ID) _, err = tbl2.AddRecord(ctx, types.MakeDatums(1, 1, 1)) - c.Assert(err, IsNil) + require.NoError(s.T(), err) _, err = tbl2.AddRecord(ctx, types.MakeDatums(2, 2, 2)) - c.Assert(err, IsNil) + require.NoError(s.T(), err) _, err = tbl2.AddRecord(ctx, types.MakeDatums(3, 3, 3)) - c.Assert(err, IsNil) + require.NoError(s.T(), err) // create database test_parallel_ddl_2; dbInfo2, err := testSchemaInfo(d, "test_parallel_ddl_2") - c.Assert(err, IsNil) - testCreateSchema(c, ctx, d, dbInfo2) + require.NoError(s.T(), err) + testCreateSchema(s.T(), ctx, d, dbInfo2) // create table t3 (c1 int, c2 int, c3 int, c4 int); tblInfo3, err := testTableInfo(d, "t3", 4) - c.Assert(err, IsNil) - testCreateTable(c, ctx, d, dbInfo2, tblInfo3) + require.NoError(s.T(), err) + testCreateTable(s.T(), ctx, d, dbInfo2, tblInfo3) // insert t3 values (11, 22, 33, 44) - tbl3 := testGetTable(c, d, dbInfo2.ID, tblInfo3.ID) + tbl3 := testGetTable(s.T(), d, dbInfo2.ID, tblInfo3.ID) _, err = tbl3.AddRecord(ctx, types.MakeDatums(11, 22, 33, 44)) - c.Assert(err, IsNil) + require.NoError(s.T(), err) // set hook to execute jobs after all jobs are in queue. jobCnt := int64(11) @@ -1632,7 +1648,7 @@ func (s *testDDLSuite) TestParallelDDL(c *C) { } if qLen1+qLen2 == jobCnt { if qLen2 != 5 { - checkErr = errors.Errorf("add index jobs cnt %v != 5", qLen2) + checkErr = errors.Errorf("add index jobs cnt %v != 6", qLen2) } break } @@ -1658,27 +1674,27 @@ func (s *testDDLSuite) TestParallelDDL(c *C) { / 11 / 2 / 2 / add index / */ job1 := buildCreateIdxJob(dbInfo1, tblInfo1, false, "db1_idx1", "c1") - addDDLJob(c, d, job1) + addDDLJob(s.T(), d, job1) job2 := buildCreateColumnJob(dbInfo1, tblInfo1, "c3", &ast.ColumnPosition{Tp: ast.ColumnPositionNone}, nil) - addDDLJob(c, d, job2) + addDDLJob(s.T(), d, job2) job3 := buildCreateIdxJob(dbInfo1, tblInfo1, false, "db1_idx2", "c3") - addDDLJob(c, d, job3) + addDDLJob(s.T(), d, job3) job4 := buildDropColumnJob(dbInfo1, tblInfo2, "c3") - addDDLJob(c, d, job4) + addDDLJob(s.T(), d, job4) job5 := buildDropIdxJob(dbInfo1, tblInfo1, "db1_idx1") - addDDLJob(c, d, job5) + addDDLJob(s.T(), d, job5) job6 := buildCreateIdxJob(dbInfo1, tblInfo2, false, "db2_idx1", "c2") - addDDLJob(c, d, job6) + addDDLJob(s.T(), d, job6) job7 := buildDropColumnJob(dbInfo2, tblInfo3, "c4") - addDDLJob(c, d, job7) + addDDLJob(s.T(), d, job7) job8 := buildRebaseAutoIDJobJob(dbInfo2, tblInfo3, 1024) - addDDLJob(c, d, job8) + addDDLJob(s.T(), d, job8) job9 := buildCreateIdxJob(dbInfo1, tblInfo1, false, "db1_idx3", "c2") - addDDLJob(c, d, job9) + addDDLJob(s.T(), d, job9) job10 := buildDropSchemaJob(dbInfo2) - addDDLJob(c, d, job10) + addDDLJob(s.T(), d, job10) job11 := buildCreateIdxJob(dbInfo2, tblInfo3, false, "db3_idx1", "c2") - addDDLJob(c, d, job11) + addDDLJob(s.T(), d, job11) // TODO: add rename table job // check results. @@ -1687,36 +1703,36 @@ func (s *testDDLSuite) TestParallelDDL(c *C) { err := kv.RunInNewTxn(context.Background(), store, false, func(ctx context.Context, txn kv.Transaction) error { m := meta.NewMeta(txn) lastJob, err := m.GetHistoryDDLJob(job11.ID) - c.Assert(err, IsNil) + require.NoError(s.T(), err) // all jobs are finished. if lastJob != nil { finishedJobs, err := m.GetAllHistoryDDLJobs() - c.Assert(err, IsNil) + require.NoError(s.T(), err) // get the last 12 jobs completed. finishedJobs = finishedJobs[len(finishedJobs)-11:] // check some jobs are ordered because of the dependence. - c.Assert(finishedJobs[0].ID, Equals, job1.ID, Commentf("%v", finishedJobs)) - c.Assert(finishedJobs[1].ID, Equals, job2.ID, Commentf("%v", finishedJobs)) - c.Assert(finishedJobs[2].ID, Equals, job3.ID, Commentf("%v", finishedJobs)) - c.Assert(finishedJobs[4].ID, Equals, job5.ID, Commentf("%v", finishedJobs)) - c.Assert(finishedJobs[10].ID, Equals, job11.ID, Commentf("%v", finishedJobs)) + require.Equal(s.T(), finishedJobs[0].ID, job1.ID, "%v", finishedJobs) + require.Equal(s.T(), finishedJobs[1].ID, job2.ID, "%v", finishedJobs) + require.Equal(s.T(), finishedJobs[2].ID, job3.ID, "%v", finishedJobs) + require.Equal(s.T(), finishedJobs[4].ID, job5.ID, "%v", finishedJobs) + require.Equal(s.T(), finishedJobs[10].ID, job11.ID, "%v", finishedJobs) // check the jobs are ordered in the backfill-job queue or general-job queue. backfillJobID := int64(0) generalJobID := int64(0) for _, job := range finishedJobs { // check jobs' order. if mayNeedReorg(job) { - c.Assert(job.ID, Greater, backfillJobID) + require.Greater(s.T(), job.ID, backfillJobID) backfillJobID = job.ID } else { - c.Assert(job.ID, Greater, generalJobID) + require.Greater(s.T(), job.ID, generalJobID) generalJobID = job.ID } // check jobs' state. if job.ID == lastJob.ID { - c.Assert(job.State, Equals, model.JobStateCancelled, Commentf("job: %v", job)) + require.Equal(s.T(), job.State, model.JobStateCancelled, "job: %v", job) } else { - c.Assert(job.State, Equals, model.JobStateSynced, Commentf("job: %v", job)) + require.Equal(s.T(), job.State, model.JobStateSynced, "job: %v", job) } } @@ -1724,20 +1740,20 @@ func (s *testDDLSuite) TestParallelDDL(c *C) { } return nil }) - c.Assert(err, IsNil) + require.NoError(s.T(), err) time.Sleep(10 * time.Millisecond) } - c.Assert(checkErr, IsNil) + require.NoError(s.T(), checkErr) tc = &TestDDLCallback{} d.SetHook(tc) } -func (s *testDDLSuite) TestDDLPackageExecuteSQL(c *C) { - store := testCreateStore(c, "test_run_sql") +func (s *testDDLSuiteToVerify) TestDDLPackageExecuteSQL() { + store := testCreateStore(s.T(), "test_run_sql") defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() d, err := testNewDDLAndStart( @@ -1745,26 +1761,26 @@ func (s *testDDLSuite) TestDDLPackageExecuteSQL(c *C) { WithStore(store), WithLease(testLease), ) - c.Assert(err, IsNil) - testCheckOwner(c, d, true) + require.NoError(s.T(), err) + testCheckOwner(s.T(), d, true) defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() worker := d.generalWorker() - c.Assert(worker, NotNil) + require.NotNil(s.T(), worker) // In test environment, worker.ctxPool will be nil, and get will return mock.Context. // We just test that can use it to call sqlexec.SQLExecutor.Execute. sess, err := worker.sessPool.get() - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer worker.sessPool.put(sess) se := sess.(sqlexec.SQLExecutor) _, _ = se.Execute(context.Background(), "create table t(a int);") } -func (s *testDDLSerialSuite) checkDropIndexes(c *C, d *ddl, schemaID int64, tableID int64, idxNames []model.CIStr, success bool) { +func (s *testDDLSerialSuiteToVerify) checkDropIndexes(d *ddl, schemaID int64, tableID int64, idxNames []model.CIStr, success bool) { for _, idxName := range idxNames { - checkIdxExist(c, d, schemaID, tableID, idxName.O, !success) + checkIdxExist(s.T(), d, schemaID, tableID, idxName.O, !success) } } diff --git a/ddl/fail_test.go b/ddl/fail_test.go index 387691c309..80673afa11 100644 --- a/ddl/fail_test.go +++ b/ddl/fail_test.go @@ -17,40 +17,40 @@ package ddl import ( "context" - . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/types" + "github.com/stretchr/testify/require" ) -func (s *testColumnChangeSuite) TestFailBeforeDecodeArgs(c *C) { +func (s *testColumnChangeSuiteToVerify) TestFailBeforeDecodeArgs() { d, err := testNewDDLAndStart( context.Background(), WithStore(s.store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() // create table t_fail (c1 int, c2 int); tblInfo, err := testTableInfo(d, "t_fail", 2) - c.Assert(err, IsNil) + require.NoError(s.T(), err) ctx := testNewContext(d) err = ctx.NewTxn(context.Background()) - c.Assert(err, IsNil) - testCreateTable(c, ctx, d, s.dbInfo, tblInfo) + require.NoError(s.T(), err) + testCreateTable(s.T(), ctx, d, s.dbInfo, tblInfo) // insert t_fail values (1, 2); - originTable := testGetTable(c, d, s.dbInfo.ID, tblInfo.ID) + originTable := testGetTable(s.T(), d, s.dbInfo.ID, tblInfo.ID) row := types.MakeDatums(1, 2) _, err = originTable.AddRecord(ctx, row) - c.Assert(err, IsNil) + require.NoError(s.T(), err) txn, err := ctx.Txn(true) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) tc := &TestDDLCallback{} first := true @@ -62,17 +62,17 @@ func (s *testColumnChangeSuite) TestFailBeforeDecodeArgs(c *C) { stateCnt++ } else if job.SchemaState == model.StateWriteReorganization { if first { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/errorBeforeDecodeArgs", `return(true)`), IsNil) + require.NoError(s.T(), failpoint.Enable("github.com/pingcap/tidb/ddl/errorBeforeDecodeArgs", `return(true)`)) first = false } else { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/errorBeforeDecodeArgs"), IsNil) + require.NoError(s.T(), failpoint.Disable("github.com/pingcap/tidb/ddl/errorBeforeDecodeArgs")) } } } d.SetHook(tc) defaultValue := int64(3) - job := testCreateColumn(c, ctx, d, s.dbInfo, tblInfo, "c3", &ast.ColumnPosition{Tp: ast.ColumnPositionNone}, defaultValue) + job := testCreateColumn(s.T(), ctx, d, s.dbInfo, tblInfo, "c3", &ast.ColumnPosition{Tp: ast.ColumnPositionNone}, defaultValue) // Make sure the schema state only appears once. - c.Assert(stateCnt, Equals, 1) - testCheckJobDone(c, d, job, true) + require.Equal(s.T(), 1, stateCnt) + testCheckJobDone(s.T(), d, job, true) } diff --git a/ddl/index_change_test.go b/ddl/index_change_test.go index f59a78cbd6..4f8356cd4a 100644 --- a/ddl/index_change_test.go +++ b/ddl/index_change_test.go @@ -16,10 +16,11 @@ package ddl import ( "context" + "testing" "time" - . "github.com/pingcap/check" "github.com/pingcap/errors" + "github.com/pingcap/tidb/domain/infosync" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" @@ -27,70 +28,78 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" ) -var _ = Suite(&testIndexChangeSuite{}) - -type testIndexChangeSuite struct { +type testIndexChangeSuiteToVerify struct { + suite.Suite store kv.Storage dbInfo *model.DBInfo } -func (s *testIndexChangeSuite) SetUpSuite(c *C) { - s.store = testCreateStore(c, "test_index_change") +func TestIndexChangeSuite(t *testing.T) { + _, err := infosync.GlobalInfoSyncerInit(context.Background(), "t", func() uint64 { return 1 }, nil, true) + require.NoError(t, err) + + suite.Run(t, new(testIndexChangeSuiteToVerify)) +} + +func (s *testIndexChangeSuiteToVerify) SetupSuite() { + s.store = testCreateStore(s.T(), "test_index_change") d, err := testNewDDLAndStart( context.Background(), WithStore(s.store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() s.dbInfo, err = testSchemaInfo(d, "test_index_change") - c.Assert(err, IsNil) - testCreateSchema(c, testNewContext(d), d, s.dbInfo) + require.NoError(s.T(), err) + testCreateSchema(s.T(), testNewContext(d), d, s.dbInfo) } -func (s *testIndexChangeSuite) TearDownSuite(c *C) { +func (s *testIndexChangeSuiteToVerify) TearDownSuite() { s.store.Close() } -func (s *testIndexChangeSuite) TestIndexChange(c *C) { +func (s *testIndexChangeSuiteToVerify) TestIndexChange() { d, err := testNewDDLAndStart( context.Background(), WithStore(s.store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() // create table t (c1 int primary key, c2 int); tblInfo, err := testTableInfo(d, "t", 2) - c.Assert(err, IsNil) + require.NoError(s.T(), err) tblInfo.Columns[0].Flag = mysql.PriKeyFlag | mysql.NotNullFlag tblInfo.PKIsHandle = true ctx := testNewContext(d) err = ctx.NewTxn(context.Background()) - c.Assert(err, IsNil) - testCreateTable(c, ctx, d, s.dbInfo, tblInfo) - originTable := testGetTable(c, d, s.dbInfo.ID, tblInfo.ID) + require.NoError(s.T(), err) + testCreateTable(s.T(), ctx, d, s.dbInfo, tblInfo) + originTable := testGetTable(s.T(), d, s.dbInfo.ID, tblInfo.ID) // insert t values (1, 1), (2, 2), (3, 3) _, err = originTable.AddRecord(ctx, types.MakeDatums(1, 1)) - c.Assert(err, IsNil) + require.NoError(s.T(), err) _, err = originTable.AddRecord(ctx, types.MakeDatums(2, 2)) - c.Assert(err, IsNil) + require.NoError(s.T(), err) _, err = originTable.AddRecord(ctx, types.MakeDatums(3, 3)) - c.Assert(err, IsNil) + require.NoError(s.T(), err) txn, err := ctx.Txn(true) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) tc := &TestDDLCallback{} // set up hook @@ -141,7 +150,7 @@ func (s *testIndexChangeSuite) TestIndexChange(c *C) { } } d.SetHook(tc) - testCreateIndex(c, ctx, d, s.dbInfo, originTable.Meta(), false, "c2", "c2") + testCreateIndex(s.T(), ctx, d, s.dbInfo, originTable.Meta(), false, "c2", "c2") // We need to make sure onJobUpdated is called in the first hook. // After testCreateIndex(), onJobUpdated() may not be called when job.state is Sync. // If we skip this check, prevState may wrongly set to StatePublic. @@ -151,10 +160,10 @@ func (s *testIndexChangeSuite) TestIndexChange(c *C) { } time.Sleep(50 * time.Millisecond) } - c.Check(checkErr, IsNil) + require.NoError(s.T(), checkErr) txn, err = ctx.Txn(true) - c.Assert(err, IsNil) - c.Assert(txn.Commit(context.Background()), IsNil) + require.NoError(s.T(), err) + require.Nil(s.T(), txn.Commit(context.Background())) prevState = model.StateNone var noneTable table.Table tc.onJobUpdated = func(job *model.Job) { @@ -193,8 +202,8 @@ func (s *testIndexChangeSuite) TestIndexChange(c *C) { } } } - testDropIndex(c, ctx, d, s.dbInfo, publicTable.Meta(), "c2") - c.Check(checkErr, IsNil) + testDropIndex(s.T(), ctx, d, s.dbInfo, publicTable.Meta(), "c2") + require.NoError(s.T(), checkErr) } func checkIndexExists(ctx sessionctx.Context, tbl table.Table, indexValue interface{}, handle int64, exists bool) error { @@ -216,7 +225,7 @@ func checkIndexExists(ctx sessionctx.Context, tbl table.Table, indexValue interf return nil } -func (s *testIndexChangeSuite) checkAddWriteOnly(d *ddl, ctx sessionctx.Context, delOnlyTbl, writeOnlyTbl table.Table) error { +func (s *testIndexChangeSuiteToVerify) checkAddWriteOnly(d *ddl, ctx sessionctx.Context, delOnlyTbl, writeOnlyTbl table.Table) error { // DeleteOnlyTable: insert t values (4, 4); err := ctx.NewTxn(context.Background()) if err != nil { @@ -289,7 +298,7 @@ func (s *testIndexChangeSuite) checkAddWriteOnly(d *ddl, ctx sessionctx.Context, return nil } -func (s *testIndexChangeSuite) checkAddPublic(d *ddl, ctx sessionctx.Context, writeTbl, publicTbl table.Table) error { +func (s *testIndexChangeSuiteToVerify) checkAddPublic(d *ddl, ctx sessionctx.Context, writeTbl, publicTbl table.Table) error { // WriteOnlyTable: insert t values (6, 6) err := ctx.NewTxn(context.Background()) if err != nil { @@ -363,7 +372,7 @@ func (s *testIndexChangeSuite) checkAddPublic(d *ddl, ctx sessionctx.Context, wr return txn.Commit(context.Background()) } -func (s *testIndexChangeSuite) checkDropWriteOnly(d *ddl, ctx sessionctx.Context, publicTbl, writeTbl table.Table) error { +func (s *testIndexChangeSuiteToVerify) checkDropWriteOnly(d *ddl, ctx sessionctx.Context, publicTbl, writeTbl table.Table) error { // WriteOnlyTable insert t values (8, 8) err := ctx.NewTxn(context.Background()) if err != nil { @@ -407,7 +416,7 @@ func (s *testIndexChangeSuite) checkDropWriteOnly(d *ddl, ctx sessionctx.Context return txn.Commit(context.Background()) } -func (s *testIndexChangeSuite) checkDropDeleteOnly(d *ddl, ctx sessionctx.Context, writeTbl, delTbl table.Table) error { +func (s *testIndexChangeSuiteToVerify) checkDropDeleteOnly(d *ddl, ctx sessionctx.Context, writeTbl, delTbl table.Table) error { // WriteOnlyTable insert t values (9, 9) err := ctx.NewTxn(context.Background()) if err != nil { diff --git a/ddl/placement_policy_ddl_test.go b/ddl/placement_policy_ddl_test.go index 28efb3bae3..00c6237e29 100644 --- a/ddl/placement_policy_ddl_test.go +++ b/ddl/placement_policy_ddl_test.go @@ -15,27 +15,28 @@ package ddl import ( "context" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx" + "github.com/stretchr/testify/require" ) -func testPlacementPolicyInfo(c *C, d *ddl, name string, settings *model.PlacementSettings) *model.PolicyInfo { +func testPlacementPolicyInfo(t *testing.T, d *ddl, name string, settings *model.PlacementSettings) *model.PolicyInfo { policy := &model.PolicyInfo{ Name: model.NewCIStr(name), PlacementSettings: settings, } genIDs, err := d.genGlobalIDs(1) - c.Assert(err, IsNil) + require.NoError(t, err) policy.ID = genIDs[0] return policy } -func testCreatePlacementPolicy(c *C, ctx sessionctx.Context, d *ddl, policyInfo *model.PolicyInfo) *model.Job { +func testCreatePlacementPolicy(t *testing.T, ctx sessionctx.Context, d *ddl, policyInfo *model.PolicyInfo) *model.Job { job := &model.Job{ SchemaName: policyInfo.Name.L, Type: model.ActionCreatePlacementPolicy, @@ -43,79 +44,79 @@ func testCreatePlacementPolicy(c *C, ctx sessionctx.Context, d *ddl, policyInfo Args: []interface{}{policyInfo}, } err := d.doDDLJob(ctx, job) - c.Assert(err, IsNil) + require.NoError(t, err) - v := getSchemaVer(c, ctx) + v := getSchemaVer(t, ctx) policyInfo.State = model.StatePublic - checkHistoryJobArgs(c, ctx, job.ID, &historyJobArgs{ver: v}) + checkHistoryJobArgs(t, ctx, job.ID, &historyJobArgs{ver: v}) policyInfo.State = model.StateNone return job } -func (s *testDDLSuite) TestPlacementPolicyInUse(c *C) { - store := testCreateStore(c, "test_placement_policy_in_use") +func (s *testDDLSuiteToVerify) TestPlacementPolicyInUse() { + store := testCreateStore(s.T(), "test_placement_policy_in_use") defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() ctx := context.Background() d, err := testNewDDLAndStart(ctx, WithStore(store)) - c.Assert(err, IsNil) + require.NoError(s.T(), err) sctx := testNewContext(d) db1, err := testSchemaInfo(d, "db1") - c.Assert(err, IsNil) - testCreateSchema(c, sctx, d, db1) + require.NoError(s.T(), err) + testCreateSchema(s.T(), sctx, d, db1) db1.State = model.StatePublic db2, err := testSchemaInfo(d, "db2") - c.Assert(err, IsNil) - testCreateSchema(c, sctx, d, db2) + require.NoError(s.T(), err) + testCreateSchema(s.T(), sctx, d, db2) db2.State = model.StatePublic policySettings := &model.PlacementSettings{PrimaryRegion: "r1", Regions: "r1,r2"} - p1 := testPlacementPolicyInfo(c, d, "p1", policySettings) - p2 := testPlacementPolicyInfo(c, d, "p2", policySettings) - p3 := testPlacementPolicyInfo(c, d, "p3", policySettings) - p4 := testPlacementPolicyInfo(c, d, "p4", policySettings) - p5 := testPlacementPolicyInfo(c, d, "p5", policySettings) - testCreatePlacementPolicy(c, sctx, d, p1) - testCreatePlacementPolicy(c, sctx, d, p2) - testCreatePlacementPolicy(c, sctx, d, p3) - testCreatePlacementPolicy(c, sctx, d, p4) - testCreatePlacementPolicy(c, sctx, d, p5) + p1 := testPlacementPolicyInfo(s.T(), d, "p1", policySettings) + p2 := testPlacementPolicyInfo(s.T(), d, "p2", policySettings) + p3 := testPlacementPolicyInfo(s.T(), d, "p3", policySettings) + p4 := testPlacementPolicyInfo(s.T(), d, "p4", policySettings) + p5 := testPlacementPolicyInfo(s.T(), d, "p5", policySettings) + testCreatePlacementPolicy(s.T(), sctx, d, p1) + testCreatePlacementPolicy(s.T(), sctx, d, p2) + testCreatePlacementPolicy(s.T(), sctx, d, p3) + testCreatePlacementPolicy(s.T(), sctx, d, p4) + testCreatePlacementPolicy(s.T(), sctx, d, p5) t1, err := testTableInfo(d, "t1", 1) - c.Assert(err, IsNil) + require.NoError(s.T(), err) t1.PlacementPolicyRef = &model.PolicyRefInfo{ID: p1.ID, Name: p1.Name} - testCreateTable(c, sctx, d, db1, t1) + testCreateTable(s.T(), sctx, d, db1, t1) t1.State = model.StatePublic db1.Tables = append(db1.Tables, t1) t2, err := testTableInfo(d, "t2", 1) - c.Assert(err, IsNil) + require.NoError(s.T(), err) t2.PlacementPolicyRef = &model.PolicyRefInfo{ID: p1.ID, Name: p1.Name} - testCreateTable(c, sctx, d, db2, t2) + testCreateTable(s.T(), sctx, d, db2, t2) t2.State = model.StatePublic db2.Tables = append(db2.Tables, t2) t3, err := testTableInfo(d, "t3", 1) - c.Assert(err, IsNil) + require.NoError(s.T(), err) t3.PlacementPolicyRef = &model.PolicyRefInfo{ID: p2.ID, Name: p2.Name} - testCreateTable(c, sctx, d, db1, t3) + testCreateTable(s.T(), sctx, d, db1, t3) t3.State = model.StatePublic db1.Tables = append(db1.Tables, t3) dbP, err := testSchemaInfo(d, "db_p") - c.Assert(err, IsNil) + require.NoError(s.T(), err) dbP.PlacementPolicyRef = &model.PolicyRefInfo{ID: p4.ID, Name: p4.Name} dbP.State = model.StatePublic - testCreateSchema(c, sctx, d, dbP) + testCreateSchema(s.T(), sctx, d, dbP) - t4 := testTableInfoWithPartition(c, d, "t4", 1) + t4 := testTableInfoWithPartition(s.T(), d, "t4", 1) t4.Partition.Definitions[0].PlacementPolicyRef = &model.PolicyRefInfo{ID: p5.ID, Name: p5.Name} - testCreateTable(c, sctx, d, db1, t4) + testCreateTable(s.T(), sctx, d, db1, t4) t4.State = model.StatePublic db1.Tables = append(db1.Tables, t4) @@ -125,22 +126,22 @@ func (s *testDDLSuite) TestPlacementPolicyInUse(c *C) { []*model.PolicyInfo{p1, p2, p3, p4, p5}, 1, ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) is := builder.Build() for _, policy := range []*model.PolicyInfo{p1, p2, p4, p5} { - c.Assert(ErrPlacementPolicyInUse.Equal(checkPlacementPolicyNotInUseFromInfoSchema(is, policy)), IsTrue) - c.Assert(kv.RunInNewTxn(ctx, sctx.GetStore(), false, func(ctx context.Context, txn kv.Transaction) error { + require.True(s.T(), ErrPlacementPolicyInUse.Equal(checkPlacementPolicyNotInUseFromInfoSchema(is, policy))) + require.Nil(s.T(), kv.RunInNewTxn(ctx, sctx.GetStore(), false, func(ctx context.Context, txn kv.Transaction) error { m := meta.NewMeta(txn) - c.Assert(ErrPlacementPolicyInUse.Equal(checkPlacementPolicyNotInUseFromMeta(m, policy)), IsTrue) + require.True(s.T(), ErrPlacementPolicyInUse.Equal(checkPlacementPolicyNotInUseFromMeta(m, policy))) return nil - }), IsNil) + })) } - c.Assert(checkPlacementPolicyNotInUseFromInfoSchema(is, p3), IsNil) - c.Assert(kv.RunInNewTxn(ctx, sctx.GetStore(), false, func(ctx context.Context, txn kv.Transaction) error { + require.Nil(s.T(), checkPlacementPolicyNotInUseFromInfoSchema(is, p3)) + require.Nil(s.T(), kv.RunInNewTxn(ctx, sctx.GetStore(), false, func(ctx context.Context, txn kv.Transaction) error { m := meta.NewMeta(txn) - c.Assert(checkPlacementPolicyNotInUseFromMeta(m, p3), IsNil) + require.Nil(s.T(), checkPlacementPolicyNotInUseFromMeta(m, p3)) return nil - }), IsNil) + })) } diff --git a/ddl/reorg_test.go b/ddl/reorg_test.go index 461b5d6cb6..64d72b080e 100644 --- a/ddl/reorg_test.go +++ b/ddl/reorg_test.go @@ -18,12 +18,12 @@ import ( "context" "time" - . "github.com/pingcap/check" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" + "github.com/stretchr/testify/require" ) type testCtxKeyType int @@ -34,11 +34,11 @@ func (k testCtxKeyType) String() string { const testCtxKey testCtxKeyType = 0 -func (s *testDDLSuite) TestReorg(c *C) { - store := testCreateStore(c, "test_reorg") +func (s *testDDLSuiteToVerify) TestReorg() { + store := testCreateStore(s.T(), "test_reorg") defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() d, err := testNewDDLAndStart( @@ -46,10 +46,10 @@ func (s *testDDLSuite) TestReorg(c *C) { WithStore(store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() time.Sleep(testLease) @@ -57,26 +57,26 @@ func (s *testDDLSuite) TestReorg(c *C) { ctx := testNewContext(d) ctx.SetValue(testCtxKey, 1) - c.Assert(ctx.Value(testCtxKey), Equals, 1) + require.Equal(s.T(), ctx.Value(testCtxKey), 1) ctx.ClearValue(testCtxKey) err = ctx.NewTxn(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) txn, err := ctx.Txn(true) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = txn.Set([]byte("a"), []byte("b")) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = txn.Rollback() - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = ctx.NewTxn(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) txn, err = ctx.Txn(true) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = txn.Set([]byte("a"), []byte("b")) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) rowCount := int64(10) handle := s.NewHandle().Int(100).Common("a", 100, "string") @@ -91,9 +91,9 @@ func (s *testDDLSuite) TestReorg(c *C) { SnapshotVer: 1, // Make sure it is not zero. So the reorgInfo's first is false. } err = ctx.NewTxn(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) txn, err = ctx.Txn(true) - c.Assert(err, IsNil) + require.NoError(s.T(), err) m := meta.NewMeta(txn) e := &meta.Element{ID: 333, TypeKey: meta.IndexElementKey} rInfo := &reorgInfo{ @@ -102,33 +102,33 @@ func (s *testDDLSuite) TestReorg(c *C) { } mockTbl := tables.MockTableFromMeta(&model.TableInfo{IsCommonHandle: s.IsCommonHandle, CommonHandleVersion: 1}) err = d.generalWorker().runReorgJob(m, rInfo, mockTbl.Meta(), d.lease, f) - c.Assert(err, NotNil) + require.Error(s.T(), err) // The longest to wait for 5 seconds to make sure the function of f is returned. for i := 0; i < 1000; i++ { time.Sleep(5 * time.Millisecond) err = d.generalWorker().runReorgJob(m, rInfo, mockTbl.Meta(), d.lease, f) if err == nil { - c.Assert(job.RowCount, Equals, rowCount) - c.Assert(d.generalWorker().reorgCtx.rowCount, Equals, int64(0)) + require.Equal(s.T(), job.RowCount, rowCount) + require.Equal(s.T(), d.generalWorker().reorgCtx.rowCount, int64(0)) // Test whether reorgInfo's Handle is update. err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = ctx.NewTxn(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) m = meta.NewMeta(txn) info, err1 := getReorgInfo(d.ddlCtx, m, job, mockTbl, nil) - c.Assert(err1, IsNil) - c.Assert(info.StartKey, DeepEquals, kv.Key(handle.Encoded())) - c.Assert(info.currElement, DeepEquals, e) + require.NoError(s.T(), err1) + require.Equal(s.T(), info.StartKey, kv.Key(handle.Encoded())) + require.Equal(s.T(), info.currElement, e) _, doneHandle, _ := d.generalWorker().reorgCtx.getRowCountAndKey() - c.Assert(doneHandle, IsNil) + require.Nil(s.T(), doneHandle) break } } - c.Assert(err, IsNil) + require.NoError(s.T(), err) job = &model.Job{ ID: 2, @@ -151,45 +151,45 @@ func (s *testDDLSuite) TestReorg(c *C) { t := meta.NewMeta(txn) var err1 error _, err1 = getReorgInfo(d.ddlCtx, t, job, mockTbl, []*meta.Element{element}) - c.Assert(meta.ErrDDLReorgElementNotExist.Equal(err1), IsTrue) - c.Assert(job.SnapshotVer, Equals, uint64(0)) + require.True(s.T(), meta.ErrDDLReorgElementNotExist.Equal(err1)) + require.Equal(s.T(), job.SnapshotVer, uint64(0)) return nil }) - c.Assert(err, IsNil) + require.NoError(s.T(), err) job.SnapshotVer = uint64(1) err = info.UpdateReorgMeta(info.StartKey) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = kv.RunInNewTxn(context.Background(), d.store, false, func(ctx context.Context, txn kv.Transaction) error { t := meta.NewMeta(txn) info1, err1 := getReorgInfo(d.ddlCtx, t, job, mockTbl, []*meta.Element{element}) - c.Assert(err1, IsNil) - c.Assert(info1.currElement, DeepEquals, info.currElement) - c.Assert(info1.StartKey, DeepEquals, info.StartKey) - c.Assert(info1.EndKey, DeepEquals, info.EndKey) - c.Assert(info1.PhysicalTableID, Equals, info.PhysicalTableID) + require.NoError(s.T(), err1) + require.Equal(s.T(), info1.currElement, info.currElement) + require.Equal(s.T(), info1.StartKey, info.StartKey) + require.Equal(s.T(), info1.EndKey, info.EndKey) + require.Equal(s.T(), info1.PhysicalTableID, info.PhysicalTableID) return nil }) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = d.generalWorker().runReorgJob(m, rInfo, mockTbl.Meta(), d.lease, func() error { time.Sleep(4 * testLease) return nil }) - c.Assert(err, NotNil) + require.Error(s.T(), err) txn, err = ctx.Txn(true) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) - s.RerunWithCommonHandleEnabled(c, s.TestReorg) + require.NoError(s.T(), err) + s.RerunWithCommonHandleEnabledWithoutCheck(s.TestReorg) } -func (s *testDDLSuite) TestReorgOwner(c *C) { - store := testCreateStore(c, "test_reorg_owner") +func (s *testDDLSuiteToVerify) TestReorgOwner() { + store := testCreateStore(s.T(), "test_reorg_owner") defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() d1, err := testNewDDLAndStart( @@ -197,65 +197,65 @@ func (s *testDDLSuite) TestReorgOwner(c *C) { WithStore(store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := d1.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() ctx := testNewContext(d1) - testCheckOwner(c, d1, true) + testCheckOwner(s.T(), d1, true) d2, err := testNewDDLAndStart( context.Background(), WithStore(store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := d2.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() dbInfo, err := testSchemaInfo(d1, "test_reorg") - c.Assert(err, IsNil) - testCreateSchema(c, ctx, d1, dbInfo) + require.NoError(s.T(), err) + testCreateSchema(s.T(), ctx, d1, dbInfo) tblInfo, err := testTableInfo(d1, "t", 3) - c.Assert(err, IsNil) - testCreateTable(c, ctx, d1, dbInfo, tblInfo) - t := testGetTable(c, d1, dbInfo.ID, tblInfo.ID) + require.NoError(s.T(), err) + testCreateTable(s.T(), ctx, d1, dbInfo, tblInfo) + t := testGetTable(s.T(), d1, dbInfo.ID, tblInfo.ID) num := 10 for i := 0; i < num; i++ { _, err := t.AddRecord(ctx, types.MakeDatums(i, i, i)) - c.Assert(err, IsNil) + require.NoError(s.T(), err) } txn, err := ctx.Txn(true) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) tc := &TestDDLCallback{} tc.onJobRunBefore = func(job *model.Job) { if job.SchemaState == model.StateDeleteReorganization { err = d1.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) } } d1.SetHook(tc) - testDropSchema(c, ctx, d1, dbInfo) + testDropSchema(s.T(), ctx, d1, dbInfo) err = kv.RunInNewTxn(context.Background(), d1.store, false, func(ctx context.Context, txn kv.Transaction) error { t := meta.NewMeta(txn) db, err1 := t.GetDatabase(dbInfo.ID) - c.Assert(err1, IsNil) - c.Assert(db, IsNil) + require.NoError(s.T(), err1) + require.Nil(s.T(), db) return nil }) - c.Assert(err, IsNil) + require.NoError(s.T(), err) } diff --git a/ddl/restart_test.go b/ddl/restart_test.go index e1cf6d15fe..1aa962bbbf 100644 --- a/ddl/restart_test.go +++ b/ddl/restart_test.go @@ -19,13 +19,16 @@ package ddl import ( "context" "errors" + "testing" "time" - . "github.com/pingcap/check" + "github.com/pingcap/tidb/domain/infosync" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/util/mock" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" ) // this test file include some test that will cause data race, mainly because restartWorkers modify d.ctx @@ -54,7 +57,7 @@ func (d *ddl) restartWorkers(ctx context.Context) { } // runInterruptedJob should be called concurrently with restartWorkers -func runInterruptedJob(c *C, d *ddl, job *model.Job, doneCh chan error) { +func runInterruptedJob(d *ddl, job *model.Job, doneCh chan error) { ctx := mock.NewContext() ctx.Store = d.store @@ -83,9 +86,9 @@ func runInterruptedJob(c *C, d *ddl, job *model.Job, doneCh chan error) { doneCh <- err } -func testRunInterruptedJob(c *C, d *ddl, job *model.Job) { +func testRunInterruptedJob(t *testing.T, d *ddl, job *model.Job) { done := make(chan error, 1) - go runInterruptedJob(c, d, job, done) + go runInterruptedJob(d, job, done) ticker := time.NewTicker(d.lease * 1) defer ticker.Stop() @@ -94,21 +97,21 @@ LOOP: select { case <-ticker.C: err := d.Stop() - c.Assert(err, IsNil) + require.Nil(t, err) d.restartWorkers(context.Background()) time.Sleep(time.Millisecond * 20) case err := <-done: - c.Assert(err, IsNil) + require.Nil(t, err) break LOOP } } } -func (s *testSchemaSuite) TestSchemaResume(c *C) { - store := testCreateStore(c, "test_schema_resume") +func TestSchemaResume(t *testing.T) { + store := testCreateStore(t, "test_schema_resume") defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(t, err) }() d1, err := testNewDDLAndStart( @@ -116,39 +119,39 @@ func (s *testSchemaSuite) TestSchemaResume(c *C) { WithStore(store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(t, err) defer func() { err := d1.Stop() - c.Assert(err, IsNil) + require.NoError(t, err) }() - testCheckOwner(c, d1, true) + testCheckOwner(t, d1, true) dbInfo, err := testSchemaInfo(d1, "test_restart") - c.Assert(err, IsNil) + require.NoError(t, err) job := &model.Job{ SchemaID: dbInfo.ID, Type: model.ActionCreateSchema, BinlogInfo: &model.HistoryInfo{}, Args: []interface{}{dbInfo}, } - testRunInterruptedJob(c, d1, job) - testCheckSchemaState(c, d1, dbInfo, model.StatePublic) + testRunInterruptedJob(t, d1, job) + testCheckSchemaState(t, d1, dbInfo, model.StatePublic) job = &model.Job{ SchemaID: dbInfo.ID, Type: model.ActionDropSchema, BinlogInfo: &model.HistoryInfo{}, } - testRunInterruptedJob(c, d1, job) - testCheckSchemaState(c, d1, dbInfo, model.StateNone) + testRunInterruptedJob(t, d1, job) + testCheckSchemaState(t, d1, dbInfo, model.StateNone) } -func (s *testStatSuite) TestStat(c *C) { - store := testCreateStore(c, "test_stat") +func (s *testStatSuiteToVerify) TestStat() { + store := testCreateStore(s.T(), "test_stat") defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() d, err := testNewDDLAndStart( @@ -156,15 +159,15 @@ func (s *testStatSuite) TestStat(c *C) { WithStore(store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() dbInfo, err := testSchemaInfo(d, "test_restart") - c.Assert(err, IsNil) - testCreateSchema(c, testNewContext(d), d, dbInfo) + require.NoError(s.T(), err) + testCreateSchema(s.T(), testNewContext(d), d, dbInfo) // TODO: Get this information from etcd. // m, err := d.Stats(nil) @@ -179,68 +182,74 @@ func (s *testStatSuite) TestStat(c *C) { } done := make(chan error, 1) - go runInterruptedJob(c, d, job, done) + go runInterruptedJob(d, job, done) ticker := time.NewTicker(d.lease * 1) defer ticker.Stop() - ver := s.getDDLSchemaVer(c, d) + ver := s.getDDLSchemaVer(d) LOOP: for { select { case <-ticker.C: err := d.Stop() - c.Assert(err, IsNil) - c.Assert(s.getDDLSchemaVer(c, d), GreaterEqual, ver) + require.Nil(s.T(), err) + require.GreaterOrEqual(s.T(), s.getDDLSchemaVer(d), ver) d.restartWorkers(context.Background()) time.Sleep(time.Millisecond * 20) case err := <-done: // TODO: Get this information from etcd. // m, err := d.Stats(nil) - c.Assert(err, IsNil) + require.Nil(s.T(), err) break LOOP } } } -var _ = Suite(&testTableSuite{}) - -type testTableSuite struct { +type testTableSuiteToVerify struct { + suite.Suite store kv.Storage dbInfo *model.DBInfo d *ddl } -func (s *testTableSuite) SetUpSuite(c *C) { - s.store = testCreateStore(c, "test_table") +func TestTableSuite(t *testing.T) { + _, err := infosync.GlobalInfoSyncerInit(context.Background(), "t", func() uint64 { return 1 }, nil, true) + require.NoError(t, err) + + suite.Run(t, new(testTableSuiteToVerify)) +} + +func (s *testTableSuiteToVerify) SetupSuite() { + s.store = testCreateStore(s.T(), "test_table") ddl, err := testNewDDLAndStart( context.Background(), WithStore(s.store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) s.d = ddl s.dbInfo, err = testSchemaInfo(s.d, "test_table") - c.Assert(err, IsNil) - testCreateSchema(c, testNewContext(s.d), s.d, s.dbInfo) + require.NoError(s.T(), err) + testCreateSchema(s.T(), testNewContext(s.d), s.d, s.dbInfo) } -func (s *testTableSuite) TearDownSuite(c *C) { - testDropSchema(c, testNewContext(s.d), s.d, s.dbInfo) +func (s *testTableSuiteToVerify) TearDownSuite() { + testDropSchema(s.T(), testNewContext(s.d), s.d, s.dbInfo) err := s.d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = s.store.Close() - c.Assert(err, IsNil) + require.NoError(s.T(), err) } -func (s *testTableSuite) TestTableResume(c *C) { +func (s *testTableSuiteToVerify) TestTableResume() { d := s.d - testCheckOwner(c, d, true) + testCheckOwner(s.T(), d, true) tblInfo, err := testTableInfo(d, "t1", 3) - c.Assert(err, IsNil) + require.NoError(s.T(), err) job := &model.Job{ SchemaID: s.dbInfo.ID, TableID: tblInfo.ID, @@ -248,8 +257,8 @@ func (s *testTableSuite) TestTableResume(c *C) { BinlogInfo: &model.HistoryInfo{}, Args: []interface{}{tblInfo}, } - testRunInterruptedJob(c, d, job) - testCheckTableState(c, d, s.dbInfo, tblInfo, model.StatePublic) + testRunInterruptedJob(s.T(), d, job) + testCheckTableState(s.T(), d, s.dbInfo, tblInfo, model.StatePublic) job = &model.Job{ SchemaID: s.dbInfo.ID, @@ -257,6 +266,6 @@ func (s *testTableSuite) TestTableResume(c *C) { Type: model.ActionDropTable, BinlogInfo: &model.HistoryInfo{}, } - testRunInterruptedJob(c, d, job) - testCheckTableState(c, d, s.dbInfo, tblInfo, model.StateNone) + testRunInterruptedJob(s.T(), d, job) + testCheckTableState(s.T(), d, s.dbInfo, tblInfo, model.StateNone) } diff --git a/ddl/schema_test.go b/ddl/schema_test.go index 2ad14b417d..9594331b77 100644 --- a/ddl/schema_test.go +++ b/ddl/schema_test.go @@ -19,7 +19,6 @@ import ( "testing" "time" - . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" @@ -31,16 +30,6 @@ import ( "github.com/stretchr/testify/require" ) -var _ = Suite(&testSchemaSuite{}) - -type testSchemaSuite struct{} - -func (s *testSchemaSuite) SetUpSuite(c *C) { -} - -func (s *testSchemaSuite) TearDownSuite(c *C) { -} - func testSchemaInfo(d *ddl, name string) (*model.DBInfo, error) { dbInfo := &model.DBInfo{ Name: model.NewCIStr(name), @@ -53,7 +42,7 @@ func testSchemaInfo(d *ddl, name string) (*model.DBInfo, error) { return dbInfo, nil } -func testCreateSchema(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo) *model.Job { +func testCreateSchema(t *testing.T, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo) *model.Job { job := &model.Job{ SchemaID: dbInfo.ID, Type: model.ActionCreateSchema, @@ -61,11 +50,11 @@ func testCreateSchema(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo Args: []interface{}{dbInfo}, } err := d.doDDLJob(ctx, job) - c.Assert(err, IsNil) + require.NoError(t, err) - v := getSchemaVer(c, ctx) + v := getSchemaVer(t, ctx) dbInfo.State = model.StatePublic - checkHistoryJobArgs(c, ctx, job.ID, &historyJobArgs{ver: v, db: dbInfo}) + checkHistoryJobArgs(t, ctx, job.ID, &historyJobArgs{ver: v, db: dbInfo}) dbInfo.State = model.StateNone return job } @@ -95,11 +84,11 @@ func buildDropSchemaJob(dbInfo *model.DBInfo) *model.Job { } } -func testDropSchema(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo) (*model.Job, int64) { +func testDropSchema(t *testing.T, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo) (*model.Job, int64) { job := buildDropSchemaJob(dbInfo) err := d.doDDLJob(ctx, job) - c.Assert(err, IsNil) - ver := getSchemaVer(c, ctx) + require.NoError(t, err) + ver := getSchemaVer(t, ctx) return job, ver } @@ -111,9 +100,9 @@ func testDropSchemaT(t *testing.T, ctx sessionctx.Context, d *ddl, dbInfo *model return job, ver } -func isDDLJobDone(c *C, t *meta.Meta) bool { +func isDDLJobDone(test *testing.T, t *meta.Meta) bool { job, err := t.GetDDLJobByIdx(0) - c.Assert(err, IsNil) + require.NoError(test, err) if job == nil { return true } @@ -122,29 +111,29 @@ func isDDLJobDone(c *C, t *meta.Meta) bool { return false } -func testCheckSchemaState(c *C, d *ddl, dbInfo *model.DBInfo, state model.SchemaState) { +func testCheckSchemaState(test *testing.T, d *ddl, dbInfo *model.DBInfo, state model.SchemaState) { isDropped := true for { err := kv.RunInNewTxn(context.Background(), d.store, false, func(ctx context.Context, txn kv.Transaction) error { t := meta.NewMeta(txn) info, err := t.GetDatabase(dbInfo.ID) - c.Assert(err, IsNil) + require.NoError(test, err) if state == model.StateNone { - isDropped = isDDLJobDone(c, t) + isDropped = isDDLJobDone(test, t) if !isDropped { return nil } - c.Assert(info, IsNil) + require.Nil(test, info) return nil } - c.Assert(info.Name, DeepEquals, dbInfo.Name) - c.Assert(info.State, Equals, state) + require.Equal(test, info.Name, dbInfo.Name) + require.Equal(test, info.State, state) return nil }) - c.Assert(err, IsNil) + require.NoError(test, err) if isDropped { break @@ -152,60 +141,60 @@ func testCheckSchemaState(c *C, d *ddl, dbInfo *model.DBInfo, state model.Schema } } -func (s *testSchemaSuite) TestSchema(c *C) { - store := testCreateStore(c, "test_schema") +func TestSchema(t *testing.T) { + store := testCreateStore(t, "test_schema") defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(t, err) }() d, err := testNewDDLAndStart( context.Background(), WithStore(store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(t, err) defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(t, err) }() ctx := testNewContext(d) dbInfo, err := testSchemaInfo(d, "test_schema") - c.Assert(err, IsNil) + require.NoError(t, err) // create a database. - job := testCreateSchema(c, ctx, d, dbInfo) - testCheckSchemaState(c, d, dbInfo, model.StatePublic) - testCheckJobDone(c, d, job, true) + job := testCreateSchema(t, ctx, d, dbInfo) + testCheckSchemaState(t, d, dbInfo, model.StatePublic) + testCheckJobDone(t, d, job, true) /*** to drop the schema with two tables. ***/ // create table t with 100 records. tblInfo1, err := testTableInfo(d, "t", 3) - c.Assert(err, IsNil) - tJob1 := testCreateTable(c, ctx, d, dbInfo, tblInfo1) - testCheckTableState(c, d, dbInfo, tblInfo1, model.StatePublic) - testCheckJobDone(c, d, tJob1, true) - tbl1 := testGetTable(c, d, dbInfo.ID, tblInfo1.ID) + require.NoError(t, err) + tJob1 := testCreateTable(t, ctx, d, dbInfo, tblInfo1) + testCheckTableState(t, d, dbInfo, tblInfo1, model.StatePublic) + testCheckJobDone(t, d, tJob1, true) + tbl1 := testGetTable(t, d, dbInfo.ID, tblInfo1.ID) for i := 1; i <= 100; i++ { _, err := tbl1.AddRecord(ctx, types.MakeDatums(i, i, i)) - c.Assert(err, IsNil) + require.NoError(t, err) } // create table t1 with 1034 records. tblInfo2, err := testTableInfo(d, "t1", 3) - c.Assert(err, IsNil) - tJob2 := testCreateTable(c, ctx, d, dbInfo, tblInfo2) - testCheckTableState(c, d, dbInfo, tblInfo2, model.StatePublic) - testCheckJobDone(c, d, tJob2, true) - tbl2 := testGetTable(c, d, dbInfo.ID, tblInfo2.ID) + require.NoError(t, err) + tJob2 := testCreateTable(t, ctx, d, dbInfo, tblInfo2) + testCheckTableState(t, d, dbInfo, tblInfo2, model.StatePublic) + testCheckJobDone(t, d, tJob2, true) + tbl2 := testGetTable(t, d, dbInfo.ID, tblInfo2.ID) for i := 1; i <= 1034; i++ { _, err := tbl2.AddRecord(ctx, types.MakeDatums(i, i, i)) - c.Assert(err, IsNil) + require.NoError(t, err) } - job, v := testDropSchema(c, ctx, d, dbInfo) - testCheckSchemaState(c, d, dbInfo, model.StateNone) + job, v := testDropSchema(t, ctx, d, dbInfo) + testCheckSchemaState(t, d, dbInfo, model.StateNone) ids := make(map[int64]struct{}) ids[tblInfo1.ID] = struct{}{} ids[tblInfo2.ID] = struct{}{} - checkHistoryJobArgs(c, ctx, job.ID, &historyJobArgs{ver: v, db: dbInfo, tblIDs: ids}) + checkHistoryJobArgs(t, ctx, job.ID, &historyJobArgs{ver: v, db: dbInfo, tblIDs: ids}) // Drop a non-existent database. job = &model.Job{ @@ -214,24 +203,24 @@ func (s *testSchemaSuite) TestSchema(c *C) { BinlogInfo: &model.HistoryInfo{}, } err = d.doDDLJob(ctx, job) - c.Assert(terror.ErrorEqual(err, infoschema.ErrDatabaseDropExists), IsTrue, Commentf("err %v", err)) + require.True(t, terror.ErrorEqual(err, infoschema.ErrDatabaseDropExists), "err %v", err) // Drop a database without a table. dbInfo1, err := testSchemaInfo(d, "test1") - c.Assert(err, IsNil) - job = testCreateSchema(c, ctx, d, dbInfo1) - testCheckSchemaState(c, d, dbInfo1, model.StatePublic) - testCheckJobDone(c, d, job, true) - job, _ = testDropSchema(c, ctx, d, dbInfo1) - testCheckSchemaState(c, d, dbInfo1, model.StateNone) - testCheckJobDone(c, d, job, false) + require.NoError(t, err) + job = testCreateSchema(t, ctx, d, dbInfo1) + testCheckSchemaState(t, d, dbInfo1, model.StatePublic) + testCheckJobDone(t, d, job, true) + job, _ = testDropSchema(t, ctx, d, dbInfo1) + testCheckSchemaState(t, d, dbInfo1, model.StateNone) + testCheckJobDone(t, d, job, false) } -func (s *testSchemaSuite) TestSchemaWaitJob(c *C) { - store := testCreateStore(c, "test_schema_wait") +func TestSchemaWaitJob(t *testing.T) { + store := testCreateStore(t, "test_schema_wait") defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(t, err) }() d1, err := testNewDDLAndStart( @@ -239,23 +228,23 @@ func (s *testSchemaSuite) TestSchemaWaitJob(c *C) { WithStore(store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(t, err) defer func() { err := d1.Stop() - c.Assert(err, IsNil) + require.NoError(t, err) }() - testCheckOwner(c, d1, true) + testCheckOwner(t, d1, true) d2, err := testNewDDLAndStart( context.Background(), WithStore(store), WithLease(testLease*4), ) - c.Assert(err, IsNil) + require.NoError(t, err) defer func() { err := d2.Stop() - c.Assert(err, IsNil) + require.NoError(t, err) }() ctx := testNewContext(d2) @@ -263,17 +252,17 @@ func (s *testSchemaSuite) TestSchemaWaitJob(c *C) { d2.ownerManager.RetireOwner() dbInfo, err := testSchemaInfo(d2, "test_schema") - c.Assert(err, IsNil) - testCreateSchema(c, ctx, d2, dbInfo) - testCheckSchemaState(c, d2, dbInfo, model.StatePublic) + require.NoError(t, err) + testCreateSchema(t, ctx, d2, dbInfo) + testCheckSchemaState(t, d2, dbInfo, model.StatePublic) // d2 must not be owner. - c.Assert(d2.ownerManager.IsOwner(), IsFalse) + require.False(t, d2.ownerManager.IsOwner()) genIDs, err := d2.genGlobalIDs(1) - c.Assert(err, IsNil) + require.NoError(t, err) schemaID := genIDs[0] - doDDLJobErr(c, schemaID, 0, model.ActionCreateSchema, []interface{}{dbInfo}, ctx, d2) + doDDLJobErr(t, schemaID, 0, model.ActionCreateSchema, []interface{}{dbInfo}, ctx, d2) } func testGetSchemaInfoWithError(d *ddl, schemaID int64) (*model.DBInfo, error) { diff --git a/ddl/stat_test.go b/ddl/stat_test.go index 933cde0ae9..ee68453849 100644 --- a/ddl/stat_test.go +++ b/ddl/stat_test.go @@ -16,39 +16,48 @@ package ddl import ( "context" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/types" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" ) -var _ = Suite(&testStatSuite{}) -var _ = SerialSuites(&testSerialStatSuite{}) - -type testStatSuite struct { +type testStatSuiteToVerify struct { + suite.Suite } -func (s *testStatSuite) SetUpSuite(c *C) { +func TestStatSuite(t *testing.T) { + suite.Run(t, new(testStatSuiteToVerify)) } -func (s *testStatSuite) TearDownSuite(c *C) { +func (s *testStatSuiteToVerify) SetupSuite() { } -type testSerialStatSuite struct { +func (s *testStatSuiteToVerify) TearDownSuite() { } -func (s *testStatSuite) getDDLSchemaVer(c *C, d *ddl) int64 { +type testSerialStatSuiteToVerify struct { + suite.Suite +} + +func TestSerialStatSuite(t *testing.T) { + suite.Run(t, new(testSerialStatSuiteToVerify)) +} + +func (s *testStatSuiteToVerify) getDDLSchemaVer(d *ddl) int64 { m, err := d.Stats(nil) - c.Assert(err, IsNil) + require.NoError(s.T(), err) v := m[ddlSchemaVersion] return v.(int64) } -func (s *testSerialStatSuite) TestDDLStatsInfo(c *C) { - store := testCreateStore(c, "test_stat") +func (s *testSerialStatSuiteToVerify) TestDDLStatsInfo() { + store := testCreateStore(s.T(), "test_stat") defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() d, err := testNewDDLAndStart( @@ -56,38 +65,38 @@ func (s *testSerialStatSuite) TestDDLStatsInfo(c *C) { WithStore(store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(s.T(), err) defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(s.T(), err) }() dbInfo, err := testSchemaInfo(d, "test_stat") - c.Assert(err, IsNil) - testCreateSchema(c, testNewContext(d), d, dbInfo) + require.NoError(s.T(), err) + testCreateSchema(s.T(), testNewContext(d), d, dbInfo) tblInfo, err := testTableInfo(d, "t", 2) - c.Assert(err, IsNil) + require.NoError(s.T(), err) ctx := testNewContext(d) - testCreateTable(c, ctx, d, dbInfo, tblInfo) + testCreateTable(s.T(), ctx, d, dbInfo, tblInfo) - t := testGetTable(c, d, dbInfo.ID, tblInfo.ID) + t := testGetTable(s.T(), d, dbInfo.ID, tblInfo.ID) // insert t values (1, 1), (2, 2), (3, 3) _, err = t.AddRecord(ctx, types.MakeDatums(1, 1)) - c.Assert(err, IsNil) + require.NoError(s.T(), err) _, err = t.AddRecord(ctx, types.MakeDatums(2, 2)) - c.Assert(err, IsNil) + require.NoError(s.T(), err) _, err = t.AddRecord(ctx, types.MakeDatums(3, 3)) - c.Assert(err, IsNil) + require.NoError(s.T(), err) txn, err := ctx.Txn(true) - c.Assert(err, IsNil) + require.NoError(s.T(), err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(s.T(), err) job := buildCreateIdxJob(dbInfo, tblInfo, true, "idx", "c1") - c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/checkBackfillWorkerNum", `return(true)`), IsNil) + require.Nil(s.T(), failpoint.Enable("github.com/pingcap/tidb/ddl/checkBackfillWorkerNum", `return(true)`)) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/checkBackfillWorkerNum"), IsNil) + require.Nil(s.T(), failpoint.Disable("github.com/pingcap/tidb/ddl/checkBackfillWorkerNum")) }() done := make(chan error, 1) @@ -99,12 +108,12 @@ func (s *testSerialStatSuite) TestDDLStatsInfo(c *C) { for !exit { select { case err := <-done: - c.Assert(err, IsNil) + require.NoError(s.T(), err) exit = true case <-TestCheckWorkerNumCh: varMap, err := d.Stats(nil) - c.Assert(err, IsNil) - c.Assert(varMap[ddlJobReorgHandle], Equals, "1") + require.NoError(s.T(), err) + require.Equal(s.T(), varMap[ddlJobReorgHandle], "1") } } } diff --git a/ddl/util_test.go b/ddl/util_test.go index 85baca2ffd..d6a050703b 100644 --- a/ddl/util_test.go +++ b/ddl/util_test.go @@ -18,8 +18,8 @@ import ( "bytes" "context" "fmt" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/parser/auth" @@ -28,11 +28,12 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" + "github.com/stretchr/testify/require" ) -func testTableInfoWith2IndexOnFirstColumn(c *C, d *ddl, name string, num int) *model.TableInfo { +func testTableInfoWith2IndexOnFirstColumn(t *testing.T, d *ddl, name string, num int) *model.TableInfo { normalInfo, err := testTableInfo(d, name, num) - c.Assert(err, IsNil) + require.NoError(t, err) idxs := make([]*model.IndexInfo, 0, 2) for i := range idxs { idx := &model.IndexInfo{ @@ -79,11 +80,11 @@ func testTableInfo(d *ddl, name string, num int) (*model.TableInfo, error) { } // testTableInfoWithPartition creates a test table with num int columns and with no index. -func testTableInfoWithPartition(c *C, d *ddl, name string, num int) *model.TableInfo { +func testTableInfoWithPartition(t *testing.T, d *ddl, name string, num int) *model.TableInfo { tblInfo, err := testTableInfo(d, name, num) - c.Assert(err, IsNil) + require.NoError(t, err) genIDs, err := d.genGlobalIDs(1) - c.Assert(err, IsNil) + require.NoError(t, err) pid := genIDs[0] tblInfo.Partition = &model.PartitionInfo{ Type: model.PartitionTypeRange, @@ -100,15 +101,15 @@ func testTableInfoWithPartition(c *C, d *ddl, name string, num int) *model.Table } // testTableInfoWithPartitionLessThan creates a test table with num int columns and one partition specified with lessthan. -func testTableInfoWithPartitionLessThan(c *C, d *ddl, name string, num int, lessthan string) *model.TableInfo { - tblInfo := testTableInfoWithPartition(c, d, name, num) +func testTableInfoWithPartitionLessThan(t *testing.T, d *ddl, name string, num int, lessthan string) *model.TableInfo { + tblInfo := testTableInfoWithPartition(t, d, name, num) tblInfo.Partition.Definitions[0].LessThan = []string{lessthan} return tblInfo } -func testAddedNewTablePartitionInfo(c *C, d *ddl, tblInfo *model.TableInfo, partName, lessthan string) *model.PartitionInfo { +func testAddedNewTablePartitionInfo(t *testing.T, d *ddl, tblInfo *model.TableInfo, partName, lessthan string) *model.PartitionInfo { genIDs, err := d.genGlobalIDs(1) - c.Assert(err, IsNil) + require.NoError(t, err) pid := genIDs[0] // the new added partition should change the partition state to state none at the beginning. return &model.PartitionInfo{ @@ -124,12 +125,12 @@ func testAddedNewTablePartitionInfo(c *C, d *ddl, tblInfo *model.TableInfo, part } // testViewInfo creates a test view with num int columns. -func testViewInfo(c *C, d *ddl, name string, num int) *model.TableInfo { +func testViewInfo(t *testing.T, d *ddl, name string, num int) *model.TableInfo { tblInfo := &model.TableInfo{ Name: model.NewCIStr(name), } genIDs, err := d.genGlobalIDs(1) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo.ID = genIDs[0] cols := make([]*model.ColumnInfo, num) @@ -160,7 +161,7 @@ func testViewInfo(c *C, d *ddl, name string, num int) *model.TableInfo { return tblInfo } -func testCreateTable(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo) *model.Job { +func testCreateTable(t *testing.T, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo) *model.Job { job := &model.Job{ SchemaID: dbInfo.ID, TableID: tblInfo.ID, @@ -169,16 +170,16 @@ func testCreateTable(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, Args: []interface{}{tblInfo}, } err := d.doDDLJob(ctx, job) - c.Assert(err, IsNil) + require.Nil(t, err) - v := getSchemaVer(c, ctx) + v := getSchemaVer(t, ctx) tblInfo.State = model.StatePublic - checkHistoryJobArgs(c, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) + checkHistoryJobArgs(t, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) tblInfo.State = model.StateNone return job } -func testCreateView(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo) *model.Job { +func testCreateView(t *testing.T, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo) *model.Job { job := &model.Job{ SchemaID: dbInfo.ID, TableID: tblInfo.ID, @@ -187,18 +188,18 @@ func testCreateView(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, Args: []interface{}{tblInfo, false, 0}, } - c.Assert(tblInfo.IsView(), IsTrue) + require.True(t, tblInfo.IsView()) err := d.doDDLJob(ctx, job) - c.Assert(err, IsNil) + require.NoError(t, err) - v := getSchemaVer(c, ctx) + v := getSchemaVer(t, ctx) tblInfo.State = model.StatePublic - checkHistoryJobArgs(c, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) + checkHistoryJobArgs(t, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) tblInfo.State = model.StateNone return job } -func testDropTable(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo) *model.Job { +func testDropTable(t *testing.T, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo) *model.Job { job := &model.Job{ SchemaID: dbInfo.ID, TableID: tblInfo.ID, @@ -206,41 +207,41 @@ func testDropTable(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, t BinlogInfo: &model.HistoryInfo{}, } err := d.doDDLJob(ctx, job) - c.Assert(err, IsNil) + require.NoError(t, err) - v := getSchemaVer(c, ctx) - checkHistoryJobArgs(c, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) + v := getSchemaVer(t, ctx) + checkHistoryJobArgs(t, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) return job } -func testCheckTableState(c *C, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, state model.SchemaState) { +func testCheckTableState(test *testing.T, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, state model.SchemaState) { err := kv.RunInNewTxn(context.Background(), d.store, false, func(ctx context.Context, txn kv.Transaction) error { t := meta.NewMeta(txn) info, err := t.GetTable(dbInfo.ID, tblInfo.ID) - c.Assert(err, IsNil) + require.NoError(test, err) if state == model.StateNone { - c.Assert(info, IsNil) + require.NoError(test, err) return nil } - c.Assert(info.Name, DeepEquals, tblInfo.Name) - c.Assert(info.State, Equals, state) + require.Equal(test, info.Name, tblInfo.Name) + require.Equal(test, info.State, state) return nil }) - c.Assert(err, IsNil) + require.NoError(test, err) } -func testGetTable(c *C, d *ddl, schemaID int64, tableID int64) table.Table { +func testGetTable(t *testing.T, d *ddl, schemaID int64, tableID int64) table.Table { tbl, err := testGetTableWithError(d, schemaID, tableID) - c.Assert(err, IsNil) + require.NoError(t, err) return tbl } // for drop indexes -func createTestTableForDropIndexes(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, name string, num int) *model.TableInfo { +func createTestTableForDropIndexes(t *testing.T, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, name string, num int) *model.TableInfo { tableInfo, err := testTableInfo(d, name, num) - c.Assert(err, IsNil) + require.NoError(t, err) var idxs []*model.IndexInfo for i := 0; i < num; i++ { idxName := model.NewCIStr(fmt.Sprintf("i%d", i+1)) @@ -252,6 +253,6 @@ func createTestTableForDropIndexes(c *C, ctx sessionctx.Context, d *ddl, dbInfo idxs = append(idxs, idx) } tableInfo.Indices = idxs - testCreateTable(c, ctx, d, dbInfo, tableInfo) + testCreateTable(t, ctx, d, dbInfo, tableInfo) return tableInfo } diff --git a/tools/check/check_testSuite.sh b/tools/check/check_testSuite.sh index efe24decbc..5387811e4c 100755 --- a/tools/check/check_testSuite.sh +++ b/tools/check/check_testSuite.sh @@ -17,7 +17,7 @@ set -euo pipefail exitCode=0 -list=$(find . -name "*_test.go" -not -path "./vendor/*" -print0 | xargs -0 grep -E "type test(.*)Suite" | awk -F ':| ' '{print $1" "$3}') +list=$(find . -name "*_test.go" -not -path "./vendor/*" -print0 | xargs -0 grep -E "type test(.*)Suite\s+" | awk -F ':| ' '{print $1" "$3}') while read -r file testSuite; do # TODO: ugly regex # TODO: check code comment diff --git a/util/testutil/testutil.go b/util/testutil/testutil.go index 4a49636743..e081f0b458 100644 --- a/util/testutil/testutil.go +++ b/util/testutil/testutil.go @@ -151,6 +151,15 @@ func (chs *CommonHandleSuite) RerunWithCommonHandleEnabled(c *check.C, f func(*c } } +// RerunWithCommonHandleEnabledWithoutCheck runs a test function with IsCommonHandle enabled but without check. +func (chs *CommonHandleSuite) RerunWithCommonHandleEnabledWithoutCheck(f func()) { + if !chs.IsCommonHandle { + chs.IsCommonHandle = true + f() + chs.IsCommonHandle = false + } +} + // NewHandle create a handle according to CommonHandleSuite.IsCommonHandle. func (chs *CommonHandleSuite) NewHandle() *commonHandleSuiteNewHandleBuilder { return &commonHandleSuiteNewHandleBuilder{isCommon: chs.IsCommonHandle}