diff --git a/ddl/ddl.go b/ddl/ddl.go index 5f00207f1d340..6d7ede29080bd 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -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 @@ -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, diff --git a/domain/domain.go b/domain/domain.go index 1ac14b6faceb1..71f00f0cf73d3 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -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 } } @@ -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") +) diff --git a/infoschema/infoschema.go b/infoschema/infoschema.go index bd7fe071abea5..f1c4bfdff7498 100644 --- a/infoschema/infoschema.go +++ b/infoschema/infoschema.go @@ -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. @@ -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, diff --git a/inspectkv/inspectkv.go b/inspectkv/inspectkv.go index 1a0efd2b1c48f..fdfafa2019785 100644 --- a/inspectkv/inspectkv.go +++ b/inspectkv/inspectkv.go @@ -196,7 +196,7 @@ 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) @@ -204,7 +204,7 @@ func checkIndexAndRecord(txn kv.Transaction, t table.Table, idx *column.IndexedC 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) } } @@ -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 @@ -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 } @@ -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) @@ -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) @@ -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 @@ -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) @@ -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") +) diff --git a/inspectkv/inspectkv_test.go b/inspectkv/inspectkv_test.go index b959a3a70dfb3..00d7628a95d0e 100644 --- a/inspectkv/inspectkv_test.go +++ b/inspectkv/inspectkv_test.go @@ -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) { @@ -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) { diff --git a/kv/error.go b/kv/error.go index 831c01c227ff8..89d118341d1ab 100644 --- a/kv/error.go +++ b/kv/error.go @@ -14,7 +14,6 @@ package kv import ( - "errors" "strings" "github.com/pingcap/go-themis" @@ -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 } diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index b9385e9edac5d..be1a9a502a70c 100644 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -20,12 +20,15 @@ import ( "github.com/ngaut/log" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" + "github.com/pingcap/tidb/terror" ) const ( step = 1000 ) +var errInvalidTableID = terror.ClassAutoid.New(codeInvalidTableID, "invalid TableID") + // Allocator is an auto increment id generator. // Just keep id unique actually. type Allocator interface { @@ -49,7 +52,7 @@ type allocator struct { // Rebase implements autoid.Allocator Rebase interface. func (alloc *allocator) Rebase(tableID, newBase int64, allocIDs bool) error { if tableID == 0 { - return errors.New("Invalid tableID") + return errInvalidTableID.Gen("Invalid tableID") } alloc.mu.Lock() @@ -93,7 +96,7 @@ func (alloc *allocator) Rebase(tableID, newBase int64, allocIDs bool) error { // Alloc implements autoid.Allocator Alloc interface. func (alloc *allocator) Alloc(tableID int64) (int64, error) { if tableID == 0 { - return 0, errors.New("Invalid tableID") + return 0, errInvalidTableID.Gen("Invalid tableID") } alloc.mu.Lock() defer alloc.mu.Unlock() @@ -149,7 +152,7 @@ func (alloc *memoryAllocator) Rebase(tableID, newBase int64, allocIDs bool) erro // Alloc implements autoid.Allocator Alloc interface. func (alloc *memoryAllocator) Alloc(tableID int64) (int64, error) { if tableID == 0 { - return 0, errors.New("Invalid tableID") + return 0, errInvalidTableID.Gen("Invalid tableID") } alloc.mu.Lock() defer alloc.mu.Unlock() @@ -178,3 +181,6 @@ func NewMemoryAllocator(dbID int64) Allocator { dbID: dbID, } } + +//autoid error codes. +const codeInvalidTableID terror.ErrCode = 1 diff --git a/meta/meta.go b/meta/meta.go index ff8b31679d33a..7780a92ad1c9f 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -25,7 +25,9 @@ import ( "github.com/juju/errors" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/model" + "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/structure" + "github.com/pingcap/tidb/terror" ) var ( @@ -58,14 +60,17 @@ var ( ) var ( + errInvalidTableKey = terror.ClassMeta.New(codeInvalidTableKey, "invalid table meta key") + errInvalidDBKey = terror.ClassMeta.New(codeInvalidDBKey, "invalid db key") + // ErrDBExists is the error for db exists. - ErrDBExists = errors.New("database already exists") + ErrDBExists = terror.ClassMeta.New(codeDatabaseExists, "database already exists") // ErrDBNotExists is the error for db not exists. - ErrDBNotExists = errors.New("database doesn't exist") + ErrDBNotExists = terror.ClassMeta.New(codeDatabaseNotExists, "database doesn't exist") // ErrTableExists is the error for table exists. - ErrTableExists = errors.New("table already exists") + ErrTableExists = terror.ClassMeta.New(codeTableExists, "table already exists") // ErrTableNotExists is the error for table not exists. - ErrTableNotExists = errors.New("table doesn't exist") + ErrTableNotExists = terror.ClassMeta.New(codeTableNotExists, "table doesn't exist") ) // Meta is for handling meta information in a transaction. @@ -648,3 +653,24 @@ func (m *Meta) GetBgJobOwner() (*model.Owner, error) { func (m *Meta) SetBgJobOwner(o *model.Owner) error { return m.setJobOwner(mBgJobOwnerKey, o) } + +// meta error codes. +const ( + codeInvalidTableKey terror.ErrCode = 1 + codeInvalidDBKey = 2 + + codeDatabaseExists = 1007 + codeDatabaseNotExists = 1049 + codeTableExists = 1050 + codeTableNotExists = 1146 +) + +func init() { + metaMySQLErrCodes := map[terror.ErrCode]uint16{ + codeDatabaseExists: mysql.ErrDbCreateExists, + codeDatabaseNotExists: mysql.ErrBadDb, + codeTableNotExists: mysql.ErrNoSuchTable, + codeTableExists: mysql.ErrTableExists, + } + terror.ErrClassToMySQLCodes[terror.ClassMeta] = metaMySQLErrCodes +} diff --git a/perfschema/init.go b/perfschema/init.go index 39e41035bf84f..d08317a53de4d 100644 --- a/perfschema/init.go +++ b/perfschema/init.go @@ -331,7 +331,7 @@ func buildEnumColumnInfo(offset int, name string, elems []string, flag uint, def func (ps *perfSchema) initRecords(tbName string, records [][]types.Datum) error { tbl, ok := ps.mTables[tbName] if !ok { - return errors.Errorf("Unknown PerformanceSchema table: %s", tbName) + return errInvalidPerfSchemaTable.Gen("Unknown PerformanceSchema table: %s", tbName) } for _, rec := range records { _, err := tbl.AddRecord(nil, rec) diff --git a/perfschema/instrument.go b/perfschema/instrument.go index 43014225e3f5b..bf68fcf6a5d41 100644 --- a/perfschema/instrument.go +++ b/perfschema/instrument.go @@ -70,7 +70,7 @@ func (ps *perfSchema) addInstrument(name string) (uint64, error) { func (ps *perfSchema) getTimerName(flag int) (enumTimerName, error) { if flag < 0 || flag >= len(setupTimersRecords) { - return timerNameNone, errors.Errorf("Unknown timerName flag %d", flag) + return timerNameNone, errInvalidTimerFlag.Gen("Unknown timerName flag %d", flag) } timerName := fmt.Sprintf("%s", setupTimersRecords[flag][1].GetString()) switch timerName { diff --git a/perfschema/perfschema.go b/perfschema/perfschema.go index 91915a54857eb..50afa4deb489e 100644 --- a/perfschema/perfschema.go +++ b/perfschema/perfschema.go @@ -17,6 +17,12 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/table" + "github.com/pingcap/tidb/terror" +) + +var ( + errInvalidPerfSchemaTable = terror.ClassPerfSchema.New(codeInvalidPerfSchemaTable, "invalid perfschema table") + errInvalidTimerFlag = terror.ClassPerfSchema.New(codeInvalidTimerFlag, "invalid timer flag") ) // StatementInstrument defines the methods for statement instrumentation points @@ -72,3 +78,9 @@ func init() { schema := &perfSchema{} PerfHandle = schema } + +// perfschema error codes. +const ( + codeInvalidPerfSchemaTable terror.ErrCode = 1 + codeInvalidTimerFlag = 2 +) diff --git a/privilege/privileges/privileges.go b/privilege/privileges/privileges.go index 7ff6588062754..f3a6ffb80a175 100644 --- a/privilege/privileges/privileges.go +++ b/privilege/privileges/privileges.go @@ -24,10 +24,22 @@ import ( "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/types" ) +// privilege error codes. +const ( + codeInvalidPrivilegeType terror.ErrCode = 1 + codeInvalidUserNameFormat = 2 +) + +var ( + errInvalidPrivilegeType = terror.ClassPrivilege.New(codeInvalidPrivilegeType, "unknown privilege type") + errInvalidUserNameFormat = terror.ClassPrivilege.New(codeInvalidUserNameFormat, "wrong username format") +) + var _ privilege.Checker = (*UserPrivileges)(nil) type privileges struct { @@ -209,7 +221,7 @@ func (p *UserPrivileges) Check(ctx context.Context, db *model.DBInfo, tbl *model func (p *UserPrivileges) loadPrivileges(ctx context.Context) error { strs := strings.Split(p.User, "@") if len(strs) != 2 { - return errors.Errorf("Wrong username format: %s", p.User) + return errInvalidUserNameFormat.Gen("Wrong username format: %s", p.User) } username, host := strs[0], strs[1] p.privs = &userPrivileges{ @@ -262,7 +274,7 @@ func (p *UserPrivileges) loadGlobalPrivileges(ctx context.Context) error { for i := userTablePrivColumnStartIndex; i < len(fs); i++ { d := row.Data[i] if d.Kind() != types.KindMysqlEnum { - return errors.Errorf("Privilege should be mysql.Enum: %v(%T)", d, d) + return errInvalidPrivilegeType.Gen("Privilege should be mysql.Enum: %v(%T)", d, d) } ed := d.GetMysqlEnum() if ed.String() != "Y" { @@ -271,7 +283,7 @@ func (p *UserPrivileges) loadGlobalPrivileges(ctx context.Context) error { f := fs[i] p, ok := mysql.Col2PrivType[f.ColumnAsName.O] if !ok { - return errors.New("Unknown Privilege Type!") + return errInvalidPrivilegeType.Gen("Unknown Privilege Type!") } ps.add(p) } @@ -307,7 +319,7 @@ func (p *UserPrivileges) loadDBScopePrivileges(ctx context.Context) error { for i := dbTablePrivColumnStartIndex; i < len(fs); i++ { d := row.Data[i] if d.Kind() != types.KindMysqlEnum { - return errors.Errorf("Privilege should be mysql.Enum: %v(%T)", d, d) + return errInvalidPrivilegeType.Gen("Privilege should be mysql.Enum: %v(%T)", d, d) } ed := d.GetMysqlEnum() if ed.String() != "Y" { @@ -316,7 +328,7 @@ func (p *UserPrivileges) loadDBScopePrivileges(ctx context.Context) error { f := fs[i] p, ok := mysql.Col2PrivType[f.ColumnAsName.O] if !ok { - return errors.New("Unknown Privilege Type!") + return errInvalidPrivilegeType.Gen("Unknown Privilege Type!") } ps[dbStr].add(p) } @@ -357,7 +369,7 @@ func (p *UserPrivileges) loadTableScopePrivileges(ctx context.Context) error { for _, d := range pvs { p, ok := mysql.SetStr2Priv[d] if !ok { - return errors.New("Unknown Privilege Type!") + return errInvalidPrivilegeType.Gen("Unknown Privilege Type!") } ps[dbStr][tblStr].add(p) } diff --git a/session_test.go b/session_test.go index 0981e3a264c80..c3c1ba77a0e1e 100644 --- a/session_test.go +++ b/session_test.go @@ -1286,12 +1286,12 @@ func (s *testSessionSuite) TestIssue461(c *C) { c.Assert(err, NotNil) // Check error type and error message c.Assert(terror.ErrorEqual(err, kv.ErrKeyExists), IsTrue) - c.Assert(err.Error(), Equals, "[kv:3]Duplicate entry '1' for key 'PRIMARY'") + c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '1' for key 'PRIMARY'") _, err = se2.Execute("commit") c.Assert(err, NotNil) c.Assert(terror.ErrorEqual(err, kv.ErrKeyExists), IsTrue) - c.Assert(err.Error(), Equals, "[kv:3]Duplicate entry '2' for key 'val'") + c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '2' for key 'val'") se := newSession(c, store, s.dbName) mustExecSQL(c, se, "drop table test;") diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 998717b89fd79..2b30f2355d2cc 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -14,10 +14,14 @@ package variable import ( - "github.com/juju/errors" "github.com/pingcap/tidb/context" + "github.com/pingcap/tidb/terror" ) +const codeCantGetValidID terror.ErrCode = 1 + +var errCantGetValidID = terror.ClassVariable.New(codeCantGetValidID, "cannot get valid auto-increment id in retry") + // RetryInfo saves retry infomation. type RetryInfo struct { Retrying bool @@ -46,7 +50,7 @@ func (r *RetryInfo) ResetOffset() { // GetCurrAutoIncrementID gets current AutoIncrementID. func (r *RetryInfo) GetCurrAutoIncrementID() (int64, error) { if r.currRetryOff >= len(r.autoIncrementIDs) { - return 0, errors.New("cannot get valid auto-increment id in retry") + return 0, errCantGetValidID } id := r.autoIncrementIDs[r.currRetryOff] r.currRetryOff++ diff --git a/structure/hash.go b/structure/hash.go index 9c0f80aedbbd5..bff8734f06a67 100644 --- a/structure/hash.go +++ b/structure/hash.go @@ -265,7 +265,7 @@ func (t *TxStructure) loadHashMeta(metaKey []byte) (hashMeta, error) { } if len(v) != 8 { - return meta, errors.New("invalid list meta data") + return meta, errInvalidListMetaData } meta.FieldCount = int64(binary.BigEndian.Uint64(v[0:8])) diff --git a/structure/list.go b/structure/list.go index b6e01cefee23f..3649450f5dffc 100644 --- a/structure/list.go +++ b/structure/list.go @@ -160,7 +160,7 @@ func (t *TxStructure) LSet(key []byte, index int64, value []byte) error { if index >= meta.LIndex && index < meta.RIndex { return t.txn.Set(t.encodeListDataKey(key, index), value) } - return errors.Errorf("invalid index %d", index) + return errInvalidListIndex.Gen("invalid list index %d", index) } // LClear removes the list of the key. @@ -195,7 +195,7 @@ func (t *TxStructure) loadListMeta(metaKey []byte) (listMeta, error) { } if len(v) != 16 { - return meta, errors.Errorf("invalid list meta data") + return meta, errInvalidListMetaData } meta.LIndex = int64(binary.BigEndian.Uint64(v[0:8])) diff --git a/structure/structure.go b/structure/structure.go index 61fe269f89fde..64320e7f3886a 100644 --- a/structure/structure.go +++ b/structure/structure.go @@ -13,7 +13,25 @@ package structure -import "github.com/pingcap/tidb/kv" +import ( + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/terror" +) + +// structure error codes. +const ( + codeInvalidHashKeyFlag terror.ErrCode = 1 + codeInvalidHashKeyPrefix = 2 + codeInvalidListIndex = 3 + codeInvalidListMetaData = 4 +) + +var ( + errInvalidHashKeyFlag = terror.ClassStructure.New(codeInvalidHashKeyFlag, "invalid encoded hash key flag") + errInvalidHashKeyPrefix = terror.ClassStructure.New(codeInvalidHashKeyPrefix, "invalid encoded hash key prefix") + errInvalidListIndex = terror.ClassStructure.New(codeInvalidListMetaData, "invalid list index") + errInvalidListMetaData = terror.ClassStructure.New(codeInvalidListMetaData, "invalid list meta data") +) // NewStructure creates a TxStructure in transaction txn and with key prefix. func NewStructure(txn kv.Transaction, prefix []byte) *TxStructure { diff --git a/structure/type.go b/structure/type.go index 5e8c87cd13962..ab3c79d34cbb6 100644 --- a/structure/type.go +++ b/structure/type.go @@ -86,7 +86,7 @@ func (t *TxStructure) decodeHashDataKey(ek kv.Key) ([]byte, []byte, error) { if err != nil { return nil, nil, errors.Trace(err) } else if TypeFlag(tp) != HashData { - return nil, nil, errors.Errorf("invalid encoded hash data key flag %c", byte(tp)) + return nil, nil, errInvalidHashKeyFlag.Gen("invalid encoded hash data key flag %c", byte(tp)) } _, field, err = codec.DecodeBytes(ek) diff --git a/terror/terror.go b/terror/terror.go index 1596d0d2fce33..d5d6537090f27 100644 --- a/terror/terror.go +++ b/terror/terror.go @@ -53,41 +53,62 @@ type ErrClass int // Error classes. const ( - ClassParser ErrClass = iota + 1 - ClassSchema - ClassOptimizer - ClassOptimizerPlan - ClassExecutor + ClassAutoid ErrClass = iota + 1 + ClassDDL + ClassDomain ClassEvaluator + ClassExecutor + ClassExpression + ClassInspectkv ClassKV + ClassMeta + ClassOptimizer + ClassOptimizerPlan + ClassParser + ClassPerfSchema + ClassPrivilege + ClassSchema ClassServer + ClassStructure ClassVariable - ClassExpression - ClassDDL // Add more as needed. ) // String implements fmt.Stringer interface. func (ec ErrClass) String() string { switch ec { - case ClassParser: - return "parser" - case ClassSchema: - return "schema" - case ClassOptimizer: - return "optimizer" + case ClassAutoid: + return "autoid" + case ClassDDL: + return "ddl" + case ClassDomain: + return "domain" case ClassExecutor: return "executor" + case ClassExpression: + return "expression" + case ClassInspectkv: + return "inspectkv" + case ClassMeta: + return "meta" case ClassKV: return "kv" + case ClassOptimizer: + return "optimizer" + case ClassParser: + return "parser" + case ClassPerfSchema: + return "perfschema" + case ClassPrivilege: + return "privilege" + case ClassSchema: + return "schema" case ClassServer: return "server" + case ClassStructure: + return "structure" case ClassVariable: return "variable" - case ClassExpression: - return "expression" - case ClassDDL: - return "ddl" } return strconv.Itoa(int(ec)) } @@ -208,10 +229,10 @@ var ( func init() { ErrClassToMySQLCodes = make(map[ErrClass](map[ErrCode]uint16)) - ErrClassToMySQLCodes[ClassParser] = parserMySQLErrCodes ErrClassToMySQLCodes[ClassExecutor] = executorMySQLErrCodes - ErrClassToMySQLCodes[ClassServer] = serverMySQLErrCodes ErrClassToMySQLCodes[ClassExpression] = expressionMySQLErrCodes + ErrClassToMySQLCodes[ClassParser] = parserMySQLErrCodes + ErrClassToMySQLCodes[ClassServer] = serverMySQLErrCodes defaultMySQLErrorCode = mysql.ErrUnknown }