*: add integration_tests (#82)

This commit is contained in:
disksing 2021-06-16 14:57:17 +08:00 committed by GitHub
parent 94f269a0f9
commit a76108498b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
26 changed files with 6301 additions and 0 deletions

View File

@ -0,0 +1,312 @@
// Copyright 2020 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_test
import (
"context"
. "github.com/pingcap/check"
"github.com/tikv/client-go/v2/metrics"
"github.com/tikv/client-go/v2/mockstore"
"github.com/tikv/client-go/v2/oracle"
"github.com/tikv/client-go/v2/tikv"
"github.com/tikv/client-go/v2/util"
)
func (s *testAsyncCommitCommon) begin1PC(c *C) tikv.TxnProbe {
txn, err := s.store.Begin()
c.Assert(err, IsNil)
txn.SetEnable1PC(true)
return tikv.TxnProbe{KVTxn: txn}
}
type testOnePCSuite struct {
OneByOneSuite
testAsyncCommitCommon
bo *tikv.Backoffer
}
var _ = SerialSuites(&testOnePCSuite{})
func (s *testOnePCSuite) SetUpTest(c *C) {
s.testAsyncCommitCommon.setUpTest(c)
s.bo = tikv.NewBackofferWithVars(context.Background(), 5000, nil)
}
func (s *testOnePCSuite) Test1PC(c *C) {
ctx := context.WithValue(context.Background(), util.SessionID, uint64(1))
k1 := []byte("k1")
v1 := []byte("v1")
txn := s.begin1PC(c)
err := txn.Set(k1, v1)
c.Assert(err, IsNil)
err = txn.Commit(ctx)
c.Assert(err, IsNil)
c.Assert(txn.GetCommitter().IsOnePC(), IsTrue)
c.Assert(txn.GetCommitter().GetOnePCCommitTS(), Equals, txn.GetCommitter().GetCommitTS())
c.Assert(txn.GetCommitter().GetOnePCCommitTS(), Greater, txn.StartTS())
// ttlManager is not used for 1PC.
c.Assert(txn.GetCommitter().IsTTLUninitialized(), IsTrue)
// 1PC doesn't work if sessionID == 0
k2 := []byte("k2")
v2 := []byte("v2")
txn = s.begin1PC(c)
err = txn.Set(k2, v2)
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
c.Assert(txn.GetCommitter().IsOnePC(), IsFalse)
c.Assert(txn.GetCommitter().GetOnePCCommitTS(), Equals, uint64(0))
c.Assert(txn.GetCommitter().GetCommitTS(), Greater, txn.StartTS())
// 1PC doesn't work if system variable not set
k3 := []byte("k3")
v3 := []byte("v3")
txn = s.begin(c)
err = txn.Set(k3, v3)
c.Assert(err, IsNil)
err = txn.Commit(ctx)
c.Assert(err, IsNil)
c.Assert(txn.GetCommitter().IsOnePC(), IsFalse)
c.Assert(txn.GetCommitter().GetOnePCCommitTS(), Equals, uint64(0))
c.Assert(txn.GetCommitter().GetCommitTS(), Greater, txn.StartTS())
// Test multiple keys
k4 := []byte("k4")
v4 := []byte("v4")
k5 := []byte("k5")
v5 := []byte("v5")
k6 := []byte("k6")
v6 := []byte("v6")
txn = s.begin1PC(c)
err = txn.Set(k4, v4)
c.Assert(err, IsNil)
err = txn.Set(k5, v5)
c.Assert(err, IsNil)
err = txn.Set(k6, v6)
c.Assert(err, IsNil)
err = txn.Commit(ctx)
c.Assert(err, IsNil)
c.Assert(txn.GetCommitter().IsOnePC(), IsTrue)
c.Assert(txn.GetCommitter().GetOnePCCommitTS(), Equals, txn.GetCommitter().GetCommitTS())
c.Assert(txn.GetCommitter().GetOnePCCommitTS(), Greater, txn.StartTS())
// Check keys are committed with the same version
s.mustGetFromSnapshot(c, txn.GetCommitTS(), k4, v4)
s.mustGetFromSnapshot(c, txn.GetCommitTS(), k5, v5)
s.mustGetFromSnapshot(c, txn.GetCommitTS(), k6, v6)
s.mustGetNoneFromSnapshot(c, txn.GetCommitTS()-1, k4)
s.mustGetNoneFromSnapshot(c, txn.GetCommitTS()-1, k5)
s.mustGetNoneFromSnapshot(c, txn.GetCommitTS()-1, k6)
// Overwriting in MVCC
v6New := []byte("v6new")
txn = s.begin1PC(c)
err = txn.Set(k6, v6New)
c.Assert(err, IsNil)
err = txn.Commit(ctx)
c.Assert(err, IsNil)
c.Assert(txn.GetCommitter().IsOnePC(), IsTrue)
c.Assert(txn.GetCommitter().GetOnePCCommitTS(), Equals, txn.GetCommitter().GetCommitTS())
c.Assert(txn.GetCommitter().GetOnePCCommitTS(), Greater, txn.StartTS())
s.mustGetFromSnapshot(c, txn.GetCommitTS(), k6, v6New)
s.mustGetFromSnapshot(c, txn.GetCommitTS()-1, k6, v6)
// Check all keys
keys := [][]byte{k1, k2, k3, k4, k5, k6}
values := [][]byte{v1, v2, v3, v4, v5, v6New}
ver, err := s.store.CurrentTimestamp(oracle.GlobalTxnScope)
c.Assert(err, IsNil)
snap := s.store.GetSnapshot(ver)
for i, k := range keys {
v, err := snap.Get(ctx, k)
c.Assert(err, IsNil)
c.Assert(v, BytesEquals, values[i])
}
}
func (s *testOnePCSuite) Test1PCIsolation(c *C) {
ctx := context.WithValue(context.Background(), util.SessionID, uint64(1))
k := []byte("k")
v1 := []byte("v1")
txn := s.begin1PC(c)
txn.Set(k, v1)
err := txn.Commit(ctx)
c.Assert(err, IsNil)
v2 := []byte("v2")
txn = s.begin1PC(c)
txn.Set(k, v2)
// Make `txn`'s commitTs more likely to be less than `txn2`'s startTs if there's bug in commitTs
// calculation.
for i := 0; i < 10; i++ {
_, err := s.store.GetOracle().GetTimestamp(ctx, &oracle.Option{TxnScope: oracle.GlobalTxnScope})
c.Assert(err, IsNil)
}
txn2 := s.begin1PC(c)
s.mustGetFromTxn(c, txn2, k, v1)
err = txn.Commit(ctx)
c.Assert(txn.GetCommitter().IsOnePC(), IsTrue)
c.Assert(err, IsNil)
s.mustGetFromTxn(c, txn2, k, v1)
c.Assert(txn2.Rollback(), IsNil)
s.mustGetFromSnapshot(c, txn.GetCommitTS(), k, v2)
s.mustGetFromSnapshot(c, txn.GetCommitTS()-1, k, v1)
}
func (s *testOnePCSuite) Test1PCDisallowMultiRegion(c *C) {
// This test doesn't support tikv mode.
if *mockstore.WithTiKV {
return
}
ctx := context.WithValue(context.Background(), util.SessionID, uint64(1))
txn := s.begin1PC(c)
keys := []string{"k0", "k1", "k2", "k3"}
values := []string{"v0", "v1", "v2", "v3"}
err := txn.Set([]byte(keys[0]), []byte(values[0]))
c.Assert(err, IsNil)
err = txn.Set([]byte(keys[3]), []byte(values[3]))
c.Assert(err, IsNil)
err = txn.Commit(ctx)
c.Assert(err, IsNil)
// 1PC doesn't work if it affects multiple regions.
loc, err := s.store.GetRegionCache().LocateKey(s.bo, []byte(keys[2]))
c.Assert(err, IsNil)
newRegionID := s.cluster.AllocID()
newPeerID := s.cluster.AllocID()
s.cluster.Split(loc.Region.GetID(), newRegionID, []byte(keys[2]), []uint64{newPeerID}, newPeerID)
txn = s.begin1PC(c)
err = txn.Set([]byte(keys[1]), []byte(values[1]))
c.Assert(err, IsNil)
err = txn.Set([]byte(keys[2]), []byte(values[2]))
c.Assert(err, IsNil)
err = txn.Commit(ctx)
c.Assert(err, IsNil)
c.Assert(txn.GetCommitter().IsOnePC(), IsFalse)
c.Assert(txn.GetCommitter().GetOnePCCommitTS(), Equals, uint64(0))
c.Assert(txn.GetCommitter().GetCommitTS(), Greater, txn.StartTS())
ver, err := s.store.CurrentTimestamp(oracle.GlobalTxnScope)
c.Assert(err, IsNil)
snap := s.store.GetSnapshot(ver)
for i, k := range keys {
v, err := snap.Get(ctx, []byte(k))
c.Assert(err, IsNil)
c.Assert(v, BytesEquals, []byte(values[i]))
}
}
// It's just a simple validation of linearizability.
// Extra tests are needed to test this feature with the control of the TiKV cluster.
func (s *testOnePCSuite) Test1PCLinearizability(c *C) {
t1 := s.begin(c)
t2 := s.begin(c)
err := t1.Set([]byte("a"), []byte("a1"))
c.Assert(err, IsNil)
err = t2.Set([]byte("b"), []byte("b1"))
c.Assert(err, IsNil)
ctx := context.WithValue(context.Background(), util.SessionID, uint64(1))
// t2 commits earlier than t1
err = t2.Commit(ctx)
c.Assert(err, IsNil)
err = t1.Commit(ctx)
c.Assert(err, IsNil)
commitTS1 := t1.GetCommitter().GetCommitTS()
commitTS2 := t2.GetCommitter().GetCommitTS()
c.Assert(commitTS2, Less, commitTS1)
}
func (s *testOnePCSuite) Test1PCWithMultiDC(c *C) {
// It requires setting placement rules to run with TiKV
if *mockstore.WithTiKV {
return
}
localTxn := s.begin1PC(c)
err := localTxn.Set([]byte("a"), []byte("a1"))
localTxn.SetScope("bj")
c.Assert(err, IsNil)
ctx := context.WithValue(context.Background(), util.SessionID, uint64(1))
err = localTxn.Commit(ctx)
c.Assert(err, IsNil)
c.Assert(localTxn.GetCommitter().IsOnePC(), IsFalse)
globalTxn := s.begin1PC(c)
err = globalTxn.Set([]byte("b"), []byte("b1"))
globalTxn.SetScope(oracle.GlobalTxnScope)
c.Assert(err, IsNil)
err = globalTxn.Commit(ctx)
c.Assert(err, IsNil)
c.Assert(globalTxn.GetCommitter().IsOnePC(), IsTrue)
}
func (s *testOnePCSuite) TestTxnCommitCounter(c *C) {
initial := metrics.GetTxnCommitCounter()
// 2PC
txn := s.begin(c)
err := txn.Set([]byte("k"), []byte("v"))
c.Assert(err, IsNil)
ctx := context.WithValue(context.Background(), util.SessionID, uint64(1))
err = txn.Commit(ctx)
c.Assert(err, IsNil)
curr := metrics.GetTxnCommitCounter()
diff := curr.Sub(initial)
c.Assert(diff.TwoPC, Equals, int64(1))
c.Assert(diff.AsyncCommit, Equals, int64(0))
c.Assert(diff.OnePC, Equals, int64(0))
// AsyncCommit
txn = s.beginAsyncCommit(c)
err = txn.Set([]byte("k1"), []byte("v1"))
c.Assert(err, IsNil)
err = txn.Commit(ctx)
c.Assert(err, IsNil)
curr = metrics.GetTxnCommitCounter()
diff = curr.Sub(initial)
c.Assert(diff.TwoPC, Equals, int64(1))
c.Assert(diff.AsyncCommit, Equals, int64(1))
c.Assert(diff.OnePC, Equals, int64(0))
// 1PC
txn = s.begin1PC(c)
err = txn.Set([]byte("k2"), []byte("v2"))
c.Assert(err, IsNil)
err = txn.Commit(ctx)
c.Assert(err, IsNil)
curr = metrics.GetTxnCommitCounter()
diff = curr.Sub(initial)
c.Assert(diff.TwoPC, Equals, int64(1))
c.Assert(diff.AsyncCommit, Equals, int64(1))
c.Assert(diff.OnePC, Equals, int64(1))
}

View File

@ -0,0 +1,134 @@
// Copyright 2017 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_test
import (
"context"
. "github.com/pingcap/check"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/parser/terror"
tikverr "github.com/tikv/client-go/v2/error"
)
// TestFailCommitPrimaryRpcErrors tests rpc errors are handled properly when
// committing primary region task.
func (s *testCommitterSuite) TestFailCommitPrimaryRpcErrors(c *C) {
c.Assert(failpoint.Enable("tikvclient/rpcCommitResult", `return("timeout")`), IsNil)
defer func() {
c.Assert(failpoint.Disable("tikvclient/rpcCommitResult"), IsNil)
}()
// The rpc error will be wrapped to ErrResultUndetermined.
t1 := s.begin(c)
err := t1.Set([]byte("a"), []byte("a1"))
c.Assert(err, IsNil)
err = t1.Commit(context.Background())
c.Assert(err, NotNil)
c.Assert(terror.ErrorEqual(err, terror.ErrResultUndetermined), IsTrue, Commentf("%s", errors.ErrorStack(err)))
// We don't need to call "Rollback" after "Commit" fails.
err = t1.Rollback()
c.Assert(err, Equals, tikverr.ErrInvalidTxn)
}
// TestFailCommitPrimaryRegionError tests RegionError is handled properly when
// committing primary region task.
func (s *testCommitterSuite) TestFailCommitPrimaryRegionError(c *C) {
c.Assert(failpoint.Enable("tikvclient/rpcCommitResult", `return("notLeader")`), IsNil)
defer func() {
c.Assert(failpoint.Disable("tikvclient/rpcCommitResult"), IsNil)
}()
// Ensure it returns the original error without wrapped to ErrResultUndetermined
// if it exceeds max retry timeout on RegionError.
t2 := s.begin(c)
err := t2.Set([]byte("b"), []byte("b1"))
c.Assert(err, IsNil)
err = t2.Commit(context.Background())
c.Assert(err, NotNil)
c.Assert(terror.ErrorNotEqual(err, terror.ErrResultUndetermined), IsTrue)
}
// TestFailCommitPrimaryRPCErrorThenRegionError tests the case when commit first
// receive a rpc timeout, then region errors afterwrards.
func (s *testCommitterSuite) TestFailCommitPrimaryRPCErrorThenRegionError(c *C) {
c.Assert(failpoint.Enable("tikvclient/rpcCommitResult", `1*return("timeout")->return("notLeader")`), IsNil)
defer func() {
c.Assert(failpoint.Disable("tikvclient/rpcCommitResult"), IsNil)
}()
// The region error will be wrapped to ErrResultUndetermined.
t1 := s.begin(c)
err := t1.Set([]byte("a"), []byte("a1"))
c.Assert(err, IsNil)
err = t1.Commit(context.Background())
c.Assert(err, NotNil)
c.Assert(terror.ErrorEqual(err, terror.ErrResultUndetermined), IsTrue, Commentf("%s", errors.ErrorStack(err)))
}
// TestFailCommitPrimaryKeyError tests KeyError is handled properly when
// committing primary region task.
func (s *testCommitterSuite) TestFailCommitPrimaryKeyError(c *C) {
c.Assert(failpoint.Enable("tikvclient/rpcCommitResult", `return("keyError")`), IsNil)
defer func() {
c.Assert(failpoint.Disable("tikvclient/rpcCommitResult"), IsNil)
}()
// Ensure it returns the original error without wrapped to ErrResultUndetermined
// if it meets KeyError.
t3 := s.begin(c)
err := t3.Set([]byte("c"), []byte("c1"))
c.Assert(err, IsNil)
err = t3.Commit(context.Background())
c.Assert(err, NotNil)
c.Assert(terror.ErrorNotEqual(err, terror.ErrResultUndetermined), IsTrue)
}
// TestFailCommitPrimaryRPCErrorThenKeyError tests KeyError overwrites the undeterminedErr.
func (s *testCommitterSuite) TestFailCommitPrimaryRPCErrorThenKeyError(c *C) {
c.Assert(failpoint.Enable("tikvclient/rpcCommitResult", `1*return("timeout")->return("keyError")`), IsNil)
defer func() {
c.Assert(failpoint.Disable("tikvclient/rpcCommitResult"), IsNil)
}()
// Ensure it returns the original error without wrapped to ErrResultUndetermined
// if it meets KeyError.
t3 := s.begin(c)
err := t3.Set([]byte("c"), []byte("c1"))
c.Assert(err, IsNil)
err = t3.Commit(context.Background())
c.Assert(err, NotNil)
c.Assert(terror.ErrorEqual(err, terror.ErrResultUndetermined), IsFalse)
}
func (s *testCommitterSuite) TestFailCommitTimeout(c *C) {
c.Assert(failpoint.Enable("tikvclient/rpcCommitTimeout", `return(true)`), IsNil)
defer func() {
c.Assert(failpoint.Disable("tikvclient/rpcCommitTimeout"), IsNil)
}()
txn := s.begin(c)
err := txn.Set([]byte("a"), []byte("a1"))
c.Assert(err, IsNil)
err = txn.Set([]byte("b"), []byte("b1"))
c.Assert(err, IsNil)
err = txn.Set([]byte("c"), []byte("c1"))
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, NotNil)
txn2 := s.begin(c)
value, err := txn2.Get(context.TODO(), []byte("a"))
c.Assert(err, IsNil)
c.Assert(len(value), Greater, 0)
_, err = txn2.Get(context.TODO(), []byte("b"))
c.Assert(err, IsNil)
c.Assert(len(value), Greater, 0)
}

View File

