From d2887d56ab8d0284e1ec23ae41d9ba9109b44a9f Mon Sep 17 00:00:00 2001 From: cfzjywxk Date: Wed, 31 Jan 2024 09:53:00 +0800 Subject: [PATCH 1/8] txn: use region pessimsitic lock rollback to speed up cleanup (#1125) * use region pessimsitic lock rollback and clean for write-write conflict processing Signed-off-by: cfzjywxk * format Signed-off-by: cfzjywxk * debug ci failure patch, disable region pessimistic rollback Signed-off-by: cfzjywxk --------- Signed-off-by: cfzjywxk --- integration_tests/lock_test.go | 146 +++++++++++++++++++++++++++++++ txnkv/transaction/pessimistic.go | 10 ++- txnkv/transaction/prewrite.go | 7 +- txnkv/txnlock/lock_resolver.go | 55 +++++++++--- txnkv/txnlock/test_probe.go | 2 +- 5 files changed, 200 insertions(+), 20 deletions(-) diff --git a/integration_tests/lock_test.go b/integration_tests/lock_test.go index 89bead1c98..81e76cbd34 100644 --- a/integration_tests/lock_test.go +++ b/integration_tests/lock_test.go @@ -56,6 +56,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/suite" "github.com/tikv/client-go/v2/config" + "github.com/tikv/client-go/v2/config/retry" tikverr "github.com/tikv/client-go/v2/error" "github.com/tikv/client-go/v2/kv" "github.com/tikv/client-go/v2/oracle" @@ -1028,6 +1029,7 @@ type testLockWithTiKVSuite struct { func (s *testLockWithTiKVSuite) SetupTest() { if *withTiKV { s.store = tikv.StoreProbe{KVStore: NewTestStore(s.T())} + s.cleanupLocks() } else { s.store = tikv.StoreProbe{KVStore: NewTestUniStore(s.T())} } @@ -1037,6 +1039,19 @@ func (s *testLockWithTiKVSuite) TearDownTest() { s.store.Close() } +func (s *testLockWithTiKVSuite) cleanupLocks() { + // Cleanup possible left locks. + bo := tikv.NewBackofferWithVars(context.Background(), int(transaction.PrewriteMaxBackoff.Load()), nil) + ctx := context.WithValue(context.Background(), util.SessionID, uint64(1)) + currentTS, err := s.store.CurrentTimestamp(oracle.GlobalTxnScope) + s.NoError(err) + remainingLocks, err := s.store.ScanLocks(ctx, []byte("k"), []byte("l"), currentTS) + s.NoError(err) + if len(remainingLocks) > 0 { + s.mustResolve(ctx, bo, remainingLocks, currentTS, []byte("k"), []byte("l")) + } +} + // TODO: Migrate FairLocking related tests here. func withRetry[T any](f func() (T, error), limit int, delay time.Duration) (T, error) { @@ -1559,3 +1574,134 @@ func (s *testLockWithTiKVSuite) TestBatchResolveLocks() { s.NoError(err) s.Equal(v3, v) } + +func (s *testLockWithTiKVSuite) makeLock(startTS uint64, forUpdateTS uint64, key []byte, primary []byte) *txnlock.Lock { + return &txnlock.Lock{ + Key: key, + Primary: primary, + TxnID: startTS, + TTL: 10, + TxnSize: 1024, + LockType: kvrpcpb.Op_PessimisticLock, + UseAsyncCommit: false, + LockForUpdateTS: forUpdateTS, + MinCommitTS: forUpdateTS, + } +} + +func (s *testLockWithTiKVSuite) mustLockNum(ctx context.Context, expectedNum int, scanTS uint64, startKey []byte, endKey []byte) { + remainingLocks, err := s.store.ScanLocks(ctx, startKey, endKey, scanTS) + s.NoError(err) + s.Len(remainingLocks, expectedNum) +} + +func (s *testLockWithTiKVSuite) mustResolve(ctx context.Context, bo *retry.Backoffer, remainingLocks []*txnlock.Lock, callerTS uint64, startKey []byte, endKey []byte) { + if len(remainingLocks) > 0 { + _, err := s.store.GetLockResolver().ResolveLocksWithOpts(bo, txnlock.ResolveLocksOptions{ + CallerStartTS: callerTS, + Locks: remainingLocks, + Lite: false, + ForRead: false, + Detail: nil, + PessimisticRegionResolve: true, + }) + s.NoError(err) + + lockAfterResolve, err := s.store.ScanLocks(ctx, startKey, endKey, callerTS) + s.NoError(err) + s.Len(lockAfterResolve, 0) + } +} + +func (s *testLockWithTiKVSuite) TestPessimisticRollbackWithRead() { + // The test relies on the pessimistic rollback read phase implementations in tikv + // https://github.com/tikv/tikv/pull/16185, which is not implemented in mockstore by now. + if !*withTiKV { + return + } + + s.NoError(failpoint.Enable("tikvclient/shortPessimisticLockTTL", "return")) + s.NoError(failpoint.Enable("tikvclient/twoPCShortLockTTL", "return")) + defer func() { + s.NoError(failpoint.Disable("tikvclient/shortPessimisticLockTTL")) + s.NoError(failpoint.Disable("tikvclient/twoPCShortLockTTL")) + }() + test := func(inMemoryLock bool) { + recoverFunc := s.trySetTiKVConfig("pessimistic-txn.in-memory", inMemoryLock) + defer recoverFunc() + + // Init, cleanup possible left locks. + bo := tikv.NewBackofferWithVars(context.Background(), int(transaction.PrewriteMaxBackoff.Load()), nil) + ctx := context.WithValue(context.Background(), util.SessionID, uint64(1)) + s.cleanupLocks() + + // Basic case, three keys could be rolled back within one pessimistic rollback request. + k1, k2, k3 := []byte("k1"), []byte("k2"), []byte("k3") + txn1, err := s.store.Begin() + s.NoError(err) + startTS := txn1.StartTS() + txn1.SetPessimistic(true) + lockCtx := kv.NewLockCtx(startTS, 200, time.Now()) + err = txn1.LockKeys(ctx, lockCtx, k1, k2, k3) + s.NoError(err) + txn1.GetCommitter().CloseTTLManager() + + time.Sleep(time.Millisecond * 100) + s.mustLockNum(ctx, 3, startTS+1, []byte("k"), []byte("l")) + locks := []*txnlock.Lock{ + s.makeLock(startTS, startTS, k3, k1), + } + s.mustResolve(ctx, bo, locks, startTS+1, []byte("k"), []byte("l")) + + time.Sleep(time.Millisecond * 100) + s.mustLockNum(ctx, 0, startTS+1, []byte("k"), []byte("l")) + + // Acquire pessimistic locks for more than 256(RESOLVE_LOCK_BATCH_SIZE) keys. + formatKey := func(prefix rune, i int) []byte { + return []byte(fmt.Sprintf("%c%04d", prefix, i)) + } + numKeys := 1000 + prewriteKeys := make([][]byte, 0, numKeys/2) + pessimisticLockKeys := make([][]byte, 0, numKeys/2) + for i := 0; i < numKeys; i++ { + key := formatKey('k', i) + if i%2 == 0 { + err = txn1.LockKeys(ctx, lockCtx, key) + pessimisticLockKeys = append(pessimisticLockKeys, key) + } else { + err = txn1.Set(key, []byte("val")) + s.NoError(err) + prewriteKeys = append(prewriteKeys, key) + } + s.NoError(err) + } + committer, err := txn1.NewCommitter(1) + s.NoError(err) + mutations := committer.MutationsOfKeys(prewriteKeys) + err = committer.PrewriteMutations(ctx, mutations) + s.NoError(err) + + // All the pessimistic locks belonging to the same transaction are pessimistic + // rolled back within one request. + time.Sleep(time.Millisecond * 100) + pessimisticLock := s.makeLock(startTS, startTS, pessimisticLockKeys[1], pessimisticLockKeys[0]) + _, err = s.store.GetLockResolver().ResolveLocksWithOpts(bo, txnlock.ResolveLocksOptions{ + CallerStartTS: startTS + 1, + Locks: []*txnlock.Lock{pessimisticLock}, + Lite: false, + ForRead: false, + Detail: nil, + PessimisticRegionResolve: true, + }) + s.NoError(err) + + time.Sleep(time.Millisecond * 100) + s.mustLockNum(ctx, numKeys/2, startTS+1, []byte("k"), []byte("l")) + + // Cleanup. + err = txn1.Rollback() + s.NoError(err) + } + test(false) + test(true) +} diff --git a/txnkv/transaction/pessimistic.go b/txnkv/transaction/pessimistic.go index d04c951cee..56384e84cb 100644 --- a/txnkv/transaction/pessimistic.go +++ b/txnkv/transaction/pessimistic.go @@ -356,8 +356,9 @@ func (action actionPessimisticLock) handlePessimisticLockResponseNormalMode( c.store.GetLockResolver().UpdateResolvingLocks(locks, c.startTS, *diagCtx.resolvingRecordToken) } resolveLockOpts := txnlock.ResolveLocksOptions{ - CallerStartTS: 0, - Locks: locks, + CallerStartTS: 0, + Locks: locks, + PessimisticRegionResolve: false, } if action.LockCtx.Stats != nil { resolveLockOpts.Detail = &action.LockCtx.Stats.ResolveLock @@ -484,8 +485,9 @@ func (action actionPessimisticLock) handlePessimisticLockResponseForceLockMode( c.store.GetLockResolver().UpdateResolvingLocks(locks, c.startTS, *diagCtx.resolvingRecordToken) } resolveLockOpts := txnlock.ResolveLocksOptions{ - CallerStartTS: 0, - Locks: locks, + CallerStartTS: 0, + Locks: locks, + PessimisticRegionResolve: false, } if action.LockCtx.Stats != nil { resolveLockOpts.Detail = &action.LockCtx.Stats.ResolveLock diff --git a/txnkv/transaction/prewrite.go b/txnkv/transaction/prewrite.go index fac2f4f5e9..1a8d24aa90 100644 --- a/txnkv/transaction/prewrite.go +++ b/txnkv/transaction/prewrite.go @@ -471,9 +471,10 @@ func (action actionPrewrite) handleSingleBatch( c.store.GetLockResolver().UpdateResolvingLocks(locks, c.startTS, *resolvingRecordToken) } resolveLockOpts := txnlock.ResolveLocksOptions{ - CallerStartTS: c.startTS, - Locks: locks, - Detail: &c.getDetail().ResolveLock, + CallerStartTS: c.startTS, + Locks: locks, + Detail: &c.getDetail().ResolveLock, + PessimisticRegionResolve: false, } resolveLockRes, err := c.store.GetLockResolver().ResolveLocksWithOpts(bo, resolveLockOpts) if err != nil { diff --git a/txnkv/txnlock/lock_resolver.go b/txnkv/txnlock/lock_resolver.go index bd668a1fb6..3d1508e9f3 100644 --- a/txnkv/txnlock/lock_resolver.go +++ b/txnkv/txnlock/lock_resolver.go @@ -160,6 +160,11 @@ func (s TxnStatus) StatusCacheable() bool { return false } +func (s TxnStatus) String() string { + // TODO: print primary lock after redact is introduced. + return fmt.Sprintf("ttl:%v commit_ts:%v action: %v", s.ttl, s.commitTS, s.action) +} + // Lock represents a lock from tikv server. type Lock struct { Key []byte @@ -261,7 +266,7 @@ func (lr *LockResolver) BatchResolveLocks(bo *retry.Backoffer, locks []*Lock, lo // // `resolvePessimisticLock` should be called after calling `getTxnStatus`. // See: https://github.com/pingcap/tidb/issues/45134 - err := lr.resolvePessimisticLock(bo, l) + err := lr.resolvePessimisticLock(bo, l, false, nil) if err != nil { return false, err } @@ -350,11 +355,12 @@ func (lr *LockResolver) BatchResolveLocks(bo *retry.Backoffer, locks []*Lock, lo // ResolveLocksOptions is the options struct for calling resolving lock. type ResolveLocksOptions struct { - CallerStartTS uint64 - Locks []*Lock - Lite bool - ForRead bool - Detail *util.ResolveLockDetail + CallerStartTS uint64 + Locks []*Lock + Lite bool + ForRead bool + Detail *util.ResolveLockDetail + PessimisticRegionResolve bool } // ResolveLockResult is the result struct for resolving lock. @@ -441,7 +447,7 @@ func (lr *LockResolver) ResolveLocksDone(callerStartTS uint64, token int) { } func (lr *LockResolver) resolveLocks(bo *retry.Backoffer, opts ResolveLocksOptions) (ResolveLockResult, error) { - callerStartTS, locks, forRead, lite, detail := opts.CallerStartTS, opts.Locks, opts.ForRead, opts.Lite, opts.Detail + callerStartTS, locks, forRead, lite, detail, pessimisticRegionResolve := opts.CallerStartTS, opts.Locks, opts.ForRead, opts.Lite, opts.Detail, opts.PessimisticRegionResolve if lr.testingKnobs.meetLock != nil { lr.testingKnobs.meetLock(locks) } @@ -464,6 +470,7 @@ func (lr *LockResolver) resolveLocks(bo *retry.Backoffer, opts ResolveLocksOptio // TxnID -> []Region, record resolved Regions. // TODO: Maybe put it in LockResolver and share by all txns. cleanTxns := make(map[uint64]map[locate.RegionVerID]struct{}) + pessimisticCleanTxns := make(map[uint64]map[locate.RegionVerID]struct{}) var resolve func(*Lock, bool) (TxnStatus, error) resolve = func(l *Lock, forceSyncCommit bool) (TxnStatus, error) { status, err := lr.getTxnStatusFromLock(bo, l, callerStartTS, forceSyncCommit, detail) @@ -482,7 +489,8 @@ func (lr *LockResolver) resolveLocks(bo *retry.Backoffer, opts ResolveLocksOptio return status, nil } - // If the lock is committed or rollbacked, resolve lock. + // If the lock is committed or rolled back, resolve lock. + // If the lock is regarded as an expired pessimistic lock, pessimistic rollback it. metrics.LockResolverCountWithExpired.Inc() cleanRegions, exists := cleanTxns[l.TxnID] if !exists { @@ -504,7 +512,16 @@ func (lr *LockResolver) resolveLocks(bo *retry.Backoffer, opts ResolveLocksOptio } if l.LockType == kvrpcpb.Op_PessimisticLock { // pessimistic locks don't block read so it needn't be async. - err = lr.resolvePessimisticLock(bo, l) + if pessimisticRegionResolve { + pessimisticCleanRegions, exists := pessimisticCleanTxns[l.TxnID] + if !exists { + pessimisticCleanRegions = make(map[locate.RegionVerID]struct{}) + pessimisticCleanTxns[l.TxnID] = pessimisticCleanRegions + } + err = lr.resolvePessimisticLock(bo, l, true, pessimisticCleanRegions) + } else { + err = lr.resolvePessimisticLock(bo, l, false, nil) + } } else { if forRead { asyncCtx := context.WithValue(lr.asyncResolveCtx, util.RequestSourceKey, bo.GetCtx().Value(util.RequestSourceKey)) @@ -669,7 +686,8 @@ func (lr *LockResolver) getTxnStatusFromLock(bo *retry.Backoffer, l *Lock, calle if lr.store.GetOracle().UntilExpired(l.TxnID, l.TTL, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) <= 0 { logutil.Logger(bo.GetCtx()).Warn("lock txn not found, lock has expired", zap.Uint64("CallerStartTs", callerStartTS), - zap.Stringer("lock str", l)) + zap.Stringer("lock str", l), + zap.Stringer("status", status)) if l.LockType == kvrpcpb.Op_PessimisticLock { if _, err := util.EvalFailpoint("txnExpireRetTTL"); err == nil { return TxnStatus{action: kvrpcpb.Action_LockNotExistDoNothing}, @@ -1178,7 +1196,10 @@ func (lr *LockResolver) resolveLock(bo *retry.Backoffer, l *Lock, status TxnStat // resolvePessimisticLock handles pessimistic locks after checking txn status. // Note that this function assumes `CheckTxnStatus` is done (or `getTxnStatusFromLock` has been called) on the lock. -func (lr *LockResolver) resolvePessimisticLock(bo *retry.Backoffer, l *Lock) error { +// When "pessimisticRegionResolve" is set to false, only pessimistic rollback input lock. Otherwise, the corresponding +// region will be scanned, and all relevant pessimistic locks that are read will be rolled back at the same time, +// similar to the `resolveLock` function. +func (lr *LockResolver) resolvePessimisticLock(bo *retry.Backoffer, l *Lock, pessimisticRegionResolve bool, pessimisticCleanRegions map[locate.RegionVerID]struct{}) error { metrics.LockResolverCountWithResolveLocks.Inc() // The lock has been resolved by getTxnStatusFromLock. if bytes.Equal(l.Key, l.Primary) { @@ -1189,6 +1210,11 @@ func (lr *LockResolver) resolvePessimisticLock(bo *retry.Backoffer, l *Lock) err if err != nil { return err } + if pessimisticRegionResolve && pessimisticCleanRegions != nil { + if _, ok := pessimisticCleanRegions[loc.Region]; ok { + return nil + } + } forUpdateTS := l.LockForUpdateTS if forUpdateTS == 0 { forUpdateTS = math.MaxUint64 @@ -1196,7 +1222,9 @@ func (lr *LockResolver) resolvePessimisticLock(bo *retry.Backoffer, l *Lock) err pessimisticRollbackReq := &kvrpcpb.PessimisticRollbackRequest{ StartVersion: l.TxnID, ForUpdateTs: forUpdateTS, - Keys: [][]byte{l.Key}, + } + if !pessimisticRegionResolve { + pessimisticRollbackReq.Keys = [][]byte{l.Key} } req := tikvrpc.NewRequest(tikvrpc.CmdPessimisticRollback, pessimisticRollbackReq, kvrpcpb.Context{ ResourceControlContext: &kvrpcpb.ResourceControlContext{ @@ -1228,6 +1256,9 @@ func (lr *LockResolver) resolvePessimisticLock(bo *retry.Backoffer, l *Lock) err logutil.Logger(bo.GetCtx()).Error("resolveLock error", zap.Error(err)) return err } + if pessimisticRegionResolve && pessimisticCleanRegions != nil { + pessimisticCleanRegions[loc.Region] = struct{}{} + } return nil } } diff --git a/txnkv/txnlock/test_probe.go b/txnkv/txnlock/test_probe.go index 67468bd344..6f91f252c2 100644 --- a/txnkv/txnlock/test_probe.go +++ b/txnkv/txnlock/test_probe.go @@ -58,7 +58,7 @@ func (l LockResolverProbe) ResolveLock(bo *retry.Backoffer, lock *Lock) error { // ResolvePessimisticLock resolves single pessimistic lock. func (l LockResolverProbe) ResolvePessimisticLock(bo *retry.Backoffer, lock *Lock) error { - return l.resolvePessimisticLock(bo, lock) + return l.resolvePessimisticLock(bo, lock, false, nil) } // GetTxnStatus sends the CheckTxnStatus request to the TiKV server. From 6e501a142dcece6672c4a255ef8a03f57f147e56 Mon Sep 17 00:00:00 2001 From: zyguan Date: Wed, 31 Jan 2024 10:14:07 +0800 Subject: [PATCH 2/8] improve region reload strategy (#1122) * refine region reload strategy Signed-off-by: zyguan * fix data race in ut Signed-off-by: zyguan * fix another data race Signed-off-by: zyguan * access store.epoch atomic Signed-off-by: zyguan * re-implement async reload by sync flags Signed-off-by: zyguan * a minor optimization Signed-off-by: zyguan * fix ut Signed-off-by: zyguan * Update internal/locate/region_cache.go Co-authored-by: ekexium Signed-off-by: zyguan * rename async-reload to delayed-reload Signed-off-by: zyguan --------- Signed-off-by: zyguan Co-authored-by: ekexium --- .github/workflows/test.yml | 1 - internal/locate/region_cache.go | 306 +++++++++---------- internal/locate/region_cache_test.go | 147 +++++++-- internal/locate/region_request.go | 8 +- internal/locate/region_request3_test.go | 47 +++ internal/locate/region_request_state_test.go | 12 +- 6 files changed, 304 insertions(+), 217 deletions(-) diff --git a/.github/workflows/test.yml b/.github/workflows/test.yml index 1dc50b8bce..f6bd35b367 100644 --- a/.github/workflows/test.yml +++ b/.github/workflows/test.yml @@ -48,4 +48,3 @@ jobs: uses: golangci/golangci-lint-action@v3 with: version: v1.55.2 - diff --git a/internal/locate/region_cache.go b/internal/locate/region_cache.go index 4c9ed782c5..3b802f4525 100644 --- a/internal/locate/region_cache.go +++ b/internal/locate/region_cache.go @@ -41,7 +41,9 @@ import ( "fmt" "math" "math/rand" + "slices" "sort" + "strconv" "strings" "sync" "sync/atomic" @@ -120,8 +122,10 @@ func SetRegionCacheTTLSec(t int64) { } const ( - updated int32 = iota // region is updated and no need to reload. - needSync // need sync new region info. + needReloadOnAccess int32 = 1 << iota // indicates the region will be reloaded on next access + needExpireAfterTTL // indicates the region will expire after RegionCacheTTL (even when it's accessed continuously) + needDelayedReloadPending // indicates the region will be reloaded later after it's scanned by GC + needDelayedReloadReady // indicates the region has been scanned by GC and can be reloaded by id on next access ) // InvalidReason is the reason why a cached region is invalidated. @@ -148,14 +152,11 @@ const ( // Region presents kv region type Region struct { - meta *metapb.Region // raw region meta from PD, immutable after init - store unsafe.Pointer // point to region store info, see RegionStore - syncFlag int32 // region need be sync in next turn - lastAccess int64 // last region access time, see checkRegionCacheTTL - invalidReason InvalidReason // the reason why the region is invalidated - asyncReload atomic.Bool // the region need to be reloaded in async mode - lastLoad int64 // last region load time - hasUnavailableTiFlashStore bool // has unavailable TiFlash store, if yes, need to trigger async reload periodically + meta *metapb.Region // raw region meta from PD, immutable after init + store unsafe.Pointer // point to region store info, see RegionStore + lastAccess int64 // last region access time, see checkRegionCacheTTL + syncFlags int32 // region need be sync later, see needReloadOnAccess, needExpireAfterTTL + invalidReason InvalidReason // the reason why the region is invalidated } // AccessIndex represent the index for accessIndex array @@ -180,9 +181,10 @@ type regionStore struct { // buckets is not accurate and it can change even if the region is not changed. // It can be stale and buckets keys can be out of the region range. buckets *metapb.Buckets - // record all storeIDs on which pending peers reside. - // key is storeID, val is peerID. - pendingTiFlashPeerStores map[uint64]uint64 + // pendingPeers refers to pdRegion.PendingPeers. It's immutable and can be used to reconstruct pdRegions. + pendingPeers []*metapb.Peer + // downPeers refers to pdRegion.DownPeers. It's immutable and can be used to reconstruct pdRegions. + downPeers []*metapb.Peer } func (r *regionStore) accessStore(mode accessMode, idx AccessIndex) (int, *Store) { @@ -275,12 +277,13 @@ func newRegion(bo *retry.Backoffer, c *RegionCache, pdRegion *pd.Region) (*Regio // regionStore pull used store from global store map // to avoid acquire storeMu in later access. rs := ®ionStore{ - workTiKVIdx: 0, - proxyTiKVIdx: -1, - stores: make([]*Store, 0, len(r.meta.Peers)), - pendingTiFlashPeerStores: map[uint64]uint64{}, - storeEpochs: make([]uint32, 0, len(r.meta.Peers)), - buckets: pdRegion.Buckets, + workTiKVIdx: 0, + proxyTiKVIdx: -1, + stores: make([]*Store, 0, len(r.meta.Peers)), + storeEpochs: make([]uint32, 0, len(r.meta.Peers)), + buckets: pdRegion.Buckets, + pendingPeers: pdRegion.PendingPeers, + downPeers: pdRegion.DownPeers, } leader := pdRegion.Leader @@ -295,8 +298,8 @@ func newRegion(bo *retry.Backoffer, c *RegionCache, pdRegion *pd.Region) (*Regio if err != nil { return nil, err } - // Filter the peer on a tombstone store. - if addr == "" { + // Filter out the peer on a tombstone or down store. + if addr == "" || slices.ContainsFunc(pdRegion.DownPeers, func(dp *metapb.Peer) bool { return isSamePeer(dp, p) }) { continue } @@ -319,11 +322,6 @@ func newRegion(bo *retry.Backoffer, c *RegionCache, pdRegion *pd.Region) (*Regio } rs.stores = append(rs.stores, store) rs.storeEpochs = append(rs.storeEpochs, atomic.LoadUint32(&store.epoch)) - for _, pendingPeer := range pdRegion.PendingPeers { - if pendingPeer.Id == p.Id { - rs.pendingTiFlashPeerStores[store.storeID] = p.Id - } - } } // TODO(youjiali1995): It's possible the region info in PD is stale for now but it can recover. // Maybe we need backoff here. @@ -331,34 +329,16 @@ func newRegion(bo *retry.Backoffer, c *RegionCache, pdRegion *pd.Region) (*Regio return nil, errors.Errorf("no available peers, region: {%v}", r.meta) } - for _, p := range pdRegion.DownPeers { - store, exists := c.getStore(p.StoreId) - if !exists { - store = c.getStoreOrInsertDefault(p.StoreId) - } - addr, err := store.initResolve(bo, c) - if err != nil { - continue - } - // Filter the peer on a tombstone store. - if addr == "" { - continue - } - - if store.storeType == tikvrpc.TiFlash { - r.hasUnavailableTiFlashStore = true - break - } - } - rs.workTiKVIdx = leaderAccessIdx - r.meta.Peers = availablePeers - r.setStore(rs) + r.meta.Peers = availablePeers + // if the region has down peers, let it expire after TTL. + if len(pdRegion.DownPeers) > 0 { + r.syncFlags |= needExpireAfterTTL + } // mark region has been init accessed. r.lastAccess = time.Now().Unix() - r.lastLoad = r.lastAccess return r, nil } @@ -391,7 +371,7 @@ func (r *Region) checkRegionCacheTTL(ts int64) bool { if ts-lastAccess > regionCacheTTLSec { return false } - if atomic.CompareAndSwapInt64(&r.lastAccess, lastAccess, ts) { + if r.checkSyncFlags(needExpireAfterTTL) || atomic.CompareAndSwapInt64(&r.lastAccess, lastAccess, ts) { return true } } @@ -410,31 +390,43 @@ func (r *Region) invalidateWithoutMetrics(reason InvalidReason) { atomic.StoreInt64(&r.lastAccess, invalidatedLastAccessTime) } -// scheduleReload schedules reload region request in next LocateKey. -func (r *Region) scheduleReload() { - oldValue := atomic.LoadInt32(&r.syncFlag) - if oldValue != updated { - return - } - atomic.CompareAndSwapInt32(&r.syncFlag, oldValue, needSync) +func (r *Region) getSyncFlags() int32 { + return atomic.LoadInt32(&r.syncFlags) } -// checkNeedReloadAndMarkUpdated returns whether the region need reload and marks the region to be updated. -func (r *Region) checkNeedReloadAndMarkUpdated() bool { - oldValue := atomic.LoadInt32(&r.syncFlag) - if oldValue == updated { - return false +// checkSyncFlags returns true if sync_flags contains any of flags. +func (r *Region) checkSyncFlags(flags int32) bool { + return atomic.LoadInt32(&r.syncFlags)&flags > 0 +} + +// setSyncFlags sets the sync_flags bits to sync_flags|flags. +func (r *Region) setSyncFlags(flags int32) { + for { + oldFlags := atomic.LoadInt32(&r.syncFlags) + if oldFlags&flags == flags { + return + } + if atomic.CompareAndSwapInt32(&r.syncFlags, oldFlags, oldFlags|flags) { + return + } } - return atomic.CompareAndSwapInt32(&r.syncFlag, oldValue, updated) } -func (r *Region) checkNeedReload() bool { - v := atomic.LoadInt32(&r.syncFlag) - return v != updated +// resetSyncFlags reverts flags from sync_flags (that is sync_flags&^flags), returns the flags that are reset (0 means no flags are reverted). +func (r *Region) resetSyncFlags(flags int32) int32 { + for { + oldFlags := atomic.LoadInt32(&r.syncFlags) + if oldFlags&flags == 0 { + return 0 + } + if atomic.CompareAndSwapInt32(&r.syncFlags, oldFlags, oldFlags&^flags) { + return oldFlags & flags + } + } } func (r *Region) isValid() bool { - return r != nil && !r.checkNeedReload() && r.checkRegionCacheTTL(time.Now().Unix()) + return r != nil && !r.checkSyncFlags(needReloadOnAccess) && r.checkRegionCacheTTL(time.Now().Unix()) } type regionIndexMu struct { @@ -490,17 +482,13 @@ type RegionCache struct { // Context for background jobs ctx context.Context cancelFunc context.CancelFunc + wg sync.WaitGroup testingKnobs struct { // Replace the requestLiveness function for test purpose. Note that in unit tests, if this is not set, // requestLiveness always returns unreachable. mockRequestLiveness atomic.Pointer[livenessFunc] } - - regionsNeedReload struct { - sync.Mutex - regions []uint64 - } } // NewRegionCache creates a RegionCache. @@ -531,16 +519,21 @@ func NewRegionCache(pdClient pd.Client) *RegionCache { c.mu = *newRegionIndexMu(nil) } + // TODO(zyguan): refine management of background cron jobs + c.wg.Add(1) go c.asyncCheckAndResolveLoop(time.Duration(interval) * time.Second) c.enableForwarding = config.GetGlobalConfig().EnableForwarding // Default use 15s as the update inerval. + c.wg.Add(1) go c.asyncUpdateStoreSlowScore(time.Duration(interval/4) * time.Second) if config.GetGlobalConfig().RegionsRefreshInterval > 0 { c.timelyRefreshCache(config.GetGlobalConfig().RegionsRefreshInterval) } else { // cacheGC is not compatible with timelyRefreshCache + c.wg.Add(1) go c.cacheGC() } + c.wg.Add(1) go c.asyncReportStoreReplicaFlows(time.Duration(interval/2) * time.Second) return c } @@ -559,7 +552,7 @@ func newTestRegionCache() *RegionCache { // clear clears all cached data in the RegionCache. It's only used in tests. func (c *RegionCache) clear() { - c.mu = *newRegionIndexMu(nil) + c.mu.refresh(nil) c.clearStores() } @@ -571,20 +564,17 @@ func (c *RegionCache) insertRegionToCache(cachedRegion *Region, invalidateOldReg // Close releases region cache's resource. func (c *RegionCache) Close() { c.cancelFunc() + c.wg.Wait() } -var reloadRegionInterval = int64(10 * time.Second) - // asyncCheckAndResolveLoop with func (c *RegionCache) asyncCheckAndResolveLoop(interval time.Duration) { ticker := time.NewTicker(interval) - reloadRegionTicker := time.NewTicker(time.Duration(atomic.LoadInt64(&reloadRegionInterval))) defer func() { + c.wg.Done() ticker.Stop() - reloadRegionTicker.Stop() }() var needCheckStores []*Store - reloadNextLoop := make(map[uint64]struct{}) for { needCheckStores = needCheckStores[:0] select { @@ -602,21 +592,6 @@ func (c *RegionCache) asyncCheckAndResolveLoop(interval time.Duration) { // there's a deleted store in the stores map which guaranteed by reReslve(). return state != unresolved && state != tombstone && state != deleted }) - case <-reloadRegionTicker.C: - for regionID := range reloadNextLoop { - c.reloadRegion(regionID) - delete(reloadNextLoop, regionID) - } - c.regionsNeedReload.Lock() - for _, regionID := range c.regionsNeedReload.regions { - // will reload in next tick, wait a while for two reasons: - // 1. there may an unavailable duration while recreating the connection. - // 2. the store may just be started, and wait safe ts synced to avoid the - // possible dataIsNotReady error. - reloadNextLoop[regionID] = struct{}{} - } - c.regionsNeedReload.regions = c.regionsNeedReload.regions[:0] - c.regionsNeedReload.Unlock() } } } @@ -874,7 +849,7 @@ func (c *RegionCache) GetAllValidTiFlashStores(id RegionVerID, currentStore *Sto allStores = append(allStores, store.storeID) } for _, storeID := range allStores { - if _, ok := regionStore.pendingTiFlashPeerStores[storeID]; !ok { + if !slices.ContainsFunc(regionStore.pendingPeers, func(p *metapb.Peer) bool { return p.StoreId == storeID }) { nonPendingStores = append(nonPendingStores, storeID) } } @@ -891,11 +866,6 @@ func (c *RegionCache) GetTiFlashRPCContext(bo *retry.Backoffer, id RegionVerID, return nil, nil } - if cachedRegion.hasUnavailableTiFlashStore && time.Now().Unix()-cachedRegion.lastLoad > regionCacheTTLSec { - /// schedule an async reload to avoid load balance issue, refer https://github.com/pingcap/tidb/issues/35418 for details - c.scheduleReloadRegion(cachedRegion) - } - regionStore := cachedRegion.getStore() // sIdx is for load balance of TiFlash store. @@ -1123,9 +1093,18 @@ func (c *RegionCache) findRegionByKey(bo *retry.Backoffer, key []byte, isEndKey c.insertRegionToCache(r, true, true) c.mu.Unlock() } - } else if r.checkNeedReloadAndMarkUpdated() { + } else if flags := r.resetSyncFlags(needReloadOnAccess | needDelayedReloadReady); flags > 0 { // load region when it be marked as need reload. - lr, err := c.loadRegion(bo, key, isEndKey) + reloadOnAccess := flags&needReloadOnAccess > 0 + var ( + lr *Region + err error + ) + if reloadOnAccess { + lr, err = c.loadRegion(bo, key, isEndKey) + } else { + lr, err = c.loadRegionByID(bo, r.GetID()) + } if err != nil { // ignore error and use old region info. logutil.Logger(bo.GetCtx()).Error("load region failure", @@ -1135,7 +1114,7 @@ func (c *RegionCache) findRegionByKey(bo *retry.Backoffer, key []byte, isEndKey logutil.Eventf(bo.GetCtx(), "load region %d from pd, due to need-reload", lr.GetID()) r = lr c.mu.Lock() - c.insertRegionToCache(r, true, true) + c.insertRegionToCache(r, reloadOnAccess, reloadOnAccess) c.mu.Unlock() } } @@ -1144,7 +1123,7 @@ func (c *RegionCache) findRegionByKey(bo *retry.Backoffer, key []byte, isEndKey func (c *RegionCache) tryFindRegionByKey(key []byte, isEndKey bool) (r *Region) { r = c.searchCachedRegion(key, isEndKey) - if r == nil || r.checkNeedReloadAndMarkUpdated() { + if r == nil || r.checkSyncFlags(needReloadOnAccess) { return nil } return r @@ -1194,7 +1173,7 @@ func (c *RegionCache) OnSendFailForTiFlash(bo *retry.Backoffer, store *Store, re // force reload region when retry all known peers in region. if scheduleReload { - r.scheduleReload() + r.setSyncFlags(needReloadOnAccess) } } @@ -1256,7 +1235,7 @@ func (c *RegionCache) OnSendFail(bo *retry.Backoffer, ctx *RPCContext, scheduleR // force reload region when retry all known peers in region. if scheduleReload { - r.scheduleReload() + r.setSyncFlags(needReloadOnAccess) } } @@ -1267,7 +1246,8 @@ func (c *RegionCache) LocateRegionByID(bo *retry.Backoffer, regionID uint64) (*K r := c.getRegionByIDFromCache(regionID) c.mu.RUnlock() if r != nil { - if r.checkNeedReloadAndMarkUpdated() { + if flags := r.resetSyncFlags(needReloadOnAccess); flags > 0 { + reloadOnAccess := flags&needReloadOnAccess > 0 lr, err := c.loadRegionByID(bo, regionID) if err != nil { // ignore error and use old region info. @@ -1276,7 +1256,7 @@ func (c *RegionCache) LocateRegionByID(bo *retry.Backoffer, regionID uint64) (*K } else { r = lr c.mu.Lock() - c.insertRegionToCache(r, true, true) + c.insertRegionToCache(r, reloadOnAccess, reloadOnAccess) c.mu.Unlock() } } @@ -1305,38 +1285,6 @@ func (c *RegionCache) LocateRegionByID(bo *retry.Backoffer, regionID uint64) (*K }, nil } -func (c *RegionCache) scheduleReloadRegion(region *Region) { - if region == nil || !region.asyncReload.CompareAndSwap(false, true) { - // async reload scheduled by other thread. - return - } - regionID := region.GetID() - if regionID > 0 { - c.regionsNeedReload.Lock() - c.regionsNeedReload.regions = append(c.regionsNeedReload.regions, regionID) - c.regionsNeedReload.Unlock() - } -} - -func (c *RegionCache) reloadRegion(regionID uint64) { - bo := retry.NewNoopBackoff(context.Background()) - lr, err := c.loadRegionByID(bo, regionID) - if err != nil { - // ignore error and use old region info. - logutil.Logger(bo.GetCtx()).Error("load region failure", - zap.Uint64("regionID", regionID), zap.Error(err)) - c.mu.RLock() - if oldRegion := c.getRegionByIDFromCache(regionID); oldRegion != nil { - oldRegion.asyncReload.Store(false) - } - c.mu.RUnlock() - return - } - c.mu.Lock() - c.insertRegionToCache(lr, false, false) - c.mu.Unlock() -} - // GroupKeysByRegion separates keys into groups by their belonging Regions. // Specially it also returns the first key's region which may be used as the // 'PrimaryLockKey' and should be committed ahead of others. @@ -1602,7 +1550,7 @@ func (c *RegionCache) getRegionByIDFromCache(regionID uint64) *Region { if ts-lastAccess > regionCacheTTLSec { return nil } - if latestRegion != nil { + if !latestRegion.checkSyncFlags(needExpireAfterTTL) { atomic.CompareAndSwapInt64(&latestRegion.lastAccess, atomic.LoadInt64(&latestRegion.lastAccess), ts) } return latestRegion @@ -1622,26 +1570,6 @@ func (c *RegionCache) GetAllStores() []*Store { }) } -func filterUnavailablePeers(region *pd.Region) { - if len(region.DownPeers) == 0 { - return - } - new := region.Meta.Peers[:0] - for _, p := range region.Meta.Peers { - available := true - for _, downPeer := range region.DownPeers { - if p.Id == downPeer.Id && p.StoreId == downPeer.StoreId { - available = false - break - } - } - if available { - new = append(new, p) - } - } - region.Meta.Peers = new -} - // loadRegion loads region from pd client, and picks the first peer as leader. // If the given key is the end key of the region that you want, you may set the second argument to true. This is useful // when processing in reverse order. @@ -1689,7 +1617,6 @@ func (c *RegionCache) loadRegion(bo *retry.Backoffer, key []byte, isEndKey bool, backoffErr = errors.Errorf("region not found for key %q, encode_key: %q", util.HexRegionKeyStr(key), util.HexRegionKey(c.codec.EncodeRegionKey(key))) continue } - filterUnavailablePeers(reg) if len(reg.Meta.Peers) == 0 { return nil, errors.New("receive Region with no available peer") } @@ -1735,7 +1662,6 @@ func (c *RegionCache) loadRegionByID(bo *retry.Backoffer, regionID uint64) (*Reg if reg == nil || reg.Meta == nil { return nil, errors.Errorf("region not found for regionID %d", regionID) } - filterUnavailablePeers(reg) if len(reg.Meta.Peers) == 0 { return nil, errors.New("receive Region with no available peer") } @@ -1765,8 +1691,12 @@ func (c *RegionCache) timelyRefreshCache(intervalS uint64) { return } ticker := time.NewTicker(time.Duration(intervalS) * time.Second) + c.wg.Add(1) go func() { - defer ticker.Stop() + defer func() { + c.wg.Done() + ticker.Stop() + }() for { select { case <-c.ctx.Done(): @@ -2156,11 +2086,15 @@ const cleanRegionNumPerRound = 50 // negligible. func (c *RegionCache) cacheGC() { ticker := time.NewTicker(cleanCacheInterval) - defer ticker.Stop() + defer func() { + c.wg.Done() + ticker.Stop() + }() beginning := newBtreeSearchItem([]byte("")) iterItem := beginning expired := make([]*btreeItem, cleanRegionNumPerRound) + remaining := make([]*Region, cleanRegionNumPerRound) for { select { case <-c.ctx.Done(): @@ -2168,6 +2102,7 @@ func (c *RegionCache) cacheGC() { case <-ticker.C: count := 0 expired = expired[:0] + remaining = remaining[:0] // Only RLock when checking TTL to avoid blocking other readers c.mu.RLock() @@ -2180,6 +2115,8 @@ func (c *RegionCache) cacheGC() { count++ if item.cachedRegion.isCacheTTLExpired(ts) { expired = append(expired, item) + } else { + remaining = append(remaining, item.cachedRegion) } return true }) @@ -2190,6 +2127,7 @@ func (c *RegionCache) cacheGC() { iterItem = beginning } + // Clean expired regions if len(expired) > 0 { c.mu.Lock() for _, item := range expired { @@ -2198,6 +2136,30 @@ func (c *RegionCache) cacheGC() { } c.mu.Unlock() } + + // Check remaining regions and update sync flags + for _, region := range remaining { + syncFlags := region.getSyncFlags() + if syncFlags&needDelayedReloadReady > 0 { + // the region will be reload soon on access + continue + } + if syncFlags&needDelayedReloadPending > 0 { + region.setSyncFlags(needDelayedReloadReady) + // the region will be reload soon on access, no need to check if it needs to be expired + continue + } + if syncFlags&needExpireAfterTTL == 0 { + regionStore := region.getStore() + for i, store := range regionStore.stores { + // if the region has a stale or unreachable store, let it expire after TTL. + if atomic.LoadUint32(&store.epoch) != regionStore.storeEpochs[i] || store.getLivenessState() != reachable { + region.setSyncFlags(needExpireAfterTTL) + break + } + } + } + } } } } @@ -3004,7 +2966,10 @@ func (s *Store) markAlreadySlow() { // asyncUpdateStoreSlowScore updates the slow score of each store periodically. func (c *RegionCache) asyncUpdateStoreSlowScore(interval time.Duration) { ticker := time.NewTicker(interval) - defer ticker.Stop() + defer func() { + c.wg.Done() + ticker.Stop() + }() for { select { case <-c.ctx.Done(): @@ -3026,13 +2991,13 @@ func (c *RegionCache) checkAndUpdateStoreSlowScores() { zap.Stack("stack trace")) } }() - slowScoreMetrics := make(map[string]float64) + slowScoreMetrics := make(map[uint64]float64) c.forEachStore(func(store *Store) { store.updateSlowScoreStat() - slowScoreMetrics[store.addr] = float64(store.getSlowScore()) + slowScoreMetrics[store.storeID] = float64(store.getSlowScore()) }) for store, score := range slowScoreMetrics { - metrics.TiKVStoreSlowScoreGauge.WithLabelValues(store).Set(score) + metrics.TiKVStoreSlowScoreGauge.WithLabelValues(strconv.FormatUint(store, 10)).Set(score) } } @@ -3054,7 +3019,10 @@ func (s *Store) recordReplicaFlowsStats(destType replicaFlowsType) { // asyncReportStoreReplicaFlows reports the statistics on the related replicaFlowsType. func (c *RegionCache) asyncReportStoreReplicaFlows(interval time.Duration) { ticker := time.NewTicker(interval) - defer ticker.Stop() + defer func() { + c.wg.Done() + ticker.Stop() + }() for { select { case <-c.ctx.Done(): diff --git a/internal/locate/region_cache_test.go b/internal/locate/region_cache_test.go index 39f4a1d7ed..2f09239d3c 100644 --- a/internal/locate/region_cache_test.go +++ b/internal/locate/region_cache_test.go @@ -58,6 +58,18 @@ import ( uatomic "go.uber.org/atomic" ) +type inspectedPDClient struct { + pd.Client + getRegion func(ctx context.Context, cli pd.Client, key []byte, opts ...pd.GetRegionOption) (*pd.Region, error) +} + +func (c *inspectedPDClient) GetRegion(ctx context.Context, key []byte, opts ...pd.GetRegionOption) (*pd.Region, error) { + if c.getRegion != nil { + return c.getRegion(ctx, c.Client, key, opts...) + } + return c.Client.GetRegion(ctx, key, opts...) +} + func TestRegionCache(t *testing.T) { suite.Run(t, new(testRegionCacheSuite)) } @@ -73,6 +85,7 @@ type testRegionCacheSuite struct { region1 uint64 cache *RegionCache bo *retry.Backoffer + onClosed func() } func (s *testRegionCacheSuite) SetupTest() { @@ -92,6 +105,9 @@ func (s *testRegionCacheSuite) SetupTest() { func (s *testRegionCacheSuite) TearDownTest() { s.cache.Close() s.mvccStore.Close() + if s.onClosed != nil { + s.onClosed() + } } func (s *testRegionCacheSuite) storeAddr(id uint64) string { @@ -293,7 +309,66 @@ func (s *testRegionCacheSuite) TestResolveStateTransition() { s.cluster.AddStore(storeMeta.GetId(), storeMeta.GetAddress(), storeMeta.GetLabels()...) } -func (s *testRegionCacheSuite) TestTiFlashDownPeersAndAsyncReload() { +func (s *testRegionCacheSuite) TestNeedExpireRegionAfterTTL() { + s.onClosed = func() { SetRegionCacheTTLSec(600) } + SetRegionCacheTTLSec(2) + + cntGetRegion := 0 + s.cache.pdClient = &inspectedPDClient{ + Client: s.cache.pdClient, + getRegion: func(ctx context.Context, cli pd.Client, key []byte, opts ...pd.GetRegionOption) (*pd.Region, error) { + cntGetRegion++ + return cli.GetRegion(ctx, key, opts...) + }, + } + + s.Run("WithDownPeers", func() { + cntGetRegion = 0 + s.cache.clear() + s.cluster.MarkPeerDown(s.peer2) + + for i := 0; i < 50; i++ { + time.Sleep(100 * time.Millisecond) + _, err := s.cache.LocateKey(s.bo, []byte("a")) + s.NoError(err) + } + s.Equal(2, cntGetRegion, "should reload region with down peers every RegionCacheTTL") + }) + + s.Run("WithStaleStores", func() { + cntGetRegion = 0 + s.cache.clear() + store2 := s.cache.getStoreOrInsertDefault(s.store2) + + for i := 0; i < 50; i++ { + atomic.StoreUint32(&store2.epoch, uint32(i)) + time.Sleep(100 * time.Millisecond) + _, err := s.cache.LocateKey(s.bo, []byte("a")) + s.NoError(err) + } + s.Equal(2, cntGetRegion, "should reload region with stale stores every RegionCacheTTL") + }) + + s.Run("WithUnreachableStores", func() { + cntGetRegion = 0 + s.cache.clear() + store2 := s.cache.getStoreOrInsertDefault(s.store2) + atomic.StoreUint32(&store2.livenessState, uint32(unreachable)) + defer atomic.StoreUint32(&store2.livenessState, uint32(reachable)) + + for i := 0; i < 50; i++ { + time.Sleep(100 * time.Millisecond) + _, err := s.cache.LocateKey(s.bo, []byte("a")) + s.NoError(err) + } + s.Equal(2, cntGetRegion, "should reload region with unreachable stores every RegionCacheTTL") + }) +} + +func (s *testRegionCacheSuite) TestTiFlashRecoveredFromDown() { + s.onClosed = func() { SetRegionCacheTTLSec(600) } + SetRegionCacheTTLSec(3) + store3 := s.cluster.AllocID() peer3 := s.cluster.AllocID() s.cluster.AddStore(store3, s.storeAddr(store3)) @@ -313,34 +388,45 @@ func (s *testRegionCacheSuite) TestTiFlashDownPeersAndAsyncReload() { s.Nil(err) s.NotNil(ctx) region := s.cache.GetCachedRegionWithRLock(loc.Region) - s.Equal(region.hasUnavailableTiFlashStore, false) - s.Equal(region.asyncReload.Load(), false) + s.Equal(region.checkSyncFlags(needExpireAfterTTL), false) s.cache.clear() s.cluster.MarkPeerDown(peer3) - s.cache.reloadRegion(loc.Region.id) loc, err = s.cache.LocateKey(s.bo, []byte("a")) s.Nil(err) s.Equal(loc.Region.id, s.region1) region = s.cache.GetCachedRegionWithRLock(loc.Region) - s.Equal(region.hasUnavailableTiFlashStore, true) - s.Equal(region.asyncReload.Load(), false) + s.Equal(region.checkSyncFlags(needExpireAfterTTL), true) - SetRegionCacheTTLSec(3) - var wg sync.WaitGroup - wg.Add(1) - go func() { - defer wg.Done() - for i := 0; i <= 3; i++ { - s.cache.GetTiFlashRPCContext(s.bo, loc.Region, true, LabelFilterNoTiFlashWriteNode) - time.Sleep(1 * time.Second) + for i := 0; i <= 3; i++ { + time.Sleep(1 * time.Second) + loc, err = s.cache.LocateKey(s.bo, []byte("a")) + s.Nil(err) + rpcCtx, err := s.cache.GetTiFlashRPCContext(s.bo, loc.Region, true, LabelFilterNoTiFlashWriteNode) + s.Nil(err) + if rpcCtx != nil { + s.NotEqual(s.storeAddr(store3), rpcCtx.Addr, "should not access peer3 when it is down") } - }() - wg.Wait() - s.cache.GetTiFlashRPCContext(s.bo, loc.Region, true, LabelFilterNoTiFlashWriteNode) - s.Equal(region.hasUnavailableTiFlashStore, true) - s.Equal(region.asyncReload.Load(), true) + } + newRegion := s.cache.GetCachedRegionWithRLock(loc.Region) + s.NotNil(newRegion) + s.NotEqual(region, newRegion) + s.cluster.RemoveDownPeer(peer3) + for i := 0; ; i++ { + if i > 10 { + s.Fail("should access peer3 after it is up") + break + } + loc, err = s.cache.LocateKey(s.bo, []byte("a")) + s.Nil(err) + rpcCtx, err := s.cache.GetTiFlashRPCContext(s.bo, loc.Region, true, LabelFilterNoTiFlashWriteNode) + s.Nil(err) + if rpcCtx != nil && rpcCtx.Addr == s.storeAddr(store3) { + break + } + time.Sleep(1 * time.Second) + } } // TestFilterDownPeersOrPeersOnTombstoneOrDroppedStore verifies the RegionCache filter @@ -1306,7 +1392,6 @@ func (s *testRegionCacheSuite) TestPeersLenChange() { Meta: cpMeta, DownPeers: []*metapb.Peer{{Id: s.peer1, StoreId: s.store1}}, } - filterUnavailablePeers(cpRegion) region, err := newRegion(s.bo, s.cache, cpRegion) s.Nil(err) s.cache.insertRegionToCache(region, true, true) @@ -1511,7 +1596,7 @@ func (s *testRegionCacheSuite) TestBuckets() { // 2. insertRegionToCache keeps old buckets information if needed. fakeRegion := &Region{ meta: cachedRegion.meta, - syncFlag: cachedRegion.syncFlag, + syncFlags: cachedRegion.syncFlags, lastAccess: cachedRegion.lastAccess, invalidReason: cachedRegion.invalidReason, } @@ -1920,7 +2005,7 @@ func (s *testRegionCacheWithDelaySuite) TestInsertStaleRegion() { s.NoError(err) fakeRegion := &Region{ meta: r.meta, - syncFlag: r.syncFlag, + syncFlags: r.syncFlags, lastAccess: r.lastAccess, invalidReason: r.invalidReason, } @@ -2053,11 +2138,10 @@ func BenchmarkInsertRegionToCache(b *testing.B) { }, } rs := ®ionStore{ - workTiKVIdx: 0, - proxyTiKVIdx: -1, - stores: make([]*Store, 0, len(r.meta.Peers)), - pendingTiFlashPeerStores: map[uint64]uint64{}, - storeEpochs: make([]uint32, 0, len(r.meta.Peers)), + workTiKVIdx: 0, + proxyTiKVIdx: -1, + stores: make([]*Store, 0, len(r.meta.Peers)), + storeEpochs: make([]uint32, 0, len(r.meta.Peers)), } r.setStore(rs) b.StartTimer() @@ -2091,11 +2175,10 @@ func BenchmarkInsertRegionToCache2(b *testing.B) { }, } rs := ®ionStore{ - workTiKVIdx: 0, - proxyTiKVIdx: -1, - stores: make([]*Store, 0, len(r.meta.Peers)), - pendingTiFlashPeerStores: map[uint64]uint64{}, - storeEpochs: make([]uint32, 0, len(r.meta.Peers)), + workTiKVIdx: 0, + proxyTiKVIdx: -1, + stores: make([]*Store, 0, len(r.meta.Peers)), + storeEpochs: make([]uint32, 0, len(r.meta.Peers)), } r.setStore(rs) b.StartTimer() diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index ebb5c7dfb5..d839d53422 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -607,7 +607,7 @@ func (state *tryNewProxy) next(bo *retry.Backoffer, selector *replicaSelector) ( if candidateNum == 0 { metrics.TiKVReplicaSelectorFailureCounter.WithLabelValues("exhausted").Inc() selector.invalidateReplicaStore(leader, errors.Errorf("all followers are tried as proxy but fail")) - selector.region.scheduleReload() + selector.region.setSyncFlags(needReloadOnAccess) return nil, nil } @@ -727,7 +727,7 @@ func (state *accessFollower) next(bo *retry.Backoffer, selector *replicaSelector } } if reloadRegion { - selector.regionCache.scheduleReloadRegion(selector.region) + selector.region.setSyncFlags(needDelayedReloadPending) } // If there is no candidate, fallback to the leader. if selector.targetIdx < 0 { @@ -760,7 +760,7 @@ func (state *accessFollower) next(bo *retry.Backoffer, selector *replicaSelector labels: state.option.labels, } if leaderEpochStale { - selector.regionCache.scheduleReloadRegion(selector.region) + selector.region.setSyncFlags(needDelayedReloadPending) } return nil, stateChanged{} } @@ -918,7 +918,7 @@ func newReplicaSelector( cachedRegion := regionCache.GetCachedRegionWithRLock(regionID) if cachedRegion == nil { return nil, errors.New("cached region not found") - } else if cachedRegion.checkNeedReload() { + } else if cachedRegion.checkSyncFlags(needReloadOnAccess) { return nil, errors.New("cached region need reload") } else if !cachedRegion.checkRegionCacheTTL(time.Now().Unix()) { return nil, errors.New("cached region ttl expired") diff --git a/internal/locate/region_request3_test.go b/internal/locate/region_request3_test.go index db83000221..b2562b4858 100644 --- a/internal/locate/region_request3_test.go +++ b/internal/locate/region_request3_test.go @@ -77,6 +77,7 @@ type testRegionRequestToThreeStoresSuite struct { bo *retry.Backoffer regionRequestSender *RegionRequestSender mvccStore mocktikv.MVCCStore + onClosed func() } func (s *testRegionRequestToThreeStoresSuite) SetupTest() { @@ -93,6 +94,9 @@ func (s *testRegionRequestToThreeStoresSuite) SetupTest() { func (s *testRegionRequestToThreeStoresSuite) TearDownTest() { s.cache.Close() s.mvccStore.Close() + if s.onClosed != nil { + s.onClosed() + } } func (s *testRegionRequestToThreeStoresSuite) TestStoreTokenLimit() { @@ -1841,3 +1845,46 @@ func (s *testRegionRequestToThreeStoresSuite) TestLeaderStuck() { s.Less(elapsed, time.Millisecond*2500) s.True(requestHandled) } + +func (s *testRegionRequestToThreeStoresSuite) TestTiKVRecoveredFromDown() { + s.onClosed = func() { SetRegionCacheTTLSec(600) } + SetRegionCacheTTLSec(2) + + bo := retry.NewBackoffer(context.Background(), -1) + key := []byte("key") + + req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{Key: key}, kv.ReplicaReadMixed, nil) + req.ReadReplicaScope = oracle.GlobalTxnScope + req.TxnScope = oracle.GlobalTxnScope + + downStore := s.cluster.GetStore(s.storeIDs[2]) + s.cluster.MarkPeerDown(s.peerIDs[2]) + s.regionRequestSender.client = &fnClient{fn: func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { + s.Require().NotEqual(addr, downStore.Address) + return &tikvrpc.Response{Resp: &kvrpcpb.GetResponse{Value: []byte(addr)}}, nil + }} + for i := 0; i < 15; i++ { + time.Sleep(200 * time.Millisecond) + loc, err := s.cache.LocateKey(bo, key) + s.Require().Nil(err) + resp, rpcCtx, _, err := s.regionRequestSender.SendReqCtx(bo, req, loc.Region, time.Second, tikvrpc.TiKV, WithMatchLabels(downStore.Labels)) + s.Require().Nil(err) + s.Require().Equal(rpcCtx.Addr, string(resp.Resp.(*kvrpcpb.GetResponse).Value), "should access other peers") + } + + s.cluster.RemoveDownPeer(s.peerIDs[2]) + s.regionRequestSender.client = &fnClient{fn: func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { + return &tikvrpc.Response{Resp: &kvrpcpb.GetResponse{Value: []byte(addr)}}, nil + }} + for i := 0; i < 15; i++ { + time.Sleep(200 * time.Millisecond) + loc, err := s.cache.LocateKey(bo, key) + s.Require().Nil(err) + _, rpcCtx, _, err := s.regionRequestSender.SendReqCtx(bo, req, loc.Region, time.Second, tikvrpc.TiKV, WithMatchLabels(downStore.Labels)) + s.Require().Nil(err) + if rpcCtx.Addr == downStore.Address { + return + } + } + s.Require().Fail("should access recovered peer after region reloading within RegionCacheTTL") +} diff --git a/internal/locate/region_request_state_test.go b/internal/locate/region_request_state_test.go index 3636355e23..f4a9881b71 100644 --- a/internal/locate/region_request_state_test.go +++ b/internal/locate/region_request_state_test.go @@ -250,13 +250,10 @@ func (s *testRegionCacheStaleReadSuite) setTimeout(id uint64) { //nolint: unused } func TestRegionCacheStaleRead(t *testing.T) { - originReloadRegionInterval := atomic.LoadInt64(&reloadRegionInterval) originBoTiKVServerBusy := retry.BoTiKVServerBusy defer func() { - atomic.StoreInt64(&reloadRegionInterval, originReloadRegionInterval) retry.BoTiKVServerBusy = originBoTiKVServerBusy }() - atomic.StoreInt64(&reloadRegionInterval, int64(24*time.Hour)) // disable reload region retry.BoTiKVServerBusy = retry.NewConfig("tikvServerBusy", &metrics.BackoffHistogramServerBusy, retry.NewBackoffFnCfg(2, 10, retry.EqualJitter), tikverr.ErrTiKVServerBusy) regionCacheTestCases := []RegionCacheTestCase{ { @@ -581,14 +578,7 @@ func testStaleRead(s *testRegionCacheStaleReadSuite, r *RegionCacheTestCase, zon return } - s.cache.regionsNeedReload.Lock() - if *asyncReload { - s.Len(s.cache.regionsNeedReload.regions, 1) - s.Equal(s.cache.regionsNeedReload.regions[0], s.regionID) - } else { - s.Empty(s.cache.regionsNeedReload.regions) - } - s.cache.regionsNeedReload.Unlock() + s.Equal(*asyncReload, region.checkSyncFlags(needDelayedReloadPending)) }() bo := retry.NewBackoffer(ctx, -1) From 70c148e84e94befab85875f7306fd805281b5197 Mon Sep 17 00:00:00 2001 From: ekexium Date: Fri, 2 Feb 2024 10:26:02 +0800 Subject: [PATCH 3/8] ErrQueryInterrupted with parameters (#1124) * feat: ErrQueryInterrupted with parameters Signed-off-by: ekexium * Revert "Revert "fix: check kill signal against 0 (#1102)" (#1129)" This reverts commit 3480b5ed7ce131ed39f2e6ffcdec603f4166e225. Signed-off-by: ekexium --------- Signed-off-by: ekexium Co-authored-by: cfzjywxk --- config/retry/backoff.go | 21 +++++++++++++++++---- error/error.go | 11 ++++++++++- integration_tests/2pc_test.go | 10 ++++++++++ internal/client/retry/backoff.go | 27 +++++++++++++++++++++------ internal/locate/region_request.go | 5 ++--- kv/variables.go | 4 ++++ txnkv/transaction/2pc.go | 2 +- 7 files changed, 65 insertions(+), 15 deletions(-) diff --git a/config/retry/backoff.go b/config/retry/backoff.go index a2723e05b3..c18577ad0c 100644 --- a/config/retry/backoff.go +++ b/config/retry/backoff.go @@ -217,10 +217,9 @@ func (b *Backoffer) BackoffWithCfgAndMaxSleep(cfg *Config, maxSleepMs int, err e atomic.AddInt64(&detail.BackoffCount, 1) } - if b.vars != nil && b.vars.Killed != nil { - if atomic.LoadUint32(b.vars.Killed) == 1 { - return errors.WithStack(tikverr.ErrQueryInterrupted) - } + err2 := b.CheckKilled() + if err2 != nil { + return err2 } var startTs interface{} @@ -382,3 +381,17 @@ func (b *Backoffer) longestSleepCfg() (*Config, int) { } return nil, 0 } + +func (b *Backoffer) CheckKilled() error { + if b.vars != nil && b.vars.Killed != nil { + killed := atomic.LoadUint32(b.vars.Killed) + if killed != 0 { + logutil.BgLogger().Info( + "backoff stops because a killed signal is received", + zap.Uint32("signal", killed), + ) + return errors.WithStack(tikverr.ErrQueryInterruptedWithSignal{Signal: killed}) + } + } + return nil +} diff --git a/error/error.go b/error/error.go index 3f10c211f9..14f29a440c 100644 --- a/error/error.go +++ b/error/error.go @@ -64,6 +64,7 @@ var ( // ErrTiFlashServerTimeout is the error when tiflash server is timeout. ErrTiFlashServerTimeout = errors.New("tiflash server timeout") // ErrQueryInterrupted is the error when the query is interrupted. + // This is deprecated. Keep it only to pass CI :-(. We can remove this later. ErrQueryInterrupted = errors.New("query interrupted") // ErrTiKVStaleCommand is the error that the command is stale in tikv. ErrTiKVStaleCommand = errors.New("tikv stale command") @@ -96,11 +97,19 @@ var ( // ErrIsWitness is the error when a request is send to a witness. ErrIsWitness = errors.New("peer is witness") // ErrUnknown is the unknow error. - ErrUnknown = errors.New("unknow") + ErrUnknown = errors.New("unknown") // ErrResultUndetermined is the error when execution result is unknown. ErrResultUndetermined = errors.New("execution result undetermined") ) +type ErrQueryInterruptedWithSignal struct { + Signal uint32 +} + +func (e ErrQueryInterruptedWithSignal) Error() string { + return fmt.Sprintf("query interrupted by signal %d", e.Signal) +} + // MismatchClusterID represents the message that the cluster ID of the PD client does not match the PD. const MismatchClusterID = "mismatch cluster id" diff --git a/integration_tests/2pc_test.go b/integration_tests/2pc_test.go index 355dafeae1..9369bfb666 100644 --- a/integration_tests/2pc_test.go +++ b/integration_tests/2pc_test.go @@ -2502,3 +2502,13 @@ func (s *testCommitterSuite) TestExtractKeyExistsErr() { s.True(txn.GetMemBuffer().TryLock()) txn.GetMemBuffer().Unlock() } + +func (s *testCommitterSuite) TestKillSignal() { + txn := s.begin() + err := txn.Set([]byte("key"), []byte("value")) + s.Nil(err) + var killed uint32 = 2 + txn.SetVars(kv.NewVariables(&killed)) + err = txn.Commit(context.Background()) + s.ErrorContains(err, "query interrupted") +} diff --git a/internal/client/retry/backoff.go b/internal/client/retry/backoff.go index a2723e05b3..229d9766cb 100644 --- a/internal/client/retry/backoff.go +++ b/internal/client/retry/backoff.go @@ -217,23 +217,38 @@ func (b *Backoffer) BackoffWithCfgAndMaxSleep(cfg *Config, maxSleepMs int, err e atomic.AddInt64(&detail.BackoffCount, 1) } - if b.vars != nil && b.vars.Killed != nil { - if atomic.LoadUint32(b.vars.Killed) == 1 { - return errors.WithStack(tikverr.ErrQueryInterrupted) - } + err2 := b.checkKilled() + if err2 != nil { + return err2 } var startTs interface{} if ts := b.ctx.Value(TxnStartKey); ts != nil { startTs = ts } - logutil.Logger(b.ctx).Debug("retry later", + logutil.Logger(b.ctx).Debug( + "retry later", zap.Error(err), zap.Int("totalSleep", b.totalSleep), zap.Int("excludedSleep", b.excludedSleep), zap.Int("maxSleep", b.maxSleep), zap.Stringer("type", cfg), - zap.Reflect("txnStartTS", startTs)) + zap.Reflect("txnStartTS", startTs), + ) + return nil +} + +func (b *Backoffer) checkKilled() error { + if b.vars != nil && b.vars.Killed != nil { + killed := atomic.LoadUint32(b.vars.Killed) + if killed != 0 { + logutil.BgLogger().Info( + "backoff stops because a killed signal is received", + zap.Uint32("signal", killed), + ) + return errors.WithStack(tikverr.ErrQueryInterruptedWithSignal{Signal: killed}) + } + } return nil } diff --git a/internal/locate/region_request.go b/internal/locate/region_request.go index d839d53422..efcafbf162 100644 --- a/internal/locate/region_request.go +++ b/internal/locate/region_request.go @@ -1499,9 +1499,8 @@ func (s *RegionRequestSender) SendReqCtx( } // recheck whether the session/query is killed during the Next() - boVars := bo.GetVars() - if boVars != nil && boVars.Killed != nil && atomic.LoadUint32(boVars.Killed) == 1 { - return nil, nil, retryTimes, errors.WithStack(tikverr.ErrQueryInterrupted) + if err2 := bo.CheckKilled(); err2 != nil { + return nil, nil, retryTimes, err2 } if val, err := util.EvalFailpoint("mockRetrySendReqToRegion"); err == nil { if val.(bool) { diff --git a/kv/variables.go b/kv/variables.go index 581be54d04..cae78c9c59 100644 --- a/kv/variables.go +++ b/kv/variables.go @@ -44,6 +44,10 @@ type Variables struct { // Pointer to SessionVars.Killed // Killed is a flag to indicate that this query is killed. + // This is an enum value rather than a boolean. See sqlkiller.go + // in TiDB for its definition. + // When its value is 0, it's not killed + // When its value is not 0, it's killed, the value indicates concrete reason. Killed *uint32 } diff --git a/txnkv/transaction/2pc.go b/txnkv/transaction/2pc.go index 866d320b36..dc281d65a9 100644 --- a/txnkv/transaction/2pc.go +++ b/txnkv/transaction/2pc.go @@ -1066,7 +1066,7 @@ func (c *twoPhaseCommitter) doActionOnBatches( ) // TODO: There might be various signals besides a query interruption, // but we are unable to differentiate them, because the definition is in TiDB. - return errors.WithStack(tikverr.ErrQueryInterrupted) + return errors.WithStack(tikverr.ErrQueryInterruptedWithSignal{Signal: status}) } } if len(batches) == 0 { From 8b3b01e8d770bbe918ce64d010103f3ad66f28e0 Mon Sep 17 00:00:00 2001 From: zyguan Date: Fri, 2 Feb 2024 20:13:23 +0800 Subject: [PATCH 4/8] introduce a random jitter to region cache ttl (#1148) * introduce a random jitter to region cache ttl Signed-off-by: zyguan * refactor searching cached region Signed-off-by: zyguan * observe load region by reason Signed-off-by: zyguan * address the comment Signed-off-by: zyguan --------- Signed-off-by: zyguan --- internal/locate/region_cache.go | 218 +++++++++++++------ internal/locate/region_cache_test.go | 34 +-- internal/locate/region_request3_test.go | 33 +-- internal/locate/region_request_state_test.go | 14 +- internal/locate/region_request_test.go | 17 +- internal/locate/sorted_btree.go | 16 +- metrics/metrics.go | 11 + tikv/region.go | 8 +- 8 files changed, 233 insertions(+), 118 deletions(-) diff --git a/internal/locate/region_cache.go b/internal/locate/region_cache.go index 3b802f4525..e8f652e00f 100644 --- a/internal/locate/region_cache.go +++ b/internal/locate/region_cache.go @@ -56,6 +56,7 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" "github.com/tikv/client-go/v2/config" "github.com/tikv/client-go/v2/config/retry" tikverr "github.com/tikv/client-go/v2/error" @@ -81,9 +82,9 @@ import ( ) const ( - btreeDegree = 32 - invalidatedLastAccessTime = -1 - defaultRegionsPerBatch = 128 + btreeDegree = 32 + expiredTTL = -1 + defaultRegionsPerBatch = 128 ) // LabelFilter returns false means label doesn't match, and will ignore this store. @@ -121,6 +122,29 @@ func SetRegionCacheTTLSec(t int64) { regionCacheTTLSec = t } +// regionCacheTTLJitterSec is the max jitter time for region cache TTL. +var regionCacheTTLJitterSec int64 = 60 + +// SetRegionCacheTTLWithJitter sets region cache TTL with jitter. The real TTL is in range of [base, base+jitter). +func SetRegionCacheTTLWithJitter(base int64, jitter int64) { + regionCacheTTLSec = base + regionCacheTTLJitterSec = jitter +} + +// nextTTL returns a random TTL in range [ts+base, ts+base+jitter). The input ts should be an epoch timestamp in seconds. +func nextTTL(ts int64) int64 { + jitter := int64(0) + if regionCacheTTLJitterSec > 0 { + jitter = rand.Int63n(regionCacheTTLJitterSec) + } + return ts + regionCacheTTLSec + jitter +} + +// nextTTLWithoutJitter is used for test. +func nextTTLWithoutJitter(ts int64) int64 { + return ts + regionCacheTTLSec +} + const ( needReloadOnAccess int32 = 1 << iota // indicates the region will be reloaded on next access needExpireAfterTTL // indicates the region will expire after RegionCacheTTL (even when it's accessed continuously) @@ -150,11 +174,30 @@ const ( Other ) +func (r InvalidReason) String() string { + switch r { + case Ok: + return "Ok" + case Other: + return "Other" + case EpochNotMatch: + return "EpochNotMatch" + case RegionNotFound: + return "RegionNotFound" + case StoreNotFound: + return "StoreNotFound" + case NoLeader: + return "NoLeader" + default: + return "Unknown" + } +} + // Region presents kv region type Region struct { meta *metapb.Region // raw region meta from PD, immutable after init store unsafe.Pointer // point to region store info, see RegionStore - lastAccess int64 // last region access time, see checkRegionCacheTTL + ttl int64 // region TTL in epoch seconds, see checkRegionCacheTTL syncFlags int32 // region need be sync later, see needReloadOnAccess, needExpireAfterTTL invalidReason InvalidReason // the reason why the region is invalidated } @@ -338,7 +381,7 @@ func newRegion(bo *retry.Backoffer, c *RegionCache, pdRegion *pd.Region) (*Regio } // mark region has been init accessed. - r.lastAccess = time.Now().Unix() + r.ttl = nextTTL(time.Now().Unix()) return r, nil } @@ -356,8 +399,7 @@ func (r *Region) compareAndSwapStore(oldStore, newStore *regionStore) bool { } func (r *Region) isCacheTTLExpired(ts int64) bool { - lastAccess := atomic.LoadInt64(&r.lastAccess) - return ts-lastAccess > regionCacheTTLSec + return ts > atomic.LoadInt64(&r.ttl) } // checkRegionCacheTTL returns false means the region cache is expired. @@ -366,28 +408,36 @@ func (r *Region) checkRegionCacheTTL(ts int64) bool { if _, err := util.EvalFailpoint("invalidateRegionCache"); err == nil { r.invalidate(Other) } + newTTL := int64(0) for { - lastAccess := atomic.LoadInt64(&r.lastAccess) - if ts-lastAccess > regionCacheTTLSec { + ttl := atomic.LoadInt64(&r.ttl) + if ts > ttl { return false } - if r.checkSyncFlags(needExpireAfterTTL) || atomic.CompareAndSwapInt64(&r.lastAccess, lastAccess, ts) { + // skip updating TTL when: + // 1. the region has been marked as `needExpireAfterTTL` + // 2. the TTL is far away from ts (still within jitter time) + if r.checkSyncFlags(needExpireAfterTTL) || ttl > ts+regionCacheTTLSec { + return true + } + if newTTL == 0 { + newTTL = nextTTL(ts) + } + // now we have ts <= ttl <= ts+regionCacheTTLSec <= newTTL = ts+regionCacheTTLSec+randomJitter + if atomic.CompareAndSwapInt64(&r.ttl, ttl, newTTL) { return true } } } // invalidate invalidates a region, next time it will got null result. -func (r *Region) invalidate(reason InvalidReason) { - metrics.RegionCacheCounterWithInvalidateRegionFromCacheOK.Inc() - atomic.StoreInt32((*int32)(&r.invalidReason), int32(reason)) - atomic.StoreInt64(&r.lastAccess, invalidatedLastAccessTime) -} - -// invalidateWithoutMetrics invalidates a region without metrics, next time it will got null result. -func (r *Region) invalidateWithoutMetrics(reason InvalidReason) { - atomic.StoreInt32((*int32)(&r.invalidReason), int32(reason)) - atomic.StoreInt64(&r.lastAccess, invalidatedLastAccessTime) +func (r *Region) invalidate(reason InvalidReason, nocount ...bool) { + if atomic.CompareAndSwapInt32((*int32)(&r.invalidReason), int32(Ok), int32(reason)) { + if len(nocount) == 0 || !nocount[0] { + metrics.RegionCacheCounterWithInvalidateRegionFromCacheOK.Inc() + } + atomic.StoreInt64(&r.ttl, expiredTTL) + } } func (r *Region) getSyncFlags() int32 { @@ -1068,9 +1118,15 @@ func (c *RegionCache) LocateEndKey(bo *retry.Backoffer, key []byte) (*KeyLocatio } func (c *RegionCache) findRegionByKey(bo *retry.Backoffer, key []byte, isEndKey bool) (r *Region, err error) { - r = c.searchCachedRegion(key, isEndKey) - if r == nil { + var expired bool + r, expired = c.searchCachedRegionByKey(key, isEndKey) + tag := "ByKey" + if isEndKey { + tag = "ByEndKey" + } + if r == nil || expired { // load region when it is not exists or expired. + observeLoadRegion(tag, r, expired, 0) lr, err := c.loadRegion(bo, key, isEndKey, pd.WithAllowFollowerHandle()) if err != nil { // no region data, return error if failure. @@ -1083,6 +1139,7 @@ func (c *RegionCache) findRegionByKey(bo *retry.Backoffer, key []byte, isEndKey c.mu.Unlock() // just retry once, it won't bring much overhead. if stale { + observeLoadRegion(tag+":Retry", r, expired, 0) lr, err = c.loadRegion(bo, key, isEndKey) if err != nil { // no region data, return error if failure. @@ -1101,8 +1158,10 @@ func (c *RegionCache) findRegionByKey(bo *retry.Backoffer, key []byte, isEndKey err error ) if reloadOnAccess { + observeLoadRegion(tag, r, expired, flags) lr, err = c.loadRegion(bo, key, isEndKey) } else { + observeLoadRegion("ByID", r, expired, flags) lr, err = c.loadRegionByID(bo, r.GetID()) } if err != nil { @@ -1122,8 +1181,9 @@ func (c *RegionCache) findRegionByKey(bo *retry.Backoffer, key []byte, isEndKey } func (c *RegionCache) tryFindRegionByKey(key []byte, isEndKey bool) (r *Region) { - r = c.searchCachedRegion(key, isEndKey) - if r == nil || r.checkSyncFlags(needReloadOnAccess) { + var expired bool + r, expired = c.searchCachedRegionByKey(key, isEndKey) + if r == nil || expired || r.checkSyncFlags(needReloadOnAccess) { return nil } return r @@ -1242,12 +1302,11 @@ func (c *RegionCache) OnSendFail(bo *retry.Backoffer, ctx *RPCContext, scheduleR // LocateRegionByID searches for the region with ID. func (c *RegionCache) LocateRegionByID(bo *retry.Backoffer, regionID uint64) (*KeyLocation, error) { - c.mu.RLock() - r := c.getRegionByIDFromCache(regionID) - c.mu.RUnlock() - if r != nil { - if flags := r.resetSyncFlags(needReloadOnAccess); flags > 0 { + r, expired := c.searchCachedRegionByID(regionID) + if r != nil && !expired { + if flags := r.resetSyncFlags(needReloadOnAccess | needDelayedReloadReady); flags > 0 { reloadOnAccess := flags&needReloadOnAccess > 0 + observeLoadRegion("ByID", r, expired, flags) lr, err := c.loadRegionByID(bo, regionID) if err != nil { // ignore error and use old region info. @@ -1269,6 +1328,7 @@ func (c *RegionCache) LocateRegionByID(bo *retry.Backoffer, regionID uint64) (*K return loc, nil } + observeLoadRegion("ByID", r, expired, 0) r, err := c.loadRegionByID(bo, regionID) if err != nil { return nil, err @@ -1500,11 +1560,7 @@ func (mu *regionIndexMu) insertRegionToCache(cachedRegion *Region, invalidateOld // If the old region is still valid, do not invalidate it to avoid unnecessary backoff. if invalidateOldRegion { // Invalidate the old region in case it's not invalidated and some requests try with the stale region information. - if shouldCount { - region.cachedRegion.invalidate(Other) - } else { - region.cachedRegion.invalidateWithoutMetrics(Other) - } + region.cachedRegion.invalidate(Other, !shouldCount) } } // update related vars. @@ -1513,47 +1569,33 @@ func (mu *regionIndexMu) insertRegionToCache(cachedRegion *Region, invalidateOld return true } -// searchCachedRegion finds a region from cache by key. Like `getCachedRegion`, -// it should be called with c.mu.RLock(), and the returned Region should not be -// used after c.mu is RUnlock(). -// If the given key is the end key of the region that you want, you may set the second argument to true. This is useful -// when processing in reverse order. -func (c *RegionCache) searchCachedRegion(key []byte, isEndKey bool) *Region { - ts := time.Now().Unix() - var r *Region +// searchCachedRegionByKey finds the region from cache by key. +func (c *RegionCache) searchCachedRegionByKey(key []byte, isEndKey bool) (*Region, bool) { c.mu.RLock() - r = c.mu.sorted.DescendLessOrEqual(key, isEndKey, ts) + region := c.mu.sorted.SearchByKey(key, isEndKey) c.mu.RUnlock() - if r != nil && (!isEndKey && r.Contains(key) || isEndKey && r.ContainsByEnd(key)) { - return r + if region == nil { + return nil, false } - return nil + return region, !region.checkRegionCacheTTL(time.Now().Unix()) } -// getRegionByIDFromCache tries to get region by regionID from cache. Like -// `getCachedRegion`, it should be called with c.mu.RLock(), and the returned -// Region should not be used after c.mu is RUnlock(). -func (c *RegionCache) getRegionByIDFromCache(regionID uint64) *Region { - ts := time.Now().Unix() +// searchCachedRegionByID finds the region from cache by id. +func (c *RegionCache) searchCachedRegionByID(regionID uint64) (*Region, bool) { + c.mu.RLock() ver, ok := c.mu.latestVersions[regionID] if !ok { - return nil + c.mu.RUnlock() + return nil, false } - latestRegion, ok := c.mu.regions[ver] + region, ok := c.mu.regions[ver] + c.mu.RUnlock() if !ok { // should not happen - logutil.BgLogger().Warn("region version not found", - zap.Uint64("regionID", regionID), zap.Stringer("version", &ver)) - return nil - } - lastAccess := atomic.LoadInt64(&latestRegion.lastAccess) - if ts-lastAccess > regionCacheTTLSec { - return nil - } - if !latestRegion.checkSyncFlags(needExpireAfterTTL) { - atomic.CompareAndSwapInt64(&latestRegion.lastAccess, atomic.LoadInt64(&latestRegion.lastAccess), ts) + logutil.BgLogger().Warn("region not found", zap.Uint64("id", regionID), zap.Stringer("ver", &ver)) + return nil, false } - return latestRegion + return region, !region.checkRegionCacheTTL(time.Now().Unix()) } // GetStoresByType gets stores by type `typ` @@ -1570,6 +1612,53 @@ func (c *RegionCache) GetAllStores() []*Store { }) } +var loadRegionCounters sync.Map + +const ( + loadRegionReasonMissing = "Missing" + loadRegionReasonExpiredNormal = "Expired:Normal" + loadRegionReasonExpiredFrozen = "Expired:Frozen" + loadRegionReasonExpiredInvalid = "Expired:Invalid:" + loadRegionReasonReloadOnAccess = "Reload:OnAccess" + loadRegionReasonReloadDelayed = "Reload:Delayed" + loadRegionReasonUpdateBuckets = "UpdateBuckets" + loadRegionReasonUnknown = "Unknown" +) + +func observeLoadRegion(tag string, region *Region, expired bool, reloadFlags int32, explicitReason ...string) { + reason := loadRegionReasonUnknown + if len(explicitReason) > 0 { + reason = strings.Join(explicitReason, ":") + } else if region == nil { + reason = loadRegionReasonMissing + } else if expired { + invalidReason := InvalidReason(atomic.LoadInt32((*int32)(®ion.invalidReason))) + if invalidReason != Ok { + reason = loadRegionReasonExpiredInvalid + invalidReason.String() + } else if region.checkSyncFlags(needExpireAfterTTL) { + reason = loadRegionReasonExpiredFrozen + } else { + reason = loadRegionReasonExpiredNormal + } + } else if reloadFlags > 0 { + if reloadFlags&needReloadOnAccess > 0 { + reason = loadRegionReasonReloadOnAccess + } else if reloadFlags&needDelayedReloadReady > 0 { + reason = loadRegionReasonReloadDelayed + } + } + type key struct { + t string + r string + } + counter, ok := loadRegionCounters.Load(key{tag, reason}) + if !ok { + counter = metrics.TiKVLoadRegionCounter.WithLabelValues(tag, reason) + loadRegionCounters.Store(key{tag, reason}, counter) + } + counter.(prometheus.Counter).Inc() +} + // loadRegion loads region from pd client, and picks the first peer as leader. // If the given key is the end key of the region that you want, you may set the second argument to true. This is useful // when processing in reverse order. @@ -2062,6 +2151,7 @@ func (c *RegionCache) UpdateBucketsIfNeeded(regionID RegionVerID, latestBucketsV // TODO(youjiali1995): use singleflight. go func() { bo := retry.NewBackoffer(context.Background(), 20000) + observeLoadRegion("ByID", r, false, 0, loadRegionReasonUpdateBuckets) new, err := c.loadRegionByID(bo, regionID.id) if err != nil { logutil.Logger(bo.GetCtx()).Error("failed to update buckets", diff --git a/internal/locate/region_cache_test.go b/internal/locate/region_cache_test.go index 2f09239d3c..2d6d9aab3c 100644 --- a/internal/locate/region_cache_test.go +++ b/internal/locate/region_cache_test.go @@ -149,7 +149,8 @@ func validRegions(regions map[RegionVerID]*Region, ts int64) (len int) { func (s *testRegionCacheSuite) getRegion(key []byte) *Region { _, err := s.cache.LocateKey(s.bo, key) s.Nil(err) - r := s.cache.searchCachedRegion(key, false) + r, expired := s.cache.searchCachedRegionByKey(key, false) + s.False(expired) s.NotNil(r) return r } @@ -157,7 +158,8 @@ func (s *testRegionCacheSuite) getRegion(key []byte) *Region { func (s *testRegionCacheSuite) getRegionWithEndKey(key []byte) *Region { _, err := s.cache.LocateEndKey(s.bo, key) s.Nil(err) - r := s.cache.searchCachedRegion(key, true) + r, expired := s.cache.searchCachedRegionByKey(key, true) + s.False(expired) s.NotNil(r) return r } @@ -211,9 +213,11 @@ func (s *testRegionCacheSuite) TestSimple() { s.checkCache(1) s.Equal(r.GetMeta(), r.meta) s.Equal(r.GetLeaderPeerID(), r.meta.Peers[r.getStore().workTiKVIdx].Id) - s.cache.mu.regions[r.VerID()].lastAccess = 0 - r = s.cache.searchCachedRegion([]byte("a"), true) - s.Nil(r) + s.cache.mu.regions[r.VerID()].ttl = 0 + var expired bool + r, expired = s.cache.searchCachedRegionByKey([]byte("a"), true) + s.True(expired) + s.NotNil(r) } // TestResolveStateTransition verifies store's resolve state transition. For example, @@ -310,8 +314,8 @@ func (s *testRegionCacheSuite) TestResolveStateTransition() { } func (s *testRegionCacheSuite) TestNeedExpireRegionAfterTTL() { - s.onClosed = func() { SetRegionCacheTTLSec(600) } - SetRegionCacheTTLSec(2) + s.onClosed = func() { SetRegionCacheTTLWithJitter(600, 60) } + SetRegionCacheTTLWithJitter(2, 0) cntGetRegion := 0 s.cache.pdClient = &inspectedPDClient{ @@ -366,8 +370,8 @@ func (s *testRegionCacheSuite) TestNeedExpireRegionAfterTTL() { } func (s *testRegionCacheSuite) TestTiFlashRecoveredFromDown() { - s.onClosed = func() { SetRegionCacheTTLSec(600) } - SetRegionCacheTTLSec(3) + s.onClosed = func() { SetRegionCacheTTLWithJitter(600, 60) } + SetRegionCacheTTLWithJitter(3, 0) store3 := s.cluster.AllocID() peer3 := s.cluster.AllocID() @@ -1508,7 +1512,7 @@ func BenchmarkOnRequestFail(b *testing.B) { if err != nil { b.Fatal(err) } - region := cache.getRegionByIDFromCache(loc.Region.id) + region, _ := cache.searchCachedRegionByID(loc.Region.id) b.ResetTimer() regionStore := region.getStore() store, peer, accessIdx, _ := region.WorkStorePeer(regionStore) @@ -1597,7 +1601,7 @@ func (s *testRegionCacheSuite) TestBuckets() { fakeRegion := &Region{ meta: cachedRegion.meta, syncFlags: cachedRegion.syncFlags, - lastAccess: cachedRegion.lastAccess, + ttl: cachedRegion.ttl, invalidReason: cachedRegion.invalidReason, } fakeRegion.setStore(cachedRegion.getStore().clone()) @@ -1817,7 +1821,7 @@ func (s *testRegionCacheSuite) TestBackgroundCacheGC() { now := time.Now().Unix() for verID, r := range s.cache.mu.regions { if verID.id%3 == 0 { - atomic.StoreInt64(&r.lastAccess, now-regionCacheTTLSec-10) + atomic.StoreInt64(&r.ttl, now-10) } else { remaining++ } @@ -1837,7 +1841,7 @@ func (s *testRegionCacheSuite) TestBackgroundCacheGC() { now = time.Now().Unix() for verID, r := range s.cache.mu.regions { if verID.id%3 == 1 { - atomic.StoreInt64(&r.lastAccess, now-regionCacheTTLSec-10) + atomic.StoreInt64(&r.ttl, now-10) } else { remaining++ } @@ -1929,7 +1933,7 @@ func (s *testRegionRequestToSingleStoreSuite) TestRefreshCache() { v2 := region.Region.confVer + 1 r2 := metapb.Region{Id: region.Region.id, RegionEpoch: &metapb.RegionEpoch{Version: region.Region.ver, ConfVer: v2}, StartKey: []byte{1}} st := &Store{storeID: s.store} - s.cache.insertRegionToCache(&Region{meta: &r2, store: unsafe.Pointer(st), lastAccess: time.Now().Unix()}, true, true) + s.cache.insertRegionToCache(&Region{meta: &r2, store: unsafe.Pointer(st), ttl: nextTTLWithoutJitter(time.Now().Unix())}, true, true) r, _ = s.cache.scanRegionsFromCache(s.bo, []byte{}, nil, 10) s.Equal(len(r), 2) @@ -2006,7 +2010,7 @@ func (s *testRegionCacheWithDelaySuite) TestInsertStaleRegion() { fakeRegion := &Region{ meta: r.meta, syncFlags: r.syncFlags, - lastAccess: r.lastAccess, + ttl: r.ttl, invalidReason: r.invalidReason, } fakeRegion.setStore(r.getStore().clone()) diff --git a/internal/locate/region_request3_test.go b/internal/locate/region_request3_test.go index b2562b4858..c5cf930a17 100644 --- a/internal/locate/region_request3_test.go +++ b/internal/locate/region_request3_test.go @@ -356,6 +356,11 @@ func (s *testRegionRequestToThreeStoresSuite) TestForwarding() { s.Nil(ctx.ProxyStore) } +func refreshRegionTTL(region *Region) { + atomic.StoreInt64(®ion.ttl, nextTTLWithoutJitter(time.Now().Unix())) + atomic.StoreInt32((*int32)(®ion.invalidReason), int32(Ok)) +} + func refreshEpochs(regionStore *regionStore) { for i, store := range regionStore.stores { regionStore.storeEpochs[i] = atomic.LoadUint32(&store.epoch) @@ -405,7 +410,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestLearnerReplicaSelector() { region = &Region{ meta: region.GetMeta(), } - atomic.StoreInt64(®ion.lastAccess, time.Now().Unix()) + refreshRegionTTL(region) region.meta.Peers = append(region.meta.Peers, tikvLearner) atomic.StorePointer(®ion.store, unsafe.Pointer(regionStore)) @@ -416,7 +421,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestLearnerReplicaSelector() { cache.mu.Unlock() // Test accessFollower state with kv.ReplicaReadLearner request type. - atomic.StoreInt64(®ion.lastAccess, time.Now().Unix()) + refreshRegionTTL(region) refreshEpochs(regionStore) req.ReplicaReadType = kv.ReplicaReadLearner replicaSelector, err := newReplicaSelector(cache, regionLoc.Region, req) @@ -425,7 +430,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestLearnerReplicaSelector() { accessLearner, _ := replicaSelector.state.(*accessFollower) // Invalidate the region if the leader is not in the region. - atomic.StoreInt64(®ion.lastAccess, time.Now().Unix()) + refreshRegionTTL(region) rpcCtx, err := replicaSelector.next(s.bo) s.Nil(err) // Should switch to the next follower. @@ -456,7 +461,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestReplicaSelector() { region = &Region{ meta: region.GetMeta(), } - atomic.StoreInt64(®ion.lastAccess, time.Now().Unix()) + refreshRegionTTL(region) region.meta.Peers = append(region.meta.Peers, tiflash) atomic.StorePointer(®ion.store, unsafe.Pointer(regionStore)) @@ -520,7 +525,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestReplicaSelector() { s.False(replicaSelector.region.isValid()) // Test switching to tryFollower if leader is unreachable - atomic.StoreInt64(®ion.lastAccess, time.Now().Unix()) + refreshRegionTTL(region) replicaSelector, err = newReplicaSelector(cache, regionLoc.Region, req) s.Nil(err) s.NotNil(replicaSelector) @@ -680,7 +685,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestReplicaSelector() { s.Nil(err) // Test accessFollower state filtering epoch-stale stores. - atomic.StoreInt64(®ion.lastAccess, time.Now().Unix()) + refreshRegionTTL(region) refreshEpochs(regionStore) // Mark all followers as stale. tiKVNum := regionStore.accessStoreNum(tiKVOnly) @@ -699,7 +704,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestReplicaSelector() { AssertRPCCtxEqual(s, rpcCtx, replicaSelector.replicas[regionStore.workTiKVIdx], nil) // Test accessFollower state filtering label-not-match stores. - atomic.StoreInt64(®ion.lastAccess, time.Now().Unix()) + refreshRegionTTL(region) refreshEpochs(regionStore) labels := []*metapb.StoreLabel{ { @@ -721,7 +726,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestReplicaSelector() { } // Test accessFollower state with leaderOnly option - atomic.StoreInt64(®ion.lastAccess, time.Now().Unix()) + refreshRegionTTL(region) refreshEpochs(regionStore) for i := 0; i < 5; i++ { replicaSelector, err = newReplicaSelector(cache, regionLoc.Region, req, WithLeaderOnly()) @@ -734,7 +739,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestReplicaSelector() { } // Test accessFollower state with kv.ReplicaReadMixed request type. - atomic.StoreInt64(®ion.lastAccess, time.Now().Unix()) + refreshRegionTTL(region) refreshEpochs(regionStore) req.ReplicaReadType = kv.ReplicaReadMixed replicaSelector, err = newReplicaSelector(cache, regionLoc.Region, req) @@ -742,7 +747,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestReplicaSelector() { s.Nil(err) // Invalidate the region if the leader is not in the region. - atomic.StoreInt64(®ion.lastAccess, time.Now().Unix()) + refreshRegionTTL(region) replicaSelector.updateLeader(&metapb.Peer{Id: s.cluster.AllocID(), StoreId: s.cluster.AllocID()}) s.False(region.isValid()) // Don't try next replica if the region is invalidated. @@ -1248,7 +1253,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestStaleReadFallback() { return &tikvrpc.Response{Resp: &kvrpcpb.GetResponse{Value: value}}, nil }} - region := s.cache.getRegionByIDFromCache(regionLoc.Region.GetID()) + region, _ := s.cache.searchCachedRegionByID(regionLoc.Region.GetID()) s.True(region.isValid()) req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{Key: []byte("key")}, kv.ReplicaReadLeader, nil) @@ -1480,7 +1485,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestReplicaReadFallbackToLeaderReg }}}, nil }} - region := s.cache.getRegionByIDFromCache(regionLoc.Region.GetID()) + region, _ := s.cache.searchCachedRegionByID(regionLoc.Region.GetID()) s.True(region.isValid()) req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{Key: []byte("key")}, kv.ReplicaReadLeader, nil) @@ -1847,8 +1852,8 @@ func (s *testRegionRequestToThreeStoresSuite) TestLeaderStuck() { } func (s *testRegionRequestToThreeStoresSuite) TestTiKVRecoveredFromDown() { - s.onClosed = func() { SetRegionCacheTTLSec(600) } - SetRegionCacheTTLSec(2) + s.onClosed = func() { SetRegionCacheTTLWithJitter(600, 60) } + SetRegionCacheTTLWithJitter(2, 0) bo := retry.NewBackoffer(context.Background(), -1) key := []byte("key") diff --git a/internal/locate/region_request_state_test.go b/internal/locate/region_request_state_test.go index f4a9881b71..6a92b52767 100644 --- a/internal/locate/region_request_state_test.go +++ b/internal/locate/region_request_state_test.go @@ -557,9 +557,7 @@ func testStaleRead(s *testRegionCacheStaleReadSuite, r *RegionCacheTestCase, zon s.Nil(err) s.NotNil(regionLoc) - s.cache.mu.RLock() - region := s.cache.getRegionByIDFromCache(s.regionID) - s.cache.mu.RUnlock() + region, _ := s.cache.searchCachedRegionByID(s.regionID) defer func() { var ( valid bool @@ -662,10 +660,9 @@ func followerDown(s *testRegionCacheStaleReadSuite) { } func followerDownAndUp(s *testRegionCacheStaleReadSuite) { - s.cache.mu.RLock() - cachedRegion := s.cache.getRegionByIDFromCache(s.regionID) - s.cache.mu.RUnlock() + cachedRegion, expired := s.cache.searchCachedRegionByID(s.regionID) _, follower := s.getFollower() + s.False(expired) s.NotNil(cachedRegion) s.NotNil(follower) regionStore := cachedRegion.getStore() @@ -751,10 +748,9 @@ func leaderDown(s *testRegionCacheStaleReadSuite) { } func leaderDownAndUp(s *testRegionCacheStaleReadSuite) { - s.cache.mu.RLock() - cachedRegion := s.cache.getRegionByIDFromCache(s.regionID) - s.cache.mu.RUnlock() + cachedRegion, expired := s.cache.searchCachedRegionByID(s.regionID) _, leader := s.getLeader() + s.False(expired) s.NotNil(cachedRegion) s.NotNil(leader) regionStore := cachedRegion.getStore() diff --git a/internal/locate/region_request_test.go b/internal/locate/region_request_test.go index b79689fb23..72cc1ac4a9 100644 --- a/internal/locate/region_request_test.go +++ b/internal/locate/region_request_test.go @@ -277,7 +277,9 @@ func (s *testRegionRequestToSingleStoreSuite) TestNoReloadRegionWhenCtxCanceled( _, _, err = sender.SendReq(bo, req, region.Region, time.Second) // Check this kind of error won't cause region cache drop. s.Equal(errors.Cause(err), context.Canceled) - s.NotNil(sender.regionCache.getRegionByIDFromCache(s.region)) + r, expired := sender.regionCache.searchCachedRegionByID(s.region) + s.False(expired) + s.NotNil(r) } // cancelContextClient wraps rpcClient and always cancels context before sending requests. @@ -547,7 +549,9 @@ func (s *testRegionRequestToSingleStoreSuite) TestNoReloadRegionForGrpcWhenCtxCa cancel() _, _, err = sender.SendReq(bo, req, region.Region, 3*time.Second) s.Equal(errors.Cause(err), context.Canceled) - s.NotNil(s.cache.getRegionByIDFromCache(s.region)) + r, expired := sender.regionCache.searchCachedRegionByID(s.region) + s.False(expired) + s.NotNil(r) // Just for covering error code = codes.Canceled. client1 := &cancelContextClient{ @@ -604,8 +608,9 @@ func (s *testRegionRequestToSingleStoreSuite) TestGetRegionByIDFromCache() { // test kv epochNotMatch return empty regions s.cache.OnRegionEpochNotMatch(s.bo, &RPCContext{Region: region.Region, Store: &Store{storeID: s.store}}, []*metapb.Region{}) s.Nil(err) - r := s.cache.getRegionByIDFromCache(s.region) - s.Nil(r) + r, expired := s.cache.searchCachedRegionByID(s.region) + s.True(expired) + s.NotNil(r) // refill cache region, err = s.cache.LocateRegionByID(s.bo, s.region) @@ -616,7 +621,7 @@ func (s *testRegionRequestToSingleStoreSuite) TestGetRegionByIDFromCache() { v2 := region.Region.confVer + 1 r2 := metapb.Region{Id: region.Region.id, RegionEpoch: &metapb.RegionEpoch{Version: region.Region.ver, ConfVer: v2}, StartKey: []byte{1}} st := &Store{storeID: s.store} - s.cache.insertRegionToCache(&Region{meta: &r2, store: unsafe.Pointer(st), lastAccess: time.Now().Unix()}, true, true) + s.cache.insertRegionToCache(&Region{meta: &r2, store: unsafe.Pointer(st), ttl: nextTTLWithoutJitter(time.Now().Unix())}, true, true) region, err = s.cache.LocateRegionByID(s.bo, s.region) s.Nil(err) s.NotNil(region) @@ -626,7 +631,7 @@ func (s *testRegionRequestToSingleStoreSuite) TestGetRegionByIDFromCache() { v3 := region.Region.confVer + 1 r3 := metapb.Region{Id: region.Region.id, RegionEpoch: &metapb.RegionEpoch{Version: v3, ConfVer: region.Region.confVer}, StartKey: []byte{2}} st = &Store{storeID: s.store} - s.cache.insertRegionToCache(&Region{meta: &r3, store: unsafe.Pointer(st), lastAccess: time.Now().Unix()}, true, true) + s.cache.insertRegionToCache(&Region{meta: &r3, store: unsafe.Pointer(st), ttl: nextTTLWithoutJitter(time.Now().Unix())}, true, true) region, err = s.cache.LocateRegionByID(s.bo, s.region) s.Nil(err) s.NotNil(region) diff --git a/internal/locate/sorted_btree.go b/internal/locate/sorted_btree.go index 0b56cc584d..5e2b88536a 100644 --- a/internal/locate/sorted_btree.go +++ b/internal/locate/sorted_btree.go @@ -63,21 +63,19 @@ func (s *SortedRegions) ReplaceOrInsert(cachedRegion *Region) *Region { return nil } -// DescendLessOrEqual returns all items that are less than or equal to the key. -func (s *SortedRegions) DescendLessOrEqual(key []byte, isEndKey bool, ts int64) (r *Region) { +// SearchByKey returns the region which contains the key. Note that the region might be expired and it's caller's duty to check the region TTL. +func (s *SortedRegions) SearchByKey(key []byte, isEndKey bool) (r *Region) { s.b.DescendLessOrEqual(newBtreeSearchItem(key), func(item *btreeItem) bool { - r = item.cachedRegion - if isEndKey && bytes.Equal(r.StartKey(), key) { - r = nil // clear result + region := item.cachedRegion + if isEndKey && bytes.Equal(region.StartKey(), key) { return true // iterate next item } - if !r.checkRegionCacheTTL(ts) { - r = nil - return true + if !isEndKey && region.Contains(key) || isEndKey && region.ContainsByEnd(key) { + r = region } return false }) - return r + return } // AscendGreaterOrEqual returns all items that are greater than or equal to the key. diff --git a/metrics/metrics.go b/metrics/metrics.go index 7a9cfcf367..875a3d44e7 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -58,6 +58,7 @@ var ( TiKVLoadSafepointCounter *prometheus.CounterVec TiKVSecondaryLockCleanupFailureCounter *prometheus.CounterVec TiKVRegionCacheCounter *prometheus.CounterVec + TiKVLoadRegionCounter *prometheus.CounterVec TiKVLoadRegionCacheHistogram *prometheus.HistogramVec TiKVLocalLatchWaitTimeHistogram prometheus.Histogram TiKVStatusDuration *prometheus.HistogramVec @@ -128,6 +129,7 @@ const ( LblInternal = "internal" LblGeneral = "general" LblDirection = "direction" + LblReason = "reason" ) func initMetrics(namespace, subsystem string, constLabels prometheus.Labels) { @@ -294,6 +296,14 @@ func initMetrics(namespace, subsystem string, constLabels prometheus.Labels) { ConstLabels: constLabels, }, []string{LblType, LblResult}) + TiKVLoadRegionCounter = prometheus.NewCounterVec(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "load_region_total", + Help: "Counter of loading region.", + ConstLabels: constLabels, + }, []string{LblType, LblReason}) + TiKVLoadRegionCacheHistogram = prometheus.NewHistogramVec( prometheus.HistogramOpts{ Namespace: namespace, @@ -763,6 +773,7 @@ func RegisterMetrics() { prometheus.MustRegister(TiKVLoadSafepointCounter) prometheus.MustRegister(TiKVSecondaryLockCleanupFailureCounter) prometheus.MustRegister(TiKVRegionCacheCounter) + prometheus.MustRegister(TiKVLoadRegionCounter) prometheus.MustRegister(TiKVLoadRegionCacheHistogram) prometheus.MustRegister(TiKVLocalLatchWaitTimeHistogram) prometheus.MustRegister(TiKVStatusDuration) diff --git a/tikv/region.go b/tikv/region.go index 6b5e4874d5..7753d4478b 100644 --- a/tikv/region.go +++ b/tikv/region.go @@ -194,11 +194,17 @@ func NewRegionRequestRuntimeStats() RegionRequestRuntimeStats { return locate.NewRegionRequestRuntimeStats() } -// SetRegionCacheTTLSec sets regionCacheTTLSec to t. +// SetRegionCacheTTLSec sets the base value of region cache TTL. +// Deprecated: use SetRegionCacheTTLWithJitter instead. func SetRegionCacheTTLSec(t int64) { locate.SetRegionCacheTTLSec(t) } +// SetRegionCacheTTLWithJitter sets region cache TTL with jitter. The real TTL is in range of [base, base+jitter). +func SetRegionCacheTTLWithJitter(base int64, jitter int64) { + locate.SetRegionCacheTTLWithJitter(base, jitter) +} + // SetStoreLivenessTimeout sets storeLivenessTimeout to t. func SetStoreLivenessTimeout(t time.Duration) { locate.SetStoreLivenessTimeout(t) From 11cb7985f0ec75018bf5ebfc67b21c55aea108e3 Mon Sep 17 00:00:00 2001 From: cfzjywxk Date: Mon, 5 Feb 2024 15:11:26 +0800 Subject: [PATCH 5/8] txn: enable pessimistic region rollback (#1149) * enable pessimistic region rollback Signed-off-by: cfzjywxk * add key range parameters Signed-off-by: cfzjywxk * change interface to use start and end keys Signed-off-by: cfzjywxk --------- Signed-off-by: cfzjywxk --- integration_tests/go.mod | 4 +- integration_tests/go.sum | 44 ++++++++++----------- integration_tests/lock_test.go | 15 +++---- internal/mockstore/mocktikv/mvcc.go | 2 +- internal/mockstore/mocktikv/mvcc_leveldb.go | 28 ++++++++++++- internal/mockstore/mocktikv/rpc.go | 2 +- txnkv/transaction/pessimistic.go | 4 +- txnkv/transaction/prewrite.go | 2 +- 8 files changed, 62 insertions(+), 39 deletions(-) diff --git a/integration_tests/go.mod b/integration_tests/go.mod index 651ed5b627..36e334b7ec 100644 --- a/integration_tests/go.mod +++ b/integration_tests/go.mod @@ -7,7 +7,7 @@ require ( github.com/pingcap/errors v0.11.5-0.20231212100244-799fae176cfb github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c github.com/pingcap/kvproto v0.0.0-20240109063850-932639606bcf - github.com/pingcap/tidb v1.1.0-beta.0.20240126041650-de177d85b19e + github.com/pingcap/tidb v1.1.0-beta.0.20240131080924-732fa8c98695 github.com/pkg/errors v0.9.1 github.com/stretchr/testify v1.8.4 github.com/tidwall/gjson v1.14.1 @@ -54,7 +54,7 @@ require ( github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 // indirect github.com/influxdata/tdigest v0.0.1 // indirect github.com/jellydator/ttlcache/v3 v3.0.1 // indirect - github.com/klauspost/compress v1.17.1 // indirect + github.com/klauspost/compress v1.17.4 // 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 diff --git a/integration_tests/go.sum b/integration_tests/go.sum index f2b3efd629..50e4bbee80 100644 --- a/integration_tests/go.sum +++ b/integration_tests/go.sum @@ -10,12 +10,12 @@ cloud.google.com/go/iam v1.1.5/go.mod h1:rB6P/Ic3mykPbFio+vo7403drjlgvoWfYpJhMXE cloud.google.com/go/storage v1.36.0 h1:P0mOkAcaJxhCTvAkMhxMfrTKiNcub4YmmPBtlhAyTr8= cloud.google.com/go/storage v1.36.0/go.mod h1:M6M/3V/D3KpzMTJyPOR/HU6n2Si5QdaXYEsng2xgOs8= github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= -github.com/Azure/azure-sdk-for-go/sdk/azcore v1.8.0 h1:9kDVnTz3vbfweTqAUmk/a/pH5pWFCHtvRpHYC0G/dcA= -github.com/Azure/azure-sdk-for-go/sdk/azcore v1.8.0/go.mod h1:3Ug6Qzto9anB6mGlEdgYMDF5zHQ+wwhEaYR4s17PHMw= +github.com/Azure/azure-sdk-for-go/sdk/azcore v1.9.0 h1:fb8kj/Dh4CSwgsOzHeZY4Xh68cFVbzXx+ONXGMY//4w= +github.com/Azure/azure-sdk-for-go/sdk/azcore v1.9.0/go.mod h1:uReU2sSxZExRPBAg3qKzmAucSi51+SP1OhohieR821Q= github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.4.0 h1:BMAjVKJM0U/CYF27gA0ZMmXGkOcvfFtD0oHVZ1TIPRI= github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.4.0/go.mod h1:1fXstnBMas5kzG+S3q8UoJcmyU6nUeunJcMDHcRYHhs= -github.com/Azure/azure-sdk-for-go/sdk/internal v1.3.0 h1:sXr+ck84g/ZlZUOZiNELInmMgOsuGwdjjVkEIde0OtY= -github.com/Azure/azure-sdk-for-go/sdk/internal v1.3.0/go.mod h1:okt5dMMTOFjX/aovMlrjvvXoPMBVSPzk9185BT0+eZM= +github.com/Azure/azure-sdk-for-go/sdk/internal v1.5.0 h1:d81/ng9rET2YqdVkVwkb6EXeRrLJIwyGnJcAlAWKwhs= +github.com/Azure/azure-sdk-for-go/sdk/internal v1.5.0/go.mod h1:s4kgfzA0covAXNicZHDMN58jExvcng2mC/DepXiF1EI= github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.0.0 h1:u/LLAOFgsMv7HmNL4Qufg58y+qElGOt5qv0z1mURkRY= github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.0.0/go.mod h1:2e8rMJtl2+2j+HXbTBwnyGpm5Nou7KhvSfxOq8JpTag= github.com/Azure/go-ntlmssp v0.0.0-20221128193559-754e69321358 h1:mFRzDkZVAjdal+s7s0MwaRv9igoPqLRdzOLzw/8Xvq8= @@ -52,8 +52,8 @@ github.com/apache/thrift v0.13.1-0.20201008052519-daf620915714/go.mod h1:cp2SuWM 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.45.25 h1:c4fLlh5sLdK2DCRTY1z0hyuJZU4ygxX8m1FswL6/nF4= -github.com/aws/aws-sdk-go v1.45.25/go.mod h1:aVsgQcEevwlmQ7qHE9I3h+dtQgpqhFB+i8Phjh7fkwI= +github.com/aws/aws-sdk-go v1.48.14 h1:nVLrp+F84SG+xGiFMfe1TE6ZV6smF+42tuuNgYGV30s= +github.com/aws/aws-sdk-go v1.48.14/go.mod h1:LF8svs817+Nz+DmiMQKTO3ubZ/6IaTpq3TjupRn3Eqk= github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/benbjohnson/clock v1.3.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= @@ -168,8 +168,8 @@ github.com/go-martini/martini v0.0.0-20170121215854-22fa46961aab/go.mod h1:/P9AE github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-ole/go-ole v1.3.0 h1:Dt6ye7+vXGIKZ7Xtk4s6/xVdGDQynvom7xCFEdWr6uE= github.com/go-ole/go-ole v1.3.0/go.mod h1:5LS6F96DhAwUc7C+1HLexzMXY1xGRSryjyPPKW6zv78= -github.com/go-resty/resty/v2 v2.7.0 h1:me+K9p3uhSmXtrBZ4k9jcEAfJmuC8IivWHwaLZwPrFY= -github.com/go-resty/resty/v2 v2.7.0/go.mod h1:9PWDzw47qPphMRFfhsyk0NnSgvluHcljSMVIq3w7q0I= +github.com/go-resty/resty/v2 v2.11.0 h1:i7jMfNOJYMp69lq7qozJP+bjgzfAzeOhuGlyDrqxT/8= +github.com/go-resty/resty/v2 v2.11.0/go.mod h1:iiP/OpA0CkcL3IGt1O0+/SIItFUbkkyw5BGXiVdTu+A= 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= @@ -305,8 +305,8 @@ github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+o github.com/klauspost/compress v1.8.2/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.9.0/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.9.5/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= -github.com/klauspost/compress v1.17.1 h1:NE3C767s2ak2bweCZo3+rdP4U/HoyVXLv/X9f2gPS5g= -github.com/klauspost/compress v1.17.1/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= +github.com/klauspost/compress v1.17.4 h1:Ej5ixsIri7BrIjBkRZLTo6ghwrEtHFk7ijlczPW4fZ4= +github.com/klauspost/compress v1.17.4/go.mod h1:/dCuZOvVtNoHsyb+cuJD3itjs3NbnF6KH9zAO4BDxPM= 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= @@ -421,8 +421,8 @@ github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22 h1:2SOzvGvE8beiC1Y4g github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 h1:QV6jqlfOkh8hqvEAgwBZa+4bSgO0EeKC7s5c6Luam2I= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21/go.mod h1:QYnjfA95ZaMefyl1NO8oPtKeb8pYUdnDVhQgf+qdpjM= -github.com/pingcap/tidb v1.1.0-beta.0.20240126041650-de177d85b19e h1:RX+LBAD8i8wWdwJ0yimxayTJUtwB8oZVcNaHPnmWOZY= -github.com/pingcap/tidb v1.1.0-beta.0.20240126041650-de177d85b19e/go.mod h1:ekt5M11y10od5MxYUSJ2B2bCqulzTd5LPqcJLFADBY4= +github.com/pingcap/tidb v1.1.0-beta.0.20240131080924-732fa8c98695 h1:7ELVL+rVzEZrDv4dEC995qHbv9K0dSKm7+wuwotiB8U= +github.com/pingcap/tidb v1.1.0-beta.0.20240131080924-732fa8c98695/go.mod h1:rfdtaBHvL6w0uFmxz0BJIBHHUpR/satNS695+def0y8= github.com/pingcap/tidb/pkg/parser v0.0.0-20240111112854-1ad36eb0ef29 h1:OPF0SMFk0O298dzHisYnhotbTcDQC2l+h0Xs7QxUF88= github.com/pingcap/tidb/pkg/parser v0.0.0-20240111112854-1ad36eb0ef29/go.mod h1:yRkiqLFwIqibYg2P7h4bclHjHcJiIFRLKhGRyBcKYus= github.com/pingcap/tipb v0.0.0-20240116032918-9bb28c43bbfc h1:sEp4lbExDfnMX8HXQyhZrhqo2/SgeFY5KOdo5akc8FM= @@ -597,10 +597,10 @@ go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.46.1 h1:aFJWCqJ go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.46.1/go.mod h1:sEGXWArGqc3tVa+ekntsN65DmVbVeW+7lTKTjZF3/Fo= go.opentelemetry.io/otel v1.21.0 h1:hzLeKBZEL7Okw2mGzZ0cc4k/A7Fta0uoPgaJCr8fsFc= go.opentelemetry.io/otel v1.21.0/go.mod h1:QZzNPQPm1zLX4gZK4cMi+71eaorMSGT3A4znnUvNNEo= -go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.19.0 h1:Mne5On7VWdx7omSrSSZvM4Kw7cS7NQkOOmLcgscI51U= -go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.19.0/go.mod h1:IPtUMKL4O3tH5y+iXVyAXqpAwMuzC1IrxVS81rummfE= -go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.19.0 h1:3d+S281UTjM+AbF31XSOYn1qXn3BgIdWl8HNEpx08Jk= -go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.19.0/go.mod h1:0+KuTDyKL4gjKCF75pHOX4wuzYDUZYfAQdSu43o+Z2I= +go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.21.0 h1:cl5P5/GIfFh4t6xyruOgJP5QiA1pw4fYYdv6nc6CBWw= +go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.21.0/go.mod h1:zgBdWWAu7oEEMC06MMKc5NLbA/1YDXV1sMpSqEeLQLg= +go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.21.0 h1:tIqheXEFWAZ7O8A7m+J0aPTmpJN3YQ7qetUAdkkkKpk= +go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.21.0/go.mod h1:nUeKExfxAQVbiVFn32YXpXZZHZ61Cc3s3Rn1pDBGAb0= go.opentelemetry.io/otel/metric v1.21.0 h1:tlYWfeo+Bocx5kLEloTjbcDwBuELRrIFxwdQ36PlJu4= go.opentelemetry.io/otel/metric v1.21.0/go.mod h1:o1p3CA8nNHW8j5yuQLdc1eeqEaPfzug24uvsyIEJRWM= go.opentelemetry.io/otel/sdk v1.21.0 h1:FTt8qirL1EysG6sTQRZ5TokkU8d0ugCj8htOgThZXQ8= @@ -857,12 +857,12 @@ gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= -k8s.io/api v0.28.2 h1:9mpl5mOb6vXZvqbQmankOfPIGiudghwCoLl1EYfUZbw= -k8s.io/api v0.28.2/go.mod h1:RVnJBsjU8tcMq7C3iaRSGMeaKt2TWEUXcpIt/90fjEg= -k8s.io/apimachinery v0.28.2 h1:KCOJLrc6gu+wV1BYgwik4AF4vXOlVJPdiqn0yAWWwXQ= -k8s.io/apimachinery v0.28.2/go.mod h1:RdzF87y/ngqk9H4z3EL2Rppv5jj95vGS/HaFXrLDApU= -k8s.io/klog/v2 v2.100.1 h1:7WCHKK6K8fNhTqfBhISHQ97KrnJNFZMcQvKp7gP/tmg= -k8s.io/klog/v2 v2.100.1/go.mod h1:y1WjHnz7Dj687irZUWR/WLkLc5N1YHtjLdmgWjndZn0= +k8s.io/api v0.28.4 h1:8ZBrLjwosLl/NYgv1P7EQLqoO8MGQApnbgH8tu3BMzY= +k8s.io/api v0.28.4/go.mod h1:axWTGrY88s/5YE+JSt4uUi6NMM+gur1en2REMR7IRj0= +k8s.io/apimachinery v0.28.4 h1:zOSJe1mc+GxuMnFzD4Z/U1wst50X28ZNsn5bhgIIao8= +k8s.io/apimachinery v0.28.4/go.mod h1:wI37ncBvfAoswfq626yPTe6Bz1c22L7uaJ8dho83mgg= +k8s.io/klog/v2 v2.110.1 h1:U/Af64HJf7FcwMcXyKm2RPM22WZzyR7OSpYj5tg3cL0= +k8s.io/klog/v2 v2.110.1/go.mod h1:YGtd1984u+GgbuZ7e08/yBuAfKLSO0+uR1Fhi6ExXjo= k8s.io/utils v0.0.0-20230711102312-30195339c3c7 h1:ZgnF1KZsYxWIifwSNZFZgNtWE89WI5yiP5WwlfDoIyc= k8s.io/utils v0.0.0-20230711102312-30195339c3c7/go.mod h1:OLgZIPagt7ERELqWJFomSt595RzquPNLL48iOWgYOg0= sigs.k8s.io/json v0.0.0-20221116044647-bc3834ca7abd h1:EDPBXCAspyGV4jQlpZSudPeMmr1bNJefnuqLsRAsHZo= diff --git a/integration_tests/lock_test.go b/integration_tests/lock_test.go index 81e76cbd34..65d04263f4 100644 --- a/integration_tests/lock_test.go +++ b/integration_tests/lock_test.go @@ -1032,6 +1032,7 @@ func (s *testLockWithTiKVSuite) SetupTest() { s.cleanupLocks() } else { s.store = tikv.StoreProbe{KVStore: NewTestUniStore(s.T())} + s.cleanupLocks() } } @@ -1609,17 +1610,11 @@ func (s *testLockWithTiKVSuite) mustResolve(ctx context.Context, bo *retry.Backo lockAfterResolve, err := s.store.ScanLocks(ctx, startKey, endKey, callerTS) s.NoError(err) - s.Len(lockAfterResolve, 0) + s.Len(lockAfterResolve, 0, "expected=%v actual=%v", 0, len(lockAfterResolve)) } } func (s *testLockWithTiKVSuite) TestPessimisticRollbackWithRead() { - // The test relies on the pessimistic rollback read phase implementations in tikv - // https://github.com/tikv/tikv/pull/16185, which is not implemented in mockstore by now. - if !*withTiKV { - return - } - s.NoError(failpoint.Enable("tikvclient/shortPessimisticLockTTL", "return")) s.NoError(failpoint.Enable("tikvclient/twoPCShortLockTTL", "return")) defer func() { @@ -1627,8 +1622,10 @@ func (s *testLockWithTiKVSuite) TestPessimisticRollbackWithRead() { s.NoError(failpoint.Disable("tikvclient/twoPCShortLockTTL")) }() test := func(inMemoryLock bool) { - recoverFunc := s.trySetTiKVConfig("pessimistic-txn.in-memory", inMemoryLock) - defer recoverFunc() + if *withTiKV { + recoverFunc := s.trySetTiKVConfig("pessimistic-txn.in-memory", inMemoryLock) + defer recoverFunc() + } // Init, cleanup possible left locks. bo := tikv.NewBackofferWithVars(context.Background(), int(transaction.PrewriteMaxBackoff.Load()), nil) diff --git a/internal/mockstore/mocktikv/mvcc.go b/internal/mockstore/mocktikv/mvcc.go index c279efb218..1ea3cf5777 100644 --- a/internal/mockstore/mocktikv/mvcc.go +++ b/internal/mockstore/mocktikv/mvcc.go @@ -269,7 +269,7 @@ type MVCCStore interface { ReverseScan(startKey, endKey []byte, limit int, startTS uint64, isoLevel kvrpcpb.IsolationLevel, resolvedLocks []uint64) []Pair BatchGet(ks [][]byte, startTS uint64, isoLevel kvrpcpb.IsolationLevel, resolvedLocks []uint64) []Pair PessimisticLock(req *kvrpcpb.PessimisticLockRequest) *kvrpcpb.PessimisticLockResponse - PessimisticRollback(keys [][]byte, startTS, forUpdateTS uint64) []error + PessimisticRollback(startKey []byte, endKey []byte, keys [][]byte, startTS, forUpdateTS uint64) []error Prewrite(req *kvrpcpb.PrewriteRequest) []error Commit(keys [][]byte, startTS, commitTS uint64) error Rollback(keys [][]byte, startTS uint64) error diff --git a/internal/mockstore/mocktikv/mvcc_leveldb.go b/internal/mockstore/mocktikv/mvcc_leveldb.go index 3d3641e0fa..8e3e4a74ce 100644 --- a/internal/mockstore/mocktikv/mvcc_leveldb.go +++ b/internal/mockstore/mocktikv/mvcc_leveldb.go @@ -732,10 +732,36 @@ func (mvcc *MVCCLevelDB) pessimisticLockMutation(batch *leveldb.Batch, mutation } // PessimisticRollback implements the MVCCStore interface. -func (mvcc *MVCCLevelDB) PessimisticRollback(keys [][]byte, startTS, forUpdateTS uint64) []error { +func (mvcc *MVCCLevelDB) PessimisticRollback(startKey []byte, endKey []byte, keys [][]byte, startTS, forUpdateTS uint64) []error { mvcc.mu.Lock() defer mvcc.mu.Unlock() + // Scan the whole region for corresponding pessimistic locks. + if len(keys) == 0 { + iter, currKey, err := newScanIterator(mvcc.getDB(""), startKey, endKey) + defer iter.Release() + if err != nil { + return []error{err} + } + for iter.Valid() { + dec := lockDecoder{expectKey: currKey} + ok, err := dec.Decode(iter) + if err != nil { + return []error{err} + } + if ok && dec.lock.op == kvrpcpb.Op_PessimisticLock && dec.lock.startTS == startTS && dec.lock.forUpdateTS <= forUpdateTS { + keys = append(keys, currKey) + } + + skip := skipDecoder{currKey: currKey} + _, err = skip.Decode(iter) + if err != nil { + return []error{err} + } + currKey = skip.currKey + } + } + anyError := false batch := &leveldb.Batch{} errs := make([]error, 0, len(keys)) diff --git a/internal/mockstore/mocktikv/rpc.go b/internal/mockstore/mocktikv/rpc.go index 09e31aa99f..bd50d91b8d 100644 --- a/internal/mockstore/mocktikv/rpc.go +++ b/internal/mockstore/mocktikv/rpc.go @@ -269,7 +269,7 @@ func (h kvHandler) handleKvPessimisticRollback(req *kvrpcpb.PessimisticRollbackR panic("KvPessimisticRollback: key not in region") } } - errs := h.mvccStore.PessimisticRollback(req.Keys, req.StartVersion, req.ForUpdateTs) + errs := h.mvccStore.PessimisticRollback(h.startKey, h.endKey, req.Keys, req.StartVersion, req.ForUpdateTs) return &kvrpcpb.PessimisticRollbackResponse{ Errors: convertToKeyErrors(errs), } diff --git a/txnkv/transaction/pessimistic.go b/txnkv/transaction/pessimistic.go index 56384e84cb..c1d9b95a13 100644 --- a/txnkv/transaction/pessimistic.go +++ b/txnkv/transaction/pessimistic.go @@ -358,7 +358,7 @@ func (action actionPessimisticLock) handlePessimisticLockResponseNormalMode( resolveLockOpts := txnlock.ResolveLocksOptions{ CallerStartTS: 0, Locks: locks, - PessimisticRegionResolve: false, + PessimisticRegionResolve: true, } if action.LockCtx.Stats != nil { resolveLockOpts.Detail = &action.LockCtx.Stats.ResolveLock @@ -487,7 +487,7 @@ func (action actionPessimisticLock) handlePessimisticLockResponseForceLockMode( resolveLockOpts := txnlock.ResolveLocksOptions{ CallerStartTS: 0, Locks: locks, - PessimisticRegionResolve: false, + PessimisticRegionResolve: true, } if action.LockCtx.Stats != nil { resolveLockOpts.Detail = &action.LockCtx.Stats.ResolveLock diff --git a/txnkv/transaction/prewrite.go b/txnkv/transaction/prewrite.go index 1a8d24aa90..7e56306515 100644 --- a/txnkv/transaction/prewrite.go +++ b/txnkv/transaction/prewrite.go @@ -474,7 +474,7 @@ func (action actionPrewrite) handleSingleBatch( CallerStartTS: c.startTS, Locks: locks, Detail: &c.getDetail().ResolveLock, - PessimisticRegionResolve: false, + PessimisticRegionResolve: true, } resolveLockRes, err := c.store.GetLockResolver().ResolveLocksWithOpts(bo, resolveLockOpts) if err != nil { From 4bb62b6db7f1754ae8c8ce08222a7b0cb0fbdd00 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Sun, 18 Feb 2024 11:41:12 +0800 Subject: [PATCH 6/8] upgrade grpc (#1156) Signed-off-by: Weizhen Wang --- examples/gcworker/go.mod | 32 +++++++++---------- examples/rawkv/go.mod | 30 +++++++++--------- examples/txnkv/1pc_txn/go.mod | 40 ++++++++++++------------ examples/txnkv/async_commit/go.mod | 40 ++++++++++++------------ examples/txnkv/delete_range/go.mod | 40 ++++++++++++------------ examples/txnkv/go.mod | 30 +++++++++--------- examples/txnkv/pessimistic_txn/go.mod | 40 ++++++++++++------------ examples/txnkv/unsafedestoryrange/go.mod | 40 ++++++++++++------------ go.mod | 12 +++---- go.sum | 24 +++++++------- integration_tests/go.mod | 14 ++++----- integration_tests/go.sum | 40 ++++++++++++------------ 12 files changed, 191 insertions(+), 191 deletions(-) diff --git a/examples/gcworker/go.mod b/examples/gcworker/go.mod index 71c776bcbe..4eedc60c18 100644 --- a/examples/gcworker/go.mod +++ b/examples/gcworker/go.mod @@ -15,22 +15,22 @@ require ( github.com/gogo/protobuf v1.3.2 // indirect github.com/golang/protobuf v1.5.3 // indirect github.com/google/btree v1.1.2 // indirect - github.com/google/uuid v1.3.1 // indirect + github.com/google/uuid v1.4.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect github.com/opentracing/opentracing-go v1.2.0 // indirect github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c // indirect - github.com/pingcap/kvproto v0.0.0-20230904082117-ecdbf1f8c130 // indirect + github.com/pingcap/kvproto v0.0.0-20231222062942-c0c73f41d0b2 // indirect github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 // indirect github.com/pkg/errors v0.9.1 // indirect - github.com/prometheus/client_golang v1.15.1 // indirect - github.com/prometheus/client_model v0.3.0 // indirect - github.com/prometheus/common v0.42.0 // indirect - github.com/prometheus/procfs v0.9.0 // indirect + github.com/prometheus/client_golang v1.18.0 // indirect + github.com/prometheus/client_model v0.5.0 // indirect + github.com/prometheus/common v0.46.0 // indirect + github.com/prometheus/procfs v0.12.0 // indirect github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 // indirect github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a // indirect - github.com/tikv/pd/client v0.0.0-20230724080549-de985b8e0afc // indirect + github.com/tikv/pd/client v0.0.0-20240124074521-ca8fd3db3e33 // indirect github.com/twmb/murmur3 v1.1.3 // indirect go.etcd.io/etcd/api/v3 v3.5.10 // indirect go.etcd.io/etcd/client/pkg/v3 v3.5.10 // indirect @@ -39,15 +39,15 @@ require ( go.uber.org/multierr v1.11.0 // indirect go.uber.org/zap v1.26.0 // indirect golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect - golang.org/x/net v0.17.0 // indirect - golang.org/x/sync v0.3.0 // indirect - golang.org/x/sys v0.13.0 // indirect - golang.org/x/text v0.13.0 // indirect - google.golang.org/genproto v0.0.0-20231016165738-49dd2c1f3d0b // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20231016165738-49dd2c1f3d0b // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b // indirect - google.golang.org/grpc v1.59.0 // indirect - google.golang.org/protobuf v1.31.0 // indirect + golang.org/x/net v0.21.0 // indirect + golang.org/x/sync v0.5.0 // indirect + golang.org/x/sys v0.17.0 // indirect + golang.org/x/text v0.14.0 // indirect + google.golang.org/genproto v0.0.0-20240205150955-31a09d347014 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240125205218-1f4bbc51befe // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240213162025-012b6fc9bca9 // indirect + google.golang.org/grpc v1.61.1 // indirect + google.golang.org/protobuf v1.32.0 // indirect gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect ) diff --git a/examples/rawkv/go.mod b/examples/rawkv/go.mod index 0c68e84a70..736a075ae5 100644 --- a/examples/rawkv/go.mod +++ b/examples/rawkv/go.mod @@ -16,22 +16,22 @@ require ( github.com/gogo/protobuf v1.3.2 // indirect github.com/golang/protobuf v1.5.3 // indirect github.com/google/btree v1.1.2 // indirect - github.com/google/uuid v1.3.1 // indirect + github.com/google/uuid v1.4.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect github.com/opentracing/opentracing-go v1.2.0 // indirect github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c // indirect - github.com/pingcap/kvproto v0.0.0-20230904082117-ecdbf1f8c130 // indirect + github.com/pingcap/kvproto v0.0.0-20231222062942-c0c73f41d0b2 // indirect github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 // indirect github.com/pkg/errors v0.9.1 // indirect - github.com/prometheus/client_golang v1.15.1 // indirect - github.com/prometheus/client_model v0.3.0 // indirect - github.com/prometheus/common v0.42.0 // indirect - github.com/prometheus/procfs v0.9.0 // indirect + github.com/prometheus/client_golang v1.18.0 // indirect + github.com/prometheus/client_model v0.5.0 // indirect + github.com/prometheus/common v0.46.0 // indirect + github.com/prometheus/procfs v0.12.0 // indirect github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 // indirect github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a // indirect - github.com/tikv/pd/client v0.0.0-20230724080549-de985b8e0afc // indirect + github.com/tikv/pd/client v0.0.0-20240124074521-ca8fd3db3e33 // indirect github.com/twmb/murmur3 v1.1.3 // indirect go.etcd.io/etcd/api/v3 v3.5.10 // indirect go.etcd.io/etcd/client/pkg/v3 v3.5.10 // indirect @@ -40,14 +40,14 @@ require ( go.uber.org/multierr v1.11.0 // indirect go.uber.org/zap v1.26.0 // indirect golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect - golang.org/x/net v0.17.0 // indirect - golang.org/x/sync v0.3.0 // indirect - golang.org/x/sys v0.13.0 // indirect - golang.org/x/text v0.13.0 // indirect - google.golang.org/genproto v0.0.0-20231016165738-49dd2c1f3d0b // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b // indirect - google.golang.org/grpc v1.59.0 // indirect - google.golang.org/protobuf v1.31.0 // indirect + golang.org/x/net v0.21.0 // indirect + golang.org/x/sync v0.5.0 // indirect + golang.org/x/sys v0.17.0 // indirect + golang.org/x/text v0.14.0 // indirect + google.golang.org/genproto v0.0.0-20240205150955-31a09d347014 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240213162025-012b6fc9bca9 // indirect + google.golang.org/grpc v1.61.1 // indirect + google.golang.org/protobuf v1.32.0 // indirect gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect ) diff --git a/examples/txnkv/1pc_txn/go.mod b/examples/txnkv/1pc_txn/go.mod index fcb48938b2..875ef93704 100644 --- a/examples/txnkv/1pc_txn/go.mod +++ b/examples/txnkv/1pc_txn/go.mod @@ -9,44 +9,44 @@ require ( github.com/beorn7/perks v1.0.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect github.com/cloudfoundry/gosigar v1.3.6 // indirect - github.com/coreos/go-semver v0.3.0 // indirect - github.com/coreos/go-systemd/v22 v22.3.2 // indirect + github.com/coreos/go-semver v0.3.1 // indirect + github.com/coreos/go-systemd/v22 v22.5.0 // indirect github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 // indirect github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2 // indirect github.com/gogo/protobuf v1.3.2 // indirect github.com/golang/protobuf v1.5.3 // indirect github.com/google/btree v1.1.2 // indirect - github.com/google/uuid v1.3.0 // indirect + github.com/google/uuid v1.4.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect github.com/opentracing/opentracing-go v1.2.0 // indirect github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c // indirect - github.com/pingcap/kvproto v0.0.0-20230904082117-ecdbf1f8c130 // indirect + github.com/pingcap/kvproto v0.0.0-20231222062942-c0c73f41d0b2 // indirect github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 // indirect github.com/pkg/errors v0.9.1 // indirect - github.com/prometheus/client_golang v1.15.1 // indirect - github.com/prometheus/client_model v0.3.0 // indirect - github.com/prometheus/common v0.42.0 // indirect - github.com/prometheus/procfs v0.9.0 // indirect + github.com/prometheus/client_golang v1.18.0 // indirect + github.com/prometheus/client_model v0.5.0 // indirect + github.com/prometheus/common v0.46.0 // indirect + github.com/prometheus/procfs v0.12.0 // indirect github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 // indirect github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a // indirect - github.com/tikv/pd/client v0.0.0-20230724080549-de985b8e0afc // indirect + github.com/tikv/pd/client v0.0.0-20240124074521-ca8fd3db3e33 // indirect github.com/twmb/murmur3 v1.1.3 // indirect - go.etcd.io/etcd/api/v3 v3.5.2 // indirect - go.etcd.io/etcd/client/pkg/v3 v3.5.2 // indirect - go.etcd.io/etcd/client/v3 v3.5.2 // indirect + go.etcd.io/etcd/api/v3 v3.5.10 // indirect + go.etcd.io/etcd/client/pkg/v3 v3.5.10 // indirect + go.etcd.io/etcd/client/v3 v3.5.10 // indirect go.uber.org/atomic v1.11.0 // indirect go.uber.org/multierr v1.11.0 // indirect - go.uber.org/zap v1.24.0 // indirect + go.uber.org/zap v1.26.0 // indirect golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect - golang.org/x/net v0.17.0 // indirect - golang.org/x/sync v0.2.0 // indirect - golang.org/x/sys v0.13.0 // indirect - golang.org/x/text v0.13.0 // indirect - google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 // indirect - google.golang.org/grpc v1.56.3 // indirect - google.golang.org/protobuf v1.30.0 // indirect + golang.org/x/net v0.21.0 // indirect + golang.org/x/sync v0.5.0 // indirect + golang.org/x/sys v0.17.0 // indirect + golang.org/x/text v0.14.0 // indirect + google.golang.org/genproto v0.0.0-20240205150955-31a09d347014 // indirect + google.golang.org/grpc v1.61.1 // indirect + google.golang.org/protobuf v1.32.0 // indirect gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect ) diff --git a/examples/txnkv/async_commit/go.mod b/examples/txnkv/async_commit/go.mod index 7d582a269e..a85bb7d812 100644 --- a/examples/txnkv/async_commit/go.mod +++ b/examples/txnkv/async_commit/go.mod @@ -9,44 +9,44 @@ require ( github.com/beorn7/perks v1.0.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect github.com/cloudfoundry/gosigar v1.3.6 // indirect - github.com/coreos/go-semver v0.3.0 // indirect - github.com/coreos/go-systemd/v22 v22.3.2 // indirect + github.com/coreos/go-semver v0.3.1 // indirect + github.com/coreos/go-systemd/v22 v22.5.0 // indirect github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 // indirect github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2 // indirect github.com/gogo/protobuf v1.3.2 // indirect github.com/golang/protobuf v1.5.3 // indirect github.com/google/btree v1.1.2 // indirect - github.com/google/uuid v1.3.0 // indirect + github.com/google/uuid v1.4.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect github.com/opentracing/opentracing-go v1.2.0 // indirect github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c // indirect - github.com/pingcap/kvproto v0.0.0-20230904082117-ecdbf1f8c130 // indirect + github.com/pingcap/kvproto v0.0.0-20231222062942-c0c73f41d0b2 // indirect github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 // indirect github.com/pkg/errors v0.9.1 // indirect - github.com/prometheus/client_golang v1.15.1 // indirect - github.com/prometheus/client_model v0.3.0 // indirect - github.com/prometheus/common v0.42.0 // indirect - github.com/prometheus/procfs v0.9.0 // indirect + github.com/prometheus/client_golang v1.18.0 // indirect + github.com/prometheus/client_model v0.5.0 // indirect + github.com/prometheus/common v0.46.0 // indirect + github.com/prometheus/procfs v0.12.0 // indirect github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 // indirect github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a // indirect - github.com/tikv/pd/client v0.0.0-20230724080549-de985b8e0afc // indirect + github.com/tikv/pd/client v0.0.0-20240124074521-ca8fd3db3e33 // indirect github.com/twmb/murmur3 v1.1.3 // indirect - go.etcd.io/etcd/api/v3 v3.5.2 // indirect - go.etcd.io/etcd/client/pkg/v3 v3.5.2 // indirect - go.etcd.io/etcd/client/v3 v3.5.2 // indirect + go.etcd.io/etcd/api/v3 v3.5.10 // indirect + go.etcd.io/etcd/client/pkg/v3 v3.5.10 // indirect + go.etcd.io/etcd/client/v3 v3.5.10 // indirect go.uber.org/atomic v1.11.0 // indirect go.uber.org/multierr v1.11.0 // indirect - go.uber.org/zap v1.24.0 // indirect + go.uber.org/zap v1.26.0 // indirect golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect - golang.org/x/net v0.17.0 // indirect - golang.org/x/sync v0.2.0 // indirect - golang.org/x/sys v0.13.0 // indirect - golang.org/x/text v0.13.0 // indirect - google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 // indirect - google.golang.org/grpc v1.56.3 // indirect - google.golang.org/protobuf v1.30.0 // indirect + golang.org/x/net v0.21.0 // indirect + golang.org/x/sync v0.5.0 // indirect + golang.org/x/sys v0.17.0 // indirect + golang.org/x/text v0.14.0 // indirect + google.golang.org/genproto v0.0.0-20240205150955-31a09d347014 // indirect + google.golang.org/grpc v1.61.1 // indirect + google.golang.org/protobuf v1.32.0 // indirect gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect ) diff --git a/examples/txnkv/delete_range/go.mod b/examples/txnkv/delete_range/go.mod index d465929f49..1b0a371454 100644 --- a/examples/txnkv/delete_range/go.mod +++ b/examples/txnkv/delete_range/go.mod @@ -9,44 +9,44 @@ require ( github.com/beorn7/perks v1.0.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect github.com/cloudfoundry/gosigar v1.3.6 // indirect - github.com/coreos/go-semver v0.3.0 // indirect - github.com/coreos/go-systemd/v22 v22.3.2 // indirect + github.com/coreos/go-semver v0.3.1 // indirect + github.com/coreos/go-systemd/v22 v22.5.0 // indirect github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 // indirect github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2 // indirect github.com/gogo/protobuf v1.3.2 // indirect github.com/golang/protobuf v1.5.3 // indirect github.com/google/btree v1.1.2 // indirect - github.com/google/uuid v1.3.0 // indirect + github.com/google/uuid v1.4.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect github.com/opentracing/opentracing-go v1.2.0 // indirect github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c // indirect - github.com/pingcap/kvproto v0.0.0-20230904082117-ecdbf1f8c130 // indirect + github.com/pingcap/kvproto v0.0.0-20231222062942-c0c73f41d0b2 // indirect github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 // indirect github.com/pkg/errors v0.9.1 // indirect - github.com/prometheus/client_golang v1.15.1 // indirect - github.com/prometheus/client_model v0.3.0 // indirect - github.com/prometheus/common v0.42.0 // indirect - github.com/prometheus/procfs v0.9.0 // indirect + github.com/prometheus/client_golang v1.18.0 // indirect + github.com/prometheus/client_model v0.5.0 // indirect + github.com/prometheus/common v0.46.0 // indirect + github.com/prometheus/procfs v0.12.0 // indirect github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 // indirect github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a // indirect - github.com/tikv/pd/client v0.0.0-20230724080549-de985b8e0afc // indirect + github.com/tikv/pd/client v0.0.0-20240124074521-ca8fd3db3e33 // indirect github.com/twmb/murmur3 v1.1.3 // indirect - go.etcd.io/etcd/api/v3 v3.5.2 // indirect - go.etcd.io/etcd/client/pkg/v3 v3.5.2 // indirect - go.etcd.io/etcd/client/v3 v3.5.2 // indirect + go.etcd.io/etcd/api/v3 v3.5.10 // indirect + go.etcd.io/etcd/client/pkg/v3 v3.5.10 // indirect + go.etcd.io/etcd/client/v3 v3.5.10 // indirect go.uber.org/atomic v1.11.0 // indirect go.uber.org/multierr v1.11.0 // indirect - go.uber.org/zap v1.24.0 // indirect + go.uber.org/zap v1.26.0 // indirect golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect - golang.org/x/net v0.17.0 // indirect - golang.org/x/sync v0.2.0 // indirect - golang.org/x/sys v0.13.0 // indirect - golang.org/x/text v0.13.0 // indirect - google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 // indirect - google.golang.org/grpc v1.56.3 // indirect - google.golang.org/protobuf v1.30.0 // indirect + golang.org/x/net v0.21.0 // indirect + golang.org/x/sync v0.5.0 // indirect + golang.org/x/sys v0.17.0 // indirect + golang.org/x/text v0.14.0 // indirect + google.golang.org/genproto v0.0.0-20240205150955-31a09d347014 // indirect + google.golang.org/grpc v1.61.1 // indirect + google.golang.org/protobuf v1.32.0 // indirect gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect ) diff --git a/examples/txnkv/go.mod b/examples/txnkv/go.mod index 798ee6f697..2f707b4487 100644 --- a/examples/txnkv/go.mod +++ b/examples/txnkv/go.mod @@ -16,22 +16,22 @@ require ( github.com/gogo/protobuf v1.3.2 // indirect github.com/golang/protobuf v1.5.3 // indirect github.com/google/btree v1.1.2 // indirect - github.com/google/uuid v1.3.1 // indirect + github.com/google/uuid v1.4.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect github.com/opentracing/opentracing-go v1.2.0 // indirect github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c // indirect - github.com/pingcap/kvproto v0.0.0-20230904082117-ecdbf1f8c130 // indirect + github.com/pingcap/kvproto v0.0.0-20231222062942-c0c73f41d0b2 // indirect github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 // indirect github.com/pkg/errors v0.9.1 // indirect - github.com/prometheus/client_golang v1.15.1 // indirect - github.com/prometheus/client_model v0.3.0 // indirect - github.com/prometheus/common v0.42.0 // indirect - github.com/prometheus/procfs v0.9.0 // indirect + github.com/prometheus/client_golang v1.18.0 // indirect + github.com/prometheus/client_model v0.5.0 // indirect + github.com/prometheus/common v0.46.0 // indirect + github.com/prometheus/procfs v0.12.0 // indirect github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 // indirect github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a // indirect - github.com/tikv/pd/client v0.0.0-20230724080549-de985b8e0afc // indirect + github.com/tikv/pd/client v0.0.0-20240124074521-ca8fd3db3e33 // indirect github.com/twmb/murmur3 v1.1.3 // indirect go.etcd.io/etcd/api/v3 v3.5.10 // indirect go.etcd.io/etcd/client/pkg/v3 v3.5.10 // indirect @@ -40,14 +40,14 @@ require ( go.uber.org/multierr v1.11.0 // indirect go.uber.org/zap v1.26.0 // indirect golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect - golang.org/x/net v0.17.0 // indirect - golang.org/x/sync v0.3.0 // indirect - golang.org/x/sys v0.13.0 // indirect - golang.org/x/text v0.13.0 // indirect - google.golang.org/genproto v0.0.0-20231016165738-49dd2c1f3d0b // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b // indirect - google.golang.org/grpc v1.59.0 // indirect - google.golang.org/protobuf v1.31.0 // indirect + golang.org/x/net v0.21.0 // indirect + golang.org/x/sync v0.5.0 // indirect + golang.org/x/sys v0.17.0 // indirect + golang.org/x/text v0.14.0 // indirect + google.golang.org/genproto v0.0.0-20240205150955-31a09d347014 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240213162025-012b6fc9bca9 // indirect + google.golang.org/grpc v1.61.1 // indirect + google.golang.org/protobuf v1.32.0 // indirect gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect ) diff --git a/examples/txnkv/pessimistic_txn/go.mod b/examples/txnkv/pessimistic_txn/go.mod index 2ca9562b18..b662b2b330 100644 --- a/examples/txnkv/pessimistic_txn/go.mod +++ b/examples/txnkv/pessimistic_txn/go.mod @@ -9,44 +9,44 @@ require ( github.com/beorn7/perks v1.0.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect github.com/cloudfoundry/gosigar v1.3.6 // indirect - github.com/coreos/go-semver v0.3.0 // indirect - github.com/coreos/go-systemd/v22 v22.3.2 // indirect + github.com/coreos/go-semver v0.3.1 // indirect + github.com/coreos/go-systemd/v22 v22.5.0 // indirect github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 // indirect github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2 // indirect github.com/gogo/protobuf v1.3.2 // indirect github.com/golang/protobuf v1.5.3 // indirect github.com/google/btree v1.1.2 // indirect - github.com/google/uuid v1.3.0 // indirect + github.com/google/uuid v1.4.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect github.com/opentracing/opentracing-go v1.2.0 // indirect github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c // indirect - github.com/pingcap/kvproto v0.0.0-20230904082117-ecdbf1f8c130 // indirect + github.com/pingcap/kvproto v0.0.0-20231222062942-c0c73f41d0b2 // indirect github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 // indirect github.com/pkg/errors v0.9.1 // indirect - github.com/prometheus/client_golang v1.15.1 // indirect - github.com/prometheus/client_model v0.3.0 // indirect - github.com/prometheus/common v0.42.0 // indirect - github.com/prometheus/procfs v0.9.0 // indirect + github.com/prometheus/client_golang v1.18.0 // indirect + github.com/prometheus/client_model v0.5.0 // indirect + github.com/prometheus/common v0.46.0 // indirect + github.com/prometheus/procfs v0.12.0 // indirect github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 // indirect github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a // indirect - github.com/tikv/pd/client v0.0.0-20230724080549-de985b8e0afc // indirect + github.com/tikv/pd/client v0.0.0-20240124074521-ca8fd3db3e33 // indirect github.com/twmb/murmur3 v1.1.3 // indirect - go.etcd.io/etcd/api/v3 v3.5.2 // indirect - go.etcd.io/etcd/client/pkg/v3 v3.5.2 // indirect - go.etcd.io/etcd/client/v3 v3.5.2 // indirect + go.etcd.io/etcd/api/v3 v3.5.10 // indirect + go.etcd.io/etcd/client/pkg/v3 v3.5.10 // indirect + go.etcd.io/etcd/client/v3 v3.5.10 // indirect go.uber.org/atomic v1.11.0 // indirect go.uber.org/multierr v1.11.0 // indirect - go.uber.org/zap v1.24.0 // indirect + go.uber.org/zap v1.26.0 // indirect golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect - golang.org/x/net v0.17.0 // indirect - golang.org/x/sync v0.2.0 // indirect - golang.org/x/sys v0.13.0 // indirect - golang.org/x/text v0.13.0 // indirect - google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 // indirect - google.golang.org/grpc v1.56.3 // indirect - google.golang.org/protobuf v1.30.0 // indirect + golang.org/x/net v0.21.0 // indirect + golang.org/x/sync v0.5.0 // indirect + golang.org/x/sys v0.17.0 // indirect + golang.org/x/text v0.14.0 // indirect + google.golang.org/genproto v0.0.0-20240205150955-31a09d347014 // indirect + google.golang.org/grpc v1.61.1 // indirect + google.golang.org/protobuf v1.32.0 // indirect gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect ) diff --git a/examples/txnkv/unsafedestoryrange/go.mod b/examples/txnkv/unsafedestoryrange/go.mod index 2166a2bf37..9edadb94a4 100644 --- a/examples/txnkv/unsafedestoryrange/go.mod +++ b/examples/txnkv/unsafedestoryrange/go.mod @@ -9,44 +9,44 @@ require ( github.com/beorn7/perks v1.0.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect github.com/cloudfoundry/gosigar v1.3.6 // indirect - github.com/coreos/go-semver v0.3.0 // indirect - github.com/coreos/go-systemd/v22 v22.3.2 // indirect + github.com/coreos/go-semver v0.3.1 // indirect + github.com/coreos/go-systemd/v22 v22.5.0 // indirect github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 // indirect github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2 // indirect github.com/gogo/protobuf v1.3.2 // indirect github.com/golang/protobuf v1.5.3 // indirect github.com/google/btree v1.1.2 // indirect - github.com/google/uuid v1.3.0 // indirect + github.com/google/uuid v1.4.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect github.com/opentracing/opentracing-go v1.2.0 // indirect github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c // indirect - github.com/pingcap/kvproto v0.0.0-20230904082117-ecdbf1f8c130 // indirect + github.com/pingcap/kvproto v0.0.0-20231222062942-c0c73f41d0b2 // indirect github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 // indirect github.com/pkg/errors v0.9.1 // indirect - github.com/prometheus/client_golang v1.15.1 // indirect - github.com/prometheus/client_model v0.3.0 // indirect - github.com/prometheus/common v0.42.0 // indirect - github.com/prometheus/procfs v0.9.0 // indirect + github.com/prometheus/client_golang v1.18.0 // indirect + github.com/prometheus/client_model v0.5.0 // indirect + github.com/prometheus/common v0.46.0 // indirect + github.com/prometheus/procfs v0.12.0 // indirect github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 // indirect github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a // indirect - github.com/tikv/pd/client v0.0.0-20230724080549-de985b8e0afc // indirect + github.com/tikv/pd/client v0.0.0-20240124074521-ca8fd3db3e33 // indirect github.com/twmb/murmur3 v1.1.3 // indirect - go.etcd.io/etcd/api/v3 v3.5.2 // indirect - go.etcd.io/etcd/client/pkg/v3 v3.5.2 // indirect - go.etcd.io/etcd/client/v3 v3.5.2 // indirect + go.etcd.io/etcd/api/v3 v3.5.10 // indirect + go.etcd.io/etcd/client/pkg/v3 v3.5.10 // indirect + go.etcd.io/etcd/client/v3 v3.5.10 // indirect go.uber.org/atomic v1.11.0 // indirect go.uber.org/multierr v1.11.0 // indirect - go.uber.org/zap v1.24.0 // indirect + go.uber.org/zap v1.26.0 // indirect golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect - golang.org/x/net v0.17.0 // indirect - golang.org/x/sync v0.2.0 // indirect - golang.org/x/sys v0.13.0 // indirect - golang.org/x/text v0.13.0 // indirect - google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 // indirect - google.golang.org/grpc v1.56.3 // indirect - google.golang.org/protobuf v1.30.0 // indirect + golang.org/x/net v0.21.0 // indirect + golang.org/x/sync v0.5.0 // indirect + golang.org/x/sys v0.17.0 // indirect + golang.org/x/text v0.14.0 // indirect + google.golang.org/genproto v0.0.0-20240205150955-31a09d347014 // indirect + google.golang.org/grpc v1.61.1 // indirect + google.golang.org/protobuf v1.32.0 // indirect gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect ) diff --git a/go.mod b/go.mod index 90299877ab..a86b50f537 100644 --- a/go.mod +++ b/go.mod @@ -29,7 +29,7 @@ require ( go.uber.org/goleak v1.2.0 go.uber.org/zap v1.26.0 golang.org/x/sync v0.5.0 - google.golang.org/grpc v1.61.0 + google.golang.org/grpc v1.61.1 ) require ( @@ -48,13 +48,13 @@ require ( go.etcd.io/etcd/client/pkg/v3 v3.5.10 // indirect go.uber.org/multierr v1.11.0 // indirect golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect - golang.org/x/net v0.20.0 // indirect - golang.org/x/sys v0.16.0 // indirect + golang.org/x/net v0.21.0 // indirect + golang.org/x/sys v0.17.0 // indirect golang.org/x/text v0.14.0 // indirect golang.org/x/tools v0.9.1 // indirect - google.golang.org/genproto v0.0.0-20240116215550-a9fa1716bcac // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240102182953-50ed04b92917 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240123012728-ef4313101c80 // indirect + google.golang.org/genproto v0.0.0-20240205150955-31a09d347014 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240125205218-1f4bbc51befe // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240213162025-012b6fc9bca9 // indirect google.golang.org/protobuf v1.32.0 // indirect gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/go.sum b/go.sum index d86eb8c675..3c2cf8ee5c 100644 --- a/go.sum +++ b/go.sum @@ -161,8 +161,8 @@ golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= -golang.org/x/net v0.20.0 h1:aCL9BSgETF1k+blQaYUBx9hJ9LOGP3gAVemcZlf1Kpo= -golang.org/x/net v0.20.0/go.mod h1:z8BVo6PvndSri0LbOE3hAn0apkU+1YvI6E70E9jsnvY= +golang.org/x/net v0.21.0 h1:AQyQV4dYCvJ7vGmJyKki9+PBdyvhkSd8EIx/qb0AYv4= +golang.org/x/net v0.21.0/go.mod h1:bIjVDfnllIU7BJ2DNgfnXvpSvtn8VRwhlsaeUTyUS44= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= 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= @@ -176,8 +176,8 @@ golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5h golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.16.0 h1:xWw16ngr6ZMtmxDyKyIgsE93KNKz5HKmMa3b8ALHidU= -golang.org/x/sys v0.16.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.17.0 h1:25cE3gD+tdBA7lp7QfhuV+rJiE9YXTcS3VG1SqssI/Y= +golang.org/x/sys v0.17.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= @@ -202,15 +202,15 @@ google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9Ywl google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= -google.golang.org/genproto v0.0.0-20240116215550-a9fa1716bcac h1:ZL/Teoy/ZGnzyrqK/Optxxp2pmVh+fmJ97slxSRyzUg= -google.golang.org/genproto v0.0.0-20240116215550-a9fa1716bcac/go.mod h1:+Rvu7ElI+aLzyDQhpHMFMMltsD6m7nqpuWDd2CwJw3k= -google.golang.org/genproto/googleapis/api v0.0.0-20240102182953-50ed04b92917 h1:rcS6EyEaoCO52hQDupoSfrxI3R6C2Tq741is7X8OvnM= -google.golang.org/genproto/googleapis/api v0.0.0-20240102182953-50ed04b92917/go.mod h1:CmlNWB9lSezaYELKS5Ym1r44VrrbPUa7JTvw+6MbpJ0= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240123012728-ef4313101c80 h1:AjyfHzEPEFp/NpvfN5g+KDla3EMojjhRVZc1i7cj+oM= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240123012728-ef4313101c80/go.mod h1:PAREbraiVEVGVdTZsVWjSbbTtSyGbAgIIvni8a8CD5s= +google.golang.org/genproto v0.0.0-20240205150955-31a09d347014 h1:g/4bk7P6TPMkAUbUhquq98xey1slwvuVJPosdBqYJlU= +google.golang.org/genproto v0.0.0-20240205150955-31a09d347014/go.mod h1:xEgQu1e4stdSSsxPDK8Azkrk/ECl5HvdPf6nbZrTS5M= +google.golang.org/genproto/googleapis/api v0.0.0-20240125205218-1f4bbc51befe h1:0poefMBYvYbs7g5UkjS6HcxBPaTRAmznle9jnxYoAI8= +google.golang.org/genproto/googleapis/api v0.0.0-20240125205218-1f4bbc51befe/go.mod h1:4jWUdICTdgc3Ibxmr8nAJiiLHwQBY0UI0XZcEMaFKaA= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240213162025-012b6fc9bca9 h1:hZB7eLIaYlW9qXRfCq/qDaPdbeY3757uARz5Vvfv+cY= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240213162025-012b6fc9bca9/go.mod h1:YUWgXUFRPfoYK1IHMuxH5K6nPEXSCzIMljnQ59lLRCk= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= -google.golang.org/grpc v1.61.0 h1:TOvOcuXn30kRao+gfcvsebNEa5iZIiLkisYEkf7R7o0= -google.golang.org/grpc v1.61.0/go.mod h1:VUbo7IFqmF1QtCAstipjG0GIoq49KvMe9+h1jFLBNJs= +google.golang.org/grpc v1.61.1 h1:kLAiWrZs7YeDM6MumDe7m3y4aM6wacLzM1Y/wiLP9XY= +google.golang.org/grpc v1.61.1/go.mod h1:VUbo7IFqmF1QtCAstipjG0GIoq49KvMe9+h1jFLBNJs= google.golang.org/grpc/examples v0.0.0-20231221225426-4f03f3ff32c9 h1:ATnmU8nL2NfIyTSiBvJVDIDIr3qBmeW+c7z7XU21eWs= google.golang.org/grpc/examples v0.0.0-20231221225426-4f03f3ff32c9/go.mod h1:j5uROIAAgi3YmtiETMt1LW0d/lHqQ7wwrIY4uGRXLQ4= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= diff --git a/integration_tests/go.mod b/integration_tests/go.mod index 36e334b7ec..20ca0b9e48 100644 --- a/integration_tests/go.mod +++ b/integration_tests/go.mod @@ -98,18 +98,18 @@ require ( go.uber.org/atomic v1.11.0 // indirect go.uber.org/multierr v1.11.0 // indirect go.uber.org/zap v1.26.0 // indirect - golang.org/x/crypto v0.18.0 // indirect + golang.org/x/crypto v0.19.0 // indirect golang.org/x/exp v0.0.0-20240110193028-0dcbfd608b1e // indirect - golang.org/x/net v0.20.0 // indirect + golang.org/x/net v0.21.0 // indirect golang.org/x/sync v0.6.0 // indirect - golang.org/x/sys v0.16.0 // indirect + golang.org/x/sys v0.17.0 // indirect golang.org/x/text v0.14.0 // indirect golang.org/x/time v0.5.0 // indirect golang.org/x/tools v0.17.0 // indirect - google.golang.org/genproto v0.0.0-20240116215550-a9fa1716bcac // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240108191215-35c7eff3a6b1 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240123012728-ef4313101c80 // indirect - google.golang.org/grpc v1.61.0 // indirect + google.golang.org/genproto v0.0.0-20240205150955-31a09d347014 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240125205218-1f4bbc51befe // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240213162025-012b6fc9bca9 // indirect + google.golang.org/grpc v1.61.1 // indirect google.golang.org/protobuf v1.32.0 // indirect gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect gopkg.in/yaml.v2 v2.4.0 // indirect diff --git a/integration_tests/go.sum b/integration_tests/go.sum index 50e4bbee80..01b619f728 100644 --- a/integration_tests/go.sum +++ b/integration_tests/go.sum @@ -1,12 +1,12 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.112.0 h1:tpFCD7hpHFlQ8yPwT3x+QeXqc2T6+n6T+hmABHfDUSM= cloud.google.com/go v0.112.0/go.mod h1:3jEEVwZ/MHU4djK5t5RHuKOA/GbLddgTdVubX1qnPD4= -cloud.google.com/go/compute v1.23.3 h1:6sVlXXBmbd7jNX0Ipq0trII3e4n1/MsADLK6a+aiVlk= -cloud.google.com/go/compute v1.23.3/go.mod h1:VCgBUoMnIVIR0CscqQiPJLAG25E3ZRZMzcFZeQ+h8CI= +cloud.google.com/go/compute v1.23.4 h1:EBT9Nw4q3zyE7G45Wvv3MzolIrCJEuHys5muLY0wvAw= +cloud.google.com/go/compute v1.23.4/go.mod h1:/EJMj55asU6kAFnuZET8zqgwgJ9FvXWXOkkfQZa4ioI= cloud.google.com/go/compute/metadata v0.2.3 h1:mg4jlk7mCAj6xXp9UJ4fjI9VUI5rubuGBW5aJ7UnBMY= cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= -cloud.google.com/go/iam v1.1.5 h1:1jTsCu4bcsNsE4iiqNT5SHwrDRCfRmIaaaVFhRveTJI= -cloud.google.com/go/iam v1.1.5/go.mod h1:rB6P/Ic3mykPbFio+vo7403drjlgvoWfYpJhMXEbzv8= +cloud.google.com/go/iam v1.1.6 h1:bEa06k05IO4f4uJonbB5iAgKTPpABy1ayxaIZV/GHVc= +cloud.google.com/go/iam v1.1.6/go.mod h1:O0zxdPeGBoFdWW3HWmBxJsk0pfvNM/p/qa82rWOGTwI= cloud.google.com/go/storage v1.36.0 h1:P0mOkAcaJxhCTvAkMhxMfrTKiNcub4YmmPBtlhAyTr8= cloud.google.com/go/storage v1.36.0/go.mod h1:M6M/3V/D3KpzMTJyPOR/HU6n2Si5QdaXYEsng2xgOs8= github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= @@ -647,8 +647,8 @@ golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.7.0/go.mod h1:pYwdfH91IfpZVANVyUOhSIPZaFoJGxTFbZhFTx+dXZU= -golang.org/x/crypto v0.18.0 h1:PGVlW0xEltQnzFZ55hkuX5+KLyrMYhHld1YHO4AKcdc= -golang.org/x/crypto v0.18.0/go.mod h1:R0j02AL6hcrfOiy9T4ZYp/rcWeMxM3L6QYxlOuEG1mg= +golang.org/x/crypto v0.19.0 h1:ENy+Az/9Y1vSrlrvBSyna3PITt4tiZLf7sgCjZBX7Wo= +golang.org/x/crypto v0.19.0/go.mod h1:Iy9bg/ha4yyC70EfRS8jz+B6ybOBKMaSxLj6P6oBDfU= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20240110193028-0dcbfd608b1e h1:723BNChdd0c2Wk6WOE320qGBiPtYx0F0Bbm1kriShfE= @@ -687,8 +687,8 @@ golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96b golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.8.0/go.mod h1:QVkue5JL9kW//ek3r6jTKnTFis1tRmNAW2P1shuFdJc= -golang.org/x/net v0.20.0 h1:aCL9BSgETF1k+blQaYUBx9hJ9LOGP3gAVemcZlf1Kpo= -golang.org/x/net v0.20.0/go.mod h1:z8BVo6PvndSri0LbOE3hAn0apkU+1YvI6E70E9jsnvY= +golang.org/x/net v0.21.0 h1:AQyQV4dYCvJ7vGmJyKki9+PBdyvhkSd8EIx/qb0AYv4= +golang.org/x/net v0.21.0/go.mod h1:bIjVDfnllIU7BJ2DNgfnXvpSvtn8VRwhlsaeUTyUS44= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.16.0 h1:aDkGMBSYxElaoP81NpoUoz2oo2R2wHdZpGToUxfyQrQ= golang.org/x/oauth2 v0.16.0/go.mod h1:hqZ+0LWXsiVoZpeld6jVt06P3adbS2Uu911W1SsJv2o= @@ -736,14 +736,14 @@ golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.16.0 h1:xWw16ngr6ZMtmxDyKyIgsE93KNKz5HKmMa3b8ALHidU= -golang.org/x/sys v0.16.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.17.0 h1:25cE3gD+tdBA7lp7QfhuV+rJiE9YXTcS3VG1SqssI/Y= +golang.org/x/sys v0.17.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.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/term v0.6.0/go.mod h1:m6U89DPEgQRMq3DNkDClhWw02AUbt2daBVO4cn4Hv9U= -golang.org/x/term v0.16.0 h1:m+B6fahuftsE9qjo0VWp2FW0mB3MTJvR0BaMQrq0pmE= -golang.org/x/term v0.16.0/go.mod h1:yn7UURbUtPyrVJPGPq404EukNFxcm/foM+bV/bfcDsY= +golang.org/x/term v0.17.0 h1:mkTF7LCd6WGJNL3K1Ad7kwxNfYAW6a8a8QqtMblp/4U= +golang.org/x/term v0.17.0/go.mod h1:lLRBjIVuehSbZlaOtGMbcMncT+aqLLLmKrsjNrUguwk= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= @@ -799,12 +799,12 @@ google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoA google.golang.org/genproto v0.0.0-20181004005441-af9cb2a35e7f/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20240116215550-a9fa1716bcac h1:ZL/Teoy/ZGnzyrqK/Optxxp2pmVh+fmJ97slxSRyzUg= -google.golang.org/genproto v0.0.0-20240116215550-a9fa1716bcac/go.mod h1:+Rvu7ElI+aLzyDQhpHMFMMltsD6m7nqpuWDd2CwJw3k= -google.golang.org/genproto/googleapis/api v0.0.0-20240108191215-35c7eff3a6b1 h1:OPXtXn7fNMaXwO3JvOmF1QyTc00jsSFFz1vXXBOdCDo= -google.golang.org/genproto/googleapis/api v0.0.0-20240108191215-35c7eff3a6b1/go.mod h1:B5xPO//w8qmBDjGReYLpR6UJPnkldGkCSMoH/2vxJeg= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240123012728-ef4313101c80 h1:AjyfHzEPEFp/NpvfN5g+KDla3EMojjhRVZc1i7cj+oM= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240123012728-ef4313101c80/go.mod h1:PAREbraiVEVGVdTZsVWjSbbTtSyGbAgIIvni8a8CD5s= +google.golang.org/genproto v0.0.0-20240205150955-31a09d347014 h1:g/4bk7P6TPMkAUbUhquq98xey1slwvuVJPosdBqYJlU= +google.golang.org/genproto v0.0.0-20240205150955-31a09d347014/go.mod h1:xEgQu1e4stdSSsxPDK8Azkrk/ECl5HvdPf6nbZrTS5M= +google.golang.org/genproto/googleapis/api v0.0.0-20240125205218-1f4bbc51befe h1:0poefMBYvYbs7g5UkjS6HcxBPaTRAmznle9jnxYoAI8= +google.golang.org/genproto/googleapis/api v0.0.0-20240125205218-1f4bbc51befe/go.mod h1:4jWUdICTdgc3Ibxmr8nAJiiLHwQBY0UI0XZcEMaFKaA= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240213162025-012b6fc9bca9 h1:hZB7eLIaYlW9qXRfCq/qDaPdbeY3757uARz5Vvfv+cY= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240213162025-012b6fc9bca9/go.mod h1:YUWgXUFRPfoYK1IHMuxH5K6nPEXSCzIMljnQ59lLRCk= google.golang.org/grpc v0.0.0-20180607172857-7a6a684ca69e/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= @@ -813,8 +813,8 @@ google.golang.org/grpc v1.24.0/go.mod h1:XDChyiUovWa60DnaeDeZmSW86xtLtjtZbwvSiRn google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.61.0 h1:TOvOcuXn30kRao+gfcvsebNEa5iZIiLkisYEkf7R7o0= -google.golang.org/grpc v1.61.0/go.mod h1:VUbo7IFqmF1QtCAstipjG0GIoq49KvMe9+h1jFLBNJs= +google.golang.org/grpc v1.61.1 h1:kLAiWrZs7YeDM6MumDe7m3y4aM6wacLzM1Y/wiLP9XY= +google.golang.org/grpc v1.61.1/go.mod h1:VUbo7IFqmF1QtCAstipjG0GIoq49KvMe9+h1jFLBNJs= google.golang.org/grpc/examples v0.0.0-20231221225426-4f03f3ff32c9 h1:ATnmU8nL2NfIyTSiBvJVDIDIr3qBmeW+c7z7XU21eWs= google.golang.org/grpc/examples v0.0.0-20231221225426-4f03f3ff32c9/go.mod h1:j5uROIAAgi3YmtiETMt1LW0d/lHqQ7wwrIY4uGRXLQ4= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= From 0f8c594090485696741c84a1441cdacb2d14d597 Mon Sep 17 00:00:00 2001 From: zyguan Date: Mon, 19 Feb 2024 11:43:20 +0800 Subject: [PATCH 7/8] fix data race in ut (#1161) Signed-off-by: zyguan --- internal/locate/region_request3_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/internal/locate/region_request3_test.go b/internal/locate/region_request3_test.go index c5cf930a17..b2ede8ab64 100644 --- a/internal/locate/region_request3_test.go +++ b/internal/locate/region_request3_test.go @@ -363,7 +363,7 @@ func refreshRegionTTL(region *Region) { func refreshEpochs(regionStore *regionStore) { for i, store := range regionStore.stores { - regionStore.storeEpochs[i] = atomic.LoadUint32(&store.epoch) + atomic.StoreUint32(&store.epoch, regionStore.storeEpochs[i]) } } From 824302acd0d8ffe847c4fe3723cba714aafb1023 Mon Sep 17 00:00:00 2001 From: tongjian <1045931706@qq.com> Date: Wed, 21 Feb 2024 14:54:33 +0800 Subject: [PATCH 8/8] client: Cache tikv request in tidb client side (#1098) * impl priority queue Signed-off-by: bufferflies <1045931706@qq.com> * replace priority queue Signed-off-by: bufferflies <1045931706@qq.com> * cache request in tidb side Signed-off-by: bufferflies <1045931706@qq.com> * fix gosimple Signed-off-by: bufferflies <1045931706@qq.com> * impl priority Signed-off-by: bufferflies <1045931706@qq.com> * pass ut Signed-off-by: bufferflies <1045931706@qq.com> * add Signed-off-by: bufferflies <1045931706@qq.com> * remove request if the request has been canceled Signed-off-by: bufferflies <1045931706@qq.com> * remove request if it has been canceled Signed-off-by: bufferflies <1045931706@qq.com> * add comment for cancel Signed-off-by: bufferflies <1045931706@qq.com> * not make the loop is busy Signed-off-by: bufferflies <1045931706@qq.com> * lint Signed-off-by: bufferflies <1045931706@qq.com> * revert busy loop Signed-off-by: bufferflies <1045931706@qq.com> * add unit test Signed-off-by: bufferflies <1045931706@qq.com> * not limit ehigh prioirty test Signed-off-by: bufferflies <1045931706@qq.com> * pass lint Signed-off-by: bufferflies <1045931706@qq.com> * support all Signed-off-by: bufferflies <1045931706@qq.com> * add comment Signed-off-by: bufferflies <1045931706@qq.com> * squash Signed-off-by: bufferflies <1045931706@qq.com> * revert all to All Signed-off-by: bufferflies <1045931706@qq.com> * remove index from entry Signed-off-by: bufferflies <1045931706@qq.com> * make fail reasons more clear Signed-off-by: bufferflies <1045931706@qq.com> --------- Signed-off-by: bufferflies <1045931706@qq.com> --- config/client.go | 7 +- examples/gcworker/go.mod | 1 - internal/client/client.go | 4 +- internal/client/client_batch.go | 130 ++++++++++++++++++------- internal/client/client_test.go | 121 +++++++++++++++++++---- internal/client/priority_queue.go | 82 +++++++++++----- internal/client/priority_queue_test.go | 58 +++++++---- 7 files changed, 306 insertions(+), 97 deletions(-) diff --git a/config/client.go b/config/client.go index 966bf2799e..29ad176670 100644 --- a/config/client.go +++ b/config/client.go @@ -36,6 +36,7 @@ package config import ( "fmt" + "math" "time" "google.golang.org/grpc/encoding/gzip" @@ -89,6 +90,9 @@ type TiKVClient struct { // TTLRefreshedTxnSize controls whether a transaction should update its TTL or not. TTLRefreshedTxnSize int64 `toml:"ttl-refreshed-txn-size" json:"ttl-refreshed-txn-size"` ResolveLockLiteThreshold uint64 `toml:"resolve-lock-lite-threshold" json:"resolve-lock-lite-threshold"` + // MaxConcurrencyRequestLimit is the max concurrency number of request to be sent the tikv + // 0 means auto adjust by feedback. + MaxConcurrencyRequestLimit int64 `toml:"max-concurrency-request-limit" json:"max-concurrency-request-limit"` } // AsyncCommit is the config for the async commit feature. The switch to enable it is a system variable. @@ -158,7 +162,8 @@ func DefaultTiKVClient() TiKVClient { }, CoprReqTimeout: 60 * time.Second, - ResolveLockLiteThreshold: 16, + ResolveLockLiteThreshold: 16, + MaxConcurrencyRequestLimit: math.MaxInt64, } } diff --git a/examples/gcworker/go.mod b/examples/gcworker/go.mod index 4eedc60c18..b9cd464356 100644 --- a/examples/gcworker/go.mod +++ b/examples/gcworker/go.mod @@ -17,7 +17,6 @@ require ( github.com/google/btree v1.1.2 // indirect github.com/google/uuid v1.4.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 // indirect - github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect github.com/opentracing/opentracing-go v1.2.0 // indirect github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c // indirect diff --git a/internal/client/client.go b/internal/client/client.go index 5a633a865e..b6052a4bc5 100644 --- a/internal/client/client.go +++ b/internal/client/client.go @@ -318,6 +318,7 @@ func (a *connArray) Init(addr string, security config.Security, idleNotify *uint dialTimeout: a.dialTimeout, tryLock: tryLock{sync.NewCond(new(sync.Mutex)), false}, } + batchClient.maxConcurrencyRequestLimit.Store(cfg.TiKVClient.MaxConcurrencyRequestLimit) a.batchCommandsClients = append(a.batchCommandsClients, batchClient) } } @@ -623,10 +624,11 @@ func (c *RPCClient) sendRequest(ctx context.Context, addr string, req *tikvrpc.R // TiDB RPC server supports batch RPC, but batch connection will send heart beat, It's not necessary since // request to TiDB is not high frequency. + pri := req.GetResourceControlContext().GetOverridePriority() if config.GetGlobalConfig().TiKVClient.MaxBatchSize > 0 && enableBatch { if batchReq := req.ToBatchCommandsRequest(); batchReq != nil { defer trace.StartRegion(ctx, req.Type.String()).End() - return sendBatchRequest(ctx, addr, req.ForwardedHost, connArray.batchConn, batchReq, timeout) + return sendBatchRequest(ctx, addr, req.ForwardedHost, connArray.batchConn, batchReq, timeout, pri) } } diff --git a/internal/client/client_batch.go b/internal/client/client_batch.go index a9956cd1d9..d241494b44 100644 --- a/internal/client/client_batch.go +++ b/internal/client/client_batch.go @@ -70,15 +70,15 @@ type batchCommandsEntry struct { // canceled indicated the request is canceled or not. canceled int32 err error + pri uint64 } func (b *batchCommandsEntry) isCanceled() bool { return atomic.LoadInt32(&b.canceled) == 1 } -// TODO: implement by the request priority. -func (b *batchCommandsEntry) priority() int { - return 0 +func (b *batchCommandsEntry) priority() uint64 { + return b.pri } func (b *batchCommandsEntry) error(err error) { @@ -107,33 +107,58 @@ func (b *batchCommandsBuilder) push(entry *batchCommandsEntry) { b.entries.Push(entry) } -// build builds BatchCommandsRequests and calls collect() for each valid entry. +const highTaskPriority = 10 + +func (b *batchCommandsBuilder) hasHighPriorityTask() bool { + return b.entries.highestPriority() >= highTaskPriority +} + +// buildWithLimit builds BatchCommandsRequests with the given limit. +// the highest priority tasks don't consume any limit, +// so the limit only works for normal tasks. // The first return value is the request that doesn't need forwarding. // The second is a map that maps forwarded hosts to requests. -func (b *batchCommandsBuilder) build( - collect func(id uint64, e *batchCommandsEntry), +func (b *batchCommandsBuilder) buildWithLimit(limit int64, collect func(id uint64, e *batchCommandsEntry), ) (*tikvpb.BatchCommandsRequest, map[string]*tikvpb.BatchCommandsRequest) { - for _, entry := range b.entries.All() { - e := entry.(*batchCommandsEntry) - if e.isCanceled() { - continue + count := int64(0) + build := func(reqs []Item) { + for _, e := range reqs { + e := e.(*batchCommandsEntry) + if e.isCanceled() { + continue + } + if e.priority() < highTaskPriority { + count++ + } + + if collect != nil { + collect(b.idAlloc, e) + } + if e.forwardedHost == "" { + b.requestIDs = append(b.requestIDs, b.idAlloc) + b.requests = append(b.requests, e.req) + } else { + batchReq, ok := b.forwardingReqs[e.forwardedHost] + if !ok { + batchReq = &tikvpb.BatchCommandsRequest{} + b.forwardingReqs[e.forwardedHost] = batchReq + } + batchReq.RequestIds = append(batchReq.RequestIds, b.idAlloc) + batchReq.Requests = append(batchReq.Requests, e.req) + } + b.idAlloc++ } - if collect != nil { - collect(b.idAlloc, e) + } + for (count < limit && b.entries.Len() > 0) || b.hasHighPriorityTask() { + n := limit + if limit == 0 { + n = 1 } - if e.forwardedHost == "" { - b.requestIDs = append(b.requestIDs, b.idAlloc) - b.requests = append(b.requests, e.req) - } else { - batchReq, ok := b.forwardingReqs[e.forwardedHost] - if !ok { - batchReq = &tikvpb.BatchCommandsRequest{} - b.forwardingReqs[e.forwardedHost] = batchReq - } - batchReq.RequestIds = append(batchReq.RequestIds, b.idAlloc) - batchReq.Requests = append(batchReq.Requests, e.req) + reqs := b.entries.Take(int(n)) + if len(reqs) == 0 { + break } - b.idAlloc++ + build(reqs) } var req *tikvpb.BatchCommandsRequest if len(b.requests) > 0 { @@ -145,20 +170,22 @@ func (b *batchCommandsBuilder) build( return req, b.forwardingReqs } +// cancel all requests, only used in test. func (b *batchCommandsBuilder) cancel(e error) { - for _, entry := range b.entries.All() { + for _, entry := range b.entries.all() { entry.(*batchCommandsEntry).error(e) } + b.entries.reset() } // reset resets the builder to the initial state. // Should call it before collecting a new batch. func (b *batchCommandsBuilder) reset() { + b.entries.clean() // NOTE: We can't simply set entries = entries[:0] here. // The data in the cap part of the slice would reference the prewrite keys whose // underlying memory is borrowed from memdb. The reference cause GC can't release // the memdb, leading to serious memory leak problems in the large transaction case. - b.entries.Reset() for i := 0; i < len(b.requests); i++ { b.requests[i] = nil } @@ -336,8 +363,7 @@ func (a *batchConn) batchSendLoop(cfg config.TiKVClient) { a.fetchMorePendingRequests(int(cfg.MaxBatchSize), int(bestBatchWaitSize), cfg.MaxBatchWaitTime) } } - a.pendingRequests.Observe(float64(len(a.batchCommandsCh))) - a.batchSize.Observe(float64(a.reqBuilder.len())) + a.pendingRequests.Observe(float64(len(a.batchCommandsCh) + a.reqBuilder.len())) length := a.reqBuilder.len() if uint(length) == 0 { // The batch command channel is closed. @@ -354,6 +380,11 @@ func (a *batchConn) batchSendLoop(cfg config.TiKVClient) { } } +const ( + SendFailedReasonNoAvailableLimit = "concurrency limit exceeded" + SendFailedReasonTryLockForSendFail = "tryLockForSend fail" +) + func (a *batchConn) getClientAndSend() { if val, err := util.EvalFailpoint("mockBatchClientSendDelay"); err == nil { if timeout, ok := val.(int); ok && timeout > 0 { @@ -366,37 +397,50 @@ func (a *batchConn) getClientAndSend() { cli *batchCommandsClient target string ) + reasons := make([]string, 0) + hasHighPriorityTask := a.reqBuilder.hasHighPriorityTask() for i := 0; i < len(a.batchCommandsClients); i++ { a.index = (a.index + 1) % uint32(len(a.batchCommandsClients)) target = a.batchCommandsClients[a.index].target // The lock protects the batchCommandsClient from been closed while it's in use. - if a.batchCommandsClients[a.index].tryLockForSend() { - cli = a.batchCommandsClients[a.index] - break + c := a.batchCommandsClients[a.index] + if hasHighPriorityTask || c.available() > 0 { + if c.tryLockForSend() { + cli = c + break + } else { + reasons = append(reasons, SendFailedReasonTryLockForSendFail) + } + } else { + reasons = append(reasons, SendFailedReasonNoAvailableLimit) } } if cli == nil { - logutil.BgLogger().Warn("no available connections", zap.String("target", target)) + logutil.BgLogger().Info("no available connections", zap.String("target", target), zap.Any("reasons", reasons)) metrics.TiKVNoAvailableConnectionCounter.Inc() - - // Please ensure the error is handled in region cache correctly. - a.reqBuilder.cancel(errors.New("no available connections")) return } defer cli.unlockForSend() - - req, forwardingReqs := a.reqBuilder.build(func(id uint64, e *batchCommandsEntry) { + available := cli.available() + batch := 0 + req, forwardingReqs := a.reqBuilder.buildWithLimit(available, func(id uint64, e *batchCommandsEntry) { cli.batched.Store(id, e) + cli.sent.Add(1) if trace.IsEnabled() { trace.Log(e.ctx, "rpc", "send") } }) if req != nil { + batch += len(req.RequestIds) cli.send("", req) } for forwardedHost, req := range forwardingReqs { + batch += len(req.RequestIds) cli.send(forwardedHost, req) } + if batch > 0 { + a.batchSize.Observe(float64(batch)) + } } type tryLock struct { @@ -507,12 +551,20 @@ type batchCommandsClient struct { closed int32 // tryLock protects client when re-create the streaming. tryLock + // sent is the number of the requests are processed by tikv server. + sent atomic.Int64 + // maxConcurrencyRequestLimit is the max allowed number of requests to be sent the tikv + maxConcurrencyRequestLimit atomic.Int64 } func (c *batchCommandsClient) isStopped() bool { return atomic.LoadInt32(&c.closed) != 0 } +func (c *batchCommandsClient) available() int64 { + return c.maxConcurrencyRequestLimit.Load() - c.sent.Load() +} + func (c *batchCommandsClient) send(forwardedHost string, req *tikvpb.BatchCommandsRequest) { err := c.initBatchClient(forwardedHost) if err != nil { @@ -549,6 +601,7 @@ func (c *batchCommandsClient) failPendingRequests(err error) { id, _ := key.(uint64) entry, _ := value.(*batchCommandsEntry) c.batched.Delete(id) + c.sent.Add(-1) entry.error(err) return true }) @@ -661,6 +714,7 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient, tikvTransport entry.res <- responses[i] } c.batched.Delete(requestID) + c.sent.Add(-1) } transportLayerLoad := resp.GetTransportLayerLoad() @@ -779,6 +833,7 @@ func sendBatchRequest( batchConn *batchConn, req *tikvpb.BatchCommandsRequest_Request, timeout time.Duration, + priority uint64, ) (*tikvrpc.Response, error) { entry := &batchCommandsEntry{ ctx: ctx, @@ -787,6 +842,7 @@ func sendBatchRequest( forwardedHost: forwardedHost, canceled: 0, err: nil, + pri: priority, } timer := time.NewTimer(timeout) defer timer.Stop() diff --git a/internal/client/client_test.go b/internal/client/client_test.go index f5c26df3ca..d4d22bfe6d 100644 --- a/internal/client/client_test.go +++ b/internal/client/client_test.go @@ -37,6 +37,7 @@ package client import ( "context" "fmt" + "math" "math/rand" "runtime" "strconv" @@ -111,10 +112,10 @@ func TestCancelTimeoutRetErr(t *testing.T) { ctx, cancel := context.WithCancel(context.TODO()) cancel() - _, err := sendBatchRequest(ctx, "", "", a, req, 2*time.Second) + _, err := sendBatchRequest(ctx, "", "", a, req, 2*time.Second, 0) assert.Equal(t, errors.Cause(err), context.Canceled) - _, err = sendBatchRequest(context.Background(), "", "", a, req, 0) + _, err = sendBatchRequest(context.Background(), "", "", a, req, 0, 0) assert.Equal(t, errors.Cause(err), context.DeadlineExceeded) } @@ -134,8 +135,8 @@ func TestSendWhenReconnect(t *testing.T) { } req := tikvrpc.NewRequest(tikvrpc.CmdEmpty, &tikvpb.BatchCommandsEmptyRequest{}) - _, err = rpcClient.SendRequest(context.Background(), addr, req, 100*time.Second) - assert.True(t, err.Error() == "no available connections") + _, err = rpcClient.SendRequest(context.Background(), addr, req, 5*time.Second) + assert.True(t, strings.Contains(err.Error(), "timeout")) server.Stop() } @@ -386,7 +387,7 @@ func TestBatchCommandsBuilder(t *testing.T) { assert.Equal(t, builder.len(), i+1) } entryMap := make(map[uint64]*batchCommandsEntry) - batchedReq, forwardingReqs := builder.build(func(id uint64, e *batchCommandsEntry) { + batchedReq, forwardingReqs := builder.buildWithLimit(math.MaxInt64, func(id uint64, e *batchCommandsEntry) { entryMap[id] = e }) assert.Equal(t, len(batchedReq.GetRequests()), 10) @@ -412,7 +413,7 @@ func TestBatchCommandsBuilder(t *testing.T) { } } entryMap = make(map[uint64]*batchCommandsEntry) - batchedReq, forwardingReqs = builder.build(func(id uint64, e *batchCommandsEntry) { + batchedReq, forwardingReqs = builder.buildWithLimit(math.MaxInt64, func(id uint64, e *batchCommandsEntry) { entryMap[id] = e }) assert.Equal(t, len(batchedReq.GetRequests()), 1) @@ -422,8 +423,8 @@ func TestBatchCommandsBuilder(t *testing.T) { assert.Equal(t, len(forwardingReqs[host].GetRequests()), i+2) assert.Equal(t, len(forwardingReqs[host].GetRequestIds()), i+2) } - assert.Equal(t, builder.idAlloc, uint64(10+builder.len())) - assert.Equal(t, len(entryMap), builder.len()) + assert.Equal(t, int(builder.idAlloc), 20) + assert.Equal(t, len(entryMap), 10) for host, forwardingReq := range forwardingReqs { for i, id := range forwardingReq.GetRequestIds() { assert.Equal(t, entryMap[id].req, forwardingReq.GetRequests()[i]) @@ -444,7 +445,7 @@ func TestBatchCommandsBuilder(t *testing.T) { builder.push(entry) } entryMap = make(map[uint64]*batchCommandsEntry) - batchedReq, forwardingReqs = builder.build(func(id uint64, e *batchCommandsEntry) { + batchedReq, forwardingReqs = builder.buildWithLimit(math.MaxInt64, func(id uint64, e *batchCommandsEntry) { entryMap[id] = e }) assert.Equal(t, len(batchedReq.GetRequests()), 2) @@ -475,7 +476,6 @@ func TestBatchCommandsBuilder(t *testing.T) { // Test reset builder.reset() assert.Equal(t, builder.len(), 0) - assert.Equal(t, builder.entries.Len(), 0) assert.Equal(t, len(builder.requests), 0) assert.Equal(t, len(builder.requestIDs), 0) assert.Equal(t, len(builder.forwardingReqs), 0) @@ -645,10 +645,6 @@ func TestTraceExecDetails(t *testing.T) { } func TestBatchClientRecoverAfterServerRestart(t *testing.T) { - config.UpdateGlobal(func(conf *config.Config) { - conf.TiKVClient.MaxBatchSize = 128 - })() - server, port := mockserver.StartMockTikvService() require.True(t, port > 0) require.True(t, server.IsRunning()) @@ -665,7 +661,7 @@ func TestBatchClientRecoverAfterServerRestart(t *testing.T) { assert.Nil(t, err) // send some request, it should be success. for i := 0; i < 100; i++ { - _, err = sendBatchRequest(context.Background(), addr, "", conn.batchConn, req, time.Second*20) + _, err = sendBatchRequest(context.Background(), addr, "", conn.batchConn, req, time.Second*20, 0) require.NoError(t, err) } @@ -674,8 +670,8 @@ func TestBatchClientRecoverAfterServerRestart(t *testing.T) { require.False(t, server.IsRunning()) // send some request, it should be failed since server is down. - for i := 0; i < 200; i++ { - _, err = sendBatchRequest(context.Background(), addr, "", conn.batchConn, req, time.Second*20) + for i := 0; i < 10; i++ { + _, err = sendBatchRequest(context.Background(), addr, "", conn.batchConn, req, time.Millisecond*100, 0) require.Error(t, err) time.Sleep(time.Millisecond * time.Duration(rand.Intn(300))) grpcConn := conn.Get() @@ -718,7 +714,96 @@ func TestBatchClientRecoverAfterServerRestart(t *testing.T) { // send some request, it should be success again. for i := 0; i < 100; i++ { - _, err = sendBatchRequest(context.Background(), addr, "", conn.batchConn, req, time.Second*20) + _, err = sendBatchRequest(context.Background(), addr, "", conn.batchConn, req, time.Second*20, 0) require.NoError(t, err) } } + +func TestLimitConcurrency(t *testing.T) { + re := require.New(t) + batch := newBatchConn(1, 128, nil) + { + batch.reqBuilder.push(&batchCommandsEntry{req: &tikvpb.BatchCommandsRequest_Request{}}) + reqs, _ := batch.reqBuilder.buildWithLimit(1, func(_ uint64, _ *batchCommandsEntry) {}) + re.Len(reqs.RequestIds, 1) + re.Equal(0, batch.reqBuilder.len()) + batch.reqBuilder.reset() + } + + // highest priority task will be sent immediately, not limited by concurrency + { + batch.reqBuilder.push(&batchCommandsEntry{req: &tikvpb.BatchCommandsRequest_Request{}, pri: highTaskPriority}) + batch.reqBuilder.push(&batchCommandsEntry{req: &tikvpb.BatchCommandsRequest_Request{}, pri: highTaskPriority - 1}) + reqs, _ := batch.reqBuilder.buildWithLimit(0, func(_ uint64, _ *batchCommandsEntry) {}) + re.Len(reqs.RequestIds, 1) + batch.reqBuilder.reset() + re.Equal(1, batch.reqBuilder.len()) + } + + // medium priority tasks are limited by concurrency + { + batch.reqBuilder.push(&batchCommandsEntry{req: &tikvpb.BatchCommandsRequest_Request{}}) + batch.reqBuilder.push(&batchCommandsEntry{req: &tikvpb.BatchCommandsRequest_Request{}}) + reqs, _ := batch.reqBuilder.buildWithLimit(2, func(_ uint64, _ *batchCommandsEntry) {}) + re.Len(reqs.RequestIds, 2) + re.Equal(1, batch.reqBuilder.len()) + batch.reqBuilder.reset() + } + + // the expired tasks should be removed from the queue. + { + batch.reqBuilder.push(&batchCommandsEntry{req: &tikvpb.BatchCommandsRequest_Request{}, canceled: 1}) + batch.reqBuilder.push(&batchCommandsEntry{req: &tikvpb.BatchCommandsRequest_Request{}, canceled: 1}) + batch.reqBuilder.reset() + re.Equal(1, batch.reqBuilder.len()) + } + +} + +func TestPrioritySentLimit(t *testing.T) { + re := require.New(t) + restoreFn := config.UpdateGlobal(func(conf *config.Config) { + conf.TiKVClient.MaxConcurrencyRequestLimit = 2 + conf.TiKVClient.GrpcConnectionCount = 1 + }) + defer restoreFn() + + server, port := mockserver.StartMockTikvService() + re.Greater(port, 0) + rpcClient := NewRPCClient() + defer rpcClient.Close() + addr := server.Addr() + wait := sync.WaitGroup{} + bench := 10 + wait.Add(bench * 2) + ctx, cancelFn := context.WithTimeout(context.Background(), time.Second*10) + defer cancelFn() + sendFn := func(pri uint64, dur *atomic.Int64, qps *atomic.Int64) { + for i := 0; i < bench; i++ { + go func() { + for { + req := tikvrpc.NewRequest(tikvrpc.CmdEmpty, &tikvpb.BatchCommandsEmptyRequest{}) + req.ResourceControlContext = &kvrpcpb.ResourceControlContext{OverridePriority: pri} + now := time.Now() + rpcClient.SendRequest(context.Background(), addr, req, 100*time.Millisecond) + dur.Add(time.Since(now).Microseconds()) + qps.Add(1) + select { + case <-ctx.Done(): + wait.Done() + return + default: + } + } + }() + } + } + + highDur, mediumDur := atomic.Int64{}, atomic.Int64{} + highQps, mediumQps := atomic.Int64{}, atomic.Int64{} + go sendFn(16, &highDur, &highQps) + go sendFn(8, &mediumDur, &mediumQps) + wait.Wait() + re.Less(highDur.Load()/highQps.Load()*2, mediumDur.Load()/mediumQps.Load()) + server.Stop() +} diff --git a/internal/client/priority_queue.go b/internal/client/priority_queue.go index f70afa106a..4622b19614 100644 --- a/internal/client/priority_queue.go +++ b/internal/client/priority_queue.go @@ -18,17 +18,18 @@ import "container/heap" // Item is the interface that all entries in a priority queue must implement. type Item interface { - priority() int + priority() uint64 + // isCanceled returns true if the item is canceled by the caller. + isCanceled() bool } // entry is an entry in a priority queue. -type entry struct { - entry Item - index int -} +//type entry struct { +// entry Item +//} // prioritySlice implements heap.Interface and holds Entries. -type prioritySlice []entry +type prioritySlice []Item // Len returns the length of the priority queue. func (ps prioritySlice) Len() int { @@ -38,20 +39,17 @@ func (ps prioritySlice) Len() int { // Less compares two entries in the priority queue. // The higher priority entry is the one with the lower value. func (ps prioritySlice) Less(i, j int) bool { - return ps[i].entry.priority() > ps[j].entry.priority() + return ps[i].priority() > ps[j].priority() } // Swap swaps two entries in the priority queue. func (ps prioritySlice) Swap(i, j int) { ps[i], ps[j] = ps[j], ps[i] - ps[i].index = i - ps[j].index = j } // Push adds an entry to the priority queue. func (ps *prioritySlice) Push(x interface{}) { - item := x.(entry) - item.index = len(*ps) + item := x.(Item) *ps = append(*ps, item) } @@ -60,7 +58,6 @@ func (ps *prioritySlice) Pop() interface{} { old := *ps n := len(old) item := old[n-1] - item.index = -1 *ps = old[0 : n-1] return item } @@ -82,27 +79,68 @@ func (pq *PriorityQueue) Len() int { // Push adds an entry to the priority queue. func (pq *PriorityQueue) Push(item Item) { - heap.Push(&pq.ps, entry{entry: item}) + heap.Push(&pq.ps, item) } -// Pop removes the highest priority entry from the priority queue. -func (pq *PriorityQueue) Pop() Item { - return heap.Pop(&pq.ps).(entry).entry +// pop removes the highest priority entry from the priority queue. +func (pq *PriorityQueue) pop() Item { + e := heap.Pop(&pq.ps) + if e == nil { + return nil + } + return e.(Item) +} + +// Take returns the highest priority entries from the priority queue. +func (pq *PriorityQueue) Take(n int) []Item { + if n <= 0 { + return nil + } + if n >= pq.Len() { + ret := pq.ps + pq.ps = pq.ps[:0] + return ret + } else { + ret := make([]Item, n) + for i := 0; i < n; i++ { + ret[i] = pq.pop() + } + return ret + } + +} + +func (pq *PriorityQueue) highestPriority() uint64 { + if pq.Len() == 0 { + return 0 + } + return pq.ps[0].priority() } -// All returns all entries in the priority queue not ensure the priority. -func (pq *PriorityQueue) All() []Item { +// all returns all entries in the priority queue not ensure the priority. +func (pq *PriorityQueue) all() []Item { items := make([]Item, 0, pq.Len()) for i := 0; i < pq.Len(); i++ { - items = append(items, pq.ps[i].entry) + items = append(items, pq.ps[i]) } return items } -// Reset resets the priority queue. -func (pq *PriorityQueue) Reset() { +// clean removes all canceled entries from the priority queue. +func (pq *PriorityQueue) clean() { + for i := 0; i < pq.Len(); { + if pq.ps[i].isCanceled() { + heap.Remove(&pq.ps, i) + continue + } + i++ + } +} + +// reset clear all entry in the queue. +func (pq *PriorityQueue) reset() { for i := 0; i < pq.Len(); i++ { - pq.ps[i].entry = nil + pq.ps[i] = nil } pq.ps = pq.ps[:0] } diff --git a/internal/client/priority_queue_test.go b/internal/client/priority_queue_test.go index e249155c9d..e64bbf5262 100644 --- a/internal/client/priority_queue_test.go +++ b/internal/client/priority_queue_test.go @@ -21,29 +21,53 @@ import ( ) type FakeItem struct { - pri int - value int + pri uint64 + value int + canceled bool } -func (f *FakeItem) priority() int { +func (f *FakeItem) priority() uint64 { return f.pri } +func (f *FakeItem) isCanceled() bool { + return f.canceled +} + func TestPriority(t *testing.T) { re := require.New(t) - pq := NewPriorityQueue() - for i := 1; i <= 5; i++ { - pq.Push(&FakeItem{value: i, pri: i}) - } - re.Equal(5, pq.Len()) - arr := pq.All() - re.Len(arr, 5) - pq.Reset() - re.Equal(0, pq.Len()) - for i := 1; i <= 5; i++ { - pq.Push(&FakeItem{value: i, pri: i}) - } - for i := pq.Len(); i > 0; i-- { - re.Equal(i, pq.Pop().(*FakeItem).value) + testFunc := func(aq *PriorityQueue) { + for i := 1; i <= 5; i++ { + aq.Push(&FakeItem{value: i, pri: uint64(i)}) + } + re.Equal(5, aq.Len()) + re.Equal(uint64(5), aq.highestPriority()) + aq.clean() + re.Equal(5, aq.Len()) + + arr := aq.Take(1) + re.Len(arr, 1) + re.Equal(uint64(5), arr[0].priority()) + re.Equal(uint64(4), aq.highestPriority()) + + arr = aq.Take(2) + re.Len(arr, 2) + re.Equal(uint64(4), arr[0].priority()) + re.Equal(uint64(3), arr[1].priority()) + re.Equal(uint64(2), aq.highestPriority()) + + arr = aq.Take(5) + re.Len(arr, 2) + re.Equal(uint64(2), arr[0].priority()) + re.Equal(uint64(1), arr[1].priority()) + re.Equal(uint64(0), aq.highestPriority()) + re.Equal(0, aq.Len()) + + aq.Push(&FakeItem{value: 1, pri: 1, canceled: true}) + re.Equal(1, aq.Len()) + aq.clean() + re.Equal(0, aq.Len()) } + hq := NewPriorityQueue() + testFunc(hq) }