Skip to content

Commit

Permalink
Merge pull request pingcap#1033 from pingcap/zimuxia/change-terror
Browse files Browse the repository at this point in the history
*: Unfiy error format
  • Loading branch information
zimulala committed Mar 31, 2016
2 parents f773610 + aaa7333 commit 690de9a
Show file tree
Hide file tree
Showing 31 changed files with 335 additions and 207 deletions.
4 changes: 2 additions & 2 deletions ddl/column_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,7 @@ func testCreateColumn(c *C, ctx context.Context, d *ddl, dbInfo *model.DBInfo, t
Args: []interface{}{col, pos, 0},
}

err = d.startDDLJob(ctx, job)
err = d.doDDLJob(ctx, job)
c.Assert(err, IsNil)
return job
}
Expand All @@ -93,7 +93,7 @@ func testDropColumn(c *C, ctx context.Context, d *ddl, dbInfo *model.DBInfo, tbl
Args: []interface{}{model.NewCIStr(colName)},
}

err := d.startDDLJob(ctx, job)
err := d.doDDLJob(ctx, job)
if isError {
c.Assert(err, NotNil)
return nil
Expand Down
86 changes: 43 additions & 43 deletions ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,41 @@ import (
"github.com/twinj/uuid"
)

var (
// errWorkerClosed means we have already closed the DDL worker.
errInvalidWorker = terror.ClassDDL.New(codeInvalidWorker, "invalid worker")
// errNotOwner means we are not owner and can't handle DDL jobs.
errNotOwner = terror.ClassDDL.New(codeNotOwner, "not Owner")
errInvalidDDLJob = terror.ClassDDL.New(codeInvalidDDLJob, "invalid ddl job")
errInvalidBgJob = terror.ClassDDL.New(codeInvalidBgJob, "invalid background job")
errInvalidJobFlag = terror.ClassDDL.New(codeInvalidJobFlag, "invalid job flag")
errRunMultiSchemaChanges = terror.ClassDDL.New(codeRunMultiSchemaChanges, "can't run multi schema change")
errWaitReorgTimeout = terror.ClassDDL.New(codeWaitReorgTimeout, "wait for reorganization timeout")
errInvalidStoreVer = terror.ClassDDL.New(codeInvalidStoreVer, "invalid storage current version")

// we don't support drop column with index covered now.
errCantDropColWithIndex = terror.ClassDDL.New(codeCantDropColWithIndex, "can't drop column with index")
errUnsupportedAddColumn = terror.ClassDDL.New(codeUnsupportedAddColumn, "unsupported add column")

// ErrInvalidDBState returns for invalid database state.
ErrInvalidDBState = terror.ClassDDL.New(codeInvalidDBState, "invalid database state")
// ErrInvalidTableState returns for invalid Table state.
ErrInvalidTableState = terror.ClassDDL.New(codeInvalidTableState, "invalid table state")
// ErrInvalidColumnState returns for invalid column state.
ErrInvalidColumnState = terror.ClassDDL.New(codeInvalidColumnState, "invalid column state")
// ErrInvalidIndexState returns for invalid index state.
ErrInvalidIndexState = terror.ClassDDL.New(codeInvalidIndexState, "invalid index state")

// ErrColumnBadNull returns for a bad null value.
ErrColumnBadNull = terror.ClassDDL.New(codeBadNull, "column cann't be null")
// ErrCantRemoveAllFields returns for deleting all columns.
ErrCantRemoveAllFields = terror.ClassDDL.New(codeCantRemoveAllFields, "can't delete all columns with ALTER TABLE")
// ErrCantDropFieldOrKey returns for dropping a non-existent field or key.
ErrCantDropFieldOrKey = terror.ClassDDL.New(codeCantDropFieldOrKey, "can't drop field; check that column/key exists")
// ErrInvalidOnUpdate returns for invalid ON UPDATE clause.
ErrInvalidOnUpdate = terror.ClassDDL.New(codeInvalidOnUpdate, "invalid ON UPDATE clause for the column")
)

// DDL is responsible for updating schema in data store and maintaining in-memory InfoSchema cache.
type DDL interface {
CreateSchema(ctx context.Context, name model.CIStr, charsetInfo *ast.CharsetOpt) error
Expand Down Expand Up @@ -275,7 +310,7 @@ func (d *ddl) CreateSchema(ctx context.Context, schema model.CIStr, charsetInfo
Args: []interface{}{dbInfo},
}

err = d.startDDLJob(ctx, job)
err = d.doDDLJob(ctx, job)
err = d.hook.OnChanged(err)
return errors.Trace(err)
}
Expand All @@ -292,7 +327,7 @@ func (d *ddl) DropSchema(ctx context.Context, schema model.CIStr) (err error) {
Type: model.ActionDropSchema,
}

err = d.startDDLJob(ctx, job)
err = d.doDDLJob(ctx, job)
err = d.hook.OnChanged(err)
return errors.Trace(err)
}
Expand Down Expand Up @@ -737,7 +772,7 @@ func (d *ddl) CreateTable(ctx context.Context, ident ast.Ident, colDefs []*ast.C
Args: []interface{}{tbInfo},
}