@ -0,0 +1,39 @@
// 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.
// +build !race
package tikv_test
import (
. "github.com/pingcap/check"
)
// TestCommitMultipleRegions tests commit multiple regions.
// The test takes too long under the race detector.
func (s *testCommitterSuite) TestCommitMultipleRegions(c *C) {
m := make(map[string]string)
for i := 0; i < 100; i++ {
k, v := randKV(10, 10)
m[k] = v
}
s.mustCommit(c, m)
// Test big values.
m = make(map[string]string)
for i := 0; i < 50; i++ {
k, v := randKV(11, int(txnCommitBatchSize)/7)
m[k] = v
}
s.mustCommit(c, m)
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,284 @@
// Copyright 2020 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_test
import (
"bytes"
"context"
"sort"
. "github.com/pingcap/check"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pingcap/parser/terror"
tikverr "github.com/tikv/client-go/v2/error"
"github.com/tikv/client-go/v2/mockstore"
"github.com/tikv/client-go/v2/tikv"
"github.com/tikv/client-go/v2/util"
)
type testAsyncCommitFailSuite struct {
OneByOneSuite
testAsyncCommitCommon
}
var _ = SerialSuites(&testAsyncCommitFailSuite{})
func (s *testAsyncCommitFailSuite) SetUpTest(c *C) {
s.testAsyncCommitCommon.setUpTest(c)
}
// TestFailCommitPrimaryRpcErrors tests rpc errors are handled properly when
// committing primary region task.
func (s *testAsyncCommitFailSuite) TestFailAsyncCommitPrewriteRpcErrors(c *C) {
// This test doesn't support tikv mode because it needs setting failpoint in unistore.
if *mockstore.WithTiKV {
return
}
c.Assert(failpoint.Enable("tikvclient/noRetryOnRpcError", "return(true)"), IsNil)
c.Assert(failpoint.Enable("tikvclient/rpcPrewriteTimeout", `return(true)`), IsNil)
defer func() {
c.Assert(failpoint.Disable("tikvclient/rpcPrewriteTimeout"), IsNil)
c.Assert(failpoint.Disable("tikvclient/noRetryOnRpcError"), IsNil)
}()
// The rpc error will be wrapped to ErrResultUndetermined.
t1 := s.beginAsyncCommit(c)
err := t1.Set([]byte("a"), []byte("a1"))
c.Assert(err, IsNil)
ctx := context.WithValue(context.Background(), util.SessionID, uint64(1))
err = t1.Commit(ctx)
c.Assert(err, NotNil)
c.Assert(terror.ErrorEqual(err, terror.ErrResultUndetermined), IsTrue, Commentf("%s", errors.ErrorStack(err)))
// We don't need to call "Rollback" after "Commit" fails.
err = t1.Rollback()
c.Assert(err, Equals, tikverr.ErrInvalidTxn)
// Create a new transaction to check. The previous transaction should actually commit.
t2 := s.beginAsyncCommit(c)
res, err := t2.Get(context.Background(), []byte("a"))
c.Assert(err, IsNil)
c.Assert(bytes.Equal(res, []byte("a1")), IsTrue)
}
func (s *testAsyncCommitFailSuite) TestAsyncCommitPrewriteCancelled(c *C) {
// This test doesn't support tikv mode because it needs setting failpoint in unistore.
if *mockstore.WithTiKV {
return
}
// Split into two regions.
splitKey := "s"
bo := tikv.NewBackofferWithVars(context.Background(), 5000, nil)
loc, err := s.store.GetRegionCache().LocateKey(bo, []byte(splitKey))
c.Assert(err, IsNil)
newRegionID := s.cluster.AllocID()
newPeerID := s.cluster.AllocID()
s.cluster.Split(loc.Region.GetID(), newRegionID, []byte(splitKey), []uint64{newPeerID}, newPeerID)
s.store.GetRegionCache().InvalidateCachedRegion(loc.Region)
c.Assert(failpoint.Enable("tikvclient/rpcPrewriteResult", `1*return("writeConflict")->sleep(50)`), IsNil)
defer func() {
c.Assert(failpoint.Disable("tikvclient/rpcPrewriteResult"), IsNil)
}()
t1 := s.beginAsyncCommit(c)
err = t1.Set([]byte("a"), []byte("a"))
c.Assert(err, IsNil)
err = t1.Set([]byte("z"), []byte("z"))
c.Assert(err, IsNil)
ctx := context.WithValue(context.Background(), util.SessionID, uint64(1))
err = t1.Commit(ctx)
c.Assert(err, NotNil)
_, ok := errors.Cause(err).(*tikverr.ErrWriteConflict)
c.Assert(ok, IsTrue, Commentf("%s", errors.ErrorStack(err)))
}
func (s *testAsyncCommitFailSuite) TestPointGetWithAsyncCommit(c *C) {
s.putAlphabets(c, true)
txn := s.beginAsyncCommit(c)
txn.Set([]byte("a"), []byte("v1"))
txn.Set([]byte("b"), []byte("v2"))
s.mustPointGet(c, []byte("a"), []byte("a"))
s.mustPointGet(c, []byte("b"), []byte("b"))
// PointGet cannot ignore async commit transactions' locks.
c.Assert(failpoint.Enable("tikvclient/asyncCommitDoNothing", "return"), IsNil)
ctx := context.WithValue(context.Background(), util.SessionID, uint64(1))
err := txn.Commit(ctx)
c.Assert(err, IsNil)
c.Assert(txn.GetCommitter().IsAsyncCommit(), IsTrue)
s.mustPointGet(c, []byte("a"), []byte("v1"))
s.mustPointGet(c, []byte("b"), []byte("v2"))
c.Assert(failpoint.Disable("tikvclient/asyncCommitDoNothing"), IsNil)
// PointGet will not push the `max_ts` to its ts which is MaxUint64.
txn2 := s.beginAsyncCommit(c)
s.mustGetFromTxn(c, txn2, []byte("a"), []byte("v1"))
s.mustGetFromTxn(c, txn2, []byte("b"), []byte("v2"))
err = txn2.Rollback()
c.Assert(err, IsNil)
}
func (s *testAsyncCommitFailSuite) TestSecondaryListInPrimaryLock(c *C) {
// This test doesn't support tikv mode.
if *mockstore.WithTiKV {
return
}
s.putAlphabets(c, true)
// Split into several regions.
for _, splitKey := range []string{"h", "o", "u"} {
bo := tikv.NewBackofferWithVars(context.Background(), 5000, nil)
loc, err := s.store.GetRegionCache().LocateKey(bo, []byte(splitKey))
c.Assert(err, IsNil)
newRegionID := s.cluster.AllocID()
newPeerID := s.cluster.AllocID()
s.cluster.Split(loc.Region.GetID(), newRegionID, []byte(splitKey), []uint64{newPeerID}, newPeerID)
s.store.GetRegionCache().InvalidateCachedRegion(loc.Region)
}
// Ensure the region has been split
bo := tikv.NewBackofferWithVars(context.Background(), 5000, nil)
loc, err := s.store.GetRegionCache().LocateKey(bo, []byte("i"))
c.Assert(err, IsNil)
c.Assert(loc.StartKey, BytesEquals, []byte("h"))
c.Assert(loc.EndKey, BytesEquals, []byte("o"))
loc, err = s.store.GetRegionCache().LocateKey(bo, []byte("p"))
c.Assert(err, IsNil)
c.Assert(loc.StartKey, BytesEquals, []byte("o"))
c.Assert(loc.EndKey, BytesEquals, []byte("u"))
var sessionID uint64 = 0
test := func(keys []string, values []string) {
sessionID++
ctx := context.WithValue(context.Background(), util.SessionID, sessionID)
txn := s.beginAsyncCommit(c)
for i := range keys {
txn.Set([]byte(keys[i]), []byte(values[i]))
}
c.Assert(failpoint.Enable("tikvclient/asyncCommitDoNothing", "return"), IsNil)
err = txn.Commit(ctx)
c.Assert(err, IsNil)
primary := txn.GetCommitter().GetPrimaryKey()
bo := tikv.NewBackofferWithVars(context.Background(), 5000, nil)
lockResolver := tikv.LockResolverProbe{LockResolver: s.store.GetLockResolver()}
txnStatus, err := lockResolver.GetTxnStatus(bo, txn.StartTS(), primary, 0, 0, false, false, nil)
c.Assert(err, IsNil)
c.Assert(txnStatus.IsCommitted(), IsFalse)
c.Assert(txnStatus.Action(), Equals, kvrpcpb.Action_NoAction)
// Currently when the transaction has no secondary, the `secondaries` field of the txnStatus
// will be set nil. So here initialize the `expectedSecondaries` to nil too.
var expectedSecondaries [][]byte
for _, k := range keys {
if !bytes.Equal([]byte(k), primary) {
expectedSecondaries = append(expectedSecondaries, []byte(k))
}
}
sort.Slice(expectedSecondaries, func(i, j int) bool {
return bytes.Compare(expectedSecondaries[i], expectedSecondaries[j]) < 0
})
gotSecondaries := lockResolver.GetSecondariesFromTxnStatus(txnStatus)
sort.Slice(gotSecondaries, func(i, j int) bool {
return bytes.Compare(gotSecondaries[i], gotSecondaries[j]) < 0
})
c.Assert(gotSecondaries, DeepEquals, expectedSecondaries)
c.Assert(failpoint.Disable("tikvclient/asyncCommitDoNothing"), IsNil)
txn.GetCommitter().Cleanup(context.Background())
}
test([]string{"a"}, []string{"a1"})
test([]string{"a", "b"}, []string{"a2", "b2"})
test([]string{"a", "b", "d"}, []string{"a3", "b3", "d3"})
test([]string{"a", "b", "h", "i", "u"}, []string{"a4", "b4", "h4", "i4", "u4"})
test([]string{"i", "a", "z", "u", "b"}, []string{"i5", "a5", "z5", "u5", "b5"})
}
func (s *testAsyncCommitFailSuite) TestAsyncCommitContextCancelCausingUndetermined(c *C) {
// For an async commit transaction, if RPC returns context.Canceled error when prewriting, the
// transaction should go to undetermined state.
txn := s.beginAsyncCommit(c)
err := txn.Set([]byte("a"), []byte("va"))
c.Assert(err, IsNil)
c.Assert(failpoint.Enable("tikvclient/rpcContextCancelErr", `return(true)`), IsNil)
defer func() {
c.Assert(failpoint.Disable("tikvclient/rpcContextCancelErr"), IsNil)
}()
ctx := context.WithValue(context.Background(), util.SessionID, uint64(1))
err = txn.Commit(ctx)
c.Assert(err, NotNil)
c.Assert(txn.GetCommitter().GetUndeterminedErr(), NotNil)
}
// TestAsyncCommitRPCErrorThenWriteConflict verifies that the determined failure error overwrites undetermined error.
func (s *testAsyncCommitFailSuite) TestAsyncCommitRPCErrorThenWriteConflict(c *C) {
// This test doesn't support tikv mode because it needs setting failpoint in unistore.
if *mockstore.WithTiKV {
return
}
txn := s.beginAsyncCommit(c)
err := txn.Set([]byte("a"), []byte("va"))
c.Assert(err, IsNil)
c.Assert(failpoint.Enable("tikvclient/rpcPrewriteResult", `1*return("timeout")->return("writeConflict")`), IsNil)
defer func() {
c.Assert(failpoint.Disable("tikvclient/rpcPrewriteResult"), IsNil)
}()
ctx := context.WithValue(context.Background(), util.SessionID, uint64(1))
err = txn.Commit(ctx)
c.Assert(err, NotNil)
c.Assert(txn.GetCommitter().GetUndeterminedErr(), IsNil)
}
// TestAsyncCommitRPCErrorThenWriteConflictInChild verifies that the determined failure error in a child recursion
// overwrites the undetermined error in the parent.
func (s *testAsyncCommitFailSuite) TestAsyncCommitRPCErrorThenWriteConflictInChild(c *C) {
// This test doesn't support tikv mode because it needs setting failpoint in unistore.
if *mockstore.WithTiKV {
return
}
txn := s.beginAsyncCommit(c)
err := txn.Set([]byte("a"), []byte("va"))
c.Assert(err, IsNil)
c.Assert(failpoint.Enable("tikvclient/rpcPrewriteResult", `1*return("timeout")->return("writeConflict")`), IsNil)
c.Assert(failpoint.Enable("tikvclient/forceRecursion", `return`), IsNil)
defer func() {
c.Assert(failpoint.Disable("tikvclient/rpcPrewriteResult"), IsNil)
c.Assert(failpoint.Disable("tikvclient/forceRecursion"), IsNil)
}()
ctx := context.WithValue(context.Background(), util.SessionID, uint64(1))
err = txn.Commit(ctx)
c.Assert(err, NotNil)
c.Assert(txn.GetCommitter().GetUndeterminedErr(), IsNil)
}

View File

@ -0,0 +1,542 @@
// Copyright 2020 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_test
import (
"bytes"
"context"
"fmt"
"math"
"sync/atomic"
"testing"
"time"
. "github.com/pingcap/check"
"github.com/pingcap/errors"
"github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pingcap/tidb/store/mockstore/unistore"
tikverr "github.com/tikv/client-go/v2/error"
"github.com/tikv/client-go/v2/mockstore"
"github.com/tikv/client-go/v2/mockstore/cluster"
"github.com/tikv/client-go/v2/oracle"
"github.com/tikv/client-go/v2/tikv"
"github.com/tikv/client-go/v2/tikvrpc"
"github.com/tikv/client-go/v2/util"
)
func TestT(t *testing.T) {
CustomVerboseFlag = true
TestingT(t)
}
// testAsyncCommitCommon is used to put common parts that will be both used by
// testAsyncCommitSuite and testAsyncCommitFailSuite.
type testAsyncCommitCommon struct {
cluster cluster.Cluster
store *tikv.KVStore
}
func (s *testAsyncCommitCommon) setUpTest(c *C) {
if *mockstore.WithTiKV {
s.store = NewTestStore(c)
return
}
client, pdClient, cluster, err := unistore.New("")
c.Assert(err, IsNil)
unistore.BootstrapWithSingleStore(cluster)
s.cluster = cluster
store, err := tikv.NewTestTiKVStore(fpClient{Client: client}, pdClient, nil, nil, 0)
c.Assert(err, IsNil)
s.store = store
}
func (s *testAsyncCommitCommon) putAlphabets(c *C, enableAsyncCommit bool) {
for ch := byte('a'); ch <= byte('z'); ch++ {
s.putKV(c, []byte{ch}, []byte{ch}, enableAsyncCommit)
}
}
func (s *testAsyncCommitCommon) putKV(c *C, key, value []byte, enableAsyncCommit bool) (uint64, uint64) {
txn := s.beginAsyncCommit(c)
err := txn.Set(key, value)
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
return txn.StartTS(), txn.GetCommitTS()
}
func (s *testAsyncCommitCommon) mustGetFromTxn(c *C, txn tikv.TxnProbe, key, expectedValue []byte) {
v, err := txn.Get(context.Background(), key)
c.Assert(err, IsNil)
c.Assert(v, BytesEquals, expectedValue)
}
func (s *testAsyncCommitCommon) mustGetLock(c *C, key []byte) *tikv.Lock {
ver, err := s.store.CurrentTimestamp(oracle.GlobalTxnScope)
c.Assert(err, IsNil)
req := tikvrpc.NewRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{
Key: key,
Version: ver,
})
bo := tikv.NewBackofferWithVars(context.Background(), 5000, nil)
loc, err := s.store.GetRegionCache().LocateKey(bo, key)
c.Assert(err, IsNil)
resp, err := s.store.SendReq(bo, req, loc.Region, time.Second*10)
c.Assert(err, IsNil)
c.Assert(resp.Resp, NotNil)
keyErr := resp.Resp.(*kvrpcpb.GetResponse).GetError()
c.Assert(keyErr, NotNil)
var lockutil tikv.LockProbe
lock, err := lockutil.ExtractLockFromKeyErr(keyErr)
c.Assert(err, IsNil)
return lock
}
func (s *testAsyncCommitCommon) mustPointGet(c *C, key, expectedValue []byte) {
snap := s.store.GetSnapshot(math.MaxUint64)
value, err := snap.Get(context.Background(), key)
c.Assert(err, IsNil)
c.Assert(value, BytesEquals, expectedValue)
}
func (s *testAsyncCommitCommon) mustGetFromSnapshot(c *C, version uint64, key, expectedValue []byte) {
snap := s.store.GetSnapshot(version)
value, err := snap.Get(context.Background(), key)
c.Assert(err, IsNil)
c.Assert(value, BytesEquals, expectedValue)
}
func (s *testAsyncCommitCommon) mustGetNoneFromSnapshot(c *C, version uint64, key []byte) {
snap := s.store.GetSnapshot(version)
_, err := snap.Get(context.Background(), key)
c.Assert(errors.Cause(err), Equals, tikverr.ErrNotExist)
}
func (s *testAsyncCommitCommon) beginAsyncCommitWithLinearizability(c *C) tikv.TxnProbe {
txn := s.beginAsyncCommit(c)
txn.SetCausalConsistency(false)
return txn
}
func (s *testAsyncCommitCommon) beginAsyncCommit(c *C) tikv.TxnProbe {
txn, err := s.store.Begin()
c.Assert(err, IsNil)
txn.SetEnableAsyncCommit(true)
return tikv.TxnProbe{KVTxn: txn}
}
func (s *testAsyncCommitCommon) begin(c *C) tikv.TxnProbe {
txn, err := s.store.Begin()
c.Assert(err, IsNil)
return tikv.TxnProbe{KVTxn: txn}
}
type testAsyncCommitSuite struct {
OneByOneSuite
testAsyncCommitCommon
bo *tikv.Backoffer
}
var _ = SerialSuites(&testAsyncCommitSuite{})
func (s *testAsyncCommitSuite) SetUpTest(c *C) {
s.testAsyncCommitCommon.setUpTest(c)
s.bo = tikv.NewBackofferWithVars(context.Background(), 5000, nil)
}
func (s *testAsyncCommitSuite) lockKeysWithAsyncCommit(c *C, keys, values [][]byte, primaryKey, primaryValue []byte, commitPrimary bool) (uint64, uint64) {
txn, err := s.store.Begin()
c.Assert(err, IsNil)
txn.SetEnableAsyncCommit(true)
for i, k := range keys {
if len(values[i]) > 0 {
err = txn.Set(k, values[i])
} else {
err = txn.Delete(k)
}
c.Assert(err, IsNil)
}
if len(primaryValue) > 0 {
err = txn.Set(primaryKey, primaryValue)
} else {
err = txn.Delete(primaryKey)
}
c.Assert(err, IsNil)
txnProbe := tikv.TxnProbe{KVTxn: txn}
tpc, err := txnProbe.NewCommitter(0)
c.Assert(err, IsNil)
tpc.SetPrimaryKey(primaryKey)
ctx := context.Background()
err = tpc.PrewriteAllMutations(ctx)
c.Assert(err, IsNil)
if commitPrimary {
commitTS, err := s.store.GetOracle().GetTimestamp(ctx, &oracle.Option{TxnScope: oracle.GlobalTxnScope})
c.Assert(err, IsNil)
tpc.SetCommitTS(commitTS)
err = tpc.CommitMutations(ctx)
c.Assert(err, IsNil)
}
return txn.StartTS(), tpc.GetCommitTS()
}
func (s *testAsyncCommitSuite) TestCheckSecondaries(c *C) {
// This test doesn't support tikv mode.
if *mockstore.WithTiKV {
return
}
s.putAlphabets(c, true)
loc, err := s.store.GetRegionCache().LocateKey(s.bo, []byte("a"))
c.Assert(err, IsNil)
newRegionID, peerID := s.cluster.AllocID(), s.cluster.AllocID()
s.cluster.Split(loc.Region.GetID(), newRegionID, []byte("e"), []uint64{peerID}, peerID)
s.store.GetRegionCache().InvalidateCachedRegion(loc.Region)
// No locks to check, only primary key is locked, should be successful.
s.lockKeysWithAsyncCommit(c, [][]byte{}, [][]byte{}, []byte("z"), []byte("z"), false)
lock := s.mustGetLock(c, []byte("z"))
lock.UseAsyncCommit = true
ts, err := s.store.GetOracle().GetTimestamp(context.Background(), &oracle.Option{TxnScope: oracle.GlobalTxnScope})
c.Assert(err, IsNil)
var lockutil tikv.LockProbe
status := lockutil.NewLockStatus(nil, true, ts)
resolver := tikv.LockResolverProbe{LockResolver: s.store.GetLockResolver()}
err = resolver.ResolveLockAsync(s.bo, lock, status)
c.Assert(err, IsNil)
currentTS, err := s.store.GetOracle().GetTimestamp(context.Background(), &oracle.Option{TxnScope: oracle.GlobalTxnScope})
c.Assert(err, IsNil)
status, err = resolver.GetTxnStatus(s.bo, lock.TxnID, []byte("z"), currentTS, currentTS, true, false, nil)
c.Assert(err, IsNil)
c.Assert(status.IsCommitted(), IsTrue)
c.Assert(status.CommitTS(), Equals, ts)
// One key is committed (i), one key is locked (a). Should get committed.
ts, err = s.store.GetOracle().GetTimestamp(context.Background(), &oracle.Option{TxnScope: oracle.GlobalTxnScope})
c.Assert(err, IsNil)
commitTs := ts + 10
gotCheckA := int64(0)
gotCheckB := int64(0)
gotResolve := int64(0)
gotOther := int64(0)
mock := mockResolveClient{
inner: s.store.GetTiKVClient(),
onCheckSecondaries: func(req *kvrpcpb.CheckSecondaryLocksRequest) (*tikvrpc.Response, error) {
if req.StartVersion != ts {
return nil, errors.Errorf("Bad start version: %d, expected: %d", req.StartVersion, ts)
}
var resp kvrpcpb.CheckSecondaryLocksResponse
for _, k := range req.Keys {
if bytes.Equal(k, []byte("a")) {
atomic.StoreInt64(&gotCheckA, 1)
resp = kvrpcpb.CheckSecondaryLocksResponse{
Locks: []*kvrpcpb.LockInfo{{Key: []byte("a"), PrimaryLock: []byte("z"), LockVersion: ts, UseAsyncCommit: true}},
CommitTs: commitTs,
}
} else if bytes.Equal(k, []byte("i")) {
atomic.StoreInt64(&gotCheckB, 1)
resp = kvrpcpb.CheckSecondaryLocksResponse{
Locks: []*kvrpcpb.LockInfo{},
CommitTs: commitTs,
}
} else {
fmt.Printf("Got other key: %s\n", k)
atomic.StoreInt64(&gotOther, 1)
}
}
return &tikvrpc.Response{Resp: &resp}, nil
},
onResolveLock: func(req *kvrpcpb.ResolveLockRequest) (*tikvrpc.Response, error) {
if req.StartVersion != ts {
return nil, errors.Errorf("Bad start version: %d, expected: %d", req.StartVersion, ts)
}
if req.CommitVersion != commitTs {
return nil, errors.Errorf("Bad commit version: %d, expected: %d", req.CommitVersion, commitTs)
}
for _, k := range req.Keys {
if bytes.Equal(k, []byte("a")) || bytes.Equal(k, []byte("z")) {
atomic.StoreInt64(&gotResolve, 1)
} else {
atomic.StoreInt64(&gotOther, 1)
}
}
resp := kvrpcpb.ResolveLockResponse{}
return &tikvrpc.Response{Resp: &resp}, nil
},
}
s.store.SetTiKVClient(&mock)
status = lockutil.NewLockStatus([][]byte{[]byte("a"), []byte("i")}, true, 0)
lock = &tikv.Lock{
Key: []byte("a"),
Primary: []byte("z"),
TxnID: ts,
LockType: kvrpcpb.Op_Put,
UseAsyncCommit: true,
MinCommitTS: ts + 5,
}
_ = s.beginAsyncCommit(c)
err = resolver.ResolveLockAsync(s.bo, lock, status)
c.Assert(err, IsNil)
c.Assert(gotCheckA, Equals, int64(1))
c.Assert(gotCheckB, Equals, int64(1))
c.Assert(gotOther, Equals, int64(0))
c.Assert(gotResolve, Equals, int64(1))
// One key has been rolled back (b), one is locked (a). Should be rolled back.
ts, err = s.store.GetOracle().GetTimestamp(context.Background(), &oracle.Option{TxnScope: oracle.GlobalTxnScope})
c.Assert(err, IsNil)
commitTs = ts + 10
gotCheckA = int64(0)
gotCheckB = int64(0)
gotResolve = int64(0)
gotOther = int64(0)
mock.onResolveLock = func(req *kvrpcpb.ResolveLockRequest) (*tikvrpc.Response, error) {
if req.StartVersion != ts {
return nil, errors.Errorf("Bad start version: %d, expected: %d", req.StartVersion, ts)
}
if req.CommitVersion != commitTs {
return nil, errors.Errorf("Bad commit version: %d, expected: 0", req.CommitVersion)
}
for _, k := range req.Keys {
if bytes.Equal(k, []byte("a")) || bytes.Equal(k, []byte("z")) {
atomic.StoreInt64(&gotResolve, 1)
} else {
atomic.StoreInt64(&gotOther, 1)
}
}
resp := kvrpcpb.ResolveLockResponse{}
return &tikvrpc.Response{Resp: &resp}, nil
}
lock.TxnID = ts
lock.MinCommitTS = ts + 5
err = resolver.ResolveLockAsync(s.bo, lock, status)
c.Assert(err, IsNil)
c.Assert(gotCheckA, Equals, int64(1))
c.Assert(gotCheckB, Equals, int64(1))
c.Assert(gotResolve, Equals, int64(1))
c.Assert(gotOther, Equals, int64(0))
}
func (s *testAsyncCommitSuite) TestRepeatableRead(c *C) {
var sessionID uint64 = 0
test := func(isPessimistic bool) {
s.putKV(c, []byte("k1"), []byte("v1"), true)
sessionID++
ctx := context.WithValue(context.Background(), util.SessionID, sessionID)
txn1 := s.beginAsyncCommit(c)
txn1.SetPessimistic(isPessimistic)
s.mustGetFromTxn(c, txn1, []byte("k1"), []byte("v1"))
txn1.Set([]byte("k1"), []byte("v2"))
for i := 0; i < 20; i++ {
_, err := s.store.GetOracle().GetTimestamp(ctx, &oracle.Option{TxnScope: oracle.GlobalTxnScope})
c.Assert(err, IsNil)
}
txn2 := s.beginAsyncCommit(c)
s.mustGetFromTxn(c, txn2, []byte("k1"), []byte("v1"))
err := txn1.Commit(ctx)
c.Assert(err, IsNil)
// Check txn1 is committed in async commit.
c.Assert(txn1.IsAsyncCommit(), IsTrue)
s.mustGetFromTxn(c, txn2, []byte("k1"), []byte("v1"))
err = txn2.Rollback()
c.Assert(err, IsNil)
txn3 := s.beginAsyncCommit(c)
s.mustGetFromTxn(c, txn3, []byte("k1"), []byte("v2"))
err = txn3.Rollback()
c.Assert(err, IsNil)
}
test(false)
test(true)
}
// It's just a simple validation of linearizability.
// Extra tests are needed to test this feature with the control of the TiKV cluster.
func (s *testAsyncCommitSuite) TestAsyncCommitLinearizability(c *C) {
t1 := s.beginAsyncCommitWithLinearizability(c)
t2 := s.beginAsyncCommitWithLinearizability(c)
err := t1.Set([]byte("a"), []byte("a1"))
c.Assert(err, IsNil)
err = t2.Set([]byte("b"), []byte("b1"))
c.Assert(err, IsNil)
ctx := context.WithValue(context.Background(), util.SessionID, uint64(1))
// t2 commits earlier than t1
err = t2.Commit(ctx)
c.Assert(err, IsNil)
err = t1.Commit(ctx)
c.Assert(err, IsNil)
commitTS1 := t1.GetCommitTS()
commitTS2 := t2.GetCommitTS()
c.Assert(commitTS2, Less, commitTS1)
}
// TestAsyncCommitWithMultiDC tests that async commit can only be enabled in global transactions
func (s *testAsyncCommitSuite) TestAsyncCommitWithMultiDC(c *C) {
// It requires setting placement rules to run with TiKV
if *mockstore.WithTiKV {
return
}
localTxn := s.beginAsyncCommit(c)
err := localTxn.Set([]byte("a"), []byte("a1"))
localTxn.SetScope("bj")
c.Assert(err, IsNil)
ctx := context.WithValue(context.Background(), util.SessionID, uint64(1))
err = localTxn.Commit(ctx)
c.Assert(err, IsNil)
c.Assert(localTxn.IsAsyncCommit(), IsFalse)
globalTxn := s.beginAsyncCommit(c)
err = globalTxn.Set([]byte("b"), []byte("b1"))
globalTxn.SetScope(oracle.GlobalTxnScope)
c.Assert(err, IsNil)
err = globalTxn.Commit(ctx)
c.Assert(err, IsNil)
c.Assert(globalTxn.IsAsyncCommit(), IsTrue)
}
func (s *testAsyncCommitSuite) TestResolveTxnFallbackFromAsyncCommit(c *C) {
keys := [][]byte{[]byte("k0"), []byte("k1")}
values := [][]byte{[]byte("v00"), []byte("v10")}
initTest := func() tikv.CommitterProbe {
t0 := s.begin(c)
err := t0.Set(keys[0], values[0])
c.Assert(err, IsNil)
err = t0.Set(keys[1], values[1])
c.Assert(err, IsNil)
err = t0.Commit(context.Background())
c.Assert(err, IsNil)
t1 := s.beginAsyncCommit(c)
err = t1.Set(keys[0], []byte("v01"))
c.Assert(err, IsNil)
err = t1.Set(keys[1], []byte("v11"))
c.Assert(err, IsNil)
committer, err := t1.NewCommitter(1)
c.Assert(err, IsNil)
committer.SetLockTTL(1)
committer.SetUseAsyncCommit()
return committer
}
prewriteKey := func(committer tikv.CommitterProbe, idx int, fallback bool) {
bo := tikv.NewBackofferWithVars(context.Background(), 5000, nil)
loc, err := s.store.GetRegionCache().LocateKey(bo, keys[idx])
c.Assert(err, IsNil)
req := committer.BuildPrewriteRequest(loc.Region.GetID(), loc.Region.GetConfVer(), loc.Region.GetVer(),
committer.GetMutations().Slice(idx, idx+1), 1)
if fallback {
req.Req.(*kvrpcpb.PrewriteRequest).MaxCommitTs = 1
}
resp, err := s.store.SendReq(bo, req, loc.Region, 5000)
c.Assert(err, IsNil)
c.Assert(resp.Resp, NotNil)
}
readKey := func(idx int) {
t2 := s.begin(c)
ctx, cancel := context.WithTimeout(context.Background(), time.Second)
defer cancel()
val, err := t2.Get(ctx, keys[idx])
c.Assert(err, IsNil)
c.Assert(val, DeepEquals, values[idx])
}
// Case 1: Fallback primary, read primary
committer := initTest()
prewriteKey(committer, 0, true)
prewriteKey(committer, 1, false)
readKey(0)
readKey(1)
// Case 2: Fallback primary, read secondary
committer = initTest()
prewriteKey(committer, 0, true)
prewriteKey(committer, 1, false)
readKey(1)
readKey(0)
// Case 3: Fallback secondary, read primary
committer = initTest()
prewriteKey(committer, 0, false)
prewriteKey(committer, 1, true)
readKey(0)
readKey(1)
// Case 4: Fallback secondary, read secondary
committer = initTest()
prewriteKey(committer, 0, false)
prewriteKey(committer, 1, true)
readKey(1)
readKey(0)
// Case 5: Fallback both, read primary
committer = initTest()
prewriteKey(committer, 0, true)
prewriteKey(committer, 1, true)
readKey(0)
readKey(1)
// Case 6: Fallback both, read secondary
committer = initTest()
prewriteKey(committer, 0, true)
prewriteKey(committer, 1, true)
readKey(1)
readKey(0)
}
type mockResolveClient struct {
inner tikv.Client
onResolveLock func(*kvrpcpb.ResolveLockRequest) (*tikvrpc.Response, error)
onCheckSecondaries func(*kvrpcpb.CheckSecondaryLocksRequest) (*tikvrpc.Response, error)
}
func (m *mockResolveClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) {
// Intercept check secondary locks and resolve lock messages if the callback is non-nil.
// If the callback returns (nil, nil), forward to the inner client.
if cr, ok := req.Req.(*kvrpcpb.CheckSecondaryLocksRequest); ok && m.onCheckSecondaries != nil {
result, err := m.onCheckSecondaries(cr)
if result != nil || err != nil {
return result, err
}
} else if rr, ok := req.Req.(*kvrpcpb.ResolveLockRequest); ok && m.onResolveLock != nil {
result, err := m.onResolveLock(rr)
if result != nil || err != nil {
return result, err
}
}
return m.inner.SendRequest(ctx, addr, req, timeout)
}
func (m *mockResolveClient) Close() error {
return m.inner.Close()
}

