mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
raftest: wait for network sending
This commit is contained in:
parent
0242faa838
commit
279b216f9a
@ -86,7 +86,11 @@ func (rn *raftNetwork) send(m raftpb.Message) {
|
|||||||
time.Sleep(time.Duration(rd))
|
time.Sleep(time.Duration(rd))
|
||||||
}
|
}
|
||||||
|
|
||||||
to <- m
|
select {
|
||||||
|
case to <- m:
|
||||||
|
default:
|
||||||
|
// drop messages when the receiver queue is full.
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (rn *raftNetwork) recvFrom(from uint64) chan raftpb.Message {
|
func (rn *raftNetwork) recvFrom(from uint64) chan raftpb.Message {
|
||||||
|
@ -49,11 +49,10 @@ func (n *node) start() {
|
|||||||
n.storage.SetHardState(n.state)
|
n.storage.SetHardState(n.state)
|
||||||
}
|
}
|
||||||
n.storage.Append(rd.Entries)
|
n.storage.Append(rd.Entries)
|
||||||
go func() {
|
// TODO: make send async, more like real world...
|
||||||
for _, m := range rd.Messages {
|
for _, m := range rd.Messages {
|
||||||
n.iface.send(m)
|
n.iface.send(m)
|
||||||
}
|
}
|
||||||
}()
|
|
||||||
n.Advance()
|
n.Advance()
|
||||||
case m := <-n.iface.recv():
|
case m := <-n.iface.recv():
|
||||||
n.Step(context.TODO(), m)
|
n.Step(context.TODO(), m)
|
||||||
|
Loading…
x
Reference in New Issue
Block a user