diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 6710a68bae24c..5b12175f57ade 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -32,7 +32,6 @@ import ( "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" - "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/model" @@ -41,7 +40,6 @@ import ( "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" @@ -49,7 +47,6 @@ import ( "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/mock" "github.com/stretchr/testify/require" - "github.com/tikv/client-go/v2/testutils" ) func TestNoZeroDateMode(t *testing.T) { @@ -1229,86 +1226,6 @@ func TestBitDefaultValue(t *testing.T) { );`) } -func TestBackwardCompatibility(t *testing.T) { - var cluster testutils.Cluster - store, dom, clean := testkit.CreateMockStoreAndDomainWithSchemaLease(t, 600*time.Millisecond, - mockstore.WithClusterInspector(func(c testutils.Cluster) { - mockstore.BootstrapWithSingleStore(c) - cluster = c - })) - defer clean() - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("create table t(a int primary key, b int)") - for i := 0; i < 200; i++ { - tk.MustExec(fmt.Sprintf("insert into t values(%v, %v)", i, i)) - } - - // alter table t add index idx_b(b); - is := dom.InfoSchema() - schemaName := model.NewCIStr("test") - tableName := model.NewCIStr("t") - schema, ok := is.SchemaByName(schemaName) - require.True(t, ok) - tbl, err := is.TableByName(schemaName, tableName) - require.NoError(t, err) - - // Split the table. - tableStart := tablecodec.GenTableRecordPrefix(tbl.Meta().ID) - cluster.SplitKeys(tableStart, tableStart.PrefixNext(), 10) - - indexName := model.NewCIStr("idx_b") - indexPartSpecification := &ast.IndexPartSpecification{ - Column: &ast.ColumnName{ - Schema: schemaName, - Table: tableName, - Name: model.NewCIStr("b"), - }, - Length: types.UnspecifiedLength, - } - indexPartSpecifications := []*ast.IndexPartSpecification{indexPartSpecification} - var indexOption *ast.IndexOption - job := &model.Job{ - SchemaID: schema.ID, - TableID: tbl.Meta().ID, - Type: model.ActionAddIndex, - BinlogInfo: &model.HistoryInfo{}, - Args: []interface{}{false /* unique */, indexName, indexPartSpecifications, indexOption}, - } - txn, err := store.Begin() - require.NoError(t, err) - m := meta.NewMeta(txn) - job.ID, err = m.GenGlobalID() - require.NoError(t, err) - job.Version = 1 - job.StartTS = txn.StartTS() - - // Simulate old TiDB init the add index job, old TiDB will not init the model.Job.ReorgMeta field, - // if we set job.SnapshotVer here, can simulate the behavior. - job.SnapshotVer = txn.StartTS() - err = m.EnQueueDDLJob(job) - require.NoError(t, err) - err = txn.Commit(context.Background()) - require.NoError(t, err) - ticker := time.NewTicker(50 * time.Millisecond) - defer ticker.Stop() - for range ticker.C { - historyJob, err := getHistoryDDLJob(store, job.ID) - require.NoError(t, err) - if historyJob == nil { - continue - } - require.Nil(t, historyJob.Error) - - if historyJob.IsSynced() { - break - } - } - - // finished add index - tk.MustExec("admin check index t idx_b") -} - func getHistoryDDLJob(store kv.Storage, id int64) (*model.Job, error) { var job *model.Job diff --git a/ddl/db_legacy_test.go b/ddl/db_legacy_test.go index 2483cffb5c8fa..705a5c9caa8f2 100644 --- a/ddl/db_legacy_test.go +++ b/ddl/db_legacy_test.go @@ -1956,54 +1956,28 @@ func (s *testSerialDBSuite) TestDDLJobErrorCount(c *C) { tk.MustExec("use test") tk.MustExec("drop table if exists ddl_error_table, new_ddl_error_table") tk.MustExec("create table ddl_error_table(a int)") - is := s.dom.InfoSchema() - schemaName := model.NewCIStr("test") - tableName := model.NewCIStr("ddl_error_table") - schema, ok := is.SchemaByName(schemaName) - c.Assert(ok, IsTrue) - tbl, err := is.TableByName(schemaName, tableName) - c.Assert(err, IsNil) - - newTableName := model.NewCIStr("new_ddl_error_table") - job := &model.Job{ - SchemaID: schema.ID, - TableID: tbl.Meta().ID, - SchemaName: schema.Name.L, - Type: model.ActionRenameTable, - BinlogInfo: &model.HistoryInfo{}, - Args: []interface{}{schema.ID, newTableName}, - } c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/mockErrEntrySizeTooLarge", `return(true)`), IsNil) defer func() { c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/mockErrEntrySizeTooLarge"), IsNil) }() - txn, err := s.store.Begin() - c.Assert(err, IsNil) - t := meta.NewMeta(txn) - job.ID, err = t.GenGlobalID() - c.Assert(err, IsNil) - job.Version = 1 - job.StartTS = txn.StartTS() + var jobID int64 + hook := &ddl.TestDDLCallback{} + hook.OnJobUpdatedExported = func(job *model.Job) { + jobID = job.ID + } + originHook := s.dom.DDL().GetHook() + s.dom.DDL().SetHook(hook) + defer s.dom.DDL().SetHook(originHook) - err = t.EnQueueDDLJob(job) - c.Assert(err, IsNil) - err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + tk.MustGetErrCode("rename table ddl_error_table to new_ddl_error_table", errno.ErrEntryTooLarge) - ticker := time.NewTicker(s.lease) - defer ticker.Stop() - for range ticker.C { - historyJob, err := getHistoryDDLJob(s.store, job.ID) - c.Assert(err, IsNil) - if historyJob == nil { - continue - } - c.Assert(historyJob.ErrorCount, Equals, int64(1), Commentf("%v", historyJob)) - kv.ErrEntryTooLarge.Equal(historyJob.Error) - break - } + historyJob, err := getHistoryDDLJob(s.store, jobID) + c.Assert(err, IsNil) + c.Assert(historyJob, NotNil) + c.Assert(historyJob.ErrorCount, Equals, int64(1), Commentf("%v", historyJob)) + kv.ErrEntryTooLarge.Equal(historyJob.Error) } func (s *testSerialDBSuite) TestCommitTxnWithIndexChange(c *C) {