operator: make an infinity retry for connecting to store (#52177)

close pingcap/tidb#52175
This commit is contained in:
山岚
2024-04-03 15:43:47 +08:00
committed by GitHub
parent 970962bdbc
commit cc3c1baf30
5 changed files with 61 additions and 4 deletions

View File

@ -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()
}

View File

@ -77,7 +77,7 @@ go_test(
],
embed = [":utils"],
flaky = True,
shard_count = 32,
shard_count = 33,
deps = [
"//br/pkg/errors",
"//pkg/kv",

View File

@ -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.

View File

@ -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)
}

View File

@ -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)):
}