operator: make an infinity retry for connecting to store (#52177)
close pingcap/tidb#52175
This commit is contained in:
@ -180,9 +180,8 @@ type RetryAndSplitRequestEnv struct {
|
||||
}
|
||||
|
||||
func (r RetryAndSplitRequestEnv) ConnectToStore(ctx context.Context, storeID uint64) (PrepareClient, error) {
|
||||
// Retry for about 2 minutes.
|
||||
rs := utils.InitialRetryState(12, 10*time.Second, 10*time.Second)
|
||||
bo := utils.Backoffer(&rs)
|
||||
rs := utils.ConstantBackoff(10 * time.Second)
|
||||
bo := utils.Backoffer(rs)
|
||||
if r.GetBackoffer != nil {
|
||||
bo = r.GetBackoffer()
|
||||
}
|
||||
|
||||
@ -77,7 +77,7 @@ go_test(
|
||||
],
|
||||
embed = [":utils"],
|
||||
flaky = True,
|
||||
shard_count = 32,
|
||||
shard_count = 33,
|
||||
deps = [
|
||||
"//br/pkg/errors",
|
||||
"//pkg/kv",
|
||||
|
||||
@ -6,6 +6,7 @@ import (
|
||||
"context"
|
||||
"database/sql"
|
||||
"io"
|
||||
"math"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
@ -62,6 +63,20 @@ func isGRPCCancel(err error) bool {
|
||||
return false
|
||||
}
|
||||
|
||||
// ConstantBackoff is a backoffer that retry forever until success.
|
||||
type ConstantBackoff time.Duration
|
||||
|
||||
// NextBackoff returns a duration to wait before retrying again
|
||||
func (c ConstantBackoff) NextBackoff(err error) time.Duration {
|
||||
return time.Duration(c)
|
||||
}
|
||||
|
||||
// Attempt returns the remain attempt times
|
||||
func (c ConstantBackoff) Attempt() int {
|
||||
// A large enough value. Also still safe for arithmetic operations (won't easily overflow).
|
||||
return math.MaxInt16
|
||||
}
|
||||
|
||||
// RetryState is the mutable state needed for retrying.
|
||||
// It likes the `utils.Backoffer`, but more fundamental:
|
||||
// this only control the backoff time and knows nothing about what error happens.
|
||||
|
||||
@ -4,7 +4,9 @@ package utils_test
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"math"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
@ -213,3 +215,43 @@ func TestNewBackupSSTBackofferWithCancel(t *testing.T) {
|
||||
context.Canceled,
|
||||
}, multierr.Errors(err))
|
||||
}
|
||||
|
||||
func TestConstantBackoff(t *testing.T) {
|
||||
backedOff := func(t *testing.T) {
|
||||
backoffer := utils.ConstantBackoff(10 * time.Millisecond)
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
i := 0
|
||||
ch := make(chan error)
|
||||
|
||||
go func() {
|
||||
_, err := utils.WithRetryV2(ctx, backoffer, func(ctx context.Context) (struct{}, error) {
|
||||
i += 1
|
||||
return struct{}{}, fmt.Errorf("%d times, no meaning", i)
|
||||
})
|
||||
ch <- err
|
||||
}()
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
cancel()
|
||||
require.Error(t, <-ch)
|
||||
// Make sure we have backed off.
|
||||
require.Less(t, i, 20)
|
||||
}
|
||||
|
||||
infRetry := func(t *testing.T) {
|
||||
backoffer := utils.ConstantBackoff(0)
|
||||
ctx := context.Background()
|
||||
i := math.MaxInt16
|
||||
|
||||
_, err := utils.WithRetryV2(ctx, backoffer, func(ctx context.Context) (struct{}, error) {
|
||||
i -= 1
|
||||
if i == 0 {
|
||||
return struct{}{}, nil
|
||||
}
|
||||
return struct{}{}, fmt.Errorf("try %d more times", i)
|
||||
})
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
t.Run("backedOff", backedOff)
|
||||
t.Run("infRetry", infRetry)
|
||||
}
|
||||
|
||||
@ -238,6 +238,7 @@ func WithRetryV2[T any](
|
||||
allErrors = multierr.Append(allErrors, err)
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
// allErrors must not be `nil` here, so ignore the context error.
|
||||
return *new(T), allErrors
|
||||
case <-time.After(backoffer.NextBackoff(err)):
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user