err = d.startDDLJob(ctx, job)
err = d.doDDLJob(ctx, job)
if err == nil {
err = d.handleTableOptions(options, tbInfo, schema.ID)
}
Expand Down Expand Up @@ -854,7 +889,7 @@ func (d *ddl) AddColumn(ctx context.Context, ti ast.Ident, spec *ast.AlterTableS
Args: []interface{}{&col.ColumnInfo, spec.Position, 0},
}

err = d.startDDLJob(ctx, job)
err = d.doDDLJob(ctx, job)
err = d.hook.OnChanged(err)
return errors.Trace(err)
}
Expand Down Expand Up @@ -885,7 +920,7 @@ func (d *ddl) DropColumn(ctx context.Context, ti ast.Ident, colName model.CIStr)
Args: []interface{}{colName},
}

err = d.startDDLJob(ctx, job)
err = d.doDDLJob(ctx, job)
err = d.hook.OnChanged(err)
return errors.Trace(err)
}
Expand All @@ -909,7 +944,7 @@ func (d *ddl) DropTable(ctx context.Context, ti ast.Ident) (err error) {
Type: model.ActionDropTable,
}

err = d.startDDLJob(ctx, job)
err = d.doDDLJob(ctx, job)
err = d.hook.OnChanged(err)
return errors.Trace(err)
}
Expand Down Expand Up @@ -937,7 +972,7 @@ func (d *ddl) CreateIndex(ctx context.Context, ti ast.Ident, unique bool, indexN
Args: []interface{}{unique, indexName, indexID, idxColNames},
}

err = d.startDDLJob(ctx, job)
err = d.doDDLJob(ctx, job)
err = d.hook.OnChanged(err)
return errors.Trace(err)
}
Expand All @@ -961,7 +996,7 @@ func (d *ddl) DropIndex(ctx context.Context, ti ast.Ident, indexName model.CIStr
Args: []interface{}{indexName},
}

err = d.startDDLJob(ctx, job)
err = d.doDDLJob(ctx, job)
err = d.hook.OnChanged(err)
return errors.Trace(err)
}
Expand Down Expand Up @@ -1003,41 +1038,6 @@ const (
codeInvalidOnUpdate = 1294
)

