clientv3: synchronous lease Close

This commit is contained in:
Anthony Romano 2016-02-01 22:06:03 -08:00
parent 24f5640d83
commit eb8ab3ace4

View File

@ -57,9 +57,11 @@ type Lease interface {
type lessor struct { type lessor struct {
c *Client c *Client
mu sync.Mutex // guards all fields mu sync.Mutex // guards all fields
conn *grpc.ClientConn // conn in-use conn *grpc.ClientConn // conn in-use
initedc chan bool
// donec is closed when recvKeepAliveLoop stops
donec chan struct{}
remote pb.LeaseClient remote pb.LeaseClient
@ -78,8 +80,7 @@ func NewLease(c *Client) Lease {
c: c, c: c,
conn: c.ActiveConnection(), conn: c.ActiveConnection(),
initedc: make(chan bool, 1), donec: make(chan struct{}),
keepAlives: make(map[lease.LeaseID]chan *LeaseKeepAliveResponse), keepAlives: make(map[lease.LeaseID]chan *LeaseKeepAliveResponse),
deadlines: make(map[lease.LeaseID]time.Time), deadlines: make(map[lease.LeaseID]time.Time),
} }
@ -87,10 +88,7 @@ func NewLease(c *Client) Lease {
l.remote = pb.NewLeaseClient(l.conn) l.remote = pb.NewLeaseClient(l.conn)
l.stopCtx, l.stopCancel = context.WithCancel(context.Background()) l.stopCtx, l.stopCancel = context.WithCancel(context.Background())
l.initedc <- false
go l.recvKeepAliveLoop() go l.recvKeepAliveLoop()
go l.sendKeepAliveLoop()
return l return l
} }
@ -181,11 +179,8 @@ func (l *lessor) KeepAliveOnce(ctx context.Context, id lease.LeaseID) (*LeaseKee
} }
func (l *lessor) Close() error { func (l *lessor) Close() error {
l.mu.Lock()
defer l.mu.Unlock()
l.stopCancel() l.stopCancel()
l.stream = nil <-l.donec
return nil return nil
} }
@ -208,56 +203,66 @@ func (l *lessor) keepAliveOnce(ctx context.Context, id lease.LeaseID) (*LeaseKee
} }
func (l *lessor) recvKeepAliveLoop() { func (l *lessor) recvKeepAliveLoop() {
if !l.initStream() { defer func() {
l.Close() l.stopCancel()
return close(l.donec)
} }()
stream, serr := l.resetRecv()
for { for {
stream := l.getKeepAliveStream()
resp, err := stream.Recv() resp, err := stream.Recv()
if err != nil { if err != nil {
err = l.switchRemoteAndStream(err) if stream, serr = l.resetRecv(); serr != nil {
if err != nil {
l.Close()
return return
} }
continue continue
} }
l.recvKeepAlive(resp)
l.mu.Lock()
lch, ok := l.keepAlives[lease.LeaseID(resp.ID)]
if !ok {
l.mu.Unlock()
continue
}
if resp.TTL <= 0 {
close(lch)
delete(l.deadlines, lease.LeaseID(resp.ID))
delete(l.keepAlives, lease.LeaseID(resp.ID))
} else {
select {
case lch <- (*LeaseKeepAliveResponse)(resp):
l.deadlines[lease.LeaseID(resp.ID)] =
time.Now().Add(1 + time.Duration(resp.TTL/3)*time.Second)
default:
}
}
l.mu.Unlock()
} }
} }
func (l *lessor) sendKeepAliveLoop() { // resetRecv opens a new lease stream and starts sending LeaseKeepAliveRequests
if !l.initStream() { func (l *lessor) resetRecv() (pb.Lease_LeaseKeepAliveClient, error) {
l.Close() if err := l.switchRemoteAndStream(nil); err != nil {
return nil, err
}
stream := l.getKeepAliveStream()
go l.sendKeepAliveLoop(stream)
return stream, nil
}
// recvKeepAlive updates a lease based on its LeaseKeepAliveResponse
func (l *lessor) recvKeepAlive(resp *pb.LeaseKeepAliveResponse) {
l.mu.Lock()
defer l.mu.Unlock()
lch, ok := l.keepAlives[lease.LeaseID(resp.ID)]
if !ok {
return return
} }
if resp.TTL <= 0 {
close(lch)
delete(l.deadlines, lease.LeaseID(resp.ID))
delete(l.keepAlives, lease.LeaseID(resp.ID))
return
}
select {
case lch <- (*LeaseKeepAliveResponse)(resp):
l.deadlines[lease.LeaseID(resp.ID)] =
time.Now().Add(1 + time.Duration(resp.TTL/3)*time.Second)
default:
}
}
// sendKeepAliveLoop sends LeaseKeepAliveRequests for the lifetime of a lease stream
func (l *lessor) sendKeepAliveLoop(stream pb.Lease_LeaseKeepAliveClient) {
for { for {
select { select {
case <-time.After(500 * time.Millisecond): case <-time.After(500 * time.Millisecond):
case <-l.donec:
return
case <-l.stopCtx.Done(): case <-l.stopCtx.Done():
return return
} }
@ -273,21 +278,10 @@ func (l *lessor) sendKeepAliveLoop() {
} }
l.mu.Unlock() l.mu.Unlock()
stream := l.getKeepAliveStream()
var err error
for _, id := range tosend { for _, id := range tosend {
r := &pb.LeaseKeepAliveRequest{ID: int64(id)} r := &pb.LeaseKeepAliveRequest{ID: int64(id)}
err = stream.Send(r) if err := stream.Send(r); err != nil {
if err != nil { // TODO do something with this error?
break
}
}
if err != nil {
err = l.switchRemoteAndStream(err)
if err != nil {
l.Close()
return return
} }
} }
@ -359,21 +353,6 @@ func (l *lessor) newStream() error {
return nil return nil
} }
func (l *lessor) initStream() bool {
ok := <-l.initedc
if ok {
return true
}
err := l.switchRemoteAndStream(nil)
if err == nil {
l.initedc <- true
return true
}
l.initedc <- false
return false
}
// cancelWhenStop calls cancel when the given stopc fires. It returns a done chan. done // cancelWhenStop calls cancel when the given stopc fires. It returns a done chan. done
// should be closed when the work is finished. When done fires, cancelWhenStop will release // should be closed when the work is finished. When done fires, cancelWhenStop will release
// its internal resource. // its internal resource.