rework peer tracking logic to handle multiple connections

This commit is contained in:
vyzo 2018-12-13 16:58:32 +02:00
parent c7528f3c99
commit 2621f893e6
4 changed files with 83 additions and 32 deletions

27
comm.go
View File

@ -10,6 +10,7 @@ import (
ggio "github.com/gogo/protobuf/io" ggio "github.com/gogo/protobuf/io"
proto "github.com/gogo/protobuf/proto" proto "github.com/gogo/protobuf/proto"
inet "github.com/libp2p/go-libp2p-net" inet "github.com/libp2p/go-libp2p-net"
peer "github.com/libp2p/go-libp2p-peer"
) )
// get the initial RPC containing all of our subscriptions to send to new peers // get the initial RPC containing all of our subscriptions to send to new peers
@ -39,10 +40,6 @@ func (p *PubSub) handleNewStream(s inet.Stream) {
// but it doesn't hurt to send it. // but it doesn't hurt to send it.
s.Close() s.Close()
} }
select {
case p.peerDead <- s.Conn().RemotePeer():
case <-p.ctx.Done():
}
return return
} }
@ -57,6 +54,24 @@ func (p *PubSub) handleNewStream(s inet.Stream) {
} }
} }
func (p *PubSub) handleNewPeer(ctx context.Context, pid peer.ID, outgoing <-chan *RPC) {
s, err := p.host.NewStream(p.ctx, pid, p.rt.Protocols()...)
if err != nil {
log.Warning("opening new stream to peer: ", err, pid)
select {
case p.newPeerError <- pid:
case <-ctx.Done():
}
return
}
go p.handleSendingMessages(ctx, s, outgoing)
select {
case p.newPeerStream <- s:
case <-ctx.Done():
}
}
func (p *PubSub) handleSendingMessages(ctx context.Context, s inet.Stream, outgoing <-chan *RPC) { func (p *PubSub) handleSendingMessages(ctx context.Context, s inet.Stream, outgoing <-chan *RPC) {
bufw := bufio.NewWriter(s) bufw := bufio.NewWriter(s)
wc := ggio.NewDelimitedWriter(bufw) wc := ggio.NewDelimitedWriter(bufw)
@ -82,10 +97,6 @@ func (p *PubSub) handleSendingMessages(ctx context.Context, s inet.Stream, outgo
if err != nil { if err != nil {
s.Reset() s.Reset()
log.Infof("writing message to %s: %s", s.Conn().RemotePeer(), err) log.Infof("writing message to %s: %s", s.Conn().RemotePeer(), err)
select {
case p.peerDead <- s.Conn().RemotePeer():
case <-ctx.Done():
}
return return
} }
case <-ctx.Done(): case <-ctx.Done():

View File

@ -874,9 +874,7 @@ func TestPeerDisconnect(t *testing.T) {
peers := psubs[0].ListPeers("foo") peers := psubs[0].ListPeers("foo")
assertPeerList(t, peers, hosts[1].ID()) assertPeerList(t, peers, hosts[1].ID())
for _, c := range hosts[1].Network().ConnsToPeer(hosts[0].ID()) { for _, c := range hosts[1].Network().ConnsToPeer(hosts[0].ID()) {
for _, s := range c.GetStreams() { c.Close()
s.Close()
}
} }
time.Sleep(time.Millisecond * 10) time.Sleep(time.Millisecond * 10)

View File

@ -17,21 +17,20 @@ func (p *PubSubNotif) ClosedStream(n inet.Network, s inet.Stream) {
func (p *PubSubNotif) Connected(n inet.Network, c inet.Conn) { func (p *PubSubNotif) Connected(n inet.Network, c inet.Conn) {
go func() { go func() {
s, err := p.host.NewStream(p.ctx, c.RemotePeer(), p.rt.Protocols()...)
if err != nil {
log.Warning("opening new stream to peer: ", err, c.LocalPeer(), c.RemotePeer())
return
}
select { select {
case p.newPeers <- s: case p.newPeers <- c.RemotePeer():
case <-p.ctx.Done(): case <-p.ctx.Done():
s.Reset()
} }
}() }()
} }
func (p *PubSubNotif) Disconnected(n inet.Network, c inet.Conn) { func (p *PubSubNotif) Disconnected(n inet.Network, c inet.Conn) {
go func() {
select {
case p.peerDead <- c.RemotePeer():
case <-p.ctx.Done():
}
}()
} }
func (p *PubSubNotif) Listen(n inet.Network, _ ma.Multiaddr) { func (p *PubSubNotif) Listen(n inet.Network, _ ma.Multiaddr) {

View File

@ -57,8 +57,14 @@ type PubSub struct {
// send subscription here to cancel it // send subscription here to cancel it
cancelCh chan *Subscription cancelCh chan *Subscription
// a notification channel for incoming streams from other peers // a notification channel for new peer connections
newPeers chan inet.Stream newPeers chan peer.ID
// a notification channel for new outoging peer streams
newPeerStream chan inet.Stream
// a notification channel for errors opening new peer streams
newPeerError chan peer.ID
// a notification channel for when our peers die // a notification channel for when our peers die
peerDead chan peer.ID peerDead chan peer.ID
@ -151,7 +157,9 @@ func NewPubSub(ctx context.Context, h host.Host, rt PubSubRouter, opts ...Option
signKey: h.Peerstore().PrivKey(h.ID()), signKey: h.Peerstore().PrivKey(h.ID()),
incoming: make(chan *RPC, 32), incoming: make(chan *RPC, 32),
publish: make(chan *Message), publish: make(chan *Message),
newPeers: make(chan inet.Stream), newPeers: make(chan peer.ID),
newPeerStream: make(chan inet.Stream),
newPeerError: make(chan peer.ID),
peerDead: make(chan peer.ID), peerDead: make(chan peer.ID),
cancelCh: make(chan *Subscription), cancelCh: make(chan *Subscription),
getPeers: make(chan *listPeerReq), getPeers: make(chan *listPeerReq),
@ -259,30 +267,65 @@ func (p *PubSub) processLoop(ctx context.Context) {
p.peers = nil p.peers = nil
p.topics = nil p.topics = nil
}() }()
for { for {
select { select {
case s := <-p.newPeers: case pid := <-p.newPeers:
pid := s.Conn().RemotePeer() _, ok := p.peers[pid]
ch, ok := p.peers[pid]
if ok { if ok {
log.Error("already have connection to peer: ", pid) log.Warning("already have connection to peer: ", pid)
close(ch) continue
} }
messages := make(chan *RPC, 32) messages := make(chan *RPC, 32)
go p.handleSendingMessages(ctx, s, messages) go p.handleNewPeer(ctx, pid, messages)
messages <- p.getHelloPacket()
p.peers[pid] = messages p.peers[pid] = messages
case s := <-p.newPeerStream:
pid := s.Conn().RemotePeer()
ch, ok := p.peers[pid]
if !ok {
log.Warning("new stream for unknown peer: ", pid)
s.Reset()
continue
}
select {
case ch <- p.getHelloPacket():
default:
log.Warning("error sending hello packet; buffer full: ", pid)
go func() {
time.Sleep(time.Duration(1+rand.Intn(1000)) * time.Millisecond)
select {
case p.newPeerStream <- s:
case <-ctx.Done():
}
}()
continue
}
p.rt.AddPeer(pid, s.Protocol()) p.rt.AddPeer(pid, s.Protocol())
case pid := <-p.newPeerError:
delete(p.peers, pid)
case pid := <-p.peerDead: case pid := <-p.peerDead:
ch, ok := p.peers[pid] ch, ok := p.peers[pid]
if ok { if !ok {
close(ch) continue
} }
if p.host.Network().Connectedness(pid) == inet.Connected {
// still connected, must be a duplicate connection being closed.
// we respawn the writer as we need to ensure there is at leat one active
// at worst we can end with two writers pushing messages from the same channel.
log.Warning("peer declared dead but still connected; respawning writer: ", pid)
go p.handleNewPeer(ctx, pid, ch)
continue
}
close(ch)
delete(p.peers, pid) delete(p.peers, pid)
for _, t := range p.topics { for _, t := range p.topics {
delete(t, pid) delete(t, pid)