Skip to content

Commit

Permalink
ddl: migrate test-infra to testify for db_partition_test
Browse files Browse the repository at this point in the history
Signed-off-by: Weizhen Wang <wangweizhen@pingcap.com>
  • Loading branch information
hawkingrei committed Mar 3, 2022
1 parent 4c16909 commit f536400
Show file tree
Hide file tree
Showing 3 changed files with 54 additions and 51 deletions.
63 changes: 31 additions & 32 deletions ddl/db_partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -641,7 +641,7 @@ func TestDisableTablePartition(t *testing.T) {
tk.MustExec("drop table if exists t")
tk.MustExec(`create table t (id int) partition by list (id) (
partition p0 values in (1,2),partition p1 values in (3,4));`)
tbl := testGetTableByNameT(t, tk.Session(), "test", "t")
tbl := tk.GetTableByName("test", "t")
require.Nil(t, tbl.Meta().Partition)
_, err := tk.Exec(`alter table t add partition (
partition p4 values in (7),
Expand Down Expand Up @@ -803,7 +803,7 @@ func TestCreateTableWithListPartition(t *testing.T) {
if id == len(validCases)-1 {
tblName = "gen_t"
}
tbl := testGetTableByNameT(t, tk.Session(), "test", tblName)
tbl := tk.GetTableByName("test", tblName)
tblInfo := tbl.Meta()
require.NotNil(t, tblInfo.Partition)
require.True(t, tblInfo.Partition.Enable)
Expand Down Expand Up @@ -1010,7 +1010,7 @@ func TestCreateTableWithListColumnsPartition(t *testing.T) {
for _, sql := range validCases {
tk.MustExec("drop table if exists t")
tk.MustExec(sql)
tbl := testGetTableByNameT(t, tk.Session(), "test", "t")
tbl := tk.GetTableByName("test", "t")
tblInfo := tbl.Meta()
require.NotNil(t, tblInfo.Partition)
require.Equal(t, true, tblInfo.Partition.Enable)
Expand Down Expand Up @@ -1293,10 +1293,10 @@ func TestAlterTableTruncatePartitionByList(t *testing.T) {
partition p3 values in (5,null)
);`)
tk.MustExec(`insert into t values (1),(3),(5),(null)`)
oldTbl := testGetTableByNameT(t, tk.Session(), "test", "t")
oldTbl := tk.GetTableByName("test", "t")
tk.MustExec(`alter table t truncate partition p1`)
tk.MustQuery("select * from t").Check(testkit.Rows("1", "5", "<nil>"))
tbl := testGetTableByNameT(t, tk.Session(), "test", "t")
tbl := tk.GetTableByName("test", "t")
require.NotNil(t, tbl.Meta().Partition)
part := tbl.Meta().Partition
require.True(t, part.Type == model.PartitionTypeList)
Expand Down Expand Up @@ -1326,10 +1326,10 @@ func TestAlterTableTruncatePartitionByListColumns(t *testing.T) {
partition p3 values in ((5,'a'),(null,null))
);`)
tk.MustExec(`insert into t values (1,'a'),(3,'a'),(5,'a'),(null,null)`)
oldTbl := testGetTableByNameT(t, tk.Session(), "test", "t")
oldTbl := tk.GetTableByName("test", "t")
tk.MustExec(`alter table t truncate partition p1`)
tk.MustQuery("select * from t").Check(testkit.Rows("1 a", "5 a", "<nil> <nil>"))
tbl := testGetTableByNameT(t, tk.Session(), "test", "t")
tbl := tk.GetTableByName("test", "t")
require.NotNil(t, tbl.Meta().Partition)
part := tbl.Meta().Partition
require.True(t, part.Type == model.PartitionTypeList)
Expand Down Expand Up @@ -1695,7 +1695,7 @@ func TestDropPartitionWithGlobalIndex(t *testing.T) {
partition p1 values less than (10),
partition p2 values less than (20)
);`)
tt := testGetTableByNameT(t, tk.Session(), "test", "test_global")
tt := tk.GetTableByName("test", "test_global")
pid := tt.Meta().Partition.Definitions[1].ID

tk.MustExec("Alter Table test_global Add Unique Index idx_b (b);")
Expand All @@ -1706,7 +1706,7 @@ func TestDropPartitionWithGlobalIndex(t *testing.T) {
result := tk.MustQuery("select * from test_global;")
result.Sort().Check(testkit.Rows(`1 1 1`, `2 2 2`))

tt = testGetTableByNameT(t, tk.Session(), "test", "test_global")
tt = tk.GetTableByName("test", "test_global")
idxInfo := tt.Meta().FindIndexByName("idx_b")
require.NotNil(t, idxInfo)
cnt := checkGlobalIndexCleanUpDone(t, tk.Session(), tt.Meta(), idxInfo, pid)
Expand Down Expand Up @@ -1889,13 +1889,13 @@ func TestAlterTableExchangePartition(t *testing.T) {
tk.MustExec("alter table e15 set tiflash replica 1;")
tk.MustExec("alter table e16 set tiflash replica 2;")

e15 := testGetTableByNameT(t, tk.Session(), "test", "e15")
e15 := tk.GetTableByName("test", "e15")
partition := e15.Meta().Partition

err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), partition.Definitions[0].ID, true)
require.NoError(t, err)

e16 := testGetTableByNameT(t, tk.Session(), "test", "e16")
e16 := tk.GetTableByName("test", "e16")
err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), e16.Meta().ID, true)
require.NoError(t, err)

Expand All @@ -1907,27 +1907,27 @@ func TestAlterTableExchangePartition(t *testing.T) {
tk.MustExec("alter table e15 set tiflash replica 1;")
tk.MustExec("alter table e16 set tiflash replica 1;")

e15 = testGetTableByNameT(t, tk.Session(), "test", "e15")
e15 = tk.GetTableByName("test", "e15")
partition = e15.Meta().Partition

err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), partition.Definitions[0].ID, true)
require.NoError(t, err)

e16 = testGetTableByNameT(t, tk.Session(), "test", "e16")
e16 = tk.GetTableByName("test", "e16")
err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), e16.Meta().ID, true)
require.NoError(t, err)

tk.MustExec("alter table e15 exchange partition p0 with table e16")

e15 = testGetTableByNameT(t, tk.Session(), "test", "e15")
e15 = tk.GetTableByName("test", "e15")

partition = e15.Meta().Partition

require.NotNil(t, e15.Meta().TiFlashReplica)
require.True(t, e15.Meta().TiFlashReplica.Available)
require.Equal(t, []int64{partition.Definitions[0].ID}, e15.Meta().TiFlashReplica.AvailablePartitionIDs)

e16 = testGetTableByNameT(t, tk.Session(), "test", "e16")
e16 = tk.GetTableByName("test", "e16")
require.NotNil(t, e16.Meta().TiFlashReplica)
require.True(t, e16.Meta().TiFlashReplica.Available)

Expand All @@ -1942,13 +1942,13 @@ func TestAlterTableExchangePartition(t *testing.T) {

tk.MustExec("alter table e16 set tiflash replica 1 location labels 'a', 'b';")

e15 = testGetTableByNameT(t, tk.Session(), "test", "e15")
e15 = tk.GetTableByName("test", "e15")
partition = e15.Meta().Partition

err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), partition.Definitions[0].ID, true)
require.NoError(t, err)

e16 = testGetTableByNameT(t, tk.Session(), "test", "e16")
e16 = tk.GetTableByName("test", "e16")
err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), e16.Meta().ID, true)
require.NoError(t, err)

