Merge pull request #7314 from heyitsanthony/fix-leadership

grpcproxy: split out / tighten up leadership detection
This commit is contained in:
Anthony Romano 2017-02-15 07:01:38 -08:00 committed by GitHub
commit 2f8b9ce9aa
4 changed files with 127 additions and 46 deletions

View File

@ -681,7 +681,9 @@ func TestWatchWithRequireLeader(t *testing.T) {
// wait for election timeout, then member[0] will not have a leader. // wait for election timeout, then member[0] will not have a leader.
tickDuration := 10 * time.Millisecond tickDuration := 10 * time.Millisecond
time.Sleep(time.Duration(3*clus.Members[0].ElectionTicks) * tickDuration) // existing streams need three elections before they're torn down; wait until 5 elections cycle
// so proxy tests receive a leader loss event on its existing watch before creating a new watch.
time.Sleep(time.Duration(5*clus.Members[0].ElectionTicks) * tickDuration)
chLeader := liveClient.Watch(clientv3.WithRequireLeader(context.TODO()), "foo", clientv3.WithRev(1)) chLeader := liveClient.Watch(clientv3.WithRequireLeader(context.TODO()), "foo", clientv3.WithRev(1))
chNoLeader := liveClient.Watch(context.TODO(), "foo", clientv3.WithRev(1)) chNoLeader := liveClient.Watch(context.TODO(), "foo", clientv3.WithRev(1))

View File

@ -33,11 +33,7 @@ func NewKvProxy(c *clientv3.Client) (pb.KVServer, <-chan struct{}) {
cache: cache.NewCache(cache.DefaultMaxEntries), cache: cache.NewCache(cache.DefaultMaxEntries),
} }
donec := make(chan struct{}) donec := make(chan struct{})
go func() { close(donec)
defer close(donec)
<-c.Ctx().Done()
kv.cache.Close()
}()
return kv, donec return kv, donec
} }

94
proxy/grpcproxy/leader.go Normal file
View File

@ -0,0 +1,94 @@
// 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 grpcproxy
import (
"math"
"sync"
"golang.org/x/net/context"
"golang.org/x/time/rate"
"google.golang.org/grpc"
"github.com/coreos/etcd/clientv3"
)
const (
lostLeaderKey = "__lostleader" // watched to detect leader loss
retryPerSecond = 10
)
type leader struct {
ctx context.Context
w clientv3.Watcher
// mu protects leaderc updates.
mu sync.RWMutex
leaderc chan struct{}
disconnc chan struct{}
donec chan struct{}
}
func newLeader(ctx context.Context, w clientv3.Watcher) *leader {
l := &leader{
ctx: clientv3.WithRequireLeader(ctx),
w: w,
leaderc: make(chan struct{}),
disconnc: make(chan struct{}),
donec: make(chan struct{}),
}
// begin assuming leader is lost
close(l.leaderc)
go l.recvLoop()
return l
}
func (l *leader) recvLoop() {
defer close(l.donec)
limiter := rate.NewLimiter(rate.Limit(retryPerSecond), retryPerSecond)
rev := int64(math.MaxInt64 - 2)
for limiter.Wait(l.ctx) == nil {
wch := l.w.Watch(l.ctx, lostLeaderKey, clientv3.WithRev(rev), clientv3.WithCreatedNotify())
cresp, ok := <-wch
if !ok {
continue
}
if cresp.Err() != nil {
if grpc.ErrorDesc(cresp.Err()) == grpc.ErrClientConnClosing.Error() {
close(l.disconnc)
return
}
continue
}
// leader is available
l.mu.Lock()
l.leaderc = make(chan struct{})
l.mu.Unlock()
<-wch
close(l.leaderc)
}
}
func (l *leader) disconnectNotify() <-chan struct{} { return l.disconnc }
func (l *leader) stopNotify() <-chan struct{} { return l.donec }
// lostNotify returns a channel that is closed if there has been
// a leader loss not yet followed by a leader reacquire.
func (l *leader) lostNotify() <-chan struct{} {
l.mu.RLock()
defer l.mu.RUnlock()
return l.leaderc
}

View File

@ -18,7 +18,7 @@ import (
"sync" "sync"
"golang.org/x/net/context" "golang.org/x/net/context"
"golang.org/x/time/rate" "google.golang.org/grpc"
"google.golang.org/grpc/metadata" "google.golang.org/grpc/metadata"
"github.com/coreos/etcd/clientv3" "github.com/coreos/etcd/clientv3"
@ -31,50 +31,35 @@ type watchProxy struct {
cw clientv3.Watcher cw clientv3.Watcher
ctx context.Context ctx context.Context
leader *leader
ranges *watchRanges ranges *watchRanges
// retryLimiter controls the create watch retry rate on lost leaders. // mu protects adding outstanding watch servers through wg.
retryLimiter *rate.Limiter mu sync.Mutex
// mu protects leaderc updates.
mu sync.RWMutex
leaderc chan struct{}
// wg waits until all outstanding watch servers quit. // wg waits until all outstanding watch servers quit.
wg sync.WaitGroup wg sync.WaitGroup
} }
const (
lostLeaderKey = "__lostleader" // watched to detect leader loss
retryPerSecond = 10
)
func NewWatchProxy(c *clientv3.Client) (pb.WatchServer, <-chan struct{}) { func NewWatchProxy(c *clientv3.Client) (pb.WatchServer, <-chan struct{}) {
cctx, cancel := context.WithCancel(c.Ctx())
wp := &watchProxy{ wp := &watchProxy{
cw: c.Watcher, cw: c.Watcher,
ctx: c.Ctx(), ctx: cctx,
retryLimiter: rate.NewLimiter(rate.Limit(retryPerSecond), retryPerSecond), leader: newLeader(c.Ctx(), c.Watcher),
leaderc: make(chan struct{}),
} }
wp.ranges = newWatchRanges(wp) wp.ranges = newWatchRanges(wp)
ch := make(chan struct{}) ch := make(chan struct{})
go func() { go func() {
defer close(ch) defer close(ch)
// a new streams without opening any watchers won't catch <-wp.leader.stopNotify()
// a lost leader event, so have a special watch to monitor it
rev := int64((uint64(1) << 63) - 2)
lctx := clientv3.WithRequireLeader(wp.ctx)
for wp.ctx.Err() == nil {
wch := wp.cw.Watch(lctx, lostLeaderKey, clientv3.WithRev(rev))
for range wch {
}
wp.mu.Lock()
close(wp.leaderc)
wp.leaderc = make(chan struct{})
wp.mu.Unlock()
wp.retryLimiter.Wait(wp.ctx)
}
wp.mu.Lock() wp.mu.Lock()
select {
case <-wp.ctx.Done():
case <-wp.leader.disconnectNotify():
cancel()
}
<-wp.ctx.Done() <-wp.ctx.Done()
wp.mu.Unlock() wp.mu.Unlock()
wp.wg.Wait() wp.wg.Wait()
@ -104,11 +89,19 @@ func (wp *watchProxy) Watch(stream pb.Watch_WatchServer) (err error) {
cancel: cancel, cancel: cancel,
} }
var leaderc <-chan struct{} var lostLeaderC <-chan struct{}
if md, ok := metadata.FromContext(stream.Context()); ok { if md, ok := metadata.FromContext(stream.Context()); ok {
v := md[rpctypes.MetadataRequireLeaderKey] v := md[rpctypes.MetadataRequireLeaderKey]
if len(v) > 0 && v[0] == rpctypes.MetadataHasLeader { if len(v) > 0 && v[0] == rpctypes.MetadataHasLeader {
leaderc = wp.lostLeaderNotify() lostLeaderC = wp.leader.lostNotify()
// if leader is known to be lost at creation time, avoid
// letting events through at all
select {
case <-lostLeaderC:
wp.wg.Done()
return rpctypes.ErrNoLeader
default:
}
} }
} }
@ -127,7 +120,7 @@ func (wp *watchProxy) Watch(stream pb.Watch_WatchServer) (err error) {
go func() { go func() {
defer func() { stopc <- struct{}{} }() defer func() { stopc <- struct{}{} }()
select { select {
case <-leaderc: case <-lostLeaderC:
case <-ctx.Done(): case <-ctx.Done():
case <-wp.ctx.Done(): case <-wp.ctx.Done():
} }
@ -146,19 +139,15 @@ func (wp *watchProxy) Watch(stream pb.Watch_WatchServer) (err error) {
}() }()
select { select {
case <-leaderc: case <-lostLeaderC:
return rpctypes.ErrNoLeader return rpctypes.ErrNoLeader
case <-wp.leader.disconnectNotify():
return grpc.ErrClientConnClosing
default: default:
return wps.ctx.Err() return wps.ctx.Err()
} }
} }
func (wp *watchProxy) lostLeaderNotify() <-chan struct{} {
wp.mu.RLock()
defer wp.mu.RUnlock()
return wp.leaderc
}
// watchProxyStream forwards etcd watch events to a proxied client stream. // watchProxyStream forwards etcd watch events to a proxied client stream.
type watchProxyStream struct { type watchProxyStream struct {
ranges *watchRanges ranges *watchRanges