Skip to content

Commit

Permalink
terror: unify util/errors and util/errors2 in terror.
Browse files Browse the repository at this point in the history
  • Loading branch information
coocood committed Nov 5, 2015
1 parent 3b0f94f commit 0943aa5
Show file tree
Hide file tree
Showing 30 changed files with 151 additions and 183 deletions.
5 changes: 2 additions & 3 deletions bootstrap.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,7 @@ import (
"github.com/ngaut/log"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/util/errors"
"github.com/pingcap/tidb/util/errors2"
"github.com/pingcap/tidb/terror"
)

const (
Expand Down Expand Up @@ -104,7 +103,7 @@ func bootstrap(s Session) {
if err == nil {
// We have already finished bootstrap.
return
} else if !errors2.ErrorEqual(err, errors.ErrDatabaseNotExist) {
} else if !terror.Schema.Equal(err, terror.DatabaseNotExists) {
log.Fatal(err)
}
mustExecute(s, fmt.Sprintf("CREATE DATABASE %s;", mysql.SystemDB))
Expand Down
10 changes: 5 additions & 5 deletions ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,9 +34,9 @@ import (
"github.com/pingcap/tidb/privilege"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/table/tables"
"github.com/pingcap/tidb/terror"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/charset"
qerror "github.com/pingcap/tidb/util/errors"
"github.com/twinj/uuid"
)

Expand Down Expand Up @@ -388,7 +388,7 @@ func (d *ddl) CreateTable(ctx context.Context, ident table.Ident, colDefs []*col
is := d.GetInformationSchema()
schema, ok := is.SchemaByName(ident.Schema)
if !ok {
return errors.Trace(qerror.ErrDatabaseNotExist)
return terror.Schema.New(terror.DatabaseNotExists, "database %s not exists", ident.Schema)
}
if is.TableExists(ident.Schema, ident.Name) {
return errors.Trace(ErrExists)
Expand Down Expand Up @@ -436,7 +436,7 @@ func (d *ddl) AlterTable(ctx context.Context, ident table.Ident, specs []*AlterS

schema, ok := is.SchemaByName(ident.Schema)
if !ok {
return errors.Trace(qerror.ErrDatabaseNotExist)
return terror.Schema.New(terror.DatabaseNotExists, "database %s not exists", ident.Schema)
}

tbl, err := is.TableByName(ident.Schema, ident.Name)
Expand Down Expand Up @@ -570,7 +570,7 @@ func (d *ddl) DropTable(ctx context.Context, ti table.Ident) (err error) {
is := d.GetInformationSchema()
schema, ok := is.SchemaByName(ti.Schema)
if !ok {
return errors.Trace(qerror.ErrDatabaseNotExist)
return terror.Schema.New(terror.DatabaseNotExists, "database %s not exists", ti.Schema)
}

tb, err := is.TableByName(ti.Schema, ti.Name)
Expand Down Expand Up @@ -633,7 +633,7 @@ func (d *ddl) CreateIndex(ctx context.Context, ti table.Ident, unique bool, inde
is := d.infoHandle.Get()
schema, ok := is.SchemaByName(ti.Schema)
if !ok {
return errors.Trace(qerror.ErrDatabaseNotExist)
return terror.Schema.New(terror.DatabaseNotExists, "database %s not exists", ti.Schema)
}

t, err := is.TableByName(ti.Schema, ti.Name)
Expand Down
11 changes: 5 additions & 6 deletions ddl/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,8 +29,7 @@ import (
"github.com/pingcap/tidb/stmt"
"github.com/pingcap/tidb/stmt/stmts"
"github.com/pingcap/tidb/table"
qerror "github.com/pingcap/tidb/util/errors"
"github.com/pingcap/tidb/util/errors2"
"github.com/pingcap/tidb/terror"
)

func TestT(t *testing.T) {
Expand Down Expand Up @@ -63,16 +62,16 @@ func (ts *testSuite) TestT(c *C) {
err := sessionctx.GetDomain(ctx).DDL().CreateSchema(ctx, tbIdent.Schema)
c.Assert(err, IsNil)
err = sessionctx.GetDomain(ctx).DDL().CreateSchema(ctx, tbIdent.Schema)
c.Assert(errors2.ErrorEqual(err, ddl.ErrExists), IsTrue)
c.Assert(terror.ErrorEqual(err, ddl.ErrExists), IsTrue)

tbStmt := statement("create table t (a int primary key not null, b varchar(255), key idx_b (b), c int, d int unique)").(*stmts.CreateTableStmt)

err = sessionctx.GetDomain(ctx).DDL().CreateTable(ctx, table.Ident{Schema: noExist, Name: tbIdent.Name}, tbStmt.Cols, tbStmt.Constraints)
c.Assert(errors2.ErrorEqual(err, qerror.ErrDatabaseNotExist), IsTrue)
c.Assert(terror.Schema.Equal(err, terror.DatabaseNotExists), IsTrue)
err = sessionctx.GetDomain(ctx).DDL().CreateTable(ctx, tbIdent, tbStmt.Cols, tbStmt.Constraints)
c.Assert(err, IsNil)
err = sessionctx.GetDomain(ctx).DDL().CreateTable(ctx, tbIdent, tbStmt.Cols, tbStmt.Constraints)
c.Assert(errors2.ErrorEqual(err, ddl.ErrExists), IsTrue)
c.Assert(terror.ErrorEqual(err, ddl.ErrExists), IsTrue)

tbIdent2 := tbIdent
tbIdent2.Name = model.NewCIStr("t2")
Expand Down Expand Up @@ -162,7 +161,7 @@ func (ts *testSuite) TestT(c *C) {
c.Assert(len(tbs), Equals, 1)

err = sessionctx.GetDomain(ctx).DDL().DropSchema(ctx, noExist)
c.Assert(errors2.ErrorEqual(err, ddl.ErrNotExists), IsTrue)
c.Assert(terror.ErrorEqual(err, ddl.ErrNotExists), IsTrue)
err = sessionctx.GetDomain(ctx).DDL().DropSchema(ctx, tbIdent.Schema)
c.Assert(err, IsNil)
}
Expand Down
9 changes: 4 additions & 5 deletions driver.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,8 +34,7 @@ import (
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/rset"
"github.com/pingcap/tidb/sessionctx"
qerror "github.com/pingcap/tidb/util/errors"
"github.com/pingcap/tidb/util/errors2"
"github.com/pingcap/tidb/terror"
)

const (
Expand Down Expand Up @@ -247,11 +246,11 @@ func (c *driverConn) Begin() (driver.Tx, error) {

func (c *driverConn) Commit() error {
if c.s == nil {
return qerror.ErrCommitNotInTransaction
return terror.Executor.New(terror.CommitNotInTransaction, "commit not in Transaction.")
}
_, err := c.s.Execute(txCommitSQL)

if errors2.ErrorEqual(err, kv.ErrConditionNotMatch) {
if terror.ErrorEqual(err, kv.ErrConditionNotMatch) {
return c.s.Retry()
}

Expand All @@ -264,7 +263,7 @@ func (c *driverConn) Commit() error {

func (c *driverConn) Rollback() error {
if c.s == nil {
return errors.Trace(qerror.ErrRollbackNotInTransaction)
return terror.Executor.New(terror.RollbackNotInTransaction, "rollback not in Transaction.")
}

if _, err := c.s.Execute(txRollbackSQL); err != nil {
Expand Down
4 changes: 2 additions & 2 deletions interpreter/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ import (
"github.com/ngaut/log"
"github.com/peterh/liner"
"github.com/pingcap/tidb"
"github.com/pingcap/tidb/util/errors2"
"github.com/pingcap/tidb/terror"
"github.com/pingcap/tidb/util/printer"
)

Expand Down Expand Up @@ -133,7 +133,7 @@ func executeLine(tx *sql.Tx, txnLine string) error {
}

func mayExit(err error, l string) bool {
if errors2.ErrorEqual(err, liner.ErrPromptAborted) || errors2.ErrorEqual(err, io.EOF) {
if terror.ErrorEqual(err, liner.ErrPromptAborted) || terror.ErrorEqual(err, io.EOF) {
fmt.Println("\nBye")
saveHistory()
return true
Expand Down
8 changes: 4 additions & 4 deletions kv/index_iter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import (
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/store/localstore"
"github.com/pingcap/tidb/store/localstore/goleveldb"
"github.com/pingcap/tidb/util/errors2"
"github.com/pingcap/tidb/terror"
)

func TestT(t *testing.T) {
Expand Down Expand Up @@ -84,7 +84,7 @@ func (s *testIndexSuite) TestIndex(c *C) {
c.Assert(err, IsNil)

_, _, err = it.Next()
c.Assert(errors2.ErrorEqual(err, io.EOF), IsTrue)
c.Assert(terror.ErrorEqual(err, io.EOF), IsTrue)
it.Close()

err = index.Create(txn, values, 0)
Expand All @@ -105,14 +105,14 @@ func (s *testIndexSuite) TestIndex(c *C) {
c.Assert(hit, IsFalse)

_, _, err = it.Next()
c.Assert(errors2.ErrorEqual(err, io.EOF), IsTrue)
c.Assert(terror.ErrorEqual(err, io.EOF), IsTrue)
it.Close()

it, err = index.SeekFirst(txn)
c.Assert(err, IsNil)

_, _, err = it.Next()
c.Assert(errors2.ErrorEqual(err, io.EOF), IsTrue)
c.Assert(terror.ErrorEqual(err, io.EOF), IsTrue)
it.Close()

err = txn.Commit()
Expand Down
4 changes: 2 additions & 2 deletions kv/memdb_buffer.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
package kv

import (
"github.com/pingcap/tidb/util/errors2"
"github.com/pingcap/tidb/terror"
"github.com/syndtr/goleveldb/leveldb"
"github.com/syndtr/goleveldb/leveldb/comparer"
"github.com/syndtr/goleveldb/leveldb/iterator"
Expand Down Expand Up @@ -52,7 +52,7 @@ func (m *memDbBuffer) NewIterator(param interface{}) Iterator {
// Get returns the value associated with key.
func (m *memDbBuffer) Get(k Key) ([]byte, error) {
v, err := m.db.Get(k)
if errors2.ErrorEqual(err, leveldb.ErrNotFound) {
if terror.ErrorEqual(err, leveldb.ErrNotFound) {
return nil, ErrNotExist
}
return v, nil
Expand Down
4 changes: 2 additions & 2 deletions kv/txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@ package kv
import (
"github.com/juju/errors"
"github.com/ngaut/log"
"github.com/pingcap/tidb/util/errors2"
"github.com/pingcap/tidb/terror"
)

// IsRetryableError checks if the err is a fatal error and the under going operation is worth to retry.
Expand All @@ -25,7 +25,7 @@ func IsRetryableError(err error) bool {
return false
}

if errors2.ErrorEqual(err, ErrLockConflict) || errors2.ErrorEqual(err, ErrConditionNotMatch) {
if terror.ErrorEqual(err, ErrLockConflict) || terror.ErrorEqual(err, ErrConditionNotMatch) {
return true
}

Expand Down
4 changes: 2 additions & 2 deletions kv/union_store.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@ package kv
import (
"github.com/juju/errors"
"github.com/ngaut/pool"
"github.com/pingcap/tidb/util/errors2"
"github.com/pingcap/tidb/terror"
)

// conditionType is the type for condition consts.
Expand Down Expand Up @@ -45,7 +45,7 @@ type conditionValue struct {

// IsErrNotFound checks if err is a kind of NotFound error.
func IsErrNotFound(err error) bool {
if errors2.ErrorEqual(err, ErrNotExist) {
if terror.ErrorEqual(err, ErrNotExist) {
return true
}

Expand Down
4 changes: 2 additions & 2 deletions session.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,8 +42,8 @@ import (
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/stmt"
"github.com/pingcap/tidb/stmt/stmts"
"github.com/pingcap/tidb/terror"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/errors2"
"github.com/pingcap/tidb/util/sqlexec"
"github.com/pingcap/tidb/util/types"
)
Expand Down Expand Up @@ -219,7 +219,7 @@ func (s *session) Retry() error {
log.Warnf("Retry %s", st.OriginText())
_, err = runStmt(s, st)
if err != nil {
if errors2.ErrorEqual(err, kv.ErrConditionNotMatch) {
if terror.ErrorEqual(err, kv.ErrConditionNotMatch) {
success = false
break
}
Expand Down
6 changes: 3 additions & 3 deletions stmt/stmts/create.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ import (
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/stmt"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/util/errors2"
"github.com/pingcap/tidb/terror"
"github.com/pingcap/tidb/util/format"
)

Expand Down Expand Up @@ -73,7 +73,7 @@ func (s *CreateDatabaseStmt) Exec(ctx context.Context) (_ rset.Recordset, err er
log.Debug("create database")
err = sessionctx.GetDomain(ctx).DDL().CreateSchema(ctx, model.NewCIStr(s.Name))
if err != nil {
if errors2.ErrorEqual(err, ddl.ErrExists) && s.IfNotExists {
if terror.ErrorEqual(err, ddl.ErrExists) && s.IfNotExists {
err = nil
}
}
Expand Down Expand Up @@ -115,7 +115,7 @@ func (s *CreateTableStmt) SetText(text string) {
// Exec implements the stmt.Statement Exec interface.
func (s *CreateTableStmt) Exec(ctx context.Context) (_ rset.Recordset, err error) {
err = sessionctx.GetDomain(ctx).DDL().CreateTable(ctx, s.Ident.Full(ctx), s.Cols, s.Constraints)
if errors2.ErrorEqual(err, ddl.ErrExists) {
if terror.ErrorEqual(err, ddl.ErrExists) {
if s.IfNotExists {
return nil, nil
}
Expand Down
4 changes: 2 additions & 2 deletions stmt/stmts/drop.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ import (
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/stmt"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/util/errors2"
"github.com/pingcap/tidb/terror"
"github.com/pingcap/tidb/util/format"
)

Expand Down Expand Up @@ -71,7 +71,7 @@ func (s *DropDatabaseStmt) SetText(text string) {
// Exec implements the stmt.Statement Exec interface.
func (s *DropDatabaseStmt) Exec(ctx context.Context) (rset.Recordset, error) {
err := sessionctx.GetDomain(ctx).DDL().DropSchema(ctx, model.NewCIStr(s.Name))
if errors2.ErrorEqual(err, ddl.ErrNotExists) && s.IfExists {
if terror.ErrorEqual(err, ddl.ErrNotExists) && s.IfExists {
err = nil
}
return nil, errors.Trace(err)
Expand Down
4 changes: 2 additions & 2 deletions stmt/stmts/insert.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ import (
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/stmt"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/util/errors2"
"github.com/pingcap/tidb/terror"
"github.com/pingcap/tidb/util/format"
"github.com/pingcap/tidb/util/types"
)
Expand Down Expand Up @@ -273,7 +273,7 @@ func (s *InsertIntoStmt) Exec(ctx context.Context) (_ rset.Recordset, err error)
continue
}

if len(s.OnDuplicate) == 0 || !errors2.ErrorEqual(err, kv.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
4 changes: 2 additions & 2 deletions stmt/stmts/replace.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ import (
"github.com/pingcap/tidb/rset"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/util/errors2"
"github.com/pingcap/tidb/terror"
"github.com/pingcap/tidb/util/format"
"github.com/pingcap/tidb/util/types"
)
Expand Down Expand Up @@ -93,7 +93,7 @@ func (s *ReplaceIntoStmt) Exec(ctx context.Context) (_ rset.Recordset, err error
if err == nil {
continue
}
if err != nil && !errors2.ErrorEqual(err, kv.ErrKeyExists) {
if err != nil && !terror.ErrorEqual(err, kv.ErrKeyExists) {
return nil, errors.Trace(err)
}

Expand Down
4 changes: 2 additions & 2 deletions stmt/stmts/use.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import (
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/db"
"github.com/pingcap/tidb/stmt"
"github.com/pingcap/tidb/util/errors"
"github.com/pingcap/tidb/terror"
"github.com/pingcap/tidb/util/format"
)

Expand Down Expand Up @@ -62,7 +62,7 @@ func (s *UseStmt) SetText(text string) {
func (s *UseStmt) Exec(ctx context.Context) (_ rset.Recordset, err error) {
dbname := model.NewCIStr(s.DBName)
if !sessionctx.GetDomain(ctx).InfoSchema().SchemaExists(dbname) {
return nil, errors.ErrDatabaseNotExist
return nil, terror.Schema.New(terror.DatabaseNotExists, "database %s not exists", dbname)
}
db.BindCurrentSchema(ctx, dbname.O)
return nil, nil
Expand Down
4 changes: 2 additions & 2 deletions store/localstore/kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,8 @@ import (
"github.com/ngaut/log"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/store/localstore/engine"
"github.com/pingcap/tidb/terror"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/errors2"
"github.com/twinj/uuid"
)

Expand Down Expand Up @@ -226,7 +226,7 @@ func (s *dbStore) tryConditionLockKey(tid uint64, key string) error {

metaKey := codec.EncodeBytes(nil, []byte(key))
currValue, err := s.db.Get(metaKey)
if errors2.ErrorEqual(err, kv.ErrNotExist) {
if terror.ErrorEqual(err, kv.ErrNotExist) {
s.keysLocked[key] = tid
return nil
}
Expand Down
Loading

0 comments on commit 0943aa5

Please sign in to comment.