fix gossipsub memory leak on disconnected peer (#371)
When messages can't be sent to peer, we try to establish a send connection - this causes messages to stack up as more and more unsent messages are blocked on the dial lock. * remove dial lock * run reconnection loop in background task
This commit is contained in:
parent
49a12e619d
commit
471e5906f6
|
@ -49,7 +49,7 @@ const
|
|||
GossipSubHeartbeatInterval* = 1.seconds
|
||||
|
||||
# fanout ttl
|
||||
const
|
||||
const
|
||||
GossipSubFanoutTTL* = 1.minutes
|
||||
|
||||
const
|
||||
|
@ -189,7 +189,7 @@ proc init*(_: type[GossipSubParams]): GossipSubParams =
|
|||
)
|
||||
|
||||
proc validateParameters*(parameters: GossipSubParams): Result[void, cstring] =
|
||||
if (parameters.dOut >= GossipSubDlo) or
|
||||
if (parameters.dOut >= GossipSubDlo) or
|
||||
(parameters.dOut > (GossipSubD div 2)):
|
||||
err("gossipsub: dOut parameter error, Number of outbound connections to keep in the mesh. Must be less than D_lo and at most D/2")
|
||||
elif parameters.gossipThreshold >= 0:
|
||||
|
@ -359,6 +359,21 @@ proc replenishFanout(g: GossipSub, topic: string) =
|
|||
|
||||
trace "fanout replenished with peers", peers = g.fanout.peers(topic)
|
||||
|
||||
method onPubSubPeerEvent*(p: GossipSub, peer: PubsubPeer, event: PubSubPeerEvent) {.gcsafe.} =
|
||||
case event.kind
|
||||
of PubSubPeerEventKind.Connected:
|
||||
discard
|
||||
of PubSubPeerEventKind.Disconnected:
|
||||
# If a send connection is lost, it's better to remove peer from the mesh -
|
||||
# if it gets reestablished, the peer will be readded to the mesh, and if it
|
||||
# doesn't, well.. then we hope the peer is going away!
|
||||
for _, peers in p.mesh.mpairs():
|
||||
peers.excl(peer)
|
||||
for _, peers in p.fanout.mpairs():
|
||||
peers.excl(peer)
|
||||
|
||||
procCall FloodSub(p).onPubSubPeerEvent(peer, event)
|
||||
|
||||
proc rebalanceMesh(g: GossipSub, topic: string) {.async.} =
|
||||
logScope:
|
||||
topic
|
||||
|
@ -379,7 +394,7 @@ proc rebalanceMesh(g: GossipSub, topic: string) {.async.} =
|
|||
grafts = toSeq(
|
||||
g.gossipsub.getOrDefault(topic, initHashSet[PubSubPeer]()) -
|
||||
g.mesh.getOrDefault(topic, initHashSet[PubSubPeer]())
|
||||
)
|
||||
).filterIt(it.connected)
|
||||
|
||||
grafts.keepIf do (x: PubSubPeer) -> bool:
|
||||
# avoid negative score peers
|
||||
|
@ -404,7 +419,7 @@ proc rebalanceMesh(g: GossipSub, topic: string) {.async.} =
|
|||
g.grafted(peer, topic)
|
||||
g.fanout.removePeer(topic, peer)
|
||||
grafting &= peer
|
||||
|
||||
|
||||
elif npeers < g.parameters.dOut:
|
||||
trace "replenishing mesh outbound quota", peers = g.mesh.peers(topic)
|
||||
# replenish the mesh if we're below Dlo
|
||||
|
@ -439,7 +454,7 @@ proc rebalanceMesh(g: GossipSub, topic: string) {.async.} =
|
|||
g.grafted(peer, topic)
|
||||
g.fanout.removePeer(topic, peer)
|
||||
grafting &= peer
|
||||
|
||||
|
||||
|
||||
if g.mesh.peers(topic) > GossipSubDhi:
|
||||
# prune peers if we've gone over Dhi
|
||||
|
@ -465,7 +480,7 @@ proc rebalanceMesh(g: GossipSub, topic: string) {.async.} =
|
|||
outbound &= peer
|
||||
else:
|
||||
inbound &= peer
|
||||
|
||||
|
||||
let pruneLen = inbound.len - GossipSubD
|
||||
if pruneLen > 0:
|
||||
# Ok we got some peers to prune,
|
||||
|
@ -506,17 +521,17 @@ proc rebalanceMesh(g: GossipSub, topic: string) {.async.} =
|
|||
x.peerId notin g.parameters.directPeers and
|
||||
# and avoid peers we are backing off
|
||||
x.peerId notin g.backingOff
|
||||
|
||||
|
||||
# by spec, grab only 2
|
||||
if avail.len > 2:
|
||||
avail.setLen(2)
|
||||
|
||||
|
||||
for peer in avail:
|
||||
if g.mesh.addPeer(topic, peer):
|
||||
g.grafted(peer, topic)
|
||||
grafting &= peer
|
||||
trace "opportunistic grafting", peer = $peer
|
||||
|
||||
|
||||
when defined(libp2p_expensive_metrics):
|
||||
libp2p_gossipsub_peers_per_topic_gossipsub
|
||||
.set(g.gossipsub.peers(topic).int64, labelValues = [topic])
|
||||
|
@ -537,7 +552,7 @@ proc rebalanceMesh(g: GossipSub, topic: string) {.async.} =
|
|||
let prune = RPCMsg(control: some(ControlMessage(
|
||||
prune: @[ControlPrune(
|
||||
topicID: topic,
|
||||
peers: g.peerExchangeList(topic),
|
||||
peers: g.peerExchangeList(topic),
|
||||
backoff: g.parameters.pruneBackoff.seconds.uint64)])))
|
||||
g.broadcast(prunes, prune)
|
||||
|
||||
|
@ -623,7 +638,7 @@ proc colocationFactor(g: GossipSub, peer: PubSubPeer): float64 =
|
|||
|
||||
proc updateScores(g: GossipSub) = # avoid async
|
||||
trace "updating scores", peers = g.peers.len
|
||||
|
||||
|
||||
let now = Moment.now()
|
||||
var evicting: seq[PubSubPeer]
|
||||
|
||||
|
@ -645,13 +660,13 @@ proc updateScores(g: GossipSub) = # avoid async
|
|||
|
||||
# Scoring
|
||||
var topicScore = 0'f64
|
||||
|
||||
|
||||
if info.inMesh:
|
||||
inc is_grafted
|
||||
info.meshTime = now - info.graftTime
|
||||
if info.meshTime > topicParams.meshMessageDeliveriesActivation:
|
||||
info.meshMessageDeliveriesActive = true
|
||||
|
||||
|
||||
var p1 = info.meshTime / topicParams.timeInMeshQuantum
|
||||
if p1 > topicParams.timeInMeshCap:
|
||||
p1 = topicParams.timeInMeshCap
|
||||
|
@ -700,7 +715,7 @@ proc updateScores(g: GossipSub) = # avoid async
|
|||
# Wrap up
|
||||
# commit our changes, mgetOrPut does NOT work as wanted with value types (lent?)
|
||||
stats.topicInfos[topic] = info
|
||||
|
||||
|
||||
peer.score += peer.appScore * g.parameters.appSpecificWeight
|
||||
|
||||
peer.score += peer.behaviourPenalty * peer.behaviourPenalty * g.parameters.behaviourPenaltyWeight
|
||||
|
@ -713,10 +728,10 @@ proc updateScores(g: GossipSub) = # avoid async
|
|||
peer.behaviourPenalty = 0
|
||||
|
||||
debug "updated peer's score", peer, score = peer.score, n_topics, is_grafted
|
||||
|
||||
|
||||
for peer in evicting:
|
||||
g.peerStats.del(peer)
|
||||
|
||||
|
||||
trace "updated scores", peers = g.peers.len
|
||||
|
||||
proc heartbeat(g: GossipSub) {.async.} =
|
||||
|
@ -755,7 +770,7 @@ proc heartbeat(g: GossipSub) {.async.} =
|
|||
prunes &= peer
|
||||
let prune = RPCMsg(control: some(ControlMessage(
|
||||
prune: @[ControlPrune(
|
||||
topicID: t,
|
||||
topicID: t,
|
||||
peers: g.peerExchangeList(t),
|
||||
backoff: g.parameters.pruneBackoff.seconds.uint64)])))
|
||||
g.broadcast(prunes, prune)
|
||||
|
@ -825,7 +840,7 @@ method unsubscribePeer*(g: GossipSub, peer: PeerID) =
|
|||
when defined(libp2p_expensive_metrics):
|
||||
libp2p_gossipsub_peers_per_topic_fanout
|
||||
.set(g.fanout.peers(t).int64, labelValues = [t])
|
||||
|
||||
|
||||
# don't retain bad score peers
|
||||
if pubSubPeer.score < 0.0:
|
||||
g.peerStats.del(pubSubPeer)
|
||||
|
@ -847,7 +862,7 @@ method subscribeTopic*(g: GossipSub,
|
|||
logScope:
|
||||
peer
|
||||
topic
|
||||
|
||||
|
||||
g.onNewPeer(peer)
|
||||
|
||||
if subscribe:
|
||||
|
@ -890,7 +905,7 @@ proc handleGraft(g: GossipSub,
|
|||
|
||||
# It is an error to GRAFT on a explicit peer
|
||||
if peer.peerId in g.parameters.directPeers:
|
||||
trace "attempt to graft an explicit peer", peer=peer.id,
|
||||
trace "attempt to graft an explicit peer", peer=peer.id,
|
||||
topicID=graft.topicID
|
||||
# and such an attempt should be logged and rejected with a PRUNE
|
||||
result.add(ControlPrune(
|
||||
|
@ -900,7 +915,7 @@ proc handleGraft(g: GossipSub,
|
|||
continue
|
||||
|
||||
if peer.peerId in g.backingOff:
|
||||
trace "attempt to graft an backingOff peer", peer=peer.id,
|
||||
trace "attempt to graft an backingOff peer", peer=peer.id,
|
||||
topicID=graft.topicID,
|
||||
expire=g.backingOff[peer.peerId]
|
||||
# and such an attempt should be logged and rejected with a PRUNE
|
||||
|
@ -909,10 +924,10 @@ proc handleGraft(g: GossipSub,
|
|||
peers: @[], # omitting heavy computation here as the remote did something illegal
|
||||
backoff: g.parameters.pruneBackoff.seconds.uint64))
|
||||
continue
|
||||
|
||||
|
||||
if peer notin g.peerStats:
|
||||
g.peerStats[peer] = PeerStats()
|
||||
|
||||
|
||||
# If they send us a graft before they send us a subscribe, what should
|
||||
# we do? For now, we add them to mesh but don't add them to gossipsub.
|
||||
if topic in g.topics:
|
||||
|
@ -927,7 +942,7 @@ proc handleGraft(g: GossipSub,
|
|||
trace "peer already in mesh"
|
||||
else:
|
||||
result.add(ControlPrune(
|
||||
topicID: topic,
|
||||
topicID: topic,
|
||||
peers: g.peerExchangeList(topic),
|
||||
backoff: g.parameters.pruneBackoff.seconds.uint64))
|
||||
else:
|
||||
|
@ -950,10 +965,10 @@ proc handlePrune(g: GossipSub, peer: PubSubPeer, prunes: seq[ControlPrune]) =
|
|||
let current = g.backingOff.getOrDefault(peer.peerId)
|
||||
if backoff > current:
|
||||
g.backingOff[peer.peerId] = backoff
|
||||
|
||||
|
||||
g.pruned(peer, prune.topicID)
|
||||
g.mesh.removePeer(prune.topicID, peer)
|
||||
|
||||
|
||||
when defined(libp2p_expensive_metrics):
|
||||
libp2p_gossipsub_peers_per_topic_mesh
|
||||
.set(g.mesh.peers(prune.topicID).int64, labelValues = [prune.topicID])
|
||||
|
@ -1026,7 +1041,7 @@ method rpcHandler*(g: GossipSub,
|
|||
|
||||
# commit back to the table
|
||||
g.peerStats[peer].topicInfos[t] = stats
|
||||
|
||||
|
||||
continue
|
||||
|
||||
g.mcache.put(msgId, msg)
|
||||
|
@ -1091,11 +1106,11 @@ method subscribe*(g: GossipSub,
|
|||
topic: string,
|
||||
handler: TopicHandler) {.async.} =
|
||||
await procCall PubSub(g).subscribe(topic, handler)
|
||||
|
||||
|
||||
# if we have a fanout on this topic break it
|
||||
if topic in g.fanout:
|
||||
g.fanout.del(topic)
|
||||
|
||||
|
||||
await g.rebalanceMesh(topic)
|
||||
|
||||
method unsubscribe*(g: GossipSub,
|
||||
|
@ -1113,7 +1128,7 @@ method unsubscribe*(g: GossipSub,
|
|||
g.pruned(peer, topic)
|
||||
let prune = RPCMsg(control: some(ControlMessage(
|
||||
prune: @[ControlPrune(
|
||||
topicID: topic,
|
||||
topicID: topic,
|
||||
peers: g.peerExchangeList(topic),
|
||||
backoff: g.parameters.pruneBackoff.seconds.uint64)])))
|
||||
g.broadcast(toSeq(peers), prune)
|
||||
|
@ -1128,7 +1143,7 @@ method unsubscribeAll*(g: GossipSub, topic: string) {.async.} =
|
|||
g.pruned(peer, topic)
|
||||
let prune = RPCMsg(control: some(ControlMessage(
|
||||
prune: @[ControlPrune(
|
||||
topicID: topic,
|
||||
topicID: topic,
|
||||
peers: g.peerExchangeList(topic),
|
||||
backoff: g.parameters.pruneBackoff.seconds.uint64)])))
|
||||
g.broadcast(toSeq(peers), prune)
|
||||
|
@ -1145,11 +1160,11 @@ method publish*(g: GossipSub,
|
|||
if topic.len <= 0: # data could be 0/empty
|
||||
debug "Empty topic, skipping publish"
|
||||
return 0
|
||||
|
||||
|
||||
var peers: HashSet[PubSubPeer]
|
||||
|
||||
if g.parameters.floodPublish:
|
||||
# With flood publishing enabled, the mesh is used when propagating messages from other peers,
|
||||
# With flood publishing enabled, the mesh is used when propagating messages from other peers,
|
||||
# but a peer's own messages will always be published to all known peers in the topic.
|
||||
for peer in g.gossipsub.getOrDefault(topic):
|
||||
if peer.score >= g.parameters.publishThreshold:
|
||||
|
@ -1248,7 +1263,7 @@ method initPubSub*(g: GossipSub) =
|
|||
|
||||
if not g.parameters.explicit:
|
||||
g.parameters = GossipSubParams.init()
|
||||
|
||||
|
||||
g.parameters.validateParameters().tryGet()
|
||||
|
||||
randomize()
|
||||
|
|
|
@ -141,6 +141,22 @@ proc replenishFanout(g: GossipSub, topic: string) =
|
|||
|
||||
trace "fanout replenished with peers", peers = g.fanout.peers(topic)
|
||||
|
||||
method onPubSubPeerEvent*(p: GossipSub, peer: PubsubPeer, event: PubSubPeerEvent) {.gcsafe.} =
|
||||
case event.kind
|
||||
of PubSubPeerEventKind.Connected:
|
||||
discard
|
||||
of PubSubPeerEventKind.Disconnected:
|
||||
# If a send connection is lost, it's better to remove peer from the mesh -
|
||||
# if it gets reestablished, the peer will be readded to the mesh, and if it
|
||||
# doesn't, well.. then we hope the peer is going away!
|
||||
for _, peers in p.mesh.mpairs():
|
||||
peers.excl(peer)
|
||||
for _, peers in p.fanout.mpairs():
|
||||
peers.excl(peer)
|
||||
|
||||
procCall FloodSub(p).onPubSubPeerEvent(peer, event)
|
||||
|
||||
|
||||
proc rebalanceMesh(g: GossipSub, topic: string) {.async.} =
|
||||
logScope:
|
||||
topic
|
||||
|
@ -160,7 +176,7 @@ proc rebalanceMesh(g: GossipSub, topic: string) {.async.} =
|
|||
grafts = toSeq(
|
||||
g.gossipsub.getOrDefault(topic, initHashSet[PubSubPeer]()) -
|
||||
g.mesh.getOrDefault(topic, initHashSet[PubSubPeer]())
|
||||
)
|
||||
).filterIt(it.connected)
|
||||
|
||||
shuffle(grafts)
|
||||
|
||||
|
|
|
@ -54,24 +54,25 @@ type
|
|||
handler*: seq[TopicHandler]
|
||||
|
||||
PubSub* = ref object of LPProtocol
|
||||
switch*: Switch # the switch used to dial/connect to peers
|
||||
peerInfo*: PeerInfo # this peer's info
|
||||
topics*: Table[string, Topic] # local topics
|
||||
peers*: Table[PeerID, PubSubPeer] # peerid to peer map
|
||||
triggerSelf*: bool # trigger own local handler on publish
|
||||
verifySignature*: bool # enable signature verification
|
||||
sign*: bool # enable message signing
|
||||
switch*: Switch # the switch used to dial/connect to peers
|
||||
peerInfo*: PeerInfo # this peer's info
|
||||
topics*: Table[string, Topic] # local topics
|
||||
peers*: Table[PeerID, PubSubPeer] ##\
|
||||
## Peers that we are interested to gossip with (but not necessarily
|
||||
## yet connected to)
|
||||
triggerSelf*: bool # trigger own local handler on publish
|
||||
verifySignature*: bool # enable signature verification
|
||||
sign*: bool # enable message signing
|
||||
validators*: Table[string, HashSet[ValidatorHandler]]
|
||||
observers: ref seq[PubSubObserver] # ref as in smart_ptr
|
||||
msgIdProvider*: MsgIdProvider # Turn message into message id (not nil)
|
||||
observers: ref seq[PubSubObserver] # ref as in smart_ptr
|
||||
msgIdProvider*: MsgIdProvider # Turn message into message id (not nil)
|
||||
msgSeqno*: uint64
|
||||
lifetimeFut*: Future[void] # pubsub liftime future
|
||||
|
||||
method unsubscribePeer*(p: PubSub, peerId: PeerID) {.base.} =
|
||||
## handle peer disconnects
|
||||
##
|
||||
|
||||
trace "unsubscribing pubsub peer", peer = $peerId
|
||||
trace "unsubscribing pubsub peer", peerId
|
||||
p.peers.del(peerId)
|
||||
|
||||
libp2p_pubsub_peers.set(p.peers.len.int64)
|
||||
|
@ -80,7 +81,7 @@ proc send*(p: PubSub, peer: PubSubPeer, msg: RPCMsg) =
|
|||
## Attempt to send `msg` to remote peer
|
||||
##
|
||||
|
||||
trace "sending pubsub message to peer", peer = $peer, msg = shortLog(msg)
|
||||
trace "sending pubsub message to peer", peer, msg = shortLog(msg)
|
||||
peer.send(msg)
|
||||
|
||||
proc broadcast*(
|
||||
|
@ -119,6 +120,14 @@ method rpcHandler*(p: PubSub,
|
|||
|
||||
method onNewPeer(p: PubSub, peer: PubSubPeer) {.base.} = discard
|
||||
|
||||
method onPubSubPeerEvent*(p: PubSub, peer: PubsubPeer, event: PubsubPeerEvent) {.base, gcsafe.} =
|
||||
# Peer event is raised for the send connection in particular
|
||||
case event.kind
|
||||
of PubSubPeerEventKind.Connected:
|
||||
p.sendSubs(peer, toSeq(p.topics.keys), true)
|
||||
of PubSubPeerEventKind.Disconnected:
|
||||
discard
|
||||
|
||||
proc getOrCreatePeer*(
|
||||
p: PubSub,
|
||||
peer: PeerID,
|
||||
|
@ -126,13 +135,15 @@ proc getOrCreatePeer*(
|
|||
if peer in p.peers:
|
||||
return p.peers[peer]
|
||||
|
||||
proc getConn(): Future[(Connection, RPCMsg)] {.async.} =
|
||||
let conn = await p.switch.dial(peer, protos)
|
||||
return (conn, RPCMsg.withSubs(toSeq(p.topics.keys), true))
|
||||
proc getConn(): Future[Connection] =
|
||||
p.switch.dial(peer, protos)
|
||||
|
||||
proc onEvent(peer: PubsubPeer, event: PubsubPeerEvent) {.gcsafe.} =
|
||||
p.onPubSubPeerEvent(peer, event)
|
||||
|
||||
# create new pubsub peer
|
||||
let pubSubPeer = newPubSubPeer(peer, getConn, protos[0])
|
||||
trace "created new pubsub peer", peerId = $peer
|
||||
let pubSubPeer = newPubSubPeer(peer, getConn, onEvent, protos[0])
|
||||
trace "created new pubsub peer", peer
|
||||
|
||||
p.peers[peer] = pubSubPeer
|
||||
pubSubPeer.observers = p.observers
|
||||
|
|
|
@ -31,17 +31,25 @@ type
|
|||
onRecv*: proc(peer: PubSubPeer; msgs: var RPCMsg) {.gcsafe, raises: [Defect].}
|
||||
onSend*: proc(peer: PubSubPeer; msgs: var RPCMsg) {.gcsafe, raises: [Defect].}
|
||||
|
||||
GetConn* = proc(): Future[(Connection, RPCMsg)] {.gcsafe.}
|
||||
PubSubPeerEventKind* {.pure.} = enum
|
||||
Connected
|
||||
Disconnected
|
||||
|
||||
PubsubPeerEvent* = object
|
||||
kind*: PubSubPeerEventKind
|
||||
|
||||
GetConn* = proc(): Future[Connection] {.gcsafe.}
|
||||
OnEvent* = proc(peer: PubSubPeer, event: PubsubPeerEvent) {.gcsafe.}
|
||||
|
||||
PubSubPeer* = ref object of RootObj
|
||||
getConn*: GetConn # callback to establish a new send connection
|
||||
onEvent*: OnEvent # Connectivity updates for peer
|
||||
codec*: string # the protocol that this peer joined from
|
||||
sendConn: Connection # cached send connection
|
||||
sendConn*: Connection # cached send connection
|
||||
connections*: seq[Connection] # connections to this peer
|
||||
peerId*: PeerID
|
||||
handler*: RPCHandler
|
||||
observers*: ref seq[PubSubObserver] # ref as in smart_ptr
|
||||
dialLock: AsyncLock
|
||||
|
||||
score*: float64
|
||||
iWantBudget*: int
|
||||
|
@ -54,7 +62,7 @@ type
|
|||
|
||||
chronicles.formatIt(PubSubPeer): $it.peerId
|
||||
|
||||
func hash*(p: PubSubPeer): Hash =
|
||||
func hash*(p: PubSubPeer): Hash =
|
||||
# int is either 32/64, so intptr basically, pubsubpeer is a ref
|
||||
cast[pointer](p).hash
|
||||
|
||||
|
@ -117,10 +125,6 @@ proc handle*(p: PubSubPeer, conn: Connection) {.async.} =
|
|||
await p.handler(p, rmsg.get())
|
||||
finally:
|
||||
await conn.close()
|
||||
|
||||
if p.sendConn == conn:
|
||||
p.sendConn = nil
|
||||
|
||||
except CancelledError:
|
||||
# This is top-level procedure which will work as separate task, so it
|
||||
# do not need to propogate CancelledError.
|
||||
|
@ -132,88 +136,54 @@ proc handle*(p: PubSubPeer, conn: Connection) {.async.} =
|
|||
debug "exiting pubsub read loop",
|
||||
conn, peer = p, closed = conn.closed
|
||||
|
||||
proc getSendConn(p: PubSubPeer): Future[Connection] {.async.} =
|
||||
## get a cached send connection or create a new one - will return nil if
|
||||
## getting a new connection fails
|
||||
##
|
||||
|
||||
block: # check if there's an existing connection that can be reused
|
||||
let current = p.sendConn
|
||||
|
||||
if not current.isNil:
|
||||
if not (current.closed() or current.atEof):
|
||||
# The existing send connection looks like it might work - reuse it
|
||||
trace "Reusing existing connection", current
|
||||
return current
|
||||
|
||||
# Send connection is set but broken - get rid of it
|
||||
p.sendConn = nil
|
||||
|
||||
# Careful, p.sendConn might change after here!
|
||||
await current.close() # TODO this might be unnecessary
|
||||
|
||||
proc connectOnce(p: PubSubPeer): Future[void] {.async.} =
|
||||
try:
|
||||
# Testing has demonstrated that when we perform concurrent meshsub dials
|
||||
# and later close one of them, other implementations such as rust-libp2p
|
||||
# become deaf to our messages (potentially due to the clean-up associated
|
||||
# with closing connections). To prevent this, we use a lock that ensures
|
||||
# that only a single dial will be performed for each peer and send the
|
||||
# subscription table every time we reconnect.
|
||||
#
|
||||
# Nevertheless, this approach is still quite problematic because the gossip
|
||||
# sends and their respective dials may be started from the mplex read loop.
|
||||
# This may cause the read loop to get stuck which ultimately results in a
|
||||
# deadlock when the other side tries to send us any other message that must
|
||||
# be routed through mplex (it will be stuck on `pushTo`). Such messages
|
||||
# naturally arise in the process of dialing itself.
|
||||
#
|
||||
# See https://github.com/status-im/nim-libp2p/issues/337
|
||||
#
|
||||
# One possible long-term solution is to avoid "blocking" the mplex read
|
||||
# loop by making the gossip send non-blocking through the use of a queue.
|
||||
await p.dialLock.acquire()
|
||||
|
||||
# Another concurrent dial may have populated p.sendConn
|
||||
if p.sendConn != nil:
|
||||
let current = p.sendConn
|
||||
if not (current.closed() or current.atEof):
|
||||
# The existing send connection looks like it might work - reuse it
|
||||
debug "Reusing existing connection", current
|
||||
return current
|
||||
else:
|
||||
p.sendConn = nil
|
||||
|
||||
# Grab a new send connection
|
||||
let (newConn, handshake) = await p.getConn() # ...and here
|
||||
let newConn = await p.getConn()
|
||||
if newConn.isNil:
|
||||
debug "Failed to get a new send connection"
|
||||
return nil
|
||||
raise (ref CatchableError)(msg: "Cannot establish send connection")
|
||||
|
||||
trace "Sending handshake", newConn, handshake = shortLog(handshake)
|
||||
await newConn.writeLp(encodeRpcMsg(handshake))
|
||||
# When the send channel goes up, subscriptions need to be sent to the
|
||||
# remote peer - if we had multiple channels up and one goes down, all
|
||||
# stop working so we make an effort to only keep a single channel alive
|
||||
|
||||
trace "Caching new send connection", newConn
|
||||
trace "Get new send connection", p, newConn
|
||||
p.sendConn = newConn
|
||||
# Start a read loop on the new connection.
|
||||
# All the errors are handled inside `handle()` procedure.
|
||||
asyncSpawn p.handle(newConn)
|
||||
return newConn
|
||||
finally:
|
||||
if p.dialLock.locked:
|
||||
p.dialLock.release()
|
||||
|
||||
proc connectImpl*(p: PubSubPeer) {.async.} =
|
||||
if p.onEvent != nil:
|
||||
p.onEvent(p, PubsubPeerEvent(kind: PubSubPeerEventKind.Connected))
|
||||
|
||||
await handle(p, newConn)
|
||||
finally:
|
||||
if p.sendConn != nil:
|
||||
trace "Removing send connection", p, conn = p.sendConn
|
||||
await p.sendConn.close()
|
||||
|
||||
p.sendConn = nil
|
||||
if p.onEvent != nil:
|
||||
p.onEvent(p, PubsubPeerEvent(kind: PubSubPeerEventKind.Disconnected))
|
||||
|
||||
proc connectImpl(p: PubSubPeer) {.async.} =
|
||||
try:
|
||||
discard await getSendConn(p)
|
||||
# Keep trying to establish a connection while it's possible to do so - the
|
||||
# send connection might get disconnected due to a timeout or an unrelated
|
||||
# issue so we try to get a new on
|
||||
while true:
|
||||
await connectOnce(p)
|
||||
|
||||
except CatchableError as exc:
|
||||
debug "Could not connect to pubsub peer", err = exc.msg
|
||||
debug "Could not establish send connection", msg = exc.msg
|
||||
|
||||
proc connect*(p: PubSubPeer) =
|
||||
asyncCheck(connectImpl(p))
|
||||
asyncSpawn connectImpl(p)
|
||||
|
||||
proc sendImpl(p: PubSubPeer, msg: RPCMsg) {.async.} =
|
||||
doAssert(not isNil(p), "pubsubpeer nil!")
|
||||
|
||||
let conn = p.sendConn
|
||||
if conn == nil:
|
||||
trace "No send connection, skipping message", p, msg
|
||||
return
|
||||
|
||||
trace "sending msg to peer", peer = p, rpcMsg = shortLog(msg)
|
||||
|
||||
# trigger send hooks
|
||||
|
@ -225,13 +195,7 @@ proc sendImpl(p: PubSubPeer, msg: RPCMsg) {.async.} =
|
|||
info "empty message, skipping"
|
||||
return
|
||||
|
||||
var conn: Connection
|
||||
try:
|
||||
conn = await p.getSendConn()
|
||||
if conn == nil:
|
||||
trace "Couldn't get send connection, dropping message", peer = p
|
||||
return
|
||||
|
||||
trace "sending encoded msgs to peer", conn, encoded = shortLog(encoded)
|
||||
await conn.writeLp(encoded)
|
||||
trace "sent pubsub message to remote", conn
|
||||
|
@ -244,31 +208,26 @@ proc sendImpl(p: PubSubPeer, msg: RPCMsg) {.async.} =
|
|||
|
||||
except CatchableError as exc:
|
||||
# Because we detach the send call from the currently executing task using
|
||||
# asyncCheck, no exceptions may leak out of it
|
||||
# asyncSpawn, no exceptions may leak out of it
|
||||
trace "Unable to send to remote", conn, exc = exc.msg
|
||||
# Next time sendConn is used, it will be have its close flag set and thus
|
||||
# will be recycled
|
||||
if not isNil(conn):
|
||||
await conn.close() # This will clean up the send connection
|
||||
|
||||
if exc is CancelledError: # TODO not handled
|
||||
debug "Send cancelled", peer = p
|
||||
|
||||
# We'll ask for a new send connection whenever possible
|
||||
if p.sendConn == conn:
|
||||
p.sendConn = nil
|
||||
await conn.close() # This will clean up the send connection
|
||||
|
||||
proc send*(p: PubSubPeer, msg: RPCMsg) =
|
||||
asyncCheck sendImpl(p, msg)
|
||||
asyncSpawn sendImpl(p, msg)
|
||||
|
||||
proc `$`*(p: PubSubPeer): string =
|
||||
$p.peerId
|
||||
|
||||
proc newPubSubPeer*(peerId: PeerID,
|
||||
getConn: GetConn,
|
||||
onEvent: OnEvent,
|
||||
codec: string): PubSubPeer =
|
||||
new result
|
||||
result.getConn = getConn
|
||||
result.codec = codec
|
||||
result.peerId = peerId
|
||||
result.dialLock = newAsyncLock()
|
||||
PubSubPeer(
|
||||
getConn: getConn,
|
||||
onEvent: onEvent,
|
||||
codec: codec,
|
||||
peerId: peerId,
|
||||
)
|
||||
|
|
|
@ -17,11 +17,10 @@ type
|
|||
proc noop(data: seq[byte]) {.async, gcsafe.} = discard
|
||||
|
||||
proc getPubSubPeer(p: TestGossipSub, peerId: PeerID): auto =
|
||||
proc getConn(): Future[(Connection, RPCMsg)] {.async.} =
|
||||
let conn = await p.switch.dial(peerId, GossipSubCodec)
|
||||
return (conn, RPCMsg.withSubs(toSeq(p.topics.keys), true))
|
||||
proc getConn(): Future[Connection] =
|
||||
p.switch.dial(peerId, GossipSubCodec)
|
||||
|
||||
newPubSubPeer(peerId, getConn, GossipSubCodec)
|
||||
newPubSubPeer(peerId, getConn, nil, GossipSubCodec)
|
||||
|
||||
proc randomPeerInfo(): PeerInfo =
|
||||
PeerInfo.init(PrivateKey.random(ECDSA, rng[]).get())
|
||||
|
@ -56,6 +55,7 @@ suite "GossipSub internal":
|
|||
let peerInfo = randomPeerInfo()
|
||||
conn.peerInfo = peerInfo
|
||||
let peer = gossipSub.getPubSubPeer(peerInfo.peerId)
|
||||
peer.sendConn = conn
|
||||
gossipSub.onNewPeer(peer)
|
||||
gossipSub.peers[peerInfo.peerId] = peer
|
||||
gossipSub.gossipsub[topic].incl(peer)
|
||||
|
|
|
@ -17,11 +17,10 @@ type
|
|||
proc noop(data: seq[byte]) {.async, gcsafe.} = discard
|
||||
|
||||
proc getPubSubPeer(p: TestGossipSub, peerId: PeerID): auto =
|
||||
proc getConn(): Future[(Connection, RPCMsg)] {.async.} =
|
||||
let conn = await p.switch.dial(peerId, GossipSubCodec)
|
||||
return (conn, RPCMsg.withSubs(toSeq(p.topics.keys), true))
|
||||
proc getConn(): Future[Connection] =
|
||||
p.switch.dial(peerId, GossipSubCodec)
|
||||
|
||||
newPubSubPeer(peerId, getConn, GossipSubCodec)
|
||||
newPubSubPeer(peerId, getConn, nil, GossipSubCodec)
|
||||
|
||||
proc randomPeerInfo(): PeerInfo =
|
||||
PeerInfo.init(PrivateKey.random(ECDSA, rng[]).get())
|
||||
|
@ -47,6 +46,7 @@ suite "GossipSub internal":
|
|||
let peerInfo = randomPeerInfo()
|
||||
conn.peerInfo = peerInfo
|
||||
let peer = gossipSub.getPubSubPeer(peerInfo.peerId)
|
||||
peer.sendConn = conn
|
||||
gossipSub.peers[peerInfo.peerId] = peer
|
||||
gossipSub.mesh[topic].incl(peer)
|
||||
|
||||
|
|
Loading…
Reference in New Issue