mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
rafttest: support node pause
This commit is contained in:
parent
3c9c4c4afa
commit
085b608de9
@ -15,6 +15,7 @@ type node struct {
|
||||
paused bool
|
||||
iface iface
|
||||
stopc chan struct{}
|
||||
pausec chan bool
|
||||
|
||||
// stable
|
||||
storage *raft.MemoryStorage
|
||||
@ -29,6 +30,7 @@ func startNode(id uint64, peers []raft.Peer, iface iface) *node {
|
||||
id: id,
|
||||
storage: st,
|
||||
iface: iface,
|
||||
pausec: make(chan bool),
|
||||
}
|
||||
n.start()
|
||||
return n
|
||||
@ -62,6 +64,22 @@ func (n *node) start() {
|
||||
n.Node = nil
|
||||
close(n.stopc)
|
||||
return
|
||||
case p := <-n.pausec:
|
||||
recvms := make([]raftpb.Message, 0)
|
||||
for p {
|
||||
// TODO: locking around paused?
|
||||
n.paused = true
|
||||
select {
|
||||
case m := <-n.iface.recv():
|
||||
recvms = append(recvms, m)
|
||||
case p = <-n.pausec:
|
||||
}
|
||||
}
|
||||
n.paused = false
|
||||
// step all pending messages
|
||||
for _, m := range recvms {
|
||||
n.Step(context.TODO(), m)
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
@ -91,12 +109,12 @@ func (n *node) restart() {
|
||||
// The paused node buffers the received messages and replies
|
||||
// all of them when it resumes.
|
||||
func (n *node) pause() {
|
||||
panic("unimplemented")
|
||||
n.pausec <- true
|
||||
}
|
||||
|
||||
// resume resumes the paused node.
|
||||
func (n *node) resume() {
|
||||
panic("unimplemented")
|
||||
n.pausec <- false
|
||||
}
|
||||
|
||||
func (n *node) isPaused() bool {
|
||||
|
@ -71,3 +71,42 @@ func TestRestart(t *testing.T) {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestPause(t *testing.T) {
|
||||
peers := []raft.Peer{{1, nil}, {2, nil}, {3, nil}, {4, nil}, {5, nil}}
|
||||
nt := newRaftNetwork(1, 2, 3, 4, 5)
|
||||
|
||||
nodes := make([]*node, 0)
|
||||
|
||||
for i := 1; i <= 5; i++ {
|
||||
n := startNode(uint64(i), peers, nt.nodeNetwork(uint64(i)))
|
||||
nodes = append(nodes, n)
|
||||
}
|
||||
|
||||
time.Sleep(50 * time.Millisecond)
|
||||
for i := 0; i < 300; i++ {
|
||||
nodes[0].Propose(context.TODO(), []byte("somedata"))
|
||||
}
|
||||
nodes[1].pause()
|
||||
for i := 0; i < 300; i++ {
|
||||
nodes[0].Propose(context.TODO(), []byte("somedata"))
|
||||
}
|
||||
nodes[2].pause()
|
||||
for i := 0; i < 300; i++ {
|
||||
nodes[0].Propose(context.TODO(), []byte("somedata"))
|
||||
}
|
||||
nodes[2].resume()
|
||||
for i := 0; i < 300; i++ {
|
||||
nodes[0].Propose(context.TODO(), []byte("somedata"))
|
||||
}
|
||||
nodes[1].resume()
|
||||
|
||||
// give some time for nodes to catch up with the raft leader
|
||||
time.Sleep(300 * time.Millisecond)
|
||||
for _, n := range nodes {
|
||||
n.stop()
|
||||
if n.state.Commit != 1206 {
|
||||
t.Errorf("commit = %d, want = 1206", n.state.Commit)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user