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
|
@ -359,6 +359,21 @@ proc replenishFanout(g: GossipSub, topic: string) =
|
||||||
|
|
||||||
trace "fanout replenished with peers", peers = g.fanout.peers(topic)
|
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.} =
|
proc rebalanceMesh(g: GossipSub, topic: string) {.async.} =
|
||||||
logScope:
|
logScope:
|
||||||
topic
|
topic
|
||||||
|
@ -379,7 +394,7 @@ proc rebalanceMesh(g: GossipSub, topic: string) {.async.} =
|
||||||
grafts = toSeq(
|
grafts = toSeq(
|
||||||
g.gossipsub.getOrDefault(topic, initHashSet[PubSubPeer]()) -
|
g.gossipsub.getOrDefault(topic, initHashSet[PubSubPeer]()) -
|
||||||
g.mesh.getOrDefault(topic, initHashSet[PubSubPeer]())
|
g.mesh.getOrDefault(topic, initHashSet[PubSubPeer]())
|
||||||
)
|
).filterIt(it.connected)
|
||||||
|
|
||||||
grafts.keepIf do (x: PubSubPeer) -> bool:
|
grafts.keepIf do (x: PubSubPeer) -> bool:
|
||||||
# avoid negative score peers
|
# avoid negative score peers
|
||||||
|
|
|
@ -141,6 +141,22 @@ proc replenishFanout(g: GossipSub, topic: string) =
|
||||||
|
|
||||||
trace "fanout replenished with peers", peers = g.fanout.peers(topic)
|
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.} =
|
proc rebalanceMesh(g: GossipSub, topic: string) {.async.} =
|
||||||
logScope:
|
logScope:
|
||||||
topic
|
topic
|
||||||
|
@ -160,7 +176,7 @@ proc rebalanceMesh(g: GossipSub, topic: string) {.async.} =
|
||||||
grafts = toSeq(
|
grafts = toSeq(
|
||||||
g.gossipsub.getOrDefault(topic, initHashSet[PubSubPeer]()) -
|
g.gossipsub.getOrDefault(topic, initHashSet[PubSubPeer]()) -
|
||||||
g.mesh.getOrDefault(topic, initHashSet[PubSubPeer]())
|
g.mesh.getOrDefault(topic, initHashSet[PubSubPeer]())
|
||||||
)
|
).filterIt(it.connected)
|
||||||
|
|
||||||
shuffle(grafts)
|
shuffle(grafts)
|
||||||
|
|
||||||
|
|
|
@ -54,24 +54,25 @@ type
|
||||||
handler*: seq[TopicHandler]
|
handler*: seq[TopicHandler]
|
||||||
|
|
||||||
PubSub* = ref object of LPProtocol
|
PubSub* = ref object of LPProtocol
|
||||||
switch*: Switch # the switch used to dial/connect to peers
|
switch*: Switch # the switch used to dial/connect to peers
|
||||||
peerInfo*: PeerInfo # this peer's info
|
peerInfo*: PeerInfo # this peer's info
|
||||||
topics*: Table[string, Topic] # local topics
|
topics*: Table[string, Topic] # local topics
|
||||||
peers*: Table[PeerID, PubSubPeer] # peerid to peer map
|
peers*: Table[PeerID, PubSubPeer] ##\
|
||||||
triggerSelf*: bool # trigger own local handler on publish
|
## Peers that we are interested to gossip with (but not necessarily
|
||||||
verifySignature*: bool # enable signature verification
|
## yet connected to)
|
||||||
sign*: bool # enable message signing
|
triggerSelf*: bool # trigger own local handler on publish
|
||||||
|
verifySignature*: bool # enable signature verification
|
||||||
|
sign*: bool # enable message signing
|
||||||
validators*: Table[string, HashSet[ValidatorHandler]]
|
validators*: Table[string, HashSet[ValidatorHandler]]
|
||||||
observers: ref seq[PubSubObserver] # ref as in smart_ptr
|
observers: ref seq[PubSubObserver] # ref as in smart_ptr
|
||||||
msgIdProvider*: MsgIdProvider # Turn message into message id (not nil)
|
msgIdProvider*: MsgIdProvider # Turn message into message id (not nil)
|
||||||
msgSeqno*: uint64
|
msgSeqno*: uint64
|
||||||
lifetimeFut*: Future[void] # pubsub liftime future
|
|
||||||
|
|
||||||
method unsubscribePeer*(p: PubSub, peerId: PeerID) {.base.} =
|
method unsubscribePeer*(p: PubSub, peerId: PeerID) {.base.} =
|
||||||
## handle peer disconnects
|
## handle peer disconnects
|
||||||
##
|
##
|
||||||
|
|
||||||
trace "unsubscribing pubsub peer", peer = $peerId
|
trace "unsubscribing pubsub peer", peerId
|
||||||
p.peers.del(peerId)
|
p.peers.del(peerId)
|
||||||
|
|
||||||
libp2p_pubsub_peers.set(p.peers.len.int64)
|
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
|
## 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)
|
peer.send(msg)
|
||||||
|
|
||||||
proc broadcast*(
|
proc broadcast*(
|
||||||
|
@ -119,6 +120,14 @@ method rpcHandler*(p: PubSub,
|
||||||
|
|
||||||
method onNewPeer(p: PubSub, peer: PubSubPeer) {.base.} = discard
|
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*(
|
proc getOrCreatePeer*(
|
||||||
p: PubSub,
|
p: PubSub,
|
||||||
peer: PeerID,
|
peer: PeerID,
|
||||||
|
@ -126,13 +135,15 @@ proc getOrCreatePeer*(
|
||||||
if peer in p.peers:
|
if peer in p.peers:
|
||||||
return p.peers[peer]
|
return p.peers[peer]
|
||||||
|
|
||||||
proc getConn(): Future[(Connection, RPCMsg)] {.async.} =
|
proc getConn(): Future[Connection] =
|
||||||
let conn = await p.switch.dial(peer, protos)
|
p.switch.dial(peer, protos)
|
||||||
return (conn, RPCMsg.withSubs(toSeq(p.topics.keys), true))
|
|
||||||
|
proc onEvent(peer: PubsubPeer, event: PubsubPeerEvent) {.gcsafe.} =
|
||||||
|
p.onPubSubPeerEvent(peer, event)
|
||||||
|
|
||||||
# create new pubsub peer
|
# create new pubsub peer
|
||||||
let pubSubPeer = newPubSubPeer(peer, getConn, protos[0])
|
let pubSubPeer = newPubSubPeer(peer, getConn, onEvent, protos[0])
|
||||||
trace "created new pubsub peer", peerId = $peer
|
trace "created new pubsub peer", peer
|
||||||
|
|
||||||
p.peers[peer] = pubSubPeer
|
p.peers[peer] = pubSubPeer
|
||||||
pubSubPeer.observers = p.observers
|
pubSubPeer.observers = p.observers
|
||||||
|
|
|
@ -31,17 +31,25 @@ type
|
||||||
onRecv*: proc(peer: PubSubPeer; msgs: var RPCMsg) {.gcsafe, raises: [Defect].}
|
onRecv*: proc(peer: PubSubPeer; msgs: var RPCMsg) {.gcsafe, raises: [Defect].}
|
||||||
onSend*: 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
|
PubSubPeer* = ref object of RootObj
|
||||||
getConn*: GetConn # callback to establish a new send connection
|
getConn*: GetConn # callback to establish a new send connection
|
||||||
|
onEvent*: OnEvent # Connectivity updates for peer
|
||||||
codec*: string # the protocol that this peer joined from
|
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
|
connections*: seq[Connection] # connections to this peer
|
||||||
peerId*: PeerID
|
peerId*: PeerID
|
||||||
handler*: RPCHandler
|
handler*: RPCHandler
|
||||||
observers*: ref seq[PubSubObserver] # ref as in smart_ptr
|
observers*: ref seq[PubSubObserver] # ref as in smart_ptr
|
||||||
dialLock: AsyncLock
|
|
||||||
|
|
||||||
score*: float64
|
score*: float64
|
||||||
iWantBudget*: int
|
iWantBudget*: int
|
||||||
|
@ -117,10 +125,6 @@ proc handle*(p: PubSubPeer, conn: Connection) {.async.} =
|
||||||
await p.handler(p, rmsg.get())
|
await p.handler(p, rmsg.get())
|
||||||
finally:
|
finally:
|
||||||
await conn.close()
|
await conn.close()
|
||||||
|
|
||||||
if p.sendConn == conn:
|
|
||||||
p.sendConn = nil
|
|
||||||
|
|
||||||
except CancelledError:
|
except CancelledError:
|
||||||
# This is top-level procedure which will work as separate task, so it
|
# This is top-level procedure which will work as separate task, so it
|
||||||
# do not need to propogate CancelledError.
|
# do not need to propogate CancelledError.
|
||||||
|
@ -132,88 +136,54 @@ proc handle*(p: PubSubPeer, conn: Connection) {.async.} =
|
||||||
debug "exiting pubsub read loop",
|
debug "exiting pubsub read loop",
|
||||||
conn, peer = p, closed = conn.closed
|
conn, peer = p, closed = conn.closed
|
||||||
|
|
||||||
proc getSendConn(p: PubSubPeer): Future[Connection] {.async.} =
|
proc connectOnce(p: PubSubPeer): Future[void] {.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
|
|
||||||
|
|
||||||
try:
|
try:
|
||||||
# Testing has demonstrated that when we perform concurrent meshsub dials
|
let newConn = await p.getConn()
|
||||||
# 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
|
|
||||||
if newConn.isNil:
|
if newConn.isNil:
|
||||||
debug "Failed to get a new send connection"
|
raise (ref CatchableError)(msg: "Cannot establish send connection")
|
||||||
return nil
|
|
||||||
|
|
||||||
trace "Sending handshake", newConn, handshake = shortLog(handshake)
|
# When the send channel goes up, subscriptions need to be sent to the
|
||||||
await newConn.writeLp(encodeRpcMsg(handshake))
|
# 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
|
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:
|
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:
|
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) =
|
proc connect*(p: PubSubPeer) =
|
||||||
asyncCheck(connectImpl(p))
|
asyncSpawn connectImpl(p)
|
||||||
|
|
||||||
proc sendImpl(p: PubSubPeer, msg: RPCMsg) {.async.} =
|
proc sendImpl(p: PubSubPeer, msg: RPCMsg) {.async.} =
|
||||||
doAssert(not isNil(p), "pubsubpeer nil!")
|
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)
|
trace "sending msg to peer", peer = p, rpcMsg = shortLog(msg)
|
||||||
|
|
||||||
# trigger send hooks
|
# trigger send hooks
|
||||||
|
@ -225,13 +195,7 @@ proc sendImpl(p: PubSubPeer, msg: RPCMsg) {.async.} =
|
||||||
info "empty message, skipping"
|
info "empty message, skipping"
|
||||||
return
|
return
|
||||||
|
|
||||||
var conn: Connection
|
|
||||||
try:
|
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)
|
trace "sending encoded msgs to peer", conn, encoded = shortLog(encoded)
|
||||||
await conn.writeLp(encoded)
|
await conn.writeLp(encoded)
|
||||||
trace "sent pubsub message to remote", conn
|
trace "sent pubsub message to remote", conn
|
||||||
|
@ -244,31 +208,26 @@ proc sendImpl(p: PubSubPeer, msg: RPCMsg) {.async.} =
|
||||||
|
|
||||||
except CatchableError as exc:
|
except CatchableError as exc:
|
||||||
# Because we detach the send call from the currently executing task using
|
# 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
|
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
|
# Next time sendConn is used, it will be have its close flag set and thus
|
||||||
# will be recycled
|
# will be recycled
|
||||||
if not isNil(conn):
|
|
||||||
await conn.close() # This will clean up the send connection
|
|
||||||
|
|
||||||
if exc is CancelledError: # TODO not handled
|
await conn.close() # This will clean up the send connection
|
||||||
debug "Send cancelled", peer = p
|
|
||||||
|
|
||||||
# We'll ask for a new send connection whenever possible
|
|
||||||
if p.sendConn == conn:
|
|
||||||
p.sendConn = nil
|
|
||||||
|
|
||||||
proc send*(p: PubSubPeer, msg: RPCMsg) =
|
proc send*(p: PubSubPeer, msg: RPCMsg) =
|
||||||
asyncCheck sendImpl(p, msg)
|
asyncSpawn sendImpl(p, msg)
|
||||||
|
|
||||||
proc `$`*(p: PubSubPeer): string =
|
proc `$`*(p: PubSubPeer): string =
|
||||||
$p.peerId
|
$p.peerId
|
||||||
|
|
||||||
proc newPubSubPeer*(peerId: PeerID,
|
proc newPubSubPeer*(peerId: PeerID,
|
||||||
getConn: GetConn,
|
getConn: GetConn,
|
||||||
|
onEvent: OnEvent,
|
||||||
codec: string): PubSubPeer =
|
codec: string): PubSubPeer =
|
||||||
new result
|
PubSubPeer(
|
||||||
result.getConn = getConn
|
getConn: getConn,
|
||||||
result.codec = codec
|
onEvent: onEvent,
|
||||||
result.peerId = peerId
|
codec: codec,
|
||||||
result.dialLock = newAsyncLock()
|
peerId: peerId,
|
||||||
|
)
|
||||||
|
|
|
@ -17,11 +17,10 @@ type
|
||||||
proc noop(data: seq[byte]) {.async, gcsafe.} = discard
|
proc noop(data: seq[byte]) {.async, gcsafe.} = discard
|
||||||
|
|
||||||
proc getPubSubPeer(p: TestGossipSub, peerId: PeerID): auto =
|
proc getPubSubPeer(p: TestGossipSub, peerId: PeerID): auto =
|
||||||
proc getConn(): Future[(Connection, RPCMsg)] {.async.} =
|
proc getConn(): Future[Connection] =
|
||||||
let conn = await p.switch.dial(peerId, GossipSubCodec)
|
p.switch.dial(peerId, GossipSubCodec)
|
||||||
return (conn, RPCMsg.withSubs(toSeq(p.topics.keys), true))
|
|
||||||
|
|
||||||
newPubSubPeer(peerId, getConn, GossipSubCodec)
|
newPubSubPeer(peerId, getConn, nil, GossipSubCodec)
|
||||||
|
|
||||||
proc randomPeerInfo(): PeerInfo =
|
proc randomPeerInfo(): PeerInfo =
|
||||||
PeerInfo.init(PrivateKey.random(ECDSA, rng[]).get())
|
PeerInfo.init(PrivateKey.random(ECDSA, rng[]).get())
|
||||||
|
@ -56,6 +55,7 @@ suite "GossipSub internal":
|
||||||
let peerInfo = randomPeerInfo()
|
let peerInfo = randomPeerInfo()
|
||||||
conn.peerInfo = peerInfo
|
conn.peerInfo = peerInfo
|
||||||
let peer = gossipSub.getPubSubPeer(peerInfo.peerId)
|
let peer = gossipSub.getPubSubPeer(peerInfo.peerId)
|
||||||
|
peer.sendConn = conn
|
||||||
gossipSub.onNewPeer(peer)
|
gossipSub.onNewPeer(peer)
|
||||||
gossipSub.peers[peerInfo.peerId] = peer
|
gossipSub.peers[peerInfo.peerId] = peer
|
||||||
gossipSub.gossipsub[topic].incl(peer)
|
gossipSub.gossipsub[topic].incl(peer)
|
||||||
|
|
|
@ -17,11 +17,10 @@ type
|
||||||
proc noop(data: seq[byte]) {.async, gcsafe.} = discard
|
proc noop(data: seq[byte]) {.async, gcsafe.} = discard
|
||||||
|
|
||||||
proc getPubSubPeer(p: TestGossipSub, peerId: PeerID): auto =
|
proc getPubSubPeer(p: TestGossipSub, peerId: PeerID): auto =
|
||||||
proc getConn(): Future[(Connection, RPCMsg)] {.async.} =
|
proc getConn(): Future[Connection] =
|
||||||
let conn = await p.switch.dial(peerId, GossipSubCodec)
|
p.switch.dial(peerId, GossipSubCodec)
|
||||||
return (conn, RPCMsg.withSubs(toSeq(p.topics.keys), true))
|
|
||||||
|
|
||||||
newPubSubPeer(peerId, getConn, GossipSubCodec)
|
newPubSubPeer(peerId, getConn, nil, GossipSubCodec)
|
||||||
|
|
||||||
proc randomPeerInfo(): PeerInfo =
|
proc randomPeerInfo(): PeerInfo =
|
||||||
PeerInfo.init(PrivateKey.random(ECDSA, rng[]).get())
|
PeerInfo.init(PrivateKey.random(ECDSA, rng[]).get())
|
||||||
|
@ -47,6 +46,7 @@ suite "GossipSub internal":
|
||||||
let peerInfo = randomPeerInfo()
|
let peerInfo = randomPeerInfo()
|
||||||
conn.peerInfo = peerInfo
|
conn.peerInfo = peerInfo
|
||||||
let peer = gossipSub.getPubSubPeer(peerInfo.peerId)
|
let peer = gossipSub.getPubSubPeer(peerInfo.peerId)
|
||||||
|
peer.sendConn = conn
|
||||||
gossipSub.peers[peerInfo.peerId] = peer
|
gossipSub.peers[peerInfo.peerId] = peer
|
||||||
gossipSub.mesh[topic].incl(peer)
|
gossipSub.mesh[topic].incl(peer)
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue