mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
clientv3: fix client balancer with gRPC v1.7
Signed-off-by: Gyu-Ho Lee <gyuhox@gmail.com>
This commit is contained in:
parent
939337f450
commit
a8c84ffc93
@ -1,6 +1,6 @@
|
||||
# etcd/clientv3
|
||||
|
||||
[](https://godoc.org/github.com/coreos/etcd/clientv3)
|
||||
[](https://godoc.org/github.com/coreos/etcd/clientv3)
|
||||
|
||||
`etcd/clientv3` is the official Go etcd client for v3.
|
||||
|
||||
|
@ -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)
|
||||
}
|
||||
|
||||
|
@ -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
|
||||
}
|
@ -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
|
||||
}
|
||||
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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()).
|
||||
|
@ -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)
|
||||
}
|
||||
|
@ -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 }
|
||||
}
|
||||
|
@ -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)
|
||||
}
|
||||
|
@ -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 }
|
||||
|
@ -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"
|
||||
)
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
}
|
||||
|
||||
|
@ -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 {
|
||||
|
@ -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.
|
||||
|
@ -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")
|
||||
|
@ -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"
|
||||
}
|
@ -19,6 +19,7 @@ import (
|
||||
"log"
|
||||
|
||||
"github.com/coreos/etcd/clientv3"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
)
|
||||
|
||||
|
@ -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()
|
||||
|
@ -19,6 +19,7 @@ import (
|
||||
"log"
|
||||
|
||||
"github.com/coreos/etcd/clientv3"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
)
|
||||
|
||||
|
@ -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() {
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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,
|
||||
|
@ -19,6 +19,7 @@ import (
|
||||
"log"
|
||||
|
||||
"github.com/coreos/etcd/clientv3"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
)
|
||||
|
||||
|
627
clientv3/health_balancer.go
Normal file
627
clientv3/health_balancer.go
Normal file
@ -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
clientv3/integration/black_hole_test.go
Normal file
211
clientv3/integration/black_hole_test.go
Normal file
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
@ -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"
|
||||
)
|
||||
|
||||
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
@ -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()
|
||||
}
|
||||
|
||||
cfg := clientv3.Config{Endpoints: eps, DialTimeout: 500 * time.Millisecond}
|
||||
cli, err := clientv3.New(cfg)
|
||||
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)
|
||||
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
|
||||
}
|
||||
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:
|
||||
}
|
||||
|
||||
resp, err := clus.Client(0).Get(context.TODO(), "k")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if resp.Kvs[0].Version > 11 {
|
||||
t.Fatalf("expected version <= 10, got %+v", resp.Kvs[0])
|
||||
}
|
||||
}
|
||||
|
@ -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})
|
||||
|
@ -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))
|
||||
}
|
||||
|
@ -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{
|
||||
|
@ -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"
|
||||
)
|
||||
|
||||
|
@ -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
clientv3/integration/network_partition_test.go
Normal file
260
clientv3/integration/network_partition_test.go
Normal file
@ -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")
|
||||
}
|
||||
}
|
@ -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
clientv3/integration/server_shutdown_test.go
Normal file
352
clientv3/integration/server_shutdown_test.go
Normal file
@ -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:
|
||||
}
|
||||
}
|
@ -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)
|
||||
|
||||
|
@ -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
clientv3/integration/util.go
Normal file
35
clientv3/integration/util.go
Normal file
@ -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)
|
||||
}
|
||||
}
|
@ -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)
|
||||
|
@ -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)
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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) }
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
}
|
||||
|
@ -17,6 +17,7 @@ package mirror
|
||||
|
||||
import (
|
||||
"github.com/coreos/etcd/clientv3"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
)
|
||||
|
||||
|
@ -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 {
|
||||
|
@ -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 {
|
||||
|
@ -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 {
|
||||
|
@ -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 {
|
||||
|
@ -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) {
|
||||
|
@ -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
clientv3/ready_wait.go
Normal file
30
clientv3/ready_wait.go
Normal file
@ -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()
|
||||
}
|
||||
}
|
@ -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 (c *Client) newRetryWrapper() retryRpcFunc {
|
||||
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(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) 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 (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
|
||||
}
|
||||
|
||||
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...)
|
||||
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 *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 (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 (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 (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 (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) 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 *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) 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 *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) 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 *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) 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 *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
|
||||
}
|
||||
|
||||
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...)
|
||||
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) 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) 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) 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) 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) 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) 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) 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) 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) 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) 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) 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) 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 *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 *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 *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 *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 *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
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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 ""
|
||||
}
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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 {
|
||||
|
Loading…
x
Reference in New Issue
Block a user