Skip to content

Commit

Permalink
store/tikv: avoid unnecessary pessimistic rollback on ErrKeyExists (p…
Browse files Browse the repository at this point in the history
  • Loading branch information
sticnarf authored May 28, 2021
1 parent 25b4276 commit 228fe64
Show file tree
Hide file tree
Showing 3 changed files with 50 additions and 2 deletions.
43 changes: 43 additions & 0 deletions store/tikv/2pc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import (
"github.com/pingcap/failpoint"
"github.com/pingcap/kvproto/pkg/kvrpcpb"
pb "github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/store/mockstore/mocktikv"
"github.com/pingcap/tidb/store/tikv/config"
Expand Down Expand Up @@ -1344,6 +1345,48 @@ func (s *testCommitterSuite) TestAsyncCommitCheck(c *C) {
c.Assert(committer.checkAsyncCommit(), IsFalse)
}

// Test that pessimistic rollback is not sent if LockKeys locks only one key and
// the error is KeyExists or WriteConflict.
func (s *testCommitterSuite) TestFailLockKeysNeedNoRollback(c *C) {
// Write key k
txn := s.begin(c)
err := txn.Set(kv.Key("k1"), []byte("v1"))
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)

// In this test case, no pessimistic rollback should be sent. So we make pessimistic rollback panic.
// If a pessimistic rollback is sent, the test should fail because of the panic.
c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/beforeAsyncPessimisticRollback", "panic"), IsNil)
defer func() {
// we should delay some time so we disable the failpoint after the background job finishes
time.Sleep(time.Second)
c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/beforeAsyncPessimisticRollback"), IsNil)
}()

txn = s.begin(c)
txn.SetOption(kv.Pessimistic, true)

// test KeyExists error
err = txn.GetMemBuffer().SetWithFlags(kv.Key("k1"), []byte{0}, kv.SetPresumeKeyNotExists)
c.Assert(err, IsNil)
lockCtx := kv.LockCtx{ForUpdateTS: txn.startTS, WaitStartTime: time.Now(), LockWaitTime: kv.LockNoWait}
err = txn.LockKeys(context.Background(), &lockCtx, kv.Key("k1"))
c.Assert(err, NotNil)
_, isErrKeyExist := errors.Cause(err).(*ErrKeyExist)
c.Assert(isErrKeyExist, IsTrue)

// test WriteConflict error
txn2 := s.begin(c)
err = txn2.Set(kv.Key("k2"), []byte("v2"))
c.Assert(err, IsNil)
err = txn2.Commit(context.Background())
c.Assert(err, IsNil)
err = txn.LockKeys(context.Background(), &lockCtx, kv.Key("k2"))
c.Assert(err, NotNil)
c.Assert(terror.ErrorEqual(kv.ErrWriteConflict, err), IsTrue)
}

type mockClient struct {
inner Client
seenPrimaryReq uint32
Expand Down
6 changes: 6 additions & 0 deletions store/tikv/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -86,3 +86,9 @@ type ErrKeyExist struct {
func (k *ErrKeyExist) Error() string {
return k.AlreadyExist.String()
}

// IsErrKeyExist returns true if it is ErrKeyExist.
func IsErrKeyExist(err error) bool {
_, ok := errors.Cause(err).(*ErrKeyExist)
return ok
}
3 changes: 1 addition & 2 deletions store/tikv/txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -486,7 +486,7 @@ func (txn *KVTxn) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keysInput .
txn.us.UnmarkPresumeKeyNotExists(key)
}
}
keyMayBeLocked := terror.ErrorNotEqual(kv.ErrWriteConflict, err) && terror.ErrorNotEqual(kv.ErrKeyExists, err)
keyMayBeLocked := terror.ErrorNotEqual(kv.ErrWriteConflict, err) && !IsErrKeyExist(err)
// If there is only 1 key and lock fails, no need to do pessimistic rollback.
if len(keys) > 1 || keyMayBeLocked {
wg := txn.asyncPessimisticRollback(ctx, keys)
Expand Down Expand Up @@ -568,7 +568,6 @@ func (txn *KVTxn) asyncPessimisticRollback(ctx context.Context, keys [][]byte) *
}
}
})

err := committer.pessimisticRollbackMutations(NewBackofferWithVars(ctx, pessimisticRollbackMaxBackoff, txn.vars), &PlainMutations{keys: keys})
if err != nil {
logutil.Logger(ctx).Warn("[kv] pessimisticRollback failed.", zap.Error(err))
Expand Down

0 comments on commit 228fe64

Please sign in to comment.