var (
// errWorkerClosed means we have already closed the DDL worker.
errInvalidWorker = terror.ClassDDL.New(codeInvalidWorker, "invalid worker")
// errNotOwner means we are not owner and can't handle DDL jobs.
errNotOwner = terror.ClassDDL.New(codeNotOwner, "not Owner")
errInvalidDDLJob = terror.ClassDDL.New(codeInvalidDDLJob, "invalid ddl job")
errInvalidBgJob = terror.ClassDDL.New(codeInvalidBgJob, "invalid background job")
errInvalidJobFlag = terror.ClassDDL.New(codeInvalidJobFlag, "invalid job flag")
errRunMultiSchemaChanges = terror.ClassDDL.New(codeRunMultiSchemaChanges, "can't run multi schema change")
errWaitReorgTimeout = terror.ClassDDL.New(codeWaitReorgTimeout, "wait for reorganization timeout")
errInvalidStoreVer = terror.ClassDDL.New(codeInvalidStoreVer, "invalid storage current version")

// we don't support drop column with index covered now.
errCantDropColWithIndex = terror.ClassDDL.New(codeCantDropColWithIndex, "can't drop column with index")
errUnsupportedAddColumn = terror.ClassDDL.New(codeUnsupportedAddColumn, "unsupported add column")

// ErrInvalidDBState returns for invalid database state.
ErrInvalidDBState = terror.ClassDDL.New(codeInvalidDBState, "invalid database state")
// ErrInvalidTableState returns for invalid Table state.
ErrInvalidTableState = terror.ClassDDL.New(codeInvalidTableState, "invalid table state")
// ErrInvalidColumnState returns for invalid column state.
ErrInvalidColumnState = terror.ClassDDL.New(codeInvalidColumnState, "invalid column state")
// ErrInvalidIndexState returns for invalid index state.
ErrInvalidIndexState = terror.ClassDDL.New(codeInvalidIndexState, "invalid index state")

// ErrColumnBadNull returns for a bad null value.
ErrColumnBadNull = terror.ClassDDL.New(codeBadNull, "column cann't be null")
// ErrCantRemoveAllFields returns for deleting all columns.
ErrCantRemoveAllFields = terror.ClassDDL.New(codeCantRemoveAllFields, "can't delete all columns with ALTER TABLE")
// ErrCantDropFieldOrKey returns for dropping a non-existent field or key.
ErrCantDropFieldOrKey = terror.ClassDDL.New(codeCantDropFieldOrKey, "can't drop field; check that column/key exists")
// ErrInvalidOnUpdate returns for invalid ON UPDATE clause.
ErrInvalidOnUpdate = terror.ClassDDL.New(codeInvalidOnUpdate, "invalid ON UPDATE clause for the column")
)

