From ea9ca6e528d382af51f06b0b192ade02df4f21cd Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Tue, 3 Dec 2024 18:30:43 +0800 Subject: [PATCH 01/11] Support validating normal read operations Signed-off-by: MyonKeminta --- oracle/oracle.go | 7 +++---- oracle/oracles/export_test.go | 2 +- oracle/oracles/local.go | 2 +- oracle/oracles/mock.go | 2 +- oracle/oracles/pd.go | 38 +++++++++++++++++++---------------- oracle/oracles/pd_test.go | 16 +++++++-------- 6 files changed, 35 insertions(+), 32 deletions(-) diff --git a/oracle/oracle.go b/oracle/oracle.go index a4ffa1b93b..cc71b5db78 100644 --- a/oracle/oracle.go +++ b/oracle/oracle.go @@ -68,12 +68,11 @@ type Oracle interface { // GetAllTSOKeyspaceGroupMinTS gets a minimum timestamp from all TSO keyspace groups. GetAllTSOKeyspaceGroupMinTS(ctx context.Context) (uint64, error) - // ValidateSnapshotReadTS verifies whether it can be guaranteed that the given readTS doesn't exceed the maximum ts - // that has been allocated by the oracle, so that it's safe to use this ts to perform snapshot read, stale read, - // etc. + // ValidateReadTS verifies whether it can be guaranteed that the given readTS doesn't exceed the maximum ts + // that has been allocated by the oracle, so that it's safe to use this ts to perform read operations. // Note that this method only checks the ts from the oracle's perspective. It doesn't check whether the snapshot // has been GCed. - ValidateSnapshotReadTS(ctx context.Context, readTS uint64, opt *Option) error + ValidateReadTS(ctx context.Context, readTS uint64, isStaleRead bool, opt *Option) error } // Future is a future which promises to return a timestamp. diff --git a/oracle/oracles/export_test.go b/oracle/oracles/export_test.go index b5e874aafd..d18c962436 100644 --- a/oracle/oracles/export_test.go +++ b/oracle/oracles/export_test.go @@ -87,6 +87,6 @@ func SetEmptyPDOracleLastTs(oc oracle.Oracle, ts uint64) { case *pdOracle: lastTSInterface, _ := o.lastTSMap.LoadOrStore(oracle.GlobalTxnScope, &atomic.Pointer[lastTSO]{}) lastTSPointer := lastTSInterface.(*atomic.Pointer[lastTSO]) - lastTSPointer.Store(&lastTSO{tso: ts, arrival: ts}) + lastTSPointer.Store(&lastTSO{tso: ts, arrival: oracle.GetTimeFromTS(ts)}) } } diff --git a/oracle/oracles/local.go b/oracle/oracles/local.go index bf9b43c892..909b35c8fc 100644 --- a/oracle/oracles/local.go +++ b/oracle/oracles/local.go @@ -150,7 +150,7 @@ func (l *localOracle) GetExternalTimestamp(ctx context.Context) (uint64, error) return l.getExternalTimestamp(ctx) } -func (l *localOracle) ValidateSnapshotReadTS(ctx context.Context, readTS uint64, opt *oracle.Option) error { +func (l *localOracle) ValidateReadTS(ctx context.Context, readTS uint64, isStaleRead bool, opt *oracle.Option) error { currentTS, err := l.GetTimestamp(ctx, opt) if err != nil { return errors.Errorf("fail to validate read timestamp: %v", err) diff --git a/oracle/oracles/mock.go b/oracle/oracles/mock.go index cab3335aba..3129664aee 100644 --- a/oracle/oracles/mock.go +++ b/oracle/oracles/mock.go @@ -137,7 +137,7 @@ func (o *MockOracle) SetLowResolutionTimestampUpdateInterval(time.Duration) erro return nil } -func (o *MockOracle) ValidateSnapshotReadTS(ctx context.Context, readTS uint64, opt *oracle.Option) error { +func (o *MockOracle) ValidateReadTS(ctx context.Context, readTS uint64, isStaleRead bool, opt *oracle.Option) error { currentTS, err := o.GetTimestamp(ctx, opt) if err != nil { return errors.Errorf("fail to validate read timestamp: %v", err) diff --git a/oracle/oracles/pd.go b/oracle/oracles/pd.go index 310e9b5f19..193d6fb045 100644 --- a/oracle/oracles/pd.go +++ b/oracle/oracles/pd.go @@ -147,7 +147,7 @@ type pdOracle struct { // When the low resolution ts is not new enough and there are many concurrent stane read / snapshot read // operations that needs to validate the read ts, we can use this to avoid too many concurrent GetTS calls by - // reusing a result for different `ValidateSnapshotReadTS` calls. This can be done because that + // reusing a result for different `ValidateReadTS` calls. This can be done because that // we don't require the ts for validation to be strictly the latest one. // Note that the result can't be reused for different txnScopes. The txnScope is used as the key. tsForValidation singleflight.Group @@ -156,7 +156,7 @@ type pdOracle struct { // lastTSO stores the last timestamp oracle gets from PD server and the local time when the TSO is fetched. type lastTSO struct { tso uint64 - arrival uint64 + arrival time.Time } type PDOracleOptions struct { @@ -290,17 +290,13 @@ func (o *pdOracle) getMinTimestampInAllTSOGroup(ctx context.Context) (uint64, er return oracle.ComposeTS(physical, logical), nil } -func (o *pdOracle) getArrivalTimestamp() uint64 { - return oracle.GoTimeToTS(time.Now()) -} - func (o *pdOracle) setLastTS(ts uint64, txnScope string) { if txnScope == "" { txnScope = oracle.GlobalTxnScope } current := &lastTSO{ tso: ts, - arrival: o.getArrivalTimestamp(), + arrival: time.Now(), } lastTSInterface, ok := o.lastTSMap.Load(txnScope) if !ok { @@ -312,7 +308,7 @@ func (o *pdOracle) setLastTS(ts uint64, txnScope string) { lastTSPointer := lastTSInterface.(*atomic.Pointer[lastTSO]) for { last := lastTSPointer.Load() - if current.tso <= last.tso || current.arrival <= last.arrival { + if current.tso <= last.tso || !current.arrival.After(last.arrival) { return } if lastTSPointer.CompareAndSwap(last, current) { @@ -614,8 +610,11 @@ func (o *pdOracle) getStaleTimestamp(txnScope string, prevSecond uint64) (uint64 if !ok { return 0, errors.Errorf("get stale timestamp fail, txnScope: %s", txnScope) } - ts, arrivalTS := last.tso, last.arrival - arrivalTime := oracle.GetTimeFromTS(arrivalTS) + return o.getStaleTimestampWithLastTS(last, prevSecond) +} + +func (o *pdOracle) getStaleTimestampWithLastTS(last *lastTSO, prevSecond uint64) (uint64, error) { + ts, arrivalTime := last.tso, last.arrival physicalTime := oracle.GetTimeFromTS(ts) if uint64(physicalTime.Unix()) <= prevSecond { return 0, errors.Errorf("invalid prevSecond %v", prevSecond) @@ -670,22 +669,24 @@ func (o *pdOracle) getCurrentTSForValidation(ctx context.Context, opt *oracle.Op } } -func (o *pdOracle) ValidateSnapshotReadTS(ctx context.Context, readTS uint64, opt *oracle.Option) error { - latestTS, err := o.GetLowResolutionTimestamp(ctx, opt) - // If we fail to get latestTS or the readTS exceeds it, get a timestamp from PD to double-check. +func (o *pdOracle) ValidateReadTS(ctx context.Context, readTS uint64, isStaleRead bool, opt *oracle.Option) error { + latestTSInfo, exists := o.getLastTSWithArrivalTS(opt.TxnScope) + // If we fail to get latestTSInfo or the readTS exceeds it, get a timestamp from PD to double-check. // But we don't need to strictly fetch the latest TS. So if there are already concurrent calls to this function // loading the latest TS, we can just reuse the same result to avoid too many concurrent GetTS calls. - if err != nil || readTS > latestTS { + if !exists || readTS > latestTSInfo.tso { currentTS, err := o.getCurrentTSForValidation(ctx, opt) if err != nil { return errors.Errorf("fail to validate read timestamp: %v", err) } - o.adjustUpdateLowResolutionTSIntervalWithRequestedStaleness(readTS, currentTS, time.Now()) + if isStaleRead { + o.adjustUpdateLowResolutionTSIntervalWithRequestedStaleness(readTS, currentTS, time.Now()) + } if readTS > currentTS { return errors.Errorf("cannot set read timestamp to a future time") } - } else { - estimatedCurrentTS, err := o.getStaleTimestamp(opt.TxnScope, 0) + } else if isStaleRead { + estimatedCurrentTS, err := o.getStaleTimestampWithLastTS(latestTSInfo, 0) if err != nil { logutil.Logger(ctx).Warn("failed to estimate current ts by getSlateTimestamp for auto-adjusting update low resolution ts interval", zap.Error(err), zap.Uint64("readTS", readTS), zap.String("txnScope", opt.TxnScope)) @@ -696,6 +697,9 @@ func (o *pdOracle) ValidateSnapshotReadTS(ctx context.Context, readTS uint64, op return nil } +// adjustUpdateLowResolutionTSIntervalWithRequestedStaleness triggers adjustments the update interval of low resolution +// ts, if necessary, to suite the usage of stale read. +// This method is not supposed to be called when performing non-stale-read operations. func (o *pdOracle) adjustUpdateLowResolutionTSIntervalWithRequestedStaleness(readTS uint64, currentTS uint64, now time.Time) { requiredStaleness := oracle.GetTimeFromTS(currentTS).Sub(oracle.GetTimeFromTS(readTS)) diff --git a/oracle/oracles/pd_test.go b/oracle/oracles/pd_test.go index b1fa6d3e33..1412795340 100644 --- a/oracle/oracles/pd_test.go +++ b/oracle/oracles/pd_test.go @@ -342,7 +342,7 @@ func TestAdaptiveUpdateTSInterval(t *testing.T) { assert.Equal(t, adaptiveUpdateTSIntervalStateUnadjustable, o.adaptiveUpdateIntervalState.state) } -func TestValidateSnapshotReadTS(t *testing.T) { +func TestValidateReadTSForStaleRead(t *testing.T) { pdClient := MockPdClient{} o, err := NewPdOracle(&pdClient, &PDOracleOptions{ UpdateInterval: time.Second * 2, @@ -356,25 +356,25 @@ func TestValidateSnapshotReadTS(t *testing.T) { assert.NoError(t, err) assert.GreaterOrEqual(t, ts, uint64(1)) - err = o.ValidateSnapshotReadTS(ctx, 1, opt) + err = o.ValidateReadTS(ctx, 1, true, opt) assert.NoError(t, err) ts, err = o.GetTimestamp(ctx, opt) assert.NoError(t, err) // The readTS exceeds the latest ts, so it first fails the check with the low resolution ts. Then it fallbacks to // the fetching-from-PD path, and it can get the previous ts + 1, which can allow this validation to pass. - err = o.ValidateSnapshotReadTS(ctx, ts+1, opt) + err = o.ValidateReadTS(ctx, ts+1, true, opt) assert.NoError(t, err) // It can't pass if the readTS is newer than previous ts + 2. ts, err = o.GetTimestamp(ctx, opt) assert.NoError(t, err) - err = o.ValidateSnapshotReadTS(ctx, ts+2, opt) + err = o.ValidateReadTS(ctx, ts+2, true, opt) assert.Error(t, err) // Simulate other PD clients requests a timestamp. ts, err = o.GetTimestamp(ctx, opt) assert.NoError(t, err) pdClient.logicalTimestamp.Add(2) - err = o.ValidateSnapshotReadTS(ctx, ts+3, opt) + err = o.ValidateReadTS(ctx, ts+3, true, opt) assert.NoError(t, err) } @@ -397,7 +397,7 @@ func (c *MockPDClientWithPause) Resume() { c.mu.Unlock() } -func TestValidateSnapshotReadTSReusingGetTSResult(t *testing.T) { +func TestValidateReadTSForStaleReadReusingGetTSResult(t *testing.T) { pdClient := &MockPDClientWithPause{} o, err := NewPdOracle(pdClient, &PDOracleOptions{ UpdateInterval: time.Second * 2, @@ -409,7 +409,7 @@ func TestValidateSnapshotReadTSReusingGetTSResult(t *testing.T) { asyncValidate := func(ctx context.Context, readTS uint64) chan error { ch := make(chan error, 1) go func() { - err := o.ValidateSnapshotReadTS(ctx, readTS, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) + err := o.ValidateReadTS(ctx, readTS, true, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) ch <- err }() return ch @@ -418,7 +418,7 @@ func TestValidateSnapshotReadTSReusingGetTSResult(t *testing.T) { noResult := func(ch chan error) { select { case <-ch: - assert.FailNow(t, "a ValidateSnapshotReadTS operation is not blocked while it's expected to be blocked") + assert.FailNow(t, "a ValidateReadTS operation is not blocked while it's expected to be blocked") default: } } From 3ffa5cabd5a7659676edeeb5282e78d81d331257 Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Wed, 4 Dec 2024 13:55:37 +0800 Subject: [PATCH 02/11] pass oracle to RegionRequestSender and do checks for all requests Signed-off-by: MyonKeminta --- internal/locate/region_cache_test.go | 5 ++-- internal/locate/region_request.go | 30 +++++++++++++++++--- internal/locate/region_request3_test.go | 10 +++---- internal/locate/region_request_state_test.go | 3 +- internal/locate/region_request_test.go | 15 +++++----- internal/locate/replica_selector_test.go | 8 +++--- oracle/oracle.go | 15 ++++++++++ rawkv/rawkv.go | 9 +++--- tikv/kv.go | 2 +- tikv/region.go | 5 ++-- tikv/split_region.go | 2 +- txnkv/transaction/commit.go | 2 +- txnkv/transaction/pessimistic.go | 2 +- txnkv/transaction/pipelined_flush.go | 2 +- txnkv/transaction/prewrite.go | 2 +- txnkv/txnsnapshot/client_helper.go | 5 +++- txnkv/txnsnapshot/scan.go | 2 +- 17 files changed, 82 insertions(+), 37 deletions(-) diff --git a/internal/locate/region_cache_test.go b/internal/locate/region_cache_test.go index 110a321463..37a17d14c9 100644 --- a/internal/locate/region_cache_test.go +++ b/internal/locate/region_cache_test.go @@ -59,6 +59,7 @@ import ( "github.com/tikv/client-go/v2/internal/apicodec" "github.com/tikv/client-go/v2/internal/mockstore/mocktikv" "github.com/tikv/client-go/v2/kv" + "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/tikvrpc" pd "github.com/tikv/pd/client" uatomic "go.uber.org/atomic" @@ -1333,7 +1334,7 @@ func (s *testRegionCacheSuite) TestRegionEpochOnTiFlash() { s.Equal(ctxTiFlash.Peer.Id, s.peer1) ctxTiFlash.Peer.Role = metapb.PeerRole_Learner r := ctxTiFlash.Meta - reqSend := NewRegionRequestSender(s.cache, nil) + reqSend := NewRegionRequestSender(s.cache, nil, oracle.NoopReadTSValidator{}) regionErr := &errorpb.Error{EpochNotMatch: &errorpb.EpochNotMatch{CurrentRegions: []*metapb.Region{r}}} reqSend.onRegionError(s.bo, ctxTiFlash, nil, regionErr) @@ -1970,7 +1971,7 @@ func (s *testRegionCacheSuite) TestShouldNotRetryFlashback() { ctx, err := s.cache.GetTiKVRPCContext(retry.NewBackofferWithVars(context.Background(), 100, nil), loc.Region, kv.ReplicaReadLeader, 0) s.NotNil(ctx) s.NoError(err) - reqSend := NewRegionRequestSender(s.cache, nil) + reqSend := NewRegionRequestSender(s.cache, nil, oracle.NoopReadTSValidator{}) shouldRetry, err := reqSend.onRegionError(s.bo, ctx, nil, &errorpb.Error{FlashbackInProgress: &errorpb.FlashbackInProgress{}}) s.Error(err) s.False(shouldRetry) diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index e30ecf0e09..5dee15a7ff 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -38,6 +38,7 @@ import ( "context" "fmt" "maps" + "math" "math/rand" "strconv" "strings" @@ -45,6 +46,7 @@ import ( "sync/atomic" "time" + "github.com/tikv/client-go/v2/oracle" "go.uber.org/zap" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" @@ -107,6 +109,7 @@ type RegionRequestSender struct { regionCache *RegionCache apiVersion kvrpcpb.APIVersion client client.Client + readTSValidator oracle.ReadTSValidator storeAddr string rpcError error replicaSelector *replicaSelector @@ -377,11 +380,12 @@ func (s *ReplicaAccessStats) String() string { } // NewRegionRequestSender creates a new sender. -func NewRegionRequestSender(regionCache *RegionCache, client client.Client) *RegionRequestSender { +func NewRegionRequestSender(regionCache *RegionCache, client client.Client, readTSValidator oracle.ReadTSValidator) *RegionRequestSender { return &RegionRequestSender{ - regionCache: regionCache, - apiVersion: regionCache.codec.GetAPIVersion(), - client: client, + regionCache: regionCache, + apiVersion: regionCache.codec.GetAPIVersion(), + client: client, + readTSValidator: readTSValidator, } } @@ -766,6 +770,10 @@ func (s *RegionRequestSender) SendReqCtx( } } + if err = s.validateReadTS(bo.GetCtx(), req); err != nil { + return nil, nil, 0, err + } + // If the MaxExecutionDurationMs is not set yet, we set it to be the RPC timeout duration // so TiKV can give up the requests whose response TiDB cannot receive due to timeout. if req.Context.MaxExecutionDurationMs == 0 { @@ -1756,6 +1764,20 @@ func (s *RegionRequestSender) onRegionError( return false, nil } +func (s *RegionRequestSender) validateReadTS(ctx context.Context, req *tikvrpc.Request) error { + var readTS uint64 + switch req.Type { + case tikvrpc.CmdGet, tikvrpc.CmdScan, tikvrpc.CmdBatchGet, tikvrpc.CmdCop, tikvrpc.CmdCopStream, tikvrpc.CmdBatchCop, tikvrpc.CmdScanLock, tikvrpc.CmdBufferBatchGet: + readTS = req.GetStartTS() + default: + return nil + } + if readTS == math.MaxUint64 { + return nil + } + return s.readTSValidator.ValidateReadTS(ctx, readTS, req.StaleRead, &oracle.Option{TxnScope: req.TxnScope}) +} + type staleReadMetricsCollector struct { } diff --git a/internal/locate/region_request3_test.go b/internal/locate/region_request3_test.go index aa051385d5..eec6e1eaa3 100644 --- a/internal/locate/region_request3_test.go +++ b/internal/locate/region_request3_test.go @@ -88,7 +88,7 @@ func (s *testRegionRequestToThreeStoresSuite) SetupTest() { s.cache = NewRegionCache(pdCli) s.bo = retry.NewNoopBackoff(context.Background()) client := mocktikv.NewRPCClient(s.cluster, s.mvccStore, nil) - s.regionRequestSender = NewRegionRequestSender(s.cache, client) + s.regionRequestSender = NewRegionRequestSender(s.cache, client, oracle.NoopReadTSValidator{}) s.NoError(failpoint.Enable("tikvclient/doNotRecoverStoreHealthCheckPanic", "return")) } @@ -185,7 +185,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestSwitchPeerWhenNoLeaderErrorWit s.Nil(err) s.NotNil(location) bo := retry.NewBackoffer(context.Background(), 1000) - resp, _, _, err := NewRegionRequestSender(s.cache, cli).SendReqCtx(bo, req, location.Region, time.Second, tikvrpc.TiKV) + resp, _, _, err := NewRegionRequestSender(s.cache, cli, oracle.NoopReadTSValidator{}).SendReqCtx(bo, req, location.Region, time.Second, tikvrpc.TiKV) s.Nil(err) s.NotNil(resp) regionErr, err := resp.GetRegionError() @@ -208,7 +208,7 @@ func (s *testRegionRequestToThreeStoresSuite) loadAndGetLeaderStore() (*Store, s } func (s *testRegionRequestToThreeStoresSuite) TestForwarding() { - sender := NewRegionRequestSender(s.cache, s.regionRequestSender.client) + sender := NewRegionRequestSender(s.cache, s.regionRequestSender.client, oracle.NoopReadTSValidator{}) sender.regionCache.enableForwarding = true // First get the leader's addr from region cache @@ -1144,7 +1144,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestSendReqFirstTimeout() { } resetStats := func() { reqTargetAddrs = make(map[string]struct{}) - s.regionRequestSender = NewRegionRequestSender(s.cache, mockRPCClient) + s.regionRequestSender = NewRegionRequestSender(s.cache, mockRPCClient, oracle.NoopReadTSValidator{}) s.regionRequestSender.Stats = NewRegionRequestRuntimeStats() } @@ -1375,7 +1375,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestStaleReadTryFollowerAfterTimeo } return &tikvrpc.Response{Resp: &kvrpcpb.GetResponse{Value: []byte("value")}}, nil }} - s.regionRequestSender = NewRegionRequestSender(s.cache, mockRPCClient) + s.regionRequestSender = NewRegionRequestSender(s.cache, mockRPCClient, oracle.NoopReadTSValidator{}) s.regionRequestSender.Stats = NewRegionRequestRuntimeStats() getLocFn := func() *KeyLocation { loc, err := s.regionRequestSender.regionCache.LocateKey(bo, []byte("a")) diff --git a/internal/locate/region_request_state_test.go b/internal/locate/region_request_state_test.go index e903cb4b79..a5b7280e4b 100644 --- a/internal/locate/region_request_state_test.go +++ b/internal/locate/region_request_state_test.go @@ -34,6 +34,7 @@ import ( "github.com/tikv/client-go/v2/internal/mockstore/mocktikv" "github.com/tikv/client-go/v2/kv" "github.com/tikv/client-go/v2/metrics" + "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/tikvrpc" "github.com/tikv/client-go/v2/util" ) @@ -77,7 +78,7 @@ func (s *testRegionCacheStaleReadSuite) SetupTest() { s.cache = NewRegionCache(pdCli) s.bo = retry.NewNoopBackoff(context.Background()) client := mocktikv.NewRPCClient(s.cluster, s.mvccStore, nil) - s.regionRequestSender = NewRegionRequestSender(s.cache, client) + s.regionRequestSender = NewRegionRequestSender(s.cache, client, oracle.NoopReadTSValidator{}) s.setClient() s.injection = testRegionCacheFSMSuiteInjection{ unavailableStoreIDs: make(map[uint64]struct{}), diff --git a/internal/locate/region_request_test.go b/internal/locate/region_request_test.go index c43aecb087..0b5d221338 100644 --- a/internal/locate/region_request_test.go +++ b/internal/locate/region_request_test.go @@ -64,6 +64,7 @@ import ( "github.com/tikv/client-go/v2/internal/client" "github.com/tikv/client-go/v2/internal/client/mockserver" "github.com/tikv/client-go/v2/internal/mockstore/mocktikv" + "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/tikvrpc" pderr "github.com/tikv/pd/client/errs" "google.golang.org/grpc" @@ -93,7 +94,7 @@ func (s *testRegionRequestToSingleStoreSuite) SetupTest() { s.cache = NewRegionCache(pdCli) s.bo = retry.NewNoopBackoff(context.Background()) client := mocktikv.NewRPCClient(s.cluster, s.mvccStore, nil) - s.regionRequestSender = NewRegionRequestSender(s.cache, client) + s.regionRequestSender = NewRegionRequestSender(s.cache, client, oracle.NoopReadTSValidator{}) s.NoError(failpoint.Enable("tikvclient/doNotRecoverStoreHealthCheckPanic", "return")) } @@ -601,7 +602,7 @@ func (s *testRegionRequestToSingleStoreSuite) TestNoReloadRegionForGrpcWhenCtxCa }() cli := client.NewRPCClient() - sender := NewRegionRequestSender(s.cache, cli) + sender := NewRegionRequestSender(s.cache, cli, oracle.NoopReadTSValidator{}) req := tikvrpc.NewRequest(tikvrpc.CmdRawPut, &kvrpcpb.RawPutRequest{ Key: []byte("key"), Value: []byte("value"), @@ -622,7 +623,7 @@ func (s *testRegionRequestToSingleStoreSuite) TestNoReloadRegionForGrpcWhenCtxCa Client: client.NewRPCClient(), redirectAddr: addr, } - sender = NewRegionRequestSender(s.cache, client1) + sender = NewRegionRequestSender(s.cache, client1, oracle.NoopReadTSValidator{}) sender.SendReq(s.bo, req, region.Region, 3*time.Second) // cleanup @@ -806,7 +807,7 @@ func (s *testRegionRequestToSingleStoreSuite) TestBatchClientSendLoopPanic() { cancel() }() req := tikvrpc.NewRequest(tikvrpc.CmdCop, &coprocessor.Request{Data: []byte("a"), StartTs: 1}) - regionRequestSender := NewRegionRequestSender(s.cache, fnClient) + regionRequestSender := NewRegionRequestSender(s.cache, fnClient, oracle.NoopReadTSValidator{}) reachable.injectConstantLiveness(regionRequestSender.regionCache.stores) regionRequestSender.SendReq(bo, req, region.Region, client.ReadTimeoutShort) } @@ -850,19 +851,19 @@ type emptyClient struct { func (s *testRegionRequestToSingleStoreSuite) TestClientExt() { var cli client.Client = client.NewRPCClient() - sender := NewRegionRequestSender(s.cache, cli) + sender := NewRegionRequestSender(s.cache, cli, oracle.NoopReadTSValidator{}) s.NotNil(sender.client) s.NotNil(sender.getClientExt()) cli.Close() cli = &emptyClient{} - sender = NewRegionRequestSender(s.cache, cli) + sender = NewRegionRequestSender(s.cache, cli, oracle.NoopReadTSValidator{}) s.NotNil(sender.client) s.Nil(sender.getClientExt()) } func (s *testRegionRequestToSingleStoreSuite) TestRegionRequestSenderString() { - sender := NewRegionRequestSender(s.cache, &fnClient{}) + sender := NewRegionRequestSender(s.cache, &fnClient{}, oracle.NoopReadTSValidator{}) loc, err := s.cache.LocateRegionByID(s.bo, s.region) s.Nil(err) // invalid region cache before sending request. diff --git a/internal/locate/replica_selector_test.go b/internal/locate/replica_selector_test.go index d928b0a02d..3721729528 100644 --- a/internal/locate/replica_selector_test.go +++ b/internal/locate/replica_selector_test.go @@ -2793,7 +2793,7 @@ func (ca *replicaSelectorAccessPathCase) run(s *testReplicaSelectorSuite) *Regio Value: []byte("hello world"), }}, nil }} - sender := NewRegionRequestSender(s.cache, fnClient) + sender := NewRegionRequestSender(s.cache, fnClient, oracle.NoopReadTSValidator{}) req, opts, timeout := ca.buildRequest(s) beforeRun(s, ca) rc := s.getRegion() @@ -3220,7 +3220,7 @@ func TestTiKVClientReadTimeout(t *testing.T) { req.ReplicaReadType = kv.ReplicaReadLeader req.MaxExecutionDurationMs = 1 bo := retry.NewBackofferWithVars(context.Background(), 2000, nil) - sender := NewRegionRequestSender(s.cache, fnClient) + sender := NewRegionRequestSender(s.cache, fnClient, oracle.NoopReadTSValidator{}) resp, _, err := sender.SendReq(bo, req, rc.VerID(), time.Millisecond) s.Nil(err) s.NotNil(resp) @@ -3234,7 +3234,7 @@ func TestTiKVClientReadTimeout(t *testing.T) { }, accessPath) // clear max execution duration for retry. req.MaxExecutionDurationMs = 0 - sender = NewRegionRequestSender(s.cache, fnClient) + sender = NewRegionRequestSender(s.cache, fnClient, oracle.NoopReadTSValidator{}) resp, _, err = sender.SendReq(bo, req, rc.VerID(), time.Second) // use a longer timeout. s.Nil(err) s.NotNil(resp) @@ -3310,7 +3310,7 @@ func BenchmarkReplicaSelector(b *testing.B) { if err != nil { b.Fail() } - sender := NewRegionRequestSender(cache, fnClient) + sender := NewRegionRequestSender(cache, fnClient, oracle.NoopReadTSValidator{}) sender.SendReqCtx(bo, req, loc.Region, client.ReadTimeoutShort, tikvrpc.TiKV) } } diff --git a/oracle/oracle.go b/oracle/oracle.go index cc71b5db78..4367bdb883 100644 --- a/oracle/oracle.go +++ b/oracle/oracle.go @@ -68,6 +68,12 @@ type Oracle interface { // GetAllTSOKeyspaceGroupMinTS gets a minimum timestamp from all TSO keyspace groups. GetAllTSOKeyspaceGroupMinTS(ctx context.Context) (uint64, error) + ReadTSValidator +} + +// ReadTSValidator is the interface for providing the ability for verifying whether a timestamp is safe to be used +// for readings, as part of the `Oracle` interface. +type ReadTSValidator interface { // ValidateReadTS verifies whether it can be guaranteed that the given readTS doesn't exceed the maximum ts // that has been allocated by the oracle, so that it's safe to use this ts to perform read operations. // Note that this method only checks the ts from the oracle's perspective. It doesn't check whether the snapshot @@ -124,3 +130,12 @@ func GoTimeToTS(t time.Time) uint64 { func GoTimeToLowerLimitStartTS(now time.Time, maxTxnTimeUse int64) uint64 { return GoTimeToTS(now.Add(-time.Duration(maxTxnTimeUse) * time.Millisecond)) } + +// NoopReadTSValidator is a dummy implementation of ReadTSValidator that always let the validation pass. +// Only use this when using RPCs that are not related to ts (e.g. rawkv), or in tests where `Oracle` is not available +// and the validation is not necessary. +type NoopReadTSValidator struct{} + +func (NoopReadTSValidator) ValidateReadTS(ctx context.Context, readTS uint64, isStaleRead bool, opt *Option) error { + return nil +} diff --git a/rawkv/rawkv.go b/rawkv/rawkv.go index 00412987f4..87ed2f84cb 100644 --- a/rawkv/rawkv.go +++ b/rawkv/rawkv.go @@ -48,6 +48,7 @@ import ( "github.com/tikv/client-go/v2/internal/kvrpc" "github.com/tikv/client-go/v2/internal/locate" "github.com/tikv/client-go/v2/metrics" + "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/tikv" "github.com/tikv/client-go/v2/tikvrpc" pd "github.com/tikv/pd/client" @@ -687,7 +688,7 @@ func (c *Client) CompareAndSwap(ctx context.Context, key, previousValue, newValu func (c *Client) sendReq(ctx context.Context, key []byte, req *tikvrpc.Request, reverse bool) (*tikvrpc.Response, *locate.KeyLocation, error) { bo := retry.NewBackofferWithVars(ctx, rawkvMaxBackoff, nil) - sender := locate.NewRegionRequestSender(c.regionCache, c.rpcClient) + sender := locate.NewRegionRequestSender(c.regionCache, c.rpcClient, oracle.NoopReadTSValidator{}) for { var loc *locate.KeyLocation var err error @@ -784,7 +785,7 @@ func (c *Client) doBatchReq(bo *retry.Backoffer, batch kvrpc.Batch, options *raw }) } - sender := locate.NewRegionRequestSender(c.regionCache, c.rpcClient) + sender := locate.NewRegionRequestSender(c.regionCache, c.rpcClient, oracle.NoopReadTSValidator{}) req.MaxExecutionDurationMs = uint64(client.MaxWriteExecutionTime.Milliseconds()) resp, _, err := sender.SendReq(bo, req, batch.RegionID, client.ReadTimeoutShort) @@ -834,7 +835,7 @@ func (c *Client) doBatchReq(bo *retry.Backoffer, batch kvrpc.Batch, options *raw // TODO: Is there any better way to avoid duplicating code with func `sendReq` ? func (c *Client) sendDeleteRangeReq(ctx context.Context, startKey []byte, endKey []byte, opts *rawOptions) (*tikvrpc.Response, []byte, error) { bo := retry.NewBackofferWithVars(ctx, rawkvMaxBackoff, nil) - sender := locate.NewRegionRequestSender(c.regionCache, c.rpcClient) + sender := locate.NewRegionRequestSender(c.regionCache, c.rpcClient, oracle.NoopReadTSValidator{}) for { loc, err := c.regionCache.LocateKey(bo, startKey) if err != nil { @@ -936,7 +937,7 @@ func (c *Client) doBatchPut(bo *retry.Backoffer, batch kvrpc.Batch, opts *rawOpt Ttl: ttl, }) - sender := locate.NewRegionRequestSender(c.regionCache, c.rpcClient) + sender := locate.NewRegionRequestSender(c.regionCache, c.rpcClient, oracle.NoopReadTSValidator{}) req.MaxExecutionDurationMs = uint64(client.MaxWriteExecutionTime.Milliseconds()) req.ApiVersion = c.apiVersion resp, _, err := sender.SendReq(bo, req, batch.RegionID, client.ReadTimeoutShort) diff --git a/tikv/kv.go b/tikv/kv.go index db375ae57f..f9bc0488a7 100644 --- a/tikv/kv.go +++ b/tikv/kv.go @@ -546,7 +546,7 @@ func (s *KVStore) SupportDeleteRange() (supported bool) { func (s *KVStore) SendReq( bo *Backoffer, req *tikvrpc.Request, regionID locate.RegionVerID, timeout time.Duration, ) (*tikvrpc.Response, error) { - sender := locate.NewRegionRequestSender(s.regionCache, s.GetTiKVClient()) + sender := locate.NewRegionRequestSender(s.regionCache, s.GetTiKVClient(), s.oracle) resp, _, err := sender.SendReq(bo, req, regionID, timeout) return resp, err } diff --git a/tikv/region.go b/tikv/region.go index 188fce60bd..1bffa6e0f9 100644 --- a/tikv/region.go +++ b/tikv/region.go @@ -42,6 +42,7 @@ import ( "github.com/tikv/client-go/v2/internal/client" "github.com/tikv/client-go/v2/internal/locate" "github.com/tikv/client-go/v2/kv" + "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/tikvrpc" pd "github.com/tikv/pd/client" ) @@ -161,8 +162,8 @@ func GetStoreTypeByMeta(store *metapb.Store) tikvrpc.EndpointType { } // NewRegionRequestSender creates a new sender. -func NewRegionRequestSender(regionCache *RegionCache, client client.Client) *RegionRequestSender { - return locate.NewRegionRequestSender(regionCache, client) +func NewRegionRequestSender(regionCache *RegionCache, client client.Client, readTSValidator oracle.ReadTSValidator) *RegionRequestSender { + return locate.NewRegionRequestSender(regionCache, client, readTSValidator) } // LoadShuttingDown atomically loads ShuttingDown. diff --git a/tikv/split_region.go b/tikv/split_region.go index 5f9a7ecef2..e7aaba022f 100644 --- a/tikv/split_region.go +++ b/tikv/split_region.go @@ -148,7 +148,7 @@ func (s *KVStore) batchSendSingleRegion(bo *Backoffer, batch kvrpc.Batch, scatte RequestSource: util.RequestSourceFromCtx(bo.GetCtx()), }) - sender := locate.NewRegionRequestSender(s.regionCache, s.GetTiKVClient()) + sender := locate.NewRegionRequestSender(s.regionCache, s.GetTiKVClient(), s.oracle) resp, _, err := sender.SendReq(bo, req, batch.RegionID, client.ReadTimeoutShort) batchResp := kvrpc.BatchResult{Response: resp} diff --git a/txnkv/transaction/commit.go b/txnkv/transaction/commit.go index a3b4be8108..749b85a5b2 100644 --- a/txnkv/transaction/commit.go +++ b/txnkv/transaction/commit.go @@ -96,7 +96,7 @@ func (action actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *retry.Bac tBegin := time.Now() attempts := 0 - sender := locate.NewRegionRequestSender(c.store.GetRegionCache(), c.store.GetTiKVClient()) + sender := locate.NewRegionRequestSender(c.store.GetRegionCache(), c.store.GetTiKVClient(), c.store.GetOracle()) for { attempts++ reqBegin := time.Now() diff --git a/txnkv/transaction/pessimistic.go b/txnkv/transaction/pessimistic.go index 637044c288..d73345c91d 100644 --- a/txnkv/transaction/pessimistic.go +++ b/txnkv/transaction/pessimistic.go @@ -184,7 +184,7 @@ func (action actionPessimisticLock) handleSingleBatch( time.Sleep(300 * time.Millisecond) return errors.WithStack(&tikverr.ErrWriteConflict{WriteConflict: nil}) } - sender := locate.NewRegionRequestSender(c.store.GetRegionCache(), c.store.GetTiKVClient()) + sender := locate.NewRegionRequestSender(c.store.GetRegionCache(), c.store.GetTiKVClient(), c.store.GetOracle()) startTime := time.Now() resp, _, err := sender.SendReq(bo, req, batch.region, client.ReadTimeoutShort) diagCtx.reqDuration = time.Since(startTime) diff --git a/txnkv/transaction/pipelined_flush.go b/txnkv/transaction/pipelined_flush.go index ce26d4647c..c745e538a5 100644 --- a/txnkv/transaction/pipelined_flush.go +++ b/txnkv/transaction/pipelined_flush.go @@ -129,7 +129,7 @@ func (action actionPipelinedFlush) handleSingleBatch( attempts := 0 req := c.buildPipelinedFlushRequest(batch, action.generation) - sender := locate.NewRegionRequestSender(c.store.GetRegionCache(), c.store.GetTiKVClient()) + sender := locate.NewRegionRequestSender(c.store.GetRegionCache(), c.store.GetTiKVClient(), c.store.GetOracle()) var resolvingRecordToken *int for { diff --git a/txnkv/transaction/prewrite.go b/txnkv/transaction/prewrite.go index 26e317e106..a6eeafb3e5 100644 --- a/txnkv/transaction/prewrite.go +++ b/txnkv/transaction/prewrite.go @@ -268,7 +268,7 @@ func (action actionPrewrite) handleSingleBatch( attempts := 0 req := c.buildPrewriteRequest(batch, txnSize) - sender := locate.NewRegionRequestSender(c.store.GetRegionCache(), c.store.GetTiKVClient()) + sender := locate.NewRegionRequestSender(c.store.GetRegionCache(), c.store.GetTiKVClient(), c.store.GetOracle()) var resolvingRecordToken *int defer func() { if err != nil { diff --git a/txnkv/txnsnapshot/client_helper.go b/txnkv/txnsnapshot/client_helper.go index 6944c75536..c3008d1eb5 100644 --- a/txnkv/txnsnapshot/client_helper.go +++ b/txnkv/txnsnapshot/client_helper.go @@ -40,6 +40,7 @@ import ( "github.com/tikv/client-go/v2/config/retry" "github.com/tikv/client-go/v2/internal/client" "github.com/tikv/client-go/v2/internal/locate" + "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/tikvrpc" "github.com/tikv/client-go/v2/txnkv/txnlock" "github.com/tikv/client-go/v2/util" @@ -62,6 +63,7 @@ type ClientHelper struct { committedLocks *util.TSSet client client.Client resolveLite bool + oracle oracle.Oracle Stats *locate.RegionRequestRuntimeStats } @@ -74,6 +76,7 @@ func NewClientHelper(store kvstore, resolvedLocks *util.TSSet, committedLocks *u committedLocks: committedLocks, client: store.GetTiKVClient(), resolveLite: resolveLite, + oracle: store.GetOracle(), } } @@ -136,7 +139,7 @@ func (ch *ClientHelper) ResolveLocksDone(callerStartTS uint64, token int) { // SendReqCtx wraps the SendReqCtx function and use the resolved lock result in the kvrpcpb.Context. func (ch *ClientHelper) SendReqCtx(bo *retry.Backoffer, req *tikvrpc.Request, regionID locate.RegionVerID, timeout time.Duration, et tikvrpc.EndpointType, directStoreAddr string, opts ...locate.StoreSelectorOption) (*tikvrpc.Response, *locate.RPCContext, string, error) { - sender := locate.NewRegionRequestSender(ch.regionCache, ch.client) + sender := locate.NewRegionRequestSender(ch.regionCache, ch.client, ch.oracle) if len(directStoreAddr) > 0 { sender.SetStoreAddr(directStoreAddr) } diff --git a/txnkv/txnsnapshot/scan.go b/txnkv/txnsnapshot/scan.go index c3fa8b021d..c099cc26b3 100644 --- a/txnkv/txnsnapshot/scan.go +++ b/txnkv/txnsnapshot/scan.go @@ -197,7 +197,7 @@ func (s *Scanner) getData(bo *retry.Backoffer) error { zap.String("nextEndKey", kv.StrKey(s.nextEndKey)), zap.Bool("reverse", s.reverse), zap.Uint64("txnStartTS", s.startTS())) - sender := locate.NewRegionRequestSender(s.snapshot.store.GetRegionCache(), s.snapshot.store.GetTiKVClient()) + sender := locate.NewRegionRequestSender(s.snapshot.store.GetRegionCache(), s.snapshot.store.GetTiKVClient(), s.snapshot.store.GetOracle()) var reqEndKey, reqStartKey []byte var loc *locate.KeyLocation var resolvingRecordToken *int From 5c02543bd1a9c7290f908c785295b0321defcca1 Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Wed, 4 Dec 2024 15:39:08 +0800 Subject: [PATCH 03/11] Cover write requests that contains implicit read; address comments Signed-off-by: MyonKeminta --- internal/locate/region_request.go | 31 ++++++++++++++++++++++++++++--- oracle/oracles/pd.go | 8 ++++++++ 2 files changed, 36 insertions(+), 3 deletions(-) diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index 5dee15a7ff..e0b0bd0d59 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -771,6 +771,7 @@ func (s *RegionRequestSender) SendReqCtx( } if err = s.validateReadTS(bo.GetCtx(), req); err != nil { + logutil.Logger(bo.GetCtx()).Error("validate read ts failed for request", zap.Stringer("reqType", req.Type), zap.Stringer("req", req.Req.(fmt.Stringer)), zap.Stringer("context", &req.Context), zap.Stack("stack"), zap.Error(err)) return nil, nil, 0, err } @@ -1765,16 +1766,40 @@ func (s *RegionRequestSender) onRegionError( } func (s *RegionRequestSender) validateReadTS(ctx context.Context, req *tikvrpc.Request) error { + if req.StoreTp != tikvrpc.TiKV { + // Only check on TiKV. + return nil + } + var readTS uint64 switch req.Type { case tikvrpc.CmdGet, tikvrpc.CmdScan, tikvrpc.CmdBatchGet, tikvrpc.CmdCop, tikvrpc.CmdCopStream, tikvrpc.CmdBatchCop, tikvrpc.CmdScanLock, tikvrpc.CmdBufferBatchGet: readTS = req.GetStartTS() + + // Check transactional write requests that has implicit read. + case tikvrpc.CmdPessimisticLock: + readTS = req.PessimisticLock().GetForUpdateTs() + case tikvrpc.CmdPrewrite: + inner := req.Prewrite() + readTS = inner.GetForUpdateTs() + if readTS == 0 { + readTS = inner.GetStartVersion() + } + case tikvrpc.CmdCheckTxnStatus: + inner := req.CheckTxnStatus() + // TiKV uses the greater one of these three fields to update the max_ts. + readTS = inner.GetLockTs() + if inner.GetCurrentTs() != math.MaxUint64 && inner.GetCurrentTs() > readTS { + readTS = inner.GetCurrentTs() + } + if inner.GetCallerStartTs() != math.MaxUint64 && inner.GetCallerStartTs() > readTS { + readTS = inner.GetCallerStartTs() + } + case tikvrpc.CmdCheckSecondaryLocks, tikvrpc.CmdCleanup, tikvrpc.CmdBatchRollback: + readTS = req.GetStartTS() default: return nil } - if readTS == math.MaxUint64 { - return nil - } return s.readTSValidator.ValidateReadTS(ctx, readTS, req.StaleRead, &oracle.Option{TxnScope: req.TxnScope}) } diff --git a/oracle/oracles/pd.go b/oracle/oracles/pd.go index 193d6fb045..8387f12c2e 100644 --- a/oracle/oracles/pd.go +++ b/oracle/oracles/pd.go @@ -37,6 +37,7 @@ package oracles import ( "context" "fmt" + "math" "strings" "sync" "sync/atomic" @@ -670,6 +671,13 @@ func (o *pdOracle) getCurrentTSForValidation(ctx context.Context, opt *oracle.Op } func (o *pdOracle) ValidateReadTS(ctx context.Context, readTS uint64, isStaleRead bool, opt *oracle.Option) error { + if readTS == math.MaxUint64 { + if isStaleRead { + return errors.Errorf("cannot set read ts to max uint64 for stale read") + } + return nil + } + latestTSInfo, exists := o.getLastTSWithArrivalTS(opt.TxnScope) // If we fail to get latestTSInfo or the readTS exceeds it, get a timestamp from PD to double-check. // But we don't need to strictly fetch the latest TS. So if there are already concurrent calls to this function From e20400625f618d2039f4a1142417c562580d724a Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Wed, 4 Dec 2024 15:56:40 +0800 Subject: [PATCH 04/11] Temporarily remove check for write path Signed-off-by: MyonKeminta --- internal/locate/region_request.go | 43 +++++++++++++++---------------- 1 file changed, 21 insertions(+), 22 deletions(-) diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index e0b0bd0d59..9f62f5701d 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -38,7 +38,6 @@ import ( "context" "fmt" "maps" - "math" "math/rand" "strconv" "strings" @@ -1776,27 +1775,27 @@ func (s *RegionRequestSender) validateReadTS(ctx context.Context, req *tikvrpc.R case tikvrpc.CmdGet, tikvrpc.CmdScan, tikvrpc.CmdBatchGet, tikvrpc.CmdCop, tikvrpc.CmdCopStream, tikvrpc.CmdBatchCop, tikvrpc.CmdScanLock, tikvrpc.CmdBufferBatchGet: readTS = req.GetStartTS() - // Check transactional write requests that has implicit read. - case tikvrpc.CmdPessimisticLock: - readTS = req.PessimisticLock().GetForUpdateTs() - case tikvrpc.CmdPrewrite: - inner := req.Prewrite() - readTS = inner.GetForUpdateTs() - if readTS == 0 { - readTS = inner.GetStartVersion() - } - case tikvrpc.CmdCheckTxnStatus: - inner := req.CheckTxnStatus() - // TiKV uses the greater one of these three fields to update the max_ts. - readTS = inner.GetLockTs() - if inner.GetCurrentTs() != math.MaxUint64 && inner.GetCurrentTs() > readTS { - readTS = inner.GetCurrentTs() - } - if inner.GetCallerStartTs() != math.MaxUint64 && inner.GetCallerStartTs() > readTS { - readTS = inner.GetCallerStartTs() - } - case tikvrpc.CmdCheckSecondaryLocks, tikvrpc.CmdCleanup, tikvrpc.CmdBatchRollback: - readTS = req.GetStartTS() + // TODO: Check transactional write requests that has implicit read. + // case tikvrpc.CmdPessimisticLock: + // readTS = req.PessimisticLock().GetForUpdateTs() + // case tikvrpc.CmdPrewrite: + // inner := req.Prewrite() + // readTS = inner.GetForUpdateTs() + // if readTS == 0 { + // readTS = inner.GetStartVersion() + // } + // case tikvrpc.CmdCheckTxnStatus: + // inner := req.CheckTxnStatus() + // // TiKV uses the greater one of these three fields to update the max_ts. + // readTS = inner.GetLockTs() + // if inner.GetCurrentTs() != math.MaxUint64 && inner.GetCurrentTs() > readTS { + // readTS = inner.GetCurrentTs() + // } + // if inner.GetCallerStartTs() != math.MaxUint64 && inner.GetCallerStartTs() > readTS { + // readTS = inner.GetCallerStartTs() + // } + // case tikvrpc.CmdCheckSecondaryLocks, tikvrpc.CmdCleanup, tikvrpc.CmdBatchRollback: + // readTS = req.GetStartTS() default: return nil } From 75cf723ae14498d2bf373355916e74a061996817 Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Wed, 4 Dec 2024 17:15:41 +0800 Subject: [PATCH 05/11] Also verify ts for tiflash Signed-off-by: MyonKeminta --- internal/locate/region_request.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index 9f62f5701d..d5450b435f 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -1765,8 +1765,8 @@ func (s *RegionRequestSender) onRegionError( } func (s *RegionRequestSender) validateReadTS(ctx context.Context, req *tikvrpc.Request) error { - if req.StoreTp != tikvrpc.TiKV { - // Only check on TiKV. + if req.StoreTp == tikvrpc.TiDB { + // Skip the checking if the store type is TiDB. return nil } From 492cbea7946a549d1eb2b51b6a7d07c1fd724dd7 Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Wed, 4 Dec 2024 17:15:49 +0800 Subject: [PATCH 06/11] Add some tests Signed-off-by: MyonKeminta --- oracle/oracles/pd.go | 2 +- oracle/oracles/pd_test.go | 119 ++++++++++++++++++++++++++++---------- 2 files changed, 88 insertions(+), 33 deletions(-) diff --git a/oracle/oracles/pd.go b/oracle/oracles/pd.go index 8387f12c2e..b1a66788db 100644 --- a/oracle/oracles/pd.go +++ b/oracle/oracles/pd.go @@ -670,7 +670,7 @@ func (o *pdOracle) getCurrentTSForValidation(ctx context.Context, opt *oracle.Op } } -func (o *pdOracle) ValidateReadTS(ctx context.Context, readTS uint64, isStaleRead bool, opt *oracle.Option) error { +func (o *pdOracle) ValidateReadTS(ctx context.Context, readTS uint64, isStaleRead bool, opt *oracle.Option) (errRet error) { if readTS == math.MaxUint64 { if isStaleRead { return errors.Errorf("cannot set read ts to max uint64 for stale read") diff --git a/oracle/oracles/pd_test.go b/oracle/oracles/pd_test.go index 1412795340..5110cc266c 100644 --- a/oracle/oracles/pd_test.go +++ b/oracle/oracles/pd_test.go @@ -342,40 +342,54 @@ func TestAdaptiveUpdateTSInterval(t *testing.T) { assert.Equal(t, adaptiveUpdateTSIntervalStateUnadjustable, o.adaptiveUpdateIntervalState.state) } -func TestValidateReadTSForStaleRead(t *testing.T) { - pdClient := MockPdClient{} - o, err := NewPdOracle(&pdClient, &PDOracleOptions{ - UpdateInterval: time.Second * 2, - }) - assert.NoError(t, err) - defer o.Close() - - ctx := context.Background() - opt := &oracle.Option{TxnScope: oracle.GlobalTxnScope} - ts, err := o.GetTimestamp(ctx, opt) - assert.NoError(t, err) - assert.GreaterOrEqual(t, ts, uint64(1)) +func TestValidateReadTS(t *testing.T) { + testImpl := func(staleRead bool) { + pdClient := MockPdClient{} + o, err := NewPdOracle(&pdClient, &PDOracleOptions{ + UpdateInterval: time.Second * 2, + }) + assert.NoError(t, err) + defer o.Close() + + ctx := context.Background() + opt := &oracle.Option{TxnScope: oracle.GlobalTxnScope} + + // Always returns error for MaxUint64 + err = o.ValidateReadTS(ctx, math.MaxUint64, staleRead, opt) + if staleRead { + assert.Error(t, err) + } else { + assert.NoError(t, err) + } - err = o.ValidateReadTS(ctx, 1, true, opt) - assert.NoError(t, err) - ts, err = o.GetTimestamp(ctx, opt) - assert.NoError(t, err) - // The readTS exceeds the latest ts, so it first fails the check with the low resolution ts. Then it fallbacks to - // the fetching-from-PD path, and it can get the previous ts + 1, which can allow this validation to pass. - err = o.ValidateReadTS(ctx, ts+1, true, opt) - assert.NoError(t, err) - // It can't pass if the readTS is newer than previous ts + 2. - ts, err = o.GetTimestamp(ctx, opt) - assert.NoError(t, err) - err = o.ValidateReadTS(ctx, ts+2, true, opt) - assert.Error(t, err) + ts, err := o.GetTimestamp(ctx, opt) + assert.NoError(t, err) + assert.GreaterOrEqual(t, ts, uint64(1)) + + err = o.ValidateReadTS(ctx, 1, staleRead, opt) + assert.NoError(t, err) + ts, err = o.GetTimestamp(ctx, opt) + assert.NoError(t, err) + // The readTS exceeds the latest ts, so it first fails the check with the low resolution ts. Then it fallbacks to + // the fetching-from-PD path, and it can get the previous ts + 1, which can allow this validation to pass. + err = o.ValidateReadTS(ctx, ts+1, staleRead, opt) + assert.NoError(t, err) + // It can't pass if the readTS is newer than previous ts + 2. + ts, err = o.GetTimestamp(ctx, opt) + assert.NoError(t, err) + err = o.ValidateReadTS(ctx, ts+2, staleRead, opt) + assert.Error(t, err) + + // Simulate other PD clients requests a timestamp. + ts, err = o.GetTimestamp(ctx, opt) + assert.NoError(t, err) + pdClient.logicalTimestamp.Add(2) + err = o.ValidateReadTS(ctx, ts+3, staleRead, opt) + assert.NoError(t, err) + } - // Simulate other PD clients requests a timestamp. - ts, err = o.GetTimestamp(ctx, opt) - assert.NoError(t, err) - pdClient.logicalTimestamp.Add(2) - err = o.ValidateReadTS(ctx, ts+3, true, opt) - assert.NoError(t, err) + testImpl(true) + testImpl(false) } type MockPDClientWithPause struct { @@ -496,3 +510,44 @@ func TestValidateReadTSForStaleReadReusingGetTSResult(t *testing.T) { } } } + +func TestValidateReadTSForNormalReadDoNotAffectUpdateInterval(t *testing.T) { + oracleInterface, err := NewPdOracle(&MockPdClient{}, &PDOracleOptions{ + UpdateInterval: time.Second * 2, + NoUpdateTS: true, + }) + assert.NoError(t, err) + o := oracleInterface.(*pdOracle) + defer o.Close() + + ctx := context.Background() + opt := &oracle.Option{TxnScope: oracle.GlobalTxnScope} + + // Validating read ts for non-stale-read requests must not trigger updating the adaptive update interval of + // low resolution ts. + mustNoNotify := func() { + select { + case <-o.adaptiveUpdateIntervalState.shrinkIntervalCh: + assert.Fail(t, "expects not notifying shrinking update interval immediately, but message was received") + default: + } + } + + ts, err := o.GetTimestamp(ctx, opt) + assert.NoError(t, err) + assert.GreaterOrEqual(t, ts, uint64(1)) + + err = o.ValidateReadTS(ctx, ts, false, opt) + assert.NoError(t, err) + mustNoNotify() + + // It loads `ts + 1` from the mock PD, and the check cannot pass. + err = o.ValidateReadTS(ctx, ts+2, false, opt) + assert.Error(t, err) + mustNoNotify() + + // Do the check again. It loads `ts + 2` from the mock PD, and the check passes. + err = o.ValidateReadTS(ctx, ts+2, false, opt) + assert.NoError(t, err) + mustNoNotify() +} From 180659921302e3ff663b6cc5fe5c7e012a1f159d Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Wed, 4 Dec 2024 17:56:49 +0800 Subject: [PATCH 07/11] Add tests Signed-off-by: MyonKeminta --- internal/locate/region_request_test.go | 56 +++++++++++++++++++++++++- oracle/oracle.go | 16 ++++++++ oracle/oracles/pd.go | 7 +++- 3 files changed, 75 insertions(+), 4 deletions(-) diff --git a/internal/locate/region_request_test.go b/internal/locate/region_request_test.go index 0b5d221338..ca03c660bf 100644 --- a/internal/locate/region_request_test.go +++ b/internal/locate/region_request_test.go @@ -65,7 +65,9 @@ import ( "github.com/tikv/client-go/v2/internal/client/mockserver" "github.com/tikv/client-go/v2/internal/mockstore/mocktikv" "github.com/tikv/client-go/v2/oracle" + "github.com/tikv/client-go/v2/oracle/oracles" "github.com/tikv/client-go/v2/tikvrpc" + pd "github.com/tikv/pd/client" pderr "github.com/tikv/pd/client/errs" "google.golang.org/grpc" ) @@ -80,6 +82,7 @@ type testRegionRequestToSingleStoreSuite struct { store uint64 peer uint64 region uint64 + pdCli pd.Client cache *RegionCache bo *retry.Backoffer regionRequestSender *RegionRequestSender @@ -90,8 +93,8 @@ func (s *testRegionRequestToSingleStoreSuite) SetupTest() { s.mvccStore = mocktikv.MustNewMVCCStore() s.cluster = mocktikv.NewCluster(s.mvccStore) s.store, s.peer, s.region = mocktikv.BootstrapWithSingleStore(s.cluster) - pdCli := &CodecPDClient{mocktikv.NewPDClient(s.cluster), apicodec.NewCodecV1(apicodec.ModeTxn)} - s.cache = NewRegionCache(pdCli) + s.pdCli = &CodecPDClient{mocktikv.NewPDClient(s.cluster), apicodec.NewCodecV1(apicodec.ModeTxn)} + s.cache = NewRegionCache(s.pdCli) s.bo = retry.NewNoopBackoff(context.Background()) client := mocktikv.NewRPCClient(s.cluster, s.mvccStore, nil) s.regionRequestSender = NewRegionRequestSender(s.cache, client, oracle.NoopReadTSValidator{}) @@ -915,6 +918,55 @@ func (s *testRegionRequestToSingleStoreSuite) TestRegionRequestStats() { s.Contains(expecteds, access.String()) } +func (s *testRegionRequestToSingleStoreSuite) TestRegionRequestValidateReadTS() { + o, err := oracles.NewPdOracle(s.pdCli, &oracles.PDOracleOptions{ + UpdateInterval: time.Second * 2, + }) + s.NoError(err) + s.regionRequestSender.readTSValidator = o + defer o.Close() + + testImpl := func(ts func() uint64, staleRead bool, expectedErrorType error) { + region, err := s.cache.LocateRegionByID(s.bo, s.region) + s.Nil(err) + s.NotNil(region) + + req := tikvrpc.NewRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{ + Key: []byte("k"), + Version: ts(), + }) + + req.StaleRead = staleRead + _, _, _, err = s.regionRequestSender.SendReqCtx(s.bo, req, region.Region, time.Second, tikvrpc.TiKV) + + if expectedErrorType == nil { + s.NoError(err) + } else { + s.Error(err) + s.IsType(err, expectedErrorType) + } + } + + getTS := func() uint64 { + ts, err := o.GetTimestamp(s.bo.GetCtx(), &oracle.Option{TxnScope: oracle.GlobalTxnScope}) + s.NoError(err) + return ts + } + + addTS := func(ts uint64, diff time.Duration) uint64 { + return oracle.ComposeTS(oracle.GetPhysical(oracle.GetTimeFromTS(ts).Add(diff)), oracle.ExtractLogical(ts)) + } + + testImpl(getTS, false, nil) + testImpl(getTS, true, nil) + testImpl(func() uint64 { return addTS(getTS(), -time.Minute) }, false, nil) + testImpl(func() uint64 { return addTS(getTS(), -time.Minute) }, true, nil) + testImpl(func() uint64 { return addTS(getTS(), +time.Minute) }, false, oracle.ErrFutureTSRead{}) + testImpl(func() uint64 { return addTS(getTS(), +time.Minute) }, true, oracle.ErrFutureTSRead{}) + testImpl(func() uint64 { return math.MaxUint64 }, false, nil) + testImpl(func() uint64 { return math.MaxUint64 }, true, oracle.ErrLatestStaleRead{}) +} + type noCauseError struct { error } diff --git a/oracle/oracle.go b/oracle/oracle.go index 4367bdb883..cd3833371e 100644 --- a/oracle/oracle.go +++ b/oracle/oracle.go @@ -36,6 +36,7 @@ package oracle import ( "context" + "fmt" "time" ) @@ -139,3 +140,18 @@ type NoopReadTSValidator struct{} func (NoopReadTSValidator) ValidateReadTS(ctx context.Context, readTS uint64, isStaleRead bool, opt *Option) error { return nil } + +type ErrFutureTSRead struct { + ReadTS uint64 + CurrentTS uint64 +} + +func (e ErrFutureTSRead) Error() string { + return fmt.Sprintf("cannot set read timestamp to a future time, readTS: %d, currentTS: %d", e.ReadTS, e.CurrentTS) +} + +type ErrLatestStaleRead struct{} + +func (ErrLatestStaleRead) Error() string { + return "cannot set read ts to max uint64 for stale read" +} diff --git a/oracle/oracles/pd.go b/oracle/oracles/pd.go index b1a66788db..b5b0c5f65c 100644 --- a/oracle/oracles/pd.go +++ b/oracle/oracles/pd.go @@ -673,7 +673,7 @@ func (o *pdOracle) getCurrentTSForValidation(ctx context.Context, opt *oracle.Op func (o *pdOracle) ValidateReadTS(ctx context.Context, readTS uint64, isStaleRead bool, opt *oracle.Option) (errRet error) { if readTS == math.MaxUint64 { if isStaleRead { - return errors.Errorf("cannot set read ts to max uint64 for stale read") + return oracle.ErrLatestStaleRead{} } return nil } @@ -691,7 +691,10 @@ func (o *pdOracle) ValidateReadTS(ctx context.Context, readTS uint64, isStaleRea o.adjustUpdateLowResolutionTSIntervalWithRequestedStaleness(readTS, currentTS, time.Now()) } if readTS > currentTS { - return errors.Errorf("cannot set read timestamp to a future time") + return oracle.ErrFutureTSRead{ + ReadTS: readTS, + CurrentTS: currentTS, + } } } else if isStaleRead { estimatedCurrentTS, err := o.getStaleTimestampWithLastTS(latestTSInfo, 0) From 4933d6ffc5ac75f278fc6894cf99d9bb6f21aaf6 Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Fri, 6 Dec 2024 16:57:03 +0800 Subject: [PATCH 08/11] fix the check in mock oracle Signed-off-by: MyonKeminta --- oracle/oracles/local.go | 13 ++++++++++++- oracle/oracles/mock.go | 13 ++++++++++++- 2 files changed, 24 insertions(+), 2 deletions(-) diff --git a/oracle/oracles/local.go b/oracle/oracles/local.go index 909b35c8fc..a980bbddbd 100644 --- a/oracle/oracles/local.go +++ b/oracle/oracles/local.go @@ -36,6 +36,7 @@ package oracles import ( "context" + "math" "sync" "time" @@ -151,12 +152,22 @@ func (l *localOracle) GetExternalTimestamp(ctx context.Context) (uint64, error) } func (l *localOracle) ValidateReadTS(ctx context.Context, readTS uint64, isStaleRead bool, opt *oracle.Option) error { + if readTS == math.MaxUint64 { + if isStaleRead { + return oracle.ErrLatestStaleRead{} + } + return nil + } + currentTS, err := l.GetTimestamp(ctx, opt) if err != nil { return errors.Errorf("fail to validate read timestamp: %v", err) } if currentTS < readTS { - return errors.Errorf("cannot set read timestamp to a future time") + return oracle.ErrFutureTSRead{ + ReadTS: readTS, + CurrentTS: currentTS, + } } return nil } diff --git a/oracle/oracles/mock.go b/oracle/oracles/mock.go index 3129664aee..757ce3158d 100644 --- a/oracle/oracles/mock.go +++ b/oracle/oracles/mock.go @@ -36,6 +36,7 @@ package oracles import ( "context" + "math" "sync" "time" @@ -138,12 +139,22 @@ func (o *MockOracle) SetLowResolutionTimestampUpdateInterval(time.Duration) erro } func (o *MockOracle) ValidateReadTS(ctx context.Context, readTS uint64, isStaleRead bool, opt *oracle.Option) error { + if readTS == math.MaxUint64 { + if isStaleRead { + return oracle.ErrLatestStaleRead{} + } + return nil + } + currentTS, err := o.GetTimestamp(ctx, opt) if err != nil { return errors.Errorf("fail to validate read timestamp: %v", err) } if currentTS < readTS { - return errors.Errorf("cannot set read timestamp to a future time") + return oracle.ErrFutureTSRead{ + ReadTS: readTS, + CurrentTS: currentTS, + } } return nil } From f9dbd2517e59817e82749f5595c77c4ff32b9737 Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Mon, 9 Dec 2024 14:06:50 +0800 Subject: [PATCH 09/11] replace tidb in integration tests Signed-off-by: MyonKeminta --- integration_tests/go.mod | 60 ++++++++------- integration_tests/go.sum | 156 +++++++++++++++++++-------------------- 2 files changed, 107 insertions(+), 109 deletions(-) diff --git a/integration_tests/go.mod b/integration_tests/go.mod index 48d623f9fa..aceec0182e 100644 --- a/integration_tests/go.mod +++ b/integration_tests/go.mod @@ -1,32 +1,30 @@ module integration_tests -go 1.23 +go 1.23.4 require ( github.com/ninedraft/israce v0.0.3 github.com/pingcap/errors v0.11.5-0.20240318064555-6bd07397691f - github.com/pingcap/failpoint v0.0.0-20240527053858-9b3b6e34194a - github.com/pingcap/kvproto v0.0.0-20241113043844-e1fa7ea8c302 + github.com/pingcap/failpoint v0.0.0-20240528011301-b51a646c7c86 + github.com/pingcap/kvproto v0.0.0-20241120022153-92b0414aeed8 github.com/pingcap/tidb v1.1.0-beta.0.20240703042657-230bbc2ef5ef github.com/pkg/errors v0.9.1 - github.com/stretchr/testify v1.9.0 + github.com/stretchr/testify v1.10.0 github.com/tidwall/gjson v1.14.1 - github.com/tikv/client-go/v2 v2.0.8-0.20240626064248-4a72526f6c30 + github.com/tikv/client-go/v2 v2.0.8-0.20241120024459-05d115b3e88b github.com/tikv/pd/client v0.0.0-20241111073742-238d4d79ea31 go.uber.org/goleak v1.3.0 ) require ( github.com/Azure/go-ntlmssp v0.0.0-20221128193559-754e69321358 // indirect - github.com/BurntSushi/toml v1.4.0 // indirect + github.com/BurntSushi/toml v1.4.1-0.20240526193622-a339e1f7089c // indirect github.com/HdrHistogram/hdrhistogram-go v1.1.2 // indirect github.com/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2 // indirect github.com/beorn7/perks v1.0.1 // indirect + github.com/bits-and-blooms/bitset v1.14.3 // indirect github.com/cespare/xxhash/v2 v2.3.0 // indirect github.com/cloudfoundry/gosigar v1.3.6 // indirect - github.com/cockroachdb/errors v1.11.1 // indirect - github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b // indirect - github.com/cockroachdb/redact v1.1.5 // indirect github.com/coocood/bbloom v0.0.0-20190830030839-58deb6228d64 // indirect github.com/coocood/freecache v1.2.1 // indirect github.com/coocood/rtutil v0.0.0-20190304133409-c84515f646f2 // indirect @@ -39,7 +37,6 @@ require ( github.com/docker/go-units v0.5.0 // indirect github.com/dolthub/maphash v0.1.0 // indirect github.com/dolthub/swiss v0.2.1 // indirect - github.com/getsentry/sentry-go v0.27.0 // indirect github.com/go-asn1-ber/asn1-ber v1.5.4 // indirect github.com/go-ldap/ldap/v3 v3.4.4 // indirect github.com/go-ole/go-ole v1.3.0 // indirect @@ -48,40 +45,39 @@ require ( github.com/golang/snappy v0.0.4 // indirect github.com/google/btree v1.1.2 // indirect github.com/google/uuid v1.6.0 // indirect - github.com/gorilla/mux v1.8.0 // indirect + github.com/gorilla/mux v1.8.1 // 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.7 // indirect + github.com/json-iterator/go v1.1.12 // indirect + github.com/klauspost/compress v1.17.9 // indirect github.com/klauspost/cpuid v1.3.1 // indirect - github.com/kr/pretty v0.3.1 // indirect - github.com/kr/text v0.2.0 // indirect github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a // indirect + github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect + github.com/modern-go/reflect2 v1.0.2 // indirect github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect github.com/ncw/directio v1.0.5 // indirect github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7 // indirect github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef // indirect github.com/opentracing/basictracer-go v1.1.0 // indirect github.com/opentracing/opentracing-go v1.2.0 // indirect - github.com/petermattis/goid v0.0.0-20231207134359-e60b3f734c67 // indirect - github.com/pingcap/badger v1.5.1-0.20230103063557-828f39b09b6d // indirect + github.com/petermattis/goid v0.0.0-20240813172612-4fcff4a6cae7 // indirect + github.com/pingcap/badger v1.5.1-0.20241015064302-38533b6cbf8d // indirect github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 // indirect github.com/pingcap/log v1.1.1-0.20240314023424-862ccc32f18d // indirect github.com/pingcap/sysutil v1.0.1-0.20241113070546-23b50de46fd3 // indirect github.com/pingcap/tidb/pkg/parser v0.0.0-20240703042657-230bbc2ef5ef // indirect - github.com/pingcap/tipb v0.0.0-20240318032315-55a7867ddd50 // indirect + github.com/pingcap/tipb v0.0.0-20241105053214-f91fdb81a69e // indirect github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b // indirect - github.com/prometheus/client_golang v1.19.1 // indirect + github.com/prometheus/client_golang v1.20.5 // indirect github.com/prometheus/client_model v0.6.1 // indirect - github.com/prometheus/common v0.55.0 // indirect + github.com/prometheus/common v0.57.0 // indirect github.com/prometheus/procfs v0.15.1 // indirect github.com/qri-io/jsonpointer v0.1.1 // indirect github.com/qri-io/jsonschema v0.2.1 // indirect github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec // indirect - github.com/rogpeppe/go-internal v1.12.0 // indirect - github.com/sasha-s/go-deadlock v0.3.1 // indirect - github.com/shirou/gopsutil/v3 v3.24.4 // indirect + github.com/sasha-s/go-deadlock v0.3.5 // indirect + github.com/shirou/gopsutil/v3 v3.24.5 // indirect github.com/shoenig/go-m1cpu v0.1.6 // indirect github.com/spf13/pflag v1.0.5 // indirect github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a // indirect @@ -99,14 +95,14 @@ require ( go.uber.org/atomic v1.11.0 // indirect go.uber.org/multierr v1.11.0 // indirect go.uber.org/zap v1.27.0 // indirect - golang.org/x/crypto v0.24.0 // indirect - golang.org/x/exp v0.0.0-20240404231335-c0f41cb1a7a0 // indirect - golang.org/x/net v0.26.0 // indirect - golang.org/x/sync v0.7.0 // indirect - golang.org/x/sys v0.21.0 // indirect - golang.org/x/text v0.16.0 // indirect - golang.org/x/time v0.5.0 // indirect - golang.org/x/tools v0.22.0 // indirect + golang.org/x/crypto v0.29.0 // indirect + golang.org/x/exp v0.0.0-20240909161429-701f63a606c0 // indirect + golang.org/x/net v0.31.0 // indirect + golang.org/x/sync v0.9.0 // indirect + golang.org/x/sys v0.27.0 // indirect + golang.org/x/text v0.20.0 // indirect + golang.org/x/time v0.7.0 // indirect + golang.org/x/tools v0.27.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240318140521-94a12d6c2237 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240515191416-fc5f0ca64291 // indirect google.golang.org/grpc v1.63.2 // indirect @@ -121,3 +117,5 @@ replace ( github.com/tikv/client-go/v2 => ../ ) + +replace github.com/pingcap/tidb => github.com/MyonKeminta/tidb v1.1.0-alpha.1.0.20241206091441-d8254eef1420 diff --git a/integration_tests/go.sum b/integration_tests/go.sum index 9e93f5575e..f5bb7732b0 100644 --- a/integration_tests/go.sum +++ b/integration_tests/go.sum @@ -462,6 +462,8 @@ cloud.google.com/go/kms v1.11.0/go.mod h1:hwdiYC0xjnWsKQQCQQmIQnS9asjYVSK6jtXm+z cloud.google.com/go/kms v1.12.1/go.mod h1:c9J991h5DTl+kg7gi3MYomh12YEENGrf48ee/N/2CDM= cloud.google.com/go/kms v1.15.0/go.mod h1:c9J991h5DTl+kg7gi3MYomh12YEENGrf48ee/N/2CDM= cloud.google.com/go/kms v1.15.2/go.mod h1:3hopT4+7ooWRCjc2DxgnpESFxhIraaI2IpAVUEhbT/w= +cloud.google.com/go/kms v1.15.7 h1:7caV9K3yIxvlQPAcaFffhlT7d1qpxjB1wHBtjWa13SM= +cloud.google.com/go/kms v1.15.7/go.mod h1:ub54lbsa6tDkUwnu4W7Yt1aAIFLnspgh0kPGToDukeI= cloud.google.com/go/language v1.4.0/go.mod h1:F9dRpNFQmJbkaop6g0JhSBXCNlO90e1KWx5iDdxbWic= cloud.google.com/go/language v1.6.0/go.mod h1:6dJ8t3B+lUYfStgls25GusK04NLh3eDLQnWM3mdEbhI= cloud.google.com/go/language v1.7.0/go.mod h1:DJ6dYN/W+SQOjF8e1hLQXMF21AkH2w9wiPzPCJa2MIE= @@ -831,8 +833,8 @@ github.com/Azure/go-ntlmssp v0.0.0-20221128193559-754e69321358/go.mod h1:chxPXzS github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2 h1:XHOnouVk1mxXfQidrMEnLlPk9UMeRtyBTnEFtxkV0kU= github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2/go.mod h1:wP83P5OoQ5p6ip3ScPr0BAq0BvuPAvacpEuSzyouqAI= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/BurntSushi/toml v1.4.0 h1:kuoIxZQy2WRRk1pttg9asf+WVv6tWQuBNVmK8+nqPr0= -github.com/BurntSushi/toml v1.4.0/go.mod h1:ukJfTF/6rtPPRCnwkur4qwRxa8vTRFBF0uk2lLoLwho= +github.com/BurntSushi/toml v1.4.1-0.20240526193622-a339e1f7089c h1:pxW6RcqyfI9/kWtOwnv/G+AzdKuy2ZrqINhenH4HyNs= +github.com/BurntSushi/toml v1.4.1-0.20240526193622-a339e1f7089c/go.mod h1:ukJfTF/6rtPPRCnwkur4qwRxa8vTRFBF0uk2lLoLwho= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= github.com/DataDog/zstd v1.5.5 h1:oWf5W7GtOLgp6bciQYDmhHHjdhYkALu6S/5Ni9ZgSvQ= github.com/DataDog/zstd v1.5.5/go.mod h1:g4AWEaM3yOg3HYfnJ3YIawPnVdXJh9QME85blwSAmyw= @@ -841,6 +843,8 @@ github.com/HdrHistogram/hdrhistogram-go v1.1.2/go.mod h1:yDgFjdqOqDEKOvasDdhWNXY github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c/go.mod h1:X0CRv0ky0k6m906ixxpzmDRLvX58TFUKS2eePweuyxk= 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.20241206091441-d8254eef1420 h1:6XEvCz7qEEU5bQxmvaqyvUbbRcoUKJy/kx7Lau7C6Y0= +github.com/MyonKeminta/tidb v1.1.0-alpha.1.0.20241206091441-d8254eef1420/go.mod h1:i0EIeFIL9EodlywrvfHoFIBjoV2MtxglWTTercdWw9I= github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= github.com/VividCortex/ewma v1.2.0 h1:f58SaIzcDXrSy3kWaHNvuJgJ3Nmz59Zji6XoJR/q1ow= github.com/VividCortex/ewma v1.2.0/go.mod h1:nz4BbCtbLyFDeC9SUHbtcT5644juEuWfUAUnGx7j5l4= @@ -870,12 +874,14 @@ github.com/apache/thrift v0.16.0/go.mod h1:PHK3hniurgQaNMZYaCLEqXKsYK8upmhPbmdP2 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= -github.com/aws/aws-sdk-go v1.50.0 h1:HBtrLeO+QyDKnc3t1+5DR1RxodOHCGr8ZcrHudpv7jI= -github.com/aws/aws-sdk-go v1.50.0/go.mod h1:LF8svs817+Nz+DmiMQKTO3ubZ/6IaTpq3TjupRn3Eqk= +github.com/aws/aws-sdk-go v1.55.5 h1:KKUZBfBoyqy5d3swXyiC7Q76ic40rYcbqH7qjh59kzU= +github.com/aws/aws-sdk-go v1.55.5/go.mod h1:eRwEWoyTWFMVYVQzKMNHWP5/RV4xIUGMQfXQHfHkpNU= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/benbjohnson/clock v1.3.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= +github.com/bits-and-blooms/bitset v1.14.3 h1:Gd2c8lSNf9pKXom5JtD7AaKO8o7fGQ2LtFj1436qilA= +github.com/bits-and-blooms/bitset v1.14.3/go.mod h1:7hO7Gc7Pp1vODcmWvKMRA9BNmbv6a/7QIWpPxHddWR8= github.com/blacktear23/go-proxyprotocol v1.0.6 h1:eTt6UMpEnq59NjON49b3Cay8Dm0sCs1nDliwgkyEsRM= github.com/blacktear23/go-proxyprotocol v1.0.6/go.mod h1:FSCbgnRZrQXazBLL5snfBbrcFSMtcmUDhSRb9OfFA1o= github.com/boombuler/barcode v1.0.0/go.mod h1:paBWMcWSl3LHKBqUq+rly7CNSldXjb2rDl3JlRe0mD8= @@ -956,7 +962,6 @@ github.com/decred/dcrd/dcrec/secp256k1/v4 v4.2.0/go.mod h1:v57UDF4pDQJcEfFUCRop3 github.com/dgraph-io/ristretto v0.1.1 h1:6CWw5tJNgpegArSHpNHJKldNeq03FQCwYvfMVWajOK8= github.com/dgraph-io/ristretto v0.1.1/go.mod h1:S1GPSBCYCIhmVNfcth17y2zZtQT6wzkzgwUve0VDWWA= github.com/dgryski/go-farm v0.0.0-20190104051053-3adb47b1fb0f/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= -github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= github.com/dgryski/go-farm v0.0.0-20240924180020-3414d57e47da h1:aIftn67I1fkbMa512G+w+Pxci9hJPB8oMnkcP3iZF38= github.com/dgryski/go-farm v0.0.0-20240924180020-3414d57e47da/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= github.com/docker/go-units v0.5.0 h1:69rxXcBk27SvSaaxTtLh/8llcHD8vYHT7WSdRZ/jvr4= @@ -991,8 +996,8 @@ github.com/envoyproxy/protoc-gen-validate v0.10.0/go.mod h1:DRjgyB0I43LtJapqN6Ni github.com/envoyproxy/protoc-gen-validate v0.10.1/go.mod h1:DRjgyB0I43LtJapqN6NiRwroiAU2PaFuvk/vjgh61ss= github.com/envoyproxy/protoc-gen-validate v1.0.1/go.mod h1:0vj8bNkYbSTNS2PIyH87KZaeN4x9zpL9Qt8fQC7d+vs= github.com/envoyproxy/protoc-gen-validate v1.0.2/go.mod h1:GpiZQP3dDbg4JouG/NNS7QWXpgx6x8QiMKdmN72jogE= -github.com/fatih/color v1.17.0 h1:GlRw1BRJxkpqUCBKzKOw098ed57fEsKeNjpTe3cSjK4= -github.com/fatih/color v1.17.0/go.mod h1:YZ7TlrGPkiz6ku9fK3TLD/pl3CpsiFyu8N92HLgmosI= +github.com/fatih/color v1.18.0 h1:S8gINlzdQ840/4pfAwic/ZE0djQEH3wM94VfqLTZcOM= +github.com/fatih/color v1.18.0/go.mod h1:4FelSpRwEGDpQ12mAdzqdOukCy4u8WUtOY6lkT/6HfU= github.com/felixge/httpsnoop v1.0.4 h1:NFTV2Zj1bL4mc9sqWACXbQFVBBg2W3GPvqp8/ESS2Wg= github.com/felixge/httpsnoop v1.0.4/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U= github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= @@ -1005,8 +1010,6 @@ github.com/getsentry/sentry-go v0.27.0/go.mod h1:lc76E2QywIyW8WuBnwl8Lc4bkmQH4+w github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/go-asn1-ber/asn1-ber v1.5.4 h1:vXT6d/FNDiELJnLb6hGNa309LMsrCoYFvpwHDF0+Y1A= github.com/go-asn1-ber/asn1-ber v1.5.4/go.mod h1:hEBeB/ic+5LoWskz+yKT7vGhhPYkProFKoKdwZRWMe0= -github.com/go-errors/errors v1.4.2 h1:J6MZopCL4uSllY1OfXM374weqZFFItUbrImctkmUxIA= -github.com/go-errors/errors v1.4.2/go.mod h1:sIVyrIiJhuEF+Pj9Ebtd6P/rEYROXFi3BopGUQ5a5Og= github.com/go-fonts/dejavu v0.1.0/go.mod h1:4Wt4I4OU2Nq9asgDCteaAaWZOV24E+0/Pwo0gppep4g= github.com/go-fonts/latin-modern v0.2.0/go.mod h1:rQVLdDMK+mK1xscDwsqM5J8U2jrRa3T0ecnM9pNujks= github.com/go-fonts/liberation v0.1.1/go.mod h1:K6qoJYypsmfVjWg8KOVDQhLc8UDgIK2HYqyqAO9z7GY= @@ -1033,19 +1036,20 @@ github.com/go-resty/resty/v2 v2.11.0/go.mod h1:iiP/OpA0CkcL3IGt1O0+/SIItFUbkkyw5 github.com/go-sql-driver/mysql v1.7.1 h1:lUIinVbN1DY0xBg0eMOzmmtGoHwWBbvnWubQUrtU8EI= github.com/go-sql-driver/mysql v1.7.1/go.mod h1:OXbVy3sEdcQ2Doequ6Z5BW6fXNQTmx+9S1MCJN5yJMI= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/gobwas/glob v0.2.3 h1:A4xDbljILXROh+kObIiy5kIaPYD8e96x1tgBhUI5J+Y= +github.com/gobwas/glob v0.2.3/go.mod h1:d3Ez4x06l9bZtSvzIay5+Yzi0fmZzPgnTbPcKjJAkT8= github.com/goccy/go-json v0.9.11/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= github.com/goccy/go-json v0.10.2 h1:CrxCmQqYDkv1z7lO7Wbh2HN93uovUHgrECaO5ZrCXAU= github.com/goccy/go-json v0.10.2/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= github.com/goccy/go-reflect v1.2.0 h1:O0T8rZCuNmGXewnATuKYnkL0xm6o8UNOJZd/gOkb9ms= github.com/goccy/go-reflect v1.2.0/go.mod h1:n0oYZn8VcV2CkWTxi8B9QjkCoq6GTtCEdfmR66YhFtE= github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= -github.com/gogo/protobuf v0.0.0-20171007142547-342cbe0a0415/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= -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.5.1 h1:JdqV9zKUdtaa9gdPlywC3aeoEsR681PlKC+4F5gQgeo= +github.com/golang-jwt/jwt/v4 v4.5.1/go.mod h1:m21LjoU+eqJr34lmDMbreY2eSTRJ1cv77w39/MY0Ch0= github.com/golang-jwt/jwt/v5 v5.2.1 h1:OuVbFODueb089Lh128TAcimifWaLhJwVflnrgM17wHk= github.com/golang-jwt/jwt/v5 v5.2.1/go.mod h1:pqrtFR0X4osieyHYxtmOUWsAWrfe1Q5UVIyoH402zdk= github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= @@ -1117,6 +1121,7 @@ github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeN github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-pkcs11 v0.2.0/go.mod h1:6eQoGcuNJpa7jnd5pMGdkSaQpNDYvPlXWMcjXXThLlY= +github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/gofuzz v1.2.0 h1:xRy4A+RhZaiKjJ1bPfwQ8sedCA+YS2YcCHW6ec7JMi0= github.com/google/gofuzz v1.2.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= @@ -1139,8 +1144,8 @@ github.com/google/pprof v0.0.0-20210226084205-cbba55b83ad5/go.mod h1:kpwsk12EmLe github.com/google/pprof v0.0.0-20210601050228-01bbb1931b22/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210609004039-a478d1d731e9/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210720184732-4bb14d4b1be1/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= -github.com/google/pprof v0.0.0-20240117000934-35fc243c5815 h1:WzfWbQz/Ze8v6l++GGbGNFZnUShVpP/0xffCPLL+ax8= -github.com/google/pprof v0.0.0-20240117000934-35fc243c5815/go.mod h1:czg5+yv1E0ZGTi6S6vVK1mke0fV+FaUhNGcd6VRS9Ik= +github.com/google/pprof v0.0.0-20241001023024-f4c0cfd0cf1d h1:Jaz2JzpQaQXyET0AjLBXShrthbpqMkhGiEfkcQAiAUs= +github.com/google/pprof v0.0.0-20241001023024-f4c0cfd0cf1d/go.mod h1:vavhavw2zAxS5dIdcRluK6cSGGPlZynqzFM8NdvU144= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/s2a-go v0.1.0/go.mod h1:OJpEgntRZo8ugHpF9hkoLJbS5dSI20XZeXJ9JVywLlM= github.com/google/s2a-go v0.1.3/go.mod h1:Ej+mSEMGRnqRzjc7VtF+jdBwYG5fuJfiZ8ELkjEwM0A= @@ -1180,8 +1185,8 @@ github.com/googleapis/gax-go/v2 v2.12.2 h1:mhN09QQW1jEWeMF74zGR81R30z4VJzjZsfkUh github.com/googleapis/gax-go/v2 v2.12.2/go.mod h1:61M8vcyyXR2kqKFxKrfA22jaA8JGF7Dc8App1U3H6jc= github.com/googleapis/go-type-adapters v1.0.0/go.mod h1:zHW75FOG2aur7gAO2B+MLby+cLsWGBF62rFAi7WjWO4= github.com/googleapis/google-cloud-go-testing v0.0.0-20200911160855-bcd43fbb19e8/go.mod h1:dvDLG8qkwmyD9a/MJJN3XJcT3xFxOKAvTZGvuZmac9g= -github.com/gorilla/mux v1.8.0 h1:i40aqfkR1h2SlN9hojwV5ZA91wcXFOvkdNIeFDP5koI= -github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= +github.com/gorilla/mux v1.8.1 h1:TuBL49tXwgrFYWhqrNgrUNEY92u81SPhu7sTdzQEiWY= +github.com/gorilla/mux v1.8.1/go.mod h1:AKf9I4AEqPTmMytcMc0KkNouC66V3BtZ4qD5fmWSiMQ= 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= @@ -1234,8 +1239,8 @@ github.com/klauspost/asmfmt v1.3.2 h1:4Ri7ox3EwapiOjCki+hw14RyKk201CN4rzyCJRFLpK github.com/klauspost/asmfmt v1.3.2/go.mod h1:AG8TuvYojzulgDAMCnYn50l/5QV3Bs/tp6j0HLHbNSE= github.com/klauspost/compress v1.9.5/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= -github.com/klauspost/compress v1.17.7 h1:ehO88t2UGzQK66LMdE8tibEd1ErmzZjNEqWkjLAKQQg= -github.com/klauspost/compress v1.17.7/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= +github.com/klauspost/compress v1.17.9 h1:6KIumPrER1LHsvBVuDa0r5xaG0Es51mhhB9BQB2qeMA= +github.com/klauspost/compress v1.17.9/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= github.com/klauspost/cpuid v1.2.1/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= github.com/klauspost/cpuid v1.3.1 h1:5JNjFYYQrZeKRJ0734q51WCEEn2huer72Dc7K+R/b6s= github.com/klauspost/cpuid v1.3.1/go.mod h1:bYW4mA6ZgKPob1/Dlai2LviZJO7KGI3uoWLd42rAQw4= @@ -1283,8 +1288,8 @@ github.com/mattn/go-isatty v0.0.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/ github.com/mattn/go-isatty v0.0.17/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= 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-runewidth v0.0.16 h1:E5ScNMtiwvlvB5paMFdw9p4kSQzbXFikJ5SQO6TULQc= +github.com/mattn/go-runewidth v0.0.16/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= github.com/mattn/go-sqlite3 v1.14.14/go.mod h1:NyWgC/yNuGj7Q9rpYnZvas74GogHl5/Z4A/KQRfk6bU= github.com/mattn/go-sqlite3 v1.14.15/go.mod h1:2eHXhiwb8IkHr+BDWZGa96P6+rkvnG63S2DGjv9HUNg= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= @@ -1294,6 +1299,7 @@ github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3 h1:+n/aFZefKZp7spd8D github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3/go.mod h1:RagcQ7I8IeTMnF8JTXieKnO4Z6JCsikNEzj0DwauVzE= github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= +github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= @@ -1324,42 +1330,39 @@ github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYr github.com/otiai10/copy v1.2.0 h1:HvG945u96iNadPoG2/Ja2+AUJeW5YuFQMixq9yirC+k= github.com/otiai10/copy v1.2.0/go.mod h1:rrF5dJ5F0t/EWSYODDu4j9/vEeYHMkc8jt0zJChqQWw= github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= -github.com/petermattis/goid v0.0.0-20180202154549-b0b1615b78e5/go.mod h1:jvVRKCrJTQWu0XVbaOlby/2lO20uSCHEMzzplHXte1o= -github.com/petermattis/goid v0.0.0-20231207134359-e60b3f734c67 h1:jik8PHtAIsPlCRJjJzl4udgEf7hawInF9texMeO2jrU= -github.com/petermattis/goid v0.0.0-20231207134359-e60b3f734c67/go.mod h1:pxMtw7cyUw6B2bRH0ZBANSPg+AoSud1I1iyJHI69jH4= +github.com/petermattis/goid v0.0.0-20240813172612-4fcff4a6cae7 h1:Dx7Ovyv/SFnMFw3fD4oEoeorXc6saIiQ23LrGLth0Gw= +github.com/petermattis/goid v0.0.0-20240813172612-4fcff4a6cae7/go.mod h1:pxMtw7cyUw6B2bRH0ZBANSPg+AoSud1I1iyJHI69jH4= github.com/phpdave11/gofpdf v1.4.2/go.mod h1:zpO6xFn9yxo3YLyMvW8HcKWVdbNqgIfOOp2dXMnm1mY= github.com/phpdave11/gofpdi v1.0.12/go.mod h1:vBmVV0Do6hSBHC8uKUQ71JGW+ZGQq74llk/7bXwjDoI= github.com/phpdave11/gofpdi v1.0.13/go.mod h1:vBmVV0Do6hSBHC8uKUQ71JGW+ZGQq74llk/7bXwjDoI= -github.com/pierrec/lz4 v2.6.1+incompatible h1:9UY3+iC23yxF0UfGaYrGplQ+79Rg+h/q9FV9ix19jjM= github.com/pierrec/lz4/v4 v4.1.15 h1:MO0/ucJhngq7299dKLwIMtgTfbkoSPF6AoMYDd8Q4q0= github.com/pierrec/lz4/v4 v4.1.15/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= -github.com/pingcap/badger v1.5.1-0.20230103063557-828f39b09b6d h1:AEcvKyVM8CUII3bYzgz8haFXtGiqcrtXW1csu/5UELY= -github.com/pingcap/badger v1.5.1-0.20230103063557-828f39b09b6d/go.mod h1:p8QnkZnmyV8L/M/jzYb8rT7kv3bz9m7bn1Ju94wDifs= +github.com/pingcap/badger v1.5.1-0.20241015064302-38533b6cbf8d h1:eHcokyHxm7HVM+7+Qy1zZwC7NhX9wVNX8oQDcSZw1qI= +github.com/pingcap/badger v1.5.1-0.20241015064302-38533b6cbf8d/go.mod h1:KiO2zumBCWx7yoVYoFRpb+DNrwEPk1pR1LF7NvOACMQ= github.com/pingcap/errors v0.11.0/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.4/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.5-0.20190809092503-95897b64e011/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.5-0.20240318064555-6bd07397691f h1:FxA+NgsdHNOv+/hZGxUh8Gb3WuZqgqmxDwztEOiA1v4= github.com/pingcap/errors v0.11.5-0.20240318064555-6bd07397691f/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= -github.com/pingcap/failpoint v0.0.0-20240527053858-9b3b6e34194a h1:UgrcL8INjEbPRKE2h8yVgZvjOn2OGkxK9CFvoBWzgbk= -github.com/pingcap/failpoint v0.0.0-20240527053858-9b3b6e34194a/go.mod h1:gPdo4h708R0CrwKM/DO0/6xJ64fz9vxzp2yKE2QON+s= +github.com/pingcap/failpoint v0.0.0-20240528011301-b51a646c7c86 h1:tdMsjOqUR7YXHoBitzdebTvOjs/swniBTOLy5XiMtuE= +github.com/pingcap/failpoint v0.0.0-20240528011301-b51a646c7c86/go.mod h1:exzhVYca3WRtd6gclGNErRWb1qEgff3LYta0LvRmON4= github.com/pingcap/fn v1.0.0 h1:CyA6AxcOZkQh52wIqYlAmaVmF6EvrcqFywP463pjA8g= github.com/pingcap/fn v1.0.0/go.mod h1:u9WZ1ZiOD1RpNhcI42RucFh/lBuzTu6rw88a+oF2Z24= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= -github.com/pingcap/kvproto v0.0.0-20241113043844-e1fa7ea8c302 h1:ynwwqr0rLliSOJcx0wHMu4T/NiPXHlK48mk2DCrBKCI= github.com/pingcap/kvproto v0.0.0-20241113043844-e1fa7ea8c302/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= +github.com/pingcap/kvproto v0.0.0-20241120022153-92b0414aeed8 h1:aNNifhc6xCjXKejjiNYtJJLFNMXnoDiXxkJIg1JErQE= +github.com/pingcap/kvproto v0.0.0-20241120022153-92b0414aeed8/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.20240314023424-862ccc32f18d h1:y3EueKVfVykdpTyfUnQGqft0ud+xVFuCdp1XkVL0X1E= github.com/pingcap/log v1.1.1-0.20240314023424-862ccc32f18d/go.mod h1:ORfBOFp1eteu2odzsyaxI+b8TzJwgjwyQcGhI+9SfEA= github.com/pingcap/sysutil v1.0.1-0.20241113070546-23b50de46fd3 h1:Q9CMGKUztbM0RWHdQu0pD9b9OC47sbISRiMvf9vJ2RY= github.com/pingcap/sysutil v1.0.1-0.20241113070546-23b50de46fd3/go.mod h1:tyo4AX5P7udiSKN0Mv3nD9DcUnuLLmFfE22+dEs4vbU= -github.com/pingcap/tidb v1.1.0-beta.0.20240703042657-230bbc2ef5ef h1:qf0QuRpodj5Yvm22BuuBoGFaiolL7KdkEYdnQ7DUstg= -github.com/pingcap/tidb v1.1.0-beta.0.20240703042657-230bbc2ef5ef/go.mod h1:vfEq5Kh+kAxB6ROcwTubBBk1lzZmIHMVzUwFYwQSJIA= github.com/pingcap/tidb/pkg/parser v0.0.0-20240703042657-230bbc2ef5ef h1:HhIocGusIROnXhcXQ9DQL/+RgJpGXGlOkI/IX25pNrA= github.com/pingcap/tidb/pkg/parser v0.0.0-20240703042657-230bbc2ef5ef/go.mod h1:c/4la2yfv1vBYvtIG8WCDyDinLMDIUC5+zLRHiafY+Y= -github.com/pingcap/tipb v0.0.0-20240318032315-55a7867ddd50 h1:fVNBE06Rjec+EIHaYAKAHa/bIt5lnu3Zh9O6kV7ZAdg= -github.com/pingcap/tipb v0.0.0-20240318032315-55a7867ddd50/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= +github.com/pingcap/tipb v0.0.0-20241105053214-f91fdb81a69e h1:7DdrYVwWpYr4o1AyKl8T376B4h2RsMEjkmom8MxQuuM= +github.com/pingcap/tipb v0.0.0-20241105053214-f91fdb81a69e/go.mod h1:zrnYy8vReNODg8G0OiYaX9OK+kpq+rK1jHmvd1DnIWw= github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c h1:+mdjkGKdHQG3305AYmdv1U2eRNDiU2ErMBj1gwrq8eQ= github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c/go.mod h1:7rwL4CYBLnjLxUqIJNnCWiEdr3bn6IUYi15bNlnbCCU= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= @@ -1375,8 +1378,8 @@ github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c/go.mod h1:Om github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b h1:0LFwY6Q3gMACTjAbMZBjXAqTOzOwFaj2Ld6cjeQ7Rig= github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= github.com/prometheus/client_golang v0.9.0/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= -github.com/prometheus/client_golang v1.19.1 h1:wZWJDwK+NameRJuPGDhlnFgx8e8HN3XHQeLaYJFJBOE= -github.com/prometheus/client_golang v1.19.1/go.mod h1:mP78NwGzrVks5S2H6ab8+ZZGJLZUq1hoULYBAYBw1Ho= +github.com/prometheus/client_golang v1.20.5 h1:cxppBPuYhUnsO6yo/aoRol4L7q7UFfdm+bR9r+8l63Y= +github.com/prometheus/client_golang v1.20.5/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= @@ -1385,8 +1388,8 @@ github.com/prometheus/client_model v0.4.0/go.mod h1:oMQmHW1/JoDwqLtg57MGgP/Fb1CJ github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= github.com/prometheus/common v0.0.0-20181020173914-7e9e6cabbd39/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= -github.com/prometheus/common v0.55.0 h1:KEi6DK7lXW/m7Ig5i47x0vRzuBsHuvJdi5ee6Y3G1dc= -github.com/prometheus/common v0.55.0/go.mod h1:2SECS4xJG1kd8XF9IcM1gMX6510RAEL65zxzNImwdc8= +github.com/prometheus/common v0.57.0 h1:Ro/rKjwdq9mZn1K5QPctzh+MA4Lp0BuYk5ZZEVhoNcY= +github.com/prometheus/common v0.57.0/go.mod h1:7uRPFSUTbfZWsJ7MHY56sqt7hLQu3bxXHDnNhl8E9qI= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= @@ -1397,29 +1400,29 @@ github.com/qri-io/jsonschema v0.2.1/go.mod h1:g7DPkiOsK1xv6T/Ao5scXRkd+yTFygcANP github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= 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.7 h1:WUdvkW8uEhrYfLC4ZzdpI2ztxP1I582+49Oc5Mq64VQ= +github.com/rivo/uniseg v0.4.7/go.mod h1:FN3SvrM+Zdj16jyLfmOkMNblXMcoc8DfTHruCPUcx88= github.com/robfig/cron/v3 v3.0.1 h1:WdRxkvbJztn8LMz/QEvLN5sBU+xKpSqwwUO1Pjr4qDs= github.com/robfig/cron/v3 v3.0.1/go.mod h1:eQICP3HwyT7UooqI/z+Ov+PtYAWygg1TEWWzGIFLtro= github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc= github.com/rogpeppe/go-internal v1.9.0/go.mod h1:WtVeX8xhTBvf0smdhujwtBcq4Qrzq/fJaraNFVN+nFs= -github.com/rogpeppe/go-internal v1.12.0 h1:exVL4IDcn6na9z1rAb56Vxr+CgyK3nn3O+epU5NdKM8= -github.com/rogpeppe/go-internal v1.12.0/go.mod h1:E+RYuTGaKKdloAfM02xzb0FW3Paa99yedzYV+kq4uf4= +github.com/rogpeppe/go-internal v1.13.1 h1:KvO1DLK/DRN07sQ1LQKScxyZJuNnedQ5/wKSR38lUII= +github.com/rogpeppe/go-internal v1.13.1/go.mod h1:uMEvuHeurkdAXX61udpOXGD/AzZDWNMNyH2VO9fmH0o= github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= github.com/ruudk/golang-pdf417 v0.0.0-20181029194003-1af4ab5afa58/go.mod h1:6lfFZQK844Gfx8o5WFuvpxWRwnSoipWe/p622j1v06w= github.com/ruudk/golang-pdf417 v0.0.0-20201230142125-a7e3863a1245/go.mod h1:pQAZKsJ8yyVxGRWYNEm9oFB8ieLgKFnamEyDmSA0BRk= -github.com/sasha-s/go-deadlock v0.3.1 h1:sqv7fDNShgjcaxkO0JNcOAlr8B9+cV5Ey/OB71efZx0= -github.com/sasha-s/go-deadlock v0.3.1/go.mod h1:F73l+cr82YSh10GxyRI6qZiCgK64VaZjwesgfQ1/iLM= +github.com/sasha-s/go-deadlock v0.3.5 h1:tNCOEEDG6tBqrNDOX35j/7hL5FcFViG6awUGROb2NsU= +github.com/sasha-s/go-deadlock v0.3.5/go.mod h1:bugP6EGbdGYObIlx7pUZtWqlvo8k9H6vCBBsiChJQ5U= github.com/segmentio/asm v1.2.0 h1:9BQrFxC+YOHJlTlHGkTrFWf59nbL3XnCoFLTwDCI7ys= github.com/segmentio/asm v1.2.0/go.mod h1:BqMnlJP91P8d+4ibuonYZw9mfnzI9HfxselHZr5aAcs= github.com/sergi/go-diff v1.0.0/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= github.com/sergi/go-diff v1.1.0 h1:we8PVUC3FE2uYfodKH/nBHMSetSfHDR6scGdBi+erh0= github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= github.com/shirou/gopsutil/v3 v3.21.12/go.mod h1:BToYZVTlSVlfazpDDYFnsVZLaoRG+g8ufT6fPQLdJzA= -github.com/shirou/gopsutil/v3 v3.24.4 h1:dEHgzZXt4LMNm+oYELpzl9YCqV65Yr/6SfrvgRBtXeU= -github.com/shirou/gopsutil/v3 v3.24.4/go.mod h1:lTd2mdiOspcqLgAnr9/nGi71NkeMpWKdmhuxm9GusH8= +github.com/shirou/gopsutil/v3 v3.24.5 h1:i0t8kL+kQTvpAYToeuiVk3TgDeKOFioZO3Ztz/iZ9pI= +github.com/shirou/gopsutil/v3 v3.24.5/go.mod h1:bsoOS1aStSs9ErQ1WWfxllSeS1K5D+U30r2NfcubMVk= github.com/shoenig/go-m1cpu v0.1.6 h1:nxdKQNcEB6vzgA2E2bvzKIYRuNj7XNJ4S/aRSwKzFtM= github.com/shoenig/go-m1cpu v0.1.6/go.mod h1:1JJMcUBvfNwpq05QDQVAnx3gUHr9IYF7GNg9SUEw2VQ= github.com/shoenig/test v0.6.4 h1:kVTaSd7WLz5WZ2IaoM0RSzRsUD+m8wRR+5qvntpn4LU= @@ -1440,8 +1443,8 @@ github.com/spf13/afero v1.6.0/go.mod h1:Ai8FlHk4v/PARR026UzYexafAt9roJ7LcLMAmO6Z github.com/spf13/afero v1.9.2/go.mod h1:iUV7ddyEEZPO5gA3zD4fJt6iStLlL+Lg4m2cihcDf8Y= 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.8.1 h1:e5/vxKd/rZsfSJMUX1agtjeTDf+qv1/JdBF8gg5k9ZM= +github.com/spf13/cobra v1.8.1/go.mod h1:wHxEcudfqmLYa8iTfL+OuZPbBZkmvliBWKIezN3kD9Y= 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= @@ -1466,9 +1469,8 @@ github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/stretchr/testify v1.8.3/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= -github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= -github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= -github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= +github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= +github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a h1:J/YdBZ46WKpXsxsW93SG+q0F8KI+yFrcIDT4c/RNoc4= @@ -1615,8 +1617,8 @@ golang.org/x/crypto v0.10.0/go.mod h1:o4eNf7Ede1fv+hwOwZsTHl9EsPFO6q6ZvYR8vYfY45 golang.org/x/crypto v0.11.0/go.mod h1:xgJhtzW8F9jGdVFWZESrid1U1bjeNy4zgy5cRr/CIio= golang.org/x/crypto v0.12.0/go.mod h1:NF0Gs7EO5K4qLn+Ylc+fih8BSTeIjAP05siRnAh98yw= golang.org/x/crypto v0.14.0/go.mod h1:MVFd36DqK4CsrnJYDkBA3VC4m2GkXAM0PvzMCn4JQf4= -golang.org/x/crypto v0.24.0 h1:mnl8DM0o513X8fdIkmyFE/5hTYxbwYOjDS/+rK6qpRI= -golang.org/x/crypto v0.24.0/go.mod h1:Z1PMYSOR5nyMcyAVAIQSKCDwalqy85Aqn1x3Ws4L5DM= +golang.org/x/crypto v0.29.0 h1:L5SG1JTTXupVV3n6sUqMTeWbjAyfPwoda2DLX8J8FrQ= +golang.org/x/crypto v0.29.0/go.mod h1:+F4F4N5hv6v38hfeYwTdx20oUvLLc+QfrE9Ax9HtgRg= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -1632,8 +1634,8 @@ golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u0 golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= golang.org/x/exp v0.0.0-20220827204233-334a2380cb91/go.mod h1:cyybsKvd6eL0RnXn6p/Grxp8F5bW7iYuBgsNCOHpMYE= -golang.org/x/exp v0.0.0-20240404231335-c0f41cb1a7a0 h1:985EYyeCOxTpcgOTJpflJUwOeEz0CQOdPt73OzpE9F8= -golang.org/x/exp v0.0.0-20240404231335-c0f41cb1a7a0/go.mod h1:/lliqkxwWAhPjf5oSOIJup2XcqJaw8RGS6k3TGEc7GI= +golang.org/x/exp v0.0.0-20240909161429-701f63a606c0 h1:e66Fs6Z+fZTbFBAxKfP3PALWBtpfqks2bwGcexMxgtk= +golang.org/x/exp v0.0.0-20240909161429-701f63a606c0/go.mod h1:2TbTHSBQa924w8M6Xs1QcRcFwyucIwBGpK1p2f1YFFY= golang.org/x/image v0.0.0-20180708004352-c73c2afc3b81/go.mod h1:ux5Hcp/YLpHSI86hEcLt0YII63i6oz57MZXIpbrjZUs= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= @@ -1678,8 +1680,8 @@ golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.9.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.10.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.11.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= -golang.org/x/mod v0.18.0 h1:5+9lSbEzPSdWkH32vYPBwEpX8KwDbM52Ud9xBUvNlb0= -golang.org/x/mod v0.18.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= +golang.org/x/mod v0.22.0 h1:D4nJWe9zXqHOmWqj4VMOJhvzj7bEZg4wEYa759z1pH4= +golang.org/x/mod v0.22.0/go.mod h1:6SkKJ3Xj0I0BrPOZoBy3bdMptDDU9oJrpohJ3eWZ1fY= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -1745,8 +1747,8 @@ golang.org/x/net v0.11.0/go.mod h1:2L/ixqYpgIVXmeoSA/4Lu7BzTG4KIyPIryS4IsOd1oQ= golang.org/x/net v0.12.0/go.mod h1:zEVYFnQC7m/vmpQFELhcD1EWkZlX69l4oqgmer6hfKA= golang.org/x/net v0.14.0/go.mod h1:PpSgVXXLK0OxS0F31C1/tv6XNguvCrnXIDrFMspZIUI= golang.org/x/net v0.17.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= -golang.org/x/net v0.26.0 h1:soB7SVo0PWrY4vPW/+ay0jKDNScG2X9wFeYlXIvJsOQ= -golang.org/x/net v0.26.0/go.mod h1:5YKkiSynbBIh3p6iOc/vibscux0x38BZDkn8sCUPxHE= +golang.org/x/net v0.31.0 h1:68CPQngjLL0r2AlUKiSxtQFKvzRVbnzLwMUn5SzcLHo= +golang.org/x/net v0.31.0/go.mod h1:P4fl1q7dY2hnZFxEk4pPSkDHF+QqjitcnDjUQyMM+pM= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -1779,8 +1781,8 @@ golang.org/x/oauth2 v0.7.0/go.mod h1:hPLQkd9LyjfXTiRohC/41GhcFqxisoUQ99sCUOHO9x4 golang.org/x/oauth2 v0.8.0/go.mod h1:yr7u4HXZRm1R1kBWqr/xKNqewf0plRYoB7sla+BCIXE= golang.org/x/oauth2 v0.10.0/go.mod h1:kTpgurOux7LqtuxjuyZa4Gj2gdezIt/jQtGnNFfypQI= golang.org/x/oauth2 v0.11.0/go.mod h1:LdF7O/8bLR/qWK9DrpXmbHLTouvRHK0SgJl0GmDBchk= -golang.org/x/oauth2 v0.21.0 h1:tsimM75w1tF/uws5rbeHzIWxEqElMehnc+iW793zsZs= -golang.org/x/oauth2 v0.21.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= +golang.org/x/oauth2 v0.23.0 h1:PbgcYx2W7i4LvjJWEbf0ngHV6qJYr86PkAV3bXdLEbs= +golang.org/x/oauth2 v0.23.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -1799,8 +1801,8 @@ golang.org/x/sync v0.0.0-20220929204114-8fcdb60fdcc0/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.2.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.3.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= -golang.org/x/sync v0.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M= -golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.9.0 h1:fEo0HyrW1GIgZdpbhCRO0PkJajUS5H9IFUztCgEo2jQ= +golang.org/x/sync v0.9.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181205085412-a5c9d58dba9a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -1891,9 +1893,8 @@ golang.org/x/sys v0.9.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.10.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.19.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.21.0 h1:rF+pYz3DAGSQAxAu1CbC7catZg4ebC4UIeIhKxBZvws= -golang.org/x/sys v0.21.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.27.0 h1:wBqf8DvsY9Y/2P8gAfPDEYNuS30J4lPHJxXSb/nJZ+s= +golang.org/x/sys v0.27.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.1.0/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= @@ -1908,8 +1909,8 @@ golang.org/x/term v0.9.0/go.mod h1:M6DEAAIenWoTxdKrOltXcmDY3rSplQUkrvaDU5FcQyo= golang.org/x/term v0.10.0/go.mod h1:lpqdcUyK/oCiQxvxVrppt5ggO2KCZ5QblwqPnfZ6d5o= golang.org/x/term v0.11.0/go.mod h1:zC9APTIj3jG3FdV/Ons+XE1riIZXG4aZ4GTHiPZJPIU= golang.org/x/term v0.13.0/go.mod h1:LTmsnFJwVN6bCy1rVCoS+qHT1HhALEFxKncY3WNNh4U= -golang.org/x/term v0.21.0 h1:WVXCp+/EBEHOj53Rvu+7KiT/iElMrO8ACK16SMZ3jaA= -golang.org/x/term v0.21.0/go.mod h1:ooXLefLobQVslOqselCNF4SxFAaoS6KujMbsGzSDmX0= +golang.org/x/term v0.26.0 h1:WEQa6V3Gja/BhNxg540hBip/kkaYtRg3cxg4oXSw4AU= +golang.org/x/term v0.26.0/go.mod h1:Si5m1o57C5nBNQo5z1iq+XDijt21BDBDp2bK0QI8e3E= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -1930,16 +1931,16 @@ golang.org/x/text v0.10.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/text v0.11.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/text v0.12.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= -golang.org/x/text v0.16.0 h1:a94ExnEXNtEwYLGJSIUxnWoxoRz/ZcCsV63ROupILh4= -golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= +golang.org/x/text v0.20.0 h1:gK/Kv2otX8gz+wn7Rmb3vT96ZwuoxnQlY+HlJVj7Qug= +golang.org/x/text v0.20.0/go.mod h1:D4IsuqiFMhST5bX19pQ9ikHC2GsaKyk/oF+pn3ducp4= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20220922220347-f3bd1da661af/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.1.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.5.0 h1:o7cqy6amK/52YcAKIPlM3a+Fpj35zvRj2TP+e1xFSfk= -golang.org/x/time v0.5.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= +golang.org/x/time v0.7.0 h1:ntUhktv3OPE6TgYxXWv9vKvUSJyIFJlyohwbkEwPrKQ= +golang.org/x/time v0.7.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180525024113-a5b4c53f6e8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -1992,7 +1993,6 @@ golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc golang.org/x/tools v0.0.0-20200904185747-39188db58858/go.mod h1:Cj7w3i3Rnn0Xh82ur9kSqwfTHTeVxaDqrfMjpcNT6bE= golang.org/x/tools v0.0.0-20201110124207-079ba7bd75cd/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201124115921-2c860bdd6e78/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20201125231158-b5590deeca9b/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201201161351-ac6f37ff4c2a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201208233053-a543418bbed2/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210105154028-b0ab187a4818/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= @@ -2012,8 +2012,8 @@ golang.org/x/tools v0.7.0/go.mod h1:4pg6aUX35JBAogB10C9AtvVL+qowtN4pT3CGSQex14s= golang.org/x/tools v0.8.0/go.mod h1:JxBZ99ISMI5ViVkT1tr6tdNmXeTrcpVSD3vZ1RsRdN4= golang.org/x/tools v0.9.1/go.mod h1:owI94Op576fPu3cIGQeHs3joujW/2Oc6MtlxbF5dfNc= golang.org/x/tools v0.10.0/go.mod h1:UJwyiVBsOA2uwvK/e5OY3GTpDUJriEd+/YlqAwLPmyM= -golang.org/x/tools v0.22.0 h1:gqSGLZqv+AI9lIQzniJ0nZDRG5GBPsSi+DRNHWNz6yA= -golang.org/x/tools v0.22.0/go.mod h1:aCwcsjqvq7Yqt6TNyX7QMU2enbQ/Gt0bo6krSeEri+c= +golang.org/x/tools v0.27.0 h1:qEKojBykQkQ4EynWy4S8Weg69NumxKdn40Fce3uc/8o= +golang.org/x/tools v0.27.0/go.mod h1:sUi0ZgbwW9ZPAq26Ekut+weQPR5eIM6GQLQ1Yjm1H0Q= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= 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= @@ -2395,10 +2395,10 @@ honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= honnef.co/go/tools v0.1.3/go.mod h1:NgwopIslSNH47DimFoV78dnkksY2EFtX0ajyb3K/las= -k8s.io/api v0.28.6 h1:yy6u9CuIhmg55YvF/BavPBBXB+5QicB64njJXxVnzLo= -k8s.io/api v0.28.6/go.mod h1:AM6Ys6g9MY3dl/XNaNfg/GePI0FT7WBGu8efU/lirAo= -k8s.io/apimachinery v0.28.6 h1:RsTeR4z6S07srPg6XYrwXpTJVMXsjPXn0ODakMytSW0= -k8s.io/apimachinery v0.28.6/go.mod h1:QFNX/kCl/EMT2WTSz8k4WLCv2XnkOLMaL8GAVRMdpsA= +k8s.io/api v0.29.11 h1:6FwDo33f1WX5Yu0RQTX9YAd3wth8Ik0B4SXQKsoQfbk= +k8s.io/api v0.29.11/go.mod h1:3TDAW1OpFbz/Yx5r0W06b6eiAfHEwtH61VYDzpTU4Ng= +k8s.io/apimachinery v0.29.11 h1:55+6ue9advpA7T0sX2ZJDHCLKuiFfrAAR/39VQN9KEQ= +k8s.io/apimachinery v0.29.11/go.mod h1:i3FJVwhvSp/6n8Fl4K97PJEP8C+MM+aoDq4+ZJBf70Y= k8s.io/klog/v2 v2.120.1 h1:QXU6cPEOIslTGvZaXvFWiP9VKyeet3sawzTOvdXb4Vw= k8s.io/klog/v2 v2.120.1/go.mod h1:3Jpz1GvMt720eyJH1ckRHK1EDfpxISzJ7I9OYgaDtPE= k8s.io/utils v0.0.0-20230726121419-3b25d923346b h1:sgn3ZU783SCgtaSJjpcVVlRqd6GSnlTLKgpAAttJvpI= From 845ff5696023a5b6822770201873d78034121825 Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Mon, 9 Dec 2024 14:12:15 +0800 Subject: [PATCH 10/11] update parser Signed-off-by: MyonKeminta --- integration_tests/go.mod | 2 ++ integration_tests/go.sum | 4 ++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/integration_tests/go.mod b/integration_tests/go.mod index aceec0182e..b1168906fa 100644 --- a/integration_tests/go.mod +++ b/integration_tests/go.mod @@ -119,3 +119,5 @@ replace ( ) replace github.com/pingcap/tidb => github.com/MyonKeminta/tidb v1.1.0-alpha.1.0.20241206091441-d8254eef1420 + +replace github.com/pingcap/tidb/pkg/parser => github.com/MyonKeminta/tidb/pkg/parser v0.0.0-20241206091441-d8254eef1420 diff --git a/integration_tests/go.sum b/integration_tests/go.sum index f5bb7732b0..f7f82eda30 100644 --- a/integration_tests/go.sum +++ b/integration_tests/go.sum @@ -845,6 +845,8 @@ github.com/Masterminds/semver v1.5.0 h1:H65muMkzWKEuNDnfl9d70GUjFniHKHRbFPGBuZ3Q github.com/Masterminds/semver v1.5.0/go.mod h1:MB6lktGJrhw8PrUyiEoblNEGEQ+RzHPF078ddwwvV3Y= github.com/MyonKeminta/tidb v1.1.0-alpha.1.0.20241206091441-d8254eef1420 h1:6XEvCz7qEEU5bQxmvaqyvUbbRcoUKJy/kx7Lau7C6Y0= github.com/MyonKeminta/tidb v1.1.0-alpha.1.0.20241206091441-d8254eef1420/go.mod h1:i0EIeFIL9EodlywrvfHoFIBjoV2MtxglWTTercdWw9I= +github.com/MyonKeminta/tidb/pkg/parser v0.0.0-20241206091441-d8254eef1420 h1:dyXe7791wvf9CuHZraJkSwuw7pR8sfIlYMT/ZEtSqYY= +github.com/MyonKeminta/tidb/pkg/parser v0.0.0-20241206091441-d8254eef1420/go.mod h1:Hju1TEWZvrctQKbztTRwXH7rd41Yq0Pgmq4PrEKcq7o= github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= github.com/VividCortex/ewma v1.2.0 h1:f58SaIzcDXrSy3kWaHNvuJgJ3Nmz59Zji6XoJR/q1ow= github.com/VividCortex/ewma v1.2.0/go.mod h1:nz4BbCtbLyFDeC9SUHbtcT5644juEuWfUAUnGx7j5l4= @@ -1359,8 +1361,6 @@ github.com/pingcap/log v1.1.1-0.20240314023424-862ccc32f18d h1:y3EueKVfVykdpTyfU github.com/pingcap/log v1.1.1-0.20240314023424-862ccc32f18d/go.mod h1:ORfBOFp1eteu2odzsyaxI+b8TzJwgjwyQcGhI+9SfEA= github.com/pingcap/sysutil v1.0.1-0.20241113070546-23b50de46fd3 h1:Q9CMGKUztbM0RWHdQu0pD9b9OC47sbISRiMvf9vJ2RY= github.com/pingcap/sysutil v1.0.1-0.20241113070546-23b50de46fd3/go.mod h1:tyo4AX5P7udiSKN0Mv3nD9DcUnuLLmFfE22+dEs4vbU= -github.com/pingcap/tidb/pkg/parser v0.0.0-20240703042657-230bbc2ef5ef h1:HhIocGusIROnXhcXQ9DQL/+RgJpGXGlOkI/IX25pNrA= -github.com/pingcap/tidb/pkg/parser v0.0.0-20240703042657-230bbc2ef5ef/go.mod h1:c/4la2yfv1vBYvtIG8WCDyDinLMDIUC5+zLRHiafY+Y= github.com/pingcap/tipb v0.0.0-20241105053214-f91fdb81a69e h1:7DdrYVwWpYr4o1AyKl8T376B4h2RsMEjkmom8MxQuuM= github.com/pingcap/tipb v0.0.0-20241105053214-f91fdb81a69e/go.mod h1:zrnYy8vReNODg8G0OiYaX9OK+kpq+rK1jHmvd1DnIWw= github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c h1:+mdjkGKdHQG3305AYmdv1U2eRNDiU2ErMBj1gwrq8eQ= From bfb9769d4b5185c831777d59ea9eea63196696a7 Mon Sep 17 00:00:00 2001 From: MyonKeminta Date: Mon, 9 Dec 2024 14:33:34 +0800 Subject: [PATCH 11/11] update ci workflow Signed-off-by: MyonKeminta --- .github/workflows/compatibility.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/workflows/compatibility.yml b/.github/workflows/compatibility.yml index b6288d3de8..8b65695f78 100644 --- a/.github/workflows/compatibility.yml +++ b/.github/workflows/compatibility.yml @@ -24,7 +24,8 @@ jobs: - name: Checkout TiDB uses: actions/checkout@v2 with: - repository: pingcap/tidb + repository: MyonKeminta/tidb + ref: m/update-ts-check path: tidb - name: Check build