From 9f94221c2dbe510d7b2e7b3636eb1caf51dee605 Mon Sep 17 00:00:00 2001 From: crazycs Date: Fri, 28 Jul 2023 20:35:05 +0800 Subject: [PATCH 01/49] add more log for diagnose (#915) * add more log for diagnose Signed-off-by: crazycs520 * fix Signed-off-by: crazycs520 * add more log for diagnose Signed-off-by: crazycs520 * add more log Signed-off-by: crazycs520 * address comment Signed-off-by: crazycs520 --------- Signed-off-by: crazycs520 --- internal/locate/region_cache.go | 38 ++++++++++++ internal/locate/region_request.go | 96 +++++++++++++++++++++++++++---- internal/retry/backoff.go | 14 +++++ kv/store_vars.go | 20 +++++++ 4 files changed, 158 insertions(+), 10 deletions(-) diff --git a/internal/locate/region_cache.go b/internal/locate/region_cache.go index a5fb471afc..c62434f128 100644 --- a/internal/locate/region_cache.go +++ b/internal/locate/region_cache.go @@ -2473,6 +2473,24 @@ const ( tombstone ) +// String implements fmt.Stringer interface. +func (s resolveState) String() string { + switch s { + case unresolved: + return "unresolved" + case resolved: + return "resolved" + case needCheck: + return "needCheck" + case deleted: + return "deleted" + case tombstone: + return "tombstone" + default: + return fmt.Sprintf("unknown-%v", uint64(s)) + } +} + // IsTiFlash returns true if the storeType is TiFlash func (s *Store) IsTiFlash() bool { return s.storeType == tikvrpc.TiFlash @@ -2612,6 +2630,12 @@ func (s *Store) changeResolveStateTo(from, to resolveState) bool { return false } if atomic.CompareAndSwapUint64(&s.state, uint64(from), uint64(to)) { + logutil.BgLogger().Info("change store resolve state", + zap.Uint64("store", s.storeID), + zap.String("addr", s.addr), + zap.String("from", from.String()), + zap.String("to", to.String()), + zap.String("liveness-state", s.getLivenessState().String())) return true } } @@ -2712,6 +2736,20 @@ const ( unknown ) +// String implements fmt.Stringer interface. +func (s livenessState) String() string { + switch s { + case unreachable: + return "unreachable" + case reachable: + return "reachable" + case unknown: + return "unknown" + default: + return fmt.Sprintf("unknown-%v", uint32(s)) + } +} + func (s *Store) startHealthCheckLoopIfNeeded(c *RegionCache, liveness livenessState) { // This mechanism doesn't support non-TiKV stores currently. if s.storeType != tikvrpc.TiKV { diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index 4308dbb2d2..0be0ab900e 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -35,6 +35,7 @@ package locate import ( + "bytes" "context" "fmt" "math" @@ -608,14 +609,15 @@ func (state *accessFollower) next(bo *retry.Backoffer, selector *replicaSelector } // If there is no candidate, fallback to the leader. if selector.targetIdx < 0 { + leader := selector.replicas[state.leaderIdx] + leaderInvalid := leader.isEpochStale() || (!state.option.leaderOnly && leader.isExhausted(1)) if len(state.option.labels) > 0 { - logutil.BgLogger().Warn( - "unable to find stores with given labels", - zap.Any("labels", state.option.labels), - ) + logutil.Logger(bo.GetCtx()).Warn("unable to find stores with given labels", + zap.Uint64("region", selector.region.GetID()), + zap.Bool("leader-invalid", leaderInvalid), + zap.Any("labels", state.option.labels)) } - leader := selector.replicas[state.leaderIdx] - if leader.isEpochStale() || (!state.option.leaderOnly && leader.isExhausted(1)) { + if leaderInvalid { metrics.TiKVReplicaSelectorFailureCounter.WithLabelValues("exhausted").Inc() selector.invalidateRegion() return nil, nil @@ -1189,6 +1191,7 @@ func (s *RegionRequestSender) SendReqCtx( }() } + totalErrors := make(map[string]int) for { if retryTimes > 0 { req.IsRetryRequest = true @@ -1221,10 +1224,7 @@ func (s *RegionRequestSender) SendReqCtx( // TODO: Change the returned error to something like "region missing in cache", // and handle this error like EpochNotMatch, which means to re-split the request and retry. - logutil.Logger(bo.GetCtx()).Debug( - "throwing pseudo region error due to region not found in cache", - zap.Stringer("region", ®ionID), - ) + s.logSendReqError(bo, "throwing pseudo region error due to no replica available", regionID, retryTimes, req, totalErrors) resp, err = tikvrpc.GenRegionErrorResp(req, &errorpb.Error{EpochNotMatch: &errorpb.EpochNotMatch{}}) return resp, nil, retryTimes, err } @@ -1250,6 +1250,8 @@ func (s *RegionRequestSender) SendReqCtx( var retry bool resp, retry, err = s.sendReqToRegion(bo, rpcCtx, req, timeout) if err != nil { + msg := fmt.Sprintf("send request failed, err: %v", err.Error()) + s.logSendReqError(bo, msg, regionID, retryTimes, req, totalErrors) return nil, nil, retryTimes, err } @@ -1281,14 +1283,19 @@ func (s *RegionRequestSender) SendReqCtx( return nil, nil, retryTimes, err } if regionErr != nil { + regionErrLabel := regionErrorToLabel(regionErr) + totalErrors[regionErrLabel]++ retry, err = s.onRegionError(bo, rpcCtx, req, regionErr) if err != nil { + msg := fmt.Sprintf("send request on region error failed, err: %v", err.Error()) + s.logSendReqError(bo, msg, regionID, retryTimes, req, totalErrors) return nil, nil, retryTimes, err } if retry { retryTimes++ continue } + s.logSendReqError(bo, "send request meet region error without retry", regionID, retryTimes, req, totalErrors) } else { if s.replicaSelector != nil { s.replicaSelector.onSendSuccess() @@ -1301,6 +1308,75 @@ func (s *RegionRequestSender) SendReqCtx( } } +func (s *RegionRequestSender) logSendReqError(bo *retry.Backoffer, msg string, regionID RegionVerID, retryTimes int, req *tikvrpc.Request, totalErrors map[string]int) { + var replicaStatus []string + replicaSelectorState := "nil" + cacheRegionIsValid := "unknown" + if s.replicaSelector != nil { + switch s.replicaSelector.state.(type) { + case *accessKnownLeader: + replicaSelectorState = "accessKnownLeader" + case *accessFollower: + replicaSelectorState = "accessFollower" + case *accessByKnownProxy: + replicaSelectorState = "accessByKnownProxy" + case *tryFollower: + replicaSelectorState = "tryFollower" + case *tryNewProxy: + replicaSelectorState = "tryNewProxy" + case *invalidLeader: + replicaSelectorState = "invalidLeader" + case *invalidStore: + replicaSelectorState = "invalidStore" + case *stateBase: + replicaSelectorState = "stateBase" + case nil: + replicaSelectorState = "nil" + } + if s.replicaSelector.region != nil { + if s.replicaSelector.region.isValid() { + cacheRegionIsValid = "true" + } else { + cacheRegionIsValid = "false" + } + } + for _, replica := range s.replicaSelector.replicas { + replicaStatus = append(replicaStatus, fmt.Sprintf("peer: %v, store: %v, isEpochStale: %v, attempts: %v, replica-epoch: %v, store-epoch: %v, store-state: %v, store-liveness-state: %v", + replica.peer.GetId(), + replica.store.storeID, + replica.isEpochStale(), + replica.attempts, + replica.epoch, + atomic.LoadUint32(&replica.store.epoch), + replica.store.getResolveState(), + replica.store.getLivenessState(), + )) + } + } + var totalErrorStr bytes.Buffer + for err, cnt := range totalErrors { + if totalErrorStr.Len() > 0 { + totalErrorStr.WriteString(", ") + } + totalErrorStr.WriteString(err) + totalErrorStr.WriteString(":") + totalErrorStr.WriteString(strconv.Itoa(cnt)) + } + logutil.Logger(bo.GetCtx()).Info(msg, + zap.Uint64("req-ts", req.GetStartTS()), + zap.String("req-type", req.Type.String()), + zap.String("region", regionID.String()), + zap.String("region-is-valid", cacheRegionIsValid), + zap.Int("retry-times", retryTimes), + zap.String("replica-read-type", req.ReplicaReadType.String()), + zap.String("replica-selector-state", replicaSelectorState), + zap.Bool("stale-read", req.StaleRead), + zap.String("replica-status", strings.Join(replicaStatus, "; ")), + zap.Int("total-backoff-ms", bo.GetTotalSleep()), + zap.Int("total-backoff-times", bo.GetTotalBackoffTimes()), + zap.String("total-region-errors", totalErrorStr.String())) +} + // RPCCancellerCtxKey is context key attach rpc send cancelFunc collector to ctx. type RPCCancellerCtxKey struct{} diff --git a/internal/retry/backoff.go b/internal/retry/backoff.go index 6a27d05938..bdefc79930 100644 --- a/internal/retry/backoff.go +++ b/internal/retry/backoff.go @@ -35,9 +35,11 @@ package retry import ( + "bytes" "context" "fmt" "math" + "strconv" "strings" "sync/atomic" "time" @@ -150,6 +152,18 @@ func (b *Backoffer) BackoffWithCfgAndMaxSleep(cfg *Config, maxSleepMs int, err e 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", totalTimes, backoffDetail.String()) returnedErr := err if longestSleepCfg != nil { errMsg += fmt.Sprintf("\nlongest sleep type: %s, time: %dms", longestSleepCfg.String(), longestSleepTime) diff --git a/kv/store_vars.go b/kv/store_vars.go index cce3e146b1..184975ca21 100644 --- a/kv/store_vars.go +++ b/kv/store_vars.go @@ -35,6 +35,8 @@ package kv import ( + "fmt" + "go.uber.org/atomic" ) @@ -72,3 +74,21 @@ const ( func (r ReplicaReadType) IsFollowerRead() bool { return r != ReplicaReadLeader } + +// String implements fmt.Stringer interface. +func (r ReplicaReadType) String() string { + switch r { + case ReplicaReadLeader: + return "leader" + case ReplicaReadFollower: + return "follower" + case ReplicaReadMixed: + return "mixed" + case ReplicaReadLearner: + return "learner" + case ReplicaReadPreferLeader: + return "prefer-leader" + default: + return fmt.Sprintf("unknown-%v", byte(r)) + } +} From 719e6456f7d5d59d45b70b77a36b450b6ce41c86 Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 31 Jul 2023 11:23:49 +0800 Subject: [PATCH 02/49] use context logger as much as possible (#908) * use context logger as much as possible Signed-off-by: crazycs520 * refine Signed-off-by: crazycs520 --------- Signed-off-by: crazycs520 --- internal/client/client_batch.go | 4 ++-- internal/locate/region_cache.go | 6 +++--- internal/locate/region_request.go | 36 +++++++++++++++---------------- internal/retry/backoff.go | 4 ++-- 4 files changed, 25 insertions(+), 25 deletions(-) diff --git a/internal/client/client_batch.go b/internal/client/client_batch.go index 6952bfc0fd..c8842e9c26 100644 --- a/internal/client/client_batch.go +++ b/internal/client/client_batch.go @@ -781,7 +781,7 @@ func sendBatchRequest( select { case batchConn.batchCommandsCh <- entry: case <-ctx.Done(): - logutil.BgLogger().Debug("send request is cancelled", + logutil.Logger(ctx).Debug("send request is cancelled", zap.String("to", addr), zap.String("cause", ctx.Err().Error())) return nil, errors.WithStack(ctx.Err()) case <-timer.C: @@ -797,7 +797,7 @@ func sendBatchRequest( return tikvrpc.FromBatchCommandsResponse(res) case <-ctx.Done(): atomic.StoreInt32(&entry.canceled, 1) - logutil.BgLogger().Debug("wait response is cancelled", + logutil.Logger(ctx).Debug("wait response is cancelled", zap.String("to", addr), zap.String("cause", ctx.Err().Error())) return nil, errors.WithStack(ctx.Err()) case <-timer.C: diff --git a/internal/locate/region_cache.go b/internal/locate/region_cache.go index c62434f128..87cd067044 100644 --- a/internal/locate/region_cache.go +++ b/internal/locate/region_cache.go @@ -776,7 +776,7 @@ func (c *RegionCache) GetTiKVRPCContext(bo *retry.Backoffer, id RegionVerID, rep storeFailEpoch := atomic.LoadUint32(&store.epoch) if storeFailEpoch != regionStore.storeEpochs[storeIdx] { cachedRegion.invalidate(Other) - logutil.BgLogger().Info("invalidate current region, because others failed on same store", + logutil.Logger(bo.GetCtx()).Info("invalidate current region, because others failed on same store", zap.Uint64("region", id.GetID()), zap.String("store", store.addr)) return nil, nil @@ -905,7 +905,7 @@ func (c *RegionCache) GetTiFlashRPCContext(bo *retry.Backoffer, id RegionVerID, storeFailEpoch := atomic.LoadUint32(&store.epoch) if storeFailEpoch != regionStore.storeEpochs[storeIdx] { cachedRegion.invalidate(Other) - logutil.BgLogger().Info("invalidate current region, because others failed on same store", + logutil.Logger(bo.GetCtx()).Info("invalidate current region, because others failed on same store", zap.Uint64("region", id.GetID()), zap.String("store", store.addr)) // TiFlash will always try to find out a valid peer, avoiding to retry too many times. @@ -1959,7 +1959,7 @@ func (c *RegionCache) OnRegionEpochNotMatch(bo *retry.Backoffer, ctx *RPCContext (meta.GetRegionEpoch().GetConfVer() < ctx.Region.confVer || meta.GetRegionEpoch().GetVersion() < ctx.Region.ver) { err := errors.Errorf("region epoch is ahead of tikv. rpc ctx: %+v, currentRegions: %+v", ctx, currentRegions) - logutil.BgLogger().Info("region epoch is ahead of tikv", zap.Error(err)) + logutil.Logger(bo.GetCtx()).Info("region epoch is ahead of tikv", zap.Error(err)) return true, bo.Backoff(retry.BoRegionMiss, err) } } diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index 0be0ab900e..3f9d9459e3 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -1645,7 +1645,7 @@ func (s *RegionRequestSender) onSendFail(bo *retry.Backoffer, ctx *RPCContext, e // If we don't cancel, but the error code is Canceled, it must be from grpc remote. // This may happen when tikv is killed and exiting. // Backoff and retry in this case. - logutil.BgLogger().Warn("receive a grpc cancel signal from remote", zap.Error(err)) + logutil.Logger(bo.GetCtx()).Warn("receive a grpc cancel signal from remote", zap.Error(err)) } } @@ -1776,7 +1776,7 @@ func (s *RegionRequestSender) onRegionError( if notLeader := regionErr.GetNotLeader(); notLeader != nil { // Retry if error is `NotLeader`. - logutil.BgLogger().Debug( + logutil.Logger(bo.GetCtx()).Debug( "tikv reports `NotLeader` retry later", zap.String("notLeader", notLeader.String()), zap.String("ctx", ctx.String()), @@ -1817,7 +1817,7 @@ func (s *RegionRequestSender) onRegionError( if regionErr.GetRecoveryInProgress() != nil { s.regionCache.InvalidateCachedRegion(ctx.Region) - logutil.BgLogger().Debug("tikv reports `RecoveryInProgress`", zap.Stringer("ctx", ctx)) + logutil.Logger(bo.GetCtx()).Debug("tikv reports `RecoveryInProgress`", zap.Stringer("ctx", ctx)) err = bo.Backoff(retry.BoRegionRecoveryInProgress, errors.Errorf("region recovery in progress, ctx: %v", ctx)) if err != nil { return false, err @@ -1827,7 +1827,7 @@ func (s *RegionRequestSender) onRegionError( if regionErr.GetIsWitness() != nil { s.regionCache.InvalidateCachedRegion(ctx.Region) - logutil.BgLogger().Debug("tikv reports `IsWitness`", zap.Stringer("ctx", ctx)) + logutil.Logger(bo.GetCtx()).Debug("tikv reports `IsWitness`", zap.Stringer("ctx", ctx)) err = bo.Backoff(retry.BoIsWitness, errors.Errorf("is witness, ctx: %v", ctx)) if err != nil { return false, err @@ -1839,7 +1839,7 @@ func (s *RegionRequestSender) onRegionError( // if a request meets the FlashbackInProgress error, it should stop retrying immediately // to avoid unnecessary backoff and potential unexpected data status to the user. if flashbackInProgress := regionErr.GetFlashbackInProgress(); flashbackInProgress != nil { - logutil.BgLogger().Debug( + logutil.Logger(bo.GetCtx()).Debug( "tikv reports `FlashbackInProgress`", zap.Stringer("req", req), zap.Stringer("ctx", ctx), @@ -1868,7 +1868,7 @@ func (s *RegionRequestSender) onRegionError( // prepared for the flashback before, it should stop retrying immediately to avoid // unnecessary backoff. if regionErr.GetFlashbackNotPrepared() != nil { - logutil.BgLogger().Debug( + logutil.Logger(bo.GetCtx()).Debug( "tikv reports `FlashbackNotPrepared`", zap.Stringer("req", req), zap.Stringer("ctx", ctx), @@ -1886,13 +1886,13 @@ func (s *RegionRequestSender) onRegionError( } if regionErr.GetKeyNotInRegion() != nil { - logutil.BgLogger().Error("tikv reports `KeyNotInRegion`", zap.Stringer("req", req), zap.Stringer("ctx", ctx)) + logutil.Logger(bo.GetCtx()).Error("tikv reports `KeyNotInRegion`", zap.Stringer("req", req), zap.Stringer("ctx", ctx)) s.regionCache.InvalidateCachedRegion(ctx.Region) return false, nil } if epochNotMatch := regionErr.GetEpochNotMatch(); epochNotMatch != nil { - logutil.BgLogger().Debug( + logutil.Logger(bo.GetCtx()).Debug( "tikv reports `EpochNotMatch` retry later", zap.Stringer("EpochNotMatch", epochNotMatch), zap.Stringer("ctx", ctx), @@ -1908,7 +1908,7 @@ func (s *RegionRequestSender) onRegionError( if s.replicaSelector != nil { return s.replicaSelector.onServerIsBusy(bo, ctx, req, serverIsBusy) } - logutil.BgLogger().Warn( + logutil.Logger(bo.GetCtx()).Warn( "tikv reports `ServerIsBusy` retry later", zap.String("reason", regionErr.GetServerIsBusy().GetReason()), zap.Stringer("ctx", ctx), @@ -1928,7 +1928,7 @@ func (s *RegionRequestSender) onRegionError( // We can't know whether the request is committed or not, so it's an undetermined error too, // but we don't handle it now. if regionErr.GetStaleCommand() != nil { - logutil.BgLogger().Debug("tikv reports `StaleCommand`", zap.Stringer("ctx", ctx)) + logutil.Logger(bo.GetCtx()).Debug("tikv reports `StaleCommand`", zap.Stringer("ctx", ctx)) if s.replicaSelector != nil { // Needn't backoff because the new leader should be elected soon // and the replicaSelector will try the next peer. @@ -1943,7 +1943,7 @@ func (s *RegionRequestSender) onRegionError( if storeNotMatch := regionErr.GetStoreNotMatch(); storeNotMatch != nil { // store not match - logutil.BgLogger().Debug( + logutil.Logger(bo.GetCtx()).Debug( "tikv reports `StoreNotMatch` retry later", zap.Stringer("storeNotMatch", storeNotMatch), zap.Stringer("ctx", ctx), @@ -1957,12 +1957,12 @@ func (s *RegionRequestSender) onRegionError( } if regionErr.GetRaftEntryTooLarge() != nil { - logutil.BgLogger().Warn("tikv reports `RaftEntryTooLarge`", zap.Stringer("ctx", ctx)) + logutil.Logger(bo.GetCtx()).Warn("tikv reports `RaftEntryTooLarge`", zap.Stringer("ctx", ctx)) return false, errors.New(regionErr.String()) } if regionErr.GetMaxTimestampNotSynced() != nil { - logutil.BgLogger().Debug("tikv reports `MaxTimestampNotSynced`", zap.Stringer("ctx", ctx)) + logutil.Logger(bo.GetCtx()).Debug("tikv reports `MaxTimestampNotSynced`", zap.Stringer("ctx", ctx)) err = bo.Backoff(retry.BoMaxTsNotSynced, errors.Errorf("max timestamp not synced, ctx: %v", ctx)) if err != nil { return false, err @@ -1972,7 +1972,7 @@ func (s *RegionRequestSender) onRegionError( // A read request may be sent to a peer which has not been initialized yet, we should retry in this case. if regionErr.GetRegionNotInitialized() != nil { - logutil.BgLogger().Debug( + logutil.Logger(bo.GetCtx()).Debug( "tikv reports `RegionNotInitialized` retry later", zap.Uint64("store-id", ctx.Store.storeID), zap.Uint64("region-id", regionErr.GetRegionNotInitialized().GetRegionId()), @@ -1987,7 +1987,7 @@ func (s *RegionRequestSender) onRegionError( // The read-index can't be handled timely because the region is splitting or merging. if regionErr.GetReadIndexNotReady() != nil { - logutil.BgLogger().Debug( + logutil.Logger(bo.GetCtx()).Debug( "tikv reports `ReadIndexNotReady` retry later", zap.Uint64("store-id", ctx.Store.storeID), zap.Uint64("region-id", regionErr.GetRegionNotInitialized().GetRegionId()), @@ -2002,7 +2002,7 @@ func (s *RegionRequestSender) onRegionError( } if regionErr.GetProposalInMergingMode() != nil { - logutil.BgLogger().Debug("tikv reports `ProposalInMergingMode`", zap.Stringer("ctx", ctx)) + logutil.Logger(bo.GetCtx()).Debug("tikv reports `ProposalInMergingMode`", zap.Stringer("ctx", ctx)) // The region is merging and it can't provide service until merge finished, so backoff. err = bo.Backoff(retry.BoRegionScheduling, errors.Errorf("region is merging, ctx: %v", ctx)) if err != nil { @@ -2015,7 +2015,7 @@ func (s *RegionRequestSender) onRegionError( // This error is specific to stale read and the target replica is randomly selected. If the request is sent // to the leader, the data must be ready, so we don't backoff here. if regionErr.GetDataIsNotReady() != nil { - logutil.BgLogger().Warn( + logutil.Logger(bo.GetCtx()).Warn( "tikv reports `DataIsNotReady` retry later", zap.Uint64("store-id", ctx.Store.storeID), zap.Uint64("peer-id", regionErr.GetDataIsNotReady().GetPeerId()), @@ -2033,7 +2033,7 @@ func (s *RegionRequestSender) onRegionError( return true, nil } - logutil.BgLogger().Debug( + logutil.Logger(bo.GetCtx()).Debug( "tikv reports region failed", zap.Stringer("regionErr", regionErr), zap.Stringer("ctx", ctx), diff --git a/internal/retry/backoff.go b/internal/retry/backoff.go index bdefc79930..19f6fca691 100644 --- a/internal/retry/backoff.go +++ b/internal/retry/backoff.go @@ -133,7 +133,7 @@ func (b *Backoffer) BackoffWithMaxSleepTxnLockFast(maxSleepMs int, err error) er // 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.BgLogger().Fatal("critical error", zap.Error(err)) + logutil.Logger(b.ctx).Fatal("critical error", zap.Error(err)) } select { case <-b.ctx.Done(): @@ -169,7 +169,7 @@ func (b *Backoffer) BackoffWithCfgAndMaxSleep(cfg *Config, maxSleepMs int, err e errMsg += fmt.Sprintf("\nlongest sleep type: %s, time: %dms", longestSleepCfg.String(), longestSleepTime) returnedErr = longestSleepCfg.err } - logutil.BgLogger().Warn(errMsg) + 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) } From c5bf330532153a4a2a066dccf1182f733fde01ba Mon Sep 17 00:00:00 2001 From: husharp Date: Wed, 2 Aug 2023 15:14:15 +0800 Subject: [PATCH 03/49] expand min-resolved-ts to support stores Signed-off-by: husharp --- integration_tests/go.mod | 14 +++++------ integration_tests/go.sum | 22 ++++++++--------- integration_tests/pd_api_test.go | 9 ++++--- tikv/kv.go | 26 ++++++++++++++------ util/pd.go | 42 +++++++++++++++++++------------- 5 files changed, 67 insertions(+), 46 deletions(-) diff --git a/integration_tests/go.mod b/integration_tests/go.mod index 97636a2f60..b89e1af2a5 100644 --- a/integration_tests/go.mod +++ b/integration_tests/go.mod @@ -6,19 +6,19 @@ require ( github.com/ninedraft/israce v0.0.3 github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32 github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c - github.com/pingcap/kvproto v0.0.0-20230720094213-a3b4a77b4333 - github.com/pingcap/tidb v1.1.0-beta.0.20230619015310-8b1006f1af04 + github.com/pingcap/kvproto v0.0.0-20230728080053-8a9db88bc88a + github.com/pingcap/tidb v1.1.0-beta.0.20230801093416-2d23240b376a github.com/pkg/errors v0.9.1 github.com/stretchr/testify v1.8.4 github.com/tidwall/gjson v1.14.1 - github.com/tikv/client-go/v2 v2.0.8-0.20230714052714-85fc8f337565 + github.com/tikv/client-go/v2 v2.0.8-0.20230731032349-719e6456f7d5 github.com/tikv/pd/client v0.0.0-20230724080549-de985b8e0afc go.uber.org/goleak v1.2.1 ) require ( github.com/Azure/go-ntlmssp v0.0.0-20221128193559-754e69321358 // indirect - github.com/BurntSushi/toml v1.3.0 // indirect + github.com/BurntSushi/toml v1.3.2 // indirect github.com/benbjohnson/clock v1.3.5 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect @@ -65,14 +65,14 @@ require ( github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 // indirect github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22 // indirect github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 // indirect - github.com/pingcap/tidb/parser v0.0.0-20230619015310-8b1006f1af04 // indirect + github.com/pingcap/tidb/parser v0.0.0-20230801093416-2d23240b376a // indirect github.com/pingcap/tipb v0.0.0-20230607071926-bda24015c2d6 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b // indirect github.com/prometheus/client_golang v1.16.0 // indirect github.com/prometheus/client_model v0.4.0 // indirect github.com/prometheus/common v0.44.0 // indirect - github.com/prometheus/procfs v0.11.0 // indirect + github.com/prometheus/procfs v0.11.1 // indirect github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec // indirect github.com/rogpeppe/go-internal v1.10.0 // indirect github.com/sasha-s/go-deadlock v0.2.0 // indirect @@ -112,7 +112,5 @@ require ( replace ( github.com/go-ldap/ldap/v3 => github.com/YangKeao/ldap/v3 v3.4.5-0.20230421065457-369a3bab1117 - github.com/pingcap/tidb => github.com/HuSharp/tidb v1.1.0-beta.0.20230726045237-a2b0085ad7c5 - github.com/pingcap/tidb/parser => github.com/HuSharp/tidb/parser v0.0.0-20230726045237-a2b0085ad7c5 github.com/tikv/client-go/v2 => ../ ) diff --git a/integration_tests/go.sum b/integration_tests/go.sum index ac10fa09a7..96255c9e4c 100644 --- a/integration_tests/go.sum +++ b/integration_tests/go.sum @@ -14,16 +14,12 @@ github.com/Azure/go-ntlmssp v0.0.0-20221128193559-754e69321358 h1:mFRzDkZVAjdal+ github.com/Azure/go-ntlmssp v0.0.0-20221128193559-754e69321358/go.mod h1:chxPXzSsl7ZWRAuOIE23GDNzjWuZquvFlgA8xmpunjU= github.com/AzureAD/microsoft-authentication-library-for-go v0.5.1 h1:BWe8a+f/t+7KY7zH2mqygeUD0t8hNFXe08p1Pb3/jKE= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/BurntSushi/toml v1.3.0 h1:Ws8e5YmnrGEHzZEzg0YvK/7COGYtTC5PbaH9oSSbgfA= -github.com/BurntSushi/toml v1.3.0/go.mod h1:CxXYINrC8qIiEnFrOxCa7Jy5BFHlXnUU2pbicEuybxQ= +github.com/BurntSushi/toml v1.3.2 h1:o7IhLm0Msx3BaB+n3Ag7L8EVlByGnpq14C4YWiu/gL8= +github.com/BurntSushi/toml v1.3.2/go.mod h1:CxXYINrC8qIiEnFrOxCa7Jy5BFHlXnUU2pbicEuybxQ= github.com/CloudyKit/fastprinter v0.0.0-20170127035650-74b38d55f37a/go.mod h1:EFZQ978U7x8IRnstaskI3IysnWY5Ao3QgZUKOXlsAdw= github.com/CloudyKit/jet v2.1.3-0.20180809161101-62edd43e4f88+incompatible/go.mod h1:HPYO+50pSWkPoj9Q/eq0aRGByCL6ScRlUmiEX5Zgm+w= github.com/DataDog/zstd v1.4.5 h1:EndNeuB0l9syBZhut0wns3gV1hL8zX8LIu6ZiVHWLIQ= github.com/HdrHistogram/hdrhistogram-go v1.1.2 h1:5IcZpTvzydCQeHzK4Ef/D5rrSqwxob0t8PQPMybUNFM= -github.com/HuSharp/tidb v1.1.0-beta.0.20230726045237-a2b0085ad7c5 h1:wSOvDYbKkvHjlWWFBihIoeJ5yBc1jZe9Ehkku3Jn8cA= -github.com/HuSharp/tidb v1.1.0-beta.0.20230726045237-a2b0085ad7c5/go.mod h1:C3tuWINS2/Vt/gxZ0OLdGI2x5crlN8E3/qNJJkIIkTI= -github.com/HuSharp/tidb/parser v0.0.0-20230726045237-a2b0085ad7c5 h1:bxwmPI7ambmbOAaozdYz81HFpIeu6ctWo7TiXfOGE14= -github.com/HuSharp/tidb/parser v0.0.0-20230726045237-a2b0085ad7c5/go.mod h1:ENXEsaVS6N3CTMpL4txc6m93y6XaztF9W4SFLjhPWJg= github.com/Joker/hpp v1.0.0/go.mod h1:8x5n+M1Hp5hC0g8okX3sR3vFQwynaX/UgSOM9MeBKzY= github.com/Joker/jade v1.0.1-0.20190614124447-d475f43051e7/go.mod h1:6E6s8o2AE4KhCrqr6GRJjdC/gNfTdxkIXvuGZZda2VM= github.com/Masterminds/semver v1.5.0 h1:H65muMkzWKEuNDnfl9d70GUjFniHKHRbFPGBuZ3QEww= @@ -363,14 +359,18 @@ github.com/pingcap/fn v1.0.0 h1:CyA6AxcOZkQh52wIqYlAmaVmF6EvrcqFywP463pjA8g= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20230720094213-a3b4a77b4333 h1:A6Wqgq0uMw51UiRAH27TVN0QlzVR5CVtV6fTQSAmvKM= -github.com/pingcap/kvproto v0.0.0-20230720094213-a3b4a77b4333/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/kvproto v0.0.0-20230728080053-8a9db88bc88a h1:5gSwd337GRaT1E0O3y10jb2ZDzv+h30pjCpRcCC5Phg= +github.com/pingcap/kvproto v0.0.0-20230728080053-8a9db88bc88a/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/log v1.1.0/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22 h1:2SOzvGvE8beiC1Y4g9Onkvu6UmuBBOeWRGQEjJaT/JY= github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 h1:QV6jqlfOkh8hqvEAgwBZa+4bSgO0EeKC7s5c6Luam2I= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21/go.mod h1:QYnjfA95ZaMefyl1NO8oPtKeb8pYUdnDVhQgf+qdpjM= +github.com/pingcap/tidb v1.1.0-beta.0.20230801093416-2d23240b376a h1:JBh4xHmZ6w33+4fFTWZi5J3yPhLnOWChlf0Za875z8Y= +github.com/pingcap/tidb v1.1.0-beta.0.20230801093416-2d23240b376a/go.mod h1:KxfAxDSUAAgPyCpaAgQ7qTYaqDonaJA73eyGnUdMlcQ= +github.com/pingcap/tidb/parser v0.0.0-20230801093416-2d23240b376a h1:GEx1TEBP1O/mUumHhJodRa2Mk3w6Jtt+yNRq3yAxPjk= +github.com/pingcap/tidb/parser v0.0.0-20230801093416-2d23240b376a/go.mod h1:ENXEsaVS6N3CTMpL4txc6m93y6XaztF9W4SFLjhPWJg= github.com/pingcap/tipb v0.0.0-20230607071926-bda24015c2d6 h1:D79RE4RVhq2ic8sqDSv7QdL0tT5aZV3CaCXUAT41iWc= github.com/pingcap/tipb v0.0.0-20230607071926-bda24015c2d6/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/browser v0.0.0-20210115035449-ce105d075bb4 h1:Qj1ukM4GlMWXNdMBuXcXfz/Kw9s1qm0CLY32QxuSImI= @@ -407,8 +407,8 @@ github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= -github.com/prometheus/procfs v0.11.0 h1:5EAgkfkMl659uZPbe9AS2N68a7Cc1TJbPEuGzFuRbyk= -github.com/prometheus/procfs v0.11.0/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM= +github.com/prometheus/procfs v0.11.1 h1:xRC8Iq1yyca5ypa9n1EZnWZkt7dwcoRPQwX/5gwaUuI= +github.com/prometheus/procfs v0.11.1/go.mod h1:eesXgaPo1q7lBpVMoMy0ZOFTth9hBn4W/y0/p/ScXhY= github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec h1:W09IVJc94icq4NjY3clb7Lk8O1qJ8BdBEF8z0ibU0rE= github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= github.com/rivo/uniseg v0.4.4 h1:8TfxU8dW6PdqD27gjM8MVNuicgxIjxpm4K7x4jp8sis= @@ -437,7 +437,7 @@ github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd h1:ug7PpSOB5RBPK1K github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= -github.com/sirupsen/logrus v1.9.2 h1:oxx1eChJGI6Uks2ZC4W1zpLlVgqB8ner4EuQwV4Ik1Y= +github.com/sirupsen/logrus v1.9.3 h1:dueUQJ1C2q9oE3F7wvmSGAaVtTmUizReu6fjN8uqzbQ= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= github.com/soheilhy/cmux v0.1.5 h1:jjzc5WVemNEDTLwv9tlmemhC73tI08BNOIGwBOo10Js= diff --git a/integration_tests/pd_api_test.go b/integration_tests/pd_api_test.go index aa5d3855c7..31dd942c65 100644 --- a/integration_tests/pd_api_test.go +++ b/integration_tests/pd_api_test.go @@ -72,8 +72,11 @@ func (s *apiTestSuite) SetupTest() { // Set PD HTTP client. store, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0, tikv.WithPDHTTPClient(nil, addrs)) s.store = store - storeID := uint64(1) - s.store.GetRegionCache().SetRegionCacheStore(storeID, s.storeAddr(storeID), s.storeAddr(storeID), tikvrpc.TiKV, 1, nil) + // Need to start cluster at least with 3 stores for test. + for i := 1; i <= 3; i++ { + storeID := uint64(i) + s.store.GetRegionCache().SetRegionCacheStore(storeID, s.storeAddr(storeID), s.storeAddr(storeID), tikvrpc.TiKV, 1, nil) + } } func (s *apiTestSuite) storeAddr(id uint64) string { @@ -120,7 +123,7 @@ func (s *apiTestSuite) TestGetStoreMinResolvedTS() { } retryCount++ } - require.Equal(atomic.LoadInt32(&mockClient.requestCount), int32(0)) + require.Equal(int32(0), atomic.LoadInt32(&mockClient.requestCount)) require.Equal(uint64(100), s.store.GetMinSafeTS(oracle.GlobalTxnScope)) require.Nil(failpoint.Disable("tikvclient/InjectMinResolvedTS")) diff --git a/tikv/kv.go b/tikv/kv.go index 3e9753612e..752e70bb96 100644 --- a/tikv/kv.go +++ b/tikv/kv.go @@ -584,6 +584,23 @@ func (s *KVStore) updateSafeTS(ctx context.Context) { tikvClient := s.GetTiKVClient() wg := &sync.WaitGroup{} wg.Add(len(stores)) + // Try to get the minimum resolved timestamp of the store from PD. + var ( + err error + ) + var storeIDs []string + for _, store := range stores { + storeIDs = append(storeIDs, strconv.FormatUint(store.StoreID(), 10)) + } + StoreMinResolvedTSs := make(map[uint64]uint64) + if s.pdHttpClient != nil { + StoreMinResolvedTSs, err = s.pdHttpClient.GetMinResolvedTSByStoresIDs(ctx, storeIDs) + if err != nil { + // If getting the minimum resolved timestamp from PD failed, log the error and need to get it from TiKV. + logutil.BgLogger().Debug("get resolved TS from PD failed", zap.Error(err), zap.Any("stores", storeIDs)) + } + } + for _, store := range stores { storeID := store.StoreID() storeAddr := store.GetAddr() @@ -595,16 +612,11 @@ func (s *KVStore) updateSafeTS(ctx context.Context) { var ( safeTS uint64 - err error ) - storeIDStr := strconv.Itoa(int(storeID)) - // Try to get the minimum resolved timestamp of the store from PD. if s.pdHttpClient != nil { - safeTS, err = s.pdHttpClient.GetStoreMinResolvedTS(ctx, storeID) - if err != nil { - logutil.BgLogger().Debug("get resolved TS from PD failed", zap.Error(err), zap.Uint64("store-id", storeID)) - } + safeTS = StoreMinResolvedTSs[storeID] } + storeIDStr := strconv.FormatUint(storeID, 10) // If getting the minimum resolved timestamp from PD failed or returned 0, try to get it from TiKV. if safeTS == 0 || err != nil { resp, err := tikvClient.SendRequest( diff --git a/util/pd.go b/util/pd.go index b4c405c153..9ac0ea288c 100644 --- a/util/pd.go +++ b/util/pd.go @@ -35,6 +35,7 @@ package util import ( + "bytes" "context" "crypto/tls" "encoding/json" @@ -56,7 +57,7 @@ const ( // pd request retry time when connection fail. pdRequestRetryTime = 10 - storeMinResolvedTSPrefix = "pd/api/v1/min-resolved-ts" + minResolvedTSPrefix = "pd/api/v1/min-resolved-ts" ) // PDHTTPClient is an HTTP client of pd. @@ -86,45 +87,52 @@ func NewPDHTTPClient( } } -// GetStoreMinResolvedTS get store-level min-resolved-ts from pd. -func (p *PDHTTPClient) GetStoreMinResolvedTS(ctx context.Context, storeID uint64) (uint64, error) { +// GetMinResolvedTSByStoresIDs get min-resolved-ts from pd by stores ids. +func (p *PDHTTPClient) GetMinResolvedTSByStoresIDs(ctx context.Context, storeIDs []string) (map[uint64]uint64, error) { var err error for _, addr := range p.addrs { - query := fmt.Sprintf("%s/%d", storeMinResolvedTSPrefix, storeID) - v, e := pdRequest(ctx, addr, query, p.cli, http.MethodGet, nil) + data, err := json.Marshal(storeIDs) + if err != nil { + logutil.BgLogger().Debug("failed to marshal store ids", zap.String("addr", addr), zap.Error(err)) + return nil, errors.Trace(err) + } + v, e := pdRequest(ctx, addr, minResolvedTSPrefix, p.cli, http.MethodGet, bytes.NewBuffer(data)) if e != nil { logutil.BgLogger().Debug("failed to get min resolved ts", zap.String("addr", addr), zap.Error(e)) err = e continue } - logutil.BgLogger().Debug("store min resolved ts", zap.String("resp", string(v))) + logutil.BgLogger().Debug("min resolved ts", zap.String("resp", string(v))) d := struct { - IsRealTime bool `json:"is_real_time,omitempty"` - MinResolvedTS uint64 `json:"min_resolved_ts"` + IsRealTime bool `json:"is_real_time,omitempty"` + StoreMinResolvedTS map[uint64]uint64 `json:"store_min_resolved_ts"` }{} err = json.Unmarshal(v, &d) if err != nil { - return 0, errors.Trace(err) + return nil, errors.Trace(err) } if !d.IsRealTime { - message := fmt.Errorf("store min resolved ts not enabled, addr: %s", addr) + message := fmt.Errorf("min resolved ts not enabled, addr: %s", addr) logutil.BgLogger().Debug(message.Error()) - return 0, errors.Trace(message) + return nil, errors.Trace(message) } if val, e := EvalFailpoint("InjectMinResolvedTS"); e == nil { // Need to make sure successfully get from real pd. - if d.MinResolvedTS != 0 { - // Should be val.(uint64) but failpoint doesn't support that. - if tmp, ok := val.(int); ok { - d.MinResolvedTS = uint64(tmp) + for storeID, v := range d.StoreMinResolvedTS { + if v != 0 { + // Should be val.(uint64) but failpoint doesn't support that. + if tmp, ok := val.(int); ok { + d.StoreMinResolvedTS[storeID] = uint64(tmp) + } } } + } - return d.MinResolvedTS, nil + return d.StoreMinResolvedTS, nil } - return 0, errors.Trace(err) + return nil, errors.Trace(err) } // pdRequest is a func to send an HTTP to pd and return the result bytes. From 359295c3e05733101e196814205b70e5d6ba8bbd Mon Sep 17 00:00:00 2001 From: husharp Date: Wed, 2 Aug 2023 15:40:15 +0800 Subject: [PATCH 04/49] refine variables name and modify tests Signed-off-by: husharp --- .github/workflows/integration.yml | 5 ++++- tikv/kv.go | 11 +++-------- 2 files changed, 7 insertions(+), 9 deletions(-) diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml index 5312ab1018..e56f91ac31 100644 --- a/.github/workflows/integration.yml +++ b/.github/workflows/integration.yml @@ -69,7 +69,10 @@ jobs: mv ../${{steps.extract-tikv.outputs.destination}}/tikv-server . ./pd-server > pd.log 2>&1 & sleep 5 - ./tikv-server -C tikv.toml > tikv.log 2>&1 & + TIKV_COUNT=3 + for i in $(seq $TIKV_COUNT); do + ./tikv-server -C tikv.toml > "tikv${i}.log" 2>&1 & + done sleep 15 working-directory: integration_tests diff --git a/tikv/kv.go b/tikv/kv.go index 752e70bb96..21c4585ec5 100644 --- a/tikv/kv.go +++ b/tikv/kv.go @@ -586,9 +586,9 @@ func (s *KVStore) updateSafeTS(ctx context.Context) { wg.Add(len(stores)) // Try to get the minimum resolved timestamp of the store from PD. var ( - err error + storeIDs []string + err error ) - var storeIDs []string for _, store := range stores { storeIDs = append(storeIDs, strconv.FormatUint(store.StoreID(), 10)) } @@ -610,12 +610,7 @@ func (s *KVStore) updateSafeTS(ctx context.Context) { go func(ctx context.Context, wg *sync.WaitGroup, storeID uint64, storeAddr string) { defer wg.Done() - var ( - safeTS uint64 - ) - if s.pdHttpClient != nil { - safeTS = StoreMinResolvedTSs[storeID] - } + safeTS := StoreMinResolvedTSs[storeID] storeIDStr := strconv.FormatUint(storeID, 10) // If getting the minimum resolved timestamp from PD failed or returned 0, try to get it from TiKV. if safeTS == 0 || err != nil { From 0930be96a5642c2fb235d3f371012fa662d3d518 Mon Sep 17 00:00:00 2001 From: husharp Date: Wed, 2 Aug 2023 18:40:29 +0800 Subject: [PATCH 05/49] remove ci Signed-off-by: husharp --- .github/workflows/integration.yml | 5 +---- integration_tests/pd_api_test.go | 7 ++----- util/pd.go | 11 ++++++----- 3 files changed, 9 insertions(+), 14 deletions(-) diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml index e56f91ac31..5312ab1018 100644 --- a/.github/workflows/integration.yml +++ b/.github/workflows/integration.yml @@ -69,10 +69,7 @@ jobs: mv ../${{steps.extract-tikv.outputs.destination}}/tikv-server . ./pd-server > pd.log 2>&1 & sleep 5 - TIKV_COUNT=3 - for i in $(seq $TIKV_COUNT); do - ./tikv-server -C tikv.toml > "tikv${i}.log" 2>&1 & - done + ./tikv-server -C tikv.toml > tikv.log 2>&1 & sleep 15 working-directory: integration_tests diff --git a/integration_tests/pd_api_test.go b/integration_tests/pd_api_test.go index 31dd942c65..81d62d1adc 100644 --- a/integration_tests/pd_api_test.go +++ b/integration_tests/pd_api_test.go @@ -72,11 +72,8 @@ func (s *apiTestSuite) SetupTest() { // Set PD HTTP client. store, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0, tikv.WithPDHTTPClient(nil, addrs)) s.store = store - // Need to start cluster at least with 3 stores for test. - for i := 1; i <= 3; i++ { - storeID := uint64(i) - s.store.GetRegionCache().SetRegionCacheStore(storeID, s.storeAddr(storeID), s.storeAddr(storeID), tikvrpc.TiKV, 1, nil) - } + storeID := uint64(1) + s.store.GetRegionCache().SetRegionCacheStore(storeID, s.storeAddr(storeID), s.storeAddr(storeID), tikvrpc.TiKV, 1, nil) } func (s *apiTestSuite) storeAddr(id uint64) string { diff --git a/util/pd.go b/util/pd.go index 9ac0ea288c..4831c22059 100644 --- a/util/pd.go +++ b/util/pd.go @@ -104,8 +104,8 @@ func (p *PDHTTPClient) GetMinResolvedTSByStoresIDs(ctx context.Context, storeIDs } logutil.BgLogger().Debug("min resolved ts", zap.String("resp", string(v))) d := struct { - IsRealTime bool `json:"is_real_time,omitempty"` - StoreMinResolvedTS map[uint64]uint64 `json:"store_min_resolved_ts"` + IsRealTime bool `json:"is_real_time,omitempty"` + StoresMinResolvedTS map[uint64]uint64 `json:"stores_min_resolved_ts"` }{} err = json.Unmarshal(v, &d) if err != nil { @@ -118,18 +118,19 @@ func (p *PDHTTPClient) GetMinResolvedTSByStoresIDs(ctx context.Context, storeIDs } if val, e := EvalFailpoint("InjectMinResolvedTS"); e == nil { // Need to make sure successfully get from real pd. - for storeID, v := range d.StoreMinResolvedTS { + for storeID, v := range d.StoresMinResolvedTS { if v != 0 { // Should be val.(uint64) but failpoint doesn't support that. if tmp, ok := val.(int); ok { - d.StoreMinResolvedTS[storeID] = uint64(tmp) + d.StoresMinResolvedTS[storeID] = uint64(tmp) + logutil.BgLogger().Info("inject min resolved ts", zap.Uint64("storeID", storeID), zap.Uint64("ts", uint64(tmp))) } } } } - return d.StoreMinResolvedTS, nil + return d.StoresMinResolvedTS, nil } return nil, errors.Trace(err) From 1ee6c84e7dcf8442bb10b4621d6d701a2e4553d0 Mon Sep 17 00:00:00 2001 From: husharp Date: Thu, 3 Aug 2023 11:25:49 +0800 Subject: [PATCH 06/49] reduce redundant export Signed-off-by: husharp --- tikv/kv.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tikv/kv.go b/tikv/kv.go index 21c4585ec5..93aec4b3a3 100644 --- a/tikv/kv.go +++ b/tikv/kv.go @@ -592,9 +592,9 @@ func (s *KVStore) updateSafeTS(ctx context.Context) { for _, store := range stores { storeIDs = append(storeIDs, strconv.FormatUint(store.StoreID(), 10)) } - StoreMinResolvedTSs := make(map[uint64]uint64) + storeMinResolvedTSs := make(map[uint64]uint64) if s.pdHttpClient != nil { - StoreMinResolvedTSs, err = s.pdHttpClient.GetMinResolvedTSByStoresIDs(ctx, storeIDs) + storeMinResolvedTSs, err = s.pdHttpClient.GetMinResolvedTSByStoresIDs(ctx, storeIDs) if err != nil { // If getting the minimum resolved timestamp from PD failed, log the error and need to get it from TiKV. logutil.BgLogger().Debug("get resolved TS from PD failed", zap.Error(err), zap.Any("stores", storeIDs)) @@ -610,7 +610,7 @@ func (s *KVStore) updateSafeTS(ctx context.Context) { go func(ctx context.Context, wg *sync.WaitGroup, storeID uint64, storeAddr string) { defer wg.Done() - safeTS := StoreMinResolvedTSs[storeID] + safeTS := storeMinResolvedTSs[storeID] storeIDStr := strconv.FormatUint(storeID, 10) // If getting the minimum resolved timestamp from PD failed or returned 0, try to get it from TiKV. if safeTS == 0 || err != nil { From a47e1c25371baafa4166ad20e6e5aea5867ed82a Mon Sep 17 00:00:00 2001 From: cfzjywxk Date: Mon, 7 Aug 2023 14:41:06 +0800 Subject: [PATCH 07/49] Resume max retry time check for stale read retry with leader option(#903) (#911) * Resume max retry time check for stale read retry with leader option Signed-off-by: cfzjywxk * add cancel Signed-off-by: cfzjywxk --------- Signed-off-by: cfzjywxk --- internal/locate/region_request.go | 15 ++++++- internal/locate/region_request3_test.go | 57 +++++++++++++++++++++++++ 2 files changed, 71 insertions(+), 1 deletion(-) diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index 3f9d9459e3..9f4840588e 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -610,7 +610,7 @@ func (state *accessFollower) next(bo *retry.Backoffer, selector *replicaSelector // If there is no candidate, fallback to the leader. if selector.targetIdx < 0 { leader := selector.replicas[state.leaderIdx] - leaderInvalid := leader.isEpochStale() || (!state.option.leaderOnly && leader.isExhausted(1)) + leaderInvalid := leader.isEpochStale() || state.IsLeaderExhausted(leader) if len(state.option.labels) > 0 { logutil.Logger(bo.GetCtx()).Warn("unable to find stores with given labels", zap.Uint64("region", selector.region.GetID()), @@ -644,6 +644,19 @@ func (state *accessFollower) next(bo *retry.Backoffer, selector *replicaSelector return rpcCtx, nil } +func (state *accessFollower) IsLeaderExhausted(leader *replica) bool { + // Allow another extra retry for the following case: + // 1. The stale read is enabled and leader peer is selected as the target peer at first. + // 2. Data is not ready is returned from the leader peer. + // 3. Stale read flag is removed and processing falls back to snapshot read on the leader peer. + // 4. The leader peer should be retried again using snapshot read. + if state.isStaleRead && state.option.leaderOnly { + return leader.isExhausted(2) + } else { + return leader.isExhausted(1) + } +} + func (state *accessFollower) onSendFailure(bo *retry.Backoffer, selector *replicaSelector, cause error) { if selector.checkLiveness(bo, selector.targetReplica()) != reachable { selector.invalidateReplicaStore(selector.targetReplica(), cause) diff --git a/internal/locate/region_request3_test.go b/internal/locate/region_request3_test.go index 7287c89d18..91f7f22f19 100644 --- a/internal/locate/region_request3_test.go +++ b/internal/locate/region_request3_test.go @@ -37,6 +37,7 @@ package locate import ( "context" "fmt" + "strconv" "sync/atomic" "testing" "time" @@ -1142,3 +1143,59 @@ func (s *testRegionRequestToThreeStoresSuite) TestAccessFollowerAfter1TiKVDown() s.Equal(0, retryTimes) } } + +func (s *testRegionRequestToThreeStoresSuite) TestStaleReadFallback() { + leaderStore, _ := s.loadAndGetLeaderStore() + leaderLabel := []*metapb.StoreLabel{ + { + Key: "id", + Value: strconv.FormatUint(leaderStore.StoreID(), 10), + }, + } + regionLoc, err := s.cache.LocateRegionByID(s.bo, s.regionID) + s.Nil(err) + s.NotNil(regionLoc) + value := []byte("value") + isFirstReq := true + + s.regionRequestSender.client = &fnClient{fn: func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (response *tikvrpc.Response, err error) { + select { + case <-ctx.Done(): + return nil, errors.New("timeout") + default: + } + // Return `DataIsNotReady` for the first time on leader. + if isFirstReq { + isFirstReq = false + return &tikvrpc.Response{Resp: &kvrpcpb.GetResponse{RegionError: &errorpb.Error{ + DataIsNotReady: &errorpb.DataIsNotReady{}, + }}}, nil + } + return &tikvrpc.Response{Resp: &kvrpcpb.GetResponse{Value: value}}, nil + }} + + region := s.cache.getRegionByIDFromCache(regionLoc.Region.GetID()) + s.True(region.isValid()) + + req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{Key: []byte("key")}, kv.ReplicaReadLeader, nil) + req.ReadReplicaScope = oracle.GlobalTxnScope + req.TxnScope = oracle.GlobalTxnScope + req.EnableStaleRead() + req.ReplicaReadType = kv.ReplicaReadMixed + var ops []StoreSelectorOption + ops = append(ops, WithMatchLabels(leaderLabel)) + + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + bo := retry.NewBackoffer(ctx, -1) + s.Nil(err) + resp, _, _, err := s.regionRequestSender.SendReqCtx(bo, req, regionLoc.Region, time.Second, tikvrpc.TiKV, ops...) + s.Nil(err) + + regionErr, err := resp.GetRegionError() + s.Nil(err) + s.Nil(regionErr) + getResp, ok := resp.Resp.(*kvrpcpb.GetResponse) + s.True(ok) + s.Equal(getResp.Value, value) +} From a0d916d1166d767c27d70b5eaea325b9fd5f9732 Mon Sep 17 00:00:00 2001 From: ShuNing Date: Wed, 9 Aug 2023 00:38:22 +0800 Subject: [PATCH 08/49] request_source: remove default label (#890) * request_source: remove default label Signed-off-by: nolouch --- util/request_source.go | 25 +++++++++++++------------ util/request_source_test.go | 10 +++++----- 2 files changed, 18 insertions(+), 17 deletions(-) diff --git a/util/request_source.go b/util/request_source.go index ed0c9bede1..663648771a 100644 --- a/util/request_source.go +++ b/util/request_source.go @@ -44,7 +44,6 @@ const ( // explicit source types. const ( ExplicitTypeEmpty = "" - ExplicitTypeDefault = "default" ExplicitTypeLightning = "lightning" ExplicitTypeBR = "br" ExplicitTypeDumpling = "dumpling" @@ -52,7 +51,7 @@ const ( ) // ExplicitTypeList is the list of all explicit source types. -var ExplicitTypeList = []string{ExplicitTypeEmpty, ExplicitTypeDefault, ExplicitTypeLightning, ExplicitTypeBR, ExplicitTypeDumpling, ExplicitTypeBackground} +var ExplicitTypeList = []string{ExplicitTypeEmpty, ExplicitTypeLightning, ExplicitTypeBR, ExplicitTypeDumpling, ExplicitTypeBackground} const ( // InternalRequest is the scope of internal queries @@ -117,24 +116,26 @@ func IsRequestSourceInternal(reqSrc *RequestSource) bool { // GetRequestSource gets the request_source field of the request. func (r *RequestSource) GetRequestSource() string { source := SourceUnknown - explicitSourceType := ExplicitTypeDefault + origin := ExternalRequest if r == nil || (len(r.RequestSourceType) == 0 && len(r.ExplicitRequestSourceType) == 0) { // if r.RequestSourceType and r.ExplicitRequestSourceType are not set, it's mostly possible that r.RequestSourceInternal is not set // to avoid internal requests be marked as external(default value), return unknown source here. - return strings.Join([]string{source, explicitSourceType}, "_") + return source } - + if r.RequestSourceInternal { + origin = InternalRequest + } + labelList := make([]string, 0, 3) + labelList = append(labelList, origin) if len(r.RequestSourceType) > 0 { source = r.RequestSourceType } - if len(r.ExplicitRequestSourceType) > 0 { - explicitSourceType = r.ExplicitRequestSourceType + labelList = append(labelList, source) + if len(r.ExplicitRequestSourceType) > 0 && r.ExplicitRequestSourceType != r.RequestSourceType { + labelList = append(labelList, r.ExplicitRequestSourceType) } - origin := ExternalRequest - if r.RequestSourceInternal { - origin = InternalRequest - } - return strings.Join([]string{origin, source, explicitSourceType}, "_") + + return strings.Join(labelList, "_") } // RequestSourceFromCtx extract source from passed context. diff --git a/util/request_source_test.go b/util/request_source_test.go index c7548bddbb..433bc42834 100644 --- a/util/request_source_test.go +++ b/util/request_source_test.go @@ -43,19 +43,19 @@ func TestGetRequestSource(t *testing.T) { // Test nil pointer rs = nil - expected = "unknown_default" + expected = "unknown" actual = rs.GetRequestSource() assert.Equal(t, expected, actual) // Test empty RequestSourceType and ExplicitRequestSourceType rs = &RequestSource{} - expected = "unknown_default" + expected = "unknown" actual = rs.GetRequestSource() assert.Equal(t, expected, actual) // Test empty ExplicitRequestSourceType rs.RequestSourceType = "test" - expected = "external_test_default" + expected = "external_test" actual = rs.GetRequestSource() assert.Equal(t, expected, actual) @@ -79,7 +79,7 @@ func TestBuildRequestSource(t *testing.T) { assert.Equal(t, expected, actual) // Test empty ExplicitRequestSourceType - expected = "external_test_default" + expected = "external_test" actual = BuildRequestSource(false, "test", "") assert.Equal(t, expected, actual) @@ -89,7 +89,7 @@ func TestBuildRequestSource(t *testing.T) { assert.Equal(t, expected, actual) // Test RequestSourceType && ExplicitRequestSourceType both empty - expected = "unknown_default" + expected = "unknown" actual = BuildRequestSource(true, "", "") assert.Equal(t, expected, actual) } From 3f36b93a7d427920afd80ae0e487191e7f15fe6a Mon Sep 17 00:00:00 2001 From: husharp Date: Wed, 9 Aug 2023 10:29:01 +0800 Subject: [PATCH 09/49] fix error Signed-off-by: husharp --- tikv/kv.go | 10 ++++------ util/pd.go | 11 +++++++---- 2 files changed, 11 insertions(+), 10 deletions(-) diff --git a/tikv/kv.go b/tikv/kv.go index 93aec4b3a3..9e47b818f8 100644 --- a/tikv/kv.go +++ b/tikv/kv.go @@ -585,12 +585,10 @@ func (s *KVStore) updateSafeTS(ctx context.Context) { wg := &sync.WaitGroup{} wg.Add(len(stores)) // Try to get the minimum resolved timestamp of the store from PD. - var ( - storeIDs []string - err error - ) - for _, store := range stores { - storeIDs = append(storeIDs, strconv.FormatUint(store.StoreID(), 10)) + var err error + storeIDs := make([]string, len(stores)) + for i, store := range stores { + storeIDs[i] = strconv.FormatUint(store.StoreID(), 10) } storeMinResolvedTSs := make(map[uint64]uint64) if s.pdHttpClient != nil { diff --git a/util/pd.go b/util/pd.go index 4831c22059..4a8b718f87 100644 --- a/util/pd.go +++ b/util/pd.go @@ -96,13 +96,16 @@ func (p *PDHTTPClient) GetMinResolvedTSByStoresIDs(ctx context.Context, storeIDs logutil.BgLogger().Debug("failed to marshal store ids", zap.String("addr", addr), zap.Error(err)) return nil, errors.Trace(err) } - v, e := pdRequest(ctx, addr, minResolvedTSPrefix, p.cli, http.MethodGet, bytes.NewBuffer(data)) - if e != nil { - logutil.BgLogger().Debug("failed to get min resolved ts", zap.String("addr", addr), zap.Error(e)) - err = e + v, err := pdRequest(ctx, addr, minResolvedTSPrefix, p.cli, http.MethodGet, bytes.NewBuffer(data)) + if err != nil { + logutil.BgLogger().Debug("failed to get min resolved ts", zap.String("addr", addr), zap.Error(err)) continue } logutil.BgLogger().Debug("min resolved ts", zap.String("resp", string(v))) + // - When no store is given, cluster-level's min_resolved_ts will be returned, + // and min_resolved_ts for each store will be empty. + // - When given a list of stores, min_resolved_ts will be provided for each store + // and the scope-specific min_resolved_ts will be returned. d := struct { IsRealTime bool `json:"is_real_time,omitempty"` StoresMinResolvedTS map[uint64]uint64 `json:"stores_min_resolved_ts"` From 300545a8a3c4bb6f5701f6ea4b5c34a52842d0d0 Mon Sep 17 00:00:00 2001 From: glorv Date: Wed, 9 Aug 2023 13:03:15 +0800 Subject: [PATCH 10/49] add a function to set request source task type (#925) * add a function to set request source task type Signed-off-by: glorv --- util/request_source.go | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/util/request_source.go b/util/request_source.go index 663648771a..6ebadfc7e6 100644 --- a/util/request_source.go +++ b/util/request_source.go @@ -48,10 +48,11 @@ const ( ExplicitTypeBR = "br" ExplicitTypeDumpling = "dumpling" ExplicitTypeBackground = "background" + ExplicitTypeDDL = "ddl" ) // ExplicitTypeList is the list of all explicit source types. -var ExplicitTypeList = []string{ExplicitTypeEmpty, ExplicitTypeLightning, ExplicitTypeBR, ExplicitTypeDumpling, ExplicitTypeBackground} +var ExplicitTypeList = []string{ExplicitTypeEmpty, ExplicitTypeLightning, ExplicitTypeBR, ExplicitTypeDumpling, ExplicitTypeBackground, ExplicitTypeDDL} const ( // InternalRequest is the scope of internal queries @@ -94,6 +95,15 @@ func WithInternalSourceType(ctx context.Context, source string) context.Context }) } +// WithInternalSourceAndTaskType create context with internal source and task name. +func WithInternalSourceAndTaskType(ctx context.Context, source, taskName string) context.Context { + return context.WithValue(ctx, RequestSourceKey, RequestSource{ + RequestSourceInternal: true, + RequestSourceType: source, + ExplicitRequestSourceType: taskName, + }) +} + // BuildRequestSource builds a request_source from internal, source and explicitSource. func BuildRequestSource(internal bool, source, explicitSource string) string { requestSource := RequestSource{ From 27f2ad0f3a5a2ff7989e26c254d4bc0a287facbe Mon Sep 17 00:00:00 2001 From: crazycs Date: Thu, 10 Aug 2023 22:05:00 +0800 Subject: [PATCH 11/49] ci: update go version (#936) * ci: update go version Signed-off-by: crazycs520 * fix test Signed-off-by: crazycs520 --------- Signed-off-by: crazycs520 --- .github/workflows/compatibility.yml | 2 +- .github/workflows/integration.yml | 8 ++++---- .github/workflows/test.yml | 6 +++--- internal/client/main_test.go | 1 + 4 files changed, 9 insertions(+), 8 deletions(-) diff --git a/.github/workflows/compatibility.yml b/.github/workflows/compatibility.yml index c62c5b7e7e..e598e4b61e 100644 --- a/.github/workflows/compatibility.yml +++ b/.github/workflows/compatibility.yml @@ -14,7 +14,7 @@ jobs: - name: Set up Go uses: actions/setup-go@v4 with: - go-version: "1.20.x" + go-version: "1.21.0" - name: Checkout Client-Go uses: actions/checkout@v2 diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml index 5312ab1018..cde5c9d87a 100644 --- a/.github/workflows/integration.yml +++ b/.github/workflows/integration.yml @@ -17,7 +17,7 @@ jobs: - name: Set up Go uses: actions/setup-go@v4 with: - go-version: 1.20.2 + go-version: 1.21.0 - name: Test run: go test ./... @@ -32,7 +32,7 @@ jobs: - name: Set up Go uses: actions/setup-go@v4 with: - go-version: 1.20.2 + go-version: 1.21.0 - name: Test run: go test ./... -race @@ -47,7 +47,7 @@ jobs: - name: Set up Go uses: actions/setup-go@v4 with: - go-version: 1.20.2 + go-version: 1.21.0 - name: Fetch PD uses: shrink/actions-docker-extract@v1 @@ -89,7 +89,7 @@ jobs: - name: Set up Go uses: actions/setup-go@v4 with: - go-version: 1.20.2 + go-version: 1.21.0 - name: Fetch PD uses: shrink/actions-docker-extract@v1 diff --git a/.github/workflows/test.yml b/.github/workflows/test.yml index 79268c86a4..50c4909737 100644 --- a/.github/workflows/test.yml +++ b/.github/workflows/test.yml @@ -15,7 +15,7 @@ jobs: - name: Set up Go uses: actions/setup-go@v4 with: - go-version: 1.20.2 + go-version: 1.21.0 - name: Test run: go test ./... @@ -28,7 +28,7 @@ jobs: - name: Set up Go uses: actions/setup-go@v4 with: - go-version: 1.20.2 + go-version: 1.21.0 - name: Test with race run: go test -race ./... @@ -42,7 +42,7 @@ jobs: - name: Set up Go uses: actions/setup-go@v4 with: - go-version: 1.20.2 + go-version: 1.21.0 - name: Lint uses: golangci/golangci-lint-action@v3 diff --git a/internal/client/main_test.go b/internal/client/main_test.go index c4d2be246f..6a22714f06 100644 --- a/internal/client/main_test.go +++ b/internal/client/main_test.go @@ -26,6 +26,7 @@ func TestMain(m *testing.M) { opts := []goleak.Option{ 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/internal/retry.(*Config).createBackoffFn.newBackoffFn.func2"), } goleak.VerifyTestMain(m, opts...) } From 8a214402da13d5bdc1ffb1d2726bdbb47f9beb74 Mon Sep 17 00:00:00 2001 From: crazycs Date: Fri, 11 Aug 2023 11:37:10 +0800 Subject: [PATCH 12/49] use tidb_kv_read_timeout as first kv request timeout (#919) * support tidb_kv_read_timeout as first round kv request timeout Signed-off-by: crazycs520 * fix ci Signed-off-by: crazycs520 * fix ci Signed-off-by: crazycs520 * fix ci Signed-off-by: crazycs520 * fix ci Signed-off-by: crazycs520 * fix ci Signed-off-by: crazycs520 * update comment Signed-off-by: crazycs520 * refine test Signed-off-by: crazycs520 --------- Signed-off-by: crazycs520 --- examples/gcworker/gcworker.go | 1 - internal/client/client_batch.go | 6 ++ internal/locate/region_request.go | 77 +++++++++++++++++--- internal/locate/region_request3_test.go | 95 +++++++++++++++++++++++-- txnkv/txnsnapshot/snapshot.go | 29 +++++++- 5 files changed, 191 insertions(+), 17 deletions(-) diff --git a/examples/gcworker/gcworker.go b/examples/gcworker/gcworker.go index 1191adcd28..1deafdd7e5 100644 --- a/examples/gcworker/gcworker.go +++ b/examples/gcworker/gcworker.go @@ -37,7 +37,6 @@ func main() { panic(err) } - sysSafepoint, err := client.GC(context.Background(), *safepoint, tikv.WithConcurrency(10)) if err != nil { panic(err) diff --git a/internal/client/client_batch.go b/internal/client/client_batch.go index c8842e9c26..6a270bc6d2 100644 --- a/internal/client/client_batch.go +++ b/internal/client/client_batch.go @@ -347,6 +347,12 @@ func (a *batchConn) batchSendLoop(cfg config.TiKVClient) { } func (a *batchConn) getClientAndSend() { + if val, err := util.EvalFailpoint("mockBatchClientSendDelay"); err == nil { + if timeout, ok := val.(int); ok && timeout > 0 { + time.Sleep(time.Duration(timeout * int(time.Millisecond))) + } + } + // Choose a connection by round-robbin. var ( cli *batchCommandsClient diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index 9f4840588e..e071d739aa 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -245,6 +245,8 @@ type replica struct { peer *metapb.Peer epoch uint32 attempts int + // deadlineErrUsingConfTimeout indicates the replica is already tried, but the received deadline exceeded error. + deadlineErrUsingConfTimeout bool } func (r *replica) isEpochStale() bool { @@ -377,7 +379,7 @@ func (state *accessKnownLeader) onSendFailure(bo *retry.Backoffer, selector *rep } func (state *accessKnownLeader) onNoLeader(selector *replicaSelector) { - selector.state = &tryFollower{leaderIdx: state.leaderIdx, lastIdx: state.leaderIdx} + selector.state = &tryFollower{leaderIdx: state.leaderIdx, lastIdx: state.leaderIdx, fromOnNotLeader: true} } // tryFollower is the state where we cannot access the known leader @@ -391,19 +393,23 @@ type tryFollower struct { stateBase leaderIdx AccessIndex lastIdx AccessIndex + // fromOnNotLeader indicates whether the state is changed from onNotLeader. + fromOnNotLeader bool } func (state *tryFollower) next(bo *retry.Backoffer, selector *replicaSelector) (*RPCContext, error) { var targetReplica *replica + hasDeadlineExceededErr := false // Search replica that is not attempted from the last accessed replica for i := 1; i < len(selector.replicas); i++ { idx := AccessIndex((int(state.lastIdx) + i) % len(selector.replicas)) + targetReplica = selector.replicas[idx] + hasDeadlineExceededErr = hasDeadlineExceededErr || targetReplica.deadlineErrUsingConfTimeout if idx == state.leaderIdx { continue } - targetReplica = selector.replicas[idx] // Each follower is only tried once - if !targetReplica.isExhausted(1) && targetReplica.store.getLivenessState() != unreachable { + if !targetReplica.isExhausted(1) && targetReplica.store.getLivenessState() != unreachable && !targetReplica.deadlineErrUsingConfTimeout { state.lastIdx = idx selector.targetIdx = idx break @@ -411,16 +417,33 @@ func (state *tryFollower) next(bo *retry.Backoffer, selector *replicaSelector) ( } // If all followers are tried and fail, backoff and retry. if selector.targetIdx < 0 { + if hasDeadlineExceededErr { + // when meet deadline exceeded error, do fast retry without invalidate region cache. + return nil, nil + } metrics.TiKVReplicaSelectorFailureCounter.WithLabelValues("exhausted").Inc() selector.invalidateRegion() return nil, nil } - return selector.buildRPCContext(bo) + rpcCtx, err := selector.buildRPCContext(bo) + if err != nil || rpcCtx == nil { + return nil, err + } + // If the state is changed from onNotLeader, the `replicaRead` flag should not be set as leader read would still be used. + if !state.fromOnNotLeader { + replicaRead := selector.targetIdx != state.leaderIdx + rpcCtx.contextPatcher.replicaRead = &replicaRead + } + disableStaleRead := false + rpcCtx.contextPatcher.staleRead = &disableStaleRead + return rpcCtx, nil } func (state *tryFollower) onSendSuccess(selector *replicaSelector) { - if !selector.region.switchWorkLeaderToPeer(selector.targetReplica().peer) { - panic("the store must exist") + if state.fromOnNotLeader { + if !selector.region.switchWorkLeaderToPeer(selector.targetReplica().peer) { + panic("the store must exist") + } } } @@ -617,6 +640,10 @@ func (state *accessFollower) next(bo *retry.Backoffer, selector *replicaSelector zap.Bool("leader-invalid", leaderInvalid), zap.Any("labels", state.option.labels)) } + // If leader tried and received deadline exceeded error, return nil to upper layer to retry with default timeout. + if leader.deadlineErrUsingConfTimeout { + return nil, nil + } if leaderInvalid { metrics.TiKVReplicaSelectorFailureCounter.WithLabelValues("exhausted").Inc() selector.invalidateRegion() @@ -665,7 +692,7 @@ func (state *accessFollower) onSendFailure(bo *retry.Backoffer, selector *replic func (state *accessFollower) isCandidate(idx AccessIndex, replica *replica) bool { // the epoch is staled or retry exhausted, or the store is unreachable. - if replica.isEpochStale() || replica.isExhausted(1) || replica.store.getLivenessState() == unreachable { + if replica.isEpochStale() || replica.isExhausted(1) || replica.store.getLivenessState() == unreachable || replica.deadlineErrUsingConfTimeout { return false } // The request can only be sent to the leader. @@ -947,6 +974,16 @@ func (s *replicaSelector) onSendFailure(bo *retry.Backoffer, err error) { s.state.onSendFailure(bo, s, err) } +func (s *replicaSelector) onDeadlineExceeded() { + if target := s.targetReplica(); target != nil { + target.deadlineErrUsingConfTimeout = true + } + if accessLeader, ok := s.state.(*accessKnownLeader); ok { + // If leader return deadline exceeded error, we should try to access follower next time. + s.state = &tryFollower{leaderIdx: accessLeader.leaderIdx, lastIdx: accessLeader.leaderIdx} + } +} + func (s *replicaSelector) checkLiveness(bo *retry.Backoffer, accessReplica *replica) livenessState { store := accessReplica.store liveness := store.requestLiveness(bo, s.regionCache) @@ -1608,7 +1645,7 @@ func (s *RegionRequestSender) sendReqToRegion( return nil, false, err } } - if e := s.onSendFail(bo, rpcCtx, err); e != nil { + if e := s.onSendFail(bo, rpcCtx, req, err); e != nil { return nil, false, err } return nil, true, nil @@ -1638,7 +1675,7 @@ func (s *RegionRequestSender) releaseStoreToken(st *Store) { logutil.BgLogger().Warn("release store token failed, count equals to 0") } -func (s *RegionRequestSender) onSendFail(bo *retry.Backoffer, ctx *RPCContext, err error) error { +func (s *RegionRequestSender) onSendFail(bo *retry.Backoffer, ctx *RPCContext, req *tikvrpc.Request, err error) error { if span := opentracing.SpanFromContext(bo.GetCtx()); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("regionRequest.onSendFail", opentracing.ChildOf(span.Context())) defer span1.Finish() @@ -1649,6 +1686,11 @@ func (s *RegionRequestSender) onSendFail(bo *retry.Backoffer, ctx *RPCContext, e return errors.WithStack(err) } else if LoadShuttingDown() > 0 { return errors.WithStack(tikverr.ErrTiDBShuttingDown) + } else if errors.Cause(err) == context.DeadlineExceeded && req.MaxExecutionDurationMs < uint64(client.ReadTimeoutShort.Milliseconds()) { + if s.replicaSelector != nil { + s.replicaSelector.onDeadlineExceeded() + return nil + } } if status.Code(errors.Cause(err)) == codes.Canceled { select { @@ -1740,6 +1782,9 @@ func regionErrorToLabel(e *errorpb.Error) string { } else if e.GetEpochNotMatch() != nil { return "epoch_not_match" } else if e.GetServerIsBusy() != nil { + if strings.Contains(e.GetServerIsBusy().GetReason(), "deadline is exceeded") { + return "deadline_exceeded" + } return "server_is_busy" } else if e.GetStaleCommand() != nil { return "stale_command" @@ -1767,10 +1812,16 @@ func regionErrorToLabel(e *errorpb.Error) string { return "flashback_not_prepared" } else if e.GetIsWitness() != nil { return "peer_is_witness" + } else if isDeadlineExceeded(e) { + return "deadline_exceeded" } return "unknown" } +func isDeadlineExceeded(e *errorpb.Error) bool { + return strings.Contains(e.GetMessage(), "Deadline is exceeded") +} + func (s *RegionRequestSender) onRegionError( bo *retry.Backoffer, ctx *RPCContext, req *tikvrpc.Request, regionErr *errorpb.Error, ) (shouldRetry bool, err error) { @@ -1918,6 +1969,10 @@ func (s *RegionRequestSender) onRegionError( } if serverIsBusy := regionErr.GetServerIsBusy(); serverIsBusy != nil { + if s.replicaSelector != nil && strings.Contains(serverIsBusy.GetReason(), "deadline is exceeded") { + s.replicaSelector.onDeadlineExceeded() + return true, nil + } if s.replicaSelector != nil { return s.replicaSelector.onServerIsBusy(bo, ctx, req, serverIsBusy) } @@ -2046,6 +2101,10 @@ func (s *RegionRequestSender) onRegionError( return true, nil } + if isDeadlineExceeded(regionErr) && s.replicaSelector != nil { + s.replicaSelector.onDeadlineExceeded() + } + logutil.Logger(bo.GetCtx()).Debug( "tikv reports region failed", zap.Stringer("regionErr", regionErr), diff --git a/internal/locate/region_request3_test.go b/internal/locate/region_request3_test.go index 91f7f22f19..f56b4022f9 100644 --- a/internal/locate/region_request3_test.go +++ b/internal/locate/region_request3_test.go @@ -46,15 +46,18 @@ import ( "github.com/pingcap/kvproto/pkg/errorpb" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/log" "github.com/pkg/errors" "github.com/stretchr/testify/suite" tikverr "github.com/tikv/client-go/v2/error" "github.com/tikv/client-go/v2/internal/apicodec" + "github.com/tikv/client-go/v2/internal/client" "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/oracle" "github.com/tikv/client-go/v2/tikvrpc" + "go.uber.org/zap" ) func TestRegionRequestToThreeStores(t *testing.T) { @@ -711,7 +714,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestSendReqWithReplicaSelector() { // Normal bo := retry.NewBackoffer(context.Background(), -1) sender := s.regionRequestSender - resp, _, err := sender.SendReq(bo, req, region.Region, time.Second) + resp, _, err := sender.SendReq(bo, req, region.Region, client.ReadTimeoutShort) s.Nil(err) s.NotNil(resp) s.True(bo.GetTotalBackoffTimes() == 0) @@ -720,7 +723,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestSendReqWithReplicaSelector() { bo = retry.NewBackoffer(context.Background(), -1) s.cluster.ChangeLeader(s.regionID, s.peerIDs[1]) s.cluster.StopStore(s.storeIDs[0]) - resp, _, err = sender.SendReq(bo, req, region.Region, time.Second) + resp, _, err = sender.SendReq(bo, req, region.Region, client.ReadTimeoutShort) s.Nil(err) s.NotNil(resp) s.Equal(sender.replicaSelector.targetIdx, AccessIndex(1)) @@ -729,8 +732,9 @@ func (s *testRegionRequestToThreeStoresSuite) TestSendReqWithReplicaSelector() { atomic.StoreUint32(®ionStore.stores[0].livenessState, uint32(reachable)) // Leader is updated because of send success, so no backoff. + reloadRegion() bo = retry.NewBackoffer(context.Background(), -1) - resp, _, err = sender.SendReq(bo, req, region.Region, time.Second) + resp, _, err = sender.SendReq(bo, req, region.Region, client.ReadTimeoutShort) s.Nil(err) s.NotNil(resp) s.Equal(sender.replicaSelector.targetIdx, AccessIndex(1)) @@ -1092,7 +1096,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestAccessFollowerAfter1TiKVDown() s.NotEqual(leaderAddr, "") for i := 0; i < 10; i++ { bo := retry.NewBackofferWithVars(context.Background(), 100, nil) - resp, _, _, err := s.regionRequestSender.SendReqCtx(bo, req, loc.Region, time.Second, tikvrpc.TiKV) + resp, _, _, err := s.regionRequestSender.SendReqCtx(bo, req, loc.Region, client.ReadTimeoutShort, tikvrpc.TiKV) s.Nil(err) s.NotNil(resp) @@ -1135,7 +1139,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestAccessFollowerAfter1TiKVDown() for i := 0; i < 100; i++ { bo := retry.NewBackofferWithVars(context.Background(), 1, nil) - resp, _, retryTimes, err := s.regionRequestSender.SendReqCtx(bo, req, loc.Region, time.Second, tikvrpc.TiKV) + resp, _, retryTimes, err := s.regionRequestSender.SendReqCtx(bo, req, loc.Region, client.ReadTimeoutShort, tikvrpc.TiKV) s.Nil(err) s.NotNil(resp) // since all follower'store is unreachable, the request will be sent to leader, the backoff times should be 0. @@ -1199,3 +1203,84 @@ func (s *testRegionRequestToThreeStoresSuite) TestStaleReadFallback() { s.True(ok) s.Equal(getResp.Value, value) } + +func (s *testRegionRequestToThreeStoresSuite) TestSendReqFirstTimeout() { + leaderAddr := "" + reqTargetAddrs := make(map[string]struct{}) + s.regionRequestSender.RegionRequestRuntimeStats = NewRegionRequestRuntimeStats() + bo := retry.NewBackoffer(context.Background(), 10000) + mockRPCClient := &fnClient{fn: func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { + reqTargetAddrs[addr] = struct{}{} + if req.Context.MaxExecutionDurationMs < 10 { + return nil, context.DeadlineExceeded + } + if addr != leaderAddr && !req.Context.ReplicaRead && !req.Context.StaleRead { + return &tikvrpc.Response{Resp: &kvrpcpb.GetResponse{RegionError: &errorpb.Error{NotLeader: &errorpb.NotLeader{}}}}, nil + } + return &tikvrpc.Response{Resp: &kvrpcpb.GetResponse{Value: []byte("value")}}, nil + }} + getLocFn := func() *KeyLocation { + loc, err := s.regionRequestSender.regionCache.LocateKey(bo, []byte("a")) + s.Nil(err) + region := s.regionRequestSender.regionCache.GetCachedRegionWithRLock(loc.Region) + leaderStore, _, _, _ := region.WorkStorePeer(region.getStore()) + leaderAddr, err = s.regionRequestSender.regionCache.getStoreAddr(s.bo, region, leaderStore) + s.Nil(err) + return loc + } + resetStats := func() { + reqTargetAddrs = make(map[string]struct{}) + s.regionRequestSender = NewRegionRequestSender(s.cache, mockRPCClient) + s.regionRequestSender.RegionRequestRuntimeStats = NewRegionRequestRuntimeStats() + } + + //Test different read type. + staleReadTypes := []bool{false, true} + replicaReadTypes := []kv.ReplicaReadType{kv.ReplicaReadLeader, kv.ReplicaReadFollower, kv.ReplicaReadMixed} + for _, staleRead := range staleReadTypes { + for _, tp := range replicaReadTypes { + log.Info("TestSendReqFirstTimeout", zap.Bool("stale-read", staleRead), zap.String("replica-read-type", tp.String())) + resetStats() + req := tikvrpc.NewRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{Key: []byte("a")}, kvrpcpb.Context{}) + if staleRead { + req.EnableStaleRead() + } else { + req.ReplicaRead = tp.IsFollowerRead() + req.ReplicaReadType = tp + } + loc := getLocFn() + resp, _, _, err := s.regionRequestSender.SendReqCtx(bo, req, loc.Region, time.Millisecond, tikvrpc.TiKV) + s.Nil(err) + regionErr, err := resp.GetRegionError() + s.Nil(err) + s.True(IsFakeRegionError(regionErr)) + s.Equal(1, len(s.regionRequestSender.Stats)) + if staleRead { + rpcNum := s.regionRequestSender.Stats[tikvrpc.CmdGet].Count + s.True(rpcNum == 1 || rpcNum == 2) // 1 rpc or 2 rpc + } else { + s.Equal(int64(3), s.regionRequestSender.Stats[tikvrpc.CmdGet].Count) // 3 rpc + s.Equal(3, len(reqTargetAddrs)) // each rpc to a different store. + } + s.Equal(0, bo.GetTotalBackoffTimes()) // no backoff since fast retry. + // warn: must rest MaxExecutionDurationMs before retry. + resetStats() + if staleRead { + req.EnableStaleRead() + } else { + req.ReplicaRead = tp.IsFollowerRead() + req.ReplicaReadType = tp + } + req.Context.MaxExecutionDurationMs = 0 + resp, _, _, err = s.regionRequestSender.SendReqCtx(bo, req, loc.Region, time.Second, tikvrpc.TiKV) + s.Nil(err) + regionErr, err = resp.GetRegionError() + s.Nil(err) + s.Nil(regionErr) + s.Equal([]byte("value"), resp.Resp.(*kvrpcpb.GetResponse).Value) + s.Equal(1, len(s.regionRequestSender.Stats)) + s.Equal(int64(1), s.regionRequestSender.Stats[tikvrpc.CmdGet].Count) // 1 rpc + s.Equal(0, bo.GetTotalBackoffTimes()) // no backoff since fast retry. + } + } +} diff --git a/txnkv/txnsnapshot/snapshot.go b/txnkv/txnsnapshot/snapshot.go index a4253d62e7..ab29655355 100644 --- a/txnkv/txnsnapshot/snapshot.go +++ b/txnkv/txnsnapshot/snapshot.go @@ -118,6 +118,7 @@ type KVSnapshot struct { resolvedLocks util.TSSet committedLocks util.TSSet scanBatchSize int + readTimeout time.Duration // Cache the result of BatchGet. // The invariance is that calling BatchGet multiple times using the same start ts, @@ -387,6 +388,7 @@ func (s *KVSnapshot) batchGetSingleRegion(bo *retry.Backoffer, batch batchKeys, pending := batch.keys var resolvingRecordToken *int + useConfigurableKVTimeout := true for { s.mu.RLock() req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdBatchGet, &kvrpcpb.BatchGetRequest{ @@ -416,6 +418,12 @@ func (s *KVSnapshot) batchGetSingleRegion(bo *retry.Backoffer, batch batchKeys, if isStaleness { req.EnableStaleRead() } + timeout := client.ReadTimeoutMedium + if useConfigurableKVTimeout && s.readTimeout > 0 { + useConfigurableKVTimeout = false + timeout = s.readTimeout + } + req.MaxExecutionDurationMs = uint64(timeout.Milliseconds()) ops := make([]locate.StoreSelectorOption, 0, 2) if len(matchStoreLabels) > 0 { ops = append(ops, locate.WithMatchLabels(matchStoreLabels)) @@ -427,7 +435,7 @@ func (s *KVSnapshot) batchGetSingleRegion(bo *retry.Backoffer, batch batchKeys, } req.ReplicaReadType = readType } - resp, _, _, err := cli.SendReqCtx(bo, req, batch.region, client.ReadTimeoutMedium, tikvrpc.TiKV, "", ops...) + resp, _, _, err := cli.SendReqCtx(bo, req, batch.region, timeout, tikvrpc.TiKV, "", ops...) if err != nil { return err } @@ -651,13 +659,20 @@ func (s *KVSnapshot) get(ctx context.Context, bo *retry.Backoffer, k []byte) ([] var firstLock *txnlock.Lock var resolvingRecordToken *int + useConfigurableKVTimeout := true for { util.EvalFailpoint("beforeSendPointGet") loc, err := s.store.GetRegionCache().LocateKey(bo, k) if err != nil { return nil, err } - resp, _, _, err := cli.SendReqCtx(bo, req, loc.Region, client.ReadTimeoutShort, tikvrpc.TiKV, "", ops...) + timeout := client.ReadTimeoutShort + if useConfigurableKVTimeout && s.readTimeout > 0 { + useConfigurableKVTimeout = false + timeout = s.readTimeout + } + req.MaxExecutionDurationMs = uint64(timeout.Milliseconds()) + resp, _, _, err := cli.SendReqCtx(bo, req, loc.Region, timeout, tikvrpc.TiKV, "", ops...) if err != nil { return nil, err } @@ -984,6 +999,16 @@ func (s *KVSnapshot) mergeRegionRequestStats(stats map[tikvrpc.CmdType]*locate.R } } +// SetKVReadTimeout sets timeout for individual KV read operations under this snapshot +func (s *KVSnapshot) SetKVReadTimeout(readTimeout time.Duration) { + s.readTimeout = readTimeout +} + +// GetKVReadTimeout returns timeout for individual KV read operations under this snapshot or 0 if timeout is not set +func (s *KVSnapshot) GetKVReadTimeout() time.Duration { + return s.readTimeout +} + func (s *KVSnapshot) getResolveLockDetail() *util.ResolveLockDetail { s.mu.RLock() defer s.mu.RUnlock() From 45894d9d36495b1d562e71700de1f2bbac883f21 Mon Sep 17 00:00:00 2001 From: ShuNing Date: Fri, 11 Aug 2023 17:40:06 +0800 Subject: [PATCH 13/49] [pick] resource_control: bypass some internal urgent request (#938) * resource_control: bypass some internal urgent request (#884) Signed-off-by: nolouch * resourcecontrol: fix nil pointer (#900) Signed-off-by: nolouch --------- Signed-off-by: nolouch --- internal/client/client_interceptor.go | 7 ++- internal/resourcecontrol/resource_control.go | 27 +++++++++-- .../resourcecontrol/resource_control_test.go | 48 +++++++++++++++++++ util/request_source.go | 2 + 4 files changed, 80 insertions(+), 4 deletions(-) create mode 100644 internal/resourcecontrol/resource_control_test.go diff --git a/internal/client/client_interceptor.go b/internal/client/client_interceptor.go index 74cabbb8cf..64ae333edd 100644 --- a/internal/client/client_interceptor.go +++ b/internal/client/client_interceptor.go @@ -107,10 +107,15 @@ func buildResourceControlInterceptor( // Build the interceptor. interceptFn := func(next interceptor.RPCInterceptorFunc) interceptor.RPCInterceptorFunc { return func(target string, req *tikvrpc.Request) (*tikvrpc.Response, error) { + // bypass some internal requests and it's may influence user experience. For example, the + // request of `alter user password`, totally bypasses the resource control. it's not cost + // many resources, but it's may influence the user experience. // If the resource group has background jobs, we should not record consumption and wait for it. - if resourceControlInterceptor.IsBackgroundRequest(ctx, resourceGroupName, req.RequestSource) { + // Background jobs will record and report in tikv side. + if reqInfo.Bypass() || resourceControlInterceptor.IsBackgroundRequest(ctx, resourceGroupName, req.RequestSource) { return next(target, req) } + consumption, penalty, err := resourceControlInterceptor.OnRequestWait(ctx, resourceGroupName, reqInfo) if err != nil { return nil, err diff --git a/internal/resourcecontrol/resource_control.go b/internal/resourcecontrol/resource_control.go index fb55d25fde..23a48b3c21 100644 --- a/internal/resourcecontrol/resource_control.go +++ b/internal/resourcecontrol/resource_control.go @@ -16,12 +16,14 @@ package resourcecontrol import ( "reflect" + "strings" "time" "github.com/pingcap/kvproto/pkg/coprocessor" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/tikv/client-go/v2/internal/logutil" "github.com/tikv/client-go/v2/tikvrpc" + "github.com/tikv/client-go/v2/util" "go.uber.org/zap" ) @@ -34,12 +36,23 @@ type RequestInfo struct { writeBytes int64 storeID uint64 replicaNumber int64 + // bypass indicates whether the request should be bypassed. + // some internal request should be bypassed, such as Privilege request. + bypass bool } // MakeRequestInfo extracts the relevant information from a BatchRequest. func MakeRequestInfo(req *tikvrpc.Request) *RequestInfo { + var bypass bool + requestSource := req.Context.GetRequestSource() + if len(requestSource) > 0 { + if strings.Contains(requestSource, util.InternalRequestPrefix+util.InternalTxnOthers) { + bypass = true + } + } + storeID := req.Context.GetPeer().GetStoreId() if !req.IsTxnWriteRequest() && !req.IsRawWriteRequest() { - return &RequestInfo{writeBytes: -1} + return &RequestInfo{writeBytes: -1, storeID: storeID, bypass: bypass} } var writeBytes int64 @@ -57,7 +70,7 @@ func MakeRequestInfo(req *tikvrpc.Request) *RequestInfo { writeBytes += int64(len(k)) } } - return &RequestInfo{writeBytes: writeBytes, storeID: req.Context.Peer.StoreId, replicaNumber: req.ReplicaNumber} + return &RequestInfo{writeBytes: writeBytes, storeID: storeID, replicaNumber: req.ReplicaNumber, bypass: bypass} } // IsWrite returns whether the request is a write request. @@ -68,13 +81,21 @@ func (req *RequestInfo) IsWrite() bool { // WriteBytes returns the actual write size of the request, // -1 will be returned if it's not a write request. func (req *RequestInfo) WriteBytes() uint64 { - return uint64(req.writeBytes) + if req.writeBytes > 0 { + return uint64(req.writeBytes) + } + return 0 } func (req *RequestInfo) ReplicaNumber() int64 { return req.replicaNumber } +// Bypass returns whether the request should be bypassed. +func (req *RequestInfo) Bypass() bool { + return req.bypass +} + func (req *RequestInfo) StoreID() uint64 { return req.storeID } diff --git a/internal/resourcecontrol/resource_control_test.go b/internal/resourcecontrol/resource_control_test.go new file mode 100644 index 0000000000..25f6f72aa4 --- /dev/null +++ b/internal/resourcecontrol/resource_control_test.go @@ -0,0 +1,48 @@ +package resourcecontrol + +import ( + "testing" + + "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/stretchr/testify/assert" + "github.com/tikv/client-go/v2/tikvrpc" +) + +func TestMakeRequestInfo(t *testing.T) { + // Test a non-write request. + req := &tikvrpc.Request{Req: &kvrpcpb.BatchGetRequest{}, Context: kvrpcpb.Context{Peer: &metapb.Peer{StoreId: 1}}} + info := MakeRequestInfo(req) + assert.False(t, info.IsWrite()) + assert.Equal(t, uint64(0), info.WriteBytes()) + assert.False(t, info.Bypass()) + assert.Equal(t, uint64(1), info.StoreID()) + + // Test a prewrite request. + mutation := &kvrpcpb.Mutation{Key: []byte("foo"), Value: []byte("bar")} + prewriteReq := &kvrpcpb.PrewriteRequest{Mutations: []*kvrpcpb.Mutation{mutation}, PrimaryLock: []byte("baz")} + req = &tikvrpc.Request{Type: tikvrpc.CmdPrewrite, Req: prewriteReq, ReplicaNumber: 1, Context: kvrpcpb.Context{Peer: &metapb.Peer{StoreId: 2}}} + requestSource := "xxx_internal_others" + req.Context.RequestSource = requestSource + info = MakeRequestInfo(req) + assert.True(t, info.IsWrite()) + assert.Equal(t, uint64(9), info.WriteBytes()) + assert.True(t, info.Bypass()) + assert.Equal(t, uint64(2), info.StoreID()) + // Test a commit request. + commitReq := &kvrpcpb.CommitRequest{Keys: [][]byte{[]byte("qux")}} + req = &tikvrpc.Request{Type: tikvrpc.CmdCommit, Req: commitReq, ReplicaNumber: 2, Context: kvrpcpb.Context{Peer: &metapb.Peer{StoreId: 3}}} + info = MakeRequestInfo(req) + assert.True(t, info.IsWrite()) + assert.Equal(t, uint64(3), info.WriteBytes()) + assert.False(t, info.Bypass()) + assert.Equal(t, uint64(3), info.StoreID()) + + // Test Nil Peer in Context + req = &tikvrpc.Request{Type: tikvrpc.CmdCommit, Req: commitReq, ReplicaNumber: 2, Context: kvrpcpb.Context{}} + info = MakeRequestInfo(req) + assert.True(t, info.IsWrite()) + assert.Equal(t, uint64(3), info.WriteBytes()) + assert.False(t, info.Bypass()) + assert.Equal(t, uint64(0), info.StoreID()) +} diff --git a/util/request_source.go b/util/request_source.go index 6ebadfc7e6..97d3b83fdb 100644 --- a/util/request_source.go +++ b/util/request_source.go @@ -57,6 +57,8 @@ var ExplicitTypeList = []string{ExplicitTypeEmpty, ExplicitTypeLightning, Explic const ( // InternalRequest is the scope of internal queries InternalRequest = "internal" + // InternalRequestPrefix is the prefix of internal queries + InternalRequestPrefix = "internal_" // ExternalRequest is the scope of external queries ExternalRequest = "external" // SourceUnknown keeps same with the default value(empty string) From 44f5025f5a4d1cab07ca17dfd70b1891b37419ee Mon Sep 17 00:00:00 2001 From: you06 Date: Fri, 11 Aug 2023 21:36:47 +0800 Subject: [PATCH 14/49] fallback to follower when leader is busy (#916) (#923) * fallback to follower when leader is busy Signed-off-by: you06 Co-authored-by: cfzjywxk Co-authored-by: cfzjywxk --- internal/locate/region_cache.go | 2 + internal/locate/region_request.go | 124 ++++++++++++++++++------ internal/locate/region_request3_test.go | 96 +++++++++++++++++- 3 files changed, 192 insertions(+), 30 deletions(-) diff --git a/internal/locate/region_cache.go b/internal/locate/region_cache.go index 87cd067044..cf57b521be 100644 --- a/internal/locate/region_cache.go +++ b/internal/locate/region_cache.go @@ -1289,9 +1289,11 @@ func (c *RegionCache) reloadRegion(regionID uint64) { // ignore error and use old region info. logutil.Logger(bo.GetCtx()).Error("load region failure", zap.Uint64("regionID", regionID), zap.Error(err)) + c.mu.RLock() if oldRegion := c.getRegionByIDFromCache(regionID); oldRegion != nil { oldRegion.asyncReload.Store(false) } + c.mu.RUnlock() return } c.mu.Lock() diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index e071d739aa..eca28b872b 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -395,26 +395,51 @@ type tryFollower struct { lastIdx AccessIndex // fromOnNotLeader indicates whether the state is changed from onNotLeader. fromOnNotLeader bool + labels []*metapb.StoreLabel } func (state *tryFollower) next(bo *retry.Backoffer, selector *replicaSelector) (*RPCContext, error) { - var targetReplica *replica hasDeadlineExceededErr := false - // Search replica that is not attempted from the last accessed replica - for i := 1; i < len(selector.replicas); i++ { - idx := AccessIndex((int(state.lastIdx) + i) % len(selector.replicas)) - targetReplica = selector.replicas[idx] - hasDeadlineExceededErr = hasDeadlineExceededErr || targetReplica.deadlineErrUsingConfTimeout - if idx == state.leaderIdx { - continue + //hasDeadlineExceededErr || targetReplica.deadlineErrUsingConfTimeout + filterReplicas := func(fn func(*replica) bool) (AccessIndex, *replica) { + for i := 0; i < len(selector.replicas); i++ { + idx := AccessIndex((int(state.lastIdx) + i) % len(selector.replicas)) + if idx == state.leaderIdx { + continue + } + selectReplica := selector.replicas[idx] + hasDeadlineExceededErr = hasDeadlineExceededErr || selectReplica.deadlineErrUsingConfTimeout + if selectReplica.store.getLivenessState() != unreachable && !selectReplica.deadlineErrUsingConfTimeout && + fn(selectReplica) { + return idx, selectReplica + } } - // Each follower is only tried once - if !targetReplica.isExhausted(1) && targetReplica.store.getLivenessState() != unreachable && !targetReplica.deadlineErrUsingConfTimeout { + return -1, nil + } + + if len(state.labels) > 0 { + idx, selectReplica := filterReplicas(func(selectReplica *replica) bool { + return selectReplica.store.IsLabelsMatch(state.labels) + }) + if selectReplica != nil && idx >= 0 { + state.lastIdx = idx + selector.targetIdx = idx + } + // labels only take effect for first try. + state.labels = nil + } + + if selector.targetIdx < 0 { + // Search replica that is not attempted from the last accessed replica + idx, selectReplica := filterReplicas(func(selectReplica *replica) bool { + return !selectReplica.isExhausted(1) + }) + if selectReplica != nil && idx >= 0 { state.lastIdx = idx selector.targetIdx = idx - break } } + // If all followers are tried and fail, backoff and retry. if selector.targetIdx < 0 { if hasDeadlineExceededErr { @@ -427,22 +452,24 @@ func (state *tryFollower) next(bo *retry.Backoffer, selector *replicaSelector) ( } rpcCtx, err := selector.buildRPCContext(bo) if err != nil || rpcCtx == nil { - return nil, err + return rpcCtx, err } - // If the state is changed from onNotLeader, the `replicaRead` flag should not be set as leader read would still be used. if !state.fromOnNotLeader { - replicaRead := selector.targetIdx != state.leaderIdx + replicaRead := true rpcCtx.contextPatcher.replicaRead = &replicaRead } - disableStaleRead := false - rpcCtx.contextPatcher.staleRead = &disableStaleRead + staleRead := false + rpcCtx.contextPatcher.staleRead = &staleRead return rpcCtx, nil } func (state *tryFollower) onSendSuccess(selector *replicaSelector) { if state.fromOnNotLeader { - if !selector.region.switchWorkLeaderToPeer(selector.targetReplica().peer) { - panic("the store must exist") + peer := selector.targetReplica().peer + if !selector.region.switchWorkLeaderToPeer(peer) { + logutil.BgLogger().Warn("the store must exist", + zap.Uint64("store", peer.StoreId), + zap.Uint64("peer", peer.Id)) } } } @@ -565,6 +592,10 @@ type accessFollower struct { learnerOnly bool } +// Follower read will try followers first, if no follower is available, it will fallback to leader. +// Specially, for stale read, it tries local peer(can be either leader or follower), then use snapshot read in the leader, +// if the leader read receive server-is-busy and connection errors, the region cache is still valid, +// and the state will be changed to tryFollower, which will read by replica read. func (state *accessFollower) next(bo *retry.Backoffer, selector *replicaSelector) (*RPCContext, error) { replicaSize := len(selector.replicas) resetStaleRead := false @@ -633,7 +664,8 @@ func (state *accessFollower) next(bo *retry.Backoffer, selector *replicaSelector // If there is no candidate, fallback to the leader. if selector.targetIdx < 0 { leader := selector.replicas[state.leaderIdx] - leaderInvalid := leader.isEpochStale() || state.IsLeaderExhausted(leader) + leaderEpochStale := leader.isEpochStale() + leaderInvalid := leaderEpochStale || state.IsLeaderExhausted(leader) if len(state.option.labels) > 0 { logutil.Logger(bo.GetCtx()).Warn("unable to find stores with given labels", zap.Uint64("region", selector.region.GetID()), @@ -645,6 +677,20 @@ func (state *accessFollower) next(bo *retry.Backoffer, selector *replicaSelector return nil, nil } if leaderInvalid { + // In stale-read, the request will fallback to leader after the local follower failure. + // If the leader is also unavailable, we can fallback to the follower and use replica-read flag again, + // The remote follower not tried yet, and the local follower can retry without stale-read flag. + if state.isStaleRead { + selector.state = &tryFollower{ + leaderIdx: state.leaderIdx, + lastIdx: state.leaderIdx, + labels: state.option.labels, + } + if leaderEpochStale { + selector.regionCache.scheduleReloadRegion(selector.region) + } + return nil, stateChanged{} + } metrics.TiKVReplicaSelectorFailureCounter.WithLabelValues("exhausted").Inc() selector.invalidateRegion() return nil, nil @@ -695,23 +741,25 @@ func (state *accessFollower) isCandidate(idx AccessIndex, replica *replica) bool if replica.isEpochStale() || replica.isExhausted(1) || replica.store.getLivenessState() == unreachable || replica.deadlineErrUsingConfTimeout { return false } - // The request can only be sent to the leader. - if state.option.leaderOnly && idx == state.leaderIdx { - return true + if state.option.leaderOnly { + // The request can only be sent to the leader. + return idx == state.leaderIdx } - // Choose a replica with matched labels. - followerCandidate := !state.option.leaderOnly && (state.tryLeader || idx != state.leaderIdx) && - replica.store.IsLabelsMatch(state.option.labels) && (!state.learnerOnly || replica.peer.Role == metapb.PeerRole_Learner) - if !followerCandidate { + if !state.tryLeader && idx == state.leaderIdx { + // The request cannot be sent to leader. return false } + if state.learnerOnly { + // The request can only be sent to the learner. + return replica.peer.Role == metapb.PeerRole_Learner + } // And If the leader store is abnormal to be accessed under `ReplicaReadPreferLeader` mode, we should choose other valid followers // as candidates to serve the Read request. if state.option.preferLeader && replica.store.isSlow() { return false } - // If the stores are limited, check if the store is in the list. - return replica.store.IsStoreMatch(state.option.stores) + // Choose a replica with matched labels. + return replica.store.IsStoreMatch(state.option.stores) && replica.store.IsLabelsMatch(state.option.labels) } // tryIdleReplica is the state where we find the leader is busy and retry the request using replica read. @@ -1101,6 +1149,9 @@ func (s *replicaSelector) onServerIsBusy( // Mark the server is busy (the next incoming READs could be redirect // to expected followers. ) ctx.Store.markAlreadySlow() + if s.canFallback2Follower() { + return true, nil + } } err = bo.Backoff(retry.BoTiKVServerBusy, errors.Errorf("server is busy, ctx: %v", ctx)) if err != nil { @@ -1109,6 +1160,23 @@ func (s *replicaSelector) onServerIsBusy( return true, nil } +// For some reasons, the leader is unreachable by now, try followers instead. +// the state is changed in accessFollower.next when leader is unavailable. +func (s *replicaSelector) canFallback2Follower() bool { + if s == nil || s.state == nil { + return false + } + state, ok := s.state.(*accessFollower) + if !ok { + return false + } + if !state.isStaleRead { + return false + } + // can fallback to follower only when the leader is exhausted. + return state.lastIdx == state.leaderIdx && state.IsLeaderExhausted(s.replicas[state.leaderIdx]) +} + func (s *replicaSelector) invalidateRegion() { if s.region != nil { s.region.invalidate(Other) diff --git a/internal/locate/region_request3_test.go b/internal/locate/region_request3_test.go index f56b4022f9..bae33b7cfd 100644 --- a/internal/locate/region_request3_test.go +++ b/internal/locate/region_request3_test.go @@ -348,7 +348,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestLearnerReplicaSelector() { atomic.StoreInt64(®ion.lastAccess, time.Now().Unix()) rpcCtx, err := replicaSelector.next(s.bo) s.Nil(err) - // Should swith to the next follower. + // Should switch to the next follower. s.Equal(AccessIndex(tikvLearnerAccessIdx), accessLearner.lastIdx) AssertRPCCtxEqual(s, rpcCtx, replicaSelector.replicas[replicaSelector.targetIdx], nil) } @@ -590,7 +590,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestReplicaSelector() { for i := 0; i < regionStore.accessStoreNum(tiKVOnly)-1; i++ { rpcCtx, err := replicaSelector.next(s.bo) s.Nil(err) - // Should swith to the next follower. + // Should switch to the next follower. s.NotEqual(lastIdx, state3.lastIdx) // Shouldn't access the leader if followers aren't exhausted. s.NotEqual(regionStore.workTiKVIdx, state3.lastIdx) @@ -1284,3 +1284,95 @@ func (s *testRegionRequestToThreeStoresSuite) TestSendReqFirstTimeout() { } } } + +func (s *testRegionRequestToThreeStoresSuite) TestStaleReadFallback2Follower() { + leaderStore, _ := s.loadAndGetLeaderStore() + leaderLabel := []*metapb.StoreLabel{ + { + Key: "id", + Value: strconv.FormatUint(leaderStore.StoreID(), 10), + }, + } + var followerID *uint64 + for _, storeID := range s.storeIDs { + if storeID != leaderStore.storeID { + id := storeID + followerID = &id + break + } + } + s.NotNil(followerID) + followerLabel := []*metapb.StoreLabel{ + { + Key: "id", + Value: strconv.FormatUint(*followerID, 10), + }, + } + + regionLoc, err := s.cache.LocateRegionByID(s.bo, s.regionID) + s.Nil(err) + s.NotNil(regionLoc) + + dataIsNotReady := false + s.regionRequestSender.client = &fnClient{fn: func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (response *tikvrpc.Response, err error) { + select { + case <-ctx.Done(): + return nil, errors.New("timeout") + default: + } + if dataIsNotReady && req.StaleRead { + dataIsNotReady = false + return &tikvrpc.Response{Resp: &kvrpcpb.GetResponse{RegionError: &errorpb.Error{ + DataIsNotReady: &errorpb.DataIsNotReady{}, + }}}, nil + } + if addr == leaderStore.addr { + return &tikvrpc.Response{Resp: &kvrpcpb.GetResponse{RegionError: &errorpb.Error{ + ServerIsBusy: &errorpb.ServerIsBusy{}, + }}}, nil + } + if !req.ReplicaRead { + return &tikvrpc.Response{Resp: &kvrpcpb.GetResponse{RegionError: &errorpb.Error{ + NotLeader: &errorpb.NotLeader{}, + }}}, nil + } + return &tikvrpc.Response{Resp: &kvrpcpb.GetResponse{Value: []byte(addr)}}, nil + }} + + for _, localLeader := range []bool{true, false} { + dataIsNotReady = true + // data is not ready, then server is busy in the first round, + // directly server is busy in the second round. + for i := 0; i < 2; i++ { + req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{Key: []byte("key")}, kv.ReplicaReadLeader, nil) + req.ReadReplicaScope = oracle.GlobalTxnScope + req.TxnScope = oracle.GlobalTxnScope + req.EnableStaleRead() + req.ReplicaReadType = kv.ReplicaReadMixed + var ops []StoreSelectorOption + if localLeader { + ops = append(ops, WithMatchLabels(leaderLabel)) + } else { + ops = append(ops, WithMatchLabels(followerLabel)) + } + + ctx, cancel := context.WithTimeout(context.Background(), 10000*time.Second) + bo := retry.NewBackoffer(ctx, -1) + s.Nil(err) + resp, _, _, err := s.regionRequestSender.SendReqCtx(bo, req, regionLoc.Region, time.Second, tikvrpc.TiKV, ops...) + s.Nil(err) + + regionErr, err := resp.GetRegionError() + s.Nil(err) + s.Nil(regionErr) + getResp, ok := resp.Resp.(*kvrpcpb.GetResponse) + s.True(ok) + if localLeader { + s.NotEqual(getResp.Value, []byte("store"+leaderLabel[0].Value)) + } else { + s.Equal(getResp.Value, []byte("store"+followerLabel[0].Value)) + } + cancel() + } + } +} From 1a932e727e8284798c1628a3448dcb5a20e873e7 Mon Sep 17 00:00:00 2001 From: husharp Date: Mon, 14 Aug 2023 13:11:16 +0800 Subject: [PATCH 15/49] suuport api bench for params Signed-off-by: husharp --- tikv/kv.go | 8 ++++---- util/pd.go | 22 +++++++++------------- 2 files changed, 13 insertions(+), 17 deletions(-) diff --git a/tikv/kv.go b/tikv/kv.go index 9e47b818f8..468ffbf154 100644 --- a/tikv/kv.go +++ b/tikv/kv.go @@ -586,12 +586,12 @@ func (s *KVStore) updateSafeTS(ctx context.Context) { wg.Add(len(stores)) // Try to get the minimum resolved timestamp of the store from PD. var err error - storeIDs := make([]string, len(stores)) - for i, store := range stores { - storeIDs[i] = strconv.FormatUint(store.StoreID(), 10) - } storeMinResolvedTSs := make(map[uint64]uint64) if s.pdHttpClient != nil { + storeIDs := make([]string, len(stores)) + for i, store := range stores { + storeIDs[i] = strconv.FormatUint(store.StoreID(), 10) + } storeMinResolvedTSs, err = s.pdHttpClient.GetMinResolvedTSByStoresIDs(ctx, storeIDs) if err != nil { // If getting the minimum resolved timestamp from PD failed, log the error and need to get it from TiKV. diff --git a/util/pd.go b/util/pd.go index 4a8b718f87..a9af5f2371 100644 --- a/util/pd.go +++ b/util/pd.go @@ -35,7 +35,6 @@ package util import ( - "bytes" "context" "crypto/tls" "encoding/json" @@ -91,21 +90,18 @@ func NewPDHTTPClient( func (p *PDHTTPClient) GetMinResolvedTSByStoresIDs(ctx context.Context, storeIDs []string) (map[uint64]uint64, error) { var err error for _, addr := range p.addrs { - data, err := json.Marshal(storeIDs) - if err != nil { - logutil.BgLogger().Debug("failed to marshal store ids", zap.String("addr", addr), zap.Error(err)) - return nil, errors.Trace(err) - } - v, err := pdRequest(ctx, addr, minResolvedTSPrefix, p.cli, http.MethodGet, bytes.NewBuffer(data)) - if err != nil { - logutil.BgLogger().Debug("failed to get min resolved ts", zap.String("addr", addr), zap.Error(err)) + // scope is an optional parameter, it can be `cluster` or specified store IDs. + // - When no scope is given, cluster-level's min_resolved_ts will be returned and storesMinResolvedTS will be nil. + // - When scope is `cluster`, cluster-level's min_resolved_ts will be returned and storesMinResolvedTS will be filled. + // - When scope given a list of stores, min_resolved_ts will be provided for each store + // and the scope-specific min_resolved_ts will be returned. + query := fmt.Sprintf("%s?scope=%s", minResolvedTSPrefix, strings.Join(storeIDs, ",")) + v, e := pdRequest(ctx, addr, query, p.cli, http.MethodGet, nil) + if e != nil { + logutil.BgLogger().Debug("failed to get min resolved ts", zap.String("addr", addr), zap.Error(e)) continue } logutil.BgLogger().Debug("min resolved ts", zap.String("resp", string(v))) - // - When no store is given, cluster-level's min_resolved_ts will be returned, - // and min_resolved_ts for each store will be empty. - // - When given a list of stores, min_resolved_ts will be provided for each store - // and the scope-specific min_resolved_ts will be returned. d := struct { IsRealTime bool `json:"is_real_time,omitempty"` StoresMinResolvedTS map[uint64]uint64 `json:"stores_min_resolved_ts"` From b771e6317faac5ae8cd2683e0c10e9957e1a8eaa Mon Sep 17 00:00:00 2001 From: husharp Date: Mon, 14 Aug 2023 14:24:53 +0800 Subject: [PATCH 16/49] make ci happy Signed-off-by: husharp --- tikv/kv.go | 8 +++++--- util/pd.go | 24 ++++++++++++++++++------ 2 files changed, 23 insertions(+), 9 deletions(-) diff --git a/tikv/kv.go b/tikv/kv.go index 468ffbf154..fbdd65d8bb 100644 --- a/tikv/kv.go +++ b/tikv/kv.go @@ -586,7 +586,7 @@ func (s *KVStore) updateSafeTS(ctx context.Context) { wg.Add(len(stores)) // Try to get the minimum resolved timestamp of the store from PD. var err error - storeMinResolvedTSs := make(map[uint64]uint64) + var storeMinResolvedTSs map[uint64]uint64 if s.pdHttpClient != nil { storeIDs := make([]string, len(stores)) for i, store := range stores { @@ -608,10 +608,10 @@ func (s *KVStore) updateSafeTS(ctx context.Context) { go func(ctx context.Context, wg *sync.WaitGroup, storeID uint64, storeAddr string) { defer wg.Done() - safeTS := storeMinResolvedTSs[storeID] + var safeTS uint64 storeIDStr := strconv.FormatUint(storeID, 10) // If getting the minimum resolved timestamp from PD failed or returned 0, try to get it from TiKV. - if safeTS == 0 || err != nil { + if storeMinResolvedTSs == nil || storeMinResolvedTSs[storeID] == 0 || err != nil { resp, err := tikvClient.SendRequest( ctx, storeAddr, tikvrpc.NewRequest( tikvrpc.CmdStoreSafeTS, &kvrpcpb.StoreSafeTSRequest{ @@ -630,6 +630,8 @@ func (s *KVStore) updateSafeTS(ctx context.Context) { return } safeTS = resp.Resp.(*kvrpcpb.StoreSafeTSResponse).GetSafeTs() + } else { + safeTS = storeMinResolvedTSs[storeID] } _, preSafeTS := s.getSafeTS(storeID) diff --git a/util/pd.go b/util/pd.go index a9af5f2371..68ac8c3e75 100644 --- a/util/pd.go +++ b/util/pd.go @@ -99,6 +99,7 @@ func (p *PDHTTPClient) GetMinResolvedTSByStoresIDs(ctx context.Context, storeIDs v, e := pdRequest(ctx, addr, query, p.cli, http.MethodGet, nil) if e != nil { logutil.BgLogger().Debug("failed to get min resolved ts", zap.String("addr", addr), zap.Error(e)) + err = e continue } logutil.BgLogger().Debug("min resolved ts", zap.String("resp", string(v))) @@ -117,14 +118,25 @@ func (p *PDHTTPClient) GetMinResolvedTSByStoresIDs(ctx context.Context, storeIDs } if val, e := EvalFailpoint("InjectMinResolvedTS"); e == nil { // Need to make sure successfully get from real pd. - for storeID, v := range d.StoresMinResolvedTS { - if v != 0 { - // Should be val.(uint64) but failpoint doesn't support that. - if tmp, ok := val.(int); ok { - d.StoresMinResolvedTS[storeID] = uint64(tmp) - logutil.BgLogger().Info("inject min resolved ts", zap.Uint64("storeID", storeID), zap.Uint64("ts", uint64(tmp))) + if d.StoresMinResolvedTS != nil { + for storeID, v := range d.StoresMinResolvedTS { + if v != 0 { + // Should be val.(uint64) but failpoint doesn't support that. + if tmp, ok := val.(int); ok { + d.StoresMinResolvedTS[storeID] = uint64(tmp) + logutil.BgLogger().Info("inject min resolved ts", zap.Uint64("storeID", storeID), zap.Uint64("ts", uint64(tmp))) + } } } + } else { + // Should be val.(uint64) but failpoint doesn't support that. + if tmp, ok := val.(int); ok { + // ci's store id is 1, we can change it if we have more stores. + // but for pool ci it's no need to do that :( + d.StoresMinResolvedTS = make(map[uint64]uint64) + d.StoresMinResolvedTS[1] = uint64(tmp) + logutil.BgLogger().Info("inject min resolved ts", zap.Uint64("ts", uint64(tmp))) + } } } From a0ac170698524cbf20a869cb3471e969608d599a Mon Sep 17 00:00:00 2001 From: you06 Date: Mon, 14 Aug 2023 15:27:20 +0800 Subject: [PATCH 17/49] handle mismatch peer id (#935) Signed-off-by: you06 --- internal/locate/region_request.go | 14 +++++++ internal/locate/region_request3_test.go | 49 +++++++++++++++++++++++++ 2 files changed, 63 insertions(+) diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index eca28b872b..47fac7b07d 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -1882,6 +1882,8 @@ func regionErrorToLabel(e *errorpb.Error) string { return "peer_is_witness" } else if isDeadlineExceeded(e) { return "deadline_exceeded" + } else if e.GetMismatchPeerId() != nil { + return "mismatch_peer_id" } return "unknown" } @@ -2173,6 +2175,18 @@ func (s *RegionRequestSender) onRegionError( s.replicaSelector.onDeadlineExceeded() } + if mismatch := regionErr.GetMismatchPeerId(); mismatch != nil { + logutil.Logger(bo.GetCtx()).Warn( + "tikv reports `MismatchPeerId`, invalidate region cache", + zap.Uint64("req peer id", mismatch.GetRequestPeerId()), + zap.Uint64("store peer id", mismatch.GetStorePeerId()), + ) + if s.replicaSelector != nil { + s.replicaSelector.invalidateRegion() + } + return false, nil + } + logutil.Logger(bo.GetCtx()).Debug( "tikv reports region failed", zap.Stringer("regionErr", regionErr), diff --git a/internal/locate/region_request3_test.go b/internal/locate/region_request3_test.go index bae33b7cfd..28bc43e87d 100644 --- a/internal/locate/region_request3_test.go +++ b/internal/locate/region_request3_test.go @@ -1376,3 +1376,52 @@ func (s *testRegionRequestToThreeStoresSuite) TestStaleReadFallback2Follower() { } } } + +func (s *testRegionRequestToThreeStoresSuite) TestReplicaReadFallbackToLeaderRegionError() { + regionLoc, err := s.cache.LocateRegionByID(s.bo, s.regionID) + s.Nil(err) + s.NotNil(regionLoc) + + s.regionRequestSender.client = &fnClient{fn: func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (response *tikvrpc.Response, err error) { + select { + case <-ctx.Done(): + return nil, errors.New("timeout") + default: + } + // Return `mismatch peer id` when accesses the leader. + if addr == s.cluster.GetStore(s.storeIDs[0]).Address { + return &tikvrpc.Response{Resp: &kvrpcpb.GetResponse{RegionError: &errorpb.Error{ + MismatchPeerId: &errorpb.MismatchPeerId{ + RequestPeerId: 1, + StorePeerId: 2, + }, + }}}, nil + } + return &tikvrpc.Response{Resp: &kvrpcpb.GetResponse{RegionError: &errorpb.Error{ + DataIsNotReady: &errorpb.DataIsNotReady{}, + }}}, nil + }} + + region := s.cache.getRegionByIDFromCache(regionLoc.Region.GetID()) + s.True(region.isValid()) + + req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{Key: []byte("key")}, kv.ReplicaReadLeader, nil) + req.ReadReplicaScope = oracle.GlobalTxnScope + req.TxnScope = oracle.GlobalTxnScope + req.EnableStaleRead() + req.ReplicaReadType = kv.ReplicaReadFollower + + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + bo := retry.NewBackoffer(ctx, -1) + s.Nil(err) + resp, _, _, err := s.regionRequestSender.SendReqCtx(bo, req, regionLoc.Region, time.Second, tikvrpc.TiKV) + s.Nil(err) + regionErr, err := resp.GetRegionError() + s.Nil(err) + s.Equal(regionErrorToLabel(regionErr), "mismatch_peer_id") + // return non-epoch-not-match region error and the upper layer can auto retry. + s.Nil(regionErr.GetEpochNotMatch()) + // after region error returned, the region should be invalidated. + s.False(region.isValid()) +} From c65aaeb8f4eaeb6a96c7a0a5b25217ac1c9085ef Mon Sep 17 00:00:00 2001 From: husharp Date: Wed, 16 Aug 2023 11:56:09 +0800 Subject: [PATCH 18/49] address comment Signed-off-by: husharp --- tikv/kv.go | 15 ++++++++------- util/pd.go | 14 ++++++-------- 2 files changed, 14 insertions(+), 15 deletions(-) diff --git a/tikv/kv.go b/tikv/kv.go index fbdd65d8bb..e1cab15bbe 100644 --- a/tikv/kv.go +++ b/tikv/kv.go @@ -585,10 +585,12 @@ func (s *KVStore) updateSafeTS(ctx context.Context) { wg := &sync.WaitGroup{} wg.Add(len(stores)) // Try to get the minimum resolved timestamp of the store from PD. - var err error - var storeMinResolvedTSs map[uint64]uint64 + var ( + err error + storeMinResolvedTSs map[uint64]uint64 + ) + storeIDs := make([]string, len(stores)) if s.pdHttpClient != nil { - storeIDs := make([]string, len(stores)) for i, store := range stores { storeIDs[i] = strconv.FormatUint(store.StoreID(), 10) } @@ -599,17 +601,16 @@ func (s *KVStore) updateSafeTS(ctx context.Context) { } } - for _, store := range stores { + for i, store := range stores { storeID := store.StoreID() storeAddr := store.GetAddr() if store.IsTiFlash() { storeAddr = store.GetPeerAddr() } - go func(ctx context.Context, wg *sync.WaitGroup, storeID uint64, storeAddr string) { + go func(ctx context.Context, wg *sync.WaitGroup, storeID uint64, storeAddr string, storeIDStr string) { defer wg.Done() var safeTS uint64 - storeIDStr := strconv.FormatUint(storeID, 10) // If getting the minimum resolved timestamp from PD failed or returned 0, try to get it from TiKV. if storeMinResolvedTSs == nil || storeMinResolvedTSs[storeID] == 0 || err != nil { resp, err := tikvClient.SendRequest( @@ -645,7 +646,7 @@ func (s *KVStore) updateSafeTS(ctx context.Context) { metrics.TiKVSafeTSUpdateCounter.WithLabelValues("success", storeIDStr).Inc() safeTSTime := oracle.GetTimeFromTS(safeTS) metrics.TiKVMinSafeTSGapSeconds.WithLabelValues(storeIDStr).Set(time.Since(safeTSTime).Seconds()) - }(ctx, wg, storeID, storeAddr) + }(ctx, wg, storeID, storeAddr, storeIDs[i]) } txnScopeMap := make(map[string][]uint64) diff --git a/util/pd.go b/util/pd.go index 68ac8c3e75..a2eb2cb893 100644 --- a/util/pd.go +++ b/util/pd.go @@ -128,15 +128,13 @@ func (p *PDHTTPClient) GetMinResolvedTSByStoresIDs(ctx context.Context, storeIDs } } } - } else { + } else if tmp, ok := val.(int); ok { // Should be val.(uint64) but failpoint doesn't support that. - if tmp, ok := val.(int); ok { - // ci's store id is 1, we can change it if we have more stores. - // but for pool ci it's no need to do that :( - d.StoresMinResolvedTS = make(map[uint64]uint64) - d.StoresMinResolvedTS[1] = uint64(tmp) - logutil.BgLogger().Info("inject min resolved ts", zap.Uint64("ts", uint64(tmp))) - } + // ci's store id is 1, we can change it if we have more stores. + // but for pool ci it's no need to do that :( + d.StoresMinResolvedTS = make(map[uint64]uint64) + d.StoresMinResolvedTS[1] = uint64(tmp) + logutil.BgLogger().Info("inject min resolved ts", zap.Uint64("ts", uint64(tmp))) } } From cedc94b3eedca1b4477b23e93ed093b6de9e0e62 Mon Sep 17 00:00:00 2001 From: husharp Date: Wed, 16 Aug 2023 15:59:30 +0800 Subject: [PATCH 19/49] use config instead Signed-off-by: husharp --- integration_tests/pd_api_test.go | 55 ++++++++++++++++++++++++++++++++ tikv/kv.go | 42 +++++++++++++++++++++++- util/pd.go | 19 ++++++----- 3 files changed, 107 insertions(+), 9 deletions(-) diff --git a/integration_tests/pd_api_test.go b/integration_tests/pd_api_test.go index 81d62d1adc..0fd9111f47 100644 --- a/integration_tests/pd_api_test.go +++ b/integration_tests/pd_api_test.go @@ -37,6 +37,8 @@ package tikv_test import ( "context" "fmt" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/client-go/v2/config" "strings" "sync/atomic" "testing" @@ -141,6 +143,59 @@ func (s *apiTestSuite) TestGetStoreMinResolvedTS() { require.Equal(uint64(150), s.store.GetMinSafeTS(oracle.GlobalTxnScope)) } +func (s *apiTestSuite) TestDCLabelClusterMinResolvedTS() { + util.EnableFailpoints() + // Try to get the minimum resolved timestamp of the cluster from PD. + require := s.Require() + require.Nil(failpoint.Enable("tikvclient/InjectMinResolvedTS", `return(100)`)) + mockClient := storeSafeTsMockClient{ + Client: s.store.GetTiKVClient(), + } + s.store.SetTiKVClient(&mockClient) + var retryCount int + for s.store.GetMinSafeTS(oracle.GlobalTxnScope) != 100 { + time.Sleep(2 * time.Second) + if retryCount > 5 { + break + } + retryCount++ + } + require.Equal(atomic.LoadInt32(&mockClient.requestCount), int32(0)) + require.Equal(uint64(100), s.store.GetMinSafeTS(oracle.GlobalTxnScope)) + defer func() { + s.Require().Nil(failpoint.Disable("tikvclient/InjectMinResolvedTS")) + }() + + // Set DC label for store 1. + dcLabel := "testDC" + restore := config.UpdateGlobal(func(conf *config.Config) { + conf.TxnScope = dcLabel + }) + defer restore() + + labels := []*metapb.StoreLabel{ + { + Key: tikv.DCLabelKey, + Value: dcLabel, + }, + } + storeID := uint64(1) + s.store.GetRegionCache().SetRegionCacheStore(storeID, s.storeAddr(storeID), s.storeAddr(storeID), tikvrpc.TiKV, 1, labels) + + // Try to get the minimum resolved timestamp of the store from TiKV. + retryCount = 0 + for s.store.GetMinSafeTS(dcLabel) != 150 { + time.Sleep(2 * time.Second) + if retryCount > 5 { + break + } + retryCount++ + } + + require.GreaterOrEqual(atomic.LoadInt32(&mockClient.requestCount), int32(1)) + require.Equal(uint64(150), s.store.GetMinSafeTS(dcLabel)) +} + func (s *apiTestSuite) TearDownTest() { if s.store != nil { s.Require().Nil(s.store.Close()) diff --git a/tikv/kv.go b/tikv/kv.go index e1cab15bbe..a2ccfcb234 100644 --- a/tikv/kv.go +++ b/tikv/kv.go @@ -580,6 +580,12 @@ func (s *KVStore) safeTSUpdater() { } func (s *KVStore) updateSafeTS(ctx context.Context) { + // Try getting the cluster-level minimum resolved timestamp from PD first. + if s.setGlobalMinSafeTSByPD(ctx) { + return + } + + // When txn scope is not global, we need to get the minimum resolved timestamp of each store. stores := s.regionCache.GetAllStores() tikvClient := s.GetTiKVClient() wg := &sync.WaitGroup{} @@ -594,7 +600,7 @@ func (s *KVStore) updateSafeTS(ctx context.Context) { for i, store := range stores { storeIDs[i] = strconv.FormatUint(store.StoreID(), 10) } - storeMinResolvedTSs, err = s.pdHttpClient.GetMinResolvedTSByStoresIDs(ctx, storeIDs) + _, storeMinResolvedTSs, err = s.pdHttpClient.GetMinResolvedTSByStoresIDs(ctx, storeIDs) if err != nil { // If getting the minimum resolved timestamp from PD failed, log the error and need to get it from TiKV. logutil.BgLogger().Debug("get resolved TS from PD failed", zap.Error(err), zap.Any("stores", storeIDs)) @@ -663,6 +669,40 @@ func (s *KVStore) updateSafeTS(ctx context.Context) { wg.Wait() } +var ( + skipSafeTSUpdateCounter = metrics.TiKVSafeTSUpdateCounter.WithLabelValues("skip", "cluster") + successSafeTSUpdateCounter = metrics.TiKVSafeTSUpdateCounter.WithLabelValues("success", "cluster") + clusterMinSafeTSGap = metrics.TiKVMinSafeTSGapSeconds.WithLabelValues("cluster") +) + +// setGlobalMinSafeTSByPD check whether it is needed to get cluster-level's min resolved ts from PD. +func (s *KVStore) setGlobalMinSafeTSByPD(ctx context.Context) bool { + isGlobal := config.GetTxnScopeFromConfig() == oracle.GlobalTxnScope + // Try to get the minimum resolved timestamp of the cluster from PD. + if s.pdHttpClient != nil && isGlobal { + clusterMinSafeTS, _, err := s.pdHttpClient.GetMinResolvedTSByStoresIDs(ctx, nil) + if err != nil { + logutil.BgLogger().Debug("get resolved TS from PD failed", zap.Error(err)) + } else if clusterMinSafeTS != 0 { + // update metrics. + preClusterMinSafeTS := s.GetMinSafeTS(oracle.GlobalTxnScope) + if preClusterMinSafeTS > clusterMinSafeTS { + skipSafeTSUpdateCounter.Inc() + preSafeTSTime := oracle.GetTimeFromTS(preClusterMinSafeTS) + clusterMinSafeTSGap.Set(time.Since(preSafeTSTime).Seconds()) + } else { + s.minSafeTS.Store(oracle.GlobalTxnScope, clusterMinSafeTS) + successSafeTSUpdateCounter.Inc() + safeTSTime := oracle.GetTimeFromTS(clusterMinSafeTS) + clusterMinSafeTSGap.Set(time.Since(safeTSTime).Seconds()) + } + return true + } + } + + return false +} + func (s *KVStore) ruRuntimeStatsMapCleaner() { defer s.wg.Done() t := time.NewTicker(ruRuntimeStatsCleanInterval) diff --git a/util/pd.go b/util/pd.go index a2eb2cb893..6f27c514fd 100644 --- a/util/pd.go +++ b/util/pd.go @@ -87,7 +87,7 @@ func NewPDHTTPClient( } // GetMinResolvedTSByStoresIDs get min-resolved-ts from pd by stores ids. -func (p *PDHTTPClient) GetMinResolvedTSByStoresIDs(ctx context.Context, storeIDs []string) (map[uint64]uint64, error) { +func (p *PDHTTPClient) GetMinResolvedTSByStoresIDs(ctx context.Context, storeIDs []string) (uint64, map[uint64]uint64, error) { var err error for _, addr := range p.addrs { // scope is an optional parameter, it can be `cluster` or specified store IDs. @@ -95,7 +95,10 @@ func (p *PDHTTPClient) GetMinResolvedTSByStoresIDs(ctx context.Context, storeIDs // - When scope is `cluster`, cluster-level's min_resolved_ts will be returned and storesMinResolvedTS will be filled. // - When scope given a list of stores, min_resolved_ts will be provided for each store // and the scope-specific min_resolved_ts will be returned. - query := fmt.Sprintf("%s?scope=%s", minResolvedTSPrefix, strings.Join(storeIDs, ",")) + query := minResolvedTSPrefix + if len(storeIDs) != 0 { + query = fmt.Sprintf("%s?scope=%s", query, strings.Join(storeIDs, ",")) + } v, e := pdRequest(ctx, addr, query, p.cli, http.MethodGet, nil) if e != nil { logutil.BgLogger().Debug("failed to get min resolved ts", zap.String("addr", addr), zap.Error(e)) @@ -104,17 +107,18 @@ func (p *PDHTTPClient) GetMinResolvedTSByStoresIDs(ctx context.Context, storeIDs } logutil.BgLogger().Debug("min resolved ts", zap.String("resp", string(v))) d := struct { + MinResolvedTS uint64 `json:"min_resolved_ts"` IsRealTime bool `json:"is_real_time,omitempty"` StoresMinResolvedTS map[uint64]uint64 `json:"stores_min_resolved_ts"` }{} err = json.Unmarshal(v, &d) if err != nil { - return nil, errors.Trace(err) + return 0, nil, errors.Trace(err) } if !d.IsRealTime { message := fmt.Errorf("min resolved ts not enabled, addr: %s", addr) logutil.BgLogger().Debug(message.Error()) - return nil, errors.Trace(message) + return 0, nil, errors.Trace(message) } if val, e := EvalFailpoint("InjectMinResolvedTS"); e == nil { // Need to make sure successfully get from real pd. @@ -132,17 +136,16 @@ func (p *PDHTTPClient) GetMinResolvedTSByStoresIDs(ctx context.Context, storeIDs // Should be val.(uint64) but failpoint doesn't support that. // ci's store id is 1, we can change it if we have more stores. // but for pool ci it's no need to do that :( - d.StoresMinResolvedTS = make(map[uint64]uint64) - d.StoresMinResolvedTS[1] = uint64(tmp) + d.MinResolvedTS = uint64(tmp) logutil.BgLogger().Info("inject min resolved ts", zap.Uint64("ts", uint64(tmp))) } } - return d.StoresMinResolvedTS, nil + return d.MinResolvedTS, d.StoresMinResolvedTS, nil } - return nil, errors.Trace(err) + return 0, nil, errors.Trace(err) } // pdRequest is a func to send an HTTP to pd and return the result bytes. From 97f9c1835bc01c56556eff60765e75c48c9ac7a8 Mon Sep 17 00:00:00 2001 From: husharp Date: Mon, 21 Aug 2023 10:26:21 +0800 Subject: [PATCH 20/49] nit change Signed-off-by: husharp --- integration_tests/pd_api_test.go | 10 +++++----- tikv/kv.go | 10 +++++----- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/integration_tests/pd_api_test.go b/integration_tests/pd_api_test.go index 0fd9111f47..91dbba1606 100644 --- a/integration_tests/pd_api_test.go +++ b/integration_tests/pd_api_test.go @@ -37,8 +37,6 @@ package tikv_test import ( "context" "fmt" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/tikv/client-go/v2/config" "strings" "sync/atomic" "testing" @@ -46,7 +44,9 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/suite" + "github.com/tikv/client-go/v2/config" "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/tikv" "github.com/tikv/client-go/v2/tikvrpc" @@ -105,9 +105,9 @@ func (c *storeSafeTsMockClient) CloseAddr(addr string) error { return c.Client.CloseAddr(addr) } -func (s *apiTestSuite) TestGetStoreMinResolvedTS() { +func (s *apiTestSuite) TestGetClusterMinResolvedTS() { util.EnableFailpoints() - // Try to get the minimum resolved timestamp of the store from PD. + // Try to get the minimum resolved timestamp of the cluster from PD. require := s.Require() require.Nil(failpoint.Enable("tikvclient/InjectMinResolvedTS", `return(100)`)) mockClient := storeSafeTsMockClient{ @@ -126,7 +126,7 @@ func (s *apiTestSuite) TestGetStoreMinResolvedTS() { require.Equal(uint64(100), s.store.GetMinSafeTS(oracle.GlobalTxnScope)) require.Nil(failpoint.Disable("tikvclient/InjectMinResolvedTS")) - // Try to get the minimum resolved timestamp of the store from TiKV. + // Try to get the minimum resolved timestamp of the cluster from TiKV. require.Nil(failpoint.Enable("tikvclient/InjectMinResolvedTS", `return(0)`)) defer func() { s.Require().Nil(failpoint.Disable("tikvclient/InjectMinResolvedTS")) diff --git a/tikv/kv.go b/tikv/kv.go index a2ccfcb234..d7f9a41996 100644 --- a/tikv/kv.go +++ b/tikv/kv.go @@ -580,8 +580,8 @@ func (s *KVStore) safeTSUpdater() { } func (s *KVStore) updateSafeTS(ctx context.Context) { - // Try getting the cluster-level minimum resolved timestamp from PD first. - if s.setGlobalMinSafeTSByPD(ctx) { + // Try to get the cluster-level minimum resolved timestamp from PD first. + if s.setClusterMinSafeTSByPD(ctx) { return } @@ -675,8 +675,8 @@ var ( clusterMinSafeTSGap = metrics.TiKVMinSafeTSGapSeconds.WithLabelValues("cluster") ) -// setGlobalMinSafeTSByPD check whether it is needed to get cluster-level's min resolved ts from PD. -func (s *KVStore) setGlobalMinSafeTSByPD(ctx context.Context) bool { +// setClusterMinSafeTSByPD check whether it is needed to get cluster-level's min resolved ts from PD. +func (s *KVStore) setClusterMinSafeTSByPD(ctx context.Context) bool { isGlobal := config.GetTxnScopeFromConfig() == oracle.GlobalTxnScope // Try to get the minimum resolved timestamp of the cluster from PD. if s.pdHttpClient != nil && isGlobal { @@ -684,7 +684,7 @@ func (s *KVStore) setGlobalMinSafeTSByPD(ctx context.Context) bool { if err != nil { logutil.BgLogger().Debug("get resolved TS from PD failed", zap.Error(err)) } else if clusterMinSafeTS != 0 { - // update metrics. + // Update metrics. preClusterMinSafeTS := s.GetMinSafeTS(oracle.GlobalTxnScope) if preClusterMinSafeTS > clusterMinSafeTS { skipSafeTSUpdateCounter.Inc() From 1ea3d66418d048cfa69beb9a099773d8ebc6b310 Mon Sep 17 00:00:00 2001 From: you06 Date: Tue, 22 Aug 2023 14:45:13 +0800 Subject: [PATCH 21/49] use leader read when `tryFollower` is fallback from `accessKnownLeader` (#952) * fix tryFollower Signed-off-by: you06 * address comment Signed-off-by: you06 --------- Signed-off-by: you06 --- internal/locate/region_request.go | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index 47fac7b07d..96d259aacf 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -347,7 +347,7 @@ func (state *accessKnownLeader) next(bo *retry.Backoffer, selector *replicaSelec // a request. So, before the new leader is elected, we should not send requests // to the unreachable old leader to avoid unnecessary timeout. if liveness != reachable || leader.isExhausted(maxReplicaAttempt) { - selector.state = &tryFollower{leaderIdx: state.leaderIdx, lastIdx: state.leaderIdx} + selector.state = &tryFollower{leaderIdx: state.leaderIdx, lastIdx: state.leaderIdx, fromAccessKnownLeader: true} return nil, stateChanged{} } if selector.busyThreshold > 0 { @@ -371,7 +371,7 @@ func (state *accessKnownLeader) onSendFailure(bo *retry.Backoffer, selector *rep return } if liveness != reachable || selector.targetReplica().isExhausted(maxReplicaAttempt) { - selector.state = &tryFollower{leaderIdx: state.leaderIdx, lastIdx: state.leaderIdx} + selector.state = &tryFollower{leaderIdx: state.leaderIdx, lastIdx: state.leaderIdx, fromAccessKnownLeader: true} } if liveness != reachable { selector.invalidateReplicaStore(selector.targetReplica(), cause) @@ -379,7 +379,7 @@ func (state *accessKnownLeader) onSendFailure(bo *retry.Backoffer, selector *rep } func (state *accessKnownLeader) onNoLeader(selector *replicaSelector) { - selector.state = &tryFollower{leaderIdx: state.leaderIdx, lastIdx: state.leaderIdx, fromOnNotLeader: true} + selector.state = &tryFollower{leaderIdx: state.leaderIdx, lastIdx: state.leaderIdx, fromAccessKnownLeader: true} } // tryFollower is the state where we cannot access the known leader @@ -393,9 +393,9 @@ type tryFollower struct { stateBase leaderIdx AccessIndex lastIdx AccessIndex - // fromOnNotLeader indicates whether the state is changed from onNotLeader. - fromOnNotLeader bool - labels []*metapb.StoreLabel + // fromAccessKnownLeader indicates whether the state is changed from `accessKnownLeader`. + fromAccessKnownLeader bool + labels []*metapb.StoreLabel } func (state *tryFollower) next(bo *retry.Backoffer, selector *replicaSelector) (*RPCContext, error) { @@ -454,7 +454,7 @@ func (state *tryFollower) next(bo *retry.Backoffer, selector *replicaSelector) ( if err != nil || rpcCtx == nil { return rpcCtx, err } - if !state.fromOnNotLeader { + if !state.fromAccessKnownLeader { replicaRead := true rpcCtx.contextPatcher.replicaRead = &replicaRead } @@ -464,7 +464,7 @@ func (state *tryFollower) next(bo *retry.Backoffer, selector *replicaSelector) ( } func (state *tryFollower) onSendSuccess(selector *replicaSelector) { - if state.fromOnNotLeader { + if state.fromAccessKnownLeader { peer := selector.targetReplica().peer if !selector.region.switchWorkLeaderToPeer(peer) { logutil.BgLogger().Warn("the store must exist", From ff39b4af09f180ff5034fdc36dbcb4a90d14eab2 Mon Sep 17 00:00:00 2001 From: buffer Date: Tue, 22 Aug 2023 17:25:20 +0800 Subject: [PATCH 22/49] error: handle bucket version not match (#918) --- go.mod | 2 +- go.sum | 4 +-- integration_tests/go.mod | 2 +- integration_tests/go.sum | 4 +-- internal/locate/region_cache.go | 41 +++++++++++++++++++++------- internal/locate/region_cache_test.go | 9 ++++++ internal/locate/region_request.go | 11 ++++++++ 7 files changed, 57 insertions(+), 16 deletions(-) diff --git a/go.mod b/go.mod index eedb327620..7fd8fd58da 100644 --- a/go.mod +++ b/go.mod @@ -14,7 +14,7 @@ require ( github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 - github.com/pingcap/kvproto v0.0.0-20230720094213-a3b4a77b4333 + github.com/pingcap/kvproto v0.0.0-20230818065851-7b612d935bf9 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.15.1 diff --git a/go.sum b/go.sum index 2df15720c3..570bd7435b 100644 --- a/go.sum +++ b/go.sum @@ -136,8 +136,8 @@ github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c h1:CgbKAHto5CQgW github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= -github.com/pingcap/kvproto v0.0.0-20230720094213-a3b4a77b4333 h1:A6Wqgq0uMw51UiRAH27TVN0QlzVR5CVtV6fTQSAmvKM= -github.com/pingcap/kvproto v0.0.0-20230720094213-a3b4a77b4333/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/kvproto v0.0.0-20230818065851-7b612d935bf9 h1:VDoZ18CAXoTUNTCxfl4BjQSD5rJQri8QlH8nu0ZuHeg= +github.com/pingcap/kvproto v0.0.0-20230818065851-7b612d935bf9/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8IDP+SZrdhV1Kibl9KrHxJ9eciw= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= diff --git a/integration_tests/go.mod b/integration_tests/go.mod index 97636a2f60..01e508d622 100644 --- a/integration_tests/go.mod +++ b/integration_tests/go.mod @@ -6,7 +6,7 @@ require ( github.com/ninedraft/israce v0.0.3 github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32 github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c - github.com/pingcap/kvproto v0.0.0-20230720094213-a3b4a77b4333 + github.com/pingcap/kvproto v0.0.0-20230818065851-7b612d935bf9 github.com/pingcap/tidb v1.1.0-beta.0.20230619015310-8b1006f1af04 github.com/pkg/errors v0.9.1 github.com/stretchr/testify v1.8.4 diff --git a/integration_tests/go.sum b/integration_tests/go.sum index ac10fa09a7..134ec393cc 100644 --- a/integration_tests/go.sum +++ b/integration_tests/go.sum @@ -363,8 +363,8 @@ github.com/pingcap/fn v1.0.0 h1:CyA6AxcOZkQh52wIqYlAmaVmF6EvrcqFywP463pjA8g= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20230720094213-a3b4a77b4333 h1:A6Wqgq0uMw51UiRAH27TVN0QlzVR5CVtV6fTQSAmvKM= -github.com/pingcap/kvproto v0.0.0-20230720094213-a3b4a77b4333/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/kvproto v0.0.0-20230818065851-7b612d935bf9 h1:VDoZ18CAXoTUNTCxfl4BjQSD5rJQri8QlH8nu0ZuHeg= +github.com/pingcap/kvproto v0.0.0-20230818065851-7b612d935bf9/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/log v1.1.0/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22 h1:2SOzvGvE8beiC1Y4g9Onkvu6UmuBBOeWRGQEjJaT/JY= diff --git a/internal/locate/region_cache.go b/internal/locate/region_cache.go index cf57b521be..56444ac093 100644 --- a/internal/locate/region_cache.go +++ b/internal/locate/region_cache.go @@ -627,16 +627,17 @@ func (c *RegionCache) SetPDClient(client pd.Client) { // RPCContext contains data that is needed to send RPC to a region. type RPCContext struct { - Region RegionVerID - Meta *metapb.Region - Peer *metapb.Peer - AccessIdx AccessIndex - Store *Store - Addr string - AccessMode accessMode - ProxyStore *Store // nil means proxy is not used - ProxyAddr string // valid when ProxyStore is not nil - TiKVNum int // Number of TiKV nodes among the region's peers. Assuming non-TiKV peers are all TiFlash peers. + Region RegionVerID + Meta *metapb.Region + Peer *metapb.Peer + AccessIdx AccessIndex + Store *Store + Addr string + AccessMode accessMode + ProxyStore *Store // nil means proxy is not used + ProxyAddr string // valid when ProxyStore is not nil + TiKVNum int // Number of TiKV nodes among the region's peers. Assuming non-TiKV peers are all TiFlash peers. + BucketVersion uint64 contextPatcher contextPatcher // kvrpcpb.Context fields that need to be overridden } @@ -1947,6 +1948,26 @@ func (c *RegionCache) getStoresByLabels(labels []*metapb.StoreLabel) []*Store { return s } +// OnBucketVersionNotMatch removes the old buckets meta if the version is stale. +func (c *RegionCache) OnBucketVersionNotMatch(ctx *RPCContext, version uint64, keys [][]byte) { + r := c.GetCachedRegionWithRLock(ctx.Region) + if r == nil { + return + } + + buckets := r.getStore().buckets + if buckets == nil || buckets.GetVersion() < version { + oldStore := r.getStore() + store := oldStore.clone() + store.buckets = &metapb.Buckets{ + Version: version, + Keys: keys, + RegionId: r.meta.GetId(), + } + r.compareAndSwapStore(oldStore, store) + } +} + // OnRegionEpochNotMatch removes the old region and inserts new regions into the cache. // It returns whether retries the request because it's possible the region epoch is ahead of TiKV's due to slow appling. func (c *RegionCache) OnRegionEpochNotMatch(bo *retry.Backoffer, ctx *RPCContext, currentRegions []*metapb.Region) (bool, error) { diff --git a/internal/locate/region_cache_test.go b/internal/locate/region_cache_test.go index 619da2d2eb..6226a1c608 100644 --- a/internal/locate/region_cache_test.go +++ b/internal/locate/region_cache_test.go @@ -1646,6 +1646,15 @@ func (s *testRegionCacheSuite) TestShouldNotRetryFlashback() { shouldRetry, err = reqSend.onRegionError(s.bo, ctx, nil, &errorpb.Error{FlashbackNotPrepared: &errorpb.FlashbackNotPrepared{}}) s.Error(err) s.False(shouldRetry) + + shouldRetry, err = reqSend.onRegionError(s.bo, ctx, nil, &errorpb.Error{BucketVersionNotMatch: &errorpb.BucketVersionNotMatch{Keys: [][]byte{[]byte("a")}, Version: 1}}) + s.Nil(err) + s.False(shouldRetry) + ctx.Region.GetID() + key, err := s.cache.LocateKey(s.bo, []byte("a")) + s.Nil(err) + s.Equal(key.Buckets.Keys, [][]byte{[]byte("a")}) + s.Equal(key.Buckets.Version, uint64(1)) } func (s *testRegionCacheSuite) TestBackgroundCacheGC() { diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index 96d259aacf..3bf19cb98b 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -2038,6 +2038,17 @@ func (s *RegionRequestSender) onRegionError( return retry, err } + if bucketVersionNotMatch := regionErr.GetBucketVersionNotMatch(); bucketVersionNotMatch != nil { + logutil.Logger(bo.GetCtx()).Debug( + "tikv reports `BucketVersionNotMatch` retry later", + zap.Stringer("bucketVersionNotMatch", bucketVersionNotMatch), + zap.Stringer("ctx", ctx), + ) + // bucket version is not match, we should split this cop request again. + s.regionCache.OnBucketVersionNotMatch(ctx, bucketVersionNotMatch.Version, bucketVersionNotMatch.Keys) + return false, nil + } + if serverIsBusy := regionErr.GetServerIsBusy(); serverIsBusy != nil { if s.replicaSelector != nil && strings.Contains(serverIsBusy.GetReason(), "deadline is exceeded") { s.replicaSelector.onDeadlineExceeded() From fc88757771f9838d8606553873444b896e016000 Mon Sep 17 00:00:00 2001 From: cfzjywxk Date: Thu, 24 Aug 2023 16:00:22 +0800 Subject: [PATCH 23/49] log: group replica selector logging and split not leader errors (#929) * group replica selector logging and split not leader errors Signed-off-by: cfzjywxk * use atomic to read epoch Signed-off-by: cfzjywxk --------- Signed-off-by: cfzjywxk --- internal/locate/region_request.go | 134 +++++++++++++++--------- internal/locate/region_request3_test.go | 28 +++++ 2 files changed, 112 insertions(+), 50 deletions(-) diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index 3bf19cb98b..c22f782e19 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -112,6 +112,10 @@ type RegionRequestSender struct { RegionRequestRuntimeStats } +func (s *RegionRequestSender) String() string { + return fmt.Sprintf("{replicaSelector: %v}", s.replicaSelector.String()) +} + // RegionRequestRuntimeStats records the runtime stats of send region requests. type RegionRequestRuntimeStats struct { Stats map[tikvrpc.CmdType]*RPCRuntimeStats @@ -249,6 +253,10 @@ type replica struct { deadlineErrUsingConfTimeout bool } +func (r *replica) getEpoch() uint32 { + return atomic.LoadUint32(&r.epoch) +} + func (r *replica) isEpochStale() bool { return r.epoch != atomic.LoadUint32(&r.store.epoch) } @@ -273,6 +281,64 @@ type replicaSelector struct { busyThreshold time.Duration } +func selectorStateToString(state selectorState) string { + replicaSelectorState := "nil" + if state != nil { + switch state.(type) { + case *accessKnownLeader: + replicaSelectorState = "accessKnownLeader" + case *accessFollower: + replicaSelectorState = "accessFollower" + case *accessByKnownProxy: + replicaSelectorState = "accessByKnownProxy" + case *tryFollower: + replicaSelectorState = "tryFollower" + case *tryNewProxy: + replicaSelectorState = "tryNewProxy" + case *invalidLeader: + replicaSelectorState = "invalidLeader" + case *invalidStore: + replicaSelectorState = "invalidStore" + case *stateBase: + replicaSelectorState = "stateBase" + case nil: + replicaSelectorState = "nil" + } + } + return replicaSelectorState +} + +func (s *replicaSelector) String() string { + var replicaStatus []string + cacheRegionIsValid := "unknown" + selectorStateStr := "nil" + if s != nil { + selectorStateStr = selectorStateToString(s.state) + if s.region != nil { + if s.region.isValid() { + cacheRegionIsValid = "true" + } else { + cacheRegionIsValid = "false" + } + } + for _, replica := range s.replicas { + replicaStatus = append(replicaStatus, fmt.Sprintf("peer: %v, store: %v, isEpochStale: %v, "+ + "attempts: %v, replica-epoch: %v, store-epoch: %v, store-state: %v, store-liveness-state: %v", + replica.peer.GetId(), + replica.store.storeID, + replica.isEpochStale(), + replica.attempts, + replica.getEpoch(), + atomic.LoadUint32(&replica.store.epoch), + replica.store.getResolveState(), + replica.store.getLivenessState(), + )) + } + } + + return fmt.Sprintf("replicaSelector{selectorStateStr: %v, cacheRegionIsValid: %v, replicaStatus: %v}", selectorStateStr, cacheRegionIsValid, replicaStatus) +} + // selectorState is the interface of states of the replicaSelector. // Here is the main state transition diagram: // @@ -1401,8 +1467,8 @@ func (s *RegionRequestSender) SendReqCtx( return nil, nil, retryTimes, err } if regionErr != nil { - regionErrLabel := regionErrorToLabel(regionErr) - totalErrors[regionErrLabel]++ + regionErrLogging := regionErrorToLogging(rpcCtx.Peer.GetId(), regionErr) + totalErrors[regionErrLogging]++ retry, err = s.onRegionError(bo, rpcCtx, req, regionErr) if err != nil { msg := fmt.Sprintf("send request on region error failed, err: %v", err.Error()) @@ -1427,50 +1493,6 @@ func (s *RegionRequestSender) SendReqCtx( } func (s *RegionRequestSender) logSendReqError(bo *retry.Backoffer, msg string, regionID RegionVerID, retryTimes int, req *tikvrpc.Request, totalErrors map[string]int) { - var replicaStatus []string - replicaSelectorState := "nil" - cacheRegionIsValid := "unknown" - if s.replicaSelector != nil { - switch s.replicaSelector.state.(type) { - case *accessKnownLeader: - replicaSelectorState = "accessKnownLeader" - case *accessFollower: - replicaSelectorState = "accessFollower" - case *accessByKnownProxy: - replicaSelectorState = "accessByKnownProxy" - case *tryFollower: - replicaSelectorState = "tryFollower" - case *tryNewProxy: - replicaSelectorState = "tryNewProxy" - case *invalidLeader: - replicaSelectorState = "invalidLeader" - case *invalidStore: - replicaSelectorState = "invalidStore" - case *stateBase: - replicaSelectorState = "stateBase" - case nil: - replicaSelectorState = "nil" - } - if s.replicaSelector.region != nil { - if s.replicaSelector.region.isValid() { - cacheRegionIsValid = "true" - } else { - cacheRegionIsValid = "false" - } - } - for _, replica := range s.replicaSelector.replicas { - replicaStatus = append(replicaStatus, fmt.Sprintf("peer: %v, store: %v, isEpochStale: %v, attempts: %v, replica-epoch: %v, store-epoch: %v, store-state: %v, store-liveness-state: %v", - replica.peer.GetId(), - replica.store.storeID, - replica.isEpochStale(), - replica.attempts, - replica.epoch, - atomic.LoadUint32(&replica.store.epoch), - replica.store.getResolveState(), - replica.store.getLivenessState(), - )) - } - } var totalErrorStr bytes.Buffer for err, cnt := range totalErrors { if totalErrorStr.Len() > 0 { @@ -1484,12 +1506,10 @@ func (s *RegionRequestSender) logSendReqError(bo *retry.Backoffer, msg string, r zap.Uint64("req-ts", req.GetStartTS()), zap.String("req-type", req.Type.String()), zap.String("region", regionID.String()), - zap.String("region-is-valid", cacheRegionIsValid), - zap.Int("retry-times", retryTimes), zap.String("replica-read-type", req.ReplicaReadType.String()), - zap.String("replica-selector-state", replicaSelectorState), zap.Bool("stale-read", req.StaleRead), - zap.String("replica-status", strings.Join(replicaStatus, "; ")), + zap.Stringer("request-sender", s), + zap.Int("retry-times", retryTimes), zap.Int("total-backoff-ms", bo.GetTotalSleep()), zap.Int("total-backoff-times", bo.GetTotalBackoffTimes()), zap.String("total-region-errors", totalErrorStr.String())) @@ -1840,6 +1860,20 @@ func (s *RegionRequestSender) NeedReloadRegion(ctx *RPCContext) (need bool) { return } +// regionErrorToLogging constructs the logging content with extra information like returned leader peer id. +func regionErrorToLogging(peerID uint64, e *errorpb.Error) string { + str := regionErrorToLabel(e) + if e.GetNotLeader() != nil { + notLeader := e.GetNotLeader() + if notLeader.GetLeader() != nil { + str = fmt.Sprintf("%v-%v", str, notLeader.GetLeader().GetId()) + } else { + str = fmt.Sprintf("%v-nil", str) + } + } + return fmt.Sprintf("%v-%v", peerID, str) +} + func regionErrorToLabel(e *errorpb.Error) string { if e.GetNotLeader() != nil { return "not_leader" diff --git a/internal/locate/region_request3_test.go b/internal/locate/region_request3_test.go index 28bc43e87d..d1549b9464 100644 --- a/internal/locate/region_request3_test.go +++ b/internal/locate/region_request3_test.go @@ -1425,3 +1425,31 @@ func (s *testRegionRequestToThreeStoresSuite) TestReplicaReadFallbackToLeaderReg // after region error returned, the region should be invalidated. s.False(region.isValid()) } + +func (s *testRegionRequestToThreeStoresSuite) TestLogging() { + req := tikvrpc.NewRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{ + Key: []byte("key"), + }) + region, err := s.cache.LocateRegionByID(s.bo, s.regionID) + s.Nil(err) + s.NotNil(region) + + oc := s.regionRequestSender.client + defer func() { + s.regionRequestSender.client = oc + }() + + s.regionRequestSender.client = &fnClient{fn: func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (response *tikvrpc.Response, err error) { + response = &tikvrpc.Response{Resp: &kvrpcpb.GetResponse{ + RegionError: &errorpb.Error{NotLeader: &errorpb.NotLeader{}}, + }} + return response, nil + }} + + bo := retry.NewBackofferWithVars(context.Background(), 5, nil) + resp, _, err := s.regionRequestSender.SendReq(bo, req, region.Region, time.Second) + s.Nil(err) + s.NotNil(resp) + regionErr, _ := resp.GetRegionError() + s.NotNil(regionErr) +} From 295094e5b534f67446155c5125f9e57fb584d72e Mon Sep 17 00:00:00 2001 From: you06 Date: Tue, 29 Aug 2023 08:28:46 +0800 Subject: [PATCH 24/49] add retry info to request source (#953) * add retry info to request source Signed-off-by: you06 * handle upper layer retry Signed-off-by: you06 * stabilize test Signed-off-by: you06 * retry in 3 dimension Signed-off-by: you06 * record and restore req.ReadType Signed-off-by: you06 --------- Signed-off-by: you06 --- internal/client/client_test.go | 2 +- internal/locate/region_request.go | 64 ++++++++++++++++++-- internal/locate/region_request3_test.go | 77 +++++++++++++++++++++++++ tikvrpc/tikvrpc.go | 4 ++ txnkv/txnsnapshot/scan.go | 9 ++- txnkv/txnsnapshot/snapshot.go | 11 +++- 6 files changed, 158 insertions(+), 9 deletions(-) diff --git a/internal/client/client_test.go b/internal/client/client_test.go index 20cae1f035..e872643f0c 100644 --- a/internal/client/client_test.go +++ b/internal/client/client_test.go @@ -708,7 +708,7 @@ func TestBatchClientRecoverAfterServerRestart(t *testing.T) { cli.unlockForSend() break } - if time.Since(start) > time.Second*5 { + if time.Since(start) > time.Second*10 { // It shouldn't take too long for batch_client to reconnect. require.Fail(t, "wait batch client reconnect timeout") } diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index c22f782e19..af0b748f59 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -1378,7 +1378,6 @@ func (s *RegionRequestSender) SendReqCtx( totalErrors := make(map[string]int) for { if retryTimes > 0 { - req.IsRetryRequest = true if retryTimes%100 == 0 { logutil.Logger(bo.GetCtx()).Warn( "retry", @@ -1431,8 +1430,17 @@ func (s *RegionRequestSender) SendReqCtx( } } + if e := tikvrpc.SetContext(req, rpcCtx.Meta, rpcCtx.Peer); e != nil { + return nil, nil, retryTimes, err + } + rpcCtx.contextPatcher.applyTo(&req.Context) + if req.InputRequestSource != "" && s.replicaSelector != nil { + s.replicaSelector.patchRequestSource(req, rpcCtx) + } + var retry bool resp, retry, err = s.sendReqToRegion(bo, rpcCtx, req, timeout) + req.IsRetryRequest = true if err != nil { msg := fmt.Sprintf("send request failed, err: %v", err.Error()) s.logSendReqError(bo, msg, regionID, retryTimes, req, totalErrors) @@ -1582,10 +1590,6 @@ func fetchRespInfo(resp *tikvrpc.Response) string { func (s *RegionRequestSender) sendReqToRegion( bo *retry.Backoffer, rpcCtx *RPCContext, req *tikvrpc.Request, timeout time.Duration, ) (resp *tikvrpc.Response, retry bool, err error) { - if e := tikvrpc.SetContext(req, rpcCtx.Meta, rpcCtx.Peer); e != nil { - return nil, false, err - } - rpcCtx.contextPatcher.applyTo(&req.Context) // judge the store limit switch. if limit := kv.StoreLimit.Load(); limit > 0 { if err := s.getStoreToken(rpcCtx.Store, limit); err != nil { @@ -2302,3 +2306,53 @@ func (s *staleReadMetricsCollector) onResp(tp tikvrpc.CmdType, resp *tikvrpc.Res metrics.StaleReadRemoteInBytes.Add(float64(size)) } } + +func (s *replicaSelector) replicaType(rpcCtx *RPCContext) string { + leaderIdx := -1 + switch v := s.state.(type) { + case *accessKnownLeader: + return "leader" + case *tryFollower: + return "follower" + case *accessFollower: + leaderIdx = int(v.leaderIdx) + case *tryIdleReplica: + leaderIdx = int(v.leaderIdx) + } + if leaderIdx > -1 && rpcCtx != nil && rpcCtx.Peer != nil { + for idx, replica := range s.replicas { + if replica.peer.Id == rpcCtx.Peer.Id { + if idx == leaderIdx { + return "leader" + } + return "follower" + } + } + } + return "unknown" +} + +func (s *replicaSelector) patchRequestSource(req *tikvrpc.Request, rpcCtx *RPCContext) { + var sb strings.Builder + sb.WriteString(req.InputRequestSource) + sb.WriteByte('-') + defer func() { + req.RequestSource = sb.String() + }() + + replicaType := s.replicaType(rpcCtx) + + if req.IsRetryRequest { + sb.WriteString("retry_") + sb.WriteString(req.ReadType) + sb.WriteByte('_') + sb.WriteString(replicaType) + return + } + if req.StaleRead { + req.ReadType = "stale_" + replicaType + } else { + req.ReadType = replicaType + } + sb.WriteString(req.ReadType) +} diff --git a/internal/locate/region_request3_test.go b/internal/locate/region_request3_test.go index d1549b9464..7a92c58f6f 100644 --- a/internal/locate/region_request3_test.go +++ b/internal/locate/region_request3_test.go @@ -1453,3 +1453,80 @@ func (s *testRegionRequestToThreeStoresSuite) TestLogging() { regionErr, _ := resp.GetRegionError() s.NotNil(regionErr) } + +func (s *testRegionRequestToThreeStoresSuite) TestRetryRequestSource() { + leaderStore, _ := s.loadAndGetLeaderStore() + regionLoc, err := s.cache.LocateRegionByID(s.bo, s.regionID) + s.Nil(err) + req := tikvrpc.NewRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{ + Key: []byte("key"), + }) + req.InputRequestSource = "test" + + setReadType := func(req *tikvrpc.Request, readType string) { + req.StaleRead = false + req.ReplicaRead = false + switch readType { + case "leader": + return + case "follower": + req.ReplicaRead = true + req.ReplicaReadType = kv.ReplicaReadFollower + case "stale_follower", "stale_leader": + req.EnableStaleRead() + default: + panic("unreachable") + } + } + + setTargetReplica := func(selector *replicaSelector, readType string) { + var leader bool + switch readType { + case "leader", "stale_leader": + leader = true + case "follower", "stale_follower": + leader = false + default: + panic("unreachable") + } + for idx, replica := range selector.replicas { + if replica.store.storeID == leaderStore.storeID && leader { + selector.targetIdx = AccessIndex(idx) + return + } + if replica.store.storeID != leaderStore.storeID && !leader { + selector.targetIdx = AccessIndex(idx) + return + } + } + panic("unreachable") + } + + firstReadReplicas := []string{"leader", "follower", "stale_follower", "stale_leader"} + retryReadReplicas := []string{"leader", "follower"} + for _, firstReplica := range firstReadReplicas { + for _, retryReplica := range retryReadReplicas { + bo := retry.NewBackoffer(context.Background(), -1) + req.IsRetryRequest = false + setReadType(req, firstReplica) + replicaSelector, err := newReplicaSelector(s.cache, regionLoc.Region, req) + s.Nil(err) + setTargetReplica(replicaSelector, firstReplica) + rpcCtx, err := replicaSelector.buildRPCContext(bo) + s.Nil(err) + replicaSelector.patchRequestSource(req, rpcCtx) + s.Equal("test-"+firstReplica, req.RequestSource) + + // retry + setReadType(req, retryReplica) + replicaSelector, err = newReplicaSelector(s.cache, regionLoc.Region, req) + s.Nil(err) + setTargetReplica(replicaSelector, retryReplica) + rpcCtx, err = replicaSelector.buildRPCContext(bo) + s.Nil(err) + req.IsRetryRequest = true + replicaSelector.patchRequestSource(req, rpcCtx) + s.Equal("test-retry_"+firstReplica+"_"+retryReplica, req.RequestSource) + } + } +} diff --git a/tikvrpc/tikvrpc.go b/tikvrpc/tikvrpc.go index 4d81e1481a..76f4e6148a 100644 --- a/tikvrpc/tikvrpc.go +++ b/tikvrpc/tikvrpc.go @@ -234,6 +234,10 @@ type Request struct { ForwardedHost string // ReplicaNumber is the number of current replicas, which is used to calculate the RU cost. ReplicaNumber int64 + // The initial read type, note this will be assigned in the first try, no need to set it outside the client. + ReadType string + // InputRequestSource is the input source of the request, if it's not empty, the final RequestSource sent to store will be attached with the retry info. + InputRequestSource string } // NewRequest returns new kv rpc request. diff --git a/txnkv/txnsnapshot/scan.go b/txnkv/txnsnapshot/scan.go index 01662f1dd8..59c8fca84b 100644 --- a/txnkv/txnsnapshot/scan.go +++ b/txnkv/txnsnapshot/scan.go @@ -202,6 +202,8 @@ func (s *Scanner) getData(bo *retry.Backoffer) error { var loc *locate.KeyLocation var resolvingRecordToken *int var err error + // the states in request need to keep when retry request. + var readType string for { if !s.reverse { loc, err = s.snapshot.store.GetRegionCache().LocateKey(bo, s.nextStartKey) @@ -245,12 +247,16 @@ func (s *Scanner) getData(bo *retry.Backoffer) error { TaskId: s.snapshot.mu.taskID, ResourceGroupTag: s.snapshot.mu.resourceGroupTag, IsolationLevel: s.snapshot.isolationLevel.ToPB(), - RequestSource: s.snapshot.GetRequestSource(), ResourceControlContext: &kvrpcpb.ResourceControlContext{ ResourceGroupName: s.snapshot.mu.resourceGroupName, }, BusyThresholdMs: uint32(s.snapshot.mu.busyThreshold.Milliseconds()), }) + if readType != "" { + req.ReadType = readType + req.IsRetryRequest = true + } + req.InputRequestSource = s.snapshot.GetRequestSource() if s.snapshot.mu.resourceGroupTag == nil && s.snapshot.mu.resourceGroupTagger != nil { s.snapshot.mu.resourceGroupTagger(req) } @@ -263,6 +269,7 @@ func (s *Scanner) getData(bo *retry.Backoffer) error { if err != nil { return err } + readType = req.ReadType if regionErr != nil { logutil.BgLogger().Debug("scanner getData failed", zap.Stringer("regionErr", regionErr)) diff --git a/txnkv/txnsnapshot/snapshot.go b/txnkv/txnsnapshot/snapshot.go index ab29655355..624dc69022 100644 --- a/txnkv/txnsnapshot/snapshot.go +++ b/txnkv/txnsnapshot/snapshot.go @@ -389,6 +389,8 @@ func (s *KVSnapshot) batchGetSingleRegion(bo *retry.Backoffer, batch batchKeys, pending := batch.keys var resolvingRecordToken *int useConfigurableKVTimeout := true + // the states in request need to keep when retry request. + var readType string for { s.mu.RLock() req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdBatchGet, &kvrpcpb.BatchGetRequest{ @@ -400,12 +402,16 @@ func (s *KVSnapshot) batchGetSingleRegion(bo *retry.Backoffer, batch batchKeys, TaskId: s.mu.taskID, ResourceGroupTag: s.mu.resourceGroupTag, IsolationLevel: s.isolationLevel.ToPB(), - RequestSource: s.GetRequestSource(), ResourceControlContext: &kvrpcpb.ResourceControlContext{ ResourceGroupName: s.mu.resourceGroupName, }, BusyThresholdMs: uint32(busyThresholdMs), }) + req.InputRequestSource = s.GetRequestSource() + if readType != "" { + req.ReadType = readType + req.IsRetryRequest = true + } if s.mu.resourceGroupTag == nil && s.mu.resourceGroupTagger != nil { s.mu.resourceGroupTagger(req) } @@ -443,6 +449,7 @@ func (s *KVSnapshot) batchGetSingleRegion(bo *retry.Backoffer, batch batchKeys, if err != nil { return err } + readType = req.ReadType if regionErr != nil { // For other region error and the fake region error, backoff because // there's something wrong. @@ -626,12 +633,12 @@ func (s *KVSnapshot) get(ctx context.Context, bo *retry.Backoffer, k []byte) ([] TaskId: s.mu.taskID, ResourceGroupTag: s.mu.resourceGroupTag, IsolationLevel: s.isolationLevel.ToPB(), - RequestSource: s.GetRequestSource(), ResourceControlContext: &kvrpcpb.ResourceControlContext{ ResourceGroupName: s.mu.resourceGroupName, }, BusyThresholdMs: uint32(s.mu.busyThreshold.Milliseconds()), }) + req.InputRequestSource = s.GetRequestSource() if s.mu.resourceGroupTag == nil && s.mu.resourceGroupTagger != nil { s.mu.resourceGroupTagger(req) } From a8860a98018e5532d6f3635c67188e236a6e717d Mon Sep 17 00:00:00 2001 From: 3pointer Date: Tue, 29 Aug 2023 13:43:23 +0800 Subject: [PATCH 25/49] gc: add resolve locks interface for tidb gc_worker (#945) * gc: add GCResolver inteface for resolve locks Signed-off-by: 3pointer * adapt scanlimit Signed-off-by: 3pointer * rename GCLockResolver to RegionLockResolver Signed-off-by: 3pointer * update Signed-off-by: 3pointer * address comments Signed-off-by: 3pointer --------- Signed-off-by: 3pointer --- tikv/gc.go | 114 +++++++++++++++++++++++++++++++++++---------- tikv/test_probe.go | 2 +- 2 files changed, 91 insertions(+), 25 deletions(-) diff --git a/tikv/gc.go b/tikv/gc.go index 2b47e6bca0..0ce3bbc5fe 100644 --- a/tikv/gc.go +++ b/tikv/gc.go @@ -35,6 +35,9 @@ import ( zap "go.uber.org/zap" ) +// We don't want gc to sweep out the cached info belong to other processes, like coprocessor. +const GCScanLockLimit = txnlock.ResolvedCacheSize / 2 + // GC does garbage collection (GC) of the TiKV cluster. // GC deletes MVCC records whose timestamp is lower than the given `safepoint`. We must guarantee // @@ -81,8 +84,9 @@ func WithConcurrency(concurrency int) GCOpt { } func (s *KVStore) resolveLocks(ctx context.Context, safePoint uint64, concurrency int) error { + lockResolver := NewRegionLockResolver("gc-client-go-api", s) handler := func(ctx context.Context, r kv.KeyRange) (rangetask.TaskStat, error) { - return s.resolveLocksForRange(ctx, safePoint, r.StartKey, r.EndKey) + return ResolveLocksForRange(ctx, lockResolver, safePoint, r.StartKey, r.EndKey, NewGcResolveLockMaxBackoffer, GCScanLockLimit) } runner := rangetask.NewRangeTaskRunner("resolve-locks-runner", s, concurrency, handler) @@ -94,72 +98,131 @@ func (s *KVStore) resolveLocks(ctx context.Context, safePoint uint64, concurrenc return nil } -// We don't want gc to sweep out the cached info belong to other processes, like coprocessor. -const gcScanLockLimit = txnlock.ResolvedCacheSize / 2 +type BaseRegionLockResolver struct { + identifier string + store Storage +} + +func NewRegionLockResolver(identifier string, store Storage) *BaseRegionLockResolver { + return &BaseRegionLockResolver{ + identifier: identifier, + store: store, + } +} + +func (l *BaseRegionLockResolver) Identifier() string { + return l.identifier +} -func (s *KVStore) resolveLocksForRange(ctx context.Context, safePoint uint64, startKey []byte, endKey []byte) (rangetask.TaskStat, error) { +func (l *BaseRegionLockResolver) ResolveLocksInOneRegion(bo *Backoffer, locks []*txnlock.Lock, loc *locate.KeyLocation) (*locate.KeyLocation, error) { + return batchResolveLocksInOneRegion(bo, l.GetStore(), locks, loc) +} + +func (l *BaseRegionLockResolver) ScanLocksInOneRegion(bo *Backoffer, key []byte, maxVersion uint64, scanLimit uint32) ([]*txnlock.Lock, *locate.KeyLocation, error) { + return scanLocksInOneRegionWithStartKey(bo, l.GetStore(), key, maxVersion, scanLimit) +} + +func (l *BaseRegionLockResolver) GetStore() Storage { + return l.store +} + +// RegionLockResolver is used for GCWorker and log backup advancer to resolve locks in a region. +type RegionLockResolver interface { + // Identifier represents the name of this resolver. + Identifier() string + + // ResolveLocksInOneRegion tries to resolve expired locks for one region. + // 1. For GCWorker it will scan locks before *safepoint*, + // and force remove these locks. rollback the txn, no matter the lock is expired of not. + // 2. For log backup advancer, it will scan all locks for a small range. + // and it will check status of the txn. resolve the locks if txn is expired, Or do nothing. + // + // regionLocation should return if resolve locks succeed. if regionLocation return nil, + // which means not all locks are resolved in someway. the caller should retry scan locks. + // ** the locks are assumed sorted by key in ascending order ** + ResolveLocksInOneRegion(bo *Backoffer, locks []*txnlock.Lock, regionLocation *locate.KeyLocation) (*locate.KeyLocation, error) + + // ScanLocksInOneRegion return locks and location with given start key in a region. + // The return result ([]*Lock, *KeyLocation, error) represents the all locks in a regionLocation. + // which will used by ResolveLocksInOneRegion later. + ScanLocksInOneRegion(bo *Backoffer, key []byte, maxVersion uint64, scanLimit uint32) ([]*txnlock.Lock, *locate.KeyLocation, error) + + // GetStore is used to get store to GetRegionCache and SendReq for this lock resolver. + GetStore() Storage +} + +func ResolveLocksForRange( + ctx context.Context, + resolver RegionLockResolver, + maxVersion uint64, + startKey []byte, + endKey []byte, + createBackoffFn func(context.Context) *Backoffer, + scanLimit uint32, +) (rangetask.TaskStat, error) { // for scan lock request, we must return all locks even if they are generated // by the same transaction. because gc worker need to make sure all locks have been // cleaned. - var stat rangetask.TaskStat key := startKey - bo := NewGcResolveLockMaxBackoffer(ctx) + // create new backoffer for every scan and resolve locks + bo := createBackoffFn(ctx) for { select { case <-ctx.Done(): return stat, errors.New("[gc worker] gc job canceled") default: } - - locks, loc, err := s.scanLocksInRegionWithStartKey(bo, key, safePoint, gcScanLockLimit) + locks, loc, err := resolver.ScanLocksInOneRegion(bo, key, maxVersion, scanLimit) if err != nil { return stat, err } - resolvedLocation, err1 := s.batchResolveLocksInARegion(bo, locks, loc) - if err1 != nil { - return stat, err1 + resolvedLocation, err := resolver.ResolveLocksInOneRegion(bo, locks, loc) + if err != nil { + return stat, err } // resolve locks failed since the locks are not in one region anymore, need retry. if resolvedLocation == nil { continue } - if len(locks) < gcScanLockLimit { + if len(locks) < int(scanLimit) { stat.CompletedRegions++ key = loc.EndKey - logutil.Logger(ctx).Info("[gc worker] one region finshed ", + logutil.Logger(ctx).Debug("resolve one region finshed ", + zap.String("identifier", resolver.Identifier()), zap.Int("regionID", int(resolvedLocation.Region.GetID())), zap.Int("resolvedLocksNum", len(locks))) } else { - logutil.Logger(ctx).Info("[gc worker] region has more than limit locks", + logutil.Logger(ctx).Info("region has more than limit locks", + zap.String("identifier", resolver.Identifier()), zap.Int("regionID", int(resolvedLocation.Region.GetID())), zap.Int("resolvedLocksNum", len(locks)), - zap.Int("scan lock limit", gcScanLockLimit)) + zap.Uint32("scan lock limit", scanLimit)) key = locks[len(locks)-1].Key } if len(key) == 0 || (len(endKey) != 0 && bytes.Compare(key, endKey) >= 0) { break } - bo = NewGcResolveLockMaxBackoffer(ctx) + bo = createBackoffFn(ctx) } return stat, nil } -func (s *KVStore) scanLocksInRegionWithStartKey(bo *retry.Backoffer, startKey []byte, maxVersion uint64, limit uint32) (locks []*txnlock.Lock, loc *locate.KeyLocation, err error) { +func scanLocksInOneRegionWithStartKey(bo *retry.Backoffer, store Storage, startKey []byte, maxVersion uint64, limit uint32) (locks []*txnlock.Lock, loc *locate.KeyLocation, err error) { for { - loc, err := s.GetRegionCache().LocateKey(bo, startKey) + loc, err := store.GetRegionCache().LocateKey(bo, startKey) if err != nil { return nil, loc, err } req := tikvrpc.NewRequest(tikvrpc.CmdScanLock, &kvrpcpb.ScanLockRequest{ MaxVersion: maxVersion, - Limit: gcScanLockLimit, + Limit: limit, StartKey: startKey, EndKey: loc.EndKey, }) - resp, err := s.SendReq(bo, req, loc.Region, ReadTimeoutMedium) + resp, err := store.SendReq(bo, req, loc.Region, ReadTimeoutMedium) if err != nil { return nil, loc, err } @@ -190,15 +253,18 @@ func (s *KVStore) scanLocksInRegionWithStartKey(bo *retry.Backoffer, startKey [] } } -// batchResolveLocksInARegion resolves locks in a region. +// batchResolveLocksInOneRegion resolves locks in a region. // It returns the real location of the resolved locks if resolve locks success. // It returns error when meet an unretryable error. // When the locks are not in one region, resolve locks should be failed, it returns with nil resolveLocation and nil err. // Used it in gcworker only! -func (s *KVStore) batchResolveLocksInARegion(bo *Backoffer, locks []*txnlock.Lock, expectedLoc *locate.KeyLocation) (resolvedLocation *locate.KeyLocation, err error) { +func batchResolveLocksInOneRegion(bo *Backoffer, store Storage, locks []*txnlock.Lock, expectedLoc *locate.KeyLocation) (resolvedLocation *locate.KeyLocation, err error) { + if expectedLoc == nil { + return nil, nil + } resolvedLocation = expectedLoc for { - ok, err := s.GetLockResolver().BatchResolveLocks(bo, locks, resolvedLocation.Region) + ok, err := store.GetLockResolver().BatchResolveLocks(bo, locks, resolvedLocation.Region) if ok { return resolvedLocation, nil } @@ -209,7 +275,7 @@ func (s *KVStore) batchResolveLocksInARegion(bo *Backoffer, locks []*txnlock.Loc if err != nil { return nil, err } - region, err1 := s.GetRegionCache().LocateKey(bo, locks[0].Key) + region, err1 := store.GetRegionCache().LocateKey(bo, locks[0].Key) if err1 != nil { return nil, err1 } diff --git a/tikv/test_probe.go b/tikv/test_probe.go index 5971480f3c..234d2585fb 100644 --- a/tikv/test_probe.go +++ b/tikv/test_probe.go @@ -123,7 +123,7 @@ func (s StoreProbe) ScanLocks(ctx context.Context, startKey, endKey []byte, maxV outerLoop: for { - locks, loc, err := s.KVStore.scanLocksInRegionWithStartKey(bo, startKey, maxVersion, limit) + locks, loc, err := scanLocksInOneRegionWithStartKey(bo, s.KVStore, startKey, maxVersion, limit) if err != nil { return nil, err } From 322775facc898fae37a47cf1351d78219c82b2f2 Mon Sep 17 00:00:00 2001 From: husharp Date: Wed, 30 Aug 2023 13:33:12 +0800 Subject: [PATCH 26/49] merge master Signed-off-by: husharp --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index eedb327620..7fd8fd58da 100644 --- a/go.mod +++ b/go.mod @@ -14,7 +14,7 @@ require ( github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 - github.com/pingcap/kvproto v0.0.0-20230720094213-a3b4a77b4333 + github.com/pingcap/kvproto v0.0.0-20230818065851-7b612d935bf9 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.15.1 diff --git a/go.sum b/go.sum index 2df15720c3..570bd7435b 100644 --- a/go.sum +++ b/go.sum @@ -136,8 +136,8 @@ github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c h1:CgbKAHto5CQgW github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= -github.com/pingcap/kvproto v0.0.0-20230720094213-a3b4a77b4333 h1:A6Wqgq0uMw51UiRAH27TVN0QlzVR5CVtV6fTQSAmvKM= -github.com/pingcap/kvproto v0.0.0-20230720094213-a3b4a77b4333/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/kvproto v0.0.0-20230818065851-7b612d935bf9 h1:VDoZ18CAXoTUNTCxfl4BjQSD5rJQri8QlH8nu0ZuHeg= +github.com/pingcap/kvproto v0.0.0-20230818065851-7b612d935bf9/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8IDP+SZrdhV1Kibl9KrHxJ9eciw= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= From 5c9fcf3e8541d675615df5a47c6c4e0771af1981 Mon Sep 17 00:00:00 2001 From: buffer <1045931706@qq.com> Date: Wed, 30 Aug 2023 18:09:50 +0800 Subject: [PATCH 27/49] metrics: add bucket version not match metrics (#957) --- internal/locate/region_request.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index af0b748f59..63242877d4 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -1922,6 +1922,8 @@ func regionErrorToLabel(e *errorpb.Error) string { return "deadline_exceeded" } else if e.GetMismatchPeerId() != nil { return "mismatch_peer_id" + } else if e.GetBucketVersionNotMatch() != nil { + return "bucket_version_not_match" } return "unknown" } @@ -2079,7 +2081,8 @@ func (s *RegionRequestSender) onRegionError( if bucketVersionNotMatch := regionErr.GetBucketVersionNotMatch(); bucketVersionNotMatch != nil { logutil.Logger(bo.GetCtx()).Debug( "tikv reports `BucketVersionNotMatch` retry later", - zap.Stringer("bucketVersionNotMatch", bucketVersionNotMatch), + zap.Uint64("latest bucket version", bucketVersionNotMatch.GetVersion()), + zap.Uint64("request bucket version", ctx.BucketVersion), zap.Stringer("ctx", ctx), ) // bucket version is not match, we should split this cop request again. From 3c7fc08748d1d09963817da1a7f7852e0e3981c7 Mon Sep 17 00:00:00 2001 From: husharp Date: Thu, 31 Aug 2023 14:27:57 +0800 Subject: [PATCH 28/49] rebase gomod Signed-off-by: husharp --- integration_tests/go.mod | 34 ++++---- integration_tests/go.sum | 170 +++++++-------------------------------- 2 files changed, 46 insertions(+), 158 deletions(-) diff --git a/integration_tests/go.mod b/integration_tests/go.mod index e76d0dc74a..01e508d622 100644 --- a/integration_tests/go.mod +++ b/integration_tests/go.mod @@ -1,26 +1,24 @@ module integration_tests -go 1.21 - -toolchain go1.21.0 +go 1.20 require ( github.com/ninedraft/israce v0.0.3 github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32 github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c - github.com/pingcap/kvproto v0.0.0-20230825101459-934e842bfd6e - github.com/pingcap/tidb v1.1.0-beta.0.20230830034708-c59bf901e052 + github.com/pingcap/kvproto v0.0.0-20230818065851-7b612d935bf9 + github.com/pingcap/tidb v1.1.0-beta.0.20230619015310-8b1006f1af04 github.com/pkg/errors v0.9.1 github.com/stretchr/testify v1.8.4 github.com/tidwall/gjson v1.14.1 - github.com/tikv/client-go/v2 v2.0.8-0.20230811033710-8a214402da13 - github.com/tikv/pd/client v0.0.0-20230728033905-31343e006842 + github.com/tikv/client-go/v2 v2.0.8-0.20230714052714-85fc8f337565 + github.com/tikv/pd/client v0.0.0-20230724080549-de985b8e0afc go.uber.org/goleak v1.2.1 ) require ( github.com/Azure/go-ntlmssp v0.0.0-20221128193559-754e69321358 // indirect - github.com/BurntSushi/toml v1.3.2 // indirect + github.com/BurntSushi/toml v1.3.0 // indirect github.com/benbjohnson/clock v1.3.5 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect @@ -67,16 +65,16 @@ require ( github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 // indirect github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22 // indirect github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 // indirect - github.com/pingcap/tidb/parser v0.0.0-20230830034708-c59bf901e052 // indirect - github.com/pingcap/tipb v0.0.0-20230822064221-711da6fede03 // indirect + github.com/pingcap/tidb/parser v0.0.0-20230619015310-8b1006f1af04 // indirect + github.com/pingcap/tipb v0.0.0-20230607071926-bda24015c2d6 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b // indirect github.com/prometheus/client_golang v1.16.0 // indirect github.com/prometheus/client_model v0.4.0 // indirect github.com/prometheus/common v0.44.0 // indirect - github.com/prometheus/procfs v0.11.1 // indirect + github.com/prometheus/procfs v0.11.0 // indirect github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec // indirect - github.com/rogpeppe/go-internal v1.11.0 // indirect + github.com/rogpeppe/go-internal v1.10.0 // indirect github.com/sasha-s/go-deadlock v0.2.0 // indirect github.com/shirou/gopsutil/v3 v3.23.5 // indirect github.com/shoenig/go-m1cpu v0.1.6 // indirect @@ -95,13 +93,13 @@ require ( go.etcd.io/etcd/client/v3 v3.5.2 // indirect go.uber.org/atomic v1.11.0 // indirect go.uber.org/multierr v1.11.0 // indirect - go.uber.org/zap v1.25.0 // indirect - golang.org/x/crypto v0.12.0 // indirect + go.uber.org/zap v1.24.0 // indirect + golang.org/x/crypto v0.11.0 // indirect golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect - golang.org/x/net v0.14.0 // indirect + golang.org/x/net v0.11.0 // indirect golang.org/x/sync v0.3.0 // indirect - golang.org/x/sys v0.11.0 // indirect - golang.org/x/text v0.12.0 // indirect + golang.org/x/sys v0.10.0 // indirect + golang.org/x/text v0.11.0 // indirect golang.org/x/time v0.3.0 // indirect golang.org/x/tools v0.10.0 // indirect google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 // indirect @@ -114,5 +112,7 @@ require ( replace ( github.com/go-ldap/ldap/v3 => github.com/YangKeao/ldap/v3 v3.4.5-0.20230421065457-369a3bab1117 + github.com/pingcap/tidb => github.com/HuSharp/tidb v1.1.0-beta.0.20230726045237-a2b0085ad7c5 + github.com/pingcap/tidb/parser => github.com/HuSharp/tidb/parser v0.0.0-20230726045237-a2b0085ad7c5 github.com/tikv/client-go/v2 => ../ ) diff --git a/integration_tests/go.sum b/integration_tests/go.sum index 4dbca4d00c..134ec393cc 100644 --- a/integration_tests/go.sum +++ b/integration_tests/go.sum @@ -1,48 +1,37 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.110.0 h1:Zc8gqp3+a9/Eyph2KDmcGaPtbKRIoqq4YTlL4NMD0Ys= -cloud.google.com/go v0.110.0/go.mod h1:SJnCLqQ0FCFGSZMUNUf84MV3Aia54kn7pi8st7tMzaY= cloud.google.com/go/compute v1.19.0 h1:+9zda3WGgW1ZSTlVppLCYFIr48Pa35q1uG2N1itbCEQ= -cloud.google.com/go/compute v1.19.0/go.mod h1:rikpw2y+UMidAe9tISo04EHNOIf42RLYF/q8Bs93scU= cloud.google.com/go/compute/metadata v0.2.3 h1:mg4jlk7mCAj6xXp9UJ4fjI9VUI5rubuGBW5aJ7UnBMY= -cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= cloud.google.com/go/iam v0.13.0 h1:+CmB+K0J/33d0zSQ9SlFWUeCCEn5XJA0ZMZ3pHE9u8k= -cloud.google.com/go/iam v0.13.0/go.mod h1:ljOg+rcNfzZ5d6f1nAUJ8ZIxOaZUVoS14bKCtaLZ/D0= cloud.google.com/go/storage v1.30.1 h1:uOdMxAs8HExqBlnLtnQyP0YkvbiDpdGShGKtx6U/oNM= -cloud.google.com/go/storage v1.30.1/go.mod h1:NfxhC0UJE1aXSx7CIIbCf7y9HKT7BiccwkR7+P7gN8E= github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= github.com/Azure/azure-sdk-for-go/sdk/azcore v1.6.0 h1:8kDqDngH+DmVBiCtIjCFTGa7MBnsIOkF9IccInFEbjk= -github.com/Azure/azure-sdk-for-go/sdk/azcore v1.6.0/go.mod h1:bjGvMhVMb+EEm3VRNQawDMUyMMjo+S5ewNjflkep/0Q= github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.1.0 h1:QkAcEIAKbNL4KoFr4SathZPhDhF4mVwpBMFlYjyAqy8= -github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.1.0/go.mod h1:bhXu1AjYL+wutSL/kpSq6s7733q2Rb0yuot9Zgfqa/0= github.com/Azure/azure-sdk-for-go/sdk/internal v1.3.0 h1:sXr+ck84g/ZlZUOZiNELInmMgOsuGwdjjVkEIde0OtY= -github.com/Azure/azure-sdk-for-go/sdk/internal v1.3.0/go.mod h1:okt5dMMTOFjX/aovMlrjvvXoPMBVSPzk9185BT0+eZM= github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.0.0 h1:u/LLAOFgsMv7HmNL4Qufg58y+qElGOt5qv0z1mURkRY= -github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.0.0/go.mod h1:2e8rMJtl2+2j+HXbTBwnyGpm5Nou7KhvSfxOq8JpTag= github.com/Azure/go-ntlmssp v0.0.0-20221128193559-754e69321358 h1:mFRzDkZVAjdal+s7s0MwaRv9igoPqLRdzOLzw/8Xvq8= github.com/Azure/go-ntlmssp v0.0.0-20221128193559-754e69321358/go.mod h1:chxPXzSsl7ZWRAuOIE23GDNzjWuZquvFlgA8xmpunjU= github.com/AzureAD/microsoft-authentication-library-for-go v0.5.1 h1:BWe8a+f/t+7KY7zH2mqygeUD0t8hNFXe08p1Pb3/jKE= -github.com/AzureAD/microsoft-authentication-library-for-go v0.5.1/go.mod h1:Vt9sXTKwMyGcOxSmLDMnGPgqsUg7m8pe215qMLrDXw4= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/BurntSushi/toml v1.3.2 h1:o7IhLm0Msx3BaB+n3Ag7L8EVlByGnpq14C4YWiu/gL8= -github.com/BurntSushi/toml v1.3.2/go.mod h1:CxXYINrC8qIiEnFrOxCa7Jy5BFHlXnUU2pbicEuybxQ= +github.com/BurntSushi/toml v1.3.0 h1:Ws8e5YmnrGEHzZEzg0YvK/7COGYtTC5PbaH9oSSbgfA= +github.com/BurntSushi/toml v1.3.0/go.mod h1:CxXYINrC8qIiEnFrOxCa7Jy5BFHlXnUU2pbicEuybxQ= github.com/CloudyKit/fastprinter v0.0.0-20170127035650-74b38d55f37a/go.mod h1:EFZQ978U7x8IRnstaskI3IysnWY5Ao3QgZUKOXlsAdw= github.com/CloudyKit/jet v2.1.3-0.20180809161101-62edd43e4f88+incompatible/go.mod h1:HPYO+50pSWkPoj9Q/eq0aRGByCL6ScRlUmiEX5Zgm+w= github.com/DataDog/zstd v1.4.5 h1:EndNeuB0l9syBZhut0wns3gV1hL8zX8LIu6ZiVHWLIQ= -github.com/DataDog/zstd v1.4.5/go.mod h1:1jcaCB/ufaK+sKp1NBhlGmpz41jOoPQ35bpF36t7BBo= github.com/HdrHistogram/hdrhistogram-go v1.1.2 h1:5IcZpTvzydCQeHzK4Ef/D5rrSqwxob0t8PQPMybUNFM= -github.com/HdrHistogram/hdrhistogram-go v1.1.2/go.mod h1:yDgFjdqOqDEKOvasDdhWNXYg9BVp4O+o5f6V/ehm6Oo= +github.com/HuSharp/tidb v1.1.0-beta.0.20230726045237-a2b0085ad7c5 h1:wSOvDYbKkvHjlWWFBihIoeJ5yBc1jZe9Ehkku3Jn8cA= +github.com/HuSharp/tidb v1.1.0-beta.0.20230726045237-a2b0085ad7c5/go.mod h1:C3tuWINS2/Vt/gxZ0OLdGI2x5crlN8E3/qNJJkIIkTI= +github.com/HuSharp/tidb/parser v0.0.0-20230726045237-a2b0085ad7c5 h1:bxwmPI7ambmbOAaozdYz81HFpIeu6ctWo7TiXfOGE14= +github.com/HuSharp/tidb/parser v0.0.0-20230726045237-a2b0085ad7c5/go.mod h1:ENXEsaVS6N3CTMpL4txc6m93y6XaztF9W4SFLjhPWJg= github.com/Joker/hpp v1.0.0/go.mod h1:8x5n+M1Hp5hC0g8okX3sR3vFQwynaX/UgSOM9MeBKzY= github.com/Joker/jade v1.0.1-0.20190614124447-d475f43051e7/go.mod h1:6E6s8o2AE4KhCrqr6GRJjdC/gNfTdxkIXvuGZZda2VM= github.com/Masterminds/semver v1.5.0 h1:H65muMkzWKEuNDnfl9d70GUjFniHKHRbFPGBuZ3QEww= -github.com/Masterminds/semver v1.5.0/go.mod h1:MB6lktGJrhw8PrUyiEoblNEGEQ+RzHPF078ddwwvV3Y= github.com/Shopify/goreferrer v0.0.0-20181106222321-ec9c9a553398/go.mod h1:a1uqRtAwp2Xwc6WNPJEufxJ7fx3npB4UV/JOLmbu5I0= github.com/VividCortex/ewma v1.2.0 h1:f58SaIzcDXrSy3kWaHNvuJgJ3Nmz59Zji6XoJR/q1ow= -github.com/VividCortex/ewma v1.2.0/go.mod h1:nz4BbCtbLyFDeC9SUHbtcT5644juEuWfUAUnGx7j5l4= github.com/YangKeao/ldap/v3 v3.4.5-0.20230421065457-369a3bab1117 h1:+OqGGFc2YHFd82aSHmjlILVt1t4JWJjrNIfV8cVEPow= github.com/YangKeao/ldap/v3 v3.4.5-0.20230421065457-369a3bab1117/go.mod h1:bMGIq3AGbytbaMwf8wdv5Phdxz0FWHTIYMSzyrYgnQs= github.com/acarl005/stripansi v0.0.0-20180116102854-5a71ef0e047d h1:licZJFw2RwpHMqeKTCYkitsPqHNxTmd4SNR5r94FGM8= -github.com/acarl005/stripansi v0.0.0-20180116102854-5a71ef0e047d/go.mod h1:asat636LX7Bqt5lYEZ27JNDcqxfjdBQuJ/MM4CN/Lzo= github.com/ajg/form v1.5.1/go.mod h1:uL1WgH+h2mgNtvBq0339dVnzXdBETtL2LeUXaIv25UY= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= @@ -52,13 +41,10 @@ github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk5 github.com/alexbrainman/sspi v0.0.0-20210105120005-909beea2cc74 h1:Kk6a4nehpJ3UuJRqlA3JxYxBZEqCeOmATOvrbT4p9RA= github.com/alexbrainman/sspi v0.0.0-20210105120005-909beea2cc74/go.mod h1:cEWa1LVoE5KvSD9ONXsZrj0z6KqySlCCNKHlLzbqAt4= github.com/aliyun/alibaba-cloud-sdk-go v1.61.1581 h1:Q/yk4z/cHUVZfgTqtD09qeYBxHwshQAjVRX73qs8UH0= -github.com/aliyun/alibaba-cloud-sdk-go v1.61.1581/go.mod h1:RcDobYh8k5VP6TNybz9m++gL3ijVI5wueVr0EM10VsU= github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/apache/thrift v0.13.1-0.20201008052519-daf620915714 h1:Jz3KVLYY5+JO7rDiX0sAuRGtuv2vG01r17Y9nLMWNUw= -github.com/apache/thrift v0.13.1-0.20201008052519-daf620915714/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= github.com/aws/aws-sdk-go v1.44.259 h1:7yDn1dcv4DZFMKpu+2exIH5O6ipNj9qXrKfdMUaIJwY= -github.com/aws/aws-sdk-go v1.44.259/go.mod h1:aVsgQcEevwlmQ7qHE9I3h+dtQgpqhFB+i8Phjh7fkwI= github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/benbjohnson/clock v1.3.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= @@ -69,16 +55,13 @@ github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+Ce github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/blacktear23/go-proxyprotocol v1.0.6 h1:eTt6UMpEnq59NjON49b3Cay8Dm0sCs1nDliwgkyEsRM= -github.com/blacktear23/go-proxyprotocol v1.0.6/go.mod h1:FSCbgnRZrQXazBLL5snfBbrcFSMtcmUDhSRb9OfFA1o= github.com/carlmjohnson/flagext v0.21.0 h1:/c4uK3ie786Z7caXLcIMvePNSSiH3bQVGDvmGLMme60= -github.com/carlmjohnson/flagext v0.21.0/go.mod h1:Eenv0epIUAr4NuedNmkzI8WmBmjIxZC239XcKxYS2ac= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.1.2/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cheggaaa/pb/v3 v3.0.8 h1:bC8oemdChbke2FHIIGy9mn4DPJ2caZYQnfbRqwmdCoA= -github.com/cheggaaa/pb/v3 v3.0.8/go.mod h1:UICbiLec/XO6Hw6k+BHEtHeQFzzBH4i2/qk/ow1EJTA= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cloudfoundry/gosigar v1.3.6 h1:gIc08FbB3QPb+nAQhINIK/qhf5REKkY0FTGgRGXkcVc= github.com/cloudfoundry/gosigar v1.3.6/go.mod h1:lNWstu5g5gw59O09Y+wsMNFzBSnU8a0u+Sfx4dq360E= @@ -90,8 +73,7 @@ github.com/cockroachdb/errors v1.8.1 h1:A5+txlVZfOqFBDa4mGz2bUWSp0aHElvHX2bKkdbQ github.com/cockroachdb/errors v1.8.1/go.mod h1:qGwQn6JmZ+oMjuLwjWzUNqblqk0xl4CVV3SQbGwK7Ac= github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f h1:o/kfcElHqOiXqcou5a3rIlMc7oJbMQkeLk0VQJ7zgqY= github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f/go.mod h1:i/u985jwjWRlyHXQbwatDASoW0RMlZ/3i9yJHE2xLkI= -github.com/cockroachdb/pebble v0.0.0-20220415182917-06c9d3be25b3 h1:snjwkhKc/ZtYIC/hg6UoT5PrhXcZmCsaB+z0bonMDcU= -github.com/cockroachdb/pebble v0.0.0-20220415182917-06c9d3be25b3/go.mod h1:buxOO9GBtOcq1DiXDpIPYrmxY020K2A8lOrwno5FetU= +github.com/cockroachdb/pebble v0.0.0-20210719141320-8c3bd06debb5 h1:Igd6YmtOZ77EgLAIaE9+mHl7+sAKaZ5m4iMI0Dz/J2A= github.com/cockroachdb/redact v1.0.8 h1:8QG/764wK+vmEYoOlfobpe12EQcS81ukx/a4hdVMxNw= github.com/cockroachdb/redact v1.0.8/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2 h1:IKgmqgMQlVJIZj19CdocBeSfSaiCbEBZGKODaixqtHM= @@ -120,22 +102,18 @@ github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSs github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/decred/dcrd/dcrec/secp256k1/v4 v4.2.0 h1:8UrgZ3GkP4i/CLijOJx79Yu+etlyjdBU4sfcs2WYQMs= -github.com/decred/dcrd/dcrec/secp256k1/v4 v4.2.0/go.mod h1:v57UDF4pDQJcEfFUCRop3lJL149eHGSe9Jvczhzjo/0= github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= github.com/dgraph-io/ristretto v0.1.1 h1:6CWw5tJNgpegArSHpNHJKldNeq03FQCwYvfMVWajOK8= -github.com/dgraph-io/ristretto v0.1.1/go.mod h1:S1GPSBCYCIhmVNfcth17y2zZtQT6wzkzgwUve0VDWWA= github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= github.com/dgryski/go-farm v0.0.0-20190104051053-3adb47b1fb0f/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13 h1:fAjc9m62+UWV/WAFKLNi6ZS0675eEUC9y3AlwSbQu1Y= github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= github.com/docker/go-units v0.4.0 h1:3uh0PgVws3nIA0Q+MwDC8yjEPf9zjRfZZWXZYDct3Tw= -github.com/docker/go-units v0.4.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= github.com/dustin/go-humanize v1.0.0 h1:VSnTsYCnlFHaM2/igO1h6X3HA71jcobQuxemgkq4zYo= github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385/go.mod h1:0vRUJqYpeSZifjYj7uP3BG/gKcuzL9xWVV/Y+cK33KM= github.com/emirpasic/gods v1.18.1 h1:FXtiHYKDGKCW2KzwZKx0iC0PQmdlorYgdFG9jPXJ1Bc= -github.com/emirpasic/gods v1.18.1/go.mod h1:8tpGGwCnJ5H4r6BWwaV6OrWmMoPhUl5jm/FMNAnJvWQ= github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= @@ -144,15 +122,12 @@ github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7 github.com/etcd-io/bbolt v1.3.3/go.mod h1:ZF2nL25h33cCyBtcyWeZ2/I3HQOfTP+0PIEvHjkjCrw= github.com/fasthttp-contrib/websocket v0.0.0-20160511215533-1f3b11f56072/go.mod h1:duJ4Jxv5lDcvg4QuQr0oowTf7dz4/CR8NtyCooz9HL8= github.com/fatih/color v1.15.0 h1:kOqh6YHBtK8aywxGerMG2Eq3H6Qgoqeo13Bk2Mv/nBs= -github.com/fatih/color v1.15.0/go.mod h1:0h5ZqXfHYED7Bhv2ZJamyIOUej9KtShiJESRwBDUSsw= github.com/fatih/structs v1.1.0/go.mod h1:9NiDSp5zOcgEDl+j00MP/WkGVPOlPRLejGD8Ga6PJ7M= github.com/flosch/pongo2 v0.0.0-20190707114632-bbf5a6c351f4/go.mod h1:T9YF2M40nIgbVgp3rreNmTged+9HrbNTIQf1PsaIiTA= github.com/form3tech-oss/jwt-go v3.2.5+incompatible h1:/l4kBbb4/vGSsdtB5nUe8L7B9mImVMaBPw9L/0TBHU8= -github.com/form3tech-oss/jwt-go v3.2.5+incompatible/go.mod h1:pbq4aXjuKjdthFRnoDwaVPLA+WlJuPGy+QneDUgJi2k= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= github.com/fsnotify/fsnotify v1.6.0 h1:n+5WquG0fcWoWp6xPWfHdbskMCQaFnG6PfBrh1Ky4HY= -github.com/fsnotify/fsnotify v1.6.0/go.mod h1:sl3t1tCWJFWoRz9R8WJCbQihKKwmorjAbSClcnxKAGw= github.com/gavv/httpexpect v2.0.0+incompatible/go.mod h1:x+9tiU1YnrOvnB725RkpoLv1M62hOWzwo5OXotisrKc= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/gin-contrib/sse v0.0.0-20190301062529-5545eab6dad3/go.mod h1:VJ0WA2NBN22VlZ2dKZQPAPnyWw5XTlK1KymzLKsr59s= @@ -169,18 +144,15 @@ github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9 github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-logr/logr v1.2.3 h1:2DntVwHkVopvECVRSlL5PSo9eG+cAkDCuckLubN+rq0= -github.com/go-logr/logr v1.2.3/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-martini/martini v0.0.0-20170121215854-22fa46961aab/go.mod h1:/P9AEU963A2AYjv4d1V5eVL1CQbEJq6aCNHDDjibzu8= github.com/go-ole/go-ole v1.2.6 h1:/Fpf6oFPoeFik9ty7siob0G6Ke8QvQEuVcuChpwXzpY= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-sql-driver/mysql v1.7.1 h1:lUIinVbN1DY0xBg0eMOzmmtGoHwWBbvnWubQUrtU8EI= -github.com/go-sql-driver/mysql v1.7.1/go.mod h1:OXbVy3sEdcQ2Doequ6Z5BW6fXNQTmx+9S1MCJN5yJMI= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee/go.mod h1:L0fX3K22YWvt/FAX9NnzrNzcI4wNYi9Yku4O0LKYflo= github.com/gobwas/pool v0.2.0/go.mod h1:q8bcK0KcYlCgd9e7WYLm9LpyS+YeLd8JVDW6WezmKEw= github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/EM= github.com/goccy/go-json v0.10.2 h1:CrxCmQqYDkv1z7lO7Wbh2HN93uovUHgrECaO5ZrCXAU= -github.com/goccy/go-json v0.10.2/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/gogo/googleapis v0.0.0-20180223154316-0cd9801be74a/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= github.com/gogo/protobuf v0.0.0-20171007142547-342cbe0a0415/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= @@ -193,12 +165,10 @@ github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/gogo/status v1.1.0/go.mod h1:BFv9nrluPLmrS0EmGVvLaPNmRosr9KapBYd5/hpY1WM= github.com/golang-jwt/jwt v3.2.1+incompatible h1:73Z+4BJcrTC+KczS6WvTPvRGOp1WmfEP4Q1lOd9Z/+c= -github.com/golang-jwt/jwt v3.2.1+incompatible/go.mod h1:8pz2t5EyA70fFQQSrl6XZXzqecmYZeUEB8OUGHkxJ+I= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/glog v1.1.1 h1:jxpi2eWoU84wbX9iIEyAeeoac3FLuifZpY9tcNUD9kw= github.com/golang/glog v1.1.1/go.mod h1:zR+okUeTbrL6EL3xHUDxZuEtGv04p5shwip1+mL/rLQ= github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da h1:oI5xCqsCo564l8iNU+DwB5epxmsaqB+rhGL0m5jtYqE= -github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/protobuf v0.0.0-20180814211427-aa810b61a9c7/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -237,23 +207,18 @@ github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeN github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/gofuzz v1.1.0 h1:Hsa8mG0dQ46ij8Sl2AYJDUv1oA9/d6Vk+3LG99Oe02g= -github.com/google/gofuzz v1.1.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/pprof v0.0.0-20211122183932-1daafda22083 h1:c8EUapQFi+kjzedr4c6WqbwMdmB95+oDBWZ5XFHFYxY= -github.com/google/pprof v0.0.0-20211122183932-1daafda22083/go.mod h1:KgnwoLYCZ8IQu3XUZ8Nc/bM9CCZFOyjUNOSygVozoDg= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I= github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/enterprise-certificate-proxy v0.2.3 h1:yk9/cqRKtT9wXZSsRH9aurXEpJX+U6FLtpYTdC3R06k= -github.com/googleapis/enterprise-certificate-proxy v0.2.3/go.mod h1:AwSRAtLfXpU5Nm3pW+v7rGDHp09LsPtGY9MduiEsR9k= github.com/googleapis/gax-go/v2 v2.7.1 h1:gF4c0zjUP2H/s/hEGyLA3I0fA2ZWjzYiONAD6cvPr8A= -github.com/googleapis/gax-go/v2 v2.7.1/go.mod h1:4orTrqY6hXxxaUL4LHIPl6lGo8vAE38/qKbhSAKP6QI= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gorilla/mux v1.8.0 h1:i40aqfkR1h2SlN9hojwV5ZA91wcXFOvkdNIeFDP5koI= github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= github.com/gorilla/websocket v1.4.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= github.com/gorilla/websocket v1.4.2 h1:+/TMaTYc4QFitKJxsQ7Yye35DkWvkdLcvGKqM+x0Ufc= -github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 h1:+9834+KizmvFV7pXQGSXQTsaWhq2GjuNUt0aUU0YBYw= github.com/grpc-ecosystem/go-grpc-middleware v1.3.0/go.mod h1:z0ButlSOZa5vEBq9m2m2hlwIgKw+rp3sdCBRoJY+30Y= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho= @@ -265,11 +230,9 @@ github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/hydrogen18/memlistener v0.0.0-20141126152155-54553eb933fb/go.mod h1:qEIFzExnS6016fRpRfxrExeVn2gbClQA99gQhnIcdhE= github.com/iancoleman/strcase v0.2.0 h1:05I4QRnGpI0m37iZQRuskXh+w77mr6Z41lwQzuHLwW0= -github.com/iancoleman/strcase v0.2.0/go.mod h1:iwCmte+B7n89clKwxIoIXy/HfoL7AsD47ZCWhYzw7ho= github.com/imkira/go-interpol v1.1.0/go.mod h1:z0h2/2T3XF8kyEPpRgJ3kmNv+C43p+I/CoI+jC3w2iA= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= github.com/inconshreveable/mousetrap v1.1.0 h1:wN+x4NVGpMsO7ErUn/mUI3vEoE6Jt13X2s0bqwp9tc8= -github.com/inconshreveable/mousetrap v1.1.0/go.mod h1:vpF70FUmC8bwa3OWnCshd2FqLfsEA9PFc4w1p2J65bw= github.com/influxdata/tdigest v0.0.1 h1:XpFptwYmnEKUqmkcDjrzffswZ3nvNeevbUSLPP/ZzIY= github.com/influxdata/tdigest v0.0.1/go.mod h1:Z0kXnxzbTC2qrx4NaIzYkE1k66+6oEDQTvL95hQFh5Y= github.com/iris-contrib/blackfriday v2.0.0+incompatible/go.mod h1:UzZ2bDEoaSGPbkg6SAB4att1aAwTmVIx/5gCVqeyUdI= @@ -277,21 +240,16 @@ github.com/iris-contrib/go.uuid v2.0.0+incompatible/go.mod h1:iz2lgM/1UnEf1kP0L/ github.com/iris-contrib/i18n v0.0.0-20171121225848-987a633949d0/go.mod h1:pMCz62A0xJL6I+umB2YTlFRwWXaDFA0jy+5HzGiJjqI= github.com/iris-contrib/schema v0.0.1/go.mod h1:urYA3uvUNG1TIIjOSCzHr9/LmbQo8LrOcOqfqxa4hXw= github.com/jedib0t/go-pretty/v6 v6.2.2 h1:o3McN0rQ4X+IU+HduppSp9TwRdGLRW2rhJXy9CJaCRw= -github.com/jedib0t/go-pretty/v6 v6.2.2/go.mod h1:+nE9fyyHGil+PuISTCrp7avEdo6bqoMwqZnuiK2r2a0= github.com/jellydator/ttlcache/v3 v3.0.1 h1:cHgCSMS7TdQcoprXnWUptJZzyFsqs18Lt8VVhRuZYVU= github.com/jellydator/ttlcache/v3 v3.0.1/go.mod h1:WwTaEmcXQ3MTjOm4bsZoDFiCu/hMvNWLO1w67RXz6h4= github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9YPoQUg= -github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= github.com/joho/sqltocsv v0.0.0-20210428211105-a6d6801d59df h1:Zrb0IbuLOGHL7nrO2WrcuNWgDTlzFv3zY69QMx4ggQE= -github.com/joho/sqltocsv v0.0.0-20210428211105-a6d6801d59df/go.mod h1:mAVCUAYtW9NG31eB30umMSLKcDt6mCUWSjoSn5qBh0k= github.com/jonboulle/clockwork v0.2.2 h1:UOGuzwb1PwsrDAObMuhUnj0p5ULPj8V/xJ7Kx9qUBdQ= -github.com/jonboulle/clockwork v0.2.2/go.mod h1:Pkfl5aHPm1nk2H9h0bjmnJD/BcgbGXUBGnn1kMkgxc8= github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.11/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= -github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/juju/errors v0.0.0-20181118221551-089d3ea4e4d5/go.mod h1:W54LbzXuIE0boCoNJfwqpmkKJ1O4TCTZMetAt6jGk7Q= github.com/juju/loggo v0.0.0-20180524022052-584905176618/go.mod h1:vgyd7OREkbtVEN/8IXZe5Ooef3LQePvuBm9UWj6ZL8U= @@ -326,35 +284,25 @@ github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/kylelemons/godebug v1.1.0 h1:RPNrshWIDI6G2gRW9EHilWtl7Z6Sb1BR0xunSBf0SNc= -github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+fNqagV/RAw= github.com/labstack/echo/v4 v4.1.11/go.mod h1:i541M3Fj6f76NZtHSj7TXnyM8n2gaodfvfxNnFqi74g= github.com/labstack/gommon v0.3.0/go.mod h1:MULnywXg0yavhxWKc+lOruYdAhDwPK9wf0OL7NoOu+k= github.com/lestrrat-go/blackmagic v1.0.1 h1:lS5Zts+5HIC/8og6cGHb0uCcNCa3OUt1ygh3Qz2Fe80= -github.com/lestrrat-go/blackmagic v1.0.1/go.mod h1:UrEqBzIR2U6CnzVyUtfM6oZNMt/7O7Vohk2J0OGSAtU= github.com/lestrrat-go/httpcc v1.0.1 h1:ydWCStUeJLkpYyjLDHihupbn2tYmZ7m22BGkcvZZrIE= -github.com/lestrrat-go/httpcc v1.0.1/go.mod h1:qiltp3Mt56+55GPVCbTdM9MlqhvzyuL6W/NMDA8vA5E= github.com/lestrrat-go/httprc v1.0.4 h1:bAZymwoZQb+Oq8MEbyipag7iSq6YIga8Wj6GOiJGdI8= -github.com/lestrrat-go/httprc v1.0.4/go.mod h1:mwwz3JMTPBjHUkkDv/IGJ39aALInZLrhBp0X7KGUZlo= github.com/lestrrat-go/iter v1.0.2 h1:gMXo1q4c2pHmC3dn8LzRhJfP1ceCbgSiT9lUydIzltI= -github.com/lestrrat-go/iter v1.0.2/go.mod h1:Momfcq3AnRlRjI5b5O8/G5/BvpzrhoFTZcn06fEOPt4= github.com/lestrrat-go/jwx/v2 v2.0.11 h1:ViHMnaMeaO0qV16RZWBHM7GTrAnX2aFLVKofc7FuKLQ= -github.com/lestrrat-go/jwx/v2 v2.0.11/go.mod h1:ZtPtMFlrfDrH2Y0iwfa3dRFn8VzwBrB+cyrm3IBWdDg= github.com/lestrrat-go/option v1.0.1 h1:oAzP2fvZGQKWkvHa1/SAcFolBEca1oN+mQ7eooNBEYU= -github.com/lestrrat-go/option v1.0.1/go.mod h1:5ZHFbivi4xwXxhxY9XHDe2FHo6/Z7WWmtT7T5nBBp3I= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a h1:N9zuLhTvBSRt0gWSiJswwQ2HqDmtX/ZCDJURnKUt1Ik= github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a/go.mod h1:JKx41uQRwqlTZabZc+kILPrO/3jlKnQ2Z8b7YiVw5cE= github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/mattn/go-colorable v0.1.2/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= github.com/mattn/go-colorable v0.1.13 h1:fFA4WZxdEF4tXPZVKMLwD8oUnCTTo08duU7wxecdEvA= -github.com/mattn/go-colorable v0.1.13/go.mod h1:7S9/ev0klgBDR4GtXTXX8a3vIGJpMovkB8vQcUbaXHg= github.com/mattn/go-isatty v0.0.7/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= github.com/mattn/go-isatty v0.0.8/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= github.com/mattn/go-isatty v0.0.9/go.mod h1:YNRxwqDuOph6SZLI9vUUz6OYw3QyUt7WiY2yME+cCiQ= github.com/mattn/go-isatty v0.0.18 h1:DOKFKCQ7FNG2L1rbrmstDN4QVRdS89Nkh85u68Uwp98= -github.com/mattn/go-isatty v0.0.18/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y= github.com/mattn/go-runewidth v0.0.14 h1:+xnbZSEeDbOIg5/mE6JF0w6n9duR1l3/WmbinWVwUuU= -github.com/mattn/go-runewidth v0.0.14/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= github.com/mattn/goveralls v0.0.2/go.mod h1:8d1ZMHsd7fW6IRPKQh46F2WRpyib5/X4FOpevwGNQEw= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zkfA9PSy5pEvNWRP0ET0TIVo= @@ -370,7 +318,6 @@ github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= -github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= @@ -388,16 +335,13 @@ github.com/ninedraft/israce v0.0.3 h1:F/Y1u6OlvgE75Syv1WbBatyg3CjGCdxLojLE7ydv2y github.com/ninedraft/israce v0.0.3/go.mod h1:4L1ITFl340650ZmexVbUcBwG18ozlWiMe47pltZAmn4= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= -github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= github.com/onsi/ginkgo v1.13.0/go.mod h1:+REjRxOmWfHCjfv9TTWB1jD1Frx4XydAD3zm1lskyM0= github.com/onsi/ginkgo v1.16.5 h1:8xi0RTUf59SOSfEtZMvwTvXYMzG4gV23XVHOZiXNtnE= -github.com/onsi/ginkgo v1.16.5/go.mod h1:+E8gABHa3K6zRBolWtd+ROzc/U5bkGt0FwiG042wbpU= github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= github.com/onsi/gomega v1.20.1 h1:PA/3qinGoukvymdIDV8pii6tiZgC8kbmJO6Z5+b002Q= -github.com/onsi/gomega v1.20.1/go.mod h1:DtrZpjmvpn2mPm4YWQa0/ALMDj9v4YxLgojwPeREyVo= github.com/opentracing/basictracer-go v1.1.0 h1:Oa1fTSBvAl8pa3U+IJYqrKm0NALwH9OsgwOqDv4xJW0= github.com/opentracing/basictracer-go v1.1.0/go.mod h1:V2HZueSJEp879yv285Aap1BS69fQMD+MNP1mRs6mBQc= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= @@ -416,26 +360,20 @@ github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32/go.mod h1:X2r9ue github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c h1:CgbKAHto5CQgWM9fSBIvaxsJHuGP0uM74HXtv3MyyGQ= github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= github.com/pingcap/fn v1.0.0 h1:CyA6AxcOZkQh52wIqYlAmaVmF6EvrcqFywP463pjA8g= -github.com/pingcap/fn v1.0.0/go.mod h1:u9WZ1ZiOD1RpNhcI42RucFh/lBuzTu6rw88a+oF2Z24= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20230825101459-934e842bfd6e h1:D47Kp8A9pvrWspyFh3KQRiKIsIik13xwzPwRXuu5ocM= -github.com/pingcap/kvproto v0.0.0-20230825101459-934e842bfd6e/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/kvproto v0.0.0-20230818065851-7b612d935bf9 h1:VDoZ18CAXoTUNTCxfl4BjQSD5rJQri8QlH8nu0ZuHeg= +github.com/pingcap/kvproto v0.0.0-20230818065851-7b612d935bf9/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/log v1.1.0/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22 h1:2SOzvGvE8beiC1Y4g9Onkvu6UmuBBOeWRGQEjJaT/JY= github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 h1:QV6jqlfOkh8hqvEAgwBZa+4bSgO0EeKC7s5c6Luam2I= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21/go.mod h1:QYnjfA95ZaMefyl1NO8oPtKeb8pYUdnDVhQgf+qdpjM= -github.com/pingcap/tidb v1.1.0-beta.0.20230830034708-c59bf901e052 h1:5zI2zZ8d0M7B/iBgp6B36ntqE1XyHP/EjXAxlplGFVs= -github.com/pingcap/tidb v1.1.0-beta.0.20230830034708-c59bf901e052/go.mod h1:vFCTutRATbvM2eDMMYxHTGEV7c2cq2MpQjvD9OwpqRI= -github.com/pingcap/tidb/parser v0.0.0-20230830034708-c59bf901e052 h1:JxsR3wJIy3Yo2aDa+N10XGJ+b3TVP7ppuyc7sblxbrA= -github.com/pingcap/tidb/parser v0.0.0-20230830034708-c59bf901e052/go.mod h1:pWA6mNa/o7UTDKrg+4H75NdpRgpWRTox/cqQjaQ4ZBU= -github.com/pingcap/tipb v0.0.0-20230822064221-711da6fede03 h1:puhZJ86ZXvSCZLTshL6xBEhvF963MAoL+/Jydh2tvO8= -github.com/pingcap/tipb v0.0.0-20230822064221-711da6fede03/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= +github.com/pingcap/tipb v0.0.0-20230607071926-bda24015c2d6 h1:D79RE4RVhq2ic8sqDSv7QdL0tT5aZV3CaCXUAT41iWc= +github.com/pingcap/tipb v0.0.0-20230607071926-bda24015c2d6/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/browser v0.0.0-20210115035449-ce105d075bb4 h1:Qj1ukM4GlMWXNdMBuXcXfz/Kw9s1qm0CLY32QxuSImI= -github.com/pkg/browser v0.0.0-20210115035449-ce105d075bb4/go.mod h1:N6UoU20jOqggOuDwUaBQpluzLNDqif3kq9z2wpdYEfQ= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= @@ -469,26 +407,22 @@ github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= -github.com/prometheus/procfs v0.11.1 h1:xRC8Iq1yyca5ypa9n1EZnWZkt7dwcoRPQwX/5gwaUuI= -github.com/prometheus/procfs v0.11.1/go.mod h1:eesXgaPo1q7lBpVMoMy0ZOFTth9hBn4W/y0/p/ScXhY= +github.com/prometheus/procfs v0.11.0 h1:5EAgkfkMl659uZPbe9AS2N68a7Cc1TJbPEuGzFuRbyk= +github.com/prometheus/procfs v0.11.0/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM= github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec h1:W09IVJc94icq4NjY3clb7Lk8O1qJ8BdBEF8z0ibU0rE= github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= github.com/rivo/uniseg v0.4.4 h1:8TfxU8dW6PdqD27gjM8MVNuicgxIjxpm4K7x4jp8sis= -github.com/rivo/uniseg v0.4.4/go.mod h1:FN3SvrM+Zdj16jyLfmOkMNblXMcoc8DfTHruCPUcx88= -github.com/robfig/cron/v3 v3.0.1 h1:WdRxkvbJztn8LMz/QEvLN5sBU+xKpSqwwUO1Pjr4qDs= -github.com/robfig/cron/v3 v3.0.1/go.mod h1:eQICP3HwyT7UooqI/z+Ov+PtYAWygg1TEWWzGIFLtro= github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.9.0/go.mod h1:WtVeX8xhTBvf0smdhujwtBcq4Qrzq/fJaraNFVN+nFs= -github.com/rogpeppe/go-internal v1.11.0 h1:cWPaGQEPrBb5/AsnsZesgZZ9yb1OQ+GOISoDNXVBh4M= -github.com/rogpeppe/go-internal v1.11.0/go.mod h1:ddIwULY96R17DhadqLgMfk9H9tvdUzkipdSkR5nkCZA= +github.com/rogpeppe/go-internal v1.10.0 h1:TMyTOH3F/DB16zRVcYyreMH6GnZZrwQVAoYjRBZyWFQ= +github.com/rogpeppe/go-internal v1.10.0/go.mod h1:UQnix2H7Ngw/k4C5ijL5+65zddjncjaFoBhdsK/akog= github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= github.com/ryanuber/columnize v2.1.0+incompatible/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= github.com/sasha-s/go-deadlock v0.2.0 h1:lMqc+fUb7RrFS3gQLtoQsJ7/6TV/pAIFvBsqX73DK8Y= github.com/sasha-s/go-deadlock v0.2.0/go.mod h1:StQn567HiB1fF2yJ44N9au7wOhrPS3iZqiDbRupzT10= github.com/sclevine/agouti v3.0.0+incompatible/go.mod h1:b4WX9W9L1sfQKXeJf1mUTLZKJ48R1S7H23Ji7oFO5Bw= github.com/segmentio/asm v1.2.0 h1:9BQrFxC+YOHJlTlHGkTrFWf59nbL3XnCoFLTwDCI7ys= -github.com/segmentio/asm v1.2.0/go.mod h1:BqMnlJP91P8d+4ibuonYZw9mfnzI9HfxselHZr5aAcs= github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= github.com/shirou/gopsutil/v3 v3.21.12/go.mod h1:BToYZVTlSVlfazpDDYFnsVZLaoRG+g8ufT6fPQLdJzA= github.com/shirou/gopsutil/v3 v3.23.5 h1:5SgDCeQ0KW0S4N0znjeM/eFHXXOKyv2dVNgRq/c9P6Y= @@ -498,31 +432,25 @@ github.com/shoenig/go-m1cpu v0.1.6/go.mod h1:1JJMcUBvfNwpq05QDQVAnx3gUHr9IYF7GNg github.com/shoenig/test v0.6.4 h1:kVTaSd7WLz5WZ2IaoM0RSzRsUD+m8wRR+5qvntpn4LU= github.com/shoenig/test v0.6.4/go.mod h1:byHiCGXqrVaflBLAMq/srcZIHynQPQgeyvkvXnjqq0k= github.com/shurcooL/httpfs v0.0.0-20190707220628-8d4bc4ba7749 h1:bUGsEnyNbVPw06Bs80sCeARAlK8lhwqGyi6UT8ymuGk= -github.com/shurcooL/httpfs v0.0.0-20190707220628-8d4bc4ba7749/go.mod h1:ZY1cvUeJuFPAdZ/B6v7RHavJWZn2YPVFQ1OSXhCGOkg= github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd h1:ug7PpSOB5RBPK1Kg6qskGBoP3Vnj/aNYFTznWvlkGo0= -github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= -github.com/sirupsen/logrus v1.9.3 h1:dueUQJ1C2q9oE3F7wvmSGAaVtTmUizReu6fjN8uqzbQ= -github.com/sirupsen/logrus v1.9.3/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ= +github.com/sirupsen/logrus v1.9.2 h1:oxx1eChJGI6Uks2ZC4W1zpLlVgqB8ner4EuQwV4Ik1Y= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= github.com/soheilhy/cmux v0.1.5 h1:jjzc5WVemNEDTLwv9tlmemhC73tI08BNOIGwBOo10Js= -github.com/soheilhy/cmux v0.1.5/go.mod h1:T7TcVDs9LWfQgPlPsdngu6I6QIoyIFZDDC6sNE1GqG0= github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU= github.com/spf13/cobra v1.7.0 h1:hyqWnYt1ZQShIddO5kBpj3vu05/++x6tJ6dg8EC572I= -github.com/spf13/cobra v1.7.0/go.mod h1:uLxZILRyS/50WlhOIKD7W6V5bgeIt+4sICxh6uRMrb0= github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s= github.com/spkg/bom v1.0.0 h1:S939THe0ukL5WcTGiGqkgtaW5JW+O6ITaIlpJXTYY64= -github.com/spkg/bom v1.0.0/go.mod h1:lAz2VbTuYNcvs7iaFF8WW0ufXrHShJ7ck1fYFFbVXJs= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= @@ -539,7 +467,6 @@ github.com/stretchr/testify v1.8.3/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXl github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= -github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a h1:J/YdBZ46WKpXsxsW93SG+q0F8KI+yFrcIDT4c/RNoc4= github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a/go.mod h1:h4xBhSNtOeEosLJ4P7JyKXX7Cabg7AVkWCK5gV2vOrM= github.com/tidwall/gjson v1.14.1 h1:iymTbGkQBhveq21bEvAQ81I0LEBork8BFe1CUZXdyuo= @@ -548,8 +475,8 @@ github.com/tidwall/match v1.1.1 h1:+Ho715JplO36QYgwN9PGYNhgZvoUSc9X2c80KVTi+GA= github.com/tidwall/match v1.1.1/go.mod h1:eRSPERbgtNPcGhD8UCthc6PmLEQXEWd3PRB5JTxsfmM= github.com/tidwall/pretty v1.2.0 h1:RWIZEg2iJ8/g6fDDYzMpobmaoGh5OLl4AXtGUGPcqCs= github.com/tidwall/pretty v1.2.0/go.mod h1:ITEVvHYasfjBbM0u2Pg8T2nJnzm8xPwvNhhsoaGGjNU= -github.com/tikv/pd/client v0.0.0-20230728033905-31343e006842 h1:TwjBJvRx/DJbgMt7Vk5cFO7tG1DZnxR+22S2VmaNGRw= -github.com/tikv/pd/client v0.0.0-20230728033905-31343e006842/go.mod h1:VJwM+qMcQxvGgyu9C6wU7fhjLaALs+odsOvpUMmnhHo= +github.com/tikv/pd/client v0.0.0-20230724080549-de985b8e0afc h1:IUg0j2nWoGYj3FQ3vA3vg97fPSpJEZQrDpgF8RkMLEU= +github.com/tikv/pd/client v0.0.0-20230724080549-de985b8e0afc/go.mod h1:wfHRc4iYaqJiOQZCHcrF+r4hYnkGDaYWDfcicee//pc= github.com/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs= github.com/tklauser/go-sysconf v0.3.11 h1:89WgdJhk5SNwJfu+GKyYveZ4IaJ7xAkecBo+KdJV0CM= github.com/tklauser/go-sysconf v0.3.11/go.mod h1:GqXfhXY3kiPa0nAXPDIQIWzJbMCB7AmcWpGR8lSZfqI= @@ -557,7 +484,6 @@ github.com/tklauser/numcpus v0.3.0/go.mod h1:yFGUr7TUHQRAhyqBcEg0Ge34zDBAsIvJJcy github.com/tklauser/numcpus v0.6.0 h1:kebhY2Qt+3U6RNK7UqpYNA+tJ23IBEGKkB7JQBfDYms= github.com/tklauser/numcpus v0.6.0/go.mod h1:FEZLMke0lhOUG6w2JadTzp0a+Nl8PF/GFkQ5UVIcaL4= github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802 h1:uruHq4dN7GR16kFc5fp3d1RIYzJW5onx8Ybykw2YQFA= -github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/twmb/murmur3 v1.1.6 h1:mqrRot1BRxm+Yct+vavLMou2/iJt0tNVTTC0QoIjaZg= github.com/twmb/murmur3 v1.1.6/go.mod h1:Qq/R7NUyOfr65zD+6Q5IHKsJLwP7exErjN6lyyq3OSQ= github.com/uber/jaeger-client-go v2.30.0+incompatible h1:D6wyKGCecFaSRUpo8lCVbaOOb6ThwMmTEbhRwtKR97o= @@ -572,16 +498,12 @@ github.com/valyala/fasthttp v1.6.0/go.mod h1:FstJa9V+Pj9vQ7OJie2qMHdwemEDaDiSdBn github.com/valyala/fasttemplate v1.0.1/go.mod h1:UQGH1tvbgY+Nz5t2n7tXsz52dQxojPUpymEIMZ47gx8= github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a/go.mod h1:v3UYOV9WzVtRmSR+PDvWpU/qWl4Wa5LApYYX4ZtKbio= github.com/vbauerster/mpb/v7 v7.5.3 h1:BkGfmb6nMrrBQDFECR/Q7RkKCw7ylMetCb4079CGs4w= -github.com/vbauerster/mpb/v7 v7.5.3/go.mod h1:i+h4QY6lmLvBNK2ah1fSreiw3ajskRlBp9AhY/PnuOE= github.com/wangjohn/quickselect v0.0.0-20161129230411-ed8402a42d5f h1:9DDCDwOyEy/gId+IEMrFHLuQ5R/WV0KNxWLler8X2OY= -github.com/wangjohn/quickselect v0.0.0-20161129230411-ed8402a42d5f/go.mod h1:8sdOQnirw1PrcnTJYkmW1iOHtUmblMmGdUOHyWYycLI= github.com/xeipuuv/gojsonpointer v0.0.0-20180127040702-4e3ac2762d5f/go.mod h1:N2zxlSyiKSe5eX1tZViRH5QA0qijqEDrYZiPEAiq3wU= github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415/go.mod h1:GwrjFmJcFw6At/Gs6z4yjiIwzuJ1/+UwLxMQDVQXShQ= github.com/xeipuuv/gojsonschema v1.2.0/go.mod h1:anYRn/JVcOK2ZgGU+IjEV4nwlhoK5sQluxsYJ78Id3Y= github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 h1:eY9dn8+vbi4tKz5Qo6v2eYzo7kUS51QINcR5jNpbZS8= -github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= github.com/xitongsys/parquet-go v1.5.5-0.20201110004701-b09c49d6d457 h1:tBbuFCtyJNKT+BFAv6qjvTFpVdy97IYNaBwGUXifIUs= -github.com/xitongsys/parquet-go v1.5.5-0.20201110004701-b09c49d6d457/go.mod h1:pheqtXeHQFzxJk45lRQ0UIGIivKnLXvialZSFWs81A8= github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q= github.com/yalp/jsonpath v0.0.0-20180802001716-5cc68e5049a0/go.mod h1:/LWChgwKmvncFJFHJ7Gvn9wZArjbV5/FppcK2fKk/tI= github.com/yudai/gojsondiff v1.0.0/go.mod h1:AY32+k2cwILAkW1fbgxQ5mUmMiZFgLIV+FBNExI05xg= @@ -595,45 +517,28 @@ github.com/yusufpapurcu/wmi v1.2.2/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQ github.com/yusufpapurcu/wmi v1.2.3 h1:E1ctvB7uKFMOJw3fdOW32DwGE9I7t++CRUEMKvFoFiw= github.com/yusufpapurcu/wmi v1.2.3/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.etcd.io/bbolt v1.3.6 h1:/ecaJf0sk1l4l6V4awd65v2C3ILy7MSj+s/x1ADCIMU= -go.etcd.io/bbolt v1.3.6/go.mod h1:qXsaaIqmgQH0T+OPdb99Bf+PKfBBQVAdyD6TY9G8XM4= go.etcd.io/etcd/api/v3 v3.5.2 h1:tXok5yLlKyuQ/SXSjtqHc4uzNaMqZi2XsoSPr/LlJXI= go.etcd.io/etcd/api/v3 v3.5.2/go.mod h1:5GB2vv4A4AOn3yk7MftYGHkUfGtDHnEraIjym4dYz5A= go.etcd.io/etcd/client/pkg/v3 v3.5.2 h1:4hzqQ6hIb3blLyQ8usCU4h3NghkqcsohEQ3o3VetYxE= go.etcd.io/etcd/client/pkg/v3 v3.5.2/go.mod h1:IJHfcCEKxYu1Os13ZdwCwIUTUVGYTSAM3YSwc9/Ac1g= go.etcd.io/etcd/client/v2 v2.305.2 h1:ymrVwTkefuqA/rPkSW7/B4ApijbPVefRumkY+stNfS0= -go.etcd.io/etcd/client/v2 v2.305.2/go.mod h1:2D7ZejHVMIfog1221iLSYlQRzrtECw3kz4I4VAQm3qI= go.etcd.io/etcd/client/v3 v3.5.2 h1:WdnejrUtQC4nCxK0/dLTMqKOB+U5TP/2Ya0BJL+1otA= go.etcd.io/etcd/client/v3 v3.5.2/go.mod h1:kOOaWFFgHygyT0WlSmL8TJiXmMysO/nNUlEsSsN6W4o= go.etcd.io/etcd/pkg/v3 v3.5.2 h1:YZUojdoPhOyl5QILYnR8LTUbbNefu/sV4ma+ZMr2tto= -go.etcd.io/etcd/pkg/v3 v3.5.2/go.mod h1:zsXz+9D/kijzRiG/UnFGDTyHKcVp0orwiO8iMLAi+k0= go.etcd.io/etcd/raft/v3 v3.5.2 h1:uCC37qOXqBvKqTGHGyhASsaCsnTuJugl1GvneJNwHWo= -go.etcd.io/etcd/raft/v3 v3.5.2/go.mod h1:G6pCP1sFgbjod7/KnEHY0vHUViqxjkdt6AiKsD0GRr8= go.etcd.io/etcd/server/v3 v3.5.2 h1:B6ytJvS4Fmt8nkjzS2/8POf4tuPhFMluE0lWd4dx/7U= -go.etcd.io/etcd/server/v3 v3.5.2/go.mod h1:mlG8znIEz4N/28GABrohZCBM11FqgGVQcpbcyJgh0j0= go.etcd.io/etcd/tests/v3 v3.5.2 h1:uk7/uMGVebpBDl+roivowHt6gJ5Fnqwik3syDkoSKdo= -go.etcd.io/etcd/tests/v3 v3.5.2/go.mod h1:Jdzbei4uFi9C3xDBfCwckRXjlX0UPooiP4g/zXgBMgQ= go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= -go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= go.opentelemetry.io/contrib v0.20.0 h1:ubFQUn0VCZ0gPwIoJfBJVpeBlyRMxu8Mm/huKWYd9p0= -go.opentelemetry.io/contrib v0.20.0/go.mod h1:G/EtFaa6qaN7+LxqfIAT3GiZa7Wv5DTBUzl5H4LY0Kc= go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.20.0 h1:sO4WKdPAudZGKPcpZT4MJn6JaDmpyLrMPDGGyA1SttE= -go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.20.0/go.mod h1:oVGt1LRbBOBq1A5BQLlUg9UaU/54aiHw8cgjV3aWZ/E= go.opentelemetry.io/otel v0.20.0 h1:eaP0Fqu7SXHwvjiqDq83zImeehOHX8doTvU9AwXON8g= -go.opentelemetry.io/otel v0.20.0/go.mod h1:Y3ugLH2oa81t5QO+Lty+zXf8zC9L26ax4Nzoxm/dooo= go.opentelemetry.io/otel/exporters/otlp v0.20.0 h1:PTNgq9MRmQqqJY0REVbZFvwkYOA85vbdQU/nVfxDyqg= -go.opentelemetry.io/otel/exporters/otlp v0.20.0/go.mod h1:YIieizyaN77rtLJra0buKiNBOm9XQfkPEKBeuhoMwAM= go.opentelemetry.io/otel/metric v0.20.0 h1:4kzhXFP+btKm4jwxpjIqjs41A7MakRFUS86bqLHTIw8= -go.opentelemetry.io/otel/metric v0.20.0/go.mod h1:598I5tYlH1vzBjn+BTuhzTCSb/9debfNp6R3s7Pr1eU= go.opentelemetry.io/otel/sdk v0.20.0 h1:JsxtGXd06J8jrnya7fdI/U/MR6yXA5DtbZy+qoHQlr8= -go.opentelemetry.io/otel/sdk v0.20.0/go.mod h1:g/IcepuwNsoiX5Byy2nNV0ySUF1em498m7hBWC279Yc= go.opentelemetry.io/otel/sdk/export/metric v0.20.0 h1:c5VRjxCXdQlx1HjzwGdQHzZaVI82b5EbBgOu2ljD92g= -go.opentelemetry.io/otel/sdk/export/metric v0.20.0/go.mod h1:h7RBNMsDJ5pmI1zExLi+bJK+Dr8NQCh0qGhm1KDnNlE= go.opentelemetry.io/otel/sdk/metric v0.20.0 h1:7ao1wpzHRVKf0OQ7GIxiQJA6X7DLX9o14gmVon7mMK8= -go.opentelemetry.io/otel/sdk/metric v0.20.0/go.mod h1:knxiS8Xd4E/N+ZqKmUPf3gTTZ4/0TjTXukfxjzSTpHE= go.opentelemetry.io/otel/trace v0.20.0 h1:1DL6EXUdcg95gukhuRRvLDO/4X5THh/5dIV52lqtnbw= -go.opentelemetry.io/otel/trace v0.20.0/go.mod h1:6GjCW8zgDjwGHGa6GkyeB8+/5vjT16gUEi0Nf1iBdgw= go.opentelemetry.io/proto/otlp v0.7.0 h1:rwOQPCuKAKmwGKq2aVNnYIibI6wnV7EvzgfTCzcdGg8= -go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= @@ -661,8 +566,8 @@ go.uber.org/zap v1.12.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= go.uber.org/zap v1.17.0/go.mod h1:MXVU+bhUf/A7Xi2HNOnopQOrmycQ5Ih87HtOu4q5SSo= go.uber.org/zap v1.19.0/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= go.uber.org/zap v1.21.0/go.mod h1:wjWOCqI0f2ZZrJF/UufIOkiC8ii6tm1iqIsLo76RfJw= -go.uber.org/zap v1.25.0 h1:4Hvk6GtkucQ790dqmj7l1eEnRdKm3k3ZUrUMS2d5+5c= -go.uber.org/zap v1.25.0/go.mod h1:JIAUzQIH94IC4fOJQm7gMmBJP5k7wQfdcnYdPoEXJYk= +go.uber.org/zap v1.24.0 h1:FiJd5l1UOLj0wCgbSE0rwwXHzEdAZS6hiiSnxJN/D60= +go.uber.org/zap v1.24.0/go.mod h1:2kMP+WWQ8aoFoedH3T2sq6iJ2yDWpHbP0f6MQbS9Gkg= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= @@ -672,8 +577,8 @@ golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.7.0/go.mod h1:pYwdfH91IfpZVANVyUOhSIPZaFoJGxTFbZhFTx+dXZU= -golang.org/x/crypto v0.12.0 h1:tFM/ta59kqch6LlvYnPa0yx5a83cL2nHflFhYKvv9Yk= -golang.org/x/crypto v0.12.0/go.mod h1:NF0Gs7EO5K4qLn+Ylc+fih8BSTeIjAP05siRnAh98yw= +golang.org/x/crypto v0.11.0 h1:6Ewdq3tDic1mg5xRO4milcWCfMVQhI4NkqWWvqejpuA= +golang.org/x/crypto v0.11.0/go.mod h1:xgJhtzW8F9jGdVFWZESrid1U1bjeNy4zgy5cRr/CIio= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 h1:QLureRX3moex6NVu/Lr4MGakp9FdA7sBHGBmvRW7NaM= @@ -718,13 +623,12 @@ golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96b golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.8.0/go.mod h1:QVkue5JL9kW//ek3r6jTKnTFis1tRmNAW2P1shuFdJc= -golang.org/x/net v0.14.0 h1:BONx9s002vGdD9umnlX1Po8vOZmrgH34qlHcD1MfK14= -golang.org/x/net v0.14.0/go.mod h1:PpSgVXXLK0OxS0F31C1/tv6XNguvCrnXIDrFMspZIUI= +golang.org/x/net v0.11.0 h1:Gi2tvZIJyBtO9SDr1q9h5hEQCp/4L2RQ+ar0qjx2oNU= +golang.org/x/net v0.11.0/go.mod h1:2L/ixqYpgIVXmeoSA/4Lu7BzTG4KIyPIryS4IsOd1oQ= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.8.0 h1:6dkIjl3j3LtZ/O3sTgZTMsLKSftL/B8Zgq4huOIIUu8= -golang.org/x/oauth2 v0.8.0/go.mod h1:yr7u4HXZRm1R1kBWqr/xKNqewf0plRYoB7sla+BCIXE= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -776,14 +680,13 @@ golang.org/x/sys v0.2.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.11.0 h1:eG7RXZHdqOJ1i+0lgLgCpSXAp6M3LYlAo6osgSi0xOM= -golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.10.0 h1:SqMFp9UcQJZa+pmYuAKjd9xq1f0j5rLcDIk0mj4qAsA= +golang.org/x/sys v0.10.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/term v0.6.0/go.mod h1:m6U89DPEgQRMq3DNkDClhWw02AUbt2daBVO4cn4Hv9U= -golang.org/x/term v0.11.0 h1:F9tnn/DA/Im8nCwm+fX+1/eBwi4qFjRT++MhtVC4ZX0= -golang.org/x/term v0.11.0/go.mod h1:zC9APTIj3jG3FdV/Ons+XE1riIZXG4aZ4GTHiPZJPIU= +golang.org/x/term v0.10.0 h1:3R7pNqamzBraeqj/Tj8qt1aQ2HpmlC+Cx/qL/7hn4/c= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= @@ -791,8 +694,8 @@ golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.8.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= -golang.org/x/text v0.12.0 h1:k+n5B8goJNdU7hSvEtMUz3d1Q6D/XW4COJSJR6fN0mc= -golang.org/x/text v0.12.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= +golang.org/x/text v0.11.0 h1:LAntKIrcmeSKERyiOh0XMV39LXS8IE9UL2yP7+f5ij4= +golang.org/x/text v0.11.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -827,17 +730,13 @@ golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 h1:H2TDz8ibqkAF6YGhCdN3jS9O0/s90v0rJh3X/OLHEUk= -golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2/go.mod h1:K8+ghG5WaK9qNqU5K3HdILfMLy1f3aNYFI/wnl100a8= gonum.org/v1/gonum v0.0.0-20181121035319-3f7ecaa7e8ca/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo= gonum.org/v1/gonum v0.8.2 h1:CCXrcPKiGGotvnN6jfUsKk4rRqm7q09/YbKb5xCEvtM= -gonum.org/v1/gonum v0.8.2/go.mod h1:oe/vMfY3deqTw+1EZJhuvEW2iwGF1bW9wwu7XCu0+v0= gonum.org/v1/netlib v0.0.0-20181029234149-ec6d1f5cefe6/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= google.golang.org/api v0.114.0 h1:1xQPji6cO2E2vLiI+C/XiFAnsn1WV3mjaEwGLhi3grE= -google.golang.org/api v0.114.0/go.mod h1:ifYI2ZsFK6/uGddGfAD5BMxlnkBqCmqHSDUVi45N5Yg= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.6.7 h1:FZR1q0exgwxzPzp/aF+VccGrSfxfPpkBqjIIEq3ru6c= -google.golang.org/appengine v1.6.7/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= google.golang.org/genproto v0.0.0-20180518175338-11a468237815/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20181004005441-af9cb2a35e7f/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= @@ -878,13 +777,11 @@ gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8 gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= -gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= gopkg.in/go-playground/assert.v1 v1.2.1/go.mod h1:9RXL0bg/zibRAgZUYszZSwO/z8Y/a8bDuhia5mkpMnE= gopkg.in/go-playground/validator.v8 v8.18.2/go.mod h1:RX2a/7Ha8BgOhfk7j780h4/u/RRjR0eouCJSH80/M2Y= gopkg.in/inf.v0 v0.9.1 h1:73M5CoZyi3ZLMOyDlQh031Cx6N9NDJ2Vvfl76EDAgDc= -gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw= gopkg.in/mgo.v2 v2.0.0-20180705113604-9856a29383ce/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA= gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= gopkg.in/natefinch/lumberjack.v2 v2.2.1 h1:bBRl1b0OH9s/DuPhuXpNl+VtCaJXFZ5/uEFST95x9zc= @@ -908,21 +805,12 @@ honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWh honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= k8s.io/api v0.27.2 h1:+H17AJpUMvl+clT+BPnKf0E3ksMAzoBBg7CntpSuADo= -k8s.io/api v0.27.2/go.mod h1:ENmbocXfBT2ADujUXcBhHV55RIT31IIEvkntP6vZKS4= k8s.io/apimachinery v0.27.2 h1:vBjGaKKieaIreI+oQwELalVG4d8f3YAMNpWLzDXkxeg= -k8s.io/apimachinery v0.27.2/go.mod h1:XNfZ6xklnMCOGGFNqXG7bUrQCoR04dh/E7FprV6pb+E= k8s.io/klog/v2 v2.90.1 h1:m4bYOKall2MmOiRaR1J+We67Do7vm9KiQVlT96lnHUw= -k8s.io/klog/v2 v2.90.1/go.mod h1:y1WjHnz7Dj687irZUWR/WLkLc5N1YHtjLdmgWjndZn0= k8s.io/utils v0.0.0-20230209194617-a36077c30491 h1:r0BAOLElQnnFhE/ApUsg3iHdVYYPBjNSSOMowRZxxsY= -k8s.io/utils v0.0.0-20230209194617-a36077c30491/go.mod h1:OLgZIPagt7ERELqWJFomSt595RzquPNLL48iOWgYOg0= sigs.k8s.io/json v0.0.0-20221116044647-bc3834ca7abd h1:EDPBXCAspyGV4jQlpZSudPeMmr1bNJefnuqLsRAsHZo= -sigs.k8s.io/json v0.0.0-20221116044647-bc3834ca7abd/go.mod h1:B8JuhiUyNFVKdsE8h686QcCxMaH6HrOAZj4vswFpcB0= sigs.k8s.io/structured-merge-diff/v4 v4.2.3 h1:PRbqxJClWWYMNV1dhaG4NsibJbArud9kFxnAMREiWFE= -sigs.k8s.io/structured-merge-diff/v4 v4.2.3/go.mod h1:qjx8mGObPmV2aSZepjQjbmb2ihdVs8cGKBraizNC69E= sigs.k8s.io/yaml v1.2.0/go.mod h1:yfXDCHCao9+ENCvLSE62v9VSji2MKu5jeNfTrofGhJc= sigs.k8s.io/yaml v1.3.0 h1:a2VclLzOGrwOHDiV8EfBGhvjHvP46CtW5j6POvhYGGo= -sigs.k8s.io/yaml v1.3.0/go.mod h1:GeOyir5tyXNByN85N/dRIT9es5UQNerPYEKK56eTBm8= sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0 h1:ucqkfpjg9WzSUubAO62csmucvxl4/JeW3F4I4909XkM= -sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0/go.mod h1:hI742Nqp5OhwiqlzhgfbWU4mW4yO10fP+LoT9WOswdU= sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 h1:e1sMhtVq9AfcEy8AXNb8eSg6gbzfdpYhoNqnPJa+GzI= -sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67/go.mod h1:L5q+DGLGOQFpo1snNEkLOJT2d1YTW66rWNzatr3He1k= From ae859e24506e68dc0221cfff1cea94cb3c95a900 Mon Sep 17 00:00:00 2001 From: husharp Date: Fri, 1 Sep 2023 15:28:18 +0800 Subject: [PATCH 29/49] refine test Signed-off-by: husharp --- integration_tests/pd_api_test.go | 62 +++++++++++++++++++------------- tikv/kv.go | 3 ++ 2 files changed, 40 insertions(+), 25 deletions(-) diff --git a/integration_tests/pd_api_test.go b/integration_tests/pd_api_test.go index 91dbba1606..d24bbbdf8f 100644 --- a/integration_tests/pd_api_test.go +++ b/integration_tests/pd_api_test.go @@ -105,48 +105,57 @@ func (c *storeSafeTsMockClient) CloseAddr(addr string) error { return c.Client.CloseAddr(addr) } -func (s *apiTestSuite) TestGetClusterMinResolvedTS() { +func (s *apiTestSuite) TestGetStoresMinResolvedTS() { util.EnableFailpoints() - // Try to get the minimum resolved timestamp of the cluster from PD. require := s.Require() + require.Nil(failpoint.Enable("tikvclient/mockFastSafeTSUpdater", `return()`)) + defer func() { + require.Nil(failpoint.Disable("tikvclient/mockFastSafeTSUpdater")) + }() + + // Set DC label for store 1. + // Mock Cluster-level min resolved ts failed. + dcLabel := "testDC" + restore := config.UpdateGlobal(func(conf *config.Config) { + conf.TxnScope = dcLabel + }) + defer restore() + + labels := []*metapb.StoreLabel{ + { + Key: tikv.DCLabelKey, + Value: dcLabel, + }, + } + storeID := uint64(1) + s.store.GetRegionCache().SetRegionCacheStore(storeID, s.storeAddr(storeID), s.storeAddr(storeID), tikvrpc.TiKV, 1, labels) + // Try to get the minimum resolved timestamp of the stores from PD. require.Nil(failpoint.Enable("tikvclient/InjectMinResolvedTS", `return(100)`)) mockClient := storeSafeTsMockClient{ Client: s.store.GetTiKVClient(), } s.store.SetTiKVClient(&mockClient) var retryCount int - for s.store.GetMinSafeTS(oracle.GlobalTxnScope) != 100 { - time.Sleep(2 * time.Second) + for s.store.GetMinSafeTS(dcLabel) != 100 { + time.Sleep(100 * time.Millisecond) if retryCount > 5 { break } retryCount++ } require.Equal(int32(0), atomic.LoadInt32(&mockClient.requestCount)) - require.Equal(uint64(100), s.store.GetMinSafeTS(oracle.GlobalTxnScope)) + require.Equal(uint64(100), s.store.GetMinSafeTS(dcLabel)) require.Nil(failpoint.Disable("tikvclient/InjectMinResolvedTS")) - - // Try to get the minimum resolved timestamp of the cluster from TiKV. - require.Nil(failpoint.Enable("tikvclient/InjectMinResolvedTS", `return(0)`)) - defer func() { - s.Require().Nil(failpoint.Disable("tikvclient/InjectMinResolvedTS")) - }() - retryCount = 0 - for s.store.GetMinSafeTS(oracle.GlobalTxnScope) != 150 { - time.Sleep(2 * time.Second) - if retryCount > 5 { - break - } - retryCount++ - } - require.GreaterOrEqual(atomic.LoadInt32(&mockClient.requestCount), int32(1)) - require.Equal(uint64(150), s.store.GetMinSafeTS(oracle.GlobalTxnScope)) } func (s *apiTestSuite) TestDCLabelClusterMinResolvedTS() { util.EnableFailpoints() // Try to get the minimum resolved timestamp of the cluster from PD. require := s.Require() + require.Nil(failpoint.Enable("tikvclient/mockFastSafeTSUpdater", `return()`)) + defer func() { + require.Nil(failpoint.Disable("tikvclient/mockFastSafeTSUpdater")) + }() require.Nil(failpoint.Enable("tikvclient/InjectMinResolvedTS", `return(100)`)) mockClient := storeSafeTsMockClient{ Client: s.store.GetTiKVClient(), @@ -154,7 +163,7 @@ func (s *apiTestSuite) TestDCLabelClusterMinResolvedTS() { s.store.SetTiKVClient(&mockClient) var retryCount int for s.store.GetMinSafeTS(oracle.GlobalTxnScope) != 100 { - time.Sleep(2 * time.Second) + time.Sleep(100 * time.Millisecond) if retryCount > 5 { break } @@ -162,11 +171,14 @@ func (s *apiTestSuite) TestDCLabelClusterMinResolvedTS() { } require.Equal(atomic.LoadInt32(&mockClient.requestCount), int32(0)) require.Equal(uint64(100), s.store.GetMinSafeTS(oracle.GlobalTxnScope)) + s.Require().Nil(failpoint.Disable("tikvclient/InjectMinResolvedTS")) + + // Set DC label for store 1. + // Mock PD server not support get min resolved ts by stores. + require.Nil(failpoint.Enable("tikvclient/InjectMinResolvedTS", `return(0)`)) defer func() { s.Require().Nil(failpoint.Disable("tikvclient/InjectMinResolvedTS")) }() - - // Set DC label for store 1. dcLabel := "testDC" restore := config.UpdateGlobal(func(conf *config.Config) { conf.TxnScope = dcLabel @@ -185,7 +197,7 @@ func (s *apiTestSuite) TestDCLabelClusterMinResolvedTS() { // Try to get the minimum resolved timestamp of the store from TiKV. retryCount = 0 for s.store.GetMinSafeTS(dcLabel) != 150 { - time.Sleep(2 * time.Second) + time.Sleep(100 * time.Millisecond) if retryCount > 5 { break } diff --git a/tikv/kv.go b/tikv/kv.go index d7f9a41996..fcd730a89e 100644 --- a/tikv/kv.go +++ b/tikv/kv.go @@ -565,6 +565,9 @@ func (s *KVStore) updateMinSafeTS(txnScope string, storeIDs []uint64) { func (s *KVStore) safeTSUpdater() { defer s.wg.Done() t := time.NewTicker(safeTSUpdateInterval) + if _, e := util.EvalFailpoint("mockFastSafeTSUpdater"); e == nil { + t.Reset(time.Millisecond * 100) + } defer t.Stop() ctx, cancel := context.WithCancel(s.ctx) ctx = util.WithInternalSourceType(ctx, util.InternalTxnGC) From 5527f3dd0e38b4fd11c4f019e490b59cd5c59a45 Mon Sep 17 00:00:00 2001 From: husharp Date: Fri, 1 Sep 2023 17:33:32 +0800 Subject: [PATCH 30/49] address comment Signed-off-by: husharp --- integration_tests/pd_api_test.go | 20 ++++++++++---------- tikv/kv.go | 9 +++++---- 2 files changed, 15 insertions(+), 14 deletions(-) diff --git a/integration_tests/pd_api_test.go b/integration_tests/pd_api_test.go index d24bbbdf8f..b47d294e92 100644 --- a/integration_tests/pd_api_test.go +++ b/integration_tests/pd_api_test.go @@ -69,7 +69,7 @@ type apiTestSuite struct { func (s *apiTestSuite) SetupTest() { addrs := strings.Split(*pdAddrs, ",") pdClient, err := pd.NewClient(addrs, pd.SecurityOption{}) - s.Require().Nil(err) + s.Require().NoError(err) rpcClient := tikv.NewRPCClient() // Set PD HTTP client. store, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0, tikv.WithPDHTTPClient(nil, addrs)) @@ -108,9 +108,9 @@ func (c *storeSafeTsMockClient) CloseAddr(addr string) error { func (s *apiTestSuite) TestGetStoresMinResolvedTS() { util.EnableFailpoints() require := s.Require() - require.Nil(failpoint.Enable("tikvclient/mockFastSafeTSUpdater", `return()`)) + require.NoError(failpoint.Enable("tikvclient/mockFastSafeTSUpdater", `return()`)) defer func() { - require.Nil(failpoint.Disable("tikvclient/mockFastSafeTSUpdater")) + require.NoError(failpoint.Disable("tikvclient/mockFastSafeTSUpdater")) }() // Set DC label for store 1. @@ -130,7 +130,7 @@ func (s *apiTestSuite) TestGetStoresMinResolvedTS() { storeID := uint64(1) s.store.GetRegionCache().SetRegionCacheStore(storeID, s.storeAddr(storeID), s.storeAddr(storeID), tikvrpc.TiKV, 1, labels) // Try to get the minimum resolved timestamp of the stores from PD. - require.Nil(failpoint.Enable("tikvclient/InjectMinResolvedTS", `return(100)`)) + require.NoError(failpoint.Enable("tikvclient/InjectMinResolvedTS", `return(100)`)) mockClient := storeSafeTsMockClient{ Client: s.store.GetTiKVClient(), } @@ -152,11 +152,11 @@ func (s *apiTestSuite) TestDCLabelClusterMinResolvedTS() { util.EnableFailpoints() // Try to get the minimum resolved timestamp of the cluster from PD. require := s.Require() - require.Nil(failpoint.Enable("tikvclient/mockFastSafeTSUpdater", `return()`)) + require.NoError(failpoint.Enable("tikvclient/mockFastSafeTSUpdater", `return()`)) defer func() { - require.Nil(failpoint.Disable("tikvclient/mockFastSafeTSUpdater")) + require.NoError(failpoint.Disable("tikvclient/mockFastSafeTSUpdater")) }() - require.Nil(failpoint.Enable("tikvclient/InjectMinResolvedTS", `return(100)`)) + require.NoError(failpoint.Enable("tikvclient/InjectMinResolvedTS", `return(100)`)) mockClient := storeSafeTsMockClient{ Client: s.store.GetTiKVClient(), } @@ -171,13 +171,13 @@ func (s *apiTestSuite) TestDCLabelClusterMinResolvedTS() { } require.Equal(atomic.LoadInt32(&mockClient.requestCount), int32(0)) require.Equal(uint64(100), s.store.GetMinSafeTS(oracle.GlobalTxnScope)) - s.Require().Nil(failpoint.Disable("tikvclient/InjectMinResolvedTS")) + require.NoError(failpoint.Disable("tikvclient/InjectMinResolvedTS")) // Set DC label for store 1. // Mock PD server not support get min resolved ts by stores. - require.Nil(failpoint.Enable("tikvclient/InjectMinResolvedTS", `return(0)`)) + require.NoError(failpoint.Enable("tikvclient/InjectMinResolvedTS", `return(0)`)) defer func() { - s.Require().Nil(failpoint.Disable("tikvclient/InjectMinResolvedTS")) + require.NoError(failpoint.Disable("tikvclient/InjectMinResolvedTS")) }() dcLabel := "testDC" restore := config.UpdateGlobal(func(conf *config.Config) { diff --git a/tikv/kv.go b/tikv/kv.go index fcd730a89e..506bfd67d7 100644 --- a/tikv/kv.go +++ b/tikv/kv.go @@ -584,7 +584,7 @@ func (s *KVStore) safeTSUpdater() { func (s *KVStore) updateSafeTS(ctx context.Context) { // Try to get the cluster-level minimum resolved timestamp from PD first. - if s.setClusterMinSafeTSByPD(ctx) { + if s.updateGlobalTxnScopeTSFromPD(ctx) { return } @@ -678,8 +678,9 @@ var ( clusterMinSafeTSGap = metrics.TiKVMinSafeTSGapSeconds.WithLabelValues("cluster") ) -// setClusterMinSafeTSByPD check whether it is needed to get cluster-level's min resolved ts from PD. -func (s *KVStore) setClusterMinSafeTSByPD(ctx context.Context) bool { +// updateGlobalTxnScopeTSFromPD check whether it is needed to get cluster-level's min resolved ts from PD +// to update min safe ts for global txn scope. +func (s *KVStore) updateGlobalTxnScopeTSFromPD(ctx context.Context) bool { isGlobal := config.GetTxnScopeFromConfig() == oracle.GlobalTxnScope // Try to get the minimum resolved timestamp of the cluster from PD. if s.pdHttpClient != nil && isGlobal { @@ -687,7 +688,7 @@ func (s *KVStore) setClusterMinSafeTSByPD(ctx context.Context) bool { if err != nil { logutil.BgLogger().Debug("get resolved TS from PD failed", zap.Error(err)) } else if clusterMinSafeTS != 0 { - // Update metrics. + // Update ts and metrics. preClusterMinSafeTS := s.GetMinSafeTS(oracle.GlobalTxnScope) if preClusterMinSafeTS > clusterMinSafeTS { skipSafeTSUpdateCounter.Inc() From 89493ff9853db75fd6065d996330685b36b024a6 Mon Sep 17 00:00:00 2001 From: you06 Date: Thu, 7 Sep 2023 21:23:09 +0800 Subject: [PATCH 31/49] set the request source at the last section (#960) Signed-off-by: you06 --- internal/locate/region_request.go | 5 +++-- internal/locate/region_request3_test.go | 4 ++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index 63242877d4..6a89942699 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -2337,9 +2337,10 @@ func (s *replicaSelector) replicaType(rpcCtx *RPCContext) string { func (s *replicaSelector) patchRequestSource(req *tikvrpc.Request, rpcCtx *RPCContext) { var sb strings.Builder - sb.WriteString(req.InputRequestSource) - sb.WriteByte('-') defer func() { + // TiKV does the limit control by the last part of the request source. + sb.WriteByte('_') + sb.WriteString(req.InputRequestSource) req.RequestSource = sb.String() }() diff --git a/internal/locate/region_request3_test.go b/internal/locate/region_request3_test.go index 7a92c58f6f..6d14fefd5b 100644 --- a/internal/locate/region_request3_test.go +++ b/internal/locate/region_request3_test.go @@ -1515,7 +1515,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestRetryRequestSource() { rpcCtx, err := replicaSelector.buildRPCContext(bo) s.Nil(err) replicaSelector.patchRequestSource(req, rpcCtx) - s.Equal("test-"+firstReplica, req.RequestSource) + s.Equal(firstReplica+"_test", req.RequestSource) // retry setReadType(req, retryReplica) @@ -1526,7 +1526,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestRetryRequestSource() { s.Nil(err) req.IsRetryRequest = true replicaSelector.patchRequestSource(req, rpcCtx) - s.Equal("test-retry_"+firstReplica+"_"+retryReplica, req.RequestSource) + s.Equal("retry_"+firstReplica+"_"+retryReplica+"_test", req.RequestSource) } } } From 94e94cb8740825e96d3968ff9a82c15976147c9b Mon Sep 17 00:00:00 2001 From: bufferflies <1045931706@qq.com> Date: Fri, 8 Sep 2023 15:50:46 +0800 Subject: [PATCH 32/49] update kvproto Signed-off-by: bufferflies <1045931706@qq.com> --- go.mod | 2 +- go.sum | 4 ++-- integration_tests/go.mod | 2 +- integration_tests/go.sum | 4 ++-- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/go.mod b/go.mod index 7fd8fd58da..b301bac77b 100644 --- a/go.mod +++ b/go.mod @@ -14,7 +14,7 @@ require ( github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 - github.com/pingcap/kvproto v0.0.0-20230818065851-7b612d935bf9 + github.com/pingcap/kvproto v0.0.0-20230904082117-ecdbf1f8c130 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.15.1 diff --git a/go.sum b/go.sum index 570bd7435b..cd65c90c6d 100644 --- a/go.sum +++ b/go.sum @@ -136,8 +136,8 @@ github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c h1:CgbKAHto5CQgW github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= -github.com/pingcap/kvproto v0.0.0-20230818065851-7b612d935bf9 h1:VDoZ18CAXoTUNTCxfl4BjQSD5rJQri8QlH8nu0ZuHeg= -github.com/pingcap/kvproto v0.0.0-20230818065851-7b612d935bf9/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/kvproto v0.0.0-20230904082117-ecdbf1f8c130 h1:qbLm5cOdCWxZ0mt6SaN2aXI+KFekbPqURd6YkNI+XRI= +github.com/pingcap/kvproto v0.0.0-20230904082117-ecdbf1f8c130/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8IDP+SZrdhV1Kibl9KrHxJ9eciw= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= diff --git a/integration_tests/go.mod b/integration_tests/go.mod index 01e508d622..a2a77d3975 100644 --- a/integration_tests/go.mod +++ b/integration_tests/go.mod @@ -6,7 +6,7 @@ require ( github.com/ninedraft/israce v0.0.3 github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32 github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c - github.com/pingcap/kvproto v0.0.0-20230818065851-7b612d935bf9 + github.com/pingcap/kvproto v0.0.0-20230904082117-ecdbf1f8c130 github.com/pingcap/tidb v1.1.0-beta.0.20230619015310-8b1006f1af04 github.com/pkg/errors v0.9.1 github.com/stretchr/testify v1.8.4 diff --git a/integration_tests/go.sum b/integration_tests/go.sum index 134ec393cc..fc6ec1e505 100644 --- a/integration_tests/go.sum +++ b/integration_tests/go.sum @@ -363,8 +363,8 @@ github.com/pingcap/fn v1.0.0 h1:CyA6AxcOZkQh52wIqYlAmaVmF6EvrcqFywP463pjA8g= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20230818065851-7b612d935bf9 h1:VDoZ18CAXoTUNTCxfl4BjQSD5rJQri8QlH8nu0ZuHeg= -github.com/pingcap/kvproto v0.0.0-20230818065851-7b612d935bf9/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/kvproto v0.0.0-20230904082117-ecdbf1f8c130 h1:qbLm5cOdCWxZ0mt6SaN2aXI+KFekbPqURd6YkNI+XRI= +github.com/pingcap/kvproto v0.0.0-20230904082117-ecdbf1f8c130/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/log v1.1.0/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22 h1:2SOzvGvE8beiC1Y4g9Onkvu6UmuBBOeWRGQEjJaT/JY= From ba3de39920f87c902796b4c2b1078b8e055aab4f Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Mon, 11 Sep 2023 14:38:07 +0800 Subject: [PATCH 33/49] *: fix panic log Signed-off-by: crazycs520 --- internal/client/client_batch.go | 6 +++--- internal/locate/region_cache.go | 4 ++-- util/misc.go | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/internal/client/client_batch.go b/internal/client/client_batch.go index 6a270bc6d2..632d6f3057 100644 --- a/internal/client/client_batch.go +++ b/internal/client/client_batch.go @@ -301,7 +301,7 @@ func (a *batchConn) batchSendLoop(cfg config.TiKVClient) { if r := recover(); r != nil { metrics.TiKVPanicCounter.WithLabelValues(metrics.LabelBatchSendLoop).Inc() logutil.BgLogger().Error("batchSendLoop", - zap.Reflect("r", r), + zap.Any("r", r), zap.Stack("stack")) logutil.BgLogger().Info("restart batchSendLoop") go a.batchSendLoop(cfg) @@ -436,7 +436,7 @@ func (s *batchCommandsStream) recv() (resp *tikvpb.BatchCommandsResponse, err er if r := recover(); r != nil { metrics.TiKVPanicCounter.WithLabelValues(metrics.LabelBatchRecvLoop).Inc() logutil.BgLogger().Error("batchCommandsClient.recv panic", - zap.Reflect("r", r), + zap.Any("r", r), zap.Stack("stack")) err = errors.New("batch conn recv paniced") } @@ -604,7 +604,7 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient, tikvTransport if r := recover(); r != nil { metrics.TiKVPanicCounter.WithLabelValues(metrics.LabelBatchRecvLoop).Inc() logutil.BgLogger().Error("batchRecvLoop", - zap.Reflect("r", r), + zap.Any("r", r), zap.Stack("stack")) logutil.BgLogger().Info("restart batchRecvLoop") go c.batchRecvLoop(cfg, tikvTransportLayerLoad, streamClient) diff --git a/internal/locate/region_cache.go b/internal/locate/region_cache.go index 56444ac093..e7cfb5b8ce 100644 --- a/internal/locate/region_cache.go +++ b/internal/locate/region_cache.go @@ -587,7 +587,7 @@ func (c *RegionCache) checkAndResolve(needCheckStores []*Store, needCheck func(* r := recover() if r != nil { logutil.BgLogger().Error("panic in the checkAndResolve goroutine", - zap.Reflect("r", r), + zap.Any("r", r), zap.Stack("stack trace")) } }() @@ -2993,7 +2993,7 @@ func (c *RegionCache) checkAndUpdateStoreSlowScores() { r := recover() if r != nil { logutil.BgLogger().Error("panic in the checkAndUpdateStoreSlowScores goroutine", - zap.Reflect("r", r), + zap.Any("r", r), zap.Stack("stack trace")) } }() diff --git a/util/misc.go b/util/misc.go index bd3e2b7793..e324bf7974 100644 --- a/util/misc.go +++ b/util/misc.go @@ -89,7 +89,7 @@ func WithRecovery(exec func(), recoverFn func(r interface{})) { } if r != nil { logutil.BgLogger().Error("panic in the recoverable goroutine", - zap.Reflect("r", r), + zap.Any("r", r), zap.Stack("stack trace")) } }() From 3216af2cb12592bc0001df187f20e6541ccd21ec Mon Sep 17 00:00:00 2001 From: bufferflies <1045931706@qq.com> Date: Tue, 12 Sep 2023 11:59:38 +0800 Subject: [PATCH 34/49] add cancel reason Signed-off-by: bufferflies <1045931706@qq.com> --- internal/locate/region_request.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index 6a89942699..c5b03d457a 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -113,7 +113,7 @@ type RegionRequestSender struct { } func (s *RegionRequestSender) String() string { - return fmt.Sprintf("{replicaSelector: %v}", s.replicaSelector.String()) + return fmt.Sprintf("{rpcError:%v,replicaSelector: %v}", s.rpcError, s.replicaSelector.String()) } // RegionRequestRuntimeStats records the runtime stats of send region requests. From 62a297ae74358ffa094e9b23e6e99f3fa4307380 Mon Sep 17 00:00:00 2001 From: bufferflies <1045931706@qq.com> Date: Tue, 12 Sep 2023 12:21:15 +0800 Subject: [PATCH 35/49] add log Signed-off-by: bufferflies <1045931706@qq.com> --- internal/client/client_batch.go | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/internal/client/client_batch.go b/internal/client/client_batch.go index 632d6f3057..b7159847b9 100644 --- a/internal/client/client_batch.go +++ b/internal/client/client_batch.go @@ -37,6 +37,7 @@ package client import ( "context" + "fmt" "math" "runtime/trace" "sync" @@ -793,7 +794,8 @@ func sendBatchRequest( case <-timer.C: return nil, errors.WithMessage(context.DeadlineExceeded, "wait sendLoop") } - metrics.TiKVBatchWaitDuration.Observe(float64(time.Since(start))) + waitDuration := time.Since(start) + metrics.TiKVBatchWaitDuration.Observe(float64(waitDuration)) select { case res, ok := <-entry.res: @@ -808,7 +810,8 @@ func sendBatchRequest( return nil, errors.WithStack(ctx.Err()) case <-timer.C: atomic.StoreInt32(&entry.canceled, 1) - return nil, errors.WithMessage(context.DeadlineExceeded, "wait recvLoop") + reason := fmt.Sprintf("wait recvLoop timeout,timeout:%s, wait_duration:%s:", timeout, waitDuration) + return nil, errors.WithMessage(context.DeadlineExceeded, reason) } } From fb8e4c0eac647fa6f9b95d2dfe703007baf08a90 Mon Sep 17 00:00:00 2001 From: you06 Date: Tue, 12 Sep 2023 12:36:53 +0800 Subject: [PATCH 36/49] add region cache state test & fix some issues of replica selector (#910) (#942) * add region cache state test & fix some issues of replica selector (#910) Signed-off-by: you06 remove duplicate code Signed-off-by: you06 * remove comment Signed-off-by: you06 * lint Signed-off-by: you06 * fix flaky test Signed-off-by: you06 --------- Signed-off-by: you06 --- internal/locate/region_cache.go | 4 +- internal/locate/region_request.go | 2 +- internal/locate/region_request_state_test.go | 841 ++++++++++++++++++ internal/mockstore/mocktikv/cluster.go | 4 +- .../mockstore/mocktikv/cluster_manipulate.go | 26 + 5 files changed, 873 insertions(+), 4 deletions(-) create mode 100644 internal/locate/region_request_state_test.go diff --git a/internal/locate/region_cache.go b/internal/locate/region_cache.go index e7cfb5b8ce..e21703f31a 100644 --- a/internal/locate/region_cache.go +++ b/internal/locate/region_cache.go @@ -534,10 +534,12 @@ func (c *RegionCache) Close() { c.cancelFunc() } +var reloadRegionInterval = int64(10 * time.Second) + // asyncCheckAndResolveLoop with func (c *RegionCache) asyncCheckAndResolveLoop(interval time.Duration) { ticker := time.NewTicker(interval) - reloadRegionTicker := time.NewTicker(10 * time.Second) + reloadRegionTicker := time.NewTicker(time.Duration(atomic.LoadInt64(&reloadRegionInterval))) defer func() { ticker.Stop() reloadRegionTicker.Stop() diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index 6a89942699..449db7be2d 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -466,7 +466,6 @@ type tryFollower struct { func (state *tryFollower) next(bo *retry.Backoffer, selector *replicaSelector) (*RPCContext, error) { hasDeadlineExceededErr := false - //hasDeadlineExceededErr || targetReplica.deadlineErrUsingConfTimeout filterReplicas := func(fn func(*replica) bool) (AccessIndex, *replica) { for i := 0; i < len(selector.replicas); i++ { idx := AccessIndex((int(state.lastIdx) + i) % len(selector.replicas)) @@ -2054,6 +2053,7 @@ func (s *RegionRequestSender) onRegionError( } // This peer is removed from the region. Invalidate the region since it's too stale. + // if the region error is from follower, can we mark the peer unavailable and reload region asynchronously? if regionErr.GetRegionNotFound() != nil { s.regionCache.InvalidateCachedRegion(ctx.Region) return false, nil diff --git a/internal/locate/region_request_state_test.go b/internal/locate/region_request_state_test.go new file mode 100644 index 0000000000..d1b5c6201e --- /dev/null +++ b/internal/locate/region_request_state_test.go @@ -0,0 +1,841 @@ +// Copyright 2023 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. + +package locate + +import ( + "context" + "fmt" + "strconv" + "strings" + "sync/atomic" + "testing" + "time" + + "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/require" + tikverr "github.com/tikv/client-go/v2/error" + "github.com/tikv/client-go/v2/internal/apicodec" + "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/metrics" + "github.com/tikv/client-go/v2/tikvrpc" +) + +type testRegionCacheStaleReadSuite struct { + *require.Assertions + cluster *mocktikv.Cluster + storeIDs []uint64 + peerIDs []uint64 + regionID uint64 + leaderPeer uint64 + store2zone map[uint64]string + cache *RegionCache + bo *retry.Backoffer + regionRequestSender *RegionRequestSender + mvccStore mocktikv.MVCCStore + injection testRegionCacheFSMSuiteInjection +} + +type testRegionCacheFSMSuiteInjection struct { + leaderRegionError func(*tikvrpc.Request, string) *errorpb.Error + followerRegionError func(*tikvrpc.Request, string) *errorpb.Error + unavailableStoreIDs map[uint64]struct{} + timeoutStoreIDs map[uint64]struct{} +} + +type SuccessReadType int + +const ( + ReadFail SuccessReadType = iota + SuccessLeaderRead + SuccessFollowerRead + SuccessStaleRead +) + +func (s *testRegionCacheStaleReadSuite) SetupTest() { + s.mvccStore = mocktikv.MustNewMVCCStore() + s.cluster = mocktikv.NewCluster(s.mvccStore) + s.storeIDs, s.peerIDs, s.regionID, s.leaderPeer, s.store2zone = mocktikv.BootstrapWithMultiZones(s.cluster, 3, 2) + pdCli := &CodecPDClient{mocktikv.NewPDClient(s.cluster), apicodec.NewCodecV1(apicodec.ModeTxn)} + s.cache = NewRegionCache(pdCli) + s.bo = retry.NewNoopBackoff(context.Background()) + client := mocktikv.NewRPCClient(s.cluster, s.mvccStore, nil) + s.regionRequestSender = NewRegionRequestSender(s.cache, client) + s.setClient() + s.injection = testRegionCacheFSMSuiteInjection{ + unavailableStoreIDs: make(map[uint64]struct{}), + } +} + +func (s *testRegionCacheStaleReadSuite) TearDownTest() { + s.cache.testingKnobs.mockRequestLiveness.Store((*livenessFunc)(nil)) + s.cache.Close() + s.mvccStore.Close() +} + +func (s *testRegionCacheStaleReadSuite) getStore(leader bool) (uint64, *metapb.Store) { + var ( + zone string + peerID uint64 + storeID uint64 + ) + if leader { + zone = "z1" + } else { + zone = "z2" + } + region, _ := s.cluster.GetRegion(s.regionID) +FIND: + for _, peer := range region.Peers { + store := s.cluster.GetStore(peer.StoreId) + for _, label := range store.Labels { + if label.Key == "zone" && label.Value == zone { + peerID = peer.Id + storeID = peer.StoreId + break FIND + } + } + } + store := s.cluster.GetStore(storeID) + if store == nil { + return 0, nil + } + return peerID, store +} + +func (s *testRegionCacheStaleReadSuite) getLeader() (uint64, *metapb.Store) { + return s.getStore(true) +} + +func (s *testRegionCacheStaleReadSuite) getFollower() (uint64, *metapb.Store) { + return s.getStore(false) +} + +func (s *testRegionCacheStaleReadSuite) setClient() { + s.regionRequestSender.client = &fnClient{fn: func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (response *tikvrpc.Response, err error) { + var store *metapb.Store + find := false + for _, one := range s.cluster.GetAllStores() { + if one.Address == addr { + store = one + find = true + break + } + } + if !find { + return nil, errors.New("no available connections") + } + if _, unavailable := s.injection.unavailableStoreIDs[store.Id]; unavailable { + return nil, errors.New("no available connections") + } + if _, timeout := s.injection.timeoutStoreIDs[store.Id]; timeout { + return nil, errors.WithMessage(context.DeadlineExceeded, "wait recvLoop") + } + + zone := "" + for _, label := range store.Labels { + if label.Key == "zone" { + zone = label.Value + break + } + } + response = &tikvrpc.Response{} + region, _ := s.cluster.GetRegion(s.regionID) + peerExist := false + for _, peer := range region.Peers { + if req.Peer.Id == peer.Id { + if peer.StoreId != store.Id { + response.Resp = &kvrpcpb.GetResponse{RegionError: &errorpb.Error{ + RegionNotFound: &errorpb.RegionNotFound{RegionId: s.regionID}, + }} + return + } + peerExist = true + } + } + if !peerExist { + response.Resp = &kvrpcpb.GetResponse{RegionError: &errorpb.Error{ + RegionNotFound: &errorpb.RegionNotFound{RegionId: s.regionID}, + }} + return + } + + _, leader := s.getLeader() + s.NotNil(leader) + isLeader := addr == leader.Address + if isLeader { + // leader region error + if s.injection.leaderRegionError != nil { + if regionRrr := s.injection.leaderRegionError(req, zone); regionRrr != nil { + response.Resp = &kvrpcpb.GetResponse{RegionError: regionRrr} + return + } + } + } else { + // follower read leader + if !req.ReplicaRead && !req.StaleRead { + _, leaderPeer, _ := s.cluster.GetRegionByID(s.regionID) + response.Resp = &kvrpcpb.GetResponse{RegionError: &errorpb.Error{ + NotLeader: &errorpb.NotLeader{ + RegionId: req.RegionId, + Leader: leaderPeer, + }, + }} + return + } + // follower region error + if s.injection.followerRegionError != nil { + if regionRrr := s.injection.followerRegionError(req, zone); regionRrr != nil { + response.Resp = &kvrpcpb.GetResponse{RegionError: regionRrr} + return + } + } + } + // no error + var successReadType SuccessReadType + if req.StaleRead { + successReadType = SuccessStaleRead + } else if isLeader { + successReadType = SuccessLeaderRead + } else { + successReadType = SuccessFollowerRead + } + s.NotEmpty(zone) + respStr := fmt.Sprintf("%d-%s-%d", store.Id, zone, successReadType) + response.Resp = &kvrpcpb.GetResponse{Value: []byte(respStr)} + return + }} + + tf := func(store *Store, bo *retry.Backoffer) livenessState { + _, ok := s.injection.unavailableStoreIDs[store.storeID] + if ok { + return unreachable + } + return reachable + } + s.cache.testingKnobs.mockRequestLiveness.Store((*livenessFunc)(&tf)) +} + +func (s *testRegionCacheStaleReadSuite) extractResp(resp *tikvrpc.Response) (uint64, string, SuccessReadType) { + resps := strings.Split(string(resp.Resp.(*kvrpcpb.GetResponse).Value), "-") + s.Len(resps, 3) + storeID, err := strconv.Atoi(resps[0]) + s.Nil(err) + successReadType, err := strconv.Atoi(resps[2]) + s.Nil(err) + return uint64(storeID), resps[1], SuccessReadType(successReadType) +} + +func (s *testRegionCacheStaleReadSuite) setUnavailableStore(id uint64) { + s.injection.unavailableStoreIDs[id] = struct{}{} +} + +func (s *testRegionCacheStaleReadSuite) setTimeout(id uint64) { //nolint: unused + s.injection.timeoutStoreIDs[id] = struct{}{} +} + +func TestRegionCacheStaleRead(t *testing.T) { + originReloadRegionInterval := atomic.LoadInt64(&reloadRegionInterval) + originBoTiKVServerBusy := retry.BoTiKVServerBusy + defer func() { + atomic.StoreInt64(&reloadRegionInterval, originReloadRegionInterval) + retry.BoTiKVServerBusy = originBoTiKVServerBusy + }() + atomic.StoreInt64(&reloadRegionInterval, int64(24*time.Hour)) // disable reload region + retry.BoTiKVServerBusy = retry.NewConfig("tikvServerBusy", &metrics.BackoffHistogramServerBusy, retry.NewBackoffFnCfg(2, 10, retry.EqualJitter), tikverr.ErrTiKVServerBusy) + regionCacheTestCases := []RegionCacheTestCase{ + { + do: followerDown, + leaderRegionValid: true, + leaderAsyncReload: Some(false), + leaderSuccessReplica: []string{"z1"}, + leaderSuccessReadType: SuccessStaleRead, + followerRegionValid: true, + followerAsyncReload: Some(false), + followerSuccessReplica: []string{"z1"}, + followerSuccessReadType: SuccessLeaderRead, + }, + { + do: followerDownAndUp, + leaderRegionValid: true, + leaderAsyncReload: None[bool](), + leaderSuccessReplica: []string{"z1"}, + leaderSuccessReadType: SuccessStaleRead, + followerRegionValid: true, + followerAsyncReload: Some(true), + followerSuccessReplica: []string{"z1"}, + // because follower's epoch is changed, leader will be selected. + followerSuccessReadType: SuccessStaleRead, + }, + { + do: followerMove, + recoverable: true, + leaderRegionValid: true, + leaderAsyncReload: Some(false), + leaderSuccessReplica: []string{"z1"}, + leaderSuccessReadType: SuccessStaleRead, + followerRegionValid: false, + followerAsyncReload: Some(false), + // may async reload region and access it from leader. + followerSuccessReplica: []string{}, + followerSuccessReadType: ReadFail, + }, + { + do: evictLeader, + leaderRegionValid: true, + leaderAsyncReload: Some(false), + // leader is evicted, but can still serve as follower. + leaderSuccessReplica: []string{"z1"}, + leaderSuccessReadType: SuccessStaleRead, + followerRegionValid: true, + followerAsyncReload: Some(false), + followerSuccessReplica: []string{"z2"}, + followerSuccessReadType: SuccessStaleRead, + }, + { + do: leaderMove, + leaderRegionValid: false, + leaderAsyncReload: Some(false), + leaderSuccessReplica: []string{}, + leaderSuccessReadType: ReadFail, + followerRegionValid: true, + followerAsyncReload: Some(false), + followerSuccessReplica: []string{"z2"}, + followerSuccessReadType: SuccessStaleRead, + }, + { + do: leaderDown, + leaderRegionValid: true, + leaderAsyncReload: Some(true), + leaderSuccessReplica: []string{"z2", "z3"}, + leaderSuccessReadType: SuccessFollowerRead, + followerRegionValid: true, + followerAsyncReload: Some(false), + followerSuccessReplica: []string{"z2"}, + followerSuccessReadType: SuccessStaleRead, + }, + { + do: leaderDownAndUp, + leaderRegionValid: true, + leaderAsyncReload: Some(true), + leaderSuccessReplica: []string{"z2", "z3"}, + leaderSuccessReadType: SuccessFollowerRead, + followerRegionValid: true, + followerAsyncReload: None[bool](), + followerSuccessReplica: []string{"z2"}, + followerSuccessReadType: SuccessStaleRead, + }, + { + do: leaderDownAndElect, + leaderRegionValid: true, + leaderAsyncReload: Some(true), + leaderSuccessReplica: []string{"z2", "z3"}, + leaderSuccessReadType: SuccessFollowerRead, + followerRegionValid: true, + followerAsyncReload: None[bool](), + followerSuccessReplica: []string{"z2"}, + followerSuccessReadType: SuccessStaleRead, + }, + { + do: leaderDataIsNotReady, + leaderRegionValid: true, + leaderAsyncReload: Some(false), + leaderSuccessReplica: []string{"z1"}, + leaderSuccessReadType: SuccessLeaderRead, + followerRegionValid: true, + followerAsyncReload: Some(false), + followerSuccessReplica: []string{"z2"}, + followerSuccessReadType: SuccessStaleRead, + }, + { + do: followerDataIsNotReady, + leaderRegionValid: true, + leaderAsyncReload: Some(false), + leaderSuccessReplica: []string{"z1"}, + leaderSuccessReadType: SuccessStaleRead, + followerRegionValid: true, + followerAsyncReload: Some(false), + followerSuccessReplica: []string{"z1"}, + followerSuccessReadType: SuccessLeaderRead, + }, + { + debug: true, + do: leaderServerIsBusy, + recoverable: true, + leaderRegionValid: true, + leaderAsyncReload: Some(false), + leaderSuccessReplica: []string{"z2", "z3"}, + leaderSuccessReadType: SuccessFollowerRead, + followerRegionValid: true, + followerAsyncReload: Some(false), + followerSuccessReplica: []string{"z2"}, + followerSuccessReadType: SuccessStaleRead, + }, + { + do: followerServerIsBusy, + recoverable: true, + leaderRegionValid: true, + leaderAsyncReload: Some(false), + leaderSuccessReplica: []string{"z1"}, + leaderSuccessReadType: SuccessStaleRead, + followerRegionValid: true, + followerAsyncReload: Some(false), + followerSuccessReplica: []string{"z1"}, + followerSuccessReadType: SuccessLeaderRead, + }, + { + do: leaderDataIsNotReady, + extra: []func(suite *testRegionCacheStaleReadSuite){followerServerIsBusy}, + recoverable: true, + leaderRegionValid: true, + leaderAsyncReload: Some(false), + leaderSuccessReplica: []string{"z1"}, + leaderSuccessReadType: SuccessLeaderRead, + followerRegionValid: true, + followerAsyncReload: Some(false), + followerSuccessReplica: []string{"z1"}, + followerSuccessReadType: SuccessLeaderRead, + }, + { + do: leaderDataIsNotReady, + extra: []func(suite *testRegionCacheStaleReadSuite){followerDataIsNotReady}, + recoverable: true, + leaderRegionValid: true, + leaderAsyncReload: Some(false), + leaderSuccessReplica: []string{"z1"}, + leaderSuccessReadType: SuccessLeaderRead, + followerRegionValid: true, + followerAsyncReload: Some(false), + followerSuccessReplica: []string{"z1"}, + followerSuccessReadType: SuccessLeaderRead, + }, + { + do: leaderDataIsNotReady, + extra: []func(suite *testRegionCacheStaleReadSuite){followerDown}, + recoverable: true, + leaderRegionValid: true, + leaderAsyncReload: Some(false), + leaderSuccessReplica: []string{"z1"}, + leaderSuccessReadType: SuccessLeaderRead, + followerRegionValid: true, + followerAsyncReload: Some(false), + followerSuccessReplica: []string{"z1"}, + followerSuccessReadType: SuccessLeaderRead, + }, + { + do: leaderServerIsBusy, + extra: []func(suite *testRegionCacheStaleReadSuite){followerServerIsBusy}, + recoverable: true, + leaderRegionValid: true, + leaderAsyncReload: Some(false), + leaderSuccessReplica: []string{"z3"}, + leaderSuccessReadType: SuccessFollowerRead, + followerRegionValid: true, + followerAsyncReload: Some(false), + followerSuccessReplica: []string{"z3"}, + followerSuccessReadType: SuccessFollowerRead, + }, + { + do: leaderServerIsBusy, + extra: []func(suite *testRegionCacheStaleReadSuite){followerDataIsNotReady}, + recoverable: true, + leaderRegionValid: true, + leaderAsyncReload: Some(false), + leaderSuccessReplica: []string{"z2", "z3"}, + leaderSuccessReadType: SuccessFollowerRead, + followerRegionValid: true, + followerAsyncReload: Some(false), + followerSuccessReplica: []string{"z2", "z3"}, + followerSuccessReadType: SuccessFollowerRead, + }, + { + do: leaderServerIsBusy, + extra: []func(suite *testRegionCacheStaleReadSuite){followerDown}, + recoverable: true, + leaderRegionValid: true, + leaderAsyncReload: Some(false), + leaderSuccessReplica: []string{"z3"}, + leaderSuccessReadType: SuccessFollowerRead, + followerRegionValid: true, + followerAsyncReload: Some(false), + followerSuccessReplica: []string{"z3"}, + followerSuccessReadType: SuccessFollowerRead, + }, + { + do: leaderDown, + extra: []func(suite *testRegionCacheStaleReadSuite){followerDataIsNotReady}, + recoverable: true, + leaderRegionValid: true, + leaderAsyncReload: Some(true), + leaderSuccessReplica: []string{"z2", "z3"}, + leaderSuccessReadType: SuccessFollowerRead, + followerRegionValid: true, + followerAsyncReload: Some(true), + followerSuccessReplica: []string{"z2", "z3"}, + followerSuccessReadType: SuccessFollowerRead, + }, + { + do: leaderDown, + extra: []func(suite *testRegionCacheStaleReadSuite){followerServerIsBusy}, + recoverable: true, + leaderRegionValid: true, + leaderAsyncReload: Some(true), + leaderSuccessReplica: []string{"z3"}, + leaderSuccessReadType: SuccessFollowerRead, + followerRegionValid: true, + followerAsyncReload: Some(true), + followerSuccessReplica: []string{"z3"}, + followerSuccessReadType: SuccessFollowerRead, + }, + { + do: leaderDown, + extra: []func(suite *testRegionCacheStaleReadSuite){followerDown}, + recoverable: true, + leaderRegionValid: true, + leaderAsyncReload: Some(true), + leaderSuccessReplica: []string{"z3"}, + leaderSuccessReadType: SuccessFollowerRead, + followerRegionValid: true, + followerAsyncReload: Some(true), + followerSuccessReplica: []string{"z3"}, + followerSuccessReadType: SuccessFollowerRead, + }, + } + tests := []func(*testRegionCacheStaleReadSuite, *RegionCacheTestCase){ + testStaleReadFollower, testStaleReadLeader, + } + for _, regionCacheTestCase := range regionCacheTestCases { + for _, test := range tests { + s := &testRegionCacheStaleReadSuite{ + Assertions: require.New(t), + } + s.SetupTest() + _, err := s.cache.LocateRegionByID(s.bo, s.regionID) + s.Nil(err) + regionCacheTestCase.do(s) + for _, extra := range regionCacheTestCase.extra { + extra(s) + } + test(s, ®ionCacheTestCase) + s.TearDownTest() + } + } +} + +func testStaleReadFollower(s *testRegionCacheStaleReadSuite, r *RegionCacheTestCase) { + testStaleRead(s, r, "z2") +} + +func testStaleReadLeader(s *testRegionCacheStaleReadSuite, r *RegionCacheTestCase) { + testStaleRead(s, r, "z1") +} + +func testStaleRead(s *testRegionCacheStaleReadSuite, r *RegionCacheTestCase, zone string) { + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + leaderZone := zone == "z1" + var available bool + if leaderZone { + available = len(r.leaderSuccessReplica) > 0 + } else { + available = len(r.followerSuccessReplica) > 0 + } + + regionLoc, err := s.cache.LocateRegionByID(s.bo, s.regionID) + s.Nil(err) + s.NotNil(regionLoc) + + s.cache.mu.RLock() + region := s.cache.getRegionByIDFromCache(s.regionID) + s.cache.mu.RUnlock() + defer func() { + var ( + valid bool + asyncReload *bool + ) + if leaderZone { + valid = r.leaderRegionValid + asyncReload = r.leaderAsyncReload.Inner() + } else { + valid = r.followerRegionValid + asyncReload = r.followerAsyncReload.Inner() + } + s.Equal(valid, region.isValid()) + + if asyncReload == nil { + return + } + + s.cache.regionsNeedReload.Lock() + if *asyncReload { + s.Len(s.cache.regionsNeedReload.regions, 1) + s.Equal(s.cache.regionsNeedReload.regions[0], s.regionID) + } else { + s.Empty(s.cache.regionsNeedReload.regions) + } + s.cache.regionsNeedReload.Unlock() + }() + + bo := retry.NewBackoffer(ctx, -1) + req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{Key: []byte("key")}, kv.ReplicaReadMixed, nil) + req.EnableStaleRead() + ops := []StoreSelectorOption{WithMatchLabels([]*metapb.StoreLabel{{ + Key: "zone", + Value: zone, + }})} + + resp, _, _, err := s.regionRequestSender.SendReqCtx(bo, req, regionLoc.Region, time.Second, tikvrpc.TiKV, ops...) + if !available { + if err != nil { + return + } + regionErr, err := resp.GetRegionError() + s.Nil(err) + s.NotNil(regionErr) + return + } + + _, successZone, successReadType := s.extractResp(resp) + find := false + if leaderZone { + s.Equal(r.leaderSuccessReadType, successReadType) + for _, z := range r.leaderSuccessReplica { + if z == successZone { + find = true + break + } + } + } else { + s.Equal(r.followerSuccessReadType, successReadType) + for _, z := range r.followerSuccessReplica { + if z == successZone { + find = true + break + } + } + } + s.True(find) +} + +type Option[T interface{}] struct { + inner *T +} + +func Some[T interface{}](inner T) Option[T] { + return Option[T]{inner: &inner} +} + +func None[T interface{}]() Option[T] { + return Option[T]{inner: nil} +} + +func (o Option[T]) Inner() *T { + return o.inner +} + +type RegionCacheTestCase struct { + debug bool + do func(s *testRegionCacheStaleReadSuite) + extra []func(s *testRegionCacheStaleReadSuite) + recoverable bool + // local peer is leader + leaderRegionValid bool + leaderAsyncReload Option[bool] + leaderSuccessReplica []string + leaderSuccessReadType SuccessReadType + // local peer is follower + followerRegionValid bool + followerAsyncReload Option[bool] + followerSuccessReplica []string + followerSuccessReadType SuccessReadType +} + +func followerDown(s *testRegionCacheStaleReadSuite) { + _, follower := s.getFollower() + s.NotNil(follower) + s.setUnavailableStore(follower.Id) +} + +func followerDownAndUp(s *testRegionCacheStaleReadSuite) { + s.cache.mu.RLock() + cachedRegion := s.cache.getRegionByIDFromCache(s.regionID) + s.cache.mu.RUnlock() + _, follower := s.getFollower() + s.NotNil(cachedRegion) + s.NotNil(follower) + regionStore := cachedRegion.getStore() + for _, storeIdx := range regionStore.accessIndex[tiKVOnly] { + if regionStore.stores[storeIdx].storeID == follower.Id { + atomic.AddUint32(®ionStore.stores[storeIdx].epoch, 1) + } + } +} + +func followerMove(s *testRegionCacheStaleReadSuite) { + peerID, follower := s.getFollower() + zone := "" + for _, label := range follower.Labels { + if label.Key == "zone" { + zone = label.Value + break + } + } + s.NotEqual("", zone) + var target *metapb.Store +FIND: + for _, store := range s.cluster.GetAllStores() { + if store.Id == follower.Id { + continue + } + for _, label := range store.Labels { + if label.Key == "zone" && label.Value == zone { + target = store + break FIND + } + } + } + s.NotNil(target) + s.cluster.RemovePeer(s.regionID, peerID) + s.cluster.AddPeer(s.regionID, target.Id, peerID) +} + +func evictLeader(s *testRegionCacheStaleReadSuite) { + region, leader := s.cluster.GetRegion(s.regionID) + for _, peer := range region.Peers { + if peer.Id != leader { + s.cluster.ChangeLeader(s.regionID, peer.Id) + return + } + } + s.Fail("unreachable") +} + +func leaderMove(s *testRegionCacheStaleReadSuite) { + peerID, leader := s.getLeader() + zone := "" + for _, label := range leader.Labels { + if label.Key == "zone" { + zone = label.Value + break + } + } + s.NotEqual("", zone) + var target *metapb.Store +FIND: + for _, store := range s.cluster.GetAllStores() { + if store.Id == leader.Id { + continue + } + for _, label := range store.Labels { + if label.Key == "zone" && label.Value == zone { + target = store + break FIND + } + } + } + s.NotNil(target) + s.cluster.RemovePeer(s.regionID, peerID) + s.cluster.AddPeer(s.regionID, target.Id, peerID) + s.cluster.ChangeLeader(s.regionID, peerID) +} + +func leaderDown(s *testRegionCacheStaleReadSuite) { + _, leader := s.getLeader() + s.NotNil(leader) + s.setUnavailableStore(leader.Id) +} + +func leaderDownAndUp(s *testRegionCacheStaleReadSuite) { + s.cache.mu.RLock() + cachedRegion := s.cache.getRegionByIDFromCache(s.regionID) + s.cache.mu.RUnlock() + _, leader := s.getLeader() + s.NotNil(cachedRegion) + s.NotNil(leader) + regionStore := cachedRegion.getStore() + for _, storeIdx := range regionStore.accessIndex[tiKVOnly] { + if regionStore.stores[storeIdx].storeID == leader.Id { + atomic.AddUint32(®ionStore.stores[storeIdx].epoch, 1) + } + } +} +func leaderDownAndElect(s *testRegionCacheStaleReadSuite) { + _, leader := s.getLeader() + s.NotNil(leader) + leaderMove(s) + s.setUnavailableStore(leader.Id) +} + +func leaderDataIsNotReady(s *testRegionCacheStaleReadSuite) { + peerID, _ := s.getLeader() + s.injection.leaderRegionError = func(req *tikvrpc.Request, zone string) *errorpb.Error { + if !req.StaleRead || zone != "z1" { + return nil + } + return &errorpb.Error{ + DataIsNotReady: &errorpb.DataIsNotReady{ + RegionId: s.regionID, + PeerId: peerID, + SafeTs: 0, + }, + } + } +} + +func leaderServerIsBusy(s *testRegionCacheStaleReadSuite) { + s.injection.leaderRegionError = func(req *tikvrpc.Request, zone string) *errorpb.Error { + if zone != "z1" { + return nil + } + return &errorpb.Error{ + ServerIsBusy: &errorpb.ServerIsBusy{ + Reason: "test", + BackoffMs: 1, + }, + } + } +} + +func followerDataIsNotReady(s *testRegionCacheStaleReadSuite) { + s.injection.followerRegionError = func(req *tikvrpc.Request, zone string) *errorpb.Error { + if !req.StaleRead || zone != "z2" { + return nil + } + return &errorpb.Error{ + DataIsNotReady: &errorpb.DataIsNotReady{ + RegionId: s.regionID, + SafeTs: 0, + }, + } + } +} + +func followerServerIsBusy(s *testRegionCacheStaleReadSuite) { + s.injection.followerRegionError = func(req *tikvrpc.Request, zone string) *errorpb.Error { + if zone != "z2" { + return nil + } + return &errorpb.Error{ + ServerIsBusy: &errorpb.ServerIsBusy{ + Reason: "test", + BackoffMs: 1, + }, + } + } +} diff --git a/internal/mockstore/mocktikv/cluster.go b/internal/mockstore/mocktikv/cluster.go index 6a65ab68b5..fc8af00d2c 100644 --- a/internal/mockstore/mocktikv/cluster.go +++ b/internal/mockstore/mocktikv/cluster.go @@ -393,11 +393,11 @@ func (c *Cluster) AddPeer(regionID, storeID, peerID uint64) { // RemovePeer removes the Peer from the Region. Note that if the Peer is leader, // the Region will have no leader before calling ChangeLeader(). -func (c *Cluster) RemovePeer(regionID, storeID uint64) { +func (c *Cluster) RemovePeer(regionID, peerID uint64) { c.Lock() defer c.Unlock() - c.regions[regionID].removePeer(storeID) + c.regions[regionID].removePeer(peerID) } // ChangeLeader sets the Region's leader Peer. Caller should guarantee the Peer diff --git a/internal/mockstore/mocktikv/cluster_manipulate.go b/internal/mockstore/mocktikv/cluster_manipulate.go index ae02beeb50..d688cde4be 100644 --- a/internal/mockstore/mocktikv/cluster_manipulate.go +++ b/internal/mockstore/mocktikv/cluster_manipulate.go @@ -80,3 +80,29 @@ func BootstrapWithMultiRegions(cluster *Cluster, splitKeys ...[]byte) (storeID u } return } + +// BootstrapWithMultiZones initializes a Cluster with 1 Region and n Zones and m Stores in each Zone. +func BootstrapWithMultiZones(cluster *Cluster, n, m int) (storeIDs, peerIDs []uint64, regionID uint64, leaderPeer uint64, store2zone map[uint64]string) { + storeIDs = cluster.AllocIDs(n * m) + peerIDs = cluster.AllocIDs(n) + leaderPeer = peerIDs[0] + regionID = cluster.AllocID() + store2zone = make(map[uint64]string, n*m) + for id, storeID := range storeIDs { + zone := fmt.Sprintf("z%d", (id%n)+1) + store2zone[storeID] = zone + labels := []*metapb.StoreLabel{ + { + Key: "id", + Value: fmt.Sprintf("%v", storeID), + }, + { + Key: "zone", + Value: zone, + }, + } + cluster.AddStore(storeID, fmt.Sprintf("store%d", storeID), labels...) + } + cluster.Bootstrap(regionID, storeIDs[:n], peerIDs, leaderPeer) + return +} From 001735b0b5f4e209f5da19422804a40062250f08 Mon Sep 17 00:00:00 2001 From: crazycs Date: Fri, 15 Sep 2023 12:53:37 +0800 Subject: [PATCH 37/49] *: try follower when stale read request timeout (#969) * fix stalre read retry logic when got timeout error Signed-off-by: crazycs520 * add test Signed-off-by: crazycs520 * refine Signed-off-by: crazycs520 * fix test Signed-off-by: crazycs520 * add log Signed-off-by: crazycs520 * add log Signed-off-by: crazycs520 * refine Signed-off-by: crazycs520 --------- Signed-off-by: crazycs520 --- internal/locate/region_request.go | 11 +++--- internal/locate/region_request3_test.go | 51 +++++++++++++++++++++---- 2 files changed, 48 insertions(+), 14 deletions(-) diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index 33d56cba2b..5b52308cb5 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -735,17 +735,15 @@ func (state *accessFollower) next(bo *retry.Backoffer, selector *replicaSelector logutil.Logger(bo.GetCtx()).Warn("unable to find stores with given labels", zap.Uint64("region", selector.region.GetID()), zap.Bool("leader-invalid", leaderInvalid), + zap.Bool("stale-read", state.isStaleRead), zap.Any("labels", state.option.labels)) } - // If leader tried and received deadline exceeded error, return nil to upper layer to retry with default timeout. - if leader.deadlineErrUsingConfTimeout { - return nil, nil - } - if leaderInvalid { + if leaderInvalid || leader.deadlineErrUsingConfTimeout { // In stale-read, the request will fallback to leader after the local follower failure. // If the leader is also unavailable, we can fallback to the follower and use replica-read flag again, // The remote follower not tried yet, and the local follower can retry without stale-read flag. - if state.isStaleRead { + // If leader tried and received deadline exceeded error, try follower. + if state.isStaleRead || leader.deadlineErrUsingConfTimeout { selector.state = &tryFollower{ leaderIdx: state.leaderIdx, lastIdx: state.leaderIdx, @@ -1519,6 +1517,7 @@ func (s *RegionRequestSender) logSendReqError(bo *retry.Backoffer, msg string, r zap.Int("retry-times", retryTimes), zap.Int("total-backoff-ms", bo.GetTotalSleep()), zap.Int("total-backoff-times", bo.GetTotalBackoffTimes()), + zap.Uint64("max-exec-timeout-ms", req.Context.MaxExecutionDurationMs), zap.String("total-region-errors", totalErrorStr.String())) } diff --git a/internal/locate/region_request3_test.go b/internal/locate/region_request3_test.go index 6d14fefd5b..a079e2c673 100644 --- a/internal/locate/region_request3_test.go +++ b/internal/locate/region_request3_test.go @@ -1255,14 +1255,9 @@ func (s *testRegionRequestToThreeStoresSuite) TestSendReqFirstTimeout() { s.Nil(err) s.True(IsFakeRegionError(regionErr)) s.Equal(1, len(s.regionRequestSender.Stats)) - if staleRead { - rpcNum := s.regionRequestSender.Stats[tikvrpc.CmdGet].Count - s.True(rpcNum == 1 || rpcNum == 2) // 1 rpc or 2 rpc - } else { - s.Equal(int64(3), s.regionRequestSender.Stats[tikvrpc.CmdGet].Count) // 3 rpc - s.Equal(3, len(reqTargetAddrs)) // each rpc to a different store. - } - s.Equal(0, bo.GetTotalBackoffTimes()) // no backoff since fast retry. + s.Equal(int64(3), s.regionRequestSender.Stats[tikvrpc.CmdGet].Count) // 3 rpc + s.Equal(3, len(reqTargetAddrs)) // each rpc to a different store. + s.Equal(0, bo.GetTotalBackoffTimes()) // no backoff since fast retry. // warn: must rest MaxExecutionDurationMs before retry. resetStats() if staleRead { @@ -1530,3 +1525,43 @@ func (s *testRegionRequestToThreeStoresSuite) TestRetryRequestSource() { } } } + +func (s *testRegionRequestToThreeStoresSuite) TestStaleReadTryFollowerAfterTimeout() { + var ( + leaderAddr string + leaderLabel []*metapb.StoreLabel + ) + bo := retry.NewBackoffer(context.Background(), 10000) + mockRPCClient := &fnClient{fn: func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { + if addr == leaderAddr { + return nil, context.DeadlineExceeded + } + return &tikvrpc.Response{Resp: &kvrpcpb.GetResponse{Value: []byte("value")}}, nil + }} + s.regionRequestSender = NewRegionRequestSender(s.cache, mockRPCClient) + s.regionRequestSender.RegionRequestRuntimeStats = NewRegionRequestRuntimeStats() + getLocFn := func() *KeyLocation { + loc, err := s.regionRequestSender.regionCache.LocateKey(bo, []byte("a")) + s.Nil(err) + region := s.regionRequestSender.regionCache.GetCachedRegionWithRLock(loc.Region) + leaderStore, _, _, _ := region.WorkStorePeer(region.getStore()) + leaderAddr, err = s.regionRequestSender.regionCache.getStoreAddr(s.bo, region, leaderStore) + s.Nil(err) + leaderLabel = []*metapb.StoreLabel{{Key: "id", Value: strconv.FormatUint(leaderStore.StoreID(), 10)}} + return loc + } + req := tikvrpc.NewRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{Key: []byte("a")}, kvrpcpb.Context{}) + req.EnableStaleRead() + loc := getLocFn() + var ops []StoreSelectorOption + ops = append(ops, WithMatchLabels(leaderLabel)) + resp, _, _, err := s.regionRequestSender.SendReqCtx(bo, req, loc.Region, time.Second, tikvrpc.TiKV, ops...) + s.Nil(err) + regionErr, err := resp.GetRegionError() + s.Nil(err) + s.Nil(regionErr) + s.Equal([]byte("value"), resp.Resp.(*kvrpcpb.GetResponse).Value) + s.Equal(1, len(s.regionRequestSender.Stats)) + s.Equal(int64(2), s.regionRequestSender.Stats[tikvrpc.CmdGet].Count) // 2 rpc + s.Equal(0, bo.GetTotalBackoffTimes()) // no backoff since fast retry. +} From 342301689f07f3674e387fcbf1174cc05197ef4d Mon Sep 17 00:00:00 2001 From: crazycs Date: Fri, 15 Sep 2023 23:49:42 +0800 Subject: [PATCH 38/49] *: fix issue of configure kv timeout not work when disable batch client (#980) * *: fix issue of configure kv timeout not work when disable batch client Signed-off-by: crazycs520 * refine test Signed-off-by: crazycs520 * address comment Signed-off-by: crazycs520 --------- Signed-off-by: crazycs520 --- internal/client/client_fail_test.go | 10 ++--- internal/client/client_test.go | 23 ++++++------ .../mock_tikv_service.go} | 37 ++++++++++++------- internal/locate/region_request.go | 8 +++- internal/locate/region_request_test.go | 34 +++++++++++++++++ 5 files changed, 82 insertions(+), 30 deletions(-) rename internal/client/{mock_tikv_service_test.go => mock_server/mock_tikv_service.go} (77%) diff --git a/internal/client/client_fail_test.go b/internal/client/client_fail_test.go index 32d976efec..c63c72272c 100644 --- a/internal/client/client_fail_test.go +++ b/internal/client/client_fail_test.go @@ -36,7 +36,6 @@ package client import ( "context" - "fmt" "sync/atomic" "testing" "time" @@ -47,6 +46,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/config" + "github.com/tikv/client-go/v2/internal/client/mock_server" "github.com/tikv/client-go/v2/tikvrpc" ) @@ -54,11 +54,11 @@ func TestPanicInRecvLoop(t *testing.T) { require.Nil(t, failpoint.Enable("tikvclient/panicInFailPendingRequests", `panic`)) require.Nil(t, failpoint.Enable("tikvclient/gotErrorInRecvLoop", `return("0")`)) - server, port := startMockTikvService() + server, port := mock_server.StartMockTikvService() require.True(t, port > 0) defer server.Stop() - addr := fmt.Sprintf("%s:%d", "127.0.0.1", port) + addr := server.Addr() rpcClient := NewRPCClient() defer rpcClient.Close() rpcClient.option.dialTimeout = time.Second / 3 @@ -81,10 +81,10 @@ func TestPanicInRecvLoop(t *testing.T) { } func TestRecvErrorInMultipleRecvLoops(t *testing.T) { - server, port := startMockTikvService() + server, port := mock_server.StartMockTikvService() require.True(t, port > 0) defer server.Stop() - addr := fmt.Sprintf("%s:%d", "127.0.0.1", port) + addr := server.Addr() // Enable batch and limit the connection count to 1 so that // there is only one BatchCommands stream for each host or forwarded host. diff --git a/internal/client/client_test.go b/internal/client/client_test.go index e872643f0c..2934c1241c 100644 --- a/internal/client/client_test.go +++ b/internal/client/client_test.go @@ -55,6 +55,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/config" + "github.com/tikv/client-go/v2/internal/client/mock_server" "github.com/tikv/client-go/v2/internal/logutil" "github.com/tikv/client-go/v2/tikvrpc" "go.uber.org/zap" @@ -118,12 +119,12 @@ func TestCancelTimeoutRetErr(t *testing.T) { } func TestSendWhenReconnect(t *testing.T) { - server, port := startMockTikvService() + server, port := mock_server.StartMockTikvService() require.True(t, port > 0) rpcClient := NewRPCClient() defer rpcClient.Close() - addr := fmt.Sprintf("%s:%d", "127.0.0.1", port) + addr := server.Addr() conn, err := rpcClient.getConnArray(addr, true) assert.Nil(t, err) @@ -242,7 +243,7 @@ func TestCollapseResolveLock(t *testing.T) { } func TestForwardMetadataByUnaryCall(t *testing.T) { - server, port := startMockTikvService() + server, port := mock_server.StartMockTikvService() require.True(t, port > 0) defer server.Stop() addr := fmt.Sprintf("%s:%d", "127.0.0.1", port) @@ -257,7 +258,7 @@ func TestForwardMetadataByUnaryCall(t *testing.T) { var checkCnt uint64 // Check no corresponding metadata if ForwardedHost is empty. - server.setMetaChecker(func(ctx context.Context) error { + server.SetMetaChecker(func(ctx context.Context) error { atomic.AddUint64(&checkCnt, 1) // gRPC may set some metadata by default, e.g. "context-type". md, ok := metadata.FromIncomingContext(ctx) @@ -285,7 +286,7 @@ func TestForwardMetadataByUnaryCall(t *testing.T) { checkCnt = 0 forwardedHost := "127.0.0.1:6666" // Check the metadata exists. - server.setMetaChecker(func(ctx context.Context) error { + server.SetMetaChecker(func(ctx context.Context) error { atomic.AddUint64(&checkCnt, 1) // gRPC may set some metadata by default, e.g. "context-type". md, ok := metadata.FromIncomingContext(ctx) @@ -310,10 +311,10 @@ func TestForwardMetadataByUnaryCall(t *testing.T) { } func TestForwardMetadataByBatchCommands(t *testing.T) { - server, port := startMockTikvService() + server, port := mock_server.StartMockTikvService() require.True(t, port > 0) defer server.Stop() - addr := fmt.Sprintf("%s:%d", "127.0.0.1", port) + addr := server.Addr() // Enable batch and limit the connection count to 1 so that // there is only one BatchCommands stream for each host or forwarded host. @@ -326,7 +327,7 @@ func TestForwardMetadataByBatchCommands(t *testing.T) { var checkCnt uint64 setCheckHandler := func(forwardedHost string) { - server.setMetaChecker(func(ctx context.Context) error { + server.SetMetaChecker(func(ctx context.Context) error { atomic.AddUint64(&checkCnt, 1) md, ok := metadata.FromIncomingContext(ctx) if forwardedHost == "" { @@ -649,10 +650,10 @@ func TestBatchClientRecoverAfterServerRestart(t *testing.T) { conf.TiKVClient.MaxBatchSize = 128 })() - server, port := startMockTikvService() + server, port := mock_server.StartMockTikvService() require.True(t, port > 0) require.True(t, server.IsRunning()) - addr := server.addr + addr := server.Addr() client := NewRPCClient() defer func() { err := client.Close() @@ -689,7 +690,7 @@ func TestBatchClientRecoverAfterServerRestart(t *testing.T) { logutil.BgLogger().Info("restart mock tikv server") server.Start(addr) require.True(t, server.IsRunning()) - require.Equal(t, addr, server.addr) + require.Equal(t, addr, server.Addr()) // Wait batch client to auto reconnect. start := time.Now() diff --git a/internal/client/mock_tikv_service_test.go b/internal/client/mock_server/mock_tikv_service.go similarity index 77% rename from internal/client/mock_tikv_service_test.go rename to internal/client/mock_server/mock_tikv_service.go index 40234c1d58..392d3a5fbd 100644 --- a/internal/client/mock_tikv_service_test.go +++ b/internal/client/mock_server/mock_tikv_service.go @@ -18,7 +18,7 @@ // https://github.com/pingcap/tidb/tree/cc5e161ac06827589c4966674597c137cc9e809c/store/tikv/client/mock_tikv_service_test.go // -package client +package mock_server import ( "context" @@ -36,7 +36,7 @@ import ( "google.golang.org/grpc" ) -type server struct { +type MockServer struct { tikvpb.TikvServer grpcServer *grpc.Server addr string @@ -49,21 +49,28 @@ type server struct { } } -func (s *server) KvPrewrite(ctx context.Context, req *kvrpcpb.PrewriteRequest) (*kvrpcpb.PrewriteResponse, error) { +func (s *MockServer) KvGet(ctx context.Context, req *kvrpcpb.GetRequest) (*kvrpcpb.GetResponse, error) { + if err := s.checkMetadata(ctx); err != nil { + return nil, err + } + return &kvrpcpb.GetResponse{}, nil +} + +func (s *MockServer) KvPrewrite(ctx context.Context, req *kvrpcpb.PrewriteRequest) (*kvrpcpb.PrewriteResponse, error) { if err := s.checkMetadata(ctx); err != nil { return nil, err } return &kvrpcpb.PrewriteResponse{}, nil } -func (s *server) CoprocessorStream(req *coprocessor.Request, ss tikvpb.Tikv_CoprocessorStreamServer) error { +func (s *MockServer) CoprocessorStream(req *coprocessor.Request, ss tikvpb.Tikv_CoprocessorStreamServer) error { if err := s.checkMetadata(ss.Context()); err != nil { return err } return ss.Send(&coprocessor.Response{}) } -func (s *server) BatchCommands(ss tikvpb.Tikv_BatchCommandsServer) error { +func (s *MockServer) BatchCommands(ss tikvpb.Tikv_BatchCommandsServer) error { if err := s.checkMetadata(ss.Context()); err != nil { return err } @@ -94,13 +101,13 @@ func (s *server) BatchCommands(ss tikvpb.Tikv_BatchCommandsServer) error { } } -func (s *server) setMetaChecker(check func(context.Context) error) { +func (s *MockServer) SetMetaChecker(check func(context.Context) error) { s.metaChecker.Lock() s.metaChecker.check = check s.metaChecker.Unlock() } -func (s *server) checkMetadata(ctx context.Context) error { +func (s *MockServer) checkMetadata(ctx context.Context) error { s.metaChecker.Lock() defer s.metaChecker.Unlock() if s.metaChecker.check != nil { @@ -109,16 +116,20 @@ func (s *server) checkMetadata(ctx context.Context) error { return nil } -func (s *server) IsRunning() bool { +func (s *MockServer) IsRunning() bool { return atomic.LoadInt64(&s.running) == 1 } -func (s *server) Stop() { +func (s *MockServer) Addr() string { + return s.addr +} + +func (s *MockServer) Stop() { s.grpcServer.Stop() atomic.StoreInt64(&s.running, 0) } -func (s *server) Start(addr string) int { +func (s *MockServer) Start(addr string) int { if addr == "" { addr = fmt.Sprintf("%s:%d", "127.0.0.1", 0) } @@ -148,9 +159,9 @@ func (s *server) Start(addr string) int { return port } -// Try to start a gRPC server and retrun the server instance and binded port. -func startMockTikvService() (*server, int) { - server := &server{} +// StartMockTikvService try to start a gRPC server and retrun the server instance and binded port. +func StartMockTikvService() (*MockServer, int) { + server := &MockServer{} port := server.Start("") return server, port } diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index 5b52308cb5..f260ad18df 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -1776,7 +1776,7 @@ func (s *RegionRequestSender) onSendFail(bo *retry.Backoffer, ctx *RPCContext, r return errors.WithStack(err) } else if LoadShuttingDown() > 0 { return errors.WithStack(tikverr.ErrTiDBShuttingDown) - } else if errors.Cause(err) == context.DeadlineExceeded && req.MaxExecutionDurationMs < uint64(client.ReadTimeoutShort.Milliseconds()) { + } else if isCauseByDeadlineExceeded(err) && req.MaxExecutionDurationMs < uint64(client.ReadTimeoutShort.Milliseconds()) { if s.replicaSelector != nil { s.replicaSelector.onDeadlineExceeded() return nil @@ -1834,6 +1834,12 @@ func (s *RegionRequestSender) onSendFail(bo *retry.Backoffer, ctx *RPCContext, r return err } +func isCauseByDeadlineExceeded(err error) bool { + causeErr := errors.Cause(err) + return causeErr == context.DeadlineExceeded || // batch-client will return this error. + status.Code(causeErr) == codes.DeadlineExceeded // when batch-client is disabled, grpc will return this error. +} + // NeedReloadRegion checks is all peers has sent failed, if so need reload. func (s *RegionRequestSender) NeedReloadRegion(ctx *RPCContext) (need bool) { if s.failStoreIDs == nil { diff --git a/internal/locate/region_request_test.go b/internal/locate/region_request_test.go index 4336d4e003..2923648fb6 100644 --- a/internal/locate/region_request_test.go +++ b/internal/locate/region_request_test.go @@ -52,8 +52,10 @@ import ( "github.com/pingcap/kvproto/pkg/tikvpb" "github.com/pkg/errors" "github.com/stretchr/testify/suite" + "github.com/tikv/client-go/v2/config" "github.com/tikv/client-go/v2/internal/apicodec" "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/mockstore/mocktikv" "github.com/tikv/client-go/v2/internal/retry" "github.com/tikv/client-go/v2/tikvrpc" @@ -697,3 +699,35 @@ func (s *testRegionRequestToSingleStoreSuite) TestStaleReadRetry() { s.Nil(regionErr) s.Equal([]byte("value"), resp.Resp.(*kvrpcpb.GetResponse).Value) } + +func (s *testRegionRequestToSingleStoreSuite) TestKVReadTimeoutWithDisableBatchClient() { + config.UpdateGlobal(func(conf *config.Config) { + conf.TiKVClient.MaxBatchSize = 0 + })() + + server, port := mock_server.StartMockTikvService() + s.True(port > 0) + server.SetMetaChecker(func(ctx context.Context) error { + return context.DeadlineExceeded + }) + rpcClient := client.NewRPCClient() + s.regionRequestSender.client = &fnClient{fn: func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (response *tikvrpc.Response, err error) { + return rpcClient.SendRequest(ctx, server.Addr(), req, timeout) + }} + defer func() { + rpcClient.Close() + server.Stop() + }() + + bo := retry.NewBackofferWithVars(context.Background(), 2000, nil) + region, err := s.cache.LocateRegionByID(bo, s.region) + s.Nil(err) + s.NotNil(region) + req := tikvrpc.NewRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{Key: []byte("a"), Version: 1}) + resp, _, err := s.regionRequestSender.SendReq(bo, req, region.Region, time.Millisecond*10) + s.Nil(err) + s.NotNil(resp) + regionErr, _ := resp.GetRegionError() + s.True(IsFakeRegionError(regionErr)) + s.Equal(0, bo.GetTotalBackoffTimes()) // use kv read timeout will do fast retry, so backoff times should be 0. +} From 39084386b3246ab9e478879dc557a6fbf8da05f6 Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 18 Sep 2023 11:17:51 +0800 Subject: [PATCH 39/49] *: configurable kv_read_timeout should not affect write request (#978) * *: configurable kv_read_timeout should not affect write request Signed-off-by: crazycs520 * refine test Signed-off-by: crazycs520 * refine logic Signed-off-by: crazycs520 --------- Signed-off-by: crazycs520 --- internal/locate/region_request.go | 38 ++++++++++++++++--------- internal/locate/region_request3_test.go | 16 +++++++++++ 2 files changed, 41 insertions(+), 13 deletions(-) diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index f260ad18df..9d823e4f04 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -1085,13 +1085,25 @@ func (s *replicaSelector) onSendFailure(bo *retry.Backoffer, err error) { s.state.onSendFailure(bo, s, err) } -func (s *replicaSelector) onDeadlineExceeded() { - if target := s.targetReplica(); target != nil { - target.deadlineErrUsingConfTimeout = true +func (s *replicaSelector) onReadReqConfigurableTimeout(req *tikvrpc.Request) bool { + if req.MaxExecutionDurationMs >= uint64(client.ReadTimeoutShort.Milliseconds()) { + // Configurable timeout should less than `ReadTimeoutShort`. + return false } - if accessLeader, ok := s.state.(*accessKnownLeader); ok { - // If leader return deadline exceeded error, we should try to access follower next time. - s.state = &tryFollower{leaderIdx: accessLeader.leaderIdx, lastIdx: accessLeader.leaderIdx} + switch req.Type { + case tikvrpc.CmdGet, tikvrpc.CmdBatchGet, tikvrpc.CmdScan, + tikvrpc.CmdCop, tikvrpc.CmdBatchCop, tikvrpc.CmdCopStream: + if target := s.targetReplica(); target != nil { + target.deadlineErrUsingConfTimeout = true + } + if accessLeader, ok := s.state.(*accessKnownLeader); ok { + // If leader return deadline exceeded error, we should try to access follower next time. + s.state = &tryFollower{leaderIdx: accessLeader.leaderIdx, lastIdx: accessLeader.leaderIdx} + } + return true + default: + // Only work for read requests, return false for non-read requests. + return false } } @@ -1776,9 +1788,8 @@ func (s *RegionRequestSender) onSendFail(bo *retry.Backoffer, ctx *RPCContext, r return errors.WithStack(err) } else if LoadShuttingDown() > 0 { return errors.WithStack(tikverr.ErrTiDBShuttingDown) - } else if isCauseByDeadlineExceeded(err) && req.MaxExecutionDurationMs < uint64(client.ReadTimeoutShort.Milliseconds()) { - if s.replicaSelector != nil { - s.replicaSelector.onDeadlineExceeded() + } else if isCauseByDeadlineExceeded(err) { + if s.replicaSelector != nil && s.replicaSelector.onReadReqConfigurableTimeout(req) { return nil } } @@ -2097,8 +2108,9 @@ func (s *RegionRequestSender) onRegionError( if serverIsBusy := regionErr.GetServerIsBusy(); serverIsBusy != nil { if s.replicaSelector != nil && strings.Contains(serverIsBusy.GetReason(), "deadline is exceeded") { - s.replicaSelector.onDeadlineExceeded() - return true, nil + if s.replicaSelector.onReadReqConfigurableTimeout(req) { + return true, nil + } } if s.replicaSelector != nil { return s.replicaSelector.onServerIsBusy(bo, ctx, req, serverIsBusy) @@ -2228,8 +2240,8 @@ func (s *RegionRequestSender) onRegionError( return true, nil } - if isDeadlineExceeded(regionErr) && s.replicaSelector != nil { - s.replicaSelector.onDeadlineExceeded() + if isDeadlineExceeded(regionErr) && s.replicaSelector != nil && s.replicaSelector.onReadReqConfigurableTimeout(req) { + return true, nil } if mismatch := regionErr.GetMismatchPeerId(); mismatch != nil { diff --git a/internal/locate/region_request3_test.go b/internal/locate/region_request3_test.go index a079e2c673..41dd69b6b3 100644 --- a/internal/locate/region_request3_test.go +++ b/internal/locate/region_request3_test.go @@ -1278,6 +1278,22 @@ func (s *testRegionRequestToThreeStoresSuite) TestSendReqFirstTimeout() { s.Equal(0, bo.GetTotalBackoffTimes()) // no backoff since fast retry. } } + + // Test for write request. + tf := func(s *Store, bo *retry.Backoffer) livenessState { + return reachable + } + s.regionRequestSender.regionCache.testingKnobs.mockRequestLiveness.Store((*livenessFunc)(&tf)) + resetStats() + req := tikvrpc.NewRequest(tikvrpc.CmdPrewrite, &kvrpcpb.PrewriteRequest{}, kvrpcpb.Context{}) + req.ReplicaReadType = kv.ReplicaReadLeader + loc := getLocFn() + bo = retry.NewBackoffer(context.Background(), 1000) + resp, _, _, err := s.regionRequestSender.SendReqCtx(bo, req, loc.Region, time.Millisecond, tikvrpc.TiKV) + s.Nil(resp) + s.Equal(context.DeadlineExceeded, err) + backoffTimes := bo.GetBackoffTimes() + s.True(backoffTimes["tikvRPC"] > 0) // write request timeout won't do fast retry, so backoff times should be more than 0. } func (s *testRegionRequestToThreeStoresSuite) TestStaleReadFallback2Follower() { From adb7db13c3e6f94256831fe0fe80f4095d93d12a Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 18 Sep 2023 11:39:42 +0800 Subject: [PATCH 40/49] *: fix batch-client wait too long and add some metrics (#973) Signed-off-by: crazycs520 Co-authored-by: disksing --- internal/client/client.go | 2 +- internal/client/client_batch.go | 42 ++++++++++++++++++++++++++++++--- metrics/metrics.go | 26 ++++++++++++++------ 3 files changed, 59 insertions(+), 11 deletions(-) diff --git a/internal/client/client.go b/internal/client/client.go index 98d847a909..3bf9b0ebf0 100644 --- a/internal/client/client.go +++ b/internal/client/client.go @@ -620,7 +620,7 @@ func (c *RPCClient) sendRequest(ctx context.Context, addr string, req *tikvrpc.R // TiDB RPC server supports batch RPC, but batch connection will send heart beat, It's not necessary since // request to TiDB is not high frequency. - if config.GetGlobalConfig().TiKVClient.MaxBatchSize > 0 && enableBatch { + if config.GetGlobalConfig().TiKVClient.MaxBatchSize > 0 && enableBatch && !connArray.batchConn.isBusy(start.UnixNano()) { if batchReq := req.ToBatchCommandsRequest(); batchReq != nil { defer trace.StartRegion(ctx, req.Type.String()).End() return sendBatchRequest(ctx, addr, req.ForwardedHost, connArray.batchConn, batchReq, timeout) diff --git a/internal/client/client_batch.go b/internal/client/client_batch.go index b7159847b9..eb58cb0333 100644 --- a/internal/client/client_batch.go +++ b/internal/client/client_batch.go @@ -70,6 +70,7 @@ type batchCommandsEntry struct { // canceled indicated the request is canceled or not. canceled int32 err error + start time.Time } func (b *batchCommandsEntry) isCanceled() bool { @@ -197,9 +198,16 @@ type batchConn struct { pendingRequests prometheus.Observer batchSize prometheus.Observer - index uint32 + index uint32 + state atomic.Int32 + startHandingTime atomic.Int64 } +var ( + batchConnIdle = int32(0) + batchConnHanding = int32(1) +) + func newBatchConn(connCount, maxBatchSize uint, idleNotify *uint32) *batchConn { return &batchConn{ batchCommandsCh: make(chan *batchCommandsEntry, maxBatchSize), @@ -216,6 +224,16 @@ func (a *batchConn) isIdle() bool { return atomic.LoadUint32(&a.idle) != 0 } +func (a *batchConn) isBusy(now int64) bool { + if len(a.batchCommandsCh) == cap(a.batchCommandsCh) { + return true + } + if a.state.Load() == batchConnHanding && (now-a.startHandingTime.Load()) > int64(time.Second) { + return true + } + return false +} + // fetchAllPendingRequests fetches all pending requests from the channel. func (a *batchConn) fetchAllPendingRequests( maxBatchSize int, @@ -311,6 +329,7 @@ func (a *batchConn) batchSendLoop(cfg config.TiKVClient) { bestBatchWaitSize := cfg.BatchWaitSize for { + a.state.Store(batchConnIdle) a.reqBuilder.reset() start := a.fetchAllPendingRequests(int(cfg.MaxBatchSize)) @@ -322,6 +341,8 @@ func (a *batchConn) batchSendLoop(cfg config.TiKVClient) { } } + a.state.Store(batchConnHanding) + a.startHandingTime.Store(start.UnixNano()) if a.reqBuilder.len() < int(cfg.MaxBatchSize) && cfg.MaxBatchWaitTime > 0 { // If the target TiKV is overload, wait a while to collect more requests. if atomic.LoadUint64(&a.tikvTransportLayerLoad) >= uint64(cfg.OverloadThreshold) { @@ -378,11 +399,14 @@ func (a *batchConn) getClientAndSend() { } defer cli.unlockForSend() + now := time.Now() + batchCmdWaitToSendDuration := metrics.TiKVBatchCmdDuration.WithLabelValues("wait-to-send", target) req, forwardingReqs := a.reqBuilder.build(func(id uint64, e *batchCommandsEntry) { cli.batched.Store(id, e) if trace.IsEnabled() { trace.Log(e.ctx, "rpc", "send") } + batchCmdWaitToSendDuration.Observe(float64(now.Sub(e.start))) }) if req != nil { cli.send("", req) @@ -507,6 +531,14 @@ func (c *batchCommandsClient) isStopped() bool { } func (c *batchCommandsClient) send(forwardedHost string, req *tikvpb.BatchCommandsRequest) { + start := time.Now() + defer func() { + if forwardedHost == "" { + metrics.TiKVBatchConnSendDuration.WithLabelValues(c.target).Observe(time.Since(start).Seconds()) + } else { + metrics.TiKVBatchConnSendDuration.WithLabelValues(forwardedHost).Observe(time.Since(start).Seconds()) + } + }() err := c.initBatchClient(forwardedHost) if err != nil { logutil.BgLogger().Warn( @@ -612,6 +644,7 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient, tikvTransport } }() + batchCmdGotRespDuration := metrics.TiKVBatchCmdDuration.WithLabelValues("got-resp", c.target) epoch := atomic.LoadUint64(&c.epoch) for { resp, err := streamClient.recv() @@ -635,6 +668,7 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient, tikvTransport } responses := resp.GetResponses() + now := time.Now() for i, requestID := range resp.GetRequestIds() { value, ok := c.batched.Load(requestID) if !ok { @@ -649,6 +683,7 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient, tikvTransport trace.Log(entry.ctx, "rpc", "received") } logutil.Eventf(entry.ctx, "receive %T response with other %d batched requests from %s", responses[i].GetCmd(), len(responses), c.target) + batchCmdGotRespDuration.Observe(float64(now.Sub(entry.start))) if atomic.LoadInt32(&entry.canceled) == 0 { // Put the response only if the request is not canceled. entry.res <- responses[i] @@ -773,6 +808,7 @@ func sendBatchRequest( req *tikvpb.BatchCommandsRequest_Request, timeout time.Duration, ) (*tikvrpc.Response, error) { + start := time.Now() entry := &batchCommandsEntry{ ctx: ctx, req: req, @@ -780,11 +816,11 @@ func sendBatchRequest( forwardedHost: forwardedHost, canceled: 0, err: nil, + start: start, } timer := time.NewTimer(timeout) defer timer.Stop() - start := time.Now() select { case batchConn.batchCommandsCh <- entry: case <-ctx.Done(): @@ -795,7 +831,7 @@ func sendBatchRequest( return nil, errors.WithMessage(context.DeadlineExceeded, "wait sendLoop") } waitDuration := time.Since(start) - metrics.TiKVBatchWaitDuration.Observe(float64(waitDuration)) + metrics.TiKVBatchCmdDuration.WithLabelValues("send-to-chan", addr).Observe(float64(waitDuration)) select { case res, ok := <-entry.res: diff --git a/metrics/metrics.go b/metrics/metrics.go index 5c72f05b28..9328ebcd84 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -62,7 +62,8 @@ var ( TiKVLocalLatchWaitTimeHistogram prometheus.Histogram TiKVStatusDuration *prometheus.HistogramVec TiKVStatusCounter *prometheus.CounterVec - TiKVBatchWaitDuration prometheus.Histogram + TiKVBatchConnSendDuration *prometheus.HistogramVec + TiKVBatchCmdDuration *prometheus.HistogramVec TiKVBatchSendLatency prometheus.Histogram TiKVBatchWaitOverLoad prometheus.Counter TiKVBatchPendingRequests *prometheus.HistogramVec @@ -333,15 +334,25 @@ func initMetrics(namespace, subsystem string, constLabels prometheus.Labels) { ConstLabels: constLabels, }, []string{LblResult}) - TiKVBatchWaitDuration = prometheus.NewHistogram( + TiKVBatchConnSendDuration = prometheus.NewHistogramVec( prometheus.HistogramOpts{ Namespace: namespace, Subsystem: subsystem, - Name: "batch_wait_duration", - Buckets: prometheus.ExponentialBuckets(1, 2, 34), // 1ns ~ 8s - Help: "batch wait duration", + Name: "batch_conn_send_seconds", + Buckets: prometheus.ExponentialBuckets(0.0005, 2, 22), // 0.5ms ~ 1048s + Help: "batch conn send duration", ConstLabels: constLabels, - }) + }, []string{LblStore}) + + TiKVBatchCmdDuration = prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "batch_cmd_duration", + Buckets: prometheus.ExponentialBuckets(16, 2, 36), // 16ns ~ 549s + Help: "batch cmd duration, unit is nanosecond", + ConstLabels: constLabels, + }, []string{LblType, LblStore}) TiKVBatchSendLatency = prometheus.NewHistogram( prometheus.HistogramOpts{ @@ -767,7 +778,8 @@ func RegisterMetrics() { prometheus.MustRegister(TiKVLocalLatchWaitTimeHistogram) prometheus.MustRegister(TiKVStatusDuration) prometheus.MustRegister(TiKVStatusCounter) - prometheus.MustRegister(TiKVBatchWaitDuration) + prometheus.MustRegister(TiKVBatchConnSendDuration) + prometheus.MustRegister(TiKVBatchCmdDuration) prometheus.MustRegister(TiKVBatchSendLatency) prometheus.MustRegister(TiKVBatchRecvLatency) prometheus.MustRegister(TiKVBatchWaitOverLoad) From 330fc8d843e837f9e8444de0c4eafc8e5be5dbe2 Mon Sep 17 00:00:00 2001 From: Smilencer Date: Mon, 18 Sep 2023 15:19:39 +0800 Subject: [PATCH 41/49] add util to set dns server (#927) Signed-off-by: Smityz Co-authored-by: disksing --- util/dns.go | 60 +++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 60 insertions(+) create mode 100644 util/dns.go diff --git a/util/dns.go b/util/dns.go new file mode 100644 index 0000000000..03b2d42341 --- /dev/null +++ b/util/dns.go @@ -0,0 +1,60 @@ +package util + +import ( + "context" + "fmt" + "net" + "strings" + "time" +) + +type dnsF func(ctx context.Context, target string) (net.Conn, error) + +func wrapWithDomain(target, domain string) (string, error) { + if len(domain) == 0 { + return target, nil + } + strlist := strings.Split(target, ":") + if len(strlist) != 2 { + return "", fmt.Errorf("target %s is not valid", target) + } + address := strlist[0] + port := strlist[1] + return fmt.Sprintf("%s.%s:%s", address, domain, port), nil +} + +// GetDefaultDNSDialer is a util which built for connecting TiKV on k8s. +// Here is an example: +// coreDNSAddr := "8.8.8.8:53" +// domain := "cluster.local" +// dialer := grpc.WithContextDialer(util.GetCustomDNSDialer(coreDNSAddr, domain)) +// cli, err := rawkv.NewClientWithOpts( +// +// context.TODO(), +// []string{"pd0.pd:2379"}, +// rawkv.WithPDOptions(pd.WithGRPCDialOptions(dialer)), +// rawkv.WithGRPCDialOptions(dialer), +// +// ) +// cli.Close() +func GetCustomDNSDialer(dnsServer, dnsDomain string) dnsF { + return func(ctx context.Context, target string) (net.Conn, error) { + r := net.Resolver{ + PreferGo: true, + Dial: func(ctx context.Context, network, _ string) (net.Conn, error) { + d := net.Dialer{ + Timeout: time.Millisecond * time.Duration(10000), + } + return d.DialContext(ctx, network, dnsServer) + }, + } + dialer := &net.Dialer{ + Resolver: &r, + } + addr, err := wrapWithDomain(target, dnsDomain) + if err != nil { + return nil, err + } + return dialer.DialContext(ctx, "tcp", addr) + } +} From be2b4c78a91020a1e73adf2577cf08109e985f87 Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 19 Sep 2023 11:15:11 +0800 Subject: [PATCH 42/49] Revert "*: fix batch-client wait too long and add some metrics (#973)" (#984) This reverts commit adb7db13c3e6f94256831fe0fe80f4095d93d12a. Signed-off-by: crazycs520 --- internal/client/client.go | 2 +- internal/client/client_batch.go | 42 +++------------------------------ metrics/metrics.go | 26 ++++++-------------- 3 files changed, 11 insertions(+), 59 deletions(-) diff --git a/internal/client/client.go b/internal/client/client.go index 3bf9b0ebf0..98d847a909 100644 --- a/internal/client/client.go +++ b/internal/client/client.go @@ -620,7 +620,7 @@ func (c *RPCClient) sendRequest(ctx context.Context, addr string, req *tikvrpc.R // TiDB RPC server supports batch RPC, but batch connection will send heart beat, It's not necessary since // request to TiDB is not high frequency. - if config.GetGlobalConfig().TiKVClient.MaxBatchSize > 0 && enableBatch && !connArray.batchConn.isBusy(start.UnixNano()) { + if config.GetGlobalConfig().TiKVClient.MaxBatchSize > 0 && enableBatch { if batchReq := req.ToBatchCommandsRequest(); batchReq != nil { defer trace.StartRegion(ctx, req.Type.String()).End() return sendBatchRequest(ctx, addr, req.ForwardedHost, connArray.batchConn, batchReq, timeout) diff --git a/internal/client/client_batch.go b/internal/client/client_batch.go index eb58cb0333..b7159847b9 100644 --- a/internal/client/client_batch.go +++ b/internal/client/client_batch.go @@ -70,7 +70,6 @@ type batchCommandsEntry struct { // canceled indicated the request is canceled or not. canceled int32 err error - start time.Time } func (b *batchCommandsEntry) isCanceled() bool { @@ -198,16 +197,9 @@ type batchConn struct { pendingRequests prometheus.Observer batchSize prometheus.Observer - index uint32 - state atomic.Int32 - startHandingTime atomic.Int64 + index uint32 } -var ( - batchConnIdle = int32(0) - batchConnHanding = int32(1) -) - func newBatchConn(connCount, maxBatchSize uint, idleNotify *uint32) *batchConn { return &batchConn{ batchCommandsCh: make(chan *batchCommandsEntry, maxBatchSize), @@ -224,16 +216,6 @@ func (a *batchConn) isIdle() bool { return atomic.LoadUint32(&a.idle) != 0 } -func (a *batchConn) isBusy(now int64) bool { - if len(a.batchCommandsCh) == cap(a.batchCommandsCh) { - return true - } - if a.state.Load() == batchConnHanding && (now-a.startHandingTime.Load()) > int64(time.Second) { - return true - } - return false -} - // fetchAllPendingRequests fetches all pending requests from the channel. func (a *batchConn) fetchAllPendingRequests( maxBatchSize int, @@ -329,7 +311,6 @@ func (a *batchConn) batchSendLoop(cfg config.TiKVClient) { bestBatchWaitSize := cfg.BatchWaitSize for { - a.state.Store(batchConnIdle) a.reqBuilder.reset() start := a.fetchAllPendingRequests(int(cfg.MaxBatchSize)) @@ -341,8 +322,6 @@ func (a *batchConn) batchSendLoop(cfg config.TiKVClient) { } } - a.state.Store(batchConnHanding) - a.startHandingTime.Store(start.UnixNano()) if a.reqBuilder.len() < int(cfg.MaxBatchSize) && cfg.MaxBatchWaitTime > 0 { // If the target TiKV is overload, wait a while to collect more requests. if atomic.LoadUint64(&a.tikvTransportLayerLoad) >= uint64(cfg.OverloadThreshold) { @@ -399,14 +378,11 @@ func (a *batchConn) getClientAndSend() { } defer cli.unlockForSend() - now := time.Now() - batchCmdWaitToSendDuration := metrics.TiKVBatchCmdDuration.WithLabelValues("wait-to-send", target) req, forwardingReqs := a.reqBuilder.build(func(id uint64, e *batchCommandsEntry) { cli.batched.Store(id, e) if trace.IsEnabled() { trace.Log(e.ctx, "rpc", "send") } - batchCmdWaitToSendDuration.Observe(float64(now.Sub(e.start))) }) if req != nil { cli.send("", req) @@ -531,14 +507,6 @@ func (c *batchCommandsClient) isStopped() bool { } func (c *batchCommandsClient) send(forwardedHost string, req *tikvpb.BatchCommandsRequest) { - start := time.Now() - defer func() { - if forwardedHost == "" { - metrics.TiKVBatchConnSendDuration.WithLabelValues(c.target).Observe(time.Since(start).Seconds()) - } else { - metrics.TiKVBatchConnSendDuration.WithLabelValues(forwardedHost).Observe(time.Since(start).Seconds()) - } - }() err := c.initBatchClient(forwardedHost) if err != nil { logutil.BgLogger().Warn( @@ -644,7 +612,6 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient, tikvTransport } }() - batchCmdGotRespDuration := metrics.TiKVBatchCmdDuration.WithLabelValues("got-resp", c.target) epoch := atomic.LoadUint64(&c.epoch) for { resp, err := streamClient.recv() @@ -668,7 +635,6 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient, tikvTransport } responses := resp.GetResponses() - now := time.Now() for i, requestID := range resp.GetRequestIds() { value, ok := c.batched.Load(requestID) if !ok { @@ -683,7 +649,6 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient, tikvTransport trace.Log(entry.ctx, "rpc", "received") } logutil.Eventf(entry.ctx, "receive %T response with other %d batched requests from %s", responses[i].GetCmd(), len(responses), c.target) - batchCmdGotRespDuration.Observe(float64(now.Sub(entry.start))) if atomic.LoadInt32(&entry.canceled) == 0 { // Put the response only if the request is not canceled. entry.res <- responses[i] @@ -808,7 +773,6 @@ func sendBatchRequest( req *tikvpb.BatchCommandsRequest_Request, timeout time.Duration, ) (*tikvrpc.Response, error) { - start := time.Now() entry := &batchCommandsEntry{ ctx: ctx, req: req, @@ -816,11 +780,11 @@ func sendBatchRequest( forwardedHost: forwardedHost, canceled: 0, err: nil, - start: start, } timer := time.NewTimer(timeout) defer timer.Stop() + start := time.Now() select { case batchConn.batchCommandsCh <- entry: case <-ctx.Done(): @@ -831,7 +795,7 @@ func sendBatchRequest( return nil, errors.WithMessage(context.DeadlineExceeded, "wait sendLoop") } waitDuration := time.Since(start) - metrics.TiKVBatchCmdDuration.WithLabelValues("send-to-chan", addr).Observe(float64(waitDuration)) + metrics.TiKVBatchWaitDuration.Observe(float64(waitDuration)) select { case res, ok := <-entry.res: diff --git a/metrics/metrics.go b/metrics/metrics.go index 9328ebcd84..5c72f05b28 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -62,8 +62,7 @@ var ( TiKVLocalLatchWaitTimeHistogram prometheus.Histogram TiKVStatusDuration *prometheus.HistogramVec TiKVStatusCounter *prometheus.CounterVec - TiKVBatchConnSendDuration *prometheus.HistogramVec - TiKVBatchCmdDuration *prometheus.HistogramVec + TiKVBatchWaitDuration prometheus.Histogram TiKVBatchSendLatency prometheus.Histogram TiKVBatchWaitOverLoad prometheus.Counter TiKVBatchPendingRequests *prometheus.HistogramVec @@ -334,25 +333,15 @@ func initMetrics(namespace, subsystem string, constLabels prometheus.Labels) { ConstLabels: constLabels, }, []string{LblResult}) - TiKVBatchConnSendDuration = prometheus.NewHistogramVec( + TiKVBatchWaitDuration = prometheus.NewHistogram( prometheus.HistogramOpts{ Namespace: namespace, Subsystem: subsystem, - Name: "batch_conn_send_seconds", - Buckets: prometheus.ExponentialBuckets(0.0005, 2, 22), // 0.5ms ~ 1048s - Help: "batch conn send duration", - ConstLabels: constLabels, - }, []string{LblStore}) - - TiKVBatchCmdDuration = prometheus.NewHistogramVec( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "batch_cmd_duration", - Buckets: prometheus.ExponentialBuckets(16, 2, 36), // 16ns ~ 549s - Help: "batch cmd duration, unit is nanosecond", + Name: "batch_wait_duration", + Buckets: prometheus.ExponentialBuckets(1, 2, 34), // 1ns ~ 8s + Help: "batch wait duration", ConstLabels: constLabels, - }, []string{LblType, LblStore}) + }) TiKVBatchSendLatency = prometheus.NewHistogram( prometheus.HistogramOpts{ @@ -778,8 +767,7 @@ func RegisterMetrics() { prometheus.MustRegister(TiKVLocalLatchWaitTimeHistogram) prometheus.MustRegister(TiKVStatusDuration) prometheus.MustRegister(TiKVStatusCounter) - prometheus.MustRegister(TiKVBatchConnSendDuration) - prometheus.MustRegister(TiKVBatchCmdDuration) + prometheus.MustRegister(TiKVBatchWaitDuration) prometheus.MustRegister(TiKVBatchSendLatency) prometheus.MustRegister(TiKVBatchRecvLatency) prometheus.MustRegister(TiKVBatchWaitOverLoad) From c8832b846685d303f5afe29e1f8da50c7aff9a2e Mon Sep 17 00:00:00 2001 From: ShuNing Date: Thu, 21 Sep 2023 19:32:00 +0800 Subject: [PATCH 43/49] resource_control: add stats task (#985) Signed-off-by: nolouch --- util/request_source.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/util/request_source.go b/util/request_source.go index 97d3b83fdb..833c30e0f3 100644 --- a/util/request_source.go +++ b/util/request_source.go @@ -49,10 +49,11 @@ const ( ExplicitTypeDumpling = "dumpling" ExplicitTypeBackground = "background" ExplicitTypeDDL = "ddl" + ExplicitTypeStats = "stats" ) // ExplicitTypeList is the list of all explicit source types. -var ExplicitTypeList = []string{ExplicitTypeEmpty, ExplicitTypeLightning, ExplicitTypeBR, ExplicitTypeDumpling, ExplicitTypeBackground, ExplicitTypeDDL} +var ExplicitTypeList = []string{ExplicitTypeEmpty, ExplicitTypeLightning, ExplicitTypeBR, ExplicitTypeDumpling, ExplicitTypeBackground, ExplicitTypeDDL, ExplicitTypeStats} const ( // InternalRequest is the scope of internal queries From 44b0cf7aba2be652cb17e36f70227c8fdb906d20 Mon Sep 17 00:00:00 2001 From: zyguan Date: Mon, 25 Sep 2023 11:25:02 +0800 Subject: [PATCH 44/49] do not try leader if it's unreachable (#971) * do not try leader if it's unreachable Signed-off-by: zyguan * fix the flaky test Signed-off-by: zyguan --------- Signed-off-by: zyguan --- internal/locate/region_request.go | 8 +++-- internal/locate/region_request3_test.go | 39 +++++++++++++++++++++++++ internal/locate/region_request_test.go | 2 ++ 3 files changed, 47 insertions(+), 2 deletions(-) diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index 9d823e4f04..f4718ceef6 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -730,11 +730,15 @@ func (state *accessFollower) next(bo *retry.Backoffer, selector *replicaSelector if selector.targetIdx < 0 { leader := selector.replicas[state.leaderIdx] leaderEpochStale := leader.isEpochStale() - leaderInvalid := leaderEpochStale || state.IsLeaderExhausted(leader) + leaderUnreachable := leader.store.getLivenessState() != reachable + leaderExhausted := state.IsLeaderExhausted(leader) + leaderInvalid := leaderEpochStale || leaderUnreachable || leaderExhausted if len(state.option.labels) > 0 { logutil.Logger(bo.GetCtx()).Warn("unable to find stores with given labels", zap.Uint64("region", selector.region.GetID()), - zap.Bool("leader-invalid", leaderInvalid), + zap.Bool("leader-epoch-stale", leaderEpochStale), + zap.Bool("leader-unreachable", leaderUnreachable), + zap.Bool("leader-exhausted", leaderExhausted), zap.Bool("stale-read", state.isStaleRead), zap.Any("labels", state.option.labels)) } diff --git a/internal/locate/region_request3_test.go b/internal/locate/region_request3_test.go index 41dd69b6b3..991bc04d5f 100644 --- a/internal/locate/region_request3_test.go +++ b/internal/locate/region_request3_test.go @@ -922,6 +922,16 @@ func (s *testRegionRequestToThreeStoresSuite) TestSendReqWithReplicaSelector() { // Verify switch to the leader immediately when stale read requests with global txn scope meet region errors. s.cluster.ChangeLeader(region.Region.id, s.peerIDs[0]) + tf = func(s *Store, bo *retry.Backoffer) livenessState { + return reachable + } + s.regionRequestSender.regionCache.testingKnobs.mockRequestLiveness.Store((*livenessFunc)(&tf)) + s.Eventually(func() bool { + stores := s.regionRequestSender.replicaSelector.regionStore.stores + return stores[0].getLivenessState() == reachable && + stores[1].getLivenessState() == reachable && + stores[2].getLivenessState() == reachable + }, 3*time.Second, 200*time.Millisecond) reloadRegion() req = tikvrpc.NewRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{Key: []byte("key")}) req.ReadReplicaScope = oracle.GlobalTxnScope @@ -1581,3 +1591,32 @@ func (s *testRegionRequestToThreeStoresSuite) TestStaleReadTryFollowerAfterTimeo s.Equal(int64(2), s.regionRequestSender.Stats[tikvrpc.CmdGet].Count) // 2 rpc s.Equal(0, bo.GetTotalBackoffTimes()) // no backoff since fast retry. } + +func (s *testRegionRequestToThreeStoresSuite) TestDoNotTryUnreachableLeader() { + key := []byte("key") + region, err := s.regionRequestSender.regionCache.findRegionByKey(s.bo, key, false) + s.Nil(err) + regionStore := region.getStore() + leader, _, _, _ := region.WorkStorePeer(regionStore) + follower, _, _, _ := region.FollowerStorePeer(regionStore, 0, &storeSelectorOp{}) + + s.regionRequestSender.client = &fnClient{fn: func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (response *tikvrpc.Response, err error) { + if req.StaleRead && addr == follower.addr { + return &tikvrpc.Response{Resp: &kvrpcpb.GetResponse{RegionError: &errorpb.Error{DataIsNotReady: &errorpb.DataIsNotReady{}}}}, nil + } + return &tikvrpc.Response{Resp: &kvrpcpb.GetResponse{ + Value: []byte(addr), + }}, nil + }} + atomic.StoreUint32(&leader.livenessState, uint32(unreachable)) + + req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{Key: key}, kv.ReplicaReadLeader, nil) + req.ReadReplicaScope = oracle.GlobalTxnScope + req.TxnScope = oracle.GlobalTxnScope + req.EnableStaleRead() + bo := retry.NewBackoffer(context.Background(), -1) + resp, _, _, err := s.regionRequestSender.SendReqCtx(bo, req, region.VerID(), time.Second, tikvrpc.TiKV, WithMatchLabels(follower.labels)) + s.Nil(err) + // `tryFollower` always try the local peer firstly + s.Equal(follower.addr, string(resp.Resp.(*kvrpcpb.GetResponse).Value)) +} diff --git a/internal/locate/region_request_test.go b/internal/locate/region_request_test.go index 2923648fb6..d14e744928 100644 --- a/internal/locate/region_request_test.go +++ b/internal/locate/region_request_test.go @@ -724,6 +724,8 @@ func (s *testRegionRequestToSingleStoreSuite) TestKVReadTimeoutWithDisableBatchC s.Nil(err) s.NotNil(region) req := tikvrpc.NewRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{Key: []byte("a"), Version: 1}) + // send a probe request to make sure the mock server is ready. + s.regionRequestSender.SendReq(retry.NewNoopBackoff(context.Background()), req, region.Region, time.Second) resp, _, err := s.regionRequestSender.SendReq(bo, req, region.Region, time.Millisecond*10) s.Nil(err) s.NotNil(resp) From 4c2ae43454de70bd8271f4d565fc90fe3db334c0 Mon Sep 17 00:00:00 2001 From: buffer <1045931706@qq.com> Date: Tue, 26 Sep 2023 13:16:11 +0800 Subject: [PATCH 45/49] dco (#988) Signed-off-by: bufferflies <1045931706@qq.com> --- internal/locate/region_cache.go | 1 + internal/locate/region_request.go | 12 +++++++++--- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/internal/locate/region_cache.go b/internal/locate/region_cache.go index e21703f31a..a3eba8d8d8 100644 --- a/internal/locate/region_cache.go +++ b/internal/locate/region_cache.go @@ -358,6 +358,7 @@ func (r *Region) isCacheTTLExpired(ts int64) bool { return ts-lastAccess > regionCacheTTLSec } +// checkRegionCacheTTL returns false means the region cache is expired. func (r *Region) checkRegionCacheTTL(ts int64) bool { // Only consider use percentage on this failpoint, for example, "2%return" if _, err := util.EvalFailpoint("invalidateRegionCache"); err == nil { diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index f4718ceef6..7d2aaf82e4 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -910,9 +910,14 @@ func newReplicaSelector( regionCache *RegionCache, regionID RegionVerID, req *tikvrpc.Request, opts ...StoreSelectorOption, ) (*replicaSelector, error) { cachedRegion := regionCache.GetCachedRegionWithRLock(regionID) - if cachedRegion == nil || !cachedRegion.isValid() { - return nil, nil + if cachedRegion == nil { + return nil, errors.New("cached region not found") + } else if cachedRegion.checkNeedReload() { + return nil, errors.New("cached region need reload") + } else if !cachedRegion.checkRegionCacheTTL(time.Now().Unix()) { + return nil, errors.New("cached region ttl expired") } + regionStore := cachedRegion.getStore() replicas := make([]*replica, 0, regionStore.accessStoreNum(tiKVOnly)) for _, storeIdx := range regionStore.accessIndex[tiKVOnly] { @@ -1274,7 +1279,8 @@ func (s *RegionRequestSender) getRPCContext( if s.replicaSelector == nil { selector, err := newReplicaSelector(s.regionCache, regionID, req, opts...) if selector == nil || err != nil { - return nil, err + s.rpcError = err + return nil, nil } s.replicaSelector = selector } From daed9fdababf45db8845c060b4825a87c9da84e1 Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Fri, 6 Oct 2023 20:20:06 -0700 Subject: [PATCH 46/49] metrics: make upper limit of RPC latency larger (#997) Signed-off-by: Jack Yu --- metrics/metrics.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/metrics/metrics.go b/metrics/metrics.go index 5c72f05b28..7a9cfcf367 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -185,7 +185,7 @@ func initMetrics(namespace, subsystem string, constLabels prometheus.Labels) { Subsystem: subsystem, Name: "rpc_net_latency_seconds", Help: "Bucketed histogram of time difference between TiDB and TiKV.", - Buckets: prometheus.ExponentialBuckets(5e-5, 2, 18), // 50us ~ 6.5s + Buckets: prometheus.ExponentialBuckets(5e-5, 2, 22), // 50us ~ 105s ConstLabels: constLabels, }, []string{LblStore, LblScope}) From b535011f12e057637de04df145f275e6b91e7ac0 Mon Sep 17 00:00:00 2001 From: Hu# Date: Sat, 7 Oct 2023 11:37:03 +0800 Subject: [PATCH 47/49] update fo mod (#995) Signed-off-by: husharp Co-authored-by: disksing --- examples/gcworker/go.mod | 48 +++++- examples/rawkv/go.mod | 48 +++++- examples/txnkv/1pc_txn/go.mod | 48 +++++- examples/txnkv/async_commit/go.mod | 48 +++++- examples/txnkv/delete_range/go.mod | 48 +++++- examples/txnkv/go.mod | 48 +++++- examples/txnkv/pessimistic_txn/go.mod | 48 +++++- examples/txnkv/unsafedestoryrange/go.mod | 48 +++++- go.mod | 2 +- go.sum | 7 + integration_tests/go.mod | 37 ++--- integration_tests/go.sum | 178 +++++++++++++++++++---- 12 files changed, 553 insertions(+), 55 deletions(-) diff --git a/examples/gcworker/go.mod b/examples/gcworker/go.mod index 87a6f233ca..acf4653e4d 100644 --- a/examples/gcworker/go.mod +++ b/examples/gcworker/go.mod @@ -1,7 +1,53 @@ module gcworker -go 1.20 +go 1.21 require github.com/tikv/client-go/v2 v2.0.0 +require ( + github.com/benbjohnson/clock v1.3.0 // indirect + github.com/beorn7/perks v1.0.1 // indirect + github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/cloudfoundry/gosigar v1.3.6 // indirect + github.com/coreos/go-semver v0.3.0 // indirect + github.com/coreos/go-systemd/v22 v22.3.2 // indirect + github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 // indirect + github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2 // indirect + github.com/gogo/protobuf v1.3.2 // indirect + github.com/golang/protobuf v1.5.3 // indirect + github.com/google/btree v1.1.2 // indirect + github.com/google/uuid v1.3.0 // indirect + github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 // indirect + github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect + github.com/opentracing/opentracing-go v1.2.0 // indirect + github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect + github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c // indirect + github.com/pingcap/kvproto v0.0.0-20230904082117-ecdbf1f8c130 // indirect + github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 // indirect + github.com/pkg/errors v0.9.1 // indirect + github.com/prometheus/client_golang v1.15.1 // indirect + github.com/prometheus/client_model v0.3.0 // indirect + github.com/prometheus/common v0.42.0 // indirect + github.com/prometheus/procfs v0.9.0 // indirect + github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 // indirect + github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a // indirect + github.com/tikv/pd/client v0.0.0-20230724080549-de985b8e0afc // indirect + github.com/twmb/murmur3 v1.1.3 // indirect + go.etcd.io/etcd/api/v3 v3.5.2 // indirect + go.etcd.io/etcd/client/pkg/v3 v3.5.2 // indirect + go.etcd.io/etcd/client/v3 v3.5.2 // indirect + go.uber.org/atomic v1.11.0 // indirect + go.uber.org/multierr v1.11.0 // indirect + go.uber.org/zap v1.24.0 // indirect + golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect + golang.org/x/net v0.10.0 // indirect + golang.org/x/sync v0.2.0 // indirect + golang.org/x/sys v0.8.0 // indirect + golang.org/x/text v0.9.0 // indirect + google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 // indirect + google.golang.org/grpc v1.54.0 // indirect + google.golang.org/protobuf v1.30.0 // indirect + gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect +) + replace github.com/tikv/client-go/v2 => ../../ diff --git a/examples/rawkv/go.mod b/examples/rawkv/go.mod index 6ea5170335..394fcfe12d 100644 --- a/examples/rawkv/go.mod +++ b/examples/rawkv/go.mod @@ -1,7 +1,53 @@ module rawkv -go 1.20 +go 1.21 require github.com/tikv/client-go/v2 v2.0.0 +require ( + github.com/benbjohnson/clock v1.3.0 // indirect + github.com/beorn7/perks v1.0.1 // indirect + github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/cloudfoundry/gosigar v1.3.6 // indirect + github.com/coreos/go-semver v0.3.0 // indirect + github.com/coreos/go-systemd/v22 v22.3.2 // indirect + github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 // indirect + github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2 // indirect + github.com/gogo/protobuf v1.3.2 // indirect + github.com/golang/protobuf v1.5.3 // indirect + github.com/google/btree v1.1.2 // indirect + github.com/google/uuid v1.3.0 // indirect + github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 // indirect + github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect + github.com/opentracing/opentracing-go v1.2.0 // indirect + github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect + github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c // indirect + github.com/pingcap/kvproto v0.0.0-20230904082117-ecdbf1f8c130 // indirect + github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 // indirect + github.com/pkg/errors v0.9.1 // indirect + github.com/prometheus/client_golang v1.15.1 // indirect + github.com/prometheus/client_model v0.3.0 // indirect + github.com/prometheus/common v0.42.0 // indirect + github.com/prometheus/procfs v0.9.0 // indirect + github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 // indirect + github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a // indirect + github.com/tikv/pd/client v0.0.0-20230724080549-de985b8e0afc // indirect + github.com/twmb/murmur3 v1.1.3 // indirect + go.etcd.io/etcd/api/v3 v3.5.2 // indirect + go.etcd.io/etcd/client/pkg/v3 v3.5.2 // indirect + go.etcd.io/etcd/client/v3 v3.5.2 // indirect + go.uber.org/atomic v1.11.0 // indirect + go.uber.org/multierr v1.11.0 // indirect + go.uber.org/zap v1.24.0 // indirect + golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect + golang.org/x/net v0.10.0 // indirect + golang.org/x/sync v0.2.0 // indirect + golang.org/x/sys v0.8.0 // indirect + golang.org/x/text v0.9.0 // indirect + google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 // indirect + google.golang.org/grpc v1.54.0 // indirect + google.golang.org/protobuf v1.30.0 // indirect + gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect +) + replace github.com/tikv/client-go/v2 => ../../ diff --git a/examples/txnkv/1pc_txn/go.mod b/examples/txnkv/1pc_txn/go.mod index 29e137b8ef..2fb3d8ae11 100644 --- a/examples/txnkv/1pc_txn/go.mod +++ b/examples/txnkv/1pc_txn/go.mod @@ -1,7 +1,53 @@ module 1pc_txn -go 1.20 +go 1.21 require github.com/tikv/client-go/v2 v2.0.0 +require ( + github.com/benbjohnson/clock v1.3.0 // indirect + github.com/beorn7/perks v1.0.1 // indirect + github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/cloudfoundry/gosigar v1.3.6 // indirect + github.com/coreos/go-semver v0.3.0 // indirect + github.com/coreos/go-systemd/v22 v22.3.2 // indirect + github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 // indirect + github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2 // indirect + github.com/gogo/protobuf v1.3.2 // indirect + github.com/golang/protobuf v1.5.3 // indirect + github.com/google/btree v1.1.2 // indirect + github.com/google/uuid v1.3.0 // indirect + github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 // indirect + github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect + github.com/opentracing/opentracing-go v1.2.0 // indirect + github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect + github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c // indirect + github.com/pingcap/kvproto v0.0.0-20230904082117-ecdbf1f8c130 // indirect + github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 // indirect + github.com/pkg/errors v0.9.1 // indirect + github.com/prometheus/client_golang v1.15.1 // indirect + github.com/prometheus/client_model v0.3.0 // indirect + github.com/prometheus/common v0.42.0 // indirect + github.com/prometheus/procfs v0.9.0 // indirect + github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 // indirect + github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a // indirect + github.com/tikv/pd/client v0.0.0-20230724080549-de985b8e0afc // indirect + github.com/twmb/murmur3 v1.1.3 // indirect + go.etcd.io/etcd/api/v3 v3.5.2 // indirect + go.etcd.io/etcd/client/pkg/v3 v3.5.2 // indirect + go.etcd.io/etcd/client/v3 v3.5.2 // indirect + go.uber.org/atomic v1.11.0 // indirect + go.uber.org/multierr v1.11.0 // indirect + go.uber.org/zap v1.24.0 // indirect + golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect + golang.org/x/net v0.10.0 // indirect + golang.org/x/sync v0.2.0 // indirect + golang.org/x/sys v0.8.0 // indirect + golang.org/x/text v0.9.0 // indirect + google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 // indirect + google.golang.org/grpc v1.54.0 // indirect + google.golang.org/protobuf v1.30.0 // indirect + gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect +) + replace github.com/tikv/client-go/v2 => ../../../ diff --git a/examples/txnkv/async_commit/go.mod b/examples/txnkv/async_commit/go.mod index df388f3ebb..fdc5c936af 100644 --- a/examples/txnkv/async_commit/go.mod +++ b/examples/txnkv/async_commit/go.mod @@ -1,7 +1,53 @@ module async_commit -go 1.20 +go 1.21 require github.com/tikv/client-go/v2 v2.0.0 +require ( + github.com/benbjohnson/clock v1.3.0 // indirect + github.com/beorn7/perks v1.0.1 // indirect + github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/cloudfoundry/gosigar v1.3.6 // indirect + github.com/coreos/go-semver v0.3.0 // indirect + github.com/coreos/go-systemd/v22 v22.3.2 // indirect + github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 // indirect + github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2 // indirect + github.com/gogo/protobuf v1.3.2 // indirect + github.com/golang/protobuf v1.5.3 // indirect + github.com/google/btree v1.1.2 // indirect + github.com/google/uuid v1.3.0 // indirect + github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 // indirect + github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect + github.com/opentracing/opentracing-go v1.2.0 // indirect + github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect + github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c // indirect + github.com/pingcap/kvproto v0.0.0-20230904082117-ecdbf1f8c130 // indirect + github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 // indirect + github.com/pkg/errors v0.9.1 // indirect + github.com/prometheus/client_golang v1.15.1 // indirect + github.com/prometheus/client_model v0.3.0 // indirect + github.com/prometheus/common v0.42.0 // indirect + github.com/prometheus/procfs v0.9.0 // indirect + github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 // indirect + github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a // indirect + github.com/tikv/pd/client v0.0.0-20230724080549-de985b8e0afc // indirect + github.com/twmb/murmur3 v1.1.3 // indirect + go.etcd.io/etcd/api/v3 v3.5.2 // indirect + go.etcd.io/etcd/client/pkg/v3 v3.5.2 // indirect + go.etcd.io/etcd/client/v3 v3.5.2 // indirect + go.uber.org/atomic v1.11.0 // indirect + go.uber.org/multierr v1.11.0 // indirect + go.uber.org/zap v1.24.0 // indirect + golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect + golang.org/x/net v0.10.0 // indirect + golang.org/x/sync v0.2.0 // indirect + golang.org/x/sys v0.8.0 // indirect + golang.org/x/text v0.9.0 // indirect + google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 // indirect + google.golang.org/grpc v1.54.0 // indirect + google.golang.org/protobuf v1.30.0 // indirect + gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect +) + replace github.com/tikv/client-go/v2 => ../../../ diff --git a/examples/txnkv/delete_range/go.mod b/examples/txnkv/delete_range/go.mod index feba0ccedc..4c73298e3a 100644 --- a/examples/txnkv/delete_range/go.mod +++ b/examples/txnkv/delete_range/go.mod @@ -1,7 +1,53 @@ module delete_range -go 1.20 +go 1.21 require github.com/tikv/client-go/v2 v2.0.0 +require ( + github.com/benbjohnson/clock v1.3.0 // indirect + github.com/beorn7/perks v1.0.1 // indirect + github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/cloudfoundry/gosigar v1.3.6 // indirect + github.com/coreos/go-semver v0.3.0 // indirect + github.com/coreos/go-systemd/v22 v22.3.2 // indirect + github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 // indirect + github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2 // indirect + github.com/gogo/protobuf v1.3.2 // indirect + github.com/golang/protobuf v1.5.3 // indirect + github.com/google/btree v1.1.2 // indirect + github.com/google/uuid v1.3.0 // indirect + github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 // indirect + github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect + github.com/opentracing/opentracing-go v1.2.0 // indirect + github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect + github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c // indirect + github.com/pingcap/kvproto v0.0.0-20230904082117-ecdbf1f8c130 // indirect + github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 // indirect + github.com/pkg/errors v0.9.1 // indirect + github.com/prometheus/client_golang v1.15.1 // indirect + github.com/prometheus/client_model v0.3.0 // indirect + github.com/prometheus/common v0.42.0 // indirect + github.com/prometheus/procfs v0.9.0 // indirect + github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 // indirect + github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a // indirect + github.com/tikv/pd/client v0.0.0-20230724080549-de985b8e0afc // indirect + github.com/twmb/murmur3 v1.1.3 // indirect + go.etcd.io/etcd/api/v3 v3.5.2 // indirect + go.etcd.io/etcd/client/pkg/v3 v3.5.2 // indirect + go.etcd.io/etcd/client/v3 v3.5.2 // indirect + go.uber.org/atomic v1.11.0 // indirect + go.uber.org/multierr v1.11.0 // indirect + go.uber.org/zap v1.24.0 // indirect + golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect + golang.org/x/net v0.10.0 // indirect + golang.org/x/sync v0.2.0 // indirect + golang.org/x/sys v0.8.0 // indirect + golang.org/x/text v0.9.0 // indirect + google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 // indirect + google.golang.org/grpc v1.54.0 // indirect + google.golang.org/protobuf v1.30.0 // indirect + gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect +) + replace github.com/tikv/client-go/v2 => ../../../ diff --git a/examples/txnkv/go.mod b/examples/txnkv/go.mod index fd7ad964f7..e70118a9f3 100644 --- a/examples/txnkv/go.mod +++ b/examples/txnkv/go.mod @@ -1,7 +1,53 @@ module txnkv -go 1.20 +go 1.21 require github.com/tikv/client-go/v2 v2.0.0 +require ( + github.com/benbjohnson/clock v1.3.0 // indirect + github.com/beorn7/perks v1.0.1 // indirect + github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/cloudfoundry/gosigar v1.3.6 // indirect + github.com/coreos/go-semver v0.3.0 // indirect + github.com/coreos/go-systemd/v22 v22.3.2 // indirect + github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 // indirect + github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2 // indirect + github.com/gogo/protobuf v1.3.2 // indirect + github.com/golang/protobuf v1.5.3 // indirect + github.com/google/btree v1.1.2 // indirect + github.com/google/uuid v1.3.0 // indirect + github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 // indirect + github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect + github.com/opentracing/opentracing-go v1.2.0 // indirect + github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect + github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c // indirect + github.com/pingcap/kvproto v0.0.0-20230904082117-ecdbf1f8c130 // indirect + github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 // indirect + github.com/pkg/errors v0.9.1 // indirect + github.com/prometheus/client_golang v1.15.1 // indirect + github.com/prometheus/client_model v0.3.0 // indirect + github.com/prometheus/common v0.42.0 // indirect + github.com/prometheus/procfs v0.9.0 // indirect + github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 // indirect + github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a // indirect + github.com/tikv/pd/client v0.0.0-20230724080549-de985b8e0afc // indirect + github.com/twmb/murmur3 v1.1.3 // indirect + go.etcd.io/etcd/api/v3 v3.5.2 // indirect + go.etcd.io/etcd/client/pkg/v3 v3.5.2 // indirect + go.etcd.io/etcd/client/v3 v3.5.2 // indirect + go.uber.org/atomic v1.11.0 // indirect + go.uber.org/multierr v1.11.0 // indirect + go.uber.org/zap v1.24.0 // indirect + golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect + golang.org/x/net v0.10.0 // indirect + golang.org/x/sync v0.2.0 // indirect + golang.org/x/sys v0.8.0 // indirect + golang.org/x/text v0.9.0 // indirect + google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 // indirect + google.golang.org/grpc v1.54.0 // indirect + google.golang.org/protobuf v1.30.0 // indirect + gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect +) + replace github.com/tikv/client-go/v2 => ../../ diff --git a/examples/txnkv/pessimistic_txn/go.mod b/examples/txnkv/pessimistic_txn/go.mod index d504598cfc..90f42b3523 100644 --- a/examples/txnkv/pessimistic_txn/go.mod +++ b/examples/txnkv/pessimistic_txn/go.mod @@ -1,7 +1,53 @@ module pessimistic_txn -go 1.20 +go 1.21 require github.com/tikv/client-go/v2 v2.0.0 +require ( + github.com/benbjohnson/clock v1.3.0 // indirect + github.com/beorn7/perks v1.0.1 // indirect + github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/cloudfoundry/gosigar v1.3.6 // indirect + github.com/coreos/go-semver v0.3.0 // indirect + github.com/coreos/go-systemd/v22 v22.3.2 // indirect + github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 // indirect + github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2 // indirect + github.com/gogo/protobuf v1.3.2 // indirect + github.com/golang/protobuf v1.5.3 // indirect + github.com/google/btree v1.1.2 // indirect + github.com/google/uuid v1.3.0 // indirect + github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 // indirect + github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect + github.com/opentracing/opentracing-go v1.2.0 // indirect + github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect + github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c // indirect + github.com/pingcap/kvproto v0.0.0-20230904082117-ecdbf1f8c130 // indirect + github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 // indirect + github.com/pkg/errors v0.9.1 // indirect + github.com/prometheus/client_golang v1.15.1 // indirect + github.com/prometheus/client_model v0.3.0 // indirect + github.com/prometheus/common v0.42.0 // indirect + github.com/prometheus/procfs v0.9.0 // indirect + github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 // indirect + github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a // indirect + github.com/tikv/pd/client v0.0.0-20230724080549-de985b8e0afc // indirect + github.com/twmb/murmur3 v1.1.3 // indirect + go.etcd.io/etcd/api/v3 v3.5.2 // indirect + go.etcd.io/etcd/client/pkg/v3 v3.5.2 // indirect + go.etcd.io/etcd/client/v3 v3.5.2 // indirect + go.uber.org/atomic v1.11.0 // indirect + go.uber.org/multierr v1.11.0 // indirect + go.uber.org/zap v1.24.0 // indirect + golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect + golang.org/x/net v0.10.0 // indirect + golang.org/x/sync v0.2.0 // indirect + golang.org/x/sys v0.8.0 // indirect + golang.org/x/text v0.9.0 // indirect + google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 // indirect + google.golang.org/grpc v1.54.0 // indirect + google.golang.org/protobuf v1.30.0 // indirect + gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect +) + replace github.com/tikv/client-go/v2 => ../../../ diff --git a/examples/txnkv/unsafedestoryrange/go.mod b/examples/txnkv/unsafedestoryrange/go.mod index 81fa60e34b..2e4ad33bf6 100644 --- a/examples/txnkv/unsafedestoryrange/go.mod +++ b/examples/txnkv/unsafedestoryrange/go.mod @@ -1,7 +1,53 @@ module unsafedestoryrange -go 1.20 +go 1.21 require github.com/tikv/client-go/v2 v2.0.0 +require ( + github.com/benbjohnson/clock v1.3.0 // indirect + github.com/beorn7/perks v1.0.1 // indirect + github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/cloudfoundry/gosigar v1.3.6 // indirect + github.com/coreos/go-semver v0.3.0 // indirect + github.com/coreos/go-systemd/v22 v22.3.2 // indirect + github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 // indirect + github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2 // indirect + github.com/gogo/protobuf v1.3.2 // indirect + github.com/golang/protobuf v1.5.3 // indirect + github.com/google/btree v1.1.2 // indirect + github.com/google/uuid v1.3.0 // indirect + github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 // indirect + github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect + github.com/opentracing/opentracing-go v1.2.0 // indirect + github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect + github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c // indirect + github.com/pingcap/kvproto v0.0.0-20230904082117-ecdbf1f8c130 // indirect + github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 // indirect + github.com/pkg/errors v0.9.1 // indirect + github.com/prometheus/client_golang v1.15.1 // indirect + github.com/prometheus/client_model v0.3.0 // indirect + github.com/prometheus/common v0.42.0 // indirect + github.com/prometheus/procfs v0.9.0 // indirect + github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 // indirect + github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a // indirect + github.com/tikv/pd/client v0.0.0-20230724080549-de985b8e0afc // indirect + github.com/twmb/murmur3 v1.1.3 // indirect + go.etcd.io/etcd/api/v3 v3.5.2 // indirect + go.etcd.io/etcd/client/pkg/v3 v3.5.2 // indirect + go.etcd.io/etcd/client/v3 v3.5.2 // indirect + go.uber.org/atomic v1.11.0 // indirect + go.uber.org/multierr v1.11.0 // indirect + go.uber.org/zap v1.24.0 // indirect + golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect + golang.org/x/net v0.10.0 // indirect + golang.org/x/sync v0.2.0 // indirect + golang.org/x/sys v0.8.0 // indirect + golang.org/x/text v0.9.0 // indirect + google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 // indirect + google.golang.org/grpc v1.54.0 // indirect + google.golang.org/protobuf v1.30.0 // indirect + gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect +) + replace github.com/tikv/client-go/v2 => ../../../ diff --git a/go.mod b/go.mod index b301bac77b..dc8e0ff995 100644 --- a/go.mod +++ b/go.mod @@ -1,6 +1,6 @@ module github.com/tikv/client-go/v2 -go 1.20 +go 1.21 require ( github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 diff --git a/go.sum b/go.sum index cd65c90c6d..8c4431dd52 100644 --- a/go.sum +++ b/go.sum @@ -42,6 +42,7 @@ github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1m github.com/envoyproxy/go-control-plane v0.9.9-0.20210217033140-668b12f5399d/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/fsnotify/fsnotify v1.6.0 h1:n+5WquG0fcWoWp6xPWfHdbskMCQaFnG6PfBrh1Ky4HY= +github.com/fsnotify/fsnotify v1.6.0/go.mod h1:sl3t1tCWJFWoRz9R8WJCbQihKKwmorjAbSClcnxKAGw= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= @@ -86,6 +87,7 @@ github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.9 h1:O2Tfq5qg4qc4AmwVlvv0oLiVAGB7enBSJ2x2DqQFi38= +github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I= @@ -108,6 +110,7 @@ github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxv github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= +github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= @@ -122,7 +125,9 @@ github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3Rllmb github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= +github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= github.com/onsi/ginkgo v1.16.5 h1:8xi0RTUf59SOSfEtZMvwTvXYMzG4gV23XVHOZiXNtnE= +github.com/onsi/ginkgo v1.16.5/go.mod h1:+E8gABHa3K6zRBolWtd+ROzc/U5bkGt0FwiG042wbpU= github.com/onsi/gomega v1.20.1 h1:PA/3qinGoukvymdIDV8pii6tiZgC8kbmJO6Z5+b002Q= github.com/onsi/gomega v1.20.1/go.mod h1:DtrZpjmvpn2mPm4YWQa0/ALMDj9v4YxLgojwPeREyVo= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= @@ -352,10 +357,12 @@ gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8 gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= gopkg.in/natefinch/lumberjack.v2 v2.2.1 h1:bBRl1b0OH9s/DuPhuXpNl+VtCaJXFZ5/uEFST95x9zc= gopkg.in/natefinch/lumberjack.v2 v2.2.1/go.mod h1:YD8tP3GAjkrDg1eZH7EGmyESg/lsYskCTPBJVb9jqSc= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ= +gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= diff --git a/integration_tests/go.mod b/integration_tests/go.mod index a2a77d3975..eb7232d60a 100644 --- a/integration_tests/go.mod +++ b/integration_tests/go.mod @@ -1,24 +1,29 @@ module integration_tests -go 1.20 +go 1.21 require ( github.com/ninedraft/israce v0.0.3 github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32 github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c github.com/pingcap/kvproto v0.0.0-20230904082117-ecdbf1f8c130 - github.com/pingcap/tidb v1.1.0-beta.0.20230619015310-8b1006f1af04 + github.com/pingcap/tidb v1.1.0-beta.0.20230927025416-38023987346f github.com/pkg/errors v0.9.1 github.com/stretchr/testify v1.8.4 github.com/tidwall/gjson v1.14.1 - github.com/tikv/client-go/v2 v2.0.8-0.20230714052714-85fc8f337565 - github.com/tikv/pd/client v0.0.0-20230724080549-de985b8e0afc + github.com/tikv/client-go/v2 v2.0.8-0.20230925032502-44b0cf7aba2b + github.com/tikv/pd/client v0.0.0-20230912103610-2f57a9f050eb go.uber.org/goleak v1.2.1 ) +require ( + github.com/dolthub/maphash v0.1.0 // indirect + github.com/dolthub/swiss v0.2.1 // indirect +) + require ( github.com/Azure/go-ntlmssp v0.0.0-20221128193559-754e69321358 // indirect - github.com/BurntSushi/toml v1.3.0 // indirect + github.com/BurntSushi/toml v1.3.2 // indirect github.com/benbjohnson/clock v1.3.5 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect @@ -49,7 +54,7 @@ require ( github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 // indirect github.com/influxdata/tdigest v0.0.1 // indirect github.com/jellydator/ttlcache/v3 v3.0.1 // indirect - github.com/klauspost/compress v1.16.5 // indirect + github.com/klauspost/compress v1.17.0 // indirect github.com/klauspost/cpuid v1.3.1 // indirect github.com/kr/pretty v0.3.1 // indirect github.com/kr/text v0.2.0 // indirect @@ -65,16 +70,16 @@ require ( github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 // indirect github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22 // indirect github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 // indirect - github.com/pingcap/tidb/parser v0.0.0-20230619015310-8b1006f1af04 // indirect - github.com/pingcap/tipb v0.0.0-20230607071926-bda24015c2d6 // indirect + github.com/pingcap/tidb/parser v0.0.0-20230927025416-38023987346f // indirect + github.com/pingcap/tipb v0.0.0-20230919054518-dfd7d194838f // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b // indirect github.com/prometheus/client_golang v1.16.0 // indirect github.com/prometheus/client_model v0.4.0 // indirect github.com/prometheus/common v0.44.0 // indirect - github.com/prometheus/procfs v0.11.0 // indirect + github.com/prometheus/procfs v0.11.1 // indirect github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec // indirect - github.com/rogpeppe/go-internal v1.10.0 // indirect + github.com/rogpeppe/go-internal v1.11.0 // indirect github.com/sasha-s/go-deadlock v0.2.0 // indirect github.com/shirou/gopsutil/v3 v3.23.5 // indirect github.com/shoenig/go-m1cpu v0.1.6 // indirect @@ -93,13 +98,13 @@ require ( go.etcd.io/etcd/client/v3 v3.5.2 // indirect go.uber.org/atomic v1.11.0 // indirect go.uber.org/multierr v1.11.0 // indirect - go.uber.org/zap v1.24.0 // indirect - golang.org/x/crypto v0.11.0 // indirect + go.uber.org/zap v1.25.0 // indirect + golang.org/x/crypto v0.13.0 // indirect golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect - golang.org/x/net v0.11.0 // indirect + golang.org/x/net v0.15.0 // indirect golang.org/x/sync v0.3.0 // indirect - golang.org/x/sys v0.10.0 // indirect - golang.org/x/text v0.11.0 // indirect + golang.org/x/sys v0.12.0 // indirect + golang.org/x/text v0.13.0 // indirect golang.org/x/time v0.3.0 // indirect golang.org/x/tools v0.10.0 // indirect google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 // indirect @@ -112,7 +117,5 @@ require ( replace ( github.com/go-ldap/ldap/v3 => github.com/YangKeao/ldap/v3 v3.4.5-0.20230421065457-369a3bab1117 - github.com/pingcap/tidb => github.com/HuSharp/tidb v1.1.0-beta.0.20230726045237-a2b0085ad7c5 - github.com/pingcap/tidb/parser => github.com/HuSharp/tidb/parser v0.0.0-20230726045237-a2b0085ad7c5 github.com/tikv/client-go/v2 => ../ ) diff --git a/integration_tests/go.sum b/integration_tests/go.sum index fc6ec1e505..3bdc647893 100644 --- a/integration_tests/go.sum +++ b/integration_tests/go.sum @@ -1,37 +1,48 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.110.0 h1:Zc8gqp3+a9/Eyph2KDmcGaPtbKRIoqq4YTlL4NMD0Ys= +cloud.google.com/go v0.110.0/go.mod h1:SJnCLqQ0FCFGSZMUNUf84MV3Aia54kn7pi8st7tMzaY= cloud.google.com/go/compute v1.19.0 h1:+9zda3WGgW1ZSTlVppLCYFIr48Pa35q1uG2N1itbCEQ= +cloud.google.com/go/compute v1.19.0/go.mod h1:rikpw2y+UMidAe9tISo04EHNOIf42RLYF/q8Bs93scU= cloud.google.com/go/compute/metadata v0.2.3 h1:mg4jlk7mCAj6xXp9UJ4fjI9VUI5rubuGBW5aJ7UnBMY= +cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= cloud.google.com/go/iam v0.13.0 h1:+CmB+K0J/33d0zSQ9SlFWUeCCEn5XJA0ZMZ3pHE9u8k= +cloud.google.com/go/iam v0.13.0/go.mod h1:ljOg+rcNfzZ5d6f1nAUJ8ZIxOaZUVoS14bKCtaLZ/D0= cloud.google.com/go/storage v1.30.1 h1:uOdMxAs8HExqBlnLtnQyP0YkvbiDpdGShGKtx6U/oNM= +cloud.google.com/go/storage v1.30.1/go.mod h1:NfxhC0UJE1aXSx7CIIbCf7y9HKT7BiccwkR7+P7gN8E= github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= github.com/Azure/azure-sdk-for-go/sdk/azcore v1.6.0 h1:8kDqDngH+DmVBiCtIjCFTGa7MBnsIOkF9IccInFEbjk= +github.com/Azure/azure-sdk-for-go/sdk/azcore v1.6.0/go.mod h1:bjGvMhVMb+EEm3VRNQawDMUyMMjo+S5ewNjflkep/0Q= github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.1.0 h1:QkAcEIAKbNL4KoFr4SathZPhDhF4mVwpBMFlYjyAqy8= +github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.1.0/go.mod h1:bhXu1AjYL+wutSL/kpSq6s7733q2Rb0yuot9Zgfqa/0= github.com/Azure/azure-sdk-for-go/sdk/internal v1.3.0 h1:sXr+ck84g/ZlZUOZiNELInmMgOsuGwdjjVkEIde0OtY= +github.com/Azure/azure-sdk-for-go/sdk/internal v1.3.0/go.mod h1:okt5dMMTOFjX/aovMlrjvvXoPMBVSPzk9185BT0+eZM= github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.0.0 h1:u/LLAOFgsMv7HmNL4Qufg58y+qElGOt5qv0z1mURkRY= +github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.0.0/go.mod h1:2e8rMJtl2+2j+HXbTBwnyGpm5Nou7KhvSfxOq8JpTag= github.com/Azure/go-ntlmssp v0.0.0-20221128193559-754e69321358 h1:mFRzDkZVAjdal+s7s0MwaRv9igoPqLRdzOLzw/8Xvq8= github.com/Azure/go-ntlmssp v0.0.0-20221128193559-754e69321358/go.mod h1:chxPXzSsl7ZWRAuOIE23GDNzjWuZquvFlgA8xmpunjU= github.com/AzureAD/microsoft-authentication-library-for-go v0.5.1 h1:BWe8a+f/t+7KY7zH2mqygeUD0t8hNFXe08p1Pb3/jKE= +github.com/AzureAD/microsoft-authentication-library-for-go v0.5.1/go.mod h1:Vt9sXTKwMyGcOxSmLDMnGPgqsUg7m8pe215qMLrDXw4= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/BurntSushi/toml v1.3.0 h1:Ws8e5YmnrGEHzZEzg0YvK/7COGYtTC5PbaH9oSSbgfA= -github.com/BurntSushi/toml v1.3.0/go.mod h1:CxXYINrC8qIiEnFrOxCa7Jy5BFHlXnUU2pbicEuybxQ= +github.com/BurntSushi/toml v1.3.2 h1:o7IhLm0Msx3BaB+n3Ag7L8EVlByGnpq14C4YWiu/gL8= +github.com/BurntSushi/toml v1.3.2/go.mod h1:CxXYINrC8qIiEnFrOxCa7Jy5BFHlXnUU2pbicEuybxQ= github.com/CloudyKit/fastprinter v0.0.0-20170127035650-74b38d55f37a/go.mod h1:EFZQ978U7x8IRnstaskI3IysnWY5Ao3QgZUKOXlsAdw= github.com/CloudyKit/jet v2.1.3-0.20180809161101-62edd43e4f88+incompatible/go.mod h1:HPYO+50pSWkPoj9Q/eq0aRGByCL6ScRlUmiEX5Zgm+w= github.com/DataDog/zstd v1.4.5 h1:EndNeuB0l9syBZhut0wns3gV1hL8zX8LIu6ZiVHWLIQ= +github.com/DataDog/zstd v1.4.5/go.mod h1:1jcaCB/ufaK+sKp1NBhlGmpz41jOoPQ35bpF36t7BBo= github.com/HdrHistogram/hdrhistogram-go v1.1.2 h1:5IcZpTvzydCQeHzK4Ef/D5rrSqwxob0t8PQPMybUNFM= -github.com/HuSharp/tidb v1.1.0-beta.0.20230726045237-a2b0085ad7c5 h1:wSOvDYbKkvHjlWWFBihIoeJ5yBc1jZe9Ehkku3Jn8cA= -github.com/HuSharp/tidb v1.1.0-beta.0.20230726045237-a2b0085ad7c5/go.mod h1:C3tuWINS2/Vt/gxZ0OLdGI2x5crlN8E3/qNJJkIIkTI= -github.com/HuSharp/tidb/parser v0.0.0-20230726045237-a2b0085ad7c5 h1:bxwmPI7ambmbOAaozdYz81HFpIeu6ctWo7TiXfOGE14= -github.com/HuSharp/tidb/parser v0.0.0-20230726045237-a2b0085ad7c5/go.mod h1:ENXEsaVS6N3CTMpL4txc6m93y6XaztF9W4SFLjhPWJg= +github.com/HdrHistogram/hdrhistogram-go v1.1.2/go.mod h1:yDgFjdqOqDEKOvasDdhWNXYg9BVp4O+o5f6V/ehm6Oo= github.com/Joker/hpp v1.0.0/go.mod h1:8x5n+M1Hp5hC0g8okX3sR3vFQwynaX/UgSOM9MeBKzY= github.com/Joker/jade v1.0.1-0.20190614124447-d475f43051e7/go.mod h1:6E6s8o2AE4KhCrqr6GRJjdC/gNfTdxkIXvuGZZda2VM= github.com/Masterminds/semver v1.5.0 h1:H65muMkzWKEuNDnfl9d70GUjFniHKHRbFPGBuZ3QEww= +github.com/Masterminds/semver v1.5.0/go.mod h1:MB6lktGJrhw8PrUyiEoblNEGEQ+RzHPF078ddwwvV3Y= github.com/Shopify/goreferrer v0.0.0-20181106222321-ec9c9a553398/go.mod h1:a1uqRtAwp2Xwc6WNPJEufxJ7fx3npB4UV/JOLmbu5I0= github.com/VividCortex/ewma v1.2.0 h1:f58SaIzcDXrSy3kWaHNvuJgJ3Nmz59Zji6XoJR/q1ow= +github.com/VividCortex/ewma v1.2.0/go.mod h1:nz4BbCtbLyFDeC9SUHbtcT5644juEuWfUAUnGx7j5l4= github.com/YangKeao/ldap/v3 v3.4.5-0.20230421065457-369a3bab1117 h1:+OqGGFc2YHFd82aSHmjlILVt1t4JWJjrNIfV8cVEPow= github.com/YangKeao/ldap/v3 v3.4.5-0.20230421065457-369a3bab1117/go.mod h1:bMGIq3AGbytbaMwf8wdv5Phdxz0FWHTIYMSzyrYgnQs= github.com/acarl005/stripansi v0.0.0-20180116102854-5a71ef0e047d h1:licZJFw2RwpHMqeKTCYkitsPqHNxTmd4SNR5r94FGM8= +github.com/acarl005/stripansi v0.0.0-20180116102854-5a71ef0e047d/go.mod h1:asat636LX7Bqt5lYEZ27JNDcqxfjdBQuJ/MM4CN/Lzo= github.com/ajg/form v1.5.1/go.mod h1:uL1WgH+h2mgNtvBq0339dVnzXdBETtL2LeUXaIv25UY= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= @@ -41,10 +52,13 @@ github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk5 github.com/alexbrainman/sspi v0.0.0-20210105120005-909beea2cc74 h1:Kk6a4nehpJ3UuJRqlA3JxYxBZEqCeOmATOvrbT4p9RA= github.com/alexbrainman/sspi v0.0.0-20210105120005-909beea2cc74/go.mod h1:cEWa1LVoE5KvSD9ONXsZrj0z6KqySlCCNKHlLzbqAt4= github.com/aliyun/alibaba-cloud-sdk-go v1.61.1581 h1:Q/yk4z/cHUVZfgTqtD09qeYBxHwshQAjVRX73qs8UH0= +github.com/aliyun/alibaba-cloud-sdk-go v1.61.1581/go.mod h1:RcDobYh8k5VP6TNybz9m++gL3ijVI5wueVr0EM10VsU= github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/apache/thrift v0.13.1-0.20201008052519-daf620915714 h1:Jz3KVLYY5+JO7rDiX0sAuRGtuv2vG01r17Y9nLMWNUw= +github.com/apache/thrift v0.13.1-0.20201008052519-daf620915714/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= github.com/aws/aws-sdk-go v1.44.259 h1:7yDn1dcv4DZFMKpu+2exIH5O6ipNj9qXrKfdMUaIJwY= +github.com/aws/aws-sdk-go v1.44.259/go.mod h1:aVsgQcEevwlmQ7qHE9I3h+dtQgpqhFB+i8Phjh7fkwI= github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/benbjohnson/clock v1.3.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= @@ -55,13 +69,16 @@ github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+Ce github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/blacktear23/go-proxyprotocol v1.0.6 h1:eTt6UMpEnq59NjON49b3Cay8Dm0sCs1nDliwgkyEsRM= +github.com/blacktear23/go-proxyprotocol v1.0.6/go.mod h1:FSCbgnRZrQXazBLL5snfBbrcFSMtcmUDhSRb9OfFA1o= github.com/carlmjohnson/flagext v0.21.0 h1:/c4uK3ie786Z7caXLcIMvePNSSiH3bQVGDvmGLMme60= +github.com/carlmjohnson/flagext v0.21.0/go.mod h1:Eenv0epIUAr4NuedNmkzI8WmBmjIxZC239XcKxYS2ac= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.1.2/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cheggaaa/pb/v3 v3.0.8 h1:bC8oemdChbke2FHIIGy9mn4DPJ2caZYQnfbRqwmdCoA= +github.com/cheggaaa/pb/v3 v3.0.8/go.mod h1:UICbiLec/XO6Hw6k+BHEtHeQFzzBH4i2/qk/ow1EJTA= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cloudfoundry/gosigar v1.3.6 h1:gIc08FbB3QPb+nAQhINIK/qhf5REKkY0FTGgRGXkcVc= github.com/cloudfoundry/gosigar v1.3.6/go.mod h1:lNWstu5g5gw59O09Y+wsMNFzBSnU8a0u+Sfx4dq360E= @@ -73,7 +90,8 @@ github.com/cockroachdb/errors v1.8.1 h1:A5+txlVZfOqFBDa4mGz2bUWSp0aHElvHX2bKkdbQ github.com/cockroachdb/errors v1.8.1/go.mod h1:qGwQn6JmZ+oMjuLwjWzUNqblqk0xl4CVV3SQbGwK7Ac= github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f h1:o/kfcElHqOiXqcou5a3rIlMc7oJbMQkeLk0VQJ7zgqY= github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f/go.mod h1:i/u985jwjWRlyHXQbwatDASoW0RMlZ/3i9yJHE2xLkI= -github.com/cockroachdb/pebble v0.0.0-20210719141320-8c3bd06debb5 h1:Igd6YmtOZ77EgLAIaE9+mHl7+sAKaZ5m4iMI0Dz/J2A= +github.com/cockroachdb/pebble v0.0.0-20220415182917-06c9d3be25b3 h1:snjwkhKc/ZtYIC/hg6UoT5PrhXcZmCsaB+z0bonMDcU= +github.com/cockroachdb/pebble v0.0.0-20220415182917-06c9d3be25b3/go.mod h1:buxOO9GBtOcq1DiXDpIPYrmxY020K2A8lOrwno5FetU= github.com/cockroachdb/redact v1.0.8 h1:8QG/764wK+vmEYoOlfobpe12EQcS81ukx/a4hdVMxNw= github.com/cockroachdb/redact v1.0.8/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2 h1:IKgmqgMQlVJIZj19CdocBeSfSaiCbEBZGKODaixqtHM= @@ -102,18 +120,26 @@ github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSs github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/decred/dcrd/dcrec/secp256k1/v4 v4.2.0 h1:8UrgZ3GkP4i/CLijOJx79Yu+etlyjdBU4sfcs2WYQMs= +github.com/decred/dcrd/dcrec/secp256k1/v4 v4.2.0/go.mod h1:v57UDF4pDQJcEfFUCRop3lJL149eHGSe9Jvczhzjo/0= github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= github.com/dgraph-io/ristretto v0.1.1 h1:6CWw5tJNgpegArSHpNHJKldNeq03FQCwYvfMVWajOK8= +github.com/dgraph-io/ristretto v0.1.1/go.mod h1:S1GPSBCYCIhmVNfcth17y2zZtQT6wzkzgwUve0VDWWA= github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= github.com/dgryski/go-farm v0.0.0-20190104051053-3adb47b1fb0f/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13 h1:fAjc9m62+UWV/WAFKLNi6ZS0675eEUC9y3AlwSbQu1Y= github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= github.com/docker/go-units v0.4.0 h1:3uh0PgVws3nIA0Q+MwDC8yjEPf9zjRfZZWXZYDct3Tw= +github.com/docker/go-units v0.4.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= +github.com/dolthub/maphash v0.1.0 h1:bsQ7JsF4FkkWyrP3oCnFJgrCUAFbFf3kOl4L/QxPDyQ= +github.com/dolthub/maphash v0.1.0/go.mod h1:gkg4Ch4CdCDu5h6PMriVLawB7koZ+5ijb9puGMV50a4= +github.com/dolthub/swiss v0.2.1 h1:gs2osYs5SJkAaH5/ggVJqXQxRXtWshF6uE0lgR/Y3Gw= +github.com/dolthub/swiss v0.2.1/go.mod h1:8AhKZZ1HK7g18j7v7k6c5cYIGEZJcPn0ARsai8cUrh0= github.com/dustin/go-humanize v1.0.0 h1:VSnTsYCnlFHaM2/igO1h6X3HA71jcobQuxemgkq4zYo= github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385/go.mod h1:0vRUJqYpeSZifjYj7uP3BG/gKcuzL9xWVV/Y+cK33KM= github.com/emirpasic/gods v1.18.1 h1:FXtiHYKDGKCW2KzwZKx0iC0PQmdlorYgdFG9jPXJ1Bc= +github.com/emirpasic/gods v1.18.1/go.mod h1:8tpGGwCnJ5H4r6BWwaV6OrWmMoPhUl5jm/FMNAnJvWQ= github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= @@ -122,12 +148,15 @@ github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7 github.com/etcd-io/bbolt v1.3.3/go.mod h1:ZF2nL25h33cCyBtcyWeZ2/I3HQOfTP+0PIEvHjkjCrw= github.com/fasthttp-contrib/websocket v0.0.0-20160511215533-1f3b11f56072/go.mod h1:duJ4Jxv5lDcvg4QuQr0oowTf7dz4/CR8NtyCooz9HL8= github.com/fatih/color v1.15.0 h1:kOqh6YHBtK8aywxGerMG2Eq3H6Qgoqeo13Bk2Mv/nBs= +github.com/fatih/color v1.15.0/go.mod h1:0h5ZqXfHYED7Bhv2ZJamyIOUej9KtShiJESRwBDUSsw= github.com/fatih/structs v1.1.0/go.mod h1:9NiDSp5zOcgEDl+j00MP/WkGVPOlPRLejGD8Ga6PJ7M= github.com/flosch/pongo2 v0.0.0-20190707114632-bbf5a6c351f4/go.mod h1:T9YF2M40nIgbVgp3rreNmTged+9HrbNTIQf1PsaIiTA= github.com/form3tech-oss/jwt-go v3.2.5+incompatible h1:/l4kBbb4/vGSsdtB5nUe8L7B9mImVMaBPw9L/0TBHU8= +github.com/form3tech-oss/jwt-go v3.2.5+incompatible/go.mod h1:pbq4aXjuKjdthFRnoDwaVPLA+WlJuPGy+QneDUgJi2k= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= github.com/fsnotify/fsnotify v1.6.0 h1:n+5WquG0fcWoWp6xPWfHdbskMCQaFnG6PfBrh1Ky4HY= +github.com/fsnotify/fsnotify v1.6.0/go.mod h1:sl3t1tCWJFWoRz9R8WJCbQihKKwmorjAbSClcnxKAGw= github.com/gavv/httpexpect v2.0.0+incompatible/go.mod h1:x+9tiU1YnrOvnB725RkpoLv1M62hOWzwo5OXotisrKc= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/gin-contrib/sse v0.0.0-20190301062529-5545eab6dad3/go.mod h1:VJ0WA2NBN22VlZ2dKZQPAPnyWw5XTlK1KymzLKsr59s= @@ -144,15 +173,18 @@ github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9 github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-logr/logr v1.2.3 h1:2DntVwHkVopvECVRSlL5PSo9eG+cAkDCuckLubN+rq0= +github.com/go-logr/logr v1.2.3/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-martini/martini v0.0.0-20170121215854-22fa46961aab/go.mod h1:/P9AEU963A2AYjv4d1V5eVL1CQbEJq6aCNHDDjibzu8= github.com/go-ole/go-ole v1.2.6 h1:/Fpf6oFPoeFik9ty7siob0G6Ke8QvQEuVcuChpwXzpY= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-sql-driver/mysql v1.7.1 h1:lUIinVbN1DY0xBg0eMOzmmtGoHwWBbvnWubQUrtU8EI= +github.com/go-sql-driver/mysql v1.7.1/go.mod h1:OXbVy3sEdcQ2Doequ6Z5BW6fXNQTmx+9S1MCJN5yJMI= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee/go.mod h1:L0fX3K22YWvt/FAX9NnzrNzcI4wNYi9Yku4O0LKYflo= github.com/gobwas/pool v0.2.0/go.mod h1:q8bcK0KcYlCgd9e7WYLm9LpyS+YeLd8JVDW6WezmKEw= github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/EM= github.com/goccy/go-json v0.10.2 h1:CrxCmQqYDkv1z7lO7Wbh2HN93uovUHgrECaO5ZrCXAU= +github.com/goccy/go-json v0.10.2/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/gogo/googleapis v0.0.0-20180223154316-0cd9801be74a/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= github.com/gogo/protobuf v0.0.0-20171007142547-342cbe0a0415/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= @@ -165,10 +197,12 @@ github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/gogo/status v1.1.0/go.mod h1:BFv9nrluPLmrS0EmGVvLaPNmRosr9KapBYd5/hpY1WM= github.com/golang-jwt/jwt v3.2.1+incompatible h1:73Z+4BJcrTC+KczS6WvTPvRGOp1WmfEP4Q1lOd9Z/+c= +github.com/golang-jwt/jwt v3.2.1+incompatible/go.mod h1:8pz2t5EyA70fFQQSrl6XZXzqecmYZeUEB8OUGHkxJ+I= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/glog v1.1.1 h1:jxpi2eWoU84wbX9iIEyAeeoac3FLuifZpY9tcNUD9kw= github.com/golang/glog v1.1.1/go.mod h1:zR+okUeTbrL6EL3xHUDxZuEtGv04p5shwip1+mL/rLQ= github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da h1:oI5xCqsCo564l8iNU+DwB5epxmsaqB+rhGL0m5jtYqE= +github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/protobuf v0.0.0-20180814211427-aa810b61a9c7/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -207,18 +241,23 @@ github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeN github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/gofuzz v1.1.0 h1:Hsa8mG0dQ46ij8Sl2AYJDUv1oA9/d6Vk+3LG99Oe02g= +github.com/google/gofuzz v1.1.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/pprof v0.0.0-20211122183932-1daafda22083 h1:c8EUapQFi+kjzedr4c6WqbwMdmB95+oDBWZ5XFHFYxY= +github.com/google/pprof v0.0.0-20211122183932-1daafda22083/go.mod h1:KgnwoLYCZ8IQu3XUZ8Nc/bM9CCZFOyjUNOSygVozoDg= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I= github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/enterprise-certificate-proxy v0.2.3 h1:yk9/cqRKtT9wXZSsRH9aurXEpJX+U6FLtpYTdC3R06k= +github.com/googleapis/enterprise-certificate-proxy v0.2.3/go.mod h1:AwSRAtLfXpU5Nm3pW+v7rGDHp09LsPtGY9MduiEsR9k= github.com/googleapis/gax-go/v2 v2.7.1 h1:gF4c0zjUP2H/s/hEGyLA3I0fA2ZWjzYiONAD6cvPr8A= +github.com/googleapis/gax-go/v2 v2.7.1/go.mod h1:4orTrqY6hXxxaUL4LHIPl6lGo8vAE38/qKbhSAKP6QI= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gorilla/mux v1.8.0 h1:i40aqfkR1h2SlN9hojwV5ZA91wcXFOvkdNIeFDP5koI= github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= github.com/gorilla/websocket v1.4.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= github.com/gorilla/websocket v1.4.2 h1:+/TMaTYc4QFitKJxsQ7Yye35DkWvkdLcvGKqM+x0Ufc= +github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 h1:+9834+KizmvFV7pXQGSXQTsaWhq2GjuNUt0aUU0YBYw= github.com/grpc-ecosystem/go-grpc-middleware v1.3.0/go.mod h1:z0ButlSOZa5vEBq9m2m2hlwIgKw+rp3sdCBRoJY+30Y= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho= @@ -230,9 +269,11 @@ github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/hydrogen18/memlistener v0.0.0-20141126152155-54553eb933fb/go.mod h1:qEIFzExnS6016fRpRfxrExeVn2gbClQA99gQhnIcdhE= github.com/iancoleman/strcase v0.2.0 h1:05I4QRnGpI0m37iZQRuskXh+w77mr6Z41lwQzuHLwW0= +github.com/iancoleman/strcase v0.2.0/go.mod h1:iwCmte+B7n89clKwxIoIXy/HfoL7AsD47ZCWhYzw7ho= github.com/imkira/go-interpol v1.1.0/go.mod h1:z0h2/2T3XF8kyEPpRgJ3kmNv+C43p+I/CoI+jC3w2iA= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= github.com/inconshreveable/mousetrap v1.1.0 h1:wN+x4NVGpMsO7ErUn/mUI3vEoE6Jt13X2s0bqwp9tc8= +github.com/inconshreveable/mousetrap v1.1.0/go.mod h1:vpF70FUmC8bwa3OWnCshd2FqLfsEA9PFc4w1p2J65bw= github.com/influxdata/tdigest v0.0.1 h1:XpFptwYmnEKUqmkcDjrzffswZ3nvNeevbUSLPP/ZzIY= github.com/influxdata/tdigest v0.0.1/go.mod h1:Z0kXnxzbTC2qrx4NaIzYkE1k66+6oEDQTvL95hQFh5Y= github.com/iris-contrib/blackfriday v2.0.0+incompatible/go.mod h1:UzZ2bDEoaSGPbkg6SAB4att1aAwTmVIx/5gCVqeyUdI= @@ -240,16 +281,25 @@ github.com/iris-contrib/go.uuid v2.0.0+incompatible/go.mod h1:iz2lgM/1UnEf1kP0L/ github.com/iris-contrib/i18n v0.0.0-20171121225848-987a633949d0/go.mod h1:pMCz62A0xJL6I+umB2YTlFRwWXaDFA0jy+5HzGiJjqI= github.com/iris-contrib/schema v0.0.1/go.mod h1:urYA3uvUNG1TIIjOSCzHr9/LmbQo8LrOcOqfqxa4hXw= github.com/jedib0t/go-pretty/v6 v6.2.2 h1:o3McN0rQ4X+IU+HduppSp9TwRdGLRW2rhJXy9CJaCRw= +github.com/jedib0t/go-pretty/v6 v6.2.2/go.mod h1:+nE9fyyHGil+PuISTCrp7avEdo6bqoMwqZnuiK2r2a0= github.com/jellydator/ttlcache/v3 v3.0.1 h1:cHgCSMS7TdQcoprXnWUptJZzyFsqs18Lt8VVhRuZYVU= github.com/jellydator/ttlcache/v3 v3.0.1/go.mod h1:WwTaEmcXQ3MTjOm4bsZoDFiCu/hMvNWLO1w67RXz6h4= +github.com/jfcg/sixb v1.3.8 h1:BKPp/mIFCkKnnqhbgasI4wO/BYas6NHNcUCowUfTzSI= +github.com/jfcg/sixb v1.3.8/go.mod h1:UWrAr1q9s7pSPPqZNccmQM4N75p8GvuBYdFuq+09Qns= +github.com/jfcg/sorty/v2 v2.1.0 h1:EjrVSL3cDRxBt/ehiYCIv10F7YHYbTzEmdv7WbkkN1k= +github.com/jfcg/sorty/v2 v2.1.0/go.mod h1:JpcSKlmtGOOAGyTdWN2ErjvxeMSJVYBsylAKepIxmNg= github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9YPoQUg= +github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= github.com/joho/sqltocsv v0.0.0-20210428211105-a6d6801d59df h1:Zrb0IbuLOGHL7nrO2WrcuNWgDTlzFv3zY69QMx4ggQE= +github.com/joho/sqltocsv v0.0.0-20210428211105-a6d6801d59df/go.mod h1:mAVCUAYtW9NG31eB30umMSLKcDt6mCUWSjoSn5qBh0k= github.com/jonboulle/clockwork v0.2.2 h1:UOGuzwb1PwsrDAObMuhUnj0p5ULPj8V/xJ7Kx9qUBdQ= +github.com/jonboulle/clockwork v0.2.2/go.mod h1:Pkfl5aHPm1nk2H9h0bjmnJD/BcgbGXUBGnn1kMkgxc8= github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.11/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= +github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/juju/errors v0.0.0-20181118221551-089d3ea4e4d5/go.mod h1:W54LbzXuIE0boCoNJfwqpmkKJ1O4TCTZMetAt6jGk7Q= github.com/juju/loggo v0.0.0-20180524022052-584905176618/go.mod h1:vgyd7OREkbtVEN/8IXZe5Ooef3LQePvuBm9UWj6ZL8U= @@ -268,8 +318,8 @@ github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+o github.com/klauspost/compress v1.8.2/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.9.0/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.9.5/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= -github.com/klauspost/compress v1.16.5 h1:IFV2oUNUzZaz+XyusxpLzpzS8Pt5rh0Z16For/djlyI= -github.com/klauspost/compress v1.16.5/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= +github.com/klauspost/compress v1.17.0 h1:Rnbp4K9EjcDuVuHtd0dgA4qNuv9yKDYKK1ulpJwgrqM= +github.com/klauspost/compress v1.17.0/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= github.com/klauspost/cpuid v1.2.1/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= github.com/klauspost/cpuid v1.3.1 h1:5JNjFYYQrZeKRJ0734q51WCEEn2huer72Dc7K+R/b6s= github.com/klauspost/cpuid v1.3.1/go.mod h1:bYW4mA6ZgKPob1/Dlai2LviZJO7KGI3uoWLd42rAQw4= @@ -284,25 +334,35 @@ github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/kylelemons/godebug v1.1.0 h1:RPNrshWIDI6G2gRW9EHilWtl7Z6Sb1BR0xunSBf0SNc= +github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+fNqagV/RAw= github.com/labstack/echo/v4 v4.1.11/go.mod h1:i541M3Fj6f76NZtHSj7TXnyM8n2gaodfvfxNnFqi74g= github.com/labstack/gommon v0.3.0/go.mod h1:MULnywXg0yavhxWKc+lOruYdAhDwPK9wf0OL7NoOu+k= github.com/lestrrat-go/blackmagic v1.0.1 h1:lS5Zts+5HIC/8og6cGHb0uCcNCa3OUt1ygh3Qz2Fe80= +github.com/lestrrat-go/blackmagic v1.0.1/go.mod h1:UrEqBzIR2U6CnzVyUtfM6oZNMt/7O7Vohk2J0OGSAtU= github.com/lestrrat-go/httpcc v1.0.1 h1:ydWCStUeJLkpYyjLDHihupbn2tYmZ7m22BGkcvZZrIE= +github.com/lestrrat-go/httpcc v1.0.1/go.mod h1:qiltp3Mt56+55GPVCbTdM9MlqhvzyuL6W/NMDA8vA5E= github.com/lestrrat-go/httprc v1.0.4 h1:bAZymwoZQb+Oq8MEbyipag7iSq6YIga8Wj6GOiJGdI8= +github.com/lestrrat-go/httprc v1.0.4/go.mod h1:mwwz3JMTPBjHUkkDv/IGJ39aALInZLrhBp0X7KGUZlo= github.com/lestrrat-go/iter v1.0.2 h1:gMXo1q4c2pHmC3dn8LzRhJfP1ceCbgSiT9lUydIzltI= +github.com/lestrrat-go/iter v1.0.2/go.mod h1:Momfcq3AnRlRjI5b5O8/G5/BvpzrhoFTZcn06fEOPt4= github.com/lestrrat-go/jwx/v2 v2.0.11 h1:ViHMnaMeaO0qV16RZWBHM7GTrAnX2aFLVKofc7FuKLQ= +github.com/lestrrat-go/jwx/v2 v2.0.11/go.mod h1:ZtPtMFlrfDrH2Y0iwfa3dRFn8VzwBrB+cyrm3IBWdDg= github.com/lestrrat-go/option v1.0.1 h1:oAzP2fvZGQKWkvHa1/SAcFolBEca1oN+mQ7eooNBEYU= +github.com/lestrrat-go/option v1.0.1/go.mod h1:5ZHFbivi4xwXxhxY9XHDe2FHo6/Z7WWmtT7T5nBBp3I= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a h1:N9zuLhTvBSRt0gWSiJswwQ2HqDmtX/ZCDJURnKUt1Ik= github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a/go.mod h1:JKx41uQRwqlTZabZc+kILPrO/3jlKnQ2Z8b7YiVw5cE= github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/mattn/go-colorable v0.1.2/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= github.com/mattn/go-colorable v0.1.13 h1:fFA4WZxdEF4tXPZVKMLwD8oUnCTTo08duU7wxecdEvA= +github.com/mattn/go-colorable v0.1.13/go.mod h1:7S9/ev0klgBDR4GtXTXX8a3vIGJpMovkB8vQcUbaXHg= github.com/mattn/go-isatty v0.0.7/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= github.com/mattn/go-isatty v0.0.8/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= github.com/mattn/go-isatty v0.0.9/go.mod h1:YNRxwqDuOph6SZLI9vUUz6OYw3QyUt7WiY2yME+cCiQ= github.com/mattn/go-isatty v0.0.18 h1:DOKFKCQ7FNG2L1rbrmstDN4QVRdS89Nkh85u68Uwp98= +github.com/mattn/go-isatty v0.0.18/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y= github.com/mattn/go-runewidth v0.0.14 h1:+xnbZSEeDbOIg5/mE6JF0w6n9duR1l3/WmbinWVwUuU= +github.com/mattn/go-runewidth v0.0.14/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= github.com/mattn/goveralls v0.0.2/go.mod h1:8d1ZMHsd7fW6IRPKQh46F2WRpyib5/X4FOpevwGNQEw= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zkfA9PSy5pEvNWRP0ET0TIVo= @@ -318,6 +378,7 @@ github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= +github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= @@ -335,13 +396,16 @@ github.com/ninedraft/israce v0.0.3 h1:F/Y1u6OlvgE75Syv1WbBatyg3CjGCdxLojLE7ydv2y github.com/ninedraft/israce v0.0.3/go.mod h1:4L1ITFl340650ZmexVbUcBwG18ozlWiMe47pltZAmn4= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= +github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= github.com/onsi/ginkgo v1.13.0/go.mod h1:+REjRxOmWfHCjfv9TTWB1jD1Frx4XydAD3zm1lskyM0= github.com/onsi/ginkgo v1.16.5 h1:8xi0RTUf59SOSfEtZMvwTvXYMzG4gV23XVHOZiXNtnE= +github.com/onsi/ginkgo v1.16.5/go.mod h1:+E8gABHa3K6zRBolWtd+ROzc/U5bkGt0FwiG042wbpU= github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= github.com/onsi/gomega v1.20.1 h1:PA/3qinGoukvymdIDV8pii6tiZgC8kbmJO6Z5+b002Q= +github.com/onsi/gomega v1.20.1/go.mod h1:DtrZpjmvpn2mPm4YWQa0/ALMDj9v4YxLgojwPeREyVo= github.com/opentracing/basictracer-go v1.1.0 h1:Oa1fTSBvAl8pa3U+IJYqrKm0NALwH9OsgwOqDv4xJW0= github.com/opentracing/basictracer-go v1.1.0/go.mod h1:V2HZueSJEp879yv285Aap1BS69fQMD+MNP1mRs6mBQc= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= @@ -360,6 +424,7 @@ github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32/go.mod h1:X2r9ue github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c h1:CgbKAHto5CQgWM9fSBIvaxsJHuGP0uM74HXtv3MyyGQ= github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= github.com/pingcap/fn v1.0.0 h1:CyA6AxcOZkQh52wIqYlAmaVmF6EvrcqFywP463pjA8g= +github.com/pingcap/fn v1.0.0/go.mod h1:u9WZ1ZiOD1RpNhcI42RucFh/lBuzTu6rw88a+oF2Z24= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= @@ -371,9 +436,14 @@ github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22 h1:2SOzvGvE8beiC1Y4g github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 h1:QV6jqlfOkh8hqvEAgwBZa+4bSgO0EeKC7s5c6Luam2I= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21/go.mod h1:QYnjfA95ZaMefyl1NO8oPtKeb8pYUdnDVhQgf+qdpjM= -github.com/pingcap/tipb v0.0.0-20230607071926-bda24015c2d6 h1:D79RE4RVhq2ic8sqDSv7QdL0tT5aZV3CaCXUAT41iWc= -github.com/pingcap/tipb v0.0.0-20230607071926-bda24015c2d6/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= +github.com/pingcap/tidb v1.1.0-beta.0.20230927025416-38023987346f h1:dn8jgHp2Xv/SfBMDvRAIZiRnQsEqI25Hzi89BHnhXa0= +github.com/pingcap/tidb v1.1.0-beta.0.20230927025416-38023987346f/go.mod h1:xlM+ek4NXAh8wgUF14D3K4S6Ba6O2eohp3glt55KeiE= +github.com/pingcap/tidb/parser v0.0.0-20230927025416-38023987346f h1:FjlKZ4IoMbxkwOAMvF8nE/ARq4t27VPEqBO9ijSdYtc= +github.com/pingcap/tidb/parser v0.0.0-20230927025416-38023987346f/go.mod h1:cwq4bKUlftpWuznB+rqNwbN0xy6/i5SL/nYvEKeJn4s= +github.com/pingcap/tipb v0.0.0-20230919054518-dfd7d194838f h1:NCiI4Wyu4GkViLGTu6cYcxt79LZ1SenBBQX1OwEV6Jg= +github.com/pingcap/tipb v0.0.0-20230919054518-dfd7d194838f/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/browser v0.0.0-20210115035449-ce105d075bb4 h1:Qj1ukM4GlMWXNdMBuXcXfz/Kw9s1qm0CLY32QxuSImI= +github.com/pkg/browser v0.0.0-20210115035449-ce105d075bb4/go.mod h1:N6UoU20jOqggOuDwUaBQpluzLNDqif3kq9z2wpdYEfQ= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= @@ -407,22 +477,26 @@ github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= -github.com/prometheus/procfs v0.11.0 h1:5EAgkfkMl659uZPbe9AS2N68a7Cc1TJbPEuGzFuRbyk= -github.com/prometheus/procfs v0.11.0/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM= +github.com/prometheus/procfs v0.11.1 h1:xRC8Iq1yyca5ypa9n1EZnWZkt7dwcoRPQwX/5gwaUuI= +github.com/prometheus/procfs v0.11.1/go.mod h1:eesXgaPo1q7lBpVMoMy0ZOFTth9hBn4W/y0/p/ScXhY= github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec h1:W09IVJc94icq4NjY3clb7Lk8O1qJ8BdBEF8z0ibU0rE= github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= github.com/rivo/uniseg v0.4.4 h1:8TfxU8dW6PdqD27gjM8MVNuicgxIjxpm4K7x4jp8sis= +github.com/rivo/uniseg v0.4.4/go.mod h1:FN3SvrM+Zdj16jyLfmOkMNblXMcoc8DfTHruCPUcx88= +github.com/robfig/cron/v3 v3.0.1 h1:WdRxkvbJztn8LMz/QEvLN5sBU+xKpSqwwUO1Pjr4qDs= +github.com/robfig/cron/v3 v3.0.1/go.mod h1:eQICP3HwyT7UooqI/z+Ov+PtYAWygg1TEWWzGIFLtro= github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.9.0/go.mod h1:WtVeX8xhTBvf0smdhujwtBcq4Qrzq/fJaraNFVN+nFs= -github.com/rogpeppe/go-internal v1.10.0 h1:TMyTOH3F/DB16zRVcYyreMH6GnZZrwQVAoYjRBZyWFQ= -github.com/rogpeppe/go-internal v1.10.0/go.mod h1:UQnix2H7Ngw/k4C5ijL5+65zddjncjaFoBhdsK/akog= +github.com/rogpeppe/go-internal v1.11.0 h1:cWPaGQEPrBb5/AsnsZesgZZ9yb1OQ+GOISoDNXVBh4M= +github.com/rogpeppe/go-internal v1.11.0/go.mod h1:ddIwULY96R17DhadqLgMfk9H9tvdUzkipdSkR5nkCZA= github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= github.com/ryanuber/columnize v2.1.0+incompatible/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= github.com/sasha-s/go-deadlock v0.2.0 h1:lMqc+fUb7RrFS3gQLtoQsJ7/6TV/pAIFvBsqX73DK8Y= github.com/sasha-s/go-deadlock v0.2.0/go.mod h1:StQn567HiB1fF2yJ44N9au7wOhrPS3iZqiDbRupzT10= github.com/sclevine/agouti v3.0.0+incompatible/go.mod h1:b4WX9W9L1sfQKXeJf1mUTLZKJ48R1S7H23Ji7oFO5Bw= github.com/segmentio/asm v1.2.0 h1:9BQrFxC+YOHJlTlHGkTrFWf59nbL3XnCoFLTwDCI7ys= +github.com/segmentio/asm v1.2.0/go.mod h1:BqMnlJP91P8d+4ibuonYZw9mfnzI9HfxselHZr5aAcs= github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= github.com/shirou/gopsutil/v3 v3.21.12/go.mod h1:BToYZVTlSVlfazpDDYFnsVZLaoRG+g8ufT6fPQLdJzA= github.com/shirou/gopsutil/v3 v3.23.5 h1:5SgDCeQ0KW0S4N0znjeM/eFHXXOKyv2dVNgRq/c9P6Y= @@ -432,25 +506,31 @@ github.com/shoenig/go-m1cpu v0.1.6/go.mod h1:1JJMcUBvfNwpq05QDQVAnx3gUHr9IYF7GNg github.com/shoenig/test v0.6.4 h1:kVTaSd7WLz5WZ2IaoM0RSzRsUD+m8wRR+5qvntpn4LU= github.com/shoenig/test v0.6.4/go.mod h1:byHiCGXqrVaflBLAMq/srcZIHynQPQgeyvkvXnjqq0k= github.com/shurcooL/httpfs v0.0.0-20190707220628-8d4bc4ba7749 h1:bUGsEnyNbVPw06Bs80sCeARAlK8lhwqGyi6UT8ymuGk= +github.com/shurcooL/httpfs v0.0.0-20190707220628-8d4bc4ba7749/go.mod h1:ZY1cvUeJuFPAdZ/B6v7RHavJWZn2YPVFQ1OSXhCGOkg= github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd h1:ug7PpSOB5RBPK1Kg6qskGBoP3Vnj/aNYFTznWvlkGo0= +github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= -github.com/sirupsen/logrus v1.9.2 h1:oxx1eChJGI6Uks2ZC4W1zpLlVgqB8ner4EuQwV4Ik1Y= +github.com/sirupsen/logrus v1.9.3 h1:dueUQJ1C2q9oE3F7wvmSGAaVtTmUizReu6fjN8uqzbQ= +github.com/sirupsen/logrus v1.9.3/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= github.com/soheilhy/cmux v0.1.5 h1:jjzc5WVemNEDTLwv9tlmemhC73tI08BNOIGwBOo10Js= +github.com/soheilhy/cmux v0.1.5/go.mod h1:T7TcVDs9LWfQgPlPsdngu6I6QIoyIFZDDC6sNE1GqG0= github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU= github.com/spf13/cobra v1.7.0 h1:hyqWnYt1ZQShIddO5kBpj3vu05/++x6tJ6dg8EC572I= +github.com/spf13/cobra v1.7.0/go.mod h1:uLxZILRyS/50WlhOIKD7W6V5bgeIt+4sICxh6uRMrb0= github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s= github.com/spkg/bom v1.0.0 h1:S939THe0ukL5WcTGiGqkgtaW5JW+O6ITaIlpJXTYY64= +github.com/spkg/bom v1.0.0/go.mod h1:lAz2VbTuYNcvs7iaFF8WW0ufXrHShJ7ck1fYFFbVXJs= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= @@ -467,6 +547,7 @@ github.com/stretchr/testify v1.8.3/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXl github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= +github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a h1:J/YdBZ46WKpXsxsW93SG+q0F8KI+yFrcIDT4c/RNoc4= github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a/go.mod h1:h4xBhSNtOeEosLJ4P7JyKXX7Cabg7AVkWCK5gV2vOrM= github.com/tidwall/gjson v1.14.1 h1:iymTbGkQBhveq21bEvAQ81I0LEBork8BFe1CUZXdyuo= @@ -475,8 +556,8 @@ github.com/tidwall/match v1.1.1 h1:+Ho715JplO36QYgwN9PGYNhgZvoUSc9X2c80KVTi+GA= github.com/tidwall/match v1.1.1/go.mod h1:eRSPERbgtNPcGhD8UCthc6PmLEQXEWd3PRB5JTxsfmM= github.com/tidwall/pretty v1.2.0 h1:RWIZEg2iJ8/g6fDDYzMpobmaoGh5OLl4AXtGUGPcqCs= github.com/tidwall/pretty v1.2.0/go.mod h1:ITEVvHYasfjBbM0u2Pg8T2nJnzm8xPwvNhhsoaGGjNU= -github.com/tikv/pd/client v0.0.0-20230724080549-de985b8e0afc h1:IUg0j2nWoGYj3FQ3vA3vg97fPSpJEZQrDpgF8RkMLEU= -github.com/tikv/pd/client v0.0.0-20230724080549-de985b8e0afc/go.mod h1:wfHRc4iYaqJiOQZCHcrF+r4hYnkGDaYWDfcicee//pc= +github.com/tikv/pd/client v0.0.0-20230912103610-2f57a9f050eb h1:hAcH9tFjQzQ3+ofrAHm4ajOTLliYCOfXpj3+boKOtac= +github.com/tikv/pd/client v0.0.0-20230912103610-2f57a9f050eb/go.mod h1:E+6qtPu8fJm5kNjvKWPVFqSgNAFPk07y2EjD03GWzuI= github.com/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs= github.com/tklauser/go-sysconf v0.3.11 h1:89WgdJhk5SNwJfu+GKyYveZ4IaJ7xAkecBo+KdJV0CM= github.com/tklauser/go-sysconf v0.3.11/go.mod h1:GqXfhXY3kiPa0nAXPDIQIWzJbMCB7AmcWpGR8lSZfqI= @@ -484,6 +565,7 @@ github.com/tklauser/numcpus v0.3.0/go.mod h1:yFGUr7TUHQRAhyqBcEg0Ge34zDBAsIvJJcy github.com/tklauser/numcpus v0.6.0 h1:kebhY2Qt+3U6RNK7UqpYNA+tJ23IBEGKkB7JQBfDYms= github.com/tklauser/numcpus v0.6.0/go.mod h1:FEZLMke0lhOUG6w2JadTzp0a+Nl8PF/GFkQ5UVIcaL4= github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802 h1:uruHq4dN7GR16kFc5fp3d1RIYzJW5onx8Ybykw2YQFA= +github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/twmb/murmur3 v1.1.6 h1:mqrRot1BRxm+Yct+vavLMou2/iJt0tNVTTC0QoIjaZg= github.com/twmb/murmur3 v1.1.6/go.mod h1:Qq/R7NUyOfr65zD+6Q5IHKsJLwP7exErjN6lyyq3OSQ= github.com/uber/jaeger-client-go v2.30.0+incompatible h1:D6wyKGCecFaSRUpo8lCVbaOOb6ThwMmTEbhRwtKR97o= @@ -498,12 +580,16 @@ github.com/valyala/fasthttp v1.6.0/go.mod h1:FstJa9V+Pj9vQ7OJie2qMHdwemEDaDiSdBn github.com/valyala/fasttemplate v1.0.1/go.mod h1:UQGH1tvbgY+Nz5t2n7tXsz52dQxojPUpymEIMZ47gx8= github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a/go.mod h1:v3UYOV9WzVtRmSR+PDvWpU/qWl4Wa5LApYYX4ZtKbio= github.com/vbauerster/mpb/v7 v7.5.3 h1:BkGfmb6nMrrBQDFECR/Q7RkKCw7ylMetCb4079CGs4w= +github.com/vbauerster/mpb/v7 v7.5.3/go.mod h1:i+h4QY6lmLvBNK2ah1fSreiw3ajskRlBp9AhY/PnuOE= github.com/wangjohn/quickselect v0.0.0-20161129230411-ed8402a42d5f h1:9DDCDwOyEy/gId+IEMrFHLuQ5R/WV0KNxWLler8X2OY= +github.com/wangjohn/quickselect v0.0.0-20161129230411-ed8402a42d5f/go.mod h1:8sdOQnirw1PrcnTJYkmW1iOHtUmblMmGdUOHyWYycLI= github.com/xeipuuv/gojsonpointer v0.0.0-20180127040702-4e3ac2762d5f/go.mod h1:N2zxlSyiKSe5eX1tZViRH5QA0qijqEDrYZiPEAiq3wU= github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415/go.mod h1:GwrjFmJcFw6At/Gs6z4yjiIwzuJ1/+UwLxMQDVQXShQ= github.com/xeipuuv/gojsonschema v1.2.0/go.mod h1:anYRn/JVcOK2ZgGU+IjEV4nwlhoK5sQluxsYJ78Id3Y= github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 h1:eY9dn8+vbi4tKz5Qo6v2eYzo7kUS51QINcR5jNpbZS8= +github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= github.com/xitongsys/parquet-go v1.5.5-0.20201110004701-b09c49d6d457 h1:tBbuFCtyJNKT+BFAv6qjvTFpVdy97IYNaBwGUXifIUs= +github.com/xitongsys/parquet-go v1.5.5-0.20201110004701-b09c49d6d457/go.mod h1:pheqtXeHQFzxJk45lRQ0UIGIivKnLXvialZSFWs81A8= github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q= github.com/yalp/jsonpath v0.0.0-20180802001716-5cc68e5049a0/go.mod h1:/LWChgwKmvncFJFHJ7Gvn9wZArjbV5/FppcK2fKk/tI= github.com/yudai/gojsondiff v1.0.0/go.mod h1:AY32+k2cwILAkW1fbgxQ5mUmMiZFgLIV+FBNExI05xg= @@ -517,28 +603,45 @@ github.com/yusufpapurcu/wmi v1.2.2/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQ github.com/yusufpapurcu/wmi v1.2.3 h1:E1ctvB7uKFMOJw3fdOW32DwGE9I7t++CRUEMKvFoFiw= github.com/yusufpapurcu/wmi v1.2.3/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.etcd.io/bbolt v1.3.6 h1:/ecaJf0sk1l4l6V4awd65v2C3ILy7MSj+s/x1ADCIMU= +go.etcd.io/bbolt v1.3.6/go.mod h1:qXsaaIqmgQH0T+OPdb99Bf+PKfBBQVAdyD6TY9G8XM4= go.etcd.io/etcd/api/v3 v3.5.2 h1:tXok5yLlKyuQ/SXSjtqHc4uzNaMqZi2XsoSPr/LlJXI= go.etcd.io/etcd/api/v3 v3.5.2/go.mod h1:5GB2vv4A4AOn3yk7MftYGHkUfGtDHnEraIjym4dYz5A= go.etcd.io/etcd/client/pkg/v3 v3.5.2 h1:4hzqQ6hIb3blLyQ8usCU4h3NghkqcsohEQ3o3VetYxE= go.etcd.io/etcd/client/pkg/v3 v3.5.2/go.mod h1:IJHfcCEKxYu1Os13ZdwCwIUTUVGYTSAM3YSwc9/Ac1g= go.etcd.io/etcd/client/v2 v2.305.2 h1:ymrVwTkefuqA/rPkSW7/B4ApijbPVefRumkY+stNfS0= +go.etcd.io/etcd/client/v2 v2.305.2/go.mod h1:2D7ZejHVMIfog1221iLSYlQRzrtECw3kz4I4VAQm3qI= go.etcd.io/etcd/client/v3 v3.5.2 h1:WdnejrUtQC4nCxK0/dLTMqKOB+U5TP/2Ya0BJL+1otA= go.etcd.io/etcd/client/v3 v3.5.2/go.mod h1:kOOaWFFgHygyT0WlSmL8TJiXmMysO/nNUlEsSsN6W4o= go.etcd.io/etcd/pkg/v3 v3.5.2 h1:YZUojdoPhOyl5QILYnR8LTUbbNefu/sV4ma+ZMr2tto= +go.etcd.io/etcd/pkg/v3 v3.5.2/go.mod h1:zsXz+9D/kijzRiG/UnFGDTyHKcVp0orwiO8iMLAi+k0= go.etcd.io/etcd/raft/v3 v3.5.2 h1:uCC37qOXqBvKqTGHGyhASsaCsnTuJugl1GvneJNwHWo= +go.etcd.io/etcd/raft/v3 v3.5.2/go.mod h1:G6pCP1sFgbjod7/KnEHY0vHUViqxjkdt6AiKsD0GRr8= go.etcd.io/etcd/server/v3 v3.5.2 h1:B6ytJvS4Fmt8nkjzS2/8POf4tuPhFMluE0lWd4dx/7U= +go.etcd.io/etcd/server/v3 v3.5.2/go.mod h1:mlG8znIEz4N/28GABrohZCBM11FqgGVQcpbcyJgh0j0= go.etcd.io/etcd/tests/v3 v3.5.2 h1:uk7/uMGVebpBDl+roivowHt6gJ5Fnqwik3syDkoSKdo= +go.etcd.io/etcd/tests/v3 v3.5.2/go.mod h1:Jdzbei4uFi9C3xDBfCwckRXjlX0UPooiP4g/zXgBMgQ= go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= +go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= go.opentelemetry.io/contrib v0.20.0 h1:ubFQUn0VCZ0gPwIoJfBJVpeBlyRMxu8Mm/huKWYd9p0= +go.opentelemetry.io/contrib v0.20.0/go.mod h1:G/EtFaa6qaN7+LxqfIAT3GiZa7Wv5DTBUzl5H4LY0Kc= go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.20.0 h1:sO4WKdPAudZGKPcpZT4MJn6JaDmpyLrMPDGGyA1SttE= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.20.0/go.mod h1:oVGt1LRbBOBq1A5BQLlUg9UaU/54aiHw8cgjV3aWZ/E= go.opentelemetry.io/otel v0.20.0 h1:eaP0Fqu7SXHwvjiqDq83zImeehOHX8doTvU9AwXON8g= +go.opentelemetry.io/otel v0.20.0/go.mod h1:Y3ugLH2oa81t5QO+Lty+zXf8zC9L26ax4Nzoxm/dooo= go.opentelemetry.io/otel/exporters/otlp v0.20.0 h1:PTNgq9MRmQqqJY0REVbZFvwkYOA85vbdQU/nVfxDyqg= +go.opentelemetry.io/otel/exporters/otlp v0.20.0/go.mod h1:YIieizyaN77rtLJra0buKiNBOm9XQfkPEKBeuhoMwAM= go.opentelemetry.io/otel/metric v0.20.0 h1:4kzhXFP+btKm4jwxpjIqjs41A7MakRFUS86bqLHTIw8= +go.opentelemetry.io/otel/metric v0.20.0/go.mod h1:598I5tYlH1vzBjn+BTuhzTCSb/9debfNp6R3s7Pr1eU= go.opentelemetry.io/otel/sdk v0.20.0 h1:JsxtGXd06J8jrnya7fdI/U/MR6yXA5DtbZy+qoHQlr8= +go.opentelemetry.io/otel/sdk v0.20.0/go.mod h1:g/IcepuwNsoiX5Byy2nNV0ySUF1em498m7hBWC279Yc= go.opentelemetry.io/otel/sdk/export/metric v0.20.0 h1:c5VRjxCXdQlx1HjzwGdQHzZaVI82b5EbBgOu2ljD92g= +go.opentelemetry.io/otel/sdk/export/metric v0.20.0/go.mod h1:h7RBNMsDJ5pmI1zExLi+bJK+Dr8NQCh0qGhm1KDnNlE= go.opentelemetry.io/otel/sdk/metric v0.20.0 h1:7ao1wpzHRVKf0OQ7GIxiQJA6X7DLX9o14gmVon7mMK8= +go.opentelemetry.io/otel/sdk/metric v0.20.0/go.mod h1:knxiS8Xd4E/N+ZqKmUPf3gTTZ4/0TjTXukfxjzSTpHE= go.opentelemetry.io/otel/trace v0.20.0 h1:1DL6EXUdcg95gukhuRRvLDO/4X5THh/5dIV52lqtnbw= +go.opentelemetry.io/otel/trace v0.20.0/go.mod h1:6GjCW8zgDjwGHGa6GkyeB8+/5vjT16gUEi0Nf1iBdgw= go.opentelemetry.io/proto/otlp v0.7.0 h1:rwOQPCuKAKmwGKq2aVNnYIibI6wnV7EvzgfTCzcdGg8= +go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= @@ -566,8 +669,8 @@ go.uber.org/zap v1.12.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= go.uber.org/zap v1.17.0/go.mod h1:MXVU+bhUf/A7Xi2HNOnopQOrmycQ5Ih87HtOu4q5SSo= go.uber.org/zap v1.19.0/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= go.uber.org/zap v1.21.0/go.mod h1:wjWOCqI0f2ZZrJF/UufIOkiC8ii6tm1iqIsLo76RfJw= -go.uber.org/zap v1.24.0 h1:FiJd5l1UOLj0wCgbSE0rwwXHzEdAZS6hiiSnxJN/D60= -go.uber.org/zap v1.24.0/go.mod h1:2kMP+WWQ8aoFoedH3T2sq6iJ2yDWpHbP0f6MQbS9Gkg= +go.uber.org/zap v1.25.0 h1:4Hvk6GtkucQ790dqmj7l1eEnRdKm3k3ZUrUMS2d5+5c= +go.uber.org/zap v1.25.0/go.mod h1:JIAUzQIH94IC4fOJQm7gMmBJP5k7wQfdcnYdPoEXJYk= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= @@ -577,8 +680,8 @@ golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.7.0/go.mod h1:pYwdfH91IfpZVANVyUOhSIPZaFoJGxTFbZhFTx+dXZU= -golang.org/x/crypto v0.11.0 h1:6Ewdq3tDic1mg5xRO4milcWCfMVQhI4NkqWWvqejpuA= -golang.org/x/crypto v0.11.0/go.mod h1:xgJhtzW8F9jGdVFWZESrid1U1bjeNy4zgy5cRr/CIio= +golang.org/x/crypto v0.13.0 h1:mvySKfSWJ+UKUii46M40LOvyWfN0s2U+46/jDd0e6Ck= +golang.org/x/crypto v0.13.0/go.mod h1:y6Z2r+Rw4iayiXXAIxJIDAJ1zMW4yaTpebo8fPOliYc= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 h1:QLureRX3moex6NVu/Lr4MGakp9FdA7sBHGBmvRW7NaM= @@ -623,12 +726,13 @@ golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96b golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.8.0/go.mod h1:QVkue5JL9kW//ek3r6jTKnTFis1tRmNAW2P1shuFdJc= -golang.org/x/net v0.11.0 h1:Gi2tvZIJyBtO9SDr1q9h5hEQCp/4L2RQ+ar0qjx2oNU= -golang.org/x/net v0.11.0/go.mod h1:2L/ixqYpgIVXmeoSA/4Lu7BzTG4KIyPIryS4IsOd1oQ= +golang.org/x/net v0.15.0 h1:ugBLEUaxABaB5AJqW9enI0ACdci2RUd4eP51NTBvuJ8= +golang.org/x/net v0.15.0/go.mod h1:idbUs1IY1+zTqbi8yxTbhexhEEk5ur9LInksu6HrEpk= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.8.0 h1:6dkIjl3j3LtZ/O3sTgZTMsLKSftL/B8Zgq4huOIIUu8= +golang.org/x/oauth2 v0.8.0/go.mod h1:yr7u4HXZRm1R1kBWqr/xKNqewf0plRYoB7sla+BCIXE= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -680,13 +784,14 @@ golang.org/x/sys v0.2.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.10.0 h1:SqMFp9UcQJZa+pmYuAKjd9xq1f0j5rLcDIk0mj4qAsA= -golang.org/x/sys v0.10.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.12.0 h1:CM0HF96J0hcLAwsHPJZjfdNzs0gftsLfgKt57wWHJ0o= +golang.org/x/sys v0.12.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/term v0.6.0/go.mod h1:m6U89DPEgQRMq3DNkDClhWw02AUbt2daBVO4cn4Hv9U= -golang.org/x/term v0.10.0 h1:3R7pNqamzBraeqj/Tj8qt1aQ2HpmlC+Cx/qL/7hn4/c= +golang.org/x/term v0.12.0 h1:/ZfYdc3zq+q02Rv9vGqTeSItdzZTSNDmfTi0mBAuidU= +golang.org/x/term v0.12.0/go.mod h1:owVbMEjm3cBLCHdkQu9b1opXd4ETQWc3BhuQGKgXgvU= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= @@ -694,8 +799,8 @@ golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.8.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= -golang.org/x/text v0.11.0 h1:LAntKIrcmeSKERyiOh0XMV39LXS8IE9UL2yP7+f5ij4= -golang.org/x/text v0.11.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= +golang.org/x/text v0.13.0 h1:ablQoSUd0tRdKxZewP80B+BaqeKJuVhuRxj/dkrun3k= +golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -730,13 +835,17 @@ golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 h1:H2TDz8ibqkAF6YGhCdN3jS9O0/s90v0rJh3X/OLHEUk= +golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2/go.mod h1:K8+ghG5WaK9qNqU5K3HdILfMLy1f3aNYFI/wnl100a8= gonum.org/v1/gonum v0.0.0-20181121035319-3f7ecaa7e8ca/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo= gonum.org/v1/gonum v0.8.2 h1:CCXrcPKiGGotvnN6jfUsKk4rRqm7q09/YbKb5xCEvtM= +gonum.org/v1/gonum v0.8.2/go.mod h1:oe/vMfY3deqTw+1EZJhuvEW2iwGF1bW9wwu7XCu0+v0= gonum.org/v1/netlib v0.0.0-20181029234149-ec6d1f5cefe6/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= google.golang.org/api v0.114.0 h1:1xQPji6cO2E2vLiI+C/XiFAnsn1WV3mjaEwGLhi3grE= +google.golang.org/api v0.114.0/go.mod h1:ifYI2ZsFK6/uGddGfAD5BMxlnkBqCmqHSDUVi45N5Yg= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.6.7 h1:FZR1q0exgwxzPzp/aF+VccGrSfxfPpkBqjIIEq3ru6c= +google.golang.org/appengine v1.6.7/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= google.golang.org/genproto v0.0.0-20180518175338-11a468237815/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20181004005441-af9cb2a35e7f/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= @@ -777,11 +886,13 @@ gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8 gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= gopkg.in/go-playground/assert.v1 v1.2.1/go.mod h1:9RXL0bg/zibRAgZUYszZSwO/z8Y/a8bDuhia5mkpMnE= gopkg.in/go-playground/validator.v8 v8.18.2/go.mod h1:RX2a/7Ha8BgOhfk7j780h4/u/RRjR0eouCJSH80/M2Y= gopkg.in/inf.v0 v0.9.1 h1:73M5CoZyi3ZLMOyDlQh031Cx6N9NDJ2Vvfl76EDAgDc= +gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw= gopkg.in/mgo.v2 v2.0.0-20180705113604-9856a29383ce/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA= gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= gopkg.in/natefinch/lumberjack.v2 v2.2.1 h1:bBRl1b0OH9s/DuPhuXpNl+VtCaJXFZ5/uEFST95x9zc= @@ -805,12 +916,21 @@ honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWh honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= k8s.io/api v0.27.2 h1:+H17AJpUMvl+clT+BPnKf0E3ksMAzoBBg7CntpSuADo= +k8s.io/api v0.27.2/go.mod h1:ENmbocXfBT2ADujUXcBhHV55RIT31IIEvkntP6vZKS4= k8s.io/apimachinery v0.27.2 h1:vBjGaKKieaIreI+oQwELalVG4d8f3YAMNpWLzDXkxeg= +k8s.io/apimachinery v0.27.2/go.mod h1:XNfZ6xklnMCOGGFNqXG7bUrQCoR04dh/E7FprV6pb+E= k8s.io/klog/v2 v2.90.1 h1:m4bYOKall2MmOiRaR1J+We67Do7vm9KiQVlT96lnHUw= +k8s.io/klog/v2 v2.90.1/go.mod h1:y1WjHnz7Dj687irZUWR/WLkLc5N1YHtjLdmgWjndZn0= k8s.io/utils v0.0.0-20230209194617-a36077c30491 h1:r0BAOLElQnnFhE/ApUsg3iHdVYYPBjNSSOMowRZxxsY= +k8s.io/utils v0.0.0-20230209194617-a36077c30491/go.mod h1:OLgZIPagt7ERELqWJFomSt595RzquPNLL48iOWgYOg0= sigs.k8s.io/json v0.0.0-20221116044647-bc3834ca7abd h1:EDPBXCAspyGV4jQlpZSudPeMmr1bNJefnuqLsRAsHZo= +sigs.k8s.io/json v0.0.0-20221116044647-bc3834ca7abd/go.mod h1:B8JuhiUyNFVKdsE8h686QcCxMaH6HrOAZj4vswFpcB0= sigs.k8s.io/structured-merge-diff/v4 v4.2.3 h1:PRbqxJClWWYMNV1dhaG4NsibJbArud9kFxnAMREiWFE= +sigs.k8s.io/structured-merge-diff/v4 v4.2.3/go.mod h1:qjx8mGObPmV2aSZepjQjbmb2ihdVs8cGKBraizNC69E= sigs.k8s.io/yaml v1.2.0/go.mod h1:yfXDCHCao9+ENCvLSE62v9VSji2MKu5jeNfTrofGhJc= sigs.k8s.io/yaml v1.3.0 h1:a2VclLzOGrwOHDiV8EfBGhvjHvP46CtW5j6POvhYGGo= +sigs.k8s.io/yaml v1.3.0/go.mod h1:GeOyir5tyXNByN85N/dRIT9es5UQNerPYEKK56eTBm8= sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0 h1:ucqkfpjg9WzSUubAO62csmucvxl4/JeW3F4I4909XkM= +sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0/go.mod h1:hI742Nqp5OhwiqlzhgfbWU4mW4yO10fP+LoT9WOswdU= sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 h1:e1sMhtVq9AfcEy8AXNb8eSg6gbzfdpYhoNqnPJa+GzI= +sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67/go.mod h1:L5q+DGLGOQFpo1snNEkLOJT2d1YTW66rWNzatr3He1k= From 1a442527792995593354e6c9298def137446a4e0 Mon Sep 17 00:00:00 2001 From: crazycs Date: Sat, 7 Oct 2023 11:46:05 +0800 Subject: [PATCH 48/49] *: add metrics for batch client (#996) Signed-off-by: crazycs520 Co-authored-by: disksing --- internal/client/client_batch.go | 17 +++++++++++++++-- metrics/metrics.go | 22 +++++++++++++++++----- 2 files changed, 32 insertions(+), 7 deletions(-) diff --git a/internal/client/client_batch.go b/internal/client/client_batch.go index b7159847b9..a8091094aa 100644 --- a/internal/client/client_batch.go +++ b/internal/client/client_batch.go @@ -70,6 +70,7 @@ type batchCommandsEntry struct { // canceled indicated the request is canceled or not. canceled int32 err error + start time.Time } func (b *batchCommandsEntry) isCanceled() bool { @@ -378,11 +379,14 @@ func (a *batchConn) getClientAndSend() { } defer cli.unlockForSend() + now := time.Now() + tiKVBatchWaitToSendDuration := metrics.TiKVBatchWaitDuration.WithLabelValues("wait-to-send", target) req, forwardingReqs := a.reqBuilder.build(func(id uint64, e *batchCommandsEntry) { cli.batched.Store(id, e) if trace.IsEnabled() { trace.Log(e.ctx, "rpc", "send") } + tiKVBatchWaitToSendDuration.Observe(float64(now.Sub(e.start))) }) if req != nil { cli.send("", req) @@ -507,6 +511,14 @@ func (c *batchCommandsClient) isStopped() bool { } func (c *batchCommandsClient) send(forwardedHost string, req *tikvpb.BatchCommandsRequest) { + start := time.Now() + defer func() { + if forwardedHost == "" { + metrics.TiKVBatchConnSendDuration.WithLabelValues(c.target).Observe(time.Since(start).Seconds()) + } else { + metrics.TiKVBatchConnSendDuration.WithLabelValues(forwardedHost).Observe(time.Since(start).Seconds()) + } + }() err := c.initBatchClient(forwardedHost) if err != nil { logutil.BgLogger().Warn( @@ -773,6 +785,7 @@ func sendBatchRequest( req *tikvpb.BatchCommandsRequest_Request, timeout time.Duration, ) (*tikvrpc.Response, error) { + start := time.Now() entry := &batchCommandsEntry{ ctx: ctx, req: req, @@ -780,11 +793,11 @@ func sendBatchRequest( forwardedHost: forwardedHost, canceled: 0, err: nil, + start: start, } timer := time.NewTimer(timeout) defer timer.Stop() - start := time.Now() select { case batchConn.batchCommandsCh <- entry: case <-ctx.Done(): @@ -795,7 +808,7 @@ func sendBatchRequest( return nil, errors.WithMessage(context.DeadlineExceeded, "wait sendLoop") } waitDuration := time.Since(start) - metrics.TiKVBatchWaitDuration.Observe(float64(waitDuration)) + metrics.TiKVBatchWaitDuration.WithLabelValues("wait-to-chan", addr).Observe(float64(waitDuration)) select { case res, ok := <-entry.res: diff --git a/metrics/metrics.go b/metrics/metrics.go index 7a9cfcf367..ff3b138026 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -62,7 +62,8 @@ var ( TiKVLocalLatchWaitTimeHistogram prometheus.Histogram TiKVStatusDuration *prometheus.HistogramVec TiKVStatusCounter *prometheus.CounterVec - TiKVBatchWaitDuration prometheus.Histogram + TiKVBatchWaitDuration *prometheus.HistogramVec + TiKVBatchConnSendDuration *prometheus.HistogramVec TiKVBatchSendLatency prometheus.Histogram TiKVBatchWaitOverLoad prometheus.Counter TiKVBatchPendingRequests *prometheus.HistogramVec @@ -333,15 +334,25 @@ func initMetrics(namespace, subsystem string, constLabels prometheus.Labels) { ConstLabels: constLabels, }, []string{LblResult}) - TiKVBatchWaitDuration = prometheus.NewHistogram( + TiKVBatchWaitDuration = prometheus.NewHistogramVec( prometheus.HistogramOpts{ Namespace: namespace, Subsystem: subsystem, Name: "batch_wait_duration", - Buckets: prometheus.ExponentialBuckets(1, 2, 34), // 1ns ~ 8s - Help: "batch wait duration", + Buckets: prometheus.ExponentialBuckets(64, 2, 34), // 64ns ~ 549s + Help: "batch-cmd wait duration, unit is nanosecond", ConstLabels: constLabels, - }) + }, []string{LblType, LblStore}) + + TiKVBatchConnSendDuration = prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "batch_conn_send_seconds", + Buckets: prometheus.ExponentialBuckets(0.0005, 2, 22), // 0.5ms ~ 1048s + Help: "batch conn send duration", + ConstLabels: constLabels, + }, []string{LblStore}) TiKVBatchSendLatency = prometheus.NewHistogram( prometheus.HistogramOpts{ @@ -768,6 +779,7 @@ func RegisterMetrics() { prometheus.MustRegister(TiKVStatusDuration) prometheus.MustRegister(TiKVStatusCounter) prometheus.MustRegister(TiKVBatchWaitDuration) + prometheus.MustRegister(TiKVBatchConnSendDuration) prometheus.MustRegister(TiKVBatchSendLatency) prometheus.MustRegister(TiKVBatchRecvLatency) prometheus.MustRegister(TiKVBatchWaitOverLoad) From a05564b183b74acbb08f82db077de83b5ca87ce1 Mon Sep 17 00:00:00 2001 From: Hu# Date: Sat, 7 Oct 2023 14:22:57 +0800 Subject: [PATCH 49/49] log: Enhanced log key output (#958) Signed-off-by: husharp Co-authored-by: disksing --- internal/locate/region_cache.go | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/internal/locate/region_cache.go b/internal/locate/region_cache.go index a3eba8d8d8..6f77980151 100644 --- a/internal/locate/region_cache.go +++ b/internal/locate/region_cache.go @@ -1098,7 +1098,8 @@ func (c *RegionCache) findRegionByKey(bo *retry.Backoffer, key []byte, isEndKey if err != nil { // ignore error and use old region info. logutil.Logger(bo.GetCtx()).Error("load region failure", - zap.String("key", util.HexRegionKeyStr(key)), zap.Error(err)) + zap.String("key", util.HexRegionKeyStr(key)), zap.Error(err), + zap.String("encode-key", util.HexRegionKeyStr(c.codec.EncodeRegionKey(key)))) } else { logutil.Eventf(bo.GetCtx(), "load region %d from pd, due to need-reload", lr.GetID()) r = lr @@ -1379,7 +1380,9 @@ func (c *RegionCache) BatchLoadRegionsWithKeyRange(bo *retry.Backoffer, startKey return } if len(regions) == 0 { - err = errors.Errorf("PD returned no region, startKey: %q, endKey: %q", util.HexRegionKeyStr(startKey), util.HexRegionKeyStr(endKey)) + err = errors.Errorf("PD returned no region, start_key: %q, end_key: %q, encode_start_key: %q, encode_end_key: %q", + util.HexRegionKeyStr(startKey), util.HexRegionKeyStr(endKey), + util.HexRegionKeyStr(c.codec.EncodeRegionKey(startKey)), util.HexRegionKeyStr(c.codec.EncodeRegionKey(endKey))) return } @@ -1644,13 +1647,14 @@ func (c *RegionCache) loadRegion(bo *retry.Backoffer, key []byte, isEndKey bool) } if err != nil { if apicodec.IsDecodeError(err) { - return nil, errors.Errorf("failed to decode region range key, key: %q, err: %v", util.HexRegionKeyStr(key), err) + return nil, errors.Errorf("failed to decode region range key, key: %q, err: %v, encode_key: %q", + util.HexRegionKeyStr(key), err, util.HexRegionKey(c.codec.EncodeRegionKey(key))) } backoffErr = errors.Errorf("loadRegion from PD failed, key: %q, err: %v", util.HexRegionKeyStr(key), err) continue } if reg == nil || reg.Meta == nil { - backoffErr = errors.Errorf("region not found for key %q", util.HexRegionKeyStr(key)) + backoffErr = errors.Errorf("region not found for key %q, encode_key: %q", util.HexRegionKeyStr(key), util.HexRegionKey(c.codec.EncodeRegionKey(key))) continue } filterUnavailablePeers(reg) @@ -1789,7 +1793,8 @@ func (c *RegionCache) scanRegions(bo *retry.Backoffer, startKey, endKey []byte, metrics.LoadRegionCacheHistogramWithRegions.Observe(time.Since(start).Seconds()) if err != nil { if apicodec.IsDecodeError(err) { - return nil, errors.Errorf("failed to decode region range key, startKey: %q, limit: %d, err: %v", util.HexRegionKeyStr(startKey), limit, err) + return nil, errors.Errorf("failed to decode region range key, startKey: %q, limit: %d, err: %v, encode_start_key: %q", + util.HexRegionKeyStr(startKey), limit, err, util.HexRegionKeyStr(c.codec.EncodeRegionKey(startKey))) } metrics.RegionCacheCounterWithScanRegionsError.Inc() backoffErr = errors.Errorf( @@ -1804,8 +1809,9 @@ func (c *RegionCache) scanRegions(bo *retry.Backoffer, startKey, endKey []byte, if len(regionsInfo) == 0 { return nil, errors.Errorf( - "PD returned no region, startKey: %q, endKey: %q, limit: %d", + "PD returned no region, startKey: %q, endKey: %q, limit: %d, encode_start_key: %q, encode_end_key: %q", util.HexRegionKeyStr(startKey), util.HexRegionKeyStr(endKey), limit, + util.HexRegionKeyStr(c.codec.EncodeRegionKey(startKey)), util.HexRegionKeyStr(c.codec.EncodeRegionKey(endKey)), ) } regions := make([]*Region, 0, len(regionsInfo))