Fix the issue that primary pessimistic lock may be left not cleared after GC (#866)

* Fix the issue that primary pessimistic lock may be left not cleared after GC

Signed-off-by: MyonKeminta <MyonKeminta@users.noreply.github.com>

* Fix mysteriously shown up thing that makes compilation failed

Signed-off-by: MyonKeminta <MyonKeminta@users.noreply.github.com>

* Fix test effectiveness (forgot to set txn2 to pessimistic txn); add more strict checks

Signed-off-by: MyonKeminta <MyonKeminta@users.noreply.github.com>

* Address comments

Signed-off-by: MyonKeminta <MyonKeminta@users.noreply.github.com>

---------

Signed-off-by: MyonKeminta <MyonKeminta@users.noreply.github.com>
Co-authored-by: MyonKeminta <MyonKeminta@users.noreply.github.com>
This commit is contained in:
MyonKeminta 2023-07-04 15:17:05 +08:00 committed by GitHub
parent fbec023060
commit c0cf773917
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 111 additions and 23 deletions

View File

@ -1472,52 +1472,81 @@ func (s *testLockWithTiKVSuite) TestBatchResolveLocks() {
s.NoError(failpoint.Enable("tikvclient/beforeAsyncPessimisticRollback", `return("skip")`)) s.NoError(failpoint.Enable("tikvclient/beforeAsyncPessimisticRollback", `return("skip")`))
s.NoError(failpoint.Enable("tikvclient/beforeCommitSecondaries", `return("skip")`)) s.NoError(failpoint.Enable("tikvclient/beforeCommitSecondaries", `return("skip")`))
s.NoError(failpoint.Enable("tikvclient/twoPCRequestBatchSizeLimit", `return("skip")`)) s.NoError(failpoint.Enable("tikvclient/twoPCRequestBatchSizeLimit", `return`))
s.NoError(failpoint.Enable("tikvclient/onRollback", `return("skipRollbackPessimisticLock")`))
defer func() { defer func() {
s.NoError(failpoint.Disable("tikvclient/beforeAsyncPessimisticRollback")) s.NoError(failpoint.Disable("tikvclient/beforeAsyncPessimisticRollback"))
s.NoError(failpoint.Disable("tikvclient/beforeCommitSecondaries")) s.NoError(failpoint.Disable("tikvclient/beforeCommitSecondaries"))
s.NoError(failpoint.Disable("tikvclient/twoPCRequestBatchSizeLimit")) s.NoError(failpoint.Disable("tikvclient/twoPCRequestBatchSizeLimit"))
s.NoError(failpoint.Disable("tikvclient/onRollback"))
}() }()
k1, k2, k3 := []byte("k1"), []byte("k2"), []byte("k3") k1, k2, k3, k4 := []byte("k1"), []byte("k2"), []byte("k3"), []byte("k4")
v2, v3 := []byte("v2"), []byte("v3") v2, v3 := []byte("v2"), []byte("v3")
ctx := context.WithValue(context.Background(), util.SessionID, uint64(1)) ctx := context.WithValue(context.Background(), util.SessionID, uint64(1))
txn, err := s.store.Begin() txn1, err := s.store.Begin()
s.NoError(err) s.NoError(err)
txn.SetPessimistic(true) txn1.SetPessimistic(true)
{ {
// Produce write conflict on key k2 // Produce write conflict on key k2
txn2, err := s.store.Begin() helperTxn, err := s.store.Begin()
s.NoError(err) s.NoError(err)
s.NoError(txn2.Set(k2, []byte("v0"))) s.NoError(helperTxn.Set(k2, []byte("v0")))
s.NoError(txn2.Commit(ctx)) s.NoError(helperTxn.Commit(ctx))
} }
lockCtx := kv.NewLockCtx(txn.StartTS(), 200, time.Now()) lockCtx := kv.NewLockCtx(txn1.StartTS(), 200, time.Now())
err = txn.LockKeys(ctx, lockCtx, k1, k2) err = txn1.LockKeys(ctx, lockCtx, k1, k2)
s.IsType(&tikverr.ErrWriteConflict{}, errors.Cause(err)) s.IsType(&tikverr.ErrWriteConflict{}, errors.Cause(err))
// k1 has txn's stale pessimistic lock now. // k1 has txn1's stale pessimistic lock now.
forUpdateTS, err := s.store.CurrentTimestamp(oracle.GlobalTxnScope) forUpdateTS, err := s.store.CurrentTimestamp(oracle.GlobalTxnScope)
s.NoError(err) s.NoError(err)
lockCtx = kv.NewLockCtx(forUpdateTS, 200, time.Now()) lockCtx = kv.NewLockCtx(forUpdateTS, 200, time.Now())
s.NoError(txn.LockKeys(ctx, lockCtx, k2, k3)) s.NoError(txn1.LockKeys(ctx, lockCtx, k2, k3))
s.NoError(txn.Set(k2, v2)) s.NoError(txn1.Set(k2, v2))
s.NoError(txn.Set(k3, v3)) s.NoError(txn1.Set(k3, v3))
s.NoError(txn.Commit(ctx)) s.NoError(txn1.Commit(ctx))
// k3 has txn's stale prewrite lock now. // k3 has txn1's stale prewrite lock now.
txn2, err := s.store.Begin()
txn2.SetPessimistic(true)
s.NoError(err)
lockCtx = kv.NewLockCtx(txn1.StartTS(), 200, time.Now())
err = txn2.LockKeys(ctx, lockCtx, k4)
s.NoError(err)
s.NoError(txn2.Rollback())
// k4 has txn2's stale primary pessimistic lock now.
currentTS, err := s.store.CurrentTimestamp(oracle.GlobalTxnScope)
remainingLocks, err := s.store.ScanLocks(ctx, []byte("k"), []byte("l"), currentTS)
s.NoError(err)
s.Len(remainingLocks, 3)
s.Equal(remainingLocks[0].Key, k1)
s.Equal(remainingLocks[0].LockType, kvrpcpb.Op_PessimisticLock)
s.Equal(remainingLocks[1].Key, k3)
s.Equal(remainingLocks[1].LockType, kvrpcpb.Op_Put)
s.Equal(remainingLocks[2].Key, k4)
s.Equal(remainingLocks[2].LockType, kvrpcpb.Op_PessimisticLock)
s.Equal(remainingLocks[2].Primary, k4)
// Perform ScanLock - BatchResolveLock. // Perform ScanLock - BatchResolveLock.
currentTS, err := s.store.CurrentTimestamp(oracle.GlobalTxnScope)
s.NoError(err) s.NoError(err)
s.NoError(s.store.GCResolveLockPhase(ctx, currentTS, 1)) s.NoError(s.store.GCResolveLockPhase(ctx, currentTS, 1))
// Do ScanLock again to make sure no locks are left.
remainingLocks, err = s.store.ScanLocks(ctx, []byte("k"), []byte("l"), currentTS)
s.NoError(err)
s.Empty(remainingLocks)
// Check data consistency // Check data consistency
readTS, err := s.store.CurrentTimestamp(oracle.GlobalTxnScope) readTS, err := s.store.CurrentTimestamp(oracle.GlobalTxnScope)
snapshot := s.store.GetSnapshot(readTS) snapshot := s.store.GetSnapshot(readTS)

View File

@ -35,6 +35,7 @@
package tikv package tikv
import ( import (
"bytes"
"context" "context"
"github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/metapb"
@ -114,6 +115,42 @@ func (s StoreProbe) GCResolveLockPhase(ctx context.Context, safepoint uint64, co
return s.resolveLocks(ctx, safepoint, concurrency) return s.resolveLocks(ctx, safepoint, concurrency)
} }
func (s StoreProbe) ScanLocks(ctx context.Context, startKey, endKey []byte, maxVersion uint64) ([]*txnlock.Lock, error) {
bo := NewGcResolveLockMaxBackoffer(ctx)
const limit = 1024
var result []*txnlock.Lock
outerLoop:
for {
locks, loc, err := s.KVStore.scanLocksInRegionWithStartKey(bo, startKey, maxVersion, limit)
if err != nil {
return nil, err
}
for _, l := range locks {
if bytes.Compare(endKey, l.Key) <= 0 {
// Finished scanning the given range.
break outerLoop
}
result = append(result, l)
}
if len(locks) < limit {
if len(loc.EndKey) == 0 {
// Scanned to the very end.
break outerLoop
}
// The current region is completely scanned.
startKey = loc.EndKey
} else {
// The current region may still have more locks.
startKey = append(locks[len(locks)-1].Key, 0)
}
}
return result, nil
}
// LockResolverProbe wraps a LockResolver and exposes internal stats for testing purpose. // LockResolverProbe wraps a LockResolver and exposes internal stats for testing purpose.
type LockResolverProbe struct { type LockResolverProbe struct {
*txnlock.LockResolverProbe *txnlock.LockResolverProbe

View File

@ -556,10 +556,27 @@ func (txn *KVTxn) Rollback() error {
txn.CancelAggressiveLocking(context.Background()) txn.CancelAggressiveLocking(context.Background())
} }
// `skipPessimisticRollback` may be true only when set by failpoint in tests.
skipPessimisticRollback := false
if val, err := util.EvalFailpoint("onRollback"); err == nil {
if s, ok := val.(string); ok {
if s == "skipRollbackPessimisticLock" {
logutil.BgLogger().Info("[failpoint] injected skip pessimistic rollback on explicit rollback",
zap.Uint64("txnStartTS", txn.startTS))
skipPessimisticRollback = true
} else {
panic(fmt.Sprintf("unknown instruction %s for failpoint \"onRollback\"", s))
}
}
}
start := time.Now() start := time.Now()
// Clean up pessimistic lock. // Clean up pessimistic lock.
if txn.IsPessimistic() && txn.committer != nil { if txn.IsPessimistic() && txn.committer != nil {
err := txn.rollbackPessimisticLocks() var err error
if !skipPessimisticRollback {
err = txn.rollbackPessimisticLocks()
}
txn.committer.ttlManager.close() txn.committer.ttlManager.close()
if err != nil { if err != nil {
logutil.BgLogger().Error(err.Error()) logutil.BgLogger().Error(err.Error())

View File

@ -245,6 +245,12 @@ func (lr *LockResolver) BatchResolveLocks(bo *retry.Backoffer, locks []*Lock, lo
} }
metrics.LockResolverCountWithExpired.Inc() metrics.LockResolverCountWithExpired.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, 0, math.MaxUint64, true, false, l)
if err != nil {
return false, err
}
if l.LockType == kvrpcpb.Op_PessimisticLock { if l.LockType == kvrpcpb.Op_PessimisticLock {
// BatchResolveLocks forces resolving the locks ignoring whether whey are expired. // BatchResolveLocks forces resolving the locks ignoring whether whey are expired.
// For pessimistic locks, committing them makes no sense, but it won't affect transaction // For pessimistic locks, committing them makes no sense, but it won't affect transaction
@ -252,6 +258,9 @@ func (lr *LockResolver) BatchResolveLocks(bo *retry.Backoffer, locks []*Lock, lo
// Pessimistic locks needs special handling logic because their primary may not point // Pessimistic locks needs special handling logic because their primary may not point
// to the real primary of that transaction, and their state cannot be put in `txnInfos`. // to the real primary of that transaction, and their state cannot be put in `txnInfos`.
// (see: https://github.com/pingcap/tidb/issues/42937). // (see: https://github.com/pingcap/tidb/issues/42937).
//
// `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)
if err != nil { if err != nil {
return false, err return false, err
@ -259,12 +268,6 @@ func (lr *LockResolver) BatchResolveLocks(bo *retry.Backoffer, locks []*Lock, lo
continue continue
} }
// 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, 0, math.MaxUint64, true, false, l)
if err != nil {
return false, err
}
// If the transaction uses async commit, CheckTxnStatus will reject rolling back the primary lock. // If the transaction uses async commit, CheckTxnStatus will reject rolling back the primary lock.
// Then we need to check the secondary locks to determine the final status of the transaction. // Then we need to check the secondary locks to determine the final status of the transaction.
if status.primaryLock != nil && status.primaryLock.UseAsyncCommit { if status.primaryLock != nil && status.primaryLock.UseAsyncCommit {
@ -1173,6 +1176,8 @@ 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 { func (lr *LockResolver) resolvePessimisticLock(bo *retry.Backoffer, l *Lock) error {
metrics.LockResolverCountWithResolveLocks.Inc() metrics.LockResolverCountWithResolveLocks.Inc()
// The lock has been resolved by getTxnStatusFromLock. // The lock has been resolved by getTxnStatusFromLock.