Skip to content

Commit

Permalink
store/tikv: export failpoints
Browse files Browse the repository at this point in the history
Signed-off-by: Andy Lok <andylokandy@hotmail.com>
  • Loading branch information
andylokandy committed Jan 27, 2021
1 parent e1e4ad1 commit be09e93
Show file tree
Hide file tree
Showing 5 changed files with 41 additions and 15 deletions.
12 changes: 6 additions & 6 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2388,7 +2388,7 @@ func (s *testSerialSuite) TestBatchPointGetRepeatableRead(c *C) {
}

func (s *testSerialSuite) TestSplitRegionTimeout(c *C) {
c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/MockSplitRegionTimeout", `return(true)`), IsNil)
c.Assert(tikv.MockSplitRegionTimeout.Enable(`return(true)`), IsNil)
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
Expand All @@ -2397,22 +2397,22 @@ func (s *testSerialSuite) TestSplitRegionTimeout(c *C) {
tk.MustExec(`set @@tidb_wait_split_region_timeout=1`)
// result 0 0 means split 0 region and 0 region finish scatter regions before timeout.
tk.MustQuery(`split table t between (0) and (10000) regions 10`).Check(testkit.Rows("0 0"))
c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/MockSplitRegionTimeout"), IsNil)
tikv.MockSplitRegionTimeout.Disable()

// Test scatter regions timeout.
c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/MockScatterRegionTimeout", `return(true)`), IsNil)
c.Assert(tikv.MockScatterRegionTimeout.Enable(`return(true)`), IsNil)
tk.MustQuery(`split table t between (0) and (10000) regions 10`).Check(testkit.Rows("10 1"))
c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/MockScatterRegionTimeout"), IsNil)
tikv.MockScatterRegionTimeout.Disable()

// Test pre-split with timeout.
tk.MustExec("drop table if exists t")
tk.MustExec("set @@global.tidb_scatter_region=1;")
c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/MockScatterRegionTimeout", `return(true)`), IsNil)
c.Assert(tikv.MockScatterRegionTimeout.Enable(`return(true)`), IsNil)
atomic.StoreUint32(&ddl.EnableSplitTableRegion, 1)
start := time.Now()
tk.MustExec("create table t (a int, b int) partition by hash(a) partitions 5;")
c.Assert(time.Since(start).Seconds(), Less, 10.0)
c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/MockScatterRegionTimeout"), IsNil)
tikv.MockScatterRegionTimeout.Disable()
}

func (s *testSuiteP2) TestRow(c *C) {
Expand Down
4 changes: 2 additions & 2 deletions session/session_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2721,9 +2721,9 @@ func (s *testSessionSerialSuite) TestTxnRetryErrMsg(c *C) {
tk1.MustExec("begin")
tk2.MustExec("update no_retry set id = id + 1")
tk1.MustExec("update no_retry set id = id + 1")
c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/ErrMockRetryableOnly", `return(true)`), IsNil)
c.Assert(tikv.MockRetryableErrorResp.Enable(`return(true)`), IsNil)
_, err := tk1.Se.Execute(context.Background(), "commit")
c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/ErrMockRetryableOnly"), IsNil)
tikv.MockRetryableErrorResp.Disable()
c.Assert(err, NotNil)
c.Assert(kv.ErrTxnRetryable.Equal(err), IsTrue, Commentf("error: %s", err))
c.Assert(strings.Contains(err.Error(), "mock retryable error"), IsTrue, Commentf("error: %s", err))
Expand Down
27 changes: 27 additions & 0 deletions store/tikv/failpoint.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
// 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

import (
"github.com/pingcap/failpoint"
)

var (
// MockRetryableErrorResp mocks an retryable error while processing response
MockRetryableErrorResp failpoint.Failpoint
// MockScatterRegionTimeout mocks timeout when trying to scatter region
MockScatterRegionTimeout failpoint.Failpoint
// MockSplitRegionTimeout mocks timeout when trying to split region
MockSplitRegionTimeout failpoint.Failpoint
)
4 changes: 2 additions & 2 deletions store/tikv/snapshot.go
Original file line number Diff line number Diff line change
Expand Up @@ -587,12 +587,12 @@ func extractLockFromKeyErr(keyErr *pb.KeyError) (*Lock, error) {
}

func extractKeyErr(keyErr *pb.KeyError) error {
failpoint.Inject("ErrMockRetryableOnly", func(val failpoint.Value) {
if val, err := MockRetryableErrorResp.Eval(); err == nil {
if val.(bool) {
keyErr.Conflict = nil
keyErr.Retryable = "mock retryable error"
}
})
}

if keyErr.Conflict != nil {
return newWriteConflictError(keyErr.Conflict)
Expand Down
9 changes: 4 additions & 5 deletions store/tikv/split_region.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import (
"sync/atomic"

"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pingcap/kvproto/pkg/metapb"
"github.com/pingcap/kvproto/pkg/pdpb"
Expand Down Expand Up @@ -108,13 +107,13 @@ func (s *tikvStore) splitBatchRegionsReq(bo *Backoffer, keys [][]byte, scatter b
}

func (s *tikvStore) batchSendSingleRegion(bo *Backoffer, batch batch, scatter bool, tableID *int64) singleBatchResp {
failpoint.Inject("MockSplitRegionTimeout", func(val failpoint.Value) {
if val, err := MockSplitRegionTimeout.Eval(); err == nil {
if val.(bool) {
if _, ok := bo.ctx.Deadline(); ok {
<-bo.ctx.Done()
}
}
})
}

req := tikvrpc.NewRequest(tikvrpc.CmdSplitRegion, &kvrpcpb.SplitRegionRequest{
SplitKeys: batch.keys,
Expand Down Expand Up @@ -217,11 +216,11 @@ func (s *tikvStore) scatterRegion(bo *Backoffer, regionID uint64, tableID *int64
}
_, err := s.pdClient.ScatterRegions(bo.ctx, []uint64{regionID}, opts...)

failpoint.Inject("MockScatterRegionTimeout", func(val failpoint.Value) {
if val, err2 := MockScatterRegionTimeout.Eval(); err2 == nil {
if val.(bool) {
err = ErrPDServerTimeout
}
})
}

if err == nil {
break
Expand Down

0 comments on commit be09e93

Please sign in to comment.