Skip to content

Commit

Permalink
ddl: Refine the error message of add index, to avoid error overwrit…
Browse files Browse the repository at this point in the history
…ten (#15808)
  • Loading branch information
tsthght authored Apr 1, 2020
1 parent dec9371 commit 959eca8
Show file tree
Hide file tree
Showing 7 changed files with 31 additions and 24 deletions.
22 changes: 11 additions & 11 deletions ddl/db_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -791,7 +791,7 @@ func (s *testStateChangeSuite) TestParallelChangeColumnName(c *C) {
oneErr = err2
}
}
c.Assert(oneErr.Error(), Equals, "[schema:1060]Duplicate column name 'aa'")
c.Assert(oneErr.Error(), Equals, "[schema:1060]current error msg: Cancelled DDL job, original error msg: Duplicate column name 'aa'")
}
s.testControlParallelExecSQL(c, sql1, sql2, f)
}
Expand All @@ -801,7 +801,7 @@ func (s *testStateChangeSuite) TestParallelAlterAddIndex(c *C) {
sql2 := "CREATE INDEX index_b ON t (c);"
f := func(c *C, err1, err2 error) {
c.Assert(err1, IsNil)
c.Assert(err2.Error(), Equals, "[ddl:1061]index already exist index_b")
c.Assert(err2.Error(), Equals, "[ddl:1061]current error msg: Cancelled DDL job, original error msg: index already exist index_b")
}
s.testControlParallelExecSQL(c, sql1, sql2, f)
}
Expand All @@ -812,7 +812,7 @@ func (s *serialTestStateChangeSuite) TestParallelAlterAddExpressionIndex(c *C) {
sql2 := "CREATE INDEX expr_index_b ON t ((c+1));"
f := func(c *C, err1, err2 error) {
c.Assert(err1, IsNil)
c.Assert(err2.Error(), Equals, "[ddl:1061]index already exist expr_index_b")
c.Assert(err2.Error(), Equals, "[ddl:1061]current error msg: Cancelled DDL job, original error msg: index already exist expr_index_b")
}
s.testControlParallelExecSQL(c, sql1, sql2, f)
}
Expand All @@ -822,7 +822,7 @@ func (s *testStateChangeSuite) TestParallelAddPrimaryKey(c *C) {
sql2 := "ALTER TABLE t add primary key index_b(c);"
f := func(c *C, err1, err2 error) {
c.Assert(err1, IsNil)
c.Assert(err2.Error(), Equals, "[schema:1068]Multiple primary key defined")
c.Assert(err2.Error(), Equals, "[schema:1068]current error msg: Cancelled DDL job, original error msg: Multiple primary key defined")
}
s.testControlParallelExecSQL(c, sql1, sql2, f)
}
Expand All @@ -833,7 +833,7 @@ func (s *testStateChangeSuite) TestParallelAlterAddPartition(c *C) {
);`
f := func(c *C, err1, err2 error) {
c.Assert(err1, IsNil)
c.Assert(err2.Error(), Equals, "[ddl:1493]VALUES LESS THAN value must be strictly increasing for each partition")
c.Assert(err2.Error(), Equals, "[ddl:1493]current error msg: Cancelled DDL job, original error msg: VALUES LESS THAN value must be strictly increasing for each partition")
}
s.testControlParallelExecSQL(c, sql1, sql1, f)
}
Expand All @@ -842,7 +842,7 @@ func (s *testStateChangeSuite) TestParallelDropColumn(c *C) {
sql := "ALTER TABLE t drop COLUMN c ;"
f := func(c *C, err1, err2 error) {
c.Assert(err1, IsNil)
c.Assert(err2.Error(), Equals, "[ddl:1091]column c doesn't exist")
c.Assert(err2.Error(), Equals, "[ddl:1091]current error msg: Cancelled DDL job, original error msg: column c doesn't exist")
}
s.testControlParallelExecSQL(c, sql, sql, f)
}
Expand All @@ -852,7 +852,7 @@ func (s *testStateChangeSuite) TestParallelDropIndex(c *C) {
sql2 := "alter table t drop index idx2 ;"
f := func(c *C, err1, err2 error) {
c.Assert(err1, IsNil)
c.Assert(err2.Error(), Equals, "[autoid:1075]Incorrect table definition; there can be only one auto column and it must be defined as a key")
c.Assert(err2.Error(), Equals, "[autoid:1075]current error msg: Cancelled DDL job, original error msg: Incorrect table definition; there can be only one auto column and it must be defined as a key")
}
s.testControlParallelExecSQL(c, sql1, sql2, f)
}
Expand All @@ -866,7 +866,7 @@ func (s *testStateChangeSuite) TestParallelDropPrimaryKey(c *C) {
sql2 := "alter table t drop primary key;"
f := func(c *C, err1, err2 error) {
c.Assert(err1, IsNil)
c.Assert(err2.Error(), Equals, "[ddl:1091]index PRIMARY doesn't exist")
c.Assert(err2.Error(), Equals, "[ddl:1091]current error msg: Cancelled DDL job, original error msg: index PRIMARY doesn't exist")
}
s.testControlParallelExecSQL(c, sql1, sql2, f)
}
Expand All @@ -877,7 +877,7 @@ func (s *testStateChangeSuite) TestParallelCreateAndRename(c *C) {
defer s.se.Execute(context.Background(), "drop table t_exists")
f := func(c *C, err1, err2 error) {
c.Assert(err1, IsNil)
c.Assert(err2.Error(), Equals, "[schema:1050]Table 't_exists' already exists")
c.Assert(err2.Error(), Equals, "[schema:1050]current error msg: Cancelled DDL job, original error msg: Table 't_exists' already exists")
}
s.testControlParallelExecSQL(c, sql1, sql2, f)
}
Expand Down Expand Up @@ -1017,7 +1017,7 @@ func (s *testStateChangeSuite) TestParallelUpdateTableReplica(c *C) {
}()
wg.Wait()
c.Assert(err1, IsNil)
c.Assert(err2.Error(), Equals, "[ddl:-1]the replica available status of table t1 is already updated")
c.Assert(err2.Error(), Equals, "[ddl:-1]current error msg: Cancelled DDL job, original error msg: the replica available status of table t1 is already updated")
}

func (s *testStateChangeSuite) testParallelExecSQL(c *C, sql string) {
Expand Down Expand Up @@ -1273,7 +1273,7 @@ func (s *serialTestStateChangeSuite) TestParallelFlashbackTable(c *C) {
f := func(c *C, err1, err2 error) {
c.Assert(err1, IsNil)
c.Assert(err2, NotNil)
c.Assert(err2.Error(), Equals, "[schema:1050]Table 't_flashback' already exists")
c.Assert(err2.Error(), Equals, "[schema:1050]current error msg: Cancelled DDL job, original error msg: Table 't_flashback' already exists")

}
s.testControlParallelExecSQL(c, sql1, sql1, f)
Expand Down
16 changes: 8 additions & 8 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -233,7 +233,7 @@ func (s *testDBSuite5) TestAddPrimaryKeyRollback1(c *C) {
hasNullValsInKey := false
idxName := "PRIMARY"
addIdxSQL := "alter table t1 add primary key c3_index (c3);"
errMsg := "[kv:1062]Duplicate entry '' for key 'PRIMARY'"
errMsg := "[kv:1062]current error msg: Cancelled DDL job, original error msg: Duplicate entry '' for key 'PRIMARY'"
testAddIndexRollback(c, s.store, s.lease, idxName, addIdxSQL, errMsg, hasNullValsInKey)
}

Expand All @@ -242,15 +242,15 @@ func (s *testDBSuite1) TestAddPrimaryKeyRollback2(c *C) {
hasNullValsInKey := true
idxName := "PRIMARY"
addIdxSQL := "alter table t1 add primary key c3_index (c3);"
errMsg := "[ddl:1138]Invalid use of NULL value"
errMsg := "[ddl:1138]current error msg: Cancelled DDL job, original error msg: Invalid use of NULL value"
testAddIndexRollback(c, s.store, s.lease, idxName, addIdxSQL, errMsg, hasNullValsInKey)
}

func (s *testDBSuite2) TestAddUniqueIndexRollback(c *C) {
hasNullValsInKey := false
idxName := "c3_index"
addIdxSQL := "create unique index c3_index on t1 (c3)"
errMsg := "[kv:1062]Duplicate entry '' for key 'c3_index'"
errMsg := "[kv:1062]current error msg: Cancelled DDL job, original error msg: Duplicate entry '' for key 'c3_index'"
testAddIndexRollback(c, s.store, s.lease, idxName, addIdxSQL, errMsg, hasNullValsInKey)
}

Expand Down Expand Up @@ -278,7 +278,7 @@ func (s *testSerialDBSuite) TestAddExpressionIndexRollback(c *C) {
}
d.(ddl.DDLForTest).SetHook(hook)

tk.MustGetErrMsg("alter table t1 add index expr_idx ((pow(c1, c2)));", "[ddl:8202]Cannot decode index value, because [types:1690]DOUBLE value is out of range in 'pow(160, 160)'")
tk.MustGetErrMsg("alter table t1 add index expr_idx ((pow(c1, c2)));", "[ddl:8202]current error msg: Cancelled DDL job, original error msg: Cannot decode index value, because [types:1690]DOUBLE value is out of range in 'pow(160, 160)'")
c.Assert(checkErr, IsNil)
tk.MustQuery("select * from t1;").Check(testkit.Rows("20 20 20", "80 80 80", "160 160 160"))
}
Expand Down Expand Up @@ -1804,7 +1804,7 @@ func (s *testDBSuite6) TestDropColumn(c *C) {
s.tk.MustExec("create table t1 (a int,b int) partition by hash(a) partitions 4;")
_, err := s.tk.Exec("alter table t1 drop column a")
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "[expression:1054]Unknown column 'a' in 'expression'")
c.Assert(err.Error(), Equals, "[expression:1054]current error msg: Cancelled DDL job, original error msg: Unknown column 'a' in 'expression'")

s.tk.MustExec("drop database drop_col_db")
}
Expand Down Expand Up @@ -3272,7 +3272,7 @@ func (s *testDBSuite1) TestModifyColumnNullToNotNull(c *C) {
_, err := s.tk.Exec("alter table t1 change c2 c2 int not null;")
c.Assert(checkErr, IsNil)
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "[ddl:1138]Invalid use of NULL value")
c.Assert(err.Error(), Equals, "[ddl:1138]current error msg: Cancelled DDL job, original error msg: Invalid use of NULL value")
s.tk.MustQuery("select * from t1").Check(testkit.Rows("<nil> <nil>"))

// Check insert error when column has PreventNullInsertFlag.
Expand Down Expand Up @@ -3571,7 +3571,7 @@ func (s *testDBSuite5) TestAddIndexForGeneratedColumn(c *C) {
s.tk.MustExec("insert into t values()")
s.tk.MustExec("ALTER TABLE t ADD COLUMN y1 year as (y + 2)")
_, err := s.tk.Exec("ALTER TABLE t ADD INDEX idx_y(y1)")
c.Assert(err.Error(), Equals, "[ddl:8202]Cannot decode index value, because cannot convert datum from unsigned bigint to type year.")
c.Assert(err.Error(), Equals, "[ddl:8202]current error msg: Cancelled DDL job, original error msg: Cannot decode index value, because cannot convert datum from unsigned bigint to type year.")

t := s.testGetTable(c, "t")
for _, idx := range t.Indices() {
Expand Down Expand Up @@ -4464,7 +4464,7 @@ func (s *testDBSuite2) TestDDLWithInvalidTableInfo(c *C) {
// Test drop partition column.
_, err = tk.Exec("alter table t drop column a;")
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "[expression:1054]Unknown column 'a' in 'expression'")
c.Assert(err.Error(), Equals, "[expression:1054]current error msg: Cancelled DDL job, original error msg: Unknown column 'a' in 'expression'")
// Test modify column with invalid expression.
_, err = tk.Exec("alter table t modify column c int GENERATED ALWAYS AS ((case when (a = 0) then 0when (a > 0) then (b / a) end));")
c.Assert(err, NotNil)
Expand Down
5 changes: 5 additions & 0 deletions ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -534,6 +534,11 @@ func (d *ddl) doDDLJob(ctx sessionctx.Context, job *model.Job) error {
}

if historyJob.Error != nil {
if (historyJob.State == model.JobStateRollbackDone || historyJob.State == model.JobStateCancelled) && !historyJob.Error.Equal(errCancelledDDLJob) {
historyJob.Error = historyJob.Error.Class().New(historyJob.Error.Code(),
fmt.Sprintf("current error msg: %s, original error msg: %s",
errCancelledDDLJob.ToSQLError().Message, historyJob.Error.ToSQLError().Message))
}
return errors.Trace(historyJob.Error)
}
panic("When the state is JobStateRollbackDone or JobStateCancelled, historyJob.Error should never be nil")
Expand Down
2 changes: 1 addition & 1 deletion ddl/ddl_worker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -204,7 +204,7 @@ func (s *testDDLSuite) TestInvalidDDLJob(c *C) {
Args: []interface{}{},
}
err := d.doDDLJob(ctx, job)
c.Assert(err.Error(), Equals, "[ddl:8204]invalid ddl job type: none")
c.Assert(err.Error(), Equals, "[ddl:8204]current error msg: Cancelled DDL job, original error msg: invalid ddl job type: none")
}

func (s *testDDLSuite) TestForeignKeyError(c *C) {
Expand Down
4 changes: 3 additions & 1 deletion ddl/rollingback.go
Original file line number Diff line number Diff line change
Expand Up @@ -311,7 +311,9 @@ func convertJob2RollbackJob(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job)
logutil.Logger(w.logCtx).Info("[ddl] the DDL job is cancelled normally", zap.String("job", job.String()), zap.Error(err))
}

job.Error = toTError(err)
if job.Error == nil {
job.Error = toTError(err)
}
job.ErrorCount++
}
return
Expand Down
4 changes: 2 additions & 2 deletions ddl/serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -514,7 +514,7 @@ func (s *testSerialSuite) TestRecoverTableByJobID(c *C) {
// if GC enable is not exists in mysql.tidb
_, err = tk.Exec(fmt.Sprintf("recover table by job %d", jobID))
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "[ddl:-1]can not get 'tikv_gc_enable'")
c.Assert(err.Error(), Equals, "[ddl:-1]current error msg: Cancelled DDL job, original error msg: can not get 'tikv_gc_enable'")

err = gcutil.EnableGC(tk.Se)
c.Assert(err, IsNil)
Expand Down Expand Up @@ -723,7 +723,7 @@ func (s *testSerialSuite) TestCancelJobByErrorCountLimit(c *C) {

_, err = tk.Exec("create table t (a int)")
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "[ddl:8214]Cancelled DDL job")
c.Assert(err.Error(), Equals, "[ddl:-1]current error msg: Cancelled DDL job, original error msg: mock do job error")
}

func (s *testSerialSuite) TestCanceledJobTakeTime(c *C) {
Expand Down
2 changes: 1 addition & 1 deletion executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4636,7 +4636,7 @@ func (s *testRecoverTable) TestRecoverTable(c *C) {
// if GC enable is not exists in mysql.tidb
_, err = tk.Exec("recover table t_recover")
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "[ddl:-1]can not get 'tikv_gc_enable'")
c.Assert(err.Error(), Equals, "[ddl:-1]current error msg: Cancelled DDL job, original error msg: can not get 'tikv_gc_enable'")

err = gcutil.EnableGC(tk.Se)
c.Assert(err, IsNil)
Expand Down

0 comments on commit 959eca8

Please sign in to comment.