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.
1147 lines
39 KiB
1147 lines
39 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"
|
|
"context"
|
|
"fmt"
|
|
"math"
|
|
"math/rand"
|
|
"strings"
|
|
"sync"
|
|
"sync/atomic"
|
|
"time"
|
|
|
|
. "github.com/pingcap/check"
|
|
"github.com/pingcap/errors"
|
|
"github.com/pingcap/failpoint"
|
|
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
|
"github.com/pingcap/tidb/kv"
|
|
"github.com/pingcap/tidb/store/mockstore/mocktikv"
|
|
"github.com/pingcap/tidb/store/tikv/oracle"
|
|
"github.com/pingcap/tidb/store/tikv/tikvrpc"
|
|
)
|
|
|
|
type testCommitterSuite struct {
|
|
OneByOneSuite
|
|
cluster *mocktikv.Cluster
|
|
store *tikvStore
|
|
mvccStore mocktikv.MVCCStore
|
|
}
|
|
|
|
var _ = SerialSuites(&testCommitterSuite{})
|
|
|
|
func (s *testCommitterSuite) SetUpSuite(c *C) {
|
|
atomic.StoreUint64(&ManagedLockTTL, 3000) // 3s
|
|
s.OneByOneSuite.SetUpSuite(c)
|
|
}
|
|
|
|
func (s *testCommitterSuite) SetUpTest(c *C) {
|
|
s.cluster = mocktikv.NewCluster()
|
|
mocktikv.BootstrapWithMultiRegions(s.cluster, []byte("a"), []byte("b"), []byte("c"))
|
|
mvccStore, err := mocktikv.NewMVCCLevelDB("")
|
|
c.Assert(err, IsNil)
|
|
s.mvccStore = mvccStore
|
|
client := mocktikv.NewRPCClient(s.cluster, mvccStore)
|
|
pdCli := &codecPDClient{mocktikv.NewPDClient(s.cluster)}
|
|
spkv := NewMockSafePointKV()
|
|
store, err := newTikvStore("mocktikv-store", pdCli, spkv, client, false, nil)
|
|
c.Assert(err, IsNil)
|
|
store.EnableTxnLocalLatches(1024000)
|
|
s.store = store
|
|
CommitMaxBackoff = 1000
|
|
}
|
|
|
|
func (s *testCommitterSuite) TearDownSuite(c *C) {
|
|
CommitMaxBackoff = 20000
|
|
s.store.Close()
|
|
s.OneByOneSuite.TearDownSuite(c)
|
|
}
|
|
|
|
func (s *testCommitterSuite) begin(c *C) *tikvTxn {
|
|
txn, err := s.store.Begin()
|
|
c.Assert(err, IsNil)
|
|
return txn.(*tikvTxn)
|
|
}
|
|
|
|
func (s *testCommitterSuite) checkValues(c *C, m map[string]string) {
|
|
txn := s.begin(c)
|
|
for k, v := range m {
|
|
val, err := txn.Get(context.TODO(), []byte(k))
|
|
c.Assert(err, IsNil)
|
|
c.Assert(string(val), Equals, v)
|
|
}
|
|
}
|
|
|
|
func (s *testCommitterSuite) mustCommit(c *C, m map[string]string) {
|
|
txn := s.begin(c)
|
|
for k, v := range m {
|
|
err := txn.Set([]byte(k), []byte(v))
|
|
c.Assert(err, IsNil)
|
|
}
|
|
err := txn.Commit(context.Background())
|
|
c.Assert(err, IsNil)
|
|
|
|
s.checkValues(c, m)
|
|
}
|
|
|
|
func randKV(keyLen, valLen int) (string, string) {
|
|
const letters = "abc"
|
|
k, v := make([]byte, keyLen), make([]byte, valLen)
|
|
for i := range k {
|
|
k[i] = letters[rand.Intn(len(letters))]
|
|
}
|
|
for i := range v {
|
|
v[i] = letters[rand.Intn(len(letters))]
|
|
}
|
|
return string(k), string(v)
|
|
}
|
|
|
|
func (s *testCommitterSuite) TestCommitRollback(c *C) {
|
|
s.mustCommit(c, map[string]string{
|
|
"a": "a",
|
|
"b": "b",
|
|
"c": "c",
|
|
})
|
|
|
|
txn := s.begin(c)
|
|
txn.Set([]byte("a"), []byte("a1"))
|
|
txn.Set([]byte("b"), []byte("b1"))
|
|
txn.Set([]byte("c"), []byte("c1"))
|
|
|
|
s.mustCommit(c, map[string]string{
|
|
"c": "c2",
|
|
})
|
|
|
|
err := txn.Commit(context.Background())
|
|
c.Assert(err, NotNil)
|
|
|
|
s.checkValues(c, map[string]string{
|
|
"a": "a",
|
|
"b": "b",
|
|
"c": "c2",
|
|
})
|
|
}
|
|
|
|
func (s *testCommitterSuite) TestPrewriteRollback(c *C) {
|
|
s.mustCommit(c, map[string]string{
|
|
"a": "a0",
|
|
"b": "b0",
|
|
})
|
|
ctx := context.Background()
|
|
txn1 := s.begin(c)
|
|
err := txn1.Set([]byte("a"), []byte("a1"))
|
|
c.Assert(err, IsNil)
|
|
err = txn1.Set([]byte("b"), []byte("b1"))
|
|
c.Assert(err, IsNil)
|
|
committer, err := newTwoPhaseCommitterWithInit(txn1, 0)
|
|
c.Assert(err, IsNil)
|
|
err = committer.prewriteMutations(NewBackofferWithVars(ctx, PrewriteMaxBackoff, nil), committer.mutations)
|
|
c.Assert(err, IsNil)
|
|
|
|
txn2 := s.begin(c)
|
|
v, err := txn2.Get(context.TODO(), []byte("a"))
|
|
c.Assert(err, IsNil)
|
|
c.Assert(v, BytesEquals, []byte("a0"))
|
|
|
|
err = committer.prewriteMutations(NewBackofferWithVars(ctx, PrewriteMaxBackoff, nil), committer.mutations)
|
|
if err != nil {
|
|
// Retry.
|
|
txn1 = s.begin(c)
|
|
err = txn1.Set([]byte("a"), []byte("a1"))
|
|
c.Assert(err, IsNil)
|
|
err = txn1.Set([]byte("b"), []byte("b1"))
|
|
c.Assert(err, IsNil)
|
|
committer, err = newTwoPhaseCommitterWithInit(txn1, 0)
|
|
c.Assert(err, IsNil)
|
|
err = committer.prewriteMutations(NewBackofferWithVars(ctx, PrewriteMaxBackoff, nil), committer.mutations)
|
|
c.Assert(err, IsNil)
|
|
}
|
|
committer.commitTS, err = s.store.oracle.GetTimestamp(ctx)
|
|
c.Assert(err, IsNil)
|
|
err = committer.commitMutations(NewBackofferWithVars(ctx, CommitMaxBackoff, nil), CommitterMutations{keys: [][]byte{[]byte("a")}})
|
|
c.Assert(err, IsNil)
|
|
|
|
txn3 := s.begin(c)
|
|
v, err = txn3.Get(context.TODO(), []byte("b"))
|
|
c.Assert(err, IsNil)
|
|
c.Assert(v, BytesEquals, []byte("b1"))
|
|
}
|
|
|
|
func (s *testCommitterSuite) TestContextCancel(c *C) {
|
|
txn1 := s.begin(c)
|
|
err := txn1.Set([]byte("a"), []byte("a1"))
|
|
c.Assert(err, IsNil)
|
|
err = txn1.Set([]byte("b"), []byte("b1"))
|
|
c.Assert(err, IsNil)
|
|
committer, err := newTwoPhaseCommitterWithInit(txn1, 0)
|
|
c.Assert(err, IsNil)
|
|
|
|
bo := NewBackofferWithVars(context.Background(), PrewriteMaxBackoff, nil)
|
|
backoffer, cancel := bo.Fork()
|
|
cancel() // cancel the context
|
|
err = committer.prewriteMutations(backoffer, committer.mutations)
|
|
c.Assert(errors.Cause(err), Equals, context.Canceled)
|
|
}
|
|
|
|
func (s *testCommitterSuite) TestContextCancel2(c *C) {
|
|
txn := s.begin(c)
|
|
err := txn.Set([]byte("a"), []byte("a"))
|
|
c.Assert(err, IsNil)
|
|
err = txn.Set([]byte("b"), []byte("b"))
|
|
c.Assert(err, IsNil)
|
|
ctx, cancel := context.WithCancel(context.Background())
|
|
err = txn.Commit(ctx)
|
|
c.Assert(err, IsNil)
|
|
cancel()
|
|
// Secondary keys should not be canceled.
|
|
time.Sleep(time.Millisecond * 20)
|
|
c.Assert(s.isKeyLocked(c, []byte("b")), IsFalse)
|
|
}
|
|
|
|
func (s *testCommitterSuite) TestContextCancelRetryable(c *C) {
|
|
txn1, txn2, txn3 := s.begin(c), s.begin(c), s.begin(c)
|
|
// txn1 locks "b"
|
|
err := txn1.Set([]byte("b"), []byte("b1"))
|
|
c.Assert(err, IsNil)
|
|
committer, err := newTwoPhaseCommitterWithInit(txn1, 0)
|
|
c.Assert(err, IsNil)
|
|
err = committer.prewriteMutations(NewBackofferWithVars(context.Background(), PrewriteMaxBackoff, nil), committer.mutations)
|
|
c.Assert(err, IsNil)
|
|
// txn3 writes "c"
|
|
err = txn3.Set([]byte("c"), []byte("c3"))
|
|
c.Assert(err, IsNil)
|
|
err = txn3.Commit(context.Background())
|
|
c.Assert(err, IsNil)
|
|
// txn2 writes "a"(PK), "b", "c" on different regions.
|
|
// "c" will return a retryable error.
|
|
// "b" will get a Locked error first, then the context must be canceled after backoff for lock.
|
|
err = txn2.Set([]byte("a"), []byte("a2"))
|
|
c.Assert(err, IsNil)
|
|
err = txn2.Set([]byte("b"), []byte("b2"))
|
|
c.Assert(err, IsNil)
|
|
err = txn2.Set([]byte("c"), []byte("c2"))
|
|
c.Assert(err, IsNil)
|
|
err = txn2.Commit(context.Background())
|
|
c.Assert(err, NotNil)
|
|
c.Assert(kv.ErrWriteConflictInTiDB.Equal(err), IsTrue, Commentf("err: %s", err))
|
|
}
|
|
|
|
func (s *testCommitterSuite) TestContextCancelCausingUndetermined(c *C) {
|
|
// For a normal transaction, if RPC returns context.Canceled error while sending commit
|
|
// requests, the transaction should go to the undetermined state.
|
|
txn := s.begin(c)
|
|
err := txn.Set([]byte("a"), []byte("va"))
|
|
c.Assert(err, IsNil)
|
|
committer, err := newTwoPhaseCommitterWithInit(txn, 0)
|
|
c.Assert(err, IsNil)
|
|
committer.prewriteMutations(NewBackofferWithVars(context.Background(), PrewriteMaxBackoff, nil), committer.mutations)
|
|
c.Assert(err, IsNil)
|
|
|
|
c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/rpcContextCancelErr", `return(true)`), IsNil)
|
|
defer func() {
|
|
c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/rpcContextCancelErr"), IsNil)
|
|
}()
|
|
|
|
err = committer.commitMutations(NewBackofferWithVars(context.Background(), PrewriteMaxBackoff, nil), committer.mutations)
|
|
c.Assert(committer.mu.undeterminedErr, NotNil)
|
|
c.Assert(errors.Cause(err), Equals, context.Canceled)
|
|
}
|
|
|
|
func (s *testCommitterSuite) mustGetRegionID(c *C, key []byte) uint64 {
|
|
loc, err := s.store.regionCache.LocateKey(NewBackofferWithVars(context.Background(), getMaxBackoff, nil), key)
|
|
c.Assert(err, IsNil)
|
|
return loc.Region.id
|
|
}
|
|
|
|
func (s *testCommitterSuite) isKeyLocked(c *C, key []byte) bool {
|
|
ver, err := s.store.CurrentVersion()
|
|
c.Assert(err, IsNil)
|
|
bo := NewBackofferWithVars(context.Background(), getMaxBackoff, nil)
|
|
req := tikvrpc.NewRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{
|
|
Key: key,
|
|
Version: ver.Ver,
|
|
})
|
|
loc, err := s.store.regionCache.LocateKey(bo, key)
|
|
c.Assert(err, IsNil)
|
|
resp, err := s.store.SendReq(bo, req, loc.Region, readTimeoutShort)
|
|
c.Assert(err, IsNil)
|
|
c.Assert(resp.Resp, NotNil)
|
|
keyErr := (resp.Resp.(*kvrpcpb.GetResponse)).GetError()
|
|
return keyErr.GetLocked() != nil
|
|
}
|
|
|
|
func (s *testCommitterSuite) TestPrewriteCancel(c *C) {
|
|
// Setup region delays for key "b" and "c".
|
|
delays := map[uint64]time.Duration{
|
|
s.mustGetRegionID(c, []byte("b")): time.Millisecond * 10,
|
|
s.mustGetRegionID(c, []byte("c")): time.Millisecond * 20,
|
|
}
|
|
s.store.client = &slowClient{
|
|
Client: s.store.client,
|
|
regionDelays: delays,
|
|
}
|
|
|
|
txn1, txn2 := s.begin(c), s.begin(c)
|
|
// txn2 writes "b"
|
|
err := txn2.Set([]byte("b"), []byte("b2"))
|
|
c.Assert(err, IsNil)
|
|
err = txn2.Commit(context.Background())
|
|
c.Assert(err, IsNil)
|
|
// txn1 writes "a"(PK), "b", "c" on different regions.
|
|
// "b" will return an error and cancel commit.
|
|
err = txn1.Set([]byte("a"), []byte("a1"))
|
|
c.Assert(err, IsNil)
|
|
err = txn1.Set([]byte("b"), []byte("b1"))
|
|
c.Assert(err, IsNil)
|
|
err = txn1.Set([]byte("c"), []byte("c1"))
|
|
c.Assert(err, IsNil)
|
|
err = txn1.Commit(context.Background())
|
|
c.Assert(err, NotNil)
|
|
// "c" should be cleaned up in reasonable time.
|
|
for i := 0; i < 50; i++ {
|
|
if !s.isKeyLocked(c, []byte("c")) {
|
|
return
|
|
}
|
|
time.Sleep(time.Millisecond * 10)
|
|
}
|
|
c.Fail()
|
|
}
|
|
|
|
// slowClient wraps rpcClient and makes some regions respond with delay.
|
|
type slowClient struct {
|
|
Client
|
|
regionDelays map[uint64]time.Duration
|
|
}
|
|
|
|
func (c *slowClient) SendReq(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) {
|
|
for id, delay := range c.regionDelays {
|
|
reqCtx := &req.Context
|
|
if reqCtx.GetRegionId() == id {
|
|
time.Sleep(delay)
|
|
}
|
|
}
|
|
return c.Client.SendRequest(ctx, addr, req, timeout)
|
|
}
|
|
|
|
func (s *testCommitterSuite) TestIllegalTso(c *C) {
|
|
txn := s.begin(c)
|
|
data := map[string]string{
|
|
"name": "aa",
|
|
"age": "12",
|
|
}
|
|
for k, v := range data {
|
|
err := txn.Set([]byte(k), []byte(v))
|
|
c.Assert(err, IsNil)
|
|
}
|
|
// make start ts bigger.
|
|
txn.startTS = uint64(math.MaxUint64)
|
|
err := txn.Commit(context.Background())
|
|
c.Assert(err, NotNil)
|
|
errMsgMustContain(c, err, "invalid txnStartTS")
|
|
}
|
|
|
|
func errMsgMustContain(c *C, err error, msg string) {
|
|
c.Assert(strings.Contains(err.Error(), msg), IsTrue)
|
|
}
|
|
|
|
func newTwoPhaseCommitterWithInit(txn *tikvTxn, connID uint64) (*twoPhaseCommitter, error) {
|
|
c, err := newTwoPhaseCommitter(txn, connID)
|
|
if err != nil {
|
|
return nil, errors.Trace(err)
|
|
}
|
|
if err = c.initKeysAndMutations(); err != nil {
|
|
return nil, errors.Trace(err)
|
|
}
|
|
return c, nil
|
|
}
|
|
|
|
func (s *testCommitterSuite) TestCommitBeforePrewrite(c *C) {
|
|
txn := s.begin(c)
|
|
err := txn.Set([]byte("a"), []byte("a1"))
|
|
c.Assert(err, IsNil)
|
|
committer, err := newTwoPhaseCommitterWithInit(txn, 0)
|
|
c.Assert(err, IsNil)
|
|
ctx := context.Background()
|
|
err = committer.cleanupMutations(NewBackofferWithVars(ctx, cleanupMaxBackoff, nil), committer.mutations)
|
|
c.Assert(err, IsNil)
|
|
err = committer.prewriteMutations(NewBackofferWithVars(ctx, PrewriteMaxBackoff, nil), committer.mutations)
|
|
c.Assert(err, NotNil)
|
|
errMsgMustContain(c, err, "already rollbacked")
|
|
}
|
|
|
|
func (s *testCommitterSuite) TestPrewritePrimaryKeyFailed(c *C) {
|
|
// commit (a,a1)
|
|
txn1 := s.begin(c)
|
|
err := txn1.Set([]byte("a"), []byte("a1"))
|
|
c.Assert(err, IsNil)
|
|
err = txn1.Commit(context.Background())
|
|
c.Assert(err, IsNil)
|
|
|
|
// check a
|
|
txn := s.begin(c)
|
|
v, err := txn.Get(context.TODO(), []byte("a"))
|
|
c.Assert(err, IsNil)
|
|
c.Assert(v, BytesEquals, []byte("a1"))
|
|
|
|
// set txn2's startTs before txn1's
|
|
txn2 := s.begin(c)
|
|
txn2.startTS = txn1.startTS - 1
|
|
err = txn2.Set([]byte("a"), []byte("a2"))
|
|
c.Assert(err, IsNil)
|
|
err = txn2.Set([]byte("b"), []byte("b2"))
|
|
c.Assert(err, IsNil)
|
|
// prewrite:primary a failed, b success
|
|
err = txn2.Commit(context.Background())
|
|
c.Assert(err, NotNil)
|
|
|
|
// txn2 failed with a rollback for record a.
|
|
txn = s.begin(c)
|
|
v, err = txn.Get(context.TODO(), []byte("a"))
|
|
c.Assert(err, IsNil)
|
|
c.Assert(v, BytesEquals, []byte("a1"))
|
|
_, err = txn.Get(context.TODO(), []byte("b"))
|
|
errMsgMustContain(c, err, "key not exist")
|
|
|
|
// clean again, shouldn't be failed when a rollback already exist.
|
|
ctx := context.Background()
|
|
committer, err := newTwoPhaseCommitterWithInit(txn2, 0)
|
|
c.Assert(err, IsNil)
|
|
err = committer.cleanupMutations(NewBackofferWithVars(ctx, cleanupMaxBackoff, nil), committer.mutations)
|
|
c.Assert(err, IsNil)
|
|
|
|
// check the data after rollback twice.
|
|
txn = s.begin(c)
|
|
v, err = txn.Get(context.TODO(), []byte("a"))
|
|
c.Assert(err, IsNil)
|
|
c.Assert(v, BytesEquals, []byte("a1"))
|
|
|
|
// update data in a new txn, should be success.
|
|
err = txn.Set([]byte("a"), []byte("a3"))
|
|
c.Assert(err, IsNil)
|
|
err = txn.Commit(context.Background())
|
|
c.Assert(err, IsNil)
|
|
// check value
|
|
txn = s.begin(c)
|
|
v, err = txn.Get(context.TODO(), []byte("a"))
|
|
c.Assert(err, IsNil)
|
|
c.Assert(v, BytesEquals, []byte("a3"))
|
|
}
|
|
|
|
func (s *testCommitterSuite) TestWrittenKeysOnConflict(c *C) {
|
|
// This test checks that when there is a write conflict, written keys is collected,
|
|
// so we can use it to clean up keys.
|
|
region, _ := s.cluster.GetRegionByKey([]byte("x"))
|
|
newRegionID := s.cluster.AllocID()
|
|
newPeerID := s.cluster.AllocID()
|
|
s.cluster.Split(region.Id, newRegionID, []byte("y"), []uint64{newPeerID}, newPeerID)
|
|
var totalTime time.Duration
|
|
for i := 0; i < 10; i++ {
|
|
txn1 := s.begin(c)
|
|
txn2 := s.begin(c)
|
|
txn2.Set([]byte("x1"), []byte("1"))
|
|
committer2, err := newTwoPhaseCommitterWithInit(txn2, 2)
|
|
c.Assert(err, IsNil)
|
|
err = committer2.execute(context.Background())
|
|
c.Assert(err, IsNil)
|
|
txn1.Set([]byte("x1"), []byte("1"))
|
|
txn1.Set([]byte("y1"), []byte("2"))
|
|
committer1, err := newTwoPhaseCommitterWithInit(txn1, 2)
|
|
c.Assert(err, IsNil)
|
|
err = committer1.execute(context.Background())
|
|
c.Assert(err, NotNil)
|
|
committer1.cleanWg.Wait()
|
|
txn3 := s.begin(c)
|
|
start := time.Now()
|
|
txn3.Get(context.TODO(), []byte("y1"))
|
|
totalTime += time.Since(start)
|
|
txn3.Commit(context.Background())
|
|
}
|
|
c.Assert(totalTime, Less, time.Millisecond*200)
|
|
}
|
|
|
|
func (s *testCommitterSuite) TestPrewriteTxnSize(c *C) {
|
|
// Prepare two regions first: (, 100) and [100, )
|
|
region, _ := s.cluster.GetRegionByKey([]byte{50})
|
|
newRegionID := s.cluster.AllocID()
|
|
newPeerID := s.cluster.AllocID()
|
|
s.cluster.Split(region.Id, newRegionID, []byte{100}, []uint64{newPeerID}, newPeerID)
|
|
|
|
txn := s.begin(c)
|
|
var val [1024]byte
|
|
for i := byte(50); i < 120; i++ {
|
|
err := txn.Set([]byte{i}, val[:])
|
|
c.Assert(err, IsNil)
|
|
}
|
|
|
|
committer, err := newTwoPhaseCommitterWithInit(txn, 1)
|
|
c.Assert(err, IsNil)
|
|
|
|
ctx := context.Background()
|
|
err = committer.prewriteMutations(NewBackofferWithVars(ctx, PrewriteMaxBackoff, nil), committer.mutations)
|
|
c.Assert(err, IsNil)
|
|
|
|
// Check the written locks in the first region (50 keys)
|
|
for i := byte(50); i < 100; i++ {
|
|
lock := s.getLockInfo(c, []byte{i})
|
|
c.Assert(int(lock.TxnSize), Equals, 50)
|
|
}
|
|
|
|
// Check the written locks in the second region (20 keys)
|
|
for i := byte(100); i < 120; i++ {
|
|
lock := s.getLockInfo(c, []byte{i})
|
|
c.Assert(int(lock.TxnSize), Equals, 20)
|
|
}
|
|
}
|
|
|
|
func (s *testCommitterSuite) TestRejectCommitTS(c *C) {
|
|
txn := s.begin(c)
|
|
c.Assert(txn.Set([]byte("x"), []byte("v")), IsNil)
|
|
|
|
committer, err := newTwoPhaseCommitterWithInit(txn, 1)
|
|
c.Assert(err, IsNil)
|
|
bo := NewBackofferWithVars(context.Background(), getMaxBackoff, nil)
|
|
loc, err := s.store.regionCache.LocateKey(bo, []byte("x"))
|
|
c.Assert(err, IsNil)
|
|
mutations := []*kvrpcpb.Mutation{
|
|
{
|
|
Op: committer.mutations.ops[0],
|
|
Key: committer.mutations.keys[0],
|
|
Value: committer.mutations.values[0],
|
|
},
|
|
}
|
|
prewrite := &kvrpcpb.PrewriteRequest{
|
|
Mutations: mutations,
|
|
PrimaryLock: committer.primary(),
|
|
StartVersion: committer.startTS,
|
|
LockTtl: committer.lockTTL,
|
|
MinCommitTs: committer.startTS + 100, // Set minCommitTS
|
|
}
|
|
req := tikvrpc.NewRequest(tikvrpc.CmdPrewrite, prewrite)
|
|
_, err = s.store.SendReq(bo, req, loc.Region, readTimeoutShort)
|
|
c.Assert(err, IsNil)
|
|
|
|
// Make commitTS less than minCommitTS.
|
|
committer.commitTS = committer.startTS + 1
|
|
// Ensure that the new commit ts is greater than minCommitTS when retry
|
|
time.Sleep(3 * time.Millisecond)
|
|
err = committer.commitMutations(bo, committer.mutations)
|
|
c.Assert(err, IsNil)
|
|
|
|
// Use startTS+2 to read the data and get nothing.
|
|
// Use max.Uint64 to read the data and success.
|
|
// That means the final commitTS > startTS+2, it's not the one we provide.
|
|
// So we cover the rety commitTS logic.
|
|
txn1, err := s.store.BeginWithStartTS(committer.startTS + 2)
|
|
c.Assert(err, IsNil)
|
|
_, err = txn1.Get(bo.ctx, []byte("x"))
|
|
c.Assert(kv.IsErrNotFound(err), IsTrue)
|
|
|
|
txn2, err := s.store.BeginWithStartTS(math.MaxUint64)
|
|
c.Assert(err, IsNil)
|
|
val, err := txn2.Get(bo.ctx, []byte("x"))
|
|
c.Assert(err, IsNil)
|
|
c.Assert(bytes.Equal(val, []byte("v")), IsTrue)
|
|
}
|
|
|
|
func (s *testCommitterSuite) TestPessimisticPrewriteRequest(c *C) {
|
|
// This test checks that the isPessimisticLock field is set in the request even when no keys are pessimistic lock.
|
|
txn := s.begin(c)
|
|
txn.SetOption(kv.Pessimistic, true)
|
|
err := txn.Set([]byte("t1"), []byte("v1"))
|
|
c.Assert(err, IsNil)
|
|
committer, err := newTwoPhaseCommitterWithInit(txn, 0)
|
|
c.Assert(err, IsNil)
|
|
committer.forUpdateTS = 100
|
|
var batch batchMutations
|
|
batch.mutations = committer.mutations.subRange(0, 1)
|
|
batch.region = RegionVerID{1, 1, 1}
|
|
req := committer.buildPrewriteRequest(batch, 1)
|
|
c.Assert(len(req.Prewrite().IsPessimisticLock), Greater, 0)
|
|
c.Assert(req.Prewrite().ForUpdateTs, Equals, uint64(100))
|
|
}
|
|
|
|
func (s *testCommitterSuite) TestUnsetPrimaryKey(c *C) {
|
|
// This test checks that the isPessimisticLock field is set in the request even when no keys are pessimistic lock.
|
|
key := kv.Key("key")
|
|
txn := s.begin(c)
|
|
c.Assert(txn.Set(key, key), IsNil)
|
|
c.Assert(txn.Commit(context.Background()), IsNil)
|
|
|
|
txn = s.begin(c)
|
|
txn.SetOption(kv.Pessimistic, true)
|
|
txn.SetOption(kv.PresumeKeyNotExists, nil)
|
|
txn.SetOption(kv.PresumeKeyNotExistsError, kv.NewExistErrInfo("name", "value"))
|
|
_, _ = txn.us.Get(context.TODO(), key)
|
|
c.Assert(txn.Set(key, key), IsNil)
|
|
txn.DelOption(kv.PresumeKeyNotExistsError)
|
|
txn.DelOption(kv.PresumeKeyNotExists)
|
|
lockCtx := &kv.LockCtx{ForUpdateTS: txn.startTS, WaitStartTime: time.Now()}
|
|
err := txn.LockKeys(context.Background(), lockCtx, key)
|
|
c.Assert(err, NotNil)
|
|
c.Assert(txn.Delete(key), IsNil)
|
|
key2 := kv.Key("key2")
|
|
c.Assert(txn.Set(key2, key2), IsNil)
|
|
err = txn.Commit(context.Background())
|
|
c.Assert(err, IsNil)
|
|
}
|
|
|
|
func (s *testCommitterSuite) TestPessimisticLockedKeysDedup(c *C) {
|
|
txn := s.begin(c)
|
|
txn.SetOption(kv.Pessimistic, true)
|
|
lockCtx := &kv.LockCtx{ForUpdateTS: 100, WaitStartTime: time.Now()}
|
|
err := txn.LockKeys(context.Background(), lockCtx, kv.Key("abc"), kv.Key("def"))
|
|
c.Assert(err, IsNil)
|
|
lockCtx = &kv.LockCtx{ForUpdateTS: 100, WaitStartTime: time.Now()}
|
|
err = txn.LockKeys(context.Background(), lockCtx, kv.Key("abc"), kv.Key("def"))
|
|
c.Assert(err, IsNil)
|
|
c.Assert(txn.lockKeys, HasLen, 2)
|
|
}
|
|
|
|
func (s *testCommitterSuite) TestPessimisticTTL(c *C) {
|
|
key := kv.Key("key")
|
|
txn := s.begin(c)
|
|
txn.SetOption(kv.Pessimistic, true)
|
|
time.Sleep(time.Millisecond * 100)
|
|
lockCtx := &kv.LockCtx{ForUpdateTS: txn.startTS, WaitStartTime: time.Now()}
|
|
err := txn.LockKeys(context.Background(), lockCtx, key)
|
|
c.Assert(err, IsNil)
|
|
time.Sleep(time.Millisecond * 100)
|
|
key2 := kv.Key("key2")
|
|
lockCtx = &kv.LockCtx{ForUpdateTS: txn.startTS, WaitStartTime: time.Now()}
|
|
err = txn.LockKeys(context.Background(), lockCtx, key2)
|
|
c.Assert(err, IsNil)
|
|
lockInfo := s.getLockInfo(c, key)
|
|
msBeforeLockExpired := s.store.GetOracle().UntilExpired(txn.StartTS(), lockInfo.LockTtl)
|
|
c.Assert(msBeforeLockExpired, GreaterEqual, int64(100))
|
|
|
|
lr := newLockResolver(s.store)
|
|
bo := NewBackofferWithVars(context.Background(), getMaxBackoff, nil)
|
|
status, err := lr.getTxnStatus(bo, txn.startTS, key2, 0, txn.startTS, true, nil)
|
|
c.Assert(err, IsNil)
|
|
c.Assert(status.ttl, GreaterEqual, lockInfo.LockTtl)
|
|
|
|
// Check primary lock TTL is auto increasing while the pessimistic txn is ongoing.
|
|
for i := 0; i < 50; i++ {
|
|
lockInfoNew := s.getLockInfo(c, key)
|
|
if lockInfoNew.LockTtl > lockInfo.LockTtl {
|
|
currentTS, err := lr.store.GetOracle().GetTimestamp(bo.ctx)
|
|
c.Assert(err, IsNil)
|
|
// Check that the TTL is update to a reasonable range.
|
|
expire := oracle.ExtractPhysical(txn.startTS) + int64(lockInfoNew.LockTtl)
|
|
now := oracle.ExtractPhysical(currentTS)
|
|
c.Assert(expire > now, IsTrue)
|
|
c.Assert(uint64(expire-now) <= atomic.LoadUint64(&ManagedLockTTL), IsTrue)
|
|
return
|
|
}
|
|
time.Sleep(100 * time.Millisecond)
|
|
}
|
|
c.Assert(false, IsTrue, Commentf("update pessimistic ttl fail"))
|
|
}
|
|
|
|
func (s *testCommitterSuite) TestPessimisticLockReturnValues(c *C) {
|
|
key := kv.Key("key")
|
|
key2 := kv.Key("key2")
|
|
txn := s.begin(c)
|
|
c.Assert(txn.Set(key, key), IsNil)
|
|
c.Assert(txn.Set(key2, key2), IsNil)
|
|
c.Assert(txn.Commit(context.Background()), IsNil)
|
|
txn = s.begin(c)
|
|
txn.SetOption(kv.Pessimistic, true)
|
|
lockCtx := &kv.LockCtx{ForUpdateTS: txn.startTS, WaitStartTime: time.Now()}
|
|
lockCtx.ReturnValues = true
|
|
lockCtx.Values = map[string]kv.ReturnedValue{}
|
|
c.Assert(txn.LockKeys(context.Background(), lockCtx, key, key2), IsNil)
|
|
c.Assert(lockCtx.Values, HasLen, 2)
|
|
c.Assert(lockCtx.Values[string(key)].Value, BytesEquals, []byte(key))
|
|
c.Assert(lockCtx.Values[string(key2)].Value, BytesEquals, []byte(key2))
|
|
}
|
|
|
|
// TestElapsedTTL tests that elapsed time is correct even if ts physical time is greater than local time.
|
|
func (s *testCommitterSuite) TestElapsedTTL(c *C) {
|
|
key := kv.Key("key")
|
|
txn := s.begin(c)
|
|
txn.startTS = oracle.ComposeTS(oracle.GetPhysical(time.Now().Add(time.Second*10)), 1)
|
|
txn.SetOption(kv.Pessimistic, true)
|
|
time.Sleep(time.Millisecond * 100)
|
|
lockCtx := &kv.LockCtx{
|
|
ForUpdateTS: oracle.ComposeTS(oracle.ExtractPhysical(txn.startTS)+100, 1),
|
|
WaitStartTime: time.Now(),
|
|
}
|
|
err := txn.LockKeys(context.Background(), lockCtx, key)
|
|
c.Assert(err, IsNil)
|
|
lockInfo := s.getLockInfo(c, key)
|
|
c.Assert(lockInfo.LockTtl-atomic.LoadUint64(&ManagedLockTTL), GreaterEqual, uint64(100))
|
|
c.Assert(lockInfo.LockTtl-atomic.LoadUint64(&ManagedLockTTL), Less, uint64(150))
|
|
}
|
|
|
|
func (s *testCommitterSuite) TestDeleteYourWriteCauseGhostPrimary(c *C) {
|
|
s.cluster.SplitKeys(s.mvccStore, kv.Key("d"), kv.Key("a"), 4)
|
|
k1 := kv.Key("a") // insert but deleted key at first pos in txn1
|
|
k2 := kv.Key("b") // insert key at second pos in txn1
|
|
k3 := kv.Key("c") // insert key in txn1 and will be conflict read by txn2
|
|
|
|
// insert k1, k2, k3 and delete k1
|
|
txn1 := s.begin(c)
|
|
txn1.DelOption(kv.Pessimistic)
|
|
txn1.SetOption(kv.PresumeKeyNotExists, nil)
|
|
txn1.SetOption(kv.PresumeKeyNotExistsError, kv.NewExistErrInfo("name", "value"))
|
|
txn1.store.txnLatches = nil
|
|
txn1.Get(context.Background(), k1)
|
|
txn1.Set(k1, []byte{0})
|
|
txn1.Set(k2, []byte{1})
|
|
txn1.Set(k3, []byte{2})
|
|
txn1.Delete(k1)
|
|
committer1, err := newTwoPhaseCommitter(txn1, 0)
|
|
c.Assert(err, IsNil)
|
|
// setup test knob in txn's committer
|
|
committer1.testingKnobs.acAfterCommitPrimary = make(chan struct{})
|
|
committer1.testingKnobs.bkAfterCommitPrimary = make(chan struct{})
|
|
txn1.committer = committer1
|
|
var txn1Done sync.WaitGroup
|
|
txn1Done.Add(1)
|
|
go func() {
|
|
err1 := txn1.Commit(context.Background())
|
|
c.Assert(err1, IsNil)
|
|
txn1Done.Done()
|
|
}()
|
|
// resume after after primary key be committed
|
|
<-txn1.committer.testingKnobs.acAfterCommitPrimary
|
|
|
|
// start txn2 to read k3(prewrite success and primary should be committed)
|
|
txn2 := s.begin(c)
|
|
txn2.DelOption(kv.Pessimistic)
|
|
txn2.store.txnLatches = nil
|
|
v, err := txn2.Get(context.Background(), k3)
|
|
c.Assert(err, IsNil) // should resolve lock and read txn1 k3 result instead of rollback it.
|
|
c.Assert(v[0], Equals, byte(2))
|
|
txn1.committer.testingKnobs.bkAfterCommitPrimary <- struct{}{}
|
|
txn1Done.Wait()
|
|
}
|
|
|
|
func (s *testCommitterSuite) TestDeleteAllYourWrites(c *C) {
|
|
s.cluster.SplitKeys(s.mvccStore, kv.Key("d"), kv.Key("a"), 4)
|
|
k1 := kv.Key("a")
|
|
k2 := kv.Key("b")
|
|
k3 := kv.Key("c")
|
|
|
|
// insert k1, k2, k3 and delete k1, k2, k3
|
|
txn1 := s.begin(c)
|
|
txn1.DelOption(kv.Pessimistic)
|
|
txn1.SetOption(kv.PresumeKeyNotExists, nil)
|
|
txn1.SetOption(kv.PresumeKeyNotExistsError, kv.NewExistErrInfo("name", "value"))
|
|
txn1.store.txnLatches = nil
|
|
txn1.Get(context.Background(), k1)
|
|
txn1.Set(k1, []byte{0})
|
|
txn1.Delete(k1)
|
|
txn1.Get(context.Background(), k2)
|
|
txn1.Set(k2, []byte{1})
|
|
txn1.Delete(k2)
|
|
txn1.Get(context.Background(), k3)
|
|
txn1.Set(k3, []byte{2})
|
|
txn1.Delete(k3)
|
|
err1 := txn1.Commit(context.Background())
|
|
c.Assert(err1, IsNil)
|
|
}
|
|
|
|
func (s *testCommitterSuite) TestDeleteAllYourWritesWithSFU(c *C) {
|
|
s.cluster.SplitKeys(s.mvccStore, kv.Key("d"), kv.Key("a"), 4)
|
|
k1 := kv.Key("a")
|
|
k2 := kv.Key("b")
|
|
k3 := kv.Key("c")
|
|
|
|
// insert k1, k2, k2 and delete k1
|
|
txn1 := s.begin(c)
|
|
txn1.DelOption(kv.Pessimistic)
|
|
txn1.SetOption(kv.PresumeKeyNotExists, nil)
|
|
txn1.SetOption(kv.PresumeKeyNotExistsError, kv.NewExistErrInfo("name", "value"))
|
|
txn1.store.txnLatches = nil
|
|
txn1.Get(context.Background(), k1)
|
|
txn1.Set(k1, []byte{0})
|
|
txn1.Delete(k1)
|
|
err := txn1.LockKeys(context.Background(), &kv.LockCtx{}, k2, k3) // select * from t where x in (k2, k3) for update
|
|
c.Assert(err, IsNil)
|
|
|
|
committer1, err := newTwoPhaseCommitter(txn1, 0)
|
|
c.Assert(err, IsNil)
|
|
// setup test knob in txn's committer
|
|
committer1.testingKnobs.acAfterCommitPrimary = make(chan struct{})
|
|
committer1.testingKnobs.bkAfterCommitPrimary = make(chan struct{})
|
|
txn1.committer = committer1
|
|
var txn1Done sync.WaitGroup
|
|
txn1Done.Add(1)
|
|
go func() {
|
|
err1 := txn1.Commit(context.Background())
|
|
c.Assert(err1, IsNil)
|
|
txn1Done.Done()
|
|
}()
|
|
// resume after after primary key be committed
|
|
<-txn1.committer.testingKnobs.acAfterCommitPrimary
|
|
// start txn2 to read k3
|
|
txn2 := s.begin(c)
|
|
txn2.DelOption(kv.Pessimistic)
|
|
txn2.store.txnLatches = nil
|
|
err = txn2.Set(k3, []byte{33})
|
|
c.Assert(err, IsNil)
|
|
var meetLocks []*Lock
|
|
txn2.store.lockResolver.testingKnobs.meetLock = func(locks []*Lock) {
|
|
meetLocks = append(meetLocks, locks...)
|
|
}
|
|
err = txn2.Commit(context.Background())
|
|
c.Assert(err, IsNil)
|
|
txn1.committer.testingKnobs.bkAfterCommitPrimary <- struct{}{}
|
|
txn1Done.Wait()
|
|
c.Assert(meetLocks[0].Primary[0], Equals, k2[0])
|
|
}
|
|
|
|
// TestAcquireFalseTimeoutLock tests acquiring a key which is a secondary key of another transaction.
|
|
// The lock's own TTL is expired but the primary key is still alive due to heartbeats.
|
|
func (s *testCommitterSuite) TestAcquireFalseTimeoutLock(c *C) {
|
|
// k1 is the primary lock of txn1
|
|
k1 := kv.Key("k1")
|
|
// k2 is a secondary lock of txn1 and a key txn2 wants to lock
|
|
k2 := kv.Key("k2")
|
|
|
|
txn1 := s.begin(c)
|
|
txn1.SetOption(kv.Pessimistic, true)
|
|
// lock the primary key
|
|
lockCtx := &kv.LockCtx{ForUpdateTS: txn1.startTS, WaitStartTime: time.Now()}
|
|
err := txn1.LockKeys(context.Background(), lockCtx, k1)
|
|
c.Assert(err, IsNil)
|
|
// lock the secondary key
|
|
lockCtx = &kv.LockCtx{ForUpdateTS: txn1.startTS, WaitStartTime: time.Now()}
|
|
err = txn1.LockKeys(context.Background(), lockCtx, k2)
|
|
c.Assert(err, IsNil)
|
|
|
|
// Heartbeats will increase the TTL of the primary key
|
|
|
|
// wait until secondary key exceeds its own TTL
|
|
time.Sleep(time.Duration(atomic.LoadUint64(&ManagedLockTTL)) * time.Millisecond)
|
|
txn2 := s.begin(c)
|
|
txn2.SetOption(kv.Pessimistic, true)
|
|
|
|
// test no wait
|
|
lockCtx = &kv.LockCtx{ForUpdateTS: txn2.startTS, LockWaitTime: kv.LockNoWait, WaitStartTime: time.Now()}
|
|
startTime := time.Now()
|
|
err = txn2.LockKeys(context.Background(), lockCtx, k2)
|
|
elapsed := time.Since(startTime)
|
|
// cannot acquire lock immediately thus error
|
|
c.Assert(err.Error(), Equals, ErrLockAcquireFailAndNoWaitSet.Error())
|
|
// it should return immediately
|
|
c.Assert(elapsed, Less, 50*time.Millisecond)
|
|
|
|
// test for wait limited time (300ms)
|
|
lockCtx = &kv.LockCtx{ForUpdateTS: txn2.startTS, LockWaitTime: 300, WaitStartTime: time.Now()}
|
|
startTime = time.Now()
|
|
err = txn2.LockKeys(context.Background(), lockCtx, k2)
|
|
elapsed = time.Since(startTime)
|
|
// cannot acquire lock in time thus error
|
|
c.Assert(err.Error(), Equals, ErrLockWaitTimeout.Error())
|
|
// it should return after about 300ms
|
|
c.Assert(elapsed, Less, 350*time.Millisecond)
|
|
}
|
|
|
|
func (s *testCommitterSuite) getLockInfo(c *C, key []byte) *kvrpcpb.LockInfo {
|
|
txn := s.begin(c)
|
|
err := txn.Set(key, key)
|
|
c.Assert(err, IsNil)
|
|
committer, err := newTwoPhaseCommitterWithInit(txn, 1)
|
|
c.Assert(err, IsNil)
|
|
bo := NewBackofferWithVars(context.Background(), getMaxBackoff, nil)
|
|
loc, err := s.store.regionCache.LocateKey(bo, key)
|
|
c.Assert(err, IsNil)
|
|
batch := batchMutations{region: loc.Region, mutations: committer.mutations.subRange(0, 1)}
|
|
req := committer.buildPrewriteRequest(batch, 1)
|
|
resp, err := s.store.SendReq(bo, req, loc.Region, readTimeoutShort)
|
|
c.Assert(err, IsNil)
|
|
c.Assert(resp.Resp, NotNil)
|
|
keyErrs := (resp.Resp.(*kvrpcpb.PrewriteResponse)).Errors
|
|
c.Assert(keyErrs, HasLen, 1)
|
|
locked := keyErrs[0].Locked
|
|
c.Assert(locked, NotNil)
|
|
return locked
|
|
}
|
|
|
|
func (s *testCommitterSuite) TestPkNotFound(c *C) {
|
|
atomic.StoreUint64(&ManagedLockTTL, 100) // 100ms
|
|
defer atomic.StoreUint64(&ManagedLockTTL, 3000) // restore default value
|
|
// k1 is the primary lock of txn1
|
|
k1 := kv.Key("k1")
|
|
// k2 is a secondary lock of txn1 and a key txn2 wants to lock
|
|
k2 := kv.Key("k2")
|
|
k3 := kv.Key("k3")
|
|
|
|
txn1 := s.begin(c)
|
|
txn1.SetOption(kv.Pessimistic, true)
|
|
// lock the primary key
|
|
lockCtx := &kv.LockCtx{ForUpdateTS: txn1.startTS, WaitStartTime: time.Now()}
|
|
err := txn1.LockKeys(context.Background(), lockCtx, k1)
|
|
c.Assert(err, IsNil)
|
|
// lock the secondary key
|
|
lockCtx = &kv.LockCtx{ForUpdateTS: txn1.startTS, WaitStartTime: time.Now()}
|
|
err = txn1.LockKeys(context.Background(), lockCtx, k2)
|
|
c.Assert(err, IsNil)
|
|
|
|
// Stop txn ttl manager and remove primary key, like tidb server crashes and the priamry key lock does not exists actually,
|
|
// while the secondary lock operation succeeded
|
|
bo := NewBackofferWithVars(context.Background(), pessimisticLockMaxBackoff, nil)
|
|
txn1.committer.ttlManager.close()
|
|
err = txn1.committer.pessimisticRollbackMutations(bo, CommitterMutations{keys: [][]byte{k1}})
|
|
c.Assert(err, IsNil)
|
|
|
|
// Txn2 tries to lock the secondary key k2, dead loop if the left secondary lock by txn1 not resolved
|
|
txn2 := s.begin(c)
|
|
txn2.SetOption(kv.Pessimistic, true)
|
|
lockCtx = &kv.LockCtx{ForUpdateTS: txn2.startTS, WaitStartTime: time.Now()}
|
|
err = txn2.LockKeys(context.Background(), lockCtx, k2)
|
|
c.Assert(err, IsNil)
|
|
|
|
// Using smaller forUpdateTS cannot rollback this lock, other lock will fail
|
|
lockKey3 := &Lock{
|
|
Key: k3,
|
|
Primary: k1,
|
|
TxnID: txn1.startTS,
|
|
TTL: ManagedLockTTL,
|
|
TxnSize: txnCommitBatchSize,
|
|
LockType: kvrpcpb.Op_PessimisticLock,
|
|
LockForUpdateTS: txn1.startTS - 1,
|
|
}
|
|
cleanTxns := make(map[RegionVerID]struct{})
|
|
err = s.store.lockResolver.resolvePessimisticLock(bo, lockKey3, cleanTxns)
|
|
c.Assert(err, IsNil)
|
|
|
|
lockCtx = &kv.LockCtx{ForUpdateTS: txn1.startTS, WaitStartTime: time.Now()}
|
|
err = txn1.LockKeys(context.Background(), lockCtx, k3)
|
|
c.Assert(err, IsNil)
|
|
txn3 := s.begin(c)
|
|
txn3.SetOption(kv.Pessimistic, true)
|
|
lockCtx = &kv.LockCtx{ForUpdateTS: txn1.startTS - 1, WaitStartTime: time.Now(), LockWaitTime: kv.LockNoWait}
|
|
c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/txnNotFoundRetTTL", "return"), IsNil)
|
|
err = txn3.LockKeys(context.Background(), lockCtx, k3)
|
|
c.Assert(err.Error(), Equals, ErrLockAcquireFailAndNoWaitSet.Error())
|
|
c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/txnNotFoundRetTTL"), IsNil)
|
|
}
|
|
|
|
func (s *testCommitterSuite) TestPessimisticLockPrimary(c *C) {
|
|
// a is the primary lock of txn1
|
|
k1 := kv.Key("a")
|
|
// b is a secondary lock of txn1 and a key txn2 wants to lock, b is on another region
|
|
k2 := kv.Key("b")
|
|
|
|
txn1 := s.begin(c)
|
|
txn1.SetOption(kv.Pessimistic, true)
|
|
// txn1 lock k1
|
|
lockCtx := &kv.LockCtx{ForUpdateTS: txn1.startTS, WaitStartTime: time.Now()}
|
|
err := txn1.LockKeys(context.Background(), lockCtx, k1)
|
|
c.Assert(err, IsNil)
|
|
|
|
// txn2 wants to lock k1, k2, k1(pk) is blocked by txn1, pessimisticLockKeys has been changed to
|
|
// lock primary key first and then secondary keys concurrently, k2 should not be locked by txn2
|
|
doneCh := make(chan error)
|
|
go func() {
|
|
txn2 := s.begin(c)
|
|
txn2.SetOption(kv.Pessimistic, true)
|
|
lockCtx2 := &kv.LockCtx{ForUpdateTS: txn2.startTS, WaitStartTime: time.Now(), LockWaitTime: 200}
|
|
waitErr := txn2.LockKeys(context.Background(), lockCtx2, k1, k2)
|
|
doneCh <- waitErr
|
|
}()
|
|
time.Sleep(50 * time.Millisecond)
|
|
|
|
// txn3 should locks k2 successfully using no wait
|
|
txn3 := s.begin(c)
|
|
txn3.SetOption(kv.Pessimistic, true)
|
|
lockCtx3 := &kv.LockCtx{ForUpdateTS: txn3.startTS, WaitStartTime: time.Now(), LockWaitTime: kv.LockNoWait}
|
|
c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/txnNotFoundRetTTL", "return"), IsNil)
|
|
err = txn3.LockKeys(context.Background(), lockCtx3, k2)
|
|
c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/txnNotFoundRetTTL"), IsNil)
|
|
c.Assert(err, IsNil)
|
|
waitErr := <-doneCh
|
|
c.Assert(ErrLockWaitTimeout.Equal(waitErr), IsTrue)
|
|
}
|
|
|
|
func (c *twoPhaseCommitter) mutationsOfKeys(keys [][]byte) CommitterMutations {
|
|
var res CommitterMutations
|
|
for i := range c.mutations.keys {
|
|
for _, key := range keys {
|
|
if bytes.Equal(c.mutations.keys[i], key) {
|
|
res.Push(c.mutations.ops[i], c.mutations.keys[i], c.mutations.values[i], c.mutations.isPessimisticLock[i])
|
|
break
|
|
}
|
|
}
|
|
}
|
|
return res
|
|
}
|
|
|
|
func (s *testCommitterSuite) TestCommitDeadLock(c *C) {
|
|
// Split into two region and let k1 k2 in different regions.
|
|
s.cluster.SplitKeys(s.mvccStore, kv.Key("z"), kv.Key("a"), 2)
|
|
k1 := kv.Key("a_deadlock_k1")
|
|
k2 := kv.Key("y_deadlock_k2")
|
|
|
|
region1, _ := s.cluster.GetRegionByKey(k1)
|
|
region2, _ := s.cluster.GetRegionByKey(k2)
|
|
c.Assert(region1.Id != region2.Id, IsTrue)
|
|
|
|
txn1 := s.begin(c)
|
|
txn1.Set(k1, []byte("t1"))
|
|
txn1.Set(k2, []byte("t1"))
|
|
commit1, err := newTwoPhaseCommitterWithInit(txn1, 1)
|
|
c.Assert(err, IsNil)
|
|
commit1.primaryKey = k1
|
|
commit1.txnSize = 1000 * 1024 * 1024
|
|
commit1.lockTTL = txnLockTTL(txn1.startTime, commit1.txnSize)
|
|
|
|
txn2 := s.begin(c)
|
|
txn2.Set(k1, []byte("t2"))
|
|
txn2.Set(k2, []byte("t2"))
|
|
commit2, err := newTwoPhaseCommitterWithInit(txn2, 2)
|
|
c.Assert(err, IsNil)
|
|
commit2.primaryKey = k2
|
|
commit2.txnSize = 1000 * 1024 * 1024
|
|
commit2.lockTTL = txnLockTTL(txn1.startTime, commit2.txnSize)
|
|
|
|
s.cluster.ScheduleDelay(txn2.startTS, region1.Id, 5*time.Millisecond)
|
|
s.cluster.ScheduleDelay(txn1.startTS, region2.Id, 5*time.Millisecond)
|
|
|
|
// Txn1 prewrites k1, k2 and txn2 prewrites k2, k1, the large txn
|
|
// protocol run ttlManager and update their TTL, cause dead lock.
|
|
ch := make(chan error, 2)
|
|
var wg sync.WaitGroup
|
|
wg.Add(1)
|
|
go func() {
|
|
ch <- commit2.execute(context.Background())
|
|
wg.Done()
|
|
}()
|
|
ch <- commit1.execute(context.Background())
|
|
wg.Wait()
|
|
close(ch)
|
|
|
|
res := 0
|
|
for e := range ch {
|
|
if e != nil {
|
|
res++
|
|
}
|
|
}
|
|
c.Assert(res, Equals, 1)
|
|
}
|
|
|
|
// TestPushPessimisticLock tests that push forward the minCommiTS of pessimistic locks.
|
|
func (s *testCommitterSuite) TestPushPessimisticLock(c *C) {
|
|
// k1 is the primary key.
|
|
k1, k2 := kv.Key("a"), kv.Key("b")
|
|
ctx := context.Background()
|
|
|
|
txn1 := s.begin(c)
|
|
txn1.SetOption(kv.Pessimistic, true)
|
|
lockCtx := &kv.LockCtx{ForUpdateTS: txn1.startTS, WaitStartTime: time.Now()}
|
|
err := txn1.LockKeys(context.Background(), lockCtx, k1, k2)
|
|
c.Assert(err, IsNil)
|
|
|
|
txn1.Set(k2, []byte("v2"))
|
|
err = txn1.committer.initKeysAndMutations()
|
|
c.Assert(err, IsNil)
|
|
// Strip the prewrite of the primary key.
|
|
txn1.committer.mutations = txn1.committer.mutations.subRange(1, 2)
|
|
c.Assert(err, IsNil)
|
|
err = txn1.committer.prewriteMutations(NewBackofferWithVars(ctx, PrewriteMaxBackoff, nil), txn1.committer.mutations)
|
|
c.Assert(err, IsNil)
|
|
// The primary lock is a pessimistic lock and the secondary lock is a optimistic lock.
|
|
lock1 := s.getLockInfo(c, k1)
|
|
c.Assert(lock1.LockType, Equals, kvrpcpb.Op_PessimisticLock)
|
|
c.Assert(lock1.PrimaryLock, BytesEquals, []byte(k1))
|
|
lock2 := s.getLockInfo(c, k2)
|
|
c.Assert(lock2.LockType, Equals, kvrpcpb.Op_Put)
|
|
c.Assert(lock2.PrimaryLock, BytesEquals, []byte(k1))
|
|
|
|
txn2 := s.begin(c)
|
|
start := time.Now()
|
|
_, err = txn2.Get(ctx, k2)
|
|
elapsed := time.Since(start)
|
|
// The optimistic lock shouldn't block reads.
|
|
c.Assert(elapsed, Less, 500*time.Millisecond)
|
|
c.Assert(kv.IsErrNotFound(err), IsTrue)
|
|
|
|
txn1.Rollback()
|
|
txn2.Rollback()
|
|
}
|
|
|
|
// TestResolveMixed tests mixed resolve with left behind optimistic locks and pessimistic locks,
|
|
// using clean whole region resolve path
|
|
func (s *testCommitterSuite) TestResolveMixed(c *C) {
|
|
atomic.StoreUint64(&ManagedLockTTL, 100) // 100ms
|
|
defer atomic.StoreUint64(&ManagedLockTTL, 3000) // restore default value
|
|
ctx := context.Background()
|
|
|
|
// pk is the primary lock of txn1
|
|
pk := kv.Key("pk")
|
|
secondaryLockkeys := make([]kv.Key, 0, bigTxnThreshold)
|
|
for i := 0; i < bigTxnThreshold; i++ {
|
|
optimisticLock := kv.Key(fmt.Sprintf("optimisticLockKey%d", i))
|
|
secondaryLockkeys = append(secondaryLockkeys, optimisticLock)
|
|
}
|
|
pessimisticLockKey := kv.Key("pessimisticLockKey")
|
|
|
|
// make the optimistic and pessimistic lock left with primary lock not found
|
|
txn1 := s.begin(c)
|
|
txn1.SetOption(kv.Pessimistic, true)
|
|
// lock the primary key
|
|
lockCtx := &kv.LockCtx{ForUpdateTS: txn1.startTS, WaitStartTime: time.Now()}
|
|
err := txn1.LockKeys(context.Background(), lockCtx, pk)
|
|
c.Assert(err, IsNil)
|
|
// lock the optimistic keys
|
|
for i := 0; i < bigTxnThreshold; i++ {
|
|
txn1.Set(secondaryLockkeys[i], []byte(fmt.Sprintf("v%d", i)))
|
|
}
|
|
err = txn1.committer.initKeysAndMutations()
|
|
c.Assert(err, IsNil)
|
|
err = txn1.committer.prewriteMutations(NewBackofferWithVars(ctx, PrewriteMaxBackoff, nil), txn1.committer.mutations)
|
|
c.Assert(err, IsNil)
|
|
// lock the pessimistic keys
|
|
err = txn1.LockKeys(context.Background(), lockCtx, pessimisticLockKey)
|
|
c.Assert(err, IsNil)
|
|
lock1 := s.getLockInfo(c, pessimisticLockKey)
|
|
c.Assert(lock1.LockType, Equals, kvrpcpb.Op_PessimisticLock)
|
|
c.Assert(lock1.PrimaryLock, BytesEquals, []byte(pk))
|
|
optimisticLockKey := secondaryLockkeys[0]
|
|
lock2 := s.getLockInfo(c, optimisticLockKey)
|
|
c.Assert(lock2.LockType, Equals, kvrpcpb.Op_Put)
|
|
c.Assert(lock2.PrimaryLock, BytesEquals, []byte(pk))
|
|
|
|
// stop txn ttl manager and remove primary key, make the other keys left behind
|
|
bo := NewBackofferWithVars(context.Background(), pessimisticLockMaxBackoff, nil)
|
|
txn1.committer.ttlManager.close()
|
|
err = txn1.committer.pessimisticRollbackMutations(bo, CommitterMutations{keys: [][]byte{pk}})
|
|
c.Assert(err, IsNil)
|
|
|
|
// try to resolve the left optimistic locks, use clean whole region
|
|
cleanTxns := make(map[RegionVerID]struct{})
|
|
time.Sleep(time.Duration(atomic.LoadUint64(&ManagedLockTTL)) * time.Millisecond)
|
|
optimisticLockInfo := s.getLockInfo(c, optimisticLockKey)
|
|
lock := NewLock(optimisticLockInfo)
|
|
err = s.store.lockResolver.resolveLock(NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, nil), lock, TxnStatus{}, false, cleanTxns)
|
|
c.Assert(err, IsNil)
|
|
|
|
// txn2 tries to lock the pessimisticLockKey, the lock should has been resolved in clean whole region resolve
|
|
txn2 := s.begin(c)
|
|
txn2.SetOption(kv.Pessimistic, true)
|
|
lockCtx = &kv.LockCtx{ForUpdateTS: txn2.startTS, WaitStartTime: time.Now(), LockWaitTime: kv.LockNoWait}
|
|
err = txn2.LockKeys(context.Background(), lockCtx, pessimisticLockKey)
|
|
c.Assert(err, IsNil)
|
|
|
|
err = txn1.Rollback()
|
|
c.Assert(err, IsNil)
|
|
err = txn2.Rollback()
|
|
c.Assert(err, IsNil)
|
|
}
|
|
|