func init() {
ddlMySQLERrCodes := map[terror.ErrCode]uint16{
codeBadNull: mysql.ErrBadNull,
Expand Down
2 changes: 1 addition & 1 deletion ddl/ddl_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import (
"github.com/pingcap/tidb/terror"
)

func (d *ddl) startDDLJob(ctx context.Context, job *model.Job) error {
func (d *ddl) doDDLJob(ctx context.Context, job *model.Job) error {
// for every DDL, we must commit current transaction.
if err := ctx.FinishTxn(false); err != nil {
return errors.Trace(err)
Expand Down
34 changes: 17 additions & 17 deletions ddl/ddl_worker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -125,7 +125,7 @@ func (s *testDDLSuite) TestSchemaError(c *C) {

ctx := testNewContext(c, d)

err := d.startDDLJob(ctx, job)
err := d.doDDLJob(ctx, job)
c.Assert(err, NotNil)
testCheckJobCancelled(c, d, job)
}
Expand All @@ -148,7 +148,7 @@ func (s *testDDLSuite) TestTableError(c *C) {

ctx := testNewContext(c, d)

err := d.startDDLJob(ctx, job)
err := d.doDDLJob(ctx, job)
c.Assert(err, NotNil)
testCheckJobCancelled(c, d, job)

Expand All @@ -157,7 +157,7 @@ func (s *testDDLSuite) TestTableError(c *C) {
tblInfo := testTableInfo(c, d, "t", 3)
job.Args = []interface{}{tblInfo}

err = d.startDDLJob(ctx, job)
err = d.doDDLJob(ctx, job)
c.Assert(err, NotNil)
testCheckJobCancelled(c, d, job)

Expand All @@ -167,7 +167,7 @@ func (s *testDDLSuite) TestTableError(c *C) {
Type: model.ActionDropTable,
}

err = d.startDDLJob(ctx, job)
err = d.doDDLJob(ctx, job)
c.Assert(err, NotNil)
testCheckJobCancelled(c, d, job)

Expand All @@ -180,7 +180,7 @@ func (s *testDDLSuite) TestTableError(c *C) {
Type: model.ActionDropTable,
}

err = d.startDDLJob(ctx, job)
err = d.doDDLJob(ctx, job)
c.Assert(err, NotNil)
testCheckJobCancelled(c, d, job)

Expand Down Expand Up @@ -217,7 +217,7 @@ func (s *testDDLSuite) TestIndexError(c *C) {
Type: model.ActionAddIndex,
}

err := d.startDDLJob(ctx, job)
err := d.doDDLJob(ctx, job)
c.Assert(err, NotNil)
testCheckJobCancelled(c, d, job)

Expand All @@ -227,7 +227,7 @@ func (s *testDDLSuite) TestIndexError(c *C) {
Type: model.ActionDropIndex,
}

err = d.startDDLJob(ctx, job)
err = d.doDDLJob(ctx, job)
c.Assert(err, NotNil)
testCheckJobCancelled(c, d, job)

Expand All @@ -243,7 +243,7 @@ func (s *testDDLSuite) TestIndexError(c *C) {
Type: model.ActionAddIndex,
Args: []interface{}{1},
}
err = d.startDDLJob(ctx, job)
err = d.doDDLJob(ctx, job)
c.Assert(err, NotNil)
testCheckJobCancelled(c, d, job)

Expand All @@ -253,7 +253,7 @@ func (s *testDDLSuite) TestIndexError(c *C) {
Type: model.ActionAddIndex,
Args: []interface{}{false, model.NewCIStr("t"), []*ast.IndexColName{{Column: &ast.ColumnName{Name: model.NewCIStr("c")}, Length: 256}}},
}
err = d.startDDLJob(ctx, job)
err = d.doDDLJob(ctx, job)
c.Assert(err, NotNil)
testCheckJobCancelled(c, d, job)

Expand All @@ -263,7 +263,7 @@ func (s *testDDLSuite) TestIndexError(c *C) {
Type: model.ActionAddIndex,
Args: []interface{}{false, model.NewCIStr("c1_index"), []*ast.IndexColName{{Column: &ast.ColumnName{Name: model.NewCIStr("c")}, Length: 256}}},
}
err = d.startDDLJob(ctx, job)
err = d.doDDLJob(ctx, job)
c.Assert(err, NotNil)
testCheckJobCancelled(c, d, job)

Expand All @@ -275,7 +275,7 @@ func (s *testDDLSuite) TestIndexError(c *C) {
Type: model.ActionAddIndex,
Args: []interface{}{false, model.NewCIStr("c1_index"), []*ast.IndexColName{{Column: &ast.ColumnName{Name: model.NewCIStr("c1")}, Length: 256}}},
}
err = d.startDDLJob(ctx, job)
err = d.doDDLJob(ctx, job)
c.Assert(err, NotNil)
testCheckJobCancelled(c, d, job)

Expand All @@ -285,7 +285,7 @@ func (s *testDDLSuite) TestIndexError(c *C) {
Type: model.ActionDropIndex,
Args: []interface{}{1},
}
err = d.startDDLJob(ctx, job)
err = d.doDDLJob(ctx, job)
c.Assert(err, NotNil)
testCheckJobCancelled(c, d, job)

Expand All @@ -297,7 +297,7 @@ func (s *testDDLSuite) TestIndexError(c *C) {
Type: model.ActionDropIndex,
Args: []interface{}{model.NewCIStr("c1_index")},
}
err = d.startDDLJob(ctx, job)
err = d.doDDLJob(ctx, job)
c.Assert(err, NotNil)
testCheckJobCancelled(c, d, job)
}
Expand All @@ -319,7 +319,7 @@ func (s *testDDLSuite) TestColumnError(c *C) {
Type: model.ActionAddColumn,
}

err := d.startDDLJob(ctx, job)
err := d.doDDLJob(ctx, job)
c.Assert(err, NotNil)
testCheckJobCancelled(c, d, job)

Expand All @@ -329,7 +329,7 @@ func (s *testDDLSuite) TestColumnError(c *C) {
Type: model.ActionDropColumn,
}

err = d.startDDLJob(ctx, job)
err = d.doDDLJob(ctx, job)
c.Assert(err, NotNil)
testCheckJobCancelled(c, d, job)

Expand Down Expand Up @@ -358,7 +358,7 @@ func (s *testDDLSuite) TestColumnError(c *C) {
Args: []interface{}{col, pos, 0},
}

err = d.startDDLJob(ctx, job)
err = d.doDDLJob(ctx, job)
c.Assert(err, NotNil)
testCheckJobCancelled(c, d, job)

Expand All @@ -369,7 +369,7 @@ func (s *testDDLSuite) TestColumnError(c *C) {
Args: []interface{}{1},
}

err = d.startDDLJob(ctx, job)
err = d.doDDLJob(ctx, job)
c.Assert(err, NotNil)
testCheckJobCancelled(c, d, job)
}
4 changes: 2 additions & 2 deletions ddl/index_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,7 @@ func testCreateIndex(c *C, ctx context.Context, d *ddl, dbInfo *model.DBInfo, tb
Args: []interface{}{unique, model.NewCIStr(indexName), id, []*ast.IndexColName{{Column: &ast.ColumnName{Name: model.NewCIStr(colName)}, Length: 256}}},
}

err = d.startDDLJob(ctx, job)
err = d.doDDLJob(ctx, job)
c.Assert(err, IsNil)
return job
}
Expand All @@ -82,7 +82,7 @@ func testDropIndex(c *C, ctx context.Context, d *ddl, dbInfo *model.DBInfo, tblI
Args: []interface{}{model.NewCIStr(indexName)},
}

err := d.startDDLJob(ctx, job)
err := d.doDDLJob(ctx, job)
c.Assert(err, IsNil)
return job
}
Expand Down
10 changes: 5 additions & 5 deletions ddl/schema_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ func testCreateSchema(c *C, ctx context.Context, d *ddl, dbInfo *model.DBInfo) *
Args: []interface{}{dbInfo},
}

err := d.startDDLJob(ctx, job)
err := d.doDDLJob(ctx, job)
c.Assert(err, IsNil)
return job
}
Expand All @@ -59,7 +59,7 @@ func testDropSchema(c *C, ctx context.Context, d *ddl, dbInfo *model.DBInfo) *mo
Type: model.ActionDropSchema,
}

err := d.startDDLJob(ctx, job)
err := d.doDDLJob(ctx, job)
c.Assert(err, IsNil)
return job
}
Expand Down Expand Up @@ -156,7 +156,7 @@ func (s *testSchemaSuite) TestSchema(c *C) {
Type: model.ActionDropSchema,
}

err := d1.startDDLJob(ctx, job)
err := d1.doDDLJob(ctx, job)
c.Assert(terror.ErrorEqual(err, infoschema.ErrDatabaseNotExists), IsTrue)
}

Expand Down Expand Up @@ -195,7 +195,7 @@ func (s *testSchemaSuite) TestSchemaWaitJob(c *C) {
Args: []interface{}{dbInfo},
}

err = d2.startDDLJob(ctx, job)
err = d2.doDDLJob(ctx, job)
c.Assert(err, NotNil)
testCheckJobCancelled(c, d2, job)

Expand All @@ -207,7 +207,7 @@ func testRunInterruptedJob(c *C, d *ddl, job *model.Job) {
ctx := mock.NewContext()
done := make(chan error, 1)
go func() {
done <- d.startDDLJob(ctx, job)
done <- d.doDDLJob(ctx, job)
}()

ticker := time.NewTicker(d.lease * 1)
Expand Down
Loading

0 comments on commit 690de9a

Please sign in to comment.