From f7f1edfead7d37d10d153f070414101e98cc0437 Mon Sep 17 00:00:00 2001 From: tangenta Date: Fri, 13 May 2022 18:42:16 +0800 Subject: [PATCH] ddl: fix build and integration tests (#97) --- ddl/cancel_test.go | 86 +++++++++++++++++++++------------ ddl/column_change_test.go | 10 ++-- ddl/ddl_api.go | 16 +++--- ddl/multi_schema_change_test.go | 3 +- 4 files changed, 73 insertions(+), 42 deletions(-) diff --git a/ddl/cancel_test.go b/ddl/cancel_test.go index 199a75346e608..c75ed0f454e81 100644 --- a/ddl/cancel_test.go +++ b/ddl/cancel_test.go @@ -24,18 +24,48 @@ import ( "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) type testCancelJob struct { sql string ok bool - cancelState model.SchemaState + cancelState interface{} // model.SchemaState | []model.SchemaState onJobBefore bool onJobUpdate bool prepareSQL []string } +type subStates = []model.SchemaState + +func testMatchCancelState(t *testing.T, job *model.Job, cancelState interface{}, sql string) bool { + switch v := cancelState.(type) { + case model.SchemaState: + if job.MultiSchemaInfo != nil && job.Type == model.ActionMultiSchemaChange { + msg := fmt.Sprintf("unexpected multi-schema change(sql: %s, cancel state: %s)", sql, v) + assert.Failf(t, msg, "use []model.SchemaState as cancel states instead") + return false + } + return job.SchemaState == v + case []model.SchemaState: // For multi-schema change sub-jobs. + if job.MultiSchemaInfo == nil { + msg := fmt.Sprintf("not multi-schema change(sql: %s, cancel state: %v)", sql, v) + assert.Failf(t, msg, "use model.SchemaState as the cancel state instead") + return false + } + assert.Equal(t, len(job.MultiSchemaInfo.SubJobs), len(v), sql) + for i, subJobSchemaState := range v { + if job.MultiSchemaInfo.SubJobs[i].SchemaState != subJobSchemaState { + return false + } + } + return true + default: + return false + } +} + var allTestCase = []testCancelJob{ // Add index. {"create unique index c3_index on t_partition (c1)", true, model.StateWriteReorganization, true, true, nil}, @@ -139,27 +169,25 @@ var allTestCase = []testCancelJob{ {"alter table t_partition truncate partition p3", true, model.StateNone, true, false, nil}, {"alter table t_partition truncate partition p3", false, model.StatePublic, false, true, nil}, // Add columns. - {"alter table t add column c41 bigint, add column c42 bigint", true, model.StateNone, true, false, nil}, - {"alter table t add column c41 bigint, add column c42 bigint", true, model.StateDeleteOnly, true, true, nil}, - {"alter table t add column c41 bigint, add column c42 bigint", true, model.StateWriteOnly, true, true, nil}, - {"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}, + {"alter table t add column c41 bigint, add column c42 bigint", true, subStates{model.StateNone, model.StateNone}, true, false, nil}, + {"alter table t add column c41 bigint, add column c42 bigint", true, subStates{model.StateDeleteOnly, model.StateNone}, true, true, nil}, + {"alter table t add column c41 bigint, add column c42 bigint", true, subStates{model.StateWriteOnly, model.StateNone}, true, true, nil}, + {"alter table t add column c41 bigint, add column c42 bigint", true, subStates{model.StateWriteReorganization, model.StateNone}, true, true, nil}, + {"alter table t add column c41 bigint, add column c42 bigint", false, subStates{model.StatePublic, 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"}}, + {"alter table t drop column c41, drop column c42", true, subStates{model.StatePublic, model.StatePublic}, true, false, nil}, + {"alter table t drop column c41, drop column c42", false, subStates{model.StateDeleteOnly, model.StateDeleteOnly}, true, false, nil}, + {"alter table t drop column c41, drop column c42", false, subStates{model.StateDeleteOnly, 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, subStates{model.StateWriteOnly, model.StateDeleteOnly}, true, true, []string{"alter table t add column c41 bigint, add column c42 bigint"}}, + {"alter table t drop column c41, drop column c42", false, subStates{model.StateDeleteReorganization, model.StateDeleteOnly}, true, true, []string{"alter table t add column c41 bigint, add column c42 bigint"}}, + {"alter table t drop column c41, drop column c42", false, subStates{model.StateNone, model.StateDeleteOnly}, 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 table t drop column c41, drop column c42", true, subStates{model.StatePublic, model.StatePublic}, 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, subStates{model.StateDeleteOnly, model.StateDeleteOnly}, true, false, nil}, + {"alter table t drop column c41, drop column c42", false, subStates{model.StateDeleteOnly, 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, subStates{model.StateWriteOnly, model.StateDeleteOnly}, 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, subStates{model.StateDeleteReorganization, model.StateDeleteOnly}, 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, subStates{model.StateNone, 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 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}, @@ -177,14 +205,13 @@ var allTestCase = []testCancelJob{ {"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.StateNone, 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)"}}, - {"alter table t drop index mul_idx1, drop index mul_idx2", false, model.StateDeleteOnly, 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.StateDeleteOnly, false, true, []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.StateDeleteReorganization, 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.StateDeleteReorganization, false, true, []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", true, subStates{model.StatePublic, model.StatePublic}, 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, subStates{model.StateWriteOnly, model.StateWriteOnly}, true, false, nil}, + {"alter table t drop index mul_idx1, drop index mul_idx2", false, subStates{model.StateWriteOnly, model.StateWriteOnly}, 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, subStates{model.StateDeleteOnly, model.StateWriteOnly}, 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, subStates{model.StateDeleteOnly, model.StateWriteOnly}, false, true, []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, subStates{model.StateDeleteReorganization, model.StateWriteOnly}, 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, subStates{model.StateDeleteReorganization, model.StateWriteOnly}, false, true, []string{"alter table t add index mul_idx1(c1)", "alter table t add index mul_idx2(c1)"}}, // 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}, @@ -237,7 +264,7 @@ func TestCancel(t *testing.T) { cancelWhenReorgNotStart := false hookFunc := func(job *model.Job) { - if job.SchemaState == allTestCase[i].cancelState && !cancel { + if testMatchCancelState(t, job, allTestCase[i].cancelState, allTestCase[i].sql) && !cancel { if !cancelWhenReorgNotStart && job.SchemaState == model.StateWriteReorganization && job.MayNeedReorg() && job.RowCount == 0 { return } @@ -268,7 +295,6 @@ func TestCancel(t *testing.T) { for _, prepareSQL := range tc.prepareSQL { tk.MustExec(prepareSQL) } - cancel = false cancelWhenReorgNotStart = true registHook(hook, true) diff --git a/ddl/column_change_test.go b/ddl/column_change_test.go index 3680ac7cc3285..7bb140aa5c198 100644 --- a/ddl/column_change_test.go +++ b/ddl/column_change_test.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/testkit/external" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mock" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -96,11 +97,14 @@ func TestColumnAdd(t *testing.T) { } } tc.OnJobUpdatedExported = func(job *model.Job) { + if job.NotStarted() { + return + } jobID = job.ID tbl := external.GetTableByName(t, internal, "test", "t") if job.SchemaState != model.StatePublic { for _, col := range tbl.Cols() { - require.NotEqualf(t, col.ID, dropCol.ID, "column is not dropped") + assert.NotEqualf(t, col.ID, dropCol.ID, "column is not dropped") } } } @@ -224,7 +228,7 @@ func checkAddWriteOnly(ctx sessionctx.Context, deleteOnlyTable, writeOnlyTable t return errors.Trace(err) } err = checkResult(ctx, writeOnlyTable, writeOnlyTable.WritableCols(), [][]string{ - {"1", "2", ""}, + {"1", "2", "3"}, {"2", "3", "3"}, }) if err != nil { @@ -236,7 +240,7 @@ func checkAddWriteOnly(ctx sessionctx.Context, deleteOnlyTable, writeOnlyTable t return errors.Trace(err) } got := fmt.Sprintf("%v", row) - expect := fmt.Sprintf("%v", []types.Datum{types.NewDatum(1), types.NewDatum(2), types.NewDatum(nil)}) + expect := fmt.Sprintf("%v", []types.Datum{types.NewDatum(1), types.NewDatum(2), types.NewDatum(3)}) if got != expect { return errors.Errorf("expect %v, got %v", expect, got) } diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index f1608fc994666..80de2afa086ff 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3942,6 +3942,7 @@ func (d *ddl) DropColumn(ctx sessionctx.Context, ti ast.Ident, spec *ast.AlterTa SchemaID: schema.ID, TableID: t.Meta().ID, SchemaName: schema.Name.L, + SchemaState: model.StatePublic, TableName: t.Meta().Name.L, Type: model.ActionDropColumn, BinlogInfo: &model.HistoryInfo{}, @@ -5882,13 +5883,14 @@ func (d *ddl) DropIndex(ctx sessionctx.Context, ti ast.Ident, indexName model.CI } job := &model.Job{ - SchemaID: schema.ID, - TableID: t.Meta().ID, - SchemaName: schema.Name.L, - TableName: t.Meta().Name.L, - Type: jobTp, - BinlogInfo: &model.HistoryInfo{}, - Args: []interface{}{indexName, ifExists}, + SchemaID: schema.ID, + TableID: t.Meta().ID, + SchemaName: schema.Name.L, + SchemaState: indexInfo.State, + TableName: t.Meta().Name.L, + Type: jobTp, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{indexName, ifExists}, } err = d.DoDDLJob(ctx, job) diff --git a/ddl/multi_schema_change_test.go b/ddl/multi_schema_change_test.go index 4b5794637e696..1eccda57dee31 100644 --- a/ddl/multi_schema_change_test.go +++ b/ddl/multi_schema_change_test.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/util/admin" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -1092,7 +1091,7 @@ func (c *cancelOnceHook) OnJobUpdated(job *model.Job) { c.triggered = true c.cancelErr = kv.RunInNewTxn(context.Background(), c.store, false, func(ctx context.Context, txn kv.Transaction) error { - errs, err := admin.CancelJobs(txn, []int64{job.ID}) + errs, err := ddl.CancelJobs(txn, []int64{job.ID}) if errs[0] != nil { return errs[0] }