mirror of https://github.com/tikv/client-go.git
*: replace pingcap/errors with pkg/errors (#357)
Signed-off-by: disksing <i@disksing.com>
This commit is contained in:
parent
cef46d13cc
commit
9ec50224be
|
|
@ -40,7 +40,7 @@ import (
|
|||
"strings"
|
||||
"sync/atomic"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/tikv/client-go/v2/internal/logutil"
|
||||
"github.com/tikv/client-go/v2/oracle"
|
||||
"github.com/tikv/client-go/v2/util"
|
||||
|
|
|
|||
|
|
@ -39,7 +39,7 @@ import (
|
|||
"crypto/x509"
|
||||
"io/ioutil"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
// Security is the security section of the config.
|
||||
|
|
|
|||
|
|
@ -38,10 +38,10 @@ import (
|
|||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pingcap/kvproto/pkg/pdpb"
|
||||
"github.com/pingcap/log"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/tikv/client-go/v2/internal/logutil"
|
||||
"github.com/tikv/client-go/v2/util"
|
||||
"go.uber.org/zap"
|
||||
|
|
@ -97,7 +97,7 @@ const MismatchClusterID = "mismatch cluster id"
|
|||
|
||||
// IsErrNotFound checks if err is a kind of NotFound error.
|
||||
func IsErrNotFound(err error) bool {
|
||||
return errors.ErrorEqual(err, ErrNotExist)
|
||||
return errors.Is(err, ErrNotExist)
|
||||
}
|
||||
|
||||
// ErrDeadlock wraps *kvrpcpb.Deadlock to implement the error interface.
|
||||
|
|
@ -267,7 +267,7 @@ func ExtractKeyErr(keyErr *kvrpcpb.KeyError) error {
|
|||
|
||||
// IsErrorUndetermined checks if the error is undetermined error.
|
||||
func IsErrorUndetermined(err error) bool {
|
||||
return errors.Cause(err) == ErrResultUndetermined
|
||||
return errors.Is(err, ErrResultUndetermined)
|
||||
}
|
||||
|
||||
// Log logs the error if it is not nil.
|
||||
|
|
|
|||
2
go.mod
2
go.mod
|
|
@ -15,11 +15,11 @@ require (
|
|||
github.com/google/uuid v1.1.1
|
||||
github.com/grpc-ecosystem/go-grpc-middleware v1.1.0
|
||||
github.com/opentracing/opentracing-go v1.1.0
|
||||
github.com/pingcap/errors v0.11.5-0.20201126102027-b0a155152ca3
|
||||
github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd
|
||||
github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989
|
||||
github.com/pingcap/kvproto v0.0.0-20210915062418-0f5764a128ad
|
||||
github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7
|
||||
github.com/pkg/errors v0.9.1
|
||||
github.com/prometheus/client_golang v1.5.1
|
||||
github.com/prometheus/client_model v0.2.0
|
||||
github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 // indirect
|
||||
|
|
|
|||
|
|
@ -48,10 +48,10 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/ninedraft/israce"
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/failpoint"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
drivertxn "github.com/pingcap/tidb/store/driver/txn"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/stretchr/testify/suite"
|
||||
"github.com/tikv/client-go/v2/config"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
|
|
@ -1432,7 +1432,7 @@ func (s *testCommitterSuite) TestFailCommitPrimaryRpcErrors() {
|
|||
s.Nil(err)
|
||||
err = t1.Commit(context.Background())
|
||||
s.NotNil(err)
|
||||
s.True(tikverr.IsErrorUndetermined(err), errors.ErrorStack(err))
|
||||
s.True(tikverr.IsErrorUndetermined(err), errors.WithStack(err))
|
||||
|
||||
// We don't need to call "Rollback" after "Commit" fails.
|
||||
err = t1.Rollback()
|
||||
|
|
@ -1453,7 +1453,7 @@ func (s *testCommitterSuite) TestFailCommitPrimaryRegionError() {
|
|||
s.Nil(err)
|
||||
err = t2.Commit(context.Background())
|
||||
s.NotNil(err)
|
||||
s.False(tikverr.IsErrorUndetermined(err), errors.ErrorStack(err))
|
||||
s.False(tikverr.IsErrorUndetermined(err), errors.WithStack(err))
|
||||
}
|
||||
|
||||
// TestFailCommitPrimaryRPCErrorThenRegionError tests the case when commit first
|
||||
|
|
@ -1469,7 +1469,7 @@ func (s *testCommitterSuite) TestFailCommitPrimaryRPCErrorThenRegionError() {
|
|||
s.Nil(err)
|
||||
err = t1.Commit(context.Background())
|
||||
s.NotNil(err)
|
||||
s.True(tikverr.IsErrorUndetermined(err), errors.ErrorStack(err))
|
||||
s.True(tikverr.IsErrorUndetermined(err), errors.WithStack(err))
|
||||
}
|
||||
|
||||
// TestFailCommitPrimaryKeyError tests KeyError is handled properly when
|
||||
|
|
|
|||
|
|
@ -41,9 +41,9 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/failpoint"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/stretchr/testify/suite"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
"github.com/tikv/client-go/v2/tikv"
|
||||
|
|
@ -87,7 +87,7 @@ func (s *testAsyncCommitFailSuite) TestFailAsyncCommitPrewriteRpcErrors() {
|
|||
ctx := context.WithValue(context.Background(), util.SessionID, uint64(1))
|
||||
err = t1.Commit(ctx)
|
||||
s.NotNil(err)
|
||||
s.True(tikverr.IsErrorUndetermined(err), errors.ErrorStack(err))
|
||||
s.True(tikverr.IsErrorUndetermined(err), errors.WithStack(err))
|
||||
|
||||
// We don't need to call "Rollback" after "Commit" fails.
|
||||
err = t1.Rollback()
|
||||
|
|
@ -130,7 +130,7 @@ func (s *testAsyncCommitFailSuite) TestAsyncCommitPrewriteCancelled() {
|
|||
err = t1.Commit(ctx)
|
||||
s.NotNil(err)
|
||||
_, ok := errors.Cause(err).(*tikverr.ErrWriteConflict)
|
||||
s.True(ok, errors.ErrorStack(err))
|
||||
s.True(ok, errors.WithStack(err))
|
||||
}
|
||||
|
||||
func (s *testAsyncCommitFailSuite) TestPointGetWithAsyncCommit() {
|
||||
|
|
|
|||
|
|
@ -43,9 +43,9 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pingcap/tidb/store/mockstore/unistore"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/stretchr/testify/suite"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
"github.com/tikv/client-go/v2/kv"
|
||||
|
|
|
|||
|
|
@ -18,8 +18,8 @@ import (
|
|||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pkg/errors"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
"github.com/tikv/client-go/v2/tikv"
|
||||
"github.com/tikv/client-go/v2/tikvrpc"
|
||||
|
|
|
|||
|
|
@ -4,11 +4,11 @@ go 1.16
|
|||
|
||||
require (
|
||||
github.com/ninedraft/israce v0.0.3
|
||||
github.com/pingcap/errors v0.11.5-0.20210425183316-da1aaba5fb63
|
||||
github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd
|
||||
github.com/pingcap/kvproto v0.0.0-20211011060348-d957056f1551
|
||||
github.com/pingcap/tidb v1.1.0-beta.0.20211031114450-f271148d792f
|
||||
github.com/pingcap/tidb/parser v0.0.0-20211031114450-f271148d792f // indirect
|
||||
github.com/pkg/errors v0.9.1
|
||||
github.com/stretchr/testify v1.7.0
|
||||
github.com/tikv/client-go/v2 v2.0.0
|
||||
github.com/tikv/pd v1.1.0-beta.0.20211029083450-e65f0c55b6ae
|
||||
|
|
|
|||
|
|
@ -44,10 +44,10 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/failpoint"
|
||||
deadlockpb "github.com/pingcap/kvproto/pkg/deadlock"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/stretchr/testify/suite"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
"github.com/tikv/client-go/v2/kv"
|
||||
|
|
|
|||
|
|
@ -41,7 +41,7 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/stretchr/testify/suite"
|
||||
"github.com/tikv/client-go/v2/error"
|
||||
"github.com/tikv/client-go/v2/tikv"
|
||||
|
|
|
|||
|
|
@ -42,9 +42,9 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/failpoint"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/stretchr/testify/suite"
|
||||
"github.com/tikv/client-go/v2/error"
|
||||
"github.com/tikv/client-go/v2/tikv"
|
||||
|
|
|
|||
|
|
@ -39,10 +39,10 @@ import (
|
|||
"sync"
|
||||
"testing"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/metapb"
|
||||
"github.com/pingcap/kvproto/pkg/pdpb"
|
||||
"github.com/pingcap/tidb/store/mockstore/mockcopr"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/stretchr/testify/suite"
|
||||
"github.com/tikv/client-go/v2/testutils"
|
||||
"github.com/tikv/client-go/v2/tikv"
|
||||
|
|
|
|||
|
|
@ -48,11 +48,11 @@ import (
|
|||
|
||||
grpc_opentracing "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing"
|
||||
"github.com/opentracing/opentracing-go"
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/coprocessor"
|
||||
"github.com/pingcap/kvproto/pkg/debugpb"
|
||||
"github.com/pingcap/kvproto/pkg/mpp"
|
||||
"github.com/pingcap/kvproto/pkg/tikvpb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
"github.com/tikv/client-go/v2/config"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
|
|
|
|||
|
|
@ -43,8 +43,8 @@ import (
|
|||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/tikvpb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
"github.com/tikv/client-go/v2/config"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
|
|
@ -432,7 +432,7 @@ func (s *batchCommandsStream) recv() (resp *tikvpb.BatchCommandsResponse, err er
|
|||
logutil.BgLogger().Error("batchCommandsClient.recv panic",
|
||||
zap.Reflect("r", r),
|
||||
zap.Stack("stack"))
|
||||
err = errors.SuspendStack(errors.New("batch conn recv paniced"))
|
||||
err = errors.New("batch conn recv paniced")
|
||||
}
|
||||
if err == nil {
|
||||
metrics.BatchRecvHistogramOK.Observe(float64(time.Since(now)))
|
||||
|
|
@ -779,7 +779,7 @@ func sendBatchRequest(
|
|||
zap.String("to", addr), zap.String("cause", ctx.Err().Error()))
|
||||
return nil, errors.WithStack(ctx.Err())
|
||||
case <-timer.C:
|
||||
return nil, errors.SuspendStack(errors.Annotate(context.DeadlineExceeded, "wait sendLoop"))
|
||||
return nil, errors.WithMessage(context.DeadlineExceeded, "wait sendLoop")
|
||||
}
|
||||
metrics.TiKVBatchWaitDuration.Observe(float64(time.Since(start)))
|
||||
|
||||
|
|
@ -796,7 +796,7 @@ func sendBatchRequest(
|
|||
return nil, errors.WithStack(ctx.Err())
|
||||
case <-timer.C:
|
||||
atomic.StoreInt32(&entry.canceled, 1)
|
||||
return nil, errors.SuspendStack(errors.Annotate(context.DeadlineExceeded, "wait recvLoop"))
|
||||
return nil, errors.WithMessage(context.DeadlineExceeded, "wait recvLoop")
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -40,7 +40,7 @@ import (
|
|||
"strconv"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/tikv/client-go/v2/tikvrpc"
|
||||
"golang.org/x/sync/singleflight"
|
||||
)
|
||||
|
|
|
|||
|
|
@ -42,11 +42,11 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/coprocessor"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pingcap/kvproto/pkg/metapb"
|
||||
"github.com/pingcap/kvproto/pkg/tikvpb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
"github.com/tikv/client-go/v2/config"
|
||||
|
|
|
|||
|
|
@ -37,8 +37,8 @@ package locate
|
|||
import (
|
||||
"context"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/metapb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/tikv/client-go/v2/util/codec"
|
||||
pd "github.com/tikv/pd/client"
|
||||
)
|
||||
|
|
|
|||
|
|
@ -48,8 +48,8 @@ import (
|
|||
"github.com/gogo/protobuf/proto"
|
||||
"github.com/google/btree"
|
||||
"github.com/opentracing/opentracing-go"
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/metapb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/tikv/client-go/v2/config"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
"github.com/tikv/client-go/v2/internal/client"
|
||||
|
|
|
|||
|
|
@ -49,11 +49,11 @@ import (
|
|||
"google.golang.org/grpc/status"
|
||||
|
||||
"github.com/opentracing/opentracing-go"
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/coprocessor"
|
||||
"github.com/pingcap/kvproto/pkg/errorpb"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pingcap/kvproto/pkg/metapb"
|
||||
"github.com/pkg/errors"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
"github.com/tikv/client-go/v2/internal/client"
|
||||
"github.com/tikv/client-go/v2/internal/logutil"
|
||||
|
|
|
|||
|
|
@ -41,10 +41,10 @@ import (
|
|||
"time"
|
||||
"unsafe"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/errorpb"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pingcap/kvproto/pkg/metapb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/stretchr/testify/suite"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
"github.com/tikv/client-go/v2/internal/mockstore/mocktikv"
|
||||
|
|
|
|||
|
|
@ -43,13 +43,13 @@ import (
|
|||
"time"
|
||||
"unsafe"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/coprocessor"
|
||||
"github.com/pingcap/kvproto/pkg/errorpb"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pingcap/kvproto/pkg/metapb"
|
||||
"github.com/pingcap/kvproto/pkg/mpp"
|
||||
"github.com/pingcap/kvproto/pkg/tikvpb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/stretchr/testify/suite"
|
||||
"github.com/tikv/client-go/v2/internal/client"
|
||||
"github.com/tikv/client-go/v2/internal/mockstore/mocktikv"
|
||||
|
|
|
|||
|
|
@ -38,8 +38,8 @@ import (
|
|||
"math"
|
||||
"testing"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
|
|
|||
|
|
@ -41,8 +41,8 @@ import (
|
|||
"math"
|
||||
|
||||
"github.com/google/btree"
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/tikv/client-go/v2/util/codec"
|
||||
)
|
||||
|
||||
|
|
|
|||
|
|
@ -40,13 +40,13 @@ import (
|
|||
"sync"
|
||||
|
||||
"github.com/dgryski/go-farm"
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/goleveldb/leveldb"
|
||||
"github.com/pingcap/goleveldb/leveldb/iterator"
|
||||
"github.com/pingcap/goleveldb/leveldb/opt"
|
||||
"github.com/pingcap/goleveldb/leveldb/storage"
|
||||
"github.com/pingcap/goleveldb/leveldb/util"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pkg/errors"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
"github.com/tikv/client-go/v2/internal/logutil"
|
||||
"github.com/tikv/client-go/v2/internal/mockstore/deadlock"
|
||||
|
|
|
|||
|
|
@ -41,9 +41,9 @@ import (
|
|||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/metapb"
|
||||
"github.com/pingcap/kvproto/pkg/pdpb"
|
||||
"github.com/pkg/errors"
|
||||
pd "github.com/tikv/pd/client"
|
||||
)
|
||||
|
||||
|
|
|
|||
|
|
@ -42,12 +42,12 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/opentracing/opentracing-go"
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/coprocessor"
|
||||
"github.com/pingcap/kvproto/pkg/debugpb"
|
||||
"github.com/pingcap/kvproto/pkg/errorpb"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pingcap/kvproto/pkg/metapb"
|
||||
"github.com/pkg/errors"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
"github.com/tikv/client-go/v2/tikvrpc"
|
||||
"github.com/tikv/client-go/v2/util"
|
||||
|
|
|
|||
|
|
@ -43,8 +43,8 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/opentracing/opentracing-go"
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/log"
|
||||
"github.com/pkg/errors"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
"github.com/tikv/client-go/v2/internal/logutil"
|
||||
"github.com/tikv/client-go/v2/kv"
|
||||
|
|
|
|||
|
|
@ -39,7 +39,7 @@ import (
|
|||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/tikv/client-go/v2/oracle"
|
||||
)
|
||||
|
||||
|
|
|
|||
|
|
@ -41,7 +41,7 @@ import (
|
|||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/tikv/client-go/v2/internal/logutil"
|
||||
"github.com/tikv/client-go/v2/metrics"
|
||||
"github.com/tikv/client-go/v2/oracle"
|
||||
|
|
|
|||
|
|
@ -39,8 +39,8 @@ import (
|
|||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/tikv/client-go/v2/config"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
"github.com/tikv/client-go/v2/internal/client"
|
||||
|
|
|
|||
|
|
@ -20,9 +20,9 @@ import (
|
|||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pingcap/kvproto/pkg/metapb"
|
||||
"github.com/pkg/errors"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
"github.com/tikv/client-go/v2/internal/locate"
|
||||
"github.com/tikv/client-go/v2/internal/logutil"
|
||||
|
|
|
|||
|
|
@ -46,9 +46,9 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/opentracing/opentracing-go"
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pingcap/kvproto/pkg/metapb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/tikv/client-go/v2/config"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
"github.com/tikv/client-go/v2/internal/client"
|
||||
|
|
|
|||
|
|
@ -42,7 +42,7 @@ import (
|
|||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/tikv/client-go/v2/internal/logutil"
|
||||
"go.etcd.io/etcd/clientv3"
|
||||
"go.etcd.io/etcd/mvcc/mvccpb"
|
||||
|
|
|
|||
|
|
@ -40,10 +40,10 @@ import (
|
|||
"fmt"
|
||||
"math"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pingcap/kvproto/pkg/metapb"
|
||||
"github.com/pingcap/kvproto/pkg/pdpb"
|
||||
"github.com/pkg/errors"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
"github.com/tikv/client-go/v2/internal/client"
|
||||
"github.com/tikv/client-go/v2/internal/kvrpc"
|
||||
|
|
@ -289,7 +289,7 @@ func (s *KVStore) WaitScatterRegionFinish(ctx context.Context, regionID uint64,
|
|||
return nil
|
||||
}
|
||||
if resp.GetHeader().GetError() != nil {
|
||||
err = errors.AddStack(&tikverr.PDError{
|
||||
err = errors.WithStack(&tikverr.PDError{
|
||||
Err: resp.Header.Error,
|
||||
})
|
||||
logutil.BgLogger().Warn("wait scatter region error",
|
||||
|
|
|
|||
|
|
@ -39,7 +39,6 @@ import (
|
|||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/coprocessor"
|
||||
"github.com/pingcap/kvproto/pkg/debugpb"
|
||||
"github.com/pingcap/kvproto/pkg/errorpb"
|
||||
|
|
@ -47,6 +46,7 @@ import (
|
|||
"github.com/pingcap/kvproto/pkg/metapb"
|
||||
"github.com/pingcap/kvproto/pkg/mpp"
|
||||
"github.com/pingcap/kvproto/pkg/tikvpb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/tikv/client-go/v2/kv"
|
||||
"github.com/tikv/client-go/v2/oracle"
|
||||
)
|
||||
|
|
|
|||
|
|
@ -39,8 +39,8 @@ import (
|
|||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pkg/errors"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
"github.com/tikv/client-go/v2/internal/client"
|
||||
"github.com/tikv/client-go/v2/internal/locate"
|
||||
|
|
|
|||
|
|
@ -41,7 +41,7 @@ import (
|
|||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/tikv/client-go/v2/internal/logutil"
|
||||
"github.com/tikv/client-go/v2/internal/retry"
|
||||
"github.com/tikv/client-go/v2/kv"
|
||||
|
|
|
|||
|
|
@ -46,8 +46,8 @@ import (
|
|||
"time"
|
||||
"unsafe"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
"github.com/tikv/client-go/v2/config"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
|
|
|
|||
|
|
@ -35,8 +35,8 @@
|
|||
package transaction
|
||||
|
||||
import (
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
"github.com/tikv/client-go/v2/internal/client"
|
||||
"github.com/tikv/client-go/v2/internal/logutil"
|
||||
|
|
|
|||
|
|
@ -39,8 +39,8 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/opentracing/opentracing-go"
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
"github.com/tikv/client-go/v2/internal/client"
|
||||
|
|
|
|||
|
|
@ -41,8 +41,8 @@ import (
|
|||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
"github.com/tikv/client-go/v2/internal/client"
|
||||
|
|
|
|||
|
|
@ -42,8 +42,8 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/opentracing/opentracing-go"
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
"github.com/tikv/client-go/v2/config"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
|
|
|
|||
|
|
@ -48,9 +48,9 @@ import (
|
|||
|
||||
"github.com/dgryski/go-farm"
|
||||
"github.com/opentracing/opentracing-go"
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/failpoint"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/tikv/client-go/v2/config"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
"github.com/tikv/client-go/v2/internal/logutil"
|
||||
|
|
|
|||
|
|
@ -24,8 +24,8 @@ import (
|
|||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/tikv/client-go/v2/config"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
"github.com/tikv/client-go/v2/internal/client"
|
||||
|
|
|
|||
|
|
@ -15,8 +15,8 @@
|
|||
package txnlock
|
||||
|
||||
import (
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/tikv/client-go/v2/internal/locate"
|
||||
"github.com/tikv/client-go/v2/internal/retry"
|
||||
)
|
||||
|
|
|
|||
|
|
@ -38,8 +38,8 @@ import (
|
|||
"bytes"
|
||||
"context"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pkg/errors"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
"github.com/tikv/client-go/v2/internal/client"
|
||||
"github.com/tikv/client-go/v2/internal/locate"
|
||||
|
|
|
|||
|
|
@ -44,10 +44,10 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/opentracing/opentracing-go"
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/failpoint"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pingcap/kvproto/pkg/metapb"
|
||||
"github.com/pkg/errors"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
"github.com/tikv/client-go/v2/internal/client"
|
||||
"github.com/tikv/client-go/v2/internal/locate"
|
||||
|
|
|
|||
|
|
@ -38,7 +38,7 @@ import (
|
|||
"runtime"
|
||||
"unsafe"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
const (
|
||||
|
|
|
|||
|
|
@ -38,7 +38,7 @@ import (
|
|||
"encoding/binary"
|
||||
"math"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
const signMask uint64 = 0x8000000000000000
|
||||
|
|
|
|||
|
|
@ -41,7 +41,7 @@ import (
|
|||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/tikv/client-go/v2/internal/logutil"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
|
|
|||
Loading…
Reference in New Issue