mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
Merge pull request #7314 from heyitsanthony/fix-leadership
grpcproxy: split out / tighten up leadership detection
This commit is contained in:
commit
2f8b9ce9aa
@ -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))
|
||||||
|
@ -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
94
proxy/grpcproxy/leader.go
Normal 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
|
||||||
|
}
|
@ -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
|
||||||
|
Loading…
x
Reference in New Issue
Block a user