From 4a02009b11a1b861eaf3ba54fad3256e0af393ff Mon Sep 17 00:00:00 2001 From: zhiqiangxu <652732310@qq.com> Date: Sat, 14 Mar 2020 22:06:31 +0800 Subject: [PATCH 1/3] increment ErrorCount when update Error --- ddl/ddl_worker.go | 1 + 1 file changed, 1 insertion(+) diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 83af5c404f70a..7cf04c1fe6af0 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -283,6 +283,7 @@ func (w *worker) handleUpdateJobError(t *meta.Meta, job *model.Job, err error) e // Reduce this txn entry size. job.BinlogInfo.Clean() job.Error = toTError(err) + job.ErrorCount++ job.SchemaState = model.StateNone job.State = model.JobStateCancelled err = w.finishDDLJob(t, job) From 0790097973f026ab57dd6d510a7cf7c7b8a3a8b3 Mon Sep 17 00:00:00 2001 From: Deardrops Date: Thu, 19 Mar 2020 12:46:48 +0800 Subject: [PATCH 2/3] add test case --- ddl/db_integration_test.go | 55 ++++++++++++++++++++++++++++++++++++++ ddl/ddl_worker.go | 5 ++++ 2 files changed, 60 insertions(+) diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 82309bc6ff114..7ab49c11f6bbe 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -22,6 +22,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/charset" "github.com/pingcap/parser/model" @@ -1081,6 +1082,60 @@ func (s *testIntegrationSuite5) TestBackwardCompatibility(c *C) { tk.MustExec("admin check index t idx_b") } +func (s *testIntegrationSuite3) TestDDLErrorCount(c *C) { + tk := testkit.NewTestKit(c, s.store) + 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() + + err = t.EnQueueDDLJob(job) + c.Assert(err, IsNil) + err = txn.Commit(context.Background()) + c.Assert(err, IsNil) + ticker := time.NewTicker(s.lease) + defer ticker.Stop() + for range ticker.C { + historyJob, err := s.getHistoryDDLJob(job.ID) + c.Assert(err, IsNil) + if historyJob == nil { + continue + } + c.Assert(historyJob.ErrorCount, Equals, int64(1)) + kv.ErrEntryTooLarge.Equal(historyJob.Error) + break + } +} + func (s *testIntegrationSuite3) TestMultiRegionGetTableEndHandle(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("drop database if exists test_get_endhandle") diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 7cf04c1fe6af0..6e9b0b1dec514 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -294,6 +294,11 @@ func (w *worker) handleUpdateJobError(t *meta.Meta, job *model.Job, err error) e // updateDDLJob updates the DDL job information. // Every time we enter another state except final state, we must call this function. func (w *worker) updateDDLJob(t *meta.Meta, job *model.Job, meetErr bool) error { + failpoint.Inject("mockErrEntrySizeTooLarge", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(kv.ErrEntryTooLarge) + } + }) updateRawArgs := true // If there is an error when running job and the RawArgs hasn't been decoded by DecodeArgs, // so we shouldn't replace RawArgs with the marshaling Args. From 3469ea1969a3e1302b93aa7e4a50bbc540d3d250 Mon Sep 17 00:00:00 2001 From: Deardrops Date: Thu, 19 Mar 2020 14:43:53 +0800 Subject: [PATCH 3/3] fix test --- ddl/db_integration_test.go | 61 ++------------------------------------ ddl/db_test.go | 56 ++++++++++++++++++++++++++++++++++ 2 files changed, 59 insertions(+), 58 deletions(-) diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 7ab49c11f6bbe..60506b397449a 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -22,7 +22,6 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" - "github.com/pingcap/failpoint" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/charset" "github.com/pingcap/parser/model" @@ -1065,7 +1064,7 @@ func (s *testIntegrationSuite5) TestBackwardCompatibility(c *C) { ticker := time.NewTicker(s.lease) defer ticker.Stop() for range ticker.C { - historyJob, err := s.getHistoryDDLJob(job.ID) + historyJob, err := getHistoryDDLJob(s.store, job.ID) c.Assert(err, IsNil) if historyJob == nil { @@ -1082,60 +1081,6 @@ func (s *testIntegrationSuite5) TestBackwardCompatibility(c *C) { tk.MustExec("admin check index t idx_b") } -func (s *testIntegrationSuite3) TestDDLErrorCount(c *C) { - tk := testkit.NewTestKit(c, s.store) - 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() - - err = t.EnQueueDDLJob(job) - c.Assert(err, IsNil) - err = txn.Commit(context.Background()) - c.Assert(err, IsNil) - ticker := time.NewTicker(s.lease) - defer ticker.Stop() - for range ticker.C { - historyJob, err := s.getHistoryDDLJob(job.ID) - c.Assert(err, IsNil) - if historyJob == nil { - continue - } - c.Assert(historyJob.ErrorCount, Equals, int64(1)) - kv.ErrEntryTooLarge.Equal(historyJob.Error) - break - } -} - func (s *testIntegrationSuite3) TestMultiRegionGetTableEndHandle(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("drop database if exists test_get_endhandle") @@ -1202,10 +1147,10 @@ func checkGetMaxTableRowID(ctx *testMaxTableRowIDContext, store kv.Storage, expe c.Assert(maxID, Equals, expectMaxID) } -func (s *testIntegrationSuite) getHistoryDDLJob(id int64) (*model.Job, error) { +func getHistoryDDLJob(store kv.Storage, id int64) (*model.Job, error) { var job *model.Job - err := kv.RunInNewTxn(s.store, false, func(txn kv.Transaction) error { + err := kv.RunInNewTxn(store, false, func(txn kv.Transaction) error { t := meta.NewMeta(txn) var err1 error job, err1 = t.GetHistoryDDLJob(id) diff --git a/ddl/db_test.go b/ddl/db_test.go index 4735916071c8c..22ea74324dffe 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" @@ -4477,6 +4478,61 @@ func (s *testDBSuite6) TestAlterOrderBy(c *C) { s.tk.MustExec("drop table if exists ob") } +func (s *testSerialDBSuite) TestDDLJobErrorCount(c *C) { + tk := testkit.NewTestKit(c, s.store) + 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() + + err = t.EnQueueDDLJob(job) + c.Assert(err, IsNil) + err = txn.Commit(context.Background()) + c.Assert(err, IsNil) + + 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)) + kv.ErrEntryTooLarge.Equal(historyJob.Error) + break + } +} + func init() { // Make sure it will only be executed once. domain.SchemaOutOfDateRetryInterval = int64(50 * time.Millisecond)