Skip to content

Commit

Permalink
ddl: rewrite some cancel tests to an unified test (#34157)
Browse files Browse the repository at this point in the history
close #32939
  • Loading branch information
wjhuang2016 committed Apr 25, 2022
1 parent 65f329e commit 9fc73ac
Show file tree
Hide file tree
Showing 7 changed files with 57 additions and 730 deletions.
52 changes: 51 additions & 1 deletion ddl/cancel_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)"}},
Expand All @@ -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},
Expand All @@ -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},
Expand All @@ -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.
Expand All @@ -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},
Expand All @@ -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)"}},
Expand All @@ -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 {
Expand Down Expand Up @@ -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))
Expand Down Expand Up @@ -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 {
Expand All @@ -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 {
Expand Down
194 changes: 0 additions & 194 deletions ddl/column_modify_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
)
Expand Down Expand Up @@ -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()
Expand Down
Loading

0 comments on commit 9fc73ac

Please sign in to comment.