Expand Down Expand Up @@ -2822,9 +2822,8 @@ func testPartitionCancelAddIndex(t *testing.T, store kv.Storage, d ddl.DDL, leas
originBatchSize := tk.MustQuery("select @@global.tidb_ddl_reorg_batch_size")
// Set batch size to lower try to slow down add-index reorganization, This if for hook to cancel this ddl job.
tk.MustExec("set @@global.tidb_ddl_reorg_batch_size = 32")
ctx := tk.Session()
defer tk.MustExec(fmt.Sprintf("set @@global.tidb_ddl_reorg_batch_size = %v", originBatchSize.Rows()[0][0]))
hook.OnJobUpdatedExported, c3IdxInfo, checkErr = backgroundExecOnJobUpdatedExportedT(t, store, ctx, hook, idxName)
hook.OnJobUpdatedExported, c3IdxInfo, checkErr = backgroundExecOnJobUpdatedExportedT(tk, store, hook, idxName)
originHook := d.GetHook()
defer d.(ddl.DDLForTest).SetHook(originHook)
jobIDExt := wrapJobIDExtCallback(hook)
Expand Down Expand Up @@ -2862,7 +2861,7 @@ LOOP:
tk.MustExec("drop table t1")
}

func backgroundExecOnJobUpdatedExportedT(t *testing.T, store kv.Storage, ctx sessionctx.Context, hook *ddl.TestDDLCallback, idxName string) (
func backgroundExecOnJobUpdatedExportedT(tk *testkit.TestKit, store kv.Storage, hook *ddl.TestDDLCallback, idxName string) (
func(*model.Job), *model.IndexInfo, error) {
var checkErr error
first := true
Expand All @@ -2877,7 +2876,7 @@ func backgroundExecOnJobUpdatedExportedT(t *testing.T, store kv.Storage, ctx ses
if c3IdxInfo.ID != 0 {
return
}
t := testGetTableByNameT(t, ctx, "test", "t1")
t := tk.GetTableByName("test", "t1")
for _, index := range t.Indices() {
if !tables.IsIndexWritable(index) {
continue
Expand Down Expand Up @@ -3051,7 +3050,7 @@ func TestDropSchemaWithPartitionTable(t *testing.T) {
);`)
tk.MustExec("insert into t_part values (1),(2),(11),(12);")
ctx := tk.Session()
tbl := testGetTableByNameT(t, ctx, "test_db_with_partition", "t_part")
tbl := tk.GetTableByName("test_db_with_partition", "t_part")

// check records num before drop database.
recordsNum := getPartitionTableRecordsNum(t, ctx, tbl.(table.PartitionedTable))
Expand Down Expand Up @@ -3341,14 +3340,13 @@ func TestCommitWhenSchemaChange(t *testing.T) {
// Test for table lock.
conf.EnableTableLock = true
conf.Log.SlowThreshold = 10000
conf.TiKVClient.AsyncCommit.SafeWindow = 0
conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0
conf.Experimental.AllowsExpressionIndex = true
})
ddl.SetWaitTimeWhenErrorOccurred(0)
store, clean := testkit.CreateMockStore(t)
store, dom, clean := testkit.CreateMockStoreAndDomainWithoutZeroSchemaLease(t)
defer clean()
dom.SetStatsUpdating(true)
tk := testkit.NewTestKit(t, store)
tk.MustExec("set @@global.tidb_max_delta_schema_count= 4096")
tk.MustExec("use test")
tk.MustExec(`create table schema_change (a int, b timestamp)
partition by range(a) (
Expand All @@ -3373,8 +3371,9 @@ func TestCommitWhenSchemaChange(t *testing.T) {
defer func() {
atomic.StoreUint32(&session.SchemaChangedWithoutRetry, 0)
}()
_, err := tk.Session().Execute(context.Background(), "commit")
require.True(t, domain.ErrInfoSchemaChanged.Equal(err))
_, err := tk.Exec("commit")
require.Error(t, err)
require.Truef(t, domain.ErrInfoSchemaChanged.Equal(err), err.Error())

// Cover a bug that schema validator does not prevent transaction commit when
// the schema has changed on the partitioned table.
Expand Down Expand Up @@ -3635,21 +3634,21 @@ func TestAddPartitionReplicaBiggerThanTiFlashStores(t *testing.T) {
tk.MustExec("use test_partition2")
tk.MustExec("drop table if exists t1")
// Build a tableInfo with replica count = 1 while there is no real tiFlash store.
require.Nil(t, failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`))
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`))
tk.MustExec(`create table t1 (c int) partition by range(c) (
partition p0 values less than (100),
partition p1 values less than (200))`)
tk.MustExec("alter table t1 set tiflash replica 1")
require.Nil(t, failpoint.Disable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount"))
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount"))
// Mock partitions replica as available.
t1 := testGetTableByNameT(t, tk.Session(), "test_partition2", "t1")
t1 := tk.GetTableByName("test_partition2", "t1")
partition := t1.Meta().Partition
require.Equal(t, 2, len(partition.Definitions))
err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), partition.Definitions[0].ID, true)
require.NoError(t, err)
err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), partition.Definitions[1].ID, true)
require.NoError(t, err)
t1 = testGetTableByNameT(t, tk.Session(), "test_partition2", "t1")
t1 = tk.GetTableByName("test_partition2", "t1")
require.True(t, t1.Meta().TiFlashReplica.Available)
// Since there is no real TiFlash store (less than replica count), adding a partition will error here.
err = tk.ExecToErr("alter table t1 add partition (partition p2 values less than (300));")
Expand Down
23 changes: 23 additions & 0 deletions testkit/mockstore.go
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,29 @@ func bootstrap(t testing.TB, store kv.Storage) (*domain.Domain, func()) {
return dom, clean
}

// CreateMockStoreAndDomainWithoutZeroSchemaLease return a new mock kv.Storage and *domain.Domain.
func CreateMockStoreAndDomainWithoutZeroSchemaLease(t testing.TB, opts ...mockstore.MockTiKVStoreOption) (kv.Storage, *domain.Domain, func()) {
store, err := mockstore.NewMockStore(opts...)
require.NoError(t, err)
dom, clean := bootstrapWithoutZeroSchemaLease(t, store)
return store, dom, clean
}

func bootstrapWithoutZeroSchemaLease(t testing.TB, store kv.Storage) (*domain.Domain, func()) {
session.DisableStats4Test()
dom, err := session.BootstrapSession(store)
require.NoError(t, err)

dom.SetStatsUpdating(true)

clean := func() {
dom.Close()
err := store.Close()
require.NoError(t, err)
}
return dom, clean
}

// CreateMockStoreWithOracle returns a new mock kv.Storage and *domain.Domain, providing the oracle for the store.
func CreateMockStoreWithOracle(t testing.TB, oracle oracle.Oracle, opts ...mockstore.MockTiKVStoreOption) (kv.Storage, *domain.Domain, func()) {
store, err := mockstore.NewMockStore(opts...)
Expand Down
19 changes: 0 additions & 19 deletions testkit/result.go
Original file line number Diff line number Diff line change
Expand Up @@ -118,22 +118,3 @@ func (res *Result) Rows() [][]interface{} {
}
return ifacesSlice
}

// CheckAt asserts the result of selected columns equals the expected results.
func (res *Result) CheckAt(cols []int, expected [][]interface{}) {
for _, e := range expected {
res.require.Equal(len(e), len(cols))
}

rows := make([][]string, 0, len(expected))
for i := range res.rows {
row := make([]string, 0, len(cols))
for _, r := range cols {
row = append(row, res.rows[i][r])
}
rows = append(rows, row)
}
got := fmt.Sprintf("%s", rows)
need := fmt.Sprintf("%s", expected)
res.require.Equal(need, got, res.comment)
}

0 comments on commit f536400

Please sign in to comment.