View File

@ -0,0 +1,97 @@
// Copyright 2021 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_test
import (
"context"
"time"
"github.com/pingcap/errors"
"github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pingcap/parser/terror"
"github.com/tikv/client-go/v2/client"
"github.com/tikv/client-go/v2/tikvrpc"
"github.com/tikv/client-go/v2/util"
)
// mock TiKV RPC client that hooks message by failpoint
type fpClient struct {
client.Client
}
func (c fpClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) {
switch req.Type {
case tikvrpc.CmdPrewrite:
if val, err := util.EvalFailpoint("rpcPrewriteResult"); err == nil && val != nil {
switch val.(string) {
case "timeout":
return nil, errors.New("timeout")
case "writeConflict":
return &tikvrpc.Response{
Resp: &kvrpcpb.PrewriteResponse{Errors: []*kvrpcpb.KeyError{{Conflict: &kvrpcpb.WriteConflict{}}}},
}, nil
}
}
case tikvrpc.CmdBatchGet:
batchGetReq := req.BatchGet()
if val, err := util.EvalFailpoint("rpcBatchGetResult"); err == nil {
switch val.(string) {
case "keyError":
return &tikvrpc.Response{
Resp: &kvrpcpb.BatchGetResponse{Error: &kvrpcpb.KeyError{
Locked: &kvrpcpb.LockInfo{
PrimaryLock: batchGetReq.Keys[0],
LockVersion: batchGetReq.Version - 1,
Key: batchGetReq.Keys[0],
LockTtl: 50,
TxnSize: 1,
LockType: kvrpcpb.Op_Put,
},
}},
}, nil
}
}
case tikvrpc.CmdScan:
kvScanReq := req.Scan()
if val, err := util.EvalFailpoint("rpcScanResult"); err == nil {
switch val.(string) {
case "keyError":
return &tikvrpc.Response{
Resp: &kvrpcpb.ScanResponse{Error: &kvrpcpb.KeyError{
Locked: &kvrpcpb.LockInfo{
PrimaryLock: kvScanReq.StartKey,
LockVersion: kvScanReq.Version - 1,
Key: kvScanReq.StartKey,
LockTtl: 50,
TxnSize: 1,
LockType: kvrpcpb.Op_Put,
},
}},
}, nil
}
}
}
res, err := c.Client.SendRequest(ctx, addr, req, timeout)
switch req.Type {
case tikvrpc.CmdPrewrite:
if val, err := util.EvalFailpoint("rpcPrewriteTimeout"); err == nil {
if val.(bool) {
return nil, terror.ErrResultUndetermined
}
}
}
return res, err
}

View File

@ -0,0 +1,153 @@
// Copyright 2018 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_test
import (
"bytes"
"context"
"math/rand"
"sort"
. "github.com/pingcap/check"
"github.com/pingcap/tidb/store/mockstore/mockcopr"
"github.com/tikv/client-go/v2/mockstore/cluster"
"github.com/tikv/client-go/v2/mockstore/mocktikv"
"github.com/tikv/client-go/v2/tikv"
)
type testDeleteRangeSuite struct {
OneByOneSuite
cluster cluster.Cluster
store *tikv.KVStore
}
var _ = Suite(&testDeleteRangeSuite{})
func (s *testDeleteRangeSuite) SetUpTest(c *C) {
client, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("", mockcopr.NewCoprRPCHandler())
c.Assert(err, IsNil)
mocktikv.BootstrapWithMultiRegions(cluster, []byte("b"), []byte("c"), []byte("d"))
s.cluster = cluster
store, err := tikv.NewTestTiKVStore(client, pdClient, nil, nil, 0)
c.Check(err, IsNil)
// TODO: make this possible
// store, err := mockstore.NewMockStore(
// mockstore.WithStoreType(mockstore.MockTiKV),
// mockstore.WithClusterInspector(func(c cluster.Cluster) {
// mockstore.BootstrapWithMultiRegions(c, []byte("b"), []byte("c"), []byte("d"))
// s.cluster = c
// }),
// )
// c.Assert(err, IsNil)
s.store = store
}
func (s *testDeleteRangeSuite) TearDownTest(c *C) {
err := s.store.Close()
c.Assert(err, IsNil)
}
func (s *testDeleteRangeSuite) checkData(c *C, expectedData map[string]string) {
txn, err := s.store.Begin()
c.Assert(err, IsNil)
it, err := txn.Iter([]byte("a"), nil)
c.Assert(err, IsNil)
// Scan all data and save into a map
data := map[string]string{}
for it.Valid() {
data[string(it.Key())] = string(it.Value())
err = it.Next()
c.Assert(err, IsNil)
}
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
// Print log
actualKeys := make([]string, 0, len(data))
expectedKeys := make([]string, 0, len(expectedData))
for key := range data {
actualKeys = append(actualKeys, key)
}
for key := range expectedData {
expectedKeys = append(expectedKeys, key)
}
sort.Strings(actualKeys)
sort.Strings(expectedKeys)
c.Log("Actual: ", actualKeys)
c.Log("Expected: ", expectedKeys)
// Assert data in the store is the same as expected
c.Assert(data, DeepEquals, expectedData)
}
func (s *testDeleteRangeSuite) deleteRange(c *C, startKey []byte, endKey []byte) int {
task := tikv.NewDeleteRangeTask(s.store, startKey, endKey, 1)
err := task.Execute(context.Background())
c.Assert(err, IsNil)
return task.CompletedRegions()
}
// deleteRangeFromMap deletes all keys in a given range from a map
func deleteRangeFromMap(m map[string]string, startKey []byte, endKey []byte) {
for keyStr := range m {
key := []byte(keyStr)
if bytes.Compare(startKey, key) <= 0 && bytes.Compare(key, endKey) < 0 {
delete(m, keyStr)
}
}
}
// mustDeleteRange does delete range on both the map and the storage, and assert they are equal after deleting
func (s *testDeleteRangeSuite) mustDeleteRange(c *C, startKey []byte, endKey []byte, expected map[string]string, regions int) {
completedRegions := s.deleteRange(c, startKey, endKey)
deleteRangeFromMap(expected, startKey, endKey)
s.checkData(c, expected)
c.Assert(completedRegions, Equals, regions)
}
func (s *testDeleteRangeSuite) TestDeleteRange(c *C) {
// Write some key-value pairs
txn, err := s.store.Begin()
c.Assert(err, IsNil)
testData := map[string]string{}
// Generate a sequence of keys and random values
for _, i := range []byte("abcd") {
for j := byte('0'); j <= byte('9'); j++ {
key := []byte{i, j}
value := []byte{byte(rand.Intn(256)), byte(rand.Intn(256))}
testData[string(key)] = string(value)
err := txn.Set(key, value)
c.Assert(err, IsNil)
}
}
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
s.checkData(c, testData)
s.mustDeleteRange(c, []byte("b"), []byte("c0"), testData, 2)
s.mustDeleteRange(c, []byte("c11"), []byte("c12"), testData, 1)
s.mustDeleteRange(c, []byte("d0"), []byte("d0"), testData, 0)
s.mustDeleteRange(c, []byte("d0\x00"), []byte("d1\x00"), testData, 1)
s.mustDeleteRange(c, []byte("c5"), []byte("d5"), testData, 2)
s.mustDeleteRange(c, []byte("a"), []byte("z"), testData, 4)
}

17
integration_tests/go.mod Normal file
View File

@ -0,0 +1,17 @@
module integration_tests
go 1.16
require (
github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712
github.com/pingcap/errors v0.11.5-0.20201126102027-b0a155152ca3
github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd
github.com/pingcap/kvproto v0.0.0-20210611081648-a215b4e61d2f
github.com/pingcap/parser v0.0.0-20210610080504-cb77169bfed9
github.com/pingcap/tidb v1.1.0-beta.0.20210616023036-9461f5ba55b1
github.com/tikv/client-go/v2 v2.0.0
github.com/tikv/pd v1.1.0-beta.0.20210323121136-78679e5e209d
go.uber.org/zap v1.17.0
)
replace github.com/tikv/client-go/v2 => ../

View File

@ -0,0 +1,198 @@
// 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.
// +build !race
package tikv_test
import (
"context"
"fmt"
"sort"
"sync"
"time"
. "github.com/pingcap/check"
"github.com/pingcap/tidb/kv"
"github.com/tikv/client-go/v2/tikv"
)
// testIsolationSuite represents test isolation suite.
// The test suite takes too long under the race detector.
type testIsolationSuite struct {
OneByOneSuite
store *tikv.KVStore
}
var _ = Suite(&testIsolationSuite{})
func (s *testIsolationSuite) SetUpSuite(c *C) {
s.OneByOneSuite.SetUpSuite(c)
s.store = NewTestStore(c)
}
func (s *testIsolationSuite) TearDownSuite(c *C) {
s.store.Close()
s.OneByOneSuite.TearDownSuite(c)
}
type writeRecord struct {
startTS uint64
commitTS uint64
}
type writeRecords []writeRecord
func (r writeRecords) Len() int { return len(r) }
func (r writeRecords) Swap(i, j int) { r[i], r[j] = r[j], r[i] }
func (r writeRecords) Less(i, j int) bool { return r[i].startTS <= r[j].startTS }
func (s *testIsolationSuite) SetWithRetry(c *C, k, v []byte) writeRecord {
for {
txnRaw, err := s.store.Begin()
c.Assert(err, IsNil)
txn := tikv.TxnProbe{KVTxn: txnRaw}
err = txn.Set(k, v)
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
if err == nil {
return writeRecord{
startTS: txn.StartTS(),
commitTS: txn.GetCommitTS(),
}
}
}
}
type readRecord struct {
startTS uint64
value []byte
}
type readRecords []readRecord
func (r readRecords) Len() int { return len(r) }
func (r readRecords) Swap(i, j int) { r[i], r[j] = r[j], r[i] }
func (r readRecords) Less(i, j int) bool { return r[i].startTS <= r[j].startTS }
func (s *testIsolationSuite) GetWithRetry(c *C, k []byte) readRecord {
for {
txn, err := s.store.Begin()
c.Assert(err, IsNil)
val, err := txn.Get(context.TODO(), k)
if err == nil {
return readRecord{
startTS: txn.StartTS(),
value: val,
}
}
c.Assert(kv.IsTxnRetryableError(err), IsTrue)
}
}
func (s *testIsolationSuite) TestWriteWriteConflict(c *C) {
const (
threadCount = 10
setPerThread = 50
)
var (
mu sync.Mutex
writes []writeRecord
wg sync.WaitGroup
)
wg.Add(threadCount)
for i := 0; i < threadCount; i++ {
go func() {
defer wg.Done()
for j := 0; j < setPerThread; j++ {
w := s.SetWithRetry(c, []byte("k"), []byte("v"))
mu.Lock()
writes = append(writes, w)
mu.Unlock()
}
}()
}
wg.Wait()
// Check all transactions' [startTS, commitTS] are not overlapped.
sort.Sort(writeRecords(writes))
for i := 0; i < len(writes)-1; i++ {
c.Assert(writes[i].commitTS, Less, writes[i+1].startTS)
}
}
func (s *testIsolationSuite) TestReadWriteConflict(c *C) {
const (
readThreadCount = 10
writeCount = 10
)
var (
writes []writeRecord
mu sync.Mutex
reads []readRecord
wg sync.WaitGroup
)
s.SetWithRetry(c, []byte("k"), []byte("0"))
writeDone := make(chan struct{})
go func() {
for i := 1; i <= writeCount; i++ {
w := s.SetWithRetry(c, []byte("k"), []byte(fmt.Sprintf("%d", i)))
writes = append(writes, w)
time.Sleep(time.Microsecond * 10)
}
close(writeDone)
}()
wg.Add(readThreadCount)
for i := 0; i < readThreadCount; i++ {
go func() {
defer wg.Done()
for {
select {
case <-writeDone:
return
default:
}
r := s.GetWithRetry(c, []byte("k"))
mu.Lock()
reads = append(reads, r)
mu.Unlock()
}
}()
}
wg.Wait()
sort.Sort(readRecords(reads))
// Check all reads got the value committed before it's startTS.
var i, j int
for ; i < len(writes); i++ {
for ; j < len(reads); j++ {
w, r := writes[i], reads[j]
if r.startTS >= w.commitTS {
break
}
c.Assert(string(r.value), Equals, fmt.Sprintf("%d", i))
}
}
for ; j < len(reads); j++ {
c.Assert(string(reads[j].value), Equals, fmt.Sprintf("%d", len(writes)))
}
}

View File

