Browse Source

clientv3: fix client balancer with gRPC v1.7

Signed-off-by: Gyu-Ho Lee <gyuhox@gmail.com>
Gyu-Ho Lee 8 years ago
parent
commit
a8c84ffc93
59 changed files with 2351 additions and 960 deletions
  1. 1 1
      clientv3/README.md
  2. 9 8
      clientv3/auth.go
  3. 0 356
      clientv3/balancer.go
  4. 54 28
      clientv3/client.go
  5. 5 5
      clientv3/client_test.go
  6. 3 2
      clientv3/clientv3util/example_key_test.go
  7. 10 18
      clientv3/cluster.go
  8. 2 4
      clientv3/compact_op.go
  9. 10 0
      clientv3/compare.go
  10. 6 3
      clientv3/concurrency/election.go
  11. 1 0
      clientv3/concurrency/key.go
  12. 10 1
      clientv3/concurrency/mutex.go
  13. 2 0
      clientv3/concurrency/session.go
  14. 3 2
      clientv3/concurrency/stm.go
  15. 9 1
      clientv3/config.go
  16. 1 1
      clientv3/doc.go
  17. 0 113
      clientv3/example_auth_test.go
  18. 1 0
      clientv3/example_cluster_test.go
  19. 6 2
      clientv3/example_kv_test.go
  20. 1 0
      clientv3/example_lease_test.go
  21. 6 12
      clientv3/example_maintenence_test.go
  22. 5 4
      clientv3/example_metrics_test.go
  23. 4 3
      clientv3/example_test.go
  24. 1 0
      clientv3/example_watch_test.go
  25. 627 0
      clientv3/health_balancer.go
  26. 211 0
      clientv3/integration/black_hole_test.go
  27. 1 0
      clientv3/integration/cluster_test.go
  28. 28 13
      clientv3/integration/dial_test.go
  29. 32 66
      clientv3/integration/kv_test.go
  30. 22 14
      clientv3/integration/lease_test.go
  31. 10 2
      clientv3/integration/logger_test.go
  32. 2 2
      clientv3/integration/metrics_test.go
  33. 1 0
      clientv3/integration/mirror_test.go
  34. 2 1
      clientv3/integration/namespace_test.go
  35. 260 0
      clientv3/integration/network_partition_test.go
  36. 1 0
      clientv3/integration/role_test.go
  37. 352 0
      clientv3/integration/server_shutdown_test.go
  38. 3 1
      clientv3/integration/txn_test.go
  39. 2 1
      clientv3/integration/user_test.go
  40. 35 0
      clientv3/integration/util.go
  41. 34 16
      clientv3/integration/watch_test.go
  42. 24 21
      clientv3/kv.go
  43. 47 55
      clientv3/lease.go
  44. 24 10
      clientv3/logger.go
  45. 8 2
      clientv3/main_test.go
  46. 12 17
      clientv3/maintenance.go
  47. 1 0
      clientv3/mirror/syncer.go
  48. 2 2
      clientv3/namespace/kv.go
  49. 2 2
      clientv3/namespace/lease.go
  50. 2 2
      clientv3/namespace/watch.go
  51. 6 5
      clientv3/naming/grpc.go
  52. 8 5
      clientv3/naming/grpc_test.go
  53. 82 8
      clientv3/op.go
  54. 30 0
      clientv3/ready_wait.go
  55. 277 99
      clientv3/retry.go
  56. 6 22
      clientv3/txn.go
  57. 32 22
      clientv3/watch.go
  58. 10 3
      clientv3/yaml/config.go
  59. 5 5
      clientv3/yaml/config_test.go

+ 1 - 1
clientv3/README.md

@@ -1,6 +1,6 @@
 # etcd/clientv3
 
-[![Godoc](http://img.shields.io/badge/go-documentation-blue.svg?style=flat-square)](https://godoc.org/github.com/coreos/etcd/clientv3)
+[![Godoc](https://img.shields.io/badge/go-documentation-blue.svg?style=flat-square)](https://godoc.org/github.com/coreos/etcd/clientv3)
 
 `etcd/clientv3` is the official Go etcd client for v3.
 

+ 9 - 8
clientv3/auth.go

@@ -20,6 +20,7 @@ import (
 
 	"github.com/coreos/etcd/auth/authpb"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
+
 	"golang.org/x/net/context"
 	"google.golang.org/grpc"
 )
@@ -104,16 +105,16 @@ type auth struct {
 }
 
 func NewAuth(c *Client) Auth {
-	return &auth{remote: pb.NewAuthClient(c.ActiveConnection())}
+	return &auth{remote: RetryAuthClient(c)}
 }
 
 func (auth *auth) AuthEnable(ctx context.Context) (*AuthEnableResponse, error) {
-	resp, err := auth.remote.AuthEnable(ctx, &pb.AuthEnableRequest{}, grpc.FailFast(false))
+	resp, err := auth.remote.AuthEnable(ctx, &pb.AuthEnableRequest{})
 	return (*AuthEnableResponse)(resp), toErr(ctx, err)
 }
 
 func (auth *auth) AuthDisable(ctx context.Context) (*AuthDisableResponse, error) {
-	resp, err := auth.remote.AuthDisable(ctx, &pb.AuthDisableRequest{}, grpc.FailFast(false))
+	resp, err := auth.remote.AuthDisable(ctx, &pb.AuthDisableRequest{})
 	return (*AuthDisableResponse)(resp), toErr(ctx, err)
 }
 
@@ -138,12 +139,12 @@ func (auth *auth) UserGrantRole(ctx context.Context, user string, role string) (
 }
 
 func (auth *auth) UserGet(ctx context.Context, name string) (*AuthUserGetResponse, error) {
-	resp, err := auth.remote.UserGet(ctx, &pb.AuthUserGetRequest{Name: name}, grpc.FailFast(false))
+	resp, err := auth.remote.UserGet(ctx, &pb.AuthUserGetRequest{Name: name})
 	return (*AuthUserGetResponse)(resp), toErr(ctx, err)
 }
 
 func (auth *auth) UserList(ctx context.Context) (*AuthUserListResponse, error) {
-	resp, err := auth.remote.UserList(ctx, &pb.AuthUserListRequest{}, grpc.FailFast(false))
+	resp, err := auth.remote.UserList(ctx, &pb.AuthUserListRequest{})
 	return (*AuthUserListResponse)(resp), toErr(ctx, err)
 }
 
@@ -168,12 +169,12 @@ func (auth *auth) RoleGrantPermission(ctx context.Context, name string, key, ran
 }
 
 func (auth *auth) RoleGet(ctx context.Context, role string) (*AuthRoleGetResponse, error) {
-	resp, err := auth.remote.RoleGet(ctx, &pb.AuthRoleGetRequest{Role: role}, grpc.FailFast(false))
+	resp, err := auth.remote.RoleGet(ctx, &pb.AuthRoleGetRequest{Role: role})
 	return (*AuthRoleGetResponse)(resp), toErr(ctx, err)
 }
 
 func (auth *auth) RoleList(ctx context.Context) (*AuthRoleListResponse, error) {
-	resp, err := auth.remote.RoleList(ctx, &pb.AuthRoleListRequest{}, grpc.FailFast(false))
+	resp, err := auth.remote.RoleList(ctx, &pb.AuthRoleListRequest{})
 	return (*AuthRoleListResponse)(resp), toErr(ctx, err)
 }
 
@@ -201,7 +202,7 @@ type authenticator struct {
 }
 
 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}, grpc.FailFast(false))
+	resp, err := auth.remote.Authenticate(ctx, &pb.AuthenticateRequest{Name: name, Password: password})
 	return (*AuthenticateResponse)(resp), toErr(ctx, err)
 }
 

+ 0 - 356
clientv3/balancer.go

@@ -1,356 +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 (
-	"net/url"
-	"strings"
-	"sync"
-
-	"golang.org/x/net/context"
-	"google.golang.org/grpc"
-	"google.golang.org/grpc/codes"
-)
-
-// ErrNoAddrAvilable is returned by Get() when the balancer does not have
-// any active connection to endpoints at the time.
-// This error is returned only when opts.BlockingWait is true.
-var ErrNoAddrAvilable = grpc.Errorf(codes.Unavailable, "there is no address available")
-
-// simpleBalancer does the bare minimum to expose multiple eps
-// to the grpc reconnection code path
-type simpleBalancer struct {
-	// addrs are the client's endpoints for grpc
-	addrs []grpc.Address
-	// notifyCh notifies grpc of the set of addresses for connecting
-	notifyCh chan []grpc.Address
-
-	// readyc closes once the first connection is up
-	readyc    chan struct{}
-	readyOnce sync.Once
-
-	// mu protects upEps, pinAddr, and connectingAddr
-	mu sync.RWMutex
-
-	// upc closes when upEps transitions from empty to non-zero or the balancer closes.
-	upc chan struct{}
-
-	// downc closes when grpc calls down() on pinAddr
-	downc chan struct{}
-
-	// stopc is closed to signal updateNotifyLoop should stop.
-	stopc chan struct{}
-
-	// donec closes when all goroutines are exited
-	donec chan struct{}
-
-	// updateAddrsC notifies updateNotifyLoop to update addrs.
-	updateAddrsC chan struct{}
-
-	// 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 map[string]string
-
-	// pinAddr is the currently pinned address; set to the empty string on
-	// intialization and shutdown.
-	pinAddr string
-
-	closed bool
-}
-
-func newSimpleBalancer(eps []string) *simpleBalancer {
-	notifyCh := make(chan []grpc.Address, 1)
-	addrs := make([]grpc.Address, len(eps))
-	for i := range eps {
-		addrs[i].Addr = getHost(eps[i])
-	}
-	sb := &simpleBalancer{
-		addrs:        addrs,
-		notifyCh:     notifyCh,
-		readyc:       make(chan struct{}),
-		upc:          make(chan struct{}),
-		stopc:        make(chan struct{}),
-		downc:        make(chan struct{}),
-		donec:        make(chan struct{}),
-		updateAddrsC: make(chan struct{}, 1),
-		host2ep:      getHost2ep(eps),
-	}
-	close(sb.downc)
-	go sb.updateNotifyLoop()
-	return sb
-}
-
-func (b *simpleBalancer) Start(target string, config grpc.BalancerConfig) error { return nil }
-
-func (b *simpleBalancer) ConnectNotify() <-chan struct{} {
-	b.mu.Lock()
-	defer b.mu.Unlock()
-	return b.upc
-}
-
-func (b *simpleBalancer) getEndpoint(host string) string {
-	b.mu.Lock()
-	defer b.mu.Unlock()
-	return b.host2ep[host]
-}
-
-func getHost2ep(eps []string) map[string]string {
-	hm := make(map[string]string, len(eps))
-	for i := range eps {
-		_, host, _ := parseEndpoint(eps[i])
-		hm[host] = eps[i]
-	}
-	return hm
-}
-
-func (b *simpleBalancer) updateAddrs(eps []string) {
-	np := getHost2ep(eps)
-
-	b.mu.Lock()
-
-	match := len(np) == len(b.host2ep)
-	for k, v := range np {
-		if b.host2ep[k] != v {
-			match = false
-			break
-		}
-	}
-	if match {
-		// same endpoints, so no need to update address
-		b.mu.Unlock()
-		return
-	}
-
-	b.host2ep = np
-
-	addrs := make([]grpc.Address, 0, len(eps))
-	for i := range eps {
-		addrs = append(addrs, grpc.Address{Addr: getHost(eps[i])})
-	}
-	b.addrs = addrs
-
-	// updating notifyCh can trigger new connections,
-	// only update addrs if all connections are down
-	// or addrs does not include pinAddr.
-	update := !hasAddr(addrs, b.pinAddr)
-	b.mu.Unlock()
-
-	if update {
-		select {
-		case b.updateAddrsC <- struct{}{}:
-		case <-b.stopc:
-		}
-	}
-}
-
-func hasAddr(addrs []grpc.Address, targetAddr string) bool {
-	for _, addr := range addrs {
-		if targetAddr == addr.Addr {
-			return true
-		}
-	}
-	return false
-}
-
-func (b *simpleBalancer) updateNotifyLoop() {
-	defer close(b.donec)
-
-	for {
-		b.mu.RLock()
-		upc, downc, addr := b.upc, b.downc, b.pinAddr
-		b.mu.RUnlock()
-		// downc or upc should be closed
-		select {
-		case <-downc:
-			downc = nil
-		default:
-		}
-		select {
-		case <-upc:
-			upc = nil
-		default:
-		}
-		switch {
-		case downc == nil && upc == nil:
-			// stale
-			select {
-			case <-b.stopc:
-				return
-			default:
-			}
-		case downc == nil:
-			b.notifyAddrs()
-			select {
-			case <-upc:
-			case <-b.updateAddrsC:
-				b.notifyAddrs()
-			case <-b.stopc:
-				return
-			}
-		case upc == nil:
-			select {
-			// close connections that are not the pinned address
-			case b.notifyCh <- []grpc.Address{{Addr: addr}}:
-			case <-downc:
-			case <-b.stopc:
-				return
-			}
-			select {
-			case <-downc:
-			case <-b.updateAddrsC:
-			case <-b.stopc:
-				return
-			}
-			b.notifyAddrs()
-		}
-	}
-}
-
-func (b *simpleBalancer) notifyAddrs() {
-	b.mu.RLock()
-	addrs := b.addrs
-	b.mu.RUnlock()
-	select {
-	case b.notifyCh <- addrs:
-	case <-b.stopc:
-	}
-}
-
-func (b *simpleBalancer) Up(addr grpc.Address) func(error) {
-	b.mu.Lock()
-	defer b.mu.Unlock()
-
-	// gRPC might call Up after it called Close. We add this check
-	// to "fix" it up at application layer. Or our simplerBalancer
-	// might panic since b.upc is closed.
-	if b.closed {
-		return func(err error) {}
-	}
-	// gRPC might call Up on a stale address.
-	// Prevent updating pinAddr with a stale address.
-	if !hasAddr(b.addrs, addr.Addr) {
-		return func(err error) {}
-	}
-	if b.pinAddr != "" {
-		return func(err error) {}
-	}
-	// notify waiting Get()s and pin first connected address
-	close(b.upc)
-	b.downc = make(chan struct{})
-	b.pinAddr = addr.Addr
-	// notify client that a connection is up
-	b.readyOnce.Do(func() { close(b.readyc) })
-	return func(err error) {
-		b.mu.Lock()
-		b.upc = make(chan struct{})
-		close(b.downc)
-		b.pinAddr = ""
-		b.mu.Unlock()
-	}
-}
-
-func (b *simpleBalancer) Get(ctx context.Context, opts grpc.BalancerGetOptions) (grpc.Address, func(), error) {
-	var (
-		addr   string
-		closed bool
-	)
-
-	// If opts.BlockingWait is false (for fail-fast RPCs), it should return
-	// an address it has notified via Notify immediately instead of blocking.
-	if !opts.BlockingWait {
-		b.mu.RLock()
-		closed = b.closed
-		addr = b.pinAddr
-		b.mu.RUnlock()
-		if closed {
-			return grpc.Address{Addr: ""}, nil, grpc.ErrClientConnClosing
-		}
-		if addr == "" {
-			return grpc.Address{Addr: ""}, nil, ErrNoAddrAvilable
-		}
-		return grpc.Address{Addr: addr}, func() {}, nil
-	}
-
-	for {
-		b.mu.RLock()
-		ch := b.upc
-		b.mu.RUnlock()
-		select {
-		case <-ch:
-		case <-b.donec:
-			return grpc.Address{Addr: ""}, nil, grpc.ErrClientConnClosing
-		case <-ctx.Done():
-			return grpc.Address{Addr: ""}, nil, ctx.Err()
-		}
-		b.mu.RLock()
-		closed = b.closed
-		addr = b.pinAddr
-		b.mu.RUnlock()
-		// Close() which sets b.closed = true can be called before Get(), Get() must exit if balancer is closed.
-		if closed {
-			return grpc.Address{Addr: ""}, nil, grpc.ErrClientConnClosing
-		}
-		if addr != "" {
-			break
-		}
-	}
-	return grpc.Address{Addr: addr}, func() {}, nil
-}
-
-func (b *simpleBalancer) Notify() <-chan []grpc.Address { return b.notifyCh }
-
-func (b *simpleBalancer) Close() error {
-	b.mu.Lock()
-	// In case gRPC calls close twice. TODO: remove the checking
-	// when we are sure that gRPC wont call close twice.
-	if b.closed {
-		b.mu.Unlock()
-		<-b.donec
-		return nil
-	}
-	b.closed = true
-	close(b.stopc)
-	b.pinAddr = ""
-
-	// In the case of following scenario:
-	//	1. upc is not closed; no pinned address
-	// 	2. client issues an rpc, calling invoke(), which calls Get(), enters for loop, blocks
-	// 	3. clientconn.Close() calls balancer.Close(); closed = true
-	// 	4. for loop in Get() never exits since ctx is the context passed in by the client and may not be canceled
-	// we must close upc so Get() exits from blocking on upc
-	select {
-	case <-b.upc:
-	default:
-		// terminate all waiting Get()s
-		close(b.upc)
-	}
-
-	b.mu.Unlock()
-
-	// wait for updateNotifyLoop to finish
-	<-b.donec
-	close(b.notifyCh)
-
-	return nil
-}
-
-func getHost(ep string) string {
-	url, uerr := url.Parse(ep)
-	if uerr != nil || !strings.Contains(ep, "://") {
-		return ep
-	}
-	return url.Host
-}

+ 54 - 28
clientv3/client.go

@@ -31,7 +31,9 @@ import (
 	"google.golang.org/grpc"
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/credentials"
+	"google.golang.org/grpc/keepalive"
 	"google.golang.org/grpc/metadata"
+	"google.golang.org/grpc/status"
 )
 
 var (
@@ -51,18 +53,17 @@ type Client struct {
 	conn     *grpc.ClientConn
 	dialerrc chan error
 
-	cfg              Config
-	creds            *credentials.TransportCredentials
-	balancer         *simpleBalancer
-	retryWrapper     retryRpcFunc
-	retryAuthWrapper retryRpcFunc
+	cfg      Config
+	creds    *credentials.TransportCredentials
+	balancer *healthBalancer
+	mu       sync.Mutex
 
 	ctx    context.Context
 	cancel context.CancelFunc
 
-	// Username is a username for authentication
+	// Username is a user name for authentication.
 	Username string
-	// Password is a password for authentication
+	// Password is a password for authentication.
 	Password string
 	// tokenCred is an instance of WithPerRPCCredentials()'s argument
 	tokenCred *authTokenCredential
@@ -116,8 +117,23 @@ func (c *Client) Endpoints() (eps []string) {
 
 // SetEndpoints updates client's endpoints.
 func (c *Client) SetEndpoints(eps ...string) {
+	c.mu.Lock()
 	c.cfg.Endpoints = eps
-	c.balancer.updateAddrs(eps)
+	c.mu.Unlock()
+	c.balancer.updateAddrs(eps...)
+
+	// updating notifyCh can trigger new connections,
+	// need update addrs if all connections are down
+	// or addrs does not include pinAddr.
+	c.balancer.mu.RLock()
+	update := !hasAddr(c.balancer.addrs, c.balancer.pinAddr)
+	c.balancer.mu.RUnlock()
+	if update {
+		select {
+		case c.balancer.updateAddrsC <- notifyNext:
+		case <-c.balancer.stopc:
+		}
+	}
 }
 
 // Sync synchronizes client's endpoints with the known endpoints from the etcd membership.
@@ -144,8 +160,10 @@ func (c *Client) autoSync() {
 		case <-c.ctx.Done():
 			return
 		case <-time.After(c.cfg.AutoSyncInterval):
-			ctx, _ := context.WithTimeout(c.ctx, 5*time.Second)
-			if err := c.Sync(ctx); err != nil && err != c.ctx.Err() {
+			ctx, cancel := context.WithTimeout(c.ctx, 5*time.Second)
+			err := c.Sync(ctx)
+			cancel()
+			if err != nil && err != c.ctx.Err() {
 				logger.Println("Auto sync endpoints failed:", err)
 			}
 		}
@@ -174,7 +192,7 @@ func parseEndpoint(endpoint string) (proto string, host string, scheme string) {
 	host = endpoint
 	url, uerr := url.Parse(endpoint)
 	if uerr != nil || !strings.Contains(endpoint, "://") {
-		return
+		return proto, host, scheme
 	}
 	scheme = url.Scheme
 
@@ -188,7 +206,7 @@ func parseEndpoint(endpoint string) (proto string, host string, scheme string) {
 	default:
 		proto, host = "", ""
 	}
-	return
+	return proto, host, scheme
 }
 
 func (c *Client) processCreds(scheme string) (creds *credentials.TransportCredentials) {
@@ -207,7 +225,7 @@ func (c *Client) processCreds(scheme string) (creds *credentials.TransportCreden
 	default:
 		creds = nil
 	}
-	return
+	return creds
 }
 
 // dialSetupOpts gives the dial opts prior to any authentication
@@ -215,10 +233,17 @@ func (c *Client) dialSetupOpts(endpoint string, dopts ...grpc.DialOption) (opts
 	if c.cfg.DialTimeout > 0 {
 		opts = []grpc.DialOption{grpc.WithTimeout(c.cfg.DialTimeout)}
 	}
+	if c.cfg.DialKeepAliveTime > 0 {
+		params := keepalive.ClientParameters{
+			Time:    c.cfg.DialKeepAliveTime,
+			Timeout: c.cfg.DialKeepAliveTimeout,
+		}
+		opts = append(opts, grpc.WithKeepaliveParams(params))
+	}
 	opts = append(opts, dopts...)
 
 	f := func(host string, t time.Duration) (net.Conn, error) {
-		proto, host, _ := parseEndpoint(c.balancer.getEndpoint(host))
+		proto, host, _ := parseEndpoint(c.balancer.endpoint(host))
 		if host == "" && endpoint != "" {
 			// dialing an endpoint not in the balancer; use
 			// endpoint passed into dial
@@ -311,7 +336,7 @@ func (c *Client) dial(endpoint string, dopts ...grpc.DialOption) (*grpc.ClientCo
 		if err != nil {
 			if toErr(ctx, err) != rpctypes.ErrAuthNotEnabled {
 				if err == ctx.Err() && ctx.Err() != c.ctx.Err() {
-					err = grpc.ErrClientConnTimeout
+					err = context.DeadlineExceeded
 				}
 				return nil, err
 			}
@@ -333,7 +358,7 @@ func (c *Client) dial(endpoint string, dopts ...grpc.DialOption) (*grpc.ClientCo
 // when the cluster has a leader.
 func WithRequireLeader(ctx context.Context) context.Context {
 	md := metadata.Pairs(rpctypes.MetadataRequireLeaderKey, rpctypes.MetadataHasLeader)
-	return metadata.NewContext(ctx, md)
+	return metadata.NewOutgoingContext(ctx, md)
 }
 
 func newClient(cfg *Config) (*Client, error) {
@@ -366,9 +391,12 @@ func newClient(cfg *Config) (*Client, error) {
 		client.Password = cfg.Password
 	}
 
-	client.balancer = newSimpleBalancer(cfg.Endpoints)
+	client.balancer = newHealthBalancer(cfg.Endpoints, cfg.DialTimeout, func(ep string) (bool, error) {
+		return grpcHealthCheck(client, ep)
+	})
+
 	// use Endpoints[0] so that for https:// without any tls config given, then
-	// grpc will assume the ServerName is in the endpoint.
+	// grpc will assume the certificate server name is the endpoint host.
 	conn, err := client.dial(cfg.Endpoints[0], grpc.WithBalancer(client.balancer))
 	if err != nil {
 		client.cancel()
@@ -376,21 +404,19 @@ func newClient(cfg *Config) (*Client, error) {
 		return nil, err
 	}
 	client.conn = conn
-	client.retryWrapper = client.newRetryWrapper()
-	client.retryAuthWrapper = client.newAuthRetryWrapper()
 
 	// wait for a connection
 	if cfg.DialTimeout > 0 {
 		hasConn := false
 		waitc := time.After(cfg.DialTimeout)
 		select {
-		case <-client.balancer.readyc:
+		case <-client.balancer.ready():
 			hasConn = true
 		case <-ctx.Done():
 		case <-waitc:
 		}
 		if !hasConn {
-			err := grpc.ErrClientConnTimeout
+			err := context.DeadlineExceeded
 			select {
 			case err = <-client.dialerrc:
 			default:
@@ -425,7 +451,7 @@ func (c *Client) checkVersion() (err error) {
 	errc := make(chan error, len(c.cfg.Endpoints))
 	ctx, cancel := context.WithCancel(c.ctx)
 	if c.cfg.DialTimeout > 0 {
-		ctx, _ = context.WithTimeout(ctx, c.cfg.DialTimeout)
+		ctx, cancel = context.WithTimeout(ctx, c.cfg.DialTimeout)
 	}
 	wg.Add(len(c.cfg.Endpoints))
 	for _, ep := range c.cfg.Endpoints {
@@ -440,7 +466,7 @@ func (c *Client) checkVersion() (err error) {
 			vs := strings.Split(resp.Version, ".")
 			maj, min := 0, 0
 			if len(vs) >= 2 {
-				maj, rerr = strconv.Atoi(vs[0])
+				maj, _ = strconv.Atoi(vs[0])
 				min, rerr = strconv.Atoi(vs[1])
 			}
 			if maj < 3 || (maj == 3 && min < 2) {
@@ -472,14 +498,14 @@ func isHaltErr(ctx context.Context, err error) bool {
 	if err == nil {
 		return false
 	}
-	code := grpc.Code(err)
+	ev, _ := status.FromError(err)
 	// Unavailable codes mean the system will be right back.
 	// (e.g., can't connect, lost leader)
 	// Treat Internal codes as if something failed, leaving the
 	// system in an inconsistent state, but retrying could make progress.
 	// (e.g., failed in middle of send, corrupted frame)
 	// TODO: are permanent Internal errors possible from grpc?
-	return code != codes.Unavailable && code != codes.Internal
+	return ev.Code() != codes.Unavailable && ev.Code() != codes.Internal
 }
 
 func toErr(ctx context.Context, err error) error {
@@ -490,7 +516,8 @@ func toErr(ctx context.Context, err error) error {
 	if _, ok := err.(rpctypes.EtcdError); ok {
 		return err
 	}
-	code := grpc.Code(err)
+	ev, _ := status.FromError(err)
+	code := ev.Code()
 	switch code {
 	case codes.DeadlineExceeded:
 		fallthrough
@@ -499,7 +526,6 @@ func toErr(ctx context.Context, err error) error {
 			err = ctx.Err()
 		}
 	case codes.Unavailable:
-		err = ErrNoAvailableEndpoints
 	case codes.FailedPrecondition:
 		err = grpc.ErrClientConnClosing
 	}

+ 5 - 5
clientv3/client_test.go

@@ -22,8 +22,8 @@ import (
 
 	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
 	"github.com/coreos/etcd/pkg/testutil"
+
 	"golang.org/x/net/context"
-	"google.golang.org/grpc"
 )
 
 func TestDialCancel(t *testing.T) {
@@ -45,7 +45,7 @@ func TestDialCancel(t *testing.T) {
 		t.Fatal(err)
 	}
 
-	// connect to ipv4 blackhole so dial blocks
+	// connect to ipv4 black hole so dial blocks
 	c.SetEndpoints("http://254.0.0.1:12345")
 
 	// issue Get to force redial attempts
@@ -97,7 +97,7 @@ func TestDialTimeout(t *testing.T) {
 	for i, cfg := range testCfgs {
 		donec := make(chan error)
 		go func() {
-			// without timeout, dial continues forever on ipv4 blackhole
+			// without timeout, dial continues forever on ipv4 black hole
 			c, err := New(cfg)
 			if c != nil || err == nil {
 				t.Errorf("#%d: new client should fail", i)
@@ -117,8 +117,8 @@ func TestDialTimeout(t *testing.T) {
 		case <-time.After(5 * time.Second):
 			t.Errorf("#%d: failed to timeout dial on time", i)
 		case err := <-donec:
-			if err != grpc.ErrClientConnTimeout {
-				t.Errorf("#%d: unexpected error %v, want %v", i, err, grpc.ErrClientConnTimeout)
+			if err != context.DeadlineExceeded {
+				t.Errorf("#%d: unexpected error %v, want %v", i, err, context.DeadlineExceeded)
 			}
 		}
 	}

+ 3 - 2
clientv3/clientv3util/example_key_test.go

@@ -15,11 +15,12 @@
 package clientv3util_test
 
 import (
-	"context"
 	"log"
 
 	"github.com/coreos/etcd/clientv3"
 	"github.com/coreos/etcd/clientv3/clientv3util"
+
+	"golang.org/x/net/context"
 )
 
 func ExampleKeyExists_put() {
@@ -33,7 +34,7 @@ func ExampleKeyExists_put() {
 	kvc := clientv3.NewKV(cli)
 
 	// perform a put only if key is missing
-	// It is useful to do the check (transactionally) to avoid overwriting
+	// It is useful to do the check atomically to avoid overwriting
 	// the existing key which would generate potentially unwanted events,
 	// unless of course you wanted to do an overwrite no matter what.
 	_, err = kvc.Txn(context.Background()).

+ 10 - 18
clientv3/cluster.go

@@ -16,8 +16,8 @@ package clientv3
 
 import (
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
+
 	"golang.org/x/net/context"
-	"google.golang.org/grpc"
 )
 
 type (
@@ -74,27 +74,19 @@ func (c *cluster) MemberRemove(ctx context.Context, id uint64) (*MemberRemoveRes
 
 func (c *cluster) MemberUpdate(ctx context.Context, id uint64, peerAddrs []string) (*MemberUpdateResponse, error) {
 	// it is safe to retry on update.
-	for {
-		r := &pb.MemberUpdateRequest{ID: id, PeerURLs: peerAddrs}
-		resp, err := c.remote.MemberUpdate(ctx, r, grpc.FailFast(false))
-		if err == nil {
-			return (*MemberUpdateResponse)(resp), nil
-		}
-		if isHaltErr(ctx, err) {
-			return nil, toErr(ctx, err)
-		}
+	r := &pb.MemberUpdateRequest{ID: id, PeerURLs: peerAddrs}
+	resp, err := c.remote.MemberUpdate(ctx, r)
+	if err == nil {
+		return (*MemberUpdateResponse)(resp), nil
 	}
+	return nil, toErr(ctx, err)
 }
 
 func (c *cluster) MemberList(ctx context.Context) (*MemberListResponse, error) {
 	// it is safe to retry on list.
-	for {
-		resp, err := c.remote.MemberList(ctx, &pb.MemberListRequest{}, grpc.FailFast(false))
-		if err == nil {
-			return (*MemberListResponse)(resp), nil
-		}
-		if isHaltErr(ctx, err) {
-			return nil, toErr(ctx, err)
-		}
+	resp, err := c.remote.MemberList(ctx, &pb.MemberListRequest{})
+	if err == nil {
+		return (*MemberListResponse)(resp), nil
 	}
+	return nil, toErr(ctx, err)
 }

+ 2 - 4
clientv3/compact_op.go

@@ -44,10 +44,8 @@ func (op CompactOp) toRequest() *pb.CompactionRequest {
 	return &pb.CompactionRequest{Revision: op.revision, Physical: op.physical}
 }
 
-// WithCompactPhysical makes compact RPC call wait until
-// the compaction is physically applied to the local database
-// such that compacted entries are totally removed from the
-// backend database.
+// WithCompactPhysical makes Compact wait until all compacted entries are
+// removed from the etcd server's storage.
 func WithCompactPhysical() CompactOption {
 	return func(op *CompactOp) { op.physical = true }
 }

+ 10 - 0
clientv3/compare.go

@@ -99,6 +99,7 @@ func (cmp *Cmp) ValueBytes() []byte {
 // WithValueBytes sets the byte slice for the comparison's value.
 func (cmp *Cmp) WithValueBytes(v []byte) { cmp.TargetUnion.(*pb.Compare_Value).Value = v }
 
+// mustInt64 panics if val isn't an int or int64. It returns an int64 otherwise.
 func mustInt64(val interface{}) int64 {
 	if v, ok := val.(int64); ok {
 		return v
@@ -108,3 +109,12 @@ func mustInt64(val interface{}) int64 {
 	}
 	panic("bad value")
 }
+
+// mustInt64orLeaseID panics if val isn't a LeaseID, int or int64. It returns an
+// int64 otherwise.
+func mustInt64orLeaseID(val interface{}) int64 {
+	if v, ok := val.(LeaseID); ok {
+		return int64(v)
+	}
+	return mustInt64(val)
+}

+ 6 - 3
clientv3/concurrency/election.go

@@ -21,6 +21,7 @@ import (
 	v3 "github.com/coreos/etcd/clientv3"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 	"github.com/coreos/etcd/mvcc/mvccpb"
+
 	"golang.org/x/net/context"
 )
 
@@ -185,12 +186,12 @@ func (e *Election) observe(ctx context.Context, ch chan<- v3.GetResponse) {
 					cancel()
 					return
 				}
-				// only accept PUTs; a DELETE will make observe() spin
+				// only accept puts; a delete will make observe() spin
 				for _, ev := range wr.Events {
 					if ev.Type == mvccpb.PUT {
 						hdr, kv = &wr.Header, ev.Kv
 						// may have multiple revs; hdr.rev = the last rev
-						// set to kv's rev in case batch has multiple PUTs
+						// set to kv's rev in case batch has multiple Puts
 						hdr.Revision = kv.ModRevision
 						break
 					}
@@ -213,6 +214,7 @@ func (e *Election) observe(ctx context.Context, ch chan<- v3.GetResponse) {
 		for !keyDeleted {
 			wr, ok := <-wch
 			if !ok {
+				cancel()
 				return
 			}
 			for _, ev := range wr.Events {
@@ -225,6 +227,7 @@ func (e *Election) observe(ctx context.Context, ch chan<- v3.GetResponse) {
 				select {
 				case ch <- *resp:
 				case <-cctx.Done():
+					cancel()
 					return
 				}
 			}
@@ -240,4 +243,4 @@ func (e *Election) Key() string { return e.leaderKey }
 func (e *Election) Rev() int64 { return e.leaderRev }
 
 // Header is the response header from the last successful election proposal.
-func (m *Election) Header() *pb.ResponseHeader { return m.hdr }
+func (e *Election) Header() *pb.ResponseHeader { return e.hdr }

+ 1 - 0
clientv3/concurrency/key.go

@@ -20,6 +20,7 @@ import (
 	v3 "github.com/coreos/etcd/clientv3"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 	"github.com/coreos/etcd/mvcc/mvccpb"
+
 	"golang.org/x/net/context"
 )
 

+ 10 - 1
clientv3/concurrency/mutex.go

@@ -20,6 +20,7 @@ import (
 
 	v3 "github.com/coreos/etcd/clientv3"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
+
 	"golang.org/x/net/context"
 )
 
@@ -49,7 +50,9 @@ func (m *Mutex) Lock(ctx context.Context) error {
 	put := v3.OpPut(m.myKey, "", v3.WithLease(s.Lease()))
 	// reuse key in case this session already holds the lock
 	get := v3.OpGet(m.myKey)
-	resp, err := client.Txn(ctx).If(cmp).Then(put).Else(get).Commit()
+	// fetch current holder to complete uncontended path with only one RPC
+	getOwner := v3.OpGet(m.pfx, v3.WithFirstCreate()...)
+	resp, err := client.Txn(ctx).If(cmp).Then(put, getOwner).Else(get, getOwner).Commit()
 	if err != nil {
 		return err
 	}
@@ -57,6 +60,12 @@ func (m *Mutex) Lock(ctx context.Context) error {
 	if !resp.Succeeded {
 		m.myRev = resp.Responses[0].GetResponseRange().Kvs[0].CreateRevision
 	}
+	// if no key on prefix / the minimum rev is key, already hold the lock
+	ownerKey := resp.Responses[1].GetResponseRange().Kvs
+	if len(ownerKey) == 0 || ownerKey[0].CreateRevision == m.myRev {
+		m.hdr = resp.Header
+		return nil
+	}
 
 	// wait for deletion revisions prior to myKey
 	hdr, werr := waitDeletes(ctx, client, m.pfx, m.myRev-1)

+ 2 - 0
clientv3/concurrency/session.go

@@ -18,6 +18,7 @@ import (
 	"time"
 
 	v3 "github.com/coreos/etcd/clientv3"
+
 	"golang.org/x/net/context"
 )
 
@@ -53,6 +54,7 @@ func NewSession(client *v3.Client, opts ...SessionOption) (*Session, error) {
 	ctx, cancel := context.WithCancel(ops.ctx)
 	keepAlive, err := client.KeepAlive(ctx, id)
 	if err != nil || keepAlive == nil {
+		cancel()
 		return nil, err
 	}
 

+ 3 - 2
clientv3/concurrency/stm.go

@@ -18,6 +18,7 @@ import (
 	"math"
 
 	v3 "github.com/coreos/etcd/clientv3"
+
 	"golang.org/x/net/context"
 )
 
@@ -46,7 +47,7 @@ const (
 	// SerializableSnapshot provides serializable isolation and also checks
 	// for write conflicts.
 	SerializableSnapshot Isolation = iota
-	// Serializable reads within the same transactiona attempt return data
+	// Serializable reads within the same transaction attempt return data
 	// from the at the revision of the first read.
 	Serializable
 	// RepeatableReads reads within the same transaction attempt always
@@ -85,7 +86,7 @@ func WithPrefetch(keys ...string) stmOption {
 	return func(so *stmOptions) { so.prefetch = append(so.prefetch, keys...) }
 }
 
-// NewSTM initiates a new STM instance, using snapshot isolation by default.
+// NewSTM initiates a new STM instance, using serializable snapshot isolation by default.
 func NewSTM(c *v3.Client, apply func(STM) error, so ...stmOption) (*v3.TxnResponse, error) {
 	opts := &stmOptions{ctx: c.Ctx()}
 	for _, f := range so {

+ 9 - 1
clientv3/config.go

@@ -33,10 +33,18 @@ type Config struct {
 	// DialTimeout is the timeout for failing to establish a connection.
 	DialTimeout time.Duration `json:"dial-timeout"`
 
+	// DialKeepAliveTime is the time in seconds after which client pings the server to see if
+	// transport is alive.
+	DialKeepAliveTime time.Duration `json:"dial-keep-alive-time"`
+
+	// DialKeepAliveTimeout is the time in seconds that the client waits for a response for the
+	// keep-alive probe.  If the response is not received in this time, the connection is closed.
+	DialKeepAliveTimeout time.Duration `json:"dial-keep-alive-timeout"`
+
 	// TLS holds the client secure credentials, if any.
 	TLS *tls.Config
 
-	// Username is a username for authentication.
+	// Username is a user name for authentication.
 	Username string `json:"username"`
 
 	// Password is a password for authentication.

+ 1 - 1
clientv3/doc.go

@@ -28,7 +28,7 @@
 // Make sure to close the client after using it. If the client is not closed, the
 // connection will have leaky goroutines.
 //
-// To specify client request timeout, pass context.WithTimeout to APIs:
+// To specify a client request timeout, wrap the context with context.WithTimeout:
 //
 //	ctx, cancel := context.WithTimeout(context.Background(), timeout)
 //	resp, err := kvc.Put(ctx, "sample_key", "sample_value")

+ 0 - 113
clientv3/example_auth_test.go

@@ -1,113 +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_test
-
-import (
-	"fmt"
-	"log"
-
-	"github.com/coreos/etcd/clientv3"
-	"golang.org/x/net/context"
-)
-
-func ExampleAuth() {
-	cli, err := clientv3.New(clientv3.Config{
-		Endpoints:   endpoints,
-		DialTimeout: dialTimeout,
-	})
-	if err != nil {
-		log.Fatal(err)
-	}
-	defer cli.Close()
-
-	if _, err = cli.RoleAdd(context.TODO(), "root"); err != nil {
-		log.Fatal(err)
-	}
-	if _, err = cli.UserAdd(context.TODO(), "root", "123"); err != nil {
-		log.Fatal(err)
-	}
-	if _, err = cli.UserGrantRole(context.TODO(), "root", "root"); err != nil {
-		log.Fatal(err)
-	}
-
-	if _, err = cli.RoleAdd(context.TODO(), "r"); err != nil {
-		log.Fatal(err)
-	}
-
-	if _, err = cli.RoleGrantPermission(
-		context.TODO(),
-		"r",   // role name
-		"foo", // key
-		"zoo", // range end
-		clientv3.PermissionType(clientv3.PermReadWrite),
-	); err != nil {
-		log.Fatal(err)
-	}
-	if _, err = cli.UserAdd(context.TODO(), "u", "123"); err != nil {
-		log.Fatal(err)
-	}
-	if _, err = cli.UserGrantRole(context.TODO(), "u", "r"); err != nil {
-		log.Fatal(err)
-	}
-	if _, err = cli.AuthEnable(context.TODO()); err != nil {
-		log.Fatal(err)
-	}
-
-	cliAuth, err := clientv3.New(clientv3.Config{
-		Endpoints:   endpoints,
-		DialTimeout: dialTimeout,
-		Username:    "u",
-		Password:    "123",
-	})
-	if err != nil {
-		log.Fatal(err)
-	}
-	defer cliAuth.Close()
-
-	if _, err = cliAuth.Put(context.TODO(), "foo1", "bar"); err != nil {
-		log.Fatal(err)
-	}
-
-	_, err = cliAuth.Txn(context.TODO()).
-		If(clientv3.Compare(clientv3.Value("zoo1"), ">", "abc")).
-		Then(clientv3.OpPut("zoo1", "XYZ")).
-		Else(clientv3.OpPut("zoo1", "ABC")).
-		Commit()
-	fmt.Println(err)
-
-	// now check the permission with the root account
-	rootCli, err := clientv3.New(clientv3.Config{
-		Endpoints:   endpoints,
-		DialTimeout: dialTimeout,
-		Username:    "root",
-		Password:    "123",
-	})
-	if err != nil {
-		log.Fatal(err)
-	}
-	defer rootCli.Close()
-
-	resp, err := rootCli.RoleGet(context.TODO(), "r")
-	if err != nil {
-		log.Fatal(err)
-	}
-	fmt.Printf("user u permission: key %q, range end %q\n", resp.Perm[0].Key, resp.Perm[0].RangeEnd)
-
-	if _, err = rootCli.AuthDisable(context.TODO()); err != nil {
-		log.Fatal(err)
-	}
-	// Output: etcdserver: permission denied
-	// user u permission: key "foo", range end "zoo"
-}

+ 1 - 0
clientv3/example_cluster_test.go

@@ -19,6 +19,7 @@ import (
 	"log"
 
 	"github.com/coreos/etcd/clientv3"
+
 	"golang.org/x/net/context"
 )
 

+ 6 - 2
clientv3/example_kv_test.go

@@ -20,6 +20,7 @@ import (
 
 	"github.com/coreos/etcd/clientv3"
 	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
+
 	"golang.org/x/net/context"
 )
 
@@ -236,8 +237,11 @@ func ExampleKV_txn() {
 
 	ctx, cancel := context.WithTimeout(context.Background(), requestTimeout)
 	_, err = kvc.Txn(ctx).
-		If(clientv3.Compare(clientv3.Value("key"), ">", "abc")). // txn value comparisons are lexical
-		Then(clientv3.OpPut("key", "XYZ")).                      // this runs, since 'xyz' > 'abc'
+		// txn value comparisons are lexical
+		If(clientv3.Compare(clientv3.Value("key"), ">", "abc")).
+		// the "Then" runs, since "xyz" > "abc"
+		Then(clientv3.OpPut("key", "XYZ")).
+		// the "Else" does not run
 		Else(clientv3.OpPut("key", "ABC")).
 		Commit()
 	cancel()

+ 1 - 0
clientv3/example_lease_test.go

@@ -19,6 +19,7 @@ import (
 	"log"
 
 	"github.com/coreos/etcd/clientv3"
+
 	"golang.org/x/net/context"
 )
 

+ 6 - 12
clientv3/example_maintenence_test.go

@@ -18,9 +18,8 @@ import (
 	"fmt"
 	"log"
 
-	"golang.org/x/net/context"
-
 	"github.com/coreos/etcd/clientv3"
+	"golang.org/x/net/context"
 )
 
 func ExampleMaintenance_status() {
@@ -34,20 +33,15 @@ func ExampleMaintenance_status() {
 		}
 		defer cli.Close()
 
-		// resp, err := cli.Status(context.Background(), ep)
-		//
-		// or
-		//
-		mapi := clientv3.NewMaintenance(cli)
-		resp, err := mapi.Status(context.Background(), ep)
+		resp, err := cli.Status(context.Background(), ep)
 		if err != nil {
 			log.Fatal(err)
 		}
-		fmt.Printf("endpoint: %s / IsLeader: %v\n", ep, resp.Header.MemberId == resp.Leader)
+		fmt.Printf("endpoint: %s / Leader: %v\n", ep, resp.Header.MemberId == resp.Leader)
 	}
-	// endpoint: localhost:2379 / IsLeader: false
-	// endpoint: localhost:22379 / IsLeader: false
-	// endpoint: localhost:32379 / IsLeader: true
+	// endpoint: localhost:2379 / Leader: false
+	// endpoint: localhost:22379 / Leader: false
+	// endpoint: localhost:32379 / Leader: true
 }
 
 func ExampleMaintenance_defragment() {

+ 5 - 4
clientv3/example_metrics_test.go

@@ -43,10 +43,10 @@ func ExampleClient_metrics() {
 	}
 	defer cli.Close()
 
-	// get a key so it shows up in the metrics as a range rpc
+	// get a key so it shows up in the metrics as a range RPC
 	cli.Get(context.TODO(), "test_key")
 
-	// listen for all prometheus metrics
+	// listen for all Prometheus metrics
 	ln, err := net.Listen("tcp", ":0")
 	if err != nil {
 		log.Fatal(err)
@@ -61,7 +61,7 @@ func ExampleClient_metrics() {
 		<-donec
 	}()
 
-	// make an http request to fetch all prometheus metrics
+	// make an http request to fetch all Prometheus metrics
 	url := "http://" + ln.Addr().String() + "/metrics"
 	resp, err := http.Get(url)
 	if err != nil {
@@ -80,5 +80,6 @@ func ExampleClient_metrics() {
 			break
 		}
 	}
-	// Output: grpc_client_started_total{grpc_method="Range",grpc_service="etcdserverpb.KV",grpc_type="unary"} 1
+	// Output:
+	//	grpc_client_started_total{grpc_method="Range",grpc_service="etcdserverpb.KV",grpc_type="unary"} 1
 }

+ 4 - 3
clientv3/example_test.go

@@ -16,12 +16,14 @@ package clientv3_test
 
 import (
 	"log"
+	"os"
 	"time"
 
 	"github.com/coreos/etcd/clientv3"
 	"github.com/coreos/etcd/pkg/transport"
-	"github.com/coreos/pkg/capnslog"
+
 	"golang.org/x/net/context"
+	"google.golang.org/grpc/grpclog"
 )
 
 var (
@@ -31,8 +33,7 @@ var (
 )
 
 func Example() {
-	var plog = capnslog.NewPackageLogger("github.com/coreos/etcd", "clientv3")
-	clientv3.SetLogger(plog)
+	clientv3.SetLogger(grpclog.NewLoggerV2(os.Stderr, os.Stderr, os.Stderr))
 
 	cli, err := clientv3.New(clientv3.Config{
 		Endpoints:   endpoints,

+ 1 - 0
clientv3/example_watch_test.go

@@ -19,6 +19,7 @@ import (
 	"log"
 
 	"github.com/coreos/etcd/clientv3"
+
 	"golang.org/x/net/context"
 )
 

+ 627 - 0
clientv3/health_balancer.go

@@ -0,0 +1,627 @@
+// Copyright 2017 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 (
+	"errors"
+	"net/url"
+	"strings"
+	"sync"
+	"time"
+
+	"golang.org/x/net/context"
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/codes"
+	healthpb "google.golang.org/grpc/health/grpc_health_v1"
+	"google.golang.org/grpc/status"
+)
+
+const (
+	minHealthRetryDuration = 3 * time.Second
+	unknownService         = "unknown service grpc.health.v1.Health"
+)
+
+// ErrNoAddrAvilable is returned by Get() when the balancer does not have
+// any active connection to endpoints at the time.
+// This error is returned only when opts.BlockingWait is true.
+var ErrNoAddrAvilable = status.Error(codes.Unavailable, "there is no address available")
+
+type healthCheckFunc func(ep string) (bool, error)
+
+type notifyMsg int
+
+const (
+	notifyReset notifyMsg = iota
+	notifyNext
+)
+
+// healthBalancer does the bare minimum to expose multiple eps
+// to the grpc reconnection code path
+type healthBalancer struct {
+	// addrs are the client's endpoint addresses for grpc
+	addrs []grpc.Address
+
+	// eps holds the raw endpoints from the client
+	eps []string
+
+	// notifyCh notifies grpc of the set of addresses for connecting
+	notifyCh chan []grpc.Address
+
+	// readyc closes once the first connection is up
+	readyc    chan struct{}
+	readyOnce sync.Once
+
+	// healthCheck checks an endpoint's health.
+	healthCheck        healthCheckFunc
+	healthCheckTimeout time.Duration
+
+	unhealthyMu        sync.RWMutex
+	unhealthyHostPorts map[string]time.Time
+
+	// mu protects all fields below.
+	mu sync.RWMutex
+
+	// upc closes when pinAddr transitions from empty to non-empty or the balancer closes.
+	upc chan struct{}
+
+	// downc closes when grpc calls down() on pinAddr
+	downc chan struct{}
+
+	// stopc is closed to signal updateNotifyLoop should stop.
+	stopc    chan struct{}
+	stopOnce sync.Once
+	wg       sync.WaitGroup
+
+	// donec closes when all goroutines are exited
+	donec chan struct{}
+
+	// updateAddrsC notifies updateNotifyLoop to update addrs.
+	updateAddrsC chan notifyMsg
+
+	// 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.
+	hostPort2ep map[string]string
+
+	// pinAddr is the currently pinned address; set to the empty string on
+	// initialization and shutdown.
+	pinAddr string
+
+	closed bool
+}
+
+func newHealthBalancer(eps []string, timeout time.Duration, hc healthCheckFunc) *healthBalancer {
+	notifyCh := make(chan []grpc.Address)
+	addrs := eps2addrs(eps)
+	hb := &healthBalancer{
+		addrs:              addrs,
+		eps:                eps,
+		notifyCh:           notifyCh,
+		readyc:             make(chan struct{}),
+		healthCheck:        hc,
+		unhealthyHostPorts: make(map[string]time.Time),
+		upc:                make(chan struct{}),
+		stopc:              make(chan struct{}),
+		downc:              make(chan struct{}),
+		donec:              make(chan struct{}),
+		updateAddrsC:       make(chan notifyMsg),
+		hostPort2ep:        getHostPort2ep(eps),
+	}
+	if timeout < minHealthRetryDuration {
+		timeout = minHealthRetryDuration
+	}
+	hb.healthCheckTimeout = timeout
+
+	close(hb.downc)
+	go hb.updateNotifyLoop()
+	hb.wg.Add(1)
+	go func() {
+		defer hb.wg.Done()
+		hb.updateUnhealthy()
+	}()
+	return hb
+}
+
+func (b *healthBalancer) Start(target string, config grpc.BalancerConfig) error { return nil }
+
+func (b *healthBalancer) ConnectNotify() <-chan struct{} {
+	b.mu.Lock()
+	defer b.mu.Unlock()
+	return b.upc
+}
+
+func (b *healthBalancer) ready() <-chan struct{} { return b.readyc }
+
+func (b *healthBalancer) endpoint(hostPort string) string {
+	b.mu.RLock()
+	defer b.mu.RUnlock()
+	return b.hostPort2ep[hostPort]
+}
+
+func (b *healthBalancer) pinned() string {
+	b.mu.RLock()
+	defer b.mu.RUnlock()
+	return b.pinAddr
+}
+
+func (b *healthBalancer) hostPortError(hostPort string, err error) {
+	if b.endpoint(hostPort) == "" {
+		if logger.V(4) {
+			logger.Infof("clientv3/balancer: %q is stale (skip marking as unhealthy on %q)", hostPort, err.Error())
+		}
+		return
+	}
+
+	b.unhealthyMu.Lock()
+	b.unhealthyHostPorts[hostPort] = time.Now()
+	b.unhealthyMu.Unlock()
+	if logger.V(4) {
+		logger.Infof("clientv3/balancer: %q is marked unhealthy (%q)", hostPort, err.Error())
+	}
+}
+
+func (b *healthBalancer) removeUnhealthy(hostPort, msg string) {
+	if b.endpoint(hostPort) == "" {
+		if logger.V(4) {
+			logger.Infof("clientv3/balancer: %q was not in unhealthy (%q)", hostPort, msg)
+		}
+		return
+	}
+
+	b.unhealthyMu.Lock()
+	delete(b.unhealthyHostPorts, hostPort)
+	b.unhealthyMu.Unlock()
+	if logger.V(4) {
+		logger.Infof("clientv3/balancer: %q is removed from unhealthy (%q)", hostPort, msg)
+	}
+}
+
+func (b *healthBalancer) countUnhealthy() (count int) {
+	b.unhealthyMu.RLock()
+	count = len(b.unhealthyHostPorts)
+	b.unhealthyMu.RUnlock()
+	return count
+}
+
+func (b *healthBalancer) isUnhealthy(hostPort string) (unhealthy bool) {
+	b.unhealthyMu.RLock()
+	_, unhealthy = b.unhealthyHostPorts[hostPort]
+	b.unhealthyMu.RUnlock()
+	return unhealthy
+}
+
+func (b *healthBalancer) cleanupUnhealthy() {
+	b.unhealthyMu.Lock()
+	for k, v := range b.unhealthyHostPorts {
+		if time.Since(v) > b.healthCheckTimeout {
+			delete(b.unhealthyHostPorts, k)
+			if logger.V(4) {
+				logger.Infof("clientv3/balancer: removed %q from unhealthy after %v", k, b.healthCheckTimeout)
+			}
+		}
+	}
+	b.unhealthyMu.Unlock()
+}
+
+func (b *healthBalancer) liveAddrs() ([]grpc.Address, map[string]struct{}) {
+	unhealthyCnt := b.countUnhealthy()
+
+	b.mu.RLock()
+	defer b.mu.RUnlock()
+
+	hbAddrs := b.addrs
+	if len(b.addrs) == 1 || unhealthyCnt == 0 || unhealthyCnt == len(b.addrs) {
+		liveHostPorts := make(map[string]struct{}, len(b.hostPort2ep))
+		for k := range b.hostPort2ep {
+			liveHostPorts[k] = struct{}{}
+		}
+		return hbAddrs, liveHostPorts
+	}
+
+	addrs := make([]grpc.Address, 0, len(b.addrs)-unhealthyCnt)
+	liveHostPorts := make(map[string]struct{}, len(addrs))
+	for _, addr := range b.addrs {
+		if !b.isUnhealthy(addr.Addr) {
+			addrs = append(addrs, addr)
+			liveHostPorts[addr.Addr] = struct{}{}
+		}
+	}
+	return addrs, liveHostPorts
+}
+
+func (b *healthBalancer) updateUnhealthy() {
+	for {
+		select {
+		case <-time.After(b.healthCheckTimeout):
+			b.cleanupUnhealthy()
+			pinned := b.pinned()
+			if pinned == "" || b.isUnhealthy(pinned) {
+				select {
+				case b.updateAddrsC <- notifyNext:
+				case <-b.stopc:
+					return
+				}
+			}
+		case <-b.stopc:
+			return
+		}
+	}
+}
+
+func (b *healthBalancer) updateAddrs(eps ...string) {
+	np := getHostPort2ep(eps)
+
+	b.mu.Lock()
+	defer b.mu.Unlock()
+
+	match := len(np) == len(b.hostPort2ep)
+	if match {
+		for k, v := range np {
+			if b.hostPort2ep[k] != v {
+				match = false
+				break
+			}
+		}
+	}
+	if match {
+		// same endpoints, so no need to update address
+		return
+	}
+
+	b.hostPort2ep = np
+	b.addrs, b.eps = eps2addrs(eps), eps
+
+	b.unhealthyMu.Lock()
+	b.unhealthyHostPorts = make(map[string]time.Time)
+	b.unhealthyMu.Unlock()
+}
+
+func (b *healthBalancer) next() {
+	b.mu.RLock()
+	downc := b.downc
+	b.mu.RUnlock()
+	select {
+	case b.updateAddrsC <- notifyNext:
+	case <-b.stopc:
+	}
+	// wait until disconnect so new RPCs are not issued on old connection
+	select {
+	case <-downc:
+	case <-b.stopc:
+	}
+}
+
+func (b *healthBalancer) updateNotifyLoop() {
+	defer close(b.donec)
+
+	for {
+		b.mu.RLock()
+		upc, downc, addr := b.upc, b.downc, b.pinAddr
+		b.mu.RUnlock()
+		// downc or upc should be closed
+		select {
+		case <-downc:
+			downc = nil
+		default:
+		}
+		select {
+		case <-upc:
+			upc = nil
+		default:
+		}
+		switch {
+		case downc == nil && upc == nil:
+			// stale
+			select {
+			case <-b.stopc:
+				return
+			default:
+			}
+		case downc == nil:
+			b.notifyAddrs(notifyReset)
+			select {
+			case <-upc:
+			case msg := <-b.updateAddrsC:
+				b.notifyAddrs(msg)
+			case <-b.stopc:
+				return
+			}
+		case upc == nil:
+			select {
+			// close connections that are not the pinned address
+			case b.notifyCh <- []grpc.Address{{Addr: addr}}:
+			case <-downc:
+			case <-b.stopc:
+				return
+			}
+			select {
+			case <-downc:
+				b.notifyAddrs(notifyReset)
+			case msg := <-b.updateAddrsC:
+				b.notifyAddrs(msg)
+			case <-b.stopc:
+				return
+			}
+		}
+	}
+}
+
+func (b *healthBalancer) notifyAddrs(msg notifyMsg) {
+	if msg == notifyNext {
+		select {
+		case b.notifyCh <- []grpc.Address{}:
+		case <-b.stopc:
+			return
+		}
+	}
+	b.mu.RLock()
+	pinAddr := b.pinAddr
+	downc := b.downc
+	b.mu.RUnlock()
+	addrs, hostPorts := b.liveAddrs()
+
+	var waitDown bool
+	if pinAddr != "" {
+		_, ok := hostPorts[pinAddr]
+		waitDown = !ok
+	}
+
+	select {
+	case b.notifyCh <- addrs:
+		if waitDown {
+			select {
+			case <-downc:
+			case <-b.stopc:
+			}
+		}
+	case <-b.stopc:
+	}
+}
+
+func (b *healthBalancer) Up(addr grpc.Address) func(error) {
+	if !b.mayPin(addr) {
+		return func(err error) {}
+	}
+
+	b.mu.Lock()
+	defer b.mu.Unlock()
+
+	// gRPC might call Up after it called Close. We add this check
+	// to "fix" it up at application layer. Otherwise, will panic
+	// if b.upc is already closed.
+	if b.closed {
+		return func(err error) {}
+	}
+
+	// gRPC might call Up on a stale address.
+	// Prevent updating pinAddr with a stale address.
+	if !hasAddr(b.addrs, addr.Addr) {
+		return func(err error) {}
+	}
+
+	if b.pinAddr != "" {
+		if logger.V(4) {
+			logger.Infof("clientv3/balancer: %q is up but not pinned (already pinned %q)", addr.Addr, b.pinAddr)
+		}
+		return func(err error) {}
+	}
+
+	// notify waiting Get()s and pin first connected address
+	close(b.upc)
+	b.downc = make(chan struct{})
+	b.pinAddr = addr.Addr
+	if logger.V(4) {
+		logger.Infof("clientv3/balancer: pin %q", addr.Addr)
+	}
+
+	// notify client that a connection is up
+	b.readyOnce.Do(func() { close(b.readyc) })
+
+	return func(err error) {
+		// If connected to a black hole endpoint or a killed server, the gRPC ping
+		// timeout will induce a network I/O error, and retrying until success;
+		// finding healthy endpoint on retry could take several timeouts and redials.
+		// To avoid wasting retries, gray-list unhealthy endpoints.
+		b.hostPortError(addr.Addr, err)
+
+		b.mu.Lock()
+		b.upc = make(chan struct{})
+		close(b.downc)
+		b.pinAddr = ""
+		b.mu.Unlock()
+		if logger.V(4) {
+			logger.Infof("clientv3/balancer: unpin %q (%q)", addr.Addr, err.Error())
+		}
+	}
+}
+
+func (b *healthBalancer) mayPin(addr grpc.Address) bool {
+	if b.endpoint(addr.Addr) == "" { // stale host:port
+		return false
+	}
+
+	b.unhealthyMu.RLock()
+	unhealthyCnt := len(b.unhealthyHostPorts)
+	failedTime, bad := b.unhealthyHostPorts[addr.Addr]
+	b.unhealthyMu.RUnlock()
+
+	b.mu.RLock()
+	skip := len(b.addrs) == 1 || unhealthyCnt == 0 || len(b.addrs) == unhealthyCnt
+	b.mu.RUnlock()
+	if skip || !bad {
+		return true
+	}
+
+	// prevent isolated member's endpoint from being infinitely retried, as follows:
+	//   1. keepalive pings detects GoAway with http2.ErrCodeEnhanceYourCalm
+	//   2. balancer 'Up' unpins with grpc: failed with network I/O error
+	//   3. grpc-healthcheck still SERVING, thus retry to pin
+	// instead, return before grpc-healthcheck if failed within healthcheck timeout
+	if elapsed := time.Since(failedTime); elapsed < b.healthCheckTimeout {
+		if logger.V(4) {
+			logger.Infof("clientv3/balancer: %q is up but not pinned (failed %v ago, require minimum %v after failure)", addr.Addr, elapsed, b.healthCheckTimeout)
+		}
+		return false
+	}
+
+	if ok, _ := b.healthCheck(addr.Addr); ok {
+		b.removeUnhealthy(addr.Addr, "health check success")
+		return true
+	}
+
+	b.hostPortError(addr.Addr, errors.New("health check failed"))
+	return false
+}
+
+func (b *healthBalancer) Get(ctx context.Context, opts grpc.BalancerGetOptions) (grpc.Address, func(), error) {
+	var (
+		addr   string
+		closed bool
+	)
+
+	// If opts.BlockingWait is false (for fail-fast RPCs), it should return
+	// an address it has notified via Notify immediately instead of blocking.
+	if !opts.BlockingWait {
+		b.mu.RLock()
+		closed = b.closed
+		addr = b.pinAddr
+		b.mu.RUnlock()
+		if closed {
+			return grpc.Address{Addr: ""}, nil, grpc.ErrClientConnClosing
+		}
+		if addr == "" {
+			return grpc.Address{Addr: ""}, nil, ErrNoAddrAvilable
+		}
+		return grpc.Address{Addr: addr}, func() {}, nil
+	}
+
+	for {
+		b.mu.RLock()
+		ch := b.upc
+		b.mu.RUnlock()
+		select {
+		case <-ch:
+		case <-b.donec:
+			return grpc.Address{Addr: ""}, nil, grpc.ErrClientConnClosing
+		case <-ctx.Done():
+			return grpc.Address{Addr: ""}, nil, ctx.Err()
+		}
+		b.mu.RLock()
+		closed = b.closed
+		addr = b.pinAddr
+		b.mu.RUnlock()
+		// Close() which sets b.closed = true can be called before Get(), Get() must exit if balancer is closed.
+		if closed {
+			return grpc.Address{Addr: ""}, nil, grpc.ErrClientConnClosing
+		}
+		if addr != "" {
+			break
+		}
+	}
+	return grpc.Address{Addr: addr}, func() {}, nil
+}
+
+func (b *healthBalancer) Notify() <-chan []grpc.Address { return b.notifyCh }
+
+func (b *healthBalancer) Close() error {
+	b.mu.Lock()
+	// In case gRPC calls close twice. TODO: remove the checking
+	// when we are sure that gRPC wont call close twice.
+	if b.closed {
+		b.mu.Unlock()
+		<-b.donec
+		return nil
+	}
+	b.closed = true
+	b.stopOnce.Do(func() { close(b.stopc) })
+	b.pinAddr = ""
+
+	// In the case of following scenario:
+	//	1. upc is not closed; no pinned address
+	// 	2. client issues an RPC, calling invoke(), which calls Get(), enters for loop, blocks
+	// 	3. client.conn.Close() calls balancer.Close(); closed = true
+	// 	4. for loop in Get() never exits since ctx is the context passed in by the client and may not be canceled
+	// we must close upc so Get() exits from blocking on upc
+	select {
+	case <-b.upc:
+	default:
+		// terminate all waiting Get()s
+		close(b.upc)
+	}
+
+	b.mu.Unlock()
+	b.wg.Wait()
+
+	// wait for updateNotifyLoop to finish
+	<-b.donec
+	close(b.notifyCh)
+
+	return nil
+}
+
+func grpcHealthCheck(client *Client, ep string) (bool, error) {
+	conn, err := client.dial(ep)
+	if err != nil {
+		return false, err
+	}
+	defer conn.Close()
+	cli := healthpb.NewHealthClient(conn)
+	ctx, cancel := context.WithTimeout(context.Background(), time.Second)
+	resp, err := cli.Check(ctx, &healthpb.HealthCheckRequest{})
+	cancel()
+	if err != nil {
+		if s, ok := status.FromError(err); ok && s.Code() == codes.Unavailable {
+			if s.Message() == unknownService { // etcd < v3.3.0
+				return true, nil
+			}
+		}
+		return false, err
+	}
+	return resp.Status == healthpb.HealthCheckResponse_SERVING, nil
+}
+
+func hasAddr(addrs []grpc.Address, targetAddr string) bool {
+	for _, addr := range addrs {
+		if targetAddr == addr.Addr {
+			return true
+		}
+	}
+	return false
+}
+
+func getHost(ep string) string {
+	url, uerr := url.Parse(ep)
+	if uerr != nil || !strings.Contains(ep, "://") {
+		return ep
+	}
+	return url.Host
+}
+
+func eps2addrs(eps []string) []grpc.Address {
+	addrs := make([]grpc.Address, len(eps))
+	for i := range eps {
+		addrs[i].Addr = getHost(eps[i])
+	}
+	return addrs
+}
+
+func getHostPort2ep(eps []string) map[string]string {
+	hm := make(map[string]string, len(eps))
+	for i := range eps {
+		_, host, _ := parseEndpoint(eps[i])
+		hm[host] = eps[i]
+	}
+	return hm
+}

+ 211 - 0
clientv3/integration/black_hole_test.go

@@ -0,0 +1,211 @@
+// Copyright 2017 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.
+
+// +build !cluster_proxy
+
+package integration
+
+import (
+	"testing"
+	"time"
+
+	"github.com/coreos/etcd/clientv3"
+	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
+	"github.com/coreos/etcd/integration"
+	"github.com/coreos/etcd/pkg/testutil"
+
+	"golang.org/x/net/context"
+)
+
+// TestBalancerUnderBlackholeKeepAliveWatch tests when watch discovers it cannot talk to
+// blackholed endpoint, client balancer switches to healthy one.
+// TODO: test server-to-client keepalive ping
+func TestBalancerUnderBlackholeKeepAliveWatch(t *testing.T) {
+	defer testutil.AfterTest(t)
+
+	clus := integration.NewClusterV3(t, &integration.ClusterConfig{
+		Size:                 2,
+		GRPCKeepAliveMinTime: 1 * time.Millisecond, // avoid too_many_pings
+	})
+	defer clus.Terminate(t)
+
+	eps := []string{clus.Members[0].GRPCAddr(), clus.Members[1].GRPCAddr()}
+
+	ccfg := clientv3.Config{
+		Endpoints:            []string{eps[0]},
+		DialTimeout:          1 * time.Second,
+		DialKeepAliveTime:    1 * time.Second,
+		DialKeepAliveTimeout: 500 * time.Millisecond,
+	}
+
+	// gRPC internal implementation related.
+	pingInterval := ccfg.DialKeepAliveTime + ccfg.DialKeepAliveTimeout
+	// 3s for slow machine to process watch and reset connections
+	// TODO: only send healthy endpoint to gRPC so gRPC wont waste time to
+	// dial for unhealthy endpoint.
+	// then we can reduce 3s to 1s.
+	timeout := pingInterval + 3*time.Second
+
+	cli, err := clientv3.New(ccfg)
+	if err != nil {
+		t.Fatal(err)
+	}
+	defer cli.Close()
+
+	wch := cli.Watch(context.Background(), "foo", clientv3.WithCreatedNotify())
+	if _, ok := <-wch; !ok {
+		t.Fatalf("watch failed on creation")
+	}
+
+	// endpoint can switch to eps[1] when it detects the failure of eps[0]
+	cli.SetEndpoints(eps...)
+
+	clus.Members[0].Blackhole()
+
+	if _, err = clus.Client(1).Put(context.TODO(), "foo", "bar"); err != nil {
+		t.Fatal(err)
+	}
+	select {
+	case <-wch:
+	case <-time.After(timeout):
+		t.Error("took too long to receive watch events")
+	}
+
+	clus.Members[0].Unblackhole()
+
+	// waiting for moving eps[0] out of unhealthy, so that it can be re-pined.
+	time.Sleep(ccfg.DialTimeout)
+
+	clus.Members[1].Blackhole()
+
+	// make sure client[0] can connect to eps[0] after remove the blackhole.
+	if _, err = clus.Client(0).Get(context.TODO(), "foo"); err != nil {
+		t.Fatal(err)
+	}
+	if _, err = clus.Client(0).Put(context.TODO(), "foo", "bar1"); err != nil {
+		t.Fatal(err)
+	}
+
+	select {
+	case <-wch:
+	case <-time.After(timeout):
+		t.Error("took too long to receive watch events")
+	}
+}
+
+func TestBalancerUnderBlackholeNoKeepAlivePut(t *testing.T) {
+	testBalancerUnderBlackholeNoKeepAlive(t, func(cli *clientv3.Client, ctx context.Context) error {
+		_, err := cli.Put(ctx, "foo", "bar")
+		if err == context.DeadlineExceeded || err == rpctypes.ErrTimeout {
+			return errExpected
+		}
+		return err
+	})
+}
+
+func TestBalancerUnderBlackholeNoKeepAliveDelete(t *testing.T) {
+	testBalancerUnderBlackholeNoKeepAlive(t, func(cli *clientv3.Client, ctx context.Context) error {
+		_, err := cli.Delete(ctx, "foo")
+		if err == context.DeadlineExceeded || err == rpctypes.ErrTimeout {
+			return errExpected
+		}
+		return err
+	})
+}
+
+func TestBalancerUnderBlackholeNoKeepAliveTxn(t *testing.T) {
+	testBalancerUnderBlackholeNoKeepAlive(t, func(cli *clientv3.Client, ctx context.Context) error {
+		_, err := cli.Txn(ctx).
+			If(clientv3.Compare(clientv3.Version("foo"), "=", 0)).
+			Then(clientv3.OpPut("foo", "bar")).
+			Else(clientv3.OpPut("foo", "baz")).Commit()
+		if err == context.DeadlineExceeded || err == rpctypes.ErrTimeout {
+			return errExpected
+		}
+		return err
+	})
+}
+
+func TestBalancerUnderBlackholeNoKeepAliveLinearizableGet(t *testing.T) {
+	testBalancerUnderBlackholeNoKeepAlive(t, func(cli *clientv3.Client, ctx context.Context) error {
+		_, err := cli.Get(ctx, "a")
+		if err == context.DeadlineExceeded || err == rpctypes.ErrTimeout {
+			return errExpected
+		}
+		return err
+	})
+}
+
+func TestBalancerUnderBlackholeNoKeepAliveSerializableGet(t *testing.T) {
+	testBalancerUnderBlackholeNoKeepAlive(t, func(cli *clientv3.Client, ctx context.Context) error {
+		_, err := cli.Get(ctx, "a", clientv3.WithSerializable())
+		if err == context.DeadlineExceeded {
+			return errExpected
+		}
+		return err
+	})
+}
+
+// testBalancerUnderBlackholeNoKeepAlive ensures that first request to blackholed endpoint
+// fails due to context timeout, but succeeds on next try, with endpoint switch.
+func testBalancerUnderBlackholeNoKeepAlive(t *testing.T, op func(*clientv3.Client, context.Context) error) {
+	defer testutil.AfterTest(t)
+
+	clus := integration.NewClusterV3(t, &integration.ClusterConfig{
+		Size:               2,
+		SkipCreatingClient: true,
+	})
+	defer clus.Terminate(t)
+
+	eps := []string{clus.Members[0].GRPCAddr(), clus.Members[1].GRPCAddr()}
+
+	ccfg := clientv3.Config{
+		Endpoints:   []string{eps[0]},
+		DialTimeout: 1 * time.Second,
+	}
+	cli, err := clientv3.New(ccfg)
+	if err != nil {
+		t.Fatal(err)
+	}
+	defer cli.Close()
+
+	// wait for eps[0] to be pinned
+	mustWaitPinReady(t, cli)
+
+	// add all eps to list, so that when the original pined one fails
+	// the client can switch to other available eps
+	cli.SetEndpoints(eps...)
+
+	// blackhole eps[0]
+	clus.Members[0].Blackhole()
+
+	// fail first due to blackhole, retry should succeed
+	// TODO: first operation can succeed
+	// when gRPC supports better retry on non-delivered request
+	for i := 0; i < 2; i++ {
+		ctx, cancel := context.WithTimeout(context.Background(), time.Second)
+		err = op(cli, ctx)
+		cancel()
+		if err == nil {
+			break
+		}
+		if i == 0 {
+			if err != errExpected {
+				t.Errorf("#%d: expected %v, got %v", i, errExpected, err)
+			}
+		} else if err != nil {
+			t.Errorf("#%d: failed with error %v", i, err)
+		}
+	}
+}

+ 1 - 0
clientv3/integration/cluster_test.go

@@ -21,6 +21,7 @@ import (
 	"github.com/coreos/etcd/integration"
 	"github.com/coreos/etcd/pkg/testutil"
 	"github.com/coreos/etcd/pkg/types"
+
 	"golang.org/x/net/context"
 )
 

+ 28 - 13
clientv3/integration/dial_test.go

@@ -16,6 +16,7 @@ package integration
 
 import (
 	"math/rand"
+	"strings"
 	"testing"
 	"time"
 
@@ -26,7 +27,6 @@ import (
 	"github.com/coreos/etcd/pkg/transport"
 
 	"golang.org/x/net/context"
-	"google.golang.org/grpc"
 )
 
 var (
@@ -48,21 +48,21 @@ var (
 // TestDialTLSExpired tests client with expired certs fails to dial.
 func TestDialTLSExpired(t *testing.T) {
 	defer testutil.AfterTest(t)
-	clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, PeerTLS: &testTLSInfo, ClientTLS: &testTLSInfo})
+	clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, PeerTLS: &testTLSInfo, ClientTLS: &testTLSInfo, SkipCreatingClient: true})
 	defer clus.Terminate(t)
 
 	tls, err := testTLSInfoExpired.ClientConfig()
 	if err != nil {
 		t.Fatal(err)
 	}
-	// expect remote errors 'tls: bad certificate'
+	// expect remote errors "tls: bad certificate"
 	_, err = clientv3.New(clientv3.Config{
 		Endpoints:   []string{clus.Members[0].GRPCAddr()},
 		DialTimeout: 3 * time.Second,
 		TLS:         tls,
 	})
-	if err != grpc.ErrClientConnTimeout {
-		t.Fatalf("expected %v, got %v", grpc.ErrClientConnTimeout, err)
+	if err != context.DeadlineExceeded {
+		t.Fatalf("expected %v, got %v", context.DeadlineExceeded, err)
 	}
 }
 
@@ -70,19 +70,20 @@ func TestDialTLSExpired(t *testing.T) {
 // when TLS endpoints (https, unixs) are given but no tls config.
 func TestDialTLSNoConfig(t *testing.T) {
 	defer testutil.AfterTest(t)
-	clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, ClientTLS: &testTLSInfo})
+	clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, ClientTLS: &testTLSInfo, SkipCreatingClient: true})
 	defer clus.Terminate(t)
-	// expect 'signed by unknown authority'
+	// expect "signed by unknown authority"
 	_, err := clientv3.New(clientv3.Config{
 		Endpoints:   []string{clus.Members[0].GRPCAddr()},
 		DialTimeout: time.Second,
 	})
-	if err != grpc.ErrClientConnTimeout {
-		t.Fatalf("expected %v, got %v", grpc.ErrClientConnTimeout, err)
+	if err != context.DeadlineExceeded {
+		t.Fatalf("expected %v, got %v", context.DeadlineExceeded, err)
 	}
 }
 
-// TestDialSetEndpoints ensures SetEndpoints can replace unavailable endpoints with available ones.
+// TestDialSetEndpointsBeforeFail ensures SetEndpoints can replace unavailable
+// endpoints with available ones.
 func TestDialSetEndpointsBeforeFail(t *testing.T) {
 	testDialSetEndpoints(t, true)
 }
@@ -94,7 +95,7 @@ func TestDialSetEndpointsAfterFail(t *testing.T) {
 // testDialSetEndpoints ensures SetEndpoints can replace unavailable endpoints with available ones.
 func testDialSetEndpoints(t *testing.T, setBefore bool) {
 	defer testutil.AfterTest(t)
-	clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
+	clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3, SkipCreatingClient: true})
 	defer clus.Terminate(t)
 
 	// get endpoint list
@@ -139,7 +140,7 @@ func TestSwitchSetEndpoints(t *testing.T) {
 	eps := []string{clus.Members[1].GRPCAddr(), clus.Members[2].GRPCAddr()}
 
 	cli := clus.Client(0)
-	clus.Members[0].InjectPartition(t, clus.Members[1:])
+	clus.Members[0].InjectPartition(t, clus.Members[1:]...)
 
 	cli.SetEndpoints(eps...)
 	ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
@@ -152,7 +153,7 @@ func TestSwitchSetEndpoints(t *testing.T) {
 func TestRejectOldCluster(t *testing.T) {
 	defer testutil.AfterTest(t)
 	// 2 endpoints to test multi-endpoint Status
-	clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 2})
+	clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 2, SkipCreatingClient: true})
 	defer clus.Terminate(t)
 
 	cfg := clientv3.Config{
@@ -189,3 +190,17 @@ func TestDialForeignEndpoint(t *testing.T) {
 		t.Fatal(err)
 	}
 }
+
+// TestSetEndpointAndPut checks that a Put following a SetEndpoints
+// to a working endpoint will always succeed.
+func TestSetEndpointAndPut(t *testing.T) {
+	defer testutil.AfterTest(t)
+	clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 2})
+	defer clus.Terminate(t)
+
+	clus.Client(1).SetEndpoints(clus.Members[0].GRPCAddr())
+	_, err := clus.Client(1).Put(context.TODO(), "foo", "bar")
+	if err != nil && !strings.Contains(err.Error(), "closing") {
+		t.Fatal(err)
+	}
+}

+ 32 - 66
clientv3/integration/kv_test.go

@@ -16,7 +16,6 @@ package integration
 
 import (
 	"bytes"
-	"math/rand"
 	"os"
 	"reflect"
 	"strings"
@@ -28,6 +27,7 @@ import (
 	"github.com/coreos/etcd/integration"
 	"github.com/coreos/etcd/mvcc/mvccpb"
 	"github.com/coreos/etcd/pkg/testutil"
+
 	"golang.org/x/net/context"
 	"google.golang.org/grpc"
 )
@@ -441,8 +441,8 @@ func TestKVGetErrConnClosed(t *testing.T) {
 	go func() {
 		defer close(donec)
 		_, err := cli.Get(context.TODO(), "foo")
-		if err != nil && err != grpc.ErrClientConnClosing {
-			t.Fatalf("expected %v, got %v", grpc.ErrClientConnClosing, err)
+		if err != nil && err != context.Canceled && err != grpc.ErrClientConnClosing {
+			t.Fatalf("expected %v or %v, got %v", context.Canceled, grpc.ErrClientConnClosing, err)
 		}
 	}()
 
@@ -472,8 +472,9 @@ func TestKVNewAfterClose(t *testing.T) {
 
 	donec := make(chan struct{})
 	go func() {
-		if _, err := cli.Get(context.TODO(), "foo"); err != grpc.ErrClientConnClosing {
-			t.Fatalf("expected %v, got %v", grpc.ErrClientConnClosing, err)
+		_, err := cli.Get(context.TODO(), "foo")
+		if err != context.Canceled && err != grpc.ErrClientConnClosing {
+			t.Fatalf("expected %v or %v, got %v", context.Canceled, grpc.ErrClientConnClosing, err)
 		}
 		close(donec)
 	}()
@@ -790,7 +791,7 @@ func TestKVGetStoppedServerAndClose(t *testing.T) {
 	// this Get fails and triggers an asynchronous connection retry
 	_, err := cli.Get(ctx, "abc")
 	cancel()
-	if !strings.Contains(err.Error(), "context deadline") {
+	if err != nil && err != context.DeadlineExceeded {
 		t.Fatal(err)
 	}
 }
@@ -812,86 +813,51 @@ func TestKVPutStoppedServerAndClose(t *testing.T) {
 	// grpc finds out the original connection is down due to the member shutdown.
 	_, err := cli.Get(ctx, "abc")
 	cancel()
-	if !strings.Contains(err.Error(), "context deadline") {
+	if err != nil && err != context.DeadlineExceeded {
 		t.Fatal(err)
 	}
 
 	// this Put fails and triggers an asynchronous connection retry
 	_, err = cli.Put(ctx, "abc", "123")
 	cancel()
-	if !strings.Contains(err.Error(), "context deadline") {
+	if err != nil && err != context.DeadlineExceeded {
 		t.Fatal(err)
 	}
 }
 
-// TestKVGetOneEndpointDown ensures a client can connect and get if one endpoint is down
-func TestKVPutOneEndpointDown(t *testing.T) {
+// TestKVPutAtMostOnce ensures that a Put will only occur at most once
+// in the presence of network errors.
+func TestKVPutAtMostOnce(t *testing.T) {
 	defer testutil.AfterTest(t)
-	clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
+	clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
 	defer clus.Terminate(t)
 
-	// get endpoint list
-	eps := make([]string, 3)
-	for i := range eps {
-		eps[i] = clus.Members[i].GRPCAddr()
-	}
-
-	// make a dead node
-	clus.Members[rand.Intn(len(eps))].Stop(t)
-
-	// try to connect with dead node in the endpoint list
-	cfg := clientv3.Config{Endpoints: eps, DialTimeout: 1 * time.Second}
-	cli, err := clientv3.New(cfg)
-	if err != nil {
+	if _, err := clus.Client(0).Put(context.TODO(), "k", "1"); err != nil {
 		t.Fatal(err)
 	}
-	defer cli.Close()
-	ctx, cancel := context.WithTimeout(context.TODO(), 3*time.Second)
-	if _, err := cli.Get(ctx, "abc", clientv3.WithSerializable()); err != nil {
-		t.Fatal(err)
-	}
-	cancel()
-}
 
-// TestKVGetResetLoneEndpoint ensures that if an endpoint resets and all other
-// endpoints are down, then it will reconnect.
-func TestKVGetResetLoneEndpoint(t *testing.T) {
-	defer testutil.AfterTest(t)
-	clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 2})
-	defer clus.Terminate(t)
-
-	// get endpoint list
-	eps := make([]string, 2)
-	for i := range eps {
-		eps[i] = clus.Members[i].GRPCAddr()
+	for i := 0; i < 10; i++ {
+		clus.Members[0].DropConnections()
+		donec := make(chan struct{})
+		go func() {
+			defer close(donec)
+			for i := 0; i < 10; i++ {
+				clus.Members[0].DropConnections()
+				time.Sleep(5 * time.Millisecond)
+			}
+		}()
+		_, err := clus.Client(0).Put(context.TODO(), "k", "v")
+		<-donec
+		if err != nil {
+			break
+		}
 	}
 
-	cfg := clientv3.Config{Endpoints: eps, DialTimeout: 500 * time.Millisecond}
-	cli, err := clientv3.New(cfg)
+	resp, err := clus.Client(0).Get(context.TODO(), "k")
 	if err != nil {
 		t.Fatal(err)
 	}
-	defer cli.Close()
-
-	// disconnect everything
-	clus.Members[0].Stop(t)
-	clus.Members[1].Stop(t)
-
-	// have Get try to reconnect
-	donec := make(chan struct{})
-	go func() {
-		ctx, cancel := context.WithTimeout(context.TODO(), 5*time.Second)
-		if _, err := cli.Get(ctx, "abc", clientv3.WithSerializable()); err != nil {
-			t.Fatal(err)
-		}
-		cancel()
-		close(donec)
-	}()
-	time.Sleep(500 * time.Millisecond)
-	clus.Members[0].Restart(t)
-	select {
-	case <-time.After(10 * time.Second):
-		t.Fatalf("timed out waiting for Get")
-	case <-donec:
+	if resp.Kvs[0].Version > 11 {
+		t.Fatalf("expected version <= 10, got %+v", resp.Kvs[0])
 	}
 }

+ 22 - 14
clientv3/integration/lease_test.go

@@ -26,6 +26,7 @@ import (
 	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
 	"github.com/coreos/etcd/integration"
 	"github.com/coreos/etcd/pkg/testutil"
+
 	"golang.org/x/net/context"
 	"google.golang.org/grpc"
 )
@@ -233,7 +234,7 @@ type leaseCh struct {
 	ch  <-chan *clientv3.LeaseKeepAliveResponse
 }
 
-// TestLeaseKeepAliveNotFound ensures a revoked lease won't stop other keep alives
+// TestLeaseKeepAliveNotFound ensures a revoked lease won't halt other leases.
 func TestLeaseKeepAliveNotFound(t *testing.T) {
 	defer testutil.AfterTest(t)
 
@@ -286,8 +287,10 @@ func TestLeaseGrantErrConnClosed(t *testing.T) {
 	go func() {
 		defer close(donec)
 		_, err := cli.Grant(context.TODO(), 5)
-		if err != nil && err != grpc.ErrClientConnClosing {
-			t.Fatalf("expected %v, got %v", grpc.ErrClientConnClosing, err)
+		if err != nil && err != grpc.ErrClientConnClosing && err != context.Canceled {
+			// grpc.ErrClientConnClosing if grpc-go balancer calls 'Get' after client.Close.
+			// context.Canceled if grpc-go balancer calls 'Get' with an inflight client.Close.
+			t.Fatalf("expected %v or %v, got %v", grpc.ErrClientConnClosing, context.Canceled, err)
 		}
 	}()
 
@@ -316,8 +319,8 @@ func TestLeaseGrantNewAfterClose(t *testing.T) {
 
 	donec := make(chan struct{})
 	go func() {
-		if _, err := cli.Grant(context.TODO(), 5); err != grpc.ErrClientConnClosing {
-			t.Fatalf("expected %v, got %v", grpc.ErrClientConnClosing, err)
+		if _, err := cli.Grant(context.TODO(), 5); err != context.Canceled && err != grpc.ErrClientConnClosing {
+			t.Fatalf("expected %v or %v, got %v", err != context.Canceled, grpc.ErrClientConnClosing, err)
 		}
 		close(donec)
 	}()
@@ -348,8 +351,8 @@ func TestLeaseRevokeNewAfterClose(t *testing.T) {
 
 	donec := make(chan struct{})
 	go func() {
-		if _, err := cli.Revoke(context.TODO(), leaseID); err != grpc.ErrClientConnClosing {
-			t.Fatalf("expected %v, got %v", grpc.ErrClientConnClosing, err)
+		if _, err := cli.Revoke(context.TODO(), leaseID); err != context.Canceled && err != grpc.ErrClientConnClosing {
+			t.Fatalf("expected %v or %v, got %v", err != context.Canceled, grpc.ErrClientConnClosing, err)
 		}
 		close(donec)
 	}()
@@ -360,7 +363,7 @@ func TestLeaseRevokeNewAfterClose(t *testing.T) {
 	}
 }
 
-// TestLeaseKeepAliveCloseAfterDisconnectExpire ensures the keep alive channel is closed
+// TestLeaseKeepAliveCloseAfterDisconnectRevoke ensures the keep alive channel is closed
 // following a disconnection, lease revoke, then reconnect.
 func TestLeaseKeepAliveCloseAfterDisconnectRevoke(t *testing.T) {
 	defer testutil.AfterTest(t)
@@ -395,7 +398,7 @@ func TestLeaseKeepAliveCloseAfterDisconnectRevoke(t *testing.T) {
 
 	clus.Members[0].Restart(t)
 
-	// some keep-alives may still be buffered; drain until close
+	// some responses may still be buffered; drain until close
 	timer := time.After(time.Duration(kresp.TTL) * time.Second)
 	for kresp != nil {
 		select {
@@ -482,7 +485,8 @@ func TestLeaseTimeToLive(t *testing.T) {
 	clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
 	defer clus.Terminate(t)
 
-	lapi := clus.RandClient()
+	c := clus.RandClient()
+	lapi := c
 
 	resp, err := lapi.Grant(context.Background(), 10)
 	if err != nil {
@@ -497,6 +501,11 @@ func TestLeaseTimeToLive(t *testing.T) {
 		}
 	}
 
+	// linearized read to ensure Puts propagated to server backing lapi
+	if _, err := c.Get(context.TODO(), "abc"); err != nil {
+		t.Fatal(err)
+	}
+
 	lresp, lerr := lapi.TimeToLive(context.Background(), resp.ID, clientv3.WithAttachedKeys())
 	if lerr != nil {
 		t.Fatal(lerr)
@@ -545,8 +554,7 @@ func TestLeaseTimeToLiveLeaseNotFound(t *testing.T) {
 	}
 
 	lresp, err := cli.TimeToLive(context.Background(), resp.ID)
-	// TimeToLive() doesn't return LeaseNotFound error
-	// but return a response with TTL to be -1
+	// TimeToLive() should return a response with TTL=-1.
 	if err != nil {
 		t.Fatalf("expected err to be nil")
 	}
@@ -636,8 +644,8 @@ func TestLeaseKeepAliveLoopExit(t *testing.T) {
 	}
 }
 
-// TestV3LeaseFailureOverlap issues Grant and Keepalive requests to a cluster
-// before, during, and after quorum loss to confirm Grant/Keepalive tolerates
+// TestV3LeaseFailureOverlap issues Grant and KeepAlive requests to a cluster
+// before, during, and after quorum loss to confirm Grant/KeepAlive tolerates
 // transient cluster failure.
 func TestV3LeaseFailureOverlap(t *testing.T) {
 	clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 2})

+ 10 - 2
clientv3/integration/logger_test.go

@@ -14,8 +14,16 @@
 
 package integration
 
-import "github.com/coreos/pkg/capnslog"
+import (
+	"io/ioutil"
+
+	"github.com/coreos/etcd/clientv3"
+
+	"github.com/coreos/pkg/capnslog"
+	"google.golang.org/grpc/grpclog"
+)
 
 func init() {
-	capnslog.SetGlobalLogLevel(capnslog.INFO)
+	capnslog.SetGlobalLogLevel(capnslog.CRITICAL)
+	clientv3.SetLogger(grpclog.NewLoggerV2(ioutil.Discard, ioutil.Discard, ioutil.Discard))
 }

+ 2 - 2
clientv3/integration/metrics_test.go

@@ -44,7 +44,7 @@ func TestV3ClientMetrics(t *testing.T) {
 		err  error
 	)
 
-	// listen for all prometheus metrics
+	// listen for all Prometheus metrics
 	donec := make(chan struct{})
 	go func() {
 		defer close(donec)
@@ -65,7 +65,7 @@ func TestV3ClientMetrics(t *testing.T) {
 
 	url := "unix://" + addr + "/metrics"
 
-	clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
+	clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, SkipCreatingClient: true})
 	defer clus.Terminate(t)
 
 	cfg := clientv3.Config{

+ 1 - 0
clientv3/integration/mirror_test.go

@@ -25,6 +25,7 @@ import (
 	"github.com/coreos/etcd/integration"
 	"github.com/coreos/etcd/mvcc/mvccpb"
 	"github.com/coreos/etcd/pkg/testutil"
+
 	"golang.org/x/net/context"
 )
 

+ 2 - 1
clientv3/integration/namespace_test.go

@@ -15,7 +15,6 @@
 package integration
 
 import (
-	"context"
 	"reflect"
 	"testing"
 
@@ -24,6 +23,8 @@ import (
 	"github.com/coreos/etcd/integration"
 	"github.com/coreos/etcd/mvcc/mvccpb"
 	"github.com/coreos/etcd/pkg/testutil"
+
+	"golang.org/x/net/context"
 )
 
 func TestNamespacePutGet(t *testing.T) {

+ 260 - 0
clientv3/integration/network_partition_test.go

@@ -0,0 +1,260 @@
+// Copyright 2017 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.
+
+// +build !cluster_proxy
+
+package integration
+
+import (
+	"errors"
+	"testing"
+	"time"
+
+	"github.com/coreos/etcd/clientv3"
+	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
+	"github.com/coreos/etcd/integration"
+	"github.com/coreos/etcd/pkg/testutil"
+
+	"golang.org/x/net/context"
+)
+
+var errExpected = errors.New("expected error")
+
+// TestBalancerUnderNetworkPartitionPut tests when one member becomes isolated,
+// first Put request fails, and following retry succeeds with client balancer
+// switching to others.
+func TestBalancerUnderNetworkPartitionPut(t *testing.T) {
+	testBalancerUnderNetworkPartition(t, func(cli *clientv3.Client, ctx context.Context) error {
+		_, err := cli.Put(ctx, "a", "b")
+		if err == context.DeadlineExceeded || err == rpctypes.ErrTimeout {
+			return errExpected
+		}
+		return err
+	}, time.Second)
+}
+
+func TestBalancerUnderNetworkPartitionDelete(t *testing.T) {
+	testBalancerUnderNetworkPartition(t, func(cli *clientv3.Client, ctx context.Context) error {
+		_, err := cli.Delete(ctx, "a")
+		if err == context.DeadlineExceeded || err == rpctypes.ErrTimeout {
+			return errExpected
+		}
+		return err
+	}, time.Second)
+}
+
+func TestBalancerUnderNetworkPartitionTxn(t *testing.T) {
+	testBalancerUnderNetworkPartition(t, func(cli *clientv3.Client, ctx context.Context) error {
+		_, err := cli.Txn(ctx).
+			If(clientv3.Compare(clientv3.Version("foo"), "=", 0)).
+			Then(clientv3.OpPut("foo", "bar")).
+			Else(clientv3.OpPut("foo", "baz")).Commit()
+		if err == context.DeadlineExceeded || err == rpctypes.ErrTimeout {
+			return errExpected
+		}
+		return err
+	}, time.Second)
+}
+
+// TestBalancerUnderNetworkPartitionLinearizableGetWithLongTimeout tests
+// when one member becomes isolated, first quorum Get request succeeds
+// by switching endpoints within the timeout (long enough to cover endpoint switch).
+func TestBalancerUnderNetworkPartitionLinearizableGetWithLongTimeout(t *testing.T) {
+	testBalancerUnderNetworkPartition(t, func(cli *clientv3.Client, ctx context.Context) error {
+		_, err := cli.Get(ctx, "a")
+		return err
+	}, 7*time.Second)
+}
+
+// TestBalancerUnderNetworkPartitionLinearizableGetWithShortTimeout tests
+// when one member becomes isolated, first quorum Get request fails,
+// and following retry succeeds with client balancer switching to others.
+func TestBalancerUnderNetworkPartitionLinearizableGetWithShortTimeout(t *testing.T) {
+	testBalancerUnderNetworkPartition(t, func(cli *clientv3.Client, ctx context.Context) error {
+		_, err := cli.Get(ctx, "a")
+		if err == context.DeadlineExceeded {
+			return errExpected
+		}
+		return err
+	}, time.Second)
+}
+
+func TestBalancerUnderNetworkPartitionSerializableGet(t *testing.T) {
+	testBalancerUnderNetworkPartition(t, func(cli *clientv3.Client, ctx context.Context) error {
+		_, err := cli.Get(ctx, "a", clientv3.WithSerializable())
+		return err
+	}, time.Second)
+}
+
+func testBalancerUnderNetworkPartition(t *testing.T, op func(*clientv3.Client, context.Context) error, timeout time.Duration) {
+	defer testutil.AfterTest(t)
+
+	clus := integration.NewClusterV3(t, &integration.ClusterConfig{
+		Size:                 3,
+		GRPCKeepAliveMinTime: time.Millisecond, // avoid too_many_pings
+		SkipCreatingClient:   true,
+	})
+	defer clus.Terminate(t)
+
+	eps := []string{clus.Members[0].GRPCAddr(), clus.Members[1].GRPCAddr(), clus.Members[2].GRPCAddr()}
+
+	// expect pin eps[0]
+	ccfg := clientv3.Config{
+		Endpoints:   []string{eps[0]},
+		DialTimeout: 3 * time.Second,
+	}
+	cli, err := clientv3.New(ccfg)
+	if err != nil {
+		t.Fatal(err)
+	}
+	defer cli.Close()
+
+	// wait for eps[0] to be pinned
+	mustWaitPinReady(t, cli)
+
+	// add other endpoints for later endpoint switch
+	cli.SetEndpoints(eps...)
+	clus.Members[0].InjectPartition(t, clus.Members[1:]...)
+
+	for i := 0; i < 2; i++ {
+		ctx, cancel := context.WithTimeout(context.Background(), timeout)
+		err = op(cli, ctx)
+		cancel()
+		if err == nil {
+			break
+		}
+		if err != errExpected {
+			t.Errorf("#%d: expected %v, got %v", i, errExpected, err)
+		}
+		// give enough time for endpoint switch
+		// TODO: remove random sleep by syncing directly with balancer
+		if i == 0 {
+			time.Sleep(5 * time.Second)
+		}
+	}
+	if err != nil {
+		t.Errorf("balancer did not switch in time (%v)", err)
+	}
+}
+
+// TestBalancerUnderNetworkPartitionLinearizableGetLeaderElection ensures balancer
+// switches endpoint when leader fails and linearizable get requests returns
+// "etcdserver: request timed out".
+func TestBalancerUnderNetworkPartitionLinearizableGetLeaderElection(t *testing.T) {
+	defer testutil.AfterTest(t)
+
+	clus := integration.NewClusterV3(t, &integration.ClusterConfig{
+		Size:               3,
+		SkipCreatingClient: true,
+	})
+	defer clus.Terminate(t)
+	eps := []string{clus.Members[0].GRPCAddr(), clus.Members[1].GRPCAddr(), clus.Members[2].GRPCAddr()}
+
+	lead := clus.WaitLeader(t)
+
+	timeout := 3 * clus.Members[(lead+1)%2].ServerConfig.ReqTimeout()
+
+	cli, err := clientv3.New(clientv3.Config{
+		Endpoints:   []string{eps[(lead+1)%2]},
+		DialTimeout: 1 * time.Second,
+	})
+	if err != nil {
+		t.Fatal(err)
+	}
+	defer cli.Close()
+
+	// wait for non-leader to be pinned
+	mustWaitPinReady(t, cli)
+
+	// add all eps to list, so that when the original pined one fails
+	// the client can switch to other available eps
+	cli.SetEndpoints(eps[lead], eps[(lead+1)%2])
+
+	// isolate leader
+	clus.Members[lead].InjectPartition(t, clus.Members[(lead+1)%3], clus.Members[(lead+2)%3])
+
+	// expects balancer endpoint switch while ongoing leader election
+	ctx, cancel := context.WithTimeout(context.TODO(), timeout)
+	_, err = cli.Get(ctx, "a")
+	cancel()
+	if err != nil {
+		t.Fatal(err)
+	}
+}
+
+func TestBalancerUnderNetworkPartitionWatchLeader(t *testing.T) {
+	testBalancerUnderNetworkPartitionWatch(t, true)
+}
+
+func TestBalancerUnderNetworkPartitionWatchFollower(t *testing.T) {
+	testBalancerUnderNetworkPartitionWatch(t, false)
+}
+
+// testBalancerUnderNetworkPartitionWatch ensures watch stream
+// to a partitioned node be closed when context requires leader.
+func testBalancerUnderNetworkPartitionWatch(t *testing.T, isolateLeader bool) {
+	defer testutil.AfterTest(t)
+
+	clus := integration.NewClusterV3(t, &integration.ClusterConfig{
+		Size:               3,
+		SkipCreatingClient: true,
+	})
+	defer clus.Terminate(t)
+
+	eps := []string{clus.Members[0].GRPCAddr(), clus.Members[1].GRPCAddr(), clus.Members[2].GRPCAddr()}
+
+	target := clus.WaitLeader(t)
+	if !isolateLeader {
+		target = (target + 1) % 3
+	}
+
+	// pin eps[target]
+	watchCli, err := clientv3.New(clientv3.Config{Endpoints: []string{eps[target]}})
+	if err != nil {
+		t.Fatal(err)
+	}
+	defer watchCli.Close()
+
+	// wait for eps[target] to be pinned
+	mustWaitPinReady(t, watchCli)
+
+	// add all eps to list, so that when the original pined one fails
+	// the client can switch to other available eps
+	watchCli.SetEndpoints(eps...)
+
+	wch := watchCli.Watch(clientv3.WithRequireLeader(context.Background()), "foo", clientv3.WithCreatedNotify())
+	select {
+	case <-wch:
+	case <-time.After(3 * time.Second):
+		t.Fatal("took too long to create watch")
+	}
+
+	// isolate eps[target]
+	clus.Members[target].InjectPartition(t,
+		clus.Members[(target+1)%3],
+		clus.Members[(target+2)%3],
+	)
+
+	select {
+	case ev := <-wch:
+		if len(ev.Events) != 0 {
+			t.Fatal("expected no event")
+		}
+		if err = ev.Err(); err != rpctypes.ErrNoLeader {
+			t.Fatalf("expected %v, got %v", rpctypes.ErrNoLeader, err)
+		}
+	case <-time.After(3 * time.Second): // enough time to detect leader lost
+		t.Fatal("took too long to detect leader lost")
+	}
+}

+ 1 - 0
clientv3/integration/role_test.go

@@ -20,6 +20,7 @@ import (
 	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
 	"github.com/coreos/etcd/integration"
 	"github.com/coreos/etcd/pkg/testutil"
+
 	"golang.org/x/net/context"
 )
 

+ 352 - 0
clientv3/integration/server_shutdown_test.go

@@ -0,0 +1,352 @@
+// Copyright 2017 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 (
+	"bytes"
+	"testing"
+	"time"
+
+	"github.com/coreos/etcd/clientv3"
+	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
+	"github.com/coreos/etcd/integration"
+	"github.com/coreos/etcd/pkg/testutil"
+
+	"golang.org/x/net/context"
+)
+
+// TestBalancerUnderServerShutdownWatch expects that watch client
+// switch its endpoints when the member of the pinned endpoint fails.
+func TestBalancerUnderServerShutdownWatch(t *testing.T) {
+	defer testutil.AfterTest(t)
+
+	clus := integration.NewClusterV3(t, &integration.ClusterConfig{
+		Size:               3,
+		SkipCreatingClient: true,
+	})
+	defer clus.Terminate(t)
+
+	eps := []string{clus.Members[0].GRPCAddr(), clus.Members[1].GRPCAddr(), clus.Members[2].GRPCAddr()}
+
+	lead := clus.WaitLeader(t)
+
+	// pin eps[lead]
+	watchCli, err := clientv3.New(clientv3.Config{Endpoints: []string{eps[lead]}})
+	if err != nil {
+		t.Fatal(err)
+	}
+	defer watchCli.Close()
+
+	// wait for eps[lead] to be pinned
+	mustWaitPinReady(t, watchCli)
+
+	// add all eps to list, so that when the original pined one fails
+	// the client can switch to other available eps
+	watchCli.SetEndpoints(eps...)
+
+	key, val := "foo", "bar"
+	wch := watchCli.Watch(context.Background(), key, clientv3.WithCreatedNotify())
+	select {
+	case <-wch:
+	case <-time.After(3 * time.Second):
+		t.Fatal("took too long to create watch")
+	}
+
+	donec := make(chan struct{})
+	go func() {
+		defer close(donec)
+
+		// switch to others when eps[lead] is shut down
+		select {
+		case ev := <-wch:
+			if werr := ev.Err(); werr != nil {
+				t.Fatal(werr)
+			}
+			if len(ev.Events) != 1 {
+				t.Fatalf("expected one event, got %+v", ev)
+			}
+			if !bytes.Equal(ev.Events[0].Kv.Value, []byte(val)) {
+				t.Fatalf("expected %q, got %+v", val, ev.Events[0].Kv)
+			}
+		case <-time.After(7 * time.Second):
+			t.Fatal("took too long to receive events")
+		}
+	}()
+
+	// shut down eps[lead]
+	clus.Members[lead].Terminate(t)
+
+	// writes to eps[lead+1]
+	putCli, err := clientv3.New(clientv3.Config{Endpoints: []string{eps[(lead+1)%3]}})
+	if err != nil {
+		t.Fatal(err)
+	}
+	defer putCli.Close()
+	for {
+		ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
+		_, err = putCli.Put(ctx, key, val)
+		cancel()
+		if err == nil {
+			break
+		}
+		if err == context.DeadlineExceeded || err == rpctypes.ErrTimeout || err == rpctypes.ErrTimeoutDueToLeaderFail {
+			continue
+		}
+		t.Fatal(err)
+	}
+
+	select {
+	case <-donec:
+	case <-time.After(5 * time.Second): // enough time for balancer switch
+		t.Fatal("took too long to receive events")
+	}
+}
+
+func TestBalancerUnderServerShutdownPut(t *testing.T) {
+	testBalancerUnderServerShutdownMutable(t, func(cli *clientv3.Client, ctx context.Context) error {
+		_, err := cli.Put(ctx, "foo", "bar")
+		return err
+	})
+}
+
+func TestBalancerUnderServerShutdownDelete(t *testing.T) {
+	testBalancerUnderServerShutdownMutable(t, func(cli *clientv3.Client, ctx context.Context) error {
+		_, err := cli.Delete(ctx, "foo")
+		return err
+	})
+}
+
+func TestBalancerUnderServerShutdownTxn(t *testing.T) {
+	testBalancerUnderServerShutdownMutable(t, func(cli *clientv3.Client, ctx context.Context) error {
+		_, err := cli.Txn(ctx).
+			If(clientv3.Compare(clientv3.Version("foo"), "=", 0)).
+			Then(clientv3.OpPut("foo", "bar")).
+			Else(clientv3.OpPut("foo", "baz")).Commit()
+		return err
+	})
+}
+
+// testBalancerUnderServerShutdownMutable expects that when the member of
+// the pinned endpoint is shut down, the balancer switches its endpoints
+// and all subsequent put/delete/txn requests succeed with new endpoints.
+func testBalancerUnderServerShutdownMutable(t *testing.T, op func(*clientv3.Client, context.Context) error) {
+	defer testutil.AfterTest(t)
+
+	clus := integration.NewClusterV3(t, &integration.ClusterConfig{
+		Size:               3,
+		SkipCreatingClient: true,
+	})
+	defer clus.Terminate(t)
+
+	eps := []string{clus.Members[0].GRPCAddr(), clus.Members[1].GRPCAddr(), clus.Members[2].GRPCAddr()}
+
+	// pin eps[0]
+	cli, err := clientv3.New(clientv3.Config{Endpoints: []string{eps[0]}})
+	if err != nil {
+		t.Fatal(err)
+	}
+	defer cli.Close()
+
+	// wait for eps[0] to be pinned
+	mustWaitPinReady(t, cli)
+
+	// add all eps to list, so that when the original pined one fails
+	// the client can switch to other available eps
+	cli.SetEndpoints(eps...)
+
+	// shut down eps[0]
+	clus.Members[0].Terminate(t)
+
+	// switched to others when eps[0] was explicitly shut down
+	// and following request should succeed
+	// TODO: remove this (expose client connection state?)
+	time.Sleep(time.Second)
+
+	cctx, ccancel := context.WithTimeout(context.Background(), time.Second)
+	err = op(cli, cctx)
+	ccancel()
+	if err != nil {
+		t.Fatal(err)
+	}
+}
+
+func TestBalancerUnderServerShutdownGetLinearizable(t *testing.T) {
+	testBalancerUnderServerShutdownImmutable(t, func(cli *clientv3.Client, ctx context.Context) error {
+		_, err := cli.Get(ctx, "foo")
+		return err
+	}, 7*time.Second) // give enough time for leader election, balancer switch
+}
+
+func TestBalancerUnderServerShutdownGetSerializable(t *testing.T) {
+	testBalancerUnderServerShutdownImmutable(t, func(cli *clientv3.Client, ctx context.Context) error {
+		_, err := cli.Get(ctx, "foo", clientv3.WithSerializable())
+		return err
+	}, 2*time.Second)
+}
+
+// testBalancerUnderServerShutdownImmutable expects that when the member of
+// the pinned endpoint is shut down, the balancer switches its endpoints
+// and all subsequent range requests succeed with new endpoints.
+func testBalancerUnderServerShutdownImmutable(t *testing.T, op func(*clientv3.Client, context.Context) error, timeout time.Duration) {
+	defer testutil.AfterTest(t)
+
+	clus := integration.NewClusterV3(t, &integration.ClusterConfig{
+		Size:               3,
+		SkipCreatingClient: true,
+	})
+	defer clus.Terminate(t)
+
+	eps := []string{clus.Members[0].GRPCAddr(), clus.Members[1].GRPCAddr(), clus.Members[2].GRPCAddr()}
+
+	// pin eps[0]
+	cli, err := clientv3.New(clientv3.Config{Endpoints: []string{eps[0]}})
+	if err != nil {
+		t.Errorf("failed to create client: %v", err)
+	}
+	defer cli.Close()
+
+	// wait for eps[0] to be pinned
+	mustWaitPinReady(t, cli)
+
+	// add all eps to list, so that when the original pined one fails
+	// the client can switch to other available eps
+	cli.SetEndpoints(eps...)
+
+	// shut down eps[0]
+	clus.Members[0].Terminate(t)
+
+	// switched to others when eps[0] was explicitly shut down
+	// and following request should succeed
+	cctx, ccancel := context.WithTimeout(context.Background(), timeout)
+	err = op(cli, cctx)
+	ccancel()
+	if err != nil {
+		t.Errorf("failed to finish range request in time %v (timeout %v)", err, timeout)
+	}
+}
+
+func TestBalancerUnderServerStopInflightLinearizableGetOnRestart(t *testing.T) {
+	tt := []pinTestOpt{
+		{pinLeader: true, stopPinFirst: true},
+		{pinLeader: true, stopPinFirst: false},
+		{pinLeader: false, stopPinFirst: true},
+		{pinLeader: false, stopPinFirst: false},
+	}
+	for i := range tt {
+		testBalancerUnderServerStopInflightRangeOnRestart(t, true, tt[i])
+	}
+}
+
+func TestBalancerUnderServerStopInflightSerializableGetOnRestart(t *testing.T) {
+	tt := []pinTestOpt{
+		{pinLeader: true, stopPinFirst: true},
+		{pinLeader: true, stopPinFirst: false},
+		{pinLeader: false, stopPinFirst: true},
+		{pinLeader: false, stopPinFirst: false},
+	}
+	for i := range tt {
+		testBalancerUnderServerStopInflightRangeOnRestart(t, false, tt[i])
+	}
+}
+
+type pinTestOpt struct {
+	pinLeader    bool
+	stopPinFirst bool
+}
+
+// testBalancerUnderServerStopInflightRangeOnRestart expects
+// inflight range request reconnects on server restart.
+func testBalancerUnderServerStopInflightRangeOnRestart(t *testing.T, linearizable bool, opt pinTestOpt) {
+	defer testutil.AfterTest(t)
+
+	cfg := &integration.ClusterConfig{
+		Size:               2,
+		SkipCreatingClient: true,
+	}
+	if linearizable {
+		cfg.Size = 3
+	}
+
+	clus := integration.NewClusterV3(t, cfg)
+	defer clus.Terminate(t)
+	eps := []string{clus.Members[0].GRPCAddr(), clus.Members[1].GRPCAddr()}
+	if linearizable {
+		eps = append(eps, clus.Members[2].GRPCAddr())
+	}
+
+	lead := clus.WaitLeader(t)
+
+	target := lead
+	if !opt.pinLeader {
+		target = (target + 1) % 2
+	}
+
+	// pin eps[target]
+	cli, err := clientv3.New(clientv3.Config{Endpoints: []string{eps[target]}})
+	if err != nil {
+		t.Errorf("failed to create client: %v", err)
+	}
+	defer cli.Close()
+
+	// wait for eps[target] to be pinned
+	mustWaitPinReady(t, cli)
+
+	// add all eps to list, so that when the original pined one fails
+	// the client can switch to other available eps
+	cli.SetEndpoints(eps...)
+
+	if opt.stopPinFirst {
+		clus.Members[target].Stop(t)
+		// give some time for balancer switch before stopping the other
+		time.Sleep(time.Second)
+		clus.Members[(target+1)%2].Stop(t)
+	} else {
+		clus.Members[(target+1)%2].Stop(t)
+		// balancer cannot pin other member since it's already stopped
+		clus.Members[target].Stop(t)
+	}
+
+	// 3-second is the minimum interval between endpoint being marked
+	// as unhealthy and being removed from unhealthy, so possibly
+	// takes >5-second to unpin and repin an endpoint
+	// TODO: decrease timeout when balancer switch rewrite
+	clientTimeout := 7 * time.Second
+
+	var gops []clientv3.OpOption
+	if !linearizable {
+		gops = append(gops, clientv3.WithSerializable())
+	}
+
+	donec, readyc := make(chan struct{}), make(chan struct{}, 1)
+	go func() {
+		defer close(donec)
+		ctx, cancel := context.WithTimeout(context.TODO(), clientTimeout)
+		readyc <- struct{}{}
+		_, err := cli.Get(ctx, "abc", gops...)
+		cancel()
+		if err != nil {
+			t.Fatal(err)
+		}
+	}()
+
+	<-readyc
+	clus.Members[target].Restart(t)
+
+	select {
+	case <-time.After(clientTimeout + 3*time.Second):
+		t.Fatalf("timed out waiting for Get [linearizable: %v, opt: %+v]", linearizable, opt)
+	case <-donec:
+	}
+}

+ 3 - 1
clientv3/integration/txn_test.go

@@ -24,6 +24,7 @@ import (
 	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
 	"github.com/coreos/etcd/integration"
 	"github.com/coreos/etcd/pkg/testutil"
+
 	"golang.org/x/net/context"
 )
 
@@ -100,6 +101,8 @@ func TestTxnWriteFail(t *testing.T) {
 }
 
 func TestTxnReadRetry(t *testing.T) {
+	t.Skipf("skipping txn read retry test: re-enable after we do retry on txn read request")
+
 	defer testutil.AfterTest(t)
 
 	clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
@@ -129,7 +132,6 @@ func TestTxnReadRetry(t *testing.T) {
 		t.Fatalf("waited too long")
 	}
 }
-
 func TestTxnSuccess(t *testing.T) {
 	defer testutil.AfterTest(t)
 

+ 2 - 1
clientv3/integration/user_test.go

@@ -21,6 +21,7 @@ import (
 	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
 	"github.com/coreos/etcd/integration"
 	"github.com/coreos/etcd/pkg/testutil"
+
 	"golang.org/x/net/context"
 )
 
@@ -62,7 +63,7 @@ func TestUserErrorAuth(t *testing.T) {
 	authapi := clus.RandClient()
 	authSetupRoot(t, authapi.Auth)
 
-	// un-authenticated client
+	// unauthenticated client
 	if _, err := authapi.UserAdd(context.TODO(), "foo", "bar"); err != rpctypes.ErrUserNotFound {
 		t.Fatalf("expected %v, got %v", rpctypes.ErrUserNotFound, err)
 	}

+ 35 - 0
clientv3/integration/util.go

@@ -0,0 +1,35 @@
+// Copyright 2017 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 (
+	"testing"
+	"time"
+
+	"github.com/coreos/etcd/clientv3"
+
+	"golang.org/x/net/context"
+)
+
+// mustWaitPinReady waits up to 3-second until connection is up (pin endpoint).
+// Fatal on time-out.
+func mustWaitPinReady(t *testing.T, cli *clientv3.Client) {
+	ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second)
+	_, err := cli.Get(ctx, "foo")
+	cancel()
+	if err != nil {
+		t.Fatal(err)
+	}
+}

+ 34 - 16
clientv3/integration/watch_test.go

@@ -28,8 +28,10 @@ import (
 	"github.com/coreos/etcd/integration"
 	mvccpb "github.com/coreos/etcd/mvcc/mvccpb"
 	"github.com/coreos/etcd/pkg/testutil"
+
 	"golang.org/x/net/context"
 	"google.golang.org/grpc"
+	"google.golang.org/grpc/metadata"
 )
 
 type watcherTest func(*testing.T, *watchctx)
@@ -51,8 +53,8 @@ func runWatchTest(t *testing.T, f watcherTest) {
 
 	wclientMember := rand.Intn(3)
 	w := clus.Client(wclientMember).Watcher
-	// select a different client from wclient so puts succeed if
-	// a test knocks out the watcher client
+	// select a different client for KV operations so puts succeed if
+	// a test knocks out the watcher client.
 	kvMember := rand.Intn(3)
 	for kvMember == wclientMember {
 		kvMember = rand.Intn(3)
@@ -309,7 +311,7 @@ func testWatchCancelRunning(t *testing.T, wctx *watchctx) {
 	select {
 	case <-time.After(time.Second):
 		t.Fatalf("took too long to cancel")
-	case v, ok := <-wctx.ch:
+	case _, ok := <-wctx.ch:
 		if !ok {
 			// closed before getting put; OK
 			break
@@ -318,8 +320,8 @@ func testWatchCancelRunning(t *testing.T, wctx *watchctx) {
 		select {
 		case <-time.After(time.Second):
 			t.Fatalf("took too long to close")
-		case v, ok = <-wctx.ch:
-			if ok {
+		case v, ok2 := <-wctx.ch:
+			if ok2 {
 				t.Fatalf("expected watcher channel to close, got %v", v)
 			}
 		}
@@ -800,7 +802,8 @@ func TestWatchWithFilter(t *testing.T) {
 	}
 }
 
-// TestWatchWithCreatedNotification checks that createdNotification works.
+// TestWatchWithCreatedNotification checks that WithCreatedNotify returns a
+// Created watch response.
 func TestWatchWithCreatedNotification(t *testing.T) {
 	cluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
 	defer cluster.Terminate(t)
@@ -837,8 +840,7 @@ func TestWatchWithCreatedNotificationDropConn(t *testing.T) {
 
 	cluster.Members[0].DropConnections()
 
-	// try to receive from watch channel again
-	// ensure it doesn't post another createNotify
+	// check watch channel doesn't post another watch response.
 	select {
 	case wresp := <-wch:
 		t.Fatalf("got unexpected watch response: %+v\n", wresp)
@@ -856,10 +858,26 @@ func TestWatchCancelOnServer(t *testing.T) {
 	client := cluster.RandClient()
 	numWatches := 10
 
+	// The grpc proxy starts watches to detect leadership after the proxy server
+	// returns as started; to avoid racing on the proxy's internal watches, wait
+	// until require leader watches get create responses to ensure the leadership
+	// watches have started.
+	for {
+		ctx, cancel := context.WithCancel(clientv3.WithRequireLeader(context.TODO()))
+		ww := client.Watch(ctx, "a", clientv3.WithCreatedNotify())
+		wresp := <-ww
+		cancel()
+		if wresp.Err() == nil {
+			break
+		}
+	}
+
 	cancels := make([]context.CancelFunc, numWatches)
 	for i := 0; i < numWatches; i++ {
-		// use WithTimeout to force separate streams in client
-		ctx, cancel := context.WithTimeout(context.Background(), time.Second)
+		// force separate streams in client
+		md := metadata.Pairs("some-key", fmt.Sprintf("%d", i))
+		mctx := metadata.NewOutgoingContext(context.Background(), md)
+		ctx, cancel := context.WithCancel(mctx)
 		cancels[i] = cancel
 		w := client.Watch(ctx, fmt.Sprintf("%d", i), clientv3.WithCreatedNotify())
 		<-w
@@ -885,7 +903,7 @@ func TestWatchCancelOnServer(t *testing.T) {
 		t.Fatalf("expected n=2 and err=nil, got n=%d and err=%v", n, serr)
 	}
 
-	if maxWatchV-minWatchV != numWatches {
+	if maxWatchV-minWatchV < numWatches {
 		t.Fatalf("expected %d canceled watchers, got %d", numWatches, maxWatchV-minWatchV)
 	}
 }
@@ -916,12 +934,12 @@ func testWatchOverlapContextCancel(t *testing.T, f func(*integration.ClusterV3))
 	clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
 	defer clus.Terminate(t)
 
-	// each unique context "%v" has a unique grpc stream
 	n := 100
 	ctxs, ctxc := make([]context.Context, 5), make([]chan struct{}, 5)
 	for i := range ctxs {
-		// make "%v" unique
-		ctxs[i] = context.WithValue(context.TODO(), "key", i)
+		// make unique stream
+		md := metadata.Pairs("some-key", fmt.Sprintf("%d", i))
+		ctxs[i] = metadata.NewOutgoingContext(context.Background(), md)
 		// limits the maximum number of outstanding watchers per stream
 		ctxc[i] = make(chan struct{}, 2)
 	}
@@ -946,7 +964,7 @@ func testWatchOverlapContextCancel(t *testing.T, f func(*integration.ClusterV3))
 					t.Fatalf("unexpected closed channel %p", wch)
 				}
 			// may take a second or two to reestablish a watcher because of
-			// grpc backoff policies for disconnects
+			// grpc back off policies for disconnects
 			case <-time.After(5 * time.Second):
 				t.Errorf("timed out waiting for watch on %p", wch)
 			}
@@ -970,7 +988,7 @@ func testWatchOverlapContextCancel(t *testing.T, f func(*integration.ClusterV3))
 	}
 }
 
-// TestWatchCanelAndCloseClient ensures that canceling a watcher then immediately
+// TestWatchCancelAndCloseClient ensures that canceling a watcher then immediately
 // closing the client does not return a client closing error.
 func TestWatchCancelAndCloseClient(t *testing.T) {
 	defer testutil.AfterTest(t)

+ 24 - 21
clientv3/kv.go

@@ -16,8 +16,8 @@ package clientv3
 
 import (
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
+
 	"golang.org/x/net/context"
-	"google.golang.org/grpc"
 )
 
 type (
@@ -66,11 +66,26 @@ type OpResponse struct {
 	put *PutResponse
 	get *GetResponse
 	del *DeleteResponse
+	txn *TxnResponse
 }
 
 func (op OpResponse) Put() *PutResponse    { return op.put }
 func (op OpResponse) Get() *GetResponse    { return op.get }
 func (op OpResponse) Del() *DeleteResponse { return op.del }
+func (op OpResponse) Txn() *TxnResponse    { return op.txn }
+
+func (resp *PutResponse) OpResponse() OpResponse {
+	return OpResponse{put: resp}
+}
+func (resp *GetResponse) OpResponse() OpResponse {
+	return OpResponse{get: resp}
+}
+func (resp *DeleteResponse) OpResponse() OpResponse {
+	return OpResponse{del: resp}
+}
+func (resp *TxnResponse) OpResponse() OpResponse {
+	return OpResponse{txn: resp}
+}
 
 type kv struct {
 	remote pb.KVClient
@@ -115,29 +130,11 @@ func (kv *kv) Txn(ctx context.Context) Txn {
 }
 
 func (kv *kv) Do(ctx context.Context, op Op) (OpResponse, error) {
-	for {
-		resp, err := kv.do(ctx, op)
-		if err == nil {
-			return resp, nil
-		}
-
-		if isHaltErr(ctx, err) {
-			return resp, toErr(ctx, err)
-		}
-		// do not retry on modifications
-		if op.isWrite() {
-			return resp, toErr(ctx, err)
-		}
-	}
-}
-
-func (kv *kv) do(ctx context.Context, op Op) (OpResponse, error) {
 	var err error
 	switch op.t {
-	// TODO: handle other ops
 	case tRange:
 		var resp *pb.RangeResponse
-		resp, err = kv.remote.Range(ctx, op.toRangeRequest(), grpc.FailFast(false))
+		resp, err = kv.remote.Range(ctx, op.toRangeRequest())
 		if err == nil {
 			return OpResponse{get: (*GetResponse)(resp)}, nil
 		}
@@ -155,8 +152,14 @@ func (kv *kv) do(ctx context.Context, op Op) (OpResponse, error) {
 		if err == nil {
 			return OpResponse{del: (*DeleteResponse)(resp)}, nil
 		}
+	case tTxn:
+		var resp *pb.TxnResponse
+		resp, err = kv.remote.Txn(ctx, op.toTxnRequest())
+		if err == nil {
+			return OpResponse{txn: (*TxnResponse)(resp)}, nil
+		}
 	default:
 		panic("Unknown op")
 	}
-	return OpResponse{}, err
+	return OpResponse{}, toErr(ctx, err)
 }

+ 47 - 55
clientv3/lease.go

@@ -20,8 +20,8 @@ import (
 
 	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
+
 	"golang.org/x/net/context"
-	"google.golang.org/grpc"
 	"google.golang.org/grpc/metadata"
 )
 
@@ -30,7 +30,7 @@ type (
 	LeaseID             int64
 )
 
-// LeaseGrantResponse is used to convert the protobuf grant response.
+// LeaseGrantResponse wraps the protobuf message LeaseGrantResponse.
 type LeaseGrantResponse struct {
 	*pb.ResponseHeader
 	ID    LeaseID
@@ -38,14 +38,14 @@ type LeaseGrantResponse struct {
 	Error string
 }
 
-// LeaseKeepAliveResponse is used to convert the protobuf keepalive response.
+// LeaseKeepAliveResponse wraps the protobuf message LeaseKeepAliveResponse.
 type LeaseKeepAliveResponse struct {
 	*pb.ResponseHeader
 	ID  LeaseID
 	TTL int64
 }
 
-// LeaseTimeToLiveResponse is used to convert the protobuf lease timetolive response.
+// LeaseTimeToLiveResponse wraps the protobuf message LeaseTimeToLiveResponse.
 type LeaseTimeToLiveResponse struct {
 	*pb.ResponseHeader
 	ID LeaseID `json:"id"`
@@ -60,6 +60,12 @@ type LeaseTimeToLiveResponse struct {
 	Keys [][]byte `json:"keys"`
 }
 
+// LeaseStatus represents a lease status.
+type LeaseStatus struct {
+	ID LeaseID `json:"id"`
+	// TODO: TTL int64
+}
+
 const (
 	// defaultTTL is the assumed lease TTL used for the first keepalive
 	// deadline before the actual TTL is known to the client.
@@ -101,7 +107,7 @@ type Lease interface {
 	// KeepAlive keeps the given lease alive forever.
 	KeepAlive(ctx context.Context, id LeaseID) (<-chan *LeaseKeepAliveResponse, error)
 
-	// KeepAliveOnce renews the lease once. In most of the cases, Keepalive
+	// KeepAliveOnce renews the lease once. In most of the cases, KeepAlive
 	// should be used instead of KeepAliveOnce.
 	KeepAliveOnce(ctx context.Context, id LeaseID) (*LeaseKeepAliveResponse, error)
 
@@ -167,56 +173,43 @@ func NewLeaseFromLeaseClient(remote pb.LeaseClient, keepAliveTimeout time.Durati
 }
 
 func (l *lessor) Grant(ctx context.Context, ttl int64) (*LeaseGrantResponse, error) {
-	for {
-		r := &pb.LeaseGrantRequest{TTL: ttl}
-		resp, err := l.remote.LeaseGrant(ctx, r)
-		if err == nil {
-			gresp := &LeaseGrantResponse{
-				ResponseHeader: resp.GetHeader(),
-				ID:             LeaseID(resp.ID),
-				TTL:            resp.TTL,
-				Error:          resp.Error,
-			}
-			return gresp, nil
-		}
-		if isHaltErr(ctx, err) {
-			return nil, toErr(ctx, err)
+	r := &pb.LeaseGrantRequest{TTL: ttl}
+	resp, err := l.remote.LeaseGrant(ctx, r)
+	if err == nil {
+		gresp := &LeaseGrantResponse{
+			ResponseHeader: resp.GetHeader(),
+			ID:             LeaseID(resp.ID),
+			TTL:            resp.TTL,
+			Error:          resp.Error,
 		}
+		return gresp, nil
 	}
+	return nil, toErr(ctx, err)
 }
 
 func (l *lessor) Revoke(ctx context.Context, id LeaseID) (*LeaseRevokeResponse, error) {
-	for {
-		r := &pb.LeaseRevokeRequest{ID: int64(id)}
-		resp, err := l.remote.LeaseRevoke(ctx, r)
-
-		if err == nil {
-			return (*LeaseRevokeResponse)(resp), nil
-		}
-		if isHaltErr(ctx, err) {
-			return nil, toErr(ctx, err)
-		}
+	r := &pb.LeaseRevokeRequest{ID: int64(id)}
+	resp, err := l.remote.LeaseRevoke(ctx, r)
+	if err == nil {
+		return (*LeaseRevokeResponse)(resp), nil
 	}
+	return nil, toErr(ctx, err)
 }
 
 func (l *lessor) TimeToLive(ctx context.Context, id LeaseID, opts ...LeaseOption) (*LeaseTimeToLiveResponse, error) {
-	for {
-		r := toLeaseTimeToLiveRequest(id, opts...)
-		resp, err := l.remote.LeaseTimeToLive(ctx, r, grpc.FailFast(false))
-		if err == nil {
-			gresp := &LeaseTimeToLiveResponse{
-				ResponseHeader: resp.GetHeader(),
-				ID:             LeaseID(resp.ID),
-				TTL:            resp.TTL,
-				GrantedTTL:     resp.GrantedTTL,
-				Keys:           resp.Keys,
-			}
-			return gresp, nil
-		}
-		if isHaltErr(ctx, err) {
-			return nil, toErr(ctx, err)
+	r := toLeaseTimeToLiveRequest(id, opts...)
+	resp, err := l.remote.LeaseTimeToLive(ctx, r)
+	if err == nil {
+		gresp := &LeaseTimeToLiveResponse{
+			ResponseHeader: resp.GetHeader(),
+			ID:             LeaseID(resp.ID),
+			TTL:            resp.TTL,
+			GrantedTTL:     resp.GrantedTTL,
+			Keys:           resp.Keys,
 		}
+		return gresp, nil
 	}
+	return nil, toErr(ctx, err)
 }
 
 func (l *lessor) KeepAlive(ctx context.Context, id LeaseID) (<-chan *LeaseKeepAliveResponse, error) {
@@ -314,7 +307,7 @@ func (l *lessor) keepAliveCtxCloser(id LeaseID, ctx context.Context, donec <-cha
 	}
 }
 
-// closeRequireLeader scans all keep alives for ctxs that have require leader
+// closeRequireLeader scans keepAlives for ctxs that have require leader
 // and closes the associated channels.
 func (l *lessor) closeRequireLeader() {
 	l.mu.Lock()
@@ -323,7 +316,7 @@ func (l *lessor) closeRequireLeader() {
 		reqIdxs := 0
 		// find all required leader channels, close, mark as nil
 		for i, ctx := range ka.ctxs {
-			md, ok := metadata.FromContext(ctx)
+			md, ok := metadata.FromOutgoingContext(ctx)
 			if !ok {
 				continue
 			}
@@ -357,7 +350,7 @@ func (l *lessor) keepAliveOnce(ctx context.Context, id LeaseID) (*LeaseKeepAlive
 	cctx, cancel := context.WithCancel(ctx)
 	defer cancel()
 
-	stream, err := l.remote.LeaseKeepAlive(cctx, grpc.FailFast(false))
+	stream, err := l.remote.LeaseKeepAlive(cctx)
 	if err != nil {
 		return nil, toErr(ctx, err)
 	}
@@ -386,7 +379,7 @@ func (l *lessor) recvKeepAliveLoop() (gerr error) {
 		close(l.donec)
 		l.loopErr = gerr
 		for _, ka := range l.keepAlives {
-			ka.Close()
+			ka.close()
 		}
 		l.keepAlives = make(map[LeaseID]*keepAlive)
 		l.mu.Unlock()
@@ -401,7 +394,6 @@ func (l *lessor) recvKeepAliveLoop() (gerr error) {
 		} else {
 			for {
 				resp, err := stream.Recv()
-
 				if err != nil {
 					if canceledByCaller(l.stopCtx, err) {
 						return err
@@ -426,10 +418,10 @@ func (l *lessor) recvKeepAliveLoop() (gerr error) {
 	}
 }
 
-// resetRecv opens a new lease stream and starts sending LeaseKeepAliveRequests
+// resetRecv opens a new lease stream and starts sending keep alive requests.
 func (l *lessor) resetRecv() (pb.Lease_LeaseKeepAliveClient, error) {
 	sctx, cancel := context.WithCancel(l.stopCtx)
-	stream, err := l.remote.LeaseKeepAlive(sctx, grpc.FailFast(false))
+	stream, err := l.remote.LeaseKeepAlive(sctx)
 	if err != nil {
 		cancel()
 		return nil, err
@@ -467,7 +459,7 @@ func (l *lessor) recvKeepAlive(resp *pb.LeaseKeepAliveResponse) {
 	if karesp.TTL <= 0 {
 		// lease expired; close all keep alive channels
 		delete(l.keepAlives, karesp.ID)
-		ka.Close()
+		ka.close()
 		return
 	}
 
@@ -497,7 +489,7 @@ func (l *lessor) deadlineLoop() {
 		for id, ka := range l.keepAlives {
 			if ka.deadline.Before(now) {
 				// waited too long for response; lease may be expired
-				ka.Close()
+				ka.close()
 				delete(l.keepAlives, id)
 			}
 		}
@@ -505,7 +497,7 @@ func (l *lessor) deadlineLoop() {
 	}
 }
 
-// sendKeepAliveLoop sends LeaseKeepAliveRequests for the lifetime of a lease stream
+// sendKeepAliveLoop sends keep alive requests for the lifetime of the given stream.
 func (l *lessor) sendKeepAliveLoop(stream pb.Lease_LeaseKeepAliveClient) {
 	for {
 		var tosend []LeaseID
@@ -539,7 +531,7 @@ func (l *lessor) sendKeepAliveLoop(stream pb.Lease_LeaseKeepAliveClient) {
 	}
 }
 
-func (ka *keepAlive) Close() {
+func (ka *keepAlive) close() {
 	close(ka.donec)
 	for _, ch := range ka.chs {
 		close(ch)

+ 24 - 10
clientv3/logger.go

@@ -16,36 +16,35 @@ package clientv3
 
 import (
 	"io/ioutil"
-	"log"
 	"sync"
 
 	"google.golang.org/grpc/grpclog"
 )
 
 // Logger is the logger used by client library.
-// It implements grpclog.Logger interface.
-type Logger grpclog.Logger
+// It implements grpclog.LoggerV2 interface.
+type Logger grpclog.LoggerV2
 
 var (
 	logger settableLogger
 )
 
 type settableLogger struct {
-	l  grpclog.Logger
+	l  grpclog.LoggerV2
 	mu sync.RWMutex
 }
 
 func init() {
 	// disable client side logs by default
 	logger.mu.Lock()
-	logger.l = log.New(ioutil.Discard, "", 0)
+	logger.l = grpclog.NewLoggerV2(ioutil.Discard, ioutil.Discard, ioutil.Discard)
 
 	// logger has to override the grpclog at initialization so that
 	// any changes to the grpclog go through logger with locking
 	// instead of through SetLogger
 	//
 	// now updates only happen through settableLogger.set
-	grpclog.SetLogger(&logger)
+	grpclog.SetLoggerV2(&logger)
 	logger.mu.Unlock()
 }
 
@@ -62,6 +61,7 @@ func GetLogger() Logger {
 func (s *settableLogger) set(l Logger) {
 	s.mu.Lock()
 	logger.l = l
+	grpclog.SetLoggerV2(&logger)
 	s.mu.Unlock()
 }
 
@@ -72,11 +72,25 @@ func (s *settableLogger) get() Logger {
 	return l
 }
 
-// implement the grpclog.Logger interface
+// implement the grpclog.LoggerV2 interface
 
+func (s *settableLogger) Info(args ...interface{})                 { s.get().Info(args...) }
+func (s *settableLogger) Infof(format string, args ...interface{}) { s.get().Infof(format, args...) }
+func (s *settableLogger) Infoln(args ...interface{})               { s.get().Infoln(args...) }
+func (s *settableLogger) Warning(args ...interface{})              { s.get().Warning(args...) }
+func (s *settableLogger) Warningf(format string, args ...interface{}) {
+	s.get().Warningf(format, args...)
+}
+func (s *settableLogger) Warningln(args ...interface{}) { s.get().Warningln(args...) }
+func (s *settableLogger) Error(args ...interface{})     { s.get().Error(args...) }
+func (s *settableLogger) Errorf(format string, args ...interface{}) {
+	s.get().Errorf(format, args...)
+}
+func (s *settableLogger) Errorln(args ...interface{})               { s.get().Errorln(args...) }
 func (s *settableLogger) Fatal(args ...interface{})                 { s.get().Fatal(args...) }
 func (s *settableLogger) Fatalf(format string, args ...interface{}) { s.get().Fatalf(format, args...) }
 func (s *settableLogger) Fatalln(args ...interface{})               { s.get().Fatalln(args...) }
-func (s *settableLogger) Print(args ...interface{})                 { s.get().Print(args...) }
-func (s *settableLogger) Printf(format string, args ...interface{}) { s.get().Printf(format, args...) }
-func (s *settableLogger) Println(args ...interface{})               { s.get().Println(args...) }
+func (s *settableLogger) Print(args ...interface{})                 { s.get().Info(args...) }
+func (s *settableLogger) Printf(format string, args ...interface{}) { s.get().Infof(format, args...) }
+func (s *settableLogger) Println(args ...interface{})               { s.get().Infoln(args...) }
+func (s *settableLogger) V(l int) bool                              { return s.get().V(l) }

+ 8 - 2
clientv3/main_test.go

@@ -32,17 +32,23 @@ func init() { auth.BcryptCost = bcrypt.MinCost }
 
 // TestMain sets up an etcd cluster if running the examples.
 func TestMain(m *testing.M) {
-	useCluster := true // default to running all tests
+	useCluster, hasRunArg := false, false // default to running only Test*
 	for _, arg := range os.Args {
 		if strings.HasPrefix(arg, "-test.run=") {
 			exp := strings.Split(arg, "=")[1]
 			match, err := regexp.MatchString(exp, "Example")
 			useCluster = (err == nil && match) || strings.Contains(exp, "Example")
+			hasRunArg = true
 			break
 		}
 	}
+	if !hasRunArg {
+		// force only running Test* if no args given to avoid leak false
+		// positives from having a long-running cluster for the examples.
+		os.Args = append(os.Args, "-test.run=Test")
+	}
 
-	v := 0
+	var v int
 	if useCluster {
 		cfg := integration.ClusterConfig{Size: 3}
 		clus := integration.NewClusterV3(nil, &cfg)

+ 12 - 17
clientv3/maintenance.go

@@ -20,7 +20,6 @@ import (
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 
 	"golang.org/x/net/context"
-	"google.golang.org/grpc"
 )
 
 type (
@@ -37,7 +36,7 @@ type Maintenance interface {
 	// AlarmDisarm disarms a given alarm.
 	AlarmDisarm(ctx context.Context, m *AlarmMember) (*AlarmResponse, error)
 
-	// Defragment defragments storage backend of the etcd member with given endpoint.
+	// Defragment releases wasted space from internal fragmentation on a given etcd member.
 	// Defragment is only needed when deleting a large number of keys and want to reclaim
 	// the resources.
 	// Defragment is an expensive operation. User should avoid defragmenting multiple members
@@ -49,7 +48,7 @@ type Maintenance interface {
 	// Status gets the status of the endpoint.
 	Status(ctx context.Context, endpoint string) (*StatusResponse, error)
 
-	// Snapshot provides a reader for a snapshot of a backend.
+	// Snapshot provides a reader for a point-in-time snapshot of etcd.
 	Snapshot(ctx context.Context) (io.ReadCloser, error)
 }
 
@@ -66,9 +65,9 @@ func NewMaintenance(c *Client) Maintenance {
 				return nil, nil, err
 			}
 			cancel := func() { conn.Close() }
-			return pb.NewMaintenanceClient(conn), cancel, nil
+			return RetryMaintenanceClient(c, conn), cancel, nil
 		},
-		remote: pb.NewMaintenanceClient(c.conn),
+		remote: RetryMaintenanceClient(c, c.conn),
 	}
 }
 
@@ -87,15 +86,11 @@ func (m *maintenance) AlarmList(ctx context.Context) (*AlarmResponse, error) {
 		MemberID: 0,                 // all
 		Alarm:    pb.AlarmType_NONE, // all
 	}
-	for {
-		resp, err := m.remote.Alarm(ctx, req, grpc.FailFast(false))
-		if err == nil {
-			return (*AlarmResponse)(resp), nil
-		}
-		if isHaltErr(ctx, err) {
-			return nil, toErr(ctx, err)
-		}
+	resp, err := m.remote.Alarm(ctx, req)
+	if err == nil {
+		return (*AlarmResponse)(resp), nil
 	}
+	return nil, toErr(ctx, err)
 }
 
 func (m *maintenance) AlarmDisarm(ctx context.Context, am *AlarmMember) (*AlarmResponse, error) {
@@ -121,7 +116,7 @@ func (m *maintenance) AlarmDisarm(ctx context.Context, am *AlarmMember) (*AlarmR
 		return &ret, nil
 	}
 
-	resp, err := m.remote.Alarm(ctx, req, grpc.FailFast(false))
+	resp, err := m.remote.Alarm(ctx, req)
 	if err == nil {
 		return (*AlarmResponse)(resp), nil
 	}
@@ -134,7 +129,7 @@ func (m *maintenance) Defragment(ctx context.Context, endpoint string) (*Defragm
 		return nil, toErr(ctx, err)
 	}
 	defer cancel()
-	resp, err := remote.Defragment(ctx, &pb.DefragmentRequest{}, grpc.FailFast(false))
+	resp, err := remote.Defragment(ctx, &pb.DefragmentRequest{})
 	if err != nil {
 		return nil, toErr(ctx, err)
 	}
@@ -147,7 +142,7 @@ func (m *maintenance) Status(ctx context.Context, endpoint string) (*StatusRespo
 		return nil, toErr(ctx, err)
 	}
 	defer cancel()
-	resp, err := remote.Status(ctx, &pb.StatusRequest{}, grpc.FailFast(false))
+	resp, err := remote.Status(ctx, &pb.StatusRequest{})
 	if err != nil {
 		return nil, toErr(ctx, err)
 	}
@@ -155,7 +150,7 @@ func (m *maintenance) Status(ctx context.Context, endpoint string) (*StatusRespo
 }
 
 func (m *maintenance) Snapshot(ctx context.Context) (io.ReadCloser, error) {
-	ss, err := m.remote.Snapshot(ctx, &pb.SnapshotRequest{}, grpc.FailFast(false))
+	ss, err := m.remote.Snapshot(ctx, &pb.SnapshotRequest{})
 	if err != nil {
 		return nil, toErr(ctx, err)
 	}

+ 1 - 0
clientv3/mirror/syncer.go

@@ -17,6 +17,7 @@ package mirror
 
 import (
 	"github.com/coreos/etcd/clientv3"
+
 	"golang.org/x/net/context"
 )
 

+ 2 - 2
clientv3/namespace/kv.go

@@ -15,11 +15,11 @@
 package namespace
 
 import (
-	"golang.org/x/net/context"
-
 	"github.com/coreos/etcd/clientv3"
 	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
+
+	"golang.org/x/net/context"
 )
 
 type kvPrefix struct {

+ 2 - 2
clientv3/namespace/lease.go

@@ -17,9 +17,9 @@ package namespace
 import (
 	"bytes"
 
-	"golang.org/x/net/context"
-
 	"github.com/coreos/etcd/clientv3"
+
+	"golang.org/x/net/context"
 )
 
 type leasePrefix struct {

+ 2 - 2
clientv3/namespace/watch.go

@@ -17,9 +17,9 @@ package namespace
 import (
 	"sync"
 
-	"golang.org/x/net/context"
-
 	"github.com/coreos/etcd/clientv3"
+
+	"golang.org/x/net/context"
 )
 
 type watcherPrefix struct {

+ 6 - 5
clientv3/naming/grpc.go

@@ -19,11 +19,12 @@ import (
 	"fmt"
 
 	etcd "github.com/coreos/etcd/clientv3"
-	"golang.org/x/net/context"
 
-	"google.golang.org/grpc"
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/naming"
+	"google.golang.org/grpc/status"
+
+	"golang.org/x/net/context"
 )
 
 var ErrWatcherClosed = fmt.Errorf("naming: watch closed")
@@ -39,13 +40,13 @@ func (gr *GRPCResolver) Update(ctx context.Context, target string, nm naming.Upd
 	case naming.Add:
 		var v []byte
 		if v, err = json.Marshal(nm); err != nil {
-			return grpc.Errorf(codes.InvalidArgument, err.Error())
+			return status.Error(codes.InvalidArgument, err.Error())
 		}
 		_, err = gr.Client.KV.Put(ctx, target+"/"+nm.Addr, string(v), opts...)
 	case naming.Delete:
 		_, err = gr.Client.Delete(ctx, target+"/"+nm.Addr, opts...)
 	default:
-		return grpc.Errorf(codes.InvalidArgument, "naming: bad naming op")
+		return status.Error(codes.InvalidArgument, "naming: bad naming op")
 	}
 	return err
 }
@@ -80,7 +81,7 @@ func (gw *gRPCWatcher) Next() ([]*naming.Update, error) {
 	// process new events on target/*
 	wr, ok := <-gw.wch
 	if !ok {
-		gw.err = grpc.Errorf(codes.Unavailable, "%s", ErrWatcherClosed)
+		gw.err = status.Error(codes.Unavailable, ErrWatcherClosed.Error())
 		return nil, gw.err
 	}
 	if gw.err = wr.Err(); gw.err != nil {

+ 8 - 5
clientv3/naming/grpc_test.go

@@ -19,12 +19,12 @@ import (
 	"reflect"
 	"testing"
 
-	"golang.org/x/net/context"
-	"google.golang.org/grpc/naming"
-
 	etcd "github.com/coreos/etcd/clientv3"
 	"github.com/coreos/etcd/integration"
 	"github.com/coreos/etcd/pkg/testutil"
+
+	"golang.org/x/net/context"
+	"google.golang.org/grpc/naming"
 )
 
 func TestGRPCResolver(t *testing.T) {
@@ -66,10 +66,13 @@ func TestGRPCResolver(t *testing.T) {
 
 	delOp := naming.Update{Op: naming.Delete, Addr: "127.0.0.1"}
 	err = r.Update(context.TODO(), "foo", delOp)
+	if err != nil {
+		t.Fatalf("failed to udpate %v", err)
+	}
 
 	us, err = w.Next()
 	if err != nil {
-		t.Fatal("failed to get udpate", err)
+		t.Fatalf("failed to get udpate %v", err)
 	}
 
 	wu = &naming.Update{
@@ -83,7 +86,7 @@ func TestGRPCResolver(t *testing.T) {
 	}
 }
 
-// TestGRPCResolverMultiInit ensures the resolver will initialize
+// TestGRPCResolverMulti ensures the resolver will initialize
 // correctly with multiple hosts and correctly receive multiple
 // updates in a single revision.
 func TestGRPCResolverMulti(t *testing.T) {

+ 82 - 8
clientv3/op.go

@@ -23,6 +23,7 @@ const (
 	tRange opType = iota + 1
 	tPut
 	tDeleteRange
+	tTxn
 )
 
 var (
@@ -67,9 +68,17 @@ type Op struct {
 	// for put
 	val     []byte
 	leaseID LeaseID
+
+	// txn
+	cmps    []Cmp
+	thenOps []Op
+	elseOps []Op
 }
 
-// accesors / mutators
+// accessors / mutators
+
+func (op Op) IsTxn() bool              { return op.t == tTxn }
+func (op Op) Txn() ([]Cmp, []Op, []Op) { return op.cmps, op.thenOps, op.elseOps }
 
 // KeyBytes returns the byte slice holding the Op's key.
 func (op Op) KeyBytes() []byte { return op.key }
@@ -80,6 +89,39 @@ func (op *Op) WithKeyBytes(key []byte) { op.key = key }
 // RangeBytes returns the byte slice holding with the Op's range end, if any.
 func (op Op) RangeBytes() []byte { return op.end }
 
+// Rev returns the requested revision, if any.
+func (op Op) Rev() int64 { return op.rev }
+
+// IsPut returns true iff the operation is a Put.
+func (op Op) IsPut() bool { return op.t == tPut }
+
+// IsGet returns true iff the operation is a Get.
+func (op Op) IsGet() bool { return op.t == tRange }
+
+// IsDelete returns true iff the operation is a Delete.
+func (op Op) IsDelete() bool { return op.t == tDeleteRange }
+
+// IsSerializable returns true if the serializable field is true.
+func (op Op) IsSerializable() bool { return op.serializable == true }
+
+// IsKeysOnly returns whether keysOnly is set.
+func (op Op) IsKeysOnly() bool { return op.keysOnly == true }
+
+// IsCountOnly returns whether countOnly is set.
+func (op Op) IsCountOnly() bool { return op.countOnly == true }
+
+// MinModRev returns the operation's minimum modify revision.
+func (op Op) MinModRev() int64 { return op.minModRev }
+
+// MaxModRev returns the operation's maximum modify revision.
+func (op Op) MaxModRev() int64 { return op.maxModRev }
+
+// MinCreateRev returns the operation's minimum create revision.
+func (op Op) MinCreateRev() int64 { return op.minCreateRev }
+
+// MaxCreateRev returns the operation's maximum create revision.
+func (op Op) MaxCreateRev() int64 { return op.maxCreateRev }
+
 // WithRangeBytes sets the byte slice for the Op's range end.
 func (op *Op) WithRangeBytes(end []byte) { op.end = end }
 
@@ -113,6 +155,22 @@ func (op Op) toRangeRequest() *pb.RangeRequest {
 	return r
 }
 
+func (op Op) toTxnRequest() *pb.TxnRequest {
+	thenOps := make([]*pb.RequestOp, len(op.thenOps))
+	for i, tOp := range op.thenOps {
+		thenOps[i] = tOp.toRequestOp()
+	}
+	elseOps := make([]*pb.RequestOp, len(op.elseOps))
+	for i, eOp := range op.elseOps {
+		elseOps[i] = eOp.toRequestOp()
+	}
+	cmps := make([]*pb.Compare, len(op.cmps))
+	for i := range op.cmps {
+		cmps[i] = (*pb.Compare)(&op.cmps[i])
+	}
+	return &pb.TxnRequest{Compare: cmps, Success: thenOps, Failure: elseOps}
+}
+
 func (op Op) toRequestOp() *pb.RequestOp {
 	switch op.t {
 	case tRange:
@@ -129,6 +187,19 @@ func (op Op) toRequestOp() *pb.RequestOp {
 }
 
 func (op Op) isWrite() bool {
+	if op.t == tTxn {
+		for _, tOp := range op.thenOps {
+			if tOp.isWrite() {
+				return true
+			}
+		}
+		for _, tOp := range op.elseOps {
+			if tOp.isWrite() {
+				return true
+			}
+		}
+		return false
+	}
 	return op.t != tRange
 }
 
@@ -194,6 +265,10 @@ func OpPut(key, val string, opts ...OpOption) Op {
 	return ret
 }
 
+func OpTxn(cmps []Cmp, thenOps []Op, elseOps []Op) Op {
+	return Op{t: tTxn, cmps: cmps, thenOps: thenOps, elseOps: elseOps}
+}
+
 func opWatch(key string, opts ...OpOption) Op {
 	ret := Op{t: tRange, key: []byte(key)}
 	ret.applyOpts(opts)
@@ -247,9 +322,9 @@ func WithSort(target SortTarget, order SortOrder) OpOption {
 		if target == SortByKey && order == SortAscend {
 			// If order != SortNone, server fetches the entire key-space,
 			// and then applies the sort and limit, if provided.
-			// Since current mvcc.Range implementation returns results
-			// sorted by keys in lexicographically ascending order,
-			// client should ignore SortOrder if the target is SortByKey.
+			// Since by default the server returns results sorted by keys
+			// in lexicographically ascending order, the client should ignore
+			// SortOrder if the target is SortByKey.
 			order = SortNone
 		}
 		op.sort = &SortOption{target, order}
@@ -390,7 +465,7 @@ func WithPrevKV() OpOption {
 }
 
 // WithIgnoreValue updates the key using its current value.
-// Empty value should be passed when ignore_value is set.
+// This option can not be combined with non-empty values.
 // Returns an error if the key does not exist.
 func WithIgnoreValue() OpOption {
 	return func(op *Op) {
@@ -399,7 +474,7 @@ func WithIgnoreValue() OpOption {
 }
 
 // WithIgnoreLease updates the key using its current lease.
-// Empty lease should be passed when ignore_lease is set.
+// This option can not be combined with WithLease.
 // Returns an error if the key does not exist.
 func WithIgnoreLease() OpOption {
 	return func(op *Op) {
@@ -424,8 +499,7 @@ func (op *LeaseOp) applyOpts(opts []LeaseOption) {
 	}
 }
 
-// WithAttachedKeys requests lease timetolive API to return
-// attached keys of given lease ID.
+// WithAttachedKeys makes TimeToLive list the keys attached to the given lease ID.
 func WithAttachedKeys() LeaseOption {
 	return func(op *LeaseOp) { op.attachedKeys = true }
 }

+ 30 - 0
clientv3/ready_wait.go

@@ -0,0 +1,30 @@
+// Copyright 2017 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 "golang.org/x/net/context"
+
+// TODO: remove this when "FailFast=false" is fixed.
+// See https://github.com/grpc/grpc-go/issues/1532.
+func readyWait(rpcCtx, clientCtx context.Context, ready <-chan struct{}) error {
+	select {
+	case <-ready:
+		return nil
+	case <-rpcCtx.Done():
+		return rpcCtx.Err()
+	case <-clientCtx.Done():
+		return clientCtx.Err()
+	}
+}

+ 277 - 99
clientv3/retry.go

@@ -17,135 +17,183 @@ package clientv3
 import (
 	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
+
 	"golang.org/x/net/context"
 	"google.golang.org/grpc"
 	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/status"
 )
 
 type rpcFunc func(ctx context.Context) error
-type retryRpcFunc func(context.Context, rpcFunc) error
+type retryRPCFunc func(context.Context, rpcFunc) error
+type retryStopErrFunc func(error) bool
+
+func isRepeatableStopError(err error) bool {
+	eErr := rpctypes.Error(err)
+	// always stop retry on etcd errors
+	if serverErr, ok := eErr.(rpctypes.EtcdError); ok && serverErr.Code() != codes.Unavailable {
+		return true
+	}
+	// only retry if unavailable
+	ev, _ := status.FromError(err)
+	return ev.Code() != codes.Unavailable
+}
+
+func isNonRepeatableStopError(err error) bool {
+	ev, _ := status.FromError(err)
+	if ev.Code() != codes.Unavailable {
+		return true
+	}
+	desc := rpctypes.ErrorDesc(err)
+	return desc != "there is no address available" && desc != "there is no connection available"
+}
 
-func (c *Client) newRetryWrapper() retryRpcFunc {
+func (c *Client) newRetryWrapper(isStop retryStopErrFunc) retryRPCFunc {
 	return func(rpcCtx context.Context, f rpcFunc) error {
 		for {
+			if err := readyWait(rpcCtx, c.ctx, c.balancer.ConnectNotify()); err != nil {
+				return err
+			}
+			pinned := c.balancer.pinned()
 			err := f(rpcCtx)
 			if err == nil {
 				return nil
 			}
-
-			eErr := rpctypes.Error(err)
-			// always stop retry on etcd errors
-			if _, ok := eErr.(rpctypes.EtcdError); ok {
-				return err
+			if logger.V(4) {
+				logger.Infof("clientv3/retry: error %q on pinned endpoint %q", err.Error(), pinned)
 			}
 
-			// only retry if unavailable
-			if grpc.Code(err) != codes.Unavailable {
-				return err
+			if s, ok := status.FromError(err); ok && (s.Code() == codes.Unavailable || s.Code() == codes.DeadlineExceeded || s.Code() == codes.Internal) {
+				// mark this before endpoint switch is triggered
+				c.balancer.hostPortError(pinned, err)
+				c.balancer.next()
+				if logger.V(4) {
+					logger.Infof("clientv3/retry: switching from %q due to error %q", pinned, err.Error())
+				}
 			}
 
-			select {
-			case <-c.balancer.ConnectNotify():
-			case <-rpcCtx.Done():
-				return rpcCtx.Err()
-			case <-c.ctx.Done():
-				return c.ctx.Err()
+			if isStop(err) {
+				return err
 			}
 		}
 	}
 }
 
-func (c *Client) newAuthRetryWrapper() retryRpcFunc {
+func (c *Client) newAuthRetryWrapper() retryRPCFunc {
 	return func(rpcCtx context.Context, f rpcFunc) error {
 		for {
+			pinned := c.balancer.pinned()
 			err := f(rpcCtx)
 			if err == nil {
 				return nil
 			}
-
+			if logger.V(4) {
+				logger.Infof("clientv3/auth-retry: error %q on pinned endpoint %q", err.Error(), pinned)
+			}
 			// always stop retry on etcd errors other than invalid auth token
 			if rpctypes.Error(err) == rpctypes.ErrInvalidAuthToken {
 				gterr := c.getToken(rpcCtx)
 				if gterr != nil {
+					if logger.V(4) {
+						logger.Infof("clientv3/auth-retry: cannot retry due to error %q(%q) on pinned endpoint %q", err.Error(), gterr.Error(), pinned)
+					}
 					return err // return the original error for simplicity
 				}
 				continue
 			}
-
 			return err
 		}
 	}
 }
 
-// RetryKVClient implements a KVClient that uses the client's FailFast retry policy.
+// RetryKVClient implements a KVClient.
 func RetryKVClient(c *Client) pb.KVClient {
-	retryWrite := &retryWriteKVClient{pb.NewKVClient(c.conn), c.retryWrapper}
-	return &retryKVClient{&retryWriteKVClient{retryWrite, c.retryAuthWrapper}}
+	repeatableRetry := c.newRetryWrapper(isRepeatableStopError)
+	nonRepeatableRetry := c.newRetryWrapper(isNonRepeatableStopError)
+	conn := pb.NewKVClient(c.conn)
+	retryBasic := &retryKVClient{&nonRepeatableKVClient{conn, nonRepeatableRetry}, repeatableRetry}
+	retryAuthWrapper := c.newAuthRetryWrapper()
+	return &retryKVClient{
+		&nonRepeatableKVClient{retryBasic, retryAuthWrapper},
+		retryAuthWrapper}
 }
 
 type retryKVClient struct {
-	*retryWriteKVClient
+	*nonRepeatableKVClient
+	repeatableRetry retryRPCFunc
 }
 
 func (rkv *retryKVClient) Range(ctx context.Context, in *pb.RangeRequest, opts ...grpc.CallOption) (resp *pb.RangeResponse, err error) {
-	err = rkv.retryf(ctx, func(rctx context.Context) error {
-		resp, err = rkv.retryWriteKVClient.Range(rctx, in, opts...)
+	err = rkv.repeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rkv.kc.Range(rctx, in, opts...)
 		return err
 	})
 	return resp, err
 }
 
-type retryWriteKVClient struct {
-	pb.KVClient
-	retryf retryRpcFunc
+type nonRepeatableKVClient struct {
+	kc                 pb.KVClient
+	nonRepeatableRetry retryRPCFunc
 }
 
-func (rkv *retryWriteKVClient) Put(ctx context.Context, in *pb.PutRequest, opts ...grpc.CallOption) (resp *pb.PutResponse, err error) {
-	err = rkv.retryf(ctx, func(rctx context.Context) error {
-		resp, err = rkv.KVClient.Put(rctx, in, opts...)
+func (rkv *nonRepeatableKVClient) Put(ctx context.Context, in *pb.PutRequest, opts ...grpc.CallOption) (resp *pb.PutResponse, err error) {
+	err = rkv.nonRepeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rkv.kc.Put(rctx, in, opts...)
 		return err
 	})
 	return resp, err
 }
 
-func (rkv *retryWriteKVClient) DeleteRange(ctx context.Context, in *pb.DeleteRangeRequest, opts ...grpc.CallOption) (resp *pb.DeleteRangeResponse, err error) {
-	err = rkv.retryf(ctx, func(rctx context.Context) error {
-		resp, err = rkv.KVClient.DeleteRange(rctx, in, opts...)
+func (rkv *nonRepeatableKVClient) DeleteRange(ctx context.Context, in *pb.DeleteRangeRequest, opts ...grpc.CallOption) (resp *pb.DeleteRangeResponse, err error) {
+	err = rkv.nonRepeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rkv.kc.DeleteRange(rctx, in, opts...)
 		return err
 	})
 	return resp, err
 }
 
-func (rkv *retryWriteKVClient) Txn(ctx context.Context, in *pb.TxnRequest, opts ...grpc.CallOption) (resp *pb.TxnResponse, err error) {
-	err = rkv.retryf(ctx, func(rctx context.Context) error {
-		resp, err = rkv.KVClient.Txn(rctx, in, opts...)
+func (rkv *nonRepeatableKVClient) Txn(ctx context.Context, in *pb.TxnRequest, opts ...grpc.CallOption) (resp *pb.TxnResponse, err error) {
+	// TODO: repeatableRetry if read-only txn
+	err = rkv.nonRepeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rkv.kc.Txn(rctx, in, opts...)
 		return err
 	})
 	return resp, err
 }
 
-func (rkv *retryWriteKVClient) Compact(ctx context.Context, in *pb.CompactionRequest, opts ...grpc.CallOption) (resp *pb.CompactionResponse, err error) {
-	err = rkv.retryf(ctx, func(rctx context.Context) error {
-		resp, err = rkv.KVClient.Compact(rctx, in, opts...)
+func (rkv *nonRepeatableKVClient) Compact(ctx context.Context, in *pb.CompactionRequest, opts ...grpc.CallOption) (resp *pb.CompactionResponse, err error) {
+	err = rkv.nonRepeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rkv.kc.Compact(rctx, in, opts...)
 		return err
 	})
 	return resp, err
 }
 
 type retryLeaseClient struct {
-	pb.LeaseClient
-	retryf retryRpcFunc
+	lc              pb.LeaseClient
+	repeatableRetry retryRPCFunc
 }
 
-// RetryLeaseClient implements a LeaseClient that uses the client's FailFast retry policy.
+// RetryLeaseClient implements a LeaseClient.
 func RetryLeaseClient(c *Client) pb.LeaseClient {
-	retry := &retryLeaseClient{pb.NewLeaseClient(c.conn), c.retryWrapper}
-	return &retryLeaseClient{retry, c.retryAuthWrapper}
+	retry := &retryLeaseClient{
+		pb.NewLeaseClient(c.conn),
+		c.newRetryWrapper(isRepeatableStopError),
+	}
+	return &retryLeaseClient{retry, c.newAuthRetryWrapper()}
+}
+
+func (rlc *retryLeaseClient) LeaseTimeToLive(ctx context.Context, in *pb.LeaseTimeToLiveRequest, opts ...grpc.CallOption) (resp *pb.LeaseTimeToLiveResponse, err error) {
+	err = rlc.repeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rlc.lc.LeaseTimeToLive(rctx, in, opts...)
+		return err
+	})
+	return resp, err
 }
 
 func (rlc *retryLeaseClient) LeaseGrant(ctx context.Context, in *pb.LeaseGrantRequest, opts ...grpc.CallOption) (resp *pb.LeaseGrantResponse, err error) {
-	err = rlc.retryf(ctx, func(rctx context.Context) error {
-		resp, err = rlc.LeaseClient.LeaseGrant(rctx, in, opts...)
+	err = rlc.repeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rlc.lc.LeaseGrant(rctx, in, opts...)
 		return err
 	})
 	return resp, err
@@ -153,140 +201,270 @@ func (rlc *retryLeaseClient) LeaseGrant(ctx context.Context, in *pb.LeaseGrantRe
 }
 
 func (rlc *retryLeaseClient) LeaseRevoke(ctx context.Context, in *pb.LeaseRevokeRequest, opts ...grpc.CallOption) (resp *pb.LeaseRevokeResponse, err error) {
-	err = rlc.retryf(ctx, func(rctx context.Context) error {
-		resp, err = rlc.LeaseClient.LeaseRevoke(rctx, in, opts...)
+	err = rlc.repeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rlc.lc.LeaseRevoke(rctx, in, opts...)
 		return err
 	})
 	return resp, err
 }
 
+func (rlc *retryLeaseClient) LeaseKeepAlive(ctx context.Context, opts ...grpc.CallOption) (stream pb.Lease_LeaseKeepAliveClient, err error) {
+	err = rlc.repeatableRetry(ctx, func(rctx context.Context) error {
+		stream, err = rlc.lc.LeaseKeepAlive(rctx, opts...)
+		return err
+	})
+	return stream, err
+}
+
 type retryClusterClient struct {
-	pb.ClusterClient
-	retryf retryRpcFunc
+	*nonRepeatableClusterClient
+	repeatableRetry retryRPCFunc
 }
 
-// RetryClusterClient implements a ClusterClient that uses the client's FailFast retry policy.
+// RetryClusterClient implements a ClusterClient.
 func RetryClusterClient(c *Client) pb.ClusterClient {
-	return &retryClusterClient{pb.NewClusterClient(c.conn), c.retryWrapper}
+	repeatableRetry := c.newRetryWrapper(isRepeatableStopError)
+	nonRepeatableRetry := c.newRetryWrapper(isNonRepeatableStopError)
+	cc := pb.NewClusterClient(c.conn)
+	return &retryClusterClient{&nonRepeatableClusterClient{cc, nonRepeatableRetry}, repeatableRetry}
+}
+
+func (rcc *retryClusterClient) MemberList(ctx context.Context, in *pb.MemberListRequest, opts ...grpc.CallOption) (resp *pb.MemberListResponse, err error) {
+	err = rcc.repeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rcc.cc.MemberList(rctx, in, opts...)
+		return err
+	})
+	return resp, err
+}
+
+type nonRepeatableClusterClient struct {
+	cc                 pb.ClusterClient
+	nonRepeatableRetry retryRPCFunc
+}
+
+func (rcc *nonRepeatableClusterClient) MemberAdd(ctx context.Context, in *pb.MemberAddRequest, opts ...grpc.CallOption) (resp *pb.MemberAddResponse, err error) {
+	err = rcc.nonRepeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rcc.cc.MemberAdd(rctx, in, opts...)
+		return err
+	})
+	return resp, err
+}
+
+func (rcc *nonRepeatableClusterClient) MemberRemove(ctx context.Context, in *pb.MemberRemoveRequest, opts ...grpc.CallOption) (resp *pb.MemberRemoveResponse, err error) {
+	err = rcc.nonRepeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rcc.cc.MemberRemove(rctx, in, opts...)
+		return err
+	})
+	return resp, err
+}
+
+func (rcc *nonRepeatableClusterClient) MemberUpdate(ctx context.Context, in *pb.MemberUpdateRequest, opts ...grpc.CallOption) (resp *pb.MemberUpdateResponse, err error) {
+	err = rcc.nonRepeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rcc.cc.MemberUpdate(rctx, in, opts...)
+		return err
+	})
+	return resp, err
+}
+
+// RetryMaintenanceClient implements a Maintenance.
+func RetryMaintenanceClient(c *Client, conn *grpc.ClientConn) pb.MaintenanceClient {
+	repeatableRetry := c.newRetryWrapper(isRepeatableStopError)
+	nonRepeatableRetry := c.newRetryWrapper(isNonRepeatableStopError)
+	mc := pb.NewMaintenanceClient(conn)
+	return &retryMaintenanceClient{&nonRepeatableMaintenanceClient{mc, nonRepeatableRetry}, repeatableRetry}
+}
+
+type retryMaintenanceClient struct {
+	*nonRepeatableMaintenanceClient
+	repeatableRetry retryRPCFunc
 }
 
-func (rcc *retryClusterClient) MemberAdd(ctx context.Context, in *pb.MemberAddRequest, opts ...grpc.CallOption) (resp *pb.MemberAddResponse, err error) {
-	err = rcc.retryf(ctx, func(rctx context.Context) error {
-		resp, err = rcc.ClusterClient.MemberAdd(rctx, in, opts...)
+func (rmc *retryMaintenanceClient) Alarm(ctx context.Context, in *pb.AlarmRequest, opts ...grpc.CallOption) (resp *pb.AlarmResponse, err error) {
+	err = rmc.repeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rmc.mc.Alarm(rctx, in, opts...)
 		return err
 	})
 	return resp, err
 }
 
-func (rcc *retryClusterClient) MemberRemove(ctx context.Context, in *pb.MemberRemoveRequest, opts ...grpc.CallOption) (resp *pb.MemberRemoveResponse, err error) {
-	err = rcc.retryf(ctx, func(rctx context.Context) error {
-		resp, err = rcc.ClusterClient.MemberRemove(rctx, in, opts...)
+func (rmc *retryMaintenanceClient) Status(ctx context.Context, in *pb.StatusRequest, opts ...grpc.CallOption) (resp *pb.StatusResponse, err error) {
+	err = rmc.repeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rmc.mc.Status(rctx, in, opts...)
 		return err
 	})
 	return resp, err
 }
 
-func (rcc *retryClusterClient) MemberUpdate(ctx context.Context, in *pb.MemberUpdateRequest, opts ...grpc.CallOption) (resp *pb.MemberUpdateResponse, err error) {
-	err = rcc.retryf(ctx, func(rctx context.Context) error {
-		resp, err = rcc.ClusterClient.MemberUpdate(rctx, in, opts...)
+func (rmc *retryMaintenanceClient) Hash(ctx context.Context, in *pb.HashRequest, opts ...grpc.CallOption) (resp *pb.HashResponse, err error) {
+	err = rmc.repeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rmc.mc.Hash(rctx, in, opts...)
+		return err
+	})
+	return resp, err
+}
+
+func (rmc *retryMaintenanceClient) Snapshot(ctx context.Context, in *pb.SnapshotRequest, opts ...grpc.CallOption) (stream pb.Maintenance_SnapshotClient, err error) {
+	err = rmc.repeatableRetry(ctx, func(rctx context.Context) error {
+		stream, err = rmc.mc.Snapshot(rctx, in, opts...)
+		return err
+	})
+	return stream, err
+}
+
+type nonRepeatableMaintenanceClient struct {
+	mc                 pb.MaintenanceClient
+	nonRepeatableRetry retryRPCFunc
+}
+
+func (rmc *nonRepeatableMaintenanceClient) Defragment(ctx context.Context, in *pb.DefragmentRequest, opts ...grpc.CallOption) (resp *pb.DefragmentResponse, err error) {
+	err = rmc.nonRepeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rmc.mc.Defragment(rctx, in, opts...)
 		return err
 	})
 	return resp, err
 }
 
 type retryAuthClient struct {
-	pb.AuthClient
-	retryf retryRpcFunc
+	*nonRepeatableAuthClient
+	repeatableRetry retryRPCFunc
 }
 
-// RetryAuthClient implements a AuthClient that uses the client's FailFast retry policy.
+// RetryAuthClient implements a AuthClient.
 func RetryAuthClient(c *Client) pb.AuthClient {
-	return &retryAuthClient{pb.NewAuthClient(c.conn), c.retryWrapper}
+	repeatableRetry := c.newRetryWrapper(isRepeatableStopError)
+	nonRepeatableRetry := c.newRetryWrapper(isNonRepeatableStopError)
+	ac := pb.NewAuthClient(c.conn)
+	return &retryAuthClient{&nonRepeatableAuthClient{ac, nonRepeatableRetry}, repeatableRetry}
+}
+
+func (rac *retryAuthClient) UserList(ctx context.Context, in *pb.AuthUserListRequest, opts ...grpc.CallOption) (resp *pb.AuthUserListResponse, err error) {
+	err = rac.repeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rac.ac.UserList(rctx, in, opts...)
+		return err
+	})
+	return resp, err
+}
+
+func (rac *retryAuthClient) UserGet(ctx context.Context, in *pb.AuthUserGetRequest, opts ...grpc.CallOption) (resp *pb.AuthUserGetResponse, err error) {
+	err = rac.repeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rac.ac.UserGet(rctx, in, opts...)
+		return err
+	})
+	return resp, err
+}
+
+func (rac *retryAuthClient) RoleGet(ctx context.Context, in *pb.AuthRoleGetRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleGetResponse, err error) {
+	err = rac.repeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rac.ac.RoleGet(rctx, in, opts...)
+		return err
+	})
+	return resp, err
+}
+
+func (rac *retryAuthClient) RoleList(ctx context.Context, in *pb.AuthRoleListRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleListResponse, err error) {
+	err = rac.repeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rac.ac.RoleList(rctx, in, opts...)
+		return err
+	})
+	return resp, err
+}
+
+type nonRepeatableAuthClient struct {
+	ac                 pb.AuthClient
+	nonRepeatableRetry retryRPCFunc
+}
+
+func (rac *nonRepeatableAuthClient) AuthEnable(ctx context.Context, in *pb.AuthEnableRequest, opts ...grpc.CallOption) (resp *pb.AuthEnableResponse, err error) {
+	err = rac.nonRepeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rac.ac.AuthEnable(rctx, in, opts...)
+		return err
+	})
+	return resp, err
 }
 
-func (rac *retryAuthClient) AuthEnable(ctx context.Context, in *pb.AuthEnableRequest, opts ...grpc.CallOption) (resp *pb.AuthEnableResponse, err error) {
-	err = rac.retryf(ctx, func(rctx context.Context) error {
-		resp, err = rac.AuthClient.AuthEnable(rctx, in, opts...)
+func (rac *nonRepeatableAuthClient) AuthDisable(ctx context.Context, in *pb.AuthDisableRequest, opts ...grpc.CallOption) (resp *pb.AuthDisableResponse, err error) {
+	err = rac.nonRepeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rac.ac.AuthDisable(rctx, in, opts...)
 		return err
 	})
 	return resp, err
 }
 
-func (rac *retryAuthClient) AuthDisable(ctx context.Context, in *pb.AuthDisableRequest, opts ...grpc.CallOption) (resp *pb.AuthDisableResponse, err error) {
-	err = rac.retryf(ctx, func(rctx context.Context) error {
-		resp, err = rac.AuthClient.AuthDisable(rctx, in, opts...)
+func (rac *nonRepeatableAuthClient) UserAdd(ctx context.Context, in *pb.AuthUserAddRequest, opts ...grpc.CallOption) (resp *pb.AuthUserAddResponse, err error) {
+	err = rac.nonRepeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rac.ac.UserAdd(rctx, in, opts...)
 		return err
 	})
 	return resp, err
 }
 
-func (rac *retryAuthClient) UserAdd(ctx context.Context, in *pb.AuthUserAddRequest, opts ...grpc.CallOption) (resp *pb.AuthUserAddResponse, err error) {
-	err = rac.retryf(ctx, func(rctx context.Context) error {
-		resp, err = rac.AuthClient.UserAdd(rctx, in, opts...)
+func (rac *nonRepeatableAuthClient) UserDelete(ctx context.Context, in *pb.AuthUserDeleteRequest, opts ...grpc.CallOption) (resp *pb.AuthUserDeleteResponse, err error) {
+	err = rac.nonRepeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rac.ac.UserDelete(rctx, in, opts...)
 		return err
 	})
 	return resp, err
 }
 
-func (rac *retryAuthClient) UserDelete(ctx context.Context, in *pb.AuthUserDeleteRequest, opts ...grpc.CallOption) (resp *pb.AuthUserDeleteResponse, err error) {
-	err = rac.retryf(ctx, func(rctx context.Context) error {
-		resp, err = rac.AuthClient.UserDelete(rctx, in, opts...)
+func (rac *nonRepeatableAuthClient) UserChangePassword(ctx context.Context, in *pb.AuthUserChangePasswordRequest, opts ...grpc.CallOption) (resp *pb.AuthUserChangePasswordResponse, err error) {
+	err = rac.nonRepeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rac.ac.UserChangePassword(rctx, in, opts...)
 		return err
 	})
 	return resp, err
 }
 
-func (rac *retryAuthClient) UserChangePassword(ctx context.Context, in *pb.AuthUserChangePasswordRequest, opts ...grpc.CallOption) (resp *pb.AuthUserChangePasswordResponse, err error) {
-	err = rac.retryf(ctx, func(rctx context.Context) error {
-		resp, err = rac.AuthClient.UserChangePassword(rctx, in, opts...)
+func (rac *nonRepeatableAuthClient) UserGrantRole(ctx context.Context, in *pb.AuthUserGrantRoleRequest, opts ...grpc.CallOption) (resp *pb.AuthUserGrantRoleResponse, err error) {
+	err = rac.nonRepeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rac.ac.UserGrantRole(rctx, in, opts...)
 		return err
 	})
 	return resp, err
 }
 
-func (rac *retryAuthClient) UserGrantRole(ctx context.Context, in *pb.AuthUserGrantRoleRequest, opts ...grpc.CallOption) (resp *pb.AuthUserGrantRoleResponse, err error) {
-	err = rac.retryf(ctx, func(rctx context.Context) error {
-		resp, err = rac.AuthClient.UserGrantRole(rctx, in, opts...)
+func (rac *nonRepeatableAuthClient) UserRevokeRole(ctx context.Context, in *pb.AuthUserRevokeRoleRequest, opts ...grpc.CallOption) (resp *pb.AuthUserRevokeRoleResponse, err error) {
+	err = rac.nonRepeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rac.ac.UserRevokeRole(rctx, in, opts...)
 		return err
 	})
 	return resp, err
 }
 
-func (rac *retryAuthClient) UserRevokeRole(ctx context.Context, in *pb.AuthUserRevokeRoleRequest, opts ...grpc.CallOption) (resp *pb.AuthUserRevokeRoleResponse, err error) {
-	err = rac.retryf(ctx, func(rctx context.Context) error {
-		resp, err = rac.AuthClient.UserRevokeRole(rctx, in, opts...)
+func (rac *nonRepeatableAuthClient) RoleAdd(ctx context.Context, in *pb.AuthRoleAddRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleAddResponse, err error) {
+	err = rac.nonRepeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rac.ac.RoleAdd(rctx, in, opts...)
 		return err
 	})
 	return resp, err
 }
 
-func (rac *retryAuthClient) RoleAdd(ctx context.Context, in *pb.AuthRoleAddRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleAddResponse, err error) {
-	err = rac.retryf(ctx, func(rctx context.Context) error {
-		resp, err = rac.AuthClient.RoleAdd(rctx, in, opts...)
+func (rac *nonRepeatableAuthClient) RoleDelete(ctx context.Context, in *pb.AuthRoleDeleteRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleDeleteResponse, err error) {
+	err = rac.nonRepeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rac.ac.RoleDelete(rctx, in, opts...)
 		return err
 	})
 	return resp, err
 }
 
-func (rac *retryAuthClient) RoleDelete(ctx context.Context, in *pb.AuthRoleDeleteRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleDeleteResponse, err error) {
-	err = rac.retryf(ctx, func(rctx context.Context) error {
-		resp, err = rac.AuthClient.RoleDelete(rctx, in, opts...)
+func (rac *nonRepeatableAuthClient) RoleGrantPermission(ctx context.Context, in *pb.AuthRoleGrantPermissionRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleGrantPermissionResponse, err error) {
+	err = rac.nonRepeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rac.ac.RoleGrantPermission(rctx, in, opts...)
 		return err
 	})
 	return resp, err
 }
 
-func (rac *retryAuthClient) RoleGrantPermission(ctx context.Context, in *pb.AuthRoleGrantPermissionRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleGrantPermissionResponse, err error) {
-	err = rac.retryf(ctx, func(rctx context.Context) error {
-		resp, err = rac.AuthClient.RoleGrantPermission(rctx, in, opts...)
+func (rac *nonRepeatableAuthClient) RoleRevokePermission(ctx context.Context, in *pb.AuthRoleRevokePermissionRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleRevokePermissionResponse, err error) {
+	err = rac.nonRepeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rac.ac.RoleRevokePermission(rctx, in, opts...)
 		return err
 	})
 	return resp, err
 }
 
-func (rac *retryAuthClient) RoleRevokePermission(ctx context.Context, in *pb.AuthRoleRevokePermissionRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleRevokePermissionResponse, err error) {
-	err = rac.retryf(ctx, func(rctx context.Context) error {
-		resp, err = rac.AuthClient.RoleRevokePermission(rctx, in, opts...)
+func (rac *nonRepeatableAuthClient) Authenticate(ctx context.Context, in *pb.AuthenticateRequest, opts ...grpc.CallOption) (resp *pb.AuthenticateResponse, err error) {
+	err = rac.nonRepeatableRetry(ctx, func(rctx context.Context) error {
+		resp, err = rac.ac.Authenticate(rctx, in, opts...)
 		return err
 	})
 	return resp, err

+ 6 - 22
clientv3/txn.go

@@ -18,13 +18,13 @@ import (
 	"sync"
 
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
+
 	"golang.org/x/net/context"
-	"google.golang.org/grpc"
 )
 
 // Txn is the interface that wraps mini-transactions.
 //
-//	 Tx.If(
+//	 Txn(context.TODO()).If(
 //	  Compare(Value(k1), ">", v1),
 //	  Compare(Version(k1), "=", 2)
 //	 ).Then(
@@ -135,30 +135,14 @@ func (txn *txn) Else(ops ...Op) Txn {
 func (txn *txn) Commit() (*TxnResponse, error) {
 	txn.mu.Lock()
 	defer txn.mu.Unlock()
-	for {
-		resp, err := txn.commit()
-		if err == nil {
-			return resp, err
-		}
-		if isHaltErr(txn.ctx, err) {
-			return nil, toErr(txn.ctx, err)
-		}
-		if txn.isWrite {
-			return nil, toErr(txn.ctx, err)
-		}
-	}
-}
 
-func (txn *txn) commit() (*TxnResponse, error) {
 	r := &pb.TxnRequest{Compare: txn.cmps, Success: txn.sus, Failure: txn.fas}
 
-	var opts []grpc.CallOption
-	if !txn.isWrite {
-		opts = []grpc.CallOption{grpc.FailFast(false)}
-	}
-	resp, err := txn.kv.remote.Txn(txn.ctx, r, opts...)
+	var resp *pb.TxnResponse
+	var err error
+	resp, err = txn.kv.remote.Txn(txn.ctx, r)
 	if err != nil {
-		return nil, err
+		return nil, toErr(txn.ctx, err)
 	}
 	return (*TxnResponse)(resp), nil
 }

+ 32 - 22
clientv3/watch.go

@@ -22,9 +22,12 @@ import (
 	v3rpc "github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 	mvccpb "github.com/coreos/etcd/mvcc/mvccpb"
+
 	"golang.org/x/net/context"
 	"google.golang.org/grpc"
 	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/metadata"
+	"google.golang.org/grpc/status"
 )
 
 const (
@@ -40,10 +43,9 @@ type WatchChan <-chan WatchResponse
 
 type Watcher interface {
 	// Watch watches on a key or prefix. The watched events will be returned
-	// through the returned channel.
-	// If the watch is slow or the required rev is compacted, the watch request
-	// might be canceled from the server-side and the chan will be closed.
-	// 'opts' can be: 'WithRev' and/or 'WithPrefix'.
+	// through the returned channel. If revisions waiting to be sent over the
+	// watch are compacted, then the watch will be canceled by the server, the
+	// client will post a compacted error watch response, and the channel will close.
 	Watch(ctx context.Context, key string, opts ...OpOption) WatchChan
 
 	// Close closes the watcher and cancels all watch requests.
@@ -90,7 +92,7 @@ func (wr *WatchResponse) Err() error {
 		return v3rpc.ErrCompacted
 	case wr.Canceled:
 		if len(wr.cancelReason) != 0 {
-			return v3rpc.Error(grpc.Errorf(codes.FailedPrecondition, "%s", wr.cancelReason))
+			return v3rpc.Error(status.Error(codes.FailedPrecondition, wr.cancelReason))
 		}
 		return v3rpc.ErrFutureRev
 	}
@@ -135,7 +137,7 @@ type watchGrpcStream struct {
 	respc chan *pb.WatchResponse
 	// donec closes to broadcast shutdown
 	donec chan struct{}
-	// errc transmits errors from grpc Recv to the watch stream reconn logic
+	// errc transmits errors from grpc Recv to the watch stream reconnect logic
 	errc chan error
 	// closingc gets the watcherStream of closing watchers
 	closingc chan *watcherStream
@@ -214,16 +216,15 @@ func (w *watcher) newWatcherGrpcStream(inctx context.Context) *watchGrpcStream {
 		owner:      w,
 		remote:     w.remote,
 		ctx:        ctx,
-		ctxKey:     fmt.Sprintf("%v", inctx),
+		ctxKey:     streamKeyFromCtx(inctx),
 		cancel:     cancel,
 		substreams: make(map[int64]*watcherStream),
-
-		respc:    make(chan *pb.WatchResponse),
-		reqc:     make(chan *watchRequest),
-		donec:    make(chan struct{}),
-		errc:     make(chan error, 1),
-		closingc: make(chan *watcherStream),
-		resumec:  make(chan struct{}),
+		respc:      make(chan *pb.WatchResponse),
+		reqc:       make(chan *watchRequest),
+		donec:      make(chan struct{}),
+		errc:       make(chan error, 1),
+		closingc:   make(chan *watcherStream),
+		resumec:    make(chan struct{}),
 	}
 	go wgs.run()
 	return wgs
@@ -254,7 +255,7 @@ func (w *watcher) Watch(ctx context.Context, key string, opts ...OpOption) Watch
 	}
 
 	ok := false
-	ctxKey := fmt.Sprintf("%v", ctx)
+	ctxKey := streamKeyFromCtx(ctx)
 
 	// find or allocate appropriate grpc watch stream
 	w.mu.Lock()
@@ -317,14 +318,14 @@ func (w *watcher) Close() (err error) {
 	w.streams = nil
 	w.mu.Unlock()
 	for _, wgs := range streams {
-		if werr := wgs.Close(); werr != nil {
+		if werr := wgs.close(); werr != nil {
 			err = werr
 		}
 	}
 	return err
 }
 
-func (w *watchGrpcStream) Close() (err error) {
+func (w *watchGrpcStream) close() (err error) {
 	w.cancel()
 	<-w.donec
 	select {
@@ -435,7 +436,7 @@ func (w *watchGrpcStream) run() {
 				initReq: *wreq,
 				id:      -1,
 				outc:    outc,
-				// unbufffered so resumes won't cause repeat events
+				// unbuffered so resumes won't cause repeat events
 				recvc: make(chan *WatchResponse),
 			}
 
@@ -487,7 +488,7 @@ func (w *watchGrpcStream) run() {
 				req := &pb.WatchRequest{RequestUnion: cr}
 				wc.Send(req)
 			}
-		// watch client failed to recv; spawn another if possible
+		// watch client failed on Recv; spawn another if possible
 		case err := <-w.errc:
 			if isHaltErr(w.ctx, err) || toErr(w.ctx, err) == v3rpc.ErrNoLeader {
 				closeErr = err
@@ -749,7 +750,7 @@ func (w *watchGrpcStream) waitCancelSubstreams(stopc <-chan struct{}) <-chan str
 	return donec
 }
 
-// joinSubstream waits for all substream goroutines to complete
+// joinSubstreams waits for all substream goroutines to complete.
 func (w *watchGrpcStream) joinSubstreams() {
 	for _, ws := range w.substreams {
 		<-ws.donec
@@ -761,7 +762,9 @@ func (w *watchGrpcStream) joinSubstreams() {
 	}
 }
 
-// openWatchClient retries opening a watchclient until retryConnection fails
+// openWatchClient retries opening a watch client until success or halt.
+// manually retry in case "ws==nil && err==nil"
+// TODO: remove FailFast=false
 func (w *watchGrpcStream) openWatchClient() (ws pb.Watch_WatchClient, err error) {
 	for {
 		select {
@@ -782,7 +785,7 @@ func (w *watchGrpcStream) openWatchClient() (ws pb.Watch_WatchClient, err error)
 	return ws, nil
 }
 
-// toPB converts an internal watch request structure to its protobuf messagefunc (wr *watchRequest)
+// toPB converts an internal watch request structure to its protobuf WatchRequest structure.
 func (wr *watchRequest) toPB() *pb.WatchRequest {
 	req := &pb.WatchCreateRequest{
 		StartRevision:  wr.rev,
@@ -795,3 +798,10 @@ func (wr *watchRequest) toPB() *pb.WatchRequest {
 	cr := &pb.WatchRequest_CreateRequest{CreateRequest: req}
 	return &pb.WatchRequest{RequestUnion: cr}
 }
+
+func streamKeyFromCtx(ctx context.Context) string {
+	if md, ok := metadata.FromOutgoingContext(ctx); ok {
+		return fmt.Sprintf("%+v", md)
+	}
+	return ""
+}

+ 10 - 3
clientv3/yaml/config.go

@@ -33,7 +33,11 @@ type yamlConfig struct {
 	InsecureSkipTLSVerify bool   `json:"insecure-skip-tls-verify"`
 	Certfile              string `json:"cert-file"`
 	Keyfile               string `json:"key-file"`
-	CAfile                string `json:"ca-file"`
+	TrustedCAfile         string `json:"trusted-ca-file"`
+
+	// CAfile is being deprecated. Use 'TrustedCAfile' instead.
+	// TODO: deprecate this in v4
+	CAfile string `json:"ca-file"`
 }
 
 // NewConfig creates a new clientv3.Config from a yaml file.
@@ -66,8 +70,11 @@ func NewConfig(fpath string) (*clientv3.Config, error) {
 		}
 	}
 
-	if yc.CAfile != "" {
-		cp, err = tlsutil.NewCertPool([]string{yc.CAfile})
+	if yc.CAfile != "" && yc.TrustedCAfile == "" {
+		yc.TrustedCAfile = yc.CAfile
+	}
+	if yc.TrustedCAfile != "" {
+		cp, err = tlsutil.NewCertPool([]string{yc.TrustedCAfile})
 		if err != nil {
 			return nil, err
 		}

+ 5 - 5
clientv3/yaml/config_test.go

@@ -50,7 +50,7 @@ func TestConfigFromFile(t *testing.T) {
 			&yamlConfig{
 				Keyfile:               privateKeyPath,
 				Certfile:              certPath,
-				CAfile:                caPath,
+				TrustedCAfile:         caPath,
 				InsecureSkipTLSVerify: true,
 			},
 			false,
@@ -64,9 +64,9 @@ func TestConfigFromFile(t *testing.T) {
 		},
 		{
 			&yamlConfig{
-				Keyfile:  privateKeyPath,
-				Certfile: certPath,
-				CAfile:   "bad",
+				Keyfile:       privateKeyPath,
+				Certfile:      certPath,
+				TrustedCAfile: "bad",
 			},
 			true,
 		},
@@ -113,7 +113,7 @@ func TestConfigFromFile(t *testing.T) {
 			if tt.ym.Certfile != "" && len(cfg.TLS.Certificates) == 0 {
 				t.Errorf("#%d: failed to load in cert", i)
 			}
-			if tt.ym.CAfile != "" && cfg.TLS.RootCAs == nil {
+			if tt.ym.TrustedCAfile != "" && cfg.TLS.RootCAs == nil {
 				t.Errorf("#%d: failed to load in ca cert", i)
 			}
 			if cfg.TLS.InsecureSkipVerify != tt.ym.InsecureSkipTLSVerify {