Etcd watcher verification
To make sure the etcd watcher works, I changed the replication controller to use watch.Interface. I made apiserver support watches on controllers, so replicationController can be run only off of the apiserver. I made sure all the etcd watch testing that used to be in replicationController is now tested on the new etcd watcher in pkg/tools/.
This commit is contained in:
@@ -18,6 +18,7 @@ package tools
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"io"
|
||||
|
||||
"github.com/GoogleCloudPlatform/kubernetes/pkg/api"
|
||||
@@ -44,7 +45,14 @@ func NewAPIEventDecoder(stream io.ReadCloser) *APIEventDecoder {
|
||||
func (d *APIEventDecoder) Decode() (action watch.EventType, object interface{}, err error) {
|
||||
var got api.WatchEvent
|
||||
err = d.decoder.Decode(&got)
|
||||
return got.Type, got.Object.Object, err
|
||||
if err != nil {
|
||||
return action, nil, err
|
||||
}
|
||||
switch got.Type {
|
||||
case watch.Added, watch.Modified, watch.Deleted:
|
||||
return got.Type, got.Object.Object, err
|
||||
}
|
||||
return action, nil, fmt.Errorf("got invalid watch event type: %v", got.Type)
|
||||
}
|
||||
|
||||
// Close closes the underlying stream.
|
||||
|
@@ -19,6 +19,7 @@ package tools
|
||||
import (
|
||||
"fmt"
|
||||
"reflect"
|
||||
"sync"
|
||||
|
||||
"github.com/GoogleCloudPlatform/kubernetes/pkg/api"
|
||||
"github.com/GoogleCloudPlatform/kubernetes/pkg/util"
|
||||
@@ -267,23 +268,31 @@ type etcdWatcher struct {
|
||||
list bool // If we're doing a recursive watch, should be true.
|
||||
filter FilterFunc
|
||||
|
||||
etcdIncoming chan *etcd.Response
|
||||
etcdStop chan bool
|
||||
etcdIncoming chan *etcd.Response
|
||||
etcdStop chan bool
|
||||
etcdCallEnded chan struct{}
|
||||
|
||||
outgoing chan watch.Event
|
||||
userStop chan struct{}
|
||||
stopped bool
|
||||
stopLock sync.Mutex
|
||||
|
||||
// Injectable for testing. Send the event down the outgoing channel.
|
||||
emit func(watch.Event)
|
||||
}
|
||||
|
||||
// Returns a new etcdWatcher; if list is true, watch sub-nodes.
|
||||
func newEtcdWatcher(list bool, filter FilterFunc) *etcdWatcher {
|
||||
w := &etcdWatcher{
|
||||
list: list,
|
||||
filter: filter,
|
||||
etcdIncoming: make(chan *etcd.Response),
|
||||
etcdStop: make(chan bool),
|
||||
outgoing: make(chan watch.Event),
|
||||
userStop: make(chan struct{}),
|
||||
list: list,
|
||||
filter: filter,
|
||||
etcdIncoming: make(chan *etcd.Response),
|
||||
etcdStop: make(chan bool),
|
||||
etcdCallEnded: make(chan struct{}),
|
||||
outgoing: make(chan watch.Event),
|
||||
userStop: make(chan struct{}),
|
||||
}
|
||||
w.emit = func(e watch.Event) { w.outgoing <- e }
|
||||
go w.translate()
|
||||
return w
|
||||
}
|
||||
@@ -292,11 +301,9 @@ func newEtcdWatcher(list bool, filter FilterFunc) *etcdWatcher {
|
||||
// as a goroutine.
|
||||
func (w *etcdWatcher) etcdWatch(client EtcdGetSet, key string) {
|
||||
defer util.HandleCrash()
|
||||
defer close(w.etcdCallEnded)
|
||||
_, err := client.Watch(key, 0, w.list, w.etcdIncoming, w.etcdStop)
|
||||
if err == etcd.ErrWatchStoppedByUser {
|
||||
// etcd doesn't close the channel in this case.
|
||||
close(w.etcdIncoming)
|
||||
} else {
|
||||
if err != etcd.ErrWatchStoppedByUser {
|
||||
glog.Errorf("etcd.Watch stopped unexpectedly: %v (%#v)", err, err)
|
||||
}
|
||||
}
|
||||
@@ -309,6 +316,8 @@ func (w *etcdWatcher) translate() {
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-w.etcdCallEnded:
|
||||
return
|
||||
case <-w.userStop:
|
||||
w.etcdStop <- true
|
||||
return
|
||||
@@ -324,7 +333,6 @@ func (w *etcdWatcher) translate() {
|
||||
func (w *etcdWatcher) sendResult(res *etcd.Response) {
|
||||
var action watch.EventType
|
||||
var data []byte
|
||||
var nodes etcd.Nodes
|
||||
switch res.Action {
|
||||
case "set":
|
||||
if res.Node == nil {
|
||||
@@ -332,7 +340,6 @@ func (w *etcdWatcher) sendResult(res *etcd.Response) {
|
||||
return
|
||||
}
|
||||
data = []byte(res.Node.Value)
|
||||
nodes = res.Node.Nodes
|
||||
// TODO: Is this conditional correct?
|
||||
if res.EtcdIndex > 0 {
|
||||
action = watch.Modified
|
||||
@@ -345,38 +352,23 @@ func (w *etcdWatcher) sendResult(res *etcd.Response) {
|
||||
return
|
||||
}
|
||||
data = []byte(res.PrevNode.Value)
|
||||
nodes = res.PrevNode.Nodes
|
||||
action = watch.Deleted
|
||||
}
|
||||
|
||||
// If listing, we're interested in sub-nodes.
|
||||
if w.list {
|
||||
for _, n := range nodes {
|
||||
obj, err := api.Decode([]byte(n.Value))
|
||||
if err != nil {
|
||||
glog.Errorf("failure to decode api object: %#v", res)
|
||||
continue
|
||||
}
|
||||
if w.filter != nil && !w.filter(obj) {
|
||||
continue
|
||||
}
|
||||
w.outgoing <- watch.Event{
|
||||
Type: action,
|
||||
Object: obj,
|
||||
}
|
||||
}
|
||||
default:
|
||||
glog.Errorf("unknown action: %v", res.Action)
|
||||
return
|
||||
}
|
||||
|
||||
obj, err := api.Decode(data)
|
||||
if err != nil {
|
||||
glog.Errorf("failure to decode api object: %#v", res)
|
||||
glog.Errorf("failure to decode api object: '%v' from %#v", string(data), res)
|
||||
// TODO: expose an error through watch.Interface?
|
||||
w.Stop()
|
||||
return
|
||||
}
|
||||
w.outgoing <- watch.Event{
|
||||
w.emit(watch.Event{
|
||||
Type: action,
|
||||
Object: obj,
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
// ResultChannel implements watch.Interface.
|
||||
@@ -386,5 +378,11 @@ func (w *etcdWatcher) ResultChan() <-chan watch.Event {
|
||||
|
||||
// Stop implements watch.Interface.
|
||||
func (w *etcdWatcher) Stop() {
|
||||
close(w.userStop)
|
||||
w.stopLock.Lock()
|
||||
defer w.stopLock.Unlock()
|
||||
// Prevent double channel closes.
|
||||
if !w.stopped {
|
||||
w.stopped = true
|
||||
close(w.userStop)
|
||||
}
|
||||
}
|
||||
|
@@ -220,71 +220,7 @@ func TestAtomicUpdate(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestWatchInterpretation_ListAdd(t *testing.T) {
|
||||
called := false
|
||||
w := newEtcdWatcher(true, func(interface{}) bool {
|
||||
called = true
|
||||
return true
|
||||
})
|
||||
pod := &api.Pod{JSONBase: api.JSONBase{ID: "foo"}}
|
||||
podBytes, _ := api.Encode(pod)
|
||||
|
||||
go w.sendResult(&etcd.Response{
|
||||
Action: "set",
|
||||
Node: &etcd.Node{
|
||||
Nodes: etcd.Nodes{
|
||||
{
|
||||
Value: string(podBytes),
|
||||
},
|
||||
},
|
||||
},
|
||||
})
|
||||
|
||||
got := <-w.outgoing
|
||||
if e, a := watch.Added, got.Type; e != a {
|
||||
t.Errorf("Expected %v, got %v", e, a)
|
||||
}
|
||||
if e, a := pod, got.Object; !reflect.DeepEqual(e, a) {
|
||||
t.Errorf("Expected %v, got %v", e, a)
|
||||
}
|
||||
if !called {
|
||||
t.Errorf("filter never called")
|
||||
}
|
||||
}
|
||||
|
||||
func TestWatchInterpretation_ListDelete(t *testing.T) {
|
||||
called := false
|
||||
w := newEtcdWatcher(true, func(interface{}) bool {
|
||||
called = true
|
||||
return true
|
||||
})
|
||||
pod := &api.Pod{JSONBase: api.JSONBase{ID: "foo"}}
|
||||
podBytes, _ := api.Encode(pod)
|
||||
|
||||
go w.sendResult(&etcd.Response{
|
||||
Action: "delete",
|
||||
PrevNode: &etcd.Node{
|
||||
Nodes: etcd.Nodes{
|
||||
{
|
||||
Value: string(podBytes),
|
||||
},
|
||||
},
|
||||
},
|
||||
})
|
||||
|
||||
got := <-w.outgoing
|
||||
if e, a := watch.Deleted, got.Type; e != a {
|
||||
t.Errorf("Expected %v, got %v", e, a)
|
||||
}
|
||||
if e, a := pod, got.Object; !reflect.DeepEqual(e, a) {
|
||||
t.Errorf("Expected %v, got %v", e, a)
|
||||
}
|
||||
if !called {
|
||||
t.Errorf("filter never called")
|
||||
}
|
||||
}
|
||||
|
||||
func TestWatchInterpretation_SingleAdd(t *testing.T) {
|
||||
w := newEtcdWatcher(false, func(interface{}) bool {
|
||||
t.Errorf("unexpected filter call")
|
||||
return true
|
||||
})
|
||||
@@ -307,8 +243,8 @@ func TestWatchInterpretation_SingleAdd(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestWatchInterpretation_SingleDelete(t *testing.T) {
|
||||
w := newEtcdWatcher(false, func(interface{}) bool {
|
||||
func TestWatchInterpretation_Delete(t *testing.T) {
|
||||
w := newEtcdWatcher(true, func(interface{}) bool {
|
||||
t.Errorf("unexpected filter call")
|
||||
return true
|
||||
})
|
||||
@@ -331,6 +267,49 @@ func TestWatchInterpretation_SingleDelete(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestWatchInterpretation_ResponseNotSet(t *testing.T) {
|
||||
w := newEtcdWatcher(false, func(interface{}) bool {
|
||||
t.Errorf("unexpected filter call")
|
||||
return true
|
||||
})
|
||||
w.emit = func(e watch.Event) {
|
||||
t.Errorf("Unexpected emit: %v", e)
|
||||
}
|
||||
|
||||
w.sendResult(&etcd.Response{
|
||||
Action: "update",
|
||||
})
|
||||
}
|
||||
|
||||
func TestWatchInterpretation_ResponseNoNode(t *testing.T) {
|
||||
w := newEtcdWatcher(false, func(interface{}) bool {
|
||||
t.Errorf("unexpected filter call")
|
||||
return true
|
||||
})
|
||||
w.emit = func(e watch.Event) {
|
||||
t.Errorf("Unexpected emit: %v", e)
|
||||
}
|
||||
w.sendResult(&etcd.Response{
|
||||
Action: "set",
|
||||
})
|
||||
}
|
||||
|
||||
func TestWatchInterpretation_ResponseBadData(t *testing.T) {
|
||||
w := newEtcdWatcher(false, func(interface{}) bool {
|
||||
t.Errorf("unexpected filter call")
|
||||
return true
|
||||
})
|
||||
w.emit = func(e watch.Event) {
|
||||
t.Errorf("Unexpected emit: %v", e)
|
||||
}
|
||||
w.sendResult(&etcd.Response{
|
||||
Action: "set",
|
||||
Node: &etcd.Node{
|
||||
Value: "foobar",
|
||||
},
|
||||
})
|
||||
}
|
||||
|
||||
func TestWatch(t *testing.T) {
|
||||
fakeEtcd := MakeFakeEtcdClient(t)
|
||||
h := EtcdHelper{fakeEtcd}
|
||||
|
@@ -215,6 +215,9 @@ func (f *FakeEtcdClient) Watch(prefix string, waitIndex uint64, recursive bool,
|
||||
|
||||
if receiver == nil {
|
||||
return f.Get(prefix, false, recursive)
|
||||
} else {
|
||||
// Emulate etcd's behavior. (I think.)
|
||||
defer close(receiver)
|
||||
}
|
||||
|
||||
f.watchCompletedChan <- true
|
||||
@@ -222,8 +225,6 @@ func (f *FakeEtcdClient) Watch(prefix string, waitIndex uint64, recursive bool,
|
||||
case <-stop:
|
||||
return nil, etcd.ErrWatchStoppedByUser
|
||||
case err := <-injectedError:
|
||||
// Emulate etcd's behavior.
|
||||
close(receiver)
|
||||
return nil, err
|
||||
}
|
||||
// Never get here.
|
||||
|
Reference in New Issue
Block a user