@ -0,0 +1,774 @@
// 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_test
import (
"bytes"
"context"
"fmt"
"math"
"runtime"
"sync"
"time"
. "github.com/pingcap/check"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
deadlockpb "github.com/pingcap/kvproto/pkg/deadlock"
"github.com/pingcap/kvproto/pkg/kvrpcpb"
tikverr "github.com/tikv/client-go/v2/error"
"github.com/tikv/client-go/v2/kv"
"github.com/tikv/client-go/v2/oracle"
"github.com/tikv/client-go/v2/tikv"
"github.com/tikv/client-go/v2/tikvrpc"
)
var getMaxBackoff = tikv.ConfigProbe{}.GetGetMaxBackoff()
type testLockSuite struct {
OneByOneSuite
store tikv.StoreProbe
}
var _ = Suite(&testLockSuite{})
func (s *testLockSuite) SetUpTest(c *C) {
s.store = tikv.StoreProbe{KVStore: NewTestStore(c)}
}
func (s *testLockSuite) TearDownTest(c *C) {
s.store.Close()
}
func (s *testLockSuite) lockKey(c *C, key, value, primaryKey, primaryValue []byte, commitPrimary bool) (uint64, uint64) {
txn, err := s.store.Begin()
c.Assert(err, IsNil)
if len(value) > 0 {
err = txn.Set(key, value)
} else {
err = txn.Delete(key)
}
c.Assert(err, IsNil)
if len(primaryValue) > 0 {
err = txn.Set(primaryKey, primaryValue)
} else {
err = txn.Delete(primaryKey)
}
c.Assert(err, IsNil)
tpc, err := txn.NewCommitter(0)
c.Assert(err, IsNil)
tpc.SetPrimaryKey(primaryKey)
ctx := context.Background()
err = tpc.PrewriteAllMutations(ctx)
c.Assert(err, IsNil)
if commitPrimary {
commitTS, err := s.store.GetOracle().GetTimestamp(ctx, &oracle.Option{TxnScope: oracle.GlobalTxnScope})
c.Assert(err, IsNil)
tpc.SetCommitTS(commitTS)
err = tpc.CommitMutations(ctx)
c.Assert(err, IsNil)
}
return txn.StartTS(), tpc.GetCommitTS()
}
func (s *testLockSuite) putAlphabets(c *C) {
for ch := byte('a'); ch <= byte('z'); ch++ {
s.putKV(c, []byte{ch}, []byte{ch})
}
}
func (s *testLockSuite) putKV(c *C, key, value []byte) (uint64, uint64) {
txn, err := s.store.Begin()
c.Assert(err, IsNil)
err = txn.Set(key, value)
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
return txn.StartTS(), txn.GetCommitTS()
}
func (s *testLockSuite) prepareAlphabetLocks(c *C) {
s.putKV(c, []byte("c"), []byte("cc"))
s.lockKey(c, []byte("c"), []byte("c"), []byte("z1"), []byte("z1"), true)
s.lockKey(c, []byte("d"), []byte("dd"), []byte("z2"), []byte("z2"), false)
s.lockKey(c, []byte("foo"), []byte("foo"), []byte("z3"), []byte("z3"), false)
s.putKV(c, []byte("bar"), []byte("bar"))
s.lockKey(c, []byte("bar"), nil, []byte("z4"), []byte("z4"), true)
}
func (s *testLockSuite) TestScanLockResolveWithGet(c *C) {
s.putAlphabets(c)
s.prepareAlphabetLocks(c)
txn, err := s.store.Begin()
c.Assert(err, IsNil)
for ch := byte('a'); ch <= byte('z'); ch++ {
v, err := txn.Get(context.TODO(), []byte{ch})
c.Assert(err, IsNil)
c.Assert(v, BytesEquals, []byte{ch})
}
}
func (s *testLockSuite) TestScanLockResolveWithSeek(c *C) {
s.putAlphabets(c)
s.prepareAlphabetLocks(c)
txn, err := s.store.Begin()
c.Assert(err, IsNil)
iter, err := txn.Iter([]byte("a"), nil)
c.Assert(err, IsNil)
for ch := byte('a'); ch <= byte('z'); ch++ {
c.Assert(iter.Valid(), IsTrue)
c.Assert(iter.Key(), BytesEquals, []byte{ch})
c.Assert(iter.Value(), BytesEquals, []byte{ch})
c.Assert(iter.Next(), IsNil)
}
}
func (s *testLockSuite) TestScanLockResolveWithSeekKeyOnly(c *C) {
s.putAlphabets(c)
s.prepareAlphabetLocks(c)
txn, err := s.store.Begin()
c.Assert(err, IsNil)
txn.GetSnapshot().SetKeyOnly(true)
iter, err := txn.Iter([]byte("a"), nil)
c.Assert(err, IsNil)
for ch := byte('a'); ch <= byte('z'); ch++ {
c.Assert(iter.Valid(), IsTrue)
c.Assert(iter.Key(), BytesEquals, []byte{ch})
c.Assert(iter.Next(), IsNil)
}
}
func (s *testLockSuite) TestScanLockResolveWithBatchGet(c *C) {
s.putAlphabets(c)
s.prepareAlphabetLocks(c)
var keys [][]byte
for ch := byte('a'); ch <= byte('z'); ch++ {
keys = append(keys, []byte{ch})
}
txn, err := s.store.Begin()
c.Assert(err, IsNil)
m, err := toTiDBTxn(&txn).BatchGet(context.Background(), toTiDBKeys(keys))
c.Assert(err, IsNil)
c.Assert(len(m), Equals, int('z'-'a'+1))
for ch := byte('a'); ch <= byte('z'); ch++ {
k := []byte{ch}
c.Assert(m[string(k)], BytesEquals, k)
}
}
func (s *testLockSuite) TestCleanLock(c *C) {
for ch := byte('a'); ch <= byte('z'); ch++ {
k := []byte{ch}
s.lockKey(c, k, k, k, k, false)
}
txn, err := s.store.Begin()
c.Assert(err, IsNil)
for ch := byte('a'); ch <= byte('z'); ch++ {
err = txn.Set([]byte{ch}, []byte{ch + 1})
c.Assert(err, IsNil)
}
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
}
func (s *testLockSuite) TestGetTxnStatus(c *C) {
startTS, commitTS := s.putKV(c, []byte("a"), []byte("a"))
status, err := s.store.GetLockResolver().GetTxnStatus(startTS, startTS, []byte("a"))
c.Assert(err, IsNil)
c.Assert(status.IsCommitted(), IsTrue)
c.Assert(status.CommitTS(), Equals, commitTS)
startTS, commitTS = s.lockKey(c, []byte("a"), []byte("a"), []byte("a"), []byte("a"), true)
status, err = s.store.GetLockResolver().GetTxnStatus(startTS, startTS, []byte("a"))
c.Assert(err, IsNil)
c.Assert(status.IsCommitted(), IsTrue)
c.Assert(status.CommitTS(), Equals, commitTS)
startTS, _ = s.lockKey(c, []byte("a"), []byte("a"), []byte("a"), []byte("a"), false)
status, err = s.store.GetLockResolver().GetTxnStatus(startTS, startTS, []byte("a"))
c.Assert(err, IsNil)
c.Assert(status.IsCommitted(), IsFalse)
c.Assert(status.TTL(), Greater, uint64(0), Commentf("action:%s", status.Action()))
}
func (s *testLockSuite) TestCheckTxnStatusTTL(c *C) {
txn, err := s.store.Begin()
c.Assert(err, IsNil)
txn.Set([]byte("key"), []byte("value"))
s.prewriteTxnWithTTL(c, txn, 1000)
bo := tikv.NewBackofferWithVars(context.Background(), tikv.PrewriteMaxBackoff, nil)
lr := s.store.NewLockResolver()
callerStartTS, err := s.store.GetOracle().GetTimestamp(bo.GetCtx(), &oracle.Option{TxnScope: oracle.GlobalTxnScope})
c.Assert(err, IsNil)
// Check the lock TTL of a transaction.
status, err := lr.LockResolver.GetTxnStatus(txn.StartTS(), callerStartTS, []byte("key"))
c.Assert(err, IsNil)
c.Assert(status.IsCommitted(), IsFalse)
c.Assert(status.TTL(), Greater, uint64(0))
c.Assert(status.CommitTS(), Equals, uint64(0))
// Rollback the txn.
lock := s.mustGetLock(c, []byte("key"))
err = s.store.NewLockResolver().ResolveLock(context.Background(), lock)
c.Assert(err, IsNil)
// Check its status is rollbacked.
status, err = lr.LockResolver.GetTxnStatus(txn.StartTS(), callerStartTS, []byte("key"))
c.Assert(err, IsNil)
c.Assert(status.TTL(), Equals, uint64(0))
c.Assert(status.CommitTS(), Equals, uint64(0))
c.Assert(status.Action(), Equals, kvrpcpb.Action_NoAction)
// Check a committed txn.
startTS, commitTS := s.putKV(c, []byte("a"), []byte("a"))
status, err = lr.LockResolver.GetTxnStatus(startTS, callerStartTS, []byte("a"))
c.Assert(err, IsNil)
c.Assert(status.TTL(), Equals, uint64(0))
c.Assert(status.CommitTS(), Equals, commitTS)
}
func (s *testLockSuite) TestTxnHeartBeat(c *C) {
txn, err := s.store.Begin()
c.Assert(err, IsNil)
txn.Set([]byte("key"), []byte("value"))
s.prewriteTxn(c, txn)
newTTL, err := s.store.SendTxnHeartbeat(context.Background(), []byte("key"), txn.StartTS(), 6666)
c.Assert(err, IsNil)
c.Assert(newTTL, Equals, uint64(6666))
newTTL, err = s.store.SendTxnHeartbeat(context.Background(), []byte("key"), txn.StartTS(), 5555)
c.Assert(err, IsNil)
c.Assert(newTTL, Equals, uint64(6666))
lock := s.mustGetLock(c, []byte("key"))
err = s.store.NewLockResolver().ResolveLock(context.Background(), lock)
c.Assert(err, IsNil)
newTTL, err = s.store.SendTxnHeartbeat(context.Background(), []byte("key"), txn.StartTS(), 6666)
c.Assert(err, NotNil)
c.Assert(newTTL, Equals, uint64(0))
}
func (s *testLockSuite) TestCheckTxnStatus(c *C) {
txn, err := s.store.Begin()
c.Assert(err, IsNil)
txn.Set([]byte("key"), []byte("value"))
txn.Set([]byte("second"), []byte("xxx"))
s.prewriteTxnWithTTL(c, txn, 1000)
o := s.store.GetOracle()
currentTS, err := o.GetTimestamp(context.Background(), &oracle.Option{TxnScope: oracle.GlobalTxnScope})
c.Assert(err, IsNil)
c.Assert(currentTS, Greater, txn.StartTS())
bo := tikv.NewBackofferWithVars(context.Background(), tikv.PrewriteMaxBackoff, nil)
resolver := s.store.NewLockResolver()
// Call getTxnStatus to check the lock status.
status, err := resolver.GetTxnStatus(bo, txn.StartTS(), []byte("key"), currentTS, currentTS, true, false, nil)
c.Assert(err, IsNil)
c.Assert(status.IsCommitted(), IsFalse)
c.Assert(status.TTL(), Greater, uint64(0))
c.Assert(status.CommitTS(), Equals, uint64(0))
c.Assert(status.Action(), Equals, kvrpcpb.Action_MinCommitTSPushed)
// Test the ResolveLocks API
lock := s.mustGetLock(c, []byte("second"))
timeBeforeExpire, _, err := resolver.ResolveLocks(bo, currentTS, []*tikv.Lock{lock})
c.Assert(err, IsNil)
c.Assert(timeBeforeExpire > int64(0), IsTrue)
// Force rollback the lock using lock.TTL = 0.
lock.TTL = uint64(0)
timeBeforeExpire, _, err = resolver.ResolveLocks(bo, currentTS, []*tikv.Lock{lock})
c.Assert(err, IsNil)
c.Assert(timeBeforeExpire, Equals, int64(0))
// Then call getTxnStatus again and check the lock status.
currentTS, err = o.GetTimestamp(context.Background(), &oracle.Option{TxnScope: oracle.GlobalTxnScope})
c.Assert(err, IsNil)
status, err = s.store.NewLockResolver().GetTxnStatus(bo, txn.StartTS(), []byte("key"), currentTS, 0, true, false, nil)
c.Assert(err, IsNil)
c.Assert(status.TTL(), Equals, uint64(0))
c.Assert(status.CommitTS(), Equals, uint64(0))
c.Assert(status.Action(), Equals, kvrpcpb.Action_NoAction)
// Call getTxnStatus on a committed transaction.
startTS, commitTS := s.putKV(c, []byte("a"), []byte("a"))
status, err = s.store.NewLockResolver().GetTxnStatus(bo, startTS, []byte("a"), currentTS, currentTS, true, false, nil)
c.Assert(err, IsNil)
c.Assert(status.TTL(), Equals, uint64(0))
c.Assert(status.CommitTS(), Equals, commitTS)
}
func (s *testLockSuite) TestCheckTxnStatusNoWait(c *C) {
txn, err := s.store.Begin()
c.Assert(err, IsNil)
txn.Set([]byte("key"), []byte("value"))
txn.Set([]byte("second"), []byte("xxx"))
committer, err := txn.NewCommitter(0)
c.Assert(err, IsNil)
// Increase lock TTL to make CI more stable.
committer.SetLockTTLByTimeAndSize(txn.GetStartTime(), 200*1024*1024)
// Only prewrite the secondary key to simulate a concurrent prewrite case:
// prewrite secondary regions success and prewrite the primary region is pending.
err = committer.PrewriteMutations(context.Background(), committer.MutationsOfKeys([][]byte{[]byte("second")}))
c.Assert(err, IsNil)
o := s.store.GetOracle()
currentTS, err := o.GetTimestamp(context.Background(), &oracle.Option{TxnScope: oracle.GlobalTxnScope})
c.Assert(err, IsNil)
bo := tikv.NewBackofferWithVars(context.Background(), tikv.PrewriteMaxBackoff, nil)
resolver := s.store.NewLockResolver()
// Call getTxnStatus for the TxnNotFound case.
_, err = resolver.GetTxnStatus(bo, txn.StartTS(), []byte("key"), currentTS, currentTS, false, false, nil)
c.Assert(err, NotNil)
c.Assert(resolver.IsErrorNotFound(err), IsTrue)
errCh := make(chan error)
go func() {
errCh <- committer.PrewriteMutations(context.Background(), committer.MutationsOfKeys([][]byte{[]byte("key")}))
}()
lock := &tikv.Lock{
Key: []byte("second"),
Primary: []byte("key"),
TxnID: txn.StartTS(),
TTL: 100000,
}
// Call getTxnStatusFromLock to cover the retry logic.
status, err := resolver.GetTxnStatusFromLock(bo, lock, currentTS, false)
c.Assert(err, IsNil)
c.Assert(status.TTL(), Greater, uint64(0))
c.Assert(<-errCh, IsNil)
c.Assert(committer.CleanupMutations(context.Background()), IsNil)
// Call getTxnStatusFromLock to cover TxnNotFound and retry timeout.
startTS, err := o.GetTimestamp(context.Background(), &oracle.Option{TxnScope: oracle.GlobalTxnScope})
c.Assert(err, IsNil)
lock = &tikv.Lock{
Key: []byte("second"),
Primary: []byte("key_not_exist"),
TxnID: startTS,
TTL: 1000,
}
status, err = resolver.GetTxnStatusFromLock(bo, lock, currentTS, false)
c.Assert(err, IsNil)
c.Assert(status.TTL(), Equals, uint64(0))
c.Assert(status.CommitTS(), Equals, uint64(0))
c.Assert(status.Action(), Equals, kvrpcpb.Action_LockNotExistRollback)
}
func (s *testLockSuite) prewriteTxn(c *C, txn tikv.TxnProbe) {
s.prewriteTxnWithTTL(c, txn, 0)
}
func (s *testLockSuite) prewriteTxnWithTTL(c *C, txn tikv.TxnProbe, ttl uint64) {
committer, err := txn.NewCommitter(0)
c.Assert(err, IsNil)
if ttl > 0 {
elapsed := time.Since(txn.GetStartTime()) / time.Millisecond
committer.SetLockTTL(uint64(elapsed) + ttl)
}
err = committer.PrewriteAllMutations(context.Background())
c.Assert(err, IsNil)
}
func (s *testLockSuite) mustGetLock(c *C, key []byte) *tikv.Lock {
ver, err := s.store.CurrentTimestamp(oracle.GlobalTxnScope)
c.Assert(err, IsNil)
bo := tikv.NewBackofferWithVars(context.Background(), getMaxBackoff, nil)
req := tikvrpc.NewRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{
Key: key,
Version: ver,
})
loc, err := s.store.GetRegionCache().LocateKey(bo, key)
c.Assert(err, IsNil)
resp, err := s.store.SendReq(bo, req, loc.Region, tikv.ReadTimeoutShort)
c.Assert(err, IsNil)
c.Assert(resp.Resp, NotNil)
keyErr := resp.Resp.(*kvrpcpb.GetResponse).GetError()
c.Assert(keyErr, NotNil)
lock, err := tikv.LockProbe{}.ExtractLockFromKeyErr(keyErr)
c.Assert(err, IsNil)
return lock
}
func (s *testLockSuite) ttlEquals(c *C, x, y uint64) {
// NOTE: On ppc64le, all integers are by default unsigned integers,
// hence we have to separately cast the value returned by "math.Abs()" function for ppc64le.
if runtime.GOARCH == "ppc64le" {
c.Assert(int(-math.Abs(float64(x-y))), LessEqual, 2)
} else {
c.Assert(int(math.Abs(float64(x-y))), LessEqual, 2)
}
}
func (s *testLockSuite) TestLockTTL(c *C) {
defaultLockTTL := tikv.ConfigProbe{}.GetDefaultLockTTL()
ttlFactor := tikv.ConfigProbe{}.GetTTLFactor()
txn, err := s.store.Begin()
c.Assert(err, IsNil)
txn.Set([]byte("key"), []byte("value"))
time.Sleep(time.Millisecond)
s.prewriteTxnWithTTL(c, txn, 3100)
l := s.mustGetLock(c, []byte("key"))
c.Assert(l.TTL >= defaultLockTTL, IsTrue)
// Huge txn has a greater TTL.
txn, err = s.store.Begin()
start := time.Now()
c.Assert(err, IsNil)
txn.Set([]byte("key"), []byte("value"))
for i := 0; i < 2048; i++ {
k, v := randKV(1024, 1024)
txn.Set([]byte(k), []byte(v))
}
s.prewriteTxn(c, txn)
l = s.mustGetLock(c, []byte("key"))
s.ttlEquals(c, l.TTL, uint64(ttlFactor*2)+uint64(time.Since(start)/time.Millisecond))
// Txn with long read time.
start = time.Now()
txn, err = s.store.Begin()
c.Assert(err, IsNil)
time.Sleep(time.Millisecond * 50)
txn.Set([]byte("key"), []byte("value"))
s.prewriteTxn(c, txn)
l = s.mustGetLock(c, []byte("key"))
s.ttlEquals(c, l.TTL, defaultLockTTL+uint64(time.Since(start)/time.Millisecond))
}
func (s *testLockSuite) TestBatchResolveLocks(c *C) {
// The first transaction is a normal transaction with a long TTL
txn, err := s.store.Begin()
c.Assert(err, IsNil)
txn.Set([]byte("k1"), []byte("v1"))
txn.Set([]byte("k2"), []byte("v2"))
s.prewriteTxnWithTTL(c, txn, 20000)
// The second transaction is an async commit transaction
txn, err = s.store.Begin()
c.Assert(err, IsNil)
txn.Set([]byte("k3"), []byte("v3"))
txn.Set([]byte("k4"), []byte("v4"))
committer, err := txn.NewCommitter(0)
c.Assert(err, IsNil)
committer.SetUseAsyncCommit()
committer.SetLockTTL(20000)
committer.PrewriteAllMutations(context.Background())
c.Assert(err, IsNil)
var locks []*tikv.Lock
for _, key := range []string{"k1", "k2", "k3", "k4"} {
l := s.mustGetLock(c, []byte(key))
locks = append(locks, l)
}
// Locks may not expired
msBeforeLockExpired := s.store.GetOracle().UntilExpired(locks[0].TxnID, locks[1].TTL, &oracle.Option{TxnScope: oracle.GlobalTxnScope})
c.Assert(msBeforeLockExpired, Greater, int64(0))
msBeforeLockExpired = s.store.GetOracle().UntilExpired(locks[3].TxnID, locks[3].TTL, &oracle.Option{TxnScope: oracle.GlobalTxnScope})
c.Assert(msBeforeLockExpired, Greater, int64(0))
lr := s.store.NewLockResolver()
bo := tikv.NewGcResolveLockMaxBackoffer(context.Background())
loc, err := s.store.GetRegionCache().LocateKey(bo, locks[0].Primary)
c.Assert(err, IsNil)
// Check BatchResolveLocks resolve the lock even the ttl is not expired.
success, err := lr.BatchResolveLocks(bo, locks, loc.Region)
c.Assert(success, IsTrue)
c.Assert(err, IsNil)
txn, err = s.store.Begin()
c.Assert(err, IsNil)
// transaction 1 is rolled back
_, err = txn.Get(context.Background(), []byte("k1"))
c.Assert(err, Equals, tikverr.ErrNotExist)
_, err = txn.Get(context.Background(), []byte("k2"))
c.Assert(err, Equals, tikverr.ErrNotExist)
// transaction 2 is committed
v, err := txn.Get(context.Background(), []byte("k3"))
c.Assert(err, IsNil)
c.Assert(bytes.Equal(v, []byte("v3")), IsTrue)
v, err = txn.Get(context.Background(), []byte("k4"))
c.Assert(err, IsNil)
c.Assert(bytes.Equal(v, []byte("v4")), IsTrue)
}
func (s *testLockSuite) TestNewLockZeroTTL(c *C) {
l := tikv.NewLock(&kvrpcpb.LockInfo{})
c.Assert(l.TTL, Equals, uint64(0))
}
func init() {
// Speed up tests.
tikv.ConfigProbe{}.SetOracleUpdateInterval(2)
}
func (s *testLockSuite) TestZeroMinCommitTS(c *C) {
txn, err := s.store.Begin()
c.Assert(err, IsNil)
txn.Set([]byte("key"), []byte("value"))
bo := tikv.NewBackofferWithVars(context.Background(), tikv.PrewriteMaxBackoff, nil)
mockValue := fmt.Sprintf(`return(%d)`, txn.StartTS())
c.Assert(failpoint.Enable("tikvclient/mockZeroCommitTS", mockValue), IsNil)
s.prewriteTxnWithTTL(c, txn, 1000)
c.Assert(failpoint.Disable("tikvclient/mockZeroCommitTS"), IsNil)
lock := s.mustGetLock(c, []byte("key"))
expire, pushed, err := s.store.NewLockResolver().ResolveLocks(bo, 0, []*tikv.Lock{lock})
c.Assert(err, IsNil)
c.Assert(pushed, HasLen, 0)
c.Assert(expire, Greater, int64(0))
expire, pushed, err = s.store.NewLockResolver().ResolveLocks(bo, math.MaxUint64, []*tikv.Lock{lock})
c.Assert(err, IsNil)
c.Assert(pushed, HasLen, 1)
c.Assert(expire, Greater, int64(0))
// Clean up this test.
lock.TTL = uint64(0)
expire, _, err = s.store.NewLockResolver().ResolveLocks(bo, 0, []*tikv.Lock{lock})
c.Assert(err, IsNil)
c.Assert(expire, Equals, int64(0))
}
func (s *testLockSuite) prepareTxnFallenBackFromAsyncCommit(c *C) {
txn, err := s.store.Begin()
c.Assert(err, IsNil)
err = txn.Set([]byte("fb1"), []byte("1"))
c.Assert(err, IsNil)
err = txn.Set([]byte("fb2"), []byte("2"))
c.Assert(err, IsNil)
committer, err := txn.NewCommitter(1)
c.Assert(err, IsNil)
c.Assert(committer.GetMutations().Len(), Equals, 2)
committer.SetLockTTL(0)
committer.SetUseAsyncCommit()
committer.SetCommitTS(committer.GetStartTS() + (100 << 18)) // 100ms
err = committer.PrewriteMutations(context.Background(), committer.GetMutations().Slice(0, 1))
c.Assert(err, IsNil)
c.Assert(committer.IsAsyncCommit(), IsTrue)
// Set an invalid maxCommitTS to produce MaxCommitTsTooLarge
committer.SetMaxCommitTS(committer.GetStartTS() - 1)
err = committer.PrewriteMutations(context.Background(), committer.GetMutations().Slice(1, 2))
c.Assert(err, IsNil)
c.Assert(committer.IsAsyncCommit(), IsFalse) // Fallback due to MaxCommitTsTooLarge
}
func (s *testLockSuite) TestCheckLocksFallenBackFromAsyncCommit(c *C) {
s.prepareTxnFallenBackFromAsyncCommit(c)
lock := s.mustGetLock(c, []byte("fb1"))
c.Assert(lock.UseAsyncCommit, IsTrue)
bo := tikv.NewBackoffer(context.Background(), getMaxBackoff)
lr := s.store.NewLockResolver()
status, err := lr.GetTxnStatusFromLock(bo, lock, 0, false)
c.Assert(err, IsNil)
c.Assert(tikv.LockProbe{}.GetPrimaryKeyFromTxnStatus(status), DeepEquals, []byte("fb1"))
err = lr.CheckAllSecondaries(bo, lock, &status)
c.Assert(lr.IsNonAsyncCommitLock(err), IsTrue)
status, err = lr.GetTxnStatusFromLock(bo, lock, 0, true)
c.Assert(err, IsNil)
c.Assert(status.Action(), Equals, kvrpcpb.Action_TTLExpireRollback)
c.Assert(status.TTL(), Equals, uint64(0))
}
func (s *testLockSuite) TestResolveTxnFallenBackFromAsyncCommit(c *C) {
s.prepareTxnFallenBackFromAsyncCommit(c)
lock := s.mustGetLock(c, []byte("fb1"))
c.Assert(lock.UseAsyncCommit, IsTrue)
bo := tikv.NewBackoffer(context.Background(), getMaxBackoff)
expire, pushed, err := s.store.NewLockResolver().ResolveLocks(bo, 0, []*tikv.Lock{lock})
c.Assert(err, IsNil)
c.Assert(expire, Equals, int64(0))
c.Assert(len(pushed), Equals, 0)
t3, err := s.store.Begin()
c.Assert(err, IsNil)
_, err = t3.Get(context.Background(), []byte("fb1"))
c.Assert(tikverr.IsErrNotFound(err), IsTrue)
_, err = t3.Get(context.Background(), []byte("fb2"))
c.Assert(tikverr.IsErrNotFound(err), IsTrue)
}
func (s *testLockSuite) TestBatchResolveTxnFallenBackFromAsyncCommit(c *C) {
s.prepareTxnFallenBackFromAsyncCommit(c)
lock := s.mustGetLock(c, []byte("fb1"))
c.Assert(lock.UseAsyncCommit, IsTrue)
bo := tikv.NewBackoffer(context.Background(), getMaxBackoff)
loc, err := s.store.GetRegionCache().LocateKey(bo, []byte("fb1"))
c.Assert(err, IsNil)
ok, err := s.store.NewLockResolver().BatchResolveLocks(bo, []*tikv.Lock{lock}, loc.Region)
c.Assert(err, IsNil)
c.Assert(ok, IsTrue)
t3, err := s.store.Begin()
c.Assert(err, IsNil)
_, err = t3.Get(context.Background(), []byte("fb1"))
c.Assert(tikverr.IsErrNotFound(err), IsTrue)
_, err = t3.Get(context.Background(), []byte("fb2"))
c.Assert(tikverr.IsErrNotFound(err), IsTrue)
}
func (s *testLockSuite) TestDeadlockReportWaitChain(c *C) {
// Utilities to make the test logic clear and simple.
type txnWrapper struct {
tikv.TxnProbe
wg sync.WaitGroup
}
makeLockCtx := func(txn *txnWrapper, resourceGroupTag string) *kv.LockCtx {
return &kv.LockCtx{
ForUpdateTS: txn.StartTS(),
WaitStartTime: time.Now(),
LockWaitTime: 1000,
ResourceGroupTag: []byte(resourceGroupTag),
}
}
// Prepares several transactions and each locks a key.
prepareTxns := func(num int) []*txnWrapper {
res := make([]*txnWrapper, 0, num)
for i := 0; i < num; i++ {
txnProbe, err := s.store.Begin()
c.Assert(err, IsNil)
txn := &txnWrapper{TxnProbe: txnProbe}
txn.SetPessimistic(true)
tag := fmt.Sprintf("tag-init%v", i)
key := []byte{'k', byte(i)}
err = txn.LockKeys(context.Background(), makeLockCtx(txn, tag), key)
c.Assert(err, IsNil)
res = append(res, txn)
}
return res
}
// Let the i-th trnasaction lock the key that has been locked by j-th transaction
tryLock := func(txns []*txnWrapper, i int, j int) error {
c.Logf("txn %v try locking %v", i, j)
txn := txns[i]
tag := fmt.Sprintf("tag-%v-%v", i, j)
key := []byte{'k', byte(j)}
return txn.LockKeys(context.Background(), makeLockCtx(txn, tag), key)
}
// Asserts the i-th transaction waits for the j-th transaction.
makeWaitFor := func(txns []*txnWrapper, i int, j int) {
txns[i].wg.Add(1)
go func() {
defer txns[i].wg.Done()
err := tryLock(txns, i, j)
// After the lock being waited for is released, the transaction returns a WriteConflict error
// unconditionally, which is by design.
c.Assert(err, NotNil)
c.Logf("txn %v wait for %v finished, err: %s", i, j, err.Error())
_, ok := errors.Cause(err).(*tikverr.ErrWriteConflict)
c.Assert(ok, IsTrue)
}()
}
waitAndRollback := func(txns []*txnWrapper, i int) {
// It's expected that each transaction should be rolled back after its blocker, so that `Rollback` will not
// run when there's concurrent `LockKeys` running.
// If it's blocked on the `Wait` forever, it means the transaction's blocker is not rolled back.
c.Logf("rollback txn %v", i)
txns[i].wg.Wait()
err := txns[i].Rollback()
c.Assert(err, IsNil)
}
// Check the given WaitForEntry is caused by txn[i] waiting for txn[j].
checkWaitChainEntry := func(txns []*txnWrapper, entry *deadlockpb.WaitForEntry, i, j int) {
c.Assert(entry.Txn, Equals, txns[i].StartTS())
c.Assert(entry.WaitForTxn, Equals, txns[j].StartTS())
c.Assert(entry.Key, BytesEquals, []byte{'k', byte(j)})
c.Assert(string(entry.ResourceGroupTag), Equals, fmt.Sprintf("tag-%v-%v", i, j))
}
c.Log("test case 1: 1->0->1")
txns := prepareTxns(2)
makeWaitFor(txns, 0, 1)
// Sleep for a while to make sure it has been blocked.
time.Sleep(time.Millisecond * 100)
// txn2 tries locking k1 and encounters deadlock error.
err := tryLock(txns, 1, 0)
c.Assert(err, NotNil)
dl, ok := errors.Cause(err).(*tikverr.ErrDeadlock)
c.Assert(ok, IsTrue)
waitChain := dl.GetWaitChain()
c.Assert(len(waitChain), Equals, 2)
checkWaitChainEntry(txns, waitChain[0], 0, 1)
checkWaitChainEntry(txns, waitChain[1], 1, 0)
// Each transaction should be rolled back after its blocker being rolled back
waitAndRollback(txns, 1)
waitAndRollback(txns, 0)
c.Log("test case 2: 3->2->0->1->3")
txns = prepareTxns(4)
makeWaitFor(txns, 0, 1)
makeWaitFor(txns, 2, 0)
makeWaitFor(txns, 1, 3)
// Sleep for a while to make sure it has been blocked.
time.Sleep(time.Millisecond * 100)
err = tryLock(txns, 3, 2)
c.Assert(err, NotNil)
dl, ok = errors.Cause(err).(*tikverr.ErrDeadlock)
c.Assert(ok, IsTrue)
waitChain = dl.GetWaitChain()
c.Assert(len(waitChain), Equals, 4)
c.Logf("wait chain: \n** %v\n**%v\n**%v\n**%v\n", waitChain[0], waitChain[1], waitChain[2], waitChain[3])
checkWaitChainEntry(txns, waitChain[0], 2, 0)
checkWaitChainEntry(txns, waitChain[1], 0, 1)
checkWaitChainEntry(txns, waitChain[2], 1, 3)
checkWaitChainEntry(txns, waitChain[3], 3, 2)
// Each transaction should be rolled back after its blocker being rolled back
waitAndRollback(txns, 3)
waitAndRollback(txns, 1)
waitAndRollback(txns, 0)
waitAndRollback(txns, 2)
}

