diff --git a/ddl/cancel_test.go b/ddl/cancel_test.go index fddd7fb220d85..57f449713c2f4 100644 --- a/ddl/cancel_test.go +++ b/ddl/cancel_test.go @@ -49,6 +49,7 @@ var allTestCase = []testCancelJob{ {"alter table t add primary key idx_pc2 (c2)", true, model.StateWriteReorganization, true, true, nil}, {"alter table t add primary key idx_pc2 (c2)", false, model.StatePublic, false, true, nil}, // Drop primary key + // TODO: fix schema state. {"alter table t drop primary key", true, model.StateNone, true, false, nil}, {"alter table t drop primary key", false, model.StateWriteOnly, true, false, nil}, {"alter table t drop primary key", false, model.StateWriteOnly, true, false, []string{"alter table t add primary key idx_pc2 (c2)"}}, @@ -73,16 +74,37 @@ var allTestCase = []testCancelJob{ // Create table. {"create table test_create_table(a int)", true, model.StateNone, true, false, nil}, {"create table test_create_table(a int)", false, model.StatePublic, false, true, nil}, + // Drop table. + // TODO: fix schema state. + {"drop table test_create_table", true, model.StateNone, true, false, nil}, + {"drop table test_create_table", false, model.StateWriteOnly, true, true, []string{"create table if not exists test_create_table(a int)"}}, + {"drop table test_create_table", false, model.StateDeleteOnly, true, true, []string{"create table if not exists test_create_table(a int)"}}, + {"drop table test_create_table", false, model.StatePublic, false, true, []string{"create table if not exists test_create_table(a int)"}}, // Create schema. {"create database test_create_db", true, model.StateNone, true, false, nil}, {"create database test_create_db", false, model.StatePublic, false, true, nil}, + // Drop schema. + // TODO: fix schema state. + {"drop database test_create_db", true, model.StateNone, true, false, nil}, + {"drop database test_create_db", false, model.StateWriteOnly, true, true, []string{"create database if not exists test_create_db"}}, + {"drop database test_create_db", false, model.StateDeleteOnly, true, true, []string{"create database if not exists test_create_db"}}, + {"drop database test_create_db", false, model.StatePublic, false, true, []string{"create database if not exists test_create_db"}}, // Drop column. + // TODO: fix schema state. {"alter table t drop column c3", true, model.StateNone, true, false, nil}, {"alter table t drop column c3", false, model.StateDeleteOnly, true, false, nil}, {"alter table t drop column c3", false, model.StateDeleteOnly, false, true, []string{"alter table t add column c3 bigint"}}, {"alter table t drop column c3", false, model.StateWriteOnly, true, true, []string{"alter table t add column c3 bigint"}}, {"alter table t drop column c3", false, model.StateDeleteReorganization, true, true, []string{"alter table t add column c3 bigint"}}, {"alter table t drop column c3", false, model.StatePublic, false, true, []string{"alter table t add column c3 bigint"}}, + // Drop column with index. + // TODO: fix schema state. + {"alter table t drop column c3", true, model.StateNone, true, false, []string{"alter table t add column c3 bigint", "alter table t add index idx_c3(c3)"}}, + {"alter table t drop column c3", false, model.StateDeleteOnly, true, false, nil}, + {"alter table t drop column c3", false, model.StateDeleteOnly, false, true, []string{"alter table t add column c3 bigint", "alter table t add index idx_c3(c3)"}}, + {"alter table t drop column c3", false, model.StateWriteOnly, true, true, []string{"alter table t add column c3 bigint", "alter table t add index idx_c3(c3)"}}, + {"alter table t drop column c3", false, model.StateDeleteReorganization, true, true, []string{"alter table t add column c3 bigint", "alter table t add index idx_c3(c3)"}}, + {"alter table t drop column c3", false, model.StatePublic, false, true, []string{"alter table t add column c3 bigint", "alter table t add index idx_c3(c3)"}}, // rebase auto ID. {"alter table t_rebase auto_increment = 6000", true, model.StateNone, true, false, []string{"create table t_rebase (c1 bigint auto_increment primary key, c2 bigint);"}}, {"alter table t_rebase auto_increment = 9000", false, model.StatePublic, false, true, nil}, @@ -92,6 +114,10 @@ var allTestCase = []testCancelJob{ // Modify column, no reorg. {"alter table t modify column c11 mediumint", true, model.StateNone, true, false, nil}, {"alter table t modify column c11 int", false, model.StatePublic, false, true, nil}, + // TODO: test cancel during second model.StateNone + {"alter table t modify column mayNullCol bigint default 1 not null", true, model.StateNone, true, false, []string{"alter table t add column mayNullCol bigint default 1"}}, + {"alter table t modify column mayNullCol bigint default 1 not null", true, model.StateNone, false, true, nil}, + {"alter table t modify column mayNullCol bigint default 1 not null", false, model.StatePublic, false, true, nil}, // Modify column, reorg. {"alter table t modify column c11 char(10)", true, model.StateNone, true, false, nil}, {"alter table t modify column c11 char(10)", true, model.StateDeleteOnly, true, true, nil}, @@ -102,6 +128,7 @@ var allTestCase = []testCancelJob{ {"alter table t add constraint fk foreign key a(c1) references t_ref(c1)", true, model.StateNone, true, false, []string{"create table t_ref (c1 int, c2 int, c3 int, c11 tinyint);"}}, {"alter table t add constraint fk foreign key a(c1) references t_ref(c1)", false, model.StatePublic, false, true, nil}, // Drop foreign key. + // TODO: fix schema state. {"alter table t drop foreign key fk", true, model.StateNone, true, false, nil}, {"alter table t drop foreign key fk", false, model.StatePublic, false, true, nil}, // Rename table. @@ -126,12 +153,21 @@ var allTestCase = []testCancelJob{ {"alter table t add column c41 bigint, add column c42 bigint", true, model.StateWriteReorganization, true, true, nil}, {"alter table t add column c41 bigint, add column c42 bigint", false, model.StatePublic, false, true, nil}, // Drop columns. + // TODO: fix schema state. {"alter table t drop column c41, drop column c42", true, model.StateNone, true, false, nil}, {"alter table t drop column c41, drop column c42", false, model.StateDeleteOnly, true, false, nil}, {"alter table t drop column c41, drop column c42", false, model.StateDeleteOnly, false, true, []string{"alter table t add column c41 bigint, add column c42 bigint"}}, {"alter table t drop column c41, drop column c42", false, model.StateWriteOnly, true, true, []string{"alter table t add column c41 bigint, add column c42 bigint"}}, {"alter table t drop column c41, drop column c42", false, model.StateDeleteReorganization, true, true, []string{"alter table t add column c41 bigint, add column c42 bigint"}}, {"alter table t drop column c41, drop column c42", false, model.StatePublic, false, true, []string{"alter table t add column c41 bigint, add column c42 bigint"}}, + // Drop columns with index. + // TODO: fix schema state. + {"alter table t drop column c41, drop column c42", true, model.StateNone, true, false, []string{"alter table t add column c41 bigint, add column c42 bigint", "alter table t add index drop_columns_idx(c41)"}}, + {"alter table t drop column c41, drop column c42", false, model.StateDeleteOnly, true, false, nil}, + {"alter table t drop column c41, drop column c42", false, model.StateDeleteOnly, false, true, []string{"alter table t add column c41 bigint, add column c42 bigint", "alter table t add index drop_columns_idx(c41)"}}, + {"alter table t drop column c41, drop column c42", false, model.StateWriteOnly, true, true, []string{"alter table t add column c41 bigint, add column c42 bigint", "alter table t add index drop_columns_idx(c41)"}}, + {"alter table t drop column c41, drop column c42", false, model.StateDeleteReorganization, true, true, []string{"alter table t add column c41 bigint, add column c42 bigint", "alter table t add index drop_columns_idx(c41)"}}, + {"alter table t drop column c41, drop column c42", false, model.StatePublic, false, true, []string{"alter table t add column c41 bigint, add column c42 bigint", "alter table t add index drop_columns_idx(c41)"}}, // Alter index visibility. {"alter table t alter index idx_v invisible", true, model.StateNone, true, false, []string{"alter table t add index idx_v(c1)"}}, {"alter table t alter index idx_v invisible", false, model.StatePublic, false, true, nil}, @@ -142,7 +178,15 @@ var allTestCase = []testCancelJob{ {"alter table t_partition add partition (partition p6 values less than (8192))", true, model.StateNone, true, false, nil}, {"alter table t_partition add partition (partition p6 values less than (8192))", true, model.StateReplicaOnly, true, true, nil}, {"alter table t_partition add partition (partition p6 values less than (8192))", false, model.StatePublic, false, true, nil}, + // Drop partition. + // TODO: fix schema state. + {"alter table t_partition drop partition p6", true, model.StateNone, true, false, nil}, + {"alter table t_partition drop partition p6", false, model.StateDeleteOnly, true, false, nil}, + {"alter table t_partition drop partition p6", false, model.StateDeleteOnly, false, true, []string{"alter table t_partition add partition (partition p6 values less than (8192))"}}, + {"alter table t_partition drop partition p6", false, model.StateDeleteReorganization, true, true, []string{"alter table t_partition add partition (partition p6 values less than (8192))"}}, + {"alter table t_partition drop partition p6", false, model.StatePublic, true, true, []string{"alter table t_partition add partition (partition p6 values less than (8192))"}}, // Drop indexes. + // TODO: fix schema state. {"alter table t drop index mul_idx1, drop index mul_idx2", true, model.StateNone, true, false, []string{"alter table t add index mul_idx1(c1)", "alter table t add index mul_idx2(c1)"}}, {"alter table t drop index mul_idx1, drop index mul_idx2", false, model.StateWriteOnly, true, false, nil}, {"alter table t drop index mul_idx1, drop index mul_idx2", false, model.StateWriteOnly, true, false, []string{"alter table t add index mul_idx1(c1)", "alter table t add index mul_idx2(c1)"}}, @@ -153,6 +197,9 @@ var allTestCase = []testCancelJob{ // Alter db placement. {"alter database db_placement placement policy = 'alter_x'", true, model.StateNone, true, false, []string{"create placement policy alter_x PRIMARY_REGION=\"cn-east-1\", REGIONS=\"cn-east-1\";", "create database db_placement"}}, {"alter database db_placement placement policy = 'alter_x'", false, model.StatePublic, false, true, nil}, + // Rename index. + {"alter table t rename index rename_idx1 to rename_idx2", true, model.StateNone, true, false, []string{"alter table t add index rename_idx1(c1)"}}, + {"alter table t rename index rename_idx1 to rename_idx2", false, model.StatePublic, false, true, nil}, } func cancelSuccess(rs *testkit.Result) bool { @@ -195,10 +242,11 @@ func TestCancel(t *testing.T) { hook := &ddl.TestDDLCallback{Do: dom} i := 0 cancel := false + cancelWhenReorgNotStart := false hookFunc := func(job *model.Job) { if job.SchemaState == allTestCase[i].cancelState && !cancel { - if job.SchemaState == model.StateWriteReorganization && job.MayNeedReorg() && job.RowCount == 0 { + if !cancelWhenReorgNotStart && job.SchemaState == model.StateWriteReorganization && job.MayNeedReorg() && job.RowCount == 0 { return } rs := tkCancel.MustQuery(fmt.Sprintf("admin cancel ddl jobs %d", job.ID)) @@ -229,6 +277,7 @@ func TestCancel(t *testing.T) { } cancel = false + cancelWhenReorgNotStart = true registHook(hook, true) logutil.BgLogger().Info("test case", zap.Int("", i)) if tc.ok { @@ -244,6 +293,7 @@ func TestCancel(t *testing.T) { } cancel = false + cancelWhenReorgNotStart = false registHook(hook, false) logutil.BgLogger().Info("test case", zap.Int("", i)) if tc.ok { diff --git a/ddl/column_modify_test.go b/ddl/column_modify_test.go index d0dedc8667818..3b6a754e484ad 100644 --- a/ddl/column_modify_test.go +++ b/ddl/column_modify_test.go @@ -39,7 +39,6 @@ import ( "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/external" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/mock" "github.com/stretchr/testify/require" ) @@ -440,199 +439,6 @@ func TestRenameColumn(t *testing.T) { tk.MustExec("drop table test_rename_column") } -// TestCancelDropColumn tests cancel ddl job which type is drop column. -func TestCancelDropColumn(t *testing.T) { - store, dom, clean := testkit.CreateMockStoreAndDomainWithSchemaLease(t, columnModifyLease) - defer clean() - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - - tk.MustExec("create table test_drop_column(c1 int, c2 int)") - defer tk.MustExec("drop table test_drop_column;") - testCases := []struct { - needAddColumn bool - jobState model.JobState - JobSchemaState model.SchemaState - cancelSucc bool - }{ - {true, model.JobStateQueueing, model.StateNone, true}, - {false, model.JobStateRunning, model.StateWriteOnly, false}, - {true, model.JobStateRunning, model.StateDeleteOnly, false}, - {true, model.JobStateRunning, model.StateDeleteReorganization, false}, - } - var checkErr error - hook := &ddl.TestDDLCallback{Do: dom} - var jobID int64 - testCase := &testCases[0] - hook.OnJobRunBeforeExported = func(job *model.Job) { - if job.Type == model.ActionDropColumn && job.State == testCase.jobState && job.SchemaState == testCase.JobSchemaState { - jobIDs := []int64{job.ID} - jobID = job.ID - hookCtx := mock.NewContext() - hookCtx.Store = store - err := hookCtx.NewTxn(context.TODO()) - if err != nil { - checkErr = errors.Trace(err) - return - } - txn, err := hookCtx.Txn(true) - if err != nil { - checkErr = errors.Trace(err) - return - } - errs, err := admin.CancelJobs(txn, jobIDs) - if err != nil { - checkErr = errors.Trace(err) - return - } - if errs[0] != nil { - checkErr = errors.Trace(errs[0]) - return - } - checkErr = txn.Commit(context.Background()) - } - } - - originalHook := dom.DDL().GetHook() - dom.DDL().SetHook(hook) - for i := range testCases { - testCase = &testCases[i] - if testCase.needAddColumn { - tk.MustExec("alter table test_drop_column add column c3 int") - tk.MustExec("alter table test_drop_column add index idx_c3(c3)") - } - - err := tk.ExecToErr("alter table test_drop_column drop column c3") - var col1 *table.Column - var idx1 table.Index - tbl := external.GetTableByName(t, tk, "test", "test_drop_column") - for _, col := range tbl.Cols() { - if strings.EqualFold(col.Name.L, "c3") { - col1 = col - break - } - } - for _, idx := range tbl.Indices() { - if strings.EqualFold(idx.Meta().Name.L, "idx_c3") { - idx1 = idx - break - } - } - if testCase.cancelSucc { - require.NoError(t, checkErr) - require.NotNil(t, col1) - require.Equal(t, "c3", col1.Name.L) - require.NotNil(t, idx1) - require.Equal(t, "idx_c3", idx1.Meta().Name.L) - require.EqualError(t, err, "[ddl:8214]Cancelled DDL job") - } else { - require.Nil(t, col1) - require.Nil(t, col1) - require.NoError(t, err) - require.EqualError(t, checkErr, admin.ErrCannotCancelDDLJob.GenWithStackByArgs(jobID).Error()) - } - } - dom.DDL().SetHook(originalHook) - tk.MustExec("alter table test_drop_column add column c3 int") - tk.MustExec("alter table test_drop_column drop column c3") -} - -// TestCancelDropColumns tests cancel ddl job which type is drop multi-columns. -func TestCancelDropColumns(t *testing.T) { - store, dom, clean := testkit.CreateMockStoreAndDomainWithSchemaLease(t, columnModifyLease) - defer clean() - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - - tk.MustExec("create table test_drop_column(c1 int, c2 int)") - defer tk.MustExec("drop table test_drop_column;") - testCases := []struct { - needAddColumn bool - jobState model.JobState - JobSchemaState model.SchemaState - cancelSucc bool - }{ - {true, model.JobStateQueueing, model.StateNone, true}, - {false, model.JobStateRunning, model.StateWriteOnly, false}, - {true, model.JobStateRunning, model.StateDeleteOnly, false}, - {true, model.JobStateRunning, model.StateDeleteReorganization, false}, - } - var checkErr error - hook := &ddl.TestDDLCallback{Do: dom} - var jobID int64 - testCase := &testCases[0] - hook.OnJobRunBeforeExported = func(job *model.Job) { - if job.Type == model.ActionDropColumns && job.State == testCase.jobState && job.SchemaState == testCase.JobSchemaState { - jobIDs := []int64{job.ID} - jobID = job.ID - hookCtx := mock.NewContext() - hookCtx.Store = store - err := hookCtx.NewTxn(context.TODO()) - if err != nil { - checkErr = errors.Trace(err) - return - } - txn, err := hookCtx.Txn(true) - if err != nil { - checkErr = errors.Trace(err) - return - } - errs, err := admin.CancelJobs(txn, jobIDs) - if err != nil { - checkErr = errors.Trace(err) - return - } - if errs[0] != nil { - checkErr = errors.Trace(errs[0]) - return - } - checkErr = txn.Commit(context.Background()) - } - } - - originalHook := dom.DDL().GetHook() - dom.DDL().SetHook(hook) - for i := range testCases { - testCase = &testCases[i] - if testCase.needAddColumn { - tk.MustExec("alter table test_drop_column add column c3 int, add column c4 int") - tk.MustExec("alter table test_drop_column add index idx_c3(c3)") - } - err := tk.ExecToErr("alter table test_drop_column drop column c3, drop column c4") - tbl := external.GetTableByName(t, tk, "test", "test_drop_column") - col3 := table.FindCol(tbl.Cols(), "c3") - col4 := table.FindCol(tbl.Cols(), "c4") - var idx3 table.Index - for _, idx := range tbl.Indices() { - if strings.EqualFold(idx.Meta().Name.L, "idx_c3") { - idx3 = idx - break - } - } - if testCase.cancelSucc { - require.NoError(t, checkErr) - require.NotNil(t, col3) - require.NotNil(t, col4) - require.NotNil(t, idx3) - require.Equal(t, "c3", col3.Name.L) - require.Equal(t, "c4", col4.Name.L) - require.Equal(t, "idx_c3", idx3.Meta().Name.L) - require.EqualError(t, err, "[ddl:8214]Cancelled DDL job") - } else { - require.Nil(t, col3) - require.Nil(t, col4) - require.Nil(t, idx3) - require.NoError(t, err) - require.EqualError(t, checkErr, admin.ErrCannotCancelDDLJob.GenWithStackByArgs(jobID).Error()) - } - } - dom.DDL().SetHook(originalHook) - tk.MustExec("alter table test_drop_column add column c3 int, add column c4 int") - tk.MustExec("alter table test_drop_column drop column c3, drop column c4") -} - func TestVirtualColumnDDL(t *testing.T) { store, clean := testkit.CreateMockStoreWithSchemaLease(t, columnModifyLease) defer clean() diff --git a/ddl/db_rename_test.go b/ddl/db_rename_test.go index b84d22e63a31c..0ed7570a62193 100644 --- a/ddl/db_rename_test.go +++ b/ddl/db_rename_test.go @@ -15,21 +15,14 @@ package ddl_test import ( - "context" "fmt" - "strings" "testing" - "github.com/pingcap/errors" "github.com/pingcap/tidb/config" - "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/testkit/external" - "github.com/pingcap/tidb/util/admin" - "github.com/pingcap/tidb/util/mock" "github.com/stretchr/testify/require" ) @@ -58,61 +51,6 @@ func TestRenameIndex(t *testing.T) { tk.MustGetErrCode("alter table t rename key k3 to K2", errno.ErrDupKeyName) } -// TestCancelRenameIndex tests cancel ddl job which type is rename index. -func TestCancelRenameIndex(t *testing.T) { - store, dom, clean := testkit.CreateMockStoreAndDomain(t) - defer clean() - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("create database if not exists test_rename_index") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(c1 int, c2 int)") - defer tk.MustExec("drop table t;") - for i := 0; i < 100; i++ { - tk.MustExec("insert into t values (?, ?)", i, i) - } - tk.MustExec("alter table t add index idx_c2(c2)") - var checkErr error - hook := &ddl.TestDDLCallback{Do: dom} - hook.OnJobRunBeforeExported = func(job *model.Job) { - if job.Type == model.ActionRenameIndex && job.State == model.JobStateQueueing { - jobIDs := []int64{job.ID} - hookCtx := mock.NewContext() - hookCtx.Store = store - err := hookCtx.NewTxn(context.Background()) - if err != nil { - checkErr = errors.Trace(err) - return - } - txn, err := hookCtx.Txn(true) - if err != nil { - checkErr = errors.Trace(err) - return - } - errs, err := admin.CancelJobs(txn, jobIDs) - if err != nil { - checkErr = errors.Trace(err) - return - } - if errs[0] != nil { - checkErr = errors.Trace(errs[0]) - return - } - checkErr = txn.Commit(context.Background()) - } - } - originalHook := dom.DDL().GetHook() - dom.DDL().SetHook(hook) - tk.MustGetErrMsg("alter table t rename index idx_c2 to idx_c3", "[ddl:8214]Cancelled DDL job") - require.NoError(t, checkErr) - dom.DDL().SetHook(originalHook) - tt := external.GetTableByName(t, tk, "test", "t") - for _, idx := range tt.Indices() { - require.False(t, strings.EqualFold(idx.Meta().Name.L, "idx_c3")) - } - tk.MustExec("alter table t rename index idx_c2 to idx_c3") -} - // See issue: https://github.com/pingcap/tidb/issues/29752 // Ref https://dev.mysql.com/doc/refman/8.0/en/rename-table.html func TestRenameTableWithLocked(t *testing.T) { diff --git a/ddl/db_table_test.go b/ddl/db_table_test.go index 146aa01e38608..46126f73c2320 100644 --- a/ddl/db_table_test.go +++ b/ddl/db_table_test.go @@ -40,108 +40,9 @@ import ( "github.com/pingcap/tidb/testkit/external" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" - "github.com/pingcap/tidb/util/admin" - "github.com/pingcap/tidb/util/mock" "github.com/stretchr/testify/require" ) -// TestCancelDropTable tests cancel ddl job which type is drop table. -func TestCancelDropTableAndSchema(t *testing.T) { - store, dom, clean := testkit.CreateMockStoreAndDomain(t) - defer clean() - tk := testkit.NewTestKit(t, store) - testCases := []struct { - needAddTableOrDB bool - action model.ActionType - jobState model.JobState - JobSchemaState model.SchemaState - cancelSucc bool - }{ - // Check drop table. - // model.JobStateNone means the jobs is canceled before the first run. - {true, model.ActionDropTable, model.JobStateQueueing, model.StateNone, true}, - {false, model.ActionDropTable, model.JobStateRunning, model.StateWriteOnly, false}, - {true, model.ActionDropTable, model.JobStateRunning, model.StateDeleteOnly, false}, - - // Check drop database. - {true, model.ActionDropSchema, model.JobStateQueueing, model.StateNone, true}, - {false, model.ActionDropSchema, model.JobStateRunning, model.StateWriteOnly, false}, - {true, model.ActionDropSchema, model.JobStateRunning, model.StateDeleteOnly, false}, - } - var checkErr error - hook := &ddl.TestDDLCallback{Do: dom} - var jobID int64 - testCase := &testCases[0] - tk.MustExec("create database if not exists test_drop_db") - dbInfo, ok := dom.InfoSchema().SchemaByName(model.NewCIStr("test_drop_db")) - require.True(t, ok) - - hook.OnJobRunBeforeExported = func(job *model.Job) { - if job.Type == testCase.action && job.State == testCase.jobState && job.SchemaState == testCase.JobSchemaState && job.SchemaID == dbInfo.ID { - jobIDs := []int64{job.ID} - jobID = job.ID - hookCtx := mock.NewContext() - hookCtx.Store = store - err := hookCtx.NewTxn(context.TODO()) - if err != nil { - checkErr = errors.Trace(err) - return - } - txn, err := hookCtx.Txn(true) - if err != nil { - checkErr = errors.Trace(err) - return - } - errs, err := admin.CancelJobs(txn, jobIDs) - if err != nil { - checkErr = errors.Trace(err) - return - } - if errs[0] != nil { - checkErr = errors.Trace(errs[0]) - return - } - checkErr = txn.Commit(context.Background()) - } - } - originHook := dom.DDL().GetHook() - defer dom.DDL().SetHook(originHook) - dom.DDL().SetHook(hook) - var err error - sql := "" - for i := range testCases { - testCase = &testCases[i] - if testCase.needAddTableOrDB { - tk.MustExec("create database if not exists test_drop_db") - tk.MustExec("use test_drop_db") - tk.MustExec("create table if not exists t(c1 int, c2 int)") - } - - dbInfo, ok = dom.InfoSchema().SchemaByName(model.NewCIStr("test_drop_db")) - require.True(t, ok) - - if testCase.action == model.ActionDropTable { - sql = "drop table t;" - } else if testCase.action == model.ActionDropSchema { - sql = "drop database test_drop_db;" - } - - _, err = tk.Exec(sql) - if testCase.cancelSucc { - require.Nil(t, checkErr) - require.Error(t, err) - require.Equal(t, "[ddl:8214]Cancelled DDL job", err.Error()) - tk.MustExec("insert into t values (?, ?)", i, i) - } else { - require.NoError(t, err) - require.NotNil(t, checkErr) - require.Equal(t, admin.ErrCannotCancelDDLJob.GenWithStackByArgs(jobID).Error(), checkErr.Error()) - _, err = tk.Exec("insert into t values (?, ?)", i, i) - require.Error(t, err) - } - } -} - func TestTableForeignKey(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() @@ -412,107 +313,6 @@ func TestCreateTableWithEnumCol(t *testing.T) { tk.MustQuery("select * from t_enum").Check(testkit.Rows("c")) } -// TestCancelAddTableAndDropTablePartition tests cancel ddl job which type is add/drop table partition. -func TestCancelAddTableAndDropTablePartition(t *testing.T) { - store, dom, clean := testkit.CreateMockStoreAndDomain(t) - defer clean() - tk := testkit.NewTestKit(t, store) - tk.MustExec("create database if not exists test_partition_table") - tk.MustExec("use test_partition_table") - tk.MustExec("drop table if exists t_part") - tk.MustExec(`create table t_part (a int key) - partition by range(a) ( - partition p0 values less than (10), - partition p1 values less than (20) - );`) - defer tk.MustExec("drop table t_part;") - base := 10 - for i := 0; i < base; i++ { - tk.MustExec("insert into t_part values (?)", i) - } - - testCases := []struct { - action model.ActionType - jobState model.JobState - JobSchemaState model.SchemaState - cancelSucc bool - }{ - {model.ActionAddTablePartition, model.JobStateQueueing, model.StateNone, true}, - {model.ActionDropTablePartition, model.JobStateQueueing, model.StateNone, true}, - // Add table partition now can be cancelled in ReplicaOnly state. - {model.ActionAddTablePartition, model.JobStateRunning, model.StateReplicaOnly, true}, - } - var checkErr error - hook := &ddl.TestDDLCallback{Do: dom} - testCase := &testCases[0] - var jobID int64 - hook.OnJobRunBeforeExported = func(job *model.Job) { - if job.Type == testCase.action && job.State == testCase.jobState && job.SchemaState == testCase.JobSchemaState { - jobIDs := []int64{job.ID} - jobID = job.ID - hookCtx := mock.NewContext() - hookCtx.Store = store - err := hookCtx.NewTxn(context.Background()) - if err != nil { - checkErr = errors.Trace(err) - return - } - txn, err := hookCtx.Txn(true) - if err != nil { - checkErr = errors.Trace(err) - return - } - errs, err := admin.CancelJobs(txn, jobIDs) - if err != nil { - checkErr = errors.Trace(err) - return - } - if errs[0] != nil { - checkErr = errors.Trace(errs[0]) - return - } - checkErr = txn.Commit(context.Background()) - } - } - originalHook := dom.DDL().GetHook() - dom.DDL().SetHook(hook) - - var err error - sql := "" - for i := range testCases { - testCase = &testCases[i] - if testCase.action == model.ActionAddTablePartition { - sql = `alter table t_part add partition ( - partition p2 values less than (30) - );` - } else if testCase.action == model.ActionDropTablePartition { - sql = "alter table t_part drop partition p1;" - } - _, err = tk.Exec(sql) - if testCase.cancelSucc { - require.Nil(t, checkErr) - require.Error(t, err) - require.Equal(t, "[ddl:8214]Cancelled DDL job", err.Error()) - tk.MustExec("insert into t_part values (?)", i+base) - - ctx := tk.Session() - is := domain.GetDomain(ctx).InfoSchema() - tbl, err := is.TableByName(model.NewCIStr("test_partition_table"), model.NewCIStr("t_part")) - require.NoError(t, err) - partitionInfo := tbl.Meta().GetPartitionInfo() - require.NotNil(t, partitionInfo) - require.Len(t, partitionInfo.AddingDefinitions, 0) - } else { - require.NoError(t, err) - require.NoError(t, checkErr) - require.Equal(t, admin.ErrCannotCancelDDLJob.GenWithStackByArgs(jobID).Error(), checkErr.Error()) - tk.MustExec("insert into t_part values (?)", i) - - } - } - dom.DDL().SetHook(originalHook) -} - func TestAlterTableWithValidation(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() diff --git a/ddl/index_modify_test.go b/ddl/index_modify_test.go index 5b5aedbe2b163..ca1b9033aa892 100644 --- a/ddl/index_modify_test.go +++ b/ddl/index_modify_test.go @@ -40,10 +40,8 @@ import ( "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/external" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/dbterror" - "github.com/pingcap/tidb/util/mock" "github.com/stretchr/testify/require" ) @@ -675,63 +673,6 @@ func TestAddIndexWithPK(t *testing.T) { } } -// TestCancelAddIndex1 tests canceling ddl job when the add index worker is not started. -func TestCancelAddIndex1(t *testing.T) { - store, dom, clean := testkit.CreateMockStoreAndDomainWithSchemaLease(t, indexModifyLease) - defer clean() - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("create table t(c1 int, c2 int)") - for i := 0; i < 50; i++ { - tk.MustExec("insert into t values (?, ?)", i, i) - } - - var checkErr error - hook := &ddl.TestDDLCallback{Do: dom} - hook.OnJobRunBeforeExported = func(job *model.Job) { - if job.Type == model.ActionAddIndex && job.State == model.JobStateRunning && job.SchemaState == model.StateWriteReorganization && job.SnapshotVer == 0 { - jobIDs := []int64{job.ID} - hookCtx := mock.NewContext() - hookCtx.Store = store - err := hookCtx.NewTxn(context.Background()) - if err != nil { - checkErr = errors.Trace(err) - return - } - txn, err := hookCtx.Txn(true) - if err != nil { - checkErr = errors.Trace(err) - return - } - errs, err := admin.CancelJobs(txn, jobIDs) - if err != nil { - checkErr = errors.Trace(err) - return - } - - if errs[0] != nil { - checkErr = errors.Trace(errs[0]) - return - } - - checkErr = txn.Commit(context.Background()) - } - } - originalHook := dom.DDL().GetHook() - dom.DDL().SetHook(hook) - err := tk.ExecToErr("alter table t add index idx_c2(c2)") - require.NoError(t, checkErr) - require.EqualError(t, err, "[ddl:8214]Cancelled DDL job") - - dom.DDL().SetHook(originalHook) - tbl := external.GetTableByName(t, tk, "test", "t") - for _, idx := range tbl.Indices() { - require.False(t, strings.EqualFold(idx.Meta().Name.L, "idx_c2")) - } - tk.MustExec("alter table t add index idx_c2(c2)") - tk.MustExec("alter table t drop index idx_c2") -} - func TestAddGlobalIndex(t *testing.T) { defer config.RestoreFunc()() config.UpdateGlobal(func(conf *config.Config) { @@ -865,7 +806,7 @@ func checkGlobalIndexRow( } func TestDropIndexes(t *testing.T) { - store, dom, clean := testkit.CreateMockStoreAndDomainWithSchemaLease(t, indexModifyLease) + store, clean := testkit.CreateMockStoreWithSchemaLease(t, indexModifyLease) defer clean() // drop multiple indexes createSQL := "create table test_drop_indexes (id int, c1 int, c2 int, primary key(id), key i1(c1), key i2(c2));" @@ -885,7 +826,6 @@ func TestDropIndexes(t *testing.T) { testDropIndexesIfExists(t, store) testDropIndexesFromPartitionedTable(t, store) - testCancelDropIndexes(t, store, dom.DDL()) } func testDropIndexes(t *testing.T, store kv.Storage, createSQL, dropIdxSQL string, idxNames []string) { @@ -975,100 +915,6 @@ func testDropIndexesFromPartitionedTable(t *testing.T, store kv.Storage) { tk.MustExec("alter table test_drop_indexes_from_partitioned_table drop column c1, drop column if exists c1;") } -func testCancelDropIndexes(t *testing.T, store kv.Storage, d ddl.DDL) { - indexesName := []string{"idx_c1", "idx_c2"} - addIdxesSQL := "alter table t add index idx_c1 (c1);alter table t add index idx_c2 (c2);" - dropIdxesSQL := "alter table t drop index idx_c1;alter table t drop index idx_c2;" - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(c1 int, c2 int)") - defer tk.MustExec("drop table t;") - for i := 0; i < 5; i++ { - tk.MustExec("insert into t values (?, ?)", i, i) - } - testCases := []struct { - needAddIndex bool - jobState model.JobState - JobSchemaState model.SchemaState - cancelSucc bool - }{ - // model.JobStateNone means the jobs is canceled before the first run. - // if we cancel successfully, we need to set needAddIndex to false in the next test case. Otherwise, set needAddIndex to true. - {true, model.JobStateQueueing, model.StateNone, true}, - {false, model.JobStateRunning, model.StateWriteOnly, false}, - {true, model.JobStateRunning, model.StateDeleteOnly, false}, - {true, model.JobStateRunning, model.StateDeleteReorganization, false}, - } - var checkErr error - hook := &ddl.TestDDLCallback{} - var jobID int64 - testCase := &testCases[0] - hook.OnJobRunBeforeExported = func(job *model.Job) { - if (job.Type == model.ActionDropIndex || job.Type == model.ActionDropPrimaryKey) && - job.State == testCase.jobState && job.SchemaState == testCase.JobSchemaState { - jobID = job.ID - jobIDs := []int64{job.ID} - hookCtx := mock.NewContext() - hookCtx.Store = store - err := hookCtx.NewTxn(context.TODO()) - if err != nil { - checkErr = errors.Trace(err) - return - } - txn, err := hookCtx.Txn(true) - if err != nil { - checkErr = errors.Trace(err) - return - } - - errs, err := admin.CancelJobs(txn, jobIDs) - if err != nil { - checkErr = errors.Trace(err) - return - } - if errs[0] != nil { - checkErr = errors.Trace(errs[0]) - return - } - checkErr = txn.Commit(context.Background()) - } - } - originalHook := d.GetHook() - d.SetHook(hook) - for i := range testCases { - testCase = &testCases[i] - if testCase.needAddIndex { - tk.MustExec(addIdxesSQL) - } - err := tk.ExecToErr(dropIdxesSQL) - tbl := external.GetTableByName(t, tk, "test", "t") - - var indexInfos []*model.IndexInfo - for _, idxName := range indexesName { - indexInfo := tbl.Meta().FindIndexByName(idxName) - if indexInfo != nil { - indexInfos = append(indexInfos, indexInfo) - } - } - - if testCase.cancelSucc { - require.NoError(t, checkErr) - require.EqualError(t, err, "[ddl:8214]Cancelled DDL job") - require.NotNil(t, indexInfos) - require.Equal(t, model.StatePublic, indexInfos[0].State) - } else { - require.NoError(t, err) - require.EqualError(t, checkErr, admin.ErrCannotCancelDDLJob.GenWithStackByArgs(jobID).Error()) - require.Nil(t, indexInfos) - } - } - d.SetHook(originalHook) - tk.MustExec(addIdxesSQL) - tk.MustExec(dropIdxesSQL) -} - func TestDropPrimaryKey(t *testing.T) { store, clean := testkit.CreateMockStoreWithSchemaLease(t, indexModifyLease) defer clean() diff --git a/ddl/modify_column_test.go b/ddl/modify_column_test.go index 8c68b74b25aad..bc2929de45eba 100644 --- a/ddl/modify_column_test.go +++ b/ddl/modify_column_test.go @@ -21,7 +21,6 @@ import ( "testing" "time" - "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/errno" @@ -30,10 +29,8 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/external" - "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/mock" "github.com/stretchr/testify/require" ) @@ -908,82 +905,3 @@ func TestModifyColumnTypeWhenInterception(t *testing.T) { res := tk.MustQuery("show warnings") require.Len(t, res.Rows(), count) } - -func TestModifyColumnRollBack(t *testing.T) { - store, dom, clean := testkit.CreateMockStoreAndDomain(t) - defer clean() - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("create table t1 (c1 int, c2 int, c3 int default 1, index (c1))") - - var c2 *table.Column - var checkErr error - hook := &ddl.TestDDLCallback{Do: dom} - hook.OnJobUpdatedExported = func(job *model.Job) { - if checkErr != nil { - return - } - - tbl := external.GetTableByName(t, tk, "test", "t1") - for _, col := range tbl.Cols() { - if col.Name.L == "c2" { - c2 = col - } - } - if mysql.HasPreventNullInsertFlag(c2.Flag) { - tk.MustGetErrCode("insert into t1(c2) values (null);", errno.ErrBadNull) - } - - hookCtx := mock.NewContext() - hookCtx.Store = store - err := hookCtx.NewTxn(context.Background()) - if err != nil { - checkErr = errors.Trace(err) - return - } - - jobIDs := []int64{job.ID} - txn, err := hookCtx.Txn(true) - if err != nil { - checkErr = errors.Trace(err) - return - } - errs, err := admin.CancelJobs(txn, jobIDs) - if err != nil { - checkErr = errors.Trace(err) - return - } - // It only tests cancel one DDL job. - if errs[0] != nil { - checkErr = errors.Trace(errs[0]) - return - } - - txn, err = hookCtx.Txn(true) - if err != nil { - checkErr = errors.Trace(err) - return - } - err = txn.Commit(context.Background()) - if err != nil { - checkErr = errors.Trace(err) - } - } - - dom.DDL().SetHook(hook) - done := make(chan error, 1) - go backgroundExec(store, "alter table test.t1 change c2 c2 bigint not null;", done) - - err := <-done - require.EqualError(t, err, "[ddl:8214]Cancelled DDL job") - tk.MustExec("insert into t1(c2) values (null);") - - tbl := external.GetTableByName(t, tk, "test", "t1") //nolint:typecheck - for _, col := range tbl.Cols() { - if col.Name.L == "c2" { - c2 = col - } - } - require.False(t, mysql.HasNotNullFlag(c2.Flag)) - tk.MustExec("drop table t1") -} diff --git a/ddl/serial_test.go b/ddl/serial_test.go index 73d08a441e8ee..646f37df5dd1a 100644 --- a/ddl/serial_test.go +++ b/ddl/serial_test.go @@ -24,7 +24,6 @@ import ( "testing" "time" - "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl" @@ -45,10 +44,8 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/external" - "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/gcutil" - "github.com/pingcap/tidb/util/mock" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/testutils" ) @@ -443,6 +440,8 @@ func TestCancelAddIndexPanic(t *testing.T) { tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(c1 int, c2 int)") + + tkCancel := testkit.NewTestKit(t, store) defer tk.MustExec("drop table t") for i := 0; i < 5; i++ { tk.MustExec("insert into t values (?, ?)", i, i) @@ -451,37 +450,10 @@ func TestCancelAddIndexPanic(t *testing.T) { oldReorgWaitTimeout := ddl.ReorgWaitTimeout ddl.ReorgWaitTimeout = 50 * time.Millisecond defer func() { ddl.ReorgWaitTimeout = oldReorgWaitTimeout }() - hook := &ddl.TestDDLCallback{} + hook := &ddl.TestDDLCallback{Do: dom} hook.OnJobRunBeforeExported = func(job *model.Job) { if job.Type == model.ActionAddIndex && job.State == model.JobStateRunning && job.SchemaState == model.StateWriteReorganization && job.SnapshotVer != 0 { - jobIDs := []int64{job.ID} - hookCtx := mock.NewContext() - hookCtx.Store = store - err := hookCtx.NewTxn(context.Background()) - if err != nil { - checkErr = errors.Trace(err) - return - } - txn, err := hookCtx.Txn(true) - if err != nil { - checkErr = errors.Trace(err) - return - } - errs, err := admin.CancelJobs(txn, jobIDs) - if err != nil { - checkErr = errors.Trace(err) - return - } - if errs[0] != nil { - checkErr = errors.Trace(errs[0]) - return - } - txn, err = hookCtx.Txn(true) - if err != nil { - checkErr = errors.Trace(err) - return - } - checkErr = txn.Commit(context.Background()) + tkCancel.MustQuery(fmt.Sprintf("admin cancel ddl jobs %d", job.ID)) } } dom.DDL().SetHook(hook) @@ -492,10 +464,7 @@ func TestCancelAddIndexPanic(t *testing.T) { require.NoError(t, checkErr) require.Error(t, err) errMsg := err.Error() - // Cancelling the job can either succeed or not, it depends on whether the cancelled job takes affect. - // For now, there's no way to guarantee that cancelling will always take effect. - // TODO: After issue #17904 is fixed, there is no need to tolerate it here. - require.True(t, strings.HasPrefix(errMsg, "[ddl:8214]Cancelled DDL job") || strings.HasPrefix(errMsg, "[ddl:8211]DDL job rollback")) + require.True(t, strings.HasPrefix(errMsg, "[ddl:8214]Cancelled DDL job")) } func TestRecoverTableByJobID(t *testing.T) {