From e714b6a72609e4ed5488428054e8bb0acea57018 Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Tue, 6 Feb 2024 16:55:17 +0800 Subject: [PATCH 01/21] Register health feedback handler in batch client Signed-off-by: MyonKeminta --- go.mod | 2 ++ go.sum | 4 +-- internal/client/client.go | 43 ++++++++++++++++++++++----------- internal/client/client_batch.go | 12 +++++++++ internal/client/client_test.go | 2 +- internal/locate/region_cache.go | 5 ++++ tikv/kv.go | 10 +++++++- 7 files changed, 60 insertions(+), 18 deletions(-) diff --git a/go.mod b/go.mod index 90299877ab..f1738612a3 100644 --- a/go.mod +++ b/go.mod @@ -59,3 +59,5 @@ require ( gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) + +replace github.com/pingcap/kvproto => github.com/MyonKeminta/kvproto v0.0.0-20240205104331-44403f2da0fd diff --git a/go.sum b/go.sum index d86eb8c675..94beeac2c3 100644 --- a/go.sum +++ b/go.sum @@ -1,6 +1,8 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/MyonKeminta/kvproto v0.0.0-20240205104331-44403f2da0fd h1:E4wJBQ03ld9RchEzVye8g+zqxpTBQQrA1+pxQt1//f8= +github.com/MyonKeminta/kvproto v0.0.0-20240205104331-44403f2da0fd/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= @@ -74,8 +76,6 @@ 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-20231222062942-c0c73f41d0b2 h1:364A6VCS+l0oHBKZKotX9LzmfEtIO/NTccTIQcPp3Ug= -github.com/pingcap/kvproto v0.0.0-20231222062942-c0c73f41d0b2/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= 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.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= diff --git a/internal/client/client.go b/internal/client/client.go index 5a633a865e..d9daa8fad5 100644 --- a/internal/client/client.go +++ b/internal/client/client.go @@ -110,6 +110,12 @@ type Client interface { SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) } +type HealthFeedbackHandler = func(feedback *tikvpb.HealthFeedback) + +type HealthFeedbackReceiver interface { + SetHealthFeedbackHandler(handler HealthFeedbackHandler) +} + type connArray struct { // The target host. target string @@ -127,7 +133,7 @@ type connArray struct { } func newConnArray(maxSize uint, addr string, security config.Security, - idleNotify *uint32, enableBatch bool, dialTimeout time.Duration, m *connMonitor, opts []grpc.DialOption) (*connArray, error) { + idleNotify *uint32, enableBatch bool, dialTimeout time.Duration, m *connMonitor, healthFeedbackHandler *atomic.Pointer[HealthFeedbackHandler], opts []grpc.DialOption) (*connArray, error) { a := &connArray{ index: 0, v: make([]*monitoredConn, maxSize), @@ -136,7 +142,7 @@ func newConnArray(maxSize uint, addr string, security config.Security, dialTimeout: dialTimeout, monitor: m, } - if err := a.Init(addr, security, idleNotify, enableBatch, opts...); err != nil { + if err := a.Init(addr, security, idleNotify, enableBatch, healthFeedbackHandler, opts...); err != nil { return nil, err } return a, nil @@ -228,7 +234,7 @@ func (c *monitoredConn) Close() error { return nil } -func (a *connArray) Init(addr string, security config.Security, idleNotify *uint32, enableBatch bool, opts ...grpc.DialOption) error { +func (a *connArray) Init(addr string, security config.Security, idleNotify *uint32, enableBatch bool, healthFeedbackHandler *atomic.Pointer[HealthFeedbackHandler], opts ...grpc.DialOption) error { a.target = addr opt := grpc.WithTransportCredentials(insecure.NewCredentials()) @@ -307,16 +313,17 @@ func (a *connArray) Init(addr string, security config.Security, idleNotify *uint if allowBatch { batchClient := &batchCommandsClient{ - target: a.target, - conn: conn.ClientConn, - forwardedClients: make(map[string]*batchCommandsStream), - batched: sync.Map{}, - epoch: 0, - closed: 0, - tikvClientCfg: cfg.TiKVClient, - tikvLoad: &a.tikvTransportLayerLoad, - dialTimeout: a.dialTimeout, - tryLock: tryLock{sync.NewCond(new(sync.Mutex)), false}, + target: a.target, + conn: conn.ClientConn, + forwardedClients: make(map[string]*batchCommandsStream), + batched: sync.Map{}, + epoch: 0, + closed: 0, + tikvClientCfg: cfg.TiKVClient, + tikvLoad: &a.tikvTransportLayerLoad, + dialTimeout: a.dialTimeout, + tryLock: tryLock{sync.NewCond(new(sync.Mutex)), false}, + healthFeedbackHandler: healthFeedbackHandler, } a.batchCommandsClients = append(a.batchCommandsClients, batchClient) } @@ -400,6 +407,8 @@ type RPCClient struct { isClosed bool connMonitor *connMonitor + + healthFeedbackHandler *atomic.Pointer[HealthFeedbackHandler] } // NewRPCClient creates a client that manages connections and rpc calls with tikv-servers. @@ -409,7 +418,8 @@ func NewRPCClient(opts ...Opt) *RPCClient { option: &option{ dialTimeout: dialTimeout, }, - connMonitor: &connMonitor{}, + connMonitor: &connMonitor{}, + healthFeedbackHandler: new(atomic.Pointer[HealthFeedbackHandler]), } for _, opt := range opts { opt(cli.option) @@ -461,6 +471,7 @@ func (c *RPCClient) createConnArray(addr string, enableBatch bool, opts ...func( enableBatch, c.option.dialTimeout, c.connMonitor, + c.healthFeedbackHandler, c.option.gRPCDialOptions) if err != nil { @@ -808,6 +819,10 @@ func (c *RPCClient) CloseAddr(addr string) error { return nil } +func (c *RPCClient) SetHealthFeedbackHandler(handler HealthFeedbackHandler) { + c.healthFeedbackHandler.Store(&handler) +} + type spanInfo struct { name string dur uint64 diff --git a/internal/client/client_batch.go b/internal/client/client_batch.go index a9956cd1d9..60f896509e 100644 --- a/internal/client/client_batch.go +++ b/internal/client/client_batch.go @@ -507,6 +507,8 @@ type batchCommandsClient struct { closed int32 // tryLock protects client when re-create the streaming. tryLock + + healthFeedbackHandler *atomic.Pointer[HealthFeedbackHandler] } func (c *batchCommandsClient) isStopped() bool { @@ -641,6 +643,10 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient, tikvTransport continue } + if resp.GetHealthFeedback() != nil { + + } + responses := resp.GetResponses() for i, requestID := range resp.GetRequestIds() { value, ok := c.batched.Load(requestID) @@ -671,6 +677,12 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient, tikvTransport } } +func (c *batchCommandsClient) onHealthFeedback(feedback *tikvpb.HealthFeedback) { + if h := c.healthFeedbackHandler.Load(); h != nil { + (*h)(feedback) + } +} + func (c *batchCommandsClient) recreateStreamingClient(err error, streamClient *batchCommandsStream, epoch *uint64) (stopped bool) { // Forbids the batchSendLoop using the old client and // blocks other streams trying to recreate. diff --git a/internal/client/client_test.go b/internal/client/client_test.go index f5c26df3ca..54b34700d9 100644 --- a/internal/client/client_test.go +++ b/internal/client/client_test.go @@ -107,7 +107,7 @@ func TestGetConnAfterClose(t *testing.T) { func TestCancelTimeoutRetErr(t *testing.T) { req := new(tikvpb.BatchCommandsRequest_Request) - a := newBatchConn(1, 1, nil) + a := newBatchConn(1, 1, nil, nil) ctx, cancel := context.WithCancel(context.TODO()) cancel() diff --git a/internal/locate/region_cache.go b/internal/locate/region_cache.go index e8f652e00f..c6b8bc9cad 100644 --- a/internal/locate/region_cache.go +++ b/internal/locate/region_cache.go @@ -55,6 +55,7 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/kvproto/pkg/tikvpb" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/tikv/client-go/v2/config" @@ -3295,3 +3296,7 @@ func (c *RegionCache) markStoreNeedCheck(store *Store) { func (c *RegionCache) getCheckStoreEvents() <-chan struct{} { return c.notifyCheckCh } + +func (c *RegionCache) OnHealthFeedback(feedback *tikvpb.HealthFeedback) { + +} diff --git a/tikv/kv.go b/tikv/kv.go index 47006f9ed4..afe9828513 100644 --- a/tikv/kv.go +++ b/tikv/kv.go @@ -40,6 +40,7 @@ import ( "fmt" "math" "math/rand" + "reflect" "strconv" "sync" "sync/atomic" @@ -228,12 +229,13 @@ func NewKVStore(uuid string, pdClient pd.Client, spkv SafePointKV, tikvclient Cl return nil, err } ctx, cancel := context.WithCancel(context.Background()) + regionCache := locate.NewRegionCache(pdClient) store := &KVStore{ clusterID: pdClient.GetClusterID(context.TODO()), uuid: uuid, oracle: o, pdClient: pdClient, - regionCache: locate.NewRegionCache(pdClient), + regionCache: regionCache, kv: spkv, safePoint: 0, spTime: time.Now(), @@ -243,6 +245,12 @@ func NewKVStore(uuid string, pdClient pd.Client, spkv SafePointKV, tikvclient Cl gP: NewSpool(128, 10*time.Second), } store.clientMu.client = client.NewReqCollapse(client.NewInterceptedClient(tikvclient)) + if r, ok := store.clientMu.client.(client.HealthFeedbackReceiver); ok { + r.SetHealthFeedbackHandler(regionCache.OnHealthFeedback) + logutil.BgLogger().Debug("health feedback handler is set") + } else { + logutil.BgLogger().Warn("health feedback handler not set due to the client type doesn't support", zap.String("type", reflect.TypeOf(tikvclient).String())) + } store.lockResolver = txnlock.NewLockResolver(store) loadOption(store, opt...) From 12166fa756763bd9e9847a9d5ba675157be06ab1 Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Wed, 7 Feb 2024 18:01:51 +0800 Subject: [PATCH 02/21] update protocol Signed-off-by: MyonKeminta --- go.mod | 2 +- go.sum | 4 ++-- internal/client/client_batch.go | 2 +- internal/locate/region_cache.go | 5 +++++ 4 files changed, 9 insertions(+), 4 deletions(-) diff --git a/go.mod b/go.mod index f1738612a3..3b4040a768 100644 --- a/go.mod +++ b/go.mod @@ -60,4 +60,4 @@ require ( gopkg.in/yaml.v3 v3.0.1 // indirect ) -replace github.com/pingcap/kvproto => github.com/MyonKeminta/kvproto v0.0.0-20240205104331-44403f2da0fd +replace github.com/pingcap/kvproto => github.com/MyonKeminta/kvproto v0.0.0-20240207064301-e87f1233c886 diff --git a/go.sum b/go.sum index 94beeac2c3..5c1fc8631e 100644 --- a/go.sum +++ b/go.sum @@ -1,8 +1,8 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/MyonKeminta/kvproto v0.0.0-20240205104331-44403f2da0fd h1:E4wJBQ03ld9RchEzVye8g+zqxpTBQQrA1+pxQt1//f8= -github.com/MyonKeminta/kvproto v0.0.0-20240205104331-44403f2da0fd/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= +github.com/MyonKeminta/kvproto v0.0.0-20240207064301-e87f1233c886 h1:rIfbGwTAP8/yZwnB7aULLPx10aF4aY5Z+NjXAjoibsU= +github.com/MyonKeminta/kvproto v0.0.0-20240207064301-e87f1233c886/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= diff --git a/internal/client/client_batch.go b/internal/client/client_batch.go index 60f896509e..f289ea1cd7 100644 --- a/internal/client/client_batch.go +++ b/internal/client/client_batch.go @@ -644,7 +644,7 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient, tikvTransport } if resp.GetHealthFeedback() != nil { - + c.onHealthFeedback(resp.GetHealthFeedback()) } responses := resp.GetResponses() diff --git a/internal/locate/region_cache.go b/internal/locate/region_cache.go index c6b8bc9cad..a55bafd46d 100644 --- a/internal/locate/region_cache.go +++ b/internal/locate/region_cache.go @@ -3298,5 +3298,10 @@ func (c *RegionCache) getCheckStoreEvents() <-chan struct{} { } func (c *RegionCache) OnHealthFeedback(feedback *tikvpb.HealthFeedback) { + store, ok := c.getStore(feedback.GetStoreId()) + if !ok { + logutil.BgLogger().Info("dropped health feedback info due to unknown store id", zap.Uint64("storeID", feedback.GetStoreId())) + return + } } From 419649a8b2f1fa909f91b23b1d89d18eccbc21aa Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Wed, 7 Feb 2024 19:47:51 +0800 Subject: [PATCH 03/21] Collect the health status and record to the stores Signed-off-by: MyonKeminta --- internal/locate/region_cache.go | 254 +++++++++++++++++++++++++----- internal/locate/region_request.go | 8 +- 2 files changed, 219 insertions(+), 43 deletions(-) diff --git a/internal/locate/region_cache.go b/internal/locate/region_cache.go index a55bafd46d..76c7b569bd 100644 --- a/internal/locate/region_cache.go +++ b/internal/locate/region_cache.go @@ -313,7 +313,7 @@ func (r *regionStore) kvPeer(seed uint32, op *storeSelectorOp) AccessIndex { func (r *regionStore) filterStoreCandidate(aidx AccessIndex, op *storeSelectorOp) bool { _, s := r.accessStore(tiKVOnly, aidx) // filter label unmatched store and slow stores when ReplicaReadMode == PreferLeader - return s.IsLabelsMatch(op.labels) && (!op.preferLeader || (aidx == r.workTiKVIdx && !s.isSlow())) + return s.IsLabelsMatch(op.labels) && (!op.preferLeader || (aidx == r.workTiKVIdx && !s.healthStatus.IsSlow())) } func newRegion(bo *retry.Backoffer, c *RegionCache, pdRegion *pd.Region) (*Region, error) { @@ -574,7 +574,7 @@ func NewRegionCache(pdClient pd.Client) *RegionCache { c.wg.Add(1) go c.asyncCheckAndResolveLoop(time.Duration(interval) * time.Second) c.enableForwarding = config.GetGlobalConfig().EnableForwarding - // Default use 15s as the update inerval. + // Default use 15s as the update interval. c.wg.Add(1) go c.asyncUpdateStoreSlowScore(time.Duration(interval/4) * time.Second) if config.GetGlobalConfig().RegionsRefreshInterval > 0 { @@ -2502,6 +2502,203 @@ func (r *Region) ContainsByEnd(key []byte) bool { (bytes.Compare(key, r.meta.GetEndKey()) <= 0 || len(r.meta.GetEndKey()) == 0) } +const ( + tikvSlowScoreDecayRate = 20. / 60. // s^(-1), linear decaying + tikvSlowScoreSlowThreshold = 80. + + tikvSlowScoreUpdateInterval = time.Millisecond * 100 + tikvSlowScoreUpdateFromPDInterval = time.Minute + tikvSlowScoreFilterCutFreq = 1. +) + +type StoreHealthStatus struct { + isSlow atomic.Bool + + // A statistic for counting the request latency to this store + clientSideSlowScore SlowScoreStat + + tikvSideSlowScore *struct { + // Assuming the type `Store` is always used in heap instead of in stack + sync.Mutex + + // These atomic fields should be read with atomic operations and written with the mutex. + + hasTiKVFeedback atomic.Bool + score atomic.Int64 + lastUpdateTime atomic.Pointer[time.Time] + } +} + +type SlownessDetail struct { + ClientSideSlowScore int64 + TiKVSideSlowScore int64 +} + +// IsSlow returns whether current Store is slow or not. +func (s *StoreHealthStatus) IsSlow() bool { + return s.isSlow.Load() +} + +func (s *StoreHealthStatus) GetHealthStatusDetail() SlownessDetail { + return SlownessDetail{ + ClientSideSlowScore: int64(s.clientSideSlowScore.getSlowScore()), + TiKVSideSlowScore: s.tikvSideSlowScore.score.Load(), + } +} + +// update updates the slow score of this store according to the timecost of current request. +func (s *StoreHealthStatus) update() { + s.clientSideSlowScore.updateSlowScore() + s.updateTiKVServerSideSlowScoreOnTick() +} + +// recordClientSideSlowScoreStat records timecost of each request. +func (s *StoreHealthStatus) recordClientSideSlowScoreStat(timecost time.Duration) { + s.clientSideSlowScore.recordSlowScoreStat(timecost) +} + +// markAlreadySlow marks the related store already slow. +func (s *StoreHealthStatus) markAlreadySlow() { + s.clientSideSlowScore.markAlreadySlow() +} + +func (s *StoreHealthStatus) updateTiKVServerSideSlowScoreOnTick() { + if !s.tikvSideSlowScore.hasTiKVFeedback.Load() { + return + } + now := time.Now() + lastUpdateTime := s.tikvSideSlowScore.lastUpdateTime.Load() + if lastUpdateTime == nil || now.Sub(*lastUpdateTime) < tikvSlowScoreUpdateFromPDInterval { + return + } + + if !s.tikvSideSlowScore.TryLock() { + // It must be being updated. + return + } + defer s.tikvSideSlowScore.Unlock() + + // Reload update time as it might be updated concurrently before acquiring mutex + lastUpdateTime = s.tikvSideSlowScore.lastUpdateTime.Load() + elapsed := now.Sub(*lastUpdateTime) + if elapsed < tikvSlowScoreUpdateFromPDInterval { + return + } + + // TODO: Try to get store status from PD. + + // If updating from PD is not successful: decay the slow score. + score := s.tikvSideSlowScore.score.Load() + if score < 1 { + return + } + // Linear decay by time + score = max(int64(math.Round(float64(score)-tikvSlowScoreDecayRate*elapsed.Seconds())), 1) + s.tikvSideSlowScore.score.Store(score) + + newUpdateTime := new(time.Time) + *newUpdateTime = now + s.tikvSideSlowScore.lastUpdateTime.Store(newUpdateTime) +} + +func (s *StoreHealthStatus) updateTiKVServerSideSlowScore(score int64, currTime time.Time) { + s.tikvSideSlowScore.hasTiKVFeedback.Store(true) + + lastScore := s.tikvSideSlowScore.score.Load() + + // Skip updating to avoid the overhead when it doesn't look slow. + if lastScore < 10 && score < 10 { + return + } + + lastUpdateTime := s.tikvSideSlowScore.lastUpdateTime.Load() + + if lastUpdateTime != nil && currTime.Sub(*lastUpdateTime) < tikvSlowScoreUpdateInterval { + return + } + + if !s.tikvSideSlowScore.TryLock() { + // It must be being updated. Skip. + return + } + defer s.tikvSideSlowScore.Unlock() + + // Reload update time as it might be updated concurrently before acquiring mutex + lastUpdateTime = s.tikvSideSlowScore.lastUpdateTime.Load() + if lastUpdateTime != nil && currTime.Sub(*lastUpdateTime) < tikvSlowScoreUpdateInterval { + return + } + + lastScore = s.tikvSideSlowScore.score.Load() + newScore := score + + if lastUpdateTime != nil { + // It must be positive, otherwise the function exits when checking the update interval. + elapsedSecs := currTime.Sub(*lastUpdateTime).Seconds() + + // We do a simple RC low-pass filter at 1Hz to ease possible jitter. + const twoPiInvert = 1. / (2 * math.Pi) + alpha := elapsedSecs * tikvSlowScoreFilterCutFreq / (elapsedSecs*tikvSlowScoreFilterCutFreq + twoPiInvert) + newScore = int64( + alpha*float64(score) + (1-alpha)*float64(lastScore), + ) + } + + newUpdateTime := new(time.Time) + *newUpdateTime = currTime + + s.tikvSideSlowScore.score.Store(newScore) + s.tikvSideSlowScore.lastUpdateTime.Store(newUpdateTime) +} + +func (s *StoreHealthStatus) updateSlowFlag() { + isSlow := s.clientSideSlowScore.isSlow() || s.tikvSideSlowScore.score.Load() >= tikvSlowScoreSlowThreshold + s.isSlow.Store(isSlow) +} + +//func (s *StoreHealthStatus) IsSlow(currTime time.Time) bool { +// isSlow := s.tikvIsSlow.Load() +// if !isSlow { +// return false +// } +// +// if !s.checkTiKVSideSlowScoreDecay(currTime) { +// return true +// } +// return s.tikvIsSlow.Load() +//} +// +//func (s *StoreHealthStatus) checkTiKVSideSlowScoreDecay(currTime time.Time) bool { +// lastCheckDecayTime := s.tikvSideSlowScoreInternal.lastCheckDecayTime.Load() +// if lastCheckDecayTime == nil || currTime.Sub(*lastCheckDecayTime) < tikvSlowScoreCheckDecayDuration { +// return false +// } +// +// if !s.tikvSideSlowScoreInternal.TryLock() { +// // It's being updated. Skip. +// return false +// } +// defer s.tikvSideSlowScoreInternal.Unlock() +// +// // Recalculate elapsed time as it might be already concurrently updated +// lastCheckDecayTime = s.tikvSideSlowScoreInternal.lastCheckDecayTime.Load() +// elapsed := currTime.Sub(*lastCheckDecayTime) +// if elapsed < tikvSlowScoreCheckDecayDuration { +// // Concurrently updated before acquiring mutex. Behave like successfully updated in this case. +// return true +// } +// +// s.tikvSideSlowScoreInternal.score -= elapsed.Seconds() * tikvSlowScoreDecayRate +// if s.tikvSideSlowScoreInternal.score < 0 { +// s.tikvSideSlowScoreInternal.score = 0 +// } +// +// if s.tikvSideSlowScoreInternal.score < tikvSlowScoreSlowThreshold { +// s.tikvIsSlow.Store(false) +// } +// return true +//} + // Store contains a kv process's address. type Store struct { addr string // loaded store address @@ -2523,8 +2720,7 @@ type Store struct { livenessState uint32 unreachableSince time.Time - // A statistic for counting the request latency to this store - slowScore SlowScoreStat + healthStatus StoreHealthStatus // A statistic for counting the flows of different replicas on this store replicaFlowsStats [numReplicaFlowsType]uint64 } @@ -2674,7 +2870,7 @@ func (s *Store) reResolve(c *RegionCache) (bool, error) { newStore.livenessState = atomic.LoadUint32(&s.livenessState) newStore.unreachableSince = s.unreachableSince if s.addr == addr { - newStore.slowScore = s.slowScore + newStore.healthStatus = s.healthStatus } c.putStore(newStore) s.setResolveState(deleted) @@ -3029,31 +3225,6 @@ func invokeKVStatusAPI(addr string, timeout time.Duration) (l livenessState) { return } -// getSlowScore returns the slow score of store. -func (s *Store) getSlowScore() uint64 { - return s.slowScore.getSlowScore() -} - -// isSlow returns whether current Store is slow or not. -func (s *Store) isSlow() bool { - return s.slowScore.isSlow() -} - -// updateSlowScore updates the slow score of this store according to the timecost of current request. -func (s *Store) updateSlowScoreStat() { - s.slowScore.updateSlowScore() -} - -// recordSlowScoreStat records timecost of each request. -func (s *Store) recordSlowScoreStat(timecost time.Duration) { - s.slowScore.recordSlowScoreStat(timecost) -} - -// markAlreadySlow marks the related store already slow. -func (s *Store) markAlreadySlow() { - s.slowScore.markAlreadySlow() -} - // asyncUpdateStoreSlowScore updates the slow score of each store periodically. func (c *RegionCache) asyncUpdateStoreSlowScore(interval time.Duration) { ticker := time.NewTicker(interval) @@ -3067,27 +3238,28 @@ func (c *RegionCache) asyncUpdateStoreSlowScore(interval time.Duration) { return case <-ticker.C: // update store slowScores - c.checkAndUpdateStoreSlowScores() + c.checkAndUpdateStoreHealthStats() } } } -// checkAndUpdateStoreSlowScores checks and updates slowScore on each store. -func (c *RegionCache) checkAndUpdateStoreSlowScores() { +// checkAndUpdateStoreHealthStats checks and updates performance status on each store. +func (c *RegionCache) checkAndUpdateStoreHealthStats() { defer func() { r := recover() if r != nil { - logutil.BgLogger().Error("panic in the checkAndUpdateStoreSlowScores goroutine", + logutil.BgLogger().Error("panic in the checkAndUpdateStoreHealthStats goroutine", zap.Any("r", r), zap.Stack("stack trace")) } }() - slowScoreMetrics := make(map[uint64]float64) + clientSideSlowScoreMetrics := make(map[uint64]float64) c.forEachStore(func(store *Store) { - store.updateSlowScoreStat() - slowScoreMetrics[store.storeID] = float64(store.getSlowScore()) + store.healthStatus.update() + healthStatusDetail := store.healthStatus.GetHealthStatusDetail() + clientSideSlowScoreMetrics[store.storeID] = float64(healthStatusDetail.ClientSideSlowScore) }) - for store, score := range slowScoreMetrics { + for store, score := range clientSideSlowScoreMetrics { metrics.TiKVStoreSlowScoreGauge.WithLabelValues(strconv.FormatUint(store, 10)).Set(score) } } @@ -3107,6 +3279,10 @@ func (s *Store) recordReplicaFlowsStats(destType replicaFlowsType) { atomic.AddUint64(&s.replicaFlowsStats[destType], 1) } +func (s *Store) recordHealthFeedback(feedback *tikvpb.HealthFeedback) { + s.healthStatus.updateTiKVServerSideSlowScore(int64(feedback.GetSlowScore()), time.Now()) +} + // asyncReportStoreReplicaFlows reports the statistics on the related replicaFlowsType. func (c *RegionCache) asyncReportStoreReplicaFlows(interval time.Duration) { ticker := time.NewTicker(interval) @@ -3303,5 +3479,5 @@ func (c *RegionCache) OnHealthFeedback(feedback *tikvpb.HealthFeedback) { logutil.BgLogger().Info("dropped health feedback info due to unknown store id", zap.Uint64("storeID", feedback.GetStoreId())) return } - + store.recordHealthFeedback(feedback) } diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index efcafbf162..b8e5214cae 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -828,7 +828,7 @@ func (state *accessFollower) isCandidate(idx AccessIndex, replica *replica) bool } // 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() { + if state.option.preferLeader && replica.store.healthStatus.IsSlow() { return false } // Choose a replica with matched labels. @@ -1161,7 +1161,7 @@ func (s *replicaSelector) invalidateReplicaStore(replica *replica, cause error) metrics.RegionCacheCounterWithInvalidateStoreRegionsOK.Inc() // schedule a store addr resolve. s.regionCache.markStoreNeedCheck(store) - store.markAlreadySlow() + store.healthStatus.markAlreadySlow() } } @@ -1257,7 +1257,7 @@ func (s *replicaSelector) onServerIsBusy( } else if ctx != nil && ctx.Store != nil { // Mark the server is busy (the next incoming READs could be redirect // to expected followers. ) - ctx.Store.markAlreadySlow() + ctx.Store.healthStatus.markAlreadySlow() if s.canFallback2Follower() { return true, nil } @@ -1709,7 +1709,7 @@ func (s *RegionRequestSender) sendReqToRegion( } // Record timecost of external requests on related Store when `ReplicaReadMode == "PreferLeader"`. if rpcCtx.Store != nil && req.ReplicaReadType == kv.ReplicaReadPreferLeader && !util.IsInternalRequest(req.RequestSource) { - rpcCtx.Store.recordSlowScoreStat(rpcDuration) + rpcCtx.Store.healthStatus.recordClientSideSlowScoreStat(rpcDuration) } if s.Stats != nil { RecordRegionRequestRuntimeStats(s.Stats, req.Type, rpcDuration) From 9b9d0d0024be166d5dd5a3d59b6d3e83ed5e9025 Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Wed, 7 Feb 2024 19:48:25 +0800 Subject: [PATCH 04/21] Remove the filter which looks not necessary Signed-off-by: MyonKeminta --- internal/locate/region_cache.go | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/internal/locate/region_cache.go b/internal/locate/region_cache.go index 76c7b569bd..6c88b80af7 100644 --- a/internal/locate/region_cache.go +++ b/internal/locate/region_cache.go @@ -2508,7 +2508,6 @@ const ( tikvSlowScoreUpdateInterval = time.Millisecond * 100 tikvSlowScoreUpdateFromPDInterval = time.Minute - tikvSlowScoreFilterCutFreq = 1. ) type StoreHealthStatus struct { @@ -2632,18 +2631,6 @@ func (s *StoreHealthStatus) updateTiKVServerSideSlowScore(score int64, currTime lastScore = s.tikvSideSlowScore.score.Load() newScore := score - if lastUpdateTime != nil { - // It must be positive, otherwise the function exits when checking the update interval. - elapsedSecs := currTime.Sub(*lastUpdateTime).Seconds() - - // We do a simple RC low-pass filter at 1Hz to ease possible jitter. - const twoPiInvert = 1. / (2 * math.Pi) - alpha := elapsedSecs * tikvSlowScoreFilterCutFreq / (elapsedSecs*tikvSlowScoreFilterCutFreq + twoPiInvert) - newScore = int64( - alpha*float64(score) + (1-alpha)*float64(lastScore), - ) - } - newUpdateTime := new(time.Time) *newUpdateTime = currTime From c381ab51d78b94e7d2caada8d20b6ecf4f6d63fb Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Thu, 8 Feb 2024 15:29:39 +0800 Subject: [PATCH 05/21] Add metrics Signed-off-by: MyonKeminta --- internal/locate/region_cache.go | 59 +++++---------------------------- metrics/metrics.go | 11 ++++++ 2 files changed, 19 insertions(+), 51 deletions(-) diff --git a/internal/locate/region_cache.go b/internal/locate/region_cache.go index 6c88b80af7..8bf3a52cb4 100644 --- a/internal/locate/region_cache.go +++ b/internal/locate/region_cache.go @@ -2528,7 +2528,7 @@ type StoreHealthStatus struct { } } -type SlownessDetail struct { +type HealthStatusDetail struct { ClientSideSlowScore int64 TiKVSideSlowScore int64 } @@ -2538,8 +2538,8 @@ func (s *StoreHealthStatus) IsSlow() bool { return s.isSlow.Load() } -func (s *StoreHealthStatus) GetHealthStatusDetail() SlownessDetail { - return SlownessDetail{ +func (s *StoreHealthStatus) GetHealthStatusDetail() HealthStatusDetail { + return HealthStatusDetail{ ClientSideSlowScore: int64(s.clientSideSlowScore.getSlowScore()), TiKVSideSlowScore: s.tikvSideSlowScore.score.Load(), } @@ -2643,49 +2643,6 @@ func (s *StoreHealthStatus) updateSlowFlag() { s.isSlow.Store(isSlow) } -//func (s *StoreHealthStatus) IsSlow(currTime time.Time) bool { -// isSlow := s.tikvIsSlow.Load() -// if !isSlow { -// return false -// } -// -// if !s.checkTiKVSideSlowScoreDecay(currTime) { -// return true -// } -// return s.tikvIsSlow.Load() -//} -// -//func (s *StoreHealthStatus) checkTiKVSideSlowScoreDecay(currTime time.Time) bool { -// lastCheckDecayTime := s.tikvSideSlowScoreInternal.lastCheckDecayTime.Load() -// if lastCheckDecayTime == nil || currTime.Sub(*lastCheckDecayTime) < tikvSlowScoreCheckDecayDuration { -// return false -// } -// -// if !s.tikvSideSlowScoreInternal.TryLock() { -// // It's being updated. Skip. -// return false -// } -// defer s.tikvSideSlowScoreInternal.Unlock() -// -// // Recalculate elapsed time as it might be already concurrently updated -// lastCheckDecayTime = s.tikvSideSlowScoreInternal.lastCheckDecayTime.Load() -// elapsed := currTime.Sub(*lastCheckDecayTime) -// if elapsed < tikvSlowScoreCheckDecayDuration { -// // Concurrently updated before acquiring mutex. Behave like successfully updated in this case. -// return true -// } -// -// s.tikvSideSlowScoreInternal.score -= elapsed.Seconds() * tikvSlowScoreDecayRate -// if s.tikvSideSlowScoreInternal.score < 0 { -// s.tikvSideSlowScoreInternal.score = 0 -// } -// -// if s.tikvSideSlowScoreInternal.score < tikvSlowScoreSlowThreshold { -// s.tikvIsSlow.Store(false) -// } -// return true -//} - // Store contains a kv process's address. type Store struct { addr string // loaded store address @@ -3240,14 +3197,14 @@ func (c *RegionCache) checkAndUpdateStoreHealthStats() { zap.Stack("stack trace")) } }() - clientSideSlowScoreMetrics := make(map[uint64]float64) + healthDetails := make(map[uint64]HealthStatusDetail) c.forEachStore(func(store *Store) { store.healthStatus.update() - healthStatusDetail := store.healthStatus.GetHealthStatusDetail() - clientSideSlowScoreMetrics[store.storeID] = float64(healthStatusDetail.ClientSideSlowScore) + healthDetails[store.storeID] = store.healthStatus.GetHealthStatusDetail() }) - for store, score := range clientSideSlowScoreMetrics { - metrics.TiKVStoreSlowScoreGauge.WithLabelValues(strconv.FormatUint(store, 10)).Set(score) + for store, details := range healthDetails { + metrics.TiKVStoreSlowScoreGauge.WithLabelValues(strconv.FormatUint(store, 10)).Set(float64(details.ClientSideSlowScore)) + metrics.TiKVFeedbackSlowScoreGauge.WithLabelValues(strconv.FormatUint(store, 10)).Set(float64(details.TiKVSideSlowScore)) } } diff --git a/metrics/metrics.go b/metrics/metrics.go index 875a3d44e7..026cc9ebd7 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -101,6 +101,7 @@ var ( TiKVGrpcConnectionState *prometheus.GaugeVec TiKVAggressiveLockedKeysCounter *prometheus.CounterVec TiKVStoreSlowScoreGauge *prometheus.GaugeVec + TiKVFeedbackSlowScoreGauge *prometheus.GaugeVec TiKVPreferLeaderFlowsGauge *prometheus.GaugeVec TiKVStaleReadCounter *prometheus.CounterVec TiKVStaleReadReqCounter *prometheus.CounterVec @@ -703,6 +704,15 @@ func initMetrics(namespace, subsystem string, constLabels prometheus.Labels) { ConstLabels: constLabels, }, []string{LblStore}) + TiKVFeedbackSlowScoreGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "feedback_slow_score", + Help: "Slow scores of each tikv node that is calculated by TiKV and sent to the client by health feedback", + ConstLabels: constLabels, + }, []string{LblStore}) + TiKVPreferLeaderFlowsGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: namespace, @@ -816,6 +826,7 @@ func RegisterMetrics() { prometheus.MustRegister(TiKVGrpcConnectionState) prometheus.MustRegister(TiKVAggressiveLockedKeysCounter) prometheus.MustRegister(TiKVStoreSlowScoreGauge) + prometheus.MustRegister(TiKVFeedbackSlowScoreGauge) prometheus.MustRegister(TiKVPreferLeaderFlowsGauge) prometheus.MustRegister(TiKVStaleReadCounter) prometheus.MustRegister(TiKVStaleReadReqCounter) From cce241a8ae8d16e8998b092806676feb2a474f77 Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Sun, 18 Feb 2024 17:13:02 +0800 Subject: [PATCH 06/21] Fix checkAndUpdateStoreHealthStatus panicking Signed-off-by: MyonKeminta --- internal/locate/region_cache.go | 78 ++++++++++++++++++------- internal/locate/region_cache_test.go | 7 ++- internal/locate/region_request3_test.go | 10 +++- internal/locate/region_request_test.go | 9 ++- 4 files changed, 78 insertions(+), 26 deletions(-) diff --git a/internal/locate/region_cache.go b/internal/locate/region_cache.go index 8bf3a52cb4..d519b9f95a 100644 --- a/internal/locate/region_cache.go +++ b/internal/locate/region_cache.go @@ -667,14 +667,7 @@ func (c *RegionCache) checkAndResolve(needCheckStores []*Store, needCheck func(* // SetRegionCacheStore is used to set a store in region cache, for testing only func (c *RegionCache) SetRegionCacheStore(id uint64, addr string, peerAddr string, storeType tikvrpc.EndpointType, state uint64, labels []*metapb.StoreLabel) { - c.putStore(&Store{ - storeID: id, - storeType: storeType, - state: state, - labels: labels, - addr: addr, - peerAddr: peerAddr, - }) + c.putStore(newStore(id, addr, peerAddr, "", storeType, resolveState(state), labels)) } // SetPDClient replaces pd client,for testing only @@ -2096,15 +2089,15 @@ func reloadTiFlashComputeStores(ctx context.Context, registry storeRegistry) (re } for _, s := range stores { if s.GetState() == metapb.StoreState_Up && isStoreContainLabel(s.GetLabels(), tikvrpc.EngineLabelKey, tikvrpc.EngineLabelTiFlashCompute) { - res = append(res, &Store{ - storeID: s.GetId(), - addr: s.GetAddress(), - peerAddr: s.GetPeerAddress(), - saddr: s.GetStatusAddress(), - storeType: tikvrpc.GetStoreTypeByMeta(s), - labels: s.GetLabels(), - state: uint64(resolved), - }) + res = append(res, newStore( + s.GetId(), + s.GetAddress(), + s.GetPeerAddress(), + s.GetStatusAddress(), + tikvrpc.GetStoreTypeByMeta(s), + resolved, + s.GetLabels(), + )) } } return res, nil @@ -2516,7 +2509,7 @@ type StoreHealthStatus struct { // A statistic for counting the request latency to this store clientSideSlowScore SlowScoreStat - tikvSideSlowScore *struct { + tikvSideSlowScore struct { // Assuming the type `Store` is always used in heap instead of in stack sync.Mutex @@ -2664,7 +2657,7 @@ type Store struct { livenessState uint32 unreachableSince time.Time - healthStatus StoreHealthStatus + healthStatus *StoreHealthStatus // A statistic for counting the flows of different replicas on this store replicaFlowsStats [numReplicaFlowsType]uint64 } @@ -2710,6 +2703,37 @@ func (s resolveState) String() string { } } +func newStore( + id uint64, + addr string, + peerAddr string, + statusAddr string, + storeType tikvrpc.EndpointType, + state resolveState, + labels []*metapb.StoreLabel, +) *Store { + return &Store{ + storeID: id, + storeType: storeType, + state: uint64(state), + labels: labels, + addr: addr, + peerAddr: peerAddr, + saddr: statusAddr, + // Make sure healthStatus field is never null. + healthStatus: &StoreHealthStatus{}, + } +} + +// newUninitializedStore creates a `Store` instance with only storeID initialized. +func newUninitializedStore(id uint64) *Store { + return &Store{ + storeID: id, + // Make sure healthStatus field is never null. + healthStatus: &StoreHealthStatus{}, + } +} + // IsTiFlash returns true if the storeType is TiFlash func (s *Store) IsTiFlash() bool { return s.storeType == tikvrpc.TiFlash @@ -2810,7 +2834,15 @@ func (s *Store) reResolve(c *RegionCache) (bool, error) { storeType := tikvrpc.GetStoreTypeByMeta(store) addr = store.GetAddress() if s.addr != addr || !s.IsSameLabels(store.GetLabels()) { - newStore := &Store{storeID: s.storeID, addr: addr, peerAddr: store.GetPeerAddress(), saddr: store.GetStatusAddress(), storeType: storeType, labels: store.GetLabels(), state: uint64(resolved)} + newStore := newStore( + s.storeID, + addr, + store.GetPeerAddress(), + store.GetStatusAddress(), + storeType, + resolved, + store.GetLabels(), + ) newStore.livenessState = atomic.LoadUint32(&s.livenessState) newStore.unreachableSince = s.unreachableSince if s.addr == addr { @@ -3196,12 +3228,16 @@ func (c *RegionCache) checkAndUpdateStoreHealthStats() { zap.Any("r", r), zap.Stack("stack trace")) } + if _, err := util.EvalFailpoint("doNotRecoverStoreHealthCheckPanic"); err == nil { + panic(r) + } }() healthDetails := make(map[uint64]HealthStatusDetail) c.forEachStore(func(store *Store) { store.healthStatus.update() healthDetails[store.storeID] = store.healthStatus.GetHealthStatusDetail() }) + logutil.BgLogger().Info("checkAndUpdateStoreHealthStats: get health details", zap.Reflect("details", healthDetails)) for store, details := range healthDetails { metrics.TiKVStoreSlowScoreGauge.WithLabelValues(strconv.FormatUint(store, 10)).Set(float64(details.ClientSideSlowScore)) metrics.TiKVFeedbackSlowScoreGauge.WithLabelValues(strconv.FormatUint(store, 10)).Set(float64(details.TiKVSideSlowScore)) @@ -3345,7 +3381,7 @@ func (c *RegionCache) getStoreOrInsertDefault(id uint64) *Store { c.storeMu.Lock() store, exists := c.storeMu.stores[id] if !exists { - store = &Store{storeID: id} + store = newUninitializedStore(id) c.storeMu.stores[id] = store } c.storeMu.Unlock() diff --git a/internal/locate/region_cache_test.go b/internal/locate/region_cache_test.go index 2d6d9aab3c..db8382aa5e 100644 --- a/internal/locate/region_cache_test.go +++ b/internal/locate/region_cache_test.go @@ -47,6 +47,7 @@ import ( "unsafe" "github.com/gogo/protobuf/proto" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/errorpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/suite" @@ -100,6 +101,8 @@ func (s *testRegionCacheSuite) SetupTest() { pdCli := &CodecPDClient{mocktikv.NewPDClient(s.cluster), apicodec.NewCodecV1(apicodec.ModeTxn)} s.cache = NewRegionCache(pdCli) s.bo = retry.NewBackofferWithVars(context.Background(), 5000, nil) + + s.NoError(failpoint.Enable("tikvclient/doNotRecoverStoreHealthCheckPanic", "return")) } func (s *testRegionCacheSuite) TearDownTest() { @@ -108,6 +111,8 @@ func (s *testRegionCacheSuite) TearDownTest() { if s.onClosed != nil { s.onClosed() } + + s.NoError(failpoint.Disable("tikvclient/doNotRecoverStoreHealthCheckPanic")) } func (s *testRegionCacheSuite) storeAddr(id uint64) string { @@ -1932,7 +1937,7 @@ func (s *testRegionRequestToSingleStoreSuite) TestRefreshCache() { region, _ := s.cache.LocateRegionByID(s.bo, s.region) v2 := region.Region.confVer + 1 r2 := metapb.Region{Id: region.Region.id, RegionEpoch: &metapb.RegionEpoch{Version: region.Region.ver, ConfVer: v2}, StartKey: []byte{1}} - st := &Store{storeID: s.store} + st := newUninitializedStore(s.store) s.cache.insertRegionToCache(&Region{meta: &r2, store: unsafe.Pointer(st), ttl: nextTTLWithoutJitter(time.Now().Unix())}, true, true) r, _ = s.cache.scanRegionsFromCache(s.bo, []byte{}, nil, 10) diff --git a/internal/locate/region_request3_test.go b/internal/locate/region_request3_test.go index c5cf930a17..dd6e3e8a3a 100644 --- a/internal/locate/region_request3_test.go +++ b/internal/locate/region_request3_test.go @@ -89,6 +89,8 @@ func (s *testRegionRequestToThreeStoresSuite) SetupTest() { s.bo = retry.NewNoopBackoff(context.Background()) client := mocktikv.NewRPCClient(s.cluster, s.mvccStore, nil) s.regionRequestSender = NewRegionRequestSender(s.cache, client) + + s.NoError(failpoint.Enable("tikvclient/doNotRecoverStoreHealthCheckPanic", "return")) } func (s *testRegionRequestToThreeStoresSuite) TearDownTest() { @@ -97,6 +99,8 @@ func (s *testRegionRequestToThreeStoresSuite) TearDownTest() { if s.onClosed != nil { s.onClosed() } + + s.NoError(failpoint.Disable("tikvclient/doNotRecoverStoreHealthCheckPanic")) } func (s *testRegionRequestToThreeStoresSuite) TestStoreTokenLimit() { @@ -404,7 +408,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestLearnerReplicaSelector() { tikvLearner := &metapb.Peer{Id: s.cluster.AllocID(), StoreId: storeID, Role: metapb.PeerRole_Learner} tikvLearnerAccessIdx := len(regionStore.stores) regionStore.accessIndex[tiKVOnly] = append(regionStore.accessIndex[tiKVOnly], tikvLearnerAccessIdx) - regionStore.stores = append(regionStore.stores, &Store{storeID: tikvLearner.StoreId}) + regionStore.stores = append(regionStore.stores, newUninitializedStore(tikvLearner.StoreId)) regionStore.storeEpochs = append(regionStore.storeEpochs, 0) region = &Region{ @@ -455,7 +459,9 @@ func (s *testRegionRequestToThreeStoresSuite) TestReplicaSelector() { // Add a TiFlash peer to the region. tiflash := &metapb.Peer{Id: s.cluster.AllocID(), StoreId: s.cluster.AllocID()} regionStore.accessIndex[tiFlashOnly] = append(regionStore.accessIndex[tiFlashOnly], len(regionStore.stores)) - regionStore.stores = append(regionStore.stores, &Store{storeID: tiflash.StoreId, storeType: tikvrpc.TiFlash}) + tiflashStore := newUninitializedStore(tiflash.StoreId) + tiflashStore.storeType = tikvrpc.TiFlash + regionStore.stores = append(regionStore.stores, tiflashStore) regionStore.storeEpochs = append(regionStore.storeEpochs, 0) region = &Region{ diff --git a/internal/locate/region_request_test.go b/internal/locate/region_request_test.go index 72cc1ac4a9..d0dd6480ab 100644 --- a/internal/locate/region_request_test.go +++ b/internal/locate/region_request_test.go @@ -45,6 +45,7 @@ import ( "time" "unsafe" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/coprocessor" "github.com/pingcap/kvproto/pkg/disaggregated" "github.com/pingcap/kvproto/pkg/errorpb" @@ -89,11 +90,15 @@ func (s *testRegionRequestToSingleStoreSuite) SetupTest() { s.bo = retry.NewNoopBackoff(context.Background()) client := mocktikv.NewRPCClient(s.cluster, s.mvccStore, nil) s.regionRequestSender = NewRegionRequestSender(s.cache, client) + + s.NoError(failpoint.Enable("tikvclient/doNotRecoverStoreHealthCheckPanic", "return")) } func (s *testRegionRequestToSingleStoreSuite) TearDownTest() { s.cache.Close() s.mvccStore.Close() + + s.NoError(failpoint.Disable("tikvclient/doNotRecoverStoreHealthCheckPanic")) } type fnClient struct { @@ -620,7 +625,7 @@ func (s *testRegionRequestToSingleStoreSuite) TestGetRegionByIDFromCache() { // test kv load new region with new start-key and new epoch v2 := region.Region.confVer + 1 r2 := metapb.Region{Id: region.Region.id, RegionEpoch: &metapb.RegionEpoch{Version: region.Region.ver, ConfVer: v2}, StartKey: []byte{1}} - st := &Store{storeID: s.store} + st := newUninitializedStore(s.store) s.cache.insertRegionToCache(&Region{meta: &r2, store: unsafe.Pointer(st), ttl: nextTTLWithoutJitter(time.Now().Unix())}, true, true) region, err = s.cache.LocateRegionByID(s.bo, s.region) s.Nil(err) @@ -630,7 +635,7 @@ func (s *testRegionRequestToSingleStoreSuite) TestGetRegionByIDFromCache() { v3 := region.Region.confVer + 1 r3 := metapb.Region{Id: region.Region.id, RegionEpoch: &metapb.RegionEpoch{Version: v3, ConfVer: region.Region.confVer}, StartKey: []byte{2}} - st = &Store{storeID: s.store} + st = newUninitializedStore(s.store) s.cache.insertRegionToCache(&Region{meta: &r3, store: unsafe.Pointer(st), ttl: nextTTLWithoutJitter(time.Now().Unix())}, true, true) region, err = s.cache.LocateRegionByID(s.bo, s.region) s.Nil(err) From f769b67ad21db8320ad1250af34c5ce6f8b71e82 Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Sun, 18 Feb 2024 19:49:58 +0800 Subject: [PATCH 07/21] Add logs Signed-off-by: MyonKeminta --- internal/client/client_batch.go | 1 + internal/locate/region_cache.go | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/internal/client/client_batch.go b/internal/client/client_batch.go index f289ea1cd7..9c4500b4b3 100644 --- a/internal/client/client_batch.go +++ b/internal/client/client_batch.go @@ -643,6 +643,7 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient, tikvTransport continue } + logutil.BgLogger().Info("received batch response", zap.Stringer("feedback", resp.GetHealthFeedback())) if resp.GetHealthFeedback() != nil { c.onHealthFeedback(resp.GetHealthFeedback()) } diff --git a/internal/locate/region_cache.go b/internal/locate/region_cache.go index d519b9f95a..3b326b3573 100644 --- a/internal/locate/region_cache.go +++ b/internal/locate/region_cache.go @@ -2597,9 +2597,9 @@ func (s *StoreHealthStatus) updateTiKVServerSideSlowScore(score int64, currTime s.tikvSideSlowScore.hasTiKVFeedback.Store(true) lastScore := s.tikvSideSlowScore.score.Load() + logutil.BgLogger().Info("updateTiKVServerSideSlowScore called", zap.Int64("score", score), zap.Int64("lastScore", lastScore)) - // Skip updating to avoid the overhead when it doesn't look slow. - if lastScore < 10 && score < 10 { + if lastScore == score { return } From 3d92daf1a7dec3ca6f5628872f4345d7eeb21a58 Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Mon, 19 Feb 2024 14:24:08 +0800 Subject: [PATCH 08/21] Merge SetHealthFeedbackHandler to Client Signed-off-by: MyonKeminta --- internal/client/client.go | 14 +++++++++++--- internal/client/client_interceptor_test.go | 2 +- internal/client/client_test.go | 3 ++- internal/locate/region_cache.go | 6 +++--- internal/locate/region_request_test.go | 1 + internal/mockstore/mocktikv/rpc.go | 2 ++ tikv/kv.go | 10 +++++++--- 7 files changed, 27 insertions(+), 11 deletions(-) diff --git a/internal/client/client.go b/internal/client/client.go index d9daa8fad5..713cec3f15 100644 --- a/internal/client/client.go +++ b/internal/client/client.go @@ -108,12 +108,19 @@ type Client interface { CloseAddr(addr string) error // SendRequest sends Request. SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) + // SetHealthFeedbackHandler sets the callback for handling health feedback information received by this client. + // Only used when building KVStore on it. Do not call this method in other places. + SetHealthFeedbackHandler(handler HealthFeedbackHandler) error } type HealthFeedbackHandler = func(feedback *tikvpb.HealthFeedback) -type HealthFeedbackReceiver interface { - SetHealthFeedbackHandler(handler HealthFeedbackHandler) +// NoHealthFeedbackClient is the helper for implementing clients that doesn't support receiving health feedback. These kinds +// of implementations are usually mocks. +type NoHealthFeedbackClient struct{} + +func (NoHealthFeedbackClient) SetHealthFeedbackHandler(_ HealthFeedbackHandler) error { + return errors.New("health feedback not supported by this client type") } type connArray struct { @@ -819,8 +826,9 @@ func (c *RPCClient) CloseAddr(addr string) error { return nil } -func (c *RPCClient) SetHealthFeedbackHandler(handler HealthFeedbackHandler) { +func (c *RPCClient) SetHealthFeedbackHandler(handler HealthFeedbackHandler) error { c.healthFeedbackHandler.Store(&handler) + return nil } type spanInfo struct { diff --git a/internal/client/client_interceptor_test.go b/internal/client/client_interceptor_test.go index 88fc0af7e8..93ebf161f7 100644 --- a/internal/client/client_interceptor_test.go +++ b/internal/client/client_interceptor_test.go @@ -25,7 +25,7 @@ import ( "github.com/tikv/client-go/v2/tikvrpc/interceptor" ) -type emptyClient struct{} +type emptyClient struct{ NoHealthFeedbackClient } func (c emptyClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { return nil, nil diff --git a/internal/client/client_test.go b/internal/client/client_test.go index 54b34700d9..8052a4d13f 100644 --- a/internal/client/client_test.go +++ b/internal/client/client_test.go @@ -107,7 +107,7 @@ func TestGetConnAfterClose(t *testing.T) { func TestCancelTimeoutRetErr(t *testing.T) { req := new(tikvpb.BatchCommandsRequest_Request) - a := newBatchConn(1, 1, nil, nil) + a := newBatchConn(1, 1, nil) ctx, cancel := context.WithCancel(context.TODO()) cancel() @@ -141,6 +141,7 @@ func TestSendWhenReconnect(t *testing.T) { // chanClient sends received requests to the channel. type chanClient struct { + NoHealthFeedbackClient wg *sync.WaitGroup ch chan<- *tikvrpc.Request } diff --git a/internal/locate/region_cache.go b/internal/locate/region_cache.go index 3b326b3573..68bccc96d9 100644 --- a/internal/locate/region_cache.go +++ b/internal/locate/region_cache.go @@ -3227,9 +3227,9 @@ func (c *RegionCache) checkAndUpdateStoreHealthStats() { logutil.BgLogger().Error("panic in the checkAndUpdateStoreHealthStats goroutine", zap.Any("r", r), zap.Stack("stack trace")) - } - if _, err := util.EvalFailpoint("doNotRecoverStoreHealthCheckPanic"); err == nil { - panic(r) + if _, err := util.EvalFailpoint("doNotRecoverStoreHealthCheckPanic"); err == nil { + panic(r) + } } }() healthDetails := make(map[uint64]HealthStatusDetail) diff --git a/internal/locate/region_request_test.go b/internal/locate/region_request_test.go index d0dd6480ab..8a5e5c481f 100644 --- a/internal/locate/region_request_test.go +++ b/internal/locate/region_request_test.go @@ -102,6 +102,7 @@ func (s *testRegionRequestToSingleStoreSuite) TearDownTest() { } type fnClient struct { + client.NoHealthFeedbackClient fn func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) closedAddr string } diff --git a/internal/mockstore/mocktikv/rpc.go b/internal/mockstore/mocktikv/rpc.go index bd50d91b8d..13ffe55fed 100644 --- a/internal/mockstore/mocktikv/rpc.go +++ b/internal/mockstore/mocktikv/rpc.go @@ -49,6 +49,7 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pkg/errors" tikverr "github.com/tikv/client-go/v2/error" + "github.com/tikv/client-go/v2/internal/client" "github.com/tikv/client-go/v2/tikvrpc" "github.com/tikv/client-go/v2/util" ) @@ -680,6 +681,7 @@ type CoprRPCHandler interface { // RPCClient sends kv RPC calls to mock cluster. RPCClient mocks the behavior of // a rpc client at tikv's side. type RPCClient struct { + client.NoHealthFeedbackClient Cluster *Cluster MvccStore MVCCStore coprHandler CoprRPCHandler diff --git a/tikv/kv.go b/tikv/kv.go index afe9828513..46733493fd 100644 --- a/tikv/kv.go +++ b/tikv/kv.go @@ -245,12 +245,16 @@ func NewKVStore(uuid string, pdClient pd.Client, spkv SafePointKV, tikvclient Cl gP: NewSpool(128, 10*time.Second), } store.clientMu.client = client.NewReqCollapse(client.NewInterceptedClient(tikvclient)) - if r, ok := store.clientMu.client.(client.HealthFeedbackReceiver); ok { - r.SetHealthFeedbackHandler(regionCache.OnHealthFeedback) + + if err = store.clientMu.client.SetHealthFeedbackHandler(regionCache.OnHealthFeedback); err == nil { logutil.BgLogger().Debug("health feedback handler is set") } else { - logutil.BgLogger().Warn("health feedback handler not set due to the client type doesn't support", zap.String("type", reflect.TypeOf(tikvclient).String())) + logutil.BgLogger().Warn("failed to set health feedback handler", zap.String("type", reflect.TypeOf(tikvclient).String()), zap.Error(err)) + if _, err := util.EvalFailpoint("healthFeedbackHandlerMustSet"); err == nil { + panic("failed to set health feedback handler, type: " + reflect.TypeOf(tikvclient).String()) + } } + store.lockResolver = txnlock.NewLockResolver(store) loadOption(store, opt...) From a29a1292fb36e0301adf8255d1e81b481165beba Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Mon, 19 Feb 2024 15:53:22 +0800 Subject: [PATCH 09/21] Try another pattern: separated callback registry Signed-off-by: MyonKeminta --- internal/client/client.go | 22 ++++++++++++++-------- internal/client/client_interceptor_test.go | 6 +++++- internal/client/client_test.go | 5 ++++- internal/locate/region_request_test.go | 5 ++++- internal/mockstore/mocktikv/rpc.go | 4 +++- tikv/client.go | 1 + tikv/kv.go | 18 +++++++++++++----- 7 files changed, 44 insertions(+), 17 deletions(-) diff --git a/internal/client/client.go b/internal/client/client.go index 713cec3f15..019e369121 100644 --- a/internal/client/client.go +++ b/internal/client/client.go @@ -108,6 +108,13 @@ type Client interface { CloseAddr(addr string) error // SendRequest sends Request. SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) + // GetCallbackRegistry returns the registry for setting callbacks for the client. For mocks that doesn't support + // this operation, nil might be returned. + GetCallbackRegistry() ClientCallbackRegistry +} + +// ClientCallbackRegistry is the registry for setting callbacks for Client. +type ClientCallbackRegistry interface { // SetHealthFeedbackHandler sets the callback for handling health feedback information received by this client. // Only used when building KVStore on it. Do not call this method in other places. SetHealthFeedbackHandler(handler HealthFeedbackHandler) error @@ -115,14 +122,6 @@ type Client interface { type HealthFeedbackHandler = func(feedback *tikvpb.HealthFeedback) -// NoHealthFeedbackClient is the helper for implementing clients that doesn't support receiving health feedback. These kinds -// of implementations are usually mocks. -type NoHealthFeedbackClient struct{} - -func (NoHealthFeedbackClient) SetHealthFeedbackHandler(_ HealthFeedbackHandler) error { - return errors.New("health feedback not supported by this client type") -} - type connArray struct { // The target host. target string @@ -418,6 +417,9 @@ type RPCClient struct { healthFeedbackHandler *atomic.Pointer[HealthFeedbackHandler] } +var _ Client = &RPCClient{} +var _ ClientCallbackRegistry = &RPCClient{} + // NewRPCClient creates a client that manages connections and rpc calls with tikv-servers. func NewRPCClient(opts ...Opt) *RPCClient { cli := &RPCClient{ @@ -826,6 +828,10 @@ func (c *RPCClient) CloseAddr(addr string) error { return nil } +func (c *RPCClient) GetCallbackRegistry() ClientCallbackRegistry { + return c +} + func (c *RPCClient) SetHealthFeedbackHandler(handler HealthFeedbackHandler) error { c.healthFeedbackHandler.Store(&handler) return nil diff --git a/internal/client/client_interceptor_test.go b/internal/client/client_interceptor_test.go index 93ebf161f7..343eaf79e9 100644 --- a/internal/client/client_interceptor_test.go +++ b/internal/client/client_interceptor_test.go @@ -25,7 +25,7 @@ import ( "github.com/tikv/client-go/v2/tikvrpc/interceptor" ) -type emptyClient struct{ NoHealthFeedbackClient } +type emptyClient struct{} func (c emptyClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { return nil, nil @@ -39,6 +39,10 @@ func (c emptyClient) CloseAddr(addr string) error { return nil } +func (c emptyClient) GetCallbackRegistry() ClientCallbackRegistry { + return nil +} + func TestInterceptedClient(t *testing.T) { executed := false client := NewInterceptedClient(emptyClient{}) diff --git a/internal/client/client_test.go b/internal/client/client_test.go index 8052a4d13f..69d32e7955 100644 --- a/internal/client/client_test.go +++ b/internal/client/client_test.go @@ -141,7 +141,6 @@ func TestSendWhenReconnect(t *testing.T) { // chanClient sends received requests to the channel. type chanClient struct { - NoHealthFeedbackClient wg *sync.WaitGroup ch chan<- *tikvrpc.Request } @@ -154,6 +153,10 @@ func (c *chanClient) CloseAddr(addr string) error { return nil } +func (c *chanClient) GetCallbackRegistry() ClientCallbackRegistry { + return nil +} + func (c *chanClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { c.wg.Wait() c.ch <- req diff --git a/internal/locate/region_request_test.go b/internal/locate/region_request_test.go index 8a5e5c481f..c03d3ebaab 100644 --- a/internal/locate/region_request_test.go +++ b/internal/locate/region_request_test.go @@ -102,7 +102,6 @@ func (s *testRegionRequestToSingleStoreSuite) TearDownTest() { } type fnClient struct { - client.NoHealthFeedbackClient fn func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) closedAddr string } @@ -116,6 +115,10 @@ func (f *fnClient) CloseAddr(addr string) error { return nil } +func (f *fnClient) GetCallbackRegistry() client.ClientCallbackRegistry { + return nil +} + func (f *fnClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { return f.fn(ctx, addr, req, timeout) } diff --git a/internal/mockstore/mocktikv/rpc.go b/internal/mockstore/mocktikv/rpc.go index 13ffe55fed..8db93cca41 100644 --- a/internal/mockstore/mocktikv/rpc.go +++ b/internal/mockstore/mocktikv/rpc.go @@ -681,7 +681,6 @@ type CoprRPCHandler interface { // RPCClient sends kv RPC calls to mock cluster. RPCClient mocks the behavior of // a rpc client at tikv's side. type RPCClient struct { - client.NoHealthFeedbackClient Cluster *Cluster MvccStore MVCCStore coprHandler CoprRPCHandler @@ -1097,3 +1096,6 @@ func (c *RPCClient) Close() error { func (c *RPCClient) CloseAddr(addr string) error { return nil } + +// GetCallbackRegistry implements the ClientCallbackRegistry interface. +func (c *RPCClient) GetCallbackRegistry() client.ClientCallbackRegistry { return nil } diff --git a/tikv/client.go b/tikv/client.go index 90e10bc1ad..950490d59c 100644 --- a/tikv/client.go +++ b/tikv/client.go @@ -43,6 +43,7 @@ import ( // Client is a client that sends RPC. // It should not be used after calling Close(). type Client = client.Client +type ClientCallbackRegistry = client.ClientCallbackRegistry // ClientOpt defines the option to create RPC client. type ClientOpt = client.Opt diff --git a/tikv/kv.go b/tikv/kv.go index 46733493fd..464dfa1345 100644 --- a/tikv/kv.go +++ b/tikv/kv.go @@ -246,12 +246,20 @@ func NewKVStore(uuid string, pdClient pd.Client, spkv SafePointKV, tikvclient Cl } store.clientMu.client = client.NewReqCollapse(client.NewInterceptedClient(tikvclient)) - if err = store.clientMu.client.SetHealthFeedbackHandler(regionCache.OnHealthFeedback); err == nil { - logutil.BgLogger().Debug("health feedback handler is set") + healthFeedbackHandlerSet := false + if r := store.clientMu.client.GetCallbackRegistry(); r != nil { + if err = r.SetHealthFeedbackHandler(regionCache.OnHealthFeedback); err == nil { + logutil.BgLogger().Debug("health feedback handler is set") + healthFeedbackHandlerSet = true + } else { + logutil.BgLogger().Warn("failed to set health feedback handler", zap.String("type", reflect.TypeOf(tikvclient).String()), zap.Error(err)) + } } else { - logutil.BgLogger().Warn("failed to set health feedback handler", zap.String("type", reflect.TypeOf(tikvclient).String()), zap.Error(err)) - if _, err := util.EvalFailpoint("healthFeedbackHandlerMustSet"); err == nil { - panic("failed to set health feedback handler, type: " + reflect.TypeOf(tikvclient).String()) + logutil.BgLogger().Warn("health feedback handler not set due to the client type doesn't support", zap.String("type", reflect.TypeOf(tikvclient).String())) + } + if _, err := util.EvalFailpoint("healthFeedbackMustSet"); err == nil { + if !healthFeedbackHandlerSet { + panic("health feedback handler not set ") } } From 1ddc60dce7a05d240c9203a775aa2b53f6ebbffb Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Mon, 19 Feb 2024 20:23:07 +0800 Subject: [PATCH 10/21] Try region cache eventListener pattern Signed-off-by: MyonKeminta --- go.mod | 4 +- go.sum | 4 +- integration_tests/go.mod | 23 +-- integration_tests/go.sum | 163 ++++++++++----------- internal/client/client.go | 61 ++++---- internal/client/client_batch.go | 14 +- internal/client/client_interceptor_test.go | 4 +- internal/client/client_test.go | 4 +- internal/locate/region_cache.go | 15 +- internal/locate/region_request_test.go | 12 +- internal/mockstore/mocktikv/rpc.go | 4 +- tikv/client.go | 3 +- tikv/kv.go | 19 +-- 13 files changed, 160 insertions(+), 170 deletions(-) diff --git a/go.mod b/go.mod index 3b4040a768..7274bf2fce 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-20231222062942-c0c73f41d0b2 + github.com/pingcap/kvproto v0.0.0-20240208102409-a554af8ee11f github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.18.0 @@ -59,5 +59,3 @@ require ( gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) - -replace github.com/pingcap/kvproto => github.com/MyonKeminta/kvproto v0.0.0-20240207064301-e87f1233c886 diff --git a/go.sum b/go.sum index 5c1fc8631e..c05cdc7564 100644 --- a/go.sum +++ b/go.sum @@ -1,8 +1,6 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/MyonKeminta/kvproto v0.0.0-20240207064301-e87f1233c886 h1:rIfbGwTAP8/yZwnB7aULLPx10aF4aY5Z+NjXAjoibsU= -github.com/MyonKeminta/kvproto v0.0.0-20240207064301-e87f1233c886/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= @@ -76,6 +74,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-20240208102409-a554af8ee11f h1:2xvTjl4OrQY+XF38p8H7qVCXpaUYc5rLiYQhSd07aTI= +github.com/pingcap/kvproto v0.0.0-20240208102409-a554af8ee11f/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= 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.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= diff --git a/integration_tests/go.mod b/integration_tests/go.mod index 36e334b7ec..6b59748a21 100644 --- a/integration_tests/go.mod +++ b/integration_tests/go.mod @@ -6,12 +6,12 @@ require ( github.com/ninedraft/israce v0.0.3 github.com/pingcap/errors v0.11.5-0.20231212100244-799fae176cfb github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c - github.com/pingcap/kvproto v0.0.0-20240109063850-932639606bcf + github.com/pingcap/kvproto v0.0.0-20240208102409-a554af8ee11f github.com/pingcap/tidb v1.1.0-beta.0.20240131080924-732fa8c98695 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.20240125030910-e6f5a45b002e + github.com/tikv/client-go/v2 v2.0.8-0.20240205071126-11cb7985f0ec github.com/tikv/pd/client v0.0.0-20240126020320-567c7d43a008 go.uber.org/goleak v1.3.0 ) @@ -51,7 +51,7 @@ require ( github.com/google/btree v1.1.2 // indirect github.com/google/uuid v1.5.0 // indirect github.com/gorilla/mux v1.8.0 // indirect - github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 // indirect + github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/influxdata/tdigest v0.0.1 // indirect github.com/jellydator/ttlcache/v3 v3.0.1 // indirect github.com/klauspost/compress v1.17.4 // indirect @@ -69,7 +69,7 @@ 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/pkg/parser v0.0.0-20240111112854-1ad36eb0ef29 // indirect + github.com/pingcap/tidb/pkg/parser v0.0.0-20240219094226-fa340f3400aa // indirect github.com/pingcap/tipb v0.0.0-20240116032918-9bb28c43bbfc // indirect github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b // indirect @@ -92,23 +92,23 @@ require ( github.com/uber/jaeger-client-go v2.30.0+incompatible // indirect github.com/uber/jaeger-lib v2.4.1+incompatible // indirect github.com/yusufpapurcu/wmi v1.2.3 // indirect - go.etcd.io/etcd/api/v3 v3.5.10 // indirect - go.etcd.io/etcd/client/pkg/v3 v3.5.10 // indirect - go.etcd.io/etcd/client/v3 v3.5.10 // indirect + go.etcd.io/etcd/api/v3 v3.5.12 // indirect + go.etcd.io/etcd/client/pkg/v3 v3.5.12 // indirect + go.etcd.io/etcd/client/v3 v3.5.12 // indirect go.uber.org/atomic v1.11.0 // indirect go.uber.org/multierr v1.11.0 // indirect go.uber.org/zap v1.26.0 // indirect golang.org/x/crypto v0.18.0 // indirect - golang.org/x/exp v0.0.0-20240110193028-0dcbfd608b1e // indirect + golang.org/x/exp v0.0.0-20240205201215-2c58cdc269a3 // indirect golang.org/x/net v0.20.0 // indirect golang.org/x/sync v0.6.0 // indirect golang.org/x/sys v0.16.0 // indirect golang.org/x/text v0.14.0 // indirect golang.org/x/time v0.5.0 // indirect golang.org/x/tools v0.17.0 // indirect - google.golang.org/genproto v0.0.0-20240116215550-a9fa1716bcac // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240108191215-35c7eff3a6b1 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240123012728-ef4313101c80 // indirect + google.golang.org/genproto v0.0.0-20240125205218-1f4bbc51befe // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240125205218-1f4bbc51befe // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240125205218-1f4bbc51befe // indirect google.golang.org/grpc v1.61.0 // indirect google.golang.org/protobuf v1.32.0 // indirect gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect @@ -118,5 +118,6 @@ 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/MyonKeminta/tidb v1.1.0-alpha.1.0.20240219090303-2af99d5fa939 github.com/tikv/client-go/v2 => ../ ) diff --git a/integration_tests/go.sum b/integration_tests/go.sum index 50e4bbee80..f72cb2992a 100644 --- a/integration_tests/go.sum +++ b/integration_tests/go.sum @@ -35,6 +35,8 @@ github.com/Joker/hpp v1.0.0/go.mod h1:8x5n+M1Hp5hC0g8okX3sR3vFQwynaX/UgSOM9MeBKz 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/MyonKeminta/tidb v1.1.0-alpha.1.0.20240219090303-2af99d5fa939 h1:LX/bEQYImIlJ+WYTWjujerBMRSyb3YoxRBKB6DpmtuM= +github.com/MyonKeminta/tidb v1.1.0-alpha.1.0.20240219090303-2af99d5fa939/go.mod h1:5oopS5coE669N5WVZwlZDyuHpAGU1XvBTmCgLlBhDk8= 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= @@ -47,8 +49,8 @@ github.com/alexbrainman/sspi v0.0.0-20210105120005-909beea2cc74 h1:Kk6a4nehpJ3Uu 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/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/apache/thrift v0.16.0 h1:qEy6UW60iVOlUy+b9ZR0d5WzUWYGOo4HfopoyBaNmoY= +github.com/apache/thrift v0.16.0/go.mod h1:PHK3hniurgQaNMZYaCLEqXKsYK8upmhPbmdP2FXSqgU= github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= github.com/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2 h1:DklsrG3dyBCFEj5IhUbnKptjxatkF07cF2ak3yi77so= github.com/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2/go.mod h1:WaHUgvxTVq04UNunO+XhnAqY/wQc+bxr74GqbsZ/Jqw= @@ -129,8 +131,9 @@ 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/dustin/go-humanize v1.0.1 h1:GzkhY7T5VNhEkwH0PVJgjz+fX1rhBrR7pRT3mDkpeCY= +github.com/dustin/go-humanize v1.0.1/go.mod h1:Mu1zIs6XwVuF/gI1OepvI0qD18qycQx+mFykh5fBlto= 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= @@ -140,8 +143,8 @@ github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1m github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= 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/color v1.16.0 h1:zmkK9Ngbjj+K0yRhTVONQh1p/HknKYSlNT+vZCzyokM= +github.com/fatih/color v1.16.0/go.mod h1:fL2Sau1YI5c0pdGEVCbKQbLXB6edEj1ZgiY4NijnWvE= github.com/fatih/structs v1.1.0/go.mod h1:9NiDSp5zOcgEDl+j00MP/WkGVPOlPRLejGD8Ga6PJ7M= github.com/felixge/httpsnoop v1.0.4 h1:NFTV2Zj1bL4mc9sqWACXbQFVBBg2W3GPvqp8/ESS2Wg= github.com/felixge/httpsnoop v1.0.4/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U= @@ -158,8 +161,8 @@ github.com/go-asn1-ber/asn1-ber v1.5.4/go.mod h1:hEBeB/ic+5LoWskz+yKT7vGhhPYkPro github.com/go-check/check v0.0.0-20180628173108-788fd7840127/go.mod h1:9ES+weclKsC9YodN5RgxqK/VD9HM9JsCSh7rNhMZE98= github.com/go-errors/errors v1.0.1 h1:LUHzmkK3GUKUrL/1gfBUxAHzcev3apQlezX/+O7ma6w= github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= -github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= -github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= +github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= +github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-logr/logr v1.4.1 h1:pKouT5E8xu9zeFC39JXRDukb6JFQPXM5p5I91188VAQ= github.com/go-logr/logr v1.4.1/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= @@ -188,8 +191,8 @@ github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXP 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/v4 v4.4.2 h1:rcc4lwaZgFMCZ5jxF9ABolDcIHdBytAFgqFPbSJQAYs= -github.com/golang-jwt/jwt/v4 v4.4.2/go.mod h1:m21LjoU+eqJr34lmDMbreY2eSTRJ1cv77w39/MY0Ch0= +github.com/golang-jwt/jwt/v4 v4.5.0 h1:7cYmW1XlMY7h7ii7UhUyChSgS5wUJEnm9uZVTGqOWzg= +github.com/golang-jwt/jwt/v4 v4.5.0/go.mod h1:m21LjoU+eqJr34lmDMbreY2eSTRJ1cv77w39/MY0Ch0= github.com/golang-jwt/jwt/v5 v5.0.0 h1:1n1XNM9hk7O9mnQoNBGolZvzebBQ7p93ULHRc28XJUE= github.com/golang-jwt/jwt/v5 v5.0.0/go.mod h1:pqrtFR0X4osieyHYxtmOUWsAWrfe1Q5UVIyoH402zdk= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= @@ -246,16 +249,16 @@ github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORR 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.5.0 h1:PPwGk2jz7EePpoHN/+ClbZu8SPxiqlu12wZP/3sWmnc= -github.com/gorilla/websocket v1.5.0/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/gorilla/websocket v1.5.1 h1:gmztn0JnHVt9JZquRuzLw3g4wouNVzKL15iLr/zn/QY= +github.com/gorilla/websocket v1.5.1/go.mod h1:x3kM2JMyaluk02fnUJpQuwD2dCS5NDG2ZHL0uE0tcaY= +github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= +github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= github.com/grpc-ecosystem/grpc-gateway v1.16.0 h1:gmcG1KaJ57LophUzW0Hy8NmPhnMZb4M0+kPpLofRdBo= github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.16.0 h1:YBftPWNWd4WwGqtY2yeZL2ef8rHAxPBD8KFhJpmcqms= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.16.0/go.mod h1:YN5jB8ie0yfIUg6VvR9Kz84aCaG7AsGZnLjhHbUqwPg= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.19.1 h1:/c3QmbOGMGTOumP2iT/rCwB7b0QDGLKzqOmktBjT+Is= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.19.1/go.mod h1:5SN9VR2LTsRFsrEC6FHgRbTWrTHu6tqPeKxEQv15giM= github.com/hashicorp/go-version v1.2.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= @@ -284,8 +287,8 @@ github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9Y 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/jonboulle/clockwork v0.4.0 h1:p4Cf1aMWXnXAUh8lVfewRBx1zaTSYKrKMF2g3ST4RZ4= +github.com/jonboulle/clockwork v0.4.0/go.mod h1:xgRqUGwRcjKCO1vbZUEtSLrqKoPSsUpK7fnezOII0kc= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= @@ -311,7 +314,6 @@ github.com/klauspost/cpuid v1.2.1/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgo github.com/klauspost/cpuid v1.3.1 h1:5JNjFYYQrZeKRJ0734q51WCEEn2huer72Dc7K+R/b6s= github.com/klauspost/cpuid v1.3.1/go.mod h1:bYW4mA6ZgKPob1/Dlai2LviZJO7KGI3uoWLd42rAQw4= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -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= @@ -347,10 +349,10 @@ github.com/mattn/go-colorable v0.1.13/go.mod h1:7S9/ev0klgBDR4GtXTXX8a3vIGJpMovk 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.19 h1:JITubQf0MOLdlGRuRq+jtsDlekdYPia9ZFsB8h/APPA= -github.com/mattn/go-isatty v0.0.19/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/go-isatty v0.0.20 h1:xfD0iDuEKnDkl03q4limB+vH+GxLEtL/jb4xVJSWWEY= +github.com/mattn/go-isatty v0.0.20/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y= +github.com/mattn/go-runewidth v0.0.15 h1:UNAjwbU9l54TA3KzvqLGxwWjHmMgBUVhBiTjelZgg3U= +github.com/mattn/go-runewidth v0.0.15/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/mediocregopher/mediocre-go-lib v0.0.0-20181029021733-cb65787f37ed/go.mod h1:dSsfyI2zABAdhcbvkXqgxOxrCsbYeHCPgrZkku60dSg= @@ -413,18 +415,16 @@ github.com/pingcap/fn v1.0.0/go.mod h1:u9WZ1ZiOD1RpNhcI42RucFh/lBuzTu6rw88a+oF2Z 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-20240109063850-932639606bcf h1:n3FMveYjc2VuETjo6YhmsgkDx0P/yLJTvk96BJdCq6Y= -github.com/pingcap/kvproto v0.0.0-20240109063850-932639606bcf/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= +github.com/pingcap/kvproto v0.0.0-20240208102409-a554af8ee11f h1:2xvTjl4OrQY+XF38p8H7qVCXpaUYc5rLiYQhSd07aTI= +github.com/pingcap/kvproto v0.0.0-20240208102409-a554af8ee11f/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= 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.20240131080924-732fa8c98695 h1:7ELVL+rVzEZrDv4dEC995qHbv9K0dSKm7+wuwotiB8U= -github.com/pingcap/tidb v1.1.0-beta.0.20240131080924-732fa8c98695/go.mod h1:rfdtaBHvL6w0uFmxz0BJIBHHUpR/satNS695+def0y8= -github.com/pingcap/tidb/pkg/parser v0.0.0-20240111112854-1ad36eb0ef29 h1:OPF0SMFk0O298dzHisYnhotbTcDQC2l+h0Xs7QxUF88= -github.com/pingcap/tidb/pkg/parser v0.0.0-20240111112854-1ad36eb0ef29/go.mod h1:yRkiqLFwIqibYg2P7h4bclHjHcJiIFRLKhGRyBcKYus= +github.com/pingcap/tidb/pkg/parser v0.0.0-20240219094226-fa340f3400aa h1:d3c9Xm6Kzwh2lT3Y/SsvMYb2jJExF+lwzXcvRtEC2qk= +github.com/pingcap/tidb/pkg/parser v0.0.0-20240219094226-fa340f3400aa/go.mod h1:MWQK6otJgZRI6zcCVPV22U4qE26qOGJnN4fq8XawgBs= github.com/pingcap/tipb v0.0.0-20240116032918-9bb28c43bbfc h1:sEp4lbExDfnMX8HXQyhZrhqo2/SgeFY5KOdo5akc8FM= github.com/pingcap/tipb v0.0.0-20240116032918-9bb28c43bbfc/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/browser v0.0.0-20210911075715-681adbf594b8 h1:KoWmjvw+nsYOo29YJK9vDA65RGE3NrOnUtO7a+RF9HU= @@ -454,8 +454,8 @@ github.com/prometheus/procfs v0.12.0 h1:jluTpSng7V9hY0O2R9DzzJHYb2xULk9VTR1V1R/k github.com/prometheus/procfs v0.12.0/go.mod h1:pcuDEFsWDnvcgNzo4EEweacyhjeA9Zk3cnaOZAZEfOo= 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/rivo/uniseg v0.4.6 h1:Sovz9sDSwbOz9tgUy8JpT+KgCkPYJEN/oYzlJiYTNLg= +github.com/rivo/uniseg v0.4.6/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/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= @@ -492,8 +492,8 @@ github.com/soheilhy/cmux v0.1.5/go.mod h1:T7TcVDs9LWfQgPlPsdngu6I6QIoyIFZDDC6sNE 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/cobra v1.8.0 h1:7aJaZx1B85qltLMc546zn58BxxfZdR/W22ej9CFoEf0= +github.com/spf13/cobra v1.8.0/go.mod h1:WXLWApfZ71AjXPya3WOlMsY9yMs7YeiHhFVlvLyhcho= 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= @@ -533,8 +533,8 @@ github.com/tklauser/go-sysconf v0.3.12/go.mod h1:Ho14jnntGE1fpdOqQEEaiKRpvIavV0h github.com/tklauser/numcpus v0.3.0/go.mod h1:yFGUr7TUHQRAhyqBcEg0Ge34zDBAsIvJJcyE6boqnA8= github.com/tklauser/numcpus v0.6.1 h1:ng9scYS7az0Bk4OZLvrNXNSAO2Pxr1XXRAPyjhIx+Fk= github.com/tklauser/numcpus v0.6.1/go.mod h1:1XfjsgE2zo8GVw7POkMbHENHzVg3GzmoZ9fESEdAacY= -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/tmc/grpc-websocket-proxy v0.0.0-20220101234140-673ab2c3ae75 h1:6fotK7otjonDflCTK0BCfls4SPy3NcCVb5dqqmbRknE= +github.com/tmc/grpc-websocket-proxy v0.0.0-20220101234140-673ab2c3ae75/go.mod h1:KO6IkyS8Y3j8OdNO85qEYBsRPuteD+YciPomcXdrMnk= 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= @@ -555,8 +555,8 @@ github.com/wangjohn/quickselect v0.0.0-20161129230411-ed8402a42d5f/go.mod h1:8sd 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/xiang90/probing v0.0.0-20221125231312-a49e3df8f510 h1:S2dVYn90KE98chqDkyE9Z4N61UnQd+KOfgp5Iu53llk= +github.com/xiang90/probing v0.0.0-20221125231312-a49e3df8f510/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= @@ -573,44 +573,43 @@ github.com/yusufpapurcu/wmi v1.2.3 h1:E1ctvB7uKFMOJw3fdOW32DwGE9I7t++CRUEMKvFoFi github.com/yusufpapurcu/wmi v1.2.3/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.etcd.io/bbolt v1.3.8 h1:xs88BrvEv273UsB79e0hcVrlUWmS0a8upikMFhSyAtA= go.etcd.io/bbolt v1.3.8/go.mod h1:N9Mkw9X8x5fupy0IKsmuqVtoGDyxsaDlbk4Rd05IAQw= -go.etcd.io/etcd/api/v3 v3.5.10 h1:szRajuUUbLyppkhs9K6BRtjY37l66XQQmw7oZRANE4k= -go.etcd.io/etcd/api/v3 v3.5.10/go.mod h1:TidfmT4Uycad3NM/o25fG3J07odo4GBB9hoxaodFCtI= -go.etcd.io/etcd/client/pkg/v3 v3.5.10 h1:kfYIdQftBnbAq8pUWFXfpuuxFSKzlmM5cSn76JByiT0= -go.etcd.io/etcd/client/pkg/v3 v3.5.10/go.mod h1:DYivfIviIuQ8+/lCq4vcxuseg2P2XbHygkKwFo9fc8U= -go.etcd.io/etcd/client/v2 v2.305.10 h1:MrmRktzv/XF8CvtQt+P6wLUlURaNpSDJHFZhe//2QE4= -go.etcd.io/etcd/client/v2 v2.305.10/go.mod h1:m3CKZi69HzilhVqtPDcjhSGp+kA1OmbNn0qamH80xjA= -go.etcd.io/etcd/client/v3 v3.5.10 h1:W9TXNZ+oB3MCd/8UjxHTWK5J9Nquw9fQBLJd5ne5/Ao= -go.etcd.io/etcd/client/v3 v3.5.10/go.mod h1:RVeBnDz2PUEZqTpgqwAtUd8nAPf5kjyFyND7P1VkOKc= -go.etcd.io/etcd/pkg/v3 v3.5.10 h1:WPR8K0e9kWl1gAhB5A7gEa5ZBTNkT9NdNWrR8Qpo1CM= -go.etcd.io/etcd/pkg/v3 v3.5.10/go.mod h1:TKTuCKKcF1zxmfKWDkfz5qqYaE3JncKKZPFf8c1nFUs= -go.etcd.io/etcd/raft/v3 v3.5.10 h1:cgNAYe7xrsrn/5kXMSaH8kM/Ky8mAdMqGOxyYwpP0LA= -go.etcd.io/etcd/raft/v3 v3.5.10/go.mod h1:odD6kr8XQXTy9oQnyMPBOr0TVe+gT0neQhElQ6jbGRc= -go.etcd.io/etcd/server/v3 v3.5.10 h1:4NOGyOwD5sUZ22PiWYKmfxqoeh72z6EhYjNosKGLmZg= -go.etcd.io/etcd/server/v3 v3.5.10/go.mod h1:gBplPHfs6YI0L+RpGkTQO7buDbHv5HJGG/Bst0/zIPo= -go.etcd.io/etcd/tests/v3 v3.5.10 h1:F1pbXwKxwZ58aBT2+CSL/r8WUCAVhob0y1y8OVJ204s= -go.etcd.io/etcd/tests/v3 v3.5.10/go.mod h1:vVMWDv9OhopxfJCd+CMI4pih0zUDqlkJj6JcBNlUVXI= +go.etcd.io/etcd/api/v3 v3.5.12 h1:W4sw5ZoU2Juc9gBWuLk5U6fHfNVyY1WC5g9uiXZio/c= +go.etcd.io/etcd/api/v3 v3.5.12/go.mod h1:Ot+o0SWSyT6uHhA56al1oCED0JImsRiU9Dc26+C2a+4= +go.etcd.io/etcd/client/pkg/v3 v3.5.12 h1:EYDL6pWwyOsylrQyLp2w+HkQ46ATiOvoEdMarindU2A= +go.etcd.io/etcd/client/pkg/v3 v3.5.12/go.mod h1:seTzl2d9APP8R5Y2hFL3NVlD6qC/dOT+3kvrqPyTas4= +go.etcd.io/etcd/client/v2 v2.305.12 h1:0m4ovXYo1CHaA/Mp3X/Fak5sRNIWf01wk/X1/G3sGKI= +go.etcd.io/etcd/client/v2 v2.305.12/go.mod h1:aQ/yhsxMu+Oht1FOupSr60oBvcS9cKXHrzBpDsPTf9E= +go.etcd.io/etcd/client/v3 v3.5.12 h1:v5lCPXn1pf1Uu3M4laUE2hp/geOTc5uPcYYsNe1lDxg= +go.etcd.io/etcd/client/v3 v3.5.12/go.mod h1:tSbBCakoWmmddL+BKVAJHa9km+O/E+bumDe9mSbPiqw= +go.etcd.io/etcd/pkg/v3 v3.5.12 h1:OK2fZKI5hX/+BTK76gXSTyZMrbnARyX9S643GenNGb8= +go.etcd.io/etcd/pkg/v3 v3.5.12/go.mod h1:UVwg/QIMoJncyeb/YxvJBJCE/NEwtHWashqc8A1nj/M= +go.etcd.io/etcd/raft/v3 v3.5.12 h1:7r22RufdDsq2z3STjoR7Msz6fYH8tmbkdheGfwJNRmU= +go.etcd.io/etcd/raft/v3 v3.5.12/go.mod h1:ERQuZVe79PI6vcC3DlKBukDCLja/L7YMu29B74Iwj4U= +go.etcd.io/etcd/server/v3 v3.5.12 h1:EtMjsbfyfkwZuA2JlKOiBfuGkFCekv5H178qjXypbG8= +go.etcd.io/etcd/server/v3 v3.5.12/go.mod h1:axB0oCjMy+cemo5290/CutIjoxlfA6KVYKD1w0uue10= +go.etcd.io/etcd/tests/v3 v3.5.12 h1:k1fG7+F87Z7zKp57EcjXu9XgOsW0sfp5USqfzmMTIwM= +go.etcd.io/etcd/tests/v3 v3.5.12/go.mod h1:CLWdnlr8bWNa8tjkmKFybPz5Ldjh9GuHbYhq1g9vpIo= 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/instrumentation/google.golang.org/grpc/otelgrpc v0.46.1 h1:SpGay3w+nEwMpfVnbqOLH5gY52/foP8RE8UzTZ1pdSE= -go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.46.1/go.mod h1:4UoMYEZOC0yN/sPGH76KPkkU7zgiEWYWL9vwmbnTJPE= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.47.0 h1:UNQQKPfTDe1J81ViolILjTKPr9WetKW6uei2hFgJmFs= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.47.0/go.mod h1:r9vWsPS/3AQItv3OSlEJ/E4mbrhUbbw18meOjArPtKQ= go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.46.1 h1:aFJWCqJMNjENlcleuuOkGAPH82y0yULBScfXcIEdS24= go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.46.1/go.mod h1:sEGXWArGqc3tVa+ekntsN65DmVbVeW+7lTKTjZF3/Fo= -go.opentelemetry.io/otel v1.21.0 h1:hzLeKBZEL7Okw2mGzZ0cc4k/A7Fta0uoPgaJCr8fsFc= -go.opentelemetry.io/otel v1.21.0/go.mod h1:QZzNPQPm1zLX4gZK4cMi+71eaorMSGT3A4znnUvNNEo= -go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.21.0 h1:cl5P5/GIfFh4t6xyruOgJP5QiA1pw4fYYdv6nc6CBWw= -go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.21.0/go.mod h1:zgBdWWAu7oEEMC06MMKc5NLbA/1YDXV1sMpSqEeLQLg= -go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.21.0 h1:tIqheXEFWAZ7O8A7m+J0aPTmpJN3YQ7qetUAdkkkKpk= -go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.21.0/go.mod h1:nUeKExfxAQVbiVFn32YXpXZZHZ61Cc3s3Rn1pDBGAb0= -go.opentelemetry.io/otel/metric v1.21.0 h1:tlYWfeo+Bocx5kLEloTjbcDwBuELRrIFxwdQ36PlJu4= -go.opentelemetry.io/otel/metric v1.21.0/go.mod h1:o1p3CA8nNHW8j5yuQLdc1eeqEaPfzug24uvsyIEJRWM= -go.opentelemetry.io/otel/sdk v1.21.0 h1:FTt8qirL1EysG6sTQRZ5TokkU8d0ugCj8htOgThZXQ8= -go.opentelemetry.io/otel/sdk v1.21.0/go.mod h1:Nna6Yv7PWTdgJHVRD9hIYywQBRx7pbox6nwBnZIxl/E= -go.opentelemetry.io/otel/trace v1.21.0 h1:WD9i5gzvoUPuXIXH24ZNBudiarZDKuekPqi/E8fpfLc= -go.opentelemetry.io/otel/trace v1.21.0/go.mod h1:LGbsEB0f9LGjN+OZaQQ26sohbOmiMR+BaslueVtS/qQ= -go.opentelemetry.io/proto/otlp v1.0.0 h1:T0TX0tmXU8a3CbNXzEKGeU5mIVOdf0oykP+u2lIVU/I= -go.opentelemetry.io/proto/otlp v1.0.0/go.mod h1:Sy6pihPLfYHkr3NkUbEhGHFhINUSI/v80hjKIs5JXpM= +go.opentelemetry.io/otel v1.22.0 h1:xS7Ku+7yTFvDfDraDIJVpw7XPyuHlB9MCiqqX5mcJ6Y= +go.opentelemetry.io/otel v1.22.0/go.mod h1:eoV4iAi3Ea8LkAEI9+GFT44O6T/D0GWAVFyZVCC6pMI= +go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.22.0 h1:9M3+rhx7kZCIQQhQRYaZCdNu1V73tm4TvXs2ntl98C4= +go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.22.0/go.mod h1:noq80iT8rrHP1SfybmPiRGc9dc5M8RPmGvtwo7Oo7tc= +go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.22.0 h1:H2JFgRcGiyHg7H7bwcwaQJYrNFqCqrbTQ8K4p1OvDu8= +go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.22.0/go.mod h1:WfCWp1bGoYK8MeULtI15MmQVczfR+bFkk0DF3h06QmQ= +go.opentelemetry.io/otel/metric v1.22.0 h1:lypMQnGyJYeuYPhOM/bgjbFM6WE44W1/T45er4d8Hhg= +go.opentelemetry.io/otel/metric v1.22.0/go.mod h1:evJGjVpZv0mQ5QBRJoBF64yMuOf4xCWdXjK8pzFvliY= +go.opentelemetry.io/otel/sdk v1.22.0 h1:6coWHw9xw7EfClIC/+O31R8IY3/+EiRFHevmHafB2Gw= +go.opentelemetry.io/otel/sdk v1.22.0/go.mod h1:iu7luyVGYovrRpe2fmj3CVKouQNdTOkxtLzPvPz1DOc= +go.opentelemetry.io/otel/trace v1.22.0 h1:Hg6pPujv0XG9QaVbGOBVHunyuLcCC3jN7WEhPx83XD0= +go.opentelemetry.io/otel/trace v1.22.0/go.mod h1:RbbHXVqKES9QhzZq/fE5UnOSILqRt40a21sPw2He1xo= +go.opentelemetry.io/proto/otlp v1.1.0 h1:2Di21piLrCqJ3U3eXGCTPHE9R8Nh+0uglSnOyxikMeI= +go.opentelemetry.io/proto/otlp v1.1.0/go.mod h1:GpBHCBWiqvVLDqmHZsoMM3C5ySeKTC7ej/RNTae6MdY= 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= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= @@ -633,8 +632,8 @@ go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0= go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.9.1/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= -go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= go.uber.org/zap v1.12.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= +go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= 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.26.0 h1:sI7k6L95XOKS281NhVKOFCUNIvv9e0w4BF8N3u+tCRo= @@ -651,8 +650,8 @@ golang.org/x/crypto v0.18.0 h1:PGVlW0xEltQnzFZ55hkuX5+KLyrMYhHld1YHO4AKcdc= golang.org/x/crypto v0.18.0/go.mod h1:R0j02AL6hcrfOiy9T4ZYp/rcWeMxM3L6QYxlOuEG1mg= 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-20240110193028-0dcbfd608b1e h1:723BNChdd0c2Wk6WOE320qGBiPtYx0F0Bbm1kriShfE= -golang.org/x/exp v0.0.0-20240110193028-0dcbfd608b1e/go.mod h1:iRJReGqOEeBhDZGkGbynYwcHlctCvnjTYIamk7uXpHI= +golang.org/x/exp v0.0.0-20240205201215-2c58cdc269a3 h1:/RIbNt/Zr7rVhIkQhooTxCxFcdWLGIKnZA4IXNFSrvo= +golang.org/x/exp v0.0.0-20240205201215-2c58cdc269a3/go.mod h1:idGWGoKP1toJGkd5/ig9ZLuPcZBC3ewk7SzmH0uou08= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= @@ -727,6 +726,7 @@ golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210816074244-15123e1e1f71/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211013075003-97ac67df715c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -784,9 +784,8 @@ golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= 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= +gonum.org/v1/gonum v0.0.0-20181121035319-3f7ecaa7e8ca h1:PupagGYwj8+I4ubCxcmcBRk3VlUWtTg5huQpZR9flmE= 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.156.0 h1:yloYcGbBtVYjLKQe4enCunxvwn3s2w/XPrrhVf6MsvQ= google.golang.org/api v0.156.0/go.mod h1:bUSmn4KFO0Q+69zo9CNIDp4Psi6BqM0np0CbzKRSiSY= @@ -799,12 +798,12 @@ google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoA google.golang.org/genproto v0.0.0-20181004005441-af9cb2a35e7f/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20240116215550-a9fa1716bcac h1:ZL/Teoy/ZGnzyrqK/Optxxp2pmVh+fmJ97slxSRyzUg= -google.golang.org/genproto v0.0.0-20240116215550-a9fa1716bcac/go.mod h1:+Rvu7ElI+aLzyDQhpHMFMMltsD6m7nqpuWDd2CwJw3k= -google.golang.org/genproto/googleapis/api v0.0.0-20240108191215-35c7eff3a6b1 h1:OPXtXn7fNMaXwO3JvOmF1QyTc00jsSFFz1vXXBOdCDo= -google.golang.org/genproto/googleapis/api v0.0.0-20240108191215-35c7eff3a6b1/go.mod h1:B5xPO//w8qmBDjGReYLpR6UJPnkldGkCSMoH/2vxJeg= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240123012728-ef4313101c80 h1:AjyfHzEPEFp/NpvfN5g+KDla3EMojjhRVZc1i7cj+oM= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240123012728-ef4313101c80/go.mod h1:PAREbraiVEVGVdTZsVWjSbbTtSyGbAgIIvni8a8CD5s= +google.golang.org/genproto v0.0.0-20240125205218-1f4bbc51befe h1:USL2DhxfgRchafRvt/wYyyQNzwgL7ZiURcozOE/Pkvo= +google.golang.org/genproto v0.0.0-20240125205218-1f4bbc51befe/go.mod h1:cc8bqMqtv9gMOr0zHg2Vzff5ULhhL2IXP4sbcn32Dro= +google.golang.org/genproto/googleapis/api v0.0.0-20240125205218-1f4bbc51befe h1:0poefMBYvYbs7g5UkjS6HcxBPaTRAmznle9jnxYoAI8= +google.golang.org/genproto/googleapis/api v0.0.0-20240125205218-1f4bbc51befe/go.mod h1:4jWUdICTdgc3Ibxmr8nAJiiLHwQBY0UI0XZcEMaFKaA= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240125205218-1f4bbc51befe h1:bQnxqljG/wqi4NTXu2+DJ3n7APcEA882QZ1JvhQAq9o= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240125205218-1f4bbc51befe/go.mod h1:PAREbraiVEVGVdTZsVWjSbbTtSyGbAgIIvni8a8CD5s= google.golang.org/grpc v0.0.0-20180607172857-7a6a684ca69e/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= @@ -869,8 +868,8 @@ sigs.k8s.io/json v0.0.0-20221116044647-bc3834ca7abd h1:EDPBXCAspyGV4jQlpZSudPeMm sigs.k8s.io/json v0.0.0-20221116044647-bc3834ca7abd/go.mod h1:B8JuhiUyNFVKdsE8h686QcCxMaH6HrOAZj4vswFpcB0= sigs.k8s.io/structured-merge-diff/v4 v4.3.0 h1:UZbZAZfX0wV2zr7YZorDz6GXROfDFj6LvqCRm4VUVKk= sigs.k8s.io/structured-merge-diff/v4 v4.3.0/go.mod h1:N8hJocpFajUSSeSJ9bOZ77VzejKZaXsTtZo4/u7Io08= -sigs.k8s.io/yaml v1.3.0 h1:a2VclLzOGrwOHDiV8EfBGhvjHvP46CtW5j6POvhYGGo= -sigs.k8s.io/yaml v1.3.0/go.mod h1:GeOyir5tyXNByN85N/dRIT9es5UQNerPYEKK56eTBm8= +sigs.k8s.io/yaml v1.4.0 h1:Mk1wCc2gy/F0THH0TAp1QYyJNzRm2KCLy3o5ASXVI5E= +sigs.k8s.io/yaml v1.4.0/go.mod h1:Ejl7/uTz7PSA4eKMyQCUTnhZYNmLIl+5c2lQPGR2BPY= 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= diff --git a/internal/client/client.go b/internal/client/client.go index 019e369121..4b2343daca 100644 --- a/internal/client/client.go +++ b/internal/client/client.go @@ -108,20 +108,15 @@ type Client interface { CloseAddr(addr string) error // SendRequest sends Request. SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) - // GetCallbackRegistry returns the registry for setting callbacks for the client. For mocks that doesn't support - // this operation, nil might be returned. - GetCallbackRegistry() ClientCallbackRegistry + // SetEventListener registers an event listener for the Client instance. If called more than once, the previously + // set one will be replaced. + SetEventListener(listener ClientEventListener) } -// ClientCallbackRegistry is the registry for setting callbacks for Client. -type ClientCallbackRegistry interface { - // SetHealthFeedbackHandler sets the callback for handling health feedback information received by this client. - // Only used when building KVStore on it. Do not call this method in other places. - SetHealthFeedbackHandler(handler HealthFeedbackHandler) error +type ClientEventListener interface { + OnHealthFeedback(feedback *tikvpb.HealthFeedback) } -type HealthFeedbackHandler = func(feedback *tikvpb.HealthFeedback) - type connArray struct { // The target host. target string @@ -139,7 +134,7 @@ type connArray struct { } func newConnArray(maxSize uint, addr string, security config.Security, - idleNotify *uint32, enableBatch bool, dialTimeout time.Duration, m *connMonitor, healthFeedbackHandler *atomic.Pointer[HealthFeedbackHandler], opts []grpc.DialOption) (*connArray, error) { + idleNotify *uint32, enableBatch bool, dialTimeout time.Duration, m *connMonitor, eventListener *atomic.Pointer[ClientEventListener], opts []grpc.DialOption) (*connArray, error) { a := &connArray{ index: 0, v: make([]*monitoredConn, maxSize), @@ -148,7 +143,7 @@ func newConnArray(maxSize uint, addr string, security config.Security, dialTimeout: dialTimeout, monitor: m, } - if err := a.Init(addr, security, idleNotify, enableBatch, healthFeedbackHandler, opts...); err != nil { + if err := a.Init(addr, security, idleNotify, enableBatch, eventListener, opts...); err != nil { return nil, err } return a, nil @@ -240,7 +235,7 @@ func (c *monitoredConn) Close() error { return nil } -func (a *connArray) Init(addr string, security config.Security, idleNotify *uint32, enableBatch bool, healthFeedbackHandler *atomic.Pointer[HealthFeedbackHandler], opts ...grpc.DialOption) error { +func (a *connArray) Init(addr string, security config.Security, idleNotify *uint32, enableBatch bool, eventListener *atomic.Pointer[ClientEventListener], opts ...grpc.DialOption) error { a.target = addr opt := grpc.WithTransportCredentials(insecure.NewCredentials()) @@ -319,17 +314,17 @@ func (a *connArray) Init(addr string, security config.Security, idleNotify *uint if allowBatch { batchClient := &batchCommandsClient{ - target: a.target, - conn: conn.ClientConn, - forwardedClients: make(map[string]*batchCommandsStream), - batched: sync.Map{}, - epoch: 0, - closed: 0, - tikvClientCfg: cfg.TiKVClient, - tikvLoad: &a.tikvTransportLayerLoad, - dialTimeout: a.dialTimeout, - tryLock: tryLock{sync.NewCond(new(sync.Mutex)), false}, - healthFeedbackHandler: healthFeedbackHandler, + target: a.target, + conn: conn.ClientConn, + forwardedClients: make(map[string]*batchCommandsStream), + batched: sync.Map{}, + epoch: 0, + closed: 0, + tikvClientCfg: cfg.TiKVClient, + tikvLoad: &a.tikvTransportLayerLoad, + dialTimeout: a.dialTimeout, + tryLock: tryLock{sync.NewCond(new(sync.Mutex)), false}, + eventListener: eventListener, } a.batchCommandsClients = append(a.batchCommandsClients, batchClient) } @@ -414,11 +409,10 @@ type RPCClient struct { connMonitor *connMonitor - healthFeedbackHandler *atomic.Pointer[HealthFeedbackHandler] + eventListener *atomic.Pointer[ClientEventListener] } var _ Client = &RPCClient{} -var _ ClientCallbackRegistry = &RPCClient{} // NewRPCClient creates a client that manages connections and rpc calls with tikv-servers. func NewRPCClient(opts ...Opt) *RPCClient { @@ -427,8 +421,8 @@ func NewRPCClient(opts ...Opt) *RPCClient { option: &option{ dialTimeout: dialTimeout, }, - connMonitor: &connMonitor{}, - healthFeedbackHandler: new(atomic.Pointer[HealthFeedbackHandler]), + connMonitor: &connMonitor{}, + eventListener: new(atomic.Pointer[ClientEventListener]), } for _, opt := range opts { opt(cli.option) @@ -480,7 +474,7 @@ func (c *RPCClient) createConnArray(addr string, enableBatch bool, opts ...func( enableBatch, c.option.dialTimeout, c.connMonitor, - c.healthFeedbackHandler, + c.eventListener, c.option.gRPCDialOptions) if err != nil { @@ -828,13 +822,8 @@ func (c *RPCClient) CloseAddr(addr string) error { return nil } -func (c *RPCClient) GetCallbackRegistry() ClientCallbackRegistry { - return c -} - -func (c *RPCClient) SetHealthFeedbackHandler(handler HealthFeedbackHandler) error { - c.healthFeedbackHandler.Store(&handler) - return nil +func (c *RPCClient) SetEventListener(listener ClientEventListener) { + c.eventListener.Store(&listener) } type spanInfo struct { diff --git a/internal/client/client_batch.go b/internal/client/client_batch.go index 9c4500b4b3..e9f96941fc 100644 --- a/internal/client/client_batch.go +++ b/internal/client/client_batch.go @@ -508,7 +508,7 @@ type batchCommandsClient struct { // tryLock protects client when re-create the streaming. tryLock - healthFeedbackHandler *atomic.Pointer[HealthFeedbackHandler] + eventListener *atomic.Pointer[ClientEventListener] } func (c *batchCommandsClient) isStopped() bool { @@ -643,8 +643,14 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient, tikvTransport continue } - logutil.BgLogger().Info("received batch response", zap.Stringer("feedback", resp.GetHealthFeedback())) if resp.GetHealthFeedback() != nil { + if val, err := util.EvalFailpoint("injectHealthFeedbackSlowScore"); err == nil { + v, ok := val.(int) + if !ok || v < 0 || v > 100 { + panic(fmt.Sprintf("invalid injection in failpoint injectHealthFeedbackSlowScore: %+q", v)) + } + resp.GetHealthFeedback().SlowScore = int32(v) + } c.onHealthFeedback(resp.GetHealthFeedback()) } @@ -679,8 +685,8 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient, tikvTransport } func (c *batchCommandsClient) onHealthFeedback(feedback *tikvpb.HealthFeedback) { - if h := c.healthFeedbackHandler.Load(); h != nil { - (*h)(feedback) + if h := c.eventListener.Load(); h != nil { + (*h).OnHealthFeedback(feedback) } } diff --git a/internal/client/client_interceptor_test.go b/internal/client/client_interceptor_test.go index 343eaf79e9..de6c6924cc 100644 --- a/internal/client/client_interceptor_test.go +++ b/internal/client/client_interceptor_test.go @@ -39,9 +39,7 @@ func (c emptyClient) CloseAddr(addr string) error { return nil } -func (c emptyClient) GetCallbackRegistry() ClientCallbackRegistry { - return nil -} +func (c emptyClient) SetEventListener(listener ClientEventListener) {} func TestInterceptedClient(t *testing.T) { executed := false diff --git a/internal/client/client_test.go b/internal/client/client_test.go index 69d32e7955..92b21bac93 100644 --- a/internal/client/client_test.go +++ b/internal/client/client_test.go @@ -153,9 +153,7 @@ func (c *chanClient) CloseAddr(addr string) error { return nil } -func (c *chanClient) GetCallbackRegistry() ClientCallbackRegistry { - return nil -} +func (c *chanClient) SetEventListener(listener ClientEventListener) {} func (c *chanClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { c.wg.Wait() diff --git a/internal/locate/region_cache.go b/internal/locate/region_cache.go index 68bccc96d9..adb0cc94b1 100644 --- a/internal/locate/region_cache.go +++ b/internal/locate/region_cache.go @@ -2597,7 +2597,6 @@ func (s *StoreHealthStatus) updateTiKVServerSideSlowScore(score int64, currTime s.tikvSideSlowScore.hasTiKVFeedback.Store(true) lastScore := s.tikvSideSlowScore.score.Load() - logutil.BgLogger().Info("updateTiKVServerSideSlowScore called", zap.Int64("score", score), zap.Int64("lastScore", lastScore)) if lastScore == score { return @@ -3453,7 +3452,7 @@ func (c *RegionCache) getCheckStoreEvents() <-chan struct{} { return c.notifyCheckCh } -func (c *RegionCache) OnHealthFeedback(feedback *tikvpb.HealthFeedback) { +func (c *RegionCache) onHealthFeedback(feedback *tikvpb.HealthFeedback) { store, ok := c.getStore(feedback.GetStoreId()) if !ok { logutil.BgLogger().Info("dropped health feedback info due to unknown store id", zap.Uint64("storeID", feedback.GetStoreId())) @@ -3461,3 +3460,15 @@ func (c *RegionCache) OnHealthFeedback(feedback *tikvpb.HealthFeedback) { } store.recordHealthFeedback(feedback) } + +func (c *RegionCache) GetClientEventListener() client.ClientEventListener { + return &RegionCacheClientEventListener{c: c} +} + +type RegionCacheClientEventListener struct { + c *RegionCache +} + +func (l *RegionCacheClientEventListener) OnHealthFeedback(feedback *tikvpb.HealthFeedback) { + l.c.onHealthFeedback(feedback) +} diff --git a/internal/locate/region_request_test.go b/internal/locate/region_request_test.go index c03d3ebaab..be990f3032 100644 --- a/internal/locate/region_request_test.go +++ b/internal/locate/region_request_test.go @@ -115,9 +115,7 @@ func (f *fnClient) CloseAddr(addr string) error { return nil } -func (f *fnClient) GetCallbackRegistry() client.ClientCallbackRegistry { - return nil -} +func (f *fnClient) SetEventListener(listener client.ClientEventListener) {} func (f *fnClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { return f.fn(ctx, addr, req, timeout) @@ -523,6 +521,14 @@ func (s *mockTikvGrpcServer) GetTiFlashSystemTable(context.Context, *kvrpcpb.TiF return nil, errors.New("unreachable") } +func (s *mockTikvGrpcServer) KvFlush(ctx context.Context, request *kvrpcpb.FlushRequest) (*kvrpcpb.FlushResponse, error) { + return nil, errors.New("unreachable") +} + +func (s *mockTikvGrpcServer) KvBufferBatchGet(ctx context.Context, request *kvrpcpb.BufferBatchGetRequest) (*kvrpcpb.BufferBatchGetResponse, error) { + return nil, errors.New("unreachable") +} + func (s *mockTikvGrpcServer) GetDisaggConfig(context.Context, *disaggregated.GetDisaggConfigRequest) (*disaggregated.GetDisaggConfigResponse, error) { return nil, errors.New("unreachable") } diff --git a/internal/mockstore/mocktikv/rpc.go b/internal/mockstore/mocktikv/rpc.go index 8db93cca41..1e1ae53419 100644 --- a/internal/mockstore/mocktikv/rpc.go +++ b/internal/mockstore/mocktikv/rpc.go @@ -1097,5 +1097,5 @@ func (c *RPCClient) CloseAddr(addr string) error { return nil } -// GetCallbackRegistry implements the ClientCallbackRegistry interface. -func (c *RPCClient) GetCallbackRegistry() client.ClientCallbackRegistry { return nil } +// SetEventListener does nothing. +func (c *RPCClient) SetEventListener(listener client.ClientEventListener) {} diff --git a/tikv/client.go b/tikv/client.go index 950490d59c..e221051cf9 100644 --- a/tikv/client.go +++ b/tikv/client.go @@ -43,7 +43,8 @@ import ( // Client is a client that sends RPC. // It should not be used after calling Close(). type Client = client.Client -type ClientCallbackRegistry = client.ClientCallbackRegistry + +type ClientEventListener = client.ClientEventListener // ClientOpt defines the option to create RPC client. type ClientOpt = client.Opt diff --git a/tikv/kv.go b/tikv/kv.go index 464dfa1345..bfd103b9e8 100644 --- a/tikv/kv.go +++ b/tikv/kv.go @@ -40,7 +40,6 @@ import ( "fmt" "math" "math/rand" - "reflect" "strconv" "sync" "sync/atomic" @@ -245,23 +244,7 @@ func NewKVStore(uuid string, pdClient pd.Client, spkv SafePointKV, tikvclient Cl gP: NewSpool(128, 10*time.Second), } store.clientMu.client = client.NewReqCollapse(client.NewInterceptedClient(tikvclient)) - - healthFeedbackHandlerSet := false - if r := store.clientMu.client.GetCallbackRegistry(); r != nil { - if err = r.SetHealthFeedbackHandler(regionCache.OnHealthFeedback); err == nil { - logutil.BgLogger().Debug("health feedback handler is set") - healthFeedbackHandlerSet = true - } else { - logutil.BgLogger().Warn("failed to set health feedback handler", zap.String("type", reflect.TypeOf(tikvclient).String()), zap.Error(err)) - } - } else { - logutil.BgLogger().Warn("health feedback handler not set due to the client type doesn't support", zap.String("type", reflect.TypeOf(tikvclient).String())) - } - if _, err := util.EvalFailpoint("healthFeedbackMustSet"); err == nil { - if !healthFeedbackHandlerSet { - panic("health feedback handler not set ") - } - } + store.clientMu.client.SetEventListener(regionCache.GetClientEventListener()) store.lockResolver = txnlock.NewLockResolver(store) loadOption(store, opt...) From 1af7e4acc0bbef2af9635af3543c8d566b3b1c19 Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Wed, 21 Feb 2024 21:48:33 +0800 Subject: [PATCH 11/21] Add tests Signed-off-by: MyonKeminta --- integration_tests/go.mod | 2 +- integration_tests/go.sum | 4 +- internal/client/client_test.go | 37 +++++++++++++++++++ .../client/mockserver/mock_tikv_service.go | 8 ++++ 4 files changed, 48 insertions(+), 3 deletions(-) diff --git a/integration_tests/go.mod b/integration_tests/go.mod index 6b59748a21..1d48cd1042 100644 --- a/integration_tests/go.mod +++ b/integration_tests/go.mod @@ -118,6 +118,6 @@ 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/MyonKeminta/tidb v1.1.0-alpha.1.0.20240219090303-2af99d5fa939 + github.com/pingcap/tidb => github.com/MyonKeminta/tidb v1.1.0-alpha.1.0.20240219124656-9c7d7bff2362 github.com/tikv/client-go/v2 => ../ ) diff --git a/integration_tests/go.sum b/integration_tests/go.sum index f72cb2992a..c24419bbe2 100644 --- a/integration_tests/go.sum +++ b/integration_tests/go.sum @@ -35,8 +35,8 @@ github.com/Joker/hpp v1.0.0/go.mod h1:8x5n+M1Hp5hC0g8okX3sR3vFQwynaX/UgSOM9MeBKz 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/MyonKeminta/tidb v1.1.0-alpha.1.0.20240219090303-2af99d5fa939 h1:LX/bEQYImIlJ+WYTWjujerBMRSyb3YoxRBKB6DpmtuM= -github.com/MyonKeminta/tidb v1.1.0-alpha.1.0.20240219090303-2af99d5fa939/go.mod h1:5oopS5coE669N5WVZwlZDyuHpAGU1XvBTmCgLlBhDk8= +github.com/MyonKeminta/tidb v1.1.0-alpha.1.0.20240219124656-9c7d7bff2362 h1:etbPZap5HTYRMqzu+dJP0tG+utTs36Es6GegTtfVp1k= +github.com/MyonKeminta/tidb v1.1.0-alpha.1.0.20240219124656-9c7d7bff2362/go.mod h1:1E0oIs7Jl5LBAvvI0ApRGXJZEdiTwQM39kuHEw9Fg4Q= 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= diff --git a/internal/client/client_test.go b/internal/client/client_test.go index 92b21bac93..b2b87f75fc 100644 --- a/internal/client/client_test.go +++ b/internal/client/client_test.go @@ -724,3 +724,40 @@ func TestBatchClientRecoverAfterServerRestart(t *testing.T) { require.NoError(t, err) } } + +func TestBatchClientReceiveHealthFeedback(t *testing.T) { + server, port := mockserver.StartMockTikvService() + require.True(t, port > 0) + require.True(t, server.IsRunning()) + defer server.Stop() + addr := server.Addr() + + client := NewRPCClient() + defer client.Close() + + conn, err := client.getConnArray(addr, true) + assert.NoError(t, err) + tikvClient := tikvpb.NewTikvClient(conn.Get()) + stream, err := tikvClient.BatchCommands(context.Background()) + assert.NoError(t, err) + + for reqID := uint64(1); reqID <= 3; reqID++ { + assert.NoError(t, stream.Send(&tikvpb.BatchCommandsRequest{ + Requests: []*tikvpb.BatchCommandsRequest_Request{{ + Cmd: &tikvpb.BatchCommandsRequest_Request_Get{Get: &kvrpcpb.GetRequest{ + Context: &kvrpcpb.Context{}, + Key: []byte("k"), + Version: 1, + }}, + }}, + RequestIds: []uint64{reqID}, + })) + resp, err := stream.Recv() + assert.NoError(t, err) + assert.Equal(t, []uint64{reqID}, resp.GetRequestIds()) + assert.Len(t, resp.GetResponses(), 1) + assert.Equal(t, uint64(1), resp.GetHealthFeedback().GetStoreId()) + assert.Equal(t, reqID, resp.GetHealthFeedback().GetFeedbackSeqNo()) + assert.Equal(t, int32(1), resp.GetHealthFeedback().GetSlowScore()) + } +} diff --git a/internal/client/mockserver/mock_tikv_service.go b/internal/client/mockserver/mock_tikv_service.go index 90c1b535dc..e0964f7359 100644 --- a/internal/client/mockserver/mock_tikv_service.go +++ b/internal/client/mockserver/mock_tikv_service.go @@ -79,6 +79,7 @@ func (s *MockServer) BatchCommands(ss tikvpb.Tikv_BatchCommandsServer) error { if err := s.checkMetadata(ss.Context()); err != nil { return err } + var feedbackSeq uint64 = 1 for { req, err := ss.Recv() if err != nil { @@ -98,7 +99,14 @@ func (s *MockServer) BatchCommands(ss tikvpb.Tikv_BatchCommandsServer) error { err = ss.Send(&tikvpb.BatchCommandsResponse{ Responses: responses, RequestIds: req.GetRequestIds(), + HealthFeedback: &tikvpb.HealthFeedback{ + StoreId: 1, + FeedbackSeqNo: feedbackSeq, + SlowScore: 1, + }, }) + feedbackSeq++ + if err != nil { logutil.BgLogger().Error("batch commands send fail", zap.Error(err)) return err From 3d685e7c6f66856f83c0c0ca7ab0f04937541335 Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Wed, 21 Feb 2024 22:02:30 +0800 Subject: [PATCH 12/21] Test receiving froim callback Signed-off-by: MyonKeminta --- internal/client/client_test.go | 36 +++++++++++++++++++++++++++++++++- 1 file changed, 35 insertions(+), 1 deletion(-) diff --git a/internal/client/client_test.go b/internal/client/client_test.go index b2b87f75fc..c3359ade71 100644 --- a/internal/client/client_test.go +++ b/internal/client/client_test.go @@ -725,6 +725,20 @@ func TestBatchClientRecoverAfterServerRestart(t *testing.T) { } } +type testClientEventListener struct { + healthFeedbackCh chan *tikvpb.HealthFeedback +} + +func newTestClientEventListener() *testClientEventListener { + return &testClientEventListener{ + healthFeedbackCh: make(chan *tikvpb.HealthFeedback, 100), + } +} + +func (l *testClientEventListener) OnHealthFeedback(feedback *tikvpb.HealthFeedback) { + l.healthFeedbackCh <- feedback +} + func TestBatchClientReceiveHealthFeedback(t *testing.T) { server, port := mockserver.StartMockTikvService() require.True(t, port > 0) @@ -738,7 +752,10 @@ func TestBatchClientReceiveHealthFeedback(t *testing.T) { conn, err := client.getConnArray(addr, true) assert.NoError(t, err) tikvClient := tikvpb.NewTikvClient(conn.Get()) - stream, err := tikvClient.BatchCommands(context.Background()) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + stream, err := tikvClient.BatchCommands(ctx) assert.NoError(t, err) for reqID := uint64(1); reqID <= 3; reqID++ { @@ -760,4 +777,21 @@ func TestBatchClientReceiveHealthFeedback(t *testing.T) { assert.Equal(t, reqID, resp.GetHealthFeedback().GetFeedbackSeqNo()) assert.Equal(t, int32(1), resp.GetHealthFeedback().GetSlowScore()) } + cancel() + + eventListener := newTestClientEventListener() + client.SetEventListener(eventListener) + ctx = context.Background() + resp, err := client.SendRequest(ctx, addr, tikvrpc.NewRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{}), time.Second) + assert.NoError(t, err) + assert.NotNil(t, resp.Resp) + + select { + case feedback := <-eventListener.healthFeedbackCh: + assert.Equal(t, uint64(1), feedback.GetStoreId()) + assert.Equal(t, int32(1), feedback.GetSlowScore()) + default: + assert.Fail(t, "health feedback not received") + } + } From 91c7bbef10330c30032982cc58602810c669e001 Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Mon, 26 Feb 2024 16:04:24 +0800 Subject: [PATCH 13/21] Add tests to region cache Signed-off-by: MyonKeminta --- internal/locate/region_cache.go | 34 ++++++-- internal/locate/region_cache_test.go | 118 +++++++++++++++++++++++++++ 2 files changed, 143 insertions(+), 9 deletions(-) diff --git a/internal/locate/region_cache.go b/internal/locate/region_cache.go index adb0cc94b1..d29cadaca5 100644 --- a/internal/locate/region_cache.go +++ b/internal/locate/region_cache.go @@ -2510,7 +2510,6 @@ type StoreHealthStatus struct { clientSideSlowScore SlowScoreStat tikvSideSlowScore struct { - // Assuming the type `Store` is always used in heap instead of in stack sync.Mutex // These atomic fields should be read with atomic operations and written with the mutex. @@ -2526,6 +2525,10 @@ type HealthStatusDetail struct { TiKVSideSlowScore int64 } +func newStoreHealthStatus() *StoreHealthStatus { + return &StoreHealthStatus{} +} + // IsSlow returns whether current Store is slow or not. func (s *StoreHealthStatus) IsSlow() bool { return s.isSlow.Load() @@ -2539,26 +2542,28 @@ func (s *StoreHealthStatus) GetHealthStatusDetail() HealthStatusDetail { } // update updates the slow score of this store according to the timecost of current request. -func (s *StoreHealthStatus) update() { +func (s *StoreHealthStatus) update(now time.Time) { s.clientSideSlowScore.updateSlowScore() - s.updateTiKVServerSideSlowScoreOnTick() + s.updateTiKVServerSideSlowScoreOnTick(now) + s.updateSlowFlag() } // recordClientSideSlowScoreStat records timecost of each request. func (s *StoreHealthStatus) recordClientSideSlowScoreStat(timecost time.Duration) { s.clientSideSlowScore.recordSlowScoreStat(timecost) + s.updateSlowFlag() } // markAlreadySlow marks the related store already slow. func (s *StoreHealthStatus) markAlreadySlow() { s.clientSideSlowScore.markAlreadySlow() + s.updateSlowFlag() } -func (s *StoreHealthStatus) updateTiKVServerSideSlowScoreOnTick() { +func (s *StoreHealthStatus) updateTiKVServerSideSlowScoreOnTick(now time.Time) { if !s.tikvSideSlowScore.hasTiKVFeedback.Load() { return } - now := time.Now() lastUpdateTime := s.tikvSideSlowScore.lastUpdateTime.Load() if lastUpdateTime == nil || now.Sub(*lastUpdateTime) < tikvSlowScoreUpdateFromPDInterval { return @@ -2577,7 +2582,8 @@ func (s *StoreHealthStatus) updateTiKVServerSideSlowScoreOnTick() { return } - // TODO: Try to get store status from PD. + // TODO: Try to get store status from PD here. But it's not mandatory. + // Don't forget to update tests if getting slow score from PD is implemented here. // If updating from PD is not successful: decay the slow score. score := s.tikvSideSlowScore.score.Load() @@ -2594,6 +2600,7 @@ func (s *StoreHealthStatus) updateTiKVServerSideSlowScoreOnTick() { } func (s *StoreHealthStatus) updateTiKVServerSideSlowScore(score int64, currTime time.Time) { + defer s.updateSlowFlag() s.tikvSideSlowScore.hasTiKVFeedback.Store(true) lastScore := s.tikvSideSlowScore.score.Load() @@ -2635,6 +2642,14 @@ func (s *StoreHealthStatus) updateSlowFlag() { s.isSlow.Store(isSlow) } +// setTiKVSlowScoreLastUpdateTimeForTest force sets last update time of TiKV server side slow score to specified value. +// For test purpose only. +func (s *StoreHealthStatus) setTiKVSlowScoreLastUpdateTimeForTest(lastUpdateTime time.Time) { + s.tikvSideSlowScore.Lock() + defer s.tikvSideSlowScore.Unlock() + s.tikvSideSlowScore.lastUpdateTime.Store(&lastUpdateTime) +} + // Store contains a kv process's address. type Store struct { addr string // loaded store address @@ -2720,7 +2735,7 @@ func newStore( peerAddr: peerAddr, saddr: statusAddr, // Make sure healthStatus field is never null. - healthStatus: &StoreHealthStatus{}, + healthStatus: newStoreHealthStatus(), } } @@ -2729,7 +2744,7 @@ func newUninitializedStore(id uint64) *Store { return &Store{ storeID: id, // Make sure healthStatus field is never null. - healthStatus: &StoreHealthStatus{}, + healthStatus: newStoreHealthStatus(), } } @@ -3232,8 +3247,9 @@ func (c *RegionCache) checkAndUpdateStoreHealthStats() { } }() healthDetails := make(map[uint64]HealthStatusDetail) + now := time.Now() c.forEachStore(func(store *Store) { - store.healthStatus.update() + store.healthStatus.update(now) healthDetails[store.storeID] = store.healthStatus.GetHealthStatusDetail() }) logutil.BgLogger().Info("checkAndUpdateStoreHealthStats: get health details", zap.Reflect("details", healthDetails)) diff --git a/internal/locate/region_cache_test.go b/internal/locate/region_cache_test.go index db8382aa5e..b97ea81845 100644 --- a/internal/locate/region_cache_test.go +++ b/internal/locate/region_cache_test.go @@ -50,6 +50,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/errorpb" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/kvproto/pkg/tikvpb" "github.com/stretchr/testify/suite" "github.com/tikv/client-go/v2/config/retry" "github.com/tikv/client-go/v2/internal/apicodec" @@ -1929,6 +1930,123 @@ func (s *testRegionCacheSuite) TestHealthCheckWithStoreReplace() { }, 3*time.Second, time.Second) } +func (s *testRegionCacheSuite) TestTiKVSideSlowScore() { + stats := newStoreHealthStatus() + s.LessOrEqual(stats.GetHealthStatusDetail().TiKVSideSlowScore, int64(1)) + now := time.Now() + stats.update(now) + s.LessOrEqual(stats.GetHealthStatusDetail().TiKVSideSlowScore, int64(1)) + s.False(stats.tikvSideSlowScore.hasTiKVFeedback.Load()) + s.False(stats.IsSlow()) + + now = now.Add(tikvSlowScoreUpdateInterval * 2) + stats.updateTiKVServerSideSlowScore(50, now) + s.Equal(int64(50), stats.GetHealthStatusDetail().TiKVSideSlowScore) + s.True(stats.tikvSideSlowScore.hasTiKVFeedback.Load()) + s.False(stats.IsSlow()) + + now = now.Add(tikvSlowScoreUpdateInterval * 2) + stats.updateTiKVServerSideSlowScore(100, now) + s.Equal(int64(100), stats.GetHealthStatusDetail().TiKVSideSlowScore) + s.True(stats.IsSlow()) + + now = now.Add(time.Minute * 2) + stats.update(now) + s.Equal(int64(60), stats.GetHealthStatusDetail().TiKVSideSlowScore) + s.False(stats.IsSlow()) + + now = now.Add(time.Minute * 3) + stats.update(now) + s.Equal(int64(1), stats.GetHealthStatusDetail().TiKVSideSlowScore) + s.False(stats.IsSlow()) + + now = now.Add(time.Minute) + stats.update(now) + s.Equal(int64(1), stats.GetHealthStatusDetail().TiKVSideSlowScore) + s.False(stats.IsSlow()) +} + +func (s *testRegionCacheSuite) TestStoreHealthStatus() { + stats := newStoreHealthStatus() + now := time.Now() + s.False(stats.IsSlow()) + + for !stats.clientSideSlowScore.isSlow() { + stats.clientSideSlowScore.recordSlowScoreStat(time.Minute) + } + stats.update(now) + s.True(stats.IsSlow()) + s.Equal(int64(stats.clientSideSlowScore.getSlowScore()), stats.GetHealthStatusDetail().ClientSideSlowScore) + + now = now.Add(time.Second) + stats.updateTiKVServerSideSlowScore(100, now) + s.True(stats.IsSlow()) + s.Equal(int64(100), stats.GetHealthStatusDetail().TiKVSideSlowScore) + + for stats.clientSideSlowScore.isSlow() { + stats.clientSideSlowScore.recordSlowScoreStat(time.Millisecond) + stats.update(now) + } + s.True(stats.IsSlow()) + s.Equal(int64(stats.clientSideSlowScore.getSlowScore()), stats.GetHealthStatusDetail().ClientSideSlowScore) + + now = now.Add(time.Second) + stats.updateTiKVServerSideSlowScore(1, now) + s.False(stats.IsSlow()) +} + +func (s *testRegionCacheSuite) TestRegionCacheHandleHealthStatus() { + _, err := s.cache.LocateKey(s.bo, []byte("k")) + s.Nil(err) + + store1, exists := s.cache.getStore(s.store1) + s.True(exists) + s.False(store1.healthStatus.IsSlow()) + + feedbackMsg := &tikvpb.HealthFeedback{ + StoreId: s.store1, + FeedbackSeqNo: 1, + SlowScore: 100, + } + s.cache.onHealthFeedback(feedbackMsg) + s.True(store1.healthStatus.IsSlow()) + s.Equal(int64(100), store1.healthStatus.GetHealthStatusDetail().TiKVSideSlowScore) + + feedbackMsg = &tikvpb.HealthFeedback{ + StoreId: s.store1, + FeedbackSeqNo: 2, + SlowScore: 90, + } + // Ignore too frequent update + s.cache.onHealthFeedback(feedbackMsg) + s.Equal(int64(100), store1.healthStatus.GetHealthStatusDetail().TiKVSideSlowScore) + + feedbackMsg = &tikvpb.HealthFeedback{ + StoreId: s.store1, + FeedbackSeqNo: 3, + SlowScore: 90, + } + store1.healthStatus.setTiKVSlowScoreLastUpdateTimeForTest(time.Now().Add(-time.Second)) + s.cache.onHealthFeedback(feedbackMsg) + s.Equal(int64(90), store1.healthStatus.GetHealthStatusDetail().TiKVSideSlowScore) + + feedbackMsg = &tikvpb.HealthFeedback{ + StoreId: s.store1, + FeedbackSeqNo: 4, + SlowScore: 50, + } + store1.healthStatus.setTiKVSlowScoreLastUpdateTimeForTest(time.Now().Add(-time.Second)) + s.cache.onHealthFeedback(feedbackMsg) + s.False(store1.healthStatus.IsSlow()) + s.Equal(int64(50), store1.healthStatus.GetHealthStatusDetail().TiKVSideSlowScore) + + store2, exists := s.cache.getStore(s.store2) + s.True(exists) + // Store 2 is never affected by updating store 1 + s.LessOrEqual(store2.healthStatus.GetHealthStatusDetail().TiKVSideSlowScore, int64(1)) + s.False(store2.healthStatus.IsSlow()) +} + func (s *testRegionRequestToSingleStoreSuite) TestRefreshCache() { _ = s.cache.refreshRegionIndex(s.bo) r, _ := s.cache.scanRegionsFromCache(s.bo, []byte{}, nil, 10) From 903ff59744e5f32a0b8e347c669309ccf42162dc Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Mon, 26 Feb 2024 16:24:30 +0800 Subject: [PATCH 14/21] Remove unnecessary debug log Signed-off-by: MyonKeminta --- internal/locate/region_cache.go | 1 - 1 file changed, 1 deletion(-) diff --git a/internal/locate/region_cache.go b/internal/locate/region_cache.go index d29cadaca5..33458bab2c 100644 --- a/internal/locate/region_cache.go +++ b/internal/locate/region_cache.go @@ -3252,7 +3252,6 @@ func (c *RegionCache) checkAndUpdateStoreHealthStats() { store.healthStatus.update(now) healthDetails[store.storeID] = store.healthStatus.GetHealthStatusDetail() }) - logutil.BgLogger().Info("checkAndUpdateStoreHealthStats: get health details", zap.Reflect("details", healthDetails)) for store, details := range healthDetails { metrics.TiKVStoreSlowScoreGauge.WithLabelValues(strconv.FormatUint(store, 10)).Set(float64(details.ClientSideSlowScore)) metrics.TiKVFeedbackSlowScoreGauge.WithLabelValues(strconv.FormatUint(store, 10)).Set(float64(details.TiKVSideSlowScore)) From daa3dd025bc8cb48bf2f7aed0e6268462ef14964 Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Mon, 26 Feb 2024 18:56:42 +0800 Subject: [PATCH 15/21] Add comments Signed-off-by: MyonKeminta --- internal/client/client.go | 2 ++ tikv/client.go | 1 + 2 files changed, 3 insertions(+) diff --git a/internal/client/client.go b/internal/client/client.go index 4b2343daca..9d54837f28 100644 --- a/internal/client/client.go +++ b/internal/client/client.go @@ -113,7 +113,9 @@ type Client interface { SetEventListener(listener ClientEventListener) } +// ClientEventListener is a listener to handle events produced by `Client`. type ClientEventListener interface { + // OnHealthFeedback is called when `Client` receives a response that carries the HealthFeedback information. OnHealthFeedback(feedback *tikvpb.HealthFeedback) } diff --git a/tikv/client.go b/tikv/client.go index e221051cf9..9e92ed69d9 100644 --- a/tikv/client.go +++ b/tikv/client.go @@ -44,6 +44,7 @@ import ( // It should not be used after calling Close(). type Client = client.Client +// ClientEventListener is a listener to handle events produced by `Client`. type ClientEventListener = client.ClientEventListener // ClientOpt defines the option to create RPC client. From f800ae1a9e696682afdbc2e10bbd4fca3bace337 Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Mon, 26 Feb 2024 19:09:18 +0800 Subject: [PATCH 16/21] remove replace of tidb repo Signed-off-by: MyonKeminta --- integration_tests/go.mod | 1 - integration_tests/go.sum | 114 +++++++++++++++++++-------------------- 2 files changed, 57 insertions(+), 58 deletions(-) diff --git a/integration_tests/go.mod b/integration_tests/go.mod index 1d48cd1042..9548dfcbc6 100644 --- a/integration_tests/go.mod +++ b/integration_tests/go.mod @@ -118,6 +118,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/MyonKeminta/tidb v1.1.0-alpha.1.0.20240219124656-9c7d7bff2362 github.com/tikv/client-go/v2 => ../ ) diff --git a/integration_tests/go.sum b/integration_tests/go.sum index c24419bbe2..8ea2b2fa14 100644 --- a/integration_tests/go.sum +++ b/integration_tests/go.sum @@ -35,8 +35,6 @@ github.com/Joker/hpp v1.0.0/go.mod h1:8x5n+M1Hp5hC0g8okX3sR3vFQwynaX/UgSOM9MeBKz 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/MyonKeminta/tidb v1.1.0-alpha.1.0.20240219124656-9c7d7bff2362 h1:etbPZap5HTYRMqzu+dJP0tG+utTs36Es6GegTtfVp1k= -github.com/MyonKeminta/tidb v1.1.0-alpha.1.0.20240219124656-9c7d7bff2362/go.mod h1:1E0oIs7Jl5LBAvvI0ApRGXJZEdiTwQM39kuHEw9Fg4Q= 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= @@ -49,8 +47,8 @@ github.com/alexbrainman/sspi v0.0.0-20210105120005-909beea2cc74 h1:Kk6a4nehpJ3Uu 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/apache/thrift v0.16.0 h1:qEy6UW60iVOlUy+b9ZR0d5WzUWYGOo4HfopoyBaNmoY= -github.com/apache/thrift v0.16.0/go.mod h1:PHK3hniurgQaNMZYaCLEqXKsYK8upmhPbmdP2FXSqgU= +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/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2 h1:DklsrG3dyBCFEj5IhUbnKptjxatkF07cF2ak3yi77so= github.com/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2/go.mod h1:WaHUgvxTVq04UNunO+XhnAqY/wQc+bxr74GqbsZ/Jqw= @@ -131,9 +129,8 @@ 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/dustin/go-humanize v1.0.1 h1:GzkhY7T5VNhEkwH0PVJgjz+fX1rhBrR7pRT3mDkpeCY= -github.com/dustin/go-humanize v1.0.1/go.mod h1:Mu1zIs6XwVuF/gI1OepvI0qD18qycQx+mFykh5fBlto= 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= @@ -143,8 +140,8 @@ github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1m github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= 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.16.0 h1:zmkK9Ngbjj+K0yRhTVONQh1p/HknKYSlNT+vZCzyokM= -github.com/fatih/color v1.16.0/go.mod h1:fL2Sau1YI5c0pdGEVCbKQbLXB6edEj1ZgiY4NijnWvE= +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/felixge/httpsnoop v1.0.4 h1:NFTV2Zj1bL4mc9sqWACXbQFVBBg2W3GPvqp8/ESS2Wg= github.com/felixge/httpsnoop v1.0.4/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U= @@ -191,8 +188,8 @@ github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXP 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/v4 v4.5.0 h1:7cYmW1XlMY7h7ii7UhUyChSgS5wUJEnm9uZVTGqOWzg= -github.com/golang-jwt/jwt/v4 v4.5.0/go.mod h1:m21LjoU+eqJr34lmDMbreY2eSTRJ1cv77w39/MY0Ch0= +github.com/golang-jwt/jwt/v4 v4.4.2 h1:rcc4lwaZgFMCZ5jxF9ABolDcIHdBytAFgqFPbSJQAYs= +github.com/golang-jwt/jwt/v4 v4.4.2/go.mod h1:m21LjoU+eqJr34lmDMbreY2eSTRJ1cv77w39/MY0Ch0= github.com/golang-jwt/jwt/v5 v5.0.0 h1:1n1XNM9hk7O9mnQoNBGolZvzebBQ7p93ULHRc28XJUE= github.com/golang-jwt/jwt/v5 v5.0.0/go.mod h1:pqrtFR0X4osieyHYxtmOUWsAWrfe1Q5UVIyoH402zdk= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= @@ -249,16 +246,16 @@ github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORR 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.5.1 h1:gmztn0JnHVt9JZquRuzLw3g4wouNVzKL15iLr/zn/QY= -github.com/gorilla/websocket v1.5.1/go.mod h1:x3kM2JMyaluk02fnUJpQuwD2dCS5NDG2ZHL0uE0tcaY= +github.com/gorilla/websocket v1.5.0 h1:PPwGk2jz7EePpoHN/+ClbZu8SPxiqlu12wZP/3sWmnc= +github.com/gorilla/websocket v1.5.0/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= github.com/grpc-ecosystem/grpc-gateway v1.16.0 h1:gmcG1KaJ57LophUzW0Hy8NmPhnMZb4M0+kPpLofRdBo= github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.19.1 h1:/c3QmbOGMGTOumP2iT/rCwB7b0QDGLKzqOmktBjT+Is= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.19.1/go.mod h1:5SN9VR2LTsRFsrEC6FHgRbTWrTHu6tqPeKxEQv15giM= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.16.0 h1:YBftPWNWd4WwGqtY2yeZL2ef8rHAxPBD8KFhJpmcqms= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.16.0/go.mod h1:YN5jB8ie0yfIUg6VvR9Kz84aCaG7AsGZnLjhHbUqwPg= github.com/hashicorp/go-version v1.2.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= @@ -287,8 +284,8 @@ github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9Y 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.4.0 h1:p4Cf1aMWXnXAUh8lVfewRBx1zaTSYKrKMF2g3ST4RZ4= -github.com/jonboulle/clockwork v0.4.0/go.mod h1:xgRqUGwRcjKCO1vbZUEtSLrqKoPSsUpK7fnezOII0kc= +github.com/jonboulle/clockwork v0.2.2 h1:UOGuzwb1PwsrDAObMuhUnj0p5ULPj8V/xJ7Kx9qUBdQ= +github.com/jonboulle/clockwork v0.2.2/go.mod h1:Pkfl5aHPm1nk2H9h0bjmnJD/BcgbGXUBGnn1kMkgxc8= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= @@ -349,10 +346,10 @@ github.com/mattn/go-colorable v0.1.13/go.mod h1:7S9/ev0klgBDR4GtXTXX8a3vIGJpMovk 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.20 h1:xfD0iDuEKnDkl03q4limB+vH+GxLEtL/jb4xVJSWWEY= -github.com/mattn/go-isatty v0.0.20/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y= -github.com/mattn/go-runewidth v0.0.15 h1:UNAjwbU9l54TA3KzvqLGxwWjHmMgBUVhBiTjelZgg3U= -github.com/mattn/go-runewidth v0.0.15/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= +github.com/mattn/go-isatty v0.0.19 h1:JITubQf0MOLdlGRuRq+jtsDlekdYPia9ZFsB8h/APPA= +github.com/mattn/go-isatty v0.0.19/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/mediocregopher/mediocre-go-lib v0.0.0-20181029021733-cb65787f37ed/go.mod h1:dSsfyI2zABAdhcbvkXqgxOxrCsbYeHCPgrZkku60dSg= @@ -423,6 +420,8 @@ 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/tidb v1.1.0-beta.0.20240131080924-732fa8c98695 h1:7ELVL+rVzEZrDv4dEC995qHbv9K0dSKm7+wuwotiB8U= +github.com/pingcap/tidb v1.1.0-beta.0.20240131080924-732fa8c98695/go.mod h1:rfdtaBHvL6w0uFmxz0BJIBHHUpR/satNS695+def0y8= github.com/pingcap/tidb/pkg/parser v0.0.0-20240219094226-fa340f3400aa h1:d3c9Xm6Kzwh2lT3Y/SsvMYb2jJExF+lwzXcvRtEC2qk= github.com/pingcap/tidb/pkg/parser v0.0.0-20240219094226-fa340f3400aa/go.mod h1:MWQK6otJgZRI6zcCVPV22U4qE26qOGJnN4fq8XawgBs= github.com/pingcap/tipb v0.0.0-20240116032918-9bb28c43bbfc h1:sEp4lbExDfnMX8HXQyhZrhqo2/SgeFY5KOdo5akc8FM= @@ -454,8 +453,8 @@ github.com/prometheus/procfs v0.12.0 h1:jluTpSng7V9hY0O2R9DzzJHYb2xULk9VTR1V1R/k github.com/prometheus/procfs v0.12.0/go.mod h1:pcuDEFsWDnvcgNzo4EEweacyhjeA9Zk3cnaOZAZEfOo= 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.6 h1:Sovz9sDSwbOz9tgUy8JpT+KgCkPYJEN/oYzlJiYTNLg= -github.com/rivo/uniseg v0.4.6/go.mod h1:FN3SvrM+Zdj16jyLfmOkMNblXMcoc8DfTHruCPUcx88= +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/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= @@ -492,8 +491,8 @@ github.com/soheilhy/cmux v0.1.5/go.mod h1:T7TcVDs9LWfQgPlPsdngu6I6QIoyIFZDDC6sNE 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.8.0 h1:7aJaZx1B85qltLMc546zn58BxxfZdR/W22ej9CFoEf0= -github.com/spf13/cobra v1.8.0/go.mod h1:WXLWApfZ71AjXPya3WOlMsY9yMs7YeiHhFVlvLyhcho= +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= @@ -533,8 +532,8 @@ github.com/tklauser/go-sysconf v0.3.12/go.mod h1:Ho14jnntGE1fpdOqQEEaiKRpvIavV0h github.com/tklauser/numcpus v0.3.0/go.mod h1:yFGUr7TUHQRAhyqBcEg0Ge34zDBAsIvJJcyE6boqnA8= github.com/tklauser/numcpus v0.6.1 h1:ng9scYS7az0Bk4OZLvrNXNSAO2Pxr1XXRAPyjhIx+Fk= github.com/tklauser/numcpus v0.6.1/go.mod h1:1XfjsgE2zo8GVw7POkMbHENHzVg3GzmoZ9fESEdAacY= -github.com/tmc/grpc-websocket-proxy v0.0.0-20220101234140-673ab2c3ae75 h1:6fotK7otjonDflCTK0BCfls4SPy3NcCVb5dqqmbRknE= -github.com/tmc/grpc-websocket-proxy v0.0.0-20220101234140-673ab2c3ae75/go.mod h1:KO6IkyS8Y3j8OdNO85qEYBsRPuteD+YciPomcXdrMnk= +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= @@ -555,8 +554,8 @@ github.com/wangjohn/quickselect v0.0.0-20161129230411-ed8402a42d5f/go.mod h1:8sd 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-20221125231312-a49e3df8f510 h1:S2dVYn90KE98chqDkyE9Z4N61UnQd+KOfgp5Iu53llk= -github.com/xiang90/probing v0.0.0-20221125231312-a49e3df8f510/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= +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= @@ -577,38 +576,38 @@ go.etcd.io/etcd/api/v3 v3.5.12 h1:W4sw5ZoU2Juc9gBWuLk5U6fHfNVyY1WC5g9uiXZio/c= go.etcd.io/etcd/api/v3 v3.5.12/go.mod h1:Ot+o0SWSyT6uHhA56al1oCED0JImsRiU9Dc26+C2a+4= go.etcd.io/etcd/client/pkg/v3 v3.5.12 h1:EYDL6pWwyOsylrQyLp2w+HkQ46ATiOvoEdMarindU2A= go.etcd.io/etcd/client/pkg/v3 v3.5.12/go.mod h1:seTzl2d9APP8R5Y2hFL3NVlD6qC/dOT+3kvrqPyTas4= -go.etcd.io/etcd/client/v2 v2.305.12 h1:0m4ovXYo1CHaA/Mp3X/Fak5sRNIWf01wk/X1/G3sGKI= -go.etcd.io/etcd/client/v2 v2.305.12/go.mod h1:aQ/yhsxMu+Oht1FOupSr60oBvcS9cKXHrzBpDsPTf9E= +go.etcd.io/etcd/client/v2 v2.305.10 h1:MrmRktzv/XF8CvtQt+P6wLUlURaNpSDJHFZhe//2QE4= +go.etcd.io/etcd/client/v2 v2.305.10/go.mod h1:m3CKZi69HzilhVqtPDcjhSGp+kA1OmbNn0qamH80xjA= go.etcd.io/etcd/client/v3 v3.5.12 h1:v5lCPXn1pf1Uu3M4laUE2hp/geOTc5uPcYYsNe1lDxg= go.etcd.io/etcd/client/v3 v3.5.12/go.mod h1:tSbBCakoWmmddL+BKVAJHa9km+O/E+bumDe9mSbPiqw= -go.etcd.io/etcd/pkg/v3 v3.5.12 h1:OK2fZKI5hX/+BTK76gXSTyZMrbnARyX9S643GenNGb8= -go.etcd.io/etcd/pkg/v3 v3.5.12/go.mod h1:UVwg/QIMoJncyeb/YxvJBJCE/NEwtHWashqc8A1nj/M= -go.etcd.io/etcd/raft/v3 v3.5.12 h1:7r22RufdDsq2z3STjoR7Msz6fYH8tmbkdheGfwJNRmU= -go.etcd.io/etcd/raft/v3 v3.5.12/go.mod h1:ERQuZVe79PI6vcC3DlKBukDCLja/L7YMu29B74Iwj4U= -go.etcd.io/etcd/server/v3 v3.5.12 h1:EtMjsbfyfkwZuA2JlKOiBfuGkFCekv5H178qjXypbG8= -go.etcd.io/etcd/server/v3 v3.5.12/go.mod h1:axB0oCjMy+cemo5290/CutIjoxlfA6KVYKD1w0uue10= -go.etcd.io/etcd/tests/v3 v3.5.12 h1:k1fG7+F87Z7zKp57EcjXu9XgOsW0sfp5USqfzmMTIwM= -go.etcd.io/etcd/tests/v3 v3.5.12/go.mod h1:CLWdnlr8bWNa8tjkmKFybPz5Ldjh9GuHbYhq1g9vpIo= +go.etcd.io/etcd/pkg/v3 v3.5.10 h1:WPR8K0e9kWl1gAhB5A7gEa5ZBTNkT9NdNWrR8Qpo1CM= +go.etcd.io/etcd/pkg/v3 v3.5.10/go.mod h1:TKTuCKKcF1zxmfKWDkfz5qqYaE3JncKKZPFf8c1nFUs= +go.etcd.io/etcd/raft/v3 v3.5.10 h1:cgNAYe7xrsrn/5kXMSaH8kM/Ky8mAdMqGOxyYwpP0LA= +go.etcd.io/etcd/raft/v3 v3.5.10/go.mod h1:odD6kr8XQXTy9oQnyMPBOr0TVe+gT0neQhElQ6jbGRc= +go.etcd.io/etcd/server/v3 v3.5.10 h1:4NOGyOwD5sUZ22PiWYKmfxqoeh72z6EhYjNosKGLmZg= +go.etcd.io/etcd/server/v3 v3.5.10/go.mod h1:gBplPHfs6YI0L+RpGkTQO7buDbHv5HJGG/Bst0/zIPo= +go.etcd.io/etcd/tests/v3 v3.5.10 h1:F1pbXwKxwZ58aBT2+CSL/r8WUCAVhob0y1y8OVJ204s= +go.etcd.io/etcd/tests/v3 v3.5.10/go.mod h1:vVMWDv9OhopxfJCd+CMI4pih0zUDqlkJj6JcBNlUVXI= 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/instrumentation/google.golang.org/grpc/otelgrpc v0.47.0 h1:UNQQKPfTDe1J81ViolILjTKPr9WetKW6uei2hFgJmFs= -go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.47.0/go.mod h1:r9vWsPS/3AQItv3OSlEJ/E4mbrhUbbw18meOjArPtKQ= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.46.1 h1:SpGay3w+nEwMpfVnbqOLH5gY52/foP8RE8UzTZ1pdSE= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.46.1/go.mod h1:4UoMYEZOC0yN/sPGH76KPkkU7zgiEWYWL9vwmbnTJPE= go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.46.1 h1:aFJWCqJMNjENlcleuuOkGAPH82y0yULBScfXcIEdS24= go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.46.1/go.mod h1:sEGXWArGqc3tVa+ekntsN65DmVbVeW+7lTKTjZF3/Fo= -go.opentelemetry.io/otel v1.22.0 h1:xS7Ku+7yTFvDfDraDIJVpw7XPyuHlB9MCiqqX5mcJ6Y= -go.opentelemetry.io/otel v1.22.0/go.mod h1:eoV4iAi3Ea8LkAEI9+GFT44O6T/D0GWAVFyZVCC6pMI= -go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.22.0 h1:9M3+rhx7kZCIQQhQRYaZCdNu1V73tm4TvXs2ntl98C4= -go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.22.0/go.mod h1:noq80iT8rrHP1SfybmPiRGc9dc5M8RPmGvtwo7Oo7tc= -go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.22.0 h1:H2JFgRcGiyHg7H7bwcwaQJYrNFqCqrbTQ8K4p1OvDu8= -go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.22.0/go.mod h1:WfCWp1bGoYK8MeULtI15MmQVczfR+bFkk0DF3h06QmQ= -go.opentelemetry.io/otel/metric v1.22.0 h1:lypMQnGyJYeuYPhOM/bgjbFM6WE44W1/T45er4d8Hhg= -go.opentelemetry.io/otel/metric v1.22.0/go.mod h1:evJGjVpZv0mQ5QBRJoBF64yMuOf4xCWdXjK8pzFvliY= -go.opentelemetry.io/otel/sdk v1.22.0 h1:6coWHw9xw7EfClIC/+O31R8IY3/+EiRFHevmHafB2Gw= -go.opentelemetry.io/otel/sdk v1.22.0/go.mod h1:iu7luyVGYovrRpe2fmj3CVKouQNdTOkxtLzPvPz1DOc= -go.opentelemetry.io/otel/trace v1.22.0 h1:Hg6pPujv0XG9QaVbGOBVHunyuLcCC3jN7WEhPx83XD0= -go.opentelemetry.io/otel/trace v1.22.0/go.mod h1:RbbHXVqKES9QhzZq/fE5UnOSILqRt40a21sPw2He1xo= -go.opentelemetry.io/proto/otlp v1.1.0 h1:2Di21piLrCqJ3U3eXGCTPHE9R8Nh+0uglSnOyxikMeI= -go.opentelemetry.io/proto/otlp v1.1.0/go.mod h1:GpBHCBWiqvVLDqmHZsoMM3C5ySeKTC7ej/RNTae6MdY= +go.opentelemetry.io/otel v1.21.0 h1:hzLeKBZEL7Okw2mGzZ0cc4k/A7Fta0uoPgaJCr8fsFc= +go.opentelemetry.io/otel v1.21.0/go.mod h1:QZzNPQPm1zLX4gZK4cMi+71eaorMSGT3A4znnUvNNEo= +go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.21.0 h1:cl5P5/GIfFh4t6xyruOgJP5QiA1pw4fYYdv6nc6CBWw= +go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.21.0/go.mod h1:zgBdWWAu7oEEMC06MMKc5NLbA/1YDXV1sMpSqEeLQLg= +go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.21.0 h1:tIqheXEFWAZ7O8A7m+J0aPTmpJN3YQ7qetUAdkkkKpk= +go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.21.0/go.mod h1:nUeKExfxAQVbiVFn32YXpXZZHZ61Cc3s3Rn1pDBGAb0= +go.opentelemetry.io/otel/metric v1.21.0 h1:tlYWfeo+Bocx5kLEloTjbcDwBuELRrIFxwdQ36PlJu4= +go.opentelemetry.io/otel/metric v1.21.0/go.mod h1:o1p3CA8nNHW8j5yuQLdc1eeqEaPfzug24uvsyIEJRWM= +go.opentelemetry.io/otel/sdk v1.21.0 h1:FTt8qirL1EysG6sTQRZ5TokkU8d0ugCj8htOgThZXQ8= +go.opentelemetry.io/otel/sdk v1.21.0/go.mod h1:Nna6Yv7PWTdgJHVRD9hIYywQBRx7pbox6nwBnZIxl/E= +go.opentelemetry.io/otel/trace v1.21.0 h1:WD9i5gzvoUPuXIXH24ZNBudiarZDKuekPqi/E8fpfLc= +go.opentelemetry.io/otel/trace v1.21.0/go.mod h1:LGbsEB0f9LGjN+OZaQQ26sohbOmiMR+BaslueVtS/qQ= +go.opentelemetry.io/proto/otlp v1.0.0 h1:T0TX0tmXU8a3CbNXzEKGeU5mIVOdf0oykP+u2lIVU/I= +go.opentelemetry.io/proto/otlp v1.0.0/go.mod h1:Sy6pihPLfYHkr3NkUbEhGHFhINUSI/v80hjKIs5JXpM= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= @@ -784,8 +783,9 @@ golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= 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= -gonum.org/v1/gonum v0.0.0-20181121035319-3f7ecaa7e8ca h1:PupagGYwj8+I4ubCxcmcBRk3VlUWtTg5huQpZR9flmE= 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.156.0 h1:yloYcGbBtVYjLKQe4enCunxvwn3s2w/XPrrhVf6MsvQ= google.golang.org/api v0.156.0/go.mod h1:bUSmn4KFO0Q+69zo9CNIDp4Psi6BqM0np0CbzKRSiSY= @@ -868,8 +868,8 @@ sigs.k8s.io/json v0.0.0-20221116044647-bc3834ca7abd h1:EDPBXCAspyGV4jQlpZSudPeMm sigs.k8s.io/json v0.0.0-20221116044647-bc3834ca7abd/go.mod h1:B8JuhiUyNFVKdsE8h686QcCxMaH6HrOAZj4vswFpcB0= sigs.k8s.io/structured-merge-diff/v4 v4.3.0 h1:UZbZAZfX0wV2zr7YZorDz6GXROfDFj6LvqCRm4VUVKk= sigs.k8s.io/structured-merge-diff/v4 v4.3.0/go.mod h1:N8hJocpFajUSSeSJ9bOZ77VzejKZaXsTtZo4/u7Io08= -sigs.k8s.io/yaml v1.4.0 h1:Mk1wCc2gy/F0THH0TAp1QYyJNzRm2KCLy3o5ASXVI5E= -sigs.k8s.io/yaml v1.4.0/go.mod h1:Ejl7/uTz7PSA4eKMyQCUTnhZYNmLIl+5c2lQPGR2BPY= +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= From f99235a1c82e9b893c0f64db2e89dff7031afec6 Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Tue, 27 Feb 2024 11:40:27 +0800 Subject: [PATCH 17/21] fix build Signed-off-by: MyonKeminta --- integration_tests/async_commit_test.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/integration_tests/async_commit_test.go b/integration_tests/async_commit_test.go index 388d7df4fc..8aa36fe986 100644 --- a/integration_tests/async_commit_test.go +++ b/integration_tests/async_commit_test.go @@ -76,9 +76,7 @@ type unistoreClientWrapper struct { *unistore.RPCClient } -func (c *unistoreClientWrapper) CloseAddr(addr string) error { - return nil -} +func (c *unistoreClientWrapper) SetEventListener(listener tikv.ClientEventListener) {} func (s *testAsyncCommitCommon) setUpTest() { if *withTiKV { From 0be8fd65d4903457344eee7397916bc3e0df799f Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Tue, 27 Feb 2024 14:06:57 +0800 Subject: [PATCH 18/21] Update comments; fix lint Signed-off-by: MyonKeminta --- internal/client/client.go | 6 +++-- internal/client/client_batch.go | 2 ++ internal/locate/region_cache.go | 38 ++++++++++++++++++---------- internal/locate/region_cache_test.go | 12 ++++----- 4 files changed, 37 insertions(+), 21 deletions(-) diff --git a/internal/client/client.go b/internal/client/client.go index f5ce41c1b3..0212081cb7 100644 --- a/internal/client/client.go +++ b/internal/client/client.go @@ -108,8 +108,8 @@ type Client interface { CloseAddr(addr string) error // SendRequest sends Request. SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) - // SetEventListener registers an event listener for the Client instance. If called more than once, the previously - // set one will be replaced. + // SetEventListener registers an event listener for the Client instance. If it's called more than once, the + // previously set one will be replaced. SetEventListener(listener ClientEventListener) } @@ -826,6 +826,8 @@ func (c *RPCClient) CloseAddr(addr string) error { return nil } +// SetEventListener registers an event listener for the Client instance. If it's called more than once, the +// previously set one will be replaced. func (c *RPCClient) SetEventListener(listener ClientEventListener) { c.eventListener.Store(&listener) } diff --git a/internal/client/client_batch.go b/internal/client/client_batch.go index 6e703dcabb..3964efc1f1 100644 --- a/internal/client/client_batch.go +++ b/internal/client/client_batch.go @@ -557,6 +557,8 @@ type batchCommandsClient struct { // maxConcurrencyRequestLimit is the max allowed number of requests to be sent the tikv maxConcurrencyRequestLimit atomic.Int64 + // eventListener is the listener set by external code to observe some events in the client. It's stored in a atomic + // pointer to make setting thread-safe. eventListener *atomic.Pointer[ClientEventListener] } diff --git a/internal/locate/region_cache.go b/internal/locate/region_cache.go index 341de77fd0..be1a0ef439 100644 --- a/internal/locate/region_cache.go +++ b/internal/locate/region_cache.go @@ -2595,7 +2595,8 @@ type StoreHealthStatus struct { tikvSideSlowScore struct { sync.Mutex - // These atomic fields should be read with atomic operations and written with the mutex. + // The following atomic fields is designed to be able to be read by atomic options directly, but only written + // while holding the mutex. hasTiKVFeedback atomic.Bool score atomic.Int64 @@ -2612,11 +2613,12 @@ func newStoreHealthStatus() *StoreHealthStatus { return &StoreHealthStatus{} } -// IsSlow returns whether current Store is slow or not. +// IsSlow returns whether current Store is slow. func (s *StoreHealthStatus) IsSlow() bool { return s.isSlow.Load() } +// GetHealthStatusDetail gets the current detailed information about the store's health status. func (s *StoreHealthStatus) GetHealthStatusDetail() HealthStatusDetail { return HealthStatusDetail{ ClientSideSlowScore: int64(s.clientSideSlowScore.getSlowScore()), @@ -2624,14 +2626,15 @@ func (s *StoreHealthStatus) GetHealthStatusDetail() HealthStatusDetail { } } -// update updates the slow score of this store according to the timecost of current request. -func (s *StoreHealthStatus) update(now time.Time) { +// tick updates the health status that changes over time, such as slow score's decaying, etc. This function is expected +// to be called periodically. +func (s *StoreHealthStatus) tick(now time.Time) { s.clientSideSlowScore.updateSlowScore() s.updateTiKVServerSideSlowScoreOnTick(now) s.updateSlowFlag() } -// recordClientSideSlowScoreStat records timecost of each request. +// recordClientSideSlowScoreStat records timecost of each request to update the client side slow score. func (s *StoreHealthStatus) recordClientSideSlowScoreStat(timecost time.Duration) { s.clientSideSlowScore.recordSlowScoreStat(timecost) s.updateSlowFlag() @@ -2643,17 +2646,21 @@ func (s *StoreHealthStatus) markAlreadySlow() { s.updateSlowFlag() } +// updateTiKVServerSideSlowScoreOnTick updates the slow score actively, which is expected to be a periodic job. +// It skips updating if the last update time didn't elapse long enough, or it's being updated concurrently. func (s *StoreHealthStatus) updateTiKVServerSideSlowScoreOnTick(now time.Time) { if !s.tikvSideSlowScore.hasTiKVFeedback.Load() { + // Do nothing if no feedback has been received from this store yet. return } lastUpdateTime := s.tikvSideSlowScore.lastUpdateTime.Load() if lastUpdateTime == nil || now.Sub(*lastUpdateTime) < tikvSlowScoreUpdateFromPDInterval { + // If the first feedback is return } if !s.tikvSideSlowScore.TryLock() { - // It must be being updated. + // It must be being updated concurrently. return } defer s.tikvSideSlowScore.Unlock() @@ -2682,9 +2689,10 @@ func (s *StoreHealthStatus) updateTiKVServerSideSlowScoreOnTick(now time.Time) { s.tikvSideSlowScore.lastUpdateTime.Store(newUpdateTime) } +// updateTiKVServerSideSlowScore updates the tikv side slow score with the given value. +// Ignores if the last update time didn't elapse long enough, or it's being updated concurrently. func (s *StoreHealthStatus) updateTiKVServerSideSlowScore(score int64, currTime time.Time) { defer s.updateSlowFlag() - s.tikvSideSlowScore.hasTiKVFeedback.Store(true) lastScore := s.tikvSideSlowScore.score.Load() @@ -2699,18 +2707,18 @@ func (s *StoreHealthStatus) updateTiKVServerSideSlowScore(score int64, currTime } if !s.tikvSideSlowScore.TryLock() { - // It must be being updated. Skip. + // It must be being updated concurrently. Skip. return } defer s.tikvSideSlowScore.Unlock() + s.tikvSideSlowScore.hasTiKVFeedback.Store(true) // Reload update time as it might be updated concurrently before acquiring mutex lastUpdateTime = s.tikvSideSlowScore.lastUpdateTime.Load() if lastUpdateTime != nil && currTime.Sub(*lastUpdateTime) < tikvSlowScoreUpdateInterval { return } - lastScore = s.tikvSideSlowScore.score.Load() newScore := score newUpdateTime := new(time.Time) @@ -3332,7 +3340,7 @@ func (c *RegionCache) checkAndUpdateStoreHealthStatus() { healthDetails := make(map[uint64]HealthStatusDetail) now := time.Now() c.forEachStore(func(store *Store) { - store.healthStatus.update(now) + store.healthStatus.tick(now) healthDetails[store.storeID] = store.healthStatus.GetHealthStatusDetail() }) for store, details := range healthDetails { @@ -3547,14 +3555,18 @@ func (c *RegionCache) onHealthFeedback(feedback *tikvpb.HealthFeedback) { store.recordHealthFeedback(feedback) } +// GetClientEventListener returns the listener to observe the RPC client's events and let the region cache respond to +// them. When creating the `KVStore` using `tikv.NewKVStore` function, the listener will be setup immediately. func (c *RegionCache) GetClientEventListener() client.ClientEventListener { - return &RegionCacheClientEventListener{c: c} + return ®ionCacheClientEventListener{c: c} } -type RegionCacheClientEventListener struct { +// regionCacheClientEventListener is the listener to let RegionCache respond to events in the RPC client. +type regionCacheClientEventListener struct { c *RegionCache } -func (l *RegionCacheClientEventListener) OnHealthFeedback(feedback *tikvpb.HealthFeedback) { +// OnHealthFeedback implements the `client.ClientEventListener` interface. +func (l *regionCacheClientEventListener) OnHealthFeedback(feedback *tikvpb.HealthFeedback) { l.c.onHealthFeedback(feedback) } diff --git a/internal/locate/region_cache_test.go b/internal/locate/region_cache_test.go index 30c03db2af..441df39c7f 100644 --- a/internal/locate/region_cache_test.go +++ b/internal/locate/region_cache_test.go @@ -2091,7 +2091,7 @@ func (s *testRegionCacheSuite) TestTiKVSideSlowScore() { stats := newStoreHealthStatus() s.LessOrEqual(stats.GetHealthStatusDetail().TiKVSideSlowScore, int64(1)) now := time.Now() - stats.update(now) + stats.tick(now) s.LessOrEqual(stats.GetHealthStatusDetail().TiKVSideSlowScore, int64(1)) s.False(stats.tikvSideSlowScore.hasTiKVFeedback.Load()) s.False(stats.IsSlow()) @@ -2108,17 +2108,17 @@ func (s *testRegionCacheSuite) TestTiKVSideSlowScore() { s.True(stats.IsSlow()) now = now.Add(time.Minute * 2) - stats.update(now) + stats.tick(now) s.Equal(int64(60), stats.GetHealthStatusDetail().TiKVSideSlowScore) s.False(stats.IsSlow()) now = now.Add(time.Minute * 3) - stats.update(now) + stats.tick(now) s.Equal(int64(1), stats.GetHealthStatusDetail().TiKVSideSlowScore) s.False(stats.IsSlow()) now = now.Add(time.Minute) - stats.update(now) + stats.tick(now) s.Equal(int64(1), stats.GetHealthStatusDetail().TiKVSideSlowScore) s.False(stats.IsSlow()) } @@ -2131,7 +2131,7 @@ func (s *testRegionCacheSuite) TestStoreHealthStatus() { for !stats.clientSideSlowScore.isSlow() { stats.clientSideSlowScore.recordSlowScoreStat(time.Minute) } - stats.update(now) + stats.tick(now) s.True(stats.IsSlow()) s.Equal(int64(stats.clientSideSlowScore.getSlowScore()), stats.GetHealthStatusDetail().ClientSideSlowScore) @@ -2142,7 +2142,7 @@ func (s *testRegionCacheSuite) TestStoreHealthStatus() { for stats.clientSideSlowScore.isSlow() { stats.clientSideSlowScore.recordSlowScoreStat(time.Millisecond) - stats.update(now) + stats.tick(now) } s.True(stats.IsSlow()) s.Equal(int64(stats.clientSideSlowScore.getSlowScore()), stats.GetHealthStatusDetail().ClientSideSlowScore) From cf5330d7c9c2132761e20db7b265ee5868665557 Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Tue, 27 Feb 2024 18:28:13 +0800 Subject: [PATCH 19/21] Add the isSlow method of Store back Signed-off-by: MyonKeminta --- internal/locate/region_cache.go | 28 ++++------------------------ 1 file changed, 4 insertions(+), 24 deletions(-) diff --git a/internal/locate/region_cache.go b/internal/locate/region_cache.go index be1a0ef439..85422e8fe1 100644 --- a/internal/locate/region_cache.go +++ b/internal/locate/region_cache.go @@ -3299,30 +3299,10 @@ func invokeKVStatusAPI(addr string, timeout time.Duration) (l livenessState) { return } -//// getSlowScore returns the slow score of store. -//func (s *Store) getSlowScore() uint64 { -// return s.slowScore.getSlowScore() -//} -// -//// isSlow returns whether current Store is slow or not. -//func (s *Store) isSlow() bool { -// return s.slowScore.isSlow() -//} -// -//// updateSlowScore updates the slow score of this store according to the timecost of current request. -//func (s *Store) updateSlowScoreStat() { -// s.slowScore.updateSlowScore() -//} -// -//// recordSlowScoreStat records timecost of each request. -//func (s *Store) recordSlowScoreStat(timecost time.Duration) { -// s.slowScore.recordSlowScoreStat(timecost) -//} -// -//// markAlreadySlow marks the related store already slow. -//func (s *Store) markAlreadySlow() { -// s.slowScore.markAlreadySlow() -//} +// isSlow returns whether current Store is slow or not. +func (s *Store) isSlow() bool { + return s.healthStatus.IsSlow() +} // checkAndUpdateStoreHealthStatus checks and updates health stats on each store. func (c *RegionCache) checkAndUpdateStoreHealthStatus() { From c6535a8750b8bcf703b773292c7491242b1e4d79 Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Wed, 28 Feb 2024 13:37:35 +0800 Subject: [PATCH 20/21] remove unused method for now to make lint happy Signed-off-by: MyonKeminta --- internal/locate/region_cache.go | 5 ----- 1 file changed, 5 deletions(-) diff --git a/internal/locate/region_cache.go b/internal/locate/region_cache.go index 85422e8fe1..78b3ee08ea 100644 --- a/internal/locate/region_cache.go +++ b/internal/locate/region_cache.go @@ -3299,11 +3299,6 @@ func invokeKVStatusAPI(addr string, timeout time.Duration) (l livenessState) { return } -// isSlow returns whether current Store is slow or not. -func (s *Store) isSlow() bool { - return s.healthStatus.IsSlow() -} - // checkAndUpdateStoreHealthStatus checks and updates health stats on each store. func (c *RegionCache) checkAndUpdateStoreHealthStatus() { defer func() { From eb1421fc9605fc69a38bbba94d3af6f30be19df6 Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Wed, 28 Feb 2024 16:56:18 +0800 Subject: [PATCH 21/21] Address comments Signed-off-by: MyonKeminta --- internal/locate/region_cache.go | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/internal/locate/region_cache.go b/internal/locate/region_cache.go index 78b3ee08ea..4b8db934d0 100644 --- a/internal/locate/region_cache.go +++ b/internal/locate/region_cache.go @@ -2579,8 +2579,8 @@ func (r *Region) ContainsByEnd(key []byte) bool { } const ( - tikvSlowScoreDecayRate = 20. / 60. // s^(-1), linear decaying - tikvSlowScoreSlowThreshold = 80. + tikvSlowScoreDecayRate float64 = 20.0 / 60.0 // s^(-1), linear decaying + tikvSlowScoreSlowThreshold int64 = 80 tikvSlowScoreUpdateInterval = time.Millisecond * 100 tikvSlowScoreUpdateFromPDInterval = time.Minute @@ -3340,6 +3340,9 @@ func (s *Store) recordReplicaFlowsStats(destType replicaFlowsType) { } func (s *Store) recordHealthFeedback(feedback *tikvpb.HealthFeedback) { + // Note that the `FeedbackSeqNo` field of `HealthFeedback` is not used yet. It's a monotonic value that can help + // to drop out-of-order feedback messages. But it's not checked for now since it's not very necessary to receive + // only a slow score. It's prepared for possible use in the future. s.healthStatus.updateTiKVServerSideSlowScore(int64(feedback.GetSlowScore()), time.Now()) }