client: add WithConnectParams to configure connection backoff and timeout (#2960)

* Implement missing pieces for connection backoff.

Spec can be found here:
https://github.com/grpc/grpc/blob/master/doc/connection-backoff.md

Summary of changes:
* Added a new type (marked experimental), ConnectParams, which contains
  the knobs defined in the spec (except for minConnectTimeout).
* Added a new API (marked experimental), WithConnectParams() to return a
  DialOption to dial with the provided parameters.
* Added new fields to the implementation of the exponential backoff in
  internal/backoff which mirror the ones in ConnectParams.
* Marked existing APIs WithBackoffMaxDelay() and WithBackoffConfig() as
  deprecated.
* Added a default exponential backoff implementation, for easy use of
  internal callers.

Added a new backoff package which defines the backoff configuration
options, and is used by both the grpc package and the internal/backoff
package. This allows us to have all backoff related options in a
separate package.
This commit is contained in:
Easwar Swaminathan 2019-10-03 16:47:13 -07:00 коммит произвёл GitHub
Родитель fb2e5cdc85
Коммит 31911ed09e
Не найден ключ, соответствующий данной подписи
Идентификатор ключа GPG: 4AEE18F83AFDEB23
11 изменённых файлов: 176 добавлений и 79 удалений

Просмотреть файл

@ -23,16 +23,36 @@ package grpc
import ( import (
"time" "time"
"google.golang.org/grpc/backoff"
) )
// DefaultBackoffConfig uses values specified for backoff in // DefaultBackoffConfig uses values specified for backoff in
// https://github.com/grpc/grpc/blob/master/doc/connection-backoff.md. // https://github.com/grpc/grpc/blob/master/doc/connection-backoff.md.
//
// Deprecated: use ConnectParams instead. Will be supported throughout 1.x.
var DefaultBackoffConfig = BackoffConfig{ var DefaultBackoffConfig = BackoffConfig{
MaxDelay: 120 * time.Second, MaxDelay: 120 * time.Second,
} }
// BackoffConfig defines the parameters for the default gRPC backoff strategy. // BackoffConfig defines the parameters for the default gRPC backoff strategy.
//
// Deprecated: use ConnectParams instead. Will be supported throughout 1.x.
type BackoffConfig struct { type BackoffConfig struct {
// MaxDelay is the upper bound of backoff delay. // MaxDelay is the upper bound of backoff delay.
MaxDelay time.Duration MaxDelay time.Duration
} }
// ConnectParams defines the parameters for connecting and retrying. Users are
// encouraged to use this instead of the BackoffConfig type defined above. See
// here for more details:
// https://github.com/grpc/grpc/blob/master/doc/connection-backoff.md.
//
// This API is EXPERIMENTAL.
type ConnectParams struct {
// Backoff specifies the configuration options for connection backoff.
Backoff backoff.Config
// MinConnectTimeout is the minimum amount of time we are willing to give a
// connection to complete.
MinConnectTimeout time.Duration
}

52
backoff/backoff.go Normal file
Просмотреть файл

@ -0,0 +1,52 @@
/*
*
* Copyright 2019 gRPC authors.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*
*/
// Package backoff provides configuration options for backoff.
//
// More details can be found at:
// https://github.com/grpc/grpc/blob/master/doc/connection-backoff.md.
//
// All APIs in this package are experimental.
package backoff
import "time"
// Config defines the configuration options for backoff.
type Config struct {
// BaseDelay is the amount of time to backoff after the first failure.
BaseDelay time.Duration
// Multiplier is the factor with which to multiply backoffs after a
// failed retry. Should ideally be greater than 1.
Multiplier float64
// Jitter is the factor with which backoffs are randomized.
Jitter float64
// MaxDelay is the upper bound of backoff delay.
MaxDelay time.Duration
}
// DefaultConfig is a backoff configuration with the default values specfied
// at https://github.com/grpc/grpc/blob/master/doc/connection-backoff.md.
//
// This should be useful for callers who want to configure backoff with
// non-default values only for a subset of the options.
var DefaultConfig = Config{
BaseDelay: 1.0 * time.Second,
Multiplier: 1.6,
Jitter: 0.2,
MaxDelay: 120 * time.Second,
}

Просмотреть файл

@ -49,19 +49,7 @@ const (
grpclbName = "grpclb" grpclbName = "grpclb"
) )
var ( var errServerTerminatedConnection = errors.New("grpclb: failed to recv server list: server terminated connection")
// defaultBackoffConfig configures the backoff strategy that's used when the
// init handshake in the RPC is unsuccessful. It's not for the clientconn
// reconnect backoff.
//
// It has the same value as the default grpc.DefaultBackoffConfig.
//
// TODO: make backoff configurable.
defaultBackoffConfig = backoff.Exponential{
MaxDelay: 120 * time.Second,
}
errServerTerminatedConnection = errors.New("grpclb: failed to recv server list: server terminated connection")
)
func convertDuration(d *durationpb.Duration) time.Duration { func convertDuration(d *durationpb.Duration) time.Duration {
if d == nil { if d == nil {
@ -155,7 +143,7 @@ func (b *lbBuilder) Build(cc balancer.ClientConn, opt balancer.BuildOptions) bal
scStates: make(map[balancer.SubConn]connectivity.State), scStates: make(map[balancer.SubConn]connectivity.State),
picker: &errPicker{err: balancer.ErrNoSubConnAvailable}, picker: &errPicker{err: balancer.ErrNoSubConnAvailable},
clientStats: newRPCStats(), clientStats: newRPCStats(),
backoff: defaultBackoffConfig, // TODO: make backoff configurable. backoff: backoff.DefaultExponential, // TODO: make backoff configurable.
} }
var err error var err error

Просмотреть файл

@ -235,9 +235,7 @@ func DialContext(ctx context.Context, target string, opts ...DialOption) (conn *
} }
} }
if cc.dopts.bs == nil { if cc.dopts.bs == nil {
cc.dopts.bs = backoff.Exponential{ cc.dopts.bs = backoff.DefaultExponential
MaxDelay: DefaultBackoffConfig.MaxDelay,
}
} }
if cc.dopts.resolverBuilder == nil { if cc.dopts.resolverBuilder == nil {
// Only try to parse target when resolver builder is not already set. // Only try to parse target when resolver builder is not already set.

Просмотреть файл

@ -30,9 +30,10 @@ import (
"time" "time"
"golang.org/x/net/http2" "golang.org/x/net/http2"
"google.golang.org/grpc/backoff"
"google.golang.org/grpc/connectivity" "google.golang.org/grpc/connectivity"
"google.golang.org/grpc/credentials" "google.golang.org/grpc/credentials"
"google.golang.org/grpc/internal/backoff" internalbackoff "google.golang.org/grpc/internal/backoff"
"google.golang.org/grpc/internal/transport" "google.golang.org/grpc/internal/transport"
"google.golang.org/grpc/keepalive" "google.golang.org/grpc/keepalive"
"google.golang.org/grpc/naming" "google.golang.org/grpc/naming"
@ -655,22 +656,39 @@ func (s) TestCredentialsMisuse(t *testing.T) {
} }
func (s) TestWithBackoffConfigDefault(t *testing.T) { func (s) TestWithBackoffConfigDefault(t *testing.T) {
testBackoffConfigSet(t, &DefaultBackoffConfig) testBackoffConfigSet(t, internalbackoff.DefaultExponential)
} }
func (s) TestWithBackoffConfig(t *testing.T) { func (s) TestWithBackoffConfig(t *testing.T) {
b := BackoffConfig{MaxDelay: DefaultBackoffConfig.MaxDelay / 2} b := BackoffConfig{MaxDelay: DefaultBackoffConfig.MaxDelay / 2}
expected := b bc := backoff.DefaultConfig
testBackoffConfigSet(t, &expected, WithBackoffConfig(b)) bc.MaxDelay = b.MaxDelay
wantBackoff := internalbackoff.Exponential{Config: bc}
testBackoffConfigSet(t, wantBackoff, WithBackoffConfig(b))
} }
func (s) TestWithBackoffMaxDelay(t *testing.T) { func (s) TestWithBackoffMaxDelay(t *testing.T) {
md := DefaultBackoffConfig.MaxDelay / 2 md := DefaultBackoffConfig.MaxDelay / 2
expected := BackoffConfig{MaxDelay: md} bc := backoff.DefaultConfig
testBackoffConfigSet(t, &expected, WithBackoffMaxDelay(md)) bc.MaxDelay = md
wantBackoff := internalbackoff.Exponential{Config: bc}
testBackoffConfigSet(t, wantBackoff, WithBackoffMaxDelay(md))
} }
func testBackoffConfigSet(t *testing.T, expected *BackoffConfig, opts ...DialOption) { func (s) TestWithConnectParams(t *testing.T) {
bd := 2 * time.Second
mltpr := 2.0
jitter := 0.0
bc := backoff.Config{BaseDelay: bd, Multiplier: mltpr, Jitter: jitter}
crt := ConnectParams{Backoff: bc}
// MaxDelay is not set in the ConnectParams. So it should not be set on
// internalbackoff.Exponential as well.
wantBackoff := internalbackoff.Exponential{Config: bc}
testBackoffConfigSet(t, wantBackoff, WithConnectParams(crt))
}
func testBackoffConfigSet(t *testing.T, wantBackoff internalbackoff.Exponential, opts ...DialOption) {
opts = append(opts, WithInsecure()) opts = append(opts, WithInsecure())
conn, err := Dial("passthrough:///foo:80", opts...) conn, err := Dial("passthrough:///foo:80", opts...)
if err != nil { if err != nil {
@ -682,16 +700,27 @@ func testBackoffConfigSet(t *testing.T, expected *BackoffConfig, opts ...DialOpt
t.Fatalf("backoff config not set") t.Fatalf("backoff config not set")
} }
actual, ok := conn.dopts.bs.(backoff.Exponential) gotBackoff, ok := conn.dopts.bs.(internalbackoff.Exponential)
if !ok { if !ok {
t.Fatalf("unexpected type of backoff config: %#v", conn.dopts.bs) t.Fatalf("unexpected type of backoff config: %#v", conn.dopts.bs)
} }
expectedValue := backoff.Exponential{ if gotBackoff != wantBackoff {
MaxDelay: expected.MaxDelay, t.Fatalf("unexpected backoff config on connection: %v, want %v", gotBackoff, wantBackoff)
} }
if actual != expectedValue { }
t.Fatalf("unexpected backoff config on connection: %v, want %v", actual, expected)
func (s) TestConnectParamsWithMinConnectTimeout(t *testing.T) {
// Default value specified for minConnectTimeout in the spec is 20 seconds.
mct := 1 * time.Minute
conn, err := Dial("passthrough:///foo:80", WithInsecure(), WithConnectParams(ConnectParams{MinConnectTimeout: mct}))
if err != nil {
t.Fatalf("unexpected error dialing connection: %v", err)
}
defer conn.Close()
if got := conn.dopts.minConnectTimeout(); got != mct {
t.Errorf("unexpect minConnectTimeout on the connection: %v, want %v", got, mct)
} }
} }

Просмотреть файл

@ -24,11 +24,12 @@ import (
"net" "net"
"time" "time"
"google.golang.org/grpc/backoff"
"google.golang.org/grpc/balancer" "google.golang.org/grpc/balancer"
"google.golang.org/grpc/credentials" "google.golang.org/grpc/credentials"
"google.golang.org/grpc/grpclog" "google.golang.org/grpc/grpclog"
"google.golang.org/grpc/internal" "google.golang.org/grpc/internal"
"google.golang.org/grpc/internal/backoff" internalbackoff "google.golang.org/grpc/internal/backoff"
"google.golang.org/grpc/internal/envconfig" "google.golang.org/grpc/internal/envconfig"
"google.golang.org/grpc/internal/transport" "google.golang.org/grpc/internal/transport"
"google.golang.org/grpc/keepalive" "google.golang.org/grpc/keepalive"
@ -47,7 +48,7 @@ type dialOptions struct {
cp Compressor cp Compressor
dc Decompressor dc Decompressor
bs backoff.Strategy bs internalbackoff.Strategy
block bool block bool
insecure bool insecure bool
timeout time.Duration timeout time.Duration
@ -246,8 +247,28 @@ func WithServiceConfig(c <-chan ServiceConfig) DialOption {
}) })
} }
// WithConnectParams configures the dialer to use the provided ConnectParams.
//
// The backoff configuration specified as part of the ConnectParams overrides
// all defaults specified in
// https://github.com/grpc/grpc/blob/master/doc/connection-backoff.md. Consider
// using the backoff.DefaultConfig as a base, in cases where you want to
// override only a subset of the backoff configuration.
//
// This API is EXPERIMENTAL.
func WithConnectParams(p ConnectParams) DialOption {
return newFuncDialOption(func(o *dialOptions) {
o.bs = internalbackoff.Exponential{Config: p.Backoff}
o.minConnectTimeout = func() time.Duration {
return p.MinConnectTimeout
}
})
}
// WithBackoffMaxDelay configures the dialer to use the provided maximum delay // WithBackoffMaxDelay configures the dialer to use the provided maximum delay
// when backing off after failed connection attempts. // when backing off after failed connection attempts.
//
// Deprecated: use WithConnectParams instead. Will be supported throughout 1.x.
func WithBackoffMaxDelay(md time.Duration) DialOption { func WithBackoffMaxDelay(md time.Duration) DialOption {
return WithBackoffConfig(BackoffConfig{MaxDelay: md}) return WithBackoffConfig(BackoffConfig{MaxDelay: md})
} }
@ -255,19 +276,18 @@ func WithBackoffMaxDelay(md time.Duration) DialOption {
// WithBackoffConfig configures the dialer to use the provided backoff // WithBackoffConfig configures the dialer to use the provided backoff
// parameters after connection failures. // parameters after connection failures.
// //
// Use WithBackoffMaxDelay until more parameters on BackoffConfig are opened up // Deprecated: use WithConnectParams instead. Will be supported throughout 1.x.
// for use.
func WithBackoffConfig(b BackoffConfig) DialOption { func WithBackoffConfig(b BackoffConfig) DialOption {
return withBackoff(backoff.Exponential{ bc := backoff.DefaultConfig
MaxDelay: b.MaxDelay, bc.MaxDelay = b.MaxDelay
}) return withBackoff(internalbackoff.Exponential{Config: bc})
} }
// withBackoff sets the backoff strategy used for connectRetryNum after a failed // withBackoff sets the backoff strategy used for connectRetryNum after a failed
// connection attempt. // connection attempt.
// //
// This can be exported if arbitrary backoff strategies are allowed by gRPC. // This can be exported if arbitrary backoff strategies are allowed by gRPC.
func withBackoff(bs backoff.Strategy) DialOption { func withBackoff(bs internalbackoff.Strategy) DialOption {
return newFuncDialOption(func(o *dialOptions) { return newFuncDialOption(func(o *dialOptions) {
o.bs = bs o.bs = bs
}) })

Просмотреть файл

@ -33,20 +33,20 @@ import (
"google.golang.org/grpc/status" "google.golang.org/grpc/status"
) )
const maxDelay = 120 * time.Second var (
backoffStrategy = backoff.DefaultExponential
var backoffStrategy = backoff.Exponential{MaxDelay: maxDelay} backoffFunc = func(ctx context.Context, retries int) bool {
var backoffFunc = func(ctx context.Context, retries int) bool { d := backoffStrategy.Backoff(retries)
d := backoffStrategy.Backoff(retries) timer := time.NewTimer(d)
timer := time.NewTimer(d) select {
select { case <-timer.C:
case <-timer.C: return true
return true case <-ctx.Done():
case <-ctx.Done(): timer.Stop()
timer.Stop() return false
return false }
} }
} )
func init() { func init() {
internal.HealthCheckFunc = clientHealthCheck internal.HealthCheckFunc = clientHealthCheck

Просмотреть файл

@ -25,44 +25,39 @@ package backoff
import ( import (
"time" "time"
grpcbackoff "google.golang.org/grpc/backoff"
"google.golang.org/grpc/internal/grpcrand" "google.golang.org/grpc/internal/grpcrand"
) )
// Strategy defines the methodology for backing off after a grpc connection // Strategy defines the methodology for backing off after a grpc connection
// failure. // failure.
//
type Strategy interface { type Strategy interface {
// Backoff returns the amount of time to wait before the next retry given // Backoff returns the amount of time to wait before the next retry given
// the number of consecutive failures. // the number of consecutive failures.
Backoff(retries int) time.Duration Backoff(retries int) time.Duration
} }
const ( // DefaultExponential is an exponential backoff implementation using the
// baseDelay is the amount of time to wait before retrying after the first // default values for all the configurable knobs defined in
// failure. // https://github.com/grpc/grpc/blob/master/doc/connection-backoff.md.
baseDelay = 1.0 * time.Second var DefaultExponential = Exponential{Config: grpcbackoff.DefaultConfig}
// factor is applied to the backoff after each retry.
factor = 1.6
// jitter provides a range to randomize backoff delays.
jitter = 0.2
)
// Exponential implements exponential backoff algorithm as defined in // Exponential implements exponential backoff algorithm as defined in
// https://github.com/grpc/grpc/blob/master/doc/connection-backoff.md. // https://github.com/grpc/grpc/blob/master/doc/connection-backoff.md.
type Exponential struct { type Exponential struct {
// MaxDelay is the upper bound of backoff delay. // Config contains all options to configure the backoff algorithm.
MaxDelay time.Duration Config grpcbackoff.Config
} }
// Backoff returns the amount of time to wait before the next retry given the // Backoff returns the amount of time to wait before the next retry given the
// number of retries. // number of retries.
func (bc Exponential) Backoff(retries int) time.Duration { func (bc Exponential) Backoff(retries int) time.Duration {
if retries == 0 { if retries == 0 {
return baseDelay return bc.Config.BaseDelay
} }
backoff, max := float64(baseDelay), float64(bc.MaxDelay) backoff, max := float64(bc.Config.BaseDelay), float64(bc.Config.MaxDelay)
for backoff < max && retries > 0 { for backoff < max && retries > 0 {
backoff *= factor backoff *= bc.Config.Multiplier
retries-- retries--
} }
if backoff > max { if backoff > max {
@ -70,7 +65,7 @@ func (bc Exponential) Backoff(retries int) time.Duration {
} }
// Randomize backoff delays so that if a cluster of requests start at // Randomize backoff delays so that if a cluster of requests start at
// the same time, they won't operate in lockstep. // the same time, they won't operate in lockstep.
backoff *= 1 + jitter*(grpcrand.Float64()*2-1) backoff *= 1 + bc.Config.Jitter*(grpcrand.Float64()*2-1)
if backoff < 0 { if backoff < 0 {
return 0 return 0
} }

Просмотреть файл

@ -32,8 +32,9 @@ import (
"sync" "sync"
"time" "time"
"google.golang.org/grpc/backoff"
"google.golang.org/grpc/grpclog" "google.golang.org/grpc/grpclog"
"google.golang.org/grpc/internal/backoff" internalbackoff "google.golang.org/grpc/internal/backoff"
"google.golang.org/grpc/internal/grpcrand" "google.golang.org/grpc/internal/grpcrand"
"google.golang.org/grpc/resolver" "google.golang.org/grpc/resolver"
) )
@ -126,9 +127,11 @@ func (b *dnsBuilder) Build(target resolver.Target, cc resolver.ClientConn, opts
// DNS address (non-IP). // DNS address (non-IP).
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
bc := backoff.DefaultConfig
bc.MaxDelay = b.minFreq
d := &dnsResolver{ d := &dnsResolver{
freq: b.minFreq, freq: b.minFreq,
backoff: backoff.Exponential{MaxDelay: b.minFreq}, backoff: internalbackoff.Exponential{Config: bc},
host: host, host: host,
port: port, port: port,
ctx: ctx, ctx: ctx,
@ -200,7 +203,7 @@ func (i *ipResolver) watcher() {
// dnsResolver watches for the name resolution update for a non-IP target. // dnsResolver watches for the name resolution update for a non-IP target.
type dnsResolver struct { type dnsResolver struct {
freq time.Duration freq time.Duration
backoff backoff.Exponential backoff internalbackoff.Exponential
retryCount int retryCount int
host string host string
port string port string

Просмотреть файл

@ -161,9 +161,7 @@ func NewStore(serviceName string) Store {
}, },
}, },
}, },
backoff: backoff.Exponential{ backoff: backoff.DefaultExponential,
MaxDelay: 120 * time.Second,
},
lastReported: time.Now(), lastReported: time.Now(),
} }
} }

Просмотреть файл

@ -48,12 +48,6 @@ const (
endpointRequired = "endpoints_required" endpointRequired = "endpoints_required"
) )
var (
defaultBackoffConfig = backoff.Exponential{
MaxDelay: 120 * time.Second,
}
)
// client is responsible for connecting to the specified traffic director, passing the received // client is responsible for connecting to the specified traffic director, passing the received
// ADS response from the traffic director, and sending notification when communication with the // ADS response from the traffic director, and sending notification when communication with the
// traffic director is lost. // traffic director is lost.
@ -266,7 +260,7 @@ func newXDSClient(balancerName string, enableCDS bool, opts balancer.BuildOption
newADS: newADS, newADS: newADS,
loseContact: loseContact, loseContact: loseContact,
cleanup: exitCleanup, cleanup: exitCleanup,
backoff: defaultBackoffConfig, backoff: backoff.DefaultExponential,
loadStore: loadStore, loadStore: loadStore,
} }