From 025f441208824959744ffc8f290b769251b77d70 Mon Sep 17 00:00:00 2001 From: shirly Date: Tue, 11 May 2021 18:15:04 +0800 Subject: [PATCH] store/driver: move error to single package Signed-off-by: shirly --- executor/point_get_test.go | 12 +- server/conn.go | 8 +- server/conn_stmt.go | 4 +- session/pessimistic_test.go | 42 +++---- store/copr/batch_coprocessor.go | 6 +- store/copr/coprocessor.go | 12 +- store/copr/mpp.go | 14 +-- store/copr/store.go | 12 +- store/driver/error/error.go | 158 ++++++++++++++++++++++++++ store/driver/tikv_driver.go | 11 +- store/driver/txn/error.go | 141 +---------------------- store/driver/txn/snapshot.go | 5 +- store/driver/txn/txn_driver.go | 11 +- store/driver/txn/unionstore_driver.go | 23 ++-- 14 files changed, 242 insertions(+), 217 deletions(-) create mode 100644 store/driver/error/error.go diff --git a/executor/point_get_test.go b/executor/point_get_test.go index 846b6f1628fe1..f66446a6bef83 100644 --- a/executor/point_get_test.go +++ b/executor/point_get_test.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" - txndriver "github.com/pingcap/tidb/store/driver/txn" + storeerr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/tablecodec" @@ -536,15 +536,15 @@ func (s *testPointGetSuite) TestSelectCheckVisibility(c *C) { c.Assert(expectErr.Equal(err), IsTrue) } // Test point get. - checkSelectResultError("select * from t where a='1'", txndriver.ErrGCTooEarly) + checkSelectResultError("select * from t where a='1'", storeerr.ErrGCTooEarly) // Test batch point get. - checkSelectResultError("select * from t where a in ('1','2')", txndriver.ErrGCTooEarly) + checkSelectResultError("select * from t where a in ('1','2')", storeerr.ErrGCTooEarly) // Test Index look up read. - checkSelectResultError("select * from t where b > 0 ", txndriver.ErrGCTooEarly) + checkSelectResultError("select * from t where b > 0 ", storeerr.ErrGCTooEarly) // Test Index read. - checkSelectResultError("select b from t where b > 0 ", txndriver.ErrGCTooEarly) + checkSelectResultError("select b from t where b > 0 ", storeerr.ErrGCTooEarly) // Test table read. - checkSelectResultError("select * from t", txndriver.ErrGCTooEarly) + checkSelectResultError("select * from t", storeerr.ErrGCTooEarly) } func (s *testPointGetSuite) TestReturnValues(c *C) { diff --git a/server/conn.go b/server/conn.go index 29c87bd0dfd86..78cdd1a46c12d 100644 --- a/server/conn.go +++ b/server/conn.go @@ -76,7 +76,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" - txndriver "github.com/pingcap/tidb/store/driver/txn" + storeerr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/arena" @@ -1569,7 +1569,7 @@ func (cc *clientConn) handleQuery(ctx context.Context, sql string) (err error) { retryable, err = cc.handleStmt(ctx, stmt, parserWarns, i == len(stmts)-1) if err != nil { _, allowTiFlashFallback := cc.ctx.GetSessionVars().AllowFallbackToTiKV[kv.TiFlash] - if allowTiFlashFallback && errors.ErrorEqual(err, txndriver.ErrTiFlashServerTimeout) && retryable { + if allowTiFlashFallback && errors.ErrorEqual(err, storeerr.ErrTiFlashServerTimeout) && retryable { // When the TiFlash server seems down, we append a warning to remind the user to check the status of the TiFlash // server and fallback to TiKV. warns := append(parserWarns, stmtctx.SQLWarn{Level: stmtctx.WarnLevelError, Err: err}) @@ -1870,10 +1870,10 @@ func (cc *clientConn) writeChunks(ctx context.Context, rs ResultSet, binary bool failpoint.Inject("fetchNextErr", func(value failpoint.Value) { switch value.(string) { case "firstNext": - failpoint.Return(firstNext, txndriver.ErrTiFlashServerTimeout) + failpoint.Return(firstNext, storeerr.ErrTiFlashServerTimeout) case "secondNext": if !firstNext { - failpoint.Return(firstNext, txndriver.ErrTiFlashServerTimeout) + failpoint.Return(firstNext, storeerr.ErrTiFlashServerTimeout) } } }) diff --git a/server/conn_stmt.go b/server/conn_stmt.go index 242b0df80fc83..e9f56306d9800 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -50,7 +50,7 @@ import ( "github.com/pingcap/tidb/metrics" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/stmtctx" - txndriver "github.com/pingcap/tidb/store/driver/txn" + storeerr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/execdetails" @@ -198,7 +198,7 @@ func (cc *clientConn) handleStmtExecute(ctx context.Context, data []byte) (err e ctx = context.WithValue(ctx, util.ExecDetailsKey, &util.ExecDetails{}) retryable, err := cc.executePreparedStmtAndWriteResult(ctx, stmt, args, useCursor) _, allowTiFlashFallback := cc.ctx.GetSessionVars().AllowFallbackToTiKV[kv.TiFlash] - if allowTiFlashFallback && err != nil && errors.ErrorEqual(err, txndriver.ErrTiFlashServerTimeout) && retryable { + if allowTiFlashFallback && err != nil && errors.ErrorEqual(err, storeerr.ErrTiFlashServerTimeout) && retryable { // When the TiFlash server seems down, we append a warning to remind the user to check the status of the TiFlash // server and fallback to TiKV. prevErr := err diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index 83f0057384aea..72853d86208a9 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -31,7 +31,7 @@ import ( plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" - txndriver "github.com/pingcap/tidb/store/driver/txn" + storeerr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/tablecodec" @@ -611,7 +611,7 @@ func (s *testPessimisticSuite) TestWaitLockKill(c *C) { _, err := tk2.Exec("update test_kill set c = c + 1 where id = 1") wg.Done() c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, txndriver.ErrQueryInterrupted), IsTrue) + c.Assert(terror.ErrorEqual(err, storeerr.ErrQueryInterrupted), IsTrue) tk.MustExec("rollback") } @@ -733,10 +733,10 @@ func (s *testPessimisticSuite) TestInnodbLockWaitTimeout(c *C) { timeoutErr := <-timeoutErrCh c.Assert(timeoutErr, NotNil) - c.Assert(timeoutErr.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) + c.Assert(timeoutErr.Error(), Equals, storeerr.ErrLockWaitTimeout.Error()) timeoutErr = <-timeoutErrCh c.Assert(timeoutErr, NotNil) - c.Assert(timeoutErr.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) + c.Assert(timeoutErr.Error(), Equals, storeerr.ErrLockWaitTimeout.Error()) // tk4 lock c1 = 2 tk4.MustExec("begin pessimistic") @@ -749,7 +749,7 @@ func (s *testPessimisticSuite) TestInnodbLockWaitTimeout(c *C) { _, err := tk2.Exec("delete from tk where c1 = 2") c.Check(time.Since(start), GreaterEqual, 1000*time.Millisecond) c.Check(time.Since(start), Less, 3000*time.Millisecond) // unit test diff should not be too big - c.Check(err.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) + c.Check(err.Error(), Equals, storeerr.ErrLockWaitTimeout.Error()) tk4.MustExec("commit") @@ -767,7 +767,7 @@ func (s *testPessimisticSuite) TestInnodbLockWaitTimeout(c *C) { _, err = tk2.Exec("delete from tk where c1 = 3") // tk2 tries to lock c1 = 3 fail, this delete should be rollback, but previous update should be keeped c.Check(time.Since(start), GreaterEqual, 1000*time.Millisecond) c.Check(time.Since(start), Less, 3000*time.Millisecond) // unit test diff should not be too big - c.Check(err.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) + c.Check(err.Error(), Equals, storeerr.ErrLockWaitTimeout.Error()) tk2.MustExec("commit") tk3.MustExec("commit") @@ -841,7 +841,7 @@ func (s *testPessimisticSuite) TestInnodbLockWaitTimeoutWaitStart(c *C) { c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/PessimisticLockErrWriteConflict"), IsNil) waitErr := <-done c.Assert(waitErr, NotNil) - c.Check(waitErr.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) + c.Check(waitErr.Error(), Equals, storeerr.ErrLockWaitTimeout.Error()) c.Check(duration, GreaterEqual, 1000*time.Millisecond) c.Check(duration, LessEqual, 3000*time.Millisecond) tk2.MustExec("rollback") @@ -1131,11 +1131,11 @@ func (s *testPessimisticSuite) TestPessimisticLockNonExistsKey(c *C) { tk1.MustExec("begin pessimistic") err := tk1.ExecToErr("select * from t where k = 2 for update nowait") - c.Check(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Check(storeerr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) err = tk1.ExecToErr("select * from t where k = 4 for update nowait") - c.Check(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Check(storeerr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) err = tk1.ExecToErr("select * from t where k = 7 for update nowait") - c.Check(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Check(storeerr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) tk.MustExec("rollback") tk1.MustExec("rollback") @@ -1147,9 +1147,9 @@ func (s *testPessimisticSuite) TestPessimisticLockNonExistsKey(c *C) { tk1.MustExec("begin pessimistic") err = tk1.ExecToErr("select * from t where k = 2 for update nowait") - c.Check(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Check(storeerr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) err = tk1.ExecToErr("select * from t where k = 6 for update nowait") - c.Check(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Check(storeerr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) tk.MustExec("rollback") tk1.MustExec("rollback") } @@ -1279,10 +1279,10 @@ func (s *testPessimisticSuite) TestBatchPointGetLockIndex(c *C) { tk2.MustExec("begin pessimistic") err := tk2.ExecToErr("insert into t1 values(2, 2, 2)") c.Assert(err, NotNil) - c.Assert(txndriver.ErrLockWaitTimeout.Equal(err), IsTrue) + c.Assert(storeerr.ErrLockWaitTimeout.Equal(err), IsTrue) err = tk2.ExecToErr("select * from t1 where c2 = 3 for update nowait") c.Assert(err, NotNil) - c.Assert(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Assert(storeerr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) tk.MustExec("rollback") tk2.MustExec("rollback") } @@ -1429,12 +1429,12 @@ func (s *testPessimisticSuite) TestGenerateColPointGet(c *C) { tk2.MustExec("begin pessimistic") err := tk2.ExecToErr("select * from tu where z = 3 for update nowait") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, txndriver.ErrLockAcquireFailAndNoWaitSet), IsTrue) + c.Assert(terror.ErrorEqual(err, storeerr.ErrLockAcquireFailAndNoWaitSet), IsTrue) tk.MustExec("begin pessimistic") tk.MustExec("insert into tu(x, y) values(2, 2);") err = tk2.ExecToErr("select * from tu where z = 4 for update nowait") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, txndriver.ErrLockAcquireFailAndNoWaitSet), IsTrue) + c.Assert(terror.ErrorEqual(err, storeerr.ErrLockAcquireFailAndNoWaitSet), IsTrue) // test batch point get lock tk.MustExec("begin pessimistic") @@ -1443,12 +1443,12 @@ func (s *testPessimisticSuite) TestGenerateColPointGet(c *C) { tk2.MustExec("begin pessimistic") err = tk2.ExecToErr("select x from tu where z in (3, 7, 9) for update nowait") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, txndriver.ErrLockAcquireFailAndNoWaitSet), IsTrue) + c.Assert(terror.ErrorEqual(err, storeerr.ErrLockAcquireFailAndNoWaitSet), IsTrue) tk.MustExec("begin pessimistic") tk.MustExec("insert into tu(x, y) values(5, 6);") err = tk2.ExecToErr("select * from tu where z = 11 for update nowait") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, txndriver.ErrLockAcquireFailAndNoWaitSet), IsTrue) + c.Assert(terror.ErrorEqual(err, storeerr.ErrLockAcquireFailAndNoWaitSet), IsTrue) tk.MustExec("commit") tk2.MustExec("commit") @@ -1996,11 +1996,11 @@ func (s *testPessimisticSuite) TestSelectForUpdateWaitSeconds(c *C) { waitErr2 := <-errCh waitErr3 := <-errCh c.Assert(waitErr, NotNil) - c.Check(waitErr.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) + c.Check(waitErr.Error(), Equals, storeerr.ErrLockWaitTimeout.Error()) c.Assert(waitErr2, NotNil) - c.Check(waitErr2.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) + c.Check(waitErr2.Error(), Equals, storeerr.ErrLockWaitTimeout.Error()) c.Assert(waitErr3, NotNil) - c.Check(waitErr3.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) + c.Check(waitErr3.Error(), Equals, storeerr.ErrLockWaitTimeout.Error()) c.Assert(time.Since(start).Seconds(), Less, 45.0) tk2.MustExec("commit") tk3.MustExec("rollback") diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index af224c59fc38a..1e6cdc3cf09bb 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/kv" - txndriver "github.com/pingcap/tidb/store/driver/txn" + derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" @@ -261,7 +261,7 @@ func (b *batchCopIterator) recvFromRespCh(ctx context.Context) (resp *batchCopRe return case <-ticker.C: if atomic.LoadUint32(b.vars.Killed) == 1 { - resp = &batchCopResponse{err: txndriver.ErrQueryInterrupted} + resp = &batchCopResponse{err: derr.ErrQueryInterrupted} ok = true return } @@ -387,7 +387,7 @@ func (b *batchCopIterator) handleStreamedBatchCopResponse(ctx context.Context, b } else { logutil.BgLogger().Info("stream unknown error", zap.Error(err)) } - return txndriver.ErrTiFlashServerTimeout + return derr.ErrTiFlashServerTimeout } } } diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 5e7eab303e84f..2c1e2d361af76 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -35,7 +35,7 @@ import ( "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" tidbmetrics "github.com/pingcap/tidb/metrics" - txndriver "github.com/pingcap/tidb/store/driver/txn" + derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" @@ -476,7 +476,7 @@ func (it *copIterator) recvFromRespCh(ctx context.Context, respCh <-chan *copRes return case <-ticker.C: if atomic.LoadUint32(it.vars.Killed) == 1 { - resp = &copResponse{err: txndriver.ErrQueryInterrupted} + resp = &copResponse{err: derr.ErrQueryInterrupted} ok = true return } @@ -717,7 +717,7 @@ func (worker *copIteratorWorker) handleTaskOnce(bo *backoffer, task *copTask, ch ops = append(ops, tikv.WithMatchLabels(worker.req.MatchStoreLabels)) } resp, rpcCtx, storeAddr, err := worker.kvclient.SendReqCtx(bo.TiKVBackoffer(), req, task.region, tikv.ReadTimeoutMedium, getEndPointType(task.storeType), task.storeAddr, ops...) - err = txndriver.ToTiDBErr(err) + err = derr.ToTiDBErr(err) if err != nil { if task.storeType == kv.TiDB { err = worker.handleTiDBSendReqErr(err, task, ch) @@ -874,7 +874,7 @@ func (worker *copIteratorWorker) handleCopResponse(bo *backoffer, rpcCtx *tikv.R logutil.BgLogger().Debug("coprocessor encounters", zap.Stringer("lock", lockErr)) msBeforeExpired, err1 := worker.kvclient.ResolveLocks(bo.TiKVBackoffer(), worker.req.StartTs, []*tikv.Lock{tikv.NewLock(lockErr)}) - err1 = txndriver.ToTiDBErr(err1) + err1 = derr.ToTiDBErr(err1) if err1 != nil { return nil, errors.Trace(err1) } @@ -982,11 +982,11 @@ type CopRuntimeStats struct { func (worker *copIteratorWorker) handleTiDBSendReqErr(err error, task *copTask, ch chan<- *copResponse) error { errCode := errno.ErrUnknown errMsg := err.Error() - if terror.ErrorEqual(err, txndriver.ErrTiKVServerTimeout) { + if terror.ErrorEqual(err, derr.ErrTiKVServerTimeout) { errCode = errno.ErrTiKVServerTimeout errMsg = "TiDB server timeout, address is " + task.storeAddr } - if terror.ErrorEqual(err, txndriver.ErrTiFlashServerTimeout) { + if terror.ErrorEqual(err, derr.ErrTiFlashServerTimeout) { errCode = errno.ErrTiFlashServerTimeout errMsg = "TiDB server timeout, address is " + task.storeAddr } diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 10784912faa9b..377e439a9392c 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/mpp" "github.com/pingcap/tidb/kv" - txndriver "github.com/pingcap/tidb/store/driver/txn" + derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/tikvrpc" @@ -225,7 +225,7 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *backoffer, req if sender.GetRPCError() != nil { logutil.BgLogger().Error("mpp dispatch meet io error", zap.String("error", sender.GetRPCError().Error())) // we return timeout to trigger tikv's fallback - m.sendError(txndriver.ErrTiFlashServerTimeout) + m.sendError(derr.ErrTiFlashServerTimeout) return } } else { @@ -235,7 +235,7 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *backoffer, req if err != nil { logutil.BgLogger().Error("mpp dispatch meet error", zap.String("error", err.Error())) // we return timeout to trigger tikv's fallback - m.sendError(txndriver.ErrTiFlashServerTimeout) + m.sendError(derr.ErrTiFlashServerTimeout) return } @@ -255,7 +255,7 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *backoffer, req failpoint.Inject("mppNonRootTaskError", func(val failpoint.Value) { if val.(bool) && !req.IsRoot { time.Sleep(1 * time.Second) - m.sendError(txndriver.ErrTiFlashServerTimeout) + m.sendError(derr.ErrTiFlashServerTimeout) return } }) @@ -318,7 +318,7 @@ func (m *mppIterator) establishMPPConns(bo *backoffer, req *kv.MPPDispatchReques if err != nil { logutil.BgLogger().Error("establish mpp connection meet error", zap.String("error", err.Error())) // we return timeout to trigger tikv's fallback - m.sendError(txndriver.ErrTiFlashServerTimeout) + m.sendError(derr.ErrTiFlashServerTimeout) return } @@ -350,7 +350,7 @@ func (m *mppIterator) establishMPPConns(bo *backoffer, req *kv.MPPDispatchReques logutil.BgLogger().Info("stream unknown error", zap.Error(err)) } } - m.sendError(txndriver.ErrTiFlashServerTimeout) + m.sendError(derr.ErrTiFlashServerTimeout) return } } @@ -405,7 +405,7 @@ func (m *mppIterator) nextImpl(ctx context.Context) (resp *mppResponse, ok bool, return case <-ticker.C: if m.vars != nil && m.vars.Killed != nil && atomic.LoadUint32(m.vars.Killed) == 1 { - err = txndriver.ErrQueryInterrupted + err = derr.ErrQueryInterrupted exit = true return } diff --git a/store/copr/store.go b/store/copr/store.go index 2cc10ee7bad38..d3f132f85238f 100644 --- a/store/copr/store.go +++ b/store/copr/store.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/kv" - txndriver "github.com/pingcap/tidb/store/driver/txn" + derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/config" "github.com/pingcap/tidb/store/tikv/tikvrpc" @@ -39,7 +39,7 @@ func (s *kvStore) GetRegionCache() *tikv.RegionCache { // CheckVisibility checks if it is safe to read using given ts. func (s *kvStore) CheckVisibility(startTime uint64) error { err := s.store.CheckVisibility(startTime) - return txndriver.ToTiDBErr(err) + return derr.ToTiDBErr(err) } // GetTiKVClient gets the client instance. @@ -54,13 +54,13 @@ type tikvClient struct { func (c *tikvClient) Close() error { err := c.c.Close() - return txndriver.ToTiDBErr(err) + return derr.ToTiDBErr(err) } // SendRequest sends Request. func (c *tikvClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { res, err := c.c.SendRequest(ctx, addr, req, timeout) - return res, txndriver.ToTiDBErr(err) + return res, derr.ToTiDBErr(err) } // Store wraps tikv.KVStore and provides coprocessor utilities. @@ -147,14 +147,14 @@ func (b *backoffer) TiKVBackoffer() *tikv.Backoffer { // It returns a retryable error if total sleep time exceeds maxSleep. func (b *backoffer) Backoff(typ tikv.BackoffType, err error) error { e := b.b.Backoff(typ, err) - return txndriver.ToTiDBErr(e) + return derr.ToTiDBErr(e) } // BackoffWithMaxSleep sleeps a while base on the backoffType and records the error message // and never sleep more than maxSleepMs for each sleep. func (b *backoffer) BackoffWithMaxSleep(typ tikv.BackoffType, maxSleepMs int, err error) error { e := b.b.BackoffWithMaxSleep(typ, maxSleepMs, err) - return txndriver.ToTiDBErr(e) + return derr.ToTiDBErr(e) } // GetBackoffTimes returns a map contains backoff time count by type. diff --git a/store/driver/error/error.go b/store/driver/error/error.go new file mode 100644 index 0000000000000..17da8f7ef2fa3 --- /dev/null +++ b/store/driver/error/error.go @@ -0,0 +1,158 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package error + +import ( + "github.com/pingcap/errors" + "github.com/pingcap/tidb/errno" + "github.com/pingcap/tidb/kv" + tikverr "github.com/pingcap/tidb/store/tikv/error" + "github.com/pingcap/tidb/util/dbterror" +) + +// tikv error instance +var ( + // ErrTokenLimit is the error that token is up to the limit. + ErrTokenLimit = dbterror.ClassTiKV.NewStd(errno.ErrTiKVStoreLimit) + // ErrTiKVServerTimeout is the error when tikv server is timeout. + ErrTiKVServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerTimeout) + ErrTiFlashServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiFlashServerTimeout) + // ErrGCTooEarly is the error that GC life time is shorter than transaction duration + ErrGCTooEarly = dbterror.ClassTiKV.NewStd(errno.ErrGCTooEarly) + // ErrTiKVStaleCommand is the error that the command is stale in tikv. + ErrTiKVStaleCommand = dbterror.ClassTiKV.NewStd(errno.ErrTiKVStaleCommand) + // ErrQueryInterrupted is the error when the query is interrupted. + ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(errno.ErrQueryInterrupted) + // ErrTiKVMaxTimestampNotSynced is the error that tikv's max timestamp is not synced. + ErrTiKVMaxTimestampNotSynced = dbterror.ClassTiKV.NewStd(errno.ErrTiKVMaxTimestampNotSynced) + // ErrLockAcquireFailAndNoWaitSet is the error that acquire the lock failed while no wait is setted. + ErrLockAcquireFailAndNoWaitSet = dbterror.ClassTiKV.NewStd(errno.ErrLockAcquireFailAndNoWaitSet) + ErrResolveLockTimeout = dbterror.ClassTiKV.NewStd(errno.ErrResolveLockTimeout) + // ErrLockWaitTimeout is the error that wait for the lock is timeout. + ErrLockWaitTimeout = dbterror.ClassTiKV.NewStd(errno.ErrLockWaitTimeout) + // ErrTiKVServerBusy is the error when tikv server is busy. + ErrTiKVServerBusy = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerBusy) + // ErrTiFlashServerBusy is the error that tiflash server is busy. + ErrTiFlashServerBusy = dbterror.ClassTiKV.NewStd(errno.ErrTiFlashServerBusy) + // ErrPDServerTimeout is the error when pd server is timeout. + ErrPDServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrPDServerTimeout) + // ErrRegionUnavailable is the error when region is not available. + ErrRegionUnavailable = dbterror.ClassTiKV.NewStd(errno.ErrRegionUnavailable) + // ErrUnknown is the unknow error. + ErrUnknown = dbterror.ClassTiKV.NewStd(errno.ErrUnknown) +) + +// Registers error returned from TiKV. +var ( + _ = dbterror.ClassTiKV.NewStd(errno.ErrDataOutOfRange) + _ = dbterror.ClassTiKV.NewStd(errno.ErrTruncatedWrongValue) + _ = dbterror.ClassTiKV.NewStd(errno.ErrDivisionByZero) +) + +// ToTiDBErr checks and converts a tikv error to a tidb error. +func ToTiDBErr(err error) error { + originErr := err + if err == nil { + return nil + } + err = errors.Cause(err) + if tikverr.IsErrNotFound(err) { + return kv.ErrNotExist + } + + if e, ok := err.(*tikverr.ErrWriteConflictInLatch); ok { + return kv.ErrWriteConflictInTiDB.FastGenByArgs(e.StartTS) + } + + if e, ok := err.(*tikverr.ErrTxnTooLarge); ok { + return kv.ErrTxnTooLarge.GenWithStackByArgs(e.Size) + } + + if errors.ErrorEqual(err, tikverr.ErrCannotSetNilValue) { + return kv.ErrCannotSetNilValue + } + + if e, ok := err.(*tikverr.ErrEntryTooLarge); ok { + return kv.ErrEntryTooLarge.GenWithStackByArgs(e.Limit, e.Size) + } + + if errors.ErrorEqual(err, tikverr.ErrInvalidTxn) { + return kv.ErrInvalidTxn + } + + if errors.ErrorEqual(err, tikverr.ErrTiKVServerTimeout) { + return ErrTiKVServerTimeout + } + + if e, ok := err.(*tikverr.ErrPDServerTimeout); ok { + if len(e.Error()) == 0 { + return ErrPDServerTimeout + } + return ErrPDServerTimeout.GenWithStackByArgs(e.Error()) + } + + if errors.ErrorEqual(err, tikverr.ErrTiFlashServerTimeout) { + return ErrTiFlashServerTimeout + } + + if errors.ErrorEqual(err, tikverr.ErrQueryInterrupted) { + return ErrQueryInterrupted + } + + if errors.ErrorEqual(err, tikverr.ErrTiKVServerBusy) { + return ErrTiKVServerBusy + } + + if errors.ErrorEqual(err, tikverr.ErrTiFlashServerBusy) { + return ErrTiFlashServerBusy + } + + if e, ok := err.(*tikverr.ErrGCTooEarly); ok { + return ErrGCTooEarly.GenWithStackByArgs(e.TxnStartTS, e.GCSafePoint) + } + + if errors.ErrorEqual(err, tikverr.ErrTiKVStaleCommand) { + return ErrTiKVStaleCommand + } + + if errors.ErrorEqual(err, tikverr.ErrTiKVMaxTimestampNotSynced) { + return ErrTiKVMaxTimestampNotSynced + } + + if errors.ErrorEqual(err, tikverr.ErrLockAcquireFailAndNoWaitSet) { + return ErrLockAcquireFailAndNoWaitSet + } + + if errors.ErrorEqual(err, tikverr.ErrResolveLockTimeout) { + return ErrResolveLockTimeout + } + + if errors.ErrorEqual(err, tikverr.ErrLockWaitTimeout) { + return ErrLockWaitTimeout + } + + if errors.ErrorEqual(err, tikverr.ErrRegionUnavailable) { + return ErrRegionUnavailable + } + + if e, ok := err.(*tikverr.ErrTokenLimit); ok { + return ErrTokenLimit.GenWithStackByArgs(e.StoreID) + } + + if errors.ErrorEqual(err, tikverr.ErrUnknown) { + return ErrUnknown + } + + return errors.Trace(originErr) +} diff --git a/store/driver/tikv_driver.go b/store/driver/tikv_driver.go index cb14736844e68..398be99520aa6 100644 --- a/store/driver/tikv_driver.go +++ b/store/driver/tikv_driver.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/copr" + derr "github.com/pingcap/tidb/store/driver/error" txn_driver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/gcworker" "github.com/pingcap/tidb/store/tikv" @@ -261,7 +262,7 @@ func (s *tikvStore) StartGCWorker() error { gcWorker, err := gcworker.NewGCWorker(s, s.pdClient) if err != nil { - return txn_driver.ToTiDBErr(err) + return derr.ToTiDBErr(err) } gcWorker.Start() s.gcWorker = gcWorker @@ -286,7 +287,7 @@ func (s *tikvStore) Close() error { } s.coprStore.Close() err := s.KVStore.Close() - return txn_driver.ToTiDBErr(err) + return derr.ToTiDBErr(err) } // GetMemCache return memory manager of the storage @@ -298,7 +299,7 @@ func (s *tikvStore) GetMemCache() kv.MemManager { func (s *tikvStore) Begin() (kv.Transaction, error) { txn, err := s.KVStore.Begin() if err != nil { - return nil, txn_driver.ToTiDBErr(err) + return nil, derr.ToTiDBErr(err) } return txn_driver.NewTiKVTxn(txn), err } @@ -307,7 +308,7 @@ func (s *tikvStore) Begin() (kv.Transaction, error) { func (s *tikvStore) BeginWithOption(option kv.TransactionOption) (kv.Transaction, error) { txn, err := s.KVStore.BeginWithOption(option) if err != nil { - return nil, txn_driver.ToTiDBErr(err) + return nil, derr.ToTiDBErr(err) } return txn_driver.NewTiKVTxn(txn), err } @@ -321,7 +322,7 @@ func (s *tikvStore) GetSnapshot(ver kv.Version) kv.Snapshot { // CurrentVersion returns current max committed version with the given txnScope (local or global). func (s *tikvStore) CurrentVersion(txnScope string) (kv.Version, error) { ver, err := s.KVStore.CurrentTimestamp(txnScope) - return kv.NewVersion(ver), txn_driver.ToTiDBErr(err) + return kv.NewVersion(ver), derr.ToTiDBErr(err) } // ShowStatus returns the specified status of the storage diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index 4c8e770c44ff7..39931357567be 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -25,56 +25,16 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" - "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" + derr "github.com/pingcap/tidb/store/driver/error" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/dbterror" "go.uber.org/zap" ) -// tikv error instance -var ( - // ErrTokenLimit is the error that token is up to the limit. - ErrTokenLimit = dbterror.ClassTiKV.NewStd(errno.ErrTiKVStoreLimit) - // ErrTiKVServerTimeout is the error when tikv server is timeout. - ErrTiKVServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerTimeout) - ErrTiFlashServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiFlashServerTimeout) - // ErrGCTooEarly is the error that GC life time is shorter than transaction duration - ErrGCTooEarly = dbterror.ClassTiKV.NewStd(errno.ErrGCTooEarly) - // ErrTiKVStaleCommand is the error that the command is stale in tikv. - ErrTiKVStaleCommand = dbterror.ClassTiKV.NewStd(errno.ErrTiKVStaleCommand) - // ErrQueryInterrupted is the error when the query is interrupted. - ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(errno.ErrQueryInterrupted) - // ErrTiKVMaxTimestampNotSynced is the error that tikv's max timestamp is not synced. - ErrTiKVMaxTimestampNotSynced = dbterror.ClassTiKV.NewStd(errno.ErrTiKVMaxTimestampNotSynced) - // ErrLockAcquireFailAndNoWaitSet is the error that acquire the lock failed while no wait is setted. - ErrLockAcquireFailAndNoWaitSet = dbterror.ClassTiKV.NewStd(errno.ErrLockAcquireFailAndNoWaitSet) - ErrResolveLockTimeout = dbterror.ClassTiKV.NewStd(errno.ErrResolveLockTimeout) - // ErrLockWaitTimeout is the error that wait for the lock is timeout. - ErrLockWaitTimeout = dbterror.ClassTiKV.NewStd(errno.ErrLockWaitTimeout) - // ErrTiKVServerBusy is the error when tikv server is busy. - ErrTiKVServerBusy = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerBusy) - // ErrTiFlashServerBusy is the error that tiflash server is busy. - ErrTiFlashServerBusy = dbterror.ClassTiKV.NewStd(errno.ErrTiFlashServerBusy) - // ErrPDServerTimeout is the error when pd server is timeout. - ErrPDServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrPDServerTimeout) - // ErrRegionUnavailable is the error when region is not available. - ErrRegionUnavailable = dbterror.ClassTiKV.NewStd(errno.ErrRegionUnavailable) - // ErrUnknown is the unknow error. - ErrUnknown = dbterror.ClassTiKV.NewStd(errno.ErrUnknown) -) - -// Registers error returned from TiKV. -var ( - _ = dbterror.ClassTiKV.NewStd(errno.ErrDataOutOfRange) - _ = dbterror.ClassTiKV.NewStd(errno.ErrTruncatedWrongValue) - _ = dbterror.ClassTiKV.NewStd(errno.ErrDivisionByZero) -) - func genKeyExistsError(name string, value string, err error) error { if err != nil { logutil.BgLogger().Info("extractKeyExistsErr meets error", zap.Error(err)) @@ -186,104 +146,7 @@ func extractKeyErr(err error) error { notFoundDetail := prettyLockNotFoundKey(e.Retryable) return kv.ErrTxnRetryable.GenWithStackByArgs(e.Retryable + " " + notFoundDetail) } - return ToTiDBErr(err) -} - -// ToTiDBErr checks and converts a tikv error to a tidb error. -func ToTiDBErr(err error) error { - originErr := err - if err == nil { - return nil - } - err = errors.Cause(err) - if tikverr.IsErrNotFound(err) { - return kv.ErrNotExist - } - - if e, ok := err.(*tikverr.ErrWriteConflictInLatch); ok { - return kv.ErrWriteConflictInTiDB.FastGenByArgs(e.StartTS) - } - - if e, ok := err.(*tikverr.ErrTxnTooLarge); ok { - return kv.ErrTxnTooLarge.GenWithStackByArgs(e.Size) - } - - if errors.ErrorEqual(err, tikverr.ErrCannotSetNilValue) { - return kv.ErrCannotSetNilValue - } - - if e, ok := err.(*tikverr.ErrEntryTooLarge); ok { - return kv.ErrEntryTooLarge.GenWithStackByArgs(e.Limit, e.Size) - } - - if errors.ErrorEqual(err, tikverr.ErrInvalidTxn) { - return kv.ErrInvalidTxn - } - - if errors.ErrorEqual(err, tikverr.ErrTiKVServerTimeout) { - return ErrTiKVServerTimeout - } - - if e, ok := err.(*tikverr.ErrPDServerTimeout); ok { - if len(e.Error()) == 0 { - return ErrPDServerTimeout - } - return ErrPDServerTimeout.GenWithStackByArgs(e.Error()) - } - - if errors.ErrorEqual(err, tikverr.ErrTiFlashServerTimeout) { - return ErrTiFlashServerTimeout - } - - if errors.ErrorEqual(err, tikverr.ErrQueryInterrupted) { - return ErrQueryInterrupted - } - - if errors.ErrorEqual(err, tikverr.ErrTiKVServerBusy) { - return ErrTiKVServerBusy - } - - if errors.ErrorEqual(err, tikverr.ErrTiFlashServerBusy) { - return ErrTiFlashServerBusy - } - - if e, ok := err.(*tikverr.ErrGCTooEarly); ok { - return ErrGCTooEarly.GenWithStackByArgs(e.TxnStartTS, e.GCSafePoint) - } - - if errors.ErrorEqual(err, tikverr.ErrTiKVStaleCommand) { - return ErrTiKVStaleCommand - } - - if errors.ErrorEqual(err, tikverr.ErrTiKVMaxTimestampNotSynced) { - return ErrTiKVMaxTimestampNotSynced - } - - if errors.ErrorEqual(err, tikverr.ErrLockAcquireFailAndNoWaitSet) { - return ErrLockAcquireFailAndNoWaitSet - } - - if errors.ErrorEqual(err, tikverr.ErrResolveLockTimeout) { - return ErrResolveLockTimeout - } - - if errors.ErrorEqual(err, tikverr.ErrLockWaitTimeout) { - return ErrLockWaitTimeout - } - - if errors.ErrorEqual(err, tikverr.ErrRegionUnavailable) { - return ErrRegionUnavailable - } - - if e, ok := err.(*tikverr.ErrTokenLimit); ok { - return ErrTokenLimit.GenWithStackByArgs(e.StoreID) - } - - if errors.ErrorEqual(err, tikverr.ErrUnknown) { - return ErrUnknown - } - - return errors.Trace(originErr) + return derr.ToTiDBErr(err) } func newWriteConflictError(conflict *kvrpcpb.WriteConflict) error { diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index 6064db02cdeba..5128259ab8e56 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -18,6 +18,7 @@ import ( "unsafe" "github.com/pingcap/tidb/kv" + derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" tikvstore "github.com/pingcap/tidb/store/tikv/kv" ) @@ -48,7 +49,7 @@ func (s *tikvSnapshot) Get(ctx context.Context, k kv.Key) ([]byte, error) { func (s *tikvSnapshot) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { scanner, err := s.KVSnapshot.Iter(k, upperBound) if err != nil { - return nil, ToTiDBErr(err) + return nil, derr.ToTiDBErr(err) } return &tikvScanner{scanner.(*tikv.Scanner)}, err } @@ -57,7 +58,7 @@ func (s *tikvSnapshot) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { func (s *tikvSnapshot) IterReverse(k kv.Key) (kv.Iterator, error) { scanner, err := s.KVSnapshot.IterReverse(k) if err != nil { - return nil, ToTiDBErr(err) + return nil, derr.ToTiDBErr(err) } return &tikvScanner{scanner.(*tikv.Scanner)}, err } diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 50bba80d2b54e..2983e6baa6242 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/binloginfo" + derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" tikvstore "github.com/pingcap/tidb/store/tikv/kv" @@ -75,7 +76,7 @@ func (txn *tikvTxn) GetSnapshot() kv.Snapshot { // The Iterator must be Closed after use. func (txn *tikvTxn) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { it, err := txn.KVTxn.Iter(k, upperBound) - return newKVIterator(it), ToTiDBErr(err) + return newKVIterator(it), derr.ToTiDBErr(err) } // IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. @@ -84,7 +85,7 @@ func (txn *tikvTxn) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { // TODO: Add lower bound limit func (txn *tikvTxn) IterReverse(k kv.Key) (kv.Iterator, error) { it, err := txn.KVTxn.IterReverse(k) - return newKVIterator(it), ToTiDBErr(err) + return newKVIterator(it), derr.ToTiDBErr(err) } // BatchGet gets kv from the memory buffer of statement and transaction, and the kv storage. @@ -101,17 +102,17 @@ func (txn *tikvTxn) BatchGet(ctx context.Context, keys []kv.Key) (map[string][]b func (txn *tikvTxn) Delete(k kv.Key) error { err := txn.KVTxn.Delete(k) - return ToTiDBErr(err) + return derr.ToTiDBErr(err) } func (txn *tikvTxn) Get(ctx context.Context, k kv.Key) ([]byte, error) { data, err := txn.KVTxn.Get(ctx, k) - return data, ToTiDBErr(err) + return data, derr.ToTiDBErr(err) } func (txn *tikvTxn) Set(k kv.Key, v []byte) error { err := txn.KVTxn.Set(k, v) - return ToTiDBErr(err) + return derr.ToTiDBErr(err) } func (txn *tikvTxn) GetMemBuffer() kv.MemBuffer { diff --git a/store/driver/txn/unionstore_driver.go b/store/driver/txn/unionstore_driver.go index 9db2325a0148f..5a2f56bfe4233 100644 --- a/store/driver/txn/unionstore_driver.go +++ b/store/driver/txn/unionstore_driver.go @@ -17,6 +17,7 @@ import ( "context" "github.com/pingcap/tidb/kv" + derr "github.com/pingcap/tidb/store/driver/error" tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/unionstore" ) @@ -39,17 +40,17 @@ func (m *memBuffer) Delete(k kv.Key) error { func (m *memBuffer) DeleteWithFlags(k kv.Key, ops ...tikvstore.FlagsOp) error { err := m.MemDB.DeleteWithFlags(k, ops...) - return ToTiDBErr(err) + return derr.ToTiDBErr(err) } func (m *memBuffer) Get(_ context.Context, key kv.Key) ([]byte, error) { data, err := m.MemDB.Get(key) - return data, ToTiDBErr(err) + return data, derr.ToTiDBErr(err) } func (m *memBuffer) GetFlags(key kv.Key) (tikvstore.KeyFlags, error) { data, err := m.MemDB.GetFlags(key) - return data, ToTiDBErr(err) + return data, derr.ToTiDBErr(err) } func (m *memBuffer) Staging() kv.StagingHandle { @@ -73,12 +74,12 @@ func (m *memBuffer) InspectStage(handle kv.StagingHandle, f func(kv.Key, tikvsto func (m *memBuffer) Set(key kv.Key, value []byte) error { err := m.MemDB.Set(key, value) - return ToTiDBErr(err) + return derr.ToTiDBErr(err) } func (m *memBuffer) SetWithFlags(key kv.Key, value []byte, ops ...kv.FlagsOp) error { err := m.MemDB.SetWithFlags(key, value, ops...) - return ToTiDBErr(err) + return derr.ToTiDBErr(err) } // Iter creates an Iterator positioned on the first entry that k <= entry's key. @@ -87,7 +88,7 @@ func (m *memBuffer) SetWithFlags(key kv.Key, value []byte, ops ...kv.FlagsOp) er // The Iterator must be Closed after use. func (m *memBuffer) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { it, err := m.MemDB.Iter(k, upperBound) - return &tikvIterator{Iterator: it}, ToTiDBErr(err) + return &tikvIterator{Iterator: it}, derr.ToTiDBErr(err) } // IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. @@ -96,7 +97,7 @@ func (m *memBuffer) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { // TODO: Add lower bound limit func (m *memBuffer) IterReverse(k kv.Key) (kv.Iterator, error) { it, err := m.MemDB.IterReverse(k) - return &tikvIterator{Iterator: it}, ToTiDBErr(err) + return &tikvIterator{Iterator: it}, derr.ToTiDBErr(err) } // SnapshotIter returns a Iterator for a snapshot of MemBuffer. @@ -121,7 +122,7 @@ func (u *tikvUnionStore) GetMemBuffer() kv.MemBuffer { func (u *tikvUnionStore) Get(ctx context.Context, k kv.Key) ([]byte, error) { data, err := u.KVUnionStore.Get(ctx, k) - return data, ToTiDBErr(err) + return data, derr.ToTiDBErr(err) } func (u *tikvUnionStore) HasPresumeKeyNotExists(k kv.Key) bool { @@ -134,7 +135,7 @@ func (u *tikvUnionStore) UnmarkPresumeKeyNotExists(k kv.Key) { func (u *tikvUnionStore) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { it, err := u.KVUnionStore.Iter(k, upperBound) - return newKVIterator(it), ToTiDBErr(err) + return newKVIterator(it), derr.ToTiDBErr(err) } // IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. @@ -143,7 +144,7 @@ func (u *tikvUnionStore) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) // TODO: Add lower bound limit func (u *tikvUnionStore) IterReverse(k kv.Key) (kv.Iterator, error) { it, err := u.KVUnionStore.IterReverse(k) - return newKVIterator(it), ToTiDBErr(err) + return newKVIterator(it), derr.ToTiDBErr(err) } type tikvGetter struct { @@ -156,7 +157,7 @@ func newKVGetter(getter unionstore.Getter) kv.Getter { func (g *tikvGetter) Get(_ context.Context, k kv.Key) ([]byte, error) { data, err := g.Getter.Get(k) - return data, ToTiDBErr(err) + return data, derr.ToTiDBErr(err) } // tikvIterator wraps unionstore.Iterator as kv.Iterator