mirror of https://github.com/tikv/client-go.git
txnkv/txnlock: init Lock in txnlock package (#241)
Signed-off-by: shirly <AndreMouche@126.com>
This commit is contained in:
parent
ce04aa0bf1
commit
654864ded8
|
|
@ -59,6 +59,8 @@ import (
|
|||
"github.com/tikv/client-go/v2/testutils"
|
||||
"github.com/tikv/client-go/v2/tikv"
|
||||
"github.com/tikv/client-go/v2/tikvrpc"
|
||||
"github.com/tikv/client-go/v2/txnkv"
|
||||
"github.com/tikv/client-go/v2/txnkv/txnlock"
|
||||
)
|
||||
|
||||
var (
|
||||
|
|
@ -842,9 +844,9 @@ func (s *testCommitterSuite) TestDeleteAllYourWritesWithSFU() {
|
|||
s.store.ClearTxnLatches()
|
||||
err = txn2.Set(k3, []byte{33})
|
||||
s.Nil(err)
|
||||
var meetLocks []*tikv.Lock
|
||||
resolver := tikv.LockResolverProbe{LockResolver: s.store.GetLockResolver()}
|
||||
resolver.SetMeetLockCallback(func(locks []*tikv.Lock) {
|
||||
var meetLocks []*txnkv.Lock
|
||||
resolver := tikv.NewLockResolverProb(s.store.GetLockResolver())
|
||||
resolver.SetMeetLockCallback(func(locks []*txnkv.Lock) {
|
||||
meetLocks = append(meetLocks, locks...)
|
||||
})
|
||||
err = txn2.Commit(context.Background())
|
||||
|
|
@ -940,9 +942,9 @@ func (s *testCommitterSuite) TestPkNotFound() {
|
|||
// while the secondary lock operation succeeded.
|
||||
txn1.GetCommitter().CloseTTLManager()
|
||||
|
||||
var status tikv.TxnStatus
|
||||
var status txnkv.TxnStatus
|
||||
bo := tikv.NewBackofferWithVars(ctx, 5000, nil)
|
||||
lockKey2 := &tikv.Lock{
|
||||
lockKey2 := &txnkv.Lock{
|
||||
Key: k2,
|
||||
Primary: k1,
|
||||
TxnID: txn1.StartTS(),
|
||||
|
|
@ -951,7 +953,8 @@ func (s *testCommitterSuite) TestPkNotFound() {
|
|||
LockType: kvrpcpb.Op_PessimisticLock,
|
||||
LockForUpdateTS: txn1.StartTS(),
|
||||
}
|
||||
resolver := tikv.LockResolverProbe{LockResolver: s.store.GetLockResolver()}
|
||||
|
||||
resolver := tikv.NewLockResolverProb(s.store.GetLockResolver())
|
||||
status, err = resolver.GetTxnStatusFromLock(bo, lockKey2, oracle.GoTimeToTS(time.Now().Add(200*time.Millisecond)), false)
|
||||
s.Nil(err)
|
||||
s.Equal(status.Action(), kvrpcpb.Action_TTLExpirePessimisticRollback)
|
||||
|
|
@ -969,7 +972,7 @@ func (s *testCommitterSuite) TestPkNotFound() {
|
|||
s.Nil(err)
|
||||
|
||||
// Pessimistic rollback using smaller forUpdateTS does not take effect.
|
||||
lockKey3 := &tikv.Lock{
|
||||
lockKey3 := &txnkv.Lock{
|
||||
Key: k3,
|
||||
Primary: k1,
|
||||
TxnID: txn1.StartTS(),
|
||||
|
|
@ -1201,8 +1204,8 @@ func (s *testCommitterSuite) TestResolveMixed() {
|
|||
// try to resolve the left optimistic locks, use clean whole region
|
||||
time.Sleep(time.Duration(atomic.LoadUint64(&tikv.ManagedLockTTL)) * time.Millisecond)
|
||||
optimisticLockInfo := s.getLockInfo(optimisticLockKey)
|
||||
lock := tikv.NewLock(optimisticLockInfo)
|
||||
resolver := tikv.LockResolverProbe{LockResolver: s.store.GetLockResolver()}
|
||||
lock := txnlock.NewLock(optimisticLockInfo)
|
||||
resolver := tikv.NewLockResolverProb(s.store.GetLockResolver())
|
||||
err = resolver.ResolveLock(ctx, lock)
|
||||
s.Nil(err)
|
||||
|
||||
|
|
|
|||
|
|
@ -206,7 +206,7 @@ func (s *testAsyncCommitFailSuite) TestSecondaryListInPrimaryLock() {
|
|||
|
||||
primary := txn.GetCommitter().GetPrimaryKey()
|
||||
bo := tikv.NewBackofferWithVars(context.Background(), 5000, nil)
|
||||
lockResolver := tikv.LockResolverProbe{LockResolver: s.store.GetLockResolver()}
|
||||
lockResolver := tikv.NewLockResolverProb(s.store.GetLockResolver())
|
||||
txnStatus, err := lockResolver.GetTxnStatus(bo, txn.StartTS(), primary, 0, 0, false, false, nil)
|
||||
s.Nil(err)
|
||||
s.False(txnStatus.IsCommitted())
|
||||
|
|
|
|||
|
|
@ -51,6 +51,8 @@ import (
|
|||
"github.com/tikv/client-go/v2/testutils"
|
||||
"github.com/tikv/client-go/v2/tikv"
|
||||
"github.com/tikv/client-go/v2/tikvrpc"
|
||||
"github.com/tikv/client-go/v2/txnkv"
|
||||
"github.com/tikv/client-go/v2/txnkv/txnlock"
|
||||
"github.com/tikv/client-go/v2/util"
|
||||
)
|
||||
|
||||
|
|
@ -108,7 +110,7 @@ func (s *testAsyncCommitCommon) mustGetFromTxn(txn tikv.TxnProbe, key, expectedV
|
|||
s.Equal(v, expectedValue)
|
||||
}
|
||||
|
||||
func (s *testAsyncCommitCommon) mustGetLock(key []byte) *tikv.Lock {
|
||||
func (s *testAsyncCommitCommon) mustGetLock(key []byte) *txnkv.Lock {
|
||||
ver, err := s.store.CurrentTimestamp(oracle.GlobalTxnScope)
|
||||
s.Nil(err)
|
||||
req := tikvrpc.NewRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{
|
||||
|
|
@ -123,8 +125,7 @@ func (s *testAsyncCommitCommon) mustGetLock(key []byte) *tikv.Lock {
|
|||
s.NotNil(resp.Resp)
|
||||
keyErr := resp.Resp.(*kvrpcpb.GetResponse).GetError()
|
||||
s.NotNil(keyErr)
|
||||
var lockutil tikv.LockProbe
|
||||
lock, err := lockutil.ExtractLockFromKeyErr(keyErr)
|
||||
lock, err := tikv.ExtractLockFromKeyErr(keyErr)
|
||||
s.Nil(err)
|
||||
return lock
|
||||
}
|
||||
|
|
@ -247,10 +248,10 @@ func (s *testAsyncCommitSuite) TestCheckSecondaries() {
|
|||
lock.UseAsyncCommit = true
|
||||
ts, err := s.store.GetOracle().GetTimestamp(context.Background(), &oracle.Option{TxnScope: oracle.GlobalTxnScope})
|
||||
s.Nil(err)
|
||||
var lockutil tikv.LockProbe
|
||||
var lockutil txnlock.LockProbe
|
||||
status := lockutil.NewLockStatus(nil, true, ts)
|
||||
|
||||
resolver := tikv.LockResolverProbe{LockResolver: s.store.GetLockResolver()}
|
||||
resolver := tikv.NewLockResolverProb(s.store.GetLockResolver())
|
||||
err = resolver.ResolveLockAsync(s.bo, lock, status)
|
||||
s.Nil(err)
|
||||
currentTS, err := s.store.GetOracle().GetTimestamp(context.Background(), &oracle.Option{TxnScope: oracle.GlobalTxnScope})
|
||||
|
|
@ -319,7 +320,7 @@ func (s *testAsyncCommitSuite) TestCheckSecondaries() {
|
|||
s.store.SetTiKVClient(&mock)
|
||||
|
||||
status = lockutil.NewLockStatus([][]byte{[]byte("a"), []byte("i")}, true, 0)
|
||||
lock = &tikv.Lock{
|
||||
lock = &txnkv.Lock{
|
||||
Key: []byte("a"),
|
||||
Primary: []byte("z"),
|
||||
TxnID: ts,
|
||||
|
|
|
|||
|
|
@ -52,6 +52,8 @@ import (
|
|||
"github.com/tikv/client-go/v2/oracle"
|
||||
"github.com/tikv/client-go/v2/tikv"
|
||||
"github.com/tikv/client-go/v2/tikvrpc"
|
||||
"github.com/tikv/client-go/v2/txnkv"
|
||||
"github.com/tikv/client-go/v2/txnkv/txnlock"
|
||||
)
|
||||
|
||||
var getMaxBackoff = tikv.ConfigProbe{}.GetGetMaxBackoff()
|
||||
|
|
@ -317,13 +319,13 @@ func (s *testLockSuite) TestCheckTxnStatus() {
|
|||
|
||||
// Test the ResolveLocks API
|
||||
lock := s.mustGetLock([]byte("second"))
|
||||
timeBeforeExpire, _, err := resolver.ResolveLocks(bo, currentTS, []*tikv.Lock{lock})
|
||||
timeBeforeExpire, _, err := resolver.ResolveLocks(bo, currentTS, []*txnkv.Lock{lock})
|
||||
s.Nil(err)
|
||||
s.True(timeBeforeExpire > int64(0))
|
||||
|
||||
// Force rollback the lock using lock.TTL = 0.
|
||||
lock.TTL = uint64(0)
|
||||
timeBeforeExpire, _, err = resolver.ResolveLocks(bo, currentTS, []*tikv.Lock{lock})
|
||||
timeBeforeExpire, _, err = resolver.ResolveLocks(bo, currentTS, []*txnkv.Lock{lock})
|
||||
s.Nil(err)
|
||||
s.Equal(timeBeforeExpire, int64(0))
|
||||
|
||||
|
|
@ -375,7 +377,7 @@ func (s *testLockSuite) TestCheckTxnStatusNoWait() {
|
|||
errCh <- committer.PrewriteMutations(context.Background(), committer.MutationsOfKeys([][]byte{[]byte("key")}))
|
||||
}()
|
||||
|
||||
lock := &tikv.Lock{
|
||||
lock := &txnkv.Lock{
|
||||
Key: []byte("second"),
|
||||
Primary: []byte("key"),
|
||||
TxnID: txn.StartTS(),
|
||||
|
|
@ -391,7 +393,7 @@ func (s *testLockSuite) TestCheckTxnStatusNoWait() {
|
|||
// Call getTxnStatusFromLock to cover TxnNotFound and retry timeout.
|
||||
startTS, err := o.GetTimestamp(context.Background(), &oracle.Option{TxnScope: oracle.GlobalTxnScope})
|
||||
s.Nil(err)
|
||||
lock = &tikv.Lock{
|
||||
lock = &txnkv.Lock{
|
||||
Key: []byte("second"),
|
||||
Primary: []byte("key_not_exist"),
|
||||
TxnID: startTS,
|
||||
|
|
@ -419,7 +421,7 @@ func (s *testLockSuite) prewriteTxnWithTTL(txn tikv.TxnProbe, ttl uint64) {
|
|||
s.Nil(err)
|
||||
}
|
||||
|
||||
func (s *testLockSuite) mustGetLock(key []byte) *tikv.Lock {
|
||||
func (s *testLockSuite) mustGetLock(key []byte) *txnkv.Lock {
|
||||
ver, err := s.store.CurrentTimestamp(oracle.GlobalTxnScope)
|
||||
s.Nil(err)
|
||||
bo := tikv.NewBackofferWithVars(context.Background(), getMaxBackoff, nil)
|
||||
|
|
@ -434,7 +436,7 @@ func (s *testLockSuite) mustGetLock(key []byte) *tikv.Lock {
|
|||
s.NotNil(resp.Resp)
|
||||
keyErr := resp.Resp.(*kvrpcpb.GetResponse).GetError()
|
||||
s.NotNil(keyErr)
|
||||
lock, err := tikv.LockProbe{}.ExtractLockFromKeyErr(keyErr)
|
||||
lock, err := tikv.ExtractLockFromKeyErr(keyErr)
|
||||
s.Nil(err)
|
||||
return lock
|
||||
}
|
||||
|
|
@ -506,7 +508,7 @@ func (s *testLockSuite) TestBatchResolveLocks() {
|
|||
committer.PrewriteAllMutations(context.Background())
|
||||
s.Nil(err)
|
||||
|
||||
var locks []*tikv.Lock
|
||||
var locks []*txnkv.Lock
|
||||
for _, key := range []string{"k1", "k2", "k3", "k4"} {
|
||||
l := s.mustGetLock([]byte(key))
|
||||
locks = append(locks, l)
|
||||
|
|
@ -544,7 +546,7 @@ func (s *testLockSuite) TestBatchResolveLocks() {
|
|||
}
|
||||
|
||||
func (s *testLockSuite) TestNewLockZeroTTL() {
|
||||
l := tikv.NewLock(&kvrpcpb.LockInfo{})
|
||||
l := txnlock.NewLock(&kvrpcpb.LockInfo{})
|
||||
s.Equal(l.TTL, uint64(0))
|
||||
}
|
||||
|
||||
|
|
@ -565,19 +567,19 @@ func (s *testLockSuite) TestZeroMinCommitTS() {
|
|||
s.Nil(failpoint.Disable("tikvclient/mockZeroCommitTS"))
|
||||
|
||||
lock := s.mustGetLock([]byte("key"))
|
||||
expire, pushed, err := s.store.NewLockResolver().ResolveLocks(bo, 0, []*tikv.Lock{lock})
|
||||
expire, pushed, err := s.store.NewLockResolver().ResolveLocks(bo, 0, []*txnkv.Lock{lock})
|
||||
s.Nil(err)
|
||||
s.Len(pushed, 0)
|
||||
s.Greater(expire, int64(0))
|
||||
|
||||
expire, pushed, err = s.store.NewLockResolver().ResolveLocks(bo, math.MaxUint64, []*tikv.Lock{lock})
|
||||
expire, pushed, err = s.store.NewLockResolver().ResolveLocks(bo, math.MaxUint64, []*txnkv.Lock{lock})
|
||||
s.Nil(err)
|
||||
s.Len(pushed, 1)
|
||||
s.Greater(expire, int64(0))
|
||||
|
||||
// Clean up this test.
|
||||
lock.TTL = uint64(0)
|
||||
expire, _, err = s.store.NewLockResolver().ResolveLocks(bo, 0, []*tikv.Lock{lock})
|
||||
expire, _, err = s.store.NewLockResolver().ResolveLocks(bo, 0, []*txnkv.Lock{lock})
|
||||
s.Nil(err)
|
||||
s.Equal(expire, int64(0))
|
||||
}
|
||||
|
|
@ -617,7 +619,7 @@ func (s *testLockSuite) TestCheckLocksFallenBackFromAsyncCommit() {
|
|||
lr := s.store.NewLockResolver()
|
||||
status, err := lr.GetTxnStatusFromLock(bo, lock, 0, false)
|
||||
s.Nil(err)
|
||||
s.Equal(tikv.LockProbe{}.GetPrimaryKeyFromTxnStatus(status), []byte("fb1"))
|
||||
s.Equal(txnlock.LockProbe{}.GetPrimaryKeyFromTxnStatus(status), []byte("fb1"))
|
||||
|
||||
err = lr.CheckAllSecondaries(bo, lock, &status)
|
||||
s.True(lr.IsNonAsyncCommitLock(err))
|
||||
|
|
@ -634,7 +636,7 @@ func (s *testLockSuite) TestResolveTxnFallenBackFromAsyncCommit() {
|
|||
lock := s.mustGetLock([]byte("fb1"))
|
||||
s.True(lock.UseAsyncCommit)
|
||||
bo := tikv.NewBackoffer(context.Background(), getMaxBackoff)
|
||||
expire, pushed, err := s.store.NewLockResolver().ResolveLocks(bo, 0, []*tikv.Lock{lock})
|
||||
expire, pushed, err := s.store.NewLockResolver().ResolveLocks(bo, 0, []*txnkv.Lock{lock})
|
||||
s.Nil(err)
|
||||
s.Equal(expire, int64(0))
|
||||
s.Equal(len(pushed), 0)
|
||||
|
|
@ -655,7 +657,7 @@ func (s *testLockSuite) TestBatchResolveTxnFallenBackFromAsyncCommit() {
|
|||
bo := tikv.NewBackoffer(context.Background(), getMaxBackoff)
|
||||
loc, err := s.store.GetRegionCache().LocateKey(bo, []byte("fb1"))
|
||||
s.Nil(err)
|
||||
ok, err := s.store.NewLockResolver().BatchResolveLocks(bo, []*tikv.Lock{lock}, loc.Region)
|
||||
ok, err := s.store.NewLockResolver().BatchResolveLocks(bo, []*txnkv.Lock{lock}, loc.Region)
|
||||
s.Nil(err)
|
||||
s.True(ok)
|
||||
|
||||
|
|
|
|||
|
|
@ -485,6 +485,14 @@ func (c *twoPhaseCommitter) asyncSecondaries() [][]byte {
|
|||
|
||||
const bytesPerMiB = 1024 * 1024
|
||||
|
||||
// ttl = ttlFactor * sqrt(writeSizeInMiB)
|
||||
var ttlFactor = 6000
|
||||
|
||||
// By default, locks after 3000ms is considered unusual (the client created the
|
||||
// lock might be dead). Other client may cleanup this kind of lock.
|
||||
// For locks created recently, we will do backoff and retry.
|
||||
var defaultLockTTL uint64 = 3000
|
||||
|
||||
func txnLockTTL(startTime time.Time, txnSize int) uint64 {
|
||||
// Increase lockTTL for large transactions.
|
||||
// The formula is `ttl = ttlFactor * sqrt(sizeInMiB)`.
|
||||
|
|
|
|||
|
|
@ -37,6 +37,7 @@ import (
|
|||
|
||||
"github.com/tikv/client-go/v2/internal/locate"
|
||||
"github.com/tikv/client-go/v2/tikvrpc"
|
||||
"github.com/tikv/client-go/v2/txnkv/txnlock"
|
||||
"github.com/tikv/client-go/v2/util"
|
||||
)
|
||||
|
||||
|
|
@ -48,7 +49,7 @@ import (
|
|||
// the request. If there is no context information about the resolved locks, we'll
|
||||
// meet the secondary lock again and run into a deadloop.
|
||||
type ClientHelper struct {
|
||||
lockResolver *LockResolver
|
||||
lockResolver *txnlock.LockResolver
|
||||
regionCache *locate.RegionCache
|
||||
resolvedLocks *util.TSSet
|
||||
client Client
|
||||
|
|
@ -67,7 +68,7 @@ func NewClientHelper(store *KVStore, resolvedLocks *util.TSSet) *ClientHelper {
|
|||
}
|
||||
|
||||
// ResolveLocks wraps the ResolveLocks function and store the resolved result.
|
||||
func (ch *ClientHelper) ResolveLocks(bo *Backoffer, callerStartTS uint64, locks []*Lock) (int64, error) {
|
||||
func (ch *ClientHelper) ResolveLocks(bo *Backoffer, callerStartTS uint64, locks []*txnlock.Lock) (int64, error) {
|
||||
var err error
|
||||
var resolvedLocks []uint64
|
||||
var msBeforeTxnExpired int64
|
||||
|
|
|
|||
11
tikv/gc.go
11
tikv/gc.go
|
|
@ -30,6 +30,7 @@ import (
|
|||
"github.com/tikv/client-go/v2/metrics"
|
||||
"github.com/tikv/client-go/v2/tikvrpc"
|
||||
"github.com/tikv/client-go/v2/txnkv/rangetask"
|
||||
"github.com/tikv/client-go/v2/txnkv/txnlock"
|
||||
zap "go.uber.org/zap"
|
||||
)
|
||||
|
||||
|
|
@ -70,7 +71,7 @@ func (s *KVStore) resolveLocks(ctx context.Context, safePoint uint64, concurrenc
|
|||
}
|
||||
|
||||
// We don't want gc to sweep out the cached info belong to other processes, like coprocessor.
|
||||
const gcScanLockLimit = ResolvedCacheSize / 2
|
||||
const gcScanLockLimit = txnlock.ResolvedCacheSize / 2
|
||||
|
||||
func (s *KVStore) resolveLocksForRange(ctx context.Context, safePoint uint64, startKey []byte, endKey []byte) (rangetask.TaskStat, error) {
|
||||
// for scan lock request, we must return all locks even if they are generated
|
||||
|
|
@ -122,7 +123,7 @@ func (s *KVStore) resolveLocksForRange(ctx context.Context, safePoint uint64, st
|
|||
return stat, nil
|
||||
}
|
||||
|
||||
func (s *KVStore) scanLocksInRegionWithStartKey(bo *retry.Backoffer, startKey []byte, maxVersion uint64, limit uint32) (locks []*Lock, loc *locate.KeyLocation, err error) {
|
||||
func (s *KVStore) scanLocksInRegionWithStartKey(bo *retry.Backoffer, startKey []byte, maxVersion uint64, limit uint32) (locks []*txnlock.Lock, loc *locate.KeyLocation, err error) {
|
||||
for {
|
||||
loc, err := s.GetRegionCache().LocateKey(bo, startKey)
|
||||
if err != nil {
|
||||
|
|
@ -157,9 +158,9 @@ func (s *KVStore) scanLocksInRegionWithStartKey(bo *retry.Backoffer, startKey []
|
|||
return nil, loc, errors.Errorf("unexpected scanlock error: %s", locksResp)
|
||||
}
|
||||
locksInfo := locksResp.GetLocks()
|
||||
locks = make([]*Lock, len(locksInfo))
|
||||
locks = make([]*txnlock.Lock, len(locksInfo))
|
||||
for i := range locksInfo {
|
||||
locks[i] = NewLock(locksInfo[i])
|
||||
locks[i] = txnlock.NewLock(locksInfo[i])
|
||||
}
|
||||
return locks, loc, nil
|
||||
}
|
||||
|
|
@ -170,7 +171,7 @@ func (s *KVStore) scanLocksInRegionWithStartKey(bo *retry.Backoffer, startKey []
|
|||
// It returns error when meet an unretryable error.
|
||||
// When the locks are not in one region, resolve locks should be failed, it returns with nil resolveLocation and nil err.
|
||||
// Used it in gcworker only!
|
||||
func (s *KVStore) batchResolveLocksInARegion(bo *Backoffer, locks []*Lock, expectedLoc *locate.KeyLocation) (resolvedLocation *locate.KeyLocation, err error) {
|
||||
func (s *KVStore) batchResolveLocksInARegion(bo *Backoffer, locks []*txnlock.Lock, expectedLoc *locate.KeyLocation) (resolvedLocation *locate.KeyLocation, err error) {
|
||||
resolvedLocation = expectedLoc
|
||||
for {
|
||||
ok, err := s.GetLockResolver().BatchResolveLocks(bo, locks, resolvedLocation.Region)
|
||||
|
|
|
|||
|
|
@ -38,6 +38,7 @@ import (
|
|||
"github.com/tikv/client-go/v2/internal/locate"
|
||||
"github.com/tikv/client-go/v2/oracle"
|
||||
"github.com/tikv/client-go/v2/tikvrpc"
|
||||
"github.com/tikv/client-go/v2/txnkv/txnlock"
|
||||
)
|
||||
|
||||
// Storage represent the kv.Storage runs on TiKV.
|
||||
|
|
@ -49,7 +50,7 @@ type Storage interface {
|
|||
SendReq(bo *Backoffer, req *tikvrpc.Request, regionID locate.RegionVerID, timeout time.Duration) (*tikvrpc.Response, error)
|
||||
|
||||
// GetLockResolver gets the LockResolver.
|
||||
GetLockResolver() *LockResolver
|
||||
GetLockResolver() *txnlock.LockResolver
|
||||
|
||||
// GetSafePointKV gets the SafePointKV.
|
||||
GetSafePointKV() SafePointKV
|
||||
|
|
|
|||
42
tikv/kv.go
42
tikv/kv.go
|
|
@ -60,6 +60,7 @@ import (
|
|||
"github.com/tikv/client-go/v2/oracle/oracles"
|
||||
"github.com/tikv/client-go/v2/tikvrpc"
|
||||
"github.com/tikv/client-go/v2/txnkv/rangetask"
|
||||
"github.com/tikv/client-go/v2/txnkv/txnlock"
|
||||
"github.com/tikv/client-go/v2/util"
|
||||
pd "github.com/tikv/pd/client"
|
||||
"go.etcd.io/etcd/clientv3"
|
||||
|
|
@ -101,7 +102,7 @@ type KVStore struct {
|
|||
}
|
||||
pdClient pd.Client
|
||||
regionCache *locate.RegionCache
|
||||
lockResolver *LockResolver
|
||||
lockResolver *txnlock.LockResolver
|
||||
txnLatches *latch.LatchesScheduler
|
||||
|
||||
mock bool
|
||||
|
|
@ -176,7 +177,7 @@ func NewKVStore(uuid string, pdClient pd.Client, spkv SafePointKV, tikvclient Cl
|
|||
cancel: cancel,
|
||||
}
|
||||
store.clientMu.client = client.NewReqCollapse(tikvclient)
|
||||
store.lockResolver = newLockResolver(store)
|
||||
store.lockResolver = txnlock.NewLockResolver(store)
|
||||
|
||||
store.wg.Add(2)
|
||||
go store.runSafePointChecker()
|
||||
|
|
@ -401,7 +402,7 @@ func (s *KVStore) GetRegionCache() *locate.RegionCache {
|
|||
}
|
||||
|
||||
// GetLockResolver returns the lock resolver instance.
|
||||
func (s *KVStore) GetLockResolver() *LockResolver {
|
||||
func (s *KVStore) GetLockResolver() *txnlock.LockResolver {
|
||||
return s.lockResolver
|
||||
}
|
||||
|
||||
|
|
@ -535,3 +536,38 @@ func (s *KVStore) updateSafeTS(ctx context.Context) {
|
|||
|
||||
// Variables defines the variables used by TiKV storage.
|
||||
type Variables = kv.Variables
|
||||
|
||||
// NewLockResolver is exported for other pkg to use, suppress unused warning.
|
||||
var _ = NewLockResolver
|
||||
|
||||
// NewLockResolver creates a LockResolver.
|
||||
// It is exported for other pkg to use. For instance, binlog service needs
|
||||
// to determine a transaction's commit state.
|
||||
func NewLockResolver(etcdAddrs []string, security config.Security, opts ...pd.ClientOption) (*txnlock.LockResolver, error) {
|
||||
pdCli, err := pd.NewClient(etcdAddrs, pd.SecurityOption{
|
||||
CAPath: security.ClusterSSLCA,
|
||||
CertPath: security.ClusterSSLCert,
|
||||
KeyPath: security.ClusterSSLKey,
|
||||
}, opts...)
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
pdCli = util.InterceptedPDClient{Client: pdCli}
|
||||
uuid := fmt.Sprintf("tikv-%v", pdCli.GetClusterID(context.TODO()))
|
||||
|
||||
tlsConfig, err := security.ToTLSConfig()
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
|
||||
spkv, err := NewEtcdSafePointKV(etcdAddrs, tlsConfig)
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
|
||||
s, err := NewKVStore(uuid, locate.NewCodeCPDClient(pdCli), spkv, client.NewRPCClient(security))
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
return s.lockResolver, nil
|
||||
}
|
||||
|
|
|
|||
|
|
@ -50,6 +50,7 @@ import (
|
|||
"github.com/tikv/client-go/v2/kv"
|
||||
"github.com/tikv/client-go/v2/metrics"
|
||||
"github.com/tikv/client-go/v2/tikvrpc"
|
||||
"github.com/tikv/client-go/v2/txnkv/txnlock"
|
||||
"github.com/tikv/client-go/v2/util"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
|
@ -186,7 +187,7 @@ func (action actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo *
|
|||
}
|
||||
return nil
|
||||
}
|
||||
var locks []*Lock
|
||||
var locks []*txnlock.Lock
|
||||
for _, keyErr := range keyErrs {
|
||||
// Check already exists error
|
||||
if alreadyExist := keyErr.GetAlreadyExist(); alreadyExist != nil {
|
||||
|
|
|
|||
|
|
@ -50,6 +50,7 @@ import (
|
|||
"github.com/tikv/client-go/v2/internal/retry"
|
||||
"github.com/tikv/client-go/v2/metrics"
|
||||
"github.com/tikv/client-go/v2/tikvrpc"
|
||||
"github.com/tikv/client-go/v2/txnkv/txnlock"
|
||||
"github.com/tikv/client-go/v2/util"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
|
@ -294,7 +295,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff
|
|||
}
|
||||
return nil
|
||||
}
|
||||
var locks []*Lock
|
||||
var locks []*txnlock.Lock
|
||||
for _, keyErr := range keyErrs {
|
||||
// Check already exists error
|
||||
if alreadyExist := keyErr.GetAlreadyExist(); alreadyExist != nil {
|
||||
|
|
@ -313,7 +314,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff
|
|||
locks = append(locks, lock)
|
||||
}
|
||||
start := time.Now()
|
||||
msBeforeExpired, err := c.store.lockResolver.resolveLocksForWrite(bo, c.startTS, c.forUpdateTS, locks)
|
||||
msBeforeExpired, err := c.store.lockResolver.ResolveLocksForWrite(bo, c.startTS, c.forUpdateTS, locks)
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -45,6 +45,7 @@ import (
|
|||
"github.com/tikv/client-go/v2/internal/retry"
|
||||
"github.com/tikv/client-go/v2/kv"
|
||||
"github.com/tikv/client-go/v2/tikvrpc"
|
||||
"github.com/tikv/client-go/v2/txnkv/txnlock"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
|
|
@ -277,7 +278,7 @@ func (s *Scanner) getData(bo *Backoffer) error {
|
|||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
msBeforeExpired, _, err := newLockResolver(s.snapshot.store).ResolveLocks(bo, s.snapshot.version, []*Lock{lock})
|
||||
msBeforeExpired, _, err := txnlock.NewLockResolver(s.snapshot.store).ResolveLocks(bo, s.snapshot.version, []*txnlock.Lock{lock})
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -54,6 +54,7 @@ import (
|
|||
"github.com/tikv/client-go/v2/kv"
|
||||
"github.com/tikv/client-go/v2/metrics"
|
||||
"github.com/tikv/client-go/v2/tikvrpc"
|
||||
"github.com/tikv/client-go/v2/txnkv/txnlock"
|
||||
"github.com/tikv/client-go/v2/util"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
|
@ -395,7 +396,7 @@ func (s *KVSnapshot) batchGetSingleRegion(bo *Backoffer, batch batchKeys, collec
|
|||
batchGetResp := resp.Resp.(*kvrpcpb.BatchGetResponse)
|
||||
var (
|
||||
lockedKeys [][]byte
|
||||
locks []*Lock
|
||||
locks []*txnlock.Lock
|
||||
)
|
||||
if keyErr := batchGetResp.GetError(); keyErr != nil {
|
||||
// If a response-level error happens, skip reading pairs.
|
||||
|
|
@ -528,7 +529,7 @@ func (s *KVSnapshot) get(ctx context.Context, bo *Backoffer, k []byte) ([]byte,
|
|||
ops = append(ops, locate.WithMatchLabels(matchStoreLabels))
|
||||
}
|
||||
|
||||
var firstLock *Lock
|
||||
var firstLock *txnlock.Lock
|
||||
for {
|
||||
util.EvalFailpoint("beforeSendPointGet")
|
||||
loc, err := s.store.regionCache.LocateKey(bo, k)
|
||||
|
|
@ -581,7 +582,7 @@ func (s *KVSnapshot) get(ctx context.Context, bo *Backoffer, k []byte) ([]byte,
|
|||
continue
|
||||
}
|
||||
|
||||
msBeforeExpired, err := cli.ResolveLocks(bo, s.version, []*Lock{lock})
|
||||
msBeforeExpired, err := cli.ResolveLocks(bo, s.version, []*txnlock.Lock{lock})
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
|
|
@ -717,9 +718,9 @@ func (s *KVSnapshot) SnapCacheSize() int {
|
|||
return len(s.mu.cached)
|
||||
}
|
||||
|
||||
func extractLockFromKeyErr(keyErr *kvrpcpb.KeyError) (*Lock, error) {
|
||||
func extractLockFromKeyErr(keyErr *kvrpcpb.KeyError) (*txnlock.Lock, error) {
|
||||
if locked := keyErr.GetLocked(); locked != nil {
|
||||
return NewLock(locked), nil
|
||||
return txnlock.NewLock(locked), nil
|
||||
}
|
||||
return nil, extractKeyErr(keyErr)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -45,6 +45,7 @@ import (
|
|||
"github.com/tikv/client-go/v2/internal/retry"
|
||||
"github.com/tikv/client-go/v2/internal/unionstore"
|
||||
"github.com/tikv/client-go/v2/tikvrpc"
|
||||
"github.com/tikv/client-go/v2/txnkv/txnlock"
|
||||
pd "github.com/tikv/pd/client"
|
||||
)
|
||||
|
||||
|
|
@ -55,7 +56,8 @@ type StoreProbe struct {
|
|||
|
||||
// NewLockResolver creates a new LockResolver instance.
|
||||
func (s StoreProbe) NewLockResolver() LockResolverProbe {
|
||||
return LockResolverProbe{LockResolver: newLockResolver(s.KVStore)}
|
||||
txnLockResolver := txnlock.LockResolverProbe{LockResolver: txnlock.NewLockResolver(s.KVStore)}
|
||||
return LockResolverProbe{&txnLockResolver}
|
||||
}
|
||||
|
||||
// GetTimestampWithRetry returns latest timestamp.
|
||||
|
|
@ -432,90 +434,32 @@ func (s SnapshotProbe) FormatStats() string {
|
|||
return s.mu.stats.String()
|
||||
}
|
||||
|
||||
// LockProbe exposes some lock utilities for testing purpose.
|
||||
type LockProbe struct{}
|
||||
|
||||
// ExtractLockFromKeyErr makes a Lock based on a key error.
|
||||
func (l LockProbe) ExtractLockFromKeyErr(err *kvrpcpb.KeyError) (*Lock, error) {
|
||||
return extractLockFromKeyErr(err)
|
||||
}
|
||||
|
||||
// NewLockStatus returns a txn state that has been locked.
|
||||
func (l LockProbe) NewLockStatus(keys [][]byte, useAsyncCommit bool, minCommitTS uint64) TxnStatus {
|
||||
return TxnStatus{
|
||||
primaryLock: &kvrpcpb.LockInfo{
|
||||
Secondaries: keys,
|
||||
UseAsyncCommit: useAsyncCommit,
|
||||
MinCommitTs: minCommitTS,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// GetPrimaryKeyFromTxnStatus returns the primary key of the transaction.
|
||||
func (l LockProbe) GetPrimaryKeyFromTxnStatus(s TxnStatus) []byte {
|
||||
return s.primaryLock.Key
|
||||
}
|
||||
|
||||
// LockResolverProbe wraps a LockResolver and exposes internal stats for testing purpose.
|
||||
type LockResolverProbe struct {
|
||||
*LockResolver
|
||||
*txnlock.LockResolverProbe
|
||||
}
|
||||
|
||||
// ResolveLockAsync tries to resolve a lock using the txn states.
|
||||
func (l LockResolverProbe) ResolveLockAsync(bo *Backoffer, lock *Lock, status TxnStatus) error {
|
||||
return l.resolveLockAsync(bo, lock, status)
|
||||
// NewLockResolverProb create a LockResolverProbe from KVStore.
|
||||
func NewLockResolverProb(r *txnlock.LockResolver) *LockResolverProbe {
|
||||
resolver := txnlock.LockResolverProbe{LockResolver: r}
|
||||
return &LockResolverProbe{&resolver}
|
||||
}
|
||||
|
||||
// ResolveLock resolves single lock.
|
||||
func (l LockResolverProbe) ResolveLock(ctx context.Context, lock *Lock) error {
|
||||
func (l LockResolverProbe) ResolveLock(ctx context.Context, lock *txnlock.Lock) error {
|
||||
bo := retry.NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, nil)
|
||||
return l.resolveLock(bo, lock, TxnStatus{}, false, make(map[locate.RegionVerID]struct{}))
|
||||
return l.LockResolverProbe.ResolveLock(bo, lock)
|
||||
}
|
||||
|
||||
// ResolvePessimisticLock resolves single pessimistic lock.
|
||||
func (l LockResolverProbe) ResolvePessimisticLock(ctx context.Context, lock *Lock) error {
|
||||
func (l LockResolverProbe) ResolvePessimisticLock(ctx context.Context, lock *txnlock.Lock) error {
|
||||
bo := retry.NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, nil)
|
||||
return l.resolvePessimisticLock(bo, lock, make(map[locate.RegionVerID]struct{}))
|
||||
return l.LockResolverProbe.ResolvePessimisticLock(bo, lock)
|
||||
}
|
||||
|
||||
// GetTxnStatus sends the CheckTxnStatus request to the TiKV server.
|
||||
func (l LockResolverProbe) GetTxnStatus(bo *Backoffer, txnID uint64, primary []byte,
|
||||
callerStartTS, currentTS uint64, rollbackIfNotExist bool, forceSyncCommit bool, lockInfo *Lock) (TxnStatus, error) {
|
||||
return l.getTxnStatus(bo, txnID, primary, callerStartTS, currentTS, rollbackIfNotExist, forceSyncCommit, lockInfo)
|
||||
}
|
||||
|
||||
// GetTxnStatusFromLock queries tikv for a txn's status.
|
||||
func (l LockResolverProbe) GetTxnStatusFromLock(bo *Backoffer, lock *Lock, callerStartTS uint64, forceSyncCommit bool) (TxnStatus, error) {
|
||||
return l.getTxnStatusFromLock(bo, lock, callerStartTS, forceSyncCommit)
|
||||
}
|
||||
|
||||
// GetSecondariesFromTxnStatus returns the secondary locks from txn status.
|
||||
func (l LockResolverProbe) GetSecondariesFromTxnStatus(status TxnStatus) [][]byte {
|
||||
return status.primaryLock.GetSecondaries()
|
||||
}
|
||||
|
||||
// SetMeetLockCallback is called whenever it meets locks.
|
||||
func (l LockResolverProbe) SetMeetLockCallback(f func([]*Lock)) {
|
||||
l.testingKnobs.meetLock = f
|
||||
}
|
||||
|
||||
// CheckAllSecondaries checks the secondary locks of an async commit transaction to find out the final
|
||||
// status of the transaction.
|
||||
func (l LockResolverProbe) CheckAllSecondaries(bo *Backoffer, lock *Lock, status *TxnStatus) error {
|
||||
_, err := l.checkAllSecondaries(bo, lock, status)
|
||||
return err
|
||||
}
|
||||
|
||||
// IsErrorNotFound checks if an error is caused by txnNotFoundErr.
|
||||
func (l LockResolverProbe) IsErrorNotFound(err error) bool {
|
||||
_, ok := errors.Cause(err).(txnNotFoundErr)
|
||||
return ok
|
||||
}
|
||||
|
||||
// IsNonAsyncCommitLock checks if an error is nonAsyncCommitLock error.
|
||||
func (l LockResolverProbe) IsNonAsyncCommitLock(err error) bool {
|
||||
_, ok := errors.Cause(err).(*nonAsyncCommitLock)
|
||||
return ok
|
||||
// ExtractLockFromKeyErr makes a Lock based on a key error.
|
||||
func ExtractLockFromKeyErr(err *kvrpcpb.KeyError) (*txnlock.Lock, error) {
|
||||
return extractLockFromKeyErr(err)
|
||||
}
|
||||
|
||||
// ConfigProbe exposes configurations and global variables for testing purpose.
|
||||
|
|
@ -528,6 +472,8 @@ func (c ConfigProbe) GetTxnCommitBatchSize() uint64 {
|
|||
|
||||
// GetBigTxnThreshold returns the txn size to be considered as big txn.
|
||||
func (c ConfigProbe) GetBigTxnThreshold() int {
|
||||
// bigTxnThreshold : transaction involves keys exceed this threshold can be treated as `big transaction`.
|
||||
const bigTxnThreshold = 16
|
||||
return bigTxnThreshold
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -0,0 +1,25 @@
|
|||
// Copyright 2021 TiKV Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package txnkv
|
||||
|
||||
import "github.com/tikv/client-go/v2/txnkv/txnlock"
|
||||
|
||||
// Lock represents a lock from tikv server.
|
||||
type Lock = txnlock.Lock
|
||||
|
||||
// LockResolver resolves locks and also caches resolved txn status.
|
||||
type LockResolver = txnlock.LockResolver
|
||||
|
||||
// TxnStatus represents a txn's final status. It should be Lock or Commit or Rollback.
|
||||
type TxnStatus = txnlock.TxnStatus
|
||||
|
|
@ -11,26 +11,7 @@
|
|||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
// NOTE: The code in this file is based on code from the
|
||||
// TiDB project, licensed under the Apache License v 2.0
|
||||
//
|
||||
// https://github.com/pingcap/tidb/tree/cc5e161ac06827589c4966674597c137cc9e809c/store/tikv/lock_resolver.go
|
||||
//
|
||||
|
||||
// Copyright 2016 PingCAP, Inc.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package tikv
|
||||
package txnlock
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
|
|
@ -44,7 +25,6 @@ import (
|
|||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/tikv/client-go/v2/config"
|
||||
tikverr "github.com/tikv/client-go/v2/error"
|
||||
"github.com/tikv/client-go/v2/internal/client"
|
||||
"github.com/tikv/client-go/v2/internal/locate"
|
||||
|
|
@ -54,19 +34,24 @@ import (
|
|||
"github.com/tikv/client-go/v2/oracle"
|
||||
"github.com/tikv/client-go/v2/tikvrpc"
|
||||
"github.com/tikv/client-go/v2/util"
|
||||
pd "github.com/tikv/pd/client"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
// ResolvedCacheSize is max number of cached txn status.
|
||||
const ResolvedCacheSize = 2048
|
||||
|
||||
// bigTxnThreshold : transaction involves keys exceed this threshold can be treated as `big transaction`.
|
||||
const bigTxnThreshold = 16
|
||||
type storage interface {
|
||||
// GetRegionCache gets the RegionCache.
|
||||
GetRegionCache() *locate.RegionCache
|
||||
// SendReq sends a request to TiKV.
|
||||
SendReq(bo *retry.Backoffer, req *tikvrpc.Request, regionID locate.RegionVerID, timeout time.Duration) (*tikvrpc.Response, error)
|
||||
// GetOracle gets a timestamp oracle client.
|
||||
GetOracle() oracle.Oracle
|
||||
}
|
||||
|
||||
// LockResolver resolves locks and also caches resolved txn status.
|
||||
type LockResolver struct {
|
||||
store *KVStore
|
||||
store storage
|
||||
mu struct {
|
||||
sync.RWMutex
|
||||
// resolved caches resolved txns (FIFO, txn id -> txnStatus).
|
||||
|
|
@ -78,7 +63,8 @@ type LockResolver struct {
|
|||
}
|
||||
}
|
||||
|
||||
func newLockResolver(store *KVStore) *LockResolver {
|
||||
// NewLockResolver creates a new LockResolver instance.
|
||||
func NewLockResolver(store storage) *LockResolver {
|
||||
r := &LockResolver{
|
||||
store: store,
|
||||
}
|
||||
|
|
@ -87,41 +73,6 @@ func newLockResolver(store *KVStore) *LockResolver {
|
|||
return r
|
||||
}
|
||||
|
||||
// NewLockResolver is exported for other pkg to use, suppress unused warning.
|
||||
var _ = NewLockResolver
|
||||
|
||||
// NewLockResolver creates a LockResolver.
|
||||
// It is exported for other pkg to use. For instance, binlog service needs
|
||||
// to determine a transaction's commit state.
|
||||
func NewLockResolver(etcdAddrs []string, security config.Security, opts ...pd.ClientOption) (*LockResolver, error) {
|
||||
pdCli, err := pd.NewClient(etcdAddrs, pd.SecurityOption{
|
||||
CAPath: security.ClusterSSLCA,
|
||||
CertPath: security.ClusterSSLCert,
|
||||
KeyPath: security.ClusterSSLKey,
|
||||
}, opts...)
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
pdCli = util.InterceptedPDClient{Client: pdCli}
|
||||
uuid := fmt.Sprintf("tikv-%v", pdCli.GetClusterID(context.TODO()))
|
||||
|
||||
tlsConfig, err := security.ToTLSConfig()
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
|
||||
spkv, err := NewEtcdSafePointKV(etcdAddrs, tlsConfig)
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
|
||||
s, err := NewKVStore(uuid, locate.NewCodeCPDClient(pdCli), spkv, client.NewRPCClient(security))
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
return s.lockResolver, nil
|
||||
}
|
||||
|
||||
// TxnStatus represents a txn's final status. It should be Lock or Commit or Rollback.
|
||||
type TxnStatus struct {
|
||||
ttl uint64
|
||||
|
|
@ -166,14 +117,6 @@ func (s TxnStatus) StatusCacheable() bool {
|
|||
return false
|
||||
}
|
||||
|
||||
// By default, locks after 3000ms is considered unusual (the client created the
|
||||
// lock might be dead). Other client may cleanup this kind of lock.
|
||||
// For locks created recently, we will do backoff and retry.
|
||||
var defaultLockTTL uint64 = 3000
|
||||
|
||||
// ttl = ttlFactor * sqrt(writeSizeInMiB)
|
||||
var ttlFactor = 6000
|
||||
|
||||
// Lock represents a lock from tikv server.
|
||||
type Lock struct {
|
||||
Key []byte
|
||||
|
|
@ -238,7 +181,7 @@ func (lr *LockResolver) getResolved(txnID uint64) (TxnStatus, bool) {
|
|||
|
||||
// BatchResolveLocks resolve locks in a batch.
|
||||
// Used it in gcworker only!
|
||||
func (lr *LockResolver) BatchResolveLocks(bo *Backoffer, locks []*Lock, loc locate.RegionVerID) (bool, error) {
|
||||
func (lr *LockResolver) BatchResolveLocks(bo *retry.Backoffer, locks []*Lock, loc locate.RegionVerID) (bool, error) {
|
||||
if len(locks) == 0 {
|
||||
return true, nil
|
||||
}
|
||||
|
|
@ -343,16 +286,16 @@ func (lr *LockResolver) BatchResolveLocks(bo *Backoffer, locks []*Lock, loc loca
|
|||
// commit status.
|
||||
// 3) Send `ResolveLock` cmd to the lock's region to resolve all locks belong to
|
||||
// the same transaction.
|
||||
func (lr *LockResolver) ResolveLocks(bo *Backoffer, callerStartTS uint64, locks []*Lock) (int64, []uint64 /*pushed*/, error) {
|
||||
func (lr *LockResolver) ResolveLocks(bo *retry.Backoffer, callerStartTS uint64, locks []*Lock) (int64, []uint64 /*pushed*/, error) {
|
||||
return lr.resolveLocks(bo, callerStartTS, locks, false, false)
|
||||
}
|
||||
|
||||
// ResolveLocksLite resolves locks while preventing scan whole region.
|
||||
func (lr *LockResolver) ResolveLocksLite(bo *Backoffer, callerStartTS uint64, locks []*Lock) (int64, []uint64 /*pushed*/, error) {
|
||||
func (lr *LockResolver) ResolveLocksLite(bo *retry.Backoffer, callerStartTS uint64, locks []*Lock) (int64, []uint64 /*pushed*/, error) {
|
||||
return lr.resolveLocks(bo, callerStartTS, locks, false, true)
|
||||
}
|
||||
|
||||
func (lr *LockResolver) resolveLocks(bo *Backoffer, callerStartTS uint64, locks []*Lock, forWrite bool, lite bool) (int64, []uint64 /*pushed*/, error) {
|
||||
func (lr *LockResolver) resolveLocks(bo *retry.Backoffer, callerStartTS uint64, locks []*Lock, forWrite bool, lite bool) (int64, []uint64 /*pushed*/, error) {
|
||||
if lr.testingKnobs.meetLock != nil {
|
||||
lr.testingKnobs.meetLock(locks)
|
||||
}
|
||||
|
|
@ -452,7 +395,8 @@ func (lr *LockResolver) resolveLocks(bo *Backoffer, callerStartTS uint64, locks
|
|||
return msBeforeTxnExpired.value(), pushed, nil
|
||||
}
|
||||
|
||||
func (lr *LockResolver) resolveLocksForWrite(bo *Backoffer, callerStartTS, callerForUpdateTS uint64, locks []*Lock) (int64, error) {
|
||||
// ResolveLocksForWrite resolves lock for write
|
||||
func (lr *LockResolver) ResolveLocksForWrite(bo *retry.Backoffer, callerStartTS, callerForUpdateTS uint64, locks []*Lock) (int64, error) {
|
||||
// The forWrite parameter is only useful for optimistic transactions which can avoid deadlock between large transactions,
|
||||
// so only use forWrite if the callerForUpdateTS is zero.
|
||||
msBeforeTxnExpired, _, err := lr.resolveLocks(bo, callerStartTS, locks, callerForUpdateTS == 0, false)
|
||||
|
|
@ -485,13 +429,15 @@ func (t *txnExpireTime) value() int64 {
|
|||
return t.txnExpire
|
||||
}
|
||||
|
||||
const getTxnStatusMaxBackoff = 20000
|
||||
|
||||
// GetTxnStatus queries tikv-server for a txn's status (commit/rollback).
|
||||
// If the primary key is still locked, it will launch a Rollback to abort it.
|
||||
// To avoid unnecessarily aborting too many txns, it is wiser to wait a few
|
||||
// seconds before calling it after Prewrite.
|
||||
func (lr *LockResolver) GetTxnStatus(txnID uint64, callerStartTS uint64, primary []byte) (TxnStatus, error) {
|
||||
var status TxnStatus
|
||||
bo := retry.NewBackoffer(context.Background(), cleanupMaxBackoff)
|
||||
bo := retry.NewBackoffer(context.Background(), getTxnStatusMaxBackoff)
|
||||
currentTS, err := lr.store.GetOracle().GetLowResolutionTimestamp(bo.GetCtx(), &oracle.Option{TxnScope: oracle.GlobalTxnScope})
|
||||
if err != nil {
|
||||
return status, err
|
||||
|
|
@ -499,7 +445,7 @@ func (lr *LockResolver) GetTxnStatus(txnID uint64, callerStartTS uint64, primary
|
|||
return lr.getTxnStatus(bo, txnID, primary, callerStartTS, currentTS, true, false, nil)
|
||||
}
|
||||
|
||||
func (lr *LockResolver) getTxnStatusFromLock(bo *Backoffer, l *Lock, callerStartTS uint64, forceSyncCommit bool) (TxnStatus, error) {
|
||||
func (lr *LockResolver) getTxnStatusFromLock(bo *retry.Backoffer, l *Lock, callerStartTS uint64, forceSyncCommit bool) (TxnStatus, error) {
|
||||
var currentTS uint64
|
||||
var err error
|
||||
var status TxnStatus
|
||||
|
|
@ -575,7 +521,7 @@ func (e txnNotFoundErr) Error() string {
|
|||
|
||||
// getTxnStatus sends the CheckTxnStatus request to the TiKV server.
|
||||
// When rollbackIfNotExist is false, the caller should be careful with the txnNotFoundErr error.
|
||||
func (lr *LockResolver) getTxnStatus(bo *Backoffer, txnID uint64, primary []byte,
|
||||
func (lr *LockResolver) getTxnStatus(bo *retry.Backoffer, txnID uint64, primary []byte,
|
||||
callerStartTS, currentTS uint64, rollbackIfNotExist bool, forceSyncCommit bool, lockInfo *Lock) (TxnStatus, error) {
|
||||
if s, ok := lr.getResolved(txnID); ok {
|
||||
return s, nil
|
||||
|
|
@ -737,7 +683,7 @@ func (data *asyncResolveData) addKeys(locks []*kvrpcpb.LockInfo, expected int, s
|
|||
return nil
|
||||
}
|
||||
|
||||
func (lr *LockResolver) checkSecondaries(bo *Backoffer, txnID uint64, curKeys [][]byte, curRegionID locate.RegionVerID, shared *asyncResolveData) error {
|
||||
func (lr *LockResolver) checkSecondaries(bo *retry.Backoffer, txnID uint64, curKeys [][]byte, curRegionID locate.RegionVerID, shared *asyncResolveData) error {
|
||||
checkReq := &kvrpcpb.CheckSecondaryLocksRequest{
|
||||
Keys: curKeys,
|
||||
StartVersion: txnID,
|
||||
|
|
@ -783,7 +729,7 @@ func (lr *LockResolver) checkSecondaries(bo *Backoffer, txnID uint64, curKeys []
|
|||
}
|
||||
|
||||
// resolveLockAsync resolves l assuming it was locked using the async commit protocol.
|
||||
func (lr *LockResolver) resolveLockAsync(bo *Backoffer, l *Lock, status TxnStatus) error {
|
||||
func (lr *LockResolver) resolveLockAsync(bo *retry.Backoffer, l *Lock, status TxnStatus) error {
|
||||
metrics.LockResolverCountWithResolveAsync.Inc()
|
||||
|
||||
resolveData, err := lr.checkAllSecondaries(bo, l, &status)
|
||||
|
|
@ -831,7 +777,7 @@ func (lr *LockResolver) resolveLockAsync(bo *Backoffer, l *Lock, status TxnStatu
|
|||
|
||||
// checkAllSecondaries checks the secondary locks of an async commit transaction to find out the final
|
||||
// status of the transaction
|
||||
func (lr *LockResolver) checkAllSecondaries(bo *Backoffer, l *Lock, status *TxnStatus) (*asyncResolveData, error) {
|
||||
func (lr *LockResolver) checkAllSecondaries(bo *retry.Backoffer, l *Lock, status *TxnStatus) (*asyncResolveData, error) {
|
||||
regions, _, err := lr.store.GetRegionCache().GroupKeysByRegion(bo, status.primaryLock.Secondaries, nil)
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
|
|
@ -867,7 +813,7 @@ func (lr *LockResolver) checkAllSecondaries(bo *Backoffer, l *Lock, status *TxnS
|
|||
}
|
||||
|
||||
// resolveRegionLocks is essentially the same as resolveLock, but we resolve all keys in the same region at the same time.
|
||||
func (lr *LockResolver) resolveRegionLocks(bo *Backoffer, l *Lock, region locate.RegionVerID, keys [][]byte, status TxnStatus) error {
|
||||
func (lr *LockResolver) resolveRegionLocks(bo *retry.Backoffer, l *Lock, region locate.RegionVerID, keys [][]byte, status TxnStatus) error {
|
||||
lreq := &kvrpcpb.ResolveLockRequest{
|
||||
StartVersion: l.TxnID,
|
||||
}
|
||||
|
|
@ -919,7 +865,10 @@ func (lr *LockResolver) resolveRegionLocks(bo *Backoffer, l *Lock, region locate
|
|||
return nil
|
||||
}
|
||||
|
||||
func (lr *LockResolver) resolveLock(bo *Backoffer, l *Lock, status TxnStatus, lite bool, cleanRegions map[locate.RegionVerID]struct{}) error {
|
||||
// bigTxnThreshold : transaction involves keys exceed this threshold can be treated as `big transaction`.
|
||||
const bigTxnThreshold = 16
|
||||
|
||||
func (lr *LockResolver) resolveLock(bo *retry.Backoffer, l *Lock, status TxnStatus, lite bool, cleanRegions map[locate.RegionVerID]struct{}) error {
|
||||
metrics.LockResolverCountWithResolveLocks.Inc()
|
||||
resolveLite := lite || l.TxnSize < bigTxnThreshold
|
||||
for {
|
||||
|
|
@ -977,7 +926,7 @@ func (lr *LockResolver) resolveLock(bo *Backoffer, l *Lock, status TxnStatus, li
|
|||
}
|
||||
}
|
||||
|
||||
func (lr *LockResolver) resolvePessimisticLock(bo *Backoffer, l *Lock, cleanRegions map[locate.RegionVerID]struct{}) error {
|
||||
func (lr *LockResolver) resolvePessimisticLock(bo *retry.Backoffer, l *Lock, cleanRegions map[locate.RegionVerID]struct{}) error {
|
||||
metrics.LockResolverCountWithResolveLocks.Inc()
|
||||
for {
|
||||
loc, err := lr.store.GetRegionCache().LocateKey(bo, l.Key)
|
||||
|
|
@ -0,0 +1,100 @@
|
|||
// Copyright 2021 TiKV Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package txnlock
|
||||
|
||||
import (
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/tikv/client-go/v2/internal/locate"
|
||||
"github.com/tikv/client-go/v2/internal/retry"
|
||||
)
|
||||
|
||||
// LockProbe exposes some lock utilities for testing purpose.
|
||||
type LockProbe struct{}
|
||||
|
||||
// NewLockStatus returns a txn state that has been locked.
|
||||
func (l LockProbe) NewLockStatus(keys [][]byte, useAsyncCommit bool, minCommitTS uint64) TxnStatus {
|
||||
return TxnStatus{
|
||||
primaryLock: &kvrpcpb.LockInfo{
|
||||
Secondaries: keys,
|
||||
UseAsyncCommit: useAsyncCommit,
|
||||
MinCommitTs: minCommitTS,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// GetPrimaryKeyFromTxnStatus returns the primary key of the transaction.
|
||||
func (l LockProbe) GetPrimaryKeyFromTxnStatus(s TxnStatus) []byte {
|
||||
return s.primaryLock.Key
|
||||
}
|
||||
|
||||
// LockResolverProbe wraps a LockResolver and exposes internal stats for testing purpose.
|
||||
type LockResolverProbe struct {
|
||||
*LockResolver
|
||||
}
|
||||
|
||||
// ResolveLockAsync tries to resolve a lock using the txn states.
|
||||
func (l LockResolverProbe) ResolveLockAsync(bo *retry.Backoffer, lock *Lock, status TxnStatus) error {
|
||||
return l.resolveLockAsync(bo, lock, status)
|
||||
}
|
||||
|
||||
// ResolveLock resolves single lock.
|
||||
func (l LockResolverProbe) ResolveLock(bo *retry.Backoffer, lock *Lock) error {
|
||||
return l.resolveLock(bo, lock, TxnStatus{}, false, make(map[locate.RegionVerID]struct{}))
|
||||
}
|
||||
|
||||
// ResolvePessimisticLock resolves single pessimistic lock.
|
||||
func (l LockResolverProbe) ResolvePessimisticLock(bo *retry.Backoffer, lock *Lock) error {
|
||||
return l.resolvePessimisticLock(bo, lock, make(map[locate.RegionVerID]struct{}))
|
||||
}
|
||||
|
||||
// GetTxnStatus sends the CheckTxnStatus request to the TiKV server.
|
||||
func (l LockResolverProbe) GetTxnStatus(bo *retry.Backoffer, txnID uint64, primary []byte,
|
||||
callerStartTS, currentTS uint64, rollbackIfNotExist bool, forceSyncCommit bool, lockInfo *Lock) (TxnStatus, error) {
|
||||
return l.getTxnStatus(bo, txnID, primary, callerStartTS, currentTS, rollbackIfNotExist, forceSyncCommit, lockInfo)
|
||||
}
|
||||
|
||||
// GetTxnStatusFromLock queries tikv for a txn's status.
|
||||
func (l LockResolverProbe) GetTxnStatusFromLock(bo *retry.Backoffer, lock *Lock, callerStartTS uint64, forceSyncCommit bool) (TxnStatus, error) {
|
||||
return l.getTxnStatusFromLock(bo, lock, callerStartTS, forceSyncCommit)
|
||||
}
|
||||
|
||||
// GetSecondariesFromTxnStatus returns the secondary locks from txn status.
|
||||
func (l LockResolverProbe) GetSecondariesFromTxnStatus(status TxnStatus) [][]byte {
|
||||
return status.primaryLock.GetSecondaries()
|
||||
}
|
||||
|
||||
// SetMeetLockCallback is called whenever it meets locks.
|
||||
func (l LockResolverProbe) SetMeetLockCallback(f func([]*Lock)) {
|
||||
l.testingKnobs.meetLock = f
|
||||
}
|
||||
|
||||
// CheckAllSecondaries checks the secondary locks of an async commit transaction to find out the final
|
||||
// status of the transaction.
|
||||
func (l LockResolverProbe) CheckAllSecondaries(bo *retry.Backoffer, lock *Lock, status *TxnStatus) error {
|
||||
_, err := l.checkAllSecondaries(bo, lock, status)
|
||||
return err
|
||||
}
|
||||
|
||||
// IsErrorNotFound checks if an error is caused by txnNotFoundErr.
|
||||
func (l LockResolverProbe) IsErrorNotFound(err error) bool {
|
||||
_, ok := errors.Cause(err).(txnNotFoundErr)
|
||||
return ok
|
||||
}
|
||||
|
||||
// IsNonAsyncCommitLock checks if an error is nonAsyncCommitLock error.
|
||||
func (l LockResolverProbe) IsNonAsyncCommitLock(err error) bool {
|
||||
_, ok := errors.Cause(err).(*nonAsyncCommitLock)
|
||||
return ok
|
||||
}
|
||||
Loading…
Reference in New Issue