mirror of
https://github.com/vacp2p/nim-libp2p.git
synced 2026-01-09 03:47:55 -05:00
gossipsub: unsubscribe fixes (#569)
* gossipsub: unsubscribe fixes * fix KeyError when updating metric of unsubscribed topic * fix unsubscribe message not being sent to all peers causing them to keep thinking we're still subscribed * release memory earlier in a few places * floodsub fix
This commit is contained in:
@@ -39,10 +39,10 @@ proc addSeen*(f: FloodSub, msgId: MessageID): bool =
|
||||
# Return true if the message has already been seen
|
||||
f.seen.put(f.seenSalt & msgId)
|
||||
|
||||
method subscribeTopic*(f: FloodSub,
|
||||
topic: string,
|
||||
subscribe: bool,
|
||||
peer: PubsubPeer) {.gcsafe.} =
|
||||
proc handleSubscribe*(f: FloodSub,
|
||||
peer: PubsubPeer,
|
||||
topic: string,
|
||||
subscribe: bool) =
|
||||
logScope:
|
||||
peer
|
||||
topic
|
||||
@@ -61,21 +61,16 @@ method subscribeTopic*(f: FloodSub,
|
||||
return
|
||||
|
||||
if subscribe:
|
||||
if topic notin f.floodsub:
|
||||
f.floodsub[topic] = initHashSet[PubSubPeer]()
|
||||
|
||||
trace "adding subscription for topic", peer, topic
|
||||
|
||||
# subscribe the peer to the topic
|
||||
f.floodsub[topic].incl(peer)
|
||||
f.floodsub.mgetOrPut(topic, HashSet[PubSubPeer]()).incl(peer)
|
||||
else:
|
||||
if topic notin f.floodsub:
|
||||
return
|
||||
f.floodsub.withValue(topic, peers):
|
||||
trace "removing subscription for topic", peer, topic
|
||||
|
||||
trace "removing subscription for topic", peer, topic
|
||||
|
||||
# unsubscribe the peer from the topic
|
||||
f.floodsub[topic].excl(peer)
|
||||
# unsubscribe the peer from the topic
|
||||
peers[].excl(peer)
|
||||
|
||||
method unsubscribePeer*(f: FloodSub, peer: PeerID) =
|
||||
## handle peer disconnects
|
||||
@@ -93,7 +88,9 @@ method unsubscribePeer*(f: FloodSub, peer: PeerID) =
|
||||
method rpcHandler*(f: FloodSub,
|
||||
peer: PubSubPeer,
|
||||
rpcMsg: RPCMsg) {.async.} =
|
||||
await procCall PubSub(f).rpcHandler(peer, rpcMsg)
|
||||
for i in 0..<min(f.topicsHigh, rpcMsg.subscriptions.len):
|
||||
template sub: untyped = rpcMsg.subscriptions[i]
|
||||
f.handleSubscribe(peer, sub.topic, sub.subscribe)
|
||||
|
||||
for msg in rpcMsg.messages: # for every message
|
||||
let msgId = f.msgIdProvider(msg)
|
||||
@@ -139,6 +136,8 @@ method rpcHandler*(f: FloodSub,
|
||||
f.broadcast(toSendPeers, RPCMsg(messages: @[msg]))
|
||||
trace "Forwared message to peers", peers = toSendPeers.len
|
||||
|
||||
f.updateMetrics(rpcMsg)
|
||||
|
||||
method init*(f: FloodSub) =
|
||||
proc handler(conn: Connection, proto: string) {.async.} =
|
||||
## main protocol handler that gets triggered on every
|
||||
@@ -202,19 +201,6 @@ method publish*(f: FloodSub,
|
||||
|
||||
return peers.len
|
||||
|
||||
method unsubscribe*(f: FloodSub,
|
||||
topics: seq[TopicPair]) =
|
||||
procCall PubSub(f).unsubscribe(topics)
|
||||
|
||||
for p in f.peers.values:
|
||||
f.sendSubs(p, topics.mapIt(it.topic).deduplicate(), false)
|
||||
|
||||
method unsubscribeAll*(f: FloodSub, topic: string) =
|
||||
procCall PubSub(f).unsubscribeAll(topic)
|
||||
|
||||
for p in f.peers.values:
|
||||
f.sendSubs(p, @[topic], false)
|
||||
|
||||
method initPubSub*(f: FloodSub) =
|
||||
procCall PubSub(f).initPubSub()
|
||||
f.seen = TimedCache[MessageID].init(2.minutes)
|
||||
|
||||
Reference in New Issue
Block a user