Skip to content

Commit

Permalink
store/tikv: fix unstable tests in snapshot_fail_test.go (#22916)
Browse files Browse the repository at this point in the history
  • Loading branch information
sticnarf authored Feb 24, 2021
1 parent 806097c commit 1970a91
Show file tree
Hide file tree
Showing 3 changed files with 33 additions and 8 deletions.
11 changes: 7 additions & 4 deletions session/session_fail_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -111,10 +111,13 @@ func (s *testSessionSerialSuite) TestAutoCommitNeedNotLinearizability(c *C) {
tk.MustExec(`create table t1 (c int)`)

c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/getMinCommitTSFromTSO", `panic`), IsNil)
defer failpoint.Disable("github.com/pingcap/tidb/store/tikv/getMinCommitTSFromTSO")
defer func() {
c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/getMinCommitTSFromTSO"), IsNil)
}()

c.Assert(tk.Se.GetSessionVars().SetSystemVar("tidb_enable_async_commit", "1"), IsNil)
c.Assert(tk.Se.GetSessionVars().SetSystemVar("tidb_guarantee_linearizability", "1"), IsNil)

tk.Se.GetSessionVars().SetSystemVar("tidb_enable_async_commit", "1")
tk.Se.GetSessionVars().SetSystemVar("tidb_guarantee_linearizability", "1")
// Auto-commit transactions don't need to get minCommitTS from TSO
tk.MustExec("INSERT INTO t1 VALUES (1)")

Expand All @@ -141,7 +144,7 @@ func (s *testSessionSerialSuite) TestAutoCommitNeedNotLinearizability(c *C) {

// Same for 1PC
tk.MustExec("set autocommit = 1")
tk.Se.GetSessionVars().SetSystemVar("tidb_enable_1pc", "1")
c.Assert(tk.Se.GetSessionVars().SetSystemVar("tidb_enable_1pc", "1"), IsNil)
tk.MustExec("INSERT INTO t1 VALUES (4)")

tk.MustExec("BEGIN")
Expand Down
3 changes: 2 additions & 1 deletion store/gcworker/gc_worker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -924,10 +924,11 @@ func (s *testGCWorkerSuite) TestResolveLockRangeMeetRegionEnlargeCausedByRegionM
mCluster := s.cluster.(*mocktikv.Cluster)
mCluster.Merge(s.initRegion.regionID, region2)
regionMeta, _ := mCluster.GetRegion(s.initRegion.regionID)
s.tikvStore.GetRegionCache().OnRegionEpochNotMatch(
err := s.tikvStore.GetRegionCache().OnRegionEpochNotMatch(
tikv.NewNoopBackoff(context.Background()),
&tikv.RPCContext{Region: regionID, Store: &tikv.Store{}},
[]*metapb.Region{regionMeta})
c.Assert(err, IsNil)
// also let region1 contains all 4 locks
s.gcWorker.testingKnobs.scanLocks = func(key []byte, regionID uint64) []*tikv.Lock {
if regionID == s.initRegion.regionID {
Expand Down
27 changes: 24 additions & 3 deletions store/tikv/snapshot_fail_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,11 +43,27 @@ 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(kv.Key(""), kv.Key(""))
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 *WithTiKV {
return
}
defer s.cleanup(c)

// Put two KV pairs
txn, err := s.store.Begin()
c.Assert(err, IsNil)
Expand Down Expand Up @@ -75,6 +91,8 @@ func (s *testSnapshotFailSuite) TestScanResponseKeyError(c *C) {
if *WithTiKV {
return
}
defer s.cleanup(c)

// Put two KV pairs
txn, err := s.store.Begin()
c.Assert(err, IsNil)
Expand Down Expand Up @@ -120,6 +138,8 @@ func (s *testSnapshotFailSuite) TestScanResponseKeyError(c *C) {
}

func (s *testSnapshotFailSuite) TestRetryPointGetWithTS(c *C) {
defer s.cleanup(c)

snapshot := s.store.GetSnapshot(kv.MaxVersion)
c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/snapshotGetTSAsync", `pause`), IsNil)
ch := make(chan error)
Expand All @@ -138,13 +158,14 @@ func (s *testSnapshotFailSuite) TestRetryPointGetWithTS(c *C) {
c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/asyncCommitDoNothing", `return`), IsNil)
committer, err := newTwoPhaseCommitterWithInit(txn.(*tikvTxn), 1)
c.Assert(err, IsNil)
// Sets its minCommitTS to a large value, so the lock can be actually ignored.
committer.minCommitTS = committer.startTS + (1 << 28)
// Sets its minCommitTS to one second later, so the lock will be ignored by point get.
committer.minCommitTS = committer.startTS + (1000 << 18)
err = committer.execute(context.Background())
c.Assert(err, IsNil)
c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/asyncCommitDoNothing"), IsNil)
c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/snapshotGetTSAsync"), IsNil)

err = <-ch
c.Assert(err, ErrorMatches, ".*key not exist")

c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/asyncCommitDoNothing"), IsNil)
}

0 comments on commit 1970a91

Please sign in to comment.