2019-12-06 02:16:18 +00:00
|
|
|
## Nim-LibP2P
|
|
|
|
## Copyright (c) 2019 Status Research & Development GmbH
|
|
|
|
## Licensed under either of
|
|
|
|
## * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE))
|
|
|
|
## * MIT license ([LICENSE-MIT](LICENSE-MIT))
|
|
|
|
## at your option.
|
|
|
|
## This file may not be copied, modified, or distributed except according to
|
|
|
|
## those terms.
|
|
|
|
|
|
|
|
import tables, sets, options, sequtils, random
|
2020-06-07 07:15:21 +00:00
|
|
|
import chronos, chronicles, metrics
|
2019-12-06 02:16:18 +00:00
|
|
|
import pubsub,
|
|
|
|
floodsub,
|
|
|
|
pubsubpeer,
|
|
|
|
mcache,
|
|
|
|
timedcache,
|
|
|
|
rpc/[messages, message],
|
|
|
|
../../crypto/crypto,
|
|
|
|
../protocol,
|
|
|
|
../../peerinfo,
|
|
|
|
../../connection,
|
2020-04-11 04:08:25 +00:00
|
|
|
../../peer,
|
2020-05-20 00:14:15 +00:00
|
|
|
../../errors,
|
|
|
|
../../utility
|
2019-12-06 02:16:18 +00:00
|
|
|
|
|
|
|
logScope:
|
2020-06-10 08:48:01 +00:00
|
|
|
topics = "gossipsub"
|
2019-12-06 02:16:18 +00:00
|
|
|
|
|
|
|
const GossipSubCodec* = "/meshsub/1.0.0"
|
|
|
|
|
|
|
|
# overlay parameters
|
2019-12-10 20:50:35 +00:00
|
|
|
const GossipSubD* = 6
|
2019-12-06 02:16:18 +00:00
|
|
|
const GossipSubDlo* = 4
|
|
|
|
const GossipSubDhi* = 12
|
|
|
|
|
|
|
|
# gossip parameters
|
|
|
|
const GossipSubHistoryLength* = 5
|
|
|
|
const GossipSubHistoryGossip* = 3
|
|
|
|
|
|
|
|
# heartbeat interval
|
|
|
|
const GossipSubHeartbeatInitialDelay* = 100.millis
|
2019-12-10 20:50:35 +00:00
|
|
|
const GossipSubHeartbeatInterval* = 1.seconds
|
2019-12-06 02:16:18 +00:00
|
|
|
|
|
|
|
# fanout ttl
|
2020-01-07 08:04:02 +00:00
|
|
|
const GossipSubFanoutTTL* = 1.minutes
|
2019-12-06 02:16:18 +00:00
|
|
|
|
|
|
|
type
|
|
|
|
GossipSub* = ref object of FloodSub
|
2020-01-07 08:04:02 +00:00
|
|
|
mesh*: Table[string, HashSet[string]] # meshes - topic to peer
|
|
|
|
fanout*: Table[string, HashSet[string]] # fanout - topic to peer
|
2019-12-06 02:16:18 +00:00
|
|
|
gossipsub*: Table[string, HashSet[string]] # topic to peer map of all gossipsub peers
|
2020-01-07 08:04:02 +00:00
|
|
|
lastFanoutPubSub*: Table[string, Moment] # last publish time for fanout topics
|
|
|
|
gossip*: Table[string, seq[ControlIHave]] # pending gossip
|
|
|
|
control*: Table[string, ControlMessage] # pending control messages
|
|
|
|
mcache*: MCache # messages cache
|
2020-05-27 18:33:49 +00:00
|
|
|
heartbeatCancel*: Future[void] # cancellation future for heartbeat interval
|
|
|
|
heartbeatLock: AsyncLock # heartbeat lock to prevent two consecutive concurrent heartbeats
|
2019-12-06 02:16:18 +00:00
|
|
|
|
2020-06-07 07:15:21 +00:00
|
|
|
declareGauge(libp2p_gossipsub_peers_per_topic_mesh, "gossipsub peers per topic in mesh", labels = ["topic"])
|
|
|
|
declareGauge(libp2p_gossipsub_peers_per_topic_fanout, "gossipsub peers per topic in fanout", labels = ["topic"])
|
|
|
|
declareGauge(libp2p_gossipsub_peers_per_topic_gossipsub, "gossipsub peers per topic in gossipsub", labels = ["topic"])
|
|
|
|
|
2019-12-10 20:50:35 +00:00
|
|
|
method init(g: GossipSub) =
|
2019-12-17 05:24:03 +00:00
|
|
|
proc handler(conn: Connection, proto: string) {.async.} =
|
2019-12-06 02:16:18 +00:00
|
|
|
## main protocol handler that gets triggered on every
|
|
|
|
## connection for a protocol string
|
|
|
|
## e.g. ``/floodsub/1.0.0``, etc...
|
|
|
|
##
|
|
|
|
|
|
|
|
await g.handleConn(conn, proto)
|
|
|
|
|
|
|
|
g.handler = handler
|
|
|
|
g.codec = GossipSubCodec
|
|
|
|
|
2020-05-21 20:24:20 +00:00
|
|
|
proc replenishFanout(g: GossipSub, topic: string) {.async.} =
|
|
|
|
## get fanout peers for a topic
|
|
|
|
trace "about to replenish fanout"
|
|
|
|
if topic notin g.fanout:
|
|
|
|
g.fanout[topic] = initHashSet[string]()
|
|
|
|
|
|
|
|
if g.fanout[topic].len < GossipSubDLo:
|
|
|
|
trace "replenishing fanout", peers = g.fanout[topic].len
|
|
|
|
if topic in g.gossipsub:
|
|
|
|
for p in g.gossipsub[topic]:
|
|
|
|
if not g.fanout[topic].containsOrIncl(p):
|
2020-06-11 18:09:34 +00:00
|
|
|
libp2p_gossipsub_peers_per_topic_fanout.set(g.fanout[topic].len.int64, labelValues = [topic])
|
2020-05-21 20:24:20 +00:00
|
|
|
if g.fanout[topic].len == GossipSubD:
|
|
|
|
break
|
|
|
|
|
|
|
|
trace "fanout replenished with peers", peers = g.fanout[topic].len
|
|
|
|
|
|
|
|
proc rebalanceMesh(g: GossipSub, topic: string) {.async.} =
|
|
|
|
try:
|
|
|
|
trace "about to rebalance mesh"
|
|
|
|
# create a mesh topic that we're subscribing to
|
|
|
|
if topic notin g.mesh:
|
|
|
|
g.mesh[topic] = initHashSet[string]()
|
|
|
|
|
|
|
|
if g.mesh[topic].len < GossipSubDlo:
|
|
|
|
trace "replenishing mesh"
|
|
|
|
# replenish the mesh if we're below GossipSubDlo
|
|
|
|
while g.mesh[topic].len < GossipSubD:
|
|
|
|
trace "gathering peers", peers = g.mesh[topic].len
|
|
|
|
var id: string
|
|
|
|
if topic in g.fanout and g.fanout[topic].len > 0:
|
|
|
|
id = sample(toSeq(g.fanout[topic]))
|
|
|
|
g.fanout[topic].excl(id)
|
2020-06-11 18:09:34 +00:00
|
|
|
libp2p_gossipsub_peers_per_topic_fanout.set(g.fanout[topic].len.int64, labelValues = [topic])
|
2020-05-21 20:24:20 +00:00
|
|
|
trace "got fanout peer", peer = id
|
|
|
|
elif topic in g.gossipsub and g.gossipsub[topic].len > 0:
|
|
|
|
id = sample(toSeq(g.gossipsub[topic]))
|
|
|
|
g.gossipsub[topic].excl(id)
|
2020-06-11 18:09:34 +00:00
|
|
|
libp2p_gossipsub_peers_per_topic_gossipsub.set(g.gossipsub[topic].len.int64, labelValues = [topic])
|
2020-05-21 20:24:20 +00:00
|
|
|
trace "got gossipsub peer", peer = id
|
|
|
|
else:
|
|
|
|
trace "no more peers"
|
|
|
|
break
|
|
|
|
|
|
|
|
g.mesh[topic].incl(id)
|
2020-06-11 18:09:34 +00:00
|
|
|
libp2p_gossipsub_peers_per_topic_mesh.set(g.mesh[topic].len.int64, labelValues = [topic])
|
2020-05-21 20:24:20 +00:00
|
|
|
if id in g.peers:
|
|
|
|
let p = g.peers[id]
|
|
|
|
# send a graft message to the peer
|
|
|
|
await p.sendGraft(@[topic])
|
|
|
|
|
|
|
|
# prune peers if we've gone over
|
|
|
|
if g.mesh[topic].len > GossipSubDhi:
|
|
|
|
trace "pruning mesh"
|
|
|
|
while g.mesh[topic].len > GossipSubD:
|
|
|
|
trace "pruning peers", peers = g.mesh[topic].len
|
|
|
|
let id = toSeq(g.mesh[topic])[rand(0..<g.mesh[topic].len)]
|
|
|
|
g.mesh[topic].excl(id)
|
2020-06-11 18:09:34 +00:00
|
|
|
libp2p_gossipsub_peers_per_topic_mesh.set(g.mesh[topic].len.int64, labelValues = [topic])
|
2020-05-21 20:24:20 +00:00
|
|
|
|
|
|
|
let p = g.peers[id]
|
|
|
|
# send a graft message to the peer
|
|
|
|
await p.sendPrune(@[topic])
|
|
|
|
|
2020-06-11 18:09:34 +00:00
|
|
|
trace "mesh balanced, got peers", peers = g.mesh[topic].len.int64, topicId = topic
|
2020-05-21 20:24:20 +00:00
|
|
|
except CatchableError as exc:
|
2020-05-27 18:33:49 +00:00
|
|
|
trace "exception occurred re-balancing mesh", exc = exc.msg
|
2020-05-21 20:24:20 +00:00
|
|
|
|
|
|
|
proc dropFanoutPeers(g: GossipSub) {.async.} =
|
|
|
|
# drop peers that we haven't published to in
|
|
|
|
# GossipSubFanoutTTL seconds
|
|
|
|
var dropping = newSeq[string]()
|
|
|
|
for topic, val in g.lastFanoutPubSub:
|
|
|
|
if Moment.now > val:
|
|
|
|
dropping.add(topic)
|
|
|
|
g.fanout.del(topic)
|
|
|
|
for topic in dropping:
|
|
|
|
g.lastFanoutPubSub.del(topic)
|
|
|
|
|
|
|
|
proc getGossipPeers(g: GossipSub): Table[string, ControlMessage] {.gcsafe.} =
|
|
|
|
## gossip iHave messages to peers
|
|
|
|
let topics = toHashSet(toSeq(g.mesh.keys)) + toHashSet(toSeq(g.fanout.keys))
|
|
|
|
for topic in topics:
|
|
|
|
let mesh: HashSet[string] =
|
|
|
|
if topic in g.mesh:
|
|
|
|
g.mesh[topic]
|
|
|
|
else:
|
|
|
|
initHashSet[string]()
|
|
|
|
|
|
|
|
let fanout: HashSet[string] =
|
|
|
|
if topic in g.fanout:
|
|
|
|
g.fanout[topic]
|
|
|
|
else:
|
|
|
|
initHashSet[string]()
|
|
|
|
|
|
|
|
let gossipPeers = mesh + fanout
|
|
|
|
let mids = g.mcache.window(topic)
|
|
|
|
if mids.len > 0:
|
|
|
|
let ihave = ControlIHave(topicID: topic,
|
|
|
|
messageIDs: toSeq(mids))
|
|
|
|
|
|
|
|
if topic notin g.gossipsub:
|
|
|
|
trace "topic not in gossip array, skipping", topicID = topic
|
|
|
|
continue
|
|
|
|
|
|
|
|
while result.len < GossipSubD:
|
|
|
|
if not (g.gossipsub[topic].len > 0):
|
|
|
|
trace "no peers for topic, skipping", topicID = topic
|
|
|
|
break
|
|
|
|
|
|
|
|
let id = toSeq(g.gossipsub[topic]).sample()
|
|
|
|
g.gossipsub[topic].excl(id)
|
2020-06-11 18:09:34 +00:00
|
|
|
libp2p_gossipsub_peers_per_topic_gossipsub.set(g.gossipsub[topic].len.int64, labelValues = [topic])
|
2020-05-21 20:24:20 +00:00
|
|
|
if id notin gossipPeers:
|
|
|
|
if id notin result:
|
|
|
|
result[id] = ControlMessage()
|
|
|
|
result[id].ihave.add(ihave)
|
|
|
|
|
|
|
|
proc heartbeat(g: GossipSub) {.async.} =
|
2020-05-27 18:33:49 +00:00
|
|
|
while true:
|
|
|
|
try:
|
|
|
|
await g.heartbeatLock.acquire()
|
|
|
|
trace "running heartbeat"
|
2020-05-21 20:24:20 +00:00
|
|
|
|
2020-05-27 18:33:49 +00:00
|
|
|
for t in g.mesh.keys:
|
|
|
|
await g.rebalanceMesh(t)
|
2020-05-21 20:24:20 +00:00
|
|
|
|
2020-05-27 18:33:49 +00:00
|
|
|
await g.dropFanoutPeers()
|
|
|
|
let peers = g.getGossipPeers()
|
|
|
|
var sent: seq[Future[void]]
|
|
|
|
for peer in peers.keys:
|
|
|
|
sent &= g.peers[peer].send(@[RPCMsg(control: some(peers[peer]))])
|
|
|
|
checkFutures(await allFinished(sent))
|
2020-05-21 20:24:20 +00:00
|
|
|
|
2020-05-27 18:33:49 +00:00
|
|
|
g.mcache.shift() # shift the cache
|
|
|
|
except CatchableError as exc:
|
|
|
|
trace "exception ocurred in gossipsub heartbeat", exc = exc.msg
|
|
|
|
finally:
|
|
|
|
g.heartbeatLock.release()
|
2020-05-21 20:24:20 +00:00
|
|
|
|
2020-05-27 18:33:49 +00:00
|
|
|
await sleepAsync(1.seconds)
|
2020-05-21 20:24:20 +00:00
|
|
|
|
2019-12-17 05:24:03 +00:00
|
|
|
method handleDisconnect(g: GossipSub, peer: PubSubPeer) {.async.} =
|
2019-12-06 02:16:18 +00:00
|
|
|
## handle peer disconnects
|
2020-04-21 01:24:42 +00:00
|
|
|
trace "peer disconnected", peer=peer.id
|
|
|
|
|
2019-12-06 02:16:18 +00:00
|
|
|
await procCall FloodSub(g).handleDisconnect(peer)
|
2020-05-06 09:26:08 +00:00
|
|
|
|
2019-12-06 02:16:18 +00:00
|
|
|
for t in g.gossipsub.keys:
|
|
|
|
g.gossipsub[t].excl(peer.id)
|
2020-06-11 18:09:34 +00:00
|
|
|
libp2p_gossipsub_peers_per_topic_gossipsub.set(g.gossipsub[t].len.int64, labelValues = [t])
|
2019-12-06 02:16:18 +00:00
|
|
|
|
|
|
|
for t in g.mesh.keys:
|
|
|
|
g.mesh[t].excl(peer.id)
|
2020-06-11 18:09:34 +00:00
|
|
|
libp2p_gossipsub_peers_per_topic_mesh.set(g.mesh[t].len.int64, labelValues = [t])
|
2019-12-06 02:16:18 +00:00
|
|
|
|
|
|
|
for t in g.fanout.keys:
|
|
|
|
g.fanout[t].excl(peer.id)
|
2020-06-11 18:09:34 +00:00
|
|
|
libp2p_gossipsub_peers_per_topic_fanout.set(g.fanout[t].len.int64, labelValues = [t])
|
2019-12-06 02:16:18 +00:00
|
|
|
|
2020-05-21 17:33:48 +00:00
|
|
|
method subscribeToPeer*(p: GossipSub,
|
|
|
|
conn: Connection) {.async.} =
|
|
|
|
await procCall PubSub(p).subscribeToPeer(conn)
|
|
|
|
asyncCheck p.handleConn(conn, GossipSubCodec)
|
|
|
|
|
2019-12-06 02:16:18 +00:00
|
|
|
method subscribeTopic*(g: GossipSub,
|
|
|
|
topic: string,
|
|
|
|
subscribe: bool,
|
2020-05-27 18:33:49 +00:00
|
|
|
peerId: string) {.gcsafe, async.} =
|
|
|
|
await procCall PubSub(g).subscribeTopic(topic, subscribe, peerId)
|
2019-12-06 02:16:18 +00:00
|
|
|
|
2019-12-10 20:50:35 +00:00
|
|
|
if topic notin g.gossipsub:
|
|
|
|
g.gossipsub[topic] = initHashSet[string]()
|
2019-12-06 02:16:18 +00:00
|
|
|
|
2019-12-10 20:50:35 +00:00
|
|
|
if subscribe:
|
|
|
|
trace "adding subscription for topic", peer = peerId, name = topic
|
2020-05-27 18:33:49 +00:00
|
|
|
# subscribe remote peer to the topic
|
2019-12-10 20:50:35 +00:00
|
|
|
g.gossipsub[topic].incl(peerId)
|
2020-06-11 18:09:34 +00:00
|
|
|
libp2p_gossipsub_peers_per_topic_gossipsub.set(g.gossipsub[topic].len.int64, labelValues = [topic])
|
2019-12-10 20:50:35 +00:00
|
|
|
else:
|
|
|
|
trace "removing subscription for topic", peer = peerId, name = topic
|
2020-05-27 18:33:49 +00:00
|
|
|
# unsubscribe remote peer from the topic
|
2019-12-10 20:50:35 +00:00
|
|
|
g.gossipsub[topic].excl(peerId)
|
2020-06-11 18:09:34 +00:00
|
|
|
libp2p_gossipsub_peers_per_topic_gossipsub.set(g.gossipsub[topic].len.int64, labelValues = [topic])
|
2019-12-06 02:16:18 +00:00
|
|
|
|
2020-05-27 18:33:49 +00:00
|
|
|
if topic in g.topics:
|
|
|
|
await g.rebalanceMesh(topic)
|
|
|
|
|
2019-12-10 20:50:35 +00:00
|
|
|
proc handleGraft(g: GossipSub,
|
2019-12-06 02:16:18 +00:00
|
|
|
peer: PubSubPeer,
|
|
|
|
grafts: seq[ControlGraft],
|
|
|
|
respControl: var ControlMessage) =
|
|
|
|
for graft in grafts:
|
2019-12-10 20:50:35 +00:00
|
|
|
trace "processing graft message", peer = peer.id,
|
2019-12-06 02:16:18 +00:00
|
|
|
topicID = graft.topicID
|
|
|
|
|
|
|
|
if graft.topicID in g.topics:
|
|
|
|
if g.mesh.len < GossipSubD:
|
|
|
|
g.mesh[graft.topicID].incl(peer.id)
|
2020-06-11 18:09:34 +00:00
|
|
|
libp2p_gossipsub_peers_per_topic_mesh.set(g.mesh[graft.topicID].len.int64, labelValues = [graft.topicID])
|
2019-12-06 02:16:18 +00:00
|
|
|
else:
|
|
|
|
g.gossipsub[graft.topicID].incl(peer.id)
|
2020-06-11 18:09:34 +00:00
|
|
|
libp2p_gossipsub_peers_per_topic_gossipsub.set(g.gossipsub[graft.topicID].len.int64, labelValues = [graft.topicID])
|
2019-12-06 02:16:18 +00:00
|
|
|
else:
|
|
|
|
respControl.prune.add(ControlPrune(topicID: graft.topicID))
|
|
|
|
|
|
|
|
proc handlePrune(g: GossipSub, peer: PubSubPeer, prunes: seq[ControlPrune]) =
|
|
|
|
for prune in prunes:
|
2019-12-10 20:50:35 +00:00
|
|
|
trace "processing prune message", peer = peer.id,
|
2019-12-06 02:16:18 +00:00
|
|
|
topicID = prune.topicID
|
|
|
|
|
|
|
|
if prune.topicID in g.mesh:
|
|
|
|
g.mesh[prune.topicID].excl(peer.id)
|
2020-06-11 18:09:34 +00:00
|
|
|
libp2p_gossipsub_peers_per_topic_mesh.set(g.mesh[prune.topicID].len.int64, labelValues = [prune.topicID])
|
2019-12-06 02:16:18 +00:00
|
|
|
|
2020-05-21 20:24:20 +00:00
|
|
|
proc handleIHave(g: GossipSub,
|
|
|
|
peer: PubSubPeer,
|
|
|
|
ihaves: seq[ControlIHave]): ControlIWant =
|
2019-12-06 02:16:18 +00:00
|
|
|
for ihave in ihaves:
|
|
|
|
trace "processing ihave message", peer = peer.id,
|
2020-05-21 20:24:20 +00:00
|
|
|
topicID = ihave.topicID,
|
|
|
|
msgs = ihave.messageIDs
|
2019-12-06 02:16:18 +00:00
|
|
|
|
|
|
|
if ihave.topicID in g.mesh:
|
|
|
|
for m in ihave.messageIDs:
|
|
|
|
if m notin g.seen:
|
|
|
|
result.messageIDs.add(m)
|
|
|
|
|
2020-05-21 20:24:20 +00:00
|
|
|
proc handleIWant(g: GossipSub,
|
|
|
|
peer: PubSubPeer,
|
|
|
|
iwants: seq[ControlIWant]): seq[Message] =
|
2019-12-06 02:16:18 +00:00
|
|
|
for iwant in iwants:
|
|
|
|
for mid in iwant.messageIDs:
|
|
|
|
trace "processing iwant message", peer = peer.id,
|
|
|
|
messageID = mid
|
|
|
|
let msg = g.mcache.get(mid)
|
|
|
|
if msg.isSome:
|
|
|
|
result.add(msg.get())
|
|
|
|
|
|
|
|
method rpcHandler(g: GossipSub,
|
|
|
|
peer: PubSubPeer,
|
2019-12-17 05:24:03 +00:00
|
|
|
rpcMsgs: seq[RPCMsg]) {.async.} =
|
2019-12-06 02:16:18 +00:00
|
|
|
await procCall PubSub(g).rpcHandler(peer, rpcMsgs)
|
|
|
|
|
2019-12-17 05:24:03 +00:00
|
|
|
for m in rpcMsgs: # for all RPC messages
|
2019-12-06 02:16:18 +00:00
|
|
|
if m.messages.len > 0: # if there are any messages
|
2020-05-21 20:24:20 +00:00
|
|
|
var toSendPeers: HashSet[string]
|
2019-12-06 02:16:18 +00:00
|
|
|
for msg in m.messages: # for every message
|
|
|
|
trace "processing message with id", msg = msg.msgId
|
|
|
|
if msg.msgId in g.seen:
|
|
|
|
trace "message already processed, skipping", msg = msg.msgId
|
|
|
|
continue
|
|
|
|
|
|
|
|
g.seen.put(msg.msgId) # add the message to the seen cache
|
|
|
|
|
2020-05-06 09:26:08 +00:00
|
|
|
if g.verifySignature and not msg.verify(peer.peerInfo):
|
2019-12-17 05:24:03 +00:00
|
|
|
trace "dropping message due to failed signature verification"
|
|
|
|
continue
|
|
|
|
|
|
|
|
if not (await g.validate(msg)):
|
|
|
|
trace "dropping message due to failed validation"
|
|
|
|
continue
|
|
|
|
|
2019-12-06 02:16:18 +00:00
|
|
|
# this shouldn't happen
|
2019-12-07 16:36:39 +00:00
|
|
|
if g.peerInfo.peerId == msg.fromPeerId():
|
2019-12-06 02:16:18 +00:00
|
|
|
trace "skipping messages from self", msg = msg.msgId
|
|
|
|
continue
|
|
|
|
|
|
|
|
for t in msg.topicIDs: # for every topic in the message
|
2020-05-21 20:24:20 +00:00
|
|
|
await g.rebalanceMesh(t) # gather peers for each topic
|
2019-12-06 02:16:18 +00:00
|
|
|
if t in g.floodsub:
|
|
|
|
toSendPeers.incl(g.floodsub[t]) # get all floodsub peers for topic
|
|
|
|
|
|
|
|
if t in g.mesh:
|
|
|
|
toSendPeers.incl(g.mesh[t]) # get all mesh peers for topic
|
|
|
|
|
|
|
|
if t in g.topics: # if we're subscribed to the topic
|
|
|
|
for h in g.topics[t].handler:
|
|
|
|
trace "calling handler for message", msg = msg.msgId,
|
|
|
|
topicId = t,
|
2019-12-07 16:36:39 +00:00
|
|
|
localPeer = g.peerInfo.id,
|
2019-12-06 02:16:18 +00:00
|
|
|
fromPeer = msg.fromPeerId().pretty
|
|
|
|
await h(t, msg.data) # trigger user provided handler
|
|
|
|
|
|
|
|
# forward the message to all peers interested in it
|
|
|
|
for p in toSendPeers:
|
2020-01-07 08:04:02 +00:00
|
|
|
if p in g.peers:
|
2019-12-10 20:50:35 +00:00
|
|
|
let id = g.peers[p].peerInfo.peerId
|
2020-05-21 20:24:20 +00:00
|
|
|
trace "about to forward message to peer", peerId = id, msgs = m.messages
|
2020-01-07 08:04:02 +00:00
|
|
|
|
2020-05-21 20:24:20 +00:00
|
|
|
if id == peer.peerInfo.peerId:
|
|
|
|
trace "not forwarding message to originator", peerId = id
|
|
|
|
continue
|
2020-01-07 08:04:02 +00:00
|
|
|
|
2020-05-21 20:24:20 +00:00
|
|
|
let msgs = m.messages.filterIt(
|
|
|
|
# don't forward to message originator
|
|
|
|
id != it.fromPeerId()
|
|
|
|
)
|
|
|
|
|
|
|
|
var sent: seq[Future[void]]
|
|
|
|
if msgs.len > 0:
|
|
|
|
trace "forwarding message to", peerId = id
|
|
|
|
sent.add(g.peers[p].send(@[RPCMsg(messages: msgs)]))
|
|
|
|
sent = await allFinished(sent)
|
|
|
|
checkFutures(sent)
|
2019-12-06 02:16:18 +00:00
|
|
|
|
|
|
|
var respControl: ControlMessage
|
|
|
|
if m.control.isSome:
|
|
|
|
var control: ControlMessage = m.control.get()
|
|
|
|
let iWant: ControlIWant = g.handleIHave(peer, control.ihave)
|
|
|
|
if iWant.messageIDs.len > 0:
|
|
|
|
respControl.iwant.add(iWant)
|
|
|
|
let messages: seq[Message] = g.handleIWant(peer, control.iwant)
|
|
|
|
|
|
|
|
g.handleGraft(peer, control.graft, respControl)
|
|
|
|
g.handlePrune(peer, control.prune)
|
|
|
|
|
|
|
|
if respControl.graft.len > 0 or respControl.prune.len > 0 or
|
|
|
|
respControl.ihave.len > 0 or respControl.iwant.len > 0:
|
2019-12-17 05:24:03 +00:00
|
|
|
await peer.send(@[RPCMsg(control: some(respControl), messages: messages)])
|
2019-12-06 02:16:18 +00:00
|
|
|
|
|
|
|
method subscribe*(g: GossipSub,
|
|
|
|
topic: string,
|
2019-12-17 05:24:03 +00:00
|
|
|
handler: TopicHandler) {.async.} =
|
2019-12-06 02:16:18 +00:00
|
|
|
await procCall PubSub(g).subscribe(topic, handler)
|
2020-05-21 20:24:20 +00:00
|
|
|
await g.rebalanceMesh(topic)
|
2019-12-06 02:16:18 +00:00
|
|
|
|
|
|
|
method unsubscribe*(g: GossipSub,
|
2019-12-17 05:24:03 +00:00
|
|
|
topics: seq[TopicPair]) {.async.} =
|
2019-12-06 02:16:18 +00:00
|
|
|
await procCall PubSub(g).unsubscribe(topics)
|
|
|
|
|
|
|
|
for pair in topics:
|
|
|
|
let topic = pair.topic
|
|
|
|
if topic in g.mesh:
|
|
|
|
let peers = g.mesh[topic]
|
|
|
|
g.mesh.del(topic)
|
|
|
|
for id in peers:
|
|
|
|
let p = g.peers[id]
|
|
|
|
await p.sendPrune(@[topic])
|
|
|
|
|
|
|
|
method publish*(g: GossipSub,
|
|
|
|
topic: string,
|
2019-12-17 05:24:03 +00:00
|
|
|
data: seq[byte]) {.async.} =
|
2019-12-06 02:16:18 +00:00
|
|
|
await procCall PubSub(g).publish(topic, data)
|
2019-12-17 05:24:03 +00:00
|
|
|
trace "about to publish message on topic", name = topic,
|
2020-05-18 20:49:49 +00:00
|
|
|
data = data.shortLog
|
2020-05-27 18:33:49 +00:00
|
|
|
|
|
|
|
var peers: HashSet[string]
|
2019-12-06 02:16:18 +00:00
|
|
|
if data.len > 0 and topic.len > 0:
|
2020-05-27 18:33:49 +00:00
|
|
|
for _ in 0..<5: # try to get peers up to 5 times
|
|
|
|
if peers.len > 0:
|
|
|
|
break
|
|
|
|
|
|
|
|
if topic in g.topics: # if we're subscribed to the topic attempt to build a mesh
|
|
|
|
await g.rebalanceMesh(topic)
|
|
|
|
peers = g.mesh[topic]
|
|
|
|
else: # send to fanout peers
|
|
|
|
await g.replenishFanout(topic)
|
|
|
|
if topic in g.fanout:
|
|
|
|
peers = g.fanout[topic]
|
|
|
|
# set the fanout expiry time
|
|
|
|
g.lastFanoutPubSub[topic] = Moment.fromNow(GossipSubFanoutTTL)
|
|
|
|
|
|
|
|
# wait a second between tries
|
|
|
|
await sleepAsync(1.seconds)
|
2019-12-06 02:16:18 +00:00
|
|
|
|
2020-05-06 09:26:08 +00:00
|
|
|
let msg = newMessage(g.peerInfo, data, topic, g.sign)
|
2020-01-07 08:04:02 +00:00
|
|
|
var sent: seq[Future[void]]
|
2019-12-06 02:16:18 +00:00
|
|
|
for p in peers:
|
2019-12-07 16:36:39 +00:00
|
|
|
if p == g.peerInfo.id:
|
2019-12-06 02:16:18 +00:00
|
|
|
continue
|
|
|
|
|
|
|
|
trace "publishing on topic", name = topic
|
|
|
|
g.mcache.put(msg)
|
2020-01-07 08:04:02 +00:00
|
|
|
sent.add(g.peers[p].send(@[RPCMsg(messages: @[msg])]))
|
2020-05-26 06:17:25 +00:00
|
|
|
checkFutures(await allFinished(sent))
|
2019-12-06 02:16:18 +00:00
|
|
|
|
|
|
|
method start*(g: GossipSub) {.async.} =
|
|
|
|
## start pubsub
|
|
|
|
## start long running/repeating procedures
|
|
|
|
|
|
|
|
# setup the heartbeat interval
|
2020-05-27 18:33:49 +00:00
|
|
|
g.heartbeatCancel = g.heartbeat()
|
2019-12-06 02:16:18 +00:00
|
|
|
|
|
|
|
method stop*(g: GossipSub) {.async.} =
|
|
|
|
## stopt pubsub
|
|
|
|
## stop long running tasks
|
|
|
|
|
|
|
|
await g.heartbeatLock.acquire()
|
|
|
|
|
|
|
|
# stop heartbeat interval
|
|
|
|
if not g.heartbeatCancel.finished:
|
|
|
|
g.heartbeatCancel.complete()
|
|
|
|
|
|
|
|
g.heartbeatLock.release()
|
|
|
|
|
2020-05-29 15:35:03 +00:00
|
|
|
method initPubSub*(g: GossipSub) =
|
2019-12-06 02:16:18 +00:00
|
|
|
procCall FloodSub(g).initPubSub()
|
|
|
|
|
2020-05-21 20:24:20 +00:00
|
|
|
randomize()
|
2019-12-06 02:16:18 +00:00
|
|
|
g.mcache = newMCache(GossipSubHistoryGossip, GossipSubHistoryLength)
|
2020-06-11 18:09:34 +00:00
|
|
|
g.mesh = initTable[string, HashSet[string]]() # meshes - topic to peer
|
|
|
|
g.fanout = initTable[string, HashSet[string]]() # fanout - topic to peer
|
|
|
|
g.gossipsub = initTable[string, HashSet[string]]()# topic to peer map of all gossipsub peers
|
|
|
|
g.lastFanoutPubSub = initTable[string, Moment]() # last publish time for fanout topics
|
2019-12-06 02:16:18 +00:00
|
|
|
g.gossip = initTable[string, seq[ControlIHave]]() # pending gossip
|
2020-06-11 18:09:34 +00:00
|
|
|
g.control = initTable[string, ControlMessage]() # pending control messages
|
2019-12-06 02:16:18 +00:00
|
|
|
g.heartbeatLock = newAsyncLock()
|