diff --git a/clientv3/concurrency/session.go b/clientv3/concurrency/session.go index 0cb5ea7cf1e..2f3281d8e35 100644 --- a/clientv3/concurrency/session.go +++ b/clientv3/concurrency/session.go @@ -51,12 +51,9 @@ 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 { - return nil, err - } - + keepAlive := client.KeepAlive(ctx, id) donec := make(chan struct{}) + s := &Session{client: client, opts: ops, id: id, cancel: cancel, donec: donec} // keep the lease alive until client error or cancelled context diff --git a/clientv3/example_lease_test.go b/clientv3/example_lease_test.go index e1bd57a15d7..2eeab77229c 100644 --- a/clientv3/example_lease_test.go +++ b/clientv3/example_lease_test.go @@ -100,12 +100,13 @@ func ExampleLease_keepAlive() { } // the key 'foo' will be kept forever - ch, kaerr := cli.KeepAlive(context.TODO(), resp.ID) - if kaerr != nil { - log.Fatal(kaerr) - } + ch := cli.KeepAlive(context.TODO(), resp.ID) ka := <-ch + if ka.Err != nil { + log.Fatal(ka.Err) + } + fmt.Println("ttl:", ka.TTL) // Output: ttl: 5 } @@ -131,9 +132,9 @@ func ExampleLease_keepAliveOnce() { } // to renew the lease only once - ka, kaerr := cli.KeepAliveOnce(context.TODO(), resp.ID) - if kaerr != nil { - log.Fatal(kaerr) + ka := cli.KeepAliveOnce(context.TODO(), resp.ID) + if ka.Err != nil { + log.Fatal(ka.Err) } fmt.Println("ttl:", ka.TTL) diff --git a/clientv3/integration/lease_test.go b/clientv3/integration/lease_test.go index a0c41182697..544ef5b66b9 100644 --- a/clientv3/integration/lease_test.go +++ b/clientv3/integration/lease_test.go @@ -104,14 +104,14 @@ func TestLeaseKeepAliveOnce(t *testing.T) { t.Errorf("failed to create lease %v", err) } - _, err = lapi.KeepAliveOnce(context.Background(), resp.ID) - if err != nil { - t.Errorf("failed to keepalive lease %v", err) + ka := lapi.KeepAliveOnce(context.Background(), resp.ID) + if ka.Err != nil { + t.Errorf("failed to keepalive lease %v", ka.Err) } - _, err = lapi.KeepAliveOnce(context.Background(), clientv3.LeaseID(0)) - if err != rpctypes.ErrLeaseNotFound { - t.Errorf("expected %v, got %v", rpctypes.ErrLeaseNotFound, err) + ka = lapi.KeepAliveOnce(context.Background(), clientv3.LeaseID(0)) + if ka.Err != rpctypes.ErrLeaseNotFound { + t.Errorf("expected %v, got %v", rpctypes.ErrLeaseNotFound, ka.Err) } } @@ -129,10 +129,7 @@ func TestLeaseKeepAlive(t *testing.T) { t.Errorf("failed to create lease %v", err) } - rc, kerr := lapi.KeepAlive(context.Background(), resp.ID) - if kerr != nil { - t.Errorf("failed to keepalive lease %v", kerr) - } + rc := lapi.KeepAlive(context.Background(), resp.ID) kresp, ok := <-rc if !ok { @@ -163,11 +160,7 @@ func TestLeaseKeepAliveOneSecond(t *testing.T) { if err != nil { t.Errorf("failed to create lease %v", err) } - rc, kerr := cli.KeepAlive(context.Background(), resp.ID) - if kerr != nil { - t.Errorf("failed to keepalive lease %v", kerr) - } - + rc := cli.KeepAlive(context.Background(), resp.ID) for i := 0; i < 3; i++ { if _, ok := <-rc; !ok { t.Errorf("chan is closed, want not closed") @@ -193,10 +186,7 @@ func TestLeaseKeepAliveHandleFailure(t *testing.T) { t.Errorf("failed to create lease %v", err) } - rc, kerr := lapi.KeepAlive(context.Background(), resp.ID) - if kerr != nil { - t.Errorf("failed to keepalive lease %v", kerr) - } + rc := lapi.KeepAlive(context.Background(), resp.ID) kresp := <-rc if kresp.ID != resp.ID { @@ -230,7 +220,7 @@ func TestLeaseKeepAliveHandleFailure(t *testing.T) { type leaseCh struct { lid clientv3.LeaseID - ch <-chan *clientv3.LeaseKeepAliveResponse + ch clientv3.LeaseKeepAliveChan } // TestLeaseKeepAliveNotFound ensures a revoked lease won't stop other keep alives @@ -247,10 +237,7 @@ func TestLeaseKeepAliveNotFound(t *testing.T) { if rerr != nil { t.Fatal(rerr) } - kach, kaerr := cli.KeepAlive(context.Background(), resp.ID) - if kaerr != nil { - t.Fatal(kaerr) - } + kach := cli.KeepAlive(context.Background(), resp.ID) lchs = append(lchs, leaseCh{resp.ID, kach}) } @@ -375,10 +362,7 @@ func TestLeaseKeepAliveCloseAfterDisconnectRevoke(t *testing.T) { if err != nil { t.Fatal(err) } - rc, kerr := cli.KeepAlive(context.Background(), resp.ID) - if kerr != nil { - t.Fatal(kerr) - } + rc := cli.KeepAlive(context.Background(), resp.ID) kresp := <-rc if kresp.ID != resp.ID { t.Fatalf("ID = %x, want %x", kresp.ID, resp.ID) @@ -397,9 +381,10 @@ func TestLeaseKeepAliveCloseAfterDisconnectRevoke(t *testing.T) { // some keep-alives may still be buffered; drain until close timer := time.After(time.Duration(kresp.TTL) * time.Second) - for kresp != nil { + loop := true + for loop { select { - case kresp = <-rc: + case _, loop = <-rc: case <-timer: t.Fatalf("keepalive channel did not close") } @@ -423,10 +408,7 @@ func TestLeaseKeepAliveInitTimeout(t *testing.T) { } // keep client disconnected clus.Members[0].Stop(t) - rc, kerr := cli.KeepAlive(context.Background(), resp.ID) - if kerr != nil { - t.Fatal(kerr) - } + rc := cli.KeepAlive(context.Background(), resp.ID) select { case ka, ok := <-rc: if ok { @@ -454,10 +436,7 @@ func TestLeaseKeepAliveTTLTimeout(t *testing.T) { if err != nil { t.Fatal(err) } - rc, kerr := cli.KeepAlive(context.Background(), resp.ID) - if kerr != nil { - t.Fatal(kerr) - } + rc := cli.KeepAlive(context.Background(), resp.ID) if kresp := <-rc; kresp.ID != resp.ID { t.Fatalf("ID = %x, want %x", kresp.ID, resp.ID) } @@ -580,10 +559,7 @@ func TestLeaseRenewLostQuorum(t *testing.T) { kctx, kcancel := context.WithCancel(context.Background()) defer kcancel() - ka, err := cli.KeepAlive(kctx, r.ID) - if err != nil { - t.Fatal(err) - } + ka := cli.KeepAlive(kctx, r.ID) // consume first keepalive so next message sends when cluster is down <-ka lastKa := time.Now() @@ -630,9 +606,9 @@ func TestLeaseKeepAliveLoopExit(t *testing.T) { } cli.Close() - _, err = cli.KeepAlive(ctx, resp.ID) - if _, ok := err.(clientv3.ErrKeepAliveHalted); !ok { - t.Fatalf("expected %T, got %v(%T)", clientv3.ErrKeepAliveHalted{}, err, err) + ka := cli.KeepAlive(ctx, resp.ID) + if resp, ok := <-ka; ok { + t.Fatalf("expected closed channel, got response %+v", resp) } } @@ -707,15 +683,9 @@ func TestLeaseWithRequireLeader(t *testing.T) { t.Fatal(err2) } // kaReqLeader close if the leader is lost - kaReqLeader, kerr1 := c.KeepAlive(clientv3.WithRequireLeader(context.TODO()), lid1.ID) - if kerr1 != nil { - t.Fatal(kerr1) - } + kaReqLeader := c.KeepAlive(clientv3.WithRequireLeader(context.TODO()), lid1.ID) // kaWait will wait even if the leader is lost - kaWait, kerr2 := c.KeepAlive(context.TODO(), lid2.ID) - if kerr2 != nil { - t.Fatal(kerr2) - } + kaWait := c.KeepAlive(context.TODO(), lid2.ID) select { case <-kaReqLeader: diff --git a/clientv3/lease.go b/clientv3/lease.go index a6494ceee45..34772315dce 100644 --- a/clientv3/lease.go +++ b/clientv3/lease.go @@ -41,8 +41,10 @@ type LeaseGrantResponse struct { // LeaseKeepAliveResponse is used to convert the protobuf keepalive response. type LeaseKeepAliveResponse struct { *pb.ResponseHeader - ID LeaseID - TTL int64 + ID LeaseID + TTL int64 + Err error + Deadline time.Time } // LeaseTimeToLiveResponse is used to convert the protobuf lease timetolive response. @@ -70,23 +72,11 @@ const ( NoLease LeaseID = 0 // retryConnWait is how long to wait before retrying on a lost leader + // or keep alive loop failure. retryConnWait = 500 * time.Millisecond ) -// ErrKeepAliveHalted is returned if client keep alive loop halts with an unexpected error. -// -// This usually means that automatic lease renewal via KeepAlive is broken, but KeepAliveOnce will still work as expected. -type ErrKeepAliveHalted struct { - Reason error -} - -func (e ErrKeepAliveHalted) Error() string { - s := "etcdclient: leases keep alive halted" - if e.Reason != nil { - s += ": " + e.Reason.Error() - } - return s -} +type LeaseKeepAliveChan <-chan LeaseKeepAliveResponse type Lease interface { // Grant creates a new lease. @@ -98,12 +88,24 @@ type Lease interface { // TimeToLive retrieves the lease information of the given lease ID. TimeToLive(ctx context.Context, id LeaseID, opts ...LeaseOption) (*LeaseTimeToLiveResponse, error) - // 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 - // should be used instead of KeepAliveOnce. - KeepAliveOnce(ctx context.Context, id LeaseID) (*LeaseKeepAliveResponse, error) + // KeepAlive keeps the given lease alive forever. If the keepalive response posted to + // the channel is not consumed immediately, the lease client will continue sending keep alive requests + // to the etcd server at least every second until latest response is consumed. + // + // The KeepAlive channel closes if the underlying keep alive stream is interrupted in some + // way the client cannot handle itself; the error will be posted in the last keep + // alive message before closing. If there is no keepalive response within the + // lease's time-out, the channel will close with no error. In most cases calling + // KeepAlive again will re-establish keepalives with the target lease if it has not + // expired. + KeepAlive(ctx context.Context, id LeaseID) LeaseKeepAliveChan + + // KeepAliveOnce renews the lease once. The response corresponds to the + // first message from calling KeepAlive. If the response has a recoverable + // error, KeepAliveOnce will retry the RPC with a new keep alive message. + // + // In most of the cases, Keepalive should be used instead of KeepAliveOnce. + KeepAliveOnce(ctx context.Context, id LeaseID) LeaseKeepAliveResponse // Close releases all resources Lease keeps for efficient communication // with the etcd server. @@ -113,9 +115,8 @@ type Lease interface { type lessor struct { mu sync.Mutex // guards all fields - // donec is closed and loopErr is set when recvKeepAliveLoop stops - donec chan struct{} - loopErr error + // donec is closed when all goroutines are torn down from Close() + donec chan struct{} remote pb.LeaseClient @@ -137,7 +138,7 @@ type lessor struct { // keepAlive multiplexes a keepalive for a lease over multiple channels type keepAlive struct { - chs []chan<- *LeaseKeepAliveResponse + chs []chan<- LeaseKeepAliveResponse ctxs []context.Context // deadline is the time the keep alive channels close if no response deadline time.Time @@ -219,24 +220,22 @@ func (l *lessor) TimeToLive(ctx context.Context, id LeaseID, opts ...LeaseOption } } -func (l *lessor) KeepAlive(ctx context.Context, id LeaseID) (<-chan *LeaseKeepAliveResponse, error) { - ch := make(chan *LeaseKeepAliveResponse, leaseResponseChSize) +func (l *lessor) KeepAlive(ctx context.Context, id LeaseID) LeaseKeepAliveChan { + ch := make(chan LeaseKeepAliveResponse, leaseResponseChSize) l.mu.Lock() // ensure that recvKeepAliveLoop is still running select { case <-l.donec: - err := l.loopErr - l.mu.Unlock() close(ch) - return ch, ErrKeepAliveHalted{Reason: err} + return ch default: } ka, ok := l.keepAlives[id] if !ok { // create fresh keep alive ka = &keepAlive{ - chs: []chan<- *LeaseKeepAliveResponse{ch}, + chs: []chan<- LeaseKeepAliveResponse{ch}, ctxs: []context.Context{ctx}, deadline: time.Now().Add(l.firstKeepAliveTimeout), nextKeepAlive: time.Now(), @@ -252,24 +251,51 @@ func (l *lessor) KeepAlive(ctx context.Context, id LeaseID) (<-chan *LeaseKeepAl go l.keepAliveCtxCloser(id, ctx, ka.donec) l.firstKeepAliveOnce.Do(func() { - go l.recvKeepAliveLoop() + go func() { + defer func() { + l.mu.Lock() + for _, ka := range l.keepAlives { + ka.Close(nil) + } + close(l.donec) + l.mu.Unlock() + }() + + for l.stopCtx.Err() == nil { + err := l.recvKeepAliveLoop() + if err == context.Canceled { + // canceled by user; no error like WatchChan + err = nil + } + l.mu.Lock() + for _, ka := range l.keepAlives { + ka.Close(err) + } + l.keepAlives = make(map[LeaseID]*keepAlive) + l.mu.Unlock() + select { + case <-l.stopCtx.Done(): + case <-time.After(retryConnWait): + } + } + }() go l.deadlineLoop() }) - return ch, nil + return ch } -func (l *lessor) KeepAliveOnce(ctx context.Context, id LeaseID) (*LeaseKeepAliveResponse, error) { +func (l *lessor) KeepAliveOnce(ctx context.Context, id LeaseID) LeaseKeepAliveResponse { for { - resp, err := l.keepAliveOnce(ctx, id) - if err == nil { + resp := l.keepAliveOnce(ctx, id) + if resp.Err == nil { if resp.TTL <= 0 { - err = rpctypes.ErrLeaseNotFound + resp.Err = rpctypes.ErrLeaseNotFound } - return resp, err + return resp } - if isHaltErr(ctx, err) { - return nil, toErr(ctx, err) + if isHaltErr(ctx, resp.Err) { + return resp } } } @@ -339,7 +365,7 @@ func (l *lessor) closeRequireLeader() { continue } // remove all channels that required a leader from keepalive - newChs := make([]chan<- *LeaseKeepAliveResponse, len(ka.chs)-reqIdxs) + newChs := make([]chan<- LeaseKeepAliveResponse, len(ka.chs)-reqIdxs) newCtxs := make([]context.Context, len(newChs)) newIdx := 0 for i := range ka.chs { @@ -353,45 +379,34 @@ func (l *lessor) closeRequireLeader() { } } -func (l *lessor) keepAliveOnce(ctx context.Context, id LeaseID) (*LeaseKeepAliveResponse, error) { +func (l *lessor) keepAliveOnce(ctx context.Context, id LeaseID) LeaseKeepAliveResponse { cctx, cancel := context.WithCancel(ctx) defer cancel() stream, err := l.remote.LeaseKeepAlive(cctx, grpc.FailFast(false)) if err != nil { - return nil, toErr(ctx, err) + return LeaseKeepAliveResponse{Err: toErr(ctx, err)} } err = stream.Send(&pb.LeaseKeepAliveRequest{ID: int64(id)}) if err != nil { - return nil, toErr(ctx, err) + return LeaseKeepAliveResponse{Err: toErr(ctx, err)} } resp, rerr := stream.Recv() if rerr != nil { - return nil, toErr(ctx, rerr) + return LeaseKeepAliveResponse{Err: toErr(ctx, rerr)} } - karesp := &LeaseKeepAliveResponse{ + return LeaseKeepAliveResponse{ ResponseHeader: resp.GetHeader(), ID: LeaseID(resp.ID), TTL: resp.TTL, + Deadline: time.Now().Add(time.Duration(resp.TTL) * time.Second), } - return karesp, nil } func (l *lessor) recvKeepAliveLoop() (gerr error) { - defer func() { - l.mu.Lock() - close(l.donec) - l.loopErr = gerr - for _, ka := range l.keepAlives { - ka.Close() - } - l.keepAlives = make(map[LeaseID]*keepAlive) - l.mu.Unlock() - }() - stream, serr := l.resetRecv() for serr == nil { resp, err := stream.Recv() @@ -443,6 +458,7 @@ func (l *lessor) recvKeepAlive(resp *pb.LeaseKeepAliveResponse) { ResponseHeader: resp.GetHeader(), ID: LeaseID(resp.ID), TTL: resp.TTL, + Deadline: time.Now().Add(time.Duration(resp.TTL) * time.Second), } l.mu.Lock() @@ -456,7 +472,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(nil) return } @@ -465,7 +481,7 @@ func (l *lessor) recvKeepAlive(resp *pb.LeaseKeepAliveResponse) { ka.deadline = time.Now().Add(time.Duration(karesp.TTL) * time.Second) for _, ch := range ka.chs { select { - case ch <- karesp: + case ch <- *karesp: ka.nextKeepAlive = nextKeepAlive default: } @@ -486,7 +502,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(nil) delete(l.keepAlives, id) } } @@ -528,9 +544,18 @@ func (l *lessor) sendKeepAliveLoop(stream pb.Lease_LeaseKeepAliveClient) { } } -func (ka *keepAlive) Close() { +func (ka *keepAlive) Close(err error) { close(ka.donec) for _, ch := range ka.chs { + if err != nil { + // try to post error if buffer space available + select { + case ch <- LeaseKeepAliveResponse{Err: err}: + default: + } + } close(ch) } + // so keepAliveCtxClose doesn't double-close ka.chs + ka.chs, ka.ctxs = nil, nil } diff --git a/etcdctl/ctlv3/command/lease_command.go b/etcdctl/ctlv3/command/lease_command.go index 0afb3d69c7c..26b9c6cb6a7 100644 --- a/etcdctl/ctlv3/command/lease_command.go +++ b/etcdctl/ctlv3/command/lease_command.go @@ -148,13 +148,12 @@ func leaseKeepAliveCommandFunc(cmd *cobra.Command, args []string) { } id := leaseFromArgs(args[0]) - respc, kerr := mustClientFromCmd(cmd).KeepAlive(context.TODO(), id) - if kerr != nil { - ExitWithError(ExitBadConnection, kerr) - } - + respc := mustClientFromCmd(cmd).KeepAlive(context.TODO(), id) for resp := range respc { - display.KeepAlive(*resp) + if resp.Err != nil { + ExitWithError(ExitError, resp.Err) + } + display.KeepAlive(resp) } if _, ok := (display).(*simplePrinter); ok { diff --git a/integration/cluster_proxy.go b/integration/cluster_proxy.go index 0152a16c67c..8593b5064f1 100644 --- a/integration/cluster_proxy.go +++ b/integration/cluster_proxy.go @@ -75,6 +75,7 @@ type proxyCloser struct { clientv3.Watcher wdonec <-chan struct{} kvdonec <-chan struct{} + lclose func() lpdonec <-chan struct{} } @@ -83,6 +84,7 @@ func (pc *proxyCloser) Close() error { <-pc.kvdonec err := pc.Watcher.Close() <-pc.wdonec + pc.lclose() <-pc.lpdonec return err } @@ -95,11 +97,13 @@ func newClientV3(cfg clientv3.Config) (*clientv3.Client, error) { rpc := toGRPC(c) c.KV = clientv3.NewKVFromKVClient(rpc.KV) pmu.Lock() + lc := c.Lease c.Lease = clientv3.NewLeaseFromLeaseClient(rpc.Lease, cfg.DialTimeout) c.Watcher = &proxyCloser{ Watcher: clientv3.NewWatchFromWatchClient(rpc.Watch), wdonec: proxies[c].wdonec, kvdonec: proxies[c].kvdonec, + lclose: func() { lc.Close() }, lpdonec: proxies[c].lpdonec, } pmu.Unlock() diff --git a/proxy/grpcproxy/lease.go b/proxy/grpcproxy/lease.go index dd23425a281..ba655d02d76 100644 --- a/proxy/grpcproxy/lease.go +++ b/proxy/grpcproxy/lease.go @@ -255,10 +255,7 @@ func (lps *leaseProxyStream) recvLoop() error { func (lps *leaseProxyStream) keepAliveLoop(leaseID int64, neededResps *atomicCounter) error { cctx, ccancel := context.WithCancel(lps.ctx) defer ccancel() - respc, err := lps.lessor.KeepAlive(cctx, clientv3.LeaseID(leaseID)) - if err != nil { - return err - } + respc := lps.lessor.KeepAlive(cctx, clientv3.LeaseID(leaseID)) // ticker expires when loop hasn't received keepalive within TTL var ticker <-chan time.Time for { @@ -276,7 +273,7 @@ func (lps *leaseProxyStream) keepAliveLoop(leaseID int64, neededResps *atomicCou lps.mu.Unlock() return nil case rp, ok := <-respc: - if !ok { + if !ok || rp.Err != nil { lps.mu.Lock() delete(lps.keepAliveLeases, leaseID) lps.mu.Unlock() diff --git a/tools/benchmark/cmd/lease.go b/tools/benchmark/cmd/lease.go index 8743ed27ead..bf59d075159 100644 --- a/tools/benchmark/cmd/lease.go +++ b/tools/benchmark/cmd/lease.go @@ -61,8 +61,8 @@ func leaseKeepaliveFunc(cmd *cobra.Command, args []string) { } for range requests { st := time.Now() - _, err := c.KeepAliveOnce(context.TODO(), resp.ID) - r.Results() <- report.Result{Err: err, Start: st, End: time.Now()} + ka := c.KeepAliveOnce(context.TODO(), resp.ID) + r.Results() <- report.Result{Err: ka.Err, Start: st, End: time.Now()} bar.Increment() } }(clients[i]) diff --git a/tools/functional-tester/etcd-runner/command/lease_renewer_command.go b/tools/functional-tester/etcd-runner/command/lease_renewer_command.go index e5257d4301b..070509747c9 100644 --- a/tools/functional-tester/etcd-runner/command/lease_renewer_command.go +++ b/tools/functional-tester/etcd-runner/command/lease_renewer_command.go @@ -50,7 +50,6 @@ func runLeaseRenewerFunc(cmd *cobra.Command, args []string) { for { var ( l *clientv3.LeaseGrantResponse - lk *clientv3.LeaseKeepAliveResponse err error ) for { @@ -62,13 +61,14 @@ func runLeaseRenewerFunc(cmd *cobra.Command, args []string) { expire := time.Now().Add(time.Duration(l.TTL-1) * time.Second) for { - lk, err = c.Lease.KeepAliveOnce(ctx, l.ID) + lk := c.Lease.KeepAliveOnce(ctx, l.ID) + err = lk.Err if grpc.Code(err) == codes.NotFound { if time.Since(expire) < 0 { log.Printf("bad renew! exceeded: %v", time.Since(expire)) for { - lk, err = c.Lease.KeepAliveOnce(ctx, l.ID) - fmt.Println(lk, err) + lk = c.Lease.KeepAliveOnce(ctx, l.ID) + fmt.Println(lk) time.Sleep(time.Second) } }