Skip to content

Commit

Permalink
br: fix the missing retry for pd batch scan error (#33420)
Browse files Browse the repository at this point in the history
close #33419
  • Loading branch information
3pointer committed Mar 25, 2022
1 parent 6517ee3 commit 9129590
Show file tree
Hide file tree
Showing 3 changed files with 25 additions and 3 deletions.
13 changes: 10 additions & 3 deletions br/pkg/restore/split.go
Original file line number Diff line number Diff line change
Expand Up @@ -438,11 +438,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 @@ -458,7 +465,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 @@ -39,6 +39,7 @@ type TestClient struct {
supportBatchScatter bool

scattered map[uint64]bool
InjectErr bool
}

func NewTestClient(
Expand Down Expand Up @@ -215,6 +216,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
10 changes: 10 additions & 0 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 @@ -228,6 +229,7 @@ func TestPaginateScanRegion(t *testing.T) {
var batch []*restore.RegionInfo
_, err := restore.PaginateScanRegion(ctx, NewTestClient(stores, regionMap, 0), []byte{}, []byte{}, 3)
require.Error(t, err)
require.True(t, berrors.ErrPDBatchScanRegion.Equal(err))
require.Regexp(t, ".*scan region return empty result.*", err.Error())

regionMap, regions = makeRegions(1)
Expand Down Expand Up @@ -268,12 +270,20 @@ func TestPaginateScanRegion(t *testing.T) {

_, err = restore.PaginateScanRegion(ctx, NewTestClient(stores, regionMap, 0), []byte{2}, []byte{1}, 3)
require.Error(t, err)
require.True(t, berrors.ErrRestoreInvalidRange.Equal(err))
require.Regexp(t, ".*startKey >= endKey.*", err.Error())

tc := NewTestClient(stores, regionMap, 0)
tc.InjectErr = true
_, err = restore.PaginateScanRegion(ctx, tc, regions[1].Region.EndKey, regions[5].Region.EndKey, 3)
require.Error(t, err)
require.Regexp(t, ".*mock scan error.*", err.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)
require.Error(t, err)
require.True(t, berrors.ErrPDBatchScanRegion.Equal(err))
require.Regexp(t, ".*region endKey not equal to next region startKey.*", err.Error())

}

0 comments on commit 9129590

Please sign in to comment.