mirror of https://github.com/grpc/grpc-go.git
health: Client LB channel health checking (#2387)
This commit is contained in:
parent
f4273b1bc8
commit
105f61423e
|
|
@ -94,6 +94,9 @@ type NewSubConnOptions struct {
|
||||||
// SubConn. If it's nil, the original creds from grpc DialOptions will be
|
// SubConn. If it's nil, the original creds from grpc DialOptions will be
|
||||||
// used.
|
// used.
|
||||||
CredsBundle credentials.Bundle
|
CredsBundle credentials.Bundle
|
||||||
|
// HealthCheckEnabled indicates whether health check service should be
|
||||||
|
// enabled on this SubConn
|
||||||
|
HealthCheckEnabled bool
|
||||||
}
|
}
|
||||||
|
|
||||||
// ClientConn represents a gRPC ClientConn.
|
// ClientConn represents a gRPC ClientConn.
|
||||||
|
|
|
||||||
|
|
@ -29,6 +29,7 @@ import (
|
||||||
type baseBuilder struct {
|
type baseBuilder struct {
|
||||||
name string
|
name string
|
||||||
pickerBuilder PickerBuilder
|
pickerBuilder PickerBuilder
|
||||||
|
config Config
|
||||||
}
|
}
|
||||||
|
|
||||||
func (bb *baseBuilder) Build(cc balancer.ClientConn, opt balancer.BuildOptions) balancer.Balancer {
|
func (bb *baseBuilder) Build(cc balancer.ClientConn, opt balancer.BuildOptions) balancer.Balancer {
|
||||||
|
|
@ -43,6 +44,7 @@ func (bb *baseBuilder) Build(cc balancer.ClientConn, opt balancer.BuildOptions)
|
||||||
// ErrNoSubConnAvailable, because when state of a SubConn changes, we
|
// ErrNoSubConnAvailable, because when state of a SubConn changes, we
|
||||||
// may call UpdateBalancerState with this picker.
|
// may call UpdateBalancerState with this picker.
|
||||||
picker: NewErrPicker(balancer.ErrNoSubConnAvailable),
|
picker: NewErrPicker(balancer.ErrNoSubConnAvailable),
|
||||||
|
config: bb.config,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -60,6 +62,7 @@ type baseBalancer struct {
|
||||||
subConns map[resolver.Address]balancer.SubConn
|
subConns map[resolver.Address]balancer.SubConn
|
||||||
scStates map[balancer.SubConn]connectivity.State
|
scStates map[balancer.SubConn]connectivity.State
|
||||||
picker balancer.Picker
|
picker balancer.Picker
|
||||||
|
config Config
|
||||||
}
|
}
|
||||||
|
|
||||||
func (b *baseBalancer) HandleResolvedAddrs(addrs []resolver.Address, err error) {
|
func (b *baseBalancer) HandleResolvedAddrs(addrs []resolver.Address, err error) {
|
||||||
|
|
@ -74,7 +77,7 @@ func (b *baseBalancer) HandleResolvedAddrs(addrs []resolver.Address, err error)
|
||||||
addrsSet[a] = struct{}{}
|
addrsSet[a] = struct{}{}
|
||||||
if _, ok := b.subConns[a]; !ok {
|
if _, ok := b.subConns[a]; !ok {
|
||||||
// a is a new address (not existing in b.subConns).
|
// a is a new address (not existing in b.subConns).
|
||||||
sc, err := b.cc.NewSubConn([]resolver.Address{a}, balancer.NewSubConnOptions{})
|
sc, err := b.cc.NewSubConn([]resolver.Address{a}, balancer.NewSubConnOptions{HealthCheckEnabled: b.config.HealthCheck})
|
||||||
if err != nil {
|
if err != nil {
|
||||||
grpclog.Warningf("base.baseBalancer: failed to create new SubConn: %v", err)
|
grpclog.Warningf("base.baseBalancer: failed to create new SubConn: %v", err)
|
||||||
continue
|
continue
|
||||||
|
|
|
||||||
|
|
@ -45,8 +45,20 @@ type PickerBuilder interface {
|
||||||
// NewBalancerBuilder returns a balancer builder. The balancers
|
// NewBalancerBuilder returns a balancer builder. The balancers
|
||||||
// built by this builder will use the picker builder to build pickers.
|
// built by this builder will use the picker builder to build pickers.
|
||||||
func NewBalancerBuilder(name string, pb PickerBuilder) balancer.Builder {
|
func NewBalancerBuilder(name string, pb PickerBuilder) balancer.Builder {
|
||||||
|
return NewBalancerBuilderWithConfig(name, pb, Config{})
|
||||||
|
}
|
||||||
|
|
||||||
|
// Config contains the config info about the base balancer builder.
|
||||||
|
type Config struct {
|
||||||
|
// HealthCheck indicates whether health checking should be enabled for this specific balancer.
|
||||||
|
HealthCheck bool
|
||||||
|
}
|
||||||
|
|
||||||
|
// NewBalancerBuilderWithConfig returns a base balancer builder configured by the provided config.
|
||||||
|
func NewBalancerBuilderWithConfig(name string, pb PickerBuilder, config Config) balancer.Builder {
|
||||||
return &baseBuilder{
|
return &baseBuilder{
|
||||||
name: name,
|
name: name,
|
||||||
pickerBuilder: pb,
|
pickerBuilder: pb,
|
||||||
|
config: config,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -36,7 +36,7 @@ const Name = "round_robin"
|
||||||
|
|
||||||
// newBuilder creates a new roundrobin balancer builder.
|
// newBuilder creates a new roundrobin balancer builder.
|
||||||
func newBuilder() balancer.Builder {
|
func newBuilder() balancer.Builder {
|
||||||
return base.NewBalancerBuilder(Name, &rrPickerBuilder{})
|
return base.NewBalancerBuilderWithConfig(Name, &rrPickerBuilder{}, base.Config{HealthCheck: true})
|
||||||
}
|
}
|
||||||
|
|
||||||
func init() {
|
func init() {
|
||||||
|
|
|
||||||
108
clientconn.go
108
clientconn.go
|
|
@ -36,6 +36,7 @@ import (
|
||||||
"google.golang.org/grpc/connectivity"
|
"google.golang.org/grpc/connectivity"
|
||||||
"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/backoff"
|
"google.golang.org/grpc/internal/backoff"
|
||||||
"google.golang.org/grpc/internal/channelz"
|
"google.golang.org/grpc/internal/channelz"
|
||||||
"google.golang.org/grpc/internal/transport"
|
"google.golang.org/grpc/internal/transport"
|
||||||
|
|
@ -306,7 +307,9 @@ func DialContext(ctx context.Context, target string, opts ...DialOption) (conn *
|
||||||
break
|
break
|
||||||
} else if cc.dopts.copts.FailOnNonTempDialError && s == connectivity.TransientFailure {
|
} else if cc.dopts.copts.FailOnNonTempDialError && s == connectivity.TransientFailure {
|
||||||
if err = cc.blockingpicker.connectionError(); err != nil {
|
if err = cc.blockingpicker.connectionError(); err != nil {
|
||||||
terr, ok := err.(interface{ Temporary() bool })
|
terr, ok := err.(interface {
|
||||||
|
Temporary() bool
|
||||||
|
})
|
||||||
if ok && !terr.Temporary() {
|
if ok && !terr.Temporary() {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
@ -715,6 +718,12 @@ func (cc *ClientConn) GetMethodConfig(method string) MethodConfig {
|
||||||
return m
|
return m
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (cc *ClientConn) healthCheckConfig() *healthCheckConfig {
|
||||||
|
cc.mu.RLock()
|
||||||
|
defer cc.mu.RUnlock()
|
||||||
|
return cc.sc.healthCheckConfig
|
||||||
|
}
|
||||||
|
|
||||||
func (cc *ClientConn) getTransport(ctx context.Context, failfast bool, method string) (transport.ClientTransport, func(balancer.DoneInfo), error) {
|
func (cc *ClientConn) getTransport(ctx context.Context, failfast bool, method string) (transport.ClientTransport, func(balancer.DoneInfo), error) {
|
||||||
hdr, _ := metadata.FromOutgoingContext(ctx)
|
hdr, _ := metadata.FromOutgoingContext(ctx)
|
||||||
t, done, err := cc.blockingpicker.pick(ctx, failfast, balancer.PickOptions{
|
t, done, err := cc.blockingpicker.pick(ctx, failfast, balancer.PickOptions{
|
||||||
|
|
@ -877,6 +886,10 @@ type addrConn struct {
|
||||||
acbw balancer.SubConn
|
acbw balancer.SubConn
|
||||||
scopts balancer.NewSubConnOptions
|
scopts balancer.NewSubConnOptions
|
||||||
|
|
||||||
|
// transport is set when there's a viable transport (note: ac state may not be READY as LB channel
|
||||||
|
// health checking may require server to report healthy to set ac to READY), and is reset
|
||||||
|
// to nil when the current transport should no longer be used to create a stream (e.g. after GoAway
|
||||||
|
// is received, transport is closed, ac has been torn down).
|
||||||
transport transport.ClientTransport // The current transport.
|
transport transport.ClientTransport // The current transport.
|
||||||
|
|
||||||
mu sync.Mutex
|
mu sync.Mutex
|
||||||
|
|
@ -903,6 +916,8 @@ type addrConn struct {
|
||||||
czData *channelzData
|
czData *channelzData
|
||||||
|
|
||||||
successfulHandshake bool
|
successfulHandshake bool
|
||||||
|
|
||||||
|
healthCheckEnabled bool
|
||||||
}
|
}
|
||||||
|
|
||||||
// Note: this requires a lock on ac.mu.
|
// Note: this requires a lock on ac.mu.
|
||||||
|
|
@ -956,6 +971,8 @@ func (ac *addrConn) resetTransport(resolveNow bool) {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// The transport that was used before is no longer viable.
|
||||||
|
ac.transport = nil
|
||||||
// If the connection is READY, a failure must have occurred.
|
// If the connection is READY, a failure must have occurred.
|
||||||
// Otherwise, we'll consider this is a transient failure when:
|
// Otherwise, we'll consider this is a transient failure when:
|
||||||
// We've exhausted all addresses
|
// We've exhausted all addresses
|
||||||
|
|
@ -1044,7 +1061,10 @@ func (ac *addrConn) createTransport(backoffNum int, addr resolver.Address, copts
|
||||||
var serverPrefaceReceived bool
|
var serverPrefaceReceived bool
|
||||||
var clientPrefaceWrote bool
|
var clientPrefaceWrote bool
|
||||||
|
|
||||||
|
hcCtx, hcCancel := context.WithCancel(ac.ctx)
|
||||||
|
|
||||||
onGoAway := func(r transport.GoAwayReason) {
|
onGoAway := func(r transport.GoAwayReason) {
|
||||||
|
hcCancel()
|
||||||
ac.mu.Lock()
|
ac.mu.Lock()
|
||||||
ac.adjustParams(r)
|
ac.adjustParams(r)
|
||||||
ac.mu.Unlock()
|
ac.mu.Unlock()
|
||||||
|
|
@ -1059,6 +1079,7 @@ func (ac *addrConn) createTransport(backoffNum int, addr resolver.Address, copts
|
||||||
prefaceTimer := time.NewTimer(connectDeadline.Sub(time.Now()))
|
prefaceTimer := time.NewTimer(connectDeadline.Sub(time.Now()))
|
||||||
|
|
||||||
onClose := func() {
|
onClose := func() {
|
||||||
|
hcCancel()
|
||||||
close(onCloseCalled)
|
close(onCloseCalled)
|
||||||
prefaceTimer.Stop()
|
prefaceTimer.Stop()
|
||||||
|
|
||||||
|
|
@ -1166,22 +1187,46 @@ func (ac *addrConn) createTransport(backoffNum int, addr resolver.Address, copts
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Now there is a viable transport to be use, so set ac.transport to reflect the new viable transport.
|
||||||
|
ac.mu.Lock()
|
||||||
|
if ac.state == connectivity.Shutdown {
|
||||||
|
ac.mu.Unlock()
|
||||||
|
close(skipReset)
|
||||||
|
newTr.Close()
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
ac.transport = newTr
|
||||||
|
ac.mu.Unlock()
|
||||||
|
|
||||||
|
healthCheckConfig := ac.cc.healthCheckConfig()
|
||||||
|
// LB channel health checking is only enabled when all the four requirements below are met:
|
||||||
|
// 1. it is not disabled by the user with the WithDisableHealthCheck DialOption,
|
||||||
|
// 2. the internal.HealthCheckFunc is set by importing the grpc/healthcheck package,
|
||||||
|
// 3. a service config with non-empty healthCheckConfig field is provided,
|
||||||
|
// 4. the current load balancer allows it.
|
||||||
|
if !ac.cc.dopts.disableHealthCheck && healthCheckConfig != nil && ac.scopts.HealthCheckEnabled {
|
||||||
|
if internal.HealthCheckFunc != nil {
|
||||||
|
go ac.startHealthCheck(hcCtx, newTr, addr, healthCheckConfig.ServiceName)
|
||||||
|
close(allowedToReset)
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
// TODO: add a link to the health check doc in the error message.
|
||||||
|
grpclog.Error("the client side LB channel health check function has not been set.")
|
||||||
|
}
|
||||||
|
|
||||||
|
// No LB channel health check case
|
||||||
ac.mu.Lock()
|
ac.mu.Lock()
|
||||||
|
|
||||||
if ac.state == connectivity.Shutdown {
|
if ac.state == connectivity.Shutdown {
|
||||||
ac.mu.Unlock()
|
ac.mu.Unlock()
|
||||||
|
|
||||||
// We don't want to reset during this close because we prefer to kick out of this function and let the loop
|
// unblock onGoAway/onClose callback.
|
||||||
// in resetTransport take care of reconnecting.
|
|
||||||
close(skipReset)
|
close(skipReset)
|
||||||
|
|
||||||
newTr.Close()
|
|
||||||
return errConnClosing
|
return errConnClosing
|
||||||
}
|
}
|
||||||
|
|
||||||
ac.updateConnectivityState(connectivity.Ready)
|
ac.updateConnectivityState(connectivity.Ready)
|
||||||
ac.cc.handleSubConnStateChange(ac.acbw, ac.state)
|
ac.cc.handleSubConnStateChange(ac.acbw, ac.state)
|
||||||
ac.transport = newTr
|
|
||||||
ac.curAddr = addr
|
ac.curAddr = addr
|
||||||
|
|
||||||
ac.mu.Unlock()
|
ac.mu.Unlock()
|
||||||
|
|
@ -1192,6 +1237,51 @@ func (ac *addrConn) createTransport(backoffNum int, addr resolver.Address, copts
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (ac *addrConn) startHealthCheck(ctx context.Context, newTr transport.ClientTransport, addr resolver.Address, serviceName string) {
|
||||||
|
// Set up the health check helper functions
|
||||||
|
newStream := func() (interface{}, error) {
|
||||||
|
return ac.newClientStream(ctx, &StreamDesc{ServerStreams: true}, "/grpc.health.v1.Health/Watch", newTr)
|
||||||
|
}
|
||||||
|
firstReady := true
|
||||||
|
reportHealth := func(ok bool) {
|
||||||
|
ac.mu.Lock()
|
||||||
|
defer ac.mu.Unlock()
|
||||||
|
if ac.transport != newTr {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
if ok {
|
||||||
|
if firstReady {
|
||||||
|
firstReady = false
|
||||||
|
ac.curAddr = addr
|
||||||
|
}
|
||||||
|
if ac.state != connectivity.Ready {
|
||||||
|
ac.updateConnectivityState(connectivity.Ready)
|
||||||
|
ac.cc.handleSubConnStateChange(ac.acbw, ac.state)
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
if ac.state != connectivity.TransientFailure {
|
||||||
|
ac.updateConnectivityState(connectivity.TransientFailure)
|
||||||
|
ac.cc.handleSubConnStateChange(ac.acbw, ac.state)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
err := internal.HealthCheckFunc(ctx, newStream, reportHealth, serviceName)
|
||||||
|
if err != nil {
|
||||||
|
if status.Code(err) == codes.Unimplemented {
|
||||||
|
if channelz.IsOn() {
|
||||||
|
channelz.AddTraceEvent(ac.channelzID, &channelz.TraceEventDesc{
|
||||||
|
Desc: "Subchannel health check is unimplemented at server side, thus health check is disabled",
|
||||||
|
Severity: channelz.CtError,
|
||||||
|
})
|
||||||
|
}
|
||||||
|
grpclog.Error("Subchannel health check is unimplemented at server side, thus health check is disabled")
|
||||||
|
} else {
|
||||||
|
grpclog.Errorf("HealthCheckFunc exits with unexpected error %v", err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// nextAddr increments the addrIdx if there are more addresses to try. If
|
// nextAddr increments the addrIdx if there are more addresses to try. If
|
||||||
// there are no more addrs to try it will re-resolve, set addrIdx to 0, and
|
// there are no more addrs to try it will re-resolve, set addrIdx to 0, and
|
||||||
// increment the backoffIdx.
|
// increment the backoffIdx.
|
||||||
|
|
@ -1279,6 +1369,8 @@ func (ac *addrConn) tearDown(err error) {
|
||||||
ac.mu.Unlock()
|
ac.mu.Unlock()
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
curTr := ac.transport
|
||||||
|
ac.transport = nil
|
||||||
// We have to set the state to Shutdown before anything else to prevent races
|
// We have to set the state to Shutdown before anything else to prevent races
|
||||||
// between setting the state and logic that waits on context cancelation / etc.
|
// between setting the state and logic that waits on context cancelation / etc.
|
||||||
ac.updateConnectivityState(connectivity.Shutdown)
|
ac.updateConnectivityState(connectivity.Shutdown)
|
||||||
|
|
@ -1286,14 +1378,14 @@ func (ac *addrConn) tearDown(err error) {
|
||||||
ac.tearDownErr = err
|
ac.tearDownErr = err
|
||||||
ac.cc.handleSubConnStateChange(ac.acbw, ac.state)
|
ac.cc.handleSubConnStateChange(ac.acbw, ac.state)
|
||||||
ac.curAddr = resolver.Address{}
|
ac.curAddr = resolver.Address{}
|
||||||
if err == errConnDrain && ac.transport != nil {
|
if err == errConnDrain && curTr != nil {
|
||||||
// GracefulClose(...) may be executed multiple times when
|
// GracefulClose(...) may be executed multiple times when
|
||||||
// i) receiving multiple GoAway frames from the server; or
|
// i) receiving multiple GoAway frames from the server; or
|
||||||
// ii) there are concurrent name resolver/Balancer triggered
|
// ii) there are concurrent name resolver/Balancer triggered
|
||||||
// address removal and GoAway.
|
// address removal and GoAway.
|
||||||
// We have to unlock and re-lock here because GracefulClose => Close => onClose, which requires locking ac.mu.
|
// We have to unlock and re-lock here because GracefulClose => Close => onClose, which requires locking ac.mu.
|
||||||
ac.mu.Unlock()
|
ac.mu.Unlock()
|
||||||
ac.transport.GracefulClose()
|
curTr.GracefulClose()
|
||||||
ac.mu.Lock()
|
ac.mu.Lock()
|
||||||
}
|
}
|
||||||
if channelz.IsOn() {
|
if channelz.IsOn() {
|
||||||
|
|
|
||||||
|
|
@ -59,6 +59,7 @@ type dialOptions struct {
|
||||||
channelzParentID int64
|
channelzParentID int64
|
||||||
disableServiceConfig bool
|
disableServiceConfig bool
|
||||||
disableRetry bool
|
disableRetry bool
|
||||||
|
disableHealthCheck bool
|
||||||
}
|
}
|
||||||
|
|
||||||
// DialOption configures how we set up the connection.
|
// DialOption configures how we set up the connection.
|
||||||
|
|
@ -454,6 +455,14 @@ func WithMaxHeaderListSize(s uint32) DialOption {
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// WithDisableHealthCheck disables the LB channel health checking for all SubConns of this ClientConn.
|
||||||
|
//
|
||||||
|
// This API is EXPERIMENTAL.
|
||||||
|
func WithDisableHealthCheck() DialOption {
|
||||||
|
return newFuncDialOption(func(o *dialOptions) {
|
||||||
|
o.disableHealthCheck = true
|
||||||
|
})
|
||||||
|
}
|
||||||
func defaultDialOptions() dialOptions {
|
func defaultDialOptions() dialOptions {
|
||||||
return dialOptions{
|
return dialOptions{
|
||||||
disableRetry: !envconfig.Retry,
|
disableRetry: !envconfig.Retry,
|
||||||
|
|
|
||||||
|
|
@ -155,7 +155,7 @@ func (d *gzipDecompressor) Type() string {
|
||||||
type callInfo struct {
|
type callInfo struct {
|
||||||
compressorType string
|
compressorType string
|
||||||
failFast bool
|
failFast bool
|
||||||
stream *clientStream
|
stream ClientStream
|
||||||
maxReceiveMessageSize *int
|
maxReceiveMessageSize *int
|
||||||
maxSendMessageSize *int
|
maxSendMessageSize *int
|
||||||
creds credentials.PerRPCCredentials
|
creds credentials.PerRPCCredentials
|
||||||
|
|
|
||||||
|
|
@ -96,6 +96,15 @@ type ServiceConfig struct {
|
||||||
// If token_count is less than or equal to maxTokens / 2, then RPCs will not
|
// If token_count is less than or equal to maxTokens / 2, then RPCs will not
|
||||||
// be retried and hedged RPCs will not be sent.
|
// be retried and hedged RPCs will not be sent.
|
||||||
retryThrottling *retryThrottlingPolicy
|
retryThrottling *retryThrottlingPolicy
|
||||||
|
// healthCheckConfig must be set as one of the requirement to enable LB channel
|
||||||
|
// health check.
|
||||||
|
healthCheckConfig *healthCheckConfig
|
||||||
|
}
|
||||||
|
|
||||||
|
// healthCheckConfig defines the go-native version of the LB channel health check config.
|
||||||
|
type healthCheckConfig struct {
|
||||||
|
// serviceName is the service name to use in the health-checking request.
|
||||||
|
ServiceName string
|
||||||
}
|
}
|
||||||
|
|
||||||
// retryPolicy defines the go-native version of the retry policy defined by the
|
// retryPolicy defines the go-native version of the retry policy defined by the
|
||||||
|
|
@ -226,6 +235,7 @@ type jsonSC struct {
|
||||||
LoadBalancingPolicy *string
|
LoadBalancingPolicy *string
|
||||||
MethodConfig *[]jsonMC
|
MethodConfig *[]jsonMC
|
||||||
RetryThrottling *retryThrottlingPolicy
|
RetryThrottling *retryThrottlingPolicy
|
||||||
|
HealthCheckConfig *healthCheckConfig
|
||||||
}
|
}
|
||||||
|
|
||||||
func parseServiceConfig(js string) (ServiceConfig, error) {
|
func parseServiceConfig(js string) (ServiceConfig, error) {
|
||||||
|
|
@ -239,9 +249,10 @@ func parseServiceConfig(js string) (ServiceConfig, error) {
|
||||||
return ServiceConfig{}, err
|
return ServiceConfig{}, err
|
||||||
}
|
}
|
||||||
sc := ServiceConfig{
|
sc := ServiceConfig{
|
||||||
LB: rsc.LoadBalancingPolicy,
|
LB: rsc.LoadBalancingPolicy,
|
||||||
Methods: make(map[string]MethodConfig),
|
Methods: make(map[string]MethodConfig),
|
||||||
retryThrottling: rsc.RetryThrottling,
|
retryThrottling: rsc.RetryThrottling,
|
||||||
|
healthCheckConfig: rsc.HealthCheckConfig,
|
||||||
}
|
}
|
||||||
if rsc.MethodConfig == nil {
|
if rsc.MethodConfig == nil {
|
||||||
return sc, nil
|
return sc, nil
|
||||||
|
|
|
||||||
295
stream.go
295
stream.go
|
|
@ -26,6 +26,8 @@ import (
|
||||||
"sync"
|
"sync"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"google.golang.org/grpc/connectivity"
|
||||||
|
|
||||||
"golang.org/x/net/context"
|
"golang.org/x/net/context"
|
||||||
"golang.org/x/net/trace"
|
"golang.org/x/net/trace"
|
||||||
"google.golang.org/grpc/balancer"
|
"google.golang.org/grpc/balancer"
|
||||||
|
|
@ -950,6 +952,299 @@ func (a *csAttempt) finish(err error) {
|
||||||
a.mu.Unlock()
|
a.mu.Unlock()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (ac *addrConn) newClientStream(ctx context.Context, desc *StreamDesc, method string, t transport.ClientTransport, opts ...CallOption) (_ ClientStream, err error) {
|
||||||
|
ac.mu.Lock()
|
||||||
|
if ac.transport != t {
|
||||||
|
ac.mu.Unlock()
|
||||||
|
return nil, status.Error(codes.Canceled, "the provided transport is no longer valid to use")
|
||||||
|
}
|
||||||
|
// transition to CONNECTING state when an attempt starts
|
||||||
|
if ac.state != connectivity.Connecting {
|
||||||
|
ac.updateConnectivityState(connectivity.Connecting)
|
||||||
|
ac.cc.handleSubConnStateChange(ac.acbw, ac.state)
|
||||||
|
}
|
||||||
|
ac.mu.Unlock()
|
||||||
|
|
||||||
|
if t == nil {
|
||||||
|
// TODO: return RPC error here?
|
||||||
|
return nil, errors.New("transport provided is nil")
|
||||||
|
}
|
||||||
|
// defaultCallInfo contains unnecessary info(i.e. failfast, maxRetryRPCBufferSize), so we just initialize an empty struct.
|
||||||
|
c := &callInfo{}
|
||||||
|
|
||||||
|
for _, o := range opts {
|
||||||
|
if err := o.before(c); err != nil {
|
||||||
|
return nil, toRPCErr(err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
c.maxReceiveMessageSize = getMaxSize(nil, c.maxReceiveMessageSize, defaultClientMaxReceiveMessageSize)
|
||||||
|
c.maxSendMessageSize = getMaxSize(nil, c.maxSendMessageSize, defaultServerMaxSendMessageSize)
|
||||||
|
|
||||||
|
// Possible context leak:
|
||||||
|
// The cancel function for the child context we create will only be called
|
||||||
|
// when RecvMsg returns a non-nil error, if the ClientConn is closed, or if
|
||||||
|
// an error is generated by SendMsg.
|
||||||
|
// https://github.com/grpc/grpc-go/issues/1818.
|
||||||
|
ctx, cancel := context.WithCancel(ctx)
|
||||||
|
defer func() {
|
||||||
|
if err != nil {
|
||||||
|
cancel()
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
|
if err := setCallInfoCodec(c); err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
callHdr := &transport.CallHdr{
|
||||||
|
Host: ac.cc.authority,
|
||||||
|
Method: method,
|
||||||
|
ContentSubtype: c.contentSubtype,
|
||||||
|
}
|
||||||
|
|
||||||
|
// Set our outgoing compression according to the UseCompressor CallOption, if
|
||||||
|
// set. In that case, also find the compressor from the encoding package.
|
||||||
|
// Otherwise, use the compressor configured by the WithCompressor DialOption,
|
||||||
|
// if set.
|
||||||
|
var cp Compressor
|
||||||
|
var comp encoding.Compressor
|
||||||
|
if ct := c.compressorType; ct != "" {
|
||||||
|
callHdr.SendCompress = ct
|
||||||
|
if ct != encoding.Identity {
|
||||||
|
comp = encoding.GetCompressor(ct)
|
||||||
|
if comp == nil {
|
||||||
|
return nil, status.Errorf(codes.Internal, "grpc: Compressor is not installed for requested grpc-encoding %q", ct)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} else if ac.cc.dopts.cp != nil {
|
||||||
|
callHdr.SendCompress = ac.cc.dopts.cp.Type()
|
||||||
|
cp = ac.cc.dopts.cp
|
||||||
|
}
|
||||||
|
if c.creds != nil {
|
||||||
|
callHdr.Creds = c.creds
|
||||||
|
}
|
||||||
|
|
||||||
|
as := &addrConnStream{
|
||||||
|
callHdr: callHdr,
|
||||||
|
ac: ac,
|
||||||
|
ctx: ctx,
|
||||||
|
cancel: cancel,
|
||||||
|
opts: opts,
|
||||||
|
callInfo: c,
|
||||||
|
desc: desc,
|
||||||
|
codec: c.codec,
|
||||||
|
cp: cp,
|
||||||
|
comp: comp,
|
||||||
|
t: t,
|
||||||
|
}
|
||||||
|
|
||||||
|
as.callInfo.stream = as
|
||||||
|
s, err := as.t.NewStream(as.ctx, as.callHdr)
|
||||||
|
if err != nil {
|
||||||
|
err = toRPCErr(err)
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
as.s = s
|
||||||
|
as.p = &parser{r: s}
|
||||||
|
ac.incrCallsStarted()
|
||||||
|
if desc != unaryStreamDesc {
|
||||||
|
// Listen on cc and stream contexts to cleanup when the user closes the
|
||||||
|
// ClientConn or cancels the stream context. In all other cases, an error
|
||||||
|
// should already be injected into the recv buffer by the transport, which
|
||||||
|
// the client will eventually receive, and then we will cancel the stream's
|
||||||
|
// context in clientStream.finish.
|
||||||
|
go func() {
|
||||||
|
select {
|
||||||
|
case <-ac.ctx.Done():
|
||||||
|
as.finish(status.Error(codes.Canceled, "grpc: the SubConn is closing"))
|
||||||
|
case <-ctx.Done():
|
||||||
|
as.finish(toRPCErr(ctx.Err()))
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
}
|
||||||
|
return as, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
type addrConnStream struct {
|
||||||
|
s *transport.Stream
|
||||||
|
ac *addrConn
|
||||||
|
callHdr *transport.CallHdr
|
||||||
|
cancel context.CancelFunc
|
||||||
|
opts []CallOption
|
||||||
|
callInfo *callInfo
|
||||||
|
t transport.ClientTransport
|
||||||
|
ctx context.Context
|
||||||
|
sentLast bool
|
||||||
|
desc *StreamDesc
|
||||||
|
codec baseCodec
|
||||||
|
cp Compressor
|
||||||
|
comp encoding.Compressor
|
||||||
|
decompSet bool
|
||||||
|
dc Decompressor
|
||||||
|
decomp encoding.Compressor
|
||||||
|
p *parser
|
||||||
|
done func(balancer.DoneInfo)
|
||||||
|
mu sync.Mutex
|
||||||
|
finished bool
|
||||||
|
}
|
||||||
|
|
||||||
|
func (as *addrConnStream) Header() (metadata.MD, error) {
|
||||||
|
m, err := as.s.Header()
|
||||||
|
if err != nil {
|
||||||
|
as.finish(toRPCErr(err))
|
||||||
|
}
|
||||||
|
return m, err
|
||||||
|
}
|
||||||
|
|
||||||
|
func (as *addrConnStream) Trailer() metadata.MD {
|
||||||
|
return as.s.Trailer()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (as *addrConnStream) CloseSend() error {
|
||||||
|
if as.sentLast {
|
||||||
|
// TODO: return an error and finish the stream instead, due to API misuse?
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
as.sentLast = true
|
||||||
|
|
||||||
|
as.t.Write(as.s, nil, nil, &transport.Options{Last: true})
|
||||||
|
// Always return nil; io.EOF is the only error that might make sense
|
||||||
|
// instead, but there is no need to signal the client to call RecvMsg
|
||||||
|
// as the only use left for the stream after CloseSend is to call
|
||||||
|
// RecvMsg. This also matches historical behavior.
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (as *addrConnStream) Context() context.Context {
|
||||||
|
return as.s.Context()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (as *addrConnStream) SendMsg(m interface{}) (err error) {
|
||||||
|
defer func() {
|
||||||
|
if err != nil && err != io.EOF {
|
||||||
|
// Call finish on the client stream for errors generated by this SendMsg
|
||||||
|
// call, as these indicate problems created by this client. (Transport
|
||||||
|
// errors are converted to an io.EOF error in csAttempt.sendMsg; the real
|
||||||
|
// error will be returned from RecvMsg eventually in that case, or be
|
||||||
|
// retried.)
|
||||||
|
as.finish(err)
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
if as.sentLast {
|
||||||
|
return status.Errorf(codes.Internal, "SendMsg called after CloseSend")
|
||||||
|
}
|
||||||
|
if !as.desc.ClientStreams {
|
||||||
|
as.sentLast = true
|
||||||
|
}
|
||||||
|
data, err := encode(as.codec, m)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
compData, err := compress(data, as.cp, as.comp)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
hdr, payld := msgHeader(data, compData)
|
||||||
|
// TODO(dfawley): should we be checking len(data) instead?
|
||||||
|
if len(payld) > *as.callInfo.maxSendMessageSize {
|
||||||
|
return status.Errorf(codes.ResourceExhausted, "trying to send message larger than max (%d vs. %d)", len(payld), *as.callInfo.maxSendMessageSize)
|
||||||
|
}
|
||||||
|
|
||||||
|
if err := as.t.Write(as.s, hdr, payld, &transport.Options{Last: !as.desc.ClientStreams}); err != nil {
|
||||||
|
if !as.desc.ClientStreams {
|
||||||
|
// For non-client-streaming RPCs, we return nil instead of EOF on error
|
||||||
|
// because the generated code requires it. finish is not called; RecvMsg()
|
||||||
|
// will call it with the stream's status independently.
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
return io.EOF
|
||||||
|
}
|
||||||
|
|
||||||
|
if channelz.IsOn() {
|
||||||
|
as.t.IncrMsgSent()
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (as *addrConnStream) RecvMsg(m interface{}) (err error) {
|
||||||
|
defer func() {
|
||||||
|
if err != nil || !as.desc.ServerStreams {
|
||||||
|
// err != nil or non-server-streaming indicates end of stream.
|
||||||
|
as.finish(err)
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
|
if !as.decompSet {
|
||||||
|
// Block until we receive headers containing received message encoding.
|
||||||
|
if ct := as.s.RecvCompress(); ct != "" && ct != encoding.Identity {
|
||||||
|
if as.dc == nil || as.dc.Type() != ct {
|
||||||
|
// No configured decompressor, or it does not match the incoming
|
||||||
|
// message encoding; attempt to find a registered compressor that does.
|
||||||
|
as.dc = nil
|
||||||
|
as.decomp = encoding.GetCompressor(ct)
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
// No compression is used; disable our decompressor.
|
||||||
|
as.dc = nil
|
||||||
|
}
|
||||||
|
// Only initialize this state once per stream.
|
||||||
|
as.decompSet = true
|
||||||
|
}
|
||||||
|
err = recv(as.p, as.codec, as.s, as.dc, m, *as.callInfo.maxReceiveMessageSize, nil, as.decomp)
|
||||||
|
if err != nil {
|
||||||
|
if err == io.EOF {
|
||||||
|
if statusErr := as.s.Status().Err(); statusErr != nil {
|
||||||
|
return statusErr
|
||||||
|
}
|
||||||
|
return io.EOF // indicates successful end of stream.
|
||||||
|
}
|
||||||
|
return toRPCErr(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
if channelz.IsOn() {
|
||||||
|
as.t.IncrMsgRecv()
|
||||||
|
}
|
||||||
|
if as.desc.ServerStreams {
|
||||||
|
// Subsequent messages should be received by subsequent RecvMsg calls.
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// Special handling for non-server-stream rpcs.
|
||||||
|
// This recv expects EOF or errors, so we don't collect inPayload.
|
||||||
|
err = recv(as.p, as.codec, as.s, as.dc, m, *as.callInfo.maxReceiveMessageSize, nil, as.decomp)
|
||||||
|
if err == nil {
|
||||||
|
return toRPCErr(errors.New("grpc: client streaming protocol violation: get <nil>, want <EOF>"))
|
||||||
|
}
|
||||||
|
if err == io.EOF {
|
||||||
|
return as.s.Status().Err() // non-server streaming Recv returns nil on success
|
||||||
|
}
|
||||||
|
return toRPCErr(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (as *addrConnStream) finish(err error) {
|
||||||
|
as.mu.Lock()
|
||||||
|
if as.finished {
|
||||||
|
as.mu.Unlock()
|
||||||
|
return
|
||||||
|
}
|
||||||
|
as.finished = true
|
||||||
|
if err == io.EOF {
|
||||||
|
// Ending a stream with EOF indicates a success.
|
||||||
|
err = nil
|
||||||
|
}
|
||||||
|
if as.s != nil {
|
||||||
|
as.t.CloseStream(as.s, err)
|
||||||
|
}
|
||||||
|
|
||||||
|
if err != nil {
|
||||||
|
as.ac.incrCallsFailed()
|
||||||
|
} else {
|
||||||
|
as.ac.incrCallsSucceeded()
|
||||||
|
}
|
||||||
|
as.cancel()
|
||||||
|
as.mu.Unlock()
|
||||||
|
}
|
||||||
|
|
||||||
// ServerStream defines the server-side behavior of a streaming RPC.
|
// ServerStream defines the server-side behavior of a streaming RPC.
|
||||||
//
|
//
|
||||||
// All errors returned from ServerStream methods are compatible with the
|
// All errors returned from ServerStream methods are compatible with the
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,955 @@
|
||||||
|
/*
|
||||||
|
*
|
||||||
|
* Copyright 2018 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 test
|
||||||
|
|
||||||
|
import (
|
||||||
|
"errors"
|
||||||
|
"fmt"
|
||||||
|
"net"
|
||||||
|
"sync"
|
||||||
|
"testing"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
"golang.org/x/net/context"
|
||||||
|
"google.golang.org/grpc"
|
||||||
|
"google.golang.org/grpc/codes"
|
||||||
|
"google.golang.org/grpc/connectivity"
|
||||||
|
_ "google.golang.org/grpc/health"
|
||||||
|
healthpb "google.golang.org/grpc/health/grpc_health_v1"
|
||||||
|
"google.golang.org/grpc/internal"
|
||||||
|
"google.golang.org/grpc/internal/channelz"
|
||||||
|
"google.golang.org/grpc/internal/leakcheck"
|
||||||
|
"google.golang.org/grpc/resolver"
|
||||||
|
"google.golang.org/grpc/resolver/manual"
|
||||||
|
"google.golang.org/grpc/status"
|
||||||
|
testpb "google.golang.org/grpc/test/grpc_testing"
|
||||||
|
)
|
||||||
|
|
||||||
|
var testHealthCheckFunc = internal.HealthCheckFunc
|
||||||
|
|
||||||
|
func replaceHealthCheckFunc(f func(context.Context, func() (interface{}, error), func(bool), string) error) func() {
|
||||||
|
oldHcFunc := internal.HealthCheckFunc
|
||||||
|
internal.HealthCheckFunc = f
|
||||||
|
return func() {
|
||||||
|
internal.HealthCheckFunc = oldHcFunc
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func newTestHealthServer() *testHealthServer {
|
||||||
|
return newTestHealthServerWithWatchFunc(defaultWatchFunc)
|
||||||
|
}
|
||||||
|
|
||||||
|
func newTestHealthServerWithWatchFunc(f func(s *testHealthServer, in *healthpb.HealthCheckRequest, stream healthpb.Health_WatchServer) error) *testHealthServer {
|
||||||
|
return &testHealthServer{
|
||||||
|
watchFunc: f,
|
||||||
|
update: make(chan struct{}, 1),
|
||||||
|
status: make(map[string]healthpb.HealthCheckResponse_ServingStatus),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// defaultWatchFunc will send a HealthCheckResponse to the client whenever SetServingStatus is called.
|
||||||
|
func defaultWatchFunc(s *testHealthServer, in *healthpb.HealthCheckRequest, stream healthpb.Health_WatchServer) error {
|
||||||
|
if in.Service != "foo" {
|
||||||
|
return status.Error(codes.FailedPrecondition,
|
||||||
|
"the defaultWatchFunc only handles request with service name to be \"foo\"")
|
||||||
|
}
|
||||||
|
var done bool
|
||||||
|
for {
|
||||||
|
select {
|
||||||
|
case <-stream.Context().Done():
|
||||||
|
done = true
|
||||||
|
case <-s.update:
|
||||||
|
}
|
||||||
|
if done {
|
||||||
|
break
|
||||||
|
}
|
||||||
|
s.mu.Lock()
|
||||||
|
resp := &healthpb.HealthCheckResponse{
|
||||||
|
Status: s.status[in.Service],
|
||||||
|
}
|
||||||
|
s.mu.Unlock()
|
||||||
|
stream.SendMsg(resp)
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
type testHealthServer struct {
|
||||||
|
watchFunc func(s *testHealthServer, in *healthpb.HealthCheckRequest, stream healthpb.Health_WatchServer) error
|
||||||
|
mu sync.Mutex
|
||||||
|
status map[string]healthpb.HealthCheckResponse_ServingStatus
|
||||||
|
update chan struct{}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (s *testHealthServer) Check(ctx context.Context, in *healthpb.HealthCheckRequest) (*healthpb.HealthCheckResponse, error) {
|
||||||
|
return &healthpb.HealthCheckResponse{
|
||||||
|
Status: healthpb.HealthCheckResponse_SERVING,
|
||||||
|
}, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (s *testHealthServer) Watch(in *healthpb.HealthCheckRequest, stream healthpb.Health_WatchServer) error {
|
||||||
|
return s.watchFunc(s, in, stream)
|
||||||
|
}
|
||||||
|
|
||||||
|
// SetServingStatus is called when need to reset the serving status of a service
|
||||||
|
// or insert a new service entry into the statusMap.
|
||||||
|
func (s *testHealthServer) SetServingStatus(service string, status healthpb.HealthCheckResponse_ServingStatus) {
|
||||||
|
s.mu.Lock()
|
||||||
|
s.status[service] = status
|
||||||
|
select {
|
||||||
|
case <-s.update:
|
||||||
|
default:
|
||||||
|
}
|
||||||
|
s.update <- struct{}{}
|
||||||
|
s.mu.Unlock()
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestHealthCheckWatchStateChange(t *testing.T) {
|
||||||
|
defer leakcheck.Check(t)
|
||||||
|
s := grpc.NewServer()
|
||||||
|
lis, err := net.Listen("tcp", "localhost:0")
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("failed to listen due to err: %v", err)
|
||||||
|
}
|
||||||
|
ts := newTestHealthServer()
|
||||||
|
healthpb.RegisterHealthServer(s, ts)
|
||||||
|
go s.Serve(lis)
|
||||||
|
defer s.Stop()
|
||||||
|
|
||||||
|
// The table below shows the expected series of addrConn connectivity transitions when server
|
||||||
|
// updates its health status. As there's only one addrConn corresponds with the ClientConn in this
|
||||||
|
// test, we use ClientConn's connectivity state as the addrConn connectivity state.
|
||||||
|
//+------------------------------+-------------------------------------------+
|
||||||
|
//| Health Check Returned Status | Expected addrConn Connectivity Transition |
|
||||||
|
//+------------------------------+-------------------------------------------+
|
||||||
|
//| NOT_SERVING | ->TRANSIENT FAILURE |
|
||||||
|
//| SERVING | ->READY |
|
||||||
|
//| SERVICE_UNKNOWN | ->TRANSIENT FAILURE |
|
||||||
|
//| SERVING | ->READY |
|
||||||
|
//| UNKNOWN | ->TRANSIENT FAILURE |
|
||||||
|
//+------------------------------+-------------------------------------------+
|
||||||
|
ts.SetServingStatus("foo", healthpb.HealthCheckResponse_NOT_SERVING)
|
||||||
|
r, rcleanup := manual.GenerateAndRegisterManualResolver()
|
||||||
|
defer rcleanup()
|
||||||
|
cc, err := grpc.Dial(r.Scheme()+":///test.server", grpc.WithInsecure(), grpc.WithBalancerName("round_robin"))
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("dial failed due to err: %v", err)
|
||||||
|
}
|
||||||
|
defer cc.Close()
|
||||||
|
|
||||||
|
r.NewServiceConfig(`{
|
||||||
|
"healthCheckConfig": {
|
||||||
|
"serviceName": "foo"
|
||||||
|
}
|
||||||
|
}`)
|
||||||
|
r.NewAddress([]resolver.Address{{Addr: lis.Addr().String()}})
|
||||||
|
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
|
||||||
|
defer cancel()
|
||||||
|
if ok := cc.WaitForStateChange(ctx, connectivity.Idle); !ok {
|
||||||
|
t.Fatal("ClientConn is still in IDLE state when the context times out.")
|
||||||
|
}
|
||||||
|
if ok := cc.WaitForStateChange(ctx, connectivity.Connecting); !ok {
|
||||||
|
t.Fatal("ClientConn is still in CONNECTING state when the context times out.")
|
||||||
|
}
|
||||||
|
if s := cc.GetState(); s != connectivity.TransientFailure {
|
||||||
|
t.Fatalf("ClientConn is in %v state, want TRANSIENT FAILURE", s)
|
||||||
|
}
|
||||||
|
|
||||||
|
ts.SetServingStatus("foo", healthpb.HealthCheckResponse_SERVING)
|
||||||
|
if ok := cc.WaitForStateChange(ctx, connectivity.TransientFailure); !ok {
|
||||||
|
t.Fatal("ClientConn is still in TRANSIENT FAILURE state when the context times out.")
|
||||||
|
}
|
||||||
|
if s := cc.GetState(); s != connectivity.Ready {
|
||||||
|
t.Fatalf("ClientConn is in %v state, want READY", s)
|
||||||
|
}
|
||||||
|
|
||||||
|
ts.SetServingStatus("foo", healthpb.HealthCheckResponse_SERVICE_UNKNOWN)
|
||||||
|
if ok := cc.WaitForStateChange(ctx, connectivity.Ready); !ok {
|
||||||
|
t.Fatal("ClientConn is still in READY state when the context times out.")
|
||||||
|
}
|
||||||
|
if s := cc.GetState(); s != connectivity.TransientFailure {
|
||||||
|
t.Fatalf("ClientConn is in %v state, want TRANSIENT FAILURE", s)
|
||||||
|
}
|
||||||
|
|
||||||
|
ts.SetServingStatus("foo", healthpb.HealthCheckResponse_SERVING)
|
||||||
|
if ok := cc.WaitForStateChange(ctx, connectivity.TransientFailure); !ok {
|
||||||
|
t.Fatal("ClientConn is still in TRANSIENT FAILURE state when the context times out.")
|
||||||
|
}
|
||||||
|
if s := cc.GetState(); s != connectivity.Ready {
|
||||||
|
t.Fatalf("ClientConn is in %v state, want READY", s)
|
||||||
|
}
|
||||||
|
|
||||||
|
ts.SetServingStatus("foo", healthpb.HealthCheckResponse_UNKNOWN)
|
||||||
|
if ok := cc.WaitForStateChange(ctx, connectivity.Ready); !ok {
|
||||||
|
t.Fatal("ClientConn is still in READY state when the context times out.")
|
||||||
|
}
|
||||||
|
if s := cc.GetState(); s != connectivity.TransientFailure {
|
||||||
|
t.Fatalf("ClientConn is in %v state, want TRANSIENT FAILURE", s)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// In the case of a goaway received, the health check stream should be terminated and health check
|
||||||
|
// function should exit.
|
||||||
|
func TestHealthCheckWithGoAway(t *testing.T) {
|
||||||
|
defer leakcheck.Check(t)
|
||||||
|
s := grpc.NewServer()
|
||||||
|
lis, err := net.Listen("tcp", "localhost:0")
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("failed to listen due to err: %v", err)
|
||||||
|
}
|
||||||
|
ts := newTestHealthServer()
|
||||||
|
healthpb.RegisterHealthServer(s, ts)
|
||||||
|
testpb.RegisterTestServiceServer(s, &testServer{})
|
||||||
|
go s.Serve(lis)
|
||||||
|
defer s.Stop()
|
||||||
|
ts.SetServingStatus("foo", healthpb.HealthCheckResponse_SERVING)
|
||||||
|
hcExitChan := make(chan struct{})
|
||||||
|
testHealthCheckFuncWrapper := func(ctx context.Context, newStream func() (interface{}, error), update func(bool), service string) error {
|
||||||
|
err := testHealthCheckFunc(ctx, newStream, update, service)
|
||||||
|
close(hcExitChan)
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
replace := replaceHealthCheckFunc(testHealthCheckFuncWrapper)
|
||||||
|
defer replace()
|
||||||
|
r, rcleanup := manual.GenerateAndRegisterManualResolver()
|
||||||
|
defer rcleanup()
|
||||||
|
|
||||||
|
cc, err := grpc.Dial(r.Scheme()+":///test.server", grpc.WithInsecure(), grpc.WithBalancerName("round_robin"))
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("dial failed due to err: %v", err)
|
||||||
|
}
|
||||||
|
defer cc.Close()
|
||||||
|
|
||||||
|
tc := testpb.NewTestServiceClient(cc)
|
||||||
|
r.NewServiceConfig(`{
|
||||||
|
"healthCheckConfig": {
|
||||||
|
"serviceName": "foo"
|
||||||
|
}
|
||||||
|
}`)
|
||||||
|
r.NewAddress([]resolver.Address{{Addr: lis.Addr().String()}})
|
||||||
|
|
||||||
|
// make some rpcs to make sure connection is working.
|
||||||
|
if err := verifyResultWithDelay(func() (bool, error) {
|
||||||
|
if _, err := tc.EmptyCall(context.Background(), &testpb.Empty{}); err != nil {
|
||||||
|
return false, fmt.Errorf("TestService/EmptyCall(_, _) = _, %v, want _, <nil>", err)
|
||||||
|
}
|
||||||
|
return true, nil
|
||||||
|
}); err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
// the stream rpc will persist through goaway event.
|
||||||
|
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
|
||||||
|
defer cancel()
|
||||||
|
stream, err := tc.FullDuplexCall(ctx, grpc.FailFast(false))
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("%v.FullDuplexCall(_) = _, %v, want <nil>", tc, err)
|
||||||
|
}
|
||||||
|
respParam := []*testpb.ResponseParameters{{Size: 1}}
|
||||||
|
payload, err := newPayload(testpb.PayloadType_COMPRESSABLE, int32(1))
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
req := &testpb.StreamingOutputCallRequest{
|
||||||
|
ResponseParameters: respParam,
|
||||||
|
Payload: payload,
|
||||||
|
}
|
||||||
|
if err := stream.Send(req); err != nil {
|
||||||
|
t.Fatalf("%v.Send(_) = %v, want <nil>", stream, err)
|
||||||
|
}
|
||||||
|
if _, err := stream.Recv(); err != nil {
|
||||||
|
t.Fatalf("%v.Recv() = _, %v, want _, <nil>", stream, err)
|
||||||
|
}
|
||||||
|
|
||||||
|
select {
|
||||||
|
case <-hcExitChan:
|
||||||
|
t.Fatal("Health check function has exited, which is not expected.")
|
||||||
|
default:
|
||||||
|
}
|
||||||
|
|
||||||
|
// server sends GoAway
|
||||||
|
go s.GracefulStop()
|
||||||
|
|
||||||
|
select {
|
||||||
|
case <-hcExitChan:
|
||||||
|
case <-time.After(5 * time.Second):
|
||||||
|
t.Fatal("Health check function has not exited after 5s.")
|
||||||
|
}
|
||||||
|
|
||||||
|
// The existing RPC should be still good to proceed.
|
||||||
|
if err := stream.Send(req); err != nil {
|
||||||
|
t.Fatalf("%v.Send(_) = %v, want <nil>", stream, err)
|
||||||
|
}
|
||||||
|
if _, err := stream.Recv(); err != nil {
|
||||||
|
t.Fatalf("%v.Recv() = _, %v, want _, <nil>", stream, err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestHealthCheckWithConnClose(t *testing.T) {
|
||||||
|
defer leakcheck.Check(t)
|
||||||
|
s := grpc.NewServer()
|
||||||
|
lis, err := net.Listen("tcp", "localhost:0")
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("failed to listen due to err: %v", err)
|
||||||
|
}
|
||||||
|
ts := newTestHealthServer()
|
||||||
|
healthpb.RegisterHealthServer(s, ts)
|
||||||
|
testpb.RegisterTestServiceServer(s, &testServer{})
|
||||||
|
go s.Serve(lis)
|
||||||
|
defer s.Stop()
|
||||||
|
ts.SetServingStatus("foo", healthpb.HealthCheckResponse_SERVING)
|
||||||
|
hcExitChan := make(chan struct{})
|
||||||
|
testHealthCheckFuncWrapper := func(ctx context.Context, newStream func() (interface{}, error), update func(bool), service string) error {
|
||||||
|
err := testHealthCheckFunc(ctx, newStream, update, service)
|
||||||
|
close(hcExitChan)
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
replace := replaceHealthCheckFunc(testHealthCheckFuncWrapper)
|
||||||
|
defer replace()
|
||||||
|
r, rcleanup := manual.GenerateAndRegisterManualResolver()
|
||||||
|
defer rcleanup()
|
||||||
|
cc, err := grpc.Dial(r.Scheme()+":///test.server", grpc.WithInsecure(), grpc.WithBalancerName("round_robin"))
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("dial failed due to err: %v", err)
|
||||||
|
}
|
||||||
|
defer cc.Close()
|
||||||
|
tc := testpb.NewTestServiceClient(cc)
|
||||||
|
|
||||||
|
r.NewServiceConfig(`{
|
||||||
|
"healthCheckConfig": {
|
||||||
|
"serviceName": "foo"
|
||||||
|
}
|
||||||
|
}`)
|
||||||
|
r.NewAddress([]resolver.Address{{Addr: lis.Addr().String()}})
|
||||||
|
|
||||||
|
// make some rpcs to make sure connection is working.
|
||||||
|
if err := verifyResultWithDelay(func() (bool, error) {
|
||||||
|
if _, err := tc.EmptyCall(context.Background(), &testpb.Empty{}); err != nil {
|
||||||
|
return false, fmt.Errorf("TestService/EmptyCall(_, _) = _, %v, want _, <nil>", err)
|
||||||
|
}
|
||||||
|
return true, nil
|
||||||
|
}); err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
select {
|
||||||
|
case <-hcExitChan:
|
||||||
|
t.Fatal("Health check function has exited, which is not expected.")
|
||||||
|
default:
|
||||||
|
}
|
||||||
|
// server closes the connection
|
||||||
|
s.Stop()
|
||||||
|
|
||||||
|
select {
|
||||||
|
case <-hcExitChan:
|
||||||
|
case <-time.After(5 * time.Second):
|
||||||
|
t.Fatal("Health check function has not exited after 5s.")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// addrConn drain happens when addrConn gets torn down due to its address being no longer in the
|
||||||
|
// address list returned by the resolver.
|
||||||
|
func TestHealthCheckWithAddrConnDrain(t *testing.T) {
|
||||||
|
defer leakcheck.Check(t)
|
||||||
|
s := grpc.NewServer()
|
||||||
|
lis, err := net.Listen("tcp", "localhost:0")
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("failed to listen due to err: %v", err)
|
||||||
|
}
|
||||||
|
ts := newTestHealthServer()
|
||||||
|
healthpb.RegisterHealthServer(s, ts)
|
||||||
|
testpb.RegisterTestServiceServer(s, &testServer{})
|
||||||
|
go s.Serve(lis)
|
||||||
|
defer s.Stop()
|
||||||
|
ts.SetServingStatus("foo", healthpb.HealthCheckResponse_SERVING)
|
||||||
|
hcExitChan := make(chan struct{})
|
||||||
|
testHealthCheckFuncWrapper := func(ctx context.Context, newStream func() (interface{}, error), update func(bool), service string) error {
|
||||||
|
err := testHealthCheckFunc(ctx, newStream, update, service)
|
||||||
|
close(hcExitChan)
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
replace := replaceHealthCheckFunc(testHealthCheckFuncWrapper)
|
||||||
|
defer replace()
|
||||||
|
r, rcleanup := manual.GenerateAndRegisterManualResolver()
|
||||||
|
defer rcleanup()
|
||||||
|
cc, err := grpc.Dial(r.Scheme()+":///test.server", grpc.WithInsecure(), grpc.WithBalancerName("round_robin"))
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("dial failed due to err: %v", err)
|
||||||
|
}
|
||||||
|
defer cc.Close()
|
||||||
|
|
||||||
|
tc := testpb.NewTestServiceClient(cc)
|
||||||
|
r.NewServiceConfig(`{
|
||||||
|
"healthCheckConfig": {
|
||||||
|
"serviceName": "foo"
|
||||||
|
}
|
||||||
|
}`)
|
||||||
|
r.NewAddress([]resolver.Address{{Addr: lis.Addr().String()}})
|
||||||
|
|
||||||
|
// make some rpcs to make sure connection is working.
|
||||||
|
if err := verifyResultWithDelay(func() (bool, error) {
|
||||||
|
if _, err := tc.EmptyCall(context.Background(), &testpb.Empty{}); err != nil {
|
||||||
|
return false, fmt.Errorf("TestService/EmptyCall(_, _) = _, %v, want _, <nil>", err)
|
||||||
|
}
|
||||||
|
return true, nil
|
||||||
|
}); err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
// the stream rpc will persist through goaway event.
|
||||||
|
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
|
||||||
|
defer cancel()
|
||||||
|
stream, err := tc.FullDuplexCall(ctx, grpc.FailFast(false))
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("%v.FullDuplexCall(_) = _, %v, want <nil>", tc, err)
|
||||||
|
}
|
||||||
|
respParam := []*testpb.ResponseParameters{{Size: 1}}
|
||||||
|
payload, err := newPayload(testpb.PayloadType_COMPRESSABLE, int32(1))
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
req := &testpb.StreamingOutputCallRequest{
|
||||||
|
ResponseParameters: respParam,
|
||||||
|
Payload: payload,
|
||||||
|
}
|
||||||
|
if err := stream.Send(req); err != nil {
|
||||||
|
t.Fatalf("%v.Send(_) = %v, want <nil>", stream, err)
|
||||||
|
}
|
||||||
|
if _, err := stream.Recv(); err != nil {
|
||||||
|
t.Fatalf("%v.Recv() = _, %v, want _, <nil>", stream, err)
|
||||||
|
}
|
||||||
|
|
||||||
|
select {
|
||||||
|
case <-hcExitChan:
|
||||||
|
t.Fatal("Health check function has exited, which is not expected.")
|
||||||
|
default:
|
||||||
|
}
|
||||||
|
// trigger teardown of the ac
|
||||||
|
r.NewAddress([]resolver.Address{})
|
||||||
|
|
||||||
|
select {
|
||||||
|
case <-hcExitChan:
|
||||||
|
case <-time.After(5 * time.Second):
|
||||||
|
t.Fatal("Health check function has not exited after 5s.")
|
||||||
|
}
|
||||||
|
|
||||||
|
// The existing RPC should be still good to proceed.
|
||||||
|
if err := stream.Send(req); err != nil {
|
||||||
|
t.Fatalf("%v.Send(_) = %v, want <nil>", stream, err)
|
||||||
|
}
|
||||||
|
if _, err := stream.Recv(); err != nil {
|
||||||
|
t.Fatalf("%v.Recv() = _, %v, want _, <nil>", stream, err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// ClientConn close will lead to its addrConns being torn down.
|
||||||
|
func TestHealthCheckWithClientConnClose(t *testing.T) {
|
||||||
|
defer leakcheck.Check(t)
|
||||||
|
s := grpc.NewServer()
|
||||||
|
lis, err := net.Listen("tcp", "localhost:0")
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("failed to listen due to err: %v", err)
|
||||||
|
}
|
||||||
|
ts := newTestHealthServer()
|
||||||
|
healthpb.RegisterHealthServer(s, ts)
|
||||||
|
testpb.RegisterTestServiceServer(s, &testServer{})
|
||||||
|
go s.Serve(lis)
|
||||||
|
defer s.Stop()
|
||||||
|
ts.SetServingStatus("foo", healthpb.HealthCheckResponse_SERVING)
|
||||||
|
hcExitChan := make(chan struct{})
|
||||||
|
testHealthCheckFuncWrapper := func(ctx context.Context, newStream func() (interface{}, error), update func(bool), service string) error {
|
||||||
|
err := testHealthCheckFunc(ctx, newStream, update, service)
|
||||||
|
close(hcExitChan)
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
replace := replaceHealthCheckFunc(testHealthCheckFuncWrapper)
|
||||||
|
defer replace()
|
||||||
|
r, rcleanup := manual.GenerateAndRegisterManualResolver()
|
||||||
|
defer rcleanup()
|
||||||
|
cc, err := grpc.Dial(r.Scheme()+":///test.server", grpc.WithInsecure(), grpc.WithBalancerName("round_robin"))
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("dial failed due to err: %v", err)
|
||||||
|
}
|
||||||
|
defer cc.Close()
|
||||||
|
|
||||||
|
tc := testpb.NewTestServiceClient(cc)
|
||||||
|
r.NewServiceConfig(`{
|
||||||
|
"healthCheckConfig": {
|
||||||
|
"serviceName": "foo"
|
||||||
|
}
|
||||||
|
}`)
|
||||||
|
r.NewAddress([]resolver.Address{{Addr: lis.Addr().String()}})
|
||||||
|
|
||||||
|
// make some rpcs to make sure connection is working.
|
||||||
|
if err := verifyResultWithDelay(func() (bool, error) {
|
||||||
|
if _, err := tc.EmptyCall(context.Background(), &testpb.Empty{}); err != nil {
|
||||||
|
return false, fmt.Errorf("TestService/EmptyCall(_, _) = _, %v, want _, <nil>", err)
|
||||||
|
}
|
||||||
|
return true, nil
|
||||||
|
}); err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
select {
|
||||||
|
case <-hcExitChan:
|
||||||
|
t.Fatal("Health check function has exited, which is not expected.")
|
||||||
|
default:
|
||||||
|
}
|
||||||
|
|
||||||
|
// trigger addrConn teardown
|
||||||
|
cc.Close()
|
||||||
|
|
||||||
|
select {
|
||||||
|
case <-hcExitChan:
|
||||||
|
case <-time.After(5 * time.Second):
|
||||||
|
t.Fatal("Health check function has not exited after 5s.")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// This test is to test the logic in the createTransport after the health check function returns which
|
||||||
|
// closes the skipReset channel(since it has not been closed inside health check func) to unblock
|
||||||
|
// onGoAway/onClose goroutine.
|
||||||
|
func TestHealthCheckWithoutReportHealthCalledAddrConnShutDown(t *testing.T) {
|
||||||
|
defer leakcheck.Check(t)
|
||||||
|
s := grpc.NewServer()
|
||||||
|
lis, err := net.Listen("tcp", "localhost:0")
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("failed to listen due to err %v", err)
|
||||||
|
}
|
||||||
|
ts := newTestHealthServerWithWatchFunc(func(s *testHealthServer, in *healthpb.HealthCheckRequest, stream healthpb.Health_WatchServer) error {
|
||||||
|
if in.Service != "delay" {
|
||||||
|
return status.Error(codes.FailedPrecondition,
|
||||||
|
"this special Watch function only handles request with service name to be \"delay\"")
|
||||||
|
}
|
||||||
|
// Do nothing to mock a delay of health check response from server side.
|
||||||
|
// This case is to help with the test that covers the condition that reportHealth is not
|
||||||
|
// called inside HealthCheckFunc before the func returns.
|
||||||
|
select {
|
||||||
|
case <-stream.Context().Done():
|
||||||
|
case <-time.After(5 * time.Second):
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
})
|
||||||
|
healthpb.RegisterHealthServer(s, ts)
|
||||||
|
testpb.RegisterTestServiceServer(s, &testServer{})
|
||||||
|
go s.Serve(lis)
|
||||||
|
defer s.Stop()
|
||||||
|
ts.SetServingStatus("delay", healthpb.HealthCheckResponse_SERVING)
|
||||||
|
|
||||||
|
hcEnterChan := make(chan struct{})
|
||||||
|
hcExitChan := make(chan struct{})
|
||||||
|
testHealthCheckFuncWrapper := func(ctx context.Context, newStream func() (interface{}, error), update func(bool), service string) error {
|
||||||
|
close(hcEnterChan)
|
||||||
|
err := testHealthCheckFunc(ctx, newStream, update, service)
|
||||||
|
close(hcExitChan)
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
replace := replaceHealthCheckFunc(testHealthCheckFuncWrapper)
|
||||||
|
defer replace()
|
||||||
|
r, rcleanup := manual.GenerateAndRegisterManualResolver()
|
||||||
|
defer rcleanup()
|
||||||
|
cc, err := grpc.Dial(r.Scheme()+":///test.server", grpc.WithInsecure(), grpc.WithBalancerName("round_robin"))
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("dial failed due to err: %v", err)
|
||||||
|
}
|
||||||
|
defer cc.Close()
|
||||||
|
|
||||||
|
// The serviceName "delay" is specially handled at server side, where response will not be sent
|
||||||
|
// back to client immediately upon receiving the request (client should receive no response until
|
||||||
|
// test ends).
|
||||||
|
r.NewServiceConfig(`{
|
||||||
|
"healthCheckConfig": {
|
||||||
|
"serviceName": "delay"
|
||||||
|
}
|
||||||
|
}`)
|
||||||
|
r.NewAddress([]resolver.Address{{Addr: lis.Addr().String()}})
|
||||||
|
|
||||||
|
select {
|
||||||
|
case <-hcExitChan:
|
||||||
|
t.Fatal("Health check function has exited, which is not expected.")
|
||||||
|
default:
|
||||||
|
}
|
||||||
|
|
||||||
|
select {
|
||||||
|
case <-hcEnterChan:
|
||||||
|
case <-time.After(5 * time.Second):
|
||||||
|
t.Fatal("Health check function has not been invoked after 5s.")
|
||||||
|
}
|
||||||
|
// trigger teardown of the ac, ac in SHUTDOWN state
|
||||||
|
r.NewAddress([]resolver.Address{})
|
||||||
|
|
||||||
|
// The health check func should exit without calling the reportHealth func, as server hasn't sent
|
||||||
|
// any response.
|
||||||
|
select {
|
||||||
|
case <-hcExitChan:
|
||||||
|
case <-time.After(5 * time.Second):
|
||||||
|
t.Fatal("Health check function has not exited after 5s.")
|
||||||
|
}
|
||||||
|
// The deferred leakcheck will check whether there's leaked goroutine, which is an indication
|
||||||
|
// whether we closes the skipReset channel to unblock onGoAway/onClose goroutine.
|
||||||
|
}
|
||||||
|
|
||||||
|
// This test is to test the logic in the createTransport after the health check function returns which
|
||||||
|
// closes the allowedToReset channel(since it has not been closed inside health check func) to unblock
|
||||||
|
// onGoAway/onClose goroutine.
|
||||||
|
func TestHealthCheckWithoutReportHealthCalled(t *testing.T) {
|
||||||
|
defer leakcheck.Check(t)
|
||||||
|
s := grpc.NewServer()
|
||||||
|
lis, err := net.Listen("tcp", "localhost:0")
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("failed to listen due to err: %v", err)
|
||||||
|
}
|
||||||
|
ts := newTestHealthServerWithWatchFunc(func(s *testHealthServer, in *healthpb.HealthCheckRequest, stream healthpb.Health_WatchServer) error {
|
||||||
|
if in.Service != "delay" {
|
||||||
|
return status.Error(codes.FailedPrecondition,
|
||||||
|
"this special Watch function only handles request with service name to be \"delay\"")
|
||||||
|
}
|
||||||
|
// Do nothing to mock a delay of health check response from server side.
|
||||||
|
// This case is to help with the test that covers the condition that reportHealth is not
|
||||||
|
// called inside HealthCheckFunc before the func returns.
|
||||||
|
select {
|
||||||
|
case <-stream.Context().Done():
|
||||||
|
case <-time.After(5 * time.Second):
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
})
|
||||||
|
healthpb.RegisterHealthServer(s, ts)
|
||||||
|
testpb.RegisterTestServiceServer(s, &testServer{})
|
||||||
|
go s.Serve(lis)
|
||||||
|
defer s.Stop()
|
||||||
|
ts.SetServingStatus("delay", healthpb.HealthCheckResponse_SERVING)
|
||||||
|
|
||||||
|
hcEnterChan := make(chan struct{})
|
||||||
|
hcExitChan := make(chan struct{})
|
||||||
|
testHealthCheckFuncWrapper := func(ctx context.Context, newStream func() (interface{}, error), update func(bool), service string) error {
|
||||||
|
close(hcEnterChan)
|
||||||
|
err := testHealthCheckFunc(ctx, newStream, update, service)
|
||||||
|
close(hcExitChan)
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
replace := replaceHealthCheckFunc(testHealthCheckFuncWrapper)
|
||||||
|
defer replace()
|
||||||
|
r, rcleanup := manual.GenerateAndRegisterManualResolver()
|
||||||
|
defer rcleanup()
|
||||||
|
cc, err := grpc.Dial(r.Scheme()+":///test.server", grpc.WithInsecure(), grpc.WithBalancerName("round_robin"))
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("dial failed due to err: %v", err)
|
||||||
|
}
|
||||||
|
defer cc.Close()
|
||||||
|
|
||||||
|
// The serviceName "delay" is specially handled at server side, where response will not be sent
|
||||||
|
// back to client immediately upon receiving the request (client should receive no response until
|
||||||
|
// test ends).
|
||||||
|
r.NewServiceConfig(`{
|
||||||
|
"healthCheckConfig": {
|
||||||
|
"serviceName": "delay"
|
||||||
|
}
|
||||||
|
}`)
|
||||||
|
r.NewAddress([]resolver.Address{{Addr: lis.Addr().String()}})
|
||||||
|
|
||||||
|
select {
|
||||||
|
case <-hcExitChan:
|
||||||
|
t.Fatal("Health check function has exited, which is not expected.")
|
||||||
|
default:
|
||||||
|
}
|
||||||
|
|
||||||
|
select {
|
||||||
|
case <-hcEnterChan:
|
||||||
|
case <-time.After(5 * time.Second):
|
||||||
|
t.Fatal("Health check function has not been invoked after 5s.")
|
||||||
|
}
|
||||||
|
// trigger transport being closed
|
||||||
|
s.Stop()
|
||||||
|
|
||||||
|
// The health check func should exit without calling the reportHealth func, as server hasn't sent
|
||||||
|
// any response.
|
||||||
|
select {
|
||||||
|
case <-hcExitChan:
|
||||||
|
case <-time.After(5 * time.Second):
|
||||||
|
t.Fatal("Health check function has not exited after 5s.")
|
||||||
|
}
|
||||||
|
// The deferred leakcheck will check whether there's leaked goroutine, which is an indication
|
||||||
|
// whether we closes the allowedToReset channel to unblock onGoAway/onClose goroutine.
|
||||||
|
}
|
||||||
|
|
||||||
|
func testHealthCheckDisableWithDialOption(t *testing.T, addr string) {
|
||||||
|
hcEnterChan := make(chan struct{})
|
||||||
|
testHealthCheckFuncWrapper := func(ctx context.Context, newStream func() (interface{}, error), update func(bool), service string) error {
|
||||||
|
close(hcEnterChan)
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
replace := replaceHealthCheckFunc(testHealthCheckFuncWrapper)
|
||||||
|
defer replace()
|
||||||
|
r, rcleanup := manual.GenerateAndRegisterManualResolver()
|
||||||
|
defer rcleanup()
|
||||||
|
cc, err := grpc.Dial(r.Scheme()+":///test.server", grpc.WithInsecure(), grpc.WithBalancerName("round_robin"), grpc.WithDisableHealthCheck())
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("dial failed due to err: %v", err)
|
||||||
|
}
|
||||||
|
tc := testpb.NewTestServiceClient(cc)
|
||||||
|
defer cc.Close()
|
||||||
|
r.NewServiceConfig(`{
|
||||||
|
"healthCheckConfig": {
|
||||||
|
"serviceName": "foo"
|
||||||
|
}
|
||||||
|
}`)
|
||||||
|
r.NewAddress([]resolver.Address{{Addr: addr}})
|
||||||
|
|
||||||
|
// send some rpcs to make sure transport has been created and is ready for use.
|
||||||
|
if err := verifyResultWithDelay(func() (bool, error) {
|
||||||
|
if _, err := tc.EmptyCall(context.Background(), &testpb.Empty{}); err != nil {
|
||||||
|
return false, fmt.Errorf("TestService/EmptyCall(_, _) = _, %v, want _, <nil>", err)
|
||||||
|
}
|
||||||
|
return true, nil
|
||||||
|
}); err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
select {
|
||||||
|
case <-hcEnterChan:
|
||||||
|
t.Fatal("Health check function has exited, which is not expected.")
|
||||||
|
default:
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func testHealthCheckDisableWithBalancer(t *testing.T, addr string) {
|
||||||
|
hcEnterChan := make(chan struct{})
|
||||||
|
testHealthCheckFuncWrapper := func(ctx context.Context, newStream func() (interface{}, error), update func(bool), service string) error {
|
||||||
|
close(hcEnterChan)
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
replace := replaceHealthCheckFunc(testHealthCheckFuncWrapper)
|
||||||
|
defer replace()
|
||||||
|
r, rcleanup := manual.GenerateAndRegisterManualResolver()
|
||||||
|
defer rcleanup()
|
||||||
|
cc, err := grpc.Dial(r.Scheme()+":///test.server", grpc.WithInsecure(), grpc.WithBalancerName("pick_first"))
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("dial failed due to err: %v", err)
|
||||||
|
}
|
||||||
|
tc := testpb.NewTestServiceClient(cc)
|
||||||
|
defer cc.Close()
|
||||||
|
r.NewServiceConfig(`{
|
||||||
|
"healthCheckConfig": {
|
||||||
|
"serviceName": "foo"
|
||||||
|
}
|
||||||
|
}`)
|
||||||
|
r.NewAddress([]resolver.Address{{Addr: addr}})
|
||||||
|
|
||||||
|
// send some rpcs to make sure transport has been created and is ready for use.
|
||||||
|
if err := verifyResultWithDelay(func() (bool, error) {
|
||||||
|
if _, err := tc.EmptyCall(context.Background(), &testpb.Empty{}); err != nil {
|
||||||
|
return false, fmt.Errorf("TestService/EmptyCall(_, _) = _, %v, want _, <nil>", err)
|
||||||
|
}
|
||||||
|
return true, nil
|
||||||
|
}); err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
select {
|
||||||
|
case <-hcEnterChan:
|
||||||
|
t.Fatal("Health check function has started, which is not expected.")
|
||||||
|
default:
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func testHealthCheckDisableWithServiceConfig(t *testing.T, addr string) {
|
||||||
|
hcEnterChan := make(chan struct{})
|
||||||
|
testHealthCheckFuncWrapper := func(ctx context.Context, newStream func() (interface{}, error), update func(bool), service string) error {
|
||||||
|
close(hcEnterChan)
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
replace := replaceHealthCheckFunc(testHealthCheckFuncWrapper)
|
||||||
|
defer replace()
|
||||||
|
r, rcleanup := manual.GenerateAndRegisterManualResolver()
|
||||||
|
defer rcleanup()
|
||||||
|
cc, err := grpc.Dial(r.Scheme()+":///test.server", grpc.WithInsecure(), grpc.WithBalancerName("round_robin"))
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("dial failed due to err: %v", err)
|
||||||
|
}
|
||||||
|
tc := testpb.NewTestServiceClient(cc)
|
||||||
|
defer cc.Close()
|
||||||
|
|
||||||
|
r.NewAddress([]resolver.Address{{Addr: addr}})
|
||||||
|
|
||||||
|
// send some rpcs to make sure transport has been created and is ready for use.
|
||||||
|
if err := verifyResultWithDelay(func() (bool, error) {
|
||||||
|
if _, err := tc.EmptyCall(context.Background(), &testpb.Empty{}); err != nil {
|
||||||
|
return false, fmt.Errorf("TestService/EmptyCall(_, _) = _, %v, want _, <nil>", err)
|
||||||
|
}
|
||||||
|
return true, nil
|
||||||
|
}); err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
select {
|
||||||
|
case <-hcEnterChan:
|
||||||
|
t.Fatal("Health check function has started, which is not expected.")
|
||||||
|
default:
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestHealthCheckDisable(t *testing.T) {
|
||||||
|
defer leakcheck.Check(t)
|
||||||
|
// set up server side
|
||||||
|
s := grpc.NewServer()
|
||||||
|
lis, err := net.Listen("tcp", "localhost:0")
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("failed to listen due to err: %v", err)
|
||||||
|
}
|
||||||
|
ts := newTestHealthServer()
|
||||||
|
healthpb.RegisterHealthServer(s, ts)
|
||||||
|
testpb.RegisterTestServiceServer(s, &testServer{})
|
||||||
|
go s.Serve(lis)
|
||||||
|
defer s.Stop()
|
||||||
|
ts.SetServingStatus("foo", healthpb.HealthCheckResponse_SERVING)
|
||||||
|
|
||||||
|
// test client side disabling configuration.
|
||||||
|
testHealthCheckDisableWithDialOption(t, lis.Addr().String())
|
||||||
|
testHealthCheckDisableWithBalancer(t, lis.Addr().String())
|
||||||
|
testHealthCheckDisableWithServiceConfig(t, lis.Addr().String())
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestHealthCheckChannelzCountingCallSuccess(t *testing.T) {
|
||||||
|
defer leakcheck.Check(t)
|
||||||
|
s := grpc.NewServer()
|
||||||
|
lis, err := net.Listen("tcp", "localhost:0")
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("failed to listen due to err: %v", err)
|
||||||
|
}
|
||||||
|
ts := newTestHealthServerWithWatchFunc(func(s *testHealthServer, in *healthpb.HealthCheckRequest, stream healthpb.Health_WatchServer) error {
|
||||||
|
if in.Service != "channelzSuccess" {
|
||||||
|
return status.Error(codes.FailedPrecondition,
|
||||||
|
"this special Watch function only handles request with service name to be \"channelzSuccess\"")
|
||||||
|
}
|
||||||
|
return status.Error(codes.OK, "fake success")
|
||||||
|
})
|
||||||
|
healthpb.RegisterHealthServer(s, ts)
|
||||||
|
testpb.RegisterTestServiceServer(s, &testServer{})
|
||||||
|
go s.Serve(lis)
|
||||||
|
defer s.Stop()
|
||||||
|
|
||||||
|
r, rcleanup := manual.GenerateAndRegisterManualResolver()
|
||||||
|
defer rcleanup()
|
||||||
|
cc, err := grpc.Dial(r.Scheme()+":///test.server", grpc.WithInsecure(), grpc.WithBalancerName("round_robin"))
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("dial failed due to err: %v", err)
|
||||||
|
}
|
||||||
|
defer cc.Close()
|
||||||
|
|
||||||
|
r.NewServiceConfig(`{
|
||||||
|
"healthCheckConfig": {
|
||||||
|
"serviceName": "channelzSuccess"
|
||||||
|
}
|
||||||
|
}`)
|
||||||
|
r.NewAddress([]resolver.Address{{Addr: lis.Addr().String()}})
|
||||||
|
|
||||||
|
if err := verifyResultWithDelay(func() (bool, error) {
|
||||||
|
cm, _ := channelz.GetTopChannels(0)
|
||||||
|
if len(cm) == 0 {
|
||||||
|
return false, errors.New("channelz.GetTopChannels return 0 top channel")
|
||||||
|
}
|
||||||
|
if len(cm[0].SubChans) == 0 {
|
||||||
|
return false, errors.New("there is 0 subchannel")
|
||||||
|
}
|
||||||
|
var id int64
|
||||||
|
for k := range cm[0].SubChans {
|
||||||
|
id = k
|
||||||
|
break
|
||||||
|
}
|
||||||
|
scm := channelz.GetSubChannel(id)
|
||||||
|
if scm == nil || scm.ChannelData == nil {
|
||||||
|
return false, errors.New("nil subchannel metric or nil subchannel metric ChannelData returned")
|
||||||
|
}
|
||||||
|
// exponential backoff retry may result in more than one health check call.
|
||||||
|
if scm.ChannelData.CallsStarted > 0 && scm.ChannelData.CallsSucceeded > 0 && scm.ChannelData.CallsFailed == 0 {
|
||||||
|
return true, nil
|
||||||
|
}
|
||||||
|
return false, fmt.Errorf("got %d CallsStarted, %d CallsSucceeded, want >0 >0", scm.ChannelData.CallsStarted, scm.ChannelData.CallsSucceeded)
|
||||||
|
}); err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestHealthCheckChannelzCountingCallFailure(t *testing.T) {
|
||||||
|
defer leakcheck.Check(t)
|
||||||
|
s := grpc.NewServer()
|
||||||
|
lis, err := net.Listen("tcp", "localhost:0")
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("failed to listen due to err: %v", err)
|
||||||
|
}
|
||||||
|
ts := newTestHealthServerWithWatchFunc(func(s *testHealthServer, in *healthpb.HealthCheckRequest, stream healthpb.Health_WatchServer) error {
|
||||||
|
if in.Service != "channelzFailure" {
|
||||||
|
return status.Error(codes.FailedPrecondition,
|
||||||
|
"this special Watch function only handles request with service name to be \"channelzFailure\"")
|
||||||
|
}
|
||||||
|
return status.Error(codes.Internal, "fake failure")
|
||||||
|
})
|
||||||
|
healthpb.RegisterHealthServer(s, ts)
|
||||||
|
testpb.RegisterTestServiceServer(s, &testServer{})
|
||||||
|
go s.Serve(lis)
|
||||||
|
defer s.Stop()
|
||||||
|
|
||||||
|
r, rcleanup := manual.GenerateAndRegisterManualResolver()
|
||||||
|
defer rcleanup()
|
||||||
|
cc, err := grpc.Dial(r.Scheme()+":///test.server", grpc.WithInsecure(), grpc.WithBalancerName("round_robin"))
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("dial failed due to err: %v", err)
|
||||||
|
}
|
||||||
|
defer cc.Close()
|
||||||
|
|
||||||
|
r.NewServiceConfig(`{
|
||||||
|
"healthCheckConfig": {
|
||||||
|
"serviceName": "channelzFailure"
|
||||||
|
}
|
||||||
|
}`)
|
||||||
|
r.NewAddress([]resolver.Address{{Addr: lis.Addr().String()}})
|
||||||
|
|
||||||
|
if err := verifyResultWithDelay(func() (bool, error) {
|
||||||
|
cm, _ := channelz.GetTopChannels(0)
|
||||||
|
if len(cm) == 0 {
|
||||||
|
return false, errors.New("channelz.GetTopChannels return 0 top channel")
|
||||||
|
}
|
||||||
|
if len(cm[0].SubChans) == 0 {
|
||||||
|
return false, errors.New("there is 0 subchannel")
|
||||||
|
}
|
||||||
|
var id int64
|
||||||
|
for k := range cm[0].SubChans {
|
||||||
|
id = k
|
||||||
|
break
|
||||||
|
}
|
||||||
|
scm := channelz.GetSubChannel(id)
|
||||||
|
if scm == nil || scm.ChannelData == nil {
|
||||||
|
return false, errors.New("nil subchannel metric or nil subchannel metric ChannelData returned")
|
||||||
|
}
|
||||||
|
// exponential backoff retry may result in more than one health check call.
|
||||||
|
if scm.ChannelData.CallsStarted > 0 && scm.ChannelData.CallsFailed > 0 && scm.ChannelData.CallsSucceeded == 0 {
|
||||||
|
return true, nil
|
||||||
|
}
|
||||||
|
return false, fmt.Errorf("got %d CallsStarted, %d CallsFailed, want >0, >0", scm.ChannelData.CallsStarted, scm.ChannelData.CallsFailed)
|
||||||
|
}); err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
}
|
||||||
Loading…
Reference in New Issue