Skip to content

Commit

Permalink
*: unify error format
Browse files Browse the repository at this point in the history
  • Loading branch information
zimulala committed Mar 31, 2016
1 parent f773610 commit 6aacddd
Show file tree
Hide file tree
Showing 19 changed files with 255 additions and 127 deletions.
70 changes: 35 additions & 35 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 @@ -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
11 changes: 10 additions & 1 deletion domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -184,7 +184,7 @@ func (do *Domain) reload() error {
case err := <-done:
return errors.Trace(err)
case <-time.After(timeout):
return errors.New("reload schema timeout")
return errLoadSchemaTimeOut
}
}

Expand Down Expand Up @@ -268,3 +268,12 @@ func NewDomain(store kv.Storage, lease time.Duration) (d *Domain, err error) {

return d, nil
}

// Domain error codes.
const (
codeLoadSchemaTimeOut terror.ErrCode = 1
)

var (
errLoadSchemaTimeOut = terror.ClassDomain.New(codeLoadSchemaTimeOut, "reload schema timeout")
)
44 changes: 22 additions & 22 deletions infoschema/infoschema.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,28 @@ import (
"github.com/pingcap/tidb/util/types"
)

var (
// ErrDatabaseDropExists returns for dropping a non-existent database.
ErrDatabaseDropExists = terror.ClassSchema.New(codeDbDropExists, "database doesn't exist")
// ErrDatabaseNotExists returns for database not exists.
ErrDatabaseNotExists = terror.ClassSchema.New(codeDatabaseNotExists, "database not exists")
// ErrTableNotExists returns for table not exists.
ErrTableNotExists = terror.ClassSchema.New(codeTableNotExists, "table not exists")
// ErrColumnNotExists returns for column not exists.
ErrColumnNotExists = terror.ClassSchema.New(codeColumnNotExists, "field not exists")

// ErrDatabaseExists returns for database already exists.
ErrDatabaseExists = terror.ClassSchema.New(codeDatabaseExists, "database already exists")
// ErrTableExists returns for table already exists.
ErrTableExists = terror.ClassSchema.New(codeTableExists, "table already exists")
// ErrTableDropExists returns for dropping a non-existent table.
ErrTableDropExists = terror.ClassSchema.New(codeBadTable, "unknown table")
// ErrColumnExists returns for column already exists.
ErrColumnExists = terror.ClassSchema.New(codeColumnExists, "Duplicate column")
// ErrIndexExists returns for index already exists.
ErrIndexExists = terror.ClassSchema.New(codeIndexExists, "Duplicate Index")
)

// InfoSchema is the interface used to retrieve the schema information.
// It works as a in memory cache and doesn't handle any schema change.
// InfoSchema is read-only, and the returned value is a copy.
Expand Down Expand Up @@ -469,28 +491,6 @@ const (
codeIndexExists = 1831
)

var (
// ErrDatabaseDropExists returns for dropping a non-existent database.
ErrDatabaseDropExists = terror.ClassSchema.New(codeDbDropExists, "database doesn't exist")
// ErrDatabaseNotExists returns for database not exists.
ErrDatabaseNotExists = terror.ClassSchema.New(codeDatabaseNotExists, "database not exists")
// ErrTableNotExists returns for table not exists.
ErrTableNotExists = terror.ClassSchema.New(codeTableNotExists, "table not exists")
// ErrColumnNotExists returns for column not exists.
ErrColumnNotExists = terror.ClassSchema.New(codeColumnNotExists, "field not exists")

// ErrDatabaseExists returns for database already exists.
ErrDatabaseExists = terror.ClassSchema.New(codeDatabaseExists, "database already exists")
// ErrTableExists returns for table already exists.
ErrTableExists = terror.ClassSchema.New(codeTableExists, "table already exists")
// ErrTableDropExists returns for dropping a non-existent table.
ErrTableDropExists = terror.ClassSchema.New(codeBadTable, "unknown table")
// ErrColumnExists returns for column already exists.
ErrColumnExists = terror.ClassSchema.New(codeColumnExists, "Duplicate column")
// ErrIndexExists returns for index already exists.
ErrIndexExists = terror.ClassSchema.New(codeIndexExists, "Duplicate Index")
)

func init() {
schemaMySQLErrCodes := map[terror.ErrCode]uint16{
codeDbDropExists: mysql.ErrDbDropExists,
Expand Down
31 changes: 22 additions & 9 deletions inspectkv/inspectkv.go
Original file line number Diff line number Diff line change
Expand Up @@ -196,15 +196,15 @@ func checkIndexAndRecord(txn kv.Transaction, t table.Table, idx *column.IndexedC
vals2, err := rowWithCols(txn, t, h, cols)
if terror.ErrorEqual(err, kv.ErrNotExist) {
record := &RecordData{Handle: h, Values: vals1}
err = errors.Errorf("index:%v != record:%v", record, nil)
err = errDateNotEqual.Gen("index:%v != record:%v", record, nil)
}
if err != nil {
return errors.Trace(err)
}
if !reflect.DeepEqual(vals1, vals2) {
record1 := &RecordData{Handle: h, Values: vals1}
record2 := &RecordData{Handle: h, Values: vals2}
return errors.Errorf("index:%v != record:%v", record1, record2)
return errDateNotEqual.Gen("index:%v != record:%v", record1, record2)
}
}

Expand All @@ -224,14 +224,14 @@ func checkRecordAndIndex(txn kv.Transaction, t table.Table, idx *column.IndexedC
if terror.ErrorEqual(err, kv.ErrKeyExists) {
record1 := &RecordData{Handle: h1, Values: vals1}
record2 := &RecordData{Handle: h2, Values: vals1}
return false, errors.Errorf("index:%v != record:%v", record2, record1)
return false, errDateNotEqual.Gen("index:%v != record:%v", record2, record1)
}
if err != nil {
return false, errors.Trace(err)
}
if !isExist {
record := &RecordData{Handle: h1, Values: vals1}
return false, errors.Errorf("index:%v != record:%v", nil, record)
return false, errDateNotEqual.Gen("index:%v != record:%v", nil, record)
}

return true, nil
Expand Down Expand Up @@ -310,7 +310,7 @@ func CompareTableRecord(txn kv.Transaction, t table.Table, data []*RecordData, e
m := make(map[int64][]types.Datum, len(data))
for _, r := range data {
if _, ok := m[r.Handle]; ok {
return errors.Errorf("handle:%d is repeated in data", r.Handle)
return errRepeatHandle.Gen("handle:%d is repeated in data", r.Handle)
}
m[r.Handle] = r.Values
}
Expand All @@ -320,7 +320,7 @@ func CompareTableRecord(txn kv.Transaction, t table.Table, data []*RecordData, e
vals2, ok := m[h]
if !ok {
record := &RecordData{Handle: h, Values: vals}
return false, errors.Errorf("data:%v != record:%v", nil, record)
return false, errDateNotEqual.Gen("data:%v != record:%v", nil, record)
}
if !exact {
delete(m, h)
Expand All @@ -330,7 +330,7 @@ func CompareTableRecord(txn kv.Transaction, t table.Table, data []*RecordData, e
if !reflect.DeepEqual(vals, vals2) {
record1 := &RecordData{Handle: h, Values: vals2}
record2 := &RecordData{Handle: h, Values: vals}
return false, errors.Errorf("data:%v != record:%v", record1, record2)
return false, errDateNotEqual.Gen("data:%v != record:%v", record1, record2)
}

delete(m, h)
Expand All @@ -344,7 +344,7 @@ func CompareTableRecord(txn kv.Transaction, t table.Table, data []*RecordData, e

for h, vals := range m {
record := &RecordData{Handle: h, Values: vals}
return errors.Errorf("data:%v != record:%v", record, nil)
return errDateNotEqual.Gen("data:%v != record:%v", record, nil)
}

return nil
Expand Down Expand Up @@ -386,7 +386,7 @@ func rowWithCols(txn kv.Retriever, t table.Table, h int64, cols []*column.Col) (
v := make([]types.Datum, len(cols))
for i, col := range cols {
if col.State != model.StatePublic {
return nil, errors.Errorf("Cannot use none public column - %v", cols)
return nil, errInvalidColumnState.Gen("Cannot use none public column - %v", cols)
}
if col.IsPKHandleColumn(t.Meta()) {
v[i].SetInt64(h)
Expand Down Expand Up @@ -450,3 +450,16 @@ func iterRecords(retriever kv.Retriever, t table.Table, startKey kv.Key, cols []

return nil
}

// inspectkv error codes.
const (
codeDataNotEqual terror.ErrCode = 1
codeRepeatHandle = 2
codeInvalidColumnState = 3
)

var (
errDateNotEqual = terror.ClassInspectkv.New(codeDataNotEqual, "data isn't equal")
errRepeatHandle = terror.ClassInspectkv.New(codeRepeatHandle, "handle is repeated")
errInvalidColumnState = terror.ClassInspectkv.New(codeInvalidColumnState, "invalid column state")
)
4 changes: 2 additions & 2 deletions inspectkv/inspectkv_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -238,7 +238,7 @@ func (s *testSuite) TestScan(c *C) {
}

func newDiffRetError(prefix string, ra, rb *RecordData) string {
return fmt.Sprintf("%s:%v != record:%v", prefix, ra, rb)
return fmt.Sprintf("[inspectkv:1]%s:%v != record:%v", prefix, ra, rb)
}

func (s *testSuite) testTableData(c *C, tb table.Table, rs []*RecordData) {
Expand Down Expand Up @@ -288,7 +288,7 @@ func (s *testSuite) testTableData(c *C, tb table.Table, rs []*RecordData) {

errRs := append(rs, &RecordData{Handle: int64(1), Values: types.MakeDatums(int64(3))})
err = CompareTableRecord(txn, tb, errRs, false)
c.Assert(err.Error(), DeepEquals, "handle:1 is repeated in data")
c.Assert(err.Error(), DeepEquals, "[inspectkv:2]handle:1 is repeated in data")
}

func (s *testSuite) testIndex(c *C, tb table.Table, idx *column.IndexedCol) {
Expand Down
37 changes: 22 additions & 15 deletions kv/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@
package kv

import (
"errors"
"strings"

"github.com/pingcap/go-themis"
Expand All @@ -24,41 +23,49 @@ import (

// KV error codes.
const (
CodeIncompatibleDBFormat terror.ErrCode = 1
CodeNoDataForHandle terror.ErrCode = 2
CodeKeyExists terror.ErrCode = 3
codeClosed terror.ErrCode = 1
codeNotExist = 2
codeCondithinNotMatch = 3
codeLockConfilict = 4
codeLazyConditionPairsNotMatch = 5
codeRetryable = 6
codeCantSetNilValue = 7
codeInvalidTxn = 8
codeNotCommitted = 9

codeKeyExists = 1062
)

var (
// ErrClosed is used when close an already closed txn.
ErrClosed = errors.New("Error: Transaction already closed")
ErrClosed = terror.ClassKV.New(codeClosed, "Error: Transaction already closed")
// ErrNotExist is used when try to get an entry with an unexist key from KV store.
ErrNotExist = errors.New("Error: key not exist")
ErrNotExist = terror.ClassKV.New(codeNotExist, "Error: key not exist")
// ErrConditionNotMatch is used when condition is not met.
ErrConditionNotMatch = errors.New("Error: Condition not match")
ErrConditionNotMatch = terror.ClassKV.New(codeCondithinNotMatch, "Error: Condition not match")
// ErrLockConflict is used when try to lock an already locked key.
ErrLockConflict = errors.New("Error: Lock conflict")
ErrLockConflict = terror.ClassKV.New(codeLockConfilict, "Error: Lock conflict")
// ErrLazyConditionPairsNotMatch is used when value in store differs from expect pairs.
ErrLazyConditionPairsNotMatch = errors.New("Error: Lazy condition pairs not match")
ErrLazyConditionPairsNotMatch = terror.ClassKV.New(codeLazyConditionPairsNotMatch, "Error: Lazy condition pairs not match")
// ErrRetryable is used when KV store occurs RPC error or some other
// errors which SQL layer can safely retry.
ErrRetryable = errors.New("Error: KV error safe to retry")
ErrRetryable = terror.ClassKV.New(codeRetryable, "Error: KV error safe to retry")
// ErrCannotSetNilValue is the error when sets an empty value.
ErrCannotSetNilValue = errors.New("can not set nil value")
ErrCannotSetNilValue = terror.ClassKV.New(codeCantSetNilValue, "can not set nil value")
// ErrInvalidTxn is the error when commits or rollbacks in an invalid transaction.
ErrInvalidTxn = errors.New("invalid transaction")
ErrInvalidTxn = terror.ClassKV.New(codeInvalidTxn, "invalid transaction")

// ErrNotCommitted is the error returned by CommitVersion when this
// transaction is not committed.
ErrNotCommitted = errors.New("this transaction has not committed")
ErrNotCommitted = terror.ClassKV.New(codeNotCommitted, "this transaction has not committed")

// ErrKeyExists returns when key is already exist.
ErrKeyExists = terror.ClassKV.New(CodeKeyExists, "key already exist")
ErrKeyExists = terror.ClassKV.New(codeKeyExists, "key already exist")
)

func init() {
kvMySQLErrCodes := map[terror.ErrCode]uint16{
CodeKeyExists: mysql.ErrDupEntry,
codeKeyExists: mysql.ErrDupEntry,
}
terror.ErrClassToMySQLCodes[terror.ClassKV] = kvMySQLErrCodes
}
Expand Down
Loading

0 comments on commit 6aacddd

Please sign in to comment.