Skip to content

Commit

Permalink
br: fix the missing retry for pd batch scan error (#33420) (#33469)
Browse files Browse the repository at this point in the history
close #33419
  • Loading branch information
ti-srebot authored Jun 17, 2022
1 parent 8c7825f commit 60c9ee7
Show file tree
Hide file tree
Showing 5 changed files with 27 additions and 9 deletions.
2 changes: 0 additions & 2 deletions .github/workflows/br_compatible_test.yml
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,6 @@ on:
push:
branches:
- master
- 'release-[0-9].[0-9]*'
paths:
- 'br/**'
- '!**.html'
Expand All @@ -17,7 +16,6 @@ on:
pull_request:
branches:
- master
- 'release-[0-9].[0-9]*'
paths:
- 'br/**'
- '!**.html'
Expand Down
2 changes: 0 additions & 2 deletions .github/workflows/compile_br.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,6 @@ on:
push:
branches:
- master
- 'release-[0-9].[0-9]*'
paths:
- 'br/**'
- '!**.html'
Expand All @@ -16,7 +15,6 @@ on:
pull_request:
branches:
- master
- 'release-[0-9].[0-9]*'
paths:
- 'br/**'
- '!**.html'
Expand Down
13 changes: 10 additions & 3 deletions br/pkg/restore/split.go
Original file line number Diff line number Diff line change
Expand Up @@ -393,11 +393,18 @@ func PaginateScanRegion(
}

var regions []*RegionInfo
err := utils.WithRetry(ctx, func() error {
var err error
// we don't need to return multierr. since there only 3 times retry.
// in most case 3 times retry have the same error. so we just return the last error.
// actually we'd better remove all multierr in br/lightning.
// because it's not easy to check multierr equals normal error.
// see https://github.com/pingcap/tidb/issues/33419.
_ = utils.WithRetry(ctx, func() error {
regions = []*RegionInfo{}
scanStartKey := startKey
for {
batch, err := client.ScanRegions(ctx, scanStartKey, endKey, limit)
var batch []*RegionInfo
batch, err = client.ScanRegions(ctx, scanStartKey, endKey, limit)
if err != nil {
return errors.Trace(err)
}
Expand All @@ -413,7 +420,7 @@ func PaginateScanRegion(
break
}
}
if err := checkRegionConsistency(startKey, endKey, regions); err != nil {
if err = checkRegionConsistency(startKey, endKey, regions); err != nil {
log.Warn("failed to scan region, retrying", logutil.ShortError(err))
return err
}
Expand Down
5 changes: 5 additions & 0 deletions br/pkg/restore/split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ type TestClient struct {
injectInScatter func(*restore.RegionInfo) error

scattered map[uint64]bool
InjectErr bool
}

func NewTestClient(
Expand Down Expand Up @@ -180,6 +181,10 @@ func (c *TestClient) GetOperator(ctx context.Context, regionID uint64) (*pdpb.Ge
}

func (c *TestClient) ScanRegions(ctx context.Context, key, endKey []byte, limit int) ([]*restore.RegionInfo, error) {
if c.InjectErr {
return nil, errors.New("mock scan error")
}

infos := c.regionsInfo.ScanRange(key, endKey, limit)
regions := make([]*restore.RegionInfo, 0, len(infos))
for _, info := range infos {
Expand Down
14 changes: 12 additions & 2 deletions br/pkg/restore/util_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ import (
backuppb "github.com/pingcap/kvproto/pkg/brpb"
"github.com/pingcap/kvproto/pkg/import_sstpb"
"github.com/pingcap/kvproto/pkg/metapb"
berrors "github.com/pingcap/tidb/br/pkg/errors"
"github.com/pingcap/tidb/br/pkg/restore"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/util/codec"
Expand Down Expand Up @@ -223,9 +224,11 @@ func (s *testRestoreUtilSuite) TestPaginateScanRegion(c *C) {

ctx := context.Background()
regionMap := make(map[uint64]*restore.RegionInfo)
regions := []*restore.RegionInfo{}
var regions []*restore.RegionInfo
var batch []*restore.RegionInfo
batch, err := restore.PaginateScanRegion(ctx, NewTestClient(stores, regionMap, 0), []byte{}, []byte{}, 3)
c.Assert(err, ErrorMatches, ".*scan region return empty result.*")
c.Assert(berrors.ErrPDBatchScanRegion.Equal(err), IsTrue)

regionMap, regions = makeRegions(1)
batch, err = restore.PaginateScanRegion(ctx, NewTestClient(stores, regionMap, 0), []byte{}, []byte{}, 3)
Expand Down Expand Up @@ -265,11 +268,18 @@ func (s *testRestoreUtilSuite) TestPaginateScanRegion(c *C) {

_, err = restore.PaginateScanRegion(ctx, NewTestClient(stores, regionMap, 0), []byte{2}, []byte{1}, 3)
c.Assert(err, ErrorMatches, ".*startKey >= endKey.*")
c.Assert(berrors.ErrRestoreInvalidRange.Equal(err), IsTrue)

tc := NewTestClient(stores, regionMap, 0)
tc.InjectErr = true
_, err = restore.PaginateScanRegion(ctx, tc, regions[1].Region.EndKey, regions[5].Region.EndKey, 3)
c.Assert(err, NotNil)
c.Assert(err, ErrorMatches, ".*mock scan error.*")

// make the regionMap losing some region, this will cause scan region check fails
delete(regionMap, uint64(3))
_, err = restore.PaginateScanRegion(
ctx, NewTestClient(stores, regionMap, 0), regions[1].Region.EndKey, regions[5].Region.EndKey, 3)
c.Assert(err, ErrorMatches, ".*region endKey not equal to next region startKey.*")

c.Assert(berrors.ErrPDBatchScanRegion.Equal(err), IsTrue)
}

0 comments on commit 60c9ee7

Please sign in to comment.