Browse Source

Merge pull request #5583 from heyitsanthony/grpc-nuke-waitstate

clientv3: use grpc balancer
Anthony Romano 9 years ago
parent
commit
ff2b24a8ac
37 changed files with 1309 additions and 1199 deletions
  1. 14 15
      clientv3/auth.go
  2. 64 0
      clientv3/balancer.go
  3. 88 216
      clientv3/client.go
  4. 11 39
      clientv3/cluster.go
  5. 0 8
      clientv3/config.go
  6. 12 21
      clientv3/integration/kv_test.go
  7. 7 6
      clientv3/integration/lease_test.go
  8. 15 27
      clientv3/integration/txn_test.go
  9. 23 18
      clientv3/integration/watch_test.go
  10. 13 46
      clientv3/kv.go
  11. 16 45
      clientv3/lease.go
  12. 16 33
      clientv3/maintenance.go
  13. 0 108
      clientv3/remote_client.go
  14. 3 14
      clientv3/txn.go
  15. 4 19
      clientv3/watch.go
  16. 10 10
      cmd/Godeps/Godeps.json
  17. 6 6
      cmd/vendor/google.golang.org/grpc/PATENTS
  18. 4 4
      cmd/vendor/google.golang.org/grpc/backoff.go
  19. 340 0
      cmd/vendor/google.golang.org/grpc/balancer.go
  20. 46 22
      cmd/vendor/google.golang.org/grpc/call.go
  21. 357 260
      cmd/vendor/google.golang.org/grpc/clientconn.go
  22. 2 1
      cmd/vendor/google.golang.org/grpc/naming/naming.go
  23. 0 243
      cmd/vendor/google.golang.org/grpc/picker.go
  24. 2 2
      cmd/vendor/google.golang.org/grpc/rpc_util.go
  25. 7 3
      cmd/vendor/google.golang.org/grpc/server.go
  26. 13 3
      cmd/vendor/google.golang.org/grpc/stream.go
  27. 1 2
      cmd/vendor/google.golang.org/grpc/trace.go
  28. 12 2
      cmd/vendor/google.golang.org/grpc/transport/handler_server.go
  29. 38 2
      cmd/vendor/google.golang.org/grpc/transport/http2_client.go
  30. 8 0
      cmd/vendor/google.golang.org/grpc/transport/http2_server.go
  31. 12 1
      cmd/vendor/google.golang.org/grpc/transport/http_util.go
  32. 6 1
      cmd/vendor/google.golang.org/grpc/transport/transport.go
  33. 0 2
      etcdserver/api/v3rpc/rpctypes/error.go
  34. 141 0
      integration/bridge.go
  35. 13 1
      integration/cluster.go
  36. 2 17
      integration/v3_grpc_test.go
  37. 3 2
      integration/v3_watch_test.go

+ 14 - 15
clientv3/auth.go

@@ -19,7 +19,6 @@ import (
 	"strings"
 	"strings"
 
 
 	"github.com/coreos/etcd/auth/authpb"
 	"github.com/coreos/etcd/auth/authpb"
-	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 	"golang.org/x/net/context"
 	"golang.org/x/net/context"
 	"google.golang.org/grpc"
 	"google.golang.org/grpc"
@@ -109,47 +108,47 @@ func NewAuth(c *Client) Auth {
 
 
 func (auth *auth) AuthEnable(ctx context.Context) (*AuthEnableResponse, error) {
 func (auth *auth) AuthEnable(ctx context.Context) (*AuthEnableResponse, error) {
 	resp, err := auth.remote.AuthEnable(ctx, &pb.AuthEnableRequest{})
 	resp, err := auth.remote.AuthEnable(ctx, &pb.AuthEnableRequest{})
-	return (*AuthEnableResponse)(resp), rpctypes.Error(err)
+	return (*AuthEnableResponse)(resp), toErr(ctx, err)
 }
 }
 
 
 func (auth *auth) AuthDisable(ctx context.Context) (*AuthDisableResponse, error) {
 func (auth *auth) AuthDisable(ctx context.Context) (*AuthDisableResponse, error) {
 	resp, err := auth.remote.AuthDisable(ctx, &pb.AuthDisableRequest{})
 	resp, err := auth.remote.AuthDisable(ctx, &pb.AuthDisableRequest{})
-	return (*AuthDisableResponse)(resp), rpctypes.Error(err)
+	return (*AuthDisableResponse)(resp), toErr(ctx, err)
 }
 }
 
 
 func (auth *auth) UserAdd(ctx context.Context, name string, password string) (*AuthUserAddResponse, error) {
 func (auth *auth) UserAdd(ctx context.Context, name string, password string) (*AuthUserAddResponse, error) {
 	resp, err := auth.remote.UserAdd(ctx, &pb.AuthUserAddRequest{Name: name, Password: password})
 	resp, err := auth.remote.UserAdd(ctx, &pb.AuthUserAddRequest{Name: name, Password: password})
-	return (*AuthUserAddResponse)(resp), rpctypes.Error(err)
+	return (*AuthUserAddResponse)(resp), toErr(ctx, err)
 }
 }
 
 
 func (auth *auth) UserDelete(ctx context.Context, name string) (*AuthUserDeleteResponse, error) {
 func (auth *auth) UserDelete(ctx context.Context, name string) (*AuthUserDeleteResponse, error) {
 	resp, err := auth.remote.UserDelete(ctx, &pb.AuthUserDeleteRequest{Name: name})
 	resp, err := auth.remote.UserDelete(ctx, &pb.AuthUserDeleteRequest{Name: name})
-	return (*AuthUserDeleteResponse)(resp), rpctypes.Error(err)
+	return (*AuthUserDeleteResponse)(resp), toErr(ctx, err)
 }
 }
 
 
 func (auth *auth) UserChangePassword(ctx context.Context, name string, password string) (*AuthUserChangePasswordResponse, error) {
 func (auth *auth) UserChangePassword(ctx context.Context, name string, password string) (*AuthUserChangePasswordResponse, error) {
 	resp, err := auth.remote.UserChangePassword(ctx, &pb.AuthUserChangePasswordRequest{Name: name, Password: password})
 	resp, err := auth.remote.UserChangePassword(ctx, &pb.AuthUserChangePasswordRequest{Name: name, Password: password})
-	return (*AuthUserChangePasswordResponse)(resp), rpctypes.Error(err)
+	return (*AuthUserChangePasswordResponse)(resp), toErr(ctx, err)
 }
 }
 
 
 func (auth *auth) UserGrantRole(ctx context.Context, user string, role string) (*AuthUserGrantRoleResponse, error) {
 func (auth *auth) UserGrantRole(ctx context.Context, user string, role string) (*AuthUserGrantRoleResponse, error) {
 	resp, err := auth.remote.UserGrantRole(ctx, &pb.AuthUserGrantRoleRequest{User: user, Role: role})
 	resp, err := auth.remote.UserGrantRole(ctx, &pb.AuthUserGrantRoleRequest{User: user, Role: role})
-	return (*AuthUserGrantRoleResponse)(resp), rpctypes.Error(err)
+	return (*AuthUserGrantRoleResponse)(resp), toErr(ctx, err)
 }
 }
 
 
 func (auth *auth) UserGet(ctx context.Context, name string) (*AuthUserGetResponse, error) {
 func (auth *auth) UserGet(ctx context.Context, name string) (*AuthUserGetResponse, error) {
 	resp, err := auth.remote.UserGet(ctx, &pb.AuthUserGetRequest{Name: name})
 	resp, err := auth.remote.UserGet(ctx, &pb.AuthUserGetRequest{Name: name})
-	return (*AuthUserGetResponse)(resp), rpctypes.Error(err)
+	return (*AuthUserGetResponse)(resp), toErr(ctx, err)
 }
 }
 
 
 func (auth *auth) UserRevokeRole(ctx context.Context, name string, role string) (*AuthUserRevokeRoleResponse, error) {
 func (auth *auth) UserRevokeRole(ctx context.Context, name string, role string) (*AuthUserRevokeRoleResponse, error) {
 	resp, err := auth.remote.UserRevokeRole(ctx, &pb.AuthUserRevokeRoleRequest{Name: name, Role: role})
 	resp, err := auth.remote.UserRevokeRole(ctx, &pb.AuthUserRevokeRoleRequest{Name: name, Role: role})
-	return (*AuthUserRevokeRoleResponse)(resp), rpctypes.Error(err)
+	return (*AuthUserRevokeRoleResponse)(resp), toErr(ctx, err)
 }
 }
 
 
 func (auth *auth) RoleAdd(ctx context.Context, name string) (*AuthRoleAddResponse, error) {
 func (auth *auth) RoleAdd(ctx context.Context, name string) (*AuthRoleAddResponse, error) {
 	resp, err := auth.remote.RoleAdd(ctx, &pb.AuthRoleAddRequest{Name: name})
 	resp, err := auth.remote.RoleAdd(ctx, &pb.AuthRoleAddRequest{Name: name})
-	return (*AuthRoleAddResponse)(resp), rpctypes.Error(err)
+	return (*AuthRoleAddResponse)(resp), toErr(ctx, err)
 }
 }
 
 
 func (auth *auth) RoleGrantPermission(ctx context.Context, name string, key string, permType PermissionType) (*AuthRoleGrantPermissionResponse, error) {
 func (auth *auth) RoleGrantPermission(ctx context.Context, name string, key string, permType PermissionType) (*AuthRoleGrantPermissionResponse, error) {
@@ -158,22 +157,22 @@ func (auth *auth) RoleGrantPermission(ctx context.Context, name string, key stri
 		PermType: authpb.Permission_Type(permType),
 		PermType: authpb.Permission_Type(permType),
 	}
 	}
 	resp, err := auth.remote.RoleGrantPermission(ctx, &pb.AuthRoleGrantPermissionRequest{Name: name, Perm: perm})
 	resp, err := auth.remote.RoleGrantPermission(ctx, &pb.AuthRoleGrantPermissionRequest{Name: name, Perm: perm})
-	return (*AuthRoleGrantPermissionResponse)(resp), rpctypes.Error(err)
+	return (*AuthRoleGrantPermissionResponse)(resp), toErr(ctx, err)
 }
 }
 
 
 func (auth *auth) RoleGet(ctx context.Context, role string) (*AuthRoleGetResponse, error) {
 func (auth *auth) RoleGet(ctx context.Context, role string) (*AuthRoleGetResponse, error) {
 	resp, err := auth.remote.RoleGet(ctx, &pb.AuthRoleGetRequest{Role: role})
 	resp, err := auth.remote.RoleGet(ctx, &pb.AuthRoleGetRequest{Role: role})
-	return (*AuthRoleGetResponse)(resp), rpctypes.Error(err)
+	return (*AuthRoleGetResponse)(resp), toErr(ctx, err)
 }
 }
 
 
 func (auth *auth) RoleRevokePermission(ctx context.Context, role string, key string) (*AuthRoleRevokePermissionResponse, error) {
 func (auth *auth) RoleRevokePermission(ctx context.Context, role string, key string) (*AuthRoleRevokePermissionResponse, error) {
 	resp, err := auth.remote.RoleRevokePermission(ctx, &pb.AuthRoleRevokePermissionRequest{Role: role, Key: key})
 	resp, err := auth.remote.RoleRevokePermission(ctx, &pb.AuthRoleRevokePermissionRequest{Role: role, Key: key})
-	return (*AuthRoleRevokePermissionResponse)(resp), rpctypes.Error(err)
+	return (*AuthRoleRevokePermissionResponse)(resp), toErr(ctx, err)
 }
 }
 
 
 func (auth *auth) RoleDelete(ctx context.Context, role string) (*AuthRoleDeleteResponse, error) {
 func (auth *auth) RoleDelete(ctx context.Context, role string) (*AuthRoleDeleteResponse, error) {
 	resp, err := auth.remote.RoleDelete(ctx, &pb.AuthRoleDeleteRequest{Role: role})
 	resp, err := auth.remote.RoleDelete(ctx, &pb.AuthRoleDeleteRequest{Role: role})
-	return (*AuthRoleDeleteResponse)(resp), rpctypes.Error(err)
+	return (*AuthRoleDeleteResponse)(resp), toErr(ctx, err)
 }
 }
 
 
 func StrToPermissionType(s string) (PermissionType, error) {
 func StrToPermissionType(s string) (PermissionType, error) {
@@ -191,7 +190,7 @@ type authenticator struct {
 
 
 func (auth *authenticator) authenticate(ctx context.Context, name string, password string) (*AuthenticateResponse, error) {
 func (auth *authenticator) authenticate(ctx context.Context, name string, password string) (*AuthenticateResponse, error) {
 	resp, err := auth.remote.Authenticate(ctx, &pb.AuthenticateRequest{Name: name, Password: password})
 	resp, err := auth.remote.Authenticate(ctx, &pb.AuthenticateRequest{Name: name, Password: password})
-	return (*AuthenticateResponse)(resp), rpctypes.Error(err)
+	return (*AuthenticateResponse)(resp), toErr(ctx, err)
 }
 }
 
 
 func (auth *authenticator) close() {
 func (auth *authenticator) close() {

+ 64 - 0
clientv3/balancer.go

@@ -0,0 +1,64 @@
+// Copyright 2016 The etcd Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package clientv3
+
+import (
+	"net/url"
+	"strings"
+	"sync/atomic"
+
+	"golang.org/x/net/context"
+	"google.golang.org/grpc"
+)
+
+// simpleBalancer does the bare minimum to expose multiple eps
+// to the grpc reconnection code path
+type simpleBalancer struct {
+	// eps are the client's endpoints stripped of any URL scheme
+	eps     []string
+	ch      chan []grpc.Address
+	numGets uint32
+}
+
+func newSimpleBalancer(eps []string) grpc.Balancer {
+	ch := make(chan []grpc.Address, 1)
+	addrs := make([]grpc.Address, len(eps))
+	for i := range eps {
+		addrs[i].Addr = getHost(eps[i])
+	}
+	ch <- addrs
+	return &simpleBalancer{eps: eps, ch: ch}
+}
+
+func (b *simpleBalancer) Start(target string) error        { return nil }
+func (b *simpleBalancer) Up(addr grpc.Address) func(error) { return func(error) {} }
+func (b *simpleBalancer) Get(ctx context.Context, opts grpc.BalancerGetOptions) (grpc.Address, func(), error) {
+	v := atomic.AddUint32(&b.numGets, 1)
+	ep := b.eps[v%uint32(len(b.eps))]
+	return grpc.Address{Addr: getHost(ep)}, func() {}, nil
+}
+func (b *simpleBalancer) Notify() <-chan []grpc.Address { return b.ch }
+func (b *simpleBalancer) Close() error {
+	close(b.ch)
+	return nil
+}
+
+func getHost(ep string) string {
+	url, uerr := url.Parse(ep)
+	if uerr != nil || !strings.Contains(ep, "://") {
+		return ep
+	}
+	return url.Host
+}

+ 88 - 216
clientv3/client.go

@@ -23,13 +23,11 @@ import (
 	"net"
 	"net"
 	"net/url"
 	"net/url"
 	"strings"
 	"strings"
-	"sync"
 	"time"
 	"time"
 
 
 	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
 	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
 
 
 	"golang.org/x/net/context"
 	"golang.org/x/net/context"
-	"golang.org/x/time/rate"
 	"google.golang.org/grpc"
 	"google.golang.org/grpc"
 	"google.golang.org/grpc/credentials"
 	"google.golang.org/grpc/credentials"
 	"google.golang.org/grpc/metadata"
 	"google.golang.org/grpc/metadata"
@@ -37,9 +35,6 @@ import (
 
 
 var (
 var (
 	ErrNoAvailableEndpoints = errors.New("etcdclient: no available endpoints")
 	ErrNoAvailableEndpoints = errors.New("etcdclient: no available endpoints")
-
-	// minConnRetryWait is the minimum time between reconnects to avoid flooding
-	minConnRetryWait = time.Second
 )
 )
 
 
 // Client provides and manages an etcd v3 client session.
 // Client provides and manages an etcd v3 client session.
@@ -51,23 +46,13 @@ type Client struct {
 	Auth
 	Auth
 	Maintenance
 	Maintenance
 
 
-	conn   *grpc.ClientConn
-	cfg    Config
-	creds  *credentials.TransportAuthenticator
-	mu     sync.RWMutex // protects connection selection and error list
-	errors []error      // errors passed to retryConnection
+	conn  *grpc.ClientConn
+	cfg   Config
+	creds *credentials.TransportAuthenticator
 
 
 	ctx    context.Context
 	ctx    context.Context
 	cancel context.CancelFunc
 	cancel context.CancelFunc
 
 
-	// fields below are managed by connMonitor
-
-	// reconnc accepts writes which signal the client should reconnect
-	reconnc chan error
-	// newconnc is closed on successful connect and set to a fresh channel
-	newconnc    chan struct{}
-	lastConnErr error
-
 	// Username is a username for authentication
 	// Username is a username for authentication
 	Username string
 	Username string
 	// Password is a password for authentication
 	// Password is a password for authentication
@@ -76,9 +61,6 @@ type Client struct {
 
 
 // New creates a new etcdv3 client from a given configuration.
 // New creates a new etcdv3 client from a given configuration.
 func New(cfg Config) (*Client, error) {
 func New(cfg Config) (*Client, error) {
-	if cfg.retryDialer == nil {
-		cfg.retryDialer = dialEndpointList
-	}
 	if len(cfg.Endpoints) == 0 {
 	if len(cfg.Endpoints) == 0 {
 		return nil, ErrNoAvailableEndpoints
 		return nil, ErrNoAvailableEndpoints
 	}
 	}
@@ -101,39 +83,9 @@ func NewFromConfigFile(path string) (*Client, error) {
 }
 }
 
 
 // Close shuts down the client's etcd connections.
 // Close shuts down the client's etcd connections.
-func (c *Client) Close() (err error) {
-	c.mu.Lock()
-	defer c.mu.Unlock()
-
-	// acquire the cancel
-	if c.cancel == nil {
-		// already canceled
-		if c.lastConnErr != c.ctx.Err() {
-			err = c.lastConnErr
-		}
-		return
-	}
-	cancel := c.cancel
-	c.cancel = nil
-	c.mu.Unlock()
-
-	// close watcher and lease before terminating connection
-	// so they don't retry on a closed client
-	c.Watcher.Close()
-	c.Lease.Close()
-
-	// cancel reconnection loop
-	cancel()
-	c.mu.Lock()
-	connc := c.newconnc
-	c.mu.Unlock()
-	// connc on cancel() is left closed
-	<-connc
-	c.mu.Lock()
-	if c.lastConnErr != c.ctx.Err() {
-		err = c.lastConnErr
-	}
-	return
+func (c *Client) Close() error {
+	c.cancel()
+	return toErr(c.ctx, c.conn.Close())
 }
 }
 
 
 // Ctx is a context for "out of band" messages (e.g., for sending
 // Ctx is a context for "out of band" messages (e.g., for sending
@@ -144,15 +96,6 @@ func (c *Client) Ctx() context.Context { return c.ctx }
 // Endpoints lists the registered endpoints for the client.
 // Endpoints lists the registered endpoints for the client.
 func (c *Client) Endpoints() []string { return c.cfg.Endpoints }
 func (c *Client) Endpoints() []string { return c.cfg.Endpoints }
 
 
-// Errors returns all errors that have been observed since called last.
-func (c *Client) Errors() (errs []error) {
-	c.mu.Lock()
-	defer c.mu.Unlock()
-	errs = c.errors
-	c.errors = nil
-	return errs
-}
-
 type authTokenCredential struct {
 type authTokenCredential struct {
 	token string
 	token string
 }
 }
@@ -167,43 +110,70 @@ func (cred authTokenCredential) GetRequestMetadata(ctx context.Context, s ...str
 	}, nil
 	}, nil
 }
 }
 
 
-// Dial establishes a connection for a given endpoint using the client's config
+func (c *Client) dialTarget(endpoint string) (proto string, host string, creds *credentials.TransportAuthenticator) {
+	proto = "tcp"
+	host = endpoint
+	creds = c.creds
+	url, uerr := url.Parse(endpoint)
+	if uerr != nil || !strings.Contains(endpoint, "://") {
+		return
+	}
+	// strip scheme:// prefix since grpc dials by host
+	host = url.Host
+	switch url.Scheme {
+	case "unix":
+		proto = "unix"
+	case "http":
+		creds = nil
+	case "https":
+		if creds != nil {
+			break
+		}
+		tlsconfig := &tls.Config{}
+		emptyCreds := credentials.NewTLS(tlsconfig)
+		creds = &emptyCreds
+	default:
+		return "", "", nil
+	}
+	return
+}
+
+// Dial connects to a single endpoint using the client's config.
 func (c *Client) Dial(endpoint string) (*grpc.ClientConn, error) {
 func (c *Client) Dial(endpoint string) (*grpc.ClientConn, error) {
+	return c.dial(endpoint)
+}
+
+func (c *Client) dial(endpoint string, dopts ...grpc.DialOption) (*grpc.ClientConn, error) {
 	opts := []grpc.DialOption{
 	opts := []grpc.DialOption{
 		grpc.WithBlock(),
 		grpc.WithBlock(),
 		grpc.WithTimeout(c.cfg.DialTimeout),
 		grpc.WithTimeout(c.cfg.DialTimeout),
 	}
 	}
+	opts = append(opts, dopts...)
+
+	// grpc issues TLS cert checks using the string passed into dial so
+	// that string must be the host. To recover the full scheme://host URL,
+	// have a map from hosts to the original endpoint.
+	host2ep := make(map[string]string)
+	for i := range c.cfg.Endpoints {
+		_, host, _ := c.dialTarget(c.cfg.Endpoints[i])
+		host2ep[host] = c.cfg.Endpoints[i]
+	}
 
 
-	proto := "tcp"
-	creds := c.creds
-	if url, uerr := url.Parse(endpoint); uerr == nil && strings.Contains(endpoint, "://") {
-		switch url.Scheme {
-		case "unix":
-			proto = "unix"
-		case "http":
-			creds = nil
-		case "https":
-			if creds == nil {
-				tlsconfig := &tls.Config{InsecureSkipVerify: true}
-				emptyCreds := credentials.NewTLS(tlsconfig)
-				creds = &emptyCreds
-			}
-		default:
-			return nil, fmt.Errorf("unknown scheme %q for %q", url.Scheme, endpoint)
+	f := func(host string, t time.Duration) (net.Conn, error) {
+		proto, host, _ := c.dialTarget(host2ep[host])
+		if proto == "" {
+			return nil, fmt.Errorf("unknown scheme for %q", endpoint)
 		}
 		}
-		// strip scheme:// prefix since grpc dials by host
-		endpoint = url.Host
-	}
-	f := func(a string, t time.Duration) (net.Conn, error) {
 		select {
 		select {
 		case <-c.ctx.Done():
 		case <-c.ctx.Done():
 			return nil, c.ctx.Err()
 			return nil, c.ctx.Err()
 		default:
 		default:
 		}
 		}
-		return net.DialTimeout(proto, a, t)
+		return net.DialTimeout(proto, host, t)
 	}
 	}
 	opts = append(opts, grpc.WithDialer(f))
 	opts = append(opts, grpc.WithDialer(f))
 
 
+	_, host, creds := c.dialTarget(endpoint)
 	if creds != nil {
 	if creds != nil {
 		opts = append(opts, grpc.WithTransportCredentials(*creds))
 		opts = append(opts, grpc.WithTransportCredentials(*creds))
 	} else {
 	} else {
@@ -211,7 +181,7 @@ func (c *Client) Dial(endpoint string) (*grpc.ClientConn, error) {
 	}
 	}
 
 
 	if c.Username != "" && c.Password != "" {
 	if c.Username != "" && c.Password != "" {
-		auth, err := newAuthenticator(endpoint, opts)
+		auth, err := newAuthenticator(host, opts)
 		if err != nil {
 		if err != nil {
 			return nil, err
 			return nil, err
 		}
 		}
@@ -225,7 +195,7 @@ func (c *Client) Dial(endpoint string) (*grpc.ClientConn, error) {
 		opts = append(opts, grpc.WithPerRPCCredentials(authTokenCredential{token: resp.Token}))
 		opts = append(opts, grpc.WithPerRPCCredentials(authTokenCredential{token: resp.Token}))
 	}
 	}
 
 
-	conn, err := grpc.Dial(endpoint, opts...)
+	conn, err := grpc.Dial(host, opts...)
 	if err != nil {
 	if err != nil {
 		return nil, err
 		return nil, err
 	}
 	}
@@ -241,7 +211,7 @@ func WithRequireLeader(ctx context.Context) context.Context {
 
 
 func newClient(cfg *Config) (*Client, error) {
 func newClient(cfg *Config) (*Client, error) {
 	if cfg == nil {
 	if cfg == nil {
-		cfg = &Config{retryDialer: dialEndpointList}
+		cfg = &Config{}
 	}
 	}
 	var creds *credentials.TransportAuthenticator
 	var creds *credentials.TransportAuthenticator
 	if cfg.TLS != nil {
 	if cfg.TLS != nil {
@@ -251,26 +221,24 @@ func newClient(cfg *Config) (*Client, error) {
 
 
 	// use a temporary skeleton client to bootstrap first connection
 	// use a temporary skeleton client to bootstrap first connection
 	ctx, cancel := context.WithCancel(context.TODO())
 	ctx, cancel := context.WithCancel(context.TODO())
-	conn, err := cfg.retryDialer(&Client{cfg: *cfg, creds: creds, ctx: ctx, Username: cfg.Username, Password: cfg.Password})
-	if err != nil {
-		return nil, err
-	}
 	client := &Client{
 	client := &Client{
-		conn:     conn,
-		cfg:      *cfg,
-		creds:    creds,
-		ctx:      ctx,
-		cancel:   cancel,
-		reconnc:  make(chan error, 1),
-		newconnc: make(chan struct{}),
+		conn:   nil,
+		cfg:    *cfg,
+		creds:  creds,
+		ctx:    ctx,
+		cancel: cancel,
 	}
 	}
-
 	if cfg.Username != "" && cfg.Password != "" {
 	if cfg.Username != "" && cfg.Password != "" {
 		client.Username = cfg.Username
 		client.Username = cfg.Username
 		client.Password = cfg.Password
 		client.Password = cfg.Password
 	}
 	}
 
 
-	go client.connMonitor()
+	b := newSimpleBalancer(cfg.Endpoints)
+	conn, err := client.dial(cfg.Endpoints[0], grpc.WithBalancer(b))
+	if err != nil {
+		return nil, err
+	}
+	client.conn = conn
 
 
 	client.Cluster = NewCluster(client)
 	client.Cluster = NewCluster(client)
 	client.KV = NewKV(client)
 	client.KV = NewKV(client)
@@ -289,126 +257,30 @@ func newClient(cfg *Config) (*Client, error) {
 }
 }
 
 
 // ActiveConnection returns the current in-use connection
 // ActiveConnection returns the current in-use connection
-func (c *Client) ActiveConnection() *grpc.ClientConn {
-	c.mu.RLock()
-	defer c.mu.RUnlock()
-	if c.conn == nil {
-		panic("trying to return nil active connection")
-	}
-	return c.conn
-}
+func (c *Client) ActiveConnection() *grpc.ClientConn { return c.conn }
 
 
-// retryConnection establishes a new connection
-func (c *Client) retryConnection(err error) {
-	oldconn := c.conn
-
-	// return holding lock so old connection can be cleaned up in this defer
-	defer func() {
-		if oldconn != nil {
-			oldconn.Close()
-			if st, _ := oldconn.State(); st != grpc.Shutdown {
-				// wait so grpc doesn't leak sleeping goroutines
-				oldconn.WaitForStateChange(context.Background(), st)
-			}
-		}
-		c.mu.Unlock()
-	}()
-
-	c.mu.Lock()
-	if err != nil {
-		c.errors = append(c.errors, err)
-	}
-	if c.cancel == nil {
-		// client has called Close() so don't try to dial out
-		return
-	}
-	c.mu.Unlock()
-
-	nc, dialErr := c.cfg.retryDialer(c)
-
-	c.mu.Lock()
-	if nc != nil {
-		c.conn = nc
-	}
-	if dialErr != nil {
-		c.errors = append(c.errors, dialErr)
-	}
-	c.lastConnErr = dialErr
-}
-
-// connStartRetry schedules a reconnect if one is not already running
-func (c *Client) connStartRetry(err error) {
-	c.mu.Lock()
-	ch := c.reconnc
-	defer c.mu.Unlock()
-	select {
-	case ch <- err:
-	default:
-	}
-}
-
-// connWait waits for a reconnect to be processed
-func (c *Client) connWait(ctx context.Context, err error) (*grpc.ClientConn, error) {
-	c.mu.RLock()
-	ch := c.newconnc
-	c.mu.RUnlock()
-	c.connStartRetry(err)
-	select {
-	case <-ctx.Done():
-		return nil, ctx.Err()
-	case <-ch:
+// isHaltErr returns true if the given error and context indicate no forward
+// progress can be made, even after reconnecting.
+func isHaltErr(ctx context.Context, err error) bool {
+	if ctx != nil && ctx.Err() != nil {
+		return true
 	}
 	}
-	c.mu.RLock()
-	defer c.mu.RUnlock()
-	if c.cancel == nil {
-		return c.conn, rpctypes.ErrConnClosed
+	if err == nil {
+		return false
 	}
 	}
-	return c.conn, c.lastConnErr
+	return strings.HasPrefix(grpc.ErrorDesc(err), "etcdserver: ") ||
+		strings.Contains(err.Error(), grpc.ErrClientConnClosing.Error())
 }
 }
 
 
-// connMonitor monitors the connection and handles retries
-func (c *Client) connMonitor() {
-	var err error
-
-	defer func() {
-		c.retryConnection(c.ctx.Err())
-		close(c.newconnc)
-	}()
-
-	limiter := rate.NewLimiter(rate.Every(minConnRetryWait), 1)
-	for limiter.Wait(c.ctx) == nil {
-		select {
-		case err = <-c.reconnc:
-		case <-c.ctx.Done():
-			return
-		}
-		c.retryConnection(err)
-		c.mu.Lock()
-		close(c.newconnc)
-		c.newconnc = make(chan struct{})
-		c.reconnc = make(chan error, 1)
-		c.mu.Unlock()
+func toErr(ctx context.Context, err error) error {
+	if err == nil {
+		return nil
 	}
 	}
-}
-
-// dialEndpointList attempts to connect to each endpoint in order until a
-// connection is established.
-func dialEndpointList(c *Client) (*grpc.ClientConn, error) {
-	var err error
-	for _, ep := range c.Endpoints() {
-		conn, curErr := c.Dial(ep)
-		if curErr != nil {
-			err = curErr
-		} else {
-			return conn, nil
-		}
+	err = rpctypes.Error(err)
+	if ctx.Err() != nil && strings.Contains(err.Error(), "context") {
+		err = ctx.Err()
+	} else if strings.Contains(err.Error(), grpc.ErrClientConnClosing.Error()) {
+		err = grpc.ErrClientConnClosing
 	}
 	}
-	return nil, err
-}
-
-// isHaltErr returns true if the given error and context indicate no forward
-// progress can be made, even after reconnecting.
-func isHaltErr(ctx context.Context, err error) bool {
-	isRPCError := strings.HasPrefix(grpc.ErrorDesc(err), "etcdserver: ")
-	return isRPCError || ctx.Err() != nil || err == rpctypes.ErrConnClosed
+	return err
 }
 }

+ 11 - 39
clientv3/cluster.go

@@ -15,10 +15,8 @@
 package clientv3
 package clientv3
 
 
 import (
 import (
-	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 	"golang.org/x/net/context"
 	"golang.org/x/net/context"
-	"google.golang.org/grpc"
 )
 )
 
 
 type (
 type (
@@ -44,62 +42,47 @@ type Cluster interface {
 }
 }
 
 
 type cluster struct {
 type cluster struct {
-	rc     *remoteClient
 	remote pb.ClusterClient
 	remote pb.ClusterClient
 }
 }
 
 
 func NewCluster(c *Client) Cluster {
 func NewCluster(c *Client) Cluster {
-	ret := &cluster{}
-	f := func(conn *grpc.ClientConn) { ret.remote = pb.NewClusterClient(conn) }
-	ret.rc = newRemoteClient(c, f)
-	return ret
+	return &cluster{remote: pb.NewClusterClient(c.conn)}
 }
 }
 
 
 func (c *cluster) MemberAdd(ctx context.Context, peerAddrs []string) (*MemberAddResponse, error) {
 func (c *cluster) MemberAdd(ctx context.Context, peerAddrs []string) (*MemberAddResponse, error) {
 	r := &pb.MemberAddRequest{PeerURLs: peerAddrs}
 	r := &pb.MemberAddRequest{PeerURLs: peerAddrs}
-	resp, err := c.getRemote().MemberAdd(ctx, r)
+	resp, err := c.remote.MemberAdd(ctx, r)
 	if err == nil {
 	if err == nil {
 		return (*MemberAddResponse)(resp), nil
 		return (*MemberAddResponse)(resp), nil
 	}
 	}
-
 	if isHaltErr(ctx, err) {
 	if isHaltErr(ctx, err) {
-		return nil, rpctypes.Error(err)
+		return nil, toErr(ctx, err)
 	}
 	}
-
-	c.rc.reconnect(err)
-	return nil, rpctypes.Error(err)
+	return nil, toErr(ctx, err)
 }
 }
 
 
 func (c *cluster) MemberRemove(ctx context.Context, id uint64) (*MemberRemoveResponse, error) {
 func (c *cluster) MemberRemove(ctx context.Context, id uint64) (*MemberRemoveResponse, error) {
 	r := &pb.MemberRemoveRequest{ID: id}
 	r := &pb.MemberRemoveRequest{ID: id}
-	resp, err := c.getRemote().MemberRemove(ctx, r)
+	resp, err := c.remote.MemberRemove(ctx, r)
 	if err == nil {
 	if err == nil {
 		return (*MemberRemoveResponse)(resp), nil
 		return (*MemberRemoveResponse)(resp), nil
 	}
 	}
-
 	if isHaltErr(ctx, err) {
 	if isHaltErr(ctx, err) {
-		return nil, rpctypes.Error(err)
+		return nil, toErr(ctx, err)
 	}
 	}
-
-	c.rc.reconnect(err)
-	return nil, rpctypes.Error(err)
+	return nil, toErr(ctx, err)
 }
 }
 
 
 func (c *cluster) MemberUpdate(ctx context.Context, id uint64, peerAddrs []string) (*MemberUpdateResponse, error) {
 func (c *cluster) MemberUpdate(ctx context.Context, id uint64, peerAddrs []string) (*MemberUpdateResponse, error) {
 	// it is safe to retry on update.
 	// it is safe to retry on update.
 	for {
 	for {
 		r := &pb.MemberUpdateRequest{ID: id, PeerURLs: peerAddrs}
 		r := &pb.MemberUpdateRequest{ID: id, PeerURLs: peerAddrs}
-		resp, err := c.getRemote().MemberUpdate(ctx, r)
+		resp, err := c.remote.MemberUpdate(ctx, r)
 		if err == nil {
 		if err == nil {
 			return (*MemberUpdateResponse)(resp), nil
 			return (*MemberUpdateResponse)(resp), nil
 		}
 		}
-
 		if isHaltErr(ctx, err) {
 		if isHaltErr(ctx, err) {
-			return nil, rpctypes.Error(err)
-		}
-
-		if err = c.rc.reconnectWait(ctx, err); err != nil {
-			return nil, rpctypes.Error(err)
+			return nil, toErr(ctx, err)
 		}
 		}
 	}
 	}
 }
 }
@@ -107,23 +90,12 @@ func (c *cluster) MemberUpdate(ctx context.Context, id uint64, peerAddrs []strin
 func (c *cluster) MemberList(ctx context.Context) (*MemberListResponse, error) {
 func (c *cluster) MemberList(ctx context.Context) (*MemberListResponse, error) {
 	// it is safe to retry on list.
 	// it is safe to retry on list.
 	for {
 	for {
-		resp, err := c.getRemote().MemberList(ctx, &pb.MemberListRequest{})
+		resp, err := c.remote.MemberList(ctx, &pb.MemberListRequest{})
 		if err == nil {
 		if err == nil {
 			return (*MemberListResponse)(resp), nil
 			return (*MemberListResponse)(resp), nil
 		}
 		}
-
 		if isHaltErr(ctx, err) {
 		if isHaltErr(ctx, err) {
-			return nil, rpctypes.Error(err)
-		}
-
-		if err = c.rc.reconnectWait(ctx, err); err != nil {
-			return nil, rpctypes.Error(err)
+			return nil, toErr(ctx, err)
 		}
 		}
 	}
 	}
 }
 }
-
-func (c *cluster) getRemote() pb.ClusterClient {
-	c.rc.mu.Lock()
-	defer c.rc.mu.Unlock()
-	return c.remote
-}

+ 0 - 8
clientv3/config.go

@@ -22,20 +22,12 @@ import (
 
 
 	"github.com/coreos/etcd/pkg/tlsutil"
 	"github.com/coreos/etcd/pkg/tlsutil"
 	"github.com/ghodss/yaml"
 	"github.com/ghodss/yaml"
-	"google.golang.org/grpc"
 )
 )
 
 
-// EndpointDialer is a policy for choosing which endpoint to dial next
-type EndpointDialer func(*Client) (*grpc.ClientConn, error)
-
 type Config struct {
 type Config struct {
 	// Endpoints is a list of URLs
 	// Endpoints is a list of URLs
 	Endpoints []string
 	Endpoints []string
 
 
-	// retryDialer chooses the next endpoint to use
-	// keep private until the grpc rebalancer is sorted out
-	retryDialer EndpointDialer
-
 	// DialTimeout is the timeout for failing to establish a connection.
 	// DialTimeout is the timeout for failing to establish a connection.
 	DialTimeout time.Duration
 	DialTimeout time.Duration
 
 

+ 12 - 21
clientv3/integration/kv_test.go

@@ -27,6 +27,7 @@ import (
 	"github.com/coreos/etcd/mvcc/mvccpb"
 	"github.com/coreos/etcd/mvcc/mvccpb"
 	"github.com/coreos/etcd/pkg/testutil"
 	"github.com/coreos/etcd/pkg/testutil"
 	"golang.org/x/net/context"
 	"golang.org/x/net/context"
+	"google.golang.org/grpc"
 )
 )
 
 
 func TestKVPutError(t *testing.T) {
 func TestKVPutError(t *testing.T) {
@@ -299,8 +300,8 @@ func TestKVGetErrConnClosed(t *testing.T) {
 	go func() {
 	go func() {
 		defer close(donec)
 		defer close(donec)
 		_, err := kv.Get(context.TODO(), "foo")
 		_, err := kv.Get(context.TODO(), "foo")
-		if err != nil && err != rpctypes.ErrConnClosed {
-			t.Fatalf("expected %v, got %v", rpctypes.ErrConnClosed, err)
+		if err != nil && err != grpc.ErrClientConnClosing {
+			t.Fatalf("expected %v, got %v", grpc.ErrClientConnClosing, err)
 		}
 		}
 	}()
 	}()
 
 
@@ -331,8 +332,8 @@ func TestKVNewAfterClose(t *testing.T) {
 	donec := make(chan struct{})
 	donec := make(chan struct{})
 	go func() {
 	go func() {
 		kv := clientv3.NewKV(cli)
 		kv := clientv3.NewKV(cli)
-		if _, err := kv.Get(context.TODO(), "foo"); err != rpctypes.ErrConnClosed {
-			t.Fatalf("expected %v, got %v", rpctypes.ErrConnClosed, err)
+		if _, err := kv.Get(context.TODO(), "foo"); err != grpc.ErrClientConnClosing {
+			t.Fatalf("expected %v, got %v", grpc.ErrClientConnClosing, err)
 		}
 		}
 		close(donec)
 		close(donec)
 	}()
 	}()
@@ -579,11 +580,10 @@ func TestKVPutFailGetRetry(t *testing.T) {
 	defer clus.Terminate(t)
 	defer clus.Terminate(t)
 
 
 	kv := clientv3.NewKV(clus.Client(0))
 	kv := clientv3.NewKV(clus.Client(0))
-	ctx := context.TODO()
-
 	clus.Members[0].Stop(t)
 	clus.Members[0].Stop(t)
-	<-clus.Members[0].StopNotify()
 
 
+	ctx, cancel := context.WithTimeout(context.TODO(), time.Second)
+	defer cancel()
 	_, err := kv.Put(ctx, "foo", "bar")
 	_, err := kv.Put(ctx, "foo", "bar")
 	if err == nil {
 	if err == nil {
 		t.Fatalf("got success on disconnected put, wanted error")
 		t.Fatalf("got success on disconnected put, wanted error")
@@ -592,7 +592,7 @@ func TestKVPutFailGetRetry(t *testing.T) {
 	donec := make(chan struct{})
 	donec := make(chan struct{})
 	go func() {
 	go func() {
 		// Get will fail, but reconnect will trigger
 		// Get will fail, but reconnect will trigger
-		gresp, gerr := kv.Get(ctx, "foo")
+		gresp, gerr := kv.Get(context.TODO(), "foo")
 		if gerr != nil {
 		if gerr != nil {
 			t.Fatal(gerr)
 			t.Fatal(gerr)
 		}
 		}
@@ -642,20 +642,11 @@ func TestKVPutStoppedServerAndClose(t *testing.T) {
 	defer clus.Terminate(t)
 	defer clus.Terminate(t)
 	cli := clus.Client(0)
 	cli := clus.Client(0)
 	clus.Members[0].Stop(t)
 	clus.Members[0].Stop(t)
+	ctx, cancel := context.WithTimeout(context.TODO(), time.Second)
 	// this Put fails and triggers an asynchronous connection retry
 	// this Put fails and triggers an asynchronous connection retry
-	_, err := cli.Put(context.TODO(), "abc", "123")
-	if err == nil ||
-		(!strings.Contains(err.Error(), "connection is closing") &&
-			!strings.Contains(err.Error(), "transport is closing")) {
-		t.Fatal(err)
-	}
-
-	// wait some so the client closes with the retry in-flight
-	time.Sleep(time.Second)
-
-	// get the timeout
-	clus.TakeClient(0)
-	if err := cli.Close(); err == nil || !strings.Contains(err.Error(), "timed out") {
+	_, err := cli.Put(ctx, "abc", "123")
+	cancel()
+	if !strings.Contains(err.Error(), "context deadline") {
 		t.Fatal(err)
 		t.Fatal(err)
 	}
 	}
 }
 }

+ 7 - 6
clientv3/integration/lease_test.go

@@ -23,6 +23,7 @@ import (
 	"github.com/coreos/etcd/integration"
 	"github.com/coreos/etcd/integration"
 	"github.com/coreos/etcd/pkg/testutil"
 	"github.com/coreos/etcd/pkg/testutil"
 	"golang.org/x/net/context"
 	"golang.org/x/net/context"
+	"google.golang.org/grpc"
 )
 )
 
 
 func TestLeaseNotFoundError(t *testing.T) {
 func TestLeaseNotFoundError(t *testing.T) {
@@ -262,8 +263,8 @@ func TestLeaseGrantErrConnClosed(t *testing.T) {
 	go func() {
 	go func() {
 		defer close(donec)
 		defer close(donec)
 		_, err := le.Grant(context.TODO(), 5)
 		_, err := le.Grant(context.TODO(), 5)
-		if err != nil && err != rpctypes.ErrConnClosed {
-			t.Fatalf("expected %v, got %v", rpctypes.ErrConnClosed, err)
+		if err != nil && err != grpc.ErrClientConnClosing {
+			t.Fatalf("expected %v, got %v", grpc.ErrClientConnClosing, err)
 		}
 		}
 	}()
 	}()
 
 
@@ -294,8 +295,8 @@ func TestLeaseGrantNewAfterClose(t *testing.T) {
 	donec := make(chan struct{})
 	donec := make(chan struct{})
 	go func() {
 	go func() {
 		le := clientv3.NewLease(cli)
 		le := clientv3.NewLease(cli)
-		if _, err := le.Grant(context.TODO(), 5); err != rpctypes.ErrConnClosed {
-			t.Fatalf("expected %v, got %v", rpctypes.ErrConnClosed, err)
+		if _, err := le.Grant(context.TODO(), 5); err != grpc.ErrClientConnClosing {
+			t.Fatalf("expected %v, got %v", grpc.ErrClientConnClosing, err)
 		}
 		}
 		close(donec)
 		close(donec)
 	}()
 	}()
@@ -327,8 +328,8 @@ func TestLeaseRevokeNewAfterClose(t *testing.T) {
 
 
 	donec := make(chan struct{})
 	donec := make(chan struct{})
 	go func() {
 	go func() {
-		if _, err := le.Revoke(context.TODO(), leaseID); err != rpctypes.ErrConnClosed {
-			t.Fatalf("expected %v, got %v", rpctypes.ErrConnClosed, err)
+		if _, err := le.Revoke(context.TODO(), leaseID); err != grpc.ErrClientConnClosing {
+			t.Fatalf("expected %v, got %v", grpc.ErrClientConnClosing, err)
 		}
 		}
 		close(donec)
 		close(donec)
 	}()
 	}()

+ 15 - 27
clientv3/integration/txn_test.go

@@ -58,57 +58,45 @@ func TestTxnWriteFail(t *testing.T) {
 	defer clus.Terminate(t)
 	defer clus.Terminate(t)
 
 
 	kv := clientv3.NewKV(clus.Client(0))
 	kv := clientv3.NewKV(clus.Client(0))
-	ctx := context.TODO()
 
 
 	clus.Members[0].Stop(t)
 	clus.Members[0].Stop(t)
-	<-clus.Members[0].StopNotify()
 
 
-	donec := make(chan struct{})
+	txnc, getc := make(chan struct{}), make(chan struct{})
 	go func() {
 	go func() {
+		ctx, cancel := context.WithTimeout(context.TODO(), time.Second)
+		defer cancel()
 		resp, err := kv.Txn(ctx).Then(clientv3.OpPut("foo", "bar")).Commit()
 		resp, err := kv.Txn(ctx).Then(clientv3.OpPut("foo", "bar")).Commit()
 		if err == nil {
 		if err == nil {
 			t.Fatalf("expected error, got response %v", resp)
 			t.Fatalf("expected error, got response %v", resp)
 		}
 		}
-		donec <- struct{}{}
+		close(txnc)
 	}()
 	}()
 
 
-	dialTimeout := 5 * time.Second
-	select {
-	case <-time.After(dialTimeout + time.Second):
-		t.Fatalf("timed out waiting for txn to fail")
-	case <-donec:
-		// don't restart cluster until txn errors out
-	}
-
 	go func() {
 	go func() {
-		// reconnect so terminate doesn't complain about double-close
-		clus.Members[0].Restart(t)
-		// wait for etcdserver to get established (CI races and get req times out)
-		time.Sleep(2 * time.Second)
-		donec <- struct{}{}
-
+		select {
+		case <-time.After(5 * time.Second):
+			t.Fatalf("timed out waiting for txn fail")
+		case <-txnc:
+		}
 		// and ensure the put didn't take
 		// and ensure the put didn't take
-		gresp, gerr := kv.Get(ctx, "foo")
+		gresp, gerr := clus.Client(1).Get(context.TODO(), "foo")
 		if gerr != nil {
 		if gerr != nil {
 			t.Fatal(gerr)
 			t.Fatal(gerr)
 		}
 		}
 		if len(gresp.Kvs) != 0 {
 		if len(gresp.Kvs) != 0 {
 			t.Fatalf("expected no keys, got %v", gresp.Kvs)
 			t.Fatalf("expected no keys, got %v", gresp.Kvs)
 		}
 		}
-		donec <- struct{}{}
+		close(getc)
 	}()
 	}()
 
 
-	select {
-	case <-time.After(5 * time.Second):
-		t.Fatalf("timed out waiting for restart")
-	case <-donec:
-	}
-
 	select {
 	select {
 	case <-time.After(5 * time.Second):
 	case <-time.After(5 * time.Second):
 		t.Fatalf("timed out waiting for get")
 		t.Fatalf("timed out waiting for get")
-	case <-donec:
+	case <-getc:
 	}
 	}
+
+	// reconnect so terminate doesn't complain about double-close
+	clus.Members[0].Restart(t)
 }
 }
 
 
 func TestTxnReadRetry(t *testing.T) {
 func TestTxnReadRetry(t *testing.T) {

+ 23 - 18
clientv3/integration/watch_test.go

@@ -16,6 +16,7 @@ package integration
 
 
 import (
 import (
 	"fmt"
 	"fmt"
+	"math/rand"
 	"reflect"
 	"reflect"
 	"sort"
 	"sort"
 	"testing"
 	"testing"
@@ -28,16 +29,19 @@ import (
 	mvccpb "github.com/coreos/etcd/mvcc/mvccpb"
 	mvccpb "github.com/coreos/etcd/mvcc/mvccpb"
 	"github.com/coreos/etcd/pkg/testutil"
 	"github.com/coreos/etcd/pkg/testutil"
 	"golang.org/x/net/context"
 	"golang.org/x/net/context"
+	"google.golang.org/grpc"
 )
 )
 
 
 type watcherTest func(*testing.T, *watchctx)
 type watcherTest func(*testing.T, *watchctx)
 
 
 type watchctx struct {
 type watchctx struct {
-	clus    *integration.ClusterV3
-	w       clientv3.Watcher
-	wclient *clientv3.Client
-	kv      clientv3.KV
-	ch      clientv3.WatchChan
+	clus          *integration.ClusterV3
+	w             clientv3.Watcher
+	wclient       *clientv3.Client
+	kv            clientv3.KV
+	wclientMember int
+	kvMember      int
+	ch            clientv3.WatchChan
 }
 }
 
 
 func runWatchTest(t *testing.T, f watcherTest) {
 func runWatchTest(t *testing.T, f watcherTest) {
@@ -46,18 +50,20 @@ func runWatchTest(t *testing.T, f watcherTest) {
 	clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
 	clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
 	defer clus.Terminate(t)
 	defer clus.Terminate(t)
 
 
-	wclient := clus.RandClient()
+	wclientMember := rand.Intn(3)
+	wclient := clus.Client(wclientMember)
 	w := clientv3.NewWatcher(wclient)
 	w := clientv3.NewWatcher(wclient)
 	defer w.Close()
 	defer w.Close()
 	// select a different client from wclient so puts succeed if
 	// select a different client from wclient so puts succeed if
 	// a test knocks out the watcher client
 	// a test knocks out the watcher client
-	kvclient := clus.RandClient()
-	for kvclient == wclient {
-		kvclient = clus.RandClient()
+	kvMember := rand.Intn(3)
+	for kvMember == wclientMember {
+		kvMember = rand.Intn(3)
 	}
 	}
+	kvclient := clus.Client(kvMember)
 	kv := clientv3.NewKV(kvclient)
 	kv := clientv3.NewKV(kvclient)
 
 
-	wctx := &watchctx{clus, w, wclient, kv, nil}
+	wctx := &watchctx{clus, w, wclient, kv, wclientMember, kvMember, nil}
 	f(t, wctx)
 	f(t, wctx)
 }
 }
 
 
@@ -185,7 +191,7 @@ func testWatchReconnRequest(t *testing.T, wctx *watchctx) {
 		defer close(donec)
 		defer close(donec)
 		// take down watcher connection
 		// take down watcher connection
 		for {
 		for {
-			wctx.wclient.ActiveConnection().Close()
+			wctx.clus.Members[wctx.wclientMember].DropConnections()
 			select {
 			select {
 			case <-timer:
 			case <-timer:
 				// spinning on close may live lock reconnection
 				// spinning on close may live lock reconnection
@@ -219,8 +225,7 @@ func testWatchReconnInit(t *testing.T, wctx *watchctx) {
 	if wctx.ch = wctx.w.Watch(context.TODO(), "a"); wctx.ch == nil {
 	if wctx.ch = wctx.w.Watch(context.TODO(), "a"); wctx.ch == nil {
 		t.Fatalf("expected non-nil channel")
 		t.Fatalf("expected non-nil channel")
 	}
 	}
-	// take down watcher connection
-	wctx.wclient.ActiveConnection().Close()
+	wctx.clus.Members[wctx.wclientMember].DropConnections()
 	// watcher should recover
 	// watcher should recover
 	putAndWatch(t, wctx, "a", "a")
 	putAndWatch(t, wctx, "a", "a")
 }
 }
@@ -237,7 +242,7 @@ func testWatchReconnRunning(t *testing.T, wctx *watchctx) {
 	}
 	}
 	putAndWatch(t, wctx, "a", "a")
 	putAndWatch(t, wctx, "a", "a")
 	// take down watcher connection
 	// take down watcher connection
-	wctx.wclient.ActiveConnection().Close()
+	wctx.clus.Members[wctx.wclientMember].DropConnections()
 	// watcher should recover
 	// watcher should recover
 	putAndWatch(t, wctx, "a", "b")
 	putAndWatch(t, wctx, "a", "b")
 }
 }
@@ -572,8 +577,8 @@ func TestWatchErrConnClosed(t *testing.T) {
 	go func() {
 	go func() {
 		defer close(donec)
 		defer close(donec)
 		wc.Watch(context.TODO(), "foo")
 		wc.Watch(context.TODO(), "foo")
-		if err := wc.Close(); err != nil && err != rpctypes.ErrConnClosed {
-			t.Fatalf("expected %v, got %v", rpctypes.ErrConnClosed, err)
+		if err := wc.Close(); err != nil && err != grpc.ErrClientConnClosing {
+			t.Fatalf("expected %v, got %v", grpc.ErrClientConnClosing, err)
 		}
 		}
 	}()
 	}()
 
 
@@ -605,8 +610,8 @@ func TestWatchAfterClose(t *testing.T) {
 	go func() {
 	go func() {
 		wc := clientv3.NewWatcher(cli)
 		wc := clientv3.NewWatcher(cli)
 		wc.Watch(context.TODO(), "foo")
 		wc.Watch(context.TODO(), "foo")
-		if err := wc.Close(); err != nil && err != rpctypes.ErrConnClosed {
-			t.Fatalf("expected %v, got %v", rpctypes.ErrConnClosed, err)
+		if err := wc.Close(); err != nil && err != grpc.ErrClientConnClosing {
+			t.Fatalf("expected %v, got %v", grpc.ErrClientConnClosing, err)
 		}
 		}
 		close(donec)
 		close(donec)
 	}()
 	}()

+ 13 - 46
clientv3/kv.go

@@ -15,10 +15,8 @@
 package clientv3
 package clientv3
 
 
 import (
 import (
-	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 	"golang.org/x/net/context"
 	"golang.org/x/net/context"
-	"google.golang.org/grpc"
 )
 )
 
 
 type (
 type (
@@ -78,47 +76,33 @@ func (op OpResponse) Get() *GetResponse    { return op.get }
 func (op OpResponse) Del() *DeleteResponse { return op.del }
 func (op OpResponse) Del() *DeleteResponse { return op.del }
 
 
 type kv struct {
 type kv struct {
-	rc     *remoteClient
 	remote pb.KVClient
 	remote pb.KVClient
 }
 }
 
 
 func NewKV(c *Client) KV {
 func NewKV(c *Client) KV {
-	ret := &kv{}
-	f := func(conn *grpc.ClientConn) { ret.remote = pb.NewKVClient(conn) }
-	ret.rc = newRemoteClient(c, f)
-	return ret
+	return &kv{remote: pb.NewKVClient(c.conn)}
 }
 }
 
 
 func (kv *kv) Put(ctx context.Context, key, val string, opts ...OpOption) (*PutResponse, error) {
 func (kv *kv) Put(ctx context.Context, key, val string, opts ...OpOption) (*PutResponse, error) {
 	r, err := kv.Do(ctx, OpPut(key, val, opts...))
 	r, err := kv.Do(ctx, OpPut(key, val, opts...))
-	return r.put, rpctypes.Error(err)
+	return r.put, toErr(ctx, err)
 }
 }
 
 
 func (kv *kv) Get(ctx context.Context, key string, opts ...OpOption) (*GetResponse, error) {
 func (kv *kv) Get(ctx context.Context, key string, opts ...OpOption) (*GetResponse, error) {
 	r, err := kv.Do(ctx, OpGet(key, opts...))
 	r, err := kv.Do(ctx, OpGet(key, opts...))
-	return r.get, rpctypes.Error(err)
+	return r.get, toErr(ctx, err)
 }
 }
 
 
 func (kv *kv) Delete(ctx context.Context, key string, opts ...OpOption) (*DeleteResponse, error) {
 func (kv *kv) Delete(ctx context.Context, key string, opts ...OpOption) (*DeleteResponse, error) {
 	r, err := kv.Do(ctx, OpDelete(key, opts...))
 	r, err := kv.Do(ctx, OpDelete(key, opts...))
-	return r.del, rpctypes.Error(err)
+	return r.del, toErr(ctx, err)
 }
 }
 
 
 func (kv *kv) Compact(ctx context.Context, rev int64) error {
 func (kv *kv) Compact(ctx context.Context, rev int64) error {
-	remote, err := kv.getRemote(ctx)
-	if err != nil {
-		return rpctypes.Error(err)
+	if _, err := kv.remote.Compact(ctx, &pb.CompactionRequest{Revision: rev}); err != nil {
+		return toErr(ctx, err)
 	}
 	}
-	defer kv.rc.release()
-	_, err = remote.Compact(ctx, &pb.CompactionRequest{Revision: rev})
-	if err == nil {
-		return nil
-	}
-	if isHaltErr(ctx, err) {
-		return rpctypes.Error(err)
-	}
-	kv.rc.reconnect(err)
-	return rpctypes.Error(err)
+	return nil
 }
 }
 
 
 func (kv *kv) Txn(ctx context.Context) Txn {
 func (kv *kv) Txn(ctx context.Context) Txn {
@@ -135,26 +119,17 @@ func (kv *kv) Do(ctx context.Context, op Op) (OpResponse, error) {
 			return resp, nil
 			return resp, nil
 		}
 		}
 		if isHaltErr(ctx, err) {
 		if isHaltErr(ctx, err) {
-			return resp, rpctypes.Error(err)
+			return resp, toErr(ctx, err)
 		}
 		}
 		// do not retry on modifications
 		// do not retry on modifications
 		if op.isWrite() {
 		if op.isWrite() {
-			kv.rc.reconnect(err)
-			return resp, rpctypes.Error(err)
-		}
-		if nerr := kv.rc.reconnectWait(ctx, err); nerr != nil {
-			return resp, rpctypes.Error(nerr)
+			return resp, toErr(ctx, err)
 		}
 		}
 	}
 	}
 }
 }
 
 
 func (kv *kv) do(ctx context.Context, op Op) (OpResponse, error) {
 func (kv *kv) do(ctx context.Context, op Op) (OpResponse, error) {
-	remote, err := kv.getRemote(ctx)
-	if err != nil {
-		return OpResponse{}, err
-	}
-	defer kv.rc.release()
-
+	var err error
 	switch op.t {
 	switch op.t {
 	// TODO: handle other ops
 	// TODO: handle other ops
 	case tRange:
 	case tRange:
@@ -165,21 +140,21 @@ func (kv *kv) do(ctx context.Context, op Op) (OpResponse, error) {
 			r.SortTarget = pb.RangeRequest_SortTarget(op.sort.Target)
 			r.SortTarget = pb.RangeRequest_SortTarget(op.sort.Target)
 		}
 		}
 
 
-		resp, err = remote.Range(ctx, r)
+		resp, err = kv.remote.Range(ctx, r)
 		if err == nil {
 		if err == nil {
 			return OpResponse{get: (*GetResponse)(resp)}, nil
 			return OpResponse{get: (*GetResponse)(resp)}, nil
 		}
 		}
 	case tPut:
 	case tPut:
 		var resp *pb.PutResponse
 		var resp *pb.PutResponse
 		r := &pb.PutRequest{Key: op.key, Value: op.val, Lease: int64(op.leaseID)}
 		r := &pb.PutRequest{Key: op.key, Value: op.val, Lease: int64(op.leaseID)}
-		resp, err = remote.Put(ctx, r)
+		resp, err = kv.remote.Put(ctx, r)
 		if err == nil {
 		if err == nil {
 			return OpResponse{put: (*PutResponse)(resp)}, nil
 			return OpResponse{put: (*PutResponse)(resp)}, nil
 		}
 		}
 	case tDeleteRange:
 	case tDeleteRange:
 		var resp *pb.DeleteRangeResponse
 		var resp *pb.DeleteRangeResponse
 		r := &pb.DeleteRangeRequest{Key: op.key, RangeEnd: op.end}
 		r := &pb.DeleteRangeRequest{Key: op.key, RangeEnd: op.end}
-		resp, err = remote.DeleteRange(ctx, r)
+		resp, err = kv.remote.DeleteRange(ctx, r)
 		if err == nil {
 		if err == nil {
 			return OpResponse{del: (*DeleteResponse)(resp)}, nil
 			return OpResponse{del: (*DeleteResponse)(resp)}, nil
 		}
 		}
@@ -188,11 +163,3 @@ func (kv *kv) do(ctx context.Context, op Op) (OpResponse, error) {
 	}
 	}
 	return OpResponse{}, err
 	return OpResponse{}, err
 }
 }
-
-// getRemote must be followed by kv.rc.release() call.
-func (kv *kv) getRemote(ctx context.Context) (pb.KVClient, error) {
-	if err := kv.rc.acquire(ctx); err != nil {
-		return nil, err
-	}
-	return kv.remote, nil
-}

+ 16 - 45
clientv3/lease.go

@@ -21,7 +21,6 @@ import (
 	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
 	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 	"golang.org/x/net/context"
 	"golang.org/x/net/context"
-	"google.golang.org/grpc"
 )
 )
 
 
 type (
 type (
@@ -76,7 +75,6 @@ type lessor struct {
 	// donec is closed when recvKeepAliveLoop stops
 	// donec is closed when recvKeepAliveLoop stops
 	donec chan struct{}
 	donec chan struct{}
 
 
-	rc     *remoteClient
 	remote pb.LeaseClient
 	remote pb.LeaseClient
 
 
 	stream       pb.Lease_LeaseKeepAliveClient
 	stream       pb.Lease_LeaseKeepAliveClient
@@ -102,14 +100,10 @@ func NewLease(c *Client) Lease {
 	l := &lessor{
 	l := &lessor{
 		donec:      make(chan struct{}),
 		donec:      make(chan struct{}),
 		keepAlives: make(map[LeaseID]*keepAlive),
 		keepAlives: make(map[LeaseID]*keepAlive),
+		remote:     pb.NewLeaseClient(c.conn),
 	}
 	}
-	f := func(conn *grpc.ClientConn) { l.remote = pb.NewLeaseClient(conn) }
-	l.rc = newRemoteClient(c, f)
-
 	l.stopCtx, l.stopCancel = context.WithCancel(context.Background())
 	l.stopCtx, l.stopCancel = context.WithCancel(context.Background())
-
 	go l.recvKeepAliveLoop()
 	go l.recvKeepAliveLoop()
-
 	return l
 	return l
 }
 }
 
 
@@ -120,7 +114,7 @@ func (l *lessor) Grant(ctx context.Context, ttl int64) (*LeaseGrantResponse, err
 
 
 	for {
 	for {
 		r := &pb.LeaseGrantRequest{TTL: ttl}
 		r := &pb.LeaseGrantRequest{TTL: ttl}
-		resp, err := l.getRemote().LeaseGrant(cctx, r)
+		resp, err := l.remote.LeaseGrant(cctx, r)
 		if err == nil {
 		if err == nil {
 			gresp := &LeaseGrantResponse{
 			gresp := &LeaseGrantResponse{
 				ResponseHeader: resp.GetHeader(),
 				ResponseHeader: resp.GetHeader(),
@@ -131,10 +125,9 @@ func (l *lessor) Grant(ctx context.Context, ttl int64) (*LeaseGrantResponse, err
 			return gresp, nil
 			return gresp, nil
 		}
 		}
 		if isHaltErr(cctx, err) {
 		if isHaltErr(cctx, err) {
-			return nil, rpctypes.Error(err)
+			return nil, toErr(ctx, err)
 		}
 		}
-
-		if nerr := l.switchRemoteAndStream(err); nerr != nil {
+		if nerr := l.newStream(); nerr != nil {
 			return nil, nerr
 			return nil, nerr
 		}
 		}
 	}
 	}
@@ -147,16 +140,15 @@ func (l *lessor) Revoke(ctx context.Context, id LeaseID) (*LeaseRevokeResponse,
 
 
 	for {
 	for {
 		r := &pb.LeaseRevokeRequest{ID: int64(id)}
 		r := &pb.LeaseRevokeRequest{ID: int64(id)}
-		resp, err := l.getRemote().LeaseRevoke(cctx, r)
+		resp, err := l.remote.LeaseRevoke(cctx, r)
 
 
 		if err == nil {
 		if err == nil {
 			return (*LeaseRevokeResponse)(resp), nil
 			return (*LeaseRevokeResponse)(resp), nil
 		}
 		}
 		if isHaltErr(ctx, err) {
 		if isHaltErr(ctx, err) {
-			return nil, rpctypes.Error(err)
+			return nil, toErr(ctx, err)
 		}
 		}
-
-		if nerr := l.switchRemoteAndStream(err); nerr != nil {
+		if nerr := l.newStream(); nerr != nil {
 			return nil, nerr
 			return nil, nerr
 		}
 		}
 	}
 	}
@@ -202,11 +194,10 @@ func (l *lessor) KeepAliveOnce(ctx context.Context, id LeaseID) (*LeaseKeepAlive
 			return resp, err
 			return resp, err
 		}
 		}
 		if isHaltErr(ctx, err) {
 		if isHaltErr(ctx, err) {
-			return nil, rpctypes.Error(err)
+			return nil, toErr(ctx, err)
 		}
 		}
 
 
-		nerr := l.switchRemoteAndStream(err)
-		if nerr != nil {
+		if nerr := l.newStream(); nerr != nil {
 			return nil, nerr
 			return nil, nerr
 		}
 		}
 	}
 	}
@@ -254,19 +245,19 @@ func (l *lessor) keepAliveOnce(ctx context.Context, id LeaseID) (*LeaseKeepAlive
 	cctx, cancel := context.WithCancel(ctx)
 	cctx, cancel := context.WithCancel(ctx)
 	defer cancel()
 	defer cancel()
 
 
-	stream, err := l.getRemote().LeaseKeepAlive(cctx)
+	stream, err := l.remote.LeaseKeepAlive(cctx)
 	if err != nil {
 	if err != nil {
-		return nil, rpctypes.Error(err)
+		return nil, toErr(ctx, err)
 	}
 	}
 
 
 	err = stream.Send(&pb.LeaseKeepAliveRequest{ID: int64(id)})
 	err = stream.Send(&pb.LeaseKeepAliveRequest{ID: int64(id)})
 	if err != nil {
 	if err != nil {
-		return nil, rpctypes.Error(err)
+		return nil, toErr(ctx, err)
 	}
 	}
 
 
 	resp, rerr := stream.Recv()
 	resp, rerr := stream.Recv()
 	if rerr != nil {
 	if rerr != nil {
-		return nil, rpctypes.Error(rerr)
+		return nil, toErr(ctx, rerr)
 	}
 	}
 
 
 	karesp := &LeaseKeepAliveResponse{
 	karesp := &LeaseKeepAliveResponse{
@@ -304,7 +295,7 @@ func (l *lessor) recvKeepAliveLoop() {
 
 
 // resetRecv opens a new lease stream and starts sending LeaseKeepAliveRequests
 // resetRecv opens a new lease stream and starts sending LeaseKeepAliveRequests
 func (l *lessor) resetRecv() (pb.Lease_LeaseKeepAliveClient, error) {
 func (l *lessor) resetRecv() (pb.Lease_LeaseKeepAliveClient, error) {
-	if err := l.switchRemoteAndStream(nil); err != nil {
+	if err := l.newStream(); err != nil {
 		return nil, err
 		return nil, err
 	}
 	}
 	stream := l.getKeepAliveStream()
 	stream := l.getKeepAliveStream()
@@ -380,38 +371,18 @@ func (l *lessor) sendKeepAliveLoop(stream pb.Lease_LeaseKeepAliveClient) {
 	}
 	}
 }
 }
 
 
-func (l *lessor) getRemote() pb.LeaseClient {
-	l.rc.mu.Lock()
-	defer l.rc.mu.Unlock()
-	return l.remote
-}
-
 func (l *lessor) getKeepAliveStream() pb.Lease_LeaseKeepAliveClient {
 func (l *lessor) getKeepAliveStream() pb.Lease_LeaseKeepAliveClient {
 	l.mu.Lock()
 	l.mu.Lock()
 	defer l.mu.Unlock()
 	defer l.mu.Unlock()
 	return l.stream
 	return l.stream
 }
 }
 
 
-func (l *lessor) switchRemoteAndStream(prevErr error) error {
-	for {
-		if prevErr != nil {
-			err := l.rc.reconnectWait(l.stopCtx, prevErr)
-			if err != nil {
-				return rpctypes.Error(err)
-			}
-		}
-		if prevErr = l.newStream(); prevErr == nil {
-			return nil
-		}
-	}
-}
-
 func (l *lessor) newStream() error {
 func (l *lessor) newStream() error {
 	sctx, cancel := context.WithCancel(l.stopCtx)
 	sctx, cancel := context.WithCancel(l.stopCtx)
-	stream, err := l.getRemote().LeaseKeepAlive(sctx)
+	stream, err := l.remote.LeaseKeepAlive(sctx)
 	if err != nil {
 	if err != nil {
 		cancel()
 		cancel()
-		return rpctypes.Error(err)
+		return toErr(sctx, err)
 	}
 	}
 
 
 	l.mu.Lock()
 	l.mu.Lock()

+ 16 - 33
clientv3/maintenance.go

@@ -17,10 +17,8 @@ package clientv3
 import (
 import (
 	"io"
 	"io"
 
 
-	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 	"golang.org/x/net/context"
 	"golang.org/x/net/context"
-	"google.golang.org/grpc"
 )
 )
 
 
 type (
 type (
@@ -54,17 +52,12 @@ type Maintenance interface {
 }
 }
 
 
 type maintenance struct {
 type maintenance struct {
-	c *Client
-
-	rc     *remoteClient
+	c      *Client
 	remote pb.MaintenanceClient
 	remote pb.MaintenanceClient
 }
 }
 
 
 func NewMaintenance(c *Client) Maintenance {
 func NewMaintenance(c *Client) Maintenance {
-	ret := &maintenance{c: c}
-	f := func(conn *grpc.ClientConn) { ret.remote = pb.NewMaintenanceClient(conn) }
-	ret.rc = newRemoteClient(c, f)
-	return ret
+	return &maintenance{c: c, remote: pb.NewMaintenanceClient(c.conn)}
 }
 }
 
 
 func (m *maintenance) AlarmList(ctx context.Context) (*AlarmResponse, error) {
 func (m *maintenance) AlarmList(ctx context.Context) (*AlarmResponse, error) {
@@ -74,15 +67,12 @@ func (m *maintenance) AlarmList(ctx context.Context) (*AlarmResponse, error) {
 		Alarm:    pb.AlarmType_NONE, // all
 		Alarm:    pb.AlarmType_NONE, // all
 	}
 	}
 	for {
 	for {
-		resp, err := m.getRemote().Alarm(ctx, req)
+		resp, err := m.remote.Alarm(ctx, req)
 		if err == nil {
 		if err == nil {
 			return (*AlarmResponse)(resp), nil
 			return (*AlarmResponse)(resp), nil
 		}
 		}
 		if isHaltErr(ctx, err) {
 		if isHaltErr(ctx, err) {
-			return nil, rpctypes.Error(err)
-		}
-		if err = m.rc.reconnectWait(ctx, err); err != nil {
-			return nil, err
+			return nil, toErr(ctx, err)
 		}
 		}
 	}
 	}
 }
 }
@@ -97,38 +87,36 @@ func (m *maintenance) AlarmDisarm(ctx context.Context, am *AlarmMember) (*AlarmR
 	if req.MemberID == 0 && req.Alarm == pb.AlarmType_NONE {
 	if req.MemberID == 0 && req.Alarm == pb.AlarmType_NONE {
 		ar, err := m.AlarmList(ctx)
 		ar, err := m.AlarmList(ctx)
 		if err != nil {
 		if err != nil {
-			return nil, rpctypes.Error(err)
+			return nil, toErr(ctx, err)
 		}
 		}
 		ret := AlarmResponse{}
 		ret := AlarmResponse{}
 		for _, am := range ar.Alarms {
 		for _, am := range ar.Alarms {
 			dresp, derr := m.AlarmDisarm(ctx, (*AlarmMember)(am))
 			dresp, derr := m.AlarmDisarm(ctx, (*AlarmMember)(am))
 			if derr != nil {
 			if derr != nil {
-				return nil, rpctypes.Error(derr)
+				return nil, toErr(ctx, derr)
 			}
 			}
 			ret.Alarms = append(ret.Alarms, dresp.Alarms...)
 			ret.Alarms = append(ret.Alarms, dresp.Alarms...)
 		}
 		}
 		return &ret, nil
 		return &ret, nil
 	}
 	}
 
 
-	resp, err := m.getRemote().Alarm(ctx, req)
+	resp, err := m.remote.Alarm(ctx, req)
 	if err == nil {
 	if err == nil {
 		return (*AlarmResponse)(resp), nil
 		return (*AlarmResponse)(resp), nil
 	}
 	}
-	if !isHaltErr(ctx, err) {
-		m.rc.reconnect(err)
-	}
-	return nil, rpctypes.Error(err)
+	return nil, toErr(ctx, err)
 }
 }
 
 
 func (m *maintenance) Defragment(ctx context.Context, endpoint string) (*DefragmentResponse, error) {
 func (m *maintenance) Defragment(ctx context.Context, endpoint string) (*DefragmentResponse, error) {
 	conn, err := m.c.Dial(endpoint)
 	conn, err := m.c.Dial(endpoint)
 	if err != nil {
 	if err != nil {
-		return nil, rpctypes.Error(err)
+		return nil, toErr(ctx, err)
 	}
 	}
+	defer conn.Close()
 	remote := pb.NewMaintenanceClient(conn)
 	remote := pb.NewMaintenanceClient(conn)
 	resp, err := remote.Defragment(ctx, &pb.DefragmentRequest{})
 	resp, err := remote.Defragment(ctx, &pb.DefragmentRequest{})
 	if err != nil {
 	if err != nil {
-		return nil, rpctypes.Error(err)
+		return nil, toErr(ctx, err)
 	}
 	}
 	return (*DefragmentResponse)(resp), nil
 	return (*DefragmentResponse)(resp), nil
 }
 }
@@ -136,20 +124,21 @@ func (m *maintenance) Defragment(ctx context.Context, endpoint string) (*Defragm
 func (m *maintenance) Status(ctx context.Context, endpoint string) (*StatusResponse, error) {
 func (m *maintenance) Status(ctx context.Context, endpoint string) (*StatusResponse, error) {
 	conn, err := m.c.Dial(endpoint)
 	conn, err := m.c.Dial(endpoint)
 	if err != nil {
 	if err != nil {
-		return nil, rpctypes.Error(err)
+		return nil, toErr(ctx, err)
 	}
 	}
+	defer conn.Close()
 	remote := pb.NewMaintenanceClient(conn)
 	remote := pb.NewMaintenanceClient(conn)
 	resp, err := remote.Status(ctx, &pb.StatusRequest{})
 	resp, err := remote.Status(ctx, &pb.StatusRequest{})
 	if err != nil {
 	if err != nil {
-		return nil, rpctypes.Error(err)
+		return nil, toErr(ctx, err)
 	}
 	}
 	return (*StatusResponse)(resp), nil
 	return (*StatusResponse)(resp), nil
 }
 }
 
 
 func (m *maintenance) Snapshot(ctx context.Context) (io.ReadCloser, error) {
 func (m *maintenance) Snapshot(ctx context.Context) (io.ReadCloser, error) {
-	ss, err := m.getRemote().Snapshot(ctx, &pb.SnapshotRequest{})
+	ss, err := m.remote.Snapshot(ctx, &pb.SnapshotRequest{})
 	if err != nil {
 	if err != nil {
-		return nil, rpctypes.Error(err)
+		return nil, toErr(ctx, err)
 	}
 	}
 
 
 	pr, pw := io.Pipe()
 	pr, pw := io.Pipe()
@@ -172,9 +161,3 @@ func (m *maintenance) Snapshot(ctx context.Context) (io.ReadCloser, error) {
 	}()
 	}()
 	return pr, nil
 	return pr, nil
 }
 }
-
-func (m *maintenance) getRemote() pb.MaintenanceClient {
-	m.rc.mu.Lock()
-	defer m.rc.mu.Unlock()
-	return m.remote
-}

+ 0 - 108
clientv3/remote_client.go

@@ -1,108 +0,0 @@
-// Copyright 2016 The etcd Authors
-//
-// Licensed under the Apache License, Version 2.0 (the "License");
-// you may not use this file except in compliance with the License.
-// You may obtain a copy of the License at
-//
-//     http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-package clientv3
-
-import (
-	"sync"
-
-	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
-
-	"golang.org/x/net/context"
-	"google.golang.org/grpc"
-)
-
-type remoteClient struct {
-	client     *Client
-	conn       *grpc.ClientConn
-	updateConn func(*grpc.ClientConn)
-	mu         sync.Mutex
-}
-
-func newRemoteClient(client *Client, update func(*grpc.ClientConn)) *remoteClient {
-	ret := &remoteClient{
-		client:     client,
-		conn:       client.ActiveConnection(),
-		updateConn: update,
-	}
-	ret.mu.Lock()
-	defer ret.mu.Unlock()
-	ret.updateConn(ret.conn)
-	return ret
-}
-
-// reconnectWait reconnects the client, returning when connection establishes/fails.
-func (r *remoteClient) reconnectWait(ctx context.Context, prevErr error) error {
-	r.mu.Lock()
-	updated := r.tryUpdate()
-	r.mu.Unlock()
-	if updated {
-		return nil
-	}
-	conn, err := r.client.connWait(ctx, prevErr)
-	if err == nil {
-		r.mu.Lock()
-		r.conn = conn
-		r.updateConn(conn)
-		r.mu.Unlock()
-	}
-	return err
-}
-
-// reconnect will reconnect the client without waiting
-func (r *remoteClient) reconnect(err error) {
-	r.mu.Lock()
-	defer r.mu.Unlock()
-	if r.tryUpdate() {
-		return
-	}
-	r.client.connStartRetry(err)
-}
-
-func (r *remoteClient) tryUpdate() bool {
-	activeConn := r.client.ActiveConnection()
-	if activeConn == nil || activeConn == r.conn {
-		return false
-	}
-	r.conn = activeConn
-	r.updateConn(activeConn)
-	return true
-}
-
-// acquire gets the client read lock on an established connection or
-// returns an error without holding the lock.
-func (r *remoteClient) acquire(ctx context.Context) error {
-	for {
-		r.mu.Lock()
-		r.client.mu.RLock()
-		closed := r.client.cancel == nil
-		c := r.client.conn
-		lastConnErr := r.client.lastConnErr
-		match := r.conn == c
-		r.mu.Unlock()
-		if lastConnErr == nil && match {
-			// new connection already
-			return nil
-		}
-		r.client.mu.RUnlock()
-		if closed {
-			return rpctypes.ErrConnClosed
-		}
-		if err := r.reconnectWait(ctx, nil); err != nil {
-			return err
-		}
-	}
-}
-
-func (r *remoteClient) release() { r.client.mu.RUnlock() }

+ 3 - 14
clientv3/txn.go

@@ -17,7 +17,6 @@ package clientv3
 import (
 import (
 	"sync"
 	"sync"
 
 
-	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 	"golang.org/x/net/context"
 	"golang.org/x/net/context"
 )
 )
@@ -143,27 +142,17 @@ func (txn *txn) Commit() (*TxnResponse, error) {
 			return resp, err
 			return resp, err
 		}
 		}
 		if isHaltErr(txn.ctx, err) {
 		if isHaltErr(txn.ctx, err) {
-			return nil, rpctypes.Error(err)
+			return nil, toErr(txn.ctx, err)
 		}
 		}
 		if txn.isWrite {
 		if txn.isWrite {
-			txn.kv.rc.reconnect(err)
-			return nil, rpctypes.Error(err)
-		}
-		if nerr := txn.kv.rc.reconnectWait(txn.ctx, err); nerr != nil {
-			return nil, nerr
+			return nil, toErr(txn.ctx, err)
 		}
 		}
 	}
 	}
 }
 }
 
 
 func (txn *txn) commit() (*TxnResponse, error) {
 func (txn *txn) commit() (*TxnResponse, error) {
-	rem, rerr := txn.kv.getRemote(txn.ctx)
-	if rerr != nil {
-		return nil, rerr
-	}
-	defer txn.kv.rc.release()
-
 	r := &pb.TxnRequest{Compare: txn.cmps, Success: txn.sus, Failure: txn.fas}
 	r := &pb.TxnRequest{Compare: txn.cmps, Success: txn.sus, Failure: txn.fas}
-	resp, err := rem.Txn(txn.ctx, r)
+	resp, err := txn.kv.remote.Txn(txn.ctx, r)
 	if err != nil {
 	if err != nil {
 		return nil, err
 		return nil, err
 	}
 	}

+ 4 - 19
clientv3/watch.go

@@ -22,7 +22,6 @@ import (
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 	mvccpb "github.com/coreos/etcd/mvcc/mvccpb"
 	mvccpb "github.com/coreos/etcd/mvcc/mvccpb"
 	"golang.org/x/net/context"
 	"golang.org/x/net/context"
-	"google.golang.org/grpc"
 )
 )
 
 
 const (
 const (
@@ -87,7 +86,6 @@ func (wr *WatchResponse) IsProgressNotify() bool {
 
 
 // watcher implements the Watcher interface
 // watcher implements the Watcher interface
 type watcher struct {
 type watcher struct {
-	rc     *remoteClient
 	remote pb.WatchClient
 	remote pb.WatchClient
 
 
 	// ctx controls internal remote.Watch requests
 	// ctx controls internal remote.Watch requests
@@ -142,6 +140,7 @@ type watcherStream struct {
 func NewWatcher(c *Client) Watcher {
 func NewWatcher(c *Client) Watcher {
 	ctx, cancel := context.WithCancel(context.Background())
 	ctx, cancel := context.WithCancel(context.Background())
 	w := &watcher{
 	w := &watcher{
+		remote:  pb.NewWatchClient(c.conn),
 		ctx:     ctx,
 		ctx:     ctx,
 		cancel:  cancel,
 		cancel:  cancel,
 		streams: make(map[int64]*watcherStream),
 		streams: make(map[int64]*watcherStream),
@@ -152,10 +151,6 @@ func NewWatcher(c *Client) Watcher {
 		donec: make(chan struct{}),
 		donec: make(chan struct{}),
 		errc:  make(chan error, 1),
 		errc:  make(chan error, 1),
 	}
 	}
-
-	f := func(conn *grpc.ClientConn) { w.remote = pb.NewWatchClient(conn) }
-	w.rc = newRemoteClient(c, f)
-
 	go w.run()
 	go w.run()
 	return w
 	return w
 }
 }
@@ -203,7 +198,7 @@ func (w *watcher) Watch(ctx context.Context, key string, opts ...OpOption) Watch
 func (w *watcher) Close() error {
 func (w *watcher) Close() error {
 	close(w.stopc)
 	close(w.stopc)
 	<-w.donec
 	<-w.donec
-	return v3rpc.Error(<-w.errc)
+	return toErr(w.ctx, <-w.errc)
 }
 }
 
 
 func (w *watcher) addStream(resp *pb.WatchResponse, pendingReq *watchRequest) {
 func (w *watcher) addStream(resp *pb.WatchResponse, pendingReq *watchRequest) {
@@ -500,29 +495,19 @@ func (w *watcher) resume() (ws pb.Watch_WatchClient, err error) {
 // openWatchClient retries opening a watchclient until retryConnection fails
 // openWatchClient retries opening a watchclient until retryConnection fails
 func (w *watcher) openWatchClient() (ws pb.Watch_WatchClient, err error) {
 func (w *watcher) openWatchClient() (ws pb.Watch_WatchClient, err error) {
 	for {
 	for {
-		if err = w.rc.acquire(w.ctx); err != nil {
-			return nil, err
-		}
-
 		select {
 		select {
 		case <-w.stopc:
 		case <-w.stopc:
 			if err == nil {
 			if err == nil {
 				err = context.Canceled
 				err = context.Canceled
 			}
 			}
-			w.rc.release()
 			return nil, err
 			return nil, err
 		default:
 		default:
 		}
 		}
 		if ws, err = w.remote.Watch(w.ctx); ws != nil && err == nil {
 		if ws, err = w.remote.Watch(w.ctx); ws != nil && err == nil {
-			w.rc.release()
 			break
 			break
-		} else if isHaltErr(w.ctx, err) {
-			w.rc.release()
-			return nil, v3rpc.Error(err)
 		}
 		}
-		w.rc.release()
-		if nerr := w.rc.reconnectWait(w.ctx, err); nerr != nil {
-			return nil, v3rpc.Error(nerr)
+		if isHaltErr(w.ctx, err) {
+			return nil, v3rpc.Error(err)
 		}
 		}
 	}
 	}
 	return ws, nil
 	return ws, nil

+ 10 - 10
cmd/Godeps/Godeps.json

@@ -1,7 +1,7 @@
 {
 {
 	"ImportPath": "github.com/coreos/etcd",
 	"ImportPath": "github.com/coreos/etcd",
 	"GoVersion": "go1.6",
 	"GoVersion": "go1.6",
-	"GodepVersion": "v71",
+	"GodepVersion": "v74",
 	"Packages": [
 	"Packages": [
 		"./..."
 		"./..."
 	],
 	],
@@ -213,39 +213,39 @@
 		},
 		},
 		{
 		{
 			"ImportPath": "google.golang.org/grpc",
 			"ImportPath": "google.golang.org/grpc",
-			"Rev": "8213ee577a465c1f314d85748fb29e4eeed59baf"
+			"Rev": "2991ee27f867c664f435417735d5024f43d0d201"
 		},
 		},
 		{
 		{
 			"ImportPath": "google.golang.org/grpc/codes",
 			"ImportPath": "google.golang.org/grpc/codes",
-			"Rev": "8213ee577a465c1f314d85748fb29e4eeed59baf"
+			"Rev": "2991ee27f867c664f435417735d5024f43d0d201"
 		},
 		},
 		{
 		{
 			"ImportPath": "google.golang.org/grpc/credentials",
 			"ImportPath": "google.golang.org/grpc/credentials",
-			"Rev": "8213ee577a465c1f314d85748fb29e4eeed59baf"
+			"Rev": "2991ee27f867c664f435417735d5024f43d0d201"
 		},
 		},
 		{
 		{
 			"ImportPath": "google.golang.org/grpc/grpclog",
 			"ImportPath": "google.golang.org/grpc/grpclog",
-			"Rev": "8213ee577a465c1f314d85748fb29e4eeed59baf"
+			"Rev": "2991ee27f867c664f435417735d5024f43d0d201"
 		},
 		},
 		{
 		{
 			"ImportPath": "google.golang.org/grpc/internal",
 			"ImportPath": "google.golang.org/grpc/internal",
-			"Rev": "8213ee577a465c1f314d85748fb29e4eeed59baf"
+			"Rev": "2991ee27f867c664f435417735d5024f43d0d201"
 		},
 		},
 		{
 		{
 			"ImportPath": "google.golang.org/grpc/metadata",
 			"ImportPath": "google.golang.org/grpc/metadata",
-			"Rev": "8213ee577a465c1f314d85748fb29e4eeed59baf"
+			"Rev": "2991ee27f867c664f435417735d5024f43d0d201"
 		},
 		},
 		{
 		{
 			"ImportPath": "google.golang.org/grpc/naming",
 			"ImportPath": "google.golang.org/grpc/naming",
-			"Rev": "8213ee577a465c1f314d85748fb29e4eeed59baf"
+			"Rev": "2991ee27f867c664f435417735d5024f43d0d201"
 		},
 		},
 		{
 		{
 			"ImportPath": "google.golang.org/grpc/peer",
 			"ImportPath": "google.golang.org/grpc/peer",
-			"Rev": "8213ee577a465c1f314d85748fb29e4eeed59baf"
+			"Rev": "2991ee27f867c664f435417735d5024f43d0d201"
 		},
 		},
 		{
 		{
 			"ImportPath": "google.golang.org/grpc/transport",
 			"ImportPath": "google.golang.org/grpc/transport",
-			"Rev": "8213ee577a465c1f314d85748fb29e4eeed59baf"
+			"Rev": "2991ee27f867c664f435417735d5024f43d0d201"
 		},
 		},
 		{
 		{
 			"ImportPath": "gopkg.in/cheggaaa/pb.v1",
 			"ImportPath": "gopkg.in/cheggaaa/pb.v1",

+ 6 - 6
cmd/vendor/google.golang.org/grpc/PATENTS

@@ -1,22 +1,22 @@
 Additional IP Rights Grant (Patents)
 Additional IP Rights Grant (Patents)
 
 
 "This implementation" means the copyrightable works distributed by
 "This implementation" means the copyrightable works distributed by
-Google as part of the GRPC project.
+Google as part of the gRPC project.
 
 
 Google hereby grants to You a perpetual, worldwide, non-exclusive,
 Google hereby grants to You a perpetual, worldwide, non-exclusive,
 no-charge, royalty-free, irrevocable (except as stated in this section)
 no-charge, royalty-free, irrevocable (except as stated in this section)
 patent license to make, have made, use, offer to sell, sell, import,
 patent license to make, have made, use, offer to sell, sell, import,
 transfer and otherwise run, modify and propagate the contents of this
 transfer and otherwise run, modify and propagate the contents of this
-implementation of GRPC, where such license applies only to those patent
+implementation of gRPC, where such license applies only to those patent
 claims, both currently owned or controlled by Google and acquired in
 claims, both currently owned or controlled by Google and acquired in
 the future, licensable by Google that are necessarily infringed by this
 the future, licensable by Google that are necessarily infringed by this
-implementation of GRPC.  This grant does not include claims that would be
+implementation of gRPC.  This grant does not include claims that would be
 infringed only as a consequence of further modification of this
 infringed only as a consequence of further modification of this
 implementation.  If you or your agent or exclusive licensee institute or
 implementation.  If you or your agent or exclusive licensee institute or
 order or agree to the institution of patent litigation against any
 order or agree to the institution of patent litigation against any
 entity (including a cross-claim or counterclaim in a lawsuit) alleging
 entity (including a cross-claim or counterclaim in a lawsuit) alleging
-that this implementation of GRPC or any code incorporated within this
-implementation of GRPC constitutes direct or contributory patent
+that this implementation of gRPC or any code incorporated within this
+implementation of gRPC constitutes direct or contributory patent
 infringement, or inducement of patent infringement, then any patent
 infringement, or inducement of patent infringement, then any patent
-rights granted to you under this License for this implementation of GRPC
+rights granted to you under this License for this implementation of gRPC
 shall terminate as of the date such litigation is filed.
 shall terminate as of the date such litigation is filed.

+ 4 - 4
cmd/vendor/google.golang.org/grpc/backoff.go

@@ -19,7 +19,7 @@ var (
 // backoffStrategy defines the methodology for backing off after a grpc
 // backoffStrategy defines the methodology for backing off after a grpc
 // connection failure.
 // connection failure.
 //
 //
-// This is unexported until the GRPC project decides whether or not to allow
+// This is unexported until the gRPC project decides whether or not to allow
 // alternative backoff strategies. Once a decision is made, this type and its
 // alternative backoff strategies. Once a decision is made, this type and its
 // method may be exported.
 // method may be exported.
 type backoffStrategy interface {
 type backoffStrategy interface {
@@ -28,14 +28,14 @@ type backoffStrategy interface {
 	backoff(retries int) time.Duration
 	backoff(retries int) time.Duration
 }
 }
 
 
-// BackoffConfig defines the parameters for the default GRPC backoff strategy.
+// BackoffConfig defines the parameters for the default gRPC backoff strategy.
 type BackoffConfig struct {
 type BackoffConfig struct {
 	// MaxDelay is the upper bound of backoff delay.
 	// MaxDelay is the upper bound of backoff delay.
 	MaxDelay time.Duration
 	MaxDelay time.Duration
 
 
 	// TODO(stevvooe): The following fields are not exported, as allowing
 	// TODO(stevvooe): The following fields are not exported, as allowing
-	// changes would violate the current GRPC specification for backoff. If
-	// GRPC decides to allow more interesting backoff strategies, these fields
+	// changes would violate the current gRPC specification for backoff. If
+	// gRPC decides to allow more interesting backoff strategies, these fields
 	// may be opened up in the future.
 	// may be opened up in the future.
 
 
 	// baseDelay is the amount of time to wait before retrying after the first
 	// baseDelay is the amount of time to wait before retrying after the first

+ 340 - 0
cmd/vendor/google.golang.org/grpc/balancer.go

@@ -0,0 +1,340 @@
+/*
+ *
+ * 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 (
+	"fmt"
+	"sync"
+
+	"golang.org/x/net/context"
+	"google.golang.org/grpc/grpclog"
+	"google.golang.org/grpc/naming"
+	"google.golang.org/grpc/transport"
+)
+
+// 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
+	// Metadata is the information associated with Addr, which may be used
+	// to make load balancing decision.
+	Metadata interface{}
+}
+
+// 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
+	// connected address.
+	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 {
+	// Start does the initialization work to bootstrap a Balancer. For example,
+	// 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
+	// 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;
+	//  or
+	//  * issues RPC on the connection otherwise.
+	// 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, 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
+	// 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. 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
+	// 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
+}
+
+// downErr implements net.Error. It is constructed 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 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
+	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
+	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() error {
+	updates, err := rr.w.Next()
+	if err != nil {
+		grpclog.Println("grpc: the naming watcher stops working due to %v.", err)
+		return err
+	}
+	rr.mu.Lock()
+	defer rr.mu.Unlock()
+	for _, update := range updates {
+		addr := Address{
+			Addr: update.Addr,
+		}
+		switch update.Op {
+		case naming.Add:
+			var exist bool
+			for _, v := range rr.open {
+				if addr == v {
+					exist = true
+					grpclog.Println("grpc: The name resolver wanted to add an existing address: ", addr)
+					break
+				}
+			}
+			if exist {
+				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), len(rr.open))
+	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 {
+		// 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)
+	if err != nil {
+		return err
+	}
+	rr.w = w
+	rr.addrCh = make(chan []Address)
+	go func() {
+		for {
+			if err := rr.watchAddrUpdates(); err != nil {
+				return
+			}
+		}
+	}()
+	return nil
+}
+
+// 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()
+	defer rr.mu.Unlock()
+	for _, a := range rr.connected {
+		if a == addr {
+			return nil
+		}
+	}
+	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)
+			rr.waitCh = nil
+		}
+	}
+	return func(err error) {
+		rr.down(addr, err)
+	}
+}
+
+// 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()
+	for i, a := range rr.connected {
+		if a == addr {
+			copy(rr.connected[i:], rr.connected[i+1:])
+			rr.connected = rr.connected[:len(rr.connected)-1]
+			return
+		}
+	}
+}
+
+// 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()
+	if rr.done {
+		rr.mu.Unlock()
+		err = ErrClientConnClosing
+		return
+	}
+	if rr.next >= len(rr.connected) {
+		rr.next = 0
+	}
+	if len(rr.connected) > 0 {
+		addr = rr.connected[rr.next]
+		rr.next++
+		rr.mu.Unlock()
+		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
+	} else {
+		ch = rr.waitCh
+	}
+	rr.mu.Unlock()
+	for {
+		select {
+		case <-ctx.Done():
+			err = transport.ContextErr(ctx.Err())
+			return
+		case <-ch:
+			rr.mu.Lock()
+			if rr.done {
+				rr.mu.Unlock()
+				err = ErrClientConnClosing
+				return
+			}
+			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
+			}
+			if rr.next >= len(rr.connected) {
+				rr.next = 0
+			}
+			addr = rr.connected[rr.next]
+			rr.next++
+			rr.mu.Unlock()
+			return
+		}
+	}
+}
+
+func (rr *roundRobin) Notify() <-chan []Address {
+	return rr.addrCh
+}
+
+func (rr *roundRobin) Close() error {
+	rr.mu.Lock()
+	defer rr.mu.Unlock()
+	rr.done = true
+	if rr.w != nil {
+		rr.w.Close()
+	}
+	if rr.waitCh != nil {
+		close(rr.waitCh)
+		rr.waitCh = nil
+	}
+	if rr.addrCh != nil {
+		close(rr.addrCh)
+	}
+	return nil
+}

+ 46 - 22
cmd/vendor/google.golang.org/grpc/call.go

@@ -132,19 +132,16 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli
 		Last:  true,
 		Last:  true,
 		Delay: false,
 		Delay: false,
 	}
 	}
-	var (
-		lastErr error // record the error that happened
-	)
 	for {
 	for {
 		var (
 		var (
 			err    error
 			err    error
 			t      transport.ClientTransport
 			t      transport.ClientTransport
 			stream *transport.Stream
 			stream *transport.Stream
+			// 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 retry strategy for non-failfast rpcs.
-		if lastErr != nil && c.failFast {
-			return toRPCErr(lastErr)
-		}
+		// TODO(zhaoq): Need a formal spec of fail-fast.
 		callHdr := &transport.CallHdr{
 		callHdr := &transport.CallHdr{
 			Host:   cc.authority,
 			Host:   cc.authority,
 			Method: method,
 			Method: method,
@@ -152,39 +149,66 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli
 		if cc.dopts.cp != nil {
 		if cc.dopts.cp != nil {
 			callHdr.SendCompress = cc.dopts.cp.Type()
 			callHdr.SendCompress = cc.dopts.cp.Type()
 		}
 		}
-		t, err = cc.dopts.picker.Pick(ctx)
+		gopts := BalancerGetOptions{
+			BlockingWait: !c.failFast,
+		}
+		t, put, err = cc.getTransport(ctx, gopts)
 		if err != nil {
 		if err != nil {
-			if lastErr != nil {
-				// This was a retry; return the error from the last attempt.
-				return toRPCErr(lastErr)
+			// TODO(zhaoq): Probably revisit the error handling.
+			if err == ErrClientConnClosing {
+				return Errorf(codes.FailedPrecondition, "%v", 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 {
 		if c.traceInfo.tr != nil {
 			c.traceInfo.tr.LazyLog(&payload{sent: true, msg: args}, true)
 			c.traceInfo.tr.LazyLog(&payload{sent: true, msg: args}, true)
 		}
 		}
 		stream, err = sendRequest(ctx, cc.dopts.codec, cc.dopts.cp, callHdr, t, args, topts)
 		stream, err = sendRequest(ctx, cc.dopts.codec, cc.dopts.cp, callHdr, t, args, topts)
 		if err != nil {
 		if err != nil {
+			if put != nil {
+				put()
+				put = nil
+			}
 			if _, ok := err.(transport.ConnectionError); ok {
 			if _, ok := err.(transport.ConnectionError); ok {
-				lastErr = err
+				if c.failFast {
+					return toRPCErr(err)
+				}
 				continue
 				continue
 			}
 			}
-			if lastErr != nil {
-				return toRPCErr(lastErr)
-			}
 			return toRPCErr(err)
 			return toRPCErr(err)
 		}
 		}
 		// Receive the response
 		// Receive the response
-		lastErr = recvResponse(cc.dopts, t, &c, stream, reply)
-		if _, ok := lastErr.(transport.ConnectionError); ok {
-			continue
+		err = recvResponse(cc.dopts, t, &c, stream, reply)
+		if err != nil {
+			if put != nil {
+				put()
+				put = nil
+			}
+			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 {
 		if c.traceInfo.tr != nil {
 			c.traceInfo.tr.LazyLog(&payload{sent: false, msg: reply}, true)
 			c.traceInfo.tr.LazyLog(&payload{sent: false, msg: reply}, true)
 		}
 		}
-		t.CloseStream(stream, lastErr)
-		if lastErr != nil {
-			return toRPCErr(lastErr)
+		t.CloseStream(stream, nil)
+		if put != nil {
+			put()
+			put = nil
 		}
 		}
 		return Errorf(stream.StatusCode(), "%s", stream.StatusDesc())
 		return Errorf(stream.StatusCode(), "%s", stream.StatusDesc())
 	}
 	}

+ 357 - 260
cmd/vendor/google.golang.org/grpc/clientconn.go

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

+ 2 - 1
cmd/vendor/google.golang.org/grpc/naming/naming.go

@@ -66,7 +66,8 @@ type Resolver interface {
 // Watcher watches for the updates on the specified target.
 // Watcher watches for the updates on the specified target.
 type Watcher interface {
 type Watcher interface {
 	// Next blocks until an update or error happens. It may return one or more
 	// 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 should
+	// return an error if and only if Watcher cannot recover.
 	Next() ([]*Update, error)
 	Next() ([]*Update, error)
 	// Close closes the Watcher.
 	// Close closes the Watcher.
 	Close()
 	Close()

+ 0 - 243
cmd/vendor/google.golang.org/grpc/picker.go

@@ -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
-}

+ 2 - 2
cmd/vendor/google.golang.org/grpc/rpc_util.go

@@ -61,7 +61,7 @@ type Codec interface {
 	String() string
 	String() string
 }
 }
 
 
-// protoCodec is a Codec implemetation with protobuf. It is the default codec for gRPC.
+// protoCodec is a Codec implementation with protobuf. It is the default codec for gRPC.
 type protoCodec struct{}
 type protoCodec struct{}
 
 
 func (protoCodec) Marshal(v interface{}) ([]byte, error) {
 func (protoCodec) Marshal(v interface{}) ([]byte, error) {
@@ -187,7 +187,7 @@ const (
 	compressionMade
 	compressionMade
 )
 )
 
 
-// parser reads complelete gRPC messages from the underlying reader.
+// parser reads complete gRPC messages from the underlying reader.
 type parser struct {
 type parser struct {
 	// r is the underlying reader.
 	// r is the underlying reader.
 	// See the comment on recvMsg for the permissible
 	// See the comment on recvMsg for the permissible

+ 7 - 3
cmd/vendor/google.golang.org/grpc/server.go

@@ -73,6 +73,7 @@ type ServiceDesc struct {
 	HandlerType interface{}
 	HandlerType interface{}
 	Methods     []MethodDesc
 	Methods     []MethodDesc
 	Streams     []StreamDesc
 	Streams     []StreamDesc
+	Metadata    interface{}
 }
 }
 
 
 // service consists of the information of the server serving this service and
 // service consists of the information of the server serving this service and
@@ -115,12 +116,14 @@ func CustomCodec(codec Codec) ServerOption {
 	}
 	}
 }
 }
 
 
+// RPCCompressor returns a ServerOption that sets a compressor for outbound message.
 func RPCCompressor(cp Compressor) ServerOption {
 func RPCCompressor(cp Compressor) ServerOption {
 	return func(o *options) {
 	return func(o *options) {
 		o.cp = cp
 		o.cp = cp
 	}
 	}
 }
 }
 
 
+// RPCDecompressor returns a ServerOption that sets a decompressor for inbound message.
 func RPCDecompressor(dc Decompressor) ServerOption {
 func RPCDecompressor(dc Decompressor) ServerOption {
 	return func(o *options) {
 	return func(o *options) {
 		o.dc = dc
 		o.dc = dc
@@ -462,6 +465,10 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport.
 			}
 			}
 		}()
 		}()
 	}
 	}
+	if s.opts.cp != nil {
+		// NOTE: this needs to be ahead of all handling, https://github.com/grpc/grpc-go/issues/686.
+		stream.SetSendCompress(s.opts.cp.Type())
+	}
 	p := &parser{r: stream}
 	p := &parser{r: stream}
 	for {
 	for {
 		pf, req, err := p.recvMsg()
 		pf, req, err := p.recvMsg()
@@ -547,9 +554,6 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport.
 			Last:  true,
 			Last:  true,
 			Delay: false,
 			Delay: false,
 		}
 		}
-		if s.opts.cp != nil {
-			stream.SetSendCompress(s.opts.cp.Type())
-		}
 		if err := s.sendResponse(t, stream, reply, s.opts.cp, opts); err != nil {
 		if err := s.sendResponse(t, stream, reply, s.opts.cp, opts); err != nil {
 			switch err := err.(type) {
 			switch err := err.(type) {
 			case transport.ConnectionError:
 			case transport.ConnectionError:

+ 13 - 3
cmd/vendor/google.golang.org/grpc/stream.go

@@ -79,7 +79,7 @@ type Stream interface {
 	RecvMsg(m interface{}) error
 	RecvMsg(m interface{}) error
 }
 }
 
 
-// ClientStream defines the interface a client stream has to satify.
+// ClientStream defines the interface a client stream has to satisfy.
 type ClientStream interface {
 type ClientStream interface {
 	// Header returns the header metadata received from the server if there
 	// Header returns the header metadata received from the server if there
 	// is any. It blocks if the metadata is not ready to read.
 	// is any. It blocks if the metadata is not ready to read.
@@ -103,12 +103,16 @@ func NewClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth
 	var (
 	var (
 		t   transport.ClientTransport
 		t   transport.ClientTransport
 		err error
 		err error
+		put func()
 	)
 	)
-	t, err = cc.dopts.picker.Pick(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 {
 	if err != nil {
 		return nil, toRPCErr(err)
 		return nil, toRPCErr(err)
 	}
 	}
-	// TODO(zhaoq): CallOption is omitted. Add support when it is needed.
 	callHdr := &transport.CallHdr{
 	callHdr := &transport.CallHdr{
 		Host:   cc.authority,
 		Host:   cc.authority,
 		Method: method,
 		Method: method,
@@ -119,6 +123,7 @@ func NewClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth
 	}
 	}
 	cs := &clientStream{
 	cs := &clientStream{
 		desc:    desc,
 		desc:    desc,
+		put:     put,
 		codec:   cc.dopts.codec,
 		codec:   cc.dopts.codec,
 		cp:      cc.dopts.cp,
 		cp:      cc.dopts.cp,
 		dc:      cc.dopts.dc,
 		dc:      cc.dopts.dc,
@@ -174,6 +179,7 @@ type clientStream struct {
 	tracing bool // set to EnableTracing when the clientStream is created.
 	tracing bool // set to EnableTracing when the clientStream is created.
 
 
 	mu     sync.Mutex
 	mu     sync.Mutex
+	put    func()
 	closed bool
 	closed bool
 	// trInfo.tr is set when the clientStream is created (if EnableTracing is true),
 	// 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.
 	// and is set to nil when the clientStream's finish method is called.
@@ -311,6 +317,10 @@ func (cs *clientStream) finish(err error) {
 	}
 	}
 	cs.mu.Lock()
 	cs.mu.Lock()
 	defer cs.mu.Unlock()
 	defer cs.mu.Unlock()
+	if cs.put != nil {
+		cs.put()
+		cs.put = nil
+	}
 	if cs.trInfo.tr != nil {
 	if cs.trInfo.tr != nil {
 		if err == nil || err == io.EOF {
 		if err == nil || err == io.EOF {
 			cs.trInfo.tr.LazyPrintf("RPC: [OK]")
 			cs.trInfo.tr.LazyPrintf("RPC: [OK]")

+ 1 - 2
cmd/vendor/google.golang.org/grpc/trace.go

@@ -101,9 +101,8 @@ type payload struct {
 func (p payload) String() string {
 func (p payload) String() string {
 	if p.sent {
 	if p.sent {
 		return fmt.Sprintf("sent: %v", p.msg)
 		return fmt.Sprintf("sent: %v", p.msg)
-	} else {
-		return fmt.Sprintf("recv: %v", p.msg)
 	}
 	}
+	return fmt.Sprintf("recv: %v", p.msg)
 }
 }
 
 
 type fmtStringer struct {
 type fmtStringer struct {

+ 12 - 2
cmd/vendor/google.golang.org/grpc/transport/handler_server.go

@@ -92,9 +92,12 @@ func NewServerHandlerTransport(w http.ResponseWriter, r *http.Request) (ServerTr
 	}
 	}
 
 
 	var metakv []string
 	var metakv []string
+	if r.Host != "" {
+		metakv = append(metakv, ":authority", r.Host)
+	}
 	for k, vv := range r.Header {
 	for k, vv := range r.Header {
 		k = strings.ToLower(k)
 		k = strings.ToLower(k)
-		if isReservedHeader(k) {
+		if isReservedHeader(k) && !isWhitelistedPseudoHeader(k){
 			continue
 			continue
 		}
 		}
 		for _, v := range vv {
 		for _, v := range vv {
@@ -108,7 +111,6 @@ func NewServerHandlerTransport(w http.ResponseWriter, r *http.Request) (ServerTr
 				}
 				}
 			}
 			}
 			metakv = append(metakv, k, v)
 			metakv = append(metakv, k, v)
-
 		}
 		}
 	}
 	}
 	st.headerMD = metadata.Pairs(metakv...)
 	st.headerMD = metadata.Pairs(metakv...)
@@ -196,6 +198,10 @@ func (ht *serverHandlerTransport) WriteStatus(s *Stream, statusCode codes.Code,
 		}
 		}
 		if md := s.Trailer(); len(md) > 0 {
 		if md := s.Trailer(); len(md) > 0 {
 			for k, vv := range md {
 			for k, vv := range md {
+				// Clients don't tolerate reading restricted headers after some non restricted ones were sent.
+				if isReservedHeader(k) {
+					continue
+				}
 				for _, v := range vv {
 				for _, v := range vv {
 					// http2 ResponseWriter mechanism to
 					// http2 ResponseWriter mechanism to
 					// send undeclared Trailers after the
 					// send undeclared Trailers after the
@@ -249,6 +255,10 @@ func (ht *serverHandlerTransport) WriteHeader(s *Stream, md metadata.MD) error {
 		ht.writeCommonHeaders(s)
 		ht.writeCommonHeaders(s)
 		h := ht.rw.Header()
 		h := ht.rw.Header()
 		for k, vv := range md {
 		for k, vv := range md {
+			// Clients don't tolerate reading restricted headers after some non restricted ones were sent.
+			if isReservedHeader(k) {
+				continue
+			}
 			for _, v := range vv {
 			for _, v := range vv {
 				h.Add(k, v)
 				h.Add(k, v)
 			}
 			}

+ 38 - 2
cmd/vendor/google.golang.org/grpc/transport/http2_client.go

@@ -35,7 +35,6 @@ package transport
 
 
 import (
 import (
 	"bytes"
 	"bytes"
-	"errors"
 	"io"
 	"io"
 	"math"
 	"math"
 	"net"
 	"net"
@@ -272,6 +271,10 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea
 		}
 		}
 	}
 	}
 	t.mu.Lock()
 	t.mu.Lock()
+	if t.activeStreams == nil {
+		t.mu.Unlock()
+		return nil, ErrConnClosing
+	}
 	if t.state != reachable {
 	if t.state != reachable {
 		t.mu.Unlock()
 		t.mu.Unlock()
 		return nil, ErrConnClosing
 		return nil, ErrConnClosing
@@ -344,6 +347,10 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea
 	if md, ok := metadata.FromContext(ctx); ok {
 	if md, ok := metadata.FromContext(ctx); ok {
 		hasMD = true
 		hasMD = true
 		for k, v := range md {
 		for k, v := range md {
+			// HTTP doesn't allow you to set pseudoheaders after non pseudoheaders were set.
+			if isReservedHeader(k) {
+				continue
+			}
 			for _, entry := range v {
 			for _, entry := range v {
 				t.hEnc.WriteField(hpack.HeaderField{Name: k, Value: entry})
 				t.hEnc.WriteField(hpack.HeaderField{Name: k, Value: entry})
 			}
 			}
@@ -393,9 +400,19 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea
 func (t *http2Client) CloseStream(s *Stream, err error) {
 func (t *http2Client) CloseStream(s *Stream, err error) {
 	var updateStreams bool
 	var updateStreams bool
 	t.mu.Lock()
 	t.mu.Lock()
+	if t.activeStreams == nil {
+		t.mu.Unlock()
+		return
+	}
 	if t.streamsQuota != nil {
 	if t.streamsQuota != nil {
 		updateStreams = true
 		updateStreams = true
 	}
 	}
+	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)
 	delete(t.activeStreams, s.id)
 	t.mu.Unlock()
 	t.mu.Unlock()
 	if updateStreams {
 	if updateStreams {
@@ -437,7 +454,7 @@ func (t *http2Client) Close() (err error) {
 	}
 	}
 	if t.state == closing {
 	if t.state == closing {
 		t.mu.Unlock()
 		t.mu.Unlock()
-		return errors.New("transport: Close() was already called")
+		return
 	}
 	}
 	t.state = closing
 	t.state = closing
 	t.mu.Unlock()
 	t.mu.Unlock()
@@ -460,6 +477,25 @@ func (t *http2Client) Close() (err error) {
 	return
 	return
 }
 }
 
 
+func (t *http2Client) GracefulClose() error {
+	t.mu.Lock()
+	if t.state == closing {
+		t.mu.Unlock()
+		return nil
+	}
+	if t.state == draining {
+		t.mu.Unlock()
+		return nil
+	}
+	t.state = draining
+	active := len(t.activeStreams)
+	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
 // Write formats the data into HTTP2 data frame(s) and sends it out. The caller
 // should proceed only if Write returns nil.
 // should proceed only if Write returns nil.
 // TODO(zhaoq): opts.Delay is ignored in this implementation. Support it later
 // TODO(zhaoq): opts.Delay is ignored in this implementation. Support it later

+ 8 - 0
cmd/vendor/google.golang.org/grpc/transport/http2_server.go

@@ -460,6 +460,10 @@ func (t *http2Server) WriteHeader(s *Stream, md metadata.MD) error {
 		t.hEnc.WriteField(hpack.HeaderField{Name: "grpc-encoding", Value: s.sendCompress})
 		t.hEnc.WriteField(hpack.HeaderField{Name: "grpc-encoding", Value: s.sendCompress})
 	}
 	}
 	for k, v := range md {
 	for k, v := range md {
+		if isReservedHeader(k) {
+			// Clients don't tolerate reading restricted headers after some non restricted ones were sent.
+			continue
+		}
 		for _, entry := range v {
 		for _, entry := range v {
 			t.hEnc.WriteField(hpack.HeaderField{Name: k, Value: entry})
 			t.hEnc.WriteField(hpack.HeaderField{Name: k, Value: entry})
 		}
 		}
@@ -502,6 +506,10 @@ func (t *http2Server) WriteStatus(s *Stream, statusCode codes.Code, statusDesc s
 	t.hEnc.WriteField(hpack.HeaderField{Name: "grpc-message", Value: statusDesc})
 	t.hEnc.WriteField(hpack.HeaderField{Name: "grpc-message", Value: statusDesc})
 	// Attach the trailer metadata.
 	// Attach the trailer metadata.
 	for k, v := range s.trailer {
 	for k, v := range s.trailer {
+		// Clients don't tolerate reading restricted headers after some non restricted ones were sent.
+		if isReservedHeader(k) {
+			continue
+		}
 		for _, entry := range v {
 		for _, entry := range v {
 			t.hEnc.WriteField(hpack.HeaderField{Name: k, Value: entry})
 			t.hEnc.WriteField(hpack.HeaderField{Name: k, Value: entry})
 		}
 		}

+ 12 - 1
cmd/vendor/google.golang.org/grpc/transport/http_util.go

@@ -127,6 +127,17 @@ func isReservedHeader(hdr string) bool {
 	}
 	}
 }
 }
 
 
+// isWhitelistedPseudoHeader checks whether hdr belongs to HTTP2 pseudoheaders
+// that should be propagated into metadata visible to users.
+func isWhitelistedPseudoHeader(hdr string) bool {
+	switch hdr {
+	case ":authority":
+		return true
+	default:
+		return false
+	}
+}
+
 func (d *decodeState) setErr(err error) {
 func (d *decodeState) setErr(err error) {
 	if d.err == nil {
 	if d.err == nil {
 		d.err = err
 		d.err = err
@@ -162,7 +173,7 @@ func (d *decodeState) processHeaderField(f hpack.HeaderField) {
 	case ":path":
 	case ":path":
 		d.method = f.Value
 		d.method = f.Value
 	default:
 	default:
-		if !isReservedHeader(f.Name) {
+		if !isReservedHeader(f.Name) || isWhitelistedPseudoHeader(f.Name) {
 			if f.Name == "user-agent" {
 			if f.Name == "user-agent" {
 				i := strings.LastIndex(f.Value, " ")
 				i := strings.LastIndex(f.Value, " ")
 				if i == -1 {
 				if i == -1 {

+ 6 - 1
cmd/vendor/google.golang.org/grpc/transport/transport.go

@@ -321,6 +321,7 @@ const (
 	reachable transportState = iota
 	reachable transportState = iota
 	unreachable
 	unreachable
 	closing
 	closing
+	draining
 )
 )
 
 
 // NewServerTransport creates a ServerTransport with conn or non-nil error
 // NewServerTransport creates a ServerTransport with conn or non-nil error
@@ -337,7 +338,7 @@ type ConnectOptions struct {
 	Dialer func(string, time.Duration) (net.Conn, error)
 	Dialer func(string, time.Duration) (net.Conn, error)
 	// AuthOptions stores the credentials required to setup a client connection and/or issue RPCs.
 	// AuthOptions stores the credentials required to setup a client connection and/or issue RPCs.
 	AuthOptions []credentials.Credentials
 	AuthOptions []credentials.Credentials
-	// Timeout specifies the timeout for dialing a client connection.
+	// Timeout specifies the timeout for dialing a ClientTransport.
 	Timeout time.Duration
 	Timeout time.Duration
 }
 }
 
 
@@ -391,6 +392,10 @@ type ClientTransport interface {
 	// is called only once.
 	// is called only once.
 	Close() error
 	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
 	// Write sends the data for the given stream. A nil stream indicates
 	// the write is to be performed on the transport as a whole.
 	// the write is to be performed on the transport as a whole.
 	Write(s *Stream, data []byte, opts *Options) error
 	Write(s *Stream, data []byte, opts *Options) error

+ 0 - 2
etcdserver/api/v3rpc/rpctypes/error.go

@@ -110,8 +110,6 @@ var (
 
 
 	ErrNoLeader   = Error(ErrGRPCNoLeader)
 	ErrNoLeader   = Error(ErrGRPCNoLeader)
 	ErrNotCapable = Error(ErrGRPCNotCapable)
 	ErrNotCapable = Error(ErrGRPCNotCapable)
-
-	ErrConnClosed = EtcdError{code: codes.Unavailable, desc: "clientv3: connection closed"}
 )
 )
 
 
 // EtcdError defines gRPC server errors.
 // EtcdError defines gRPC server errors.

+ 141 - 0
integration/bridge.go

@@ -0,0 +1,141 @@
+// Copyright 2016 The etcd Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package integration
+
+import (
+	"fmt"
+	"io"
+	"net"
+	"os"
+	"sync"
+)
+
+// bridge creates a unix socket bridge to another unix socket, making it possible
+// to disconnect grpc network connections without closing the logical grpc connection.
+type bridge struct {
+	inaddr  string
+	outaddr string
+	l       net.Listener
+	conns   map[*bridgeConn]struct{}
+
+	stopc chan struct{}
+	wg    sync.WaitGroup
+
+	mu sync.Mutex
+}
+
+func newBridge(addr string) (*bridge, error) {
+	b := &bridge{
+		inaddr:  addr + ".bridge",
+		outaddr: addr,
+		conns:   make(map[*bridgeConn]struct{}),
+		stopc:   make(chan struct{}, 1),
+	}
+	if err := os.RemoveAll(b.inaddr); err != nil {
+		return nil, err
+	}
+	l, err := net.Listen("unix", b.inaddr)
+	if err != nil {
+		return nil, fmt.Errorf("listen failed on socket %s (%v)", addr, err)
+	}
+	b.l = l
+	b.wg.Add(1)
+	go b.serveListen()
+	return b, nil
+}
+
+func (b *bridge) URL() string { return "unix://" + b.inaddr }
+
+func (b *bridge) Close() {
+	b.l.Close()
+	select {
+	case b.stopc <- struct{}{}:
+	default:
+	}
+	b.wg.Wait()
+}
+
+func (b *bridge) Reset() {
+	b.mu.Lock()
+	defer b.mu.Unlock()
+	for bc := range b.conns {
+		bc.Close()
+	}
+	b.conns = make(map[*bridgeConn]struct{})
+}
+
+func (b *bridge) serveListen() {
+	defer func() {
+		b.l.Close()
+		os.RemoveAll(b.inaddr)
+		b.mu.Lock()
+		for bc := range b.conns {
+			bc.Close()
+		}
+		b.mu.Unlock()
+		b.wg.Done()
+	}()
+
+	for {
+		inc, ierr := b.l.Accept()
+		if ierr != nil {
+			return
+		}
+		outc, oerr := net.Dial("unix", b.outaddr)
+		if oerr != nil {
+			inc.Close()
+			return
+		}
+
+		bc := &bridgeConn{inc, outc}
+		b.wg.Add(1)
+		b.mu.Lock()
+		b.conns[bc] = struct{}{}
+		go b.serveConn(bc)
+		b.mu.Unlock()
+	}
+}
+
+func (b *bridge) serveConn(bc *bridgeConn) {
+	defer func() {
+		bc.Close()
+		b.mu.Lock()
+		delete(b.conns, bc)
+		b.mu.Unlock()
+		b.wg.Done()
+	}()
+
+	var wg sync.WaitGroup
+	wg.Add(2)
+	go func() {
+		io.Copy(bc.out, bc.in)
+		wg.Done()
+	}()
+	go func() {
+		io.Copy(bc.in, bc.out)
+		wg.Done()
+	}()
+	wg.Wait()
+}
+
+type bridgeConn struct {
+	in  net.Conn
+	out net.Conn
+}
+
+func (bc *bridgeConn) Close() {
+	bc.in.Close()
+	bc.out.Close()
+}

+ 13 - 1
integration/cluster.go

@@ -431,6 +431,7 @@ type member struct {
 
 
 	grpcServer *grpc.Server
 	grpcServer *grpc.Server
 	grpcAddr   string
 	grpcAddr   string
+	grpcBridge *bridge
 }
 }
 
 
 func (m *member) GRPCAddr() string { return m.grpcAddr }
 func (m *member) GRPCAddr() string { return m.grpcAddr }
@@ -506,11 +507,18 @@ func (m *member) listenGRPC() error {
 	if err != nil {
 	if err != nil {
 		return fmt.Errorf("listen failed on grpc socket %s (%v)", m.grpcAddr, err)
 		return fmt.Errorf("listen failed on grpc socket %s (%v)", m.grpcAddr, err)
 	}
 	}
-	m.grpcAddr = "unix://" + m.grpcAddr
+	m.grpcBridge, err = newBridge(m.grpcAddr)
+	if err != nil {
+		l.Close()
+		return err
+	}
+	m.grpcAddr = m.grpcBridge.URL()
 	m.grpcListener = l
 	m.grpcListener = l
 	return nil
 	return nil
 }
 }
 
 
+func (m *member) DropConnections() { m.grpcBridge.Reset() }
+
 // NewClientV3 creates a new grpc client connection to the member
 // NewClientV3 creates a new grpc client connection to the member
 func NewClientV3(m *member) (*clientv3.Client, error) {
 func NewClientV3(m *member) (*clientv3.Client, error) {
 	if m.grpcAddr == "" {
 	if m.grpcAddr == "" {
@@ -659,6 +667,10 @@ func (m *member) Resume() {
 
 
 // Close stops the member's etcdserver and closes its connections
 // Close stops the member's etcdserver and closes its connections
 func (m *member) Close() {
 func (m *member) Close() {
+	if m.grpcBridge != nil {
+		m.grpcBridge.Close()
+		m.grpcBridge = nil
+	}
 	if m.grpcServer != nil {
 	if m.grpcServer != nil {
 		m.grpcServer.Stop()
 		m.grpcServer.Stop()
 		m.grpcServer = nil
 		m.grpcServer = nil

+ 2 - 17
integration/v3_grpc_test.go

@@ -909,30 +909,15 @@ func TestTLSGRPCRejectInsecureClient(t *testing.T) {
 	}
 	}
 	defer client.Close()
 	defer client.Close()
 
 
-	ctx, cancel := context.WithTimeout(context.TODO(), 5*time.Second)
-	conn := client.ActiveConnection()
-	st, err := conn.State()
-	if err != nil {
-		t.Fatal(err)
-	} else if st != grpc.Ready {
-		t.Fatalf("expected Ready, got %v", st)
-	}
-
-	// rpc will fail to handshake, triggering a connection state change
 	donec := make(chan error, 1)
 	donec := make(chan error, 1)
 	go func() {
 	go func() {
+		ctx, cancel := context.WithTimeout(context.TODO(), 5*time.Second)
 		reqput := &pb.PutRequest{Key: []byte("foo"), Value: []byte("bar")}
 		reqput := &pb.PutRequest{Key: []byte("foo"), Value: []byte("bar")}
 		_, perr := toGRPC(client).KV.Put(ctx, reqput)
 		_, perr := toGRPC(client).KV.Put(ctx, reqput)
+		cancel()
 		donec <- perr
 		donec <- perr
 	}()
 	}()
 
 
-	st, err = conn.WaitForStateChange(ctx, st)
-	if err != nil {
-		t.Fatalf("unexpected error waiting for change (%v)", err)
-	} else if st == grpc.Ready {
-		t.Fatalf("expected failure state, got %v", st)
-	}
-	cancel()
 	if perr := <-donec; perr == nil {
 	if perr := <-donec; perr == nil {
 		t.Fatalf("expected client error on put")
 		t.Fatalf("expected client error on put")
 	}
 	}

+ 3 - 2
integration/v3_watch_test.go

@@ -983,7 +983,7 @@ func TestV3WatchClose(t *testing.T) {
 	clus := NewClusterV3(t, &ClusterConfig{Size: 1})
 	clus := NewClusterV3(t, &ClusterConfig{Size: 1})
 	defer clus.Terminate(t)
 	defer clus.Terminate(t)
 
 
-	c := clus.RandClient()
+	c := clus.Client(0)
 	wapi := toGRPC(c).Watch
 	wapi := toGRPC(c).Watch
 
 
 	var wg sync.WaitGroup
 	var wg sync.WaitGroup
@@ -1007,6 +1007,7 @@ func TestV3WatchClose(t *testing.T) {
 			ws.Recv()
 			ws.Recv()
 		}()
 		}()
 	}
 	}
-	c.ActiveConnection().Close()
+
+	clus.Members[0].DropConnections()
 	wg.Wait()
 	wg.Wait()
 }
 }