mirror of https://github.com/tikv/client-go.git
config: enable user configuration for retry strategy. (#1055)
Signed-off-by: Smityz <smityz@qq.com> Co-authored-by: disksing <i@disksing.com>
This commit is contained in:
parent
8fc8a52821
commit
6ba909c4ad
|
|
@ -0,0 +1,207 @@
|
||||||
|
// Copyright 2021 TiKV Authors
|
||||||
|
//
|
||||||
|
// 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,
|
||||||
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
// See the License for the specific language governing permissions and
|
||||||
|
// limitations under the License.
|
||||||
|
|
||||||
|
// NOTE: The code in this file is based on code from the
|
||||||
|
// TiDB project, licensed under the Apache License v 2.0
|
||||||
|
//
|
||||||
|
// https://github.com/pingcap/tidb/tree/cc5e161ac06827589c4966674597c137cc9e809c/store/tikv/retry/config.go
|
||||||
|
//
|
||||||
|
|
||||||
|
// 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,
|
||||||
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
// See the License for the specific language governing permissions and
|
||||||
|
// limitations under the License.
|
||||||
|
|
||||||
|
package retry
|
||||||
|
|
||||||
|
import (
|
||||||
|
"context"
|
||||||
|
"math"
|
||||||
|
"math/rand"
|
||||||
|
"strings"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
"github.com/prometheus/client_golang/prometheus"
|
||||||
|
tikverr "github.com/tikv/client-go/v2/error"
|
||||||
|
"github.com/tikv/client-go/v2/internal/logutil"
|
||||||
|
"github.com/tikv/client-go/v2/kv"
|
||||||
|
"github.com/tikv/client-go/v2/metrics"
|
||||||
|
"go.uber.org/zap"
|
||||||
|
)
|
||||||
|
|
||||||
|
// Config is the configuration of the Backoff function.
|
||||||
|
type Config struct {
|
||||||
|
name string
|
||||||
|
metric *prometheus.Observer
|
||||||
|
fnCfg *BackoffFnCfg
|
||||||
|
err error
|
||||||
|
}
|
||||||
|
|
||||||
|
// backoffFn is the backoff function which compute the sleep time and do sleep.
|
||||||
|
type backoffFn func(ctx context.Context, maxSleepMs int) int
|
||||||
|
|
||||||
|
func (c *Config) createBackoffFn(vars *kv.Variables) backoffFn {
|
||||||
|
if strings.EqualFold(c.name, txnLockFastName) {
|
||||||
|
return newBackoffFn(vars.BackoffLockFast, c.fnCfg.cap, c.fnCfg.jitter)
|
||||||
|
}
|
||||||
|
return newBackoffFn(c.fnCfg.base, c.fnCfg.cap, c.fnCfg.jitter)
|
||||||
|
}
|
||||||
|
|
||||||
|
// BackoffFnCfg is the configuration for the backoff func which implements exponential backoff with
|
||||||
|
// optional jitters.
|
||||||
|
// See http://www.awsarchitectureblog.com/2015/03/backoff.html
|
||||||
|
type BackoffFnCfg struct {
|
||||||
|
base int
|
||||||
|
cap int
|
||||||
|
jitter int
|
||||||
|
}
|
||||||
|
|
||||||
|
// NewBackoffFnCfg creates the config for BackoffFn.
|
||||||
|
func NewBackoffFnCfg(base, cap, jitter int) *BackoffFnCfg {
|
||||||
|
return &BackoffFnCfg{
|
||||||
|
base,
|
||||||
|
cap,
|
||||||
|
jitter,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// NewConfig creates a new Config for the Backoff operation.
|
||||||
|
func NewConfig(name string, metric *prometheus.Observer, backoffFnCfg *BackoffFnCfg, err error) *Config {
|
||||||
|
return &Config{
|
||||||
|
name: name,
|
||||||
|
metric: metric,
|
||||||
|
fnCfg: backoffFnCfg,
|
||||||
|
err: err,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (c *Config) String() string {
|
||||||
|
return c.name
|
||||||
|
}
|
||||||
|
|
||||||
|
// SetErrors sets a more detailed error instead of the default bo config.
|
||||||
|
func (c *Config) SetErrors(err error) {
|
||||||
|
c.err = err
|
||||||
|
}
|
||||||
|
|
||||||
|
func (c *Config) SetBackoffFnCfg(fnCfg *BackoffFnCfg) {
|
||||||
|
c.fnCfg = fnCfg
|
||||||
|
}
|
||||||
|
|
||||||
|
const txnLockFastName = "txnLockFast"
|
||||||
|
|
||||||
|
// Backoff Config variables.
|
||||||
|
var (
|
||||||
|
// TODO: distinguish tikv and tiflash in metrics
|
||||||
|
BoTiKVRPC = NewConfig("tikvRPC", &metrics.BackoffHistogramRPC, NewBackoffFnCfg(100, 2000, EqualJitter), tikverr.ErrTiKVServerTimeout)
|
||||||
|
BoTiFlashRPC = NewConfig("tiflashRPC", &metrics.BackoffHistogramRPC, NewBackoffFnCfg(100, 2000, EqualJitter), tikverr.ErrTiFlashServerTimeout)
|
||||||
|
BoTxnLock = NewConfig("txnLock", &metrics.BackoffHistogramLock, NewBackoffFnCfg(100, 3000, EqualJitter), tikverr.ErrResolveLockTimeout)
|
||||||
|
BoPDRPC = NewConfig("pdRPC", &metrics.BackoffHistogramPD, NewBackoffFnCfg(500, 3000, EqualJitter), tikverr.NewErrPDServerTimeout(""))
|
||||||
|
// change base time to 2ms, because it may recover soon.
|
||||||
|
BoRegionMiss = NewConfig("regionMiss", &metrics.BackoffHistogramRegionMiss, NewBackoffFnCfg(2, 500, NoJitter), tikverr.ErrRegionUnavailable)
|
||||||
|
BoRegionScheduling = NewConfig("regionScheduling", &metrics.BackoffHistogramRegionScheduling, NewBackoffFnCfg(2, 500, NoJitter), tikverr.ErrRegionUnavailable)
|
||||||
|
BoTiKVServerBusy = NewConfig("tikvServerBusy", &metrics.BackoffHistogramServerBusy, NewBackoffFnCfg(2000, 10000, EqualJitter), tikverr.ErrTiKVServerBusy)
|
||||||
|
BoTiKVDiskFull = NewConfig("tikvDiskFull", &metrics.BackoffHistogramTiKVDiskFull, NewBackoffFnCfg(500, 5000, NoJitter), tikverr.ErrTiKVDiskFull)
|
||||||
|
BoRegionRecoveryInProgress = NewConfig("regionRecoveryInProgress", &metrics.BackoffHistogramRegionRecoveryInProgress, NewBackoffFnCfg(100, 10000, EqualJitter), tikverr.ErrRegionRecoveryInProgress)
|
||||||
|
BoTiFlashServerBusy = NewConfig("tiflashServerBusy", &metrics.BackoffHistogramServerBusy, NewBackoffFnCfg(2000, 10000, EqualJitter), tikverr.ErrTiFlashServerBusy)
|
||||||
|
BoTxnNotFound = NewConfig("txnNotFound", &metrics.BackoffHistogramEmpty, NewBackoffFnCfg(2, 500, NoJitter), tikverr.ErrResolveLockTimeout)
|
||||||
|
BoStaleCmd = NewConfig("staleCommand", &metrics.BackoffHistogramStaleCmd, NewBackoffFnCfg(2, 1000, NoJitter), tikverr.ErrTiKVStaleCommand)
|
||||||
|
BoMaxTsNotSynced = NewConfig("maxTsNotSynced", &metrics.BackoffHistogramEmpty, NewBackoffFnCfg(2, 500, NoJitter), tikverr.ErrTiKVMaxTimestampNotSynced)
|
||||||
|
BoMaxDataNotReady = NewConfig("dataNotReady", &metrics.BackoffHistogramDataNotReady, NewBackoffFnCfg(2, 2000, NoJitter), tikverr.ErrRegionDataNotReady)
|
||||||
|
BoMaxRegionNotInitialized = NewConfig("regionNotInitialized", &metrics.BackoffHistogramEmpty, NewBackoffFnCfg(2, 1000, NoJitter), tikverr.ErrRegionNotInitialized)
|
||||||
|
BoIsWitness = NewConfig("isWitness", &metrics.BackoffHistogramIsWitness, NewBackoffFnCfg(1000, 10000, EqualJitter), tikverr.ErrIsWitness)
|
||||||
|
// TxnLockFast's `base` load from vars.BackoffLockFast when create BackoffFn.
|
||||||
|
BoTxnLockFast = NewConfig(txnLockFastName, &metrics.BackoffHistogramLockFast, NewBackoffFnCfg(2, 3000, EqualJitter), tikverr.ErrResolveLockTimeout)
|
||||||
|
)
|
||||||
|
|
||||||
|
var isSleepExcluded = map[string]int{
|
||||||
|
BoTiKVServerBusy.name: 600000, // The max excluded limit is 10min.
|
||||||
|
// add BoTiFlashServerBusy if appropriate
|
||||||
|
}
|
||||||
|
|
||||||
|
// setBackoffExcluded is used for test only.
|
||||||
|
func setBackoffExcluded(name string, maxVal int) {
|
||||||
|
if _, ok := isSleepExcluded[name]; ok {
|
||||||
|
isSleepExcluded[name] = maxVal
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
const (
|
||||||
|
// NoJitter makes the backoff sequence strict exponential.
|
||||||
|
NoJitter = 1 + iota
|
||||||
|
// FullJitter applies random factors to strict exponential.
|
||||||
|
FullJitter
|
||||||
|
// EqualJitter is also randomized, but prevents very short sleeps.
|
||||||
|
EqualJitter
|
||||||
|
// DecorrJitter increases the maximum jitter based on the last random value.
|
||||||
|
DecorrJitter
|
||||||
|
)
|
||||||
|
|
||||||
|
// newBackoffFn creates a backoff func which implements exponential backoff with
|
||||||
|
// optional jitters.
|
||||||
|
// See http://www.awsarchitectureblog.com/2015/03/backoff.html
|
||||||
|
func newBackoffFn(base, cap, jitter int) backoffFn {
|
||||||
|
if base < 2 {
|
||||||
|
// Top prevent panic in 'rand.Intn'.
|
||||||
|
base = 2
|
||||||
|
}
|
||||||
|
attempts := 0
|
||||||
|
lastSleep := base
|
||||||
|
return func(ctx context.Context, maxSleepMs int) int {
|
||||||
|
var sleep int
|
||||||
|
switch jitter {
|
||||||
|
case NoJitter:
|
||||||
|
sleep = expo(base, cap, attempts)
|
||||||
|
case FullJitter:
|
||||||
|
v := expo(base, cap, attempts)
|
||||||
|
sleep = rand.Intn(v)
|
||||||
|
case EqualJitter:
|
||||||
|
v := expo(base, cap, attempts)
|
||||||
|
sleep = v/2 + rand.Intn(v/2)
|
||||||
|
case DecorrJitter:
|
||||||
|
sleep = int(math.Min(float64(cap), float64(base+rand.Intn(lastSleep*3-base))))
|
||||||
|
}
|
||||||
|
logutil.BgLogger().Debug("backoff",
|
||||||
|
zap.Int("base", base),
|
||||||
|
zap.Int("sleep", sleep),
|
||||||
|
zap.Int("attempts", attempts))
|
||||||
|
|
||||||
|
realSleep := sleep
|
||||||
|
// when set maxSleepMs >= 0 in `tikv.BackoffWithMaxSleep` will force sleep maxSleepMs milliseconds.
|
||||||
|
if maxSleepMs >= 0 && realSleep > maxSleepMs {
|
||||||
|
realSleep = maxSleepMs
|
||||||
|
}
|
||||||
|
select {
|
||||||
|
case <-time.After(time.Duration(realSleep) * time.Millisecond):
|
||||||
|
attempts++
|
||||||
|
lastSleep = sleep
|
||||||
|
return realSleep
|
||||||
|
case <-ctx.Done():
|
||||||
|
return 0
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func expo(base, cap, n int) int {
|
||||||
|
return int(math.Min(float64(cap), float64(base)*math.Pow(2.0, float64(n))))
|
||||||
|
}
|
||||||
|
|
@ -48,9 +48,9 @@ import (
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
"github.com/prometheus/client_golang/prometheus"
|
"github.com/prometheus/client_golang/prometheus"
|
||||||
"github.com/tikv/client-go/v2/config"
|
"github.com/tikv/client-go/v2/config"
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
tikverr "github.com/tikv/client-go/v2/error"
|
tikverr "github.com/tikv/client-go/v2/error"
|
||||||
"github.com/tikv/client-go/v2/internal/logutil"
|
"github.com/tikv/client-go/v2/internal/logutil"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/metrics"
|
"github.com/tikv/client-go/v2/metrics"
|
||||||
"github.com/tikv/client-go/v2/tikvrpc"
|
"github.com/tikv/client-go/v2/tikvrpc"
|
||||||
"github.com/tikv/client-go/v2/util"
|
"github.com/tikv/client-go/v2/util"
|
||||||
|
|
|
||||||
|
|
@ -25,8 +25,8 @@ func TestMain(m *testing.M) {
|
||||||
util.EnableFailpoints()
|
util.EnableFailpoints()
|
||||||
opts := []goleak.Option{
|
opts := []goleak.Option{
|
||||||
goleak.IgnoreTopFunction("google.golang.org/grpc.(*ClientConn).WaitForStateChange"),
|
goleak.IgnoreTopFunction("google.golang.org/grpc.(*ClientConn).WaitForStateChange"),
|
||||||
goleak.IgnoreTopFunction("github.com/tikv/client-go/v2/internal/retry.newBackoffFn.func1"),
|
goleak.IgnoreTopFunction("github.com/tikv/client-go/v2/config/retry.newBackoffFn.func1"),
|
||||||
goleak.IgnoreTopFunction("github.com/tikv/client-go/v2/internal/retry.(*Config).createBackoffFn.newBackoffFn.func2"),
|
goleak.IgnoreTopFunction("github.com/tikv/client-go/v2/config/retry.(*Config).createBackoffFn.newBackoffFn.func2"),
|
||||||
}
|
}
|
||||||
goleak.VerifyTestMain(m, opts...)
|
goleak.VerifyTestMain(m, opts...)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,384 @@
|
||||||
|
// Copyright 2021 TiKV Authors
|
||||||
|
//
|
||||||
|
// 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,
|
||||||
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
// See the License for the specific language governing permissions and
|
||||||
|
// limitations under the License.
|
||||||
|
|
||||||
|
// NOTE: The code in this file is based on code from the
|
||||||
|
// TiDB project, licensed under the Apache License v 2.0
|
||||||
|
//
|
||||||
|
// https://github.com/pingcap/tidb/tree/cc5e161ac06827589c4966674597c137cc9e809c/store/tikv/retry/backoff.go
|
||||||
|
//
|
||||||
|
|
||||||
|
// Copyright 2016 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,
|
||||||
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
// See the License for the specific language governing permissions and
|
||||||
|
// limitations under the License.
|
||||||
|
|
||||||
|
package retry
|
||||||
|
|
||||||
|
import (
|
||||||
|
"bytes"
|
||||||
|
"context"
|
||||||
|
"fmt"
|
||||||
|
"math"
|
||||||
|
"strconv"
|
||||||
|
"strings"
|
||||||
|
"sync/atomic"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
"github.com/opentracing/opentracing-go"
|
||||||
|
"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"
|
||||||
|
"github.com/tikv/client-go/v2/util"
|
||||||
|
"go.uber.org/zap"
|
||||||
|
"go.uber.org/zap/zapcore"
|
||||||
|
)
|
||||||
|
|
||||||
|
// Backoffer is a utility for retrying queries.
|
||||||
|
type Backoffer struct {
|
||||||
|
ctx context.Context
|
||||||
|
|
||||||
|
fn map[string]backoffFn
|
||||||
|
maxSleep int
|
||||||
|
totalSleep int
|
||||||
|
excludedSleep int
|
||||||
|
|
||||||
|
vars *kv.Variables
|
||||||
|
noop bool
|
||||||
|
|
||||||
|
errors []error
|
||||||
|
configs []*Config
|
||||||
|
backoffSleepMS map[string]int
|
||||||
|
backoffTimes map[string]int
|
||||||
|
parent *Backoffer
|
||||||
|
}
|
||||||
|
|
||||||
|
type txnStartCtxKeyType struct{}
|
||||||
|
|
||||||
|
// TxnStartKey is a key for transaction start_ts info in context.Context.
|
||||||
|
var TxnStartKey interface{} = txnStartCtxKeyType{}
|
||||||
|
|
||||||
|
// NewBackoffer (Deprecated) creates a Backoffer with maximum sleep time(in ms).
|
||||||
|
func NewBackoffer(ctx context.Context, maxSleep int) *Backoffer {
|
||||||
|
return &Backoffer{
|
||||||
|
ctx: ctx,
|
||||||
|
maxSleep: maxSleep,
|
||||||
|
vars: kv.DefaultVars,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// NewBackofferWithVars creates a Backoffer with maximum sleep time(in ms) and kv.Variables.
|
||||||
|
func NewBackofferWithVars(ctx context.Context, maxSleep int, vars *kv.Variables) *Backoffer {
|
||||||
|
return NewBackoffer(ctx, maxSleep).withVars(vars)
|
||||||
|
}
|
||||||
|
|
||||||
|
// NewNoopBackoff create a Backoffer do nothing just return error directly
|
||||||
|
func NewNoopBackoff(ctx context.Context) *Backoffer {
|
||||||
|
return &Backoffer{ctx: ctx, noop: true}
|
||||||
|
}
|
||||||
|
|
||||||
|
// withVars sets the kv.Variables to the Backoffer and return it.
|
||||||
|
func (b *Backoffer) withVars(vars *kv.Variables) *Backoffer {
|
||||||
|
if vars != nil {
|
||||||
|
b.vars = vars
|
||||||
|
}
|
||||||
|
// maxSleep is the max sleep time in millisecond.
|
||||||
|
// When it is multiplied by BackOffWeight, it should not be greater than MaxInt32.
|
||||||
|
if b.maxSleep > 0 && math.MaxInt32/b.vars.BackOffWeight >= b.maxSleep {
|
||||||
|
b.maxSleep *= b.vars.BackOffWeight
|
||||||
|
}
|
||||||
|
return b
|
||||||
|
}
|
||||||
|
|
||||||
|
// Backoff sleeps a while base on the Config and records the error message.
|
||||||
|
// It returns a retryable error if total sleep time exceeds maxSleep.
|
||||||
|
func (b *Backoffer) Backoff(cfg *Config, err error) error {
|
||||||
|
if span := opentracing.SpanFromContext(b.ctx); span != nil && span.Tracer() != nil {
|
||||||
|
span1 := span.Tracer().StartSpan(fmt.Sprintf("tikv.backoff.%s", cfg), opentracing.ChildOf(span.Context()))
|
||||||
|
defer span1.Finish()
|
||||||
|
opentracing.ContextWithSpan(b.ctx, span1)
|
||||||
|
}
|
||||||
|
return b.BackoffWithCfgAndMaxSleep(cfg, -1, err)
|
||||||
|
}
|
||||||
|
|
||||||
|
// BackoffWithMaxSleepTxnLockFast sleeps a while base on the MaxSleepTxnLock and records the error message
|
||||||
|
// and never sleep more than maxSleepMs for each sleep.
|
||||||
|
func (b *Backoffer) BackoffWithMaxSleepTxnLockFast(maxSleepMs int, err error) error {
|
||||||
|
cfg := BoTxnLockFast
|
||||||
|
return b.BackoffWithCfgAndMaxSleep(cfg, maxSleepMs, err)
|
||||||
|
}
|
||||||
|
|
||||||
|
// BackoffWithCfgAndMaxSleep sleeps a while base on the Config and records the error message
|
||||||
|
// and never sleep more than maxSleepMs for each sleep.
|
||||||
|
func (b *Backoffer) BackoffWithCfgAndMaxSleep(cfg *Config, maxSleepMs int, err error) error {
|
||||||
|
if strings.Contains(err.Error(), tikverr.MismatchClusterID) {
|
||||||
|
logutil.Logger(b.ctx).Fatal("critical error", zap.Error(err))
|
||||||
|
}
|
||||||
|
select {
|
||||||
|
case <-b.ctx.Done():
|
||||||
|
return errors.WithStack(err)
|
||||||
|
default:
|
||||||
|
}
|
||||||
|
if b.noop {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
maxBackoffTimeExceeded := (b.totalSleep - b.excludedSleep) >= b.maxSleep
|
||||||
|
maxExcludedTimeExceeded := false
|
||||||
|
if maxLimit, ok := isSleepExcluded[cfg.name]; ok {
|
||||||
|
maxExcludedTimeExceeded = b.excludedSleep >= maxLimit && b.excludedSleep >= b.maxSleep
|
||||||
|
}
|
||||||
|
maxTimeExceeded := maxBackoffTimeExceeded || maxExcludedTimeExceeded
|
||||||
|
if b.maxSleep > 0 && maxTimeExceeded {
|
||||||
|
longestSleepCfg, longestSleepTime := b.longestSleepCfg()
|
||||||
|
errMsg := fmt.Sprintf("%s backoffer.maxSleep %dms is exceeded, errors:", cfg.String(), b.maxSleep)
|
||||||
|
for i, err := range b.errors {
|
||||||
|
// Print only last 3 errors for non-DEBUG log levels.
|
||||||
|
if log.GetLevel() == zapcore.DebugLevel || i >= len(b.errors)-3 {
|
||||||
|
errMsg += "\n" + err.Error()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
var backoffDetail bytes.Buffer
|
||||||
|
totalTimes := 0
|
||||||
|
for name, times := range b.backoffTimes {
|
||||||
|
totalTimes += times
|
||||||
|
if backoffDetail.Len() > 0 {
|
||||||
|
backoffDetail.WriteString(", ")
|
||||||
|
}
|
||||||
|
backoffDetail.WriteString(name)
|
||||||
|
backoffDetail.WriteString(":")
|
||||||
|
backoffDetail.WriteString(strconv.Itoa(times))
|
||||||
|
}
|
||||||
|
errMsg += fmt.Sprintf("\ntotal-backoff-times: %v, backoff-detail: %v, maxBackoffTimeExceeded: %v, maxExcludedTimeExceeded: %v",
|
||||||
|
totalTimes, backoffDetail.String(), maxBackoffTimeExceeded, maxExcludedTimeExceeded)
|
||||||
|
returnedErr := err
|
||||||
|
if longestSleepCfg != nil {
|
||||||
|
errMsg += fmt.Sprintf("\nlongest sleep type: %s, time: %dms", longestSleepCfg.String(), longestSleepTime)
|
||||||
|
returnedErr = longestSleepCfg.err
|
||||||
|
}
|
||||||
|
logutil.Logger(b.ctx).Warn(errMsg)
|
||||||
|
// Use the backoff type that contributes most to the timeout to generate a MySQL error.
|
||||||
|
return errors.WithStack(returnedErr)
|
||||||
|
}
|
||||||
|
b.errors = append(b.errors, errors.Errorf("%s at %s", err.Error(), time.Now().Format(time.RFC3339Nano)))
|
||||||
|
b.configs = append(b.configs, cfg)
|
||||||
|
|
||||||
|
// Lazy initialize.
|
||||||
|
if b.fn == nil {
|
||||||
|
b.fn = make(map[string]backoffFn)
|
||||||
|
}
|
||||||
|
f, ok := b.fn[cfg.name]
|
||||||
|
if !ok {
|
||||||
|
f = cfg.createBackoffFn(b.vars)
|
||||||
|
b.fn[cfg.name] = f
|
||||||
|
}
|
||||||
|
realSleep := f(b.ctx, maxSleepMs)
|
||||||
|
if cfg.metric != nil {
|
||||||
|
(*cfg.metric).Observe(float64(realSleep) / 1000)
|
||||||
|
}
|
||||||
|
|
||||||
|
b.totalSleep += realSleep
|
||||||
|
if _, ok := isSleepExcluded[cfg.name]; ok {
|
||||||
|
b.excludedSleep += realSleep
|
||||||
|
}
|
||||||
|
if b.backoffSleepMS == nil {
|
||||||
|
b.backoffSleepMS = make(map[string]int)
|
||||||
|
}
|
||||||
|
b.backoffSleepMS[cfg.name] += realSleep
|
||||||
|
if b.backoffTimes == nil {
|
||||||
|
b.backoffTimes = make(map[string]int)
|
||||||
|
}
|
||||||
|
b.backoffTimes[cfg.name]++
|
||||||
|
|
||||||
|
stmtExec := b.ctx.Value(util.ExecDetailsKey)
|
||||||
|
if stmtExec != nil {
|
||||||
|
detail := stmtExec.(*util.ExecDetails)
|
||||||
|
atomic.AddInt64(&detail.BackoffDuration, int64(realSleep)*int64(time.Millisecond))
|
||||||
|
atomic.AddInt64(&detail.BackoffCount, 1)
|
||||||
|
}
|
||||||
|
|
||||||
|
if b.vars != nil && b.vars.Killed != nil {
|
||||||
|
if atomic.LoadUint32(b.vars.Killed) == 1 {
|
||||||
|
return errors.WithStack(tikverr.ErrQueryInterrupted)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
var startTs interface{}
|
||||||
|
if ts := b.ctx.Value(TxnStartKey); ts != nil {
|
||||||
|
startTs = ts
|
||||||
|
}
|
||||||
|
logutil.Logger(b.ctx).Debug("retry later",
|
||||||
|
zap.Error(err),
|
||||||
|
zap.Int("totalSleep", b.totalSleep),
|
||||||
|
zap.Int("excludedSleep", b.excludedSleep),
|
||||||
|
zap.Int("maxSleep", b.maxSleep),
|
||||||
|
zap.Stringer("type", cfg),
|
||||||
|
zap.Reflect("txnStartTS", startTs))
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (b *Backoffer) String() string {
|
||||||
|
if b.totalSleep == 0 {
|
||||||
|
return ""
|
||||||
|
}
|
||||||
|
return fmt.Sprintf(" backoff(%dms %v)", b.totalSleep, b.configs)
|
||||||
|
}
|
||||||
|
|
||||||
|
// copyMapWithoutRecursive is only used to deep copy map fields in the Backoffer type.
|
||||||
|
func copyMapWithoutRecursive(srcMap map[string]int) map[string]int {
|
||||||
|
result := map[string]int{}
|
||||||
|
for k, v := range srcMap {
|
||||||
|
result[k] = v
|
||||||
|
}
|
||||||
|
return result
|
||||||
|
}
|
||||||
|
|
||||||
|
// Clone creates a new Backoffer which keeps current Backoffer's sleep time and errors, and shares
|
||||||
|
// current Backoffer's context.
|
||||||
|
// Some fields like `configs` and `vars` are concurrently used by all the backoffers in different threads,
|
||||||
|
// try not to modify the referenced content directly.
|
||||||
|
func (b *Backoffer) Clone() *Backoffer {
|
||||||
|
return &Backoffer{
|
||||||
|
ctx: b.ctx,
|
||||||
|
maxSleep: b.maxSleep,
|
||||||
|
totalSleep: b.totalSleep,
|
||||||
|
excludedSleep: b.excludedSleep,
|
||||||
|
vars: b.vars,
|
||||||
|
errors: append([]error{}, b.errors...),
|
||||||
|
configs: append([]*Config{}, b.configs...),
|
||||||
|
backoffSleepMS: copyMapWithoutRecursive(b.backoffSleepMS),
|
||||||
|
backoffTimes: copyMapWithoutRecursive(b.backoffTimes),
|
||||||
|
parent: b.parent,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Fork creates a new Backoffer which keeps current Backoffer's sleep time and errors, and holds
|
||||||
|
// a child context of current Backoffer's context.
|
||||||
|
// Some fields like `configs` and `vars` are concurrently used by all the backoffers in different threads,
|
||||||
|
// try not to modify the referenced content directly.
|
||||||
|
func (b *Backoffer) Fork() (*Backoffer, context.CancelFunc) {
|
||||||
|
ctx, cancel := context.WithCancel(b.ctx)
|
||||||
|
return &Backoffer{
|
||||||
|
ctx: ctx,
|
||||||
|
maxSleep: b.maxSleep,
|
||||||
|
totalSleep: b.totalSleep,
|
||||||
|
excludedSleep: b.excludedSleep,
|
||||||
|
errors: append([]error{}, b.errors...),
|
||||||
|
configs: append([]*Config{}, b.configs...),
|
||||||
|
backoffSleepMS: copyMapWithoutRecursive(b.backoffSleepMS),
|
||||||
|
backoffTimes: copyMapWithoutRecursive(b.backoffTimes),
|
||||||
|
vars: b.vars,
|
||||||
|
parent: b,
|
||||||
|
}, cancel
|
||||||
|
}
|
||||||
|
|
||||||
|
// GetVars returns the binded vars.
|
||||||
|
func (b *Backoffer) GetVars() *kv.Variables {
|
||||||
|
return b.vars
|
||||||
|
}
|
||||||
|
|
||||||
|
// GetTotalSleep returns total sleep time.
|
||||||
|
func (b *Backoffer) GetTotalSleep() int {
|
||||||
|
return b.totalSleep
|
||||||
|
}
|
||||||
|
|
||||||
|
// GetTypes returns type list of this backoff and all its ancestors.
|
||||||
|
func (b *Backoffer) GetTypes() []string {
|
||||||
|
typs := make([]string, 0, len(b.configs))
|
||||||
|
for b != nil {
|
||||||
|
for _, cfg := range b.configs {
|
||||||
|
typs = append(typs, cfg.String())
|
||||||
|
}
|
||||||
|
b = b.parent
|
||||||
|
}
|
||||||
|
return typs
|
||||||
|
}
|
||||||
|
|
||||||
|
// GetCtx returns the binded context.
|
||||||
|
func (b *Backoffer) GetCtx() context.Context {
|
||||||
|
return b.ctx
|
||||||
|
}
|
||||||
|
|
||||||
|
// SetCtx sets the binded context to ctx.
|
||||||
|
func (b *Backoffer) SetCtx(ctx context.Context) {
|
||||||
|
b.ctx = ctx
|
||||||
|
}
|
||||||
|
|
||||||
|
// GetBackoffTimes returns a map contains backoff time count by type.
|
||||||
|
func (b *Backoffer) GetBackoffTimes() map[string]int {
|
||||||
|
return b.backoffTimes
|
||||||
|
}
|
||||||
|
|
||||||
|
// GetTotalBackoffTimes returns the total backoff times of the backoffer.
|
||||||
|
func (b *Backoffer) GetTotalBackoffTimes() int {
|
||||||
|
total := 0
|
||||||
|
for _, time := range b.backoffTimes {
|
||||||
|
total += time
|
||||||
|
}
|
||||||
|
return total
|
||||||
|
}
|
||||||
|
|
||||||
|
// GetBackoffSleepMS returns a map contains backoff sleep time by type.
|
||||||
|
func (b *Backoffer) GetBackoffSleepMS() map[string]int {
|
||||||
|
return b.backoffSleepMS
|
||||||
|
}
|
||||||
|
|
||||||
|
// ErrorsNum returns the number of errors.
|
||||||
|
func (b *Backoffer) ErrorsNum() int {
|
||||||
|
return len(b.errors)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Reset resets the sleep state of the backoffer, so that following backoff
|
||||||
|
// can sleep shorter. The reason why we don't create a new backoffer is that
|
||||||
|
// backoffer is similar to context and it records some metrics that we
|
||||||
|
// want to record for an entire process which is composed of serveral stages.
|
||||||
|
func (b *Backoffer) Reset() {
|
||||||
|
b.fn = nil
|
||||||
|
b.totalSleep = 0
|
||||||
|
b.excludedSleep = 0
|
||||||
|
}
|
||||||
|
|
||||||
|
// ResetMaxSleep resets the sleep state and max sleep limit of the backoffer.
|
||||||
|
// It's used when switches to the next stage of the process.
|
||||||
|
func (b *Backoffer) ResetMaxSleep(maxSleep int) {
|
||||||
|
b.Reset()
|
||||||
|
b.maxSleep = maxSleep
|
||||||
|
b.withVars(b.vars)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (b *Backoffer) longestSleepCfg() (*Config, int) {
|
||||||
|
candidate := ""
|
||||||
|
maxSleep := 0
|
||||||
|
for cfgName, sleepTime := range b.backoffSleepMS {
|
||||||
|
if _, ok := isSleepExcluded[cfgName]; sleepTime > maxSleep && !ok {
|
||||||
|
maxSleep = sleepTime
|
||||||
|
candidate = cfgName
|
||||||
|
}
|
||||||
|
}
|
||||||
|
for _, cfg := range b.configs {
|
||||||
|
if cfg.name == candidate {
|
||||||
|
return cfg, maxSleep
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return nil, 0
|
||||||
|
}
|
||||||
|
|
@ -0,0 +1,112 @@
|
||||||
|
// Copyright 2021 TiKV Authors
|
||||||
|
//
|
||||||
|
// 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,
|
||||||
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
// See the License for the specific language governing permissions and
|
||||||
|
// limitations under the License.
|
||||||
|
|
||||||
|
// NOTE: The code in this file is based on code from the
|
||||||
|
// TiDB project, licensed under the Apache License v 2.0
|
||||||
|
//
|
||||||
|
// https://github.com/pingcap/tidb/tree/cc5e161ac06827589c4966674597c137cc9e809c/store/tikv/retry/backoff_test.go
|
||||||
|
//
|
||||||
|
|
||||||
|
// Copyright 2019 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,
|
||||||
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
// See the License for the specific language governing permissions and
|
||||||
|
// limitations under the License.
|
||||||
|
|
||||||
|
package retry
|
||||||
|
|
||||||
|
import (
|
||||||
|
"context"
|
||||||
|
"errors"
|
||||||
|
"testing"
|
||||||
|
|
||||||
|
"github.com/stretchr/testify/assert"
|
||||||
|
)
|
||||||
|
|
||||||
|
func TestBackoffWithMax(t *testing.T) {
|
||||||
|
b := NewBackofferWithVars(context.TODO(), 2000, nil)
|
||||||
|
err := b.BackoffWithMaxSleepTxnLockFast(5, errors.New("test"))
|
||||||
|
|
||||||
|
assert.Nil(t, err)
|
||||||
|
assert.Equal(t, 5, b.totalSleep)
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestBackoffErrorType(t *testing.T) {
|
||||||
|
// the actual maxSleep is multiplied by weight, which is 1600ms
|
||||||
|
b := NewBackofferWithVars(context.TODO(), 800, nil)
|
||||||
|
err := b.Backoff(BoRegionMiss, errors.New("region miss")) // 2ms sleep
|
||||||
|
assert.Nil(t, err)
|
||||||
|
// 6ms sleep at most in total
|
||||||
|
for i := 0; i < 2; i++ {
|
||||||
|
err = b.Backoff(BoMaxDataNotReady, errors.New("data not ready"))
|
||||||
|
assert.Nil(t, err)
|
||||||
|
}
|
||||||
|
// 100ms sleep at most in total
|
||||||
|
err = b.Backoff(BoRegionRecoveryInProgress, errors.New("recovery in progress"))
|
||||||
|
assert.Nil(t, err)
|
||||||
|
|
||||||
|
// sleep from ServerIsBusy is not counted
|
||||||
|
err = b.Backoff(BoTiKVServerBusy, errors.New("server is busy"))
|
||||||
|
assert.Nil(t, err)
|
||||||
|
// 1000ms sleep at most in total
|
||||||
|
err = b.Backoff(BoIsWitness, errors.New("peer is witness"))
|
||||||
|
assert.Nil(t, err)
|
||||||
|
// wait it exceed max sleep
|
||||||
|
for i := 0; i < 15; i++ {
|
||||||
|
err = b.Backoff(BoTxnNotFound, errors.New("txn not found"))
|
||||||
|
if err != nil {
|
||||||
|
// Next backoff should return error of backoff that sleeps for longest time.
|
||||||
|
assert.ErrorIs(t, err, BoTxnNotFound.err)
|
||||||
|
return
|
||||||
|
}
|
||||||
|
}
|
||||||
|
assert.Fail(t, "should not be here")
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestBackoffDeepCopy(t *testing.T) {
|
||||||
|
var err error
|
||||||
|
b := NewBackofferWithVars(context.TODO(), 4, nil)
|
||||||
|
// 700 ms sleep in total and the backoffer will return an error next time.
|
||||||
|
for i := 0; i < 3; i++ {
|
||||||
|
err = b.Backoff(BoMaxDataNotReady, errors.New("data not ready"))
|
||||||
|
assert.Nil(t, err)
|
||||||
|
}
|
||||||
|
bForked, cancel := b.Fork()
|
||||||
|
defer cancel()
|
||||||
|
bCloned := b.Clone()
|
||||||
|
for _, b := range []*Backoffer{bForked, bCloned} {
|
||||||
|
err = b.Backoff(BoTiKVRPC, errors.New("tikv rpc"))
|
||||||
|
assert.ErrorIs(t, err, BoMaxDataNotReady.err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestBackoffWithMaxExcludedExceed(t *testing.T) {
|
||||||
|
setBackoffExcluded(BoTiKVServerBusy.name, 1)
|
||||||
|
b := NewBackofferWithVars(context.TODO(), 1, nil)
|
||||||
|
err := b.Backoff(BoTiKVServerBusy, errors.New("server is busy"))
|
||||||
|
assert.Nil(t, err)
|
||||||
|
|
||||||
|
// As the total excluded sleep is greater than the max limited value, error should be returned.
|
||||||
|
err = b.Backoff(BoTiKVServerBusy, errors.New("server is busy"))
|
||||||
|
assert.NotNil(t, err)
|
||||||
|
assert.Greater(t, b.excludedSleep, b.maxSleep)
|
||||||
|
}
|
||||||
|
|
@ -55,11 +55,11 @@ import (
|
||||||
"github.com/pingcap/kvproto/pkg/metapb"
|
"github.com/pingcap/kvproto/pkg/metapb"
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
"github.com/tikv/client-go/v2/config"
|
"github.com/tikv/client-go/v2/config"
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
tikverr "github.com/tikv/client-go/v2/error"
|
tikverr "github.com/tikv/client-go/v2/error"
|
||||||
"github.com/tikv/client-go/v2/internal/apicodec"
|
"github.com/tikv/client-go/v2/internal/apicodec"
|
||||||
"github.com/tikv/client-go/v2/internal/client"
|
"github.com/tikv/client-go/v2/internal/client"
|
||||||
"github.com/tikv/client-go/v2/internal/logutil"
|
"github.com/tikv/client-go/v2/internal/logutil"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/kv"
|
"github.com/tikv/client-go/v2/kv"
|
||||||
"github.com/tikv/client-go/v2/metrics"
|
"github.com/tikv/client-go/v2/metrics"
|
||||||
"github.com/tikv/client-go/v2/tikvrpc"
|
"github.com/tikv/client-go/v2/tikvrpc"
|
||||||
|
|
|
||||||
|
|
@ -50,9 +50,9 @@ import (
|
||||||
"github.com/pingcap/kvproto/pkg/errorpb"
|
"github.com/pingcap/kvproto/pkg/errorpb"
|
||||||
"github.com/pingcap/kvproto/pkg/metapb"
|
"github.com/pingcap/kvproto/pkg/metapb"
|
||||||
"github.com/stretchr/testify/suite"
|
"github.com/stretchr/testify/suite"
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
"github.com/tikv/client-go/v2/internal/apicodec"
|
"github.com/tikv/client-go/v2/internal/apicodec"
|
||||||
"github.com/tikv/client-go/v2/internal/mockstore/mocktikv"
|
"github.com/tikv/client-go/v2/internal/mockstore/mocktikv"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/kv"
|
"github.com/tikv/client-go/v2/kv"
|
||||||
pd "github.com/tikv/pd/client"
|
pd "github.com/tikv/pd/client"
|
||||||
)
|
)
|
||||||
|
|
|
||||||
|
|
@ -56,10 +56,10 @@ import (
|
||||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||||
"github.com/pingcap/kvproto/pkg/metapb"
|
"github.com/pingcap/kvproto/pkg/metapb"
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
tikverr "github.com/tikv/client-go/v2/error"
|
tikverr "github.com/tikv/client-go/v2/error"
|
||||||
"github.com/tikv/client-go/v2/internal/client"
|
"github.com/tikv/client-go/v2/internal/client"
|
||||||
"github.com/tikv/client-go/v2/internal/logutil"
|
"github.com/tikv/client-go/v2/internal/logutil"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/kv"
|
"github.com/tikv/client-go/v2/kv"
|
||||||
"github.com/tikv/client-go/v2/metrics"
|
"github.com/tikv/client-go/v2/metrics"
|
||||||
"github.com/tikv/client-go/v2/tikvrpc"
|
"github.com/tikv/client-go/v2/tikvrpc"
|
||||||
|
|
|
||||||
|
|
@ -49,11 +49,11 @@ import (
|
||||||
"github.com/pingcap/log"
|
"github.com/pingcap/log"
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
"github.com/stretchr/testify/suite"
|
"github.com/stretchr/testify/suite"
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
tikverr "github.com/tikv/client-go/v2/error"
|
tikverr "github.com/tikv/client-go/v2/error"
|
||||||
"github.com/tikv/client-go/v2/internal/apicodec"
|
"github.com/tikv/client-go/v2/internal/apicodec"
|
||||||
"github.com/tikv/client-go/v2/internal/client"
|
"github.com/tikv/client-go/v2/internal/client"
|
||||||
"github.com/tikv/client-go/v2/internal/mockstore/mocktikv"
|
"github.com/tikv/client-go/v2/internal/mockstore/mocktikv"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/kv"
|
"github.com/tikv/client-go/v2/kv"
|
||||||
"github.com/tikv/client-go/v2/oracle"
|
"github.com/tikv/client-go/v2/oracle"
|
||||||
"github.com/tikv/client-go/v2/tikvrpc"
|
"github.com/tikv/client-go/v2/tikvrpc"
|
||||||
|
|
|
||||||
|
|
@ -28,10 +28,10 @@ import (
|
||||||
"github.com/pingcap/kvproto/pkg/metapb"
|
"github.com/pingcap/kvproto/pkg/metapb"
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
"github.com/stretchr/testify/require"
|
"github.com/stretchr/testify/require"
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
tikverr "github.com/tikv/client-go/v2/error"
|
tikverr "github.com/tikv/client-go/v2/error"
|
||||||
"github.com/tikv/client-go/v2/internal/apicodec"
|
"github.com/tikv/client-go/v2/internal/apicodec"
|
||||||
"github.com/tikv/client-go/v2/internal/mockstore/mocktikv"
|
"github.com/tikv/client-go/v2/internal/mockstore/mocktikv"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/kv"
|
"github.com/tikv/client-go/v2/kv"
|
||||||
"github.com/tikv/client-go/v2/metrics"
|
"github.com/tikv/client-go/v2/metrics"
|
||||||
"github.com/tikv/client-go/v2/tikvrpc"
|
"github.com/tikv/client-go/v2/tikvrpc"
|
||||||
|
|
|
||||||
|
|
@ -55,11 +55,11 @@ import (
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
"github.com/stretchr/testify/suite"
|
"github.com/stretchr/testify/suite"
|
||||||
"github.com/tikv/client-go/v2/config"
|
"github.com/tikv/client-go/v2/config"
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
"github.com/tikv/client-go/v2/internal/apicodec"
|
"github.com/tikv/client-go/v2/internal/apicodec"
|
||||||
"github.com/tikv/client-go/v2/internal/client"
|
"github.com/tikv/client-go/v2/internal/client"
|
||||||
"github.com/tikv/client-go/v2/internal/client/mock_server"
|
"github.com/tikv/client-go/v2/internal/client/mock_server"
|
||||||
"github.com/tikv/client-go/v2/internal/mockstore/mocktikv"
|
"github.com/tikv/client-go/v2/internal/mockstore/mocktikv"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/tikvrpc"
|
"github.com/tikv/client-go/v2/tikvrpc"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
)
|
)
|
||||||
|
|
|
||||||
|
|
@ -42,11 +42,11 @@ import (
|
||||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
"github.com/tikv/client-go/v2/config"
|
"github.com/tikv/client-go/v2/config"
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
tikverr "github.com/tikv/client-go/v2/error"
|
tikverr "github.com/tikv/client-go/v2/error"
|
||||||
"github.com/tikv/client-go/v2/internal/client"
|
"github.com/tikv/client-go/v2/internal/client"
|
||||||
"github.com/tikv/client-go/v2/internal/kvrpc"
|
"github.com/tikv/client-go/v2/internal/kvrpc"
|
||||||
"github.com/tikv/client-go/v2/internal/locate"
|
"github.com/tikv/client-go/v2/internal/locate"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/metrics"
|
"github.com/tikv/client-go/v2/metrics"
|
||||||
"github.com/tikv/client-go/v2/tikv"
|
"github.com/tikv/client-go/v2/tikv"
|
||||||
"github.com/tikv/client-go/v2/tikvrpc"
|
"github.com/tikv/client-go/v2/tikvrpc"
|
||||||
|
|
|
||||||
|
|
@ -42,9 +42,9 @@ import (
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
"github.com/stretchr/testify/suite"
|
"github.com/stretchr/testify/suite"
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
"github.com/tikv/client-go/v2/internal/locate"
|
"github.com/tikv/client-go/v2/internal/locate"
|
||||||
"github.com/tikv/client-go/v2/internal/mockstore/mocktikv"
|
"github.com/tikv/client-go/v2/internal/mockstore/mocktikv"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/kv"
|
"github.com/tikv/client-go/v2/kv"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -37,7 +37,7 @@ package tikv
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
"github.com/tikv/client-go/v2/kv"
|
"github.com/tikv/client-go/v2/kv"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -23,10 +23,10 @@ import (
|
||||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||||
"github.com/pingcap/kvproto/pkg/metapb"
|
"github.com/pingcap/kvproto/pkg/metapb"
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
tikverr "github.com/tikv/client-go/v2/error"
|
tikverr "github.com/tikv/client-go/v2/error"
|
||||||
"github.com/tikv/client-go/v2/internal/locate"
|
"github.com/tikv/client-go/v2/internal/locate"
|
||||||
"github.com/tikv/client-go/v2/internal/logutil"
|
"github.com/tikv/client-go/v2/internal/logutil"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/kv"
|
"github.com/tikv/client-go/v2/kv"
|
||||||
"github.com/tikv/client-go/v2/metrics"
|
"github.com/tikv/client-go/v2/metrics"
|
||||||
"github.com/tikv/client-go/v2/tikvrpc"
|
"github.com/tikv/client-go/v2/tikvrpc"
|
||||||
|
|
|
||||||
|
|
@ -49,12 +49,12 @@ import (
|
||||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
"github.com/tikv/client-go/v2/config"
|
"github.com/tikv/client-go/v2/config"
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
tikverr "github.com/tikv/client-go/v2/error"
|
tikverr "github.com/tikv/client-go/v2/error"
|
||||||
"github.com/tikv/client-go/v2/internal/client"
|
"github.com/tikv/client-go/v2/internal/client"
|
||||||
"github.com/tikv/client-go/v2/internal/latch"
|
"github.com/tikv/client-go/v2/internal/latch"
|
||||||
"github.com/tikv/client-go/v2/internal/locate"
|
"github.com/tikv/client-go/v2/internal/locate"
|
||||||
"github.com/tikv/client-go/v2/internal/logutil"
|
"github.com/tikv/client-go/v2/internal/logutil"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/kv"
|
"github.com/tikv/client-go/v2/kv"
|
||||||
"github.com/tikv/client-go/v2/metrics"
|
"github.com/tikv/client-go/v2/metrics"
|
||||||
"github.com/tikv/client-go/v2/oracle"
|
"github.com/tikv/client-go/v2/oracle"
|
||||||
|
|
|
||||||
|
|
@ -44,12 +44,12 @@ import (
|
||||||
"github.com/pingcap/kvproto/pkg/metapb"
|
"github.com/pingcap/kvproto/pkg/metapb"
|
||||||
"github.com/pingcap/kvproto/pkg/pdpb"
|
"github.com/pingcap/kvproto/pkg/pdpb"
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
tikverr "github.com/tikv/client-go/v2/error"
|
tikverr "github.com/tikv/client-go/v2/error"
|
||||||
"github.com/tikv/client-go/v2/internal/client"
|
"github.com/tikv/client-go/v2/internal/client"
|
||||||
"github.com/tikv/client-go/v2/internal/kvrpc"
|
"github.com/tikv/client-go/v2/internal/kvrpc"
|
||||||
"github.com/tikv/client-go/v2/internal/locate"
|
"github.com/tikv/client-go/v2/internal/locate"
|
||||||
"github.com/tikv/client-go/v2/internal/logutil"
|
"github.com/tikv/client-go/v2/internal/logutil"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/kv"
|
"github.com/tikv/client-go/v2/kv"
|
||||||
"github.com/tikv/client-go/v2/tikvrpc"
|
"github.com/tikv/client-go/v2/tikvrpc"
|
||||||
"github.com/tikv/client-go/v2/txnkv/rangetask"
|
"github.com/tikv/client-go/v2/txnkv/rangetask"
|
||||||
|
|
|
||||||
|
|
@ -39,7 +39,7 @@ import (
|
||||||
"context"
|
"context"
|
||||||
|
|
||||||
"github.com/pingcap/kvproto/pkg/metapb"
|
"github.com/pingcap/kvproto/pkg/metapb"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
"github.com/tikv/client-go/v2/tikvrpc"
|
"github.com/tikv/client-go/v2/tikvrpc"
|
||||||
"github.com/tikv/client-go/v2/txnkv/transaction"
|
"github.com/tikv/client-go/v2/txnkv/transaction"
|
||||||
"github.com/tikv/client-go/v2/txnkv/txnlock"
|
"github.com/tikv/client-go/v2/txnkv/txnlock"
|
||||||
|
|
|
||||||
|
|
@ -21,7 +21,7 @@ import (
|
||||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
"github.com/tikv/client-go/v2/config"
|
"github.com/tikv/client-go/v2/config"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
"github.com/tikv/client-go/v2/oracle"
|
"github.com/tikv/client-go/v2/oracle"
|
||||||
"github.com/tikv/client-go/v2/tikv"
|
"github.com/tikv/client-go/v2/tikv"
|
||||||
"github.com/tikv/client-go/v2/txnkv/transaction"
|
"github.com/tikv/client-go/v2/txnkv/transaction"
|
||||||
|
|
|
||||||
|
|
@ -41,10 +41,10 @@ import (
|
||||||
|
|
||||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
tikverr "github.com/tikv/client-go/v2/error"
|
tikverr "github.com/tikv/client-go/v2/error"
|
||||||
"github.com/tikv/client-go/v2/internal/client"
|
"github.com/tikv/client-go/v2/internal/client"
|
||||||
"github.com/tikv/client-go/v2/internal/locate"
|
"github.com/tikv/client-go/v2/internal/locate"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/kv"
|
"github.com/tikv/client-go/v2/kv"
|
||||||
"github.com/tikv/client-go/v2/tikvrpc"
|
"github.com/tikv/client-go/v2/tikvrpc"
|
||||||
)
|
)
|
||||||
|
|
|
||||||
|
|
@ -42,8 +42,8 @@ import (
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
"github.com/tikv/client-go/v2/internal/logutil"
|
"github.com/tikv/client-go/v2/internal/logutil"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/kv"
|
"github.com/tikv/client-go/v2/kv"
|
||||||
"github.com/tikv/client-go/v2/metrics"
|
"github.com/tikv/client-go/v2/metrics"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
|
|
|
||||||
|
|
@ -52,12 +52,12 @@ import (
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
"github.com/prometheus/client_golang/prometheus"
|
"github.com/prometheus/client_golang/prometheus"
|
||||||
"github.com/tikv/client-go/v2/config"
|
"github.com/tikv/client-go/v2/config"
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
tikverr "github.com/tikv/client-go/v2/error"
|
tikverr "github.com/tikv/client-go/v2/error"
|
||||||
"github.com/tikv/client-go/v2/internal/client"
|
"github.com/tikv/client-go/v2/internal/client"
|
||||||
"github.com/tikv/client-go/v2/internal/latch"
|
"github.com/tikv/client-go/v2/internal/latch"
|
||||||
"github.com/tikv/client-go/v2/internal/locate"
|
"github.com/tikv/client-go/v2/internal/locate"
|
||||||
"github.com/tikv/client-go/v2/internal/logutil"
|
"github.com/tikv/client-go/v2/internal/logutil"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/internal/unionstore"
|
"github.com/tikv/client-go/v2/internal/unionstore"
|
||||||
"github.com/tikv/client-go/v2/kv"
|
"github.com/tikv/client-go/v2/kv"
|
||||||
"github.com/tikv/client-go/v2/metrics"
|
"github.com/tikv/client-go/v2/metrics"
|
||||||
|
|
|
||||||
|
|
@ -38,9 +38,9 @@ import (
|
||||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
"github.com/prometheus/client_golang/prometheus"
|
"github.com/prometheus/client_golang/prometheus"
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
"github.com/tikv/client-go/v2/internal/client"
|
"github.com/tikv/client-go/v2/internal/client"
|
||||||
"github.com/tikv/client-go/v2/internal/logutil"
|
"github.com/tikv/client-go/v2/internal/logutil"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/metrics"
|
"github.com/tikv/client-go/v2/metrics"
|
||||||
"github.com/tikv/client-go/v2/tikvrpc"
|
"github.com/tikv/client-go/v2/tikvrpc"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
|
|
|
||||||
|
|
@ -42,11 +42,11 @@ import (
|
||||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
"github.com/prometheus/client_golang/prometheus"
|
"github.com/prometheus/client_golang/prometheus"
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
tikverr "github.com/tikv/client-go/v2/error"
|
tikverr "github.com/tikv/client-go/v2/error"
|
||||||
"github.com/tikv/client-go/v2/internal/client"
|
"github.com/tikv/client-go/v2/internal/client"
|
||||||
"github.com/tikv/client-go/v2/internal/locate"
|
"github.com/tikv/client-go/v2/internal/locate"
|
||||||
"github.com/tikv/client-go/v2/internal/logutil"
|
"github.com/tikv/client-go/v2/internal/logutil"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/metrics"
|
"github.com/tikv/client-go/v2/metrics"
|
||||||
"github.com/tikv/client-go/v2/tikvrpc"
|
"github.com/tikv/client-go/v2/tikvrpc"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
|
|
|
||||||
|
|
@ -45,11 +45,11 @@ import (
|
||||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
"github.com/prometheus/client_golang/prometheus"
|
"github.com/prometheus/client_golang/prometheus"
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
tikverr "github.com/tikv/client-go/v2/error"
|
tikverr "github.com/tikv/client-go/v2/error"
|
||||||
"github.com/tikv/client-go/v2/internal/client"
|
"github.com/tikv/client-go/v2/internal/client"
|
||||||
"github.com/tikv/client-go/v2/internal/locate"
|
"github.com/tikv/client-go/v2/internal/locate"
|
||||||
"github.com/tikv/client-go/v2/internal/logutil"
|
"github.com/tikv/client-go/v2/internal/logutil"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/kv"
|
"github.com/tikv/client-go/v2/kv"
|
||||||
"github.com/tikv/client-go/v2/metrics"
|
"github.com/tikv/client-go/v2/metrics"
|
||||||
"github.com/tikv/client-go/v2/tikvrpc"
|
"github.com/tikv/client-go/v2/tikvrpc"
|
||||||
|
|
|
||||||
|
|
@ -46,11 +46,11 @@ import (
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
"github.com/prometheus/client_golang/prometheus"
|
"github.com/prometheus/client_golang/prometheus"
|
||||||
"github.com/tikv/client-go/v2/config"
|
"github.com/tikv/client-go/v2/config"
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
tikverr "github.com/tikv/client-go/v2/error"
|
tikverr "github.com/tikv/client-go/v2/error"
|
||||||
"github.com/tikv/client-go/v2/internal/client"
|
"github.com/tikv/client-go/v2/internal/client"
|
||||||
"github.com/tikv/client-go/v2/internal/locate"
|
"github.com/tikv/client-go/v2/internal/locate"
|
||||||
"github.com/tikv/client-go/v2/internal/logutil"
|
"github.com/tikv/client-go/v2/internal/logutil"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/metrics"
|
"github.com/tikv/client-go/v2/metrics"
|
||||||
"github.com/tikv/client-go/v2/oracle"
|
"github.com/tikv/client-go/v2/oracle"
|
||||||
"github.com/tikv/client-go/v2/tikvrpc"
|
"github.com/tikv/client-go/v2/tikvrpc"
|
||||||
|
|
|
||||||
|
|
@ -20,8 +20,8 @@ import (
|
||||||
"sync/atomic"
|
"sync/atomic"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
"github.com/tikv/client-go/v2/internal/locate"
|
"github.com/tikv/client-go/v2/internal/locate"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/internal/unionstore"
|
"github.com/tikv/client-go/v2/internal/unionstore"
|
||||||
"github.com/tikv/client-go/v2/kv"
|
"github.com/tikv/client-go/v2/kv"
|
||||||
"github.com/tikv/client-go/v2/tikvrpc"
|
"github.com/tikv/client-go/v2/tikvrpc"
|
||||||
|
|
|
||||||
|
|
@ -54,9 +54,9 @@ import (
|
||||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
"github.com/tikv/client-go/v2/config"
|
"github.com/tikv/client-go/v2/config"
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
tikverr "github.com/tikv/client-go/v2/error"
|
tikverr "github.com/tikv/client-go/v2/error"
|
||||||
"github.com/tikv/client-go/v2/internal/logutil"
|
"github.com/tikv/client-go/v2/internal/logutil"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/internal/unionstore"
|
"github.com/tikv/client-go/v2/internal/unionstore"
|
||||||
tikv "github.com/tikv/client-go/v2/kv"
|
tikv "github.com/tikv/client-go/v2/kv"
|
||||||
"github.com/tikv/client-go/v2/metrics"
|
"github.com/tikv/client-go/v2/metrics"
|
||||||
|
|
|
||||||
|
|
@ -28,11 +28,11 @@ import (
|
||||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
"github.com/tikv/client-go/v2/config"
|
"github.com/tikv/client-go/v2/config"
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
tikverr "github.com/tikv/client-go/v2/error"
|
tikverr "github.com/tikv/client-go/v2/error"
|
||||||
"github.com/tikv/client-go/v2/internal/client"
|
"github.com/tikv/client-go/v2/internal/client"
|
||||||
"github.com/tikv/client-go/v2/internal/locate"
|
"github.com/tikv/client-go/v2/internal/locate"
|
||||||
"github.com/tikv/client-go/v2/internal/logutil"
|
"github.com/tikv/client-go/v2/internal/logutil"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/metrics"
|
"github.com/tikv/client-go/v2/metrics"
|
||||||
"github.com/tikv/client-go/v2/oracle"
|
"github.com/tikv/client-go/v2/oracle"
|
||||||
"github.com/tikv/client-go/v2/tikvrpc"
|
"github.com/tikv/client-go/v2/tikvrpc"
|
||||||
|
|
|
||||||
|
|
@ -17,8 +17,8 @@ package txnlock
|
||||||
import (
|
import (
|
||||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
"github.com/tikv/client-go/v2/internal/locate"
|
"github.com/tikv/client-go/v2/internal/locate"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
// LockProbe exposes some lock utilities for testing purpose.
|
// LockProbe exposes some lock utilities for testing purpose.
|
||||||
|
|
|
||||||
|
|
@ -37,9 +37,9 @@ package txnsnapshot
|
||||||
import (
|
import (
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
"github.com/tikv/client-go/v2/internal/client"
|
"github.com/tikv/client-go/v2/internal/client"
|
||||||
"github.com/tikv/client-go/v2/internal/locate"
|
"github.com/tikv/client-go/v2/internal/locate"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/tikvrpc"
|
"github.com/tikv/client-go/v2/tikvrpc"
|
||||||
"github.com/tikv/client-go/v2/txnkv/txnlock"
|
"github.com/tikv/client-go/v2/txnkv/txnlock"
|
||||||
"github.com/tikv/client-go/v2/util"
|
"github.com/tikv/client-go/v2/util"
|
||||||
|
|
|
||||||
|
|
@ -40,11 +40,11 @@ import (
|
||||||
|
|
||||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
tikverr "github.com/tikv/client-go/v2/error"
|
tikverr "github.com/tikv/client-go/v2/error"
|
||||||
"github.com/tikv/client-go/v2/internal/client"
|
"github.com/tikv/client-go/v2/internal/client"
|
||||||
"github.com/tikv/client-go/v2/internal/locate"
|
"github.com/tikv/client-go/v2/internal/locate"
|
||||||
"github.com/tikv/client-go/v2/internal/logutil"
|
"github.com/tikv/client-go/v2/internal/logutil"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/kv"
|
"github.com/tikv/client-go/v2/kv"
|
||||||
"github.com/tikv/client-go/v2/tikvrpc"
|
"github.com/tikv/client-go/v2/tikvrpc"
|
||||||
"github.com/tikv/client-go/v2/tikvrpc/interceptor"
|
"github.com/tikv/client-go/v2/tikvrpc/interceptor"
|
||||||
|
|
|
||||||
|
|
@ -47,11 +47,11 @@ import (
|
||||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||||
"github.com/pingcap/kvproto/pkg/metapb"
|
"github.com/pingcap/kvproto/pkg/metapb"
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
tikverr "github.com/tikv/client-go/v2/error"
|
tikverr "github.com/tikv/client-go/v2/error"
|
||||||
"github.com/tikv/client-go/v2/internal/client"
|
"github.com/tikv/client-go/v2/internal/client"
|
||||||
"github.com/tikv/client-go/v2/internal/locate"
|
"github.com/tikv/client-go/v2/internal/locate"
|
||||||
"github.com/tikv/client-go/v2/internal/logutil"
|
"github.com/tikv/client-go/v2/internal/logutil"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/internal/unionstore"
|
"github.com/tikv/client-go/v2/internal/unionstore"
|
||||||
"github.com/tikv/client-go/v2/kv"
|
"github.com/tikv/client-go/v2/kv"
|
||||||
"github.com/tikv/client-go/v2/metrics"
|
"github.com/tikv/client-go/v2/metrics"
|
||||||
|
|
|
||||||
|
|
@ -16,8 +16,8 @@ package txnsnapshot
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||||
|
"github.com/tikv/client-go/v2/config/retry"
|
||||||
"github.com/tikv/client-go/v2/internal/locate"
|
"github.com/tikv/client-go/v2/internal/locate"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
|
||||||
"github.com/tikv/client-go/v2/tikvrpc"
|
"github.com/tikv/client-go/v2/tikvrpc"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
|
|
||||||
Loading…
Reference in New Issue