Add new Resolver and Balancer APIs (gRFC L9) (#1408)
- Add package balancer and resolver. - Change ClientConn internals to new APIs and adds a wrapper for v1 balancer.
This commit is contained in:
498
clientconn.go
498
clientconn.go
@ -22,16 +22,19 @@ import (
|
||||
"errors"
|
||||
"math"
|
||||
"net"
|
||||
"reflect"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
"golang.org/x/net/trace"
|
||||
"google.golang.org/grpc/balancer"
|
||||
"google.golang.org/grpc/connectivity"
|
||||
"google.golang.org/grpc/credentials"
|
||||
"google.golang.org/grpc/grpclog"
|
||||
"google.golang.org/grpc/keepalive"
|
||||
"google.golang.org/grpc/resolver"
|
||||
"google.golang.org/grpc/stats"
|
||||
"google.golang.org/grpc/transport"
|
||||
)
|
||||
@ -79,13 +82,14 @@ type dialOptions struct {
|
||||
cp Compressor
|
||||
dc Decompressor
|
||||
bs backoffStrategy
|
||||
balancer Balancer
|
||||
block bool
|
||||
insecure bool
|
||||
timeout time.Duration
|
||||
scChan <-chan ServiceConfig
|
||||
copts transport.ConnectOptions
|
||||
callOptions []CallOption
|
||||
// This is to support v1 balancer.
|
||||
balancerBuilder balancer.Builder
|
||||
}
|
||||
|
||||
const (
|
||||
@ -147,10 +151,14 @@ func WithDecompressor(dc Decompressor) DialOption {
|
||||
}
|
||||
}
|
||||
|
||||
// WithBalancer returns a DialOption which sets a load balancer.
|
||||
// WithBalancer returns a DialOption which sets a load balancer with the v1 API.
|
||||
// Name resolver will be ignored if this DialOption is specified.
|
||||
// Deprecated: use the new balancer APIs in balancer package instead.
|
||||
func WithBalancer(b Balancer) DialOption {
|
||||
return func(o *dialOptions) {
|
||||
o.balancer = b
|
||||
o.balancerBuilder = &balancerWrapperBuilder{
|
||||
b: b,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -314,7 +322,7 @@ func DialContext(ctx context.Context, target string, opts ...DialOption) (conn *
|
||||
cc := &ClientConn{
|
||||
target: target,
|
||||
csMgr: &connectivityStateManager{},
|
||||
conns: make(map[Address]*addrConn),
|
||||
conns: make(map[*addrConn]struct{}),
|
||||
}
|
||||
cc.csEvltr = &connectivityStateEvaluator{csMgr: cc.csMgr}
|
||||
cc.ctx, cc.cancel = context.WithCancel(context.Background())
|
||||
@ -383,49 +391,42 @@ func DialContext(ctx context.Context, target string, opts ...DialOption) (conn *
|
||||
} else {
|
||||
cc.authority = target
|
||||
}
|
||||
waitC := make(chan error, 1)
|
||||
go func() {
|
||||
defer close(waitC)
|
||||
if cc.dopts.balancer == nil && cc.sc.LB != nil {
|
||||
cc.dopts.balancer = cc.sc.LB
|
||||
|
||||
// TODO(bar) parse scheme and start resolver.
|
||||
if cc.dopts.balancerBuilder != nil {
|
||||
var credsClone credentials.TransportCredentials
|
||||
if creds != nil {
|
||||
credsClone = creds.Clone()
|
||||
}
|
||||
if cc.dopts.balancer != nil {
|
||||
var credsClone credentials.TransportCredentials
|
||||
if creds != nil {
|
||||
credsClone = creds.Clone()
|
||||
}
|
||||
config := BalancerConfig{
|
||||
DialCreds: credsClone,
|
||||
Dialer: cc.dopts.copts.Dialer,
|
||||
}
|
||||
if err := cc.dopts.balancer.Start(target, config); err != nil {
|
||||
buildOpts := balancer.BuildOptions{
|
||||
DialCreds: credsClone,
|
||||
Dialer: cc.dopts.copts.Dialer,
|
||||
}
|
||||
// Build should not take long time. So it's ok to not have a goroutine for it.
|
||||
// TODO(bar) init balancer after first resolver result to support service config balancer.
|
||||
cc.balancer = cc.dopts.balancerBuilder.Build(&ccBalancerWrapper{cc: cc}, buildOpts)
|
||||
} else {
|
||||
waitC := make(chan error, 1)
|
||||
go func() {
|
||||
defer close(waitC)
|
||||
// No balancer, or no resolver within the balancer. Connect directly.
|
||||
ac, err := cc.newAddrConn([]resolver.Address{{Addr: target}})
|
||||
if err != nil {
|
||||
waitC <- err
|
||||
return
|
||||
}
|
||||
ch := cc.dopts.balancer.Notify()
|
||||
if ch != nil {
|
||||
if cc.dopts.block {
|
||||
doneChan := make(chan struct{})
|
||||
go cc.lbWatcher(doneChan)
|
||||
<-doneChan
|
||||
} else {
|
||||
go cc.lbWatcher(nil)
|
||||
}
|
||||
if err := ac.connect(cc.dopts.block); err != nil {
|
||||
waitC <- err
|
||||
return
|
||||
}
|
||||
}
|
||||
// No balancer, or no resolver within the balancer. Connect directly.
|
||||
if err := cc.resetAddrConn([]Address{{Addr: target}}, cc.dopts.block, nil); err != nil {
|
||||
waitC <- err
|
||||
return
|
||||
}
|
||||
}()
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return nil, ctx.Err()
|
||||
case err := <-waitC:
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}()
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return nil, ctx.Err()
|
||||
case err := <-waitC:
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
}
|
||||
if cc.dopts.scChan != nil && !scSet {
|
||||
@ -443,6 +444,27 @@ func DialContext(ctx context.Context, target string, opts ...DialOption) (conn *
|
||||
go cc.scWatcher()
|
||||
}
|
||||
|
||||
if cc.balancer != nil {
|
||||
// Unblock balancer initialization with a fake resolver update.
|
||||
// The balancer wrapper will not read the addresses, so an empty list works.
|
||||
// TODO(bar) remove this after the real resolver is started.
|
||||
cc.balancer.HandleResolvedAddrs([]resolver.Address{}, nil)
|
||||
}
|
||||
|
||||
// A blocking dial blocks until the clientConn is ready.
|
||||
if cc.dopts.block {
|
||||
for {
|
||||
s := cc.GetState()
|
||||
if s == connectivity.Ready {
|
||||
break
|
||||
}
|
||||
if !cc.WaitForStateChange(ctx, s) {
|
||||
// ctx got timeout or canceled.
|
||||
return nil, ctx.Err()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return cc, nil
|
||||
}
|
||||
|
||||
@ -548,9 +570,15 @@ type ClientConn struct {
|
||||
csMgr *connectivityStateManager
|
||||
csEvltr *connectivityStateEvaluator // This will eventually be part of balancer.
|
||||
|
||||
balancer balancer.Balancer
|
||||
|
||||
// TODO(bar) move the mutex and picker into a struct that does blocking pick().
|
||||
pmu sync.Mutex
|
||||
picker balancer.Picker
|
||||
|
||||
mu sync.RWMutex
|
||||
sc ServiceConfig
|
||||
conns map[Address]*addrConn
|
||||
conns map[*addrConn]struct{}
|
||||
// Keepalive parameter can be updated if a GoAway is received.
|
||||
mkp keepalive.ClientParameters
|
||||
}
|
||||
@ -577,84 +605,6 @@ func (cc *ClientConn) GetState() connectivity.State {
|
||||
return cc.csMgr.getState()
|
||||
}
|
||||
|
||||
// lbWatcher watches the Notify channel of the balancer in cc and manages
|
||||
// connections accordingly. If doneChan is not nil, it is closed after the
|
||||
// first successfull connection is made.
|
||||
func (cc *ClientConn) lbWatcher(doneChan chan struct{}) {
|
||||
defer func() {
|
||||
// In case channel from cc.dopts.balancer.Notify() gets closed before a
|
||||
// successful connection gets established, don't forget to notify the
|
||||
// caller.
|
||||
if doneChan != nil {
|
||||
close(doneChan)
|
||||
}
|
||||
}()
|
||||
|
||||
_, isPickFirst := cc.dopts.balancer.(*pickFirst)
|
||||
for addrs := range cc.dopts.balancer.Notify() {
|
||||
if isPickFirst {
|
||||
if len(addrs) == 0 {
|
||||
// No address can be connected, should teardown current addrconn if exists
|
||||
cc.mu.Lock()
|
||||
if len(cc.conns) != 0 {
|
||||
cc.pickFirstAddrConnTearDown()
|
||||
}
|
||||
cc.mu.Unlock()
|
||||
} else {
|
||||
cc.resetAddrConn(addrs, true, nil)
|
||||
if doneChan != nil {
|
||||
close(doneChan)
|
||||
doneChan = nil
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// Not pickFirst, create a new addrConn for each address.
|
||||
var (
|
||||
add []Address // Addresses need to setup connections.
|
||||
del []*addrConn // Connections need to tear down.
|
||||
)
|
||||
cc.mu.Lock()
|
||||
for _, a := range addrs {
|
||||
if _, ok := cc.conns[a]; !ok {
|
||||
add = append(add, a)
|
||||
}
|
||||
}
|
||||
for k, c := range cc.conns {
|
||||
var keep bool
|
||||
for _, a := range addrs {
|
||||
if k == a {
|
||||
keep = true
|
||||
break
|
||||
}
|
||||
}
|
||||
if !keep {
|
||||
del = append(del, c)
|
||||
delete(cc.conns, k)
|
||||
}
|
||||
}
|
||||
cc.mu.Unlock()
|
||||
for _, a := range add {
|
||||
var err error
|
||||
if doneChan != nil {
|
||||
err = cc.resetAddrConn([]Address{a}, true, nil)
|
||||
if err == nil {
|
||||
close(doneChan)
|
||||
doneChan = nil
|
||||
}
|
||||
} else {
|
||||
err = cc.resetAddrConn([]Address{a}, false, nil)
|
||||
}
|
||||
if err != nil {
|
||||
grpclog.Warningf("Error creating connection to %v. Err: %v", a, err)
|
||||
}
|
||||
}
|
||||
for _, c := range del {
|
||||
c.tearDown(errConnDrain)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (cc *ClientConn) scWatcher() {
|
||||
for {
|
||||
select {
|
||||
@ -673,72 +623,8 @@ func (cc *ClientConn) scWatcher() {
|
||||
}
|
||||
}
|
||||
|
||||
// pickFirstUpdateAddresses checks whether current address in the updating list, Update the list if true.
|
||||
// It is only used when the balancer is pick first.
|
||||
func (cc *ClientConn) pickFirstUpdateAddresses(addrs []Address) bool {
|
||||
if len(cc.conns) == 0 {
|
||||
// No addrconn. Should go resetting addrconn.
|
||||
return false
|
||||
}
|
||||
var currentAc *addrConn
|
||||
for _, currentAc = range cc.conns {
|
||||
break
|
||||
}
|
||||
var addrInNewSlice bool
|
||||
for _, addr := range addrs {
|
||||
if strings.Compare(addr.Addr, currentAc.curAddr.Addr) == 0 {
|
||||
addrInNewSlice = true
|
||||
break
|
||||
}
|
||||
}
|
||||
if addrInNewSlice {
|
||||
cc.conns = make(map[Address]*addrConn)
|
||||
for _, addr := range addrs {
|
||||
cc.conns[addr] = currentAc
|
||||
}
|
||||
currentAc.addrs = addrs
|
||||
return true
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
// pickFirstAddrConnTearDown() should be called after lock.
|
||||
func (cc *ClientConn) pickFirstAddrConnTearDown() {
|
||||
if len(cc.conns) == 0 {
|
||||
return
|
||||
}
|
||||
var currentAc *addrConn
|
||||
for _, currentAc = range cc.conns {
|
||||
break
|
||||
}
|
||||
for k := range cc.conns {
|
||||
delete(cc.conns, k)
|
||||
}
|
||||
currentAc.tearDown(errConnDrain)
|
||||
}
|
||||
|
||||
// resetAddrConn creates an addrConn for addr and adds it to cc.conns.
|
||||
// If there is an old addrConn for addr, it will be torn down, using tearDownErr as the reason.
|
||||
// If tearDownErr is nil, errConnDrain will be used instead.
|
||||
//
|
||||
// We should never need to replace an addrConn with a new one. This function is only used
|
||||
// as newAddrConn to create new addrConn.
|
||||
// TODO rename this function and clean up the code.
|
||||
func (cc *ClientConn) resetAddrConn(addrs []Address, block bool, tearDownErr error) error {
|
||||
// if current transport in addrs, just change lists to update order and new addresses
|
||||
// not work for roundrobin
|
||||
cc.mu.Lock()
|
||||
if _, isPickFirst := cc.dopts.balancer.(*pickFirst); isPickFirst {
|
||||
// If Current address in use in the updating list, just update the list.
|
||||
// Otherwise, teardown current addrconn and create a new one.
|
||||
if cc.pickFirstUpdateAddresses(addrs) {
|
||||
cc.mu.Unlock()
|
||||
return nil
|
||||
}
|
||||
cc.pickFirstAddrConnTearDown()
|
||||
}
|
||||
cc.mu.Unlock()
|
||||
|
||||
// newAddrConn creates an addrConn for addrs and adds it to cc.conns.
|
||||
func (cc *ClientConn) newAddrConn(addrs []resolver.Address) (*addrConn, error) {
|
||||
ac := &addrConn{
|
||||
cc: cc,
|
||||
addrs: addrs,
|
||||
@ -746,6 +632,42 @@ func (cc *ClientConn) resetAddrConn(addrs []Address, block bool, tearDownErr err
|
||||
}
|
||||
ac.ctx, ac.cancel = context.WithCancel(cc.ctx)
|
||||
ac.csEvltr = cc.csEvltr
|
||||
// Track ac in cc. This needs to be done before any getTransport(...) is called.
|
||||
cc.mu.Lock()
|
||||
if cc.conns == nil {
|
||||
cc.mu.Unlock()
|
||||
return nil, ErrClientConnClosing
|
||||
}
|
||||
cc.conns[ac] = struct{}{}
|
||||
cc.mu.Unlock()
|
||||
return ac, nil
|
||||
}
|
||||
|
||||
// removeAddrConn removes the addrConn in the subConn from clientConn.
|
||||
// It also tears down the ac with the given error.
|
||||
func (cc *ClientConn) removeAddrConn(ac *addrConn, err error) {
|
||||
cc.mu.Lock()
|
||||
if cc.conns == nil {
|
||||
cc.mu.Unlock()
|
||||
return
|
||||
}
|
||||
delete(cc.conns, ac)
|
||||
cc.mu.Unlock()
|
||||
ac.tearDown(err)
|
||||
}
|
||||
|
||||
// connect starts to creating transport and also starts the transport monitor
|
||||
// goroutine for this ac.
|
||||
// TODO(bar) Move this to the addrConn section.
|
||||
// This was part of resetAddrConn, keep it here to make the diff look clean.
|
||||
func (ac *addrConn) connect(block bool) error {
|
||||
ac.mu.Lock()
|
||||
if ac.state == connectivity.Shutdown {
|
||||
ac.mu.Unlock()
|
||||
return errConnClosing
|
||||
}
|
||||
ac.mu.Unlock()
|
||||
|
||||
if EnableTracing {
|
||||
ac.events = trace.NewEventLog("grpc.ClientConn", ac.addrs[0].Addr)
|
||||
}
|
||||
@ -763,37 +685,10 @@ func (cc *ClientConn) resetAddrConn(addrs []Address, block bool, tearDownErr err
|
||||
}
|
||||
}
|
||||
}
|
||||
// Track ac in cc. This needs to be done before any getTransport(...) is called.
|
||||
cc.mu.Lock()
|
||||
if cc.conns == nil {
|
||||
cc.mu.Unlock()
|
||||
return ErrClientConnClosing
|
||||
}
|
||||
stale := cc.conns[ac.addrs[0]]
|
||||
for _, a := range ac.addrs {
|
||||
cc.conns[a] = ac
|
||||
}
|
||||
cc.mu.Unlock()
|
||||
if stale != nil {
|
||||
// There is an addrConn alive on ac.addr already. This could be due to
|
||||
// a buggy Balancer that reports duplicated Addresses.
|
||||
if tearDownErr == nil {
|
||||
// tearDownErr is nil if resetAddrConn is called by
|
||||
// 1) Dial
|
||||
// 2) lbWatcher
|
||||
// In both cases, the stale ac should drain, not close.
|
||||
stale.tearDown(errConnDrain)
|
||||
} else {
|
||||
stale.tearDown(tearDownErr)
|
||||
}
|
||||
}
|
||||
|
||||
if block {
|
||||
if err := ac.resetTransport(false); err != nil {
|
||||
if err != errConnClosing {
|
||||
// Tear down ac and delete it from cc.conns.
|
||||
cc.mu.Lock()
|
||||
delete(cc.conns, ac.addrs[0])
|
||||
cc.mu.Unlock()
|
||||
ac.tearDown(err)
|
||||
}
|
||||
if e, ok := err.(transport.ConnectionError); ok && !e.Temporary() {
|
||||
@ -820,6 +715,36 @@ func (cc *ClientConn) resetAddrConn(addrs []Address, block bool, tearDownErr err
|
||||
return nil
|
||||
}
|
||||
|
||||
// tryUpdateAddrs tries to update ac.addrs with the new addresses list.
|
||||
//
|
||||
// It checks whether current connected address of ac is in the new addrs list.
|
||||
// - If true, it updates ac.addrs and returns true. The ac will keep using
|
||||
// the existing connection.
|
||||
// - If false, it does nothing and returns false.
|
||||
func (ac *addrConn) tryUpdateAddrs(addrs []resolver.Address) bool {
|
||||
ac.mu.Lock()
|
||||
defer ac.mu.Unlock()
|
||||
grpclog.Infof("addrConn: tryUpdateAddrs curAddr: %v, addrs: %v", ac.curAddr, addrs)
|
||||
if ac.state == connectivity.Shutdown {
|
||||
ac.addrs = addrs
|
||||
return true
|
||||
}
|
||||
|
||||
var curAddrFound bool
|
||||
for _, a := range addrs {
|
||||
if reflect.DeepEqual(ac.curAddr, a) {
|
||||
curAddrFound = true
|
||||
break
|
||||
}
|
||||
}
|
||||
grpclog.Infof("addrConn: tryUpdateAddrs curAddrFound: %v", curAddrFound)
|
||||
if curAddrFound {
|
||||
ac.addrs = addrs
|
||||
}
|
||||
|
||||
return curAddrFound
|
||||
}
|
||||
|
||||
// GetMethodConfig gets the method config of the input method.
|
||||
// If there's an exact match for input method (i.e. /service/method), we return
|
||||
// the corresponding MethodConfig.
|
||||
@ -839,54 +764,52 @@ func (cc *ClientConn) GetMethodConfig(method string) MethodConfig {
|
||||
return m
|
||||
}
|
||||
|
||||
func (cc *ClientConn) getTransport(ctx context.Context, opts BalancerGetOptions) (transport.ClientTransport, func(), error) {
|
||||
func (cc *ClientConn) getTransport(ctx context.Context, opts BalancerGetOptions) (transport.ClientTransport, func(balancer.DoneInfo), error) {
|
||||
var (
|
||||
ac *addrConn
|
||||
ok bool
|
||||
put func()
|
||||
put func(balancer.DoneInfo)
|
||||
)
|
||||
if cc.dopts.balancer == nil {
|
||||
if cc.balancer == nil {
|
||||
// If balancer is nil, there should be only one addrConn available.
|
||||
cc.mu.RLock()
|
||||
if cc.conns == nil {
|
||||
cc.mu.RUnlock()
|
||||
return nil, nil, toRPCErr(ErrClientConnClosing)
|
||||
}
|
||||
for _, ac = range cc.conns {
|
||||
for ac = range cc.conns {
|
||||
// Break after the first iteration to get the first addrConn.
|
||||
ok = true
|
||||
break
|
||||
}
|
||||
cc.mu.RUnlock()
|
||||
} else {
|
||||
cc.pmu.Lock()
|
||||
// TODO(bar) call pick on struct blockPicker instead of the real picker.
|
||||
p := cc.picker
|
||||
cc.pmu.Unlock()
|
||||
|
||||
var (
|
||||
addr Address
|
||||
err error
|
||||
err error
|
||||
sc balancer.SubConn
|
||||
)
|
||||
addr, put, err = cc.dopts.balancer.Get(ctx, opts)
|
||||
sc, put, err = p.Pick(ctx, balancer.PickOptions{})
|
||||
if err != nil {
|
||||
return nil, nil, toRPCErr(err)
|
||||
}
|
||||
cc.mu.RLock()
|
||||
if cc.conns == nil {
|
||||
cc.mu.RUnlock()
|
||||
return nil, nil, toRPCErr(ErrClientConnClosing)
|
||||
}
|
||||
ac, ok = cc.conns[addr]
|
||||
cc.mu.RUnlock()
|
||||
}
|
||||
if !ok {
|
||||
if put != nil {
|
||||
if acbw, ok := sc.(*acBalancerWrapper); ok {
|
||||
ac = acbw.getAddrConn()
|
||||
} else if put != nil {
|
||||
updateRPCInfoInContext(ctx, rpcInfo{bytesSent: false, bytesReceived: false})
|
||||
put()
|
||||
put(balancer.DoneInfo{Err: errors.New("SubConn returned by pick cannot be recognized")})
|
||||
}
|
||||
}
|
||||
if ac == nil {
|
||||
return nil, nil, errConnClosing
|
||||
}
|
||||
t, err := ac.wait(ctx, cc.dopts.balancer != nil, !opts.BlockingWait)
|
||||
t, err := ac.wait(ctx, cc.balancer != nil, !opts.BlockingWait)
|
||||
if err != nil {
|
||||
if put != nil {
|
||||
updateRPCInfoInContext(ctx, rpcInfo{bytesSent: false, bytesReceived: false})
|
||||
put()
|
||||
put(balancer.DoneInfo{Err: err})
|
||||
}
|
||||
return nil, nil, err
|
||||
}
|
||||
@ -906,10 +829,10 @@ func (cc *ClientConn) Close() error {
|
||||
cc.conns = nil
|
||||
cc.csMgr.updateState(connectivity.Shutdown)
|
||||
cc.mu.Unlock()
|
||||
if cc.dopts.balancer != nil {
|
||||
cc.dopts.balancer.Close()
|
||||
if cc.balancer != nil {
|
||||
cc.balancer.Close()
|
||||
}
|
||||
for _, ac := range conns {
|
||||
for ac := range conns {
|
||||
ac.tearDown(ErrClientConnClosing)
|
||||
}
|
||||
return nil
|
||||
@ -921,16 +844,16 @@ type addrConn struct {
|
||||
cancel context.CancelFunc
|
||||
|
||||
cc *ClientConn
|
||||
curAddr Address
|
||||
addrs []Address
|
||||
curAddr resolver.Address
|
||||
addrs []resolver.Address
|
||||
dopts dialOptions
|
||||
events trace.EventLog
|
||||
acbw balancer.SubConn
|
||||
|
||||
csEvltr *connectivityStateEvaluator
|
||||
|
||||
mu sync.Mutex
|
||||
state connectivity.State
|
||||
down func(error) // the handler called when a connection is down.
|
||||
// ready is closed and becomes nil when a new transport is up or failed
|
||||
// due to timeout.
|
||||
ready chan struct{}
|
||||
@ -974,20 +897,24 @@ func (ac *addrConn) errorf(format string, a ...interface{}) {
|
||||
// For the old transport:
|
||||
// - if drain is true, it will be gracefully closed.
|
||||
// - otherwise, it will be closed.
|
||||
// TODO(bar) make sure all state transitions are valid.
|
||||
func (ac *addrConn) resetTransport(drain bool) error {
|
||||
ac.mu.Lock()
|
||||
if ac.state == connectivity.Shutdown {
|
||||
ac.mu.Unlock()
|
||||
return errConnClosing
|
||||
}
|
||||
ac.printf("connecting")
|
||||
if ac.down != nil {
|
||||
ac.down(downErrorf(false, true, "%v", errNetworkIO))
|
||||
ac.down = nil
|
||||
}
|
||||
oldState := ac.state
|
||||
ac.state = connectivity.Connecting
|
||||
ac.csEvltr.recordTransition(oldState, ac.state)
|
||||
if ac.cc.balancer != nil {
|
||||
ac.cc.balancer.HandleSubConnStateChange(ac.acbw, ac.state)
|
||||
}
|
||||
// TODO(bar) don't call balancer functions to handle subconn state change if ac.acbw is nil.
|
||||
if ac.ready != nil {
|
||||
close(ac.ready)
|
||||
ac.ready = nil
|
||||
}
|
||||
t := ac.transport
|
||||
ac.transport = nil
|
||||
ac.mu.Unlock()
|
||||
@ -998,15 +925,27 @@ func (ac *addrConn) resetTransport(drain bool) error {
|
||||
ac.dopts.copts.KeepaliveParams = ac.cc.mkp
|
||||
ac.cc.mu.RUnlock()
|
||||
for retries := 0; ; retries++ {
|
||||
ac.mu.Lock()
|
||||
sleepTime := ac.dopts.bs.backoff(retries)
|
||||
timeout := minConnectTimeout
|
||||
ac.mu.Lock()
|
||||
if timeout < time.Duration(int(sleepTime)/len(ac.addrs)) {
|
||||
timeout = time.Duration(int(sleepTime) / len(ac.addrs))
|
||||
}
|
||||
connectTime := time.Now()
|
||||
if ac.state == connectivity.Shutdown {
|
||||
ac.mu.Unlock()
|
||||
return errConnClosing
|
||||
}
|
||||
ac.printf("connecting")
|
||||
oldState := ac.state
|
||||
ac.state = connectivity.Connecting
|
||||
ac.csEvltr.recordTransition(oldState, ac.state)
|
||||
// TODO(bar) remove condition once we always have a balancer.
|
||||
if ac.cc.balancer != nil {
|
||||
ac.cc.balancer.HandleSubConnStateChange(ac.acbw, ac.state)
|
||||
}
|
||||
// copy ac.addrs in case of race
|
||||
addrsIter := make([]Address, len(ac.addrs))
|
||||
addrsIter := make([]resolver.Address, len(ac.addrs))
|
||||
copy(addrsIter, ac.addrs)
|
||||
ac.mu.Unlock()
|
||||
for _, addr := range addrsIter {
|
||||
@ -1038,14 +977,6 @@ func (ac *addrConn) resetTransport(drain bool) error {
|
||||
ac.mu.Unlock()
|
||||
return errConnClosing
|
||||
}
|
||||
ac.errorf("transient failure: %v", err)
|
||||
oldState = ac.state
|
||||
ac.state = connectivity.TransientFailure
|
||||
ac.csEvltr.recordTransition(oldState, ac.state)
|
||||
if ac.ready != nil {
|
||||
close(ac.ready)
|
||||
ac.ready = nil
|
||||
}
|
||||
ac.mu.Unlock()
|
||||
continue
|
||||
}
|
||||
@ -1060,18 +991,30 @@ func (ac *addrConn) resetTransport(drain bool) error {
|
||||
oldState = ac.state
|
||||
ac.state = connectivity.Ready
|
||||
ac.csEvltr.recordTransition(oldState, ac.state)
|
||||
if ac.cc.balancer != nil {
|
||||
ac.cc.balancer.HandleSubConnStateChange(ac.acbw, ac.state)
|
||||
}
|
||||
ac.transport = newTransport
|
||||
ac.curAddr = addr
|
||||
if ac.ready != nil {
|
||||
close(ac.ready)
|
||||
ac.ready = nil
|
||||
}
|
||||
if ac.cc.dopts.balancer != nil {
|
||||
ac.down = ac.cc.dopts.balancer.Up(addr)
|
||||
}
|
||||
ac.curAddr = addr
|
||||
ac.mu.Unlock()
|
||||
return nil
|
||||
}
|
||||
ac.mu.Lock()
|
||||
oldState = ac.state
|
||||
ac.state = connectivity.TransientFailure
|
||||
ac.csEvltr.recordTransition(oldState, ac.state)
|
||||
if ac.cc.balancer != nil {
|
||||
ac.cc.balancer.HandleSubConnStateChange(ac.acbw, ac.state)
|
||||
}
|
||||
if ac.ready != nil {
|
||||
close(ac.ready)
|
||||
ac.ready = nil
|
||||
}
|
||||
ac.mu.Unlock()
|
||||
timer := time.NewTimer(sleepTime - time.Since(connectTime))
|
||||
select {
|
||||
case <-timer.C:
|
||||
@ -1128,26 +1071,8 @@ func (ac *addrConn) transportMonitor() {
|
||||
return
|
||||
case <-t.GoAway():
|
||||
ac.adjustParams(t.GetGoAwayReason())
|
||||
if err := ac.resetTransport(false); err != nil {
|
||||
grpclog.Infof("get error from resetTransport %v, transportMonitor returning", err)
|
||||
if err != errConnClosing {
|
||||
// Keep this ac in cc.conns, to get the reason it's torn down.
|
||||
ac.tearDown(err)
|
||||
}
|
||||
return
|
||||
}
|
||||
default:
|
||||
}
|
||||
ac.mu.Lock()
|
||||
if ac.state == connectivity.Shutdown {
|
||||
// ac has been shutdown.
|
||||
ac.mu.Unlock()
|
||||
return
|
||||
}
|
||||
oldState := ac.state
|
||||
ac.state = connectivity.TransientFailure
|
||||
ac.csEvltr.recordTransition(oldState, ac.state)
|
||||
ac.mu.Unlock()
|
||||
if err := ac.resetTransport(false); err != nil {
|
||||
grpclog.Infof("get error from resetTransport %v, transportMonitor returning", err)
|
||||
ac.mu.Lock()
|
||||
@ -1213,12 +1138,8 @@ func (ac *addrConn) tearDown(err error) {
|
||||
ac.cancel()
|
||||
|
||||
ac.mu.Lock()
|
||||
ac.curAddr = Address{}
|
||||
ac.curAddr = resolver.Address{}
|
||||
defer ac.mu.Unlock()
|
||||
if ac.down != nil {
|
||||
ac.down(downErrorf(false, false, "%v", err))
|
||||
ac.down = nil
|
||||
}
|
||||
if err == errConnDrain && ac.transport != nil {
|
||||
// GracefulClose(...) may be executed multiple times when
|
||||
// i) receiving multiple GoAway frames from the server; or
|
||||
@ -1233,6 +1154,9 @@ func (ac *addrConn) tearDown(err error) {
|
||||
ac.state = connectivity.Shutdown
|
||||
ac.tearDownErr = err
|
||||
ac.csEvltr.recordTransition(oldState, ac.state)
|
||||
if ac.cc.balancer != nil {
|
||||
ac.cc.balancer.HandleSubConnStateChange(ac.acbw, ac.state)
|
||||
}
|
||||
if ac.events != nil {
|
||||
ac.events.Finish()
|
||||
ac.events = nil
|
||||
@ -1246,3 +1170,9 @@ func (ac *addrConn) tearDown(err error) {
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
func (ac *addrConn) getState() connectivity.State {
|
||||
ac.mu.Lock()
|
||||
defer ac.mu.Unlock()
|
||||
return ac.state
|
||||
}
|
||||
|
Reference in New Issue
Block a user