diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 417d77445d00b..fd83e306ee14c 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -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), @@ -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) @@ -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) @@ -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", "")) - 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) @@ -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", " ")) - 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) @@ -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);") @@ -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) @@ -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) @@ -1907,19 +1907,19 @@ 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 @@ -1927,7 +1927,7 @@ func TestAlterTableExchangePartition(t *testing.T) { 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) @@ -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) @@ -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) @@ -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 @@ -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 @@ -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)) @@ -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) ( @@ -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. @@ -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));") diff --git a/testkit/mockstore.go b/testkit/mockstore.go index 181d1c609e364..0b2b4ca1c837d 100644 --- a/testkit/mockstore.go +++ b/testkit/mockstore.go @@ -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...) diff --git a/testkit/result.go b/testkit/result.go index 31c896da97460..7ad2575f468d6 100644 --- a/testkit/result.go +++ b/testkit/result.go @@ -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) -}