Skip to content

Commit

Permalink
*: Convert ErrKeyExists to MySQL error
Browse files Browse the repository at this point in the history
  • Loading branch information
shenli committed Dec 17, 2015
1 parent 2523e49 commit 67ad875
Show file tree
Hide file tree
Showing 10 changed files with 80 additions and 14 deletions.
5 changes: 3 additions & 2 deletions kv/index_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"strings"

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

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

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

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

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

return true, handle, nil
Expand Down
2 changes: 0 additions & 2 deletions kv/iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,6 @@ var (
ErrClosed = errors.New("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")
// ErrKeyExists is used when try to put an entry to KV store.
ErrKeyExists = errors.New("Error: key already exist")
// ErrConditionNotMatch is used when condition is not met.
ErrConditionNotMatch = errors.New("Error: Condition not match")
// ErrLockConflict is used when try to lock an already locked key.
Expand Down
2 changes: 1 addition & 1 deletion kv/union_store.go
Original file line number Diff line number Diff line change
Expand Up @@ -186,7 +186,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(ErrKeyExists)
return errors.Trace(terror.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 kv.ErrKeyExists Error because we pass -1 as the handle which is not equal to the existed handle.
// We expect a terror.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(kv.ErrKeyExists, err) {
if terror.ErrorNotEqual(terror.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, kv.ErrKeyExists) {
if len(s.OnDuplicate) == 0 || !terror.ErrorEqual(err, terror.ErrKeyExists) {
return nil, errors.Trace(err)
}
if err = execOnDuplicateUpdate(ctx, t, row, h, toUpdateColumns); err != nil {
Expand Down
3 changes: 1 addition & 2 deletions stmt/stmts/replace.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ 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 @@ -83,7 +82,7 @@ func (s *ReplaceIntoStmt) Exec(ctx context.Context) (_ rset.Recordset, err error
if err == nil {
continue
}
if err != nil && !terror.ErrorEqual(err, kv.ErrKeyExists) {
if err != nil && !terror.ErrorEqual(err, terror.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, kv.ErrKeyExists) {
if terror.ErrorEqual(err, terror.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
55 changes: 55 additions & 0 deletions terror/terror.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@ import (
"strconv"

"github.com/juju/errors"
"github.com/ngaut/log"
"github.com/pingcap/tidb/mysql"
)

// Common base error instances.
Expand All @@ -34,6 +36,8 @@ 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 @@ -57,6 +61,7 @@ const (
const (
CodeIncompatibleDBFormat ErrCode = iota + 1
CodeNoDataForHandle
CodeKeyExists
)

// Variable error codes.
Expand Down Expand Up @@ -190,6 +195,56 @@ func (e *Error) NotEqual(err error) bool {
return !e.Equal(err)
}

// ToSQLError convert Error to mysql.SQLError.
func (e *Error) ToSQLError() *mysql.SQLError {
code := e.getMySQLErrorCode()
return &mysql.SQLError{
Code: code,
Message: e.message,
}
}

var defaultMySQLErrorCode uint16

func (e *Error) getMySQLErrorCode() uint16 {
codeMap, ok := errClassToMySQLCodes[e.class]
if !ok {
log.Warnf("Unknown error class: %v", e.class)
return defaultMySQLErrorCode
}
code, ok := codeMap[e.code]
if !ok {
log.Warnf("Unknown error class: %v", e.class)
return defaultMySQLErrorCode
}
return code
}

var parserMySQLErrCodes = map[ErrCode]uint16{}
var schemaMySQLErrCodes = map[ErrCode]uint16{}
var optimizerMySQLErrCodes = map[ErrCode]uint16{}
var executorMySQLErrCodes = map[ErrCode]uint16{}
var kvMySQLErrCodes = map[ErrCode]uint16{
CodeKeyExists: mysql.ErrDupEntry,
}
var serverMySQLErrCodes = map[ErrCode]uint16{}
var expressionMySQLErrCodes = map[ErrCode]uint16{}

var errClassToMySQLCodes map[ErrClass](map[ErrCode]uint16)

func init() {
errClassToMySQLCodes = make(map[ErrClass](map[ErrCode]uint16))
errClassToMySQLCodes[ClassParser] = parserMySQLErrCodes
errClassToMySQLCodes[ClassSchema] = schemaMySQLErrCodes
errClassToMySQLCodes[ClassOptimizer] = optimizerMySQLErrCodes
errClassToMySQLCodes[ClassExecutor] = executorMySQLErrCodes
errClassToMySQLCodes[ClassKV] = kvMySQLErrCodes
errClassToMySQLCodes[ClassServer] = serverMySQLErrCodes
errClassToMySQLCodes[ClassExpression] = expressionMySQLErrCodes
defaultMySQLErrorCode = mysql.ErrDupEntry

}

// ErrorEqual returns a boolean indicating whether err1 is equal to err2.
func ErrorEqual(err1, err2 error) bool {
e1 := errors.Cause(err1)
Expand Down
8 changes: 8 additions & 0 deletions terror/terror_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (

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

func TestT(t *testing.T) {
Expand Down Expand Up @@ -110,3 +111,10 @@ 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")
}
11 changes: 8 additions & 3 deletions tidb-server/server/conn.go
Original file line number Diff line number Diff line change
Expand Up @@ -319,10 +319,15 @@ func (cc *clientConn) writeOK() error {
}

func (cc *clientConn) writeError(e error) error {
var m *mysql.SQLError
var ok bool
var (
m *mysql.SQLError
te *terror.Error
ok bool
)
originErr := errors.Cause(e)
if m, ok = originErr.(*mysql.SQLError); !ok {
if te, ok = originErr.(*terror.Error); ok {
m = te.ToSQLError()
} else {
m = mysql.NewErrf(mysql.ErrUnknown, e.Error())
}

Expand Down

0 comments on commit 67ad875

Please sign in to comment.