You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
 
 

704 lines
23 KiB

// 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
import (
"bytes"
"container/list"
"context"
"fmt"
"math"
"sync"
"time"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/store/tikv/tikvrpc"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/logutil"
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
var (
tikvLockResolverCountWithBatchResolve = metrics.TiKVLockResolverCounter.WithLabelValues("batch_resolve")
tikvLockResolverCountWithExpired = metrics.TiKVLockResolverCounter.WithLabelValues("expired")
tikvLockResolverCountWithNotExpired = metrics.TiKVLockResolverCounter.WithLabelValues("not_expired")
tikvLockResolverCountWithWaitExpired = metrics.TiKVLockResolverCounter.WithLabelValues("wait_expired")
tikvLockResolverCountWithResolve = metrics.TiKVLockResolverCounter.WithLabelValues("resolve")
tikvLockResolverCountWithResolveForWrite = metrics.TiKVLockResolverCounter.WithLabelValues("resolve_for_write")
tikvLockResolverCountWithWriteConflict = metrics.TiKVLockResolverCounter.WithLabelValues("write_conflict")
tikvLockResolverCountWithQueryTxnStatus = metrics.TiKVLockResolverCounter.WithLabelValues("query_txn_status")
tikvLockResolverCountWithQueryTxnStatusCommitted = metrics.TiKVLockResolverCounter.WithLabelValues("query_txn_status_committed")
tikvLockResolverCountWithQueryTxnStatusRolledBack = metrics.TiKVLockResolverCounter.WithLabelValues("query_txn_status_rolled_back")
tikvLockResolverCountWithResolveLocks = metrics.TiKVLockResolverCounter.WithLabelValues("query_resolve_locks")
tikvLockResolverCountWithResolveLockLite = metrics.TiKVLockResolverCounter.WithLabelValues("query_resolve_lock_lite")
)
// LockResolver resolves locks and also caches resolved txn status.
type LockResolver struct {
store Storage
mu struct {
sync.RWMutex
// resolved caches resolved txns (FIFO, txn id -> txnStatus).
resolved map[uint64]TxnStatus
recentResolved *list.List
}
testingKnobs struct {
meetLock func(locks []*Lock)
}
}
func newLockResolver(store Storage) *LockResolver {
r := &LockResolver{
store: store,
}
r.mu.resolved = make(map[uint64]TxnStatus)
r.mu.recentResolved = list.New()
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 = execdetails.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 := newTikvStore(uuid, &codecPDClient{pdCli}, spkv, newRPCClient(security), false, nil)
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
commitTS uint64
action kvrpcpb.Action
}
// IsCommitted returns true if the txn's final status is Commit.
func (s TxnStatus) IsCommitted() bool { return s.ttl == 0 && s.commitTS > 0 }
// CommitTS returns the txn's commitTS. It is valid iff `IsCommitted` is true.
func (s TxnStatus) CommitTS() uint64 { return uint64(s.commitTS) }
// TTL returns the TTL of the transaction if the transaction is still alive.
func (s TxnStatus) TTL() uint64 { return s.ttl }
// Action returns what the CheckTxnStatus request have done to the transaction.
func (s TxnStatus) Action() kvrpcpb.Action { return s.action }
// 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
Primary []byte
TxnID uint64
TTL uint64
TxnSize uint64
LockType kvrpcpb.Op
LockForUpdateTS uint64
}
func (l *Lock) String() string {
buf := bytes.NewBuffer(make([]byte, 0, 128))
buf.WriteString("key: ")
prettyWriteKey(buf, l.Key)
buf.WriteString(", primary: ")
prettyWriteKey(buf, l.Primary)
return fmt.Sprintf("%s, txnStartTS: %d, lockForUpdateTS:%d, ttl: %d, type: %s", buf.String(), l.TxnID, l.LockForUpdateTS, l.TTL, l.LockType)
}
// NewLock creates a new *Lock.
func NewLock(l *kvrpcpb.LockInfo) *Lock {
return &Lock{
Key: l.GetKey(),
Primary: l.GetPrimaryLock(),
TxnID: l.GetLockVersion(),
TTL: l.GetLockTtl(),
TxnSize: l.GetTxnSize(),
LockType: l.LockType,
LockForUpdateTS: l.LockForUpdateTs,
}
}
func (lr *LockResolver) saveResolved(txnID uint64, status TxnStatus) {
lr.mu.Lock()
defer lr.mu.Unlock()
if _, ok := lr.mu.resolved[txnID]; ok {
return
}
lr.mu.resolved[txnID] = status
lr.mu.recentResolved.PushBack(txnID)
if len(lr.mu.resolved) > ResolvedCacheSize {
front := lr.mu.recentResolved.Front()
delete(lr.mu.resolved, front.Value.(uint64))
lr.mu.recentResolved.Remove(front)
}
}
func (lr *LockResolver) getResolved(txnID uint64) (TxnStatus, bool) {
lr.mu.RLock()
defer lr.mu.RUnlock()
s, ok := lr.mu.resolved[txnID]
return s, ok
}
// BatchResolveLocks resolve locks in a batch.
// Used it in gcworker only!
func (lr *LockResolver) BatchResolveLocks(bo *Backoffer, locks []*Lock, loc RegionVerID) (bool, error) {
if len(locks) == 0 {
return true, nil
}
tikvLockResolverCountWithBatchResolve.Inc()
// The GCWorker kill all ongoing transactions, because it must make sure all
// locks have been cleaned before GC.
expiredLocks := locks
callerStartTS, err := lr.store.GetOracle().GetTimestamp(bo.ctx)
if err != nil {
return false, errors.Trace(err)
}
txnInfos := make(map[uint64]uint64)
startTime := time.Now()
for _, l := range expiredLocks {
if _, ok := txnInfos[l.TxnID]; ok {
continue
}
tikvLockResolverCountWithExpired.Inc()
// Use currentTS = math.MaxUint64 means rollback the txn, no matter the lock is expired or not!
status, err := lr.getTxnStatus(bo, l.TxnID, l.Primary, callerStartTS, math.MaxUint64, true, l)
if err != nil {
return false, err
}
if status.ttl > 0 {
logutil.BgLogger().Error("BatchResolveLocks fail to clean locks, this result is not expected!")
return false, errors.New("TiDB ask TiKV to rollback locks but it doesn't, the protocol maybe wrong")
}
txnInfos[l.TxnID] = uint64(status.commitTS)
}
logutil.BgLogger().Info("BatchResolveLocks: lookup txn status",
zap.Duration("cost time", time.Since(startTime)),
zap.Int("num of txn", len(txnInfos)))
listTxnInfos := make([]*kvrpcpb.TxnInfo, 0, len(txnInfos))
for txnID, status := range txnInfos {
listTxnInfos = append(listTxnInfos, &kvrpcpb.TxnInfo{
Txn: txnID,
Status: status,
})
}
req := tikvrpc.NewRequest(tikvrpc.CmdResolveLock, &kvrpcpb.ResolveLockRequest{TxnInfos: listTxnInfos})
startTime = time.Now()
resp, err := lr.store.SendReq(bo, req, loc, readTimeoutShort)
if err != nil {
return false, errors.Trace(err)
}
regionErr, err := resp.GetRegionError()
if err != nil {
return false, errors.Trace(err)
}
if regionErr != nil {
err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String()))
if err != nil {
return false, errors.Trace(err)
}
return false, nil
}
if resp.Resp == nil {
return false, errors.Trace(ErrBodyMissing)
}
cmdResp := resp.Resp.(*kvrpcpb.ResolveLockResponse)
if keyErr := cmdResp.GetError(); keyErr != nil {
return false, errors.Errorf("unexpected resolve err: %s", keyErr)
}
logutil.BgLogger().Info("BatchResolveLocks: resolve locks in a batch",
zap.Duration("cost time", time.Since(startTime)),
zap.Int("num of locks", len(expiredLocks)))
return true, nil
}
// ResolveLocks tries to resolve Locks. The resolving process is in 3 steps:
// 1. Use the `lockTTL` to pick up all expired locks. Only locks that are too
// old are considered orphan locks and will be handled later. If all locks
// are expired then all locks will be resolved so the returned `ok` will be
// true, otherwise caller should sleep a while before retry.
// 2. For each lock, query the primary key to get txn(which left the lock)'s
// 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) {
return lr.resolveLocks(bo, callerStartTS, locks, false, false)
}
func (lr *LockResolver) resolveLocksLite(bo *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) {
if lr.testingKnobs.meetLock != nil {
lr.testingKnobs.meetLock(locks)
}
var msBeforeTxnExpired txnExpireTime
if len(locks) == 0 {
return msBeforeTxnExpired.value(), nil, nil
}
if forWrite {
tikvLockResolverCountWithResolveForWrite.Inc()
} else {
tikvLockResolverCountWithResolve.Inc()
}
var pushFail bool
// TxnID -> []Region, record resolved Regions.
// TODO: Maybe put it in LockResolver and share by all txns.
cleanTxns := make(map[uint64]map[RegionVerID]struct{})
var pushed []uint64
// pushed is only used in the read operation.
if !forWrite {
pushed = make([]uint64, 0, len(locks))
}
for _, l := range locks {
status, err := lr.getTxnStatusFromLock(bo, l, callerStartTS)
if err != nil {
msBeforeTxnExpired.update(0)
err = errors.Trace(err)
return msBeforeTxnExpired.value(), nil, err
}
if status.ttl == 0 {
tikvLockResolverCountWithExpired.Inc()
// If the lock is committed or rollbacked, resolve lock.
cleanRegions, exists := cleanTxns[l.TxnID]
if !exists {
cleanRegions = make(map[RegionVerID]struct{})
cleanTxns[l.TxnID] = cleanRegions
}
if l.LockType == kvrpcpb.Op_PessimisticLock {
err = lr.resolvePessimisticLock(bo, l, cleanRegions)
} else {
err = lr.resolveLock(bo, l, status, lite, cleanRegions)
}
if err != nil {
msBeforeTxnExpired.update(0)
err = errors.Trace(err)
return msBeforeTxnExpired.value(), nil, err
}
} else {
tikvLockResolverCountWithNotExpired.Inc()
// If the lock is valid, the txn may be a pessimistic transaction.
// Update the txn expire time.
msBeforeLockExpired := lr.store.GetOracle().UntilExpired(l.TxnID, status.ttl)
msBeforeTxnExpired.update(msBeforeLockExpired)
if forWrite {
// Write conflict detected!
// If it's a optimistic conflict and current txn is earlier than the lock owner,
// abort current transaction.
// This could avoids the deadlock scene of two large transaction.
if l.LockType != kvrpcpb.Op_PessimisticLock && l.TxnID > callerStartTS {
tikvLockResolverCountWithWriteConflict.Inc()
return msBeforeTxnExpired.value(), nil, kv.ErrWriteConflict.GenWithStackByArgs(callerStartTS, l.TxnID, status.commitTS, l.Key)
}
} else {
if status.action != kvrpcpb.Action_MinCommitTSPushed {
pushFail = true
continue
}
pushed = append(pushed, l.TxnID)
}
}
}
if pushFail {
// If any of the lock fails to push minCommitTS, don't return the pushed array.
pushed = nil
}
if msBeforeTxnExpired.value() > 0 && len(pushed) == 0 {
// If len(pushed) > 0, the caller will not block on the locks, it push the minCommitTS instead.
tikvLockResolverCountWithWaitExpired.Inc()
}
return msBeforeTxnExpired.value(), pushed, nil
}
func (lr *LockResolver) resolveLocksForWrite(bo *Backoffer, callerStartTS uint64, locks []*Lock) (int64, error) {
msBeforeTxnExpired, _, err := lr.resolveLocks(bo, callerStartTS, locks, true, false)
return msBeforeTxnExpired, err
}
type txnExpireTime struct {
initialized bool
txnExpire int64
}
func (t *txnExpireTime) update(lockExpire int64) {
if lockExpire <= 0 {
lockExpire = 0
}
if !t.initialized {
t.txnExpire = lockExpire
t.initialized = true
return
}
if lockExpire < t.txnExpire {
t.txnExpire = lockExpire
}
}
func (t *txnExpireTime) value() int64 {
if !t.initialized {
return 0
}
return t.txnExpire
}
// 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 := NewBackofferWithVars(context.Background(), cleanupMaxBackoff, nil)
currentTS, err := lr.store.GetOracle().GetLowResolutionTimestamp(bo.ctx)
if err != nil {
return status, err
}
return lr.getTxnStatus(bo, txnID, primary, callerStartTS, currentTS, true, nil)
}
func (lr *LockResolver) getTxnStatusFromLock(bo *Backoffer, l *Lock, callerStartTS uint64) (TxnStatus, error) {
var currentTS uint64
var err error
var status TxnStatus
if l.TTL == 0 {
// NOTE: l.TTL = 0 is a special protocol!!!
// When the pessimistic txn prewrite meets locks of a txn, it should resolve the lock **unconditionally**.
// In this case, TiKV use lock TTL = 0 to notify TiDB, and TiDB should resolve the lock!
// Set currentTS to max uint64 to make the lock expired.
currentTS = math.MaxUint64
} else {
currentTS, err = lr.store.GetOracle().GetLowResolutionTimestamp(bo.ctx)
if err != nil {
return TxnStatus{}, err
}
}
rollbackIfNotExist := false
failpoint.Inject("getTxnStatusDelay", func() {
time.Sleep(100 * time.Millisecond)
})
for {
status, err = lr.getTxnStatus(bo, l.TxnID, l.Primary, callerStartTS, currentTS, rollbackIfNotExist, l)
if err == nil {
return status, nil
}
// If the error is something other than txnNotFoundErr, throw the error (network
// unavailable, tikv down, backoff timeout etc) to the caller.
if _, ok := errors.Cause(err).(txnNotFoundErr); !ok {
return TxnStatus{}, err
}
failpoint.Inject("txnNotFoundRetTTL", func() {
failpoint.Return(TxnStatus{l.TTL, 0, kvrpcpb.Action_NoAction}, nil)
})
// Handle txnNotFound error.
// getTxnStatus() returns it when the secondary locks exist while the primary lock doesn't.
// This is likely to happen in the concurrently prewrite when secondary regions
// success before the primary region.
if err := bo.Backoff(boTxnNotFound, err); err != nil {
logutil.Logger(bo.ctx).Warn("getTxnStatusFromLock backoff fail", zap.Error(err))
}
if lr.store.GetOracle().UntilExpired(l.TxnID, l.TTL) <= 0 {
logutil.Logger(bo.ctx).Warn("lock txn not found, lock has expired",
zap.Uint64("CallerStartTs", callerStartTS),
zap.Stringer("lock str", l))
if l.LockType == kvrpcpb.Op_PessimisticLock {
failpoint.Inject("txnExpireRetTTL", func() {
failpoint.Return(TxnStatus{l.TTL, 0, kvrpcpb.Action_NoAction},
errors.New("error txn not found and lock expired"))
})
return TxnStatus{}, nil
}
rollbackIfNotExist = true
} else {
if l.LockType == kvrpcpb.Op_PessimisticLock {
return TxnStatus{ttl: l.TTL}, nil
}
}
}
}
type txnNotFoundErr struct {
*kvrpcpb.TxnNotFound
}
func (e txnNotFoundErr) Error() string {
return e.TxnNotFound.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, callerStartTS, currentTS uint64,
rollbackIfNotExist bool, lockInfo *Lock) (TxnStatus, error) {
if s, ok := lr.getResolved(txnID); ok {
return s, nil
}
tikvLockResolverCountWithQueryTxnStatus.Inc()
// CheckTxnStatus may meet the following cases:
// 1. LOCK
// 1.1 Lock expired -- orphan lock, fail to update TTL, crash recovery etc.
// 1.2 Lock TTL -- active transaction holding the lock.
// 2. NO LOCK
// 2.1 Txn Committed
// 2.2 Txn Rollbacked -- rollback itself, rollback by others, GC tomb etc.
// 2.3 No lock -- pessimistic lock rollback, concurrence prewrite.
var status TxnStatus
req := tikvrpc.NewRequest(tikvrpc.CmdCheckTxnStatus, &kvrpcpb.CheckTxnStatusRequest{
PrimaryKey: primary,
LockTs: txnID,
CallerStartTs: callerStartTS,
CurrentTs: currentTS,
RollbackIfNotExist: rollbackIfNotExist,
})
for {
loc, err := lr.store.GetRegionCache().LocateKey(bo, primary)
if err != nil {
return status, errors.Trace(err)
}
resp, err := lr.store.SendReq(bo, req, loc.Region, readTimeoutShort)
if err != nil {
return status, errors.Trace(err)
}
regionErr, err := resp.GetRegionError()
if err != nil {
return status, errors.Trace(err)
}
if regionErr != nil {
err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String()))
if err != nil {
return status, errors.Trace(err)
}
continue
}
if resp.Resp == nil {
return status, errors.Trace(ErrBodyMissing)
}
cmdResp := resp.Resp.(*kvrpcpb.CheckTxnStatusResponse)
if keyErr := cmdResp.GetError(); keyErr != nil {
txnNotFound := keyErr.GetTxnNotFound()
if txnNotFound != nil {
return status, txnNotFoundErr{txnNotFound}
}
err = errors.Errorf("unexpected err: %s, tid: %v", keyErr, txnID)
logutil.BgLogger().Error("getTxnStatus error", zap.Error(err))
return status, err
}
status.action = cmdResp.Action
if cmdResp.LockTtl != 0 {
status.ttl = cmdResp.LockTtl
} else {
if cmdResp.CommitVersion == 0 {
tikvLockResolverCountWithQueryTxnStatusRolledBack.Inc()
} else {
tikvLockResolverCountWithQueryTxnStatusCommitted.Inc()
}
status.commitTS = cmdResp.CommitVersion
// If the transaction is still valid with ttl greater than zero, do nothing.
// If its status is certain:
// If transaction is already committed, the result could be cached.
// Otherwise:
// If l.LockType is pessimistic lock type:
// - if its primary lock is pessimistic too, the check txn status result should not be cached.
// - if its primary lock is prewrite lock type, the check txn status could be cached, todo.
// If l.lockType is prewrite lock type:
// - always cache the check txn status result.
// For prewrite locks, their primary keys should ALWAYS be the correct one and will NOT change.
if status.ttl == 0 {
if status.IsCommitted() || (lockInfo != nil && lockInfo.LockType != kvrpcpb.Op_PessimisticLock) {
lr.saveResolved(txnID, status)
}
}
}
return status, nil
}
}
func (lr *LockResolver) resolveLock(bo *Backoffer, l *Lock, status TxnStatus, lite bool, cleanRegions map[RegionVerID]struct{}) error {
tikvLockResolverCountWithResolveLocks.Inc()
resolveLite := lite || l.TxnSize < bigTxnThreshold
for {
loc, err := lr.store.GetRegionCache().LocateKey(bo, l.Key)
if err != nil {
return errors.Trace(err)
}
if _, ok := cleanRegions[loc.Region]; ok {
return nil
}
lreq := &kvrpcpb.ResolveLockRequest{
StartVersion: l.TxnID,
}
if status.IsCommitted() {
lreq.CommitVersion = status.CommitTS()
} else {
logutil.BgLogger().Info("resolveLock rollback", zap.String("lock", l.String()))
}
if resolveLite {
// Only resolve specified keys when it is a small transaction,
// prevent from scanning the whole region in this case.
tikvLockResolverCountWithResolveLockLite.Inc()
lreq.Keys = [][]byte{l.Key}
}
req := tikvrpc.NewRequest(tikvrpc.CmdResolveLock, lreq)
resp, err := lr.store.SendReq(bo, req, loc.Region, readTimeoutShort)
if err != nil {
return errors.Trace(err)
}
regionErr, err := resp.GetRegionError()
if err != nil {
return errors.Trace(err)
}
if regionErr != nil {
err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String()))
if err != nil {
return errors.Trace(err)
}
continue
}
if resp.Resp == nil {
return errors.Trace(ErrBodyMissing)
}
cmdResp := resp.Resp.(*kvrpcpb.ResolveLockResponse)
if keyErr := cmdResp.GetError(); keyErr != nil {
err = errors.Errorf("unexpected resolve err: %s, lock: %v", keyErr, l)
logutil.BgLogger().Error("resolveLock error", zap.Error(err))
return err
}
if !resolveLite {
cleanRegions[loc.Region] = struct{}{}
}
return nil
}
}
func (lr *LockResolver) resolvePessimisticLock(bo *Backoffer, l *Lock, cleanRegions map[RegionVerID]struct{}) error {
tikvLockResolverCountWithResolveLocks.Inc()
for {
loc, err := lr.store.GetRegionCache().LocateKey(bo, l.Key)
if err != nil {
return errors.Trace(err)
}
if _, ok := cleanRegions[loc.Region]; ok {
return nil
}
forUpdateTS := l.LockForUpdateTS
if forUpdateTS == 0 {
forUpdateTS = math.MaxUint64
}
pessimisticRollbackReq := &kvrpcpb.PessimisticRollbackRequest{
StartVersion: l.TxnID,
ForUpdateTs: forUpdateTS,
Keys: [][]byte{l.Key},
}
req := tikvrpc.NewRequest(tikvrpc.CmdPessimisticRollback, pessimisticRollbackReq)
resp, err := lr.store.SendReq(bo, req, loc.Region, readTimeoutShort)
if err != nil {
return errors.Trace(err)
}
regionErr, err := resp.GetRegionError()
if err != nil {
return errors.Trace(err)
}
if regionErr != nil {
err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String()))
if err != nil {
return errors.Trace(err)
}
continue
}
if resp.Resp == nil {
return errors.Trace(ErrBodyMissing)
}
cmdResp := resp.Resp.(*kvrpcpb.PessimisticRollbackResponse)
if keyErr := cmdResp.GetErrors(); len(keyErr) > 0 {
err = errors.Errorf("unexpected resolve pessimistic lock err: %s, lock: %v", keyErr[0], l)
logutil.Logger(bo.ctx).Error("resolveLock error", zap.Error(err))
return err
}
return nil
}
}