Skip to content

Commit

Permalink
log_backup: fix the retry over wrapped errors (#35218)
Browse files Browse the repository at this point in the history
close #35204
  • Loading branch information
YuJuncen authored Jun 14, 2022
1 parent b0b1c7c commit 837e264
Show file tree
Hide file tree
Showing 2 changed files with 28 additions and 18 deletions.
36 changes: 18 additions & 18 deletions br/pkg/restore/import_retry.go
Original file line number Diff line number Diff line change
Expand Up @@ -151,16 +151,16 @@ func (o *OverRegionsInRangeController) runInRegion(ctx context.Context, f Region
if !result.OK() {
o.onError(ctx, result, region)
switch result.StrategyForRetry() {
case giveUp:
case StrategyGiveUp:
logutil.CL(ctx).Warn("unexpected error, should stop to retry", logutil.ShortError(&result), logutil.Region(region.Region))
return false, o.errors
case fromThisRegion:
case StrategyFromThisRegion:
logutil.CL(ctx).Warn("retry for region", logutil.Region(region.Region), logutil.ShortError(&result))
if !o.handleInRegionError(ctx, result, region) {
return false, o.Run(ctx, f)
}
return o.runInRegion(ctx, f, region)
case fromStart:
case StrategyFromStart:
logutil.CL(ctx).Warn("retry for execution over regions", logutil.ShortError(&result))
// TODO: make a backoffer considering more about the error info,
// instead of ingore the result and retry.
Expand Down Expand Up @@ -196,48 +196,48 @@ func RPCResultOK() RPCResult {
return RPCResult{}
}

type retryStrategy int
type RetryStrategy int

const (
giveUp retryStrategy = iota
fromThisRegion
fromStart
StrategyGiveUp RetryStrategy = iota
StrategyFromThisRegion
StrategyFromStart
)

func (r *RPCResult) StrategyForRetry() retryStrategy {
func (r *RPCResult) StrategyForRetry() RetryStrategy {
if r.Err != nil {
return r.StrategyForRetryGoError()
}
return r.StrategyForRetryStoreError()
}

func (r *RPCResult) StrategyForRetryStoreError() retryStrategy {
func (r *RPCResult) StrategyForRetryStoreError() RetryStrategy {
if r.StoreError == nil && r.ImportError == "" {
return giveUp
return StrategyGiveUp
}

if r.StoreError.GetServerIsBusy() != nil ||
r.StoreError.GetRegionNotInitialized() != nil ||
r.StoreError.GetNotLeader() != nil {
return fromThisRegion
return StrategyFromThisRegion
}

return fromStart
return StrategyFromStart
}

func (r *RPCResult) StrategyForRetryGoError() retryStrategy {
func (r *RPCResult) StrategyForRetryGoError() RetryStrategy {
if r.Err == nil {
return giveUp
return StrategyGiveUp
}

if gRPCErr, ok := status.FromError(r.Err); ok {
// we should unwrap the error or we cannot get the write gRPC status.
if gRPCErr, ok := status.FromError(errors.Cause(r.Err)); ok {
switch gRPCErr.Code() {
case codes.Unavailable, codes.Aborted, codes.ResourceExhausted:
return fromThisRegion
return StrategyFromThisRegion
}
}

return giveUp
return StrategyGiveUp
}

func (r *RPCResult) Error() string {
Expand Down
10 changes: 10 additions & 0 deletions br/pkg/restore/import_retry_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ import (
"testing"
"time"

"github.com/pingcap/errors"
"github.com/pingcap/kvproto/pkg/errorpb"
"github.com/pingcap/kvproto/pkg/import_sstpb"
"github.com/pingcap/kvproto/pkg/metapb"
Expand All @@ -17,6 +18,8 @@ import (
"github.com/pingcap/tidb/store/pdtypes"
"github.com/pingcap/tidb/util/codec"
"github.com/stretchr/testify/require"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
)

func assertDecode(t *testing.T, key []byte) []byte {
Expand Down Expand Up @@ -306,3 +309,10 @@ func TestRetryBackoff(t *testing.T) {
require.Equal(t, 2*time.Millisecond, rs.ExponentialBackoff())
require.NoError(t, err)
}

func TestWrappedError(t *testing.T) {
result := restore.RPCResultFromError(errors.Trace(status.Error(codes.Unavailable, "the server is slacking. ><=·>")))
require.Equal(t, result.StrategyForRetry(), restore.StrategyFromThisRegion)
result = restore.RPCResultFromError(errors.Trace(status.Error(codes.Unknown, "the server said something hard to understand")))
require.Equal(t, result.StrategyForRetry(), restore.StrategyGiveUp)
}

0 comments on commit 837e264

Please sign in to comment.