Skip to content

Commit

Permalink
*: Move kv errors to kv package
Browse files Browse the repository at this point in the history
  • Loading branch information
shenli committed Dec 24, 2015
1 parent 852ad1d commit 87d007d
Show file tree
Hide file tree
Showing 9 changed files with 29 additions and 35 deletions.
18 changes: 18 additions & 0 deletions kv/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,9 +18,17 @@ import (
"strings"

"github.com/pingcap/go-themis"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/terror"
)

// KV error codes.
const (
CodeIncompatibleDBFormat terror.ErrCode = 1
CodeNoDataForHandle = 2
CodeKeyExists = 3
)

var (
// ErrClosed is used when close an already closed txn.
ErrClosed = errors.New("Error: Transaction already closed")
Expand All @@ -43,8 +51,18 @@ var (
// ErrNotCommitted is the error returned by CommitVersion when this
// transaction is not committed.
ErrNotCommitted = errors.New("this transaction has not committed")

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

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

// IsRetryableError checks if the err is a fatal error and the under going operation is worth to retry.
func IsRetryableError(err error) bool {
if err == nil {
Expand Down
5 changes: 2 additions & 3 deletions kv/index_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ import (
"strings"

"github.com/juju/errors"
"github.com/pingcap/tidb/terror"
"github.com/pingcap/tidb/util/codec"
)

Expand Down Expand Up @@ -194,7 +193,7 @@ func (c *kvIndex) Create(rm RetrieverMutator, indexedValues []interface{}, h int
return errors.Trace(err)
}

return errors.Trace(terror.ErrKeyExists)
return errors.Trace(ErrKeyExists)
}

// Delete removes the entry for handle h and indexdValues from KV index.
Expand Down Expand Up @@ -283,7 +282,7 @@ func (c *kvIndex) Exist(rm RetrieverMutator, indexedValues []interface{}, h int6
}

if handle != h {
return true, handle, errors.Trace(terror.ErrKeyExists)
return true, handle, errors.Trace(ErrKeyExists)
}

return true, handle, nil
Expand Down
3 changes: 1 addition & 2 deletions kv/union_store.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ import (

"github.com/juju/errors"
"github.com/ngaut/pool"
"github.com/pingcap/tidb/terror"
)

// UnionStore is a store that wraps a snapshot for read and a BufferStore for buffered write.
Expand Down Expand Up @@ -213,7 +212,7 @@ func (us *unionStore) CheckLazyConditionPairs() error {
for ; it.Valid(); it.Next() {
if len(it.Value()) == 0 {
if _, exist := values[it.Key()]; exist {
return errors.Trace(terror.ErrKeyExists)
return errors.Trace(ErrKeyExists)
}
} else {
if bytes.Compare(values[it.Key()], it.Value()) != 0 {
Expand Down
4 changes: 2 additions & 2 deletions plan/plans/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -436,12 +436,12 @@ func (r *indexPlan) pointLookup(ctx context.Context, val interface{}) (*plan.Row
}
var exist bool
var h int64
// We expect a terror.ErrKeyExists Error because we pass -1 as the handle which is not equal to the existed handle.
// We expect a kv.ErrKeyExists Error because we pass -1 as the handle which is not equal to the existed handle.
exist, h, err = r.idx.Exist(txn, []interface{}{val}, -1)
if !exist {
return nil, errors.Trace(err)
}
if terror.ErrorNotEqual(terror.ErrKeyExists, err) {
if terror.ErrorNotEqual(kv.ErrKeyExists, err) {
return nil, errors.Trace(err)
}
var row *plan.Row
Expand Down
2 changes: 1 addition & 1 deletion stmt/stmts/insert.go
Original file line number Diff line number Diff line change
Expand Up @@ -215,7 +215,7 @@ func (s *InsertIntoStmt) Exec(ctx context.Context) (_ rset.Recordset, err error)
continue
}

if len(s.OnDuplicate) == 0 || !terror.ErrorEqual(err, terror.ErrKeyExists) {
if len(s.OnDuplicate) == 0 || !terror.ErrorEqual(err, kv.ErrKeyExists) {
return nil, errors.Trace(err)
}
if err = execOnDuplicateUpdate(ctx, t, row, h, toUpdateColumns); err != nil {
Expand Down
3 changes: 2 additions & 1 deletion stmt/stmts/replace.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ package stmts
import (
"github.com/juju/errors"
"github.com/pingcap/tidb/context"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/rset"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/table"
Expand Down Expand Up @@ -82,7 +83,7 @@ func (s *ReplaceIntoStmt) Exec(ctx context.Context) (_ rset.Recordset, err error
if err == nil {
continue
}
if err != nil && !terror.ErrorEqual(err, terror.ErrKeyExists) {
if err != nil && !terror.ErrorEqual(err, kv.ErrKeyExists) {
return nil, errors.Trace(err)
}

Expand Down
2 changes: 1 addition & 1 deletion table/tables/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -430,7 +430,7 @@ func (t *Table) AddRecord(ctx context.Context, r []interface{}, h int64) (record
}
colVals, _ := v.FetchValues(r)
if err = v.X.Create(bs, colVals, recordID); err != nil {
if terror.ErrorEqual(err, terror.ErrKeyExists) {
if terror.ErrorEqual(err, kv.ErrKeyExists) {
// Get the duplicate row handle
// For insert on duplicate syntax, we should update the row
iter, _, err1 := v.X.Seek(bs, colVals)
Expand Down
19 changes: 2 additions & 17 deletions terror/terror.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,8 +33,6 @@ var (
UnknownSystemVar = ClassVariable.New(CodeUnknownSystemVar, "unknown system variable")

MissConnectionID = ClassExpression.New(CodeMissConnectionID, "miss connection id information")

ErrKeyExists = ClassKV.New(CodeKeyExists, "key already exist")
)

// ErrCode represents a specific error type in a error class.
Expand All @@ -48,13 +46,6 @@ const (
CodeExecResultIsEmpty = 3
)

// KV error codes.
const (
CodeIncompatibleDBFormat ErrCode = 1
CodeNoDataForHandle = 2
CodeKeyExists = 3
)

// Variable error codes.
const (
CodeUnknownStatusVar ErrCode = iota + 1
Expand Down Expand Up @@ -210,12 +201,8 @@ func (e *Error) getMySQLErrorCode() uint16 {

var (
// ErrCode to mysql error code map.
parserMySQLErrCodes = map[ErrCode]uint16{}
optimizerMySQLErrCodes = map[ErrCode]uint16{}
executorMySQLErrCodes = map[ErrCode]uint16{}
kvMySQLErrCodes = map[ErrCode]uint16{
CodeKeyExists: mysql.ErrDupEntry,
}
parserMySQLErrCodes = map[ErrCode]uint16{}
executorMySQLErrCodes = map[ErrCode]uint16{}
serverMySQLErrCodes = map[ErrCode]uint16{}
expressionMySQLErrCodes = map[ErrCode]uint16{}

Expand All @@ -226,9 +213,7 @@ var (
func init() {
ErrClassToMySQLCodes = make(map[ErrClass](map[ErrCode]uint16))
ErrClassToMySQLCodes[ClassParser] = parserMySQLErrCodes
ErrClassToMySQLCodes[ClassOptimizer] = optimizerMySQLErrCodes
ErrClassToMySQLCodes[ClassExecutor] = executorMySQLErrCodes
ErrClassToMySQLCodes[ClassKV] = kvMySQLErrCodes
ErrClassToMySQLCodes[ClassServer] = serverMySQLErrCodes
ErrClassToMySQLCodes[ClassExpression] = expressionMySQLErrCodes
defaultMySQLErrorCode = mysql.ErrUnknown
Expand Down
8 changes: 0 additions & 8 deletions terror/terror_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ import (

"github.com/juju/errors"
. "github.com/pingcap/check"
"github.com/pingcap/tidb/mysql"
)

func TestT(t *testing.T) {
Expand Down Expand Up @@ -111,10 +110,3 @@ func (s *testTErrorSuite) TestErrorEqual(c *C) {
c.Assert(ErrorEqual(te1, te3), IsFalse)
c.Assert(ErrorEqual(te3, te4), IsFalse)
}

func (s *testTErrorSuite) TestMySQLErrorCode(c *C) {
ke := ErrKeyExists.Gen("key exists")
me := ke.ToSQLError()
c.Assert(me.Code, Equals, uint16(mysql.ErrDupEntry))
c.Assert(me.Message, Equals, "key exists")
}

0 comments on commit 87d007d

Please sign in to comment.