Skip to content

Commit

Permalink
unistore: Adjust some behaviors to be consistent with TiKV (#43397) (#…
Browse files Browse the repository at this point in the history
  • Loading branch information
ti-chi-bot committed May 11, 2023
1 parent 5654903 commit 805d481
Show file tree
Hide file tree
Showing 9 changed files with 60 additions and 18 deletions.
4 changes: 2 additions & 2 deletions DEPS.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -3381,8 +3381,8 @@ def go_deps():
name = "com_github_pingcap_kvproto",
build_file_proto_mode = "disable_global",
importpath = "github.com/pingcap/kvproto",
sum = "h1:aGROoQpU8Sx9MhCspeSrDXpNkW1pcG+EWdMYxg4d5uo=",
version = "v0.0.0-20230419072653-dc3cd8784a19",
sum = "h1:58k95xMDOJkpwKs2ULr/KbEZmU2+UprcguoR/pYy6MA=",
version = "v0.0.0-20230424092600-14ac513b9eff",
)
go_repository(
name = "com_github_pingcap_log",
Expand Down
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ require (
github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32
github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c
github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059
github.com/pingcap/kvproto v0.0.0-20230419072653-dc3cd8784a19
github.com/pingcap/kvproto v0.0.0-20230424092600-14ac513b9eff
github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22
github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21
github.com/pingcap/tidb/parser v0.0.0-20211011031125-9b13dc409c5e
Expand Down
4 changes: 2 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -774,8 +774,8 @@ github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059/go.mod h1:fMRU1BA1y+r89
github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E=
github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw=
github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w=
github.com/pingcap/kvproto v0.0.0-20230419072653-dc3cd8784a19 h1:aGROoQpU8Sx9MhCspeSrDXpNkW1pcG+EWdMYxg4d5uo=
github.com/pingcap/kvproto v0.0.0-20230419072653-dc3cd8784a19/go.mod h1:guCyM5N+o+ru0TsoZ1hi9lDjUMs2sIBjW3ARTEpVbnk=
github.com/pingcap/kvproto v0.0.0-20230424092600-14ac513b9eff h1:58k95xMDOJkpwKs2ULr/KbEZmU2+UprcguoR/pYy6MA=
github.com/pingcap/kvproto v0.0.0-20230424092600-14ac513b9eff/go.mod h1:guCyM5N+o+ru0TsoZ1hi9lDjUMs2sIBjW3ARTEpVbnk=
github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8=
github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM=
github.com/pingcap/log v1.1.0/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4=
Expand Down
14 changes: 12 additions & 2 deletions store/mockstore/unistore/tikv/kverrors/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,8 +43,8 @@ func BuildLockErr(key []byte, lock *mvcc.Lock) *ErrLocked {
func (e *ErrLocked) Error() string {
lock := e.Lock
return fmt.Sprintf(
"key is locked, key: %q, Type: %v, primary: %q, startTS: %v, forUpdateTS: %v, useAsyncCommit: %v",
e.Key, lock.Op, lock.Primary, lock.StartTS, lock.ForUpdateTS, lock.UseAsyncCommit,
"key is locked, key: %v, lock: %v",
hex.EncodeToString(e.Key), lock.String(),
)
}

Expand Down Expand Up @@ -148,3 +148,13 @@ func (e *ErrAssertionFailed) Error() string {
return fmt.Sprintf("AssertionFailed { StartTS: %v, Key: %v, Assertion: %v, ExistingStartTS: %v, ExistingCommitTS: %v }",
e.StartTS, hex.EncodeToString(e.Key), e.Assertion.String(), e.ExistingStartTS, e.ExistingCommitTS)
}

// ErrPrimaryMismatch is returned if CheckTxnStatus request is sent to a secondary lock.
type ErrPrimaryMismatch struct {
Key []byte
Lock *mvcc.Lock
}

func (e *ErrPrimaryMismatch) Error() string {
return fmt.Sprintf("primary mismatch, key: %v, lock: %v", hex.EncodeToString(e.Key), e.Lock.String())
}
18 changes: 8 additions & 10 deletions store/mockstore/unistore/tikv/mvcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -487,6 +487,13 @@ func (store *MVCCStore) CheckTxnStatus(reqCtx *requestCtx,
lock := store.getLock(reqCtx, req.PrimaryKey)
batch := store.dbWriter.NewWriteBatch(req.LockTs, 0, reqCtx.rpcCtx)
if lock != nil && lock.StartTS == req.LockTs {
if !bytes.Equal(req.PrimaryKey, lock.Primary) {
return TxnStatus{}, &kverrors.ErrPrimaryMismatch{
Key: req.PrimaryKey,
Lock: lock,
}
}

// For an async-commit lock, never roll it back or push forward it MinCommitTS.
if lock.UseAsyncCommit && !req.ForceSyncCommit {
log.S().Debugf("async commit startTS=%v secondaries=%v minCommitTS=%v", lock.StartTS, lock.Secondaries, lock.MinCommitTS)
Expand Down Expand Up @@ -1144,10 +1151,6 @@ func (store *MVCCStore) Commit(req *requestCtx, keys [][]byte, startTS, commitTS
Key: key,
}
}
if lock.Op == uint8(kvrpcpb.Op_PessimisticLock) {
log.Warn("commit a pessimistic lock with Lock type", zap.Binary("key", key), zap.Uint64("start ts", startTS), zap.Uint64("commit ts", commitTS))
lock.Op = uint8(kvrpcpb.Op_Lock)
}
isPessimisticTxn = lock.ForUpdateTS > 0
tmpDiff += len(key) + len(lock.Value)
batch.Commit(key, &lock)
Expand Down Expand Up @@ -1423,12 +1426,7 @@ func (store *MVCCStore) Cleanup(reqCtx *requestCtx, key []byte, startTS, current
func (store *MVCCStore) appendScannedLock(locks []*kvrpcpb.LockInfo, it *lockstore.Iterator, maxTS uint64) []*kvrpcpb.LockInfo {
lock := mvcc.DecodeLock(it.Value())
if lock.StartTS < maxTS {
locks = append(locks, &kvrpcpb.LockInfo{
PrimaryLock: lock.Primary,
LockVersion: lock.StartTS,
Key: safeCopy(it.Key()),
LockTtl: uint64(lock.TTL),
})
locks = append(locks, lock.ToLockInfo(append([]byte{}, it.Key()...)))
}
return locks
}
Expand Down
14 changes: 14 additions & 0 deletions store/mockstore/unistore/tikv/mvcc/mvcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,8 @@ package mvcc

import (
"encoding/binary"
"encoding/hex"
"fmt"
"unsafe"

"github.com/pingcap/kvproto/pkg/kvrpcpb"
Expand Down Expand Up @@ -113,6 +115,18 @@ func (l *Lock) ToLockInfo(key []byte) *kvrpcpb.LockInfo {
}
}

// String implements fmt.Stringer for Lock.
func (l *Lock) String() string {
return fmt.Sprintf(
"Lock { Type: %v, StartTS: %v, ForUpdateTS: %v, Primary: %v, UseAsyncCommit: %v }",
kvrpcpb.Op(l.Op).String(),
l.StartTS,
l.ForUpdateTS,
hex.EncodeToString(l.Primary),
l.UseAsyncCommit,
)
}

// UserMeta value for lock.
const (
LockUserMetaNoneByte = 0
Expand Down
9 changes: 9 additions & 0 deletions store/mockstore/unistore/tikv/mvcc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -715,6 +715,15 @@ func TestCheckTxnStatus(t *testing.T) {
require.Equal(t, uint64(41), resCommitTs)
require.NoError(t, err)
require.Equal(t, kvrpcpb.Action_NoAction, action)

// check on mismatching primary
startTs = 43
callerStartTs = 44
currentTs = 44
MustAcquirePessimisticLock([]byte("another_key"), pk, startTs, startTs, store)
_, _, _, err = CheckTxnStatus(pk, startTs, callerStartTs, currentTs, true, store)
require.IsType(t, &kverrors.ErrPrimaryMismatch{}, errors.Cause(err))
MustPessimisticRollback(pk, startTs, startTs, store)
}

func TestCheckSecondaryLocksStatus(t *testing.T) {
Expand Down
6 changes: 6 additions & 0 deletions store/mockstore/unistore/tikv/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -1138,6 +1138,12 @@ func convertToKeyError(err error) *kvrpcpb.KeyError {
ExistingCommitTs: x.ExistingCommitTS,
},
}
case *kverrors.ErrPrimaryMismatch:
return &kvrpcpb.KeyError{
PrimaryMismatch: &kvrpcpb.PrimaryMismatch{
LockInfo: x.Lock.ToLockInfo(x.Key),
},
}
default:
return &kvrpcpb.KeyError{
Abort: err.Error(),
Expand Down
7 changes: 6 additions & 1 deletion store/mockstore/unistore/tikv/write.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,10 +23,12 @@ import (
"github.com/pingcap/badger"
"github.com/pingcap/badger/y"
"github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pingcap/log"
"github.com/pingcap/tidb/store/mockstore/unistore/lockstore"
"github.com/pingcap/tidb/store/mockstore/unistore/tikv/dbreader"
"github.com/pingcap/tidb/store/mockstore/unistore/tikv/mvcc"
"github.com/pingcap/tidb/util/mathutil"
"go.uber.org/zap"
)

const (
Expand Down Expand Up @@ -249,7 +251,10 @@ func (wb *writeBatch) Prewrite(key []byte, lock *mvcc.Lock) {
func (wb *writeBatch) Commit(key []byte, lock *mvcc.Lock) {
userMeta := mvcc.NewDBUserMeta(wb.startTS, wb.commitTS)
k := y.KeyWithTs(key, wb.commitTS)
if lock.Op != uint8(kvrpcpb.Op_Lock) {
if lock.Op == uint8(kvrpcpb.Op_PessimisticLock) {
log.Info("removing a pessimistic lock when committing", zap.Binary("key", key), zap.Uint64("startTS", wb.startTS), zap.Uint64("commitTS", wb.commitTS))
// Write nothing as if PessimisticRollback is called.
} else if lock.Op != uint8(kvrpcpb.Op_Lock) {
wb.dbBatch.set(k, lock.Value, userMeta)
} else if bytes.Equal(key, lock.Primary) {
opLockKey := y.KeyWithTs(mvcc.EncodeExtraTxnStatusKey(key, wb.startTS), wb.startTS)
Expand Down

0 comments on commit 805d481

Please sign in to comment.