View File

@ -0,0 +1,67 @@
// Copyright 2020 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_test
import (
. "github.com/pingcap/check"
"github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pingcap/tidb/store/mockstore/unistore"
"github.com/tikv/client-go/v2/tikv"
)
type testPrewriteSuite struct {
store *tikv.KVStore
}
var _ = Suite(&testPrewriteSuite{})
func (s *testPrewriteSuite) SetUpTest(c *C) {
client, pdClient, cluster, err := unistore.New("")
c.Assert(err, IsNil)
unistore.BootstrapWithSingleStore(cluster)
store, err := tikv.NewTestTiKVStore(client, pdClient, nil, nil, 0)
c.Assert(err, IsNil)
s.store = store
}
func (s *testPrewriteSuite) TestSetMinCommitTSInAsyncCommit(c *C) {
t, err := s.store.Begin()
c.Assert(err, IsNil)
txn := tikv.TxnProbe{KVTxn: t}
err = txn.Set([]byte("k"), []byte("v"))
c.Assert(err, IsNil)
committer, err := txn.NewCommitter(1)
c.Assert(err, IsNil)
committer.SetUseAsyncCommit()
buildRequest := func() *kvrpcpb.PrewriteRequest {
req := committer.BuildPrewriteRequest(1, 1, 1, committer.GetMutations(), 1)
return req.Req.(*kvrpcpb.PrewriteRequest)
}
// no forUpdateTS
req := buildRequest()
c.Assert(req.MinCommitTs, Equals, txn.StartTS()+1)
// forUpdateTS is set
committer.SetForUpdateTS(txn.StartTS() + (5 << 18))
req = buildRequest()
c.Assert(req.MinCommitTs, Equals, committer.GetForUpdateTS()+1)
// minCommitTS is set
committer.SetMinCommitTS(txn.StartTS() + (10 << 18))
req = buildRequest()
c.Assert(req.MinCommitTs, Equals, committer.GetMinCommitTS())
}

View File

@ -0,0 +1,242 @@
// Copyright 2019 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_test
import (
"bytes"
"context"
"errors"
"sort"
. "github.com/pingcap/check"
"github.com/pingcap/tidb/store/mockstore/mockcopr"
"github.com/tikv/client-go/v2/kv"
"github.com/tikv/client-go/v2/mockstore/cluster"
"github.com/tikv/client-go/v2/mockstore/mocktikv"
"github.com/tikv/client-go/v2/tikv"
)
type testRangeTaskSuite struct {
OneByOneSuite
cluster cluster.Cluster
store *tikv.KVStore
testRanges []kv.KeyRange
expectedRanges [][]kv.KeyRange
}
var _ = Suite(&testRangeTaskSuite{})
func makeRange(startKey string, endKey string) kv.KeyRange {
return kv.KeyRange{
StartKey: []byte(startKey),
EndKey: []byte(endKey),
}
}
func (s *testRangeTaskSuite) SetUpTest(c *C) {
// Split the store at "a" to "z"
splitKeys := make([][]byte, 0)
for k := byte('a'); k <= byte('z'); k++ {
splitKeys = append(splitKeys, []byte{k})
}
// Calculate all region's ranges
allRegionRanges := []kv.KeyRange{makeRange("", "a")}
for i := 0; i < len(splitKeys)-1; i++ {
allRegionRanges = append(allRegionRanges, kv.KeyRange{
StartKey: splitKeys[i],
EndKey: splitKeys[i+1],
})
}
allRegionRanges = append(allRegionRanges, makeRange("z", ""))
client, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("", mockcopr.NewCoprRPCHandler())
c.Assert(err, IsNil)
mocktikv.BootstrapWithMultiRegions(cluster, splitKeys...)
s.cluster = cluster
store, err := tikv.NewTestTiKVStore(client, pdClient, nil, nil, 0)
c.Assert(err, IsNil)
// TODO: make this possible
// store, err := mockstore.NewMockStore(
// mockstore.WithStoreType(mockstore.MockTiKV),
// mockstore.WithClusterInspector(func(c cluster.Cluster) {
// mockstore.BootstrapWithMultiRegions(c, splitKeys...)
// s.cluster = c
// }),
// )
// c.Assert(err, IsNil)
s.store = store
s.testRanges = []kv.KeyRange{
makeRange("", ""),
makeRange("", "b"),
makeRange("b", ""),
makeRange("b", "x"),
makeRange("a", "d"),
makeRange("a\x00", "d\x00"),
makeRange("a\xff\xff\xff", "c\xff\xff\xff"),
makeRange("a1", "a2"),
makeRange("a", "a"),
makeRange("a3", "a3"),
}
s.expectedRanges = [][]kv.KeyRange{
allRegionRanges,
allRegionRanges[:2],
allRegionRanges[2:],
allRegionRanges[2:24],
{
makeRange("a", "b"),
makeRange("b", "c"),
makeRange("c", "d"),
},
{
makeRange("a\x00", "b"),
makeRange("b", "c"),
makeRange("c", "d"),
makeRange("d", "d\x00"),
},
{
makeRange("a\xff\xff\xff", "b"),
makeRange("b", "c"),
makeRange("c", "c\xff\xff\xff"),
},
{
makeRange("a1", "a2"),
},
{},
{},
}
}
func (s *testRangeTaskSuite) TearDownTest(c *C) {
err := s.store.Close()
c.Assert(err, IsNil)
}
func collect(c chan *kv.KeyRange) []kv.KeyRange {
c <- nil
ranges := make([]kv.KeyRange, 0)
for {
r := <-c
if r == nil {
break
}
ranges = append(ranges, *r)
}
return ranges
}
func (s *testRangeTaskSuite) checkRanges(c *C, obtained []kv.KeyRange, expected []kv.KeyRange) {
sort.Slice(obtained, func(i, j int) bool {
return bytes.Compare(obtained[i].StartKey, obtained[j].StartKey) < 0
})
c.Assert(obtained, DeepEquals, expected)
}
func batchRanges(ranges []kv.KeyRange, batchSize int) []kv.KeyRange {
result := make([]kv.KeyRange, 0, len(ranges))
for i := 0; i < len(ranges); i += batchSize {
lastRange := i + batchSize - 1
if lastRange >= len(ranges) {
lastRange = len(ranges) - 1
}
result = append(result, kv.KeyRange{
StartKey: ranges[i].StartKey,
EndKey: ranges[lastRange].EndKey,
})
}
return result
}
func (s *testRangeTaskSuite) testRangeTaskImpl(c *C, concurrency int) {
c.Logf("Test RangeTask, concurrency: %v", concurrency)
ranges := make(chan *kv.KeyRange, 100)
handler := func(ctx context.Context, r kv.KeyRange) (tikv.RangeTaskStat, error) {
ranges <- &r
stat := tikv.RangeTaskStat{
CompletedRegions: 1,
}
return stat, nil
}
runner := tikv.NewRangeTaskRunner("test-runner", s.store, concurrency, handler)
for regionsPerTask := 1; regionsPerTask <= 5; regionsPerTask++ {
for i, r := range s.testRanges {
runner.SetRegionsPerTask(regionsPerTask)
expectedRanges := batchRanges(s.expectedRanges[i], regionsPerTask)
err := runner.RunOnRange(context.Background(), r.StartKey, r.EndKey)
c.Assert(err, IsNil)
s.checkRanges(c, collect(ranges), expectedRanges)
c.Assert(runner.CompletedRegions(), Equals, len(expectedRanges))
c.Assert(runner.FailedRegions(), Equals, 0)
}
}
}
func (s *testRangeTaskSuite) TestRangeTask(c *C) {
for concurrency := 1; concurrency < 5; concurrency++ {
s.testRangeTaskImpl(c, concurrency)
}
}
func (s *testRangeTaskSuite) testRangeTaskErrorImpl(c *C, concurrency int) {
for i, r := range s.testRanges {
// Iterate all sub tasks and make it an error
subRanges := s.expectedRanges[i]
for _, subRange := range subRanges {
errKey := subRange.StartKey
c.Logf("Test RangeTask Error concurrency: %v, range: [%+q, %+q), errKey: %+q", concurrency, r.StartKey, r.EndKey, errKey)
handler := func(ctx context.Context, r kv.KeyRange) (tikv.RangeTaskStat, error) {
stat := tikv.RangeTaskStat{CompletedRegions: 0, FailedRegions: 0}
if bytes.Equal(r.StartKey, errKey) {
stat.FailedRegions++
return stat, errors.New("test error")
}
stat.CompletedRegions++
return stat, nil
}
runner := tikv.NewRangeTaskRunner("test-error-runner", s.store, concurrency, handler)
runner.SetRegionsPerTask(1)
err := runner.RunOnRange(context.Background(), r.StartKey, r.EndKey)
// RunOnRange returns no error only when all sub tasks are done successfully.
c.Assert(err, NotNil)
c.Assert(runner.CompletedRegions(), Less, len(subRanges))
c.Assert(runner.FailedRegions(), Equals, 1)
}
}
}
func (s *testRangeTaskSuite) TestRangeTaskError(c *C) {
for concurrency := 1; concurrency < 5; concurrency++ {
s.testRangeTaskErrorImpl(c, concurrency)
}
}

View File

@ -0,0 +1,312 @@
// 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_test
import (
"bytes"
"context"
"fmt"
. "github.com/pingcap/check"
"github.com/pingcap/tidb/store/mockstore/unistore"
"github.com/tikv/client-go/v2/mockstore/cluster"
"github.com/tikv/client-go/v2/tikv"
)
type testRawKVSuite struct {
OneByOneSuite
cluster cluster.Cluster
client tikv.RawKVClientProbe
bo *tikv.Backoffer
}
var _ = Suite(&testRawKVSuite{})
func (s *testRawKVSuite) SetUpTest(c *C) {
client, pdClient, cluster, err := unistore.New("")
c.Assert(err, IsNil)
unistore.BootstrapWithSingleStore(cluster)
s.cluster = cluster
s.client = tikv.RawKVClientProbe{RawKVClient: &tikv.RawKVClient{}}
s.client.SetPDClient(pdClient)
s.client.SetRegionCache(tikv.NewRegionCache(pdClient))
s.client.SetRPCClient(client)
s.bo = tikv.NewBackofferWithVars(context.Background(), 5000, nil)
}
func (s *testRawKVSuite) TearDownTest(c *C) {
s.client.Close()
}
func (s *testRawKVSuite) mustNotExist(c *C, key []byte) {
v, err := s.client.Get(key)
c.Assert(err, IsNil)
c.Assert(v, IsNil)
}
func (s *testRawKVSuite) mustBatchNotExist(c *C, keys [][]byte) {
values, err := s.client.BatchGet(keys)
c.Assert(err, IsNil)
c.Assert(values, NotNil)
c.Assert(len(keys), Equals, len(values))
for _, value := range values {
c.Assert([]byte{}, BytesEquals, value)
}
}
func (s *testRawKVSuite) mustGet(c *C, key, value []byte) {
v, err := s.client.Get(key)
c.Assert(err, IsNil)
c.Assert(v, NotNil)
c.Assert(v, BytesEquals, value)
}
func (s *testRawKVSuite) mustBatchGet(c *C, keys, values [][]byte) {
checkValues, err := s.client.BatchGet(keys)
c.Assert(err, IsNil)
c.Assert(checkValues, NotNil)
c.Assert(len(keys), Equals, len(checkValues))
for i := range keys {
c.Check(values[i], BytesEquals, checkValues[i])
}
}
func (s *testRawKVSuite) mustPut(c *C, key, value []byte) {
err := s.client.Put(key, value)
c.Assert(err, IsNil)
}
func (s *testRawKVSuite) mustBatchPut(c *C, keys, values [][]byte) {
err := s.client.BatchPut(keys, values)
c.Assert(err, IsNil)
}
func (s *testRawKVSuite) mustDelete(c *C, key []byte) {
err := s.client.Delete(key)
c.Assert(err, IsNil)
}
func (s *testRawKVSuite) mustBatchDelete(c *C, keys [][]byte) {
err := s.client.BatchDelete(keys)
c.Assert(err, IsNil)
}
func (s *testRawKVSuite) mustScan(c *C, startKey string, limit int, expect ...string) {
keys, values, err := s.client.Scan([]byte(startKey), nil, limit)
c.Assert(err, IsNil)
c.Assert(len(keys)*2, Equals, len(expect))
for i := range keys {
c.Assert(string(keys[i]), Equals, expect[i*2])
c.Assert(string(values[i]), Equals, expect[i*2+1])
}
}
func (s *testRawKVSuite) mustScanRange(c *C, startKey string, endKey string, limit int, expect ...string) {
keys, values, err := s.client.Scan([]byte(startKey), []byte(endKey), limit)
c.Assert(err, IsNil)
c.Assert(len(keys)*2, Equals, len(expect))
for i := range keys {
c.Assert(string(keys[i]), Equals, expect[i*2])
c.Assert(string(values[i]), Equals, expect[i*2+1])
}
}
func (s *testRawKVSuite) mustReverseScan(c *C, startKey []byte, limit int, expect ...string) {
keys, values, err := s.client.ReverseScan(startKey, nil, limit)
c.Assert(err, IsNil)
c.Assert(len(keys)*2, Equals, len(expect))
for i := range keys {
c.Assert(string(keys[i]), Equals, expect[i*2])
c.Assert(string(values[i]), Equals, expect[i*2+1])
}
}
func (s *testRawKVSuite) mustReverseScanRange(c *C, startKey, endKey []byte, limit int, expect ...string) {
keys, values, err := s.client.ReverseScan(startKey, endKey, limit)
c.Assert(err, IsNil)
c.Assert(len(keys)*2, Equals, len(expect))
for i := range keys {
c.Assert(string(keys[i]), Equals, expect[i*2])
c.Assert(string(values[i]), Equals, expect[i*2+1])
}
}
func (s *testRawKVSuite) mustDeleteRange(c *C, startKey, endKey []byte, expected map[string]string) {
err := s.client.DeleteRange(startKey, endKey)
c.Assert(err, IsNil)
for keyStr := range expected {
key := []byte(keyStr)
if bytes.Compare(startKey, key) <= 0 && bytes.Compare(key, endKey) < 0 {
delete(expected, keyStr)
}
}
s.checkData(c, expected)
}
func (s *testRawKVSuite) checkData(c *C, expected map[string]string) {
keys, values, err := s.client.Scan([]byte(""), nil, len(expected)+1)
c.Assert(err, IsNil)
c.Assert(len(expected), Equals, len(keys))
for i, key := range keys {
c.Assert(expected[string(key)], Equals, string(values[i]))
}
}
func (s *testRawKVSuite) split(c *C, regionKey, splitKey string) error {
loc, err := s.client.GetRegionCache().LocateKey(s.bo, []byte(regionKey))
if err != nil {
return err
}
newRegionID, peerID := s.cluster.AllocID(), s.cluster.AllocID()
s.cluster.SplitRaw(loc.Region.GetID(), newRegionID, []byte(splitKey), []uint64{peerID}, peerID)
return nil
}
func (s *testRawKVSuite) TestSimple(c *C) {
s.mustNotExist(c, []byte("key"))
s.mustPut(c, []byte("key"), []byte("value"))
s.mustGet(c, []byte("key"), []byte("value"))
s.mustDelete(c, []byte("key"))
s.mustNotExist(c, []byte("key"))
err := s.client.Put([]byte("key"), []byte(""))
c.Assert(err, NotNil)
}
func (s *testRawKVSuite) TestRawBatch(c *C) {
testNum := 0
size := 0
var testKeys [][]byte
var testValues [][]byte
for i := 0; size/(tikv.ConfigProbe{}.GetRawBatchPutSize()) < 4; i++ {
key := fmt.Sprint("key", i)
size += len(key)
testKeys = append(testKeys, []byte(key))
value := fmt.Sprint("value", i)
size += len(value)
testValues = append(testValues, []byte(value))
s.mustNotExist(c, []byte(key))
testNum = i
}
err := s.split(c, "", fmt.Sprint("key", testNum/2))
c.Assert(err, IsNil)
s.mustBatchPut(c, testKeys, testValues)
s.mustBatchGet(c, testKeys, testValues)
s.mustBatchDelete(c, testKeys)
s.mustBatchNotExist(c, testKeys)
}
func (s *testRawKVSuite) TestSplit(c *C) {
s.mustPut(c, []byte("k1"), []byte("v1"))
s.mustPut(c, []byte("k3"), []byte("v3"))
err := s.split(c, "k", "k2")
c.Assert(err, IsNil)
s.mustGet(c, []byte("k1"), []byte("v1"))
s.mustGet(c, []byte("k3"), []byte("v3"))
}
func (s *testRawKVSuite) TestScan(c *C) {
s.mustPut(c, []byte("k1"), []byte("v1"))
s.mustPut(c, []byte("k3"), []byte("v3"))
s.mustPut(c, []byte("k5"), []byte("v5"))
s.mustPut(c, []byte("k7"), []byte("v7"))
check := func() {
s.mustScan(c, "", 1, "k1", "v1")
s.mustScan(c, "k1", 2, "k1", "v1", "k3", "v3")
s.mustScan(c, "", 10, "k1", "v1", "k3", "v3", "k5", "v5", "k7", "v7")
s.mustScan(c, "k2", 2, "k3", "v3", "k5", "v5")
s.mustScan(c, "k2", 3, "k3", "v3", "k5", "v5", "k7", "v7")
s.mustScanRange(c, "", "k1", 1)
s.mustScanRange(c, "k1", "k3", 2, "k1", "v1")
s.mustScanRange(c, "k1", "k5", 10, "k1", "v1", "k3", "v3")
s.mustScanRange(c, "k1", "k5\x00", 10, "k1", "v1", "k3", "v3", "k5", "v5")
s.mustScanRange(c, "k5\x00", "k5\x00\x00", 10)
}
check()
err := s.split(c, "k", "k2")
c.Assert(err, IsNil)
check()
err = s.split(c, "k2", "k5")
c.Assert(err, IsNil)
check()
}
func (s *testRawKVSuite) TestReverseScan(c *C) {
s.mustPut(c, []byte("k1"), []byte("v1"))
s.mustPut(c, []byte("k3"), []byte("v3"))
s.mustPut(c, []byte("k5"), []byte("v5"))
s.mustPut(c, []byte("k7"), []byte("v7"))
s.checkReverseScan(c)
err := s.split(c, "k", "k2")
c.Assert(err, IsNil)
s.checkReverseScan(c)
err = s.split(c, "k2", "k5")
c.Assert(err, IsNil)
s.checkReverseScan(c)
}
func (s *testRawKVSuite) checkReverseScan(c *C) {
s.mustReverseScan(c, []byte(""), 10)
s.mustReverseScan(c, []byte("z"), 1, "k7", "v7")
s.mustReverseScan(c, []byte("z"), 2, "k7", "v7", "k5", "v5")
s.mustReverseScan(c, []byte("z"), 10, "k7", "v7", "k5", "v5", "k3", "v3", "k1", "v1")
s.mustReverseScan(c, []byte("k2"), 10, "k1", "v1")
s.mustReverseScan(c, []byte("k6"), 2, "k5", "v5", "k3", "v3")
s.mustReverseScan(c, []byte("k5"), 1, "k3", "v3")
s.mustReverseScan(c, append([]byte("k5"), 0), 1, "k5", "v5")
s.mustReverseScan(c, []byte("k6"), 3, "k5", "v5", "k3", "v3", "k1", "v1")
s.mustReverseScanRange(c, []byte("z"), []byte("k3"), 10, "k7", "v7", "k5", "v5", "k3", "v3")
s.mustReverseScanRange(c, []byte("k7"), append([]byte("k3"), 0), 10, "k5", "v5")
}
func (s *testRawKVSuite) TestDeleteRange(c *C) {
// Init data
testData := map[string]string{}
for _, i := range []byte("abcd") {
for j := byte('0'); j <= byte('9'); j++ {
key := []byte{i, j}
value := []byte{'v', i, j}
s.mustPut(c, key, value)
testData[string(key)] = string(value)
}
}
err := s.split(c, "b", "b")
c.Assert(err, IsNil)
err = s.split(c, "c", "c")
c.Assert(err, IsNil)
err = s.split(c, "d", "d")
c.Assert(err, IsNil)
s.checkData(c, testData)
s.mustDeleteRange(c, []byte("b"), []byte("c0"), testData)
s.mustDeleteRange(c, []byte("c11"), []byte("c12"), testData)
s.mustDeleteRange(c, []byte("d0"), []byte("d0"), testData)
s.mustDeleteRange(c, []byte("c5"), []byte("d5"), testData)
s.mustDeleteRange(c, []byte("a"), []byte("z"), testData)
}

View File

@ -0,0 +1,124 @@
// Copyright 2017 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_test
import (
"context"
"fmt"
"time"
. "github.com/pingcap/check"
"github.com/pingcap/errors"
"github.com/pingcap/parser/terror"
tikverr "github.com/tikv/client-go/v2/error"
"github.com/tikv/client-go/v2/tikv"
)
type testSafePointSuite struct {
OneByOneSuite
store tikv.StoreProbe
prefix string
}
var _ = Suite(&testSafePointSuite{})
func (s *testSafePointSuite) SetUpSuite(c *C) {
s.OneByOneSuite.SetUpSuite(c)
s.store = tikv.StoreProbe{KVStore: NewTestStore(c)}
s.prefix = fmt.Sprintf("seek_%d", time.Now().Unix())
}
func (s *testSafePointSuite) TearDownSuite(c *C) {
err := s.store.Close()
c.Assert(err, IsNil)
s.OneByOneSuite.TearDownSuite(c)
}
func (s *testSafePointSuite) beginTxn(c *C) tikv.TxnProbe {
txn, err := s.store.Begin()
c.Assert(err, IsNil)
return txn
}
func mymakeKeys(rowNum int, prefix string) [][]byte {
keys := make([][]byte, 0, rowNum)
for i := 0; i < rowNum; i++ {
k := encodeKey(prefix, s08d("key", i))
keys = append(keys, k)
}
return keys
}
func (s *testSafePointSuite) waitUntilErrorPlugIn(t uint64) {
for {
s.store.SaveSafePoint(t + 10)
cachedTime := time.Now()
newSafePoint, err := s.store.LoadSafePoint()
if err == nil {
s.store.UpdateSPCache(newSafePoint, cachedTime)
break
}
time.Sleep(time.Second)
}
}
func (s *testSafePointSuite) TestSafePoint(c *C) {
txn := s.beginTxn(c)
for i := 0; i < 10; i++ {
err := txn.Set(encodeKey(s.prefix, s08d("key", i)), valueBytes(i))
c.Assert(err, IsNil)
}
err := txn.Commit(context.Background())
c.Assert(err, IsNil)
// for txn get
txn2 := s.beginTxn(c)
_, err = txn2.Get(context.TODO(), encodeKey(s.prefix, s08d("key", 0)))
c.Assert(err, IsNil)
s.waitUntilErrorPlugIn(txn2.StartTS())
_, geterr2 := txn2.Get(context.TODO(), encodeKey(s.prefix, s08d("key", 0)))
c.Assert(geterr2, NotNil)
_, isFallBehind := errors.Cause(geterr2).(*tikverr.ErrGCTooEarly)
isMayFallBehind := terror.ErrorEqual(errors.Cause(geterr2), tikverr.NewErrPDServerTimeout("start timestamp may fall behind safe point"))
isBehind := isFallBehind || isMayFallBehind
c.Assert(isBehind, IsTrue)
// for txn seek
txn3 := s.beginTxn(c)
s.waitUntilErrorPlugIn(txn3.StartTS())
_, seekerr := txn3.Iter(encodeKey(s.prefix, ""), nil)
c.Assert(seekerr, NotNil)
_, isFallBehind = errors.Cause(geterr2).(*tikverr.ErrGCTooEarly)
isMayFallBehind = terror.ErrorEqual(errors.Cause(geterr2), tikverr.NewErrPDServerTimeout("start timestamp may fall behind safe point"))
isBehind = isFallBehind || isMayFallBehind
c.Assert(isBehind, IsTrue)
// for snapshot batchGet
keys := mymakeKeys(10, s.prefix)
txn4 := s.beginTxn(c)
s.waitUntilErrorPlugIn(txn4.StartTS())
_, batchgeterr := toTiDBTxn(&txn4).BatchGet(context.Background(), toTiDBKeys(keys))
c.Assert(batchgeterr, NotNil)
_, isFallBehind = errors.Cause(geterr2).(*tikverr.ErrGCTooEarly)
isMayFallBehind = terror.ErrorEqual(errors.Cause(geterr2), tikverr.NewErrPDServerTimeout("start timestamp may fall behind safe point"))
isBehind = isFallBehind || isMayFallBehind
c.Assert(isBehind, IsTrue)
}

View File

@ -0,0 +1,91 @@
// 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_test
import (
"context"
. "github.com/pingcap/check"
"github.com/tikv/client-go/v2/tikv"
)
type testScanMockSuite struct {
OneByOneSuite
}
var _ = Suite(&testScanMockSuite{})
func (s *testScanMockSuite) TestScanMultipleRegions(c *C) {
store := tikv.StoreProbe{KVStore: NewTestStore(c)}
defer store.Close()
txn, err := store.Begin()
c.Assert(err, IsNil)
for ch := byte('a'); ch <= byte('z'); ch++ {
err = txn.Set([]byte{ch}, []byte{ch})
c.Assert(err, IsNil)
}
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
txn, err = store.Begin()
c.Assert(err, IsNil)
scanner, err := txn.NewScanner([]byte("a"), nil, 10, false)
c.Assert(err, IsNil)
for ch := byte('a'); ch <= byte('z'); ch++ {
c.Assert([]byte{ch}, BytesEquals, scanner.Key())
c.Assert(scanner.Next(), IsNil)
}
c.Assert(scanner.Valid(), IsFalse)
scanner, err = txn.NewScanner([]byte("a"), []byte("i"), 10, false)
c.Assert(err, IsNil)
for ch := byte('a'); ch <= byte('h'); ch++ {
c.Assert([]byte{ch}, BytesEquals, scanner.Key())
c.Assert(scanner.Next(), IsNil)
}
c.Assert(scanner.Valid(), IsFalse)
}
func (s *testScanMockSuite) TestReverseScan(c *C) {
store := tikv.StoreProbe{KVStore: NewTestStore(c)}
defer store.Close()
txn, err := store.Begin()
c.Assert(err, IsNil)
for ch := byte('a'); ch <= byte('z'); ch++ {
err = txn.Set([]byte{ch}, []byte{ch})
c.Assert(err, IsNil)
}
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
txn, err = store.Begin()
c.Assert(err, IsNil)
scanner, err := txn.NewScanner(nil, []byte("z"), 10, true)
c.Assert(err, IsNil)
for ch := byte('y'); ch >= byte('a'); ch-- {
c.Assert(string([]byte{ch}), Equals, string(scanner.Key()))
c.Assert(scanner.Next(), IsNil)
}
c.Assert(scanner.Valid(), IsFalse)
scanner, err = txn.NewScanner([]byte("a"), []byte("i"), 10, true)
c.Assert(err, IsNil)
for ch := byte('h'); ch >= byte('a'); ch-- {
c.Assert(string([]byte{ch}), Equals, string(scanner.Key()))
c.Assert(scanner.Next(), IsNil)
}
c.Assert(scanner.Valid(), IsFalse)
}

View File

@ -0,0 +1,169 @@
// 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_test
import (
"bytes"
"context"
"fmt"
. "github.com/pingcap/check"
"github.com/tikv/client-go/v2/kv"
"github.com/tikv/client-go/v2/logutil"
"github.com/tikv/client-go/v2/tikv"
"github.com/tikv/client-go/v2/unionstore"
"github.com/tikv/client-go/v2/util"
"go.uber.org/zap"
)
var scanBatchSize = tikv.ConfigProbe{}.GetScanBatchSize()
type testScanSuite struct {
OneByOneSuite
store *tikv.KVStore
recordPrefix []byte
rowNums []int
ctx context.Context
}
var _ = SerialSuites(&testScanSuite{})
func (s *testScanSuite) SetUpSuite(c *C) {
s.OneByOneSuite.SetUpSuite(c)
s.store = NewTestStore(c)
s.recordPrefix = []byte("prefix")
s.rowNums = append(s.rowNums, 1, scanBatchSize, scanBatchSize+1, scanBatchSize*3)
// Avoid using async commit logic.
s.ctx = context.WithValue(context.Background(), util.SessionID, uint64(0))
}
func (s *testScanSuite) TearDownSuite(c *C) {
txn := s.beginTxn(c)
scanner, err := txn.Iter(s.recordPrefix, nil)
c.Assert(err, IsNil)
c.Assert(scanner, NotNil)
for scanner.Valid() {
k := scanner.Key()
err = txn.Delete(k)
c.Assert(err, IsNil)
scanner.Next()
}
err = txn.Commit(s.ctx)
c.Assert(err, IsNil)
err = s.store.Close()
c.Assert(err, IsNil)
s.OneByOneSuite.TearDownSuite(c)
}
func (s *testScanSuite) beginTxn(c *C) *tikv.KVTxn {
txn, err := s.store.Begin()
c.Assert(err, IsNil)
return txn
}
func (s *testScanSuite) makeKey(i int) []byte {
var key []byte
key = append(key, s.recordPrefix...)
key = append(key, []byte(fmt.Sprintf("%10d", i))...)
return key
}
func (s *testScanSuite) makeValue(i int) []byte {
return []byte(fmt.Sprintf("%d", i))
}
func (s *testScanSuite) TestScan(c *C) {
check := func(c *C, scan unionstore.Iterator, rowNum int, keyOnly bool) {
for i := 0; i < rowNum; i++ {
k := scan.Key()
expectedKey := s.makeKey(i)
if ok := bytes.Equal(k, expectedKey); !ok {
logutil.BgLogger().Error("bytes equal check fail",
zap.Int("i", i),
zap.Int("rowNum", rowNum),
zap.String("obtained key", kv.StrKey(k)),
zap.String("obtained val", kv.StrKey(scan.Value())),
zap.String("expected", kv.StrKey(expectedKey)),
zap.Bool("keyOnly", keyOnly))
}
c.Assert(k, BytesEquals, expectedKey)
if !keyOnly {
v := scan.Value()
c.Assert(v, BytesEquals, s.makeValue(i))
}
// Because newScan return first item without calling scan.Next() just like go-hbase,
// for-loop count will decrease 1.
if i < rowNum-1 {
scan.Next()
}
}
scan.Next()
c.Assert(scan.Valid(), IsFalse)
}
for _, rowNum := range s.rowNums {
txn := s.beginTxn(c)
for i := 0; i < rowNum; i++ {
err := txn.Set(s.makeKey(i), s.makeValue(i))
c.Assert(err, IsNil)
}
err := txn.Commit(s.ctx)
c.Assert(err, IsNil)
mockTableID := int64(999)
if rowNum > 123 {
_, err = s.store.SplitRegions(s.ctx, [][]byte{s.makeKey(123)}, false, &mockTableID)
c.Assert(err, IsNil)
}
if rowNum > 456 {
_, err = s.store.SplitRegions(s.ctx, [][]byte{s.makeKey(456)}, false, &mockTableID)
c.Assert(err, IsNil)
}
txn2 := s.beginTxn(c)
val, err := txn2.Get(context.TODO(), s.makeKey(0))
c.Assert(err, IsNil)
c.Assert(val, BytesEquals, s.makeValue(0))
// Test scan without upperBound
scan, err := txn2.Iter(s.recordPrefix, nil)
c.Assert(err, IsNil)
check(c, scan, rowNum, false)
// Test scan with upperBound
upperBound := rowNum / 2
scan, err = txn2.Iter(s.recordPrefix, s.makeKey(upperBound))
c.Assert(err, IsNil)
check(c, scan, upperBound, false)
txn3 := s.beginTxn(c)
txn3.GetSnapshot().SetKeyOnly(true)
// Test scan without upper bound
scan, err = txn3.Iter(s.recordPrefix, nil)
c.Assert(err, IsNil)
check(c, scan, rowNum, true)
// test scan with upper bound
scan, err = txn3.Iter(s.recordPrefix, s.makeKey(upperBound))
c.Assert(err, IsNil)
check(c, scan, upperBound, true)
// Restore KeyOnly to false
txn3.GetSnapshot().SetKeyOnly(false)
scan, err = txn3.Iter(s.recordPrefix, nil)
c.Assert(err, IsNil)
check(c, scan, rowNum, true)
// test scan with upper bound
scan, err = txn3.Iter(s.recordPrefix, s.makeKey(upperBound))
c.Assert(err, IsNil)
check(c, scan, upperBound, true)
}
}

View File

@ -0,0 +1,245 @@
// 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_test
import (
"context"
"math"
"time"
. "github.com/pingcap/check"
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/store/mockstore/unistore"
tikverr "github.com/tikv/client-go/v2/error"
"github.com/tikv/client-go/v2/mockstore"
"github.com/tikv/client-go/v2/tikv"
)
type testSnapshotFailSuite struct {
OneByOneSuite
store tikv.StoreProbe
}
var _ = SerialSuites(&testSnapshotFailSuite{})
func (s *testSnapshotFailSuite) SetUpSuite(c *C) {
s.OneByOneSuite.SetUpSuite(c)
client, pdClient, cluster, err := unistore.New("")
c.Assert(err, IsNil)
unistore.BootstrapWithSingleStore(cluster)
store, err := tikv.NewTestTiKVStore(fpClient{Client: client}, pdClient, nil, nil, 0)
c.Assert(err, IsNil)
s.store = tikv.StoreProbe{KVStore: store}
}
func (s *testSnapshotFailSuite) TearDownSuite(c *C) {
s.OneByOneSuite.TearDownSuite(c)
}
func (s *testSnapshotFailSuite) cleanup(c *C) {
txn, err := s.store.Begin()
c.Assert(err, IsNil)
iter, err := txn.Iter([]byte(""), []byte(""))
c.Assert(err, IsNil)
for iter.Valid() {
err = txn.Delete(iter.Key())
c.Assert(err, IsNil)
err = iter.Next()
c.Assert(err, IsNil)
}
c.Assert(txn.Commit(context.TODO()), IsNil)
}
func (s *testSnapshotFailSuite) TestBatchGetResponseKeyError(c *C) {
// Meaningless to test with tikv because it has a mock key error
if *mockstore.WithTiKV {
return
}
defer s.cleanup(c)
// Put two KV pairs
txn, err := s.store.Begin()
c.Assert(err, IsNil)
err = txn.Set([]byte("k1"), []byte("v1"))
c.Assert(err, IsNil)
err = txn.Set([]byte("k2"), []byte("v2"))
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
c.Assert(failpoint.Enable("tikvclient/rpcBatchGetResult", `1*return("keyError")`), IsNil)
defer func() {
c.Assert(failpoint.Disable("tikvclient/rpcBatchGetResult"), IsNil)
}()
txn, err = s.store.Begin()
c.Assert(err, IsNil)
res, err := toTiDBTxn(&txn).BatchGet(context.Background(), toTiDBKeys([][]byte{[]byte("k1"), []byte("k2")}))
c.Assert(err, IsNil)
c.Assert(res, DeepEquals, map[string][]byte{"k1": []byte("v1"), "k2": []byte("v2")})
}
func (s *testSnapshotFailSuite) TestScanResponseKeyError(c *C) {
// Meaningless to test with tikv because it has a mock key error
if *mockstore.WithTiKV {
return
}
defer s.cleanup(c)
// Put two KV pairs
txn, err := s.store.Begin()
c.Assert(err, IsNil)
err = txn.Set([]byte("k1"), []byte("v1"))
c.Assert(err, IsNil)
err = txn.Set([]byte("k2"), []byte("v2"))
c.Assert(err, IsNil)
err = txn.Set([]byte("k3"), []byte("v3"))
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
c.Assert(failpoint.Enable("tikvclient/rpcScanResult", `1*return("keyError")`), IsNil)
txn, err = s.store.Begin()
c.Assert(err, IsNil)
iter, err := txn.Iter([]byte("a"), []byte("z"))
c.Assert(err, IsNil)
c.Assert(iter.Key(), DeepEquals, []byte("k1"))
c.Assert(iter.Value(), DeepEquals, []byte("v1"))
c.Assert(iter.Next(), IsNil)
c.Assert(iter.Key(), DeepEquals, []byte("k2"))
c.Assert(iter.Value(), DeepEquals, []byte("v2"))
c.Assert(iter.Next(), IsNil)
c.Assert(iter.Key(), DeepEquals, []byte("k3"))
c.Assert(iter.Value(), DeepEquals, []byte("v3"))
c.Assert(iter.Next(), IsNil)
c.Assert(iter.Valid(), IsFalse)
c.Assert(failpoint.Disable("tikvclient/rpcScanResult"), IsNil)
c.Assert(failpoint.Enable("tikvclient/rpcScanResult", `1*return("keyError")`), IsNil)
txn, err = s.store.Begin()
c.Assert(err, IsNil)
iter, err = txn.Iter([]byte("k2"), []byte("k4"))
c.Assert(err, IsNil)
c.Assert(iter.Key(), DeepEquals, []byte("k2"))
c.Assert(iter.Value(), DeepEquals, []byte("v2"))
c.Assert(iter.Next(), IsNil)
c.Assert(iter.Key(), DeepEquals, []byte("k3"))
c.Assert(iter.Value(), DeepEquals, []byte("v3"))
c.Assert(iter.Next(), IsNil)
c.Assert(iter.Valid(), IsFalse)
c.Assert(failpoint.Disable("tikvclient/rpcScanResult"), IsNil)
}
func (s *testSnapshotFailSuite) TestRetryMaxTsPointGetSkipLock(c *C) {
defer s.cleanup(c)
// Prewrite k1 and k2 with async commit but don't commit them
txn, err := s.store.Begin()
c.Assert(err, IsNil)
err = txn.Set([]byte("k1"), []byte("v1"))
c.Assert(err, IsNil)
err = txn.Set([]byte("k2"), []byte("v2"))
c.Assert(err, IsNil)
txn.SetEnableAsyncCommit(true)
c.Assert(failpoint.Enable("tikvclient/asyncCommitDoNothing", "return"), IsNil)
c.Assert(failpoint.Enable("tikvclient/twoPCShortLockTTL", "return"), IsNil)
committer, err := txn.NewCommitter(1)
c.Assert(err, IsNil)
err = committer.Execute(context.Background())
c.Assert(err, IsNil)
c.Assert(failpoint.Disable("tikvclient/twoPCShortLockTTL"), IsNil)
snapshot := s.store.GetSnapshot(math.MaxUint64)
getCh := make(chan []byte)
go func() {
// Sleep a while to make the TTL of the first txn expire, then we make sure we resolve lock by this get
time.Sleep(200 * time.Millisecond)
c.Assert(failpoint.Enable("tikvclient/beforeSendPointGet", "1*off->pause"), IsNil)
res, err := snapshot.Get(context.Background(), []byte("k2"))
c.Assert(err, IsNil)
getCh <- res
}()
// The get should be blocked by the failpoint. But the lock should have been resolved.
select {
case res := <-getCh:
c.Errorf("too early %s", string(res))
case <-time.After(1 * time.Second):
}
// Prewrite k1 and k2 again without committing them
txn, err = s.store.Begin()
c.Assert(err, IsNil)
txn.SetEnableAsyncCommit(true)
err = txn.Set([]byte("k1"), []byte("v3"))
c.Assert(err, IsNil)
err = txn.Set([]byte("k2"), []byte("v4"))
c.Assert(err, IsNil)
committer, err = txn.NewCommitter(1)
c.Assert(err, IsNil)
err = committer.Execute(context.Background())
c.Assert(err, IsNil)
c.Assert(failpoint.Disable("tikvclient/beforeSendPointGet"), IsNil)
// After disabling the failpoint, the get request should bypass the new locks and read the old result
select {
case res := <-getCh:
c.Assert(res, DeepEquals, []byte("v2"))
case <-time.After(1 * time.Second):
c.Errorf("get timeout")
}
}
func (s *testSnapshotFailSuite) TestRetryPointGetResolveTS(c *C) {
defer s.cleanup(c)
txn, err := s.store.Begin()
c.Assert(err, IsNil)
c.Assert(txn.Set([]byte("k1"), []byte("v1")), IsNil)
err = txn.Set([]byte("k2"), []byte("v2"))
c.Assert(err, IsNil)
txn.SetEnableAsyncCommit(false)
txn.SetEnable1PC(false)
txn.SetCausalConsistency(true)
// Prewrite the lock without committing it
c.Assert(failpoint.Enable("tikvclient/beforeCommit", `pause`), IsNil)
ch := make(chan struct{})
committer, err := txn.NewCommitter(1)
c.Assert(committer.GetPrimaryKey(), DeepEquals, []byte("k1"))
go func() {
c.Assert(err, IsNil)
err = committer.Execute(context.Background())
c.Assert(err, IsNil)
ch <- struct{}{}
}()
// Wait until prewrite finishes
time.Sleep(200 * time.Millisecond)
// Should get nothing with max version, and **not pushing forward minCommitTS** of the primary lock
snapshot := s.store.GetSnapshot(math.MaxUint64)
_, err = snapshot.Get(context.Background(), []byte("k2"))
c.Assert(tikverr.IsErrNotFound(err), IsTrue)
initialCommitTS := committer.GetCommitTS()
c.Assert(failpoint.Disable("tikvclient/beforeCommit"), IsNil)
<-ch
// check the minCommitTS is not pushed forward
snapshot = s.store.GetSnapshot(initialCommitTS)
v, err := snapshot.Get(context.Background(), []byte("k2"))
c.Assert(err, IsNil)
c.Assert(v, DeepEquals, []byte("v2"))
}

View File

@ -0,0 +1,315 @@
// 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_test
import (
"context"
"fmt"
"math"
"sync"
"time"
. "github.com/pingcap/check"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/kvproto/pkg/kvrpcpb"
tikverr "github.com/tikv/client-go/v2/error"
"github.com/tikv/client-go/v2/logutil"
"github.com/tikv/client-go/v2/tikv"
"github.com/tikv/client-go/v2/tikvrpc"
"go.uber.org/zap"
)
type testSnapshotSuite struct {
OneByOneSuite
store tikv.StoreProbe
prefix string
rowNums []int
}
var _ = Suite(&testSnapshotSuite{})
func (s *testSnapshotSuite) SetUpSuite(c *C) {
s.OneByOneSuite.SetUpSuite(c)
s.store = tikv.StoreProbe{KVStore: NewTestStore(c)}
s.prefix = fmt.Sprintf("snapshot_%d", time.Now().Unix())
s.rowNums = append(s.rowNums, 1, 100, 191)
}
func (s *testSnapshotSuite) TearDownSuite(c *C) {
txn := s.beginTxn(c)
scanner, err := txn.Iter(encodeKey(s.prefix, ""), nil)
c.Assert(err, IsNil)
c.Assert(scanner, NotNil)
for scanner.Valid() {
k := scanner.Key()
err = txn.Delete(k)
c.Assert(err, IsNil)
scanner.Next()
}
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
err = s.store.Close()
c.Assert(err, IsNil)
s.OneByOneSuite.TearDownSuite(c)
}
func (s *testSnapshotSuite) beginTxn(c *C) tikv.TxnProbe {
txn, err := s.store.Begin()
c.Assert(err, IsNil)
return txn
}
func (s *testSnapshotSuite) checkAll(keys [][]byte, c *C) {
txn := s.beginTxn(c)
snapshot := txn.GetSnapshot()
m, err := snapshot.BatchGet(context.Background(), keys)
c.Assert(err, IsNil)
scan, err := txn.Iter(encodeKey(s.prefix, ""), nil)
c.Assert(err, IsNil)
cnt := 0
for scan.Valid() {
cnt++
k := scan.Key()
v := scan.Value()
v2, ok := m[string(k)]
c.Assert(ok, IsTrue, Commentf("key: %q", k))
c.Assert(v, BytesEquals, v2)
scan.Next()
}
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
c.Assert(m, HasLen, cnt)
}
func (s *testSnapshotSuite) deleteKeys(keys [][]byte, c *C) {
txn := s.beginTxn(c)
for _, k := range keys {
err := txn.Delete(k)
c.Assert(err, IsNil)
}
err := txn.Commit(context.Background())
c.Assert(err, IsNil)
}
func (s *testSnapshotSuite) TestBatchGet(c *C) {
for _, rowNum := range s.rowNums {
logutil.BgLogger().Debug("test BatchGet",
zap.Int("length", rowNum))
txn := s.beginTxn(c)
for i := 0; i < rowNum; i++ {
k := encodeKey(s.prefix, s08d("key", i))
err := txn.Set(k, valueBytes(i))
c.Assert(err, IsNil)
}
err := txn.Commit(context.Background())
c.Assert(err, IsNil)
keys := makeKeys(rowNum, s.prefix)
s.checkAll(keys, c)
s.deleteKeys(keys, c)
}
}
type contextKey string
func (s *testSnapshotSuite) TestSnapshotCache(c *C) {
txn := s.beginTxn(c)
c.Assert(txn.Set([]byte("x"), []byte("x")), IsNil)
c.Assert(txn.Delete([]byte("y")), IsNil) // store data is affected by other tests.
c.Assert(txn.Commit(context.Background()), IsNil)
txn = s.beginTxn(c)
snapshot := txn.GetSnapshot()
_, err := snapshot.BatchGet(context.Background(), [][]byte{[]byte("x"), []byte("y")})
c.Assert(err, IsNil)
c.Assert(failpoint.Enable("tikvclient/snapshot-get-cache-fail", `return(true)`), IsNil)
ctx := context.WithValue(context.Background(), contextKey("TestSnapshotCache"), true)
_, err = snapshot.Get(ctx, []byte("x"))
c.Assert(err, IsNil)
_, err = snapshot.Get(ctx, []byte("y"))
c.Assert(tikverr.IsErrNotFound(err), IsTrue)
c.Assert(failpoint.Disable("tikvclient/snapshot-get-cache-fail"), IsNil)
}
func (s *testSnapshotSuite) TestBatchGetNotExist(c *C) {
for _, rowNum := range s.rowNums {
logutil.BgLogger().Debug("test BatchGetNotExist",
zap.Int("length", rowNum))
txn := s.beginTxn(c)
for i := 0; i < rowNum; i++ {
k := encodeKey(s.prefix, s08d("key", i))
err := txn.Set(k, valueBytes(i))
c.Assert(err, IsNil)
}
err := txn.Commit(context.Background())
c.Assert(err, IsNil)
keys := makeKeys(rowNum, s.prefix)
keys = append(keys, []byte("noSuchKey"))
s.checkAll(keys, c)
s.deleteKeys(keys, c)
}
}
func makeKeys(rowNum int, prefix string) [][]byte {
keys := make([][]byte, 0, rowNum)
for i := 0; i < rowNum; i++ {
k := encodeKey(prefix, s08d("key", i))
keys = append(keys, k)
}
return keys
}
func (s *testSnapshotSuite) TestSkipLargeTxnLock(c *C) {
x := []byte("x_key_TestSkipLargeTxnLock")
y := []byte("y_key_TestSkipLargeTxnLock")
txn := s.beginTxn(c)
c.Assert(txn.Set(x, []byte("x")), IsNil)
c.Assert(txn.Set(y, []byte("y")), IsNil)
ctx := context.Background()
committer, err := txn.NewCommitter(0)
c.Assert(err, IsNil)
committer.SetLockTTL(3000)
c.Assert(committer.PrewriteAllMutations(ctx), IsNil)
txn1 := s.beginTxn(c)
// txn1 is not blocked by txn in the large txn protocol.
_, err = txn1.Get(ctx, x)
c.Assert(tikverr.IsErrNotFound(errors.Trace(err)), IsTrue)
res, err := toTiDBTxn(&txn1).BatchGet(ctx, toTiDBKeys([][]byte{x, y, []byte("z")}))
c.Assert(err, IsNil)
c.Assert(res, HasLen, 0)
// Commit txn, check the final commit ts is pushed.
committer.SetCommitTS(txn.StartTS() + 1)
c.Assert(committer.CommitMutations(ctx), IsNil)
status, err := s.store.GetLockResolver().GetTxnStatus(txn.StartTS(), 0, x)
c.Assert(err, IsNil)
c.Assert(status.IsCommitted(), IsTrue)
c.Assert(status.CommitTS(), Greater, txn1.StartTS())
}
func (s *testSnapshotSuite) TestPointGetSkipTxnLock(c *C) {
x := []byte("x_key_TestPointGetSkipTxnLock")
y := []byte("y_key_TestPointGetSkipTxnLock")
txn := s.beginTxn(c)
c.Assert(txn.Set(x, []byte("x")), IsNil)
c.Assert(txn.Set(y, []byte("y")), IsNil)
ctx := context.Background()
committer, err := txn.NewCommitter(0)
c.Assert(err, IsNil)
committer.SetLockTTL(3000)
c.Assert(committer.PrewriteAllMutations(ctx), IsNil)
snapshot := s.store.GetSnapshot(math.MaxUint64)
start := time.Now()
c.Assert(committer.GetPrimaryKey(), BytesEquals, x)
// Point get secondary key. Shouldn't be blocked by the lock and read old data.
_, err = snapshot.Get(ctx, y)
c.Assert(tikverr.IsErrNotFound(errors.Trace(err)), IsTrue)
c.Assert(time.Since(start), Less, 500*time.Millisecond)
// Commit the primary key
committer.SetCommitTS(txn.StartTS() + 1)
committer.CommitMutations(ctx)
snapshot = s.store.GetSnapshot(math.MaxUint64)
start = time.Now()
// Point get secondary key. Should read committed data.
value, err := snapshot.Get(ctx, y)
c.Assert(err, IsNil)
c.Assert(value, BytesEquals, []byte("y"))
c.Assert(time.Since(start), Less, 500*time.Millisecond)
}
func (s *testSnapshotSuite) TestSnapshotThreadSafe(c *C) {
txn := s.beginTxn(c)
key := []byte("key_test_snapshot_threadsafe")
c.Assert(txn.Set(key, []byte("x")), IsNil)
ctx := context.Background()
err := txn.Commit(context.Background())
c.Assert(err, IsNil)
snapshot := s.store.GetSnapshot(math.MaxUint64)
var wg sync.WaitGroup
wg.Add(5)
for i := 0; i < 5; i++ {
go func() {
for i := 0; i < 30; i++ {
_, err := snapshot.Get(ctx, key)
c.Assert(err, IsNil)
_, err = snapshot.BatchGet(ctx, [][]byte{key, []byte("key_not_exist")})
c.Assert(err, IsNil)
}
wg.Done()
}()
}
wg.Wait()
}
func (s *testSnapshotSuite) TestSnapshotRuntimeStats(c *C) {
reqStats := tikv.NewRegionRequestRuntimeStats()
tikv.RecordRegionRequestRuntimeStats(reqStats.Stats, tikvrpc.CmdGet, time.Second)
tikv.RecordRegionRequestRuntimeStats(reqStats.Stats, tikvrpc.CmdGet, time.Millisecond)
snapshot := s.store.GetSnapshot(0)
snapshot.SetRuntimeStats(&tikv.SnapshotRuntimeStats{})
snapshot.MergeRegionRequestStats(reqStats.Stats)
snapshot.MergeRegionRequestStats(reqStats.Stats)
bo := tikv.NewBackofferWithVars(context.Background(), 2000, nil)
err := bo.BackoffWithMaxSleepTxnLockFast(30, errors.New("test"))
c.Assert(err, IsNil)
snapshot.RecordBackoffInfo(bo)
snapshot.RecordBackoffInfo(bo)
expect := "Get:{num_rpc:4, total_time:2s},txnLockFast_backoff:{num:2, total_time:60ms}"
c.Assert(snapshot.FormatStats(), Equals, expect)
detail := &kvrpcpb.ExecDetailsV2{
TimeDetail: &kvrpcpb.TimeDetail{
WaitWallTimeMs: 100,
ProcessWallTimeMs: 100,
},
ScanDetailV2: &kvrpcpb.ScanDetailV2{
ProcessedVersions: 10,
TotalVersions: 15,
RocksdbBlockReadCount: 20,
RocksdbBlockReadByte: 15,
RocksdbDeleteSkippedCount: 5,
RocksdbKeySkippedCount: 1,
RocksdbBlockCacheHitCount: 10,
},
}
snapshot.MergeExecDetail(detail)
expect = "Get:{num_rpc:4, total_time:2s},txnLockFast_backoff:{num:2, total_time:60ms}, " +
"total_process_time: 100ms, total_wait_time: 100ms, " +
"scan_detail: {total_process_keys: 10, " +
"total_keys: 15, " +
"rocksdb: {delete_skipped_count: 5, " +
"key_skipped_count: 1, " +
"block: {cache_hit_count: 10, read_count: 20, read_byte: 15 Bytes}}}"
c.Assert(snapshot.FormatStats(), Equals, expect)
snapshot.MergeExecDetail(detail)
expect = "Get:{num_rpc:4, total_time:2s},txnLockFast_backoff:{num:2, total_time:60ms}, " +
"total_process_time: 200ms, total_wait_time: 200ms, " +
"scan_detail: {total_process_keys: 20, " +
"total_keys: 30, " +
"rocksdb: {delete_skipped_count: 10, " +
"key_skipped_count: 2, " +
"block: {cache_hit_count: 20, read_count: 40, read_byte: 30 Bytes}}}"
c.Assert(snapshot.FormatStats(), Equals, expect)
}

View File

@ -0,0 +1,250 @@
// 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_test
import (
"context"
"sync"
. "github.com/pingcap/check"
"github.com/pingcap/errors"
"github.com/pingcap/kvproto/pkg/metapb"
"github.com/pingcap/kvproto/pkg/pdpb"
"github.com/pingcap/tidb/store/mockstore/mockcopr"
"github.com/tikv/client-go/v2/mockstore/cluster"
"github.com/tikv/client-go/v2/mockstore/mocktikv"
"github.com/tikv/client-go/v2/tikv"
pd "github.com/tikv/pd/client"
)
type testSplitSuite struct {
OneByOneSuite
cluster cluster.Cluster
store tikv.StoreProbe
bo *tikv.Backoffer
}
var _ = Suite(&testSplitSuite{})
func (s *testSplitSuite) SetUpTest(c *C) {
client, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("", mockcopr.NewCoprRPCHandler())
c.Assert(err, IsNil)
mocktikv.BootstrapWithSingleStore(cluster)
s.cluster = cluster
store, err := tikv.NewTestTiKVStore(client, pdClient, nil, nil, 0)
c.Assert(err, IsNil)
// TODO: make this possible
// store, err := mockstore.NewMockStore(
// mockstore.WithClusterInspector(func(c cluster.Cluster) {
// mockstore.BootstrapWithSingleStore(c)
// s.cluster = c
// }),
// )
// c.Assert(err, IsNil)
s.store = tikv.StoreProbe{KVStore: store}
s.bo = tikv.NewBackofferWithVars(context.Background(), 5000, nil)
}
func (s *testSplitSuite) begin(c *C) tikv.TxnProbe {
txn, err := s.store.Begin()
c.Assert(err, IsNil)
return txn
}
func (s *testSplitSuite) split(c *C, regionID uint64, key []byte) {
newRegionID, peerID := s.cluster.AllocID(), s.cluster.AllocID()
s.cluster.Split(regionID, newRegionID, key, []uint64{peerID}, peerID)
}
func (s *testSplitSuite) TestSplitBatchGet(c *C) {
loc, err := s.store.GetRegionCache().LocateKey(s.bo, []byte("a"))
c.Assert(err, IsNil)
txn := s.begin(c)
keys := [][]byte{{'a'}, {'b'}, {'c'}}
_, region, err := s.store.GetRegionCache().GroupKeysByRegion(s.bo, keys, nil)
c.Assert(err, IsNil)
s.split(c, loc.Region.GetID(), []byte("b"))
s.store.GetRegionCache().InvalidateCachedRegion(loc.Region)
// mocktikv will panic if it meets a not-in-region key.
err = txn.BatchGetSingleRegion(s.bo, region, keys, func([]byte, []byte) {})
c.Assert(err, IsNil)
}
func (s *testSplitSuite) TestStaleEpoch(c *C) {
mockPDClient := &mockPDClient{client: s.store.GetRegionCache().PDClient()}
s.store.SetRegionCachePDClient(mockPDClient)
loc, err := s.store.GetRegionCache().LocateKey(s.bo, []byte("a"))
c.Assert(err, IsNil)
txn := s.begin(c)
err = txn.Set([]byte("a"), []byte("a"))
c.Assert(err, IsNil)
err = txn.Set([]byte("c"), []byte("c"))
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
// Initiate a split and disable the PD client. If it still works, the
// new region is updated from kvrpc.
s.split(c, loc.Region.GetID(), []byte("b"))
mockPDClient.disable()
txn = s.begin(c)
_, err = txn.Get(context.TODO(), []byte("a"))
c.Assert(err, IsNil)
_, err = txn.Get(context.TODO(), []byte("c"))
c.Assert(err, IsNil)
}
var errStopped = errors.New("stopped")
type mockPDClient struct {
sync.RWMutex
client pd.Client
stop bool
}
func (c *mockPDClient) disable() {
c.Lock()
defer c.Unlock()
c.stop = true
}
func (c *mockPDClient) GetAllMembers(ctx context.Context) ([]*pdpb.Member, error) {
return nil, nil
}
func (c *mockPDClient) GetClusterID(context.Context) uint64 {
return 1
}
func (c *mockPDClient) GetTS(ctx context.Context) (int64, int64, error) {
c.RLock()
defer c.RUnlock()
if c.stop {
return 0, 0, errors.Trace(errStopped)
}
return c.client.GetTS(ctx)
}
func (c *mockPDClient) GetLocalTS(ctx context.Context, dcLocation string) (int64, int64, error) {
return c.GetTS(ctx)
}
func (c *mockPDClient) GetTSAsync(ctx context.Context) pd.TSFuture {
return nil
}
func (c *mockPDClient) GetLocalTSAsync(ctx context.Context, dcLocation string) pd.TSFuture {
return nil
}
func (c *mockPDClient) GetRegion(ctx context.Context, key []byte) (*pd.Region, error) {
c.RLock()
defer c.RUnlock()
if c.stop {
return nil, errors.Trace(errStopped)
}
return c.client.GetRegion(ctx, key)
}
func (c *mockPDClient) GetRegionFromMember(ctx context.Context, key []byte, memberURLs []string) (*pd.Region, error) {
return nil, nil
}
func (c *mockPDClient) GetPrevRegion(ctx context.Context, key []byte) (*pd.Region, error) {
c.RLock()
defer c.RUnlock()
if c.stop {
return nil, errors.Trace(errStopped)
}
return c.client.GetPrevRegion(ctx, key)
}
func (c *mockPDClient) GetRegionByID(ctx context.Context, regionID uint64) (*pd.Region, error) {
c.RLock()
defer c.RUnlock()
if c.stop {
return nil, errors.Trace(errStopped)
}
return c.client.GetRegionByID(ctx, regionID)
}
func (c *mockPDClient) ScanRegions(ctx context.Context, startKey []byte, endKey []byte, limit int) ([]*pd.Region, error) {
c.RLock()
defer c.RUnlock()
if c.stop {
return nil, errors.Trace(errStopped)
}
return c.client.ScanRegions(ctx, startKey, endKey, limit)
}
func (c *mockPDClient) GetStore(ctx context.Context, storeID uint64) (*metapb.Store, error) {
c.RLock()
defer c.RUnlock()
if c.stop {
return nil, errors.Trace(errStopped)
}
return c.client.GetStore(ctx, storeID)
}
func (c *mockPDClient) GetAllStores(ctx context.Context, opts ...pd.GetStoreOption) ([]*metapb.Store, error) {
c.RLock()
defer c.Unlock()
if c.stop {
return nil, errors.Trace(errStopped)
}
return c.client.GetAllStores(ctx)
}
func (c *mockPDClient) UpdateGCSafePoint(ctx context.Context, safePoint uint64) (uint64, error) {
panic("unimplemented")
}
func (c *mockPDClient) UpdateServiceGCSafePoint(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) {
panic("unimplemented")
}
func (c *mockPDClient) Close() {}
func (c *mockPDClient) ScatterRegion(ctx context.Context, regionID uint64) error {
return nil
}
func (c *mockPDClient) ScatterRegions(ctx context.Context, regionsID []uint64, opts ...pd.RegionsOption) (*pdpb.ScatterRegionResponse, error) {
return nil, nil
}
func (c *mockPDClient) SplitRegions(ctx context.Context, splitKeys [][]byte, opts ...pd.RegionsOption) (*pdpb.SplitRegionsResponse, error) {
return nil, nil
}
func (c *mockPDClient) GetOperator(ctx context.Context, regionID uint64) (*pdpb.GetOperatorResponse, error) {
return &pdpb.GetOperatorResponse{Status: pdpb.OperatorStatus_SUCCESS}, nil
}
func (c *mockPDClient) GetLeaderAddr() string { return "mockpd" }

View File

@ -0,0 +1,53 @@
// Copyright 2017 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_test
import (
"context"
"sync"
"time"
. "github.com/pingcap/check"
"github.com/pingcap/failpoint"
)
func (s *testStoreSerialSuite) TestFailBusyServerKV(c *C) {
txn, err := s.store.Begin()
c.Assert(err, IsNil)
err = txn.Set([]byte("key"), []byte("value"))
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
var wg sync.WaitGroup
wg.Add(2)
c.Assert(failpoint.Enable("tikvclient/rpcServerBusy", `return(true)`), IsNil)
go func() {
defer wg.Done()
time.Sleep(time.Millisecond * 100)
c.Assert(failpoint.Disable("tikvclient/rpcServerBusy"), IsNil)
}()
go func() {
defer wg.Done()
txn, err := s.store.Begin()
c.Assert(err, IsNil)
val, err := txn.Get(context.TODO(), []byte("key"))
c.Assert(err, IsNil)
c.Assert(val, BytesEquals, []byte("value"))
}()
wg.Wait()
}

View File

@ -0,0 +1,155 @@
// 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_test
import (
"context"
"sync"
"time"
. "github.com/pingcap/check"
"github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/tikv/client-go/v2/oracle"
"github.com/tikv/client-go/v2/oracle/oracles"
"github.com/tikv/client-go/v2/tikv"
"github.com/tikv/client-go/v2/tikvrpc"
)
type testStoreSuite struct {
testStoreSuiteBase
}
type testStoreSerialSuite struct {
testStoreSuiteBase
}
type testStoreSuiteBase struct {
OneByOneSuite
store tikv.StoreProbe
}
var _ = Suite(&testStoreSuite{})
var _ = SerialSuites(&testStoreSerialSuite{})
func (s *testStoreSuiteBase) SetUpTest(c *C) {
s.store = tikv.StoreProbe{KVStore: NewTestStore(c)}
}
func (s *testStoreSuiteBase) TearDownTest(c *C) {
c.Assert(s.store.Close(), IsNil)
}
func (s *testStoreSuite) TestOracle(c *C) {
o := &oracles.MockOracle{}
s.store.SetOracle(o)
ctx := context.Background()
t1, err := s.store.GetTimestampWithRetry(tikv.NewBackofferWithVars(ctx, 100, nil), oracle.GlobalTxnScope)
c.Assert(err, IsNil)
t2, err := s.store.GetTimestampWithRetry(tikv.NewBackofferWithVars(ctx, 100, nil), oracle.GlobalTxnScope)
c.Assert(err, IsNil)
c.Assert(t1, Less, t2)
t1, err = o.GetLowResolutionTimestamp(ctx, &oracle.Option{})
c.Assert(err, IsNil)
t2, err = o.GetLowResolutionTimestamp(ctx, &oracle.Option{})
c.Assert(err, IsNil)
c.Assert(t1, Less, t2)
f := o.GetLowResolutionTimestampAsync(ctx, &oracle.Option{})
c.Assert(f, NotNil)
_ = o.UntilExpired(0, 0, &oracle.Option{})
// Check retry.
var wg sync.WaitGroup
wg.Add(2)
o.Disable()
go func() {
defer wg.Done()
time.Sleep(time.Millisecond * 100)
o.Enable()
}()
go func() {
defer wg.Done()
t3, err := s.store.GetTimestampWithRetry(tikv.NewBackofferWithVars(ctx, 5000, nil), oracle.GlobalTxnScope)
c.Assert(err, IsNil)
c.Assert(t2, Less, t3)
expired := s.store.GetOracle().IsExpired(t2, 50, &oracle.Option{})
c.Assert(expired, IsTrue)
}()
wg.Wait()
}
type checkRequestClient struct {
tikv.Client
priority kvrpcpb.CommandPri
}
func (c *checkRequestClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) {
resp, err := c.Client.SendRequest(ctx, addr, req, timeout)
if c.priority != req.Priority {
if resp.Resp != nil {
if getResp, ok := resp.Resp.(*kvrpcpb.GetResponse); ok {
getResp.Error = &kvrpcpb.KeyError{
Abort: "request check error",
}
}
}
}
return resp, err
}
func (s *testStoreSuite) TestRequestPriority(c *C) {
client := &checkRequestClient{
Client: s.store.GetTiKVClient(),
}
s.store.SetTiKVClient(client)
// Cover 2PC commit.
txn, err := s.store.Begin()
c.Assert(err, IsNil)
client.priority = kvrpcpb.CommandPri_High
txn.SetPriority(tikv.PriorityHigh)
err = txn.Set([]byte("key"), []byte("value"))
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
// Cover the basic Get request.
txn, err = s.store.Begin()
c.Assert(err, IsNil)
client.priority = kvrpcpb.CommandPri_Low
txn.SetPriority(tikv.PriorityLow)
_, err = txn.Get(context.TODO(), []byte("key"))
c.Assert(err, IsNil)
// A counter example.
client.priority = kvrpcpb.CommandPri_Low
txn.SetPriority(tikv.PriorityNormal)
_, err = txn.Get(context.TODO(), []byte("key"))
// err is translated to "try again later" by backoffer, so doesn't check error value here.
c.Assert(err, NotNil)
// Cover Seek request.
client.priority = kvrpcpb.CommandPri_High
txn.SetPriority(tikv.PriorityHigh)
iter, err := txn.Iter([]byte("key"), nil)
c.Assert(err, IsNil)
for iter.Valid() {
c.Assert(iter.Next(), IsNil)
}
iter.Close()
}

View File

@ -0,0 +1,93 @@
// 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.
// +build !race
package tikv_test
import (
"context"
. "github.com/pingcap/check"
"github.com/tikv/client-go/v2/kv"
"github.com/tikv/client-go/v2/mockstore"
"github.com/tikv/client-go/v2/tikv"
)
func (s *testTiclientSuite) TestSplitRegionIn2PC(c *C) {
if *mockstore.WithTiKV {
c.Skip("scatter will timeout with single node TiKV")
}
config := tikv.ConfigProbe{}
const preSplitThresholdInTest = 500
old := config.LoadPreSplitDetectThreshold()
defer config.StorePreSplitDetectThreshold(old)
config.StorePreSplitDetectThreshold(preSplitThresholdInTest)
old = config.LoadPreSplitSizeThreshold()
defer config.StorePreSplitSizeThreshold(old)
config.StorePreSplitSizeThreshold(5000)
bo := tikv.NewBackofferWithVars(context.Background(), 1, nil)
checkKeyRegion := func(bo *tikv.Backoffer, start, end []byte, checker Checker) {
// Check regions after split.
loc1, err := s.store.GetRegionCache().LocateKey(bo, start)
c.Assert(err, IsNil)
loc2, err := s.store.GetRegionCache().LocateKey(bo, end)
c.Assert(err, IsNil)
c.Assert(loc1.Region.GetID(), checker, loc2.Region.GetID())
}
mode := []string{"optimistic", "pessimistic"}
var (
startKey []byte
endKey []byte
)
ctx := context.Background()
for _, m := range mode {
if m == "optimistic" {
startKey = encodeKey(s.prefix, s08d("key", 0))
endKey = encodeKey(s.prefix, s08d("key", preSplitThresholdInTest))
} else {
startKey = encodeKey(s.prefix, s08d("pkey", 0))
endKey = encodeKey(s.prefix, s08d("pkey", preSplitThresholdInTest))
}
// Check before test.
checkKeyRegion(bo, startKey, endKey, Equals)
txn := s.beginTxn(c)
if m == "pessimistic" {
txn.SetPessimistic(true)
lockCtx := &kv.LockCtx{}
lockCtx.ForUpdateTS = txn.StartTS()
keys := make([][]byte, 0, preSplitThresholdInTest)
for i := 0; i < preSplitThresholdInTest; i++ {
keys = append(keys, encodeKey(s.prefix, s08d("pkey", i)))
}
err := txn.LockKeys(ctx, lockCtx, keys...)
c.Assert(err, IsNil)
checkKeyRegion(bo, startKey, endKey, Not(Equals))
}
var err error
for i := 0; i < preSplitThresholdInTest; i++ {
if m == "optimistic" {
err = txn.Set(encodeKey(s.prefix, s08d("key", i)), valueBytes(i))
} else {
err = txn.Set(encodeKey(s.prefix, s08d("pkey", i)), valueBytes(i))
}
c.Assert(err, IsNil)
}
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
// Check region split after test.
checkKeyRegion(bo, startKey, endKey, Not(Equals))
}
}

View File

@ -0,0 +1,131 @@
// Copyright 2021 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_test
import (
"context"
"fmt"
"time"
. "github.com/pingcap/check"
"github.com/pingcap/tidb/kv"
tikvstore "github.com/tikv/client-go/v2/kv"
"github.com/tikv/client-go/v2/tikv"
)
type testTiclientSuite struct {
OneByOneSuite
store *tikv.KVStore
// prefix is prefix of each key in this test. It is used for table isolation,
// or it may pollute other data.
prefix string
}
var _ = Suite(&testTiclientSuite{})
func (s *testTiclientSuite) SetUpSuite(c *C) {
s.OneByOneSuite.SetUpSuite(c)
s.store = NewTestStore(c)
s.prefix = fmt.Sprintf("ticlient_%d", time.Now().Unix())
}
func (s *testTiclientSuite) TearDownSuite(c *C) {
// Clean all data, or it may pollute other data.
txn := s.beginTxn(c)
scanner, err := txn.Iter(encodeKey(s.prefix, ""), nil)
c.Assert(err, IsNil)
c.Assert(scanner, NotNil)
for scanner.Valid() {
k := scanner.Key()
err = txn.Delete(k)
c.Assert(err, IsNil)
scanner.Next()
}
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
err = s.store.Close()
c.Assert(err, IsNil)
s.OneByOneSuite.TearDownSuite(c)
}
func (s *testTiclientSuite) beginTxn(c *C) *tikv.KVTxn {
txn, err := s.store.Begin()
c.Assert(err, IsNil)
return txn
}
func (s *testTiclientSuite) TestSingleKey(c *C) {
txn := s.beginTxn(c)
err := txn.Set(encodeKey(s.prefix, "key"), []byte("value"))
c.Assert(err, IsNil)
err = txn.LockKeys(context.Background(), new(tikvstore.LockCtx), encodeKey(s.prefix, "key"))
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
txn = s.beginTxn(c)
val, err := txn.Get(context.TODO(), encodeKey(s.prefix, "key"))
c.Assert(err, IsNil)
c.Assert(val, BytesEquals, []byte("value"))
txn = s.beginTxn(c)
err = txn.Delete(encodeKey(s.prefix, "key"))
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
}
func (s *testTiclientSuite) TestMultiKeys(c *C) {
const keyNum = 100
txn := s.beginTxn(c)
for i := 0; i < keyNum; i++ {
err := txn.Set(encodeKey(s.prefix, s08d("key", i)), valueBytes(i))
c.Assert(err, IsNil)
}
err := txn.Commit(context.Background())
c.Assert(err, IsNil)
txn = s.beginTxn(c)
for i := 0; i < keyNum; i++ {
val, err1 := txn.Get(context.TODO(), encodeKey(s.prefix, s08d("key", i)))
c.Assert(err1, IsNil)
c.Assert(val, BytesEquals, valueBytes(i))
}
txn = s.beginTxn(c)
for i := 0; i < keyNum; i++ {
err = txn.Delete(encodeKey(s.prefix, s08d("key", i)))
c.Assert(err, IsNil)
}
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
}
func (s *testTiclientSuite) TestNotExist(c *C) {
txn := s.beginTxn(c)
_, err := txn.Get(context.TODO(), encodeKey(s.prefix, "noSuchKey"))
c.Assert(err, NotNil)
}
func (s *testTiclientSuite) TestLargeRequest(c *C) {
largeValue := make([]byte, 9*1024*1024) // 9M value.
txn := s.beginTxn(c)
txn.GetUnionStore().SetEntrySizeLimit(1024*1024, 100*1024*1024)
err := txn.Set([]byte("key"), largeValue)
c.Assert(err, NotNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
c.Assert(kv.IsTxnRetryableError(err), IsFalse)
}

View File

@ -0,0 +1,110 @@
// Copyright 2021 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_test
import (
"context"
"flag"
"fmt"
"strings"
"unsafe"
. "github.com/pingcap/check"
"github.com/pingcap/errors"
"github.com/pingcap/tidb/kv"
txndriver "github.com/pingcap/tidb/store/driver/txn"
"github.com/pingcap/tidb/store/mockstore/unistore"
"github.com/tikv/client-go/v2/config"
"github.com/tikv/client-go/v2/mockstore"
"github.com/tikv/client-go/v2/tikv"
"github.com/tikv/client-go/v2/util/codec"
pd "github.com/tikv/pd/client"
)
var (
pdAddrs = flag.String("pd-addrs", "127.0.0.1:2379", "pd addrs")
)
// NewTestStore creates a KVStore for testing purpose.
func NewTestStore(c *C) *tikv.KVStore {
if !flag.Parsed() {
flag.Parse()
}
if *mockstore.WithTiKV {
addrs := strings.Split(*pdAddrs, ",")
pdClient, err := pd.NewClient(addrs, pd.SecurityOption{})
c.Assert(err, IsNil)
var securityConfig config.Security
tlsConfig, err := securityConfig.ToTLSConfig()
c.Assert(err, IsNil)
spKV, err := tikv.NewEtcdSafePointKV(addrs, tlsConfig)
c.Assert(err, IsNil)
store, err := tikv.NewKVStore("test-store", &tikv.CodecPDClient{Client: pdClient}, spKV, tikv.NewRPCClient(securityConfig))
c.Assert(err, IsNil)
err = clearStorage(store)
c.Assert(err, IsNil)
return store
}
client, pdClient, cluster, err := unistore.New("")
c.Assert(err, IsNil)
unistore.BootstrapWithSingleStore(cluster)
store, err := tikv.NewTestTiKVStore(client, pdClient, nil, nil, 0)
c.Assert(err, IsNil)
return store
}
func clearStorage(store *tikv.KVStore) error {
txn, err := store.Begin()
if err != nil {
return errors.Trace(err)
}
iter, err := txn.Iter(nil, nil)
if err != nil {
return errors.Trace(err)
}
for iter.Valid() {
txn.Delete(iter.Key())
if err := iter.Next(); err != nil {
return errors.Trace(err)
}
}
return txn.Commit(context.Background())
}
// OneByOneSuite is a suite, When with-tikv flag is true, there is only one storage, so the test suite have to run one by one.
type OneByOneSuite = mockstore.OneByOneSuite
func encodeKey(prefix, s string) []byte {
return codec.EncodeBytes(nil, []byte(fmt.Sprintf("%s_%s", prefix, s)))
}
func valueBytes(n int) []byte {
return []byte(fmt.Sprintf("value%d", n))
}
// s08d is for returning format string "%s%08d" to keep string sorted.
// e.g.: "0002" < "0011", otherwise "2" > "11"
func s08d(prefix string, n int) string {
return fmt.Sprintf("%s%08d", prefix, n)
}
func toTiDBTxn(txn *tikv.TxnProbe) kv.Transaction {
return txndriver.NewTiKVTxn(txn.KVTxn)
}
func toTiDBKeys(keys [][]byte) []kv.Key {
kvKeys := *(*[]kv.Key)(unsafe.Pointer(&keys))
return kvKeys
}