|
|
@@ -43,28 +43,35 @@ import (
|
|
|
|
|
|
"golang.org/x/net/context"
|
|
|
"golang.org/x/net/trace"
|
|
|
+ "google.golang.org/grpc/codes"
|
|
|
"google.golang.org/grpc/credentials"
|
|
|
"google.golang.org/grpc/grpclog"
|
|
|
"google.golang.org/grpc/transport"
|
|
|
)
|
|
|
|
|
|
var (
|
|
|
- // ErrUnspecTarget indicates that the target address is unspecified.
|
|
|
- ErrUnspecTarget = errors.New("grpc: target is unspecified")
|
|
|
- // ErrNoTransportSecurity indicates that there is no transport security
|
|
|
+ // ErrClientConnClosing indicates that the operation is illegal because
|
|
|
+ // the ClientConn is closing.
|
|
|
+ ErrClientConnClosing = errors.New("grpc: the client connection is closing")
|
|
|
+ // ErrClientConnTimeout indicates that the ClientConn cannot establish the
|
|
|
+ // underlying connections within the specified timeout.
|
|
|
+ ErrClientConnTimeout = errors.New("grpc: timed out when dialing")
|
|
|
+
|
|
|
+ // errNoTransportSecurity indicates that there is no transport security
|
|
|
// being set for ClientConn. Users should either set one or explicitly
|
|
|
// call WithInsecure DialOption to disable security.
|
|
|
- ErrNoTransportSecurity = errors.New("grpc: no transport security set (use grpc.WithInsecure() explicitly or set credentials)")
|
|
|
- // ErrCredentialsMisuse indicates that users want to transmit security information
|
|
|
+ errNoTransportSecurity = errors.New("grpc: no transport security set (use grpc.WithInsecure() explicitly or set credentials)")
|
|
|
+ // errCredentialsMisuse indicates that users want to transmit security information
|
|
|
// (e.g., oauth2 token) which requires secure connection on an insecure
|
|
|
// connection.
|
|
|
- ErrCredentialsMisuse = errors.New("grpc: the credentials require transport level security (use grpc.WithTransportAuthenticator() to set)")
|
|
|
- // ErrClientConnClosing indicates that the operation is illegal because
|
|
|
- // the session is closing.
|
|
|
- ErrClientConnClosing = errors.New("grpc: the client connection is closing")
|
|
|
- // ErrClientConnTimeout indicates that the connection could not be
|
|
|
- // established or re-established within the specified timeout.
|
|
|
- ErrClientConnTimeout = errors.New("grpc: timed out trying to connect")
|
|
|
+ errCredentialsMisuse = errors.New("grpc: the credentials require transport level security (use grpc.WithTransportAuthenticator() to set)")
|
|
|
+ // errNetworkIP indicates that the connection is down due to some network I/O error.
|
|
|
+ errNetworkIO = errors.New("grpc: failed with network I/O error")
|
|
|
+ // errConnDrain indicates that the connection starts to be drained and does not accept any new RPCs.
|
|
|
+ errConnDrain = errors.New("grpc: the connection is drained")
|
|
|
+ // errConnClosing indicates that the connection is closing.
|
|
|
+ errConnClosing = errors.New("grpc: the connection is closing")
|
|
|
+ errNoAddr = errors.New("grpc: there is no address available to dial")
|
|
|
// minimum time to give a connection to complete
|
|
|
minConnectTimeout = 20 * time.Second
|
|
|
)
|
|
|
@@ -76,9 +83,10 @@ type dialOptions struct {
|
|
|
cp Compressor
|
|
|
dc Decompressor
|
|
|
bs backoffStrategy
|
|
|
- picker Picker
|
|
|
+ balancer Balancer
|
|
|
block bool
|
|
|
insecure bool
|
|
|
+ timeout time.Duration
|
|
|
copts transport.ConnectOptions
|
|
|
}
|
|
|
|
|
|
@@ -108,10 +116,10 @@ func WithDecompressor(dc Decompressor) DialOption {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-// WithPicker returns a DialOption which sets a picker for connection selection.
|
|
|
-func WithPicker(p Picker) DialOption {
|
|
|
+// WithBalancer returns a DialOption which sets a load balancer.
|
|
|
+func WithBalancer(b Balancer) DialOption {
|
|
|
return func(o *dialOptions) {
|
|
|
- o.picker = p
|
|
|
+ o.balancer = b
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@@ -136,7 +144,7 @@ func WithBackoffConfig(b BackoffConfig) DialOption {
|
|
|
// withBackoff sets the backoff strategy used for retries after a
|
|
|
// failed connection attempt.
|
|
|
//
|
|
|
-// This can be exported if arbitrary backoff strategies are allowed by GRPC.
|
|
|
+// This can be exported if arbitrary backoff strategies are allowed by gRPC.
|
|
|
func withBackoff(bs backoffStrategy) DialOption {
|
|
|
return func(o *dialOptions) {
|
|
|
o.bs = bs
|
|
|
@@ -176,10 +184,11 @@ func WithPerRPCCredentials(creds credentials.Credentials) DialOption {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-// WithTimeout returns a DialOption that configures a timeout for dialing a client connection.
|
|
|
+// WithTimeout returns a DialOption that configures a timeout for dialing a ClientConn
|
|
|
+// initially. This is valid if and only if WithBlock() is present.
|
|
|
func WithTimeout(d time.Duration) DialOption {
|
|
|
return func(o *dialOptions) {
|
|
|
- o.copts.Timeout = d
|
|
|
+ o.timeout = d
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@@ -201,6 +210,7 @@ func WithUserAgent(s string) DialOption {
|
|
|
func Dial(target string, opts ...DialOption) (*ClientConn, error) {
|
|
|
cc := &ClientConn{
|
|
|
target: target,
|
|
|
+ conns: make(map[Address]*addrConn),
|
|
|
}
|
|
|
for _, opt := range opts {
|
|
|
opt(&cc.dopts)
|
|
|
@@ -214,14 +224,54 @@ func Dial(target string, opts ...DialOption) (*ClientConn, error) {
|
|
|
cc.dopts.bs = DefaultBackoffConfig
|
|
|
}
|
|
|
|
|
|
- if cc.dopts.picker == nil {
|
|
|
- cc.dopts.picker = &unicastPicker{
|
|
|
- target: target,
|
|
|
- }
|
|
|
+ cc.balancer = cc.dopts.balancer
|
|
|
+ if cc.balancer == nil {
|
|
|
+ cc.balancer = RoundRobin(nil)
|
|
|
}
|
|
|
- if err := cc.dopts.picker.Init(cc); err != nil {
|
|
|
+ if err := cc.balancer.Start(target); err != nil {
|
|
|
return nil, err
|
|
|
}
|
|
|
+ var (
|
|
|
+ ok bool
|
|
|
+ addrs []Address
|
|
|
+ )
|
|
|
+ ch := cc.balancer.Notify()
|
|
|
+ if ch == nil {
|
|
|
+ // There is no name resolver installed.
|
|
|
+ addrs = append(addrs, Address{Addr: target})
|
|
|
+ } else {
|
|
|
+ addrs, ok = <-ch
|
|
|
+ if !ok || len(addrs) == 0 {
|
|
|
+ return nil, errNoAddr
|
|
|
+ }
|
|
|
+ }
|
|
|
+ waitC := make(chan error, 1)
|
|
|
+ go func() {
|
|
|
+ for _, a := range addrs {
|
|
|
+ if err := cc.newAddrConn(a, false); err != nil {
|
|
|
+ waitC <- err
|
|
|
+ return
|
|
|
+ }
|
|
|
+ }
|
|
|
+ close(waitC)
|
|
|
+ }()
|
|
|
+ var timeoutCh <-chan time.Time
|
|
|
+ if cc.dopts.timeout > 0 {
|
|
|
+ timeoutCh = time.After(cc.dopts.timeout)
|
|
|
+ }
|
|
|
+ select {
|
|
|
+ case err := <-waitC:
|
|
|
+ if err != nil {
|
|
|
+ cc.Close()
|
|
|
+ return nil, err
|
|
|
+ }
|
|
|
+ case <-timeoutCh:
|
|
|
+ cc.Close()
|
|
|
+ return nil, ErrClientConnTimeout
|
|
|
+ }
|
|
|
+ if ok {
|
|
|
+ go cc.lbWatcher()
|
|
|
+ }
|
|
|
colonPos := strings.LastIndex(target, ":")
|
|
|
if colonPos == -1 {
|
|
|
colonPos = len(target)
|
|
|
@@ -263,328 +313,361 @@ func (s ConnectivityState) String() string {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-// ClientConn represents a client connection to an RPC service.
|
|
|
+// ClientConn represents a client connection to an RPC server.
|
|
|
type ClientConn struct {
|
|
|
target string
|
|
|
+ balancer Balancer
|
|
|
authority string
|
|
|
dopts dialOptions
|
|
|
-}
|
|
|
|
|
|
-// State returns the connectivity state of cc.
|
|
|
-// This is EXPERIMENTAL API.
|
|
|
-func (cc *ClientConn) State() (ConnectivityState, error) {
|
|
|
- return cc.dopts.picker.State()
|
|
|
+ mu sync.RWMutex
|
|
|
+ conns map[Address]*addrConn
|
|
|
}
|
|
|
|
|
|
-// WaitForStateChange blocks until the state changes to something other than the sourceState.
|
|
|
-// It returns the new state or error.
|
|
|
-// This is EXPERIMENTAL API.
|
|
|
-func (cc *ClientConn) WaitForStateChange(ctx context.Context, sourceState ConnectivityState) (ConnectivityState, error) {
|
|
|
- return cc.dopts.picker.WaitForStateChange(ctx, sourceState)
|
|
|
-}
|
|
|
-
|
|
|
-// Close starts to tear down the ClientConn.
|
|
|
-func (cc *ClientConn) Close() error {
|
|
|
- return cc.dopts.picker.Close()
|
|
|
-}
|
|
|
-
|
|
|
-// Conn is a client connection to a single destination.
|
|
|
-type Conn struct {
|
|
|
- target string
|
|
|
- dopts dialOptions
|
|
|
- resetChan chan int
|
|
|
- shutdownChan chan struct{}
|
|
|
- events trace.EventLog
|
|
|
-
|
|
|
- mu sync.Mutex
|
|
|
- state ConnectivityState
|
|
|
- stateCV *sync.Cond
|
|
|
- // ready is closed and becomes nil when a new transport is up or failed
|
|
|
- // due to timeout.
|
|
|
- ready chan struct{}
|
|
|
- transport transport.ClientTransport
|
|
|
+func (cc *ClientConn) lbWatcher() {
|
|
|
+ for addrs := range cc.balancer.Notify() {
|
|
|
+ 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)
|
|
|
+ }
|
|
|
+ }
|
|
|
+ cc.mu.Unlock()
|
|
|
+ for _, a := range add {
|
|
|
+ cc.newAddrConn(a, true)
|
|
|
+ }
|
|
|
+ for _, c := range del {
|
|
|
+ c.tearDown(errConnDrain)
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
-// NewConn creates a Conn.
|
|
|
-func NewConn(cc *ClientConn) (*Conn, error) {
|
|
|
- if cc.target == "" {
|
|
|
- return nil, ErrUnspecTarget
|
|
|
- }
|
|
|
- c := &Conn{
|
|
|
- target: cc.target,
|
|
|
+func (cc *ClientConn) newAddrConn(addr Address, skipWait bool) error {
|
|
|
+ ac := &addrConn{
|
|
|
+ cc: cc,
|
|
|
+ addr: addr,
|
|
|
dopts: cc.dopts,
|
|
|
- resetChan: make(chan int, 1),
|
|
|
shutdownChan: make(chan struct{}),
|
|
|
}
|
|
|
if EnableTracing {
|
|
|
- c.events = trace.NewEventLog("grpc.ClientConn", c.target)
|
|
|
+ ac.events = trace.NewEventLog("grpc.ClientConn", ac.addr.Addr)
|
|
|
}
|
|
|
- if !c.dopts.insecure {
|
|
|
+ if !ac.dopts.insecure {
|
|
|
var ok bool
|
|
|
- for _, cd := range c.dopts.copts.AuthOptions {
|
|
|
+ for _, cd := range ac.dopts.copts.AuthOptions {
|
|
|
if _, ok = cd.(credentials.TransportAuthenticator); ok {
|
|
|
break
|
|
|
}
|
|
|
}
|
|
|
if !ok {
|
|
|
- return nil, ErrNoTransportSecurity
|
|
|
+ return errNoTransportSecurity
|
|
|
}
|
|
|
} else {
|
|
|
- for _, cd := range c.dopts.copts.AuthOptions {
|
|
|
+ for _, cd := range ac.dopts.copts.AuthOptions {
|
|
|
if cd.RequireTransportSecurity() {
|
|
|
- return nil, ErrCredentialsMisuse
|
|
|
+ return errCredentialsMisuse
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
- c.stateCV = sync.NewCond(&c.mu)
|
|
|
- if c.dopts.block {
|
|
|
- if err := c.resetTransport(false); err != nil {
|
|
|
- c.Close()
|
|
|
- return nil, err
|
|
|
+ // Insert ac into ac.cc.conns. This needs to be done before any getTransport(...) is called.
|
|
|
+ ac.cc.mu.Lock()
|
|
|
+ if ac.cc.conns == nil {
|
|
|
+ ac.cc.mu.Unlock()
|
|
|
+ return ErrClientConnClosing
|
|
|
+ }
|
|
|
+ stale := ac.cc.conns[ac.addr]
|
|
|
+ ac.cc.conns[ac.addr] = ac
|
|
|
+ ac.cc.mu.Unlock()
|
|
|
+ if stale != nil {
|
|
|
+ // There is an addrConn alive on ac.addr already. This could be due to
|
|
|
+ // i) stale's Close is undergoing;
|
|
|
+ // ii) a buggy Balancer notifies duplicated Addresses.
|
|
|
+ stale.tearDown(errConnDrain)
|
|
|
+ }
|
|
|
+ ac.stateCV = sync.NewCond(&ac.mu)
|
|
|
+ // skipWait may overwrite the decision in ac.dopts.block.
|
|
|
+ if ac.dopts.block && !skipWait {
|
|
|
+ if err := ac.resetTransport(false); err != nil {
|
|
|
+ ac.tearDown(err)
|
|
|
+ return err
|
|
|
}
|
|
|
// Start to monitor the error status of transport.
|
|
|
- go c.transportMonitor()
|
|
|
+ go ac.transportMonitor()
|
|
|
} else {
|
|
|
// Start a goroutine connecting to the server asynchronously.
|
|
|
go func() {
|
|
|
- if err := c.resetTransport(false); err != nil {
|
|
|
- grpclog.Printf("Failed to dial %s: %v; please retry.", c.target, err)
|
|
|
- c.Close()
|
|
|
+ if err := ac.resetTransport(false); err != nil {
|
|
|
+ grpclog.Printf("Failed to dial %s: %v; please retry.", ac.addr.Addr, err)
|
|
|
+ ac.tearDown(err)
|
|
|
return
|
|
|
}
|
|
|
- c.transportMonitor()
|
|
|
+ ac.transportMonitor()
|
|
|
}()
|
|
|
}
|
|
|
- return c, nil
|
|
|
+ return nil
|
|
|
}
|
|
|
|
|
|
-// printf records an event in cc's event log, unless cc has been closed.
|
|
|
-// REQUIRES cc.mu is held.
|
|
|
-func (cc *Conn) printf(format string, a ...interface{}) {
|
|
|
- if cc.events != nil {
|
|
|
- cc.events.Printf(format, a...)
|
|
|
+func (cc *ClientConn) getTransport(ctx context.Context, opts BalancerGetOptions) (transport.ClientTransport, func(), error) {
|
|
|
+ // TODO(zhaoq): Implement fail-fast logic.
|
|
|
+ addr, put, err := cc.balancer.Get(ctx, opts)
|
|
|
+ if err != nil {
|
|
|
+ return nil, nil, err
|
|
|
+ }
|
|
|
+ cc.mu.RLock()
|
|
|
+ if cc.conns == nil {
|
|
|
+ cc.mu.RUnlock()
|
|
|
+ return nil, nil, ErrClientConnClosing
|
|
|
}
|
|
|
+ ac, ok := cc.conns[addr]
|
|
|
+ cc.mu.RUnlock()
|
|
|
+ if !ok {
|
|
|
+ if put != nil {
|
|
|
+ put()
|
|
|
+ }
|
|
|
+ return nil, nil, transport.StreamErrorf(codes.Internal, "grpc: failed to find the transport to send the rpc")
|
|
|
+ }
|
|
|
+ t, err := ac.wait(ctx)
|
|
|
+ if err != nil {
|
|
|
+ if put != nil {
|
|
|
+ put()
|
|
|
+ }
|
|
|
+ return nil, nil, err
|
|
|
+ }
|
|
|
+ return t, put, nil
|
|
|
}
|
|
|
|
|
|
-// errorf records an error in cc's event log, unless cc has been closed.
|
|
|
-// REQUIRES cc.mu is held.
|
|
|
-func (cc *Conn) errorf(format string, a ...interface{}) {
|
|
|
- if cc.events != nil {
|
|
|
- cc.events.Errorf(format, a...)
|
|
|
+// Close tears down the ClientConn and all underlying connections.
|
|
|
+func (cc *ClientConn) Close() error {
|
|
|
+ cc.mu.Lock()
|
|
|
+ if cc.conns == nil {
|
|
|
+ cc.mu.Unlock()
|
|
|
+ return ErrClientConnClosing
|
|
|
+ }
|
|
|
+ conns := cc.conns
|
|
|
+ cc.conns = nil
|
|
|
+ cc.mu.Unlock()
|
|
|
+ cc.balancer.Close()
|
|
|
+ for _, ac := range conns {
|
|
|
+ ac.tearDown(ErrClientConnClosing)
|
|
|
}
|
|
|
+ return nil
|
|
|
}
|
|
|
|
|
|
-// State returns the connectivity state of the Conn
|
|
|
-func (cc *Conn) State() ConnectivityState {
|
|
|
- cc.mu.Lock()
|
|
|
- defer cc.mu.Unlock()
|
|
|
- return cc.state
|
|
|
+// addrConn is a network connection to a given address.
|
|
|
+type addrConn struct {
|
|
|
+ cc *ClientConn
|
|
|
+ addr Address
|
|
|
+ dopts dialOptions
|
|
|
+ shutdownChan chan struct{}
|
|
|
+ events trace.EventLog
|
|
|
+
|
|
|
+ mu sync.Mutex
|
|
|
+ state ConnectivityState
|
|
|
+ stateCV *sync.Cond
|
|
|
+ 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{}
|
|
|
+ transport transport.ClientTransport
|
|
|
}
|
|
|
|
|
|
-// WaitForStateChange blocks until the state changes to something other than the sourceState.
|
|
|
-func (cc *Conn) WaitForStateChange(ctx context.Context, sourceState ConnectivityState) (ConnectivityState, error) {
|
|
|
- cc.mu.Lock()
|
|
|
- defer cc.mu.Unlock()
|
|
|
- if sourceState != cc.state {
|
|
|
- return cc.state, nil
|
|
|
+// printf records an event in ac's event log, unless ac has been closed.
|
|
|
+// REQUIRES ac.mu is held.
|
|
|
+func (ac *addrConn) printf(format string, a ...interface{}) {
|
|
|
+ if ac.events != nil {
|
|
|
+ ac.events.Printf(format, a...)
|
|
|
+ }
|
|
|
+}
|
|
|
+
|
|
|
+// errorf records an error in ac's event log, unless ac has been closed.
|
|
|
+// REQUIRES ac.mu is held.
|
|
|
+func (ac *addrConn) errorf(format string, a ...interface{}) {
|
|
|
+ if ac.events != nil {
|
|
|
+ ac.events.Errorf(format, a...)
|
|
|
+ }
|
|
|
+}
|
|
|
+
|
|
|
+// getState returns the connectivity state of the Conn
|
|
|
+func (ac *addrConn) getState() ConnectivityState {
|
|
|
+ ac.mu.Lock()
|
|
|
+ defer ac.mu.Unlock()
|
|
|
+ return ac.state
|
|
|
+}
|
|
|
+
|
|
|
+// waitForStateChange blocks until the state changes to something other than the sourceState.
|
|
|
+func (ac *addrConn) waitForStateChange(ctx context.Context, sourceState ConnectivityState) (ConnectivityState, error) {
|
|
|
+ ac.mu.Lock()
|
|
|
+ defer ac.mu.Unlock()
|
|
|
+ if sourceState != ac.state {
|
|
|
+ return ac.state, nil
|
|
|
}
|
|
|
done := make(chan struct{})
|
|
|
var err error
|
|
|
go func() {
|
|
|
select {
|
|
|
case <-ctx.Done():
|
|
|
- cc.mu.Lock()
|
|
|
+ ac.mu.Lock()
|
|
|
err = ctx.Err()
|
|
|
- cc.stateCV.Broadcast()
|
|
|
- cc.mu.Unlock()
|
|
|
+ ac.stateCV.Broadcast()
|
|
|
+ ac.mu.Unlock()
|
|
|
case <-done:
|
|
|
}
|
|
|
}()
|
|
|
defer close(done)
|
|
|
- for sourceState == cc.state {
|
|
|
- cc.stateCV.Wait()
|
|
|
+ for sourceState == ac.state {
|
|
|
+ ac.stateCV.Wait()
|
|
|
if err != nil {
|
|
|
- return cc.state, err
|
|
|
+ return ac.state, err
|
|
|
}
|
|
|
}
|
|
|
- return cc.state, nil
|
|
|
+ return ac.state, nil
|
|
|
}
|
|
|
|
|
|
-// NotifyReset tries to signal the underlying transport needs to be reset due to
|
|
|
-// for example a name resolution change in flight.
|
|
|
-func (cc *Conn) NotifyReset() {
|
|
|
- select {
|
|
|
- case cc.resetChan <- 0:
|
|
|
- default:
|
|
|
- }
|
|
|
-}
|
|
|
-
|
|
|
-func (cc *Conn) resetTransport(closeTransport bool) error {
|
|
|
+func (ac *addrConn) resetTransport(closeTransport bool) error {
|
|
|
var retries int
|
|
|
- start := time.Now()
|
|
|
for {
|
|
|
- cc.mu.Lock()
|
|
|
- cc.printf("connecting")
|
|
|
- if cc.state == Shutdown {
|
|
|
- // cc.Close() has been invoked.
|
|
|
- cc.mu.Unlock()
|
|
|
- return ErrClientConnClosing
|
|
|
+ ac.mu.Lock()
|
|
|
+ ac.printf("connecting")
|
|
|
+ if ac.state == Shutdown {
|
|
|
+ // ac.tearDown(...) has been invoked.
|
|
|
+ ac.mu.Unlock()
|
|
|
+ return errConnClosing
|
|
|
}
|
|
|
- cc.state = Connecting
|
|
|
- cc.stateCV.Broadcast()
|
|
|
- cc.mu.Unlock()
|
|
|
- if closeTransport {
|
|
|
- cc.transport.Close()
|
|
|
- }
|
|
|
- // Adjust timeout for the current try.
|
|
|
- copts := cc.dopts.copts
|
|
|
- if copts.Timeout < 0 {
|
|
|
- cc.Close()
|
|
|
- return ErrClientConnTimeout
|
|
|
- }
|
|
|
- if copts.Timeout > 0 {
|
|
|
- copts.Timeout -= time.Since(start)
|
|
|
- if copts.Timeout <= 0 {
|
|
|
- cc.Close()
|
|
|
- return ErrClientConnTimeout
|
|
|
- }
|
|
|
+ if ac.down != nil {
|
|
|
+ ac.down(downErrorf(false, true, "%v", errNetworkIO))
|
|
|
+ ac.down = nil
|
|
|
}
|
|
|
- sleepTime := cc.dopts.bs.backoff(retries)
|
|
|
- timeout := sleepTime
|
|
|
- if timeout < minConnectTimeout {
|
|
|
- timeout = minConnectTimeout
|
|
|
+ ac.state = Connecting
|
|
|
+ ac.stateCV.Broadcast()
|
|
|
+ t := ac.transport
|
|
|
+ ac.mu.Unlock()
|
|
|
+ if closeTransport && t != nil {
|
|
|
+ t.Close()
|
|
|
}
|
|
|
- if copts.Timeout == 0 || copts.Timeout > timeout {
|
|
|
- copts.Timeout = timeout
|
|
|
+ sleepTime := ac.dopts.bs.backoff(retries)
|
|
|
+ ac.dopts.copts.Timeout = sleepTime
|
|
|
+ if sleepTime < minConnectTimeout {
|
|
|
+ ac.dopts.copts.Timeout = minConnectTimeout
|
|
|
}
|
|
|
connectTime := time.Now()
|
|
|
- addr, err := cc.dopts.picker.PickAddr()
|
|
|
- var newTransport transport.ClientTransport
|
|
|
- if err == nil {
|
|
|
- newTransport, err = transport.NewClientTransport(addr, &copts)
|
|
|
- }
|
|
|
+ newTransport, err := transport.NewClientTransport(ac.addr.Addr, &ac.dopts.copts)
|
|
|
if err != nil {
|
|
|
- cc.mu.Lock()
|
|
|
- if cc.state == Shutdown {
|
|
|
- // cc.Close() has been invoked.
|
|
|
- cc.mu.Unlock()
|
|
|
- return ErrClientConnClosing
|
|
|
+ ac.mu.Lock()
|
|
|
+ if ac.state == Shutdown {
|
|
|
+ // ac.tearDown(...) has been invoked.
|
|
|
+ ac.mu.Unlock()
|
|
|
+ return errConnClosing
|
|
|
}
|
|
|
- cc.errorf("transient failure: %v", err)
|
|
|
- cc.state = TransientFailure
|
|
|
- cc.stateCV.Broadcast()
|
|
|
- if cc.ready != nil {
|
|
|
- close(cc.ready)
|
|
|
- cc.ready = nil
|
|
|
+ ac.errorf("transient failure: %v", err)
|
|
|
+ ac.state = TransientFailure
|
|
|
+ ac.stateCV.Broadcast()
|
|
|
+ if ac.ready != nil {
|
|
|
+ close(ac.ready)
|
|
|
+ ac.ready = nil
|
|
|
}
|
|
|
- cc.mu.Unlock()
|
|
|
+ ac.mu.Unlock()
|
|
|
sleepTime -= time.Since(connectTime)
|
|
|
if sleepTime < 0 {
|
|
|
sleepTime = 0
|
|
|
}
|
|
|
- // Fail early before falling into sleep.
|
|
|
- if cc.dopts.copts.Timeout > 0 && cc.dopts.copts.Timeout < sleepTime+time.Since(start) {
|
|
|
- cc.mu.Lock()
|
|
|
- cc.errorf("connection timeout")
|
|
|
- cc.mu.Unlock()
|
|
|
- cc.Close()
|
|
|
- return ErrClientConnTimeout
|
|
|
- }
|
|
|
closeTransport = false
|
|
|
select {
|
|
|
case <-time.After(sleepTime):
|
|
|
- case <-cc.shutdownChan:
|
|
|
+ case <-ac.shutdownChan:
|
|
|
}
|
|
|
retries++
|
|
|
- grpclog.Printf("grpc: Conn.resetTransport failed to create client transport: %v; Reconnecting to %q", err, cc.target)
|
|
|
+ grpclog.Printf("grpc: addrConn.resetTransport failed to create client transport: %v; Reconnecting to %q", err, ac.addr)
|
|
|
continue
|
|
|
}
|
|
|
- cc.mu.Lock()
|
|
|
- cc.printf("ready")
|
|
|
- if cc.state == Shutdown {
|
|
|
- // cc.Close() has been invoked.
|
|
|
- cc.mu.Unlock()
|
|
|
+ ac.mu.Lock()
|
|
|
+ ac.printf("ready")
|
|
|
+ if ac.state == Shutdown {
|
|
|
+ // ac.tearDown(...) has been invoked.
|
|
|
+ ac.mu.Unlock()
|
|
|
newTransport.Close()
|
|
|
- return ErrClientConnClosing
|
|
|
+ return errConnClosing
|
|
|
}
|
|
|
- cc.state = Ready
|
|
|
- cc.stateCV.Broadcast()
|
|
|
- cc.transport = newTransport
|
|
|
- if cc.ready != nil {
|
|
|
- close(cc.ready)
|
|
|
- cc.ready = nil
|
|
|
+ ac.state = Ready
|
|
|
+ ac.stateCV.Broadcast()
|
|
|
+ ac.transport = newTransport
|
|
|
+ if ac.ready != nil {
|
|
|
+ close(ac.ready)
|
|
|
+ ac.ready = nil
|
|
|
}
|
|
|
- cc.mu.Unlock()
|
|
|
+ ac.down = ac.cc.balancer.Up(ac.addr)
|
|
|
+ ac.mu.Unlock()
|
|
|
return nil
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-func (cc *Conn) reconnect() bool {
|
|
|
- cc.mu.Lock()
|
|
|
- if cc.state == Shutdown {
|
|
|
- // cc.Close() has been invoked.
|
|
|
- cc.mu.Unlock()
|
|
|
- return false
|
|
|
- }
|
|
|
- cc.state = TransientFailure
|
|
|
- cc.stateCV.Broadcast()
|
|
|
- cc.mu.Unlock()
|
|
|
- if err := cc.resetTransport(true); err != nil {
|
|
|
- // The ClientConn is closing.
|
|
|
- cc.mu.Lock()
|
|
|
- cc.printf("transport exiting: %v", err)
|
|
|
- cc.mu.Unlock()
|
|
|
- grpclog.Printf("grpc: Conn.transportMonitor exits due to: %v", err)
|
|
|
- return false
|
|
|
- }
|
|
|
- return true
|
|
|
-}
|
|
|
-
|
|
|
// Run in a goroutine to track the error in transport and create the
|
|
|
// new transport if an error happens. It returns when the channel is closing.
|
|
|
-func (cc *Conn) transportMonitor() {
|
|
|
+func (ac *addrConn) transportMonitor() {
|
|
|
for {
|
|
|
+ ac.mu.Lock()
|
|
|
+ t := ac.transport
|
|
|
+ ac.mu.Unlock()
|
|
|
select {
|
|
|
// shutdownChan is needed to detect the teardown when
|
|
|
- // the ClientConn is idle (i.e., no RPC in flight).
|
|
|
- case <-cc.shutdownChan:
|
|
|
+ // the addrConn is idle (i.e., no RPC in flight).
|
|
|
+ case <-ac.shutdownChan:
|
|
|
return
|
|
|
- case <-cc.resetChan:
|
|
|
- if !cc.reconnect() {
|
|
|
+ case <-t.Error():
|
|
|
+ ac.mu.Lock()
|
|
|
+ if ac.state == Shutdown {
|
|
|
+ // ac.tearDown(...) has been invoked.
|
|
|
+ ac.mu.Unlock()
|
|
|
return
|
|
|
}
|
|
|
- case <-cc.transport.Error():
|
|
|
- if !cc.reconnect() {
|
|
|
+ ac.state = TransientFailure
|
|
|
+ ac.stateCV.Broadcast()
|
|
|
+ ac.mu.Unlock()
|
|
|
+ if err := ac.resetTransport(true); err != nil {
|
|
|
+ ac.mu.Lock()
|
|
|
+ ac.printf("transport exiting: %v", err)
|
|
|
+ ac.mu.Unlock()
|
|
|
+ grpclog.Printf("grpc: addrConn.transportMonitor exits due to: %v", err)
|
|
|
return
|
|
|
}
|
|
|
- // Tries to drain reset signal if there is any since it is out-dated.
|
|
|
- select {
|
|
|
- case <-cc.resetChan:
|
|
|
- default:
|
|
|
- }
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-// Wait blocks until i) the new transport is up or ii) ctx is done or iii) cc is closed.
|
|
|
-func (cc *Conn) Wait(ctx context.Context) (transport.ClientTransport, error) {
|
|
|
+// wait blocks until i) the new transport is up or ii) ctx is done or iii) ac is closed.
|
|
|
+func (ac *addrConn) wait(ctx context.Context) (transport.ClientTransport, error) {
|
|
|
for {
|
|
|
- cc.mu.Lock()
|
|
|
+ ac.mu.Lock()
|
|
|
switch {
|
|
|
- case cc.state == Shutdown:
|
|
|
- cc.mu.Unlock()
|
|
|
- return nil, ErrClientConnClosing
|
|
|
- case cc.state == Ready:
|
|
|
- ct := cc.transport
|
|
|
- cc.mu.Unlock()
|
|
|
+ case ac.state == Shutdown:
|
|
|
+ ac.mu.Unlock()
|
|
|
+ return nil, errConnClosing
|
|
|
+ case ac.state == Ready:
|
|
|
+ ct := ac.transport
|
|
|
+ ac.mu.Unlock()
|
|
|
return ct, nil
|
|
|
default:
|
|
|
- ready := cc.ready
|
|
|
+ ready := ac.ready
|
|
|
if ready == nil {
|
|
|
ready = make(chan struct{})
|
|
|
- cc.ready = ready
|
|
|
+ ac.ready = ready
|
|
|
}
|
|
|
- cc.mu.Unlock()
|
|
|
+ ac.mu.Unlock()
|
|
|
select {
|
|
|
case <-ctx.Done():
|
|
|
return nil, transport.ContextErr(ctx.Err())
|
|
|
@@ -595,32 +678,46 @@ func (cc *Conn) Wait(ctx context.Context) (transport.ClientTransport, error) {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-// Close starts to tear down the Conn. Returns ErrClientConnClosing if
|
|
|
-// it has been closed (mostly due to dial time-out).
|
|
|
+// tearDown starts to tear down the addrConn.
|
|
|
// TODO(zhaoq): Make this synchronous to avoid unbounded memory consumption in
|
|
|
-// some edge cases (e.g., the caller opens and closes many ClientConn's in a
|
|
|
+// some edge cases (e.g., the caller opens and closes many addrConn's in a
|
|
|
// tight loop.
|
|
|
-func (cc *Conn) Close() error {
|
|
|
- cc.mu.Lock()
|
|
|
- defer cc.mu.Unlock()
|
|
|
- if cc.state == Shutdown {
|
|
|
- return ErrClientConnClosing
|
|
|
- }
|
|
|
- cc.state = Shutdown
|
|
|
- cc.stateCV.Broadcast()
|
|
|
- if cc.events != nil {
|
|
|
- cc.events.Finish()
|
|
|
- cc.events = nil
|
|
|
- }
|
|
|
- if cc.ready != nil {
|
|
|
- close(cc.ready)
|
|
|
- cc.ready = nil
|
|
|
- }
|
|
|
- if cc.transport != nil {
|
|
|
- cc.transport.Close()
|
|
|
+func (ac *addrConn) tearDown(err error) {
|
|
|
+ ac.mu.Lock()
|
|
|
+ defer func() {
|
|
|
+ ac.mu.Unlock()
|
|
|
+ ac.cc.mu.Lock()
|
|
|
+ if ac.cc.conns != nil {
|
|
|
+ delete(ac.cc.conns, ac.addr)
|
|
|
+ }
|
|
|
+ ac.cc.mu.Unlock()
|
|
|
+ }()
|
|
|
+ if ac.state == Shutdown {
|
|
|
+ return
|
|
|
+ }
|
|
|
+ ac.state = Shutdown
|
|
|
+ if ac.down != nil {
|
|
|
+ ac.down(downErrorf(false, false, "%v", err))
|
|
|
+ ac.down = nil
|
|
|
+ }
|
|
|
+ ac.stateCV.Broadcast()
|
|
|
+ if ac.events != nil {
|
|
|
+ ac.events.Finish()
|
|
|
+ ac.events = nil
|
|
|
+ }
|
|
|
+ if ac.ready != nil {
|
|
|
+ close(ac.ready)
|
|
|
+ ac.ready = nil
|
|
|
+ }
|
|
|
+ if ac.transport != nil {
|
|
|
+ if err == errConnDrain {
|
|
|
+ ac.transport.GracefulClose()
|
|
|
+ } else {
|
|
|
+ ac.transport.Close()
|
|
|
+ }
|
|
|
}
|
|
|
- if cc.shutdownChan != nil {
|
|
|
- close(cc.shutdownChan)
|
|
|
+ if ac.shutdownChan != nil {
|
|
|
+ close(ac.shutdownChan)
|
|
|
}
|
|
|
- return nil
|
|
|
+ return
|
|
|
}
|