From 9c2d8546bf9fdbae6679bf11f1e420730277659a Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Fri, 6 May 2016 15:47:09 -0700 Subject: [PATCH 01/40] load balancer --- balancer.go | 137 +++++ call.go | 6 +- clientconn.go | 593 ++++++++++++++-------- examples/pipe/main.go | 67 +++ picker.go | 243 --------- picker_test.go | 188 ------- stream.go | 10 +- test/end2end_test.go | 101 ++-- test/out.txt | 1015 +++++++++++++++++++++++++++++++++++++ transport/http2_client.go | 20 + transport/transport.go | 2 + 11 files changed, 1685 insertions(+), 697 deletions(-) create mode 100644 balancer.go create mode 100644 examples/pipe/main.go delete mode 100644 picker.go delete mode 100644 picker_test.go create mode 100644 test/out.txt diff --git a/balancer.go b/balancer.go new file mode 100644 index 00000000..998c7373 --- /dev/null +++ b/balancer.go @@ -0,0 +1,137 @@ +package grpc + +import ( + "sync" + + "golang.org/x/net/context" + "google.golang.org/grpc/transport" +) + +type Address struct { + // Addr is the peer address on which a connection will be established. + Addr string + // Metadata is the information associated with Addr, which may be used + // to make load balancing decision. This is from the metadata attached + // in the address updates from name resolver. + Metadata interface{} +} + +// Balancer chooses network addresses for RPCs. +type Balancer interface { + // Up informs the balancer that gRPC has a connection to the server at + // addr. It returns down which will be called once the connection gets + // lost. Once down is called, addr may no longer be returned by Get. + Up(addr Address) (down func(error)) + // Get gets the address of a server for the rpc corresponding to ctx. + // It may block if there is no server available. It respects the + // timeout or cancellation of ctx when blocking. It returns put which + // is called once the rpc has completed or failed. put can collect and + // report rpc stats to remote load balancer. + Get(ctx context.Context) (addr Address, put func(), err error) + // Close shuts down the balancer. + Close() error +} + +func RoundRobin() Balancer { + return &roundRobin{} +} + +type roundRobin struct { + mu sync.Mutex + addrs []Address + next int + waitCh chan struct{} + pending int +} + +func (rr *roundRobin) Up(addr Address) func() { + rr.mu.Lock() + defer rr.mu.Unlock() + for _, a := range rr.addrs { + if a == addr { + return nil + } + } + rr.addrs = append(rr.addrs, addr) + if len(rr.addrs) == 1 { + if rr.waitCh != nil { + close(rr.waitCh) + rr.waitCh = nil + } + } + return func() { + rr.down(addr) + } +} + +func (rr *roundRobin) down(addr Address) { + rr.mu.Lock() + defer rr.mu.Unlock() + for i, a := range rr.addrs { + if a == addr { + copy(rr.addrs[i:], rr.addrs[i+1:]) + rr.addrs = rr.addrs[:len(rr.addrs)-1] + return + } + } +} + +func (rr *roundRobin) Get(ctx context.Context) (addr Address, put func(), err error) { + var ch chan struct{} + rr.mu.Lock() + if rr.next >= len(rr.addrs) { + rr.next = 0 + } + if len(rr.addrs) > 0 { + addr = rr.addrs[rr.next] + rr.next++ + rr.pending++ + rr.mu.Unlock() + put = func() { + rr.put(ctx, addr) + } + return + } + if rr.waitCh == nil { + ch = make(chan struct{}) + rr.waitCh = ch + } else { + ch = rr.waitCh + } + rr.mu.Unlock() + for { + select { + case <-ctx.Done(): + err = transport.ContextErr(ctx.Err()) + return + case <-ch: + rr.mu.Lock() + if len(rr.addrs) == 0 { + // The newly added addr got removed by Down() again. + rr.mu.Unlock() + continue + } + if rr.next >= len(rr.addrs) { + rr.next = 0 + } + addr = rr.addrs[rr.next] + rr.next++ + rr.pending++ + rr.mu.Unlock() + put = func() { + rr.put(ctx, addr) + } + return + } + } +} + +func (rr *roundRobin) put(ctx context.Context, addr Address) { + rr.mu.Lock() + defer rr.mu.Unlock() + rr.pending-- +} + +func (rr *roundRobin) Close() error { + return nil +} diff --git a/call.go b/call.go index 9d0fc8ee..67604f77 100644 --- a/call.go +++ b/call.go @@ -134,6 +134,7 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli } var ( lastErr error // record the error that happened + put func() ) for { var ( @@ -152,7 +153,7 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli if cc.dopts.cp != nil { callHdr.SendCompress = cc.dopts.cp.Type() } - t, err = cc.dopts.picker.Pick(ctx) + t, put, err = cc.getTransport(ctx) if err != nil { if lastErr != nil { // This was a retry; return the error from the last attempt. @@ -165,6 +166,7 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli } stream, err = sendRequest(ctx, cc.dopts.codec, cc.dopts.cp, callHdr, t, args, topts) if err != nil { + put() if _, ok := err.(transport.ConnectionError); ok { lastErr = err continue @@ -177,12 +179,14 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli // Receive the response lastErr = recvResponse(cc.dopts, t, &c, stream, reply) if _, ok := lastErr.(transport.ConnectionError); ok { + put() continue } if c.traceInfo.tr != nil { c.traceInfo.tr.LazyLog(&payload{sent: false, msg: reply}, true) } t.CloseStream(stream, lastErr) + put() if lastErr != nil { return toRPCErr(lastErr) } diff --git a/clientconn.go b/clientconn.go index 6de86e9e..312e07a9 100644 --- a/clientconn.go +++ b/clientconn.go @@ -43,14 +43,14 @@ 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/naming" "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 // being set for ClientConn. Users should either set one or explicitly // call WithInsecure DialOption to disable security. @@ -60,11 +60,16 @@ var ( // 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. + // the ClientConn 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") + // 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: ") + errConnClosing = errors.New("grpc: the addrConn is closing") // minimum time to give a connection to complete minConnectTimeout = 20 * time.Second ) @@ -76,7 +81,8 @@ type dialOptions struct { cp Compressor dc Decompressor bs backoffStrategy - picker Picker + resolver naming.Resolver + balancer Balancer block bool insecure bool copts transport.ConnectOptions @@ -108,10 +114,9 @@ func WithDecompressor(dc Decompressor) DialOption { } } -// WithPicker returns a DialOption which sets a picker for connection selection. -func WithPicker(p Picker) DialOption { +func WithResolver(r naming.Resolver) DialOption { return func(o *dialOptions) { - o.picker = p + o.resolver = r } } @@ -201,6 +206,7 @@ func WithUserAgent(s string) DialOption { func Dial(target string, opts ...DialOption) (*ClientConn, error) { cc := &ClientConn{ target: target, + infos: make(map[Address]*addrInfo), } for _, opt := range opts { opt(&cc.dopts) @@ -214,14 +220,44 @@ 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() + } + + // Start the first connection + if cc.dopts.resolver == nil { + addr := Address{ + Addr: cc.target, } + ac, err := cc.newAddrConn(addr) + if err != nil { + return nil, err + } + cc.mu.Lock() + cc.infos[addr] = &addrInfo{ + ac: ac, + } + cc.mu.Unlock() + } else { + w, err := cc.dopts.resolver.Resolve(cc.target) + if err != nil { + return nil, err + } + cc.watcher = w + // Get the initial name resolution which starts dialing. + if err := cc.watchAddrUpdates(); err != nil { + return nil, err + } + go func() { + for { + if err := cc.watchAddrUpdates(); err != nil { + return + } + } + }() } - if err := cc.dopts.picker.Init(cc); err != nil { - return nil, err - } + colonPos := strings.LastIndex(target, ":") if colonPos == -1 { colonPos = len(target) @@ -263,61 +299,88 @@ func (s ConnectivityState) String() string { } } +type addrInfo struct { + ac *addrConn +} + // ClientConn represents a client connection to an RPC service. type ClientConn struct { target string + watcher naming.Watcher + balancer Balancer authority string dopts dialOptions + + mu sync.RWMutex + infos map[Address]*addrInfo } -// State returns the connectivity state of cc. -// This is EXPERIMENTAL API. -func (cc *ClientConn) State() (ConnectivityState, error) { - return cc.dopts.picker.State() -} - -// 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 -} - -// NewConn creates a Conn. -func NewConn(cc *ClientConn) (*Conn, error) { - if cc.target == "" { - return nil, ErrUnspecTarget +func (cc *ClientConn) watchAddrUpdates() error { + updates, err := cc.watcher.Next() + if err != nil { + return err } - c := &Conn{ - target: cc.target, - dopts: cc.dopts, - resetChan: make(chan int, 1), + for _, update := range updates { + switch update.Op { + case naming.Add: + cc.mu.Lock() + addr := Address{ + Addr: update.Addr, + Metadata: update.Metadata, + } + if _, ok := cc.infos[addr]; ok { + cc.mu.Unlock() + grpclog.Println("grpc: The name resolver wanted to add an existing address: ", addr) + continue + } + cc.mu.Unlock() + ac, err := cc.newAddrConn(addr) + if err != nil { + cc.mu.Unlock() + return err + } + cc.mu.Lock() + cc.infos[addr] = &addrInfo{ + ac: ac, + } + cc.mu.Unlock() + case naming.Delete: + cc.mu.Lock() + addr := Address{ + Addr: update.Addr, + Metadata: update.Metadata, + } + i, ok := cc.infos[addr] + if !ok { + cc.mu.Unlock() + grpclog.Println("grpc: The name resolver wanted to delete a non-exist address: ", addr) + continue + } + delete(cc.infos, addr) + cc.mu.Unlock() + i.ac.startDrain() + default: + grpclog.Println("Unknown update.Op ", update.Op) + } + } + return nil +} + +func (cc *ClientConn) newAddrConn(addr Address) (*addrConn, error) { + /* + if cc.target == "" { + return nil, ErrUnspecTarget + } + */ + c := &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) + c.events = trace.NewEventLog("grpc.ClientConn", c.addr.Addr) } if !c.dopts.insecure { var ok bool @@ -339,7 +402,7 @@ func NewConn(cc *ClientConn) (*Conn, error) { c.stateCV = sync.NewCond(&c.mu) if c.dopts.block { if err := c.resetTransport(false); err != nil { - c.Close() + c.tearDown(err) return nil, err } // Start to monitor the error status of transport. @@ -348,108 +411,200 @@ func NewConn(cc *ClientConn) (*Conn, error) { // 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() + grpclog.Printf("Failed to dial %s: %v; please retry.", c.addr.Addr, err) + c.tearDown(err) return } + grpclog.Println("DEBUG ugh here resetTransport") c.transportMonitor() }() } return c, 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) (transport.ClientTransport, func(), error) { + addr, put, err := cc.balancer.Get(ctx) + if err != nil { + return nil, nil, err } + cc.mu.RLock() + if cc.infos == nil { + cc.mu.RUnlock() + return nil, nil, ErrClientConnClosing + } + info, ok := cc.infos[addr] + cc.mu.RUnlock() + if !ok { + put() + return nil, nil, transport.StreamErrorf(codes.Internal, "grpc: failed to find the transport to send the rpc") + } + t, err := info.ac.wait(ctx) + if err != 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...) - } -} - -// State returns the connectivity state of the Conn -func (cc *Conn) State() ConnectivityState { - cc.mu.Lock() - defer cc.mu.Unlock() - return cc.state +/* +// State returns the connectivity state of cc. +// This is EXPERIMENTAL API. +func (cc *ClientConn) State() (ConnectivityState, error) { + return cc.dopts.picker.State() } // WaitForStateChange blocks until the state changes to something other than the sourceState. -func (cc *Conn) WaitForStateChange(ctx context.Context, sourceState ConnectivityState) (ConnectivityState, error) { +// 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 { cc.mu.Lock() - defer cc.mu.Unlock() - if sourceState != cc.state { - return cc.state, nil + if cc.infos == nil { + cc.mu.Unlock() + return ErrClientConnClosing + } + infos := cc.infos + cc.infos = nil + cc.mu.Unlock() + cc.balancer.Close() + if cc.watcher != nil { + cc.watcher.Close() + } + for _, i := range infos { + i.ac.tearDown(ErrClientClosing) + } + return nil +} + +// addrConn is a network connection to a given address. +type addrConn struct { + cc *ClientConn + addr Address + dopts dialOptions + //resetChan chan int + 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. + drain bool + // ready is closed and becomes nil when a new transport is up or failed + // due to timeout. + ready chan struct{} + transport transport.ClientTransport +} + +func (ac *addrConn) startDrain() { + ac.mu.Lock() + t := ac.transport + ac.drain = true + if ac.down != nil { + ac.down(ErrConnDrain) + ac.down = nil + } + ac.mu.Unlock() + t.GracefulClose() +} + +// 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() + if ac.drain { + ac.mu.Unlock() + return nil + } + if ac.down != nil { + ac.down(ErrNetworkIO) + ac.down = nil + } + ac.state = Connecting + ac.stateCV.Broadcast() + t := ac.transport + ac.mu.Unlock() + if closeTransport && t != nil { + t.Close() } // Adjust timeout for the current try. - copts := cc.dopts.copts + copts := ac.dopts.copts if copts.Timeout < 0 { - cc.Close() + ac.tearDown(ErrClientConnTimeout) return ErrClientConnTimeout } if copts.Timeout > 0 { copts.Timeout -= time.Since(start) if copts.Timeout <= 0 { - cc.Close() + ac.tearDown(ErrClientConnTimeout) return ErrClientConnTimeout } } - sleepTime := cc.dopts.bs.backoff(retries) + sleepTime := ac.dopts.bs.backoff(retries) timeout := sleepTime if timeout < minConnectTimeout { timeout = minConnectTimeout @@ -458,130 +613,134 @@ func (cc *Conn) resetTransport(closeTransport bool) error { copts.Timeout = timeout } connectTime := time.Now() - addr, err := cc.dopts.picker.PickAddr() - var newTransport transport.ClientTransport - if err == nil { - newTransport, err = transport.NewClientTransport(addr, &copts) - } + grpclog.Println("DEBUG reach inside resetTransport 1") + newTransport, err := transport.NewClientTransport(ac.addr.Addr, &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() + if ac.dopts.copts.Timeout > 0 && ac.dopts.copts.Timeout < sleepTime+time.Since(start) { + ac.mu.Lock() + ac.errorf("connection timeout") + ac.mu.Unlock() + ac.tearDown(ErrClientTimeout) return ErrClientConnTimeout } closeTransport = false time.Sleep(sleepTime) 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() + grpclog.Println("DEBUG reach inside resetTransport 2") + 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 + grpclog.Println("DEBUG reach inside resetTransport 3: ", ac.addr) + ac.state = Ready + ac.stateCV.Broadcast() + ac.transport = newTransport + ac.down = ac.cc.balancer.Up(ac.addr) + if ac.ready != nil { + close(ac.ready) + ac.ready = nil } - cc.mu.Unlock() + 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 <-ac.resetChan: + if !ac.reconnect() { + return + } + */ + 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: - } + /* + if !ac.reconnect() { + return + } + */ + /* + // Tries to drain reset signal if there is any since it is out-dated. + select { + case <-ac.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()) @@ -592,32 +751,36 @@ func (cc *Conn) Wait(ctx context.Context) (transport.ClientTransport, error) { } } -// Close starts to tear down the Conn. Returns ErrClientConnClosing if +// tearDown starts to tear down the Conn. Returns errConnClosing if // it has been closed (mostly due to dial time-out). // 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 +func (ac *addrConn) tearDown(err error) { + ac.mu.Lock() + defer ac.mu.Unlock() + if ac.down != nil { + ac.down(err) + ac.down = nil } - cc.state = Shutdown - cc.stateCV.Broadcast() - if cc.events != nil { - cc.events.Finish() - cc.events = nil + if ac.state == Shutdown { + return } - if cc.ready != nil { - close(cc.ready) - cc.ready = nil + ac.state = Shutdown + ac.stateCV.Broadcast() + if ac.events != nil { + ac.events.Finish() + ac.events = nil } - if cc.transport != nil { - cc.transport.Close() + if ac.ready != nil { + close(ac.ready) + ac.ready = nil } - if cc.shutdownChan != nil { - close(cc.shutdownChan) + if ac.transport != nil { + ac.transport.Close() } - return nil + if ac.shutdownChan != nil { + close(ac.shutdownChan) + } + return } diff --git a/examples/pipe/main.go b/examples/pipe/main.go new file mode 100644 index 00000000..84a10726 --- /dev/null +++ b/examples/pipe/main.go @@ -0,0 +1,67 @@ +package main + +import ( + "fmt" + "log" + "net" + "time" + + "google.golang.org/grpc" +) + +type memAddr string + +func (a memAddr) Network() string { return "mem" } +func (a memAddr) String() string { return string(a) } + +type memListener struct { + c chan net.Conn +} + +func (ln *memListener) Accept() (net.Conn, error) { + conn, ok := <-ln.c + if !ok { + return nil, fmt.Errorf("closed") + } + return conn, nil +} + +func (ln *memListener) Addr() net.Addr { + return memAddr(fmt.Sprintf("%p", ln)) +} + +func (ln *memListener) Close() error { + close(ln.c) + return nil +} + +func main() { + grpc.EnableTracing = true + + ln := &memListener{ + c: make(chan net.Conn, 1), + } + + go func() { + s := grpc.NewServer() + log.Fatal(s.Serve(ln)) + }() + + log.Printf("Dialing to server over a synchronous pipe...") + serverConn, err := grpc.Dial("inmemory", + grpc.WithInsecure(), + grpc.WithBlock(), + grpc.WithDialer(func(_ string, _ time.Duration) (net.Conn, error) { + c1, c2 := net.Pipe() + log.Printf("Pipe created: %v %v", c1, c2) + ln.c <- c2 + log.Printf("Pipe accepted: %v %v", c1, c2) + return c1, nil + })) + if err != nil { + log.Fatal(err) + } + + // BUG: never reached + log.Printf("SUCCESS! Connected to server: %v", serverConn) +} diff --git a/picker.go b/picker.go deleted file mode 100644 index 50f315b4..00000000 --- a/picker.go +++ /dev/null @@ -1,243 +0,0 @@ -/* - * - * Copyright 2014, Google Inc. - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are - * met: - * - * * Redistributions of source code must retain the above copyright - * notice, this list of conditions and the following disclaimer. - * * Redistributions in binary form must reproduce the above - * copyright notice, this list of conditions and the following disclaimer - * in the documentation and/or other materials provided with the - * distribution. - * * Neither the name of Google Inc. nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR - * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT - * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, - * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT - * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, - * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY - * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT - * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - */ - -package grpc - -import ( - "container/list" - "fmt" - "sync" - - "golang.org/x/net/context" - "google.golang.org/grpc/grpclog" - "google.golang.org/grpc/naming" - "google.golang.org/grpc/transport" -) - -// Picker picks a Conn for RPC requests. -// This is EXPERIMENTAL and please do not implement your own Picker for now. -type Picker interface { - // Init does initial processing for the Picker, e.g., initiate some connections. - Init(cc *ClientConn) error - // Pick blocks until either a transport.ClientTransport is ready for the upcoming RPC - // or some error happens. - Pick(ctx context.Context) (transport.ClientTransport, error) - // PickAddr picks a peer address for connecting. This will be called repeated for - // connecting/reconnecting. - PickAddr() (string, error) - // State returns the connectivity state of the underlying connections. - State() (ConnectivityState, error) - // WaitForStateChange blocks until the state changes to something other than - // the sourceState. It returns the new state or error. - WaitForStateChange(ctx context.Context, sourceState ConnectivityState) (ConnectivityState, error) - // Close closes all the Conn's owned by this Picker. - Close() error -} - -// unicastPicker is the default Picker which is used when there is no custom Picker -// specified by users. It always picks the same Conn. -type unicastPicker struct { - target string - conn *Conn -} - -func (p *unicastPicker) Init(cc *ClientConn) error { - c, err := NewConn(cc) - if err != nil { - return err - } - p.conn = c - return nil -} - -func (p *unicastPicker) Pick(ctx context.Context) (transport.ClientTransport, error) { - return p.conn.Wait(ctx) -} - -func (p *unicastPicker) PickAddr() (string, error) { - return p.target, nil -} - -func (p *unicastPicker) State() (ConnectivityState, error) { - return p.conn.State(), nil -} - -func (p *unicastPicker) WaitForStateChange(ctx context.Context, sourceState ConnectivityState) (ConnectivityState, error) { - return p.conn.WaitForStateChange(ctx, sourceState) -} - -func (p *unicastPicker) Close() error { - if p.conn != nil { - return p.conn.Close() - } - return nil -} - -// unicastNamingPicker picks an address from a name resolver to set up the connection. -type unicastNamingPicker struct { - cc *ClientConn - resolver naming.Resolver - watcher naming.Watcher - mu sync.Mutex - // The list of the addresses are obtained from watcher. - addrs *list.List - // It tracks the current picked addr by PickAddr(). The next PickAddr may - // push it forward on addrs. - pickedAddr *list.Element - conn *Conn -} - -// NewUnicastNamingPicker creates a Picker to pick addresses from a name resolver -// to connect. -func NewUnicastNamingPicker(r naming.Resolver) Picker { - return &unicastNamingPicker{ - resolver: r, - addrs: list.New(), - } -} - -type addrInfo struct { - addr string - // Set to true if this addrInfo needs to be deleted in the next PickAddrr() call. - deleting bool -} - -// processUpdates calls Watcher.Next() once and processes the obtained updates. -func (p *unicastNamingPicker) processUpdates() error { - updates, err := p.watcher.Next() - if err != nil { - return err - } - for _, update := range updates { - switch update.Op { - case naming.Add: - p.mu.Lock() - p.addrs.PushBack(&addrInfo{ - addr: update.Addr, - }) - p.mu.Unlock() - // Initial connection setup - if p.conn == nil { - conn, err := NewConn(p.cc) - if err != nil { - return err - } - p.conn = conn - } - case naming.Delete: - p.mu.Lock() - for e := p.addrs.Front(); e != nil; e = e.Next() { - if update.Addr == e.Value.(*addrInfo).addr { - if e == p.pickedAddr { - // Do not remove the element now if it is the current picked - // one. We leave the deletion to the next PickAddr() call. - e.Value.(*addrInfo).deleting = true - // Notify Conn to close it. All the live RPCs on this connection - // will be aborted. - p.conn.NotifyReset() - } else { - p.addrs.Remove(e) - } - } - } - p.mu.Unlock() - default: - grpclog.Println("Unknown update.Op ", update.Op) - } - } - return nil -} - -// monitor runs in a standalone goroutine to keep watching name resolution updates until the watcher -// is closed. -func (p *unicastNamingPicker) monitor() { - for { - if err := p.processUpdates(); err != nil { - return - } - } -} - -func (p *unicastNamingPicker) Init(cc *ClientConn) error { - w, err := p.resolver.Resolve(cc.target) - if err != nil { - return err - } - p.watcher = w - p.cc = cc - // Get the initial name resolution. - if err := p.processUpdates(); err != nil { - return err - } - go p.monitor() - return nil -} - -func (p *unicastNamingPicker) Pick(ctx context.Context) (transport.ClientTransport, error) { - return p.conn.Wait(ctx) -} - -func (p *unicastNamingPicker) PickAddr() (string, error) { - p.mu.Lock() - defer p.mu.Unlock() - if p.pickedAddr == nil { - p.pickedAddr = p.addrs.Front() - } else { - pa := p.pickedAddr - p.pickedAddr = pa.Next() - if pa.Value.(*addrInfo).deleting { - p.addrs.Remove(pa) - } - if p.pickedAddr == nil { - p.pickedAddr = p.addrs.Front() - } - } - if p.pickedAddr == nil { - return "", fmt.Errorf("there is no address available to pick") - } - return p.pickedAddr.Value.(*addrInfo).addr, nil -} - -func (p *unicastNamingPicker) State() (ConnectivityState, error) { - return 0, fmt.Errorf("State() is not supported for unicastNamingPicker") -} - -func (p *unicastNamingPicker) WaitForStateChange(ctx context.Context, sourceState ConnectivityState) (ConnectivityState, error) { - return 0, fmt.Errorf("WaitForStateChange is not supported for unicastNamingPciker") -} - -func (p *unicastNamingPicker) Close() error { - p.watcher.Close() - p.conn.Close() - return nil -} diff --git a/picker_test.go b/picker_test.go deleted file mode 100644 index dd29497b..00000000 --- a/picker_test.go +++ /dev/null @@ -1,188 +0,0 @@ -/* - * - * Copyright 2014, Google Inc. - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are - * met: - * - * * Redistributions of source code must retain the above copyright - * notice, this list of conditions and the following disclaimer. - * * Redistributions in binary form must reproduce the above - * copyright notice, this list of conditions and the following disclaimer - * in the documentation and/or other materials provided with the - * distribution. - * * Neither the name of Google Inc. nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR - * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT - * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, - * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT - * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, - * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY - * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT - * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - * - */ - -package grpc - -import ( - "fmt" - "math" - "testing" - "time" - - "golang.org/x/net/context" - "google.golang.org/grpc/naming" -) - -type testWatcher struct { - // the channel to receives name resolution updates - update chan *naming.Update - // the side channel to get to know how many updates in a batch - side chan int - // the channel to notifiy update injector that the update reading is done - readDone chan int -} - -func (w *testWatcher) Next() (updates []*naming.Update, err error) { - n := <-w.side - if n == 0 { - return nil, fmt.Errorf("w.side is closed") - } - for i := 0; i < n; i++ { - u := <-w.update - if u != nil { - updates = append(updates, u) - } - } - w.readDone <- 0 - return -} - -func (w *testWatcher) Close() { -} - -func (w *testWatcher) inject(updates []*naming.Update) { - w.side <- len(updates) - for _, u := range updates { - w.update <- u - } - <-w.readDone -} - -type testNameResolver struct { - w *testWatcher - addr string -} - -func (r *testNameResolver) Resolve(target string) (naming.Watcher, error) { - r.w = &testWatcher{ - update: make(chan *naming.Update, 1), - side: make(chan int, 1), - readDone: make(chan int), - } - r.w.side <- 1 - r.w.update <- &naming.Update{ - Op: naming.Add, - Addr: r.addr, - } - go func() { - <-r.w.readDone - }() - return r.w, nil -} - -func startServers(t *testing.T, numServers, port int, maxStreams uint32) ([]*server, *testNameResolver) { - var servers []*server - for i := 0; i < numServers; i++ { - s := newTestServer() - servers = append(servers, s) - go s.start(t, port, maxStreams) - s.wait(t, 2*time.Second) - } - // Point to server1 - addr := "127.0.0.1:" + servers[0].port - return servers, &testNameResolver{ - addr: addr, - } -} - -func TestNameDiscovery(t *testing.T) { - // Start 3 servers on 3 ports. - servers, r := startServers(t, 3, 0, math.MaxUint32) - cc, err := Dial("foo.bar.com", WithPicker(NewUnicastNamingPicker(r)), WithBlock(), WithInsecure(), WithCodec(testCodec{})) - if err != nil { - t.Fatalf("Failed to create ClientConn: %v", err) - } - var reply string - if err := Invoke(context.Background(), "/foo/bar", &expectedRequest, &reply, cc); err != nil || reply != expectedResponse { - t.Fatalf("grpc.Invoke(_, _, _, _, _) = %v, want ", err) - } - // Inject name resolution change to point to the second server now. - var updates []*naming.Update - updates = append(updates, &naming.Update{ - Op: naming.Delete, - Addr: "127.0.0.1:" + servers[0].port, - }) - updates = append(updates, &naming.Update{ - Op: naming.Add, - Addr: "127.0.0.1:" + servers[1].port, - }) - r.w.inject(updates) - servers[0].stop() - if err := Invoke(context.Background(), "/foo/bar", &expectedRequest, &reply, cc); err != nil || reply != expectedResponse { - t.Fatalf("grpc.Invoke(_, _, _, _, _) = %v, want ", err) - } - // Add another server address (server#3) to name resolution - updates = nil - updates = append(updates, &naming.Update{ - Op: naming.Add, - Addr: "127.0.0.1:" + servers[2].port, - }) - r.w.inject(updates) - // Stop server#2. The library should direct to server#3 automatically. - servers[1].stop() - if err := Invoke(context.Background(), "/foo/bar", &expectedRequest, &reply, cc); err != nil || reply != expectedResponse { - t.Fatalf("grpc.Invoke(_, _, _, _, _) = %v, want ", err) - } - cc.Close() - servers[2].stop() -} - -func TestEmptyAddrs(t *testing.T) { - servers, r := startServers(t, 1, 0, math.MaxUint32) - cc, err := Dial("foo.bar.com", WithPicker(NewUnicastNamingPicker(r)), WithBlock(), WithInsecure(), WithCodec(testCodec{})) - if err != nil { - t.Fatalf("Failed to create ClientConn: %v", err) - } - var reply string - if err := Invoke(context.Background(), "/foo/bar", &expectedRequest, &reply, cc); err != nil || reply != expectedResponse { - t.Fatalf("grpc.Invoke(_, _, _, _, _) = %v, want ", err) - } - // Inject name resolution change to remove the server address so that there is no address - // available after that. - var updates []*naming.Update - updates = append(updates, &naming.Update{ - Op: naming.Delete, - Addr: "127.0.0.1:" + servers[0].port, - }) - r.w.inject(updates) - // Loop until the above updates apply. - for { - time.Sleep(10 * time.Millisecond) - ctx, _ := context.WithTimeout(context.Background(), 10*time.Millisecond) - if err := Invoke(ctx, "/foo/bar", &expectedRequest, &reply, cc); err != nil { - break - } - } - cc.Close() - servers[0].stop() -} diff --git a/stream.go b/stream.go index 22e49cb5..51735e19 100644 --- a/stream.go +++ b/stream.go @@ -103,8 +103,10 @@ func NewClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth var ( t transport.ClientTransport err error + put func() ) - t, err = cc.dopts.picker.Pick(ctx) + //t, err = cc.dopts.picker.Pick(ctx) + t, put, err = cc.getTransport(ctx) if err != nil { return nil, toRPCErr(err) } @@ -119,6 +121,7 @@ func NewClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth } cs := &clientStream{ desc: desc, + put: put, codec: cc.dopts.codec, cp: cc.dopts.cp, dc: cc.dopts.dc, @@ -174,6 +177,7 @@ type clientStream struct { tracing bool // set to EnableTracing when the clientStream is created. mu sync.Mutex + put func() closed bool // trInfo.tr is set when the clientStream is created (if EnableTracing is true), // and is set to nil when the clientStream's finish method is called. @@ -311,6 +315,10 @@ func (cs *clientStream) finish(err error) { } cs.mu.Lock() defer cs.mu.Unlock() + if cs.put != nil { + cs.put() + cs.put = nil + } if cs.trInfo.tr != nil { if err == nil || err == io.EOF { cs.trInfo.tr.LazyPrintf("RPC: [OK]") diff --git a/test/end2end_test.go b/test/end2end_test.go index d3be1eb0..0bfa48d1 100644 --- a/test/end2end_test.go +++ b/test/end2end_test.go @@ -332,14 +332,15 @@ func TestReconnectTimeout(t *testing.T) { ResponseSize: proto.Int32(respSize), Payload: payload, } - if _, err := tc.UnaryCall(context.Background(), req); err == nil { + ctx, _ := context.WithTimeout(context.Background(), 10*time.Millisecond) + if _, err := tc.UnaryCall(ctx, req); err == nil { t.Errorf("TestService/UnaryCall(_, _) = _, , want _, non-nil") return } }() // Block until reconnect times out. <-waitC - if err := conn.Close(); err != grpc.ErrClientConnClosing { + if err := conn.Close(); err != nil { t.Fatalf("%v.Close() = %v, want %v", conn, err, grpc.ErrClientConnClosing) } } @@ -594,36 +595,36 @@ func testTimeoutOnDeadServer(t *testing.T, e env) { cc := te.clientConn() tc := testpb.NewTestServiceClient(cc) - ctx, _ := context.WithTimeout(context.Background(), time.Second) - if _, err := cc.WaitForStateChange(ctx, grpc.Idle); err != nil { - t.Fatalf("cc.WaitForStateChange(_, %s) = _, %v, want _, ", grpc.Idle, err) - } - ctx, _ = context.WithTimeout(context.Background(), time.Second) - if _, err := cc.WaitForStateChange(ctx, grpc.Connecting); err != nil { - t.Fatalf("cc.WaitForStateChange(_, %s) = _, %v, want _, ", grpc.Connecting, err) - } - if state, err := cc.State(); err != nil || state != grpc.Ready { - t.Fatalf("cc.State() = %s, %v, want %s, ", state, err, grpc.Ready) - } - ctx, _ = context.WithTimeout(context.Background(), time.Second) - if _, err := cc.WaitForStateChange(ctx, grpc.Ready); err != context.DeadlineExceeded { - t.Fatalf("cc.WaitForStateChange(_, %s) = _, %v, want _, %v", grpc.Ready, err, context.DeadlineExceeded) - } + //ctx, _ := context.WithTimeout(context.Background(), time.Second) + //if _, err := cc.WaitForStateChange(ctx, grpc.Idle); err != nil { + // t.Fatalf("cc.WaitForStateChange(_, %s) = _, %v, want _, ", grpc.Idle, err) + //} + //ctx, _ = context.WithTimeout(context.Background(), time.Second) + //if _, err := cc.WaitForStateChange(ctx, grpc.Connecting); err != nil { + // t.Fatalf("cc.WaitForStateChange(_, %s) = _, %v, want _, ", grpc.Connecting, err) + //} + //if state, err := cc.State(); err != nil || state != grpc.Ready { + // t.Fatalf("cc.State() = %s, %v, want %s, ", state, err, grpc.Ready) + //} + //ctx, _ = context.WithTimeout(context.Background(), time.Second) + //if _, err := cc.WaitForStateChange(ctx, grpc.Ready); err != context.DeadlineExceeded { + // t.Fatalf("cc.WaitForStateChange(_, %s) = _, %v, want _, %v", grpc.Ready, err, context.DeadlineExceeded) + //} te.srv.Stop() // Set -1 as the timeout to make sure if transportMonitor gets error // notification in time the failure path of the 1st invoke of // ClientConn.wait hits the deadline exceeded error. - ctx, _ = context.WithTimeout(context.Background(), -1) + ctx, _ := context.WithTimeout(context.Background(), -1) if _, err := tc.EmptyCall(ctx, &testpb.Empty{}); grpc.Code(err) != codes.DeadlineExceeded { t.Fatalf("TestService/EmptyCall(%v, _) = _, error %v, want _, error code: %d", ctx, err, codes.DeadlineExceeded) } - ctx, _ = context.WithTimeout(context.Background(), time.Second) - if _, err := cc.WaitForStateChange(ctx, grpc.Ready); err != nil { - t.Fatalf("cc.WaitForStateChange(_, %s) = _, %v, want _, ", grpc.Ready, err) - } - if state, err := cc.State(); err != nil || (state != grpc.Connecting && state != grpc.TransientFailure) { - t.Fatalf("cc.State() = %s, %v, want %s or %s, ", state, err, grpc.Connecting, grpc.TransientFailure) - } + //ctx, _ = context.WithTimeout(context.Background(), time.Second) + //if _, err := cc.WaitForStateChange(ctx, grpc.Ready); err != nil { + // t.Fatalf("cc.WaitForStateChange(_, %s) = _, %v, want _, ", grpc.Ready, err) + //} + //if state, err := cc.State(); err != nil || (state != grpc.Connecting && state != grpc.TransientFailure) { + // t.Fatalf("cc.State() = %s, %v, want %s or %s, ", state, err, grpc.Connecting, grpc.TransientFailure) + //} cc.Close() awaitNewConnLogOutput() } @@ -784,21 +785,23 @@ func testEmptyUnaryWithUserAgent(t *testing.T, e env) { cc := te.clientConn() // Wait until cc is connected. - ctx, _ := context.WithTimeout(context.Background(), time.Second) - if _, err := cc.WaitForStateChange(ctx, grpc.Idle); err != nil { - t.Fatalf("cc.WaitForStateChange(_, %s) = _, %v, want _, ", grpc.Idle, err) - } - ctx, _ = context.WithTimeout(context.Background(), time.Second) - if _, err := cc.WaitForStateChange(ctx, grpc.Connecting); err != nil { - t.Fatalf("cc.WaitForStateChange(_, %s) = _, %v, want _, ", grpc.Connecting, err) - } - if state, err := cc.State(); err != nil || state != grpc.Ready { - t.Fatalf("cc.State() = %s, %v, want %s, ", state, err, grpc.Ready) - } - ctx, _ = context.WithTimeout(context.Background(), time.Second) - if _, err := cc.WaitForStateChange(ctx, grpc.Ready); err == nil { - t.Fatalf("cc.WaitForStateChange(_, %s) = _, , want _, %v", grpc.Ready, context.DeadlineExceeded) - } + //ctx, _ := context.WithTimeout(context.Background(), time.Second) + //if _, err := cc.WaitForStateChange(ctx, grpc.Idle); err != nil { + // t.Fatalf("cc.WaitForStateChange(_, %s) = _, %v, want _, ", grpc.Idle, err) + //} + //ctx, _ = context.WithTimeout(context.Background(), time.Second) + //if _, err := cc.WaitForStateChange(ctx, grpc.Connecting); err != nil { + // t.Fatalf("cc.WaitForStateChange(_, %s) = _, %v, want _, ", grpc.Connecting, err) + //} + //if state, err := cc.State(); err != nil || state != grpc.Ready { + // t.Fatalf("cc.State() = %s, %v, want %s, ", state, err, grpc.Ready) + //} + /* + ctx, _ = context.WithTimeout(context.Background(), time.Second) + if _, err := cc.WaitForStateChange(ctx, grpc.Ready); err == nil { + t.Fatalf("cc.WaitForStateChange(_, %s) = _, , want _, %v", grpc.Ready, context.DeadlineExceeded) + } + */ tc := testpb.NewTestServiceClient(cc) var header metadata.MD reply, err := tc.EmptyCall(context.Background(), &testpb.Empty{}, grpc.Header(&header)) @@ -811,14 +814,15 @@ func testEmptyUnaryWithUserAgent(t *testing.T, e env) { te.srv.Stop() cc.Close() - - ctx, _ = context.WithTimeout(context.Background(), 5*time.Second) - if _, err := cc.WaitForStateChange(ctx, grpc.Ready); err != nil { - t.Fatalf("cc.WaitForStateChange(_, %s) = _, %v, want _, ", grpc.Ready, err) - } - if state, err := cc.State(); err != nil || state != grpc.Shutdown { - t.Fatalf("cc.State() = %s, %v, want %s, ", state, err, grpc.Shutdown) - } + /* + ctx, _ = context.WithTimeout(context.Background(), 5*time.Second) + if _, err := cc.WaitForStateChange(ctx, grpc.Ready); err != nil { + t.Fatalf("cc.WaitForStateChange(_, %s) = _, %v, want _, ", grpc.Ready, err) + } + if state, err := cc.State(); err != nil || state != grpc.Shutdown { + t.Fatalf("cc.State() = %s, %v, want %s, ", state, err, grpc.Shutdown) + } + */ } func TestFailedEmptyUnary(t *testing.T) { @@ -1000,7 +1004,6 @@ func testRetry(t *testing.T, e env) { cc := te.clientConn() tc := testpb.NewTestServiceClient(cc) - var wg sync.WaitGroup numRPC := 1000 diff --git a/test/out.txt b/test/out.txt new file mode 100644 index 00000000..ad188a8d --- /dev/null +++ b/test/out.txt @@ -0,0 +1,1015 @@ +=== RUN TestRetry +2016/05/05 17:56:45 DEBUG ugh here +2016/05/05 17:56:45 DEBUG reach inside resetTransport -2 +2016/05/05 17:56:45 DEBUG reach inside resetTransport -1 +2016/05/05 17:56:45 DEBUG reach inside resetTransport 0 +2016/05/05 17:56:45 DEBUG reach inside resetTransport 1 +2016/05/05 17:56:45 DEBUG reach inside resetTransport 2 +2016/05/05 17:56:45 DEBUG reach inside resetTransport 3: {localhost:46298 } +2016/05/05 17:56:45 DEBUG ugh here resetTransport +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:45 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG reach inside resetTransport -2 +2016/05/05 17:56:46 DEBUG reach inside resetTransport -1 +2016/05/05 17:56:46 DEBUG reach inside resetTransport 0 +2016/05/05 17:56:46 DEBUG reach inside resetTransport 1 +2016/05/05 17:56:46 DEBUG reach inside resetTransport 2 +2016/05/05 17:56:46 DEBUG reach inside resetTransport 3: {localhost:46298 } +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:46 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: +2016/05/05 17:56:47 DEBUG get transport: diff --git a/transport/http2_client.go b/transport/http2_client.go index 8e916b00..77b2e522 100644 --- a/transport/http2_client.go +++ b/transport/http2_client.go @@ -272,6 +272,10 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea } } t.mu.Lock() + if t.activeStreams == nil { + t.mu.Unlock() + return nil, ErrConnClosing + } if t.state != reachable { t.mu.Unlock() return nil, ErrConnClosing @@ -390,6 +394,11 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea func (t *http2Client) CloseStream(s *Stream, err error) { var updateStreams bool t.mu.Lock() + if t.activeStreams == nil { + t.mu.Unlock() + t.Close() + return + } if t.streamsQuota != nil { updateStreams = true } @@ -457,6 +466,17 @@ func (t *http2Client) Close() (err error) { return } +func (t *http2Client) GracefulClose() error { + t.mu.Lock() + active := len(t.activeStreams) + t.activeStreams = nil + t.mu.Unlock() + if active == 0 { + return t.Close() + } + return nil +} + // Write formats the data into HTTP2 data frame(s) and sends it out. The caller // should proceed only if Write returns nil. // TODO(zhaoq): opts.Delay is ignored in this implementation. Support it later diff --git a/transport/transport.go b/transport/transport.go index 87fdf532..b85c0ac4 100644 --- a/transport/transport.go +++ b/transport/transport.go @@ -391,6 +391,8 @@ type ClientTransport interface { // is called only once. Close() error + GracefulClose() error + // Write sends the data for the given stream. A nil stream indicates // the write is to be performed on the transport as a whole. Write(s *Stream, data []byte, opts *Options) error From 19ded2395157c11fc46624a5038f1551f7a2bc44 Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Tue, 10 May 2016 19:29:44 -0700 Subject: [PATCH 02/40] graceful close and test --- balancer.go | 8 +-- clientconn.go | 123 ++++++++++++------------------------ transport/http2_client.go | 15 ++++- transport/transport.go | 1 + transport/transport_test.go | 58 ++++++++++++++--- 5 files changed, 108 insertions(+), 97 deletions(-) diff --git a/balancer.go b/balancer.go index 998c7373..78e4d331 100644 --- a/balancer.go +++ b/balancer.go @@ -44,7 +44,7 @@ type roundRobin struct { pending int } -func (rr *roundRobin) Up(addr Address) func() { +func (rr *roundRobin) Up(addr Address) func(error) { rr.mu.Lock() defer rr.mu.Unlock() for _, a := range rr.addrs { @@ -59,12 +59,12 @@ func (rr *roundRobin) Up(addr Address) func() { rr.waitCh = nil } } - return func() { - rr.down(addr) + return func(err error) { + rr.down(addr, err) } } -func (rr *roundRobin) down(addr Address) { +func (rr *roundRobin) down(addr Address, err error) { rr.mu.Lock() defer rr.mu.Unlock() for i, a := range rr.addrs { diff --git a/clientconn.go b/clientconn.go index 312e07a9..75e59d07 100644 --- a/clientconn.go +++ b/clientconn.go @@ -206,7 +206,7 @@ func WithUserAgent(s string) DialOption { func Dial(target string, opts ...DialOption) (*ClientConn, error) { cc := &ClientConn{ target: target, - infos: make(map[Address]*addrInfo), + conns: make(map[Address]*addrConn), } for _, opt := range opts { opt(&cc.dopts) @@ -235,9 +235,7 @@ func Dial(target string, opts ...DialOption) (*ClientConn, error) { return nil, err } cc.mu.Lock() - cc.infos[addr] = &addrInfo{ - ac: ac, - } + cc.conns[addr] = ac cc.mu.Unlock() } else { w, err := cc.dopts.resolver.Resolve(cc.target) @@ -299,10 +297,6 @@ func (s ConnectivityState) String() string { } } -type addrInfo struct { - ac *addrConn -} - // ClientConn represents a client connection to an RPC service. type ClientConn struct { target string @@ -312,7 +306,7 @@ type ClientConn struct { dopts dialOptions mu sync.RWMutex - infos map[Address]*addrInfo + conns map[Address]*addrConn } func (cc *ClientConn) watchAddrUpdates() error { @@ -328,7 +322,7 @@ func (cc *ClientConn) watchAddrUpdates() error { Addr: update.Addr, Metadata: update.Metadata, } - if _, ok := cc.infos[addr]; ok { + if _, ok := cc.conns[addr]; ok { cc.mu.Unlock() grpclog.Println("grpc: The name resolver wanted to add an existing address: ", addr) continue @@ -340,9 +334,7 @@ func (cc *ClientConn) watchAddrUpdates() error { return err } cc.mu.Lock() - cc.infos[addr] = &addrInfo{ - ac: ac, - } + cc.conns[addr] = ac cc.mu.Unlock() case naming.Delete: cc.mu.Lock() @@ -350,15 +342,16 @@ func (cc *ClientConn) watchAddrUpdates() error { Addr: update.Addr, Metadata: update.Metadata, } - i, ok := cc.infos[addr] + ac, ok := cc.conns[addr] if !ok { cc.mu.Unlock() grpclog.Println("grpc: The name resolver wanted to delete a non-exist address: ", addr) continue } - delete(cc.infos, addr) + delete(cc.conns, addr) cc.mu.Unlock() - i.ac.startDrain() + ac.tearDown(ErrConnDrain) + //ac.startDrain() default: grpclog.Println("Unknown update.Op ", update.Op) } @@ -367,16 +360,10 @@ func (cc *ClientConn) watchAddrUpdates() error { } func (cc *ClientConn) newAddrConn(addr Address) (*addrConn, error) { - /* - if cc.target == "" { - return nil, ErrUnspecTarget - } - */ c := &addrConn{ - cc: cc, - addr: addr, - dopts: cc.dopts, - //resetChan: make(chan int, 1), + cc: cc, + addr: addr, + dopts: cc.dopts, shutdownChan: make(chan struct{}), } if EnableTracing { @@ -415,7 +402,6 @@ func (cc *ClientConn) newAddrConn(addr Address) (*addrConn, error) { c.tearDown(err) return } - grpclog.Println("DEBUG ugh here resetTransport") c.transportMonitor() }() } @@ -428,17 +414,17 @@ func (cc *ClientConn) getTransport(ctx context.Context) (transport.ClientTranspo return nil, nil, err } cc.mu.RLock() - if cc.infos == nil { + if cc.conns == nil { cc.mu.RUnlock() return nil, nil, ErrClientConnClosing } - info, ok := cc.infos[addr] + ac, ok := cc.conns[addr] cc.mu.RUnlock() if !ok { put() return nil, nil, transport.StreamErrorf(codes.Internal, "grpc: failed to find the transport to send the rpc") } - t, err := info.ac.wait(ctx) + t, err := ac.wait(ctx) if err != nil { put() return nil, nil, err @@ -446,47 +432,31 @@ func (cc *ClientConn) getTransport(ctx context.Context) (transport.ClientTranspo return t, put, nil } -/* -// State returns the connectivity state of cc. -// This is EXPERIMENTAL API. -func (cc *ClientConn) State() (ConnectivityState, error) { - return cc.dopts.picker.State() -} - -// 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 { cc.mu.Lock() - if cc.infos == nil { + if cc.conns == nil { cc.mu.Unlock() return ErrClientConnClosing } - infos := cc.infos - cc.infos = nil + conns := cc.conns + cc.conns = nil cc.mu.Unlock() cc.balancer.Close() if cc.watcher != nil { cc.watcher.Close() } - for _, i := range infos { - i.ac.tearDown(ErrClientClosing) + for _, ac := range conns { + ac.tearDown(ErrClientConnClosing) } return nil } // addrConn is a network connection to a given address. type addrConn struct { - cc *ClientConn - addr Address - dopts dialOptions - //resetChan chan int + cc *ClientConn + addr Address + dopts dialOptions shutdownChan chan struct{} events trace.EventLog @@ -494,13 +464,13 @@ type addrConn struct { state ConnectivityState stateCV *sync.Cond down func(error) // the handler called when a connection is down. - drain bool // ready is closed and becomes nil when a new transport is up or failed // due to timeout. ready chan struct{} transport transport.ClientTransport } +/* func (ac *addrConn) startDrain() { ac.mu.Lock() t := ac.transport @@ -510,8 +480,9 @@ func (ac *addrConn) startDrain() { ac.down = nil } ac.mu.Unlock() - t.GracefulClose() + ac.tearDown(ErrConnDrain) } +*/ // printf records an event in ac's event log, unless ac has been closed. // REQUIRES ac.mu is held. @@ -576,10 +547,12 @@ func (ac *addrConn) resetTransport(closeTransport bool) error { ac.mu.Unlock() return errConnClosing } - if ac.drain { - ac.mu.Unlock() - return nil - } + /* + if ac.drain { + ac.mu.Unlock() + return nil + } + */ if ac.down != nil { ac.down(ErrNetworkIO) ac.down = nil @@ -613,7 +586,6 @@ func (ac *addrConn) resetTransport(closeTransport bool) error { copts.Timeout = timeout } connectTime := time.Now() - grpclog.Println("DEBUG reach inside resetTransport 1") newTransport, err := transport.NewClientTransport(ac.addr.Addr, &copts) if err != nil { ac.mu.Lock() @@ -639,7 +611,7 @@ func (ac *addrConn) resetTransport(closeTransport bool) error { ac.mu.Lock() ac.errorf("connection timeout") ac.mu.Unlock() - ac.tearDown(ErrClientTimeout) + ac.tearDown(ErrClientConnTimeout) return ErrClientConnTimeout } closeTransport = false @@ -649,7 +621,6 @@ func (ac *addrConn) resetTransport(closeTransport bool) error { continue } ac.mu.Lock() - grpclog.Println("DEBUG reach inside resetTransport 2") ac.printf("ready") if ac.state == Shutdown { // ac.tearDown(...) has been invoked. @@ -657,7 +628,6 @@ func (ac *addrConn) resetTransport(closeTransport bool) error { newTransport.Close() return errConnClosing } - grpclog.Println("DEBUG reach inside resetTransport 3: ", ac.addr) ac.state = Ready ac.stateCV.Broadcast() ac.transport = newTransport @@ -683,12 +653,6 @@ func (ac *addrConn) transportMonitor() { // the addrConn is idle (i.e., no RPC in flight). case <-ac.shutdownChan: return - /* - case <-ac.resetChan: - if !ac.reconnect() { - return - } - */ case <-t.Error(): ac.mu.Lock() if ac.state == Shutdown { @@ -706,18 +670,6 @@ func (ac *addrConn) transportMonitor() { grpclog.Printf("grpc: addrConn.transportMonitor exits due to: %v", err) return } - /* - if !ac.reconnect() { - return - } - */ - /* - // Tries to drain reset signal if there is any since it is out-dated. - select { - case <-ac.resetChan: - default: - } - */ } } } @@ -751,8 +703,7 @@ func (ac *addrConn) wait(ctx context.Context) (transport.ClientTransport, error) } } -// tearDown starts to tear down the Conn. Returns errConnClosing if -// it has been closed (mostly due to dial time-out). +// tearDown starts to tear down the Conn. // TODO(zhaoq): Make this synchronous to avoid unbounded memory consumption in // some edge cases (e.g., the caller opens and closes many addrConn's in a // tight loop. @@ -777,7 +728,11 @@ func (ac *addrConn) tearDown(err error) { ac.ready = nil } if ac.transport != nil { - ac.transport.Close() + if err == ErrConnDrain { + ac.transport.GracefulClose() + } else { + ac.transport.Close() + } } if ac.shutdownChan != nil { close(ac.shutdownChan) diff --git a/transport/http2_client.go b/transport/http2_client.go index 0fc6a668..4027614b 100644 --- a/transport/http2_client.go +++ b/transport/http2_client.go @@ -403,6 +403,11 @@ func (t *http2Client) CloseStream(s *Stream, err error) { updateStreams = true } delete(t.activeStreams, s.id) + if t.state == draining && len(t.activeStreams) == 0 { + t.mu.Unlock() + t.Close() + return + } t.mu.Unlock() if updateStreams { t.streamsQuota.add(1) @@ -468,8 +473,16 @@ func (t *http2Client) Close() (err error) { func (t *http2Client) GracefulClose() error { t.mu.Lock() + if t.state == closing { + t.mu.Unlock() + return errors.New("transport: Graceful close on a closed transport") + } + if t.state == draining { + t.mu.Unlock() + return nil + } + t.state = draining active := len(t.activeStreams) - t.activeStreams = nil t.mu.Unlock() if active == 0 { return t.Close() diff --git a/transport/transport.go b/transport/transport.go index b85c0ac4..230e215d 100644 --- a/transport/transport.go +++ b/transport/transport.go @@ -321,6 +321,7 @@ const ( reachable transportState = iota unreachable closing + draining ) // NewServerTransport creates a ServerTransport with conn or non-nil error diff --git a/transport/transport_test.go b/transport/transport_test.go index d63dba31..6ebec452 100644 --- a/transport/transport_test.go +++ b/transport/transport_test.go @@ -331,19 +331,17 @@ func TestLargeMessage(t *testing.T) { defer wg.Done() s, err := ct.NewStream(context.Background(), callHdr) if err != nil { - t.Errorf("failed to open stream: %v", err) + t.Errorf("%v.NewStream(_, _) = _, %v, want _, ", ct, err) } if err := ct.Write(s, expectedRequestLarge, &Options{Last: true, Delay: false}); err != nil { - t.Errorf("failed to send data: %v", err) + t.Errorf("%v.Write(_, _, _) = %v, want ", ct, err) } p := make([]byte, len(expectedResponseLarge)) - _, recvErr := io.ReadFull(s, p) - if recvErr != nil || !bytes.Equal(p, expectedResponseLarge) { - t.Errorf("Error: %v, want ; Result len: %d, want len %d", recvErr, len(p), len(expectedResponseLarge)) + if _, err := io.ReadFull(s, p); err != nil || !bytes.Equal(p, expectedResponseLarge) { + t.Errorf("io.ReadFull(_, %v) = _, %v, want %v, ", err, p, expectedResponse) } - _, recvErr = io.ReadFull(s, p) - if recvErr != io.EOF { - t.Errorf("Error: %v; want ", recvErr) + if _, err = io.ReadFull(s, p); err != io.EOF { + t.Errorf("Failed to complete the stream %v; want ", err) } }() } @@ -352,6 +350,50 @@ func TestLargeMessage(t *testing.T) { server.stop() } +func TestGracefulClose(t *testing.T) { + server, ct := setUp(t, 0, math.MaxUint32, normal) + callHdr := &CallHdr{ + Host: "localhost", + Method: "foo.Small", + } + s, err := ct.NewStream(context.Background(), callHdr) + if err != nil { + t.Fatalf("%v.NewStream(_, _) = _, %v, want _, ", ct, err) + } + if err = ct.GracefulClose(); err != nil { + t.Fatalf("%v.GracefulClose() = %v, want ", ct, err) + } + var wg sync.WaitGroup + // Expect the failure for all the follow-up streams because ct has been closed gracefully. + for i := 0; i < 100; i++ { + wg.Add(1) + go func() { + defer wg.Done() + if _, err := ct.NewStream(context.Background(), callHdr); err != ErrConnClosing { + t.Errorf("%v.NewStream(_, _) = _, %v, want _, %v", err, ErrConnClosing) + } + }() + } + opts := Options{ + Last: true, + Delay: false, + } + // The stream which was created before graceful close can still proceed. + if err := ct.Write(s, expectedRequest, &opts); err != nil { + t.Fatalf("%v.Write(_, _, _) = %v, want ", ct, err) + } + p := make([]byte, len(expectedResponse)) + if _, err := io.ReadFull(s, p); err != nil || !bytes.Equal(p, expectedResponse) { + t.Fatalf("io.ReadFull(_, %v) = _, %v, want %v, ", err, p, expectedResponse) + } + if _, err = io.ReadFull(s, p); err != io.EOF { + t.Fatalf("Failed to complete the stream %v; want ", err) + } + wg.Wait() + ct.Close() + server.stop() +} + func TestLargeMessageSuspension(t *testing.T) { server, ct := setUp(t, 0, math.MaxUint32, suspended) callHdr := &CallHdr{ From a1b60d7b43696187fcd83baf55d3a1ffe7676304 Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Thu, 12 May 2016 17:01:58 -0700 Subject: [PATCH 03/40] Add the tests for balancer and resolver --- balancer_test.go | 222 +++++++++++++++++++++++++++++++++++++++++++++++ call_test.go | 13 ++- clientconn.go | 3 +- 3 files changed, 235 insertions(+), 3 deletions(-) create mode 100644 balancer_test.go diff --git a/balancer_test.go b/balancer_test.go new file mode 100644 index 00000000..9fa76b4b --- /dev/null +++ b/balancer_test.go @@ -0,0 +1,222 @@ +/* + * + * Copyright 2014, Google Inc. + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are + * met: + * + * * Redistributions of source code must retain the above copyright + * notice, this list of conditions and the following disclaimer. + * * Redistributions in binary form must reproduce the above + * copyright notice, this list of conditions and the following disclaimer + * in the documentation and/or other materials provided with the + * distribution. + * * Neither the name of Google Inc. nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + */ + +package grpc + +import ( + "fmt" + "math" + "testing" + "time" + + "golang.org/x/net/context" + "google.golang.org/grpc/naming" +) + +type testWatcher struct { + // the channel to receives name resolution updates + update chan *naming.Update + // the side channel to get to know how many updates in a batch + side chan int + // the channel to notifiy update injector that the update reading is done + readDone chan int +} + +func (w *testWatcher) Next() (updates []*naming.Update, err error) { + n := <-w.side + if n == 0 { + return nil, fmt.Errorf("w.side is closed") + } + for i := 0; i < n; i++ { + u := <-w.update + if u != nil { + updates = append(updates, u) + } + } + w.readDone <- 0 + return +} + +func (w *testWatcher) Close() { +} + +func (w *testWatcher) inject(updates []*naming.Update) { + w.side <- len(updates) + for _, u := range updates { + w.update <- u + } + <-w.readDone +} + +type testNameResolver struct { + w *testWatcher + addr string +} + +func (r *testNameResolver) Resolve(target string) (naming.Watcher, error) { + r.w = &testWatcher{ + update: make(chan *naming.Update, 1), + side: make(chan int, 1), + readDone: make(chan int), + } + r.w.side <- 1 + r.w.update <- &naming.Update{ + Op: naming.Add, + Addr: r.addr, + } + go func() { + <-r.w.readDone + }() + return r.w, nil +} + +func startServers(t *testing.T, numServers, port int, maxStreams uint32) ([]*server, *testNameResolver) { + var servers []*server + for i := 0; i < numServers; i++ { + s := newTestServer() + servers = append(servers, s) + go s.start(t, port, maxStreams) + s.wait(t, 2*time.Second) + } + // Point to server[0] + addr := "127.0.0.1:" + servers[0].port + return servers, &testNameResolver{ + addr: addr, + } +} + +func TestNameDiscovery(t *testing.T) { + // Start 2 servers on 2 ports. + numServers := 2 + servers, r := startServers(t, numServers, 0, math.MaxUint32) + cc, err := Dial("foo.bar.com", WithNameResolver(r), WithBlock(), WithInsecure(), WithCodec(testCodec{})) + if err != nil { + t.Fatalf("Failed to create ClientConn: %v", err) + } + req := "port" + var reply string + if err := Invoke(context.Background(), "/foo/bar", &req, &reply, cc); err == nil || ErrorDesc(err) != servers[0].port { + t.Fatalf("grpc.Invoke(_, _, _, _, _) = %v, want %s", err, servers[0].port) + } + // Inject name resolution change to point to the second server now. + var updates []*naming.Update + updates = append(updates, &naming.Update{ + Op: naming.Delete, + Addr: "127.0.0.1:" + servers[0].port, + }) + updates = append(updates, &naming.Update{ + Op: naming.Add, + Addr: "127.0.0.1:" + servers[1].port, + }) + r.w.inject(updates) + for { + if err := Invoke(context.Background(), "/foo/bar", &req, &reply, cc); err != nil && ErrorDesc(err) == servers[1].port { + break + } + time.Sleep(10 * time.Millisecond) + } + cc.Close() + for i := 0; i < numServers; i++ { + servers[i].stop() + } +} + +func TestEmptyAddrs(t *testing.T) { + servers, r := startServers(t, 1, 0, math.MaxUint32) + cc, err := Dial("foo.bar.com", WithNameResolver(r), WithBlock(), WithInsecure(), WithCodec(testCodec{})) + if err != nil { + t.Fatalf("Failed to create ClientConn: %v", err) + } + var reply string + if err := Invoke(context.Background(), "/foo/bar", &expectedRequest, &reply, cc); err != nil || reply != expectedResponse { + t.Fatalf("grpc.Invoke(_, _, _, _, _) = %v, want ", err) + } + // Inject name resolution change to remove the server address so that there is no address + // available after that. + var updates []*naming.Update + updates = append(updates, &naming.Update{ + Op: naming.Delete, + Addr: "127.0.0.1:" + servers[0].port, + }) + r.w.inject(updates) + // Loop until the above updates apply. + for { + time.Sleep(10 * time.Millisecond) + ctx, _ := context.WithTimeout(context.Background(), 10*time.Millisecond) + if err := Invoke(ctx, "/foo/bar", &expectedRequest, &reply, cc); err != nil { + break + } + } + cc.Close() + servers[0].stop() +} + +func TestRoundRobin(t *testing.T) { + // Start 3 servers on 3 ports. + numServers := 3 + servers, r := startServers(t, numServers, 0, math.MaxUint32) + cc, err := Dial("foo.bar.com", WithNameResolver(r), WithBlock(), WithInsecure(), WithCodec(testCodec{})) + if err != nil { + t.Fatalf("Failed to create ClientConn: %v", err) + } + // Add the other 2 servers as the name updates. + var updates []*naming.Update + updates = append(updates, &naming.Update{ + Op: naming.Add, + Addr: "127.0.0.1:" + servers[1].port, + }) + updates = append(updates, &naming.Update{ + Op: naming.Add, + Addr: "127.0.0.1:" + servers[2].port, + }) + r.w.inject(updates) + req := "port" + var reply string + // Loop until an RPC is completed by servers[2]. + for { + if err := Invoke(context.Background(), "/foo/bar", &req, &reply, cc); err != nil && ErrorDesc(err) == servers[2].port { + break + } + time.Sleep(10 * time.Millisecond) + } + // Check it works in round-robin manner. + for i := 0; i < 10; i++ { + if err := Invoke(context.Background(), "/foo/bar", &req, &reply, cc); err == nil || ErrorDesc(err) != servers[i%numServers].port { + t.Fatalf("Invoke(_, _, _, _, _) = %v, want %s", err, servers[i%numServers].port) + } + } + cc.Close() + for i := 0; i < numServers; i++ { + servers[i].stop() + } +} diff --git a/call_test.go b/call_test.go index 7d01f457..20d4adb4 100644 --- a/call_test.go +++ b/call_test.go @@ -74,7 +74,8 @@ func (testCodec) String() string { } type testStreamHandler struct { - t transport.ServerTransport + port string + t transport.ServerTransport } func (h *testStreamHandler) handleStream(t *testing.T, s *transport.Stream) { @@ -106,6 +107,11 @@ func (h *testStreamHandler) handleStream(t *testing.T, s *transport.Stream) { h.t.WriteStatus(s, codes.Internal, "") return } + if v == "port" { + h.t.WriteStatus(s, codes.Internal, h.port) + return + } + if v != expectedRequest { h.t.WriteStatus(s, codes.Internal, strings.Repeat("A", sizeLargeErr)) return @@ -170,7 +176,10 @@ func (s *server) start(t *testing.T, port int, maxStreams uint32) { } s.conns[st] = true s.mu.Unlock() - h := &testStreamHandler{st} + h := &testStreamHandler{ + port: s.port, + t: st, + } go st.HandleStreams(func(s *transport.Stream) { go h.handleStream(t, s) }) diff --git a/clientconn.go b/clientconn.go index 75e59d07..8059e6f6 100644 --- a/clientconn.go +++ b/clientconn.go @@ -114,7 +114,8 @@ func WithDecompressor(dc Decompressor) DialOption { } } -func WithResolver(r naming.Resolver) DialOption { +// WithNameResolver returns a DialOption which sets a name resolver for service discovery. +func WithNameResolver(r naming.Resolver) DialOption { return func(o *dialOptions) { o.resolver = r } From 32eec1acef1a69612a8dcb83098543803d0a3896 Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Thu, 12 May 2016 18:52:24 -0700 Subject: [PATCH 04/40] add comments --- balancer.go | 23 ++++++++++++----------- clientconn.go | 2 +- 2 files changed, 13 insertions(+), 12 deletions(-) diff --git a/balancer.go b/balancer.go index 78e4d331..d5f34433 100644 --- a/balancer.go +++ b/balancer.go @@ -7,8 +7,9 @@ import ( "google.golang.org/grpc/transport" ) +// Address represents a server the client connects to. type Address struct { - // Addr is the peer address on which a connection will be established. + // Addr is the server address on which a connection will be established. Addr string // Metadata is the information associated with Addr, which may be used // to make load balancing decision. This is from the metadata attached @@ -32,18 +33,20 @@ type Balancer interface { Close() error } +// RoundRobin returns a Balancer that selects addresses round-robin. func RoundRobin() Balancer { return &roundRobin{} } type roundRobin struct { - mu sync.Mutex - addrs []Address - next int - waitCh chan struct{} - pending int + mu sync.Mutex + addrs []Address + next int // index of the next address to return for Get() + waitCh chan struct{} } +// Up appends addr to the end of rr.addrs and sends notification if there +// are pending Get() calls. func (rr *roundRobin) Up(addr Address) func(error) { rr.mu.Lock() defer rr.mu.Unlock() @@ -64,6 +67,7 @@ func (rr *roundRobin) Up(addr Address) func(error) { } } +// down removes addr from rr.addrs and moves the remaining addrs forward. func (rr *roundRobin) down(addr Address, err error) { rr.mu.Lock() defer rr.mu.Unlock() @@ -76,6 +80,7 @@ func (rr *roundRobin) down(addr Address, err error) { } } +// Get returns the next addr in the rotation. It blocks if there is no address available. func (rr *roundRobin) Get(ctx context.Context) (addr Address, put func(), err error) { var ch chan struct{} rr.mu.Lock() @@ -85,13 +90,13 @@ func (rr *roundRobin) Get(ctx context.Context) (addr Address, put func(), err er if len(rr.addrs) > 0 { addr = rr.addrs[rr.next] rr.next++ - rr.pending++ rr.mu.Unlock() put = func() { rr.put(ctx, addr) } return } + // There is no address available. Wait on rr.waitCh. if rr.waitCh == nil { ch = make(chan struct{}) rr.waitCh = ch @@ -116,7 +121,6 @@ func (rr *roundRobin) Get(ctx context.Context) (addr Address, put func(), err er } addr = rr.addrs[rr.next] rr.next++ - rr.pending++ rr.mu.Unlock() put = func() { rr.put(ctx, addr) @@ -127,9 +131,6 @@ func (rr *roundRobin) Get(ctx context.Context) (addr Address, put func(), err er } func (rr *roundRobin) put(ctx context.Context, addr Address) { - rr.mu.Lock() - defer rr.mu.Unlock() - rr.pending-- } func (rr *roundRobin) Close() error { diff --git a/clientconn.go b/clientconn.go index 8f373640..4abfc32a 100644 --- a/clientconn.go +++ b/clientconn.go @@ -618,7 +618,7 @@ func (ac *addrConn) resetTransport(closeTransport bool) error { closeTransport = false select { case <-time.After(sleepTime): - case <-cc.shutdownChan: + case <-ac.shutdownChan: } retries++ grpclog.Printf("grpc: addrConn.resetTransport failed to create client transport: %v; Reconnecting to %q", err, ac.addr) From 482b6e5c345ef9195d7ff163c40b62da8dd7291d Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Thu, 12 May 2016 19:01:25 -0700 Subject: [PATCH 05/40] more comments --- balancer.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/balancer.go b/balancer.go index d5f34433..671d2b3d 100644 --- a/balancer.go +++ b/balancer.go @@ -41,8 +41,8 @@ func RoundRobin() Balancer { type roundRobin struct { mu sync.Mutex addrs []Address - next int // index of the next address to return for Get() - waitCh chan struct{} + next int // index of the next address to return for Get() + waitCh chan struct{} // channel to block when there is no address available } // Up appends addr to the end of rr.addrs and sends notification if there From 634e886afa1e21d3d2b6fbf5ba6495cc366cf5fb Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Thu, 12 May 2016 19:19:14 -0700 Subject: [PATCH 06/40] complete some unfinished error --- clientconn.go | 23 ++++++++--------------- 1 file changed, 8 insertions(+), 15 deletions(-) diff --git a/clientconn.go b/clientconn.go index 4abfc32a..040ca830 100644 --- a/clientconn.go +++ b/clientconn.go @@ -68,7 +68,7 @@ var ( // 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: ") + ErrConnDrain = errors.New("grpc: the connection is drained") errConnClosing = errors.New("grpc: the addrConn is closing") // minimum time to give a connection to complete minConnectTimeout = 20 * time.Second @@ -121,6 +121,13 @@ func WithNameResolver(r naming.Resolver) DialOption { } } +// WithBalancer returns a DialOption which sets a load balancer. +func WithBalancer(b Balancer) DialOption { + return func(o *dialOptions) { + o.balancer = b + } +} + // WithBackoffMaxDelay configures the dialer to use the provided maximum delay // when backing off after failed connection attempts. func WithBackoffMaxDelay(md time.Duration) DialOption { @@ -471,20 +478,6 @@ type addrConn struct { transport transport.ClientTransport } -/* -func (ac *addrConn) startDrain() { - ac.mu.Lock() - t := ac.transport - ac.drain = true - if ac.down != nil { - ac.down(ErrConnDrain) - ac.down = nil - } - ac.mu.Unlock() - ac.tearDown(ErrConnDrain) -} -*/ - // 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{}) { From e3351672baaa3c3526209677a5f7a5f156b1716f Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Thu, 12 May 2016 19:19:14 -0700 Subject: [PATCH 07/40] complete some unfinished error --- clientconn.go | 23 ++++++++--------------- 1 file changed, 8 insertions(+), 15 deletions(-) diff --git a/clientconn.go b/clientconn.go index 4abfc32a..040ca830 100644 --- a/clientconn.go +++ b/clientconn.go @@ -68,7 +68,7 @@ var ( // 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: ") + ErrConnDrain = errors.New("grpc: the connection is drained") errConnClosing = errors.New("grpc: the addrConn is closing") // minimum time to give a connection to complete minConnectTimeout = 20 * time.Second @@ -121,6 +121,13 @@ func WithNameResolver(r naming.Resolver) DialOption { } } +// WithBalancer returns a DialOption which sets a load balancer. +func WithBalancer(b Balancer) DialOption { + return func(o *dialOptions) { + o.balancer = b + } +} + // WithBackoffMaxDelay configures the dialer to use the provided maximum delay // when backing off after failed connection attempts. func WithBackoffMaxDelay(md time.Duration) DialOption { @@ -471,20 +478,6 @@ type addrConn struct { transport transport.ClientTransport } -/* -func (ac *addrConn) startDrain() { - ac.mu.Lock() - t := ac.transport - ac.drain = true - if ac.down != nil { - ac.down(ErrConnDrain) - ac.down = nil - } - ac.mu.Unlock() - ac.tearDown(ErrConnDrain) -} -*/ - // 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{}) { From f4b89fe44f0e457d5f4251e0c1978c78f97ac987 Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Fri, 13 May 2016 13:27:35 -0700 Subject: [PATCH 08/40] some fixes --- clientconn.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/clientconn.go b/clientconn.go index 040ca830..723b2882 100644 --- a/clientconn.go +++ b/clientconn.go @@ -233,7 +233,6 @@ func Dial(target string, opts ...DialOption) (*ClientConn, error) { cc.balancer = RoundRobin() } - // Start the first connection if cc.dopts.resolver == nil { addr := Address{ Addr: cc.target, @@ -251,10 +250,11 @@ func Dial(target string, opts ...DialOption) (*ClientConn, error) { return nil, err } cc.watcher = w - // Get the initial name resolution which starts dialing. + // Get the initial name resolution and dial the first connection. if err := cc.watchAddrUpdates(); err != nil { return nil, err } + // Start a goroutine to watch for the future name resolution changes. go func() { for { if err := cc.watchAddrUpdates(); err != nil { @@ -338,7 +338,6 @@ func (cc *ClientConn) watchAddrUpdates() error { cc.mu.Unlock() ac, err := cc.newAddrConn(addr) if err != nil { - cc.mu.Unlock() return err } cc.mu.Lock() @@ -359,7 +358,6 @@ func (cc *ClientConn) watchAddrUpdates() error { delete(cc.conns, addr) cc.mu.Unlock() ac.tearDown(ErrConnDrain) - //ac.startDrain() default: grpclog.Println("Unknown update.Op ", update.Op) } From 31ef3ef957261c1d8191da19530f366ece5ec835 Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Fri, 13 May 2016 15:08:09 -0700 Subject: [PATCH 09/40] some touchups --- clientconn.go | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/clientconn.go b/clientconn.go index 723b2882..8327fed1 100644 --- a/clientconn.go +++ b/clientconn.go @@ -325,13 +325,13 @@ func (cc *ClientConn) watchAddrUpdates() error { for _, update := range updates { switch update.Op { case naming.Add: - cc.mu.Lock() + cc.mu.RLock() addr := Address{ Addr: update.Addr, Metadata: update.Metadata, } if _, ok := cc.conns[addr]; ok { - cc.mu.Unlock() + cc.mu.RUnlock() grpclog.Println("grpc: The name resolver wanted to add an existing address: ", addr) continue } @@ -539,12 +539,6 @@ func (ac *addrConn) resetTransport(closeTransport bool) error { ac.mu.Unlock() return errConnClosing } - /* - if ac.drain { - ac.mu.Unlock() - return nil - } - */ if ac.down != nil { ac.down(ErrNetworkIO) ac.down = nil From 2161303fcd04fc9fb1c7e0f6c754cc7f9d4190ac Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Fri, 13 May 2016 15:14:38 -0700 Subject: [PATCH 10/40] Remove pipe from this change. --- examples/pipe/main.go | 67 ------------------------------------------- 1 file changed, 67 deletions(-) delete mode 100644 examples/pipe/main.go diff --git a/examples/pipe/main.go b/examples/pipe/main.go deleted file mode 100644 index 84a10726..00000000 --- a/examples/pipe/main.go +++ /dev/null @@ -1,67 +0,0 @@ -package main - -import ( - "fmt" - "log" - "net" - "time" - - "google.golang.org/grpc" -) - -type memAddr string - -func (a memAddr) Network() string { return "mem" } -func (a memAddr) String() string { return string(a) } - -type memListener struct { - c chan net.Conn -} - -func (ln *memListener) Accept() (net.Conn, error) { - conn, ok := <-ln.c - if !ok { - return nil, fmt.Errorf("closed") - } - return conn, nil -} - -func (ln *memListener) Addr() net.Addr { - return memAddr(fmt.Sprintf("%p", ln)) -} - -func (ln *memListener) Close() error { - close(ln.c) - return nil -} - -func main() { - grpc.EnableTracing = true - - ln := &memListener{ - c: make(chan net.Conn, 1), - } - - go func() { - s := grpc.NewServer() - log.Fatal(s.Serve(ln)) - }() - - log.Printf("Dialing to server over a synchronous pipe...") - serverConn, err := grpc.Dial("inmemory", - grpc.WithInsecure(), - grpc.WithBlock(), - grpc.WithDialer(func(_ string, _ time.Duration) (net.Conn, error) { - c1, c2 := net.Pipe() - log.Printf("Pipe created: %v %v", c1, c2) - ln.c <- c2 - log.Printf("Pipe accepted: %v %v", c1, c2) - return c1, nil - })) - if err != nil { - log.Fatal(err) - } - - // BUG: never reached - log.Printf("SUCCESS! Connected to server: %v", serverConn) -} From aa532d5baf0b6b739d7bcf70a277206e68ade40a Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Mon, 16 May 2016 15:31:00 -0700 Subject: [PATCH 11/40] Fix some issues and remove garbbage files --- balancer.go | 18 + balancer_test.go | 100 +++- call_test.go | 2 +- clientconn.go | 48 +- stream.go | 1 - test/end2end_test.go | 13 +- test/out.txt | 1015 ------------------------------------- transport/http2_client.go | 5 +- 8 files changed, 152 insertions(+), 1050 deletions(-) delete mode 100644 test/out.txt diff --git a/balancer.go b/balancer.go index 671d2b3d..fa09318d 100644 --- a/balancer.go +++ b/balancer.go @@ -43,6 +43,7 @@ type roundRobin struct { addrs []Address next int // index of the next address to return for Get() waitCh chan struct{} // channel to block when there is no address available + done bool // The Balancer is closed. } // Up appends addr to the end of rr.addrs and sends notification if there @@ -84,6 +85,11 @@ func (rr *roundRobin) down(addr Address, err error) { func (rr *roundRobin) Get(ctx context.Context) (addr Address, put func(), err error) { var ch chan struct{} rr.mu.Lock() + if rr.done { + rr.mu.Unlock() + err = ErrClientConnClosing + return + } if rr.next >= len(rr.addrs) { rr.next = 0 } @@ -111,6 +117,11 @@ func (rr *roundRobin) Get(ctx context.Context) (addr Address, put func(), err er return case <-ch: rr.mu.Lock() + if rr.done { + rr.mu.Unlock() + err = ErrClientConnClosing + return + } if len(rr.addrs) == 0 { // The newly added addr got removed by Down() again. rr.mu.Unlock() @@ -134,5 +145,12 @@ func (rr *roundRobin) put(ctx context.Context, addr Address) { } func (rr *roundRobin) Close() error { + rr.mu.Lock() + defer rr.mu.Unlock() + rr.done = true + if rr.waitCh != nil { + close(rr.waitCh) + rr.waitCh = nil + } return nil } diff --git a/balancer_test.go b/balancer_test.go index 9fa76b4b..f09ef1ac 100644 --- a/balancer_test.go +++ b/balancer_test.go @@ -1,6 +1,6 @@ /* * - * Copyright 2014, Google Inc. + * Copyright 2016, Google Inc. * All rights reserved. * * Redistribution and use in source and binary forms, with or without @@ -36,10 +36,12 @@ package grpc import ( "fmt" "math" + "sync" "testing" "time" "golang.org/x/net/context" + "google.golang.org/grpc/codes" "google.golang.org/grpc/naming" ) @@ -100,12 +102,12 @@ func (r *testNameResolver) Resolve(target string) (naming.Watcher, error) { return r.w, nil } -func startServers(t *testing.T, numServers, port int, maxStreams uint32) ([]*server, *testNameResolver) { +func startServers(t *testing.T, numServers int, maxStreams uint32) ([]*server, *testNameResolver) { var servers []*server for i := 0; i < numServers; i++ { s := newTestServer() servers = append(servers, s) - go s.start(t, port, maxStreams) + go s.start(t, 0, maxStreams) s.wait(t, 2*time.Second) } // Point to server[0] @@ -118,7 +120,7 @@ func startServers(t *testing.T, numServers, port int, maxStreams uint32) ([]*ser func TestNameDiscovery(t *testing.T) { // Start 2 servers on 2 ports. numServers := 2 - servers, r := startServers(t, numServers, 0, math.MaxUint32) + servers, r := startServers(t, numServers, math.MaxUint32) cc, err := Dial("foo.bar.com", WithNameResolver(r), WithBlock(), WithInsecure(), WithCodec(testCodec{})) if err != nil { t.Fatalf("Failed to create ClientConn: %v", err) @@ -152,7 +154,7 @@ func TestNameDiscovery(t *testing.T) { } func TestEmptyAddrs(t *testing.T) { - servers, r := startServers(t, 1, 0, math.MaxUint32) + servers, r := startServers(t, 1, math.MaxUint32) cc, err := Dial("foo.bar.com", WithNameResolver(r), WithBlock(), WithInsecure(), WithCodec(testCodec{})) if err != nil { t.Fatalf("Failed to create ClientConn: %v", err) @@ -184,7 +186,7 @@ func TestEmptyAddrs(t *testing.T) { func TestRoundRobin(t *testing.T) { // Start 3 servers on 3 ports. numServers := 3 - servers, r := startServers(t, numServers, 0, math.MaxUint32) + servers, r := startServers(t, numServers, math.MaxUint32) cc, err := Dial("foo.bar.com", WithNameResolver(r), WithBlock(), WithInsecure(), WithCodec(testCodec{})) if err != nil { t.Fatalf("Failed to create ClientConn: %v", err) @@ -220,3 +222,89 @@ func TestRoundRobin(t *testing.T) { servers[i].stop() } } + +func TestCloseWithPendingRPC(t *testing.T) { + servers, r := startServers(t, 1, math.MaxUint32) + cc, err := Dial("foo.bar.com", WithNameResolver(r), WithBlock(), WithInsecure(), WithCodec(testCodec{})) + if err != nil { + t.Fatalf("Failed to create ClientConn: %v", err) + } + var reply string + if err := Invoke(context.Background(), "/foo/bar", &expectedRequest, &reply, cc); err != nil { + t.Fatalf("grpc.Invoke(_, _, _, _, _) = %v, want %s", err, servers[0].port) + } + // Remove the server. + updates := []*naming.Update{&naming.Update{ + Op: naming.Delete, + Addr: "127.0.0.1:" + servers[0].port, + }} + r.w.inject(updates) + for { + ctx, _ := context.WithTimeout(context.Background(), 10*time.Millisecond) + if err := Invoke(ctx, "/foo/bar", &expectedRequest, &reply, cc); Code(err) == codes.DeadlineExceeded { + break + } + time.Sleep(10 * time.Millisecond) + } + var wg sync.WaitGroup + wg.Add(2) + go func() { + defer wg.Done() + var reply string + if err := Invoke(context.Background(), "/foo/bar", &expectedRequest, &reply, cc); err == nil { + t.Errorf("grpc.Invoke(_, _, _, _, _) = %v, want not", err) + } + }() + go func() { + defer wg.Done() + var reply string + time.Sleep(5 * time.Millisecond) + if err := Invoke(context.Background(), "/foo/bar", &expectedRequest, &reply, cc); err == nil { + t.Errorf("grpc.Invoke(_, _, _, _, _) = %v, want not nil", err) + } + }() + time.Sleep(5 * time.Millisecond) + cc.Close() + wg.Wait() + servers[0].stop() +} + +func TestGetOnWaitChannel(t *testing.T) { + servers, r := startServers(t, 1, math.MaxUint32) + cc, err := Dial("foo.bar.com", WithNameResolver(r), WithBlock(), WithInsecure(), WithCodec(testCodec{})) + if err != nil { + t.Fatalf("Failed to create ClientConn: %v", err) + } + // Remove all servers so that all upcoming RPCs will block on waitCh. + updates := []*naming.Update{&naming.Update{ + Op: naming.Delete, + Addr: "127.0.0.1:" + servers[0].port, + }} + r.w.inject(updates) + for { + var reply string + ctx, _ := context.WithTimeout(context.Background(), 10*time.Millisecond) + if err := Invoke(ctx, "/foo/bar", &expectedRequest, &reply, cc); Code(err) == codes.DeadlineExceeded { + break + } + time.Sleep(10 * time.Millisecond) + } + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + var reply string + if err := Invoke(context.Background(), "/foo/bar", &expectedRequest, &reply, cc); err != nil { + t.Errorf("grpc.Invoke(_, _, _, _, _) = %v, want ", err) + } + }() + // Add a connected server. + updates = []*naming.Update{&naming.Update{ + Op: naming.Add, + Addr: "127.0.0.1:" + servers[0].port, + }} + r.w.inject(updates) + wg.Wait() + cc.Close() + servers[0].stop() +} diff --git a/call_test.go b/call_test.go index 20d4adb4..380bf872 100644 --- a/call_test.go +++ b/call_test.go @@ -166,7 +166,7 @@ func (s *server) start(t *testing.T, port int, maxStreams uint32) { } st, err := transport.NewServerTransport("http2", conn, maxStreams, nil) if err != nil { - return + continue } s.mu.Lock() if s.conns == nil { diff --git a/clientconn.go b/clientconn.go index 8327fed1..2db6b903 100644 --- a/clientconn.go +++ b/clientconn.go @@ -237,13 +237,9 @@ func Dial(target string, opts ...DialOption) (*ClientConn, error) { addr := Address{ Addr: cc.target, } - ac, err := cc.newAddrConn(addr) - if err != nil { + if err := cc.newAddrConn(addr); err != nil { return nil, err } - cc.mu.Lock() - cc.conns[addr] = ac - cc.mu.Unlock() } else { w, err := cc.dopts.resolver.Resolve(cc.target) if err != nil { @@ -335,14 +331,15 @@ func (cc *ClientConn) watchAddrUpdates() error { grpclog.Println("grpc: The name resolver wanted to add an existing address: ", addr) continue } - cc.mu.Unlock() - ac, err := cc.newAddrConn(addr) - if err != nil { + cc.mu.RUnlock() + if err := cc.newAddrConn(addr); err != nil { return err } - cc.mu.Lock() - cc.conns[addr] = ac - cc.mu.Unlock() + /* + cc.mu.Lock() + cc.conns[addr] = ac + cc.mu.Unlock() + */ case naming.Delete: cc.mu.Lock() addr := Address{ @@ -355,7 +352,6 @@ func (cc *ClientConn) watchAddrUpdates() error { grpclog.Println("grpc: The name resolver wanted to delete a non-exist address: ", addr) continue } - delete(cc.conns, addr) cc.mu.Unlock() ac.tearDown(ErrConnDrain) default: @@ -365,7 +361,7 @@ func (cc *ClientConn) watchAddrUpdates() error { return nil } -func (cc *ClientConn) newAddrConn(addr Address) (*addrConn, error) { +func (cc *ClientConn) newAddrConn(addr Address) error { c := &addrConn{ cc: cc, addr: addr, @@ -383,12 +379,12 @@ func (cc *ClientConn) newAddrConn(addr Address) (*addrConn, error) { } } if !ok { - return nil, ErrNoTransportSecurity + return ErrNoTransportSecurity } } else { for _, cd := range c.dopts.copts.AuthOptions { if cd.RequireTransportSecurity() { - return nil, ErrCredentialsMisuse + return ErrCredentialsMisuse } } } @@ -396,7 +392,7 @@ func (cc *ClientConn) newAddrConn(addr Address) (*addrConn, error) { if c.dopts.block { if err := c.resetTransport(false); err != nil { c.tearDown(err) - return nil, err + return err } // Start to monitor the error status of transport. go c.transportMonitor() @@ -411,7 +407,7 @@ func (cc *ClientConn) newAddrConn(addr Address) (*addrConn, error) { c.transportMonitor() }() } - return c, nil + return nil } func (cc *ClientConn) getTransport(ctx context.Context) (transport.ClientTransport, func(), error) { @@ -529,6 +525,13 @@ func (ac *addrConn) waitForStateChange(ctx context.Context, sourceState Connecti } func (ac *addrConn) resetTransport(closeTransport bool) error { + ac.cc.mu.Lock() + if ac.cc.conns == nil { + ac.cc.mu.Unlock() + return ErrClientConnClosing + } + ac.cc.conns[ac.addr] = ac + ac.cc.mu.Unlock() var retries int start := time.Now() for { @@ -692,13 +695,20 @@ func (ac *addrConn) wait(ctx context.Context) (transport.ClientTransport, error) } } -// tearDown starts to tear down the Conn. +// 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 addrConn's in a // tight loop. func (ac *addrConn) tearDown(err error) { ac.mu.Lock() - defer ac.mu.Unlock() + 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.down != nil { ac.down(err) ac.down = nil diff --git a/stream.go b/stream.go index d11eff18..7f0e7c3e 100644 --- a/stream.go +++ b/stream.go @@ -105,7 +105,6 @@ func NewClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth err error put func() ) - //t, err = cc.dopts.picker.Pick(ctx) t, put, err = cc.getTransport(ctx) if err != nil { return nil, toRPCErr(err) diff --git a/test/end2end_test.go b/test/end2end_test.go index acfb3828..ea87c7c3 100644 --- a/test/end2end_test.go +++ b/test/end2end_test.go @@ -435,14 +435,17 @@ type test struct { func (te *test) tearDown() { if te.cancel != nil { te.cancel() + te.cancel = nil } - te.srv.Stop() if te.cc != nil { te.cc.Close() + te.cc = nil } if te.restoreLogs != nil { te.restoreLogs() + te.restoreLogs = nil } + te.srv.Stop() } // newTest returns a new test using the provided testing.T and @@ -625,8 +628,8 @@ func testTimeoutOnDeadServer(t *testing.T, e env) { //if state, err := cc.State(); err != nil || (state != grpc.Connecting && state != grpc.TransientFailure) { // t.Fatalf("cc.State() = %s, %v, want %s or %s, ", state, err, grpc.Connecting, grpc.TransientFailure) //} - cc.Close() - awaitNewConnLogOutput() + //cc.Close() + //awaitNewConnLogOutput() } func healthCheck(d time.Duration, cc *grpc.ClientConn, serviceName string) (*healthpb.HealthCheckResponse, error) { @@ -1076,7 +1079,7 @@ func testRPCTimeout(t *testing.T, e env) { } } -func TestCancel(t *testing.T) { +func TestCancelX(t *testing.T) { defer leakCheck(t)() for _, e := range listTestEnv() { testCancel(t, e) @@ -1111,8 +1114,6 @@ func testCancel(t *testing.T, e env) { if grpc.Code(err) != codes.Canceled { t.Fatalf(`TestService/UnaryCall(_, _) = %v, %v; want , error code: %d`, reply, err, codes.Canceled) } - cc.Close() - awaitNewConnLogOutput() } diff --git a/test/out.txt b/test/out.txt deleted file mode 100644 index ad188a8d..00000000 --- a/test/out.txt +++ /dev/null @@ -1,1015 +0,0 @@ -=== RUN TestRetry -2016/05/05 17:56:45 DEBUG ugh here -2016/05/05 17:56:45 DEBUG reach inside resetTransport -2 -2016/05/05 17:56:45 DEBUG reach inside resetTransport -1 -2016/05/05 17:56:45 DEBUG reach inside resetTransport 0 -2016/05/05 17:56:45 DEBUG reach inside resetTransport 1 -2016/05/05 17:56:45 DEBUG reach inside resetTransport 2 -2016/05/05 17:56:45 DEBUG reach inside resetTransport 3: {localhost:46298 } -2016/05/05 17:56:45 DEBUG ugh here resetTransport -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:45 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG reach inside resetTransport -2 -2016/05/05 17:56:46 DEBUG reach inside resetTransport -1 -2016/05/05 17:56:46 DEBUG reach inside resetTransport 0 -2016/05/05 17:56:46 DEBUG reach inside resetTransport 1 -2016/05/05 17:56:46 DEBUG reach inside resetTransport 2 -2016/05/05 17:56:46 DEBUG reach inside resetTransport 3: {localhost:46298 } -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:46 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: -2016/05/05 17:56:47 DEBUG get transport: diff --git a/transport/http2_client.go b/transport/http2_client.go index 10004e4d..d5bcca03 100644 --- a/transport/http2_client.go +++ b/transport/http2_client.go @@ -405,12 +405,13 @@ func (t *http2Client) CloseStream(s *Stream, err error) { if t.streamsQuota != nil { updateStreams = true } - delete(t.activeStreams, s.id) - if t.state == draining && len(t.activeStreams) == 0 { + if t.state == draining && len(t.activeStreams) == 1 { + // The transport is draining and s is the last live stream on t. t.mu.Unlock() t.Close() return } + delete(t.activeStreams, s.id) t.mu.Unlock() if updateStreams { t.streamsQuota.add(1) From 73ff8375a671b4cbd956f5abb610d1eb304b8e33 Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Mon, 16 May 2016 15:47:46 -0700 Subject: [PATCH 12/40] Remove debugging logs and add license. --- balancer.go | 33 +++++++++++++++++++++++++++ clientconn.go | 5 ---- test/end2end_test.go | 54 +------------------------------------------- 3 files changed, 34 insertions(+), 58 deletions(-) diff --git a/balancer.go b/balancer.go index fa09318d..2d45eee5 100644 --- a/balancer.go +++ b/balancer.go @@ -1,3 +1,36 @@ +/* + * + * Copyright 2016, Google Inc. + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are + * met: + * + * * Redistributions of source code must retain the above copyright + * notice, this list of conditions and the following disclaimer. + * * Redistributions in binary form must reproduce the above + * copyright notice, this list of conditions and the following disclaimer + * in the documentation and/or other materials provided with the + * distribution. + * * Neither the name of Google Inc. nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + * + */ + package grpc import ( diff --git a/clientconn.go b/clientconn.go index 2db6b903..09136204 100644 --- a/clientconn.go +++ b/clientconn.go @@ -335,11 +335,6 @@ func (cc *ClientConn) watchAddrUpdates() error { if err := cc.newAddrConn(addr); err != nil { return err } - /* - cc.mu.Lock() - cc.conns[addr] = ac - cc.mu.Unlock() - */ case naming.Delete: cc.mu.Lock() addr := Address{ diff --git a/test/end2end_test.go b/test/end2end_test.go index ea87c7c3..ccf7cf43 100644 --- a/test/end2end_test.go +++ b/test/end2end_test.go @@ -598,21 +598,6 @@ func testTimeoutOnDeadServer(t *testing.T, e env) { cc := te.clientConn() tc := testpb.NewTestServiceClient(cc) - //ctx, _ := context.WithTimeout(context.Background(), time.Second) - //if _, err := cc.WaitForStateChange(ctx, grpc.Idle); err != nil { - // t.Fatalf("cc.WaitForStateChange(_, %s) = _, %v, want _, ", grpc.Idle, err) - //} - //ctx, _ = context.WithTimeout(context.Background(), time.Second) - //if _, err := cc.WaitForStateChange(ctx, grpc.Connecting); err != nil { - // t.Fatalf("cc.WaitForStateChange(_, %s) = _, %v, want _, ", grpc.Connecting, err) - //} - //if state, err := cc.State(); err != nil || state != grpc.Ready { - // t.Fatalf("cc.State() = %s, %v, want %s, ", state, err, grpc.Ready) - //} - //ctx, _ = context.WithTimeout(context.Background(), time.Second) - //if _, err := cc.WaitForStateChange(ctx, grpc.Ready); err != context.DeadlineExceeded { - // t.Fatalf("cc.WaitForStateChange(_, %s) = _, %v, want _, %v", grpc.Ready, err, context.DeadlineExceeded) - //} te.srv.Stop() // Set -1 as the timeout to make sure if transportMonitor gets error // notification in time the failure path of the 1st invoke of @@ -621,15 +606,7 @@ func testTimeoutOnDeadServer(t *testing.T, e env) { if _, err := tc.EmptyCall(ctx, &testpb.Empty{}); grpc.Code(err) != codes.DeadlineExceeded { t.Fatalf("TestService/EmptyCall(%v, _) = _, error %v, want _, error code: %d", ctx, err, codes.DeadlineExceeded) } - //ctx, _ = context.WithTimeout(context.Background(), time.Second) - //if _, err := cc.WaitForStateChange(ctx, grpc.Ready); err != nil { - // t.Fatalf("cc.WaitForStateChange(_, %s) = _, %v, want _, ", grpc.Ready, err) - //} - //if state, err := cc.State(); err != nil || (state != grpc.Connecting && state != grpc.TransientFailure) { - // t.Fatalf("cc.State() = %s, %v, want %s or %s, ", state, err, grpc.Connecting, grpc.TransientFailure) - //} - //cc.Close() - //awaitNewConnLogOutput() + awaitNewConnLogOutput() } func healthCheck(d time.Duration, cc *grpc.ClientConn, serviceName string) (*healthpb.HealthCheckResponse, error) { @@ -786,25 +763,6 @@ func testEmptyUnaryWithUserAgent(t *testing.T, e env) { defer te.tearDown() cc := te.clientConn() - - // Wait until cc is connected. - //ctx, _ := context.WithTimeout(context.Background(), time.Second) - //if _, err := cc.WaitForStateChange(ctx, grpc.Idle); err != nil { - // t.Fatalf("cc.WaitForStateChange(_, %s) = _, %v, want _, ", grpc.Idle, err) - //} - //ctx, _ = context.WithTimeout(context.Background(), time.Second) - //if _, err := cc.WaitForStateChange(ctx, grpc.Connecting); err != nil { - // t.Fatalf("cc.WaitForStateChange(_, %s) = _, %v, want _, ", grpc.Connecting, err) - //} - //if state, err := cc.State(); err != nil || state != grpc.Ready { - // t.Fatalf("cc.State() = %s, %v, want %s, ", state, err, grpc.Ready) - //} - /* - ctx, _ = context.WithTimeout(context.Background(), time.Second) - if _, err := cc.WaitForStateChange(ctx, grpc.Ready); err == nil { - t.Fatalf("cc.WaitForStateChange(_, %s) = _, , want _, %v", grpc.Ready, context.DeadlineExceeded) - } - */ tc := testpb.NewTestServiceClient(cc) var header metadata.MD reply, err := tc.EmptyCall(context.Background(), &testpb.Empty{}, grpc.Header(&header)) @@ -816,16 +774,6 @@ func testEmptyUnaryWithUserAgent(t *testing.T, e env) { } te.srv.Stop() - cc.Close() - /* - ctx, _ = context.WithTimeout(context.Background(), 5*time.Second) - if _, err := cc.WaitForStateChange(ctx, grpc.Ready); err != nil { - t.Fatalf("cc.WaitForStateChange(_, %s) = _, %v, want _, ", grpc.Ready, err) - } - if state, err := cc.State(); err != nil || state != grpc.Shutdown { - t.Fatalf("cc.State() = %s, %v, want %s, ", state, err, grpc.Shutdown) - } - */ } func TestFailedEmptyUnary(t *testing.T) { From 5710bedd7005b8d04da236d71cd6e0fa93b9886c Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Mon, 16 May 2016 16:36:40 -0700 Subject: [PATCH 13/40] polish Balancer comments --- balancer.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/balancer.go b/balancer.go index 2d45eee5..ecfaeb21 100644 --- a/balancer.go +++ b/balancer.go @@ -53,8 +53,9 @@ type Address struct { // Balancer chooses network addresses for RPCs. type Balancer interface { // Up informs the balancer that gRPC has a connection to the server at - // addr. It returns down which will be called once the connection gets - // lost. Once down is called, addr may no longer be returned by Get. + // addr. It returns down which is called once the connection to addr gets + // lost or closed. Once down is called, addr may no longer be returned + // by Get. Up(addr Address) (down func(error)) // Get gets the address of a server for the rpc corresponding to ctx. // It may block if there is no server available. It respects the @@ -91,6 +92,7 @@ func (rr *roundRobin) Up(addr Address) func(error) { } rr.addrs = append(rr.addrs, addr) if len(rr.addrs) == 1 { + // addr is only one available. Notify the Get() callers who are blocking. if rr.waitCh != nil { close(rr.waitCh) rr.waitCh = nil From c22f95cf180fb41522dccc780dc9e11c0417398b Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Mon, 16 May 2016 17:06:09 -0700 Subject: [PATCH 14/40] Polish the Balancer tests --- balancer_test.go | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/balancer_test.go b/balancer_test.go index f09ef1ac..d0379497 100644 --- a/balancer_test.go +++ b/balancer_test.go @@ -72,6 +72,7 @@ func (w *testWatcher) Next() (updates []*naming.Update, err error) { func (w *testWatcher) Close() { } +// Inject naming resolution updates to the testWatcher. func (w *testWatcher) inject(updates []*naming.Update) { w.side <- len(updates) for _, u := range updates { @@ -130,7 +131,7 @@ func TestNameDiscovery(t *testing.T) { if err := Invoke(context.Background(), "/foo/bar", &req, &reply, cc); err == nil || ErrorDesc(err) != servers[0].port { t.Fatalf("grpc.Invoke(_, _, _, _, _) = %v, want %s", err, servers[0].port) } - // Inject name resolution change to point to the second server now. + // Inject the name resolution change to remove servers[0] and add servers[1]. var updates []*naming.Update updates = append(updates, &naming.Update{ Op: naming.Delete, @@ -141,6 +142,7 @@ func TestNameDiscovery(t *testing.T) { Addr: "127.0.0.1:" + servers[1].port, }) r.w.inject(updates) + // Loop until the rpcs in flight talks to servers[1]. for { if err := Invoke(context.Background(), "/foo/bar", &req, &reply, cc); err != nil && ErrorDesc(err) == servers[1].port { break @@ -161,9 +163,9 @@ func TestEmptyAddrs(t *testing.T) { } var reply string if err := Invoke(context.Background(), "/foo/bar", &expectedRequest, &reply, cc); err != nil || reply != expectedResponse { - t.Fatalf("grpc.Invoke(_, _, _, _, _) = %v, want ", err) + t.Fatalf("grpc.Invoke(_, _, _, _, _) = %v, reply = %q, want %q, ", err, reply, expectedResponse) } - // Inject name resolution change to remove the server address so that there is no address + // Inject name resolution change to remove the server so that there is no address // available after that. var updates []*naming.Update updates = append(updates, &naming.Update{ @@ -191,7 +193,7 @@ func TestRoundRobin(t *testing.T) { if err != nil { t.Fatalf("Failed to create ClientConn: %v", err) } - // Add the other 2 servers as the name updates. + // Add servers[1] and servers[2] to the service discovery. var updates []*naming.Update updates = append(updates, &naming.Update{ Op: naming.Add, @@ -211,7 +213,7 @@ func TestRoundRobin(t *testing.T) { } time.Sleep(10 * time.Millisecond) } - // Check it works in round-robin manner. + // Check the incoming RPCs served in a round-robin manner. for i := 0; i < 10; i++ { if err := Invoke(context.Background(), "/foo/bar", &req, &reply, cc); err == nil || ErrorDesc(err) != servers[i%numServers].port { t.Fatalf("Invoke(_, _, _, _, _) = %v, want %s", err, servers[i%numServers].port) @@ -239,6 +241,7 @@ func TestCloseWithPendingRPC(t *testing.T) { Addr: "127.0.0.1:" + servers[0].port, }} r.w.inject(updates) + // Loop until the above update applies. for { ctx, _ := context.WithTimeout(context.Background(), 10*time.Millisecond) if err := Invoke(ctx, "/foo/bar", &expectedRequest, &reply, cc); Code(err) == codes.DeadlineExceeded { @@ -246,6 +249,7 @@ func TestCloseWithPendingRPC(t *testing.T) { } time.Sleep(10 * time.Millisecond) } + // Issue 2 RPCs which should be completed with error status once cc is closed. var wg sync.WaitGroup wg.Add(2) go func() { @@ -298,12 +302,13 @@ func TestGetOnWaitChannel(t *testing.T) { t.Errorf("grpc.Invoke(_, _, _, _, _) = %v, want ", err) } }() - // Add a connected server. + // Add a connected server to get the above RPC through. updates = []*naming.Update{&naming.Update{ Op: naming.Add, Addr: "127.0.0.1:" + servers[0].port, }} r.w.inject(updates) + // Wait until the above RPC succeeds. wg.Wait() cc.Close() servers[0].stop() From 9a13167f9ba49944959f671c7df8042df9cbc174 Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Mon, 16 May 2016 17:12:44 -0700 Subject: [PATCH 15/40] Tag Balancer as experimental API --- balancer.go | 1 + 1 file changed, 1 insertion(+) diff --git a/balancer.go b/balancer.go index ecfaeb21..1a9c7501 100644 --- a/balancer.go +++ b/balancer.go @@ -51,6 +51,7 @@ type Address struct { } // Balancer chooses network addresses for RPCs. +// This is the EXPERIMENTAL API and may be changed or extended in the future. type Balancer interface { // Up informs the balancer that gRPC has a connection to the server at // addr. It returns down which is called once the connection to addr gets From 65969fb499eb53f145847ad8775ae2d9c80cd0d8 Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Mon, 16 May 2016 19:19:35 -0700 Subject: [PATCH 16/40] lock optimization --- clientconn.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/clientconn.go b/clientconn.go index 09136204..0263d81e 100644 --- a/clientconn.go +++ b/clientconn.go @@ -336,18 +336,18 @@ func (cc *ClientConn) watchAddrUpdates() error { return err } case naming.Delete: - cc.mu.Lock() + cc.mu.RLock() addr := Address{ Addr: update.Addr, Metadata: update.Metadata, } ac, ok := cc.conns[addr] if !ok { - cc.mu.Unlock() + cc.mu.RUnlock() grpclog.Println("grpc: The name resolver wanted to delete a non-exist address: ", addr) continue } - cc.mu.Unlock() + cc.mu.RUnlock() ac.tearDown(ErrConnDrain) default: grpclog.Println("Unknown update.Op ", update.Op) From e9e748b9782666e0c9d94533ee2c3cbccb8b45d1 Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Tue, 17 May 2016 17:18:54 -0700 Subject: [PATCH 17/40] small fix and more comments --- clientconn.go | 52 ++++++++++++++++++++++++++++++--------------------- 1 file changed, 31 insertions(+), 21 deletions(-) diff --git a/clientconn.go b/clientconn.go index 0263d81e..dc3fc689 100644 --- a/clientconn.go +++ b/clientconn.go @@ -357,18 +357,18 @@ func (cc *ClientConn) watchAddrUpdates() error { } func (cc *ClientConn) newAddrConn(addr Address) error { - c := &addrConn{ + ac := &addrConn{ cc: cc, addr: addr, dopts: cc.dopts, shutdownChan: make(chan struct{}), } if EnableTracing { - c.events = trace.NewEventLog("grpc.ClientConn", c.addr.Addr) + 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 } @@ -377,29 +377,37 @@ func (cc *ClientConn) newAddrConn(addr Address) error { return ErrNoTransportSecurity } } else { - for _, cd := range c.dopts.copts.AuthOptions { + for _, cd := range ac.dopts.copts.AuthOptions { if cd.RequireTransportSecurity() { return ErrCredentialsMisuse } } } - c.stateCV = sync.NewCond(&c.mu) - if c.dopts.block { - if err := c.resetTransport(false); err != nil { - c.tearDown(err) + ac.cc.mu.Lock() + if ac.cc.conns == nil { + ac.cc.mu.Unlock() + return ErrClientConnClosing + } + ac.cc.conns[ac.addr] = ac + ac.cc.mu.Unlock() + + ac.stateCV = sync.NewCond(&ac.mu) + if ac.dopts.block { + 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.addr.Addr, err) - c.tearDown(err) + 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 nil @@ -429,7 +437,7 @@ func (cc *ClientConn) getTransport(ctx context.Context) (transport.ClientTranspo return t, put, nil } -// Close starts to tear down the ClientConn. +// Close tears down the ClientConn and all underlying connections. func (cc *ClientConn) Close() error { cc.mu.Lock() if cc.conns == nil { @@ -520,13 +528,15 @@ func (ac *addrConn) waitForStateChange(ctx context.Context, sourceState Connecti } func (ac *addrConn) resetTransport(closeTransport bool) error { - ac.cc.mu.Lock() - if ac.cc.conns == nil { + /* + ac.cc.mu.Lock() + if ac.cc.conns == nil { + ac.cc.mu.Unlock() + return ErrClientConnClosing + } + ac.cc.conns[ac.addr] = ac ac.cc.mu.Unlock() - return ErrClientConnClosing - } - ac.cc.conns[ac.addr] = ac - ac.cc.mu.Unlock() + */ var retries int start := time.Now() for { From 162d8d2d33952d148a8677af94a4786ec62b8f1d Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Wed, 18 May 2016 11:18:10 -0700 Subject: [PATCH 18/40] Tune Invoke behavior for the new Balancer. --- call.go | 51 ++++++++++++++++++++++++-------------------- clientconn.go | 28 +++++++++--------------- test/end2end_test.go | 13 +++++------ 3 files changed, 43 insertions(+), 49 deletions(-) diff --git a/call.go b/call.go index 67604f77..ac2d99d6 100644 --- a/call.go +++ b/call.go @@ -132,20 +132,14 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli Last: true, Delay: false, } - var ( - lastErr error // record the error that happened - put func() - ) + var put func() for { var ( err error t transport.ClientTransport stream *transport.Stream ) - // TODO(zhaoq): Need a formal spec of retry strategy for non-failfast rpcs. - if lastErr != nil && c.failFast { - return toRPCErr(lastErr) - } + // TODO(zhaoq): Need a formal spec of retry strategy for non-failFast rpcs. callHdr := &transport.CallHdr{ Host: cc.authority, Method: method, @@ -155,11 +149,19 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli } t, put, err = cc.getTransport(ctx) if err != nil { - if lastErr != nil { - // This was a retry; return the error from the last attempt. - return toRPCErr(lastErr) + if err == ErrClientConnClosing { + return toRPCErr(err) } - return toRPCErr(err) + if _, ok := err.(transport.StreamError); ok { + return toRPCErr(err) + } + if _, ok := err.(transport.ConnectionError); ok { + if c.failFast { + return toRPCErr(err) + } + } + // All the remaining cases are treated as retryable. + continue } if c.traceInfo.tr != nil { c.traceInfo.tr.LazyLog(&payload{sent: true, msg: args}, true) @@ -168,28 +170,31 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli if err != nil { put() if _, ok := err.(transport.ConnectionError); ok { - lastErr = err + if c.failFast { + return toRPCErr(err) + } continue } - if lastErr != nil { - return toRPCErr(lastErr) - } return toRPCErr(err) } // Receive the response - lastErr = recvResponse(cc.dopts, t, &c, stream, reply) - if _, ok := lastErr.(transport.ConnectionError); ok { + err = recvResponse(cc.dopts, t, &c, stream, reply) + if err != nil { put() - continue + if _, ok := err.(transport.ConnectionError); ok { + if c.failFast { + return toRPCErr(err) + } + continue + } + t.CloseStream(stream, err) + return toRPCErr(err) } if c.traceInfo.tr != nil { c.traceInfo.tr.LazyLog(&payload{sent: false, msg: reply}, true) } - t.CloseStream(stream, lastErr) + t.CloseStream(stream, nil) put() - if lastErr != nil { - return toRPCErr(lastErr) - } return Errorf(stream.StatusCode(), "%s", stream.StatusDesc()) } } diff --git a/clientconn.go b/clientconn.go index dc3fc689..6e6cb38b 100644 --- a/clientconn.go +++ b/clientconn.go @@ -61,14 +61,15 @@ var ( ErrCredentialsMisuse = errors.New("grpc: the credentials require transport level security (use grpc.WithTransportAuthenticator() to set)") // ErrClientConnClosing indicates that the operation is illegal because // the ClientConn is closing. - ErrClientConnClosing = errors.New("grpc: the client connection is closing") + ErrClientConnClosing = Errorf(codes.FailedPrecondition, "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") - // 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") + + // 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 = errors.New("grpc: the addrConn is closing") // minimum time to give a connection to complete minConnectTimeout = 20 * time.Second @@ -301,7 +302,7 @@ 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 watcher naming.Watcher @@ -348,7 +349,7 @@ func (cc *ClientConn) watchAddrUpdates() error { continue } cc.mu.RUnlock() - ac.tearDown(ErrConnDrain) + ac.tearDown(errConnDrain) default: grpclog.Println("Unknown update.Op ", update.Op) } @@ -528,15 +529,6 @@ func (ac *addrConn) waitForStateChange(ctx context.Context, sourceState Connecti } func (ac *addrConn) resetTransport(closeTransport bool) error { - /* - ac.cc.mu.Lock() - if ac.cc.conns == nil { - ac.cc.mu.Unlock() - return ErrClientConnClosing - } - ac.cc.conns[ac.addr] = ac - ac.cc.mu.Unlock() - */ var retries int start := time.Now() for { @@ -548,7 +540,7 @@ func (ac *addrConn) resetTransport(closeTransport bool) error { return errConnClosing } if ac.down != nil { - ac.down(ErrNetworkIO) + ac.down(errNetworkIO) ac.down = nil } ac.state = Connecting @@ -732,7 +724,7 @@ func (ac *addrConn) tearDown(err error) { ac.ready = nil } if ac.transport != nil { - if err == ErrConnDrain { + if err == errConnDrain { ac.transport.GracefulClose() } else { ac.transport.Close() diff --git a/test/end2end_test.go b/test/end2end_test.go index 78e9eac3..70f85325 100644 --- a/test/end2end_test.go +++ b/test/end2end_test.go @@ -159,7 +159,6 @@ func (s *testServer) UnaryCall(ctx context.Context, in *testpb.SimpleRequest) (* return nil, fmt.Errorf("Unknown server name %q", serverName) } } - // Simulate some service delay. time.Sleep(time.Second) @@ -1020,14 +1019,13 @@ func testRPCTimeout(t *testing.T, e env) { } for i := -1; i <= 10; i++ { ctx, _ := context.WithTimeout(context.Background(), time.Duration(i)*time.Millisecond) - reply, err := tc.UnaryCall(ctx, req) - if grpc.Code(err) != codes.DeadlineExceeded { - t.Fatalf(`TestService/UnaryCallv(_, _) = %v, %v; want , error code: %d`, reply, err, codes.DeadlineExceeded) + if _, err := tc.UnaryCall(ctx, req); grpc.Code(err) != codes.DeadlineExceeded { + t.Fatalf("TestService/UnaryCallv(_, _) = _, %v; want , error code: %d", err, codes.DeadlineExceeded) } } } -func TestCancelX(t *testing.T) { +func TestCancel(t *testing.T) { defer leakCheck(t)() for _, e := range listTestEnv() { testCancel(t, e) @@ -1058,9 +1056,8 @@ func testCancel(t *testing.T, e env) { } ctx, cancel := context.WithCancel(context.Background()) time.AfterFunc(1*time.Millisecond, cancel) - reply, err := tc.UnaryCall(ctx, req) - if grpc.Code(err) != codes.Canceled { - t.Fatalf(`TestService/UnaryCall(_, _) = %v, %v; want , error code: %d`, reply, err, codes.Canceled) + if r, err := tc.UnaryCall(ctx, req); grpc.Code(err) != codes.Canceled { + t.Fatalf("TestService/UnaryCall(_, _) = %v, %v; want _, error code: %d", r, err, codes.Canceled) } awaitNewConnLogOutput() } From 5d62215b4148b9ff45e1f5f42d9e37f11ccc7998 Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Wed, 18 May 2016 16:26:12 -0700 Subject: [PATCH 19/40] some minor fixes --- call.go | 3 ++- clientconn.go | 3 ++- test/end2end_test.go | 8 ++++++-- transport/transport.go | 2 ++ 4 files changed, 12 insertions(+), 4 deletions(-) diff --git a/call.go b/call.go index ac2d99d6..5fe5af26 100644 --- a/call.go +++ b/call.go @@ -132,12 +132,12 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli Last: true, Delay: false, } - var put func() for { var ( err error t transport.ClientTransport stream *transport.Stream + put func() ) // TODO(zhaoq): Need a formal spec of retry strategy for non-failFast rpcs. callHdr := &transport.CallHdr{ @@ -149,6 +149,7 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli } t, put, err = cc.getTransport(ctx) if err != nil { + // TODO(zhaoq): Probably revisit the error handling. if err == ErrClientConnClosing { return toRPCErr(err) } diff --git a/clientconn.go b/clientconn.go index 6e6cb38b..81c2c731 100644 --- a/clientconn.go +++ b/clientconn.go @@ -384,6 +384,7 @@ func (cc *ClientConn) newAddrConn(addr Address) error { } } } + // 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() @@ -620,11 +621,11 @@ func (ac *addrConn) resetTransport(closeTransport bool) error { ac.state = Ready ac.stateCV.Broadcast() ac.transport = newTransport - ac.down = ac.cc.balancer.Up(ac.addr) if ac.ready != nil { close(ac.ready) ac.ready = nil } + ac.down = ac.cc.balancer.Up(ac.addr) ac.mu.Unlock() return nil } diff --git a/test/end2end_test.go b/test/end2end_test.go index 70f85325..ecb1b527 100644 --- a/test/end2end_test.go +++ b/test/end2end_test.go @@ -340,7 +340,7 @@ func TestReconnectTimeout(t *testing.T) { // Block until reconnect times out. <-waitC if err := conn.Close(); err != nil { - t.Fatalf("%v.Close() = %v, want %v", conn, err, grpc.ErrClientConnClosing) + t.Fatalf("%v.Close() = %v, want ", conn, err) } } @@ -586,6 +586,7 @@ func TestTimeoutOnDeadServer(t *testing.T) { func testTimeoutOnDeadServer(t *testing.T, e env) { te := newTest(t, e) + te.userAgent = testAppUA te.declareLogNoise( "transport: http2Client.notifyError got notified that the client transport was broken EOF", "grpc: Conn.transportMonitor exits due to: grpc: the client connection is closing", @@ -597,13 +598,16 @@ func testTimeoutOnDeadServer(t *testing.T, e env) { cc := te.clientConn() tc := testpb.NewTestServiceClient(cc) + if _, err := tc.EmptyCall(context.Background(), &testpb.Empty{}); err != nil { + t.Fatalf("TestService/EmptyCall(_, _) = _, %v, want _, ", err) + } te.srv.Stop() // Set -1 as the timeout to make sure if transportMonitor gets error // notification in time the failure path of the 1st invoke of // ClientConn.wait hits the deadline exceeded error. ctx, _ := context.WithTimeout(context.Background(), -1) if _, err := tc.EmptyCall(ctx, &testpb.Empty{}); grpc.Code(err) != codes.DeadlineExceeded { - t.Fatalf("TestService/EmptyCall(%v, _) = _, error %v, want _, error code: %d", ctx, err, codes.DeadlineExceeded) + t.Fatalf("TestService/EmptyCall(%v, _) = _, %v, want _, error code: %d", ctx, err, codes.DeadlineExceeded) } awaitNewConnLogOutput() } diff --git a/transport/transport.go b/transport/transport.go index 230e215d..1c9af545 100644 --- a/transport/transport.go +++ b/transport/transport.go @@ -392,6 +392,8 @@ type ClientTransport interface { // is called only once. Close() error + // GracefulClose starts to tear down the transport. It stops accepting + // new RPCs and wait the completion of the pending RPCs. GracefulClose() error // Write sends the data for the given stream. A nil stream indicates From 2151d51eaafca9eb0b5940f59fa13f1adb7ce588 Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Wed, 18 May 2016 17:19:50 -0700 Subject: [PATCH 20/40] small error touchup --- transport/http2_client.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/transport/http2_client.go b/transport/http2_client.go index d5bcca03..73222012 100644 --- a/transport/http2_client.go +++ b/transport/http2_client.go @@ -452,7 +452,7 @@ func (t *http2Client) Close() (err error) { } if t.state == closing { t.mu.Unlock() - return errors.New("transport: Close() was already called") + return } t.state = closing t.mu.Unlock() @@ -479,7 +479,7 @@ func (t *http2Client) GracefulClose() error { t.mu.Lock() if t.state == closing { t.mu.Unlock() - return errors.New("transport: Graceful close on a closed transport") + return nil } if t.state == draining { t.mu.Unlock() From fda7cb3cdfc28b47e03218a1baa55ed1689505cf Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Thu, 19 May 2016 10:59:35 -0700 Subject: [PATCH 21/40] Remove unused import --- transport/http2_client.go | 1 - 1 file changed, 1 deletion(-) diff --git a/transport/http2_client.go b/transport/http2_client.go index 73222012..d6fde4f1 100644 --- a/transport/http2_client.go +++ b/transport/http2_client.go @@ -35,7 +35,6 @@ package transport import ( "bytes" - "errors" "io" "math" "net" From 5b484e4099d2b12a20a1b673e31de6597fa11992 Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Mon, 23 May 2016 19:25:01 -0700 Subject: [PATCH 22/40] add Notify API; move the name resolving into Balancer --- balancer.go | 138 +++++++++++++++++++++++++++++++++--------- balancer_test.go | 47 ++++++++------- call.go | 14 ++++- clientconn.go | 152 ++++++++++++++++++++++------------------------- 4 files changed, 220 insertions(+), 131 deletions(-) diff --git a/balancer.go b/balancer.go index 1a9c7501..279474b9 100644 --- a/balancer.go +++ b/balancer.go @@ -37,6 +37,8 @@ import ( "sync" "golang.org/x/net/context" + "google.golang.org/grpc/grpclog" + "google.golang.org/grpc/naming" "google.golang.org/grpc/transport" ) @@ -53,6 +55,10 @@ type Address struct { // Balancer chooses network addresses for RPCs. // This is the EXPERIMENTAL API and may be changed or extended in the future. type Balancer interface { + // Start does the initialization work to bootstrap a Balancer. For example, + // this function may start the service discovery and watch the name resolution + // updates. + Start(target string) error // Up informs the balancer that gRPC has a connection to the server at // addr. It returns down which is called once the connection to addr gets // lost or closed. Once down is called, addr may no longer be returned @@ -64,21 +70,101 @@ type Balancer interface { // is called once the rpc has completed or failed. put can collect and // report rpc stats to remote load balancer. Get(ctx context.Context) (addr Address, put func(), err error) + // Notify gRPC internals the list of Address which should be connected. gRPC + // internals will compare it with the exisiting connected addresses. If the + // address Balancer notified is not in the list of the connected addresses, + // gRPC starts to connect the address. If an address in the connected + // addresses is not in the notification list, the corresponding connect will be + // shutdown gracefully. Otherwise, there are no operations. Note that this + // function must return the full list of the Addrresses which should be connected. + // It is NOT delta. + Notify() <-chan []Address // Close shuts down the balancer. Close() error } -// RoundRobin returns a Balancer that selects addresses round-robin. -func RoundRobin() Balancer { - return &roundRobin{} +// RoundRobin returns a Balancer that selects addresses round-robin. It starts to watch +// the name resolution updates. +func RoundRobin(r naming.Resolver) Balancer { + return &roundRobin{r: r} } type roundRobin struct { - mu sync.Mutex - addrs []Address - next int // index of the next address to return for Get() - waitCh chan struct{} // channel to block when there is no address available - done bool // The Balancer is closed. + r naming.Resolver + open []Address // all the known addresses the client can potentially connect + mu sync.Mutex + addrCh chan []Address // the channel to notify gRPC internals the list of addresses the client should connect to. + connected []Address // all the connected addresses + next int // index of the next address to return for Get() + waitCh chan struct{} // the channel to block when there is no connected address available + done bool // The Balancer is closed. +} + +func (rr *roundRobin) watchAddrUpdates(w naming.Watcher) error { + updates, err := w.Next() + if err != nil { + return err + } + for _, update := range updates { + addr := Address{ + Addr: update.Addr, + Metadata: update.Metadata, + } + switch update.Op { + case naming.Add: + var exisit bool + for _, v := range rr.open { + if addr == v { + exisit = true + grpclog.Println("grpc: The name resolver wanted to add an existing address: ", addr) + break + } + } + if exisit { + continue + } + rr.open = append(rr.open, addr) + case naming.Delete: + for i, v := range rr.open { + if v == addr { + copy(rr.open[i:], rr.open[i+1:]) + rr.open = rr.open[:len(rr.open)-1] + break + } + } + default: + grpclog.Println("Unknown update.Op ", update.Op) + } + } + // Make a copy of rr.open and write it onto rr.addrCh so that gRPC internals gets notified. + open := make([]Address, len(rr.open)) + rr.mu.Lock() + defer rr.mu.Unlock() + copy(open, rr.open) + if rr.done { + return ErrClientConnClosing + } + rr.addrCh <- open + return nil +} + +func (rr *roundRobin) Start(target string) error { + if rr.r == nil { + return nil + } + w, err := rr.r.Resolve(target) + if err != nil { + return err + } + rr.addrCh = make(chan []Address) + go func() { + for { + if err := rr.watchAddrUpdates(w); err != nil { + return + } + } + }() + return nil } // Up appends addr to the end of rr.addrs and sends notification if there @@ -86,13 +172,13 @@ type roundRobin struct { func (rr *roundRobin) Up(addr Address) func(error) { rr.mu.Lock() defer rr.mu.Unlock() - for _, a := range rr.addrs { + for _, a := range rr.connected { if a == addr { return nil } } - rr.addrs = append(rr.addrs, addr) - if len(rr.addrs) == 1 { + rr.connected = append(rr.connected, addr) + if len(rr.connected) == 1 { // addr is only one available. Notify the Get() callers who are blocking. if rr.waitCh != nil { close(rr.waitCh) @@ -108,10 +194,10 @@ func (rr *roundRobin) Up(addr Address) func(error) { func (rr *roundRobin) down(addr Address, err error) { rr.mu.Lock() defer rr.mu.Unlock() - for i, a := range rr.addrs { + for i, a := range rr.connected { if a == addr { - copy(rr.addrs[i:], rr.addrs[i+1:]) - rr.addrs = rr.addrs[:len(rr.addrs)-1] + copy(rr.connected[i:], rr.connected[i+1:]) + rr.connected = rr.connected[:len(rr.connected)-1] return } } @@ -126,16 +212,13 @@ func (rr *roundRobin) Get(ctx context.Context) (addr Address, put func(), err er err = ErrClientConnClosing return } - if rr.next >= len(rr.addrs) { + if rr.next >= len(rr.connected) { rr.next = 0 } - if len(rr.addrs) > 0 { - addr = rr.addrs[rr.next] + if len(rr.connected) > 0 { + addr = rr.connected[rr.next] rr.next++ rr.mu.Unlock() - put = func() { - rr.put(ctx, addr) - } return } // There is no address available. Wait on rr.waitCh. @@ -158,26 +241,24 @@ func (rr *roundRobin) Get(ctx context.Context) (addr Address, put func(), err er err = ErrClientConnClosing return } - if len(rr.addrs) == 0 { + if len(rr.connected) == 0 { // The newly added addr got removed by Down() again. rr.mu.Unlock() continue } - if rr.next >= len(rr.addrs) { + if rr.next >= len(rr.connected) { rr.next = 0 } - addr = rr.addrs[rr.next] + addr = rr.connected[rr.next] rr.next++ rr.mu.Unlock() - put = func() { - rr.put(ctx, addr) - } return } } } -func (rr *roundRobin) put(ctx context.Context, addr Address) { +func (rr *roundRobin) Notify() <-chan []Address { + return rr.addrCh } func (rr *roundRobin) Close() error { @@ -188,5 +269,8 @@ func (rr *roundRobin) Close() error { close(rr.waitCh) rr.waitCh = nil } + if rr.addrCh != nil { + close(rr.addrCh) + } return nil } diff --git a/balancer_test.go b/balancer_test.go index d0379497..976432f4 100644 --- a/balancer_test.go +++ b/balancer_test.go @@ -122,7 +122,7 @@ func TestNameDiscovery(t *testing.T) { // Start 2 servers on 2 ports. numServers := 2 servers, r := startServers(t, numServers, math.MaxUint32) - cc, err := Dial("foo.bar.com", WithNameResolver(r), WithBlock(), WithInsecure(), WithCodec(testCodec{})) + cc, err := Dial("foo.bar.com", WithBalancer(RoundRobin(r)), WithBlock(), WithInsecure(), WithCodec(testCodec{})) if err != nil { t.Fatalf("Failed to create ClientConn: %v", err) } @@ -157,7 +157,7 @@ func TestNameDiscovery(t *testing.T) { func TestEmptyAddrs(t *testing.T) { servers, r := startServers(t, 1, math.MaxUint32) - cc, err := Dial("foo.bar.com", WithNameResolver(r), WithBlock(), WithInsecure(), WithCodec(testCodec{})) + cc, err := Dial("foo.bar.com", WithBalancer(RoundRobin(r)), WithBlock(), WithInsecure(), WithCodec(testCodec{})) if err != nil { t.Fatalf("Failed to create ClientConn: %v", err) } @@ -167,12 +167,11 @@ func TestEmptyAddrs(t *testing.T) { } // Inject name resolution change to remove the server so that there is no address // available after that. - var updates []*naming.Update - updates = append(updates, &naming.Update{ + u := &naming.Update{ Op: naming.Delete, Addr: "127.0.0.1:" + servers[0].port, - }) - r.w.inject(updates) + } + r.w.inject([]*naming.Update{u}) // Loop until the above updates apply. for { time.Sleep(10 * time.Millisecond) @@ -189,24 +188,32 @@ func TestRoundRobin(t *testing.T) { // Start 3 servers on 3 ports. numServers := 3 servers, r := startServers(t, numServers, math.MaxUint32) - cc, err := Dial("foo.bar.com", WithNameResolver(r), WithBlock(), WithInsecure(), WithCodec(testCodec{})) + cc, err := Dial("foo.bar.com", WithBalancer(RoundRobin(r)), WithBlock(), WithInsecure(), WithCodec(testCodec{})) if err != nil { t.Fatalf("Failed to create ClientConn: %v", err) } - // Add servers[1] and servers[2] to the service discovery. - var updates []*naming.Update - updates = append(updates, &naming.Update{ + // Add servers[1] to the service discovery. + u := &naming.Update{ Op: naming.Add, Addr: "127.0.0.1:" + servers[1].port, - }) - updates = append(updates, &naming.Update{ - Op: naming.Add, - Addr: "127.0.0.1:" + servers[2].port, - }) - r.w.inject(updates) + } + r.w.inject([]*naming.Update{u}) req := "port" var reply string - // Loop until an RPC is completed by servers[2]. + // Loop until servers[1] is up + for { + if err := Invoke(context.Background(), "/foo/bar", &req, &reply, cc); err != nil && ErrorDesc(err) == servers[1].port { + break + } + time.Sleep(10 * time.Millisecond) + } + // Add server2[2] to the service discovery. + u = &naming.Update{ + Op: naming.Add, + Addr: "127.0.0.1:" + servers[2].port, + } + r.w.inject([]*naming.Update{u}) + // Loop until both servers[2] are up. for { if err := Invoke(context.Background(), "/foo/bar", &req, &reply, cc); err != nil && ErrorDesc(err) == servers[2].port { break @@ -216,7 +223,7 @@ func TestRoundRobin(t *testing.T) { // Check the incoming RPCs served in a round-robin manner. for i := 0; i < 10; i++ { if err := Invoke(context.Background(), "/foo/bar", &req, &reply, cc); err == nil || ErrorDesc(err) != servers[i%numServers].port { - t.Fatalf("Invoke(_, _, _, _, _) = %v, want %s", err, servers[i%numServers].port) + t.Fatalf("Index %d: Invoke(_, _, _, _, _) = %v, want %s", i, err, servers[i%numServers].port) } } cc.Close() @@ -227,7 +234,7 @@ func TestRoundRobin(t *testing.T) { func TestCloseWithPendingRPC(t *testing.T) { servers, r := startServers(t, 1, math.MaxUint32) - cc, err := Dial("foo.bar.com", WithNameResolver(r), WithBlock(), WithInsecure(), WithCodec(testCodec{})) + cc, err := Dial("foo.bar.com", WithBalancer(RoundRobin(r)), WithBlock(), WithInsecure(), WithCodec(testCodec{})) if err != nil { t.Fatalf("Failed to create ClientConn: %v", err) } @@ -275,7 +282,7 @@ func TestCloseWithPendingRPC(t *testing.T) { func TestGetOnWaitChannel(t *testing.T) { servers, r := startServers(t, 1, math.MaxUint32) - cc, err := Dial("foo.bar.com", WithNameResolver(r), WithBlock(), WithInsecure(), WithCodec(testCodec{})) + cc, err := Dial("foo.bar.com", WithBalancer(RoundRobin(r)), WithBlock(), WithInsecure(), WithCodec(testCodec{})) if err != nil { t.Fatalf("Failed to create ClientConn: %v", err) } diff --git a/call.go b/call.go index 5fe5af26..98b8e2b1 100644 --- a/call.go +++ b/call.go @@ -169,7 +169,10 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli } stream, err = sendRequest(ctx, cc.dopts.codec, cc.dopts.cp, callHdr, t, args, topts) if err != nil { - put() + if put != nil { + put() + put = nil + } if _, ok := err.(transport.ConnectionError); ok { if c.failFast { return toRPCErr(err) @@ -181,7 +184,9 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli // Receive the response err = recvResponse(cc.dopts, t, &c, stream, reply) if err != nil { - put() + if put != nil { + put() + } if _, ok := err.(transport.ConnectionError); ok { if c.failFast { return toRPCErr(err) @@ -195,7 +200,10 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli c.traceInfo.tr.LazyLog(&payload{sent: false, msg: reply}, true) } t.CloseStream(stream, nil) - put() + if put != nil { + put() + put = nil + } return Errorf(stream.StatusCode(), "%s", stream.StatusDesc()) } } diff --git a/clientconn.go b/clientconn.go index 81c2c731..141cbbf0 100644 --- a/clientconn.go +++ b/clientconn.go @@ -65,12 +65,12 @@ var ( // 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") - - // 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 = errors.New("grpc: the addrConn is closing") + // 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 + ErrConnClosing = errors.New("grpc: the addrConn is closing") // minimum time to give a connection to complete minConnectTimeout = 20 * time.Second ) @@ -82,7 +82,6 @@ type dialOptions struct { cp Compressor dc Decompressor bs backoffStrategy - resolver naming.Resolver balancer Balancer block bool insecure bool @@ -115,13 +114,6 @@ func WithDecompressor(dc Decompressor) DialOption { } } -// WithNameResolver returns a DialOption which sets a name resolver for service discovery. -func WithNameResolver(r naming.Resolver) DialOption { - return func(o *dialOptions) { - o.resolver = r - } -} - // WithBalancer returns a DialOption which sets a load balancer. func WithBalancer(b Balancer) DialOption { return func(o *dialOptions) { @@ -231,34 +223,29 @@ func Dial(target string, opts ...DialOption) (*ClientConn, error) { cc.balancer = cc.dopts.balancer if cc.balancer == nil { - cc.balancer = RoundRobin() + cc.balancer = RoundRobin(nil) } - - if cc.dopts.resolver == nil { - addr := Address{ - Addr: cc.target, - } - if err := cc.newAddrConn(addr); err != nil { + if err := cc.balancer.Start(target); err != nil { + return nil, err + } + ch := cc.balancer.Notify() + if ch == nil { + // There is no name resolver installed. + addr := Address{Addr: target} + if err := cc.newAddrConn(addr, false); err != nil { return nil, err } } else { - w, err := cc.dopts.resolver.Resolve(cc.target) - if err != nil { - return nil, err + addrs, ok := <-ch + if !ok || len(addrs) == 0 { + return nil, fmt.Errorf("grpc: there is no address available to dial") } - cc.watcher = w - // Get the initial name resolution and dial the first connection. - if err := cc.watchAddrUpdates(); err != nil { - return nil, err - } - // Start a goroutine to watch for the future name resolution changes. - go func() { - for { - if err := cc.watchAddrUpdates(); err != nil { - return - } + for _, a := range addrs { + if err := cc.newAddrConn(a, false); err != nil { + return nil, err } - }() + } + go cc.controller() } colonPos := strings.LastIndex(target, ":") @@ -314,50 +301,48 @@ type ClientConn struct { conns map[Address]*addrConn } -func (cc *ClientConn) watchAddrUpdates() error { - updates, err := cc.watcher.Next() - if err != nil { - return err - } - for _, update := range updates { - switch update.Op { - case naming.Add: - cc.mu.RLock() - addr := Address{ - Addr: update.Addr, - Metadata: update.Metadata, +func (cc *ClientConn) controller() { + for { + addrs, ok := <-cc.balancer.Notify() + if !ok { + // cc has been closed. + return + } + 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) } - if _, ok := cc.conns[addr]; ok { - cc.mu.RUnlock() - grpclog.Println("grpc: The name resolver wanted to add an existing address: ", addr) - continue + } + for k, c := range cc.conns { + var keep bool + for _, a := range addrs { + if k == a { + keep = true + break + } } - cc.mu.RUnlock() - if err := cc.newAddrConn(addr); err != nil { - return err + if !keep { + del = append(del, c) } - case naming.Delete: - cc.mu.RLock() - addr := Address{ - Addr: update.Addr, - Metadata: update.Metadata, + } + cc.mu.Unlock() + for _, a := range addrs { + if err := cc.newAddrConn(a, true); err != nil { + } - ac, ok := cc.conns[addr] - if !ok { - cc.mu.RUnlock() - grpclog.Println("grpc: The name resolver wanted to delete a non-exist address: ", addr) - continue - } - cc.mu.RUnlock() - ac.tearDown(errConnDrain) - default: - grpclog.Println("Unknown update.Op ", update.Op) + } + for _, c := range del { + c.tearDown(ErrConnDrain) } } - return nil } -func (cc *ClientConn) newAddrConn(addr Address) error { +func (cc *ClientConn) newAddrConn(addr Address, skipWait bool) error { ac := &addrConn{ cc: cc, addr: addr, @@ -394,7 +379,8 @@ func (cc *ClientConn) newAddrConn(addr Address) error { ac.cc.mu.Unlock() ac.stateCV = sync.NewCond(&ac.mu) - if ac.dopts.block { + // 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 @@ -428,12 +414,16 @@ func (cc *ClientConn) getTransport(ctx context.Context) (transport.ClientTranspo ac, ok := cc.conns[addr] cc.mu.RUnlock() if !ok { - put() + 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 { - put() + if put != nil { + put() + } return nil, nil, err } return t, put, nil @@ -538,10 +528,10 @@ func (ac *addrConn) resetTransport(closeTransport bool) error { if ac.state == Shutdown { // ac.tearDown(...) has been invoked. ac.mu.Unlock() - return errConnClosing + return ErrConnClosing } if ac.down != nil { - ac.down(errNetworkIO) + ac.down(ErrNetworkIO) ac.down = nil } ac.state = Connecting @@ -579,7 +569,7 @@ func (ac *addrConn) resetTransport(closeTransport bool) error { if ac.state == Shutdown { // ac.tearDown(...) has been invoked. ac.mu.Unlock() - return errConnClosing + return ErrConnClosing } ac.errorf("transient failure: %v", err) ac.state = TransientFailure @@ -616,7 +606,7 @@ func (ac *addrConn) resetTransport(closeTransport bool) error { // ac.tearDown(...) has been invoked. ac.mu.Unlock() newTransport.Close() - return errConnClosing + return ErrConnClosing } ac.state = Ready ac.stateCV.Broadcast() @@ -671,7 +661,7 @@ func (ac *addrConn) wait(ctx context.Context) (transport.ClientTransport, error) switch { case ac.state == Shutdown: ac.mu.Unlock() - return nil, errConnClosing + return nil, ErrConnClosing case ac.state == Ready: ct := ac.transport ac.mu.Unlock() @@ -725,7 +715,7 @@ func (ac *addrConn) tearDown(err error) { ac.ready = nil } if ac.transport != nil { - if err == errConnDrain { + if err == ErrConnDrain { ac.transport.GracefulClose() } else { ac.transport.Close() From 0b1df3bca2197b6c636cb19c9b6c4f310be95a03 Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Tue, 24 May 2016 17:19:44 -0700 Subject: [PATCH 23/40] add BalancerGetOption --- balancer.go | 36 +++++++++++++++++++++++------------- call.go | 5 ++++- clientconn.go | 5 +++-- stream.go | 7 +++++-- 4 files changed, 35 insertions(+), 18 deletions(-) diff --git a/balancer.go b/balancer.go index 279474b9..1aa170dd 100644 --- a/balancer.go +++ b/balancer.go @@ -52,6 +52,13 @@ type Address struct { Metadata interface{} } +// BalancerGetOption can configure a Get call. +type BalancerGetOptions struct { + // BlockingWait specifies whether Get should block when there is no + // address ready for transmission. + BlockingWait bool +} + // Balancer chooses network addresses for RPCs. // This is the EXPERIMENTAL API and may be changed or extended in the future. type Balancer interface { @@ -65,19 +72,21 @@ type Balancer interface { // by Get. Up(addr Address) (down func(error)) // Get gets the address of a server for the rpc corresponding to ctx. - // It may block if there is no server available. It respects the - // timeout or cancellation of ctx when blocking. It returns put which - // is called once the rpc has completed or failed. put can collect and + // If opts.BlockingWait is true, it blocks if there is no connection available, + // i.e., invocations of Up(...) is equal to that of Down(...). It respects the + // timeout or cancellation of ctx when blocking. If opts.BlockingWait is + // false, it may return any address it has notified via Notify(...). It returns + // put which is called once the rpc has completed or failed. put can collect and // report rpc stats to remote load balancer. - Get(ctx context.Context) (addr Address, put func(), err error) - // Notify gRPC internals the list of Address which should be connected. gRPC - // internals will compare it with the exisiting connected addresses. If the - // address Balancer notified is not in the list of the connected addresses, - // gRPC starts to connect the address. If an address in the connected - // addresses is not in the notification list, the corresponding connect will be - // shutdown gracefully. Otherwise, there are no operations. Note that this - // function must return the full list of the Addrresses which should be connected. - // It is NOT delta. + Get(ctx context.Context, opts BalancerGetOptions) (addr Address, put func(), err error) + // Notify gRPC internals the list of Address which should be connected. The list + // may be from a name resolver or remote load balancer. gRPC internals will + // compare it with the exisiting connected addresses. If the address Balancer + // notified is not in the list of the connected addresses, gRPC starts to connect + // the address. If an address in the connected addresses is not in the notification + // list, the corresponding connect will be shutdown gracefully. Otherwise, there + // are no operations. Note that this function must return the full list of the + // Addrresses which should be connected. It is NOT delta. Notify() <-chan []Address // Close shuts down the balancer. Close() error @@ -204,7 +213,7 @@ func (rr *roundRobin) down(addr Address, err error) { } // Get returns the next addr in the rotation. It blocks if there is no address available. -func (rr *roundRobin) Get(ctx context.Context) (addr Address, put func(), err error) { +func (rr *roundRobin) Get(ctx context.Context, opts BalancerGetOptions) (addr Address, put func(), err error) { var ch chan struct{} rr.mu.Lock() if rr.done { @@ -222,6 +231,7 @@ func (rr *roundRobin) Get(ctx context.Context) (addr Address, put func(), err er return } // There is no address available. Wait on rr.waitCh. + // TODO(zhaoq): Handle the case when opts.BlockingWait is false. if rr.waitCh == nil { ch = make(chan struct{}) rr.waitCh = ch diff --git a/call.go b/call.go index 98b8e2b1..2149ce29 100644 --- a/call.go +++ b/call.go @@ -147,7 +147,10 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli if cc.dopts.cp != nil { callHdr.SendCompress = cc.dopts.cp.Type() } - t, put, err = cc.getTransport(ctx) + gopts := BalancerGetOptions{ + BlockingWait: !c.failFast, + } + t, put, err = cc.getTransport(ctx, gopts) if err != nil { // TODO(zhaoq): Probably revisit the error handling. if err == ErrClientConnClosing { diff --git a/clientconn.go b/clientconn.go index 141cbbf0..81e75d01 100644 --- a/clientconn.go +++ b/clientconn.go @@ -401,8 +401,9 @@ func (cc *ClientConn) newAddrConn(addr Address, skipWait bool) error { return nil } -func (cc *ClientConn) getTransport(ctx context.Context) (transport.ClientTransport, func(), error) { - addr, put, err := cc.balancer.Get(ctx) +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 } diff --git a/stream.go b/stream.go index 7f0e7c3e..de125d5b 100644 --- a/stream.go +++ b/stream.go @@ -105,11 +105,14 @@ func NewClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth err error put func() ) - t, put, err = cc.getTransport(ctx) + // TODO(zhaoq): CallOption is omitted. Add support when it is needed. + gopts := BalancerGetOptions{ + BlockingWait: false, + } + t, put, err = cc.getTransport(ctx, gopts) if err != nil { return nil, toRPCErr(err) } - // TODO(zhaoq): CallOption is omitted. Add support when it is needed. callHdr := &transport.CallHdr{ Host: cc.authority, Method: method, From 53fe56a73e6809bfc4821d022fb6c2f365d72319 Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Tue, 24 May 2016 17:37:44 -0700 Subject: [PATCH 24/40] Polish the comments --- balancer.go | 35 ++++++++++++++++++----------------- 1 file changed, 18 insertions(+), 17 deletions(-) diff --git a/balancer.go b/balancer.go index 1aa170dd..8c25a9dc 100644 --- a/balancer.go +++ b/balancer.go @@ -43,6 +43,7 @@ import ( ) // Address represents a server the client connects to. +// This is the EXPERIMENTAL API and may be changed or extended in the future. type Address struct { // Addr is the server address on which a connection will be established. Addr string @@ -52,10 +53,11 @@ type Address struct { Metadata interface{} } -// BalancerGetOption can configure a Get call. +// BalancerGetOption configures a Get call. +// This is the EXPERIMENTAL API and may be changed or extended in the future. type BalancerGetOptions struct { // BlockingWait specifies whether Get should block when there is no - // address ready for transmission. + // connected address. BlockingWait bool } @@ -63,30 +65,29 @@ type BalancerGetOptions struct { // This is the EXPERIMENTAL API and may be changed or extended in the future. type Balancer interface { // Start does the initialization work to bootstrap a Balancer. For example, - // this function may start the service discovery and watch the name resolution - // updates. + // this function may start the name resolution and watch the updates. Start(target string) error - // Up informs the balancer that gRPC has a connection to the server at + // Up informs the Balancer that gRPC has a connection to the server at // addr. It returns down which is called once the connection to addr gets - // lost or closed. Once down is called, addr may no longer be returned - // by Get. + // lost or closed. Up(addr Address) (down func(error)) // Get gets the address of a server for the rpc corresponding to ctx. // If opts.BlockingWait is true, it blocks if there is no connection available, - // i.e., invocations of Up(...) is equal to that of Down(...). It respects the + // i.e., invocations of Up(...) is equal to those of Down(...). It respects the // timeout or cancellation of ctx when blocking. If opts.BlockingWait is - // false, it may return any address it has notified via Notify(...). It returns + // false, it may return any address it has notified via Notify(...) instead of + // blocking. The returned address may or may not be connected. The function returns // put which is called once the rpc has completed or failed. put can collect and - // report rpc stats to remote load balancer. + // report rpc stats to a remote load balancer. Get(ctx context.Context, opts BalancerGetOptions) (addr Address, put func(), err error) - // Notify gRPC internals the list of Address which should be connected. The list + // Notify notifies gRPC internals the list of Address to be connected. The list // may be from a name resolver or remote load balancer. gRPC internals will // compare it with the exisiting connected addresses. If the address Balancer - // notified is not in the list of the connected addresses, gRPC starts to connect - // the address. If an address in the connected addresses is not in the notification - // list, the corresponding connect will be shutdown gracefully. Otherwise, there - // are no operations. Note that this function must return the full list of the - // Addrresses which should be connected. It is NOT delta. + // notified is not in the exisitng connected addresses, gRPC starts to connect + // the address. If an address in the exisiting connected addresses is not in + // the notification list, the corresponding connection is shutdown gracefully. + // Otherwise, there are no operations to take. Note that this function must + // return the full list of the Addrresses which should be connected. It is NOT delta. Notify() <-chan []Address // Close shuts down the balancer. Close() error @@ -100,7 +101,7 @@ func RoundRobin(r naming.Resolver) Balancer { type roundRobin struct { r naming.Resolver - open []Address // all the known addresses the client can potentially connect + open []Address // all the addresses the client should potentially connect mu sync.Mutex addrCh chan []Address // the channel to notify gRPC internals the list of addresses the client should connect to. connected []Address // all the connected addresses From 26c0af8adc07e29fc8644e00620b1d4cd4dbba1b Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Tue, 24 May 2016 17:50:02 -0700 Subject: [PATCH 25/40] some touchups --- balancer.go | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/balancer.go b/balancer.go index 8c25a9dc..34ada6bf 100644 --- a/balancer.go +++ b/balancer.go @@ -115,6 +115,8 @@ func (rr *roundRobin) watchAddrUpdates(w naming.Watcher) error { if err != nil { return err } + rr.mu.Lock() + defer rr.mu.Unlock() for _, update := range updates { addr := Address{ Addr: update.Addr, @@ -122,15 +124,15 @@ func (rr *roundRobin) watchAddrUpdates(w naming.Watcher) error { } switch update.Op { case naming.Add: - var exisit bool + var exist bool for _, v := range rr.open { if addr == v { - exisit = true + exist = true grpclog.Println("grpc: The name resolver wanted to add an existing address: ", addr) break } } - if exisit { + if exist { continue } rr.open = append(rr.open, addr) @@ -148,8 +150,6 @@ func (rr *roundRobin) watchAddrUpdates(w naming.Watcher) error { } // Make a copy of rr.open and write it onto rr.addrCh so that gRPC internals gets notified. open := make([]Address, len(rr.open)) - rr.mu.Lock() - defer rr.mu.Unlock() copy(open, rr.open) if rr.done { return ErrClientConnClosing @@ -160,6 +160,8 @@ func (rr *roundRobin) watchAddrUpdates(w naming.Watcher) error { func (rr *roundRobin) Start(target string) error { if rr.r == nil { + // If there is no name resolver installed, it is not needed to + // do name resolution. In this case, rr.addrCh stays nil. return nil } w, err := rr.r.Resolve(target) @@ -213,7 +215,7 @@ func (rr *roundRobin) down(addr Address, err error) { } } -// Get returns the next addr in the rotation. It blocks if there is no address available. +// Get returns the next addr in the rotation. func (rr *roundRobin) Get(ctx context.Context, opts BalancerGetOptions) (addr Address, put func(), err error) { var ch chan struct{} rr.mu.Lock() From 8eab9cb6bf3ab3ba2ba831d0948493150df21c9e Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Tue, 24 May 2016 18:14:24 -0700 Subject: [PATCH 26/40] rewrite Balancer Get comments --- balancer.go | 26 ++++++++++++++++++-------- call.go | 2 +- 2 files changed, 19 insertions(+), 9 deletions(-) diff --git a/balancer.go b/balancer.go index 34ada6bf..5841c558 100644 --- a/balancer.go +++ b/balancer.go @@ -71,14 +71,24 @@ type Balancer interface { // addr. It returns down which is called once the connection to addr gets // lost or closed. Up(addr Address) (down func(error)) - // Get gets the address of a server for the rpc corresponding to ctx. - // If opts.BlockingWait is true, it blocks if there is no connection available, - // i.e., invocations of Up(...) is equal to those of Down(...). It respects the - // timeout or cancellation of ctx when blocking. If opts.BlockingWait is - // false, it may return any address it has notified via Notify(...) instead of - // blocking. The returned address may or may not be connected. The function returns - // put which is called once the rpc has completed or failed. put can collect and - // report rpc stats to a remote load balancer. + // Get gets the address of a server for the RPC corresponding to ctx. + // If it returns + // i) a connected address, gRPC internals issues the RPC on the connection to + // this address; + // ii) an address which is notified but not connected, gRPC internals + // * fails RPC if the RPC is fail-fast and connection is in the TransientFailure + // or Shutdown state; + // * issues RPC on the connection otherwise. + // ii) an address which was not notified, gRPC internals treats it as an error + // and will fail the corresponding RPC. + // + // Therefore if opts.BlockingWait is true, it should return a connected address or + // block if there is no connected address. It respects the timeout or + // cancellation of ctx when blocking. If opts.BlockingWait is false (for fail-fast + // RPCs), it should return an address it has notified via Notify(...) immediately. + // + // The function returns put which is called once the rpc has completed or failed. + // put can collect and report rpc stats to a remote load balancer. Get(ctx context.Context, opts BalancerGetOptions) (addr Address, put func(), err error) // Notify notifies gRPC internals the list of Address to be connected. The list // may be from a name resolver or remote load balancer. gRPC internals will diff --git a/call.go b/call.go index 2149ce29..3c34e6f8 100644 --- a/call.go +++ b/call.go @@ -139,7 +139,7 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli stream *transport.Stream put func() ) - // TODO(zhaoq): Need a formal spec of retry strategy for non-failFast rpcs. + // TODO(zhaoq): Need a formal spec of fail-fast. callHdr := &transport.CallHdr{ Host: cc.authority, Method: method, From 9dc3da06330de371475818cac387c2aa4d4cde0d Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Wed, 25 May 2016 11:28:45 -0700 Subject: [PATCH 27/40] make downErr for Balancer down closure --- balancer.go | 21 +++++++++++++++ call.go | 3 ++- clientconn.go | 67 +++++++++++++++++++++++----------------------- clientconn_test.go | 16 +++++------ 4 files changed, 65 insertions(+), 42 deletions(-) diff --git a/balancer.go b/balancer.go index 5841c558..4edfd2fa 100644 --- a/balancer.go +++ b/balancer.go @@ -34,6 +34,7 @@ package grpc import ( + "fmt" "sync" "golang.org/x/net/context" @@ -103,6 +104,26 @@ type Balancer interface { Close() error } +// downErr implements net.Error. It is contructed by gRPC internals and passed to the down +// call of Balancer. +type downErr struct { + timeout bool + temporary bool + desc string +} + +func (e downErr) Error() string { return e.desc } +func (e downErr) Timeout() bool { return e.timeout } +func (e downErr) Temporary() bool { return e.temporary } + +func downErrorf(timeout, temporary bool, format string, a ...interface{}) downErr { + return downErr{ + timeout: timeout, + temporary: temporary, + desc: fmt.Sprintf(format, a...), + } +} + // RoundRobin returns a Balancer that selects addresses round-robin. It starts to watch // the name resolution updates. func RoundRobin(r naming.Resolver) Balancer { diff --git a/call.go b/call.go index 3c34e6f8..e8adad42 100644 --- a/call.go +++ b/call.go @@ -154,7 +154,7 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli if err != nil { // TODO(zhaoq): Probably revisit the error handling. if err == ErrClientConnClosing { - return toRPCErr(err) + return Errorf(codes.FailedPrecondition, "%v", err) } if _, ok := err.(transport.StreamError); ok { return toRPCErr(err) @@ -189,6 +189,7 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli if err != nil { if put != nil { put() + put = nil } if _, ok := err.(transport.ConnectionError); ok { if c.failFast { diff --git a/clientconn.go b/clientconn.go index 81e75d01..274e9e8c 100644 --- a/clientconn.go +++ b/clientconn.go @@ -51,26 +51,27 @@ import ( ) var ( - // 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 - // (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 ClientConn is closing. - ErrClientConnClosing = Errorf(codes.FailedPrecondition, "grpc: the client connection is closing") - // ErrClientConnTimeout indicates that the connection could not be + ErrClientConnClosing = errors.New("grpc: the client connection is closing") + + // 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 + // (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)") + // 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") - // 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 - ErrConnClosing = errors.New("grpc: the addrConn is closing") + errClientConnTimeout = errors.New("grpc: timed out trying to connect") + // 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") // minimum time to give a connection to complete minConnectTimeout = 20 * time.Second ) @@ -337,7 +338,7 @@ func (cc *ClientConn) controller() { } } for _, c := range del { - c.tearDown(ErrConnDrain) + c.tearDown(errConnDrain) } } } @@ -360,12 +361,12 @@ func (cc *ClientConn) newAddrConn(addr Address, skipWait bool) error { } } if !ok { - return ErrNoTransportSecurity + return errNoTransportSecurity } } else { for _, cd := range ac.dopts.copts.AuthOptions { if cd.RequireTransportSecurity() { - return ErrCredentialsMisuse + return errCredentialsMisuse } } } @@ -529,10 +530,10 @@ func (ac *addrConn) resetTransport(closeTransport bool) error { if ac.state == Shutdown { // ac.tearDown(...) has been invoked. ac.mu.Unlock() - return ErrConnClosing + return errConnClosing } if ac.down != nil { - ac.down(ErrNetworkIO) + ac.down(downErrorf(false, true, "%v", errNetworkIO)) ac.down = nil } ac.state = Connecting @@ -545,14 +546,14 @@ func (ac *addrConn) resetTransport(closeTransport bool) error { // Adjust timeout for the current try. copts := ac.dopts.copts if copts.Timeout < 0 { - ac.tearDown(ErrClientConnTimeout) - return ErrClientConnTimeout + ac.tearDown(errClientConnTimeout) + return errClientConnTimeout } if copts.Timeout > 0 { copts.Timeout -= time.Since(start) if copts.Timeout <= 0 { - ac.tearDown(ErrClientConnTimeout) - return ErrClientConnTimeout + ac.tearDown(errClientConnTimeout) + return errClientConnTimeout } } sleepTime := ac.dopts.bs.backoff(retries) @@ -570,7 +571,7 @@ func (ac *addrConn) resetTransport(closeTransport bool) error { if ac.state == Shutdown { // ac.tearDown(...) has been invoked. ac.mu.Unlock() - return ErrConnClosing + return errConnClosing } ac.errorf("transient failure: %v", err) ac.state = TransientFailure @@ -589,8 +590,8 @@ func (ac *addrConn) resetTransport(closeTransport bool) error { ac.mu.Lock() ac.errorf("connection timeout") ac.mu.Unlock() - ac.tearDown(ErrClientConnTimeout) - return ErrClientConnTimeout + ac.tearDown(errClientConnTimeout) + return errClientConnTimeout } closeTransport = false select { @@ -607,7 +608,7 @@ func (ac *addrConn) resetTransport(closeTransport bool) error { // ac.tearDown(...) has been invoked. ac.mu.Unlock() newTransport.Close() - return ErrConnClosing + return errConnClosing } ac.state = Ready ac.stateCV.Broadcast() @@ -662,7 +663,7 @@ func (ac *addrConn) wait(ctx context.Context) (transport.ClientTransport, error) switch { case ac.state == Shutdown: ac.mu.Unlock() - return nil, ErrConnClosing + return nil, errConnClosing case ac.state == Ready: ct := ac.transport ac.mu.Unlock() @@ -699,7 +700,7 @@ func (ac *addrConn) tearDown(err error) { ac.cc.mu.Unlock() }() if ac.down != nil { - ac.down(err) + ac.down(downErrorf(false, false, "%v", err)) ac.down = nil } if ac.state == Shutdown { @@ -716,7 +717,7 @@ func (ac *addrConn) tearDown(err error) { ac.ready = nil } if ac.transport != nil { - if err == ErrConnDrain { + if err == errConnDrain { ac.transport.GracefulClose() } else { ac.transport.Close() diff --git a/clientconn_test.go b/clientconn_test.go index b44f5b29..09d7f110 100644 --- a/clientconn_test.go +++ b/clientconn_test.go @@ -47,8 +47,8 @@ func TestDialTimeout(t *testing.T) { if err == nil { conn.Close() } - if err != ErrClientConnTimeout { - t.Fatalf("Dial(_, _) = %v, %v, want %v", conn, err, ErrClientConnTimeout) + if err != errClientConnTimeout { + t.Fatalf("Dial(_, _) = %v, %v, want %v", conn, err, errClientConnTimeout) } } @@ -61,8 +61,8 @@ func TestTLSDialTimeout(t *testing.T) { if err == nil { conn.Close() } - if err != ErrClientConnTimeout { - t.Fatalf("grpc.Dial(_, _) = %v, %v, want %v", conn, err, ErrClientConnTimeout) + if err != errClientConnTimeout { + t.Fatalf("grpc.Dial(_, _) = %v, %v, want %v", conn, err, errClientConnTimeout) } } @@ -72,12 +72,12 @@ func TestCredentialsMisuse(t *testing.T) { t.Fatalf("Failed to create credentials %v", err) } // Two conflicting credential configurations - if _, err := Dial("Non-Existent.Server:80", WithTransportCredentials(creds), WithTimeout(time.Millisecond), WithBlock(), WithInsecure()); err != ErrCredentialsMisuse { - t.Fatalf("Dial(_, _) = _, %v, want _, %v", err, ErrCredentialsMisuse) + if _, err := Dial("Non-Existent.Server:80", WithTransportCredentials(creds), WithTimeout(time.Millisecond), WithBlock(), WithInsecure()); err != errCredentialsMisuse { + t.Fatalf("Dial(_, _) = _, %v, want _, %v", err, errCredentialsMisuse) } // security info on insecure connection - if _, err := Dial("Non-Existent.Server:80", WithPerRPCCredentials(creds), WithTimeout(time.Millisecond), WithBlock(), WithInsecure()); err != ErrCredentialsMisuse { - t.Fatalf("Dial(_, _) = _, %v, want _, %v", err, ErrCredentialsMisuse) + if _, err := Dial("Non-Existent.Server:80", WithPerRPCCredentials(creds), WithTimeout(time.Millisecond), WithBlock(), WithInsecure()); err != errCredentialsMisuse { + t.Fatalf("Dial(_, _) = _, %v, want _, %v", err, errCredentialsMisuse) } } From 2a6703fb095df8b728d9879c552c5f004a811f16 Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Wed, 25 May 2016 11:38:29 -0700 Subject: [PATCH 28/40] Improve Get comments --- balancer.go | 21 +++++++++++++-------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/balancer.go b/balancer.go index 4edfd2fa..b0b8a5bc 100644 --- a/balancer.go +++ b/balancer.go @@ -73,20 +73,25 @@ type Balancer interface { // lost or closed. Up(addr Address) (down func(error)) // Get gets the address of a server for the RPC corresponding to ctx. - // If it returns - // i) a connected address, gRPC internals issues the RPC on the connection to - // this address; - // ii) an address which is notified but not connected, gRPC internals + // i) If it returns a connected address, gRPC internals issues the RPC on the + // connection to this address; + // ii) If it returns an address on which the connection is under construction + // (initiated by Notify(...)) but not connected, gRPC internals // * fails RPC if the RPC is fail-fast and connection is in the TransientFailure // or Shutdown state; + // or // * issues RPC on the connection otherwise. - // ii) an address which was not notified, gRPC internals treats it as an error - // and will fail the corresponding RPC. + // iii) If it returns an address on which the connection does not exist, gRPC + // internals treats it as an error and will fail the corresponding RPC. // - // Therefore if opts.BlockingWait is true, it should return a connected address or + // Therefore, we recommend the following general rule when you write your own + // custom Balancer. + // + // If opts.BlockingWait is true, it should return a connected address or // block if there is no connected address. It respects the timeout or // cancellation of ctx when blocking. If opts.BlockingWait is false (for fail-fast - // RPCs), it should return an address it has notified via Notify(...) immediately. + // RPCs), it should return an address it has notified via Notify(...) immediately + // instead of blocking. // // The function returns put which is called once the rpc has completed or failed. // put can collect and report rpc stats to a remote load balancer. From fc73096412f0f37586440960a7f675c12ba36cdb Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Wed, 25 May 2016 11:52:51 -0700 Subject: [PATCH 29/40] fix some typos --- balancer.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/balancer.go b/balancer.go index b0b8a5bc..e1dfb54a 100644 --- a/balancer.go +++ b/balancer.go @@ -54,7 +54,7 @@ type Address struct { Metadata interface{} } -// BalancerGetOption configures a Get call. +// BalancerGetOptions configures a Get call. // This is the EXPERIMENTAL API and may be changed or extended in the future. type BalancerGetOptions struct { // BlockingWait specifies whether Get should block when there is no @@ -94,22 +94,22 @@ type Balancer interface { // instead of blocking. // // The function returns put which is called once the rpc has completed or failed. - // put can collect and report rpc stats to a remote load balancer. + // put can collect and report RPC stats to a remote load balancer. Get(ctx context.Context, opts BalancerGetOptions) (addr Address, put func(), err error) // Notify notifies gRPC internals the list of Address to be connected. The list // may be from a name resolver or remote load balancer. gRPC internals will - // compare it with the exisiting connected addresses. If the address Balancer - // notified is not in the exisitng connected addresses, gRPC starts to connect - // the address. If an address in the exisiting connected addresses is not in + // compare it with the existing connected addresses. If the address Balancer + // notified is not in the existing connected addresses, gRPC starts to connect + // the address. If an address in the existing connected addresses is not in // the notification list, the corresponding connection is shutdown gracefully. // Otherwise, there are no operations to take. Note that this function must - // return the full list of the Addrresses which should be connected. It is NOT delta. + // return the full list of the Addresses which should be connected. It is NOT delta. Notify() <-chan []Address // Close shuts down the balancer. Close() error } -// downErr implements net.Error. It is contructed by gRPC internals and passed to the down +// downErr implements net.Error. It is constructed by gRPC internals and passed to the down // call of Balancer. type downErr struct { timeout bool From d4fcb1a6512943f0d7229799af69c0dfcd7dfee3 Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Wed, 25 May 2016 15:31:54 -0700 Subject: [PATCH 30/40] fix some comments --- balancer.go | 8 +++++++- naming/naming.go | 3 ++- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/balancer.go b/balancer.go index e1dfb54a..da6cde1d 100644 --- a/balancer.go +++ b/balancer.go @@ -88,7 +88,7 @@ type Balancer interface { // custom Balancer. // // If opts.BlockingWait is true, it should return a connected address or - // block if there is no connected address. It respects the timeout or + // block if there is no connected address. It should respect the timeout or // cancellation of ctx when blocking. If opts.BlockingWait is false (for fail-fast // RPCs), it should return an address it has notified via Notify(...) immediately // instead of blocking. @@ -137,6 +137,7 @@ func RoundRobin(r naming.Resolver) Balancer { type roundRobin struct { r naming.Resolver + w []naming.Watcher open []Address // all the addresses the client should potentially connect mu sync.Mutex addrCh chan []Address // the channel to notify gRPC internals the list of addresses the client should connect to. @@ -149,6 +150,7 @@ type roundRobin struct { func (rr *roundRobin) watchAddrUpdates(w naming.Watcher) error { updates, err := w.Next() if err != nil { + grpclog.Println("grpc: the naming watcher stops working due to %v.", err) return err } rr.mu.Lock() @@ -204,6 +206,7 @@ func (rr *roundRobin) Start(target string) error { if err != nil { return err } + rr.w = []naming.Watcher{w} rr.addrCh = make(chan []Address) go func() { for { @@ -314,6 +317,9 @@ func (rr *roundRobin) Close() error { rr.mu.Lock() defer rr.mu.Unlock() rr.done = true + for _, w := range rr.w { + w.Close() + } if rr.waitCh != nil { close(rr.waitCh) rr.waitCh = nil diff --git a/naming/naming.go b/naming/naming.go index 06605607..d2b617ca 100644 --- a/naming/naming.go +++ b/naming/naming.go @@ -66,7 +66,8 @@ type Resolver interface { // Watcher watches for the updates on the specified target. type Watcher interface { // Next blocks until an update or error happens. It may return one or more - // updates. The first call should get the full set of the results. + // updates. The first call should get the full set of the results. It only + // returns the error Watcher cannot recover. Next() ([]*Update, error) // Close closes the Watcher. Close() From 98c0b8aa3e6ae828ecaa365050874470cc5d15cc Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Wed, 25 May 2016 15:55:03 -0700 Subject: [PATCH 31/40] Fix a race conditon and add some small touchups --- balancer.go | 30 ++++++++++++++++++------------ 1 file changed, 18 insertions(+), 12 deletions(-) diff --git a/balancer.go b/balancer.go index da6cde1d..80220cf6 100644 --- a/balancer.go +++ b/balancer.go @@ -129,15 +129,15 @@ func downErrorf(timeout, temporary bool, format string, a ...interface{}) downEr } } -// RoundRobin returns a Balancer that selects addresses round-robin. It starts to watch -// the name resolution updates. +// RoundRobin returns a Balancer that selects addresses round-robin. It uses r to watch +// the name resolution updates and updates the addresses available correspondingly. func RoundRobin(r naming.Resolver) Balancer { return &roundRobin{r: r} } type roundRobin struct { r naming.Resolver - w []naming.Watcher + w naming.Watcher open []Address // all the addresses the client should potentially connect mu sync.Mutex addrCh chan []Address // the channel to notify gRPC internals the list of addresses the client should connect to. @@ -147,8 +147,8 @@ type roundRobin struct { done bool // The Balancer is closed. } -func (rr *roundRobin) watchAddrUpdates(w naming.Watcher) error { - updates, err := w.Next() +func (rr *roundRobin) watchAddrUpdates() error { + updates, err := rr.w.Next() if err != nil { grpclog.Println("grpc: the naming watcher stops working due to %v.", err) return err @@ -187,7 +187,7 @@ func (rr *roundRobin) watchAddrUpdates(w naming.Watcher) error { } } // Make a copy of rr.open and write it onto rr.addrCh so that gRPC internals gets notified. - open := make([]Address, len(rr.open)) + open := make([]Address, len(rr.open), len(rr.open)) copy(open, rr.open) if rr.done { return ErrClientConnClosing @@ -206,11 +206,11 @@ func (rr *roundRobin) Start(target string) error { if err != nil { return err } - rr.w = []naming.Watcher{w} + rr.w = w rr.addrCh = make(chan []Address) go func() { for { - if err := rr.watchAddrUpdates(w); err != nil { + if err := rr.watchAddrUpdates(); err != nil { return } } @@ -218,7 +218,7 @@ func (rr *roundRobin) Start(target string) error { return nil } -// Up appends addr to the end of rr.addrs and sends notification if there +// Up appends addr to the end of rr.connected and sends notification if there // are pending Get() calls. func (rr *roundRobin) Up(addr Address) func(error) { rr.mu.Lock() @@ -241,7 +241,7 @@ func (rr *roundRobin) Up(addr Address) func(error) { } } -// down removes addr from rr.addrs and moves the remaining addrs forward. +// down removes addr from rr.connected and moves the remaining addrs forward. func (rr *roundRobin) down(addr Address, err error) { rr.mu.Lock() defer rr.mu.Unlock() @@ -295,6 +295,12 @@ func (rr *roundRobin) Get(ctx context.Context, opts BalancerGetOptions) (addr Ad } if len(rr.connected) == 0 { // The newly added addr got removed by Down() again. + if rr.waitCh == nil { + ch = make(chan struct{}) + rr.waitCh = ch + } else { + ch = rr.waitCh + } rr.mu.Unlock() continue } @@ -317,8 +323,8 @@ func (rr *roundRobin) Close() error { rr.mu.Lock() defer rr.mu.Unlock() rr.done = true - for _, w := range rr.w { - w.Close() + if rr.w != nil { + rr.w.Close() } if rr.waitCh != nil { close(rr.waitCh) From 9a14ca798d4f0a5f62d4df99f049cd8e4151ba06 Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Wed, 25 May 2016 17:18:05 -0700 Subject: [PATCH 32/40] Add more comments --- call.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/call.go b/call.go index e8adad42..d6d993b4 100644 --- a/call.go +++ b/call.go @@ -137,7 +137,9 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli err error t transport.ClientTransport stream *transport.Stream - put func() + // Record the put handler from Balancer.Get(...). It is called once the + // RPC has completed or failed. + put func() ) // TODO(zhaoq): Need a formal spec of fail-fast. callHdr := &transport.CallHdr{ From db887c9bcd224a919fae77c63283655504aa2fcd Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Wed, 25 May 2016 18:17:23 -0700 Subject: [PATCH 33/40] rename controller --- clientconn.go | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/clientconn.go b/clientconn.go index 274e9e8c..3bb48a4c 100644 --- a/clientconn.go +++ b/clientconn.go @@ -246,7 +246,7 @@ func Dial(target string, opts ...DialOption) (*ClientConn, error) { return nil, err } } - go cc.controller() + go cc.lbWatcher() } colonPos := strings.LastIndex(target, ":") @@ -302,7 +302,7 @@ type ClientConn struct { conns map[Address]*addrConn } -func (cc *ClientConn) controller() { +func (cc *ClientConn) lbWatcher() { for { addrs, ok := <-cc.balancer.Notify() if !ok { @@ -333,9 +333,7 @@ func (cc *ClientConn) controller() { } cc.mu.Unlock() for _, a := range addrs { - if err := cc.newAddrConn(a, true); err != nil { - - } + cc.newAddrConn(a, true) } for _, c := range del { c.tearDown(errConnDrain) From 817b48219c5056ef3d371d617affcec4a7901e4a Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Thu, 26 May 2016 14:20:31 -0700 Subject: [PATCH 34/40] Fix a typo bug and handle stale addrConns --- clientconn.go | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/clientconn.go b/clientconn.go index 3bb48a4c..9868c10e 100644 --- a/clientconn.go +++ b/clientconn.go @@ -46,7 +46,6 @@ import ( "google.golang.org/grpc/codes" "google.golang.org/grpc/credentials" "google.golang.org/grpc/grpclog" - "google.golang.org/grpc/naming" "google.golang.org/grpc/transport" ) @@ -293,7 +292,6 @@ func (s ConnectivityState) String() string { // ClientConn represents a client connection to an RPC server. type ClientConn struct { target string - watcher naming.Watcher balancer Balancer authority string dopts dialOptions @@ -332,7 +330,7 @@ func (cc *ClientConn) lbWatcher() { } } cc.mu.Unlock() - for _, a := range addrs { + for _, a := range add { cc.newAddrConn(a, true) } for _, c := range del { @@ -374,9 +372,15 @@ func (cc *ClientConn) newAddrConn(addr Address, skipWait bool) error { 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 { @@ -440,9 +444,6 @@ func (cc *ClientConn) Close() error { cc.conns = nil cc.mu.Unlock() cc.balancer.Close() - if cc.watcher != nil { - cc.watcher.Close() - } for _, ac := range conns { ac.tearDown(ErrClientConnClosing) } From fca009f66ffa7df0587e64163df4863b8c0900c3 Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Thu, 26 May 2016 14:53:32 -0700 Subject: [PATCH 35/40] Fix some small comments --- balancer.go | 3 ++- clientconn.go | 8 ++++---- naming/naming.go | 4 ++-- transport/http2_client.go | 1 - 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/balancer.go b/balancer.go index 80220cf6..23431c4e 100644 --- a/balancer.go +++ b/balancer.go @@ -66,7 +66,8 @@ type BalancerGetOptions struct { // This is the EXPERIMENTAL API and may be changed or extended in the future. type Balancer interface { // Start does the initialization work to bootstrap a Balancer. For example, - // this function may start the name resolution and watch the updates. + // this function may start the name resolution and watch the updates. It will + // be called when dialing. Start(target string) error // Up informs the Balancer that gRPC has a connection to the server at // addr. It returns down which is called once the connection to addr gets diff --git a/clientconn.go b/clientconn.go index 9868c10e..39cee3db 100644 --- a/clientconn.go +++ b/clientconn.go @@ -698,14 +698,14 @@ func (ac *addrConn) tearDown(err error) { } ac.cc.mu.Unlock() }() - if ac.down != nil { - ac.down(downErrorf(false, false, "%v", err)) - ac.down = nil - } 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() diff --git a/naming/naming.go b/naming/naming.go index d2b617ca..c2e0871e 100644 --- a/naming/naming.go +++ b/naming/naming.go @@ -66,8 +66,8 @@ type Resolver interface { // Watcher watches for the updates on the specified target. type Watcher interface { // Next blocks until an update or error happens. It may return one or more - // updates. The first call should get the full set of the results. It only - // returns the error Watcher cannot recover. + // updates. The first call should get the full set of the results. It should + // return an error if and only if Watcher cannot recover. Next() ([]*Update, error) // Close closes the Watcher. Close() diff --git a/transport/http2_client.go b/transport/http2_client.go index d6fde4f1..05d6e496 100644 --- a/transport/http2_client.go +++ b/transport/http2_client.go @@ -398,7 +398,6 @@ func (t *http2Client) CloseStream(s *Stream, err error) { t.mu.Lock() if t.activeStreams == nil { t.mu.Unlock() - t.Close() return } if t.streamsQuota != nil { From 7acc9989fd7f13526ce729b97c52eec2c7b3d10f Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Thu, 26 May 2016 15:34:18 -0700 Subject: [PATCH 36/40] polish the comments of Notify() and Get(). --- balancer.go | 21 ++++++++++----------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/balancer.go b/balancer.go index 23431c4e..5fec2c9c 100644 --- a/balancer.go +++ b/balancer.go @@ -85,9 +85,7 @@ type Balancer interface { // iii) If it returns an address on which the connection does not exist, gRPC // internals treats it as an error and will fail the corresponding RPC. // - // Therefore, we recommend the following general rule when you write your own - // custom Balancer. - // + // Therefore, the following is the recommended rule when writing a custom Balancer. // If opts.BlockingWait is true, it should return a connected address or // block if there is no connected address. It should respect the timeout or // cancellation of ctx when blocking. If opts.BlockingWait is false (for fail-fast @@ -97,14 +95,15 @@ type Balancer interface { // The function returns put which is called once the rpc has completed or failed. // put can collect and report RPC stats to a remote load balancer. Get(ctx context.Context, opts BalancerGetOptions) (addr Address, put func(), err error) - // Notify notifies gRPC internals the list of Address to be connected. The list - // may be from a name resolver or remote load balancer. gRPC internals will - // compare it with the existing connected addresses. If the address Balancer - // notified is not in the existing connected addresses, gRPC starts to connect - // the address. If an address in the existing connected addresses is not in - // the notification list, the corresponding connection is shutdown gracefully. - // Otherwise, there are no operations to take. Note that this function must - // return the full list of the Addresses which should be connected. It is NOT delta. + // Notify returns a channel that is used by gRPC internals to watch the addresses + // gRPC needs to connect. The addresses might be from a name resolver or remote + // load balancer. gRPC internals will compare it with the existing connected + // addresses. If the address Balancer notified is not in the existing connected + // addresses, gRPC starts to connect the address. If an address in the existing + // connected addresses is not in the notification list, the corresponding connection + // is shutdown gracefully. Otherwise, there are no operations to take. Note that + // the Address slice must be the full list of the Addresses which should be connected. + // It is NOT delta. Notify() <-chan []Address // Close shuts down the balancer. Close() error From cf595a8fcdac74c5dd0098b539860677d28d3cdb Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Thu, 26 May 2016 15:49:50 -0700 Subject: [PATCH 37/40] polish Address comments --- balancer.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/balancer.go b/balancer.go index 5fec2c9c..91e03b65 100644 --- a/balancer.go +++ b/balancer.go @@ -49,8 +49,7 @@ type Address struct { // Addr is the server address on which a connection will be established. Addr string // Metadata is the information associated with Addr, which may be used - // to make load balancing decision. This is from the metadata attached - // in the address updates from name resolver. + // to make load balancing decision. Metadata interface{} } From 5d8c1e2d84502be80f51ef76b02bca1b91e47d98 Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Thu, 26 May 2016 16:14:27 -0700 Subject: [PATCH 38/40] Remove metadata when creating an Address --- balancer.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/balancer.go b/balancer.go index 91e03b65..64043b5d 100644 --- a/balancer.go +++ b/balancer.go @@ -156,8 +156,7 @@ func (rr *roundRobin) watchAddrUpdates() error { defer rr.mu.Unlock() for _, update := range updates { addr := Address{ - Addr: update.Addr, - Metadata: update.Metadata, + Addr: update.Addr, } switch update.Op { case naming.Add: From 3d000f3e5f0097385bd5a232a7ecd5dc2611c4fa Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Thu, 26 May 2016 16:50:29 -0700 Subject: [PATCH 39/40] fix a test error msg --- balancer_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/balancer_test.go b/balancer_test.go index 976432f4..9d8d2bcd 100644 --- a/balancer_test.go +++ b/balancer_test.go @@ -263,7 +263,7 @@ func TestCloseWithPendingRPC(t *testing.T) { defer wg.Done() var reply string if err := Invoke(context.Background(), "/foo/bar", &expectedRequest, &reply, cc); err == nil { - t.Errorf("grpc.Invoke(_, _, _, _, _) = %v, want not", err) + t.Errorf("grpc.Invoke(_, _, _, _, _) = %v, want not nil", err) } }() go func() { From 2700f043b937c2b59b4a520bc6ddbb440a2de20e Mon Sep 17 00:00:00 2001 From: iamqizhao Date: Wed, 1 Jun 2016 14:09:21 -0700 Subject: [PATCH 40/40] some small fixes --- balancer.go | 11 ++++++++--- clientconn.go | 7 +------ 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/balancer.go b/balancer.go index 64043b5d..348bf975 100644 --- a/balancer.go +++ b/balancer.go @@ -71,14 +71,16 @@ type Balancer interface { // Up informs the Balancer that gRPC has a connection to the server at // addr. It returns down which is called once the connection to addr gets // lost or closed. + // TODO: It is not clear how to construct and take advantage the meaningful error + // parameter for down. Need realistic demands to guide. Up(addr Address) (down func(error)) // Get gets the address of a server for the RPC corresponding to ctx. // i) If it returns a connected address, gRPC internals issues the RPC on the // connection to this address; // ii) If it returns an address on which the connection is under construction // (initiated by Notify(...)) but not connected, gRPC internals - // * fails RPC if the RPC is fail-fast and connection is in the TransientFailure - // or Shutdown state; + // * fails RPC if the RPC is fail-fast and connection is in the TransientFailure or + // Shutdown state; // or // * issues RPC on the connection otherwise. // iii) If it returns an address on which the connection does not exist, gRPC @@ -92,7 +94,10 @@ type Balancer interface { // instead of blocking. // // The function returns put which is called once the rpc has completed or failed. - // put can collect and report RPC stats to a remote load balancer. + // put can collect and report RPC stats to a remote load balancer. gRPC internals + // will try to call this again if err is non-nil (unless err is ErrClientConnClosing). + // + // TODO: Add other non-recoverable errors? Get(ctx context.Context, opts BalancerGetOptions) (addr Address, put func(), err error) // Notify returns a channel that is used by gRPC internals to watch the addresses // gRPC needs to connect. The addresses might be from a name resolver or remote diff --git a/clientconn.go b/clientconn.go index 39cee3db..0d640b47 100644 --- a/clientconn.go +++ b/clientconn.go @@ -301,12 +301,7 @@ type ClientConn struct { } func (cc *ClientConn) lbWatcher() { - for { - addrs, ok := <-cc.balancer.Notify() - if !ok { - // cc has been closed. - return - } + for addrs := range cc.balancer.Notify() { var ( add []Address // Addresses need to setup connections. del []*addrConn // Connections need to tear down.