mirror of
https://github.com/codex-storage/nim-libp2p.git
synced 2025-01-12 20:14:09 +00:00
b99d2039a8
* allow multiple codecs per protocol (without breaking things) * add 1.1 protocol to gossip * explicit peering part 1 * explicit peering part 2 * explicit peering part 3 * PeerInfo and ControlPrune protocols * fix encodePrune * validated always, even explicit peers * prune by score (score is stub still) * add a way to pass parameters to gossip * standard setup fixes * take into account explicit direct peers in publish * add floodPublish logic * small fixes, publish still half broken * make sure to waitsub in sparse test * use var semantics to optimize table access * wip... lvalues don't work properly sadly... * big publish refactor, replenish and balance * fix internal tests * use g.peers for fanout (todo: don't include flood peers) * exclude non gossip from fanout * internal test fixes * fix flood tests * fix test's trypublish * test interop fixes * make sure to not remove peers from gossip table * restore old replenishFanout * cleanups * restore utility module import * restore trace vs debug in gossip * improve fanout replenish behavior further * triage publish nil peers (issue is on master too but just hidden behind a if/in) * getGossipPeers fixes * remove topics from pubsubpeer (was unused) * simplify rebalanceMesh (following spec) and make it finally reach D_high * better diagnostics * merge new pubsubpeer, copy 1.1 to new module * fix up merge * conditional enable gossip11 module * add back topics in peers, re-enable flood publish * add more heartbeat locking to prevent races * actually lock the heartbeat * minor fixes * with sugar * merge 1.0 * remove assertion in publish * fix multistream 1.1 multi proto * Fix merge oops * wip * fix gossip 11 upstream * gossipsub11 -> gossipsub * support interop testing * tests fixing * fix directchat build * control prune updates (pb) * wip parameters * gossip internal tests fixes * parameters wip * finishup with params * cleanups/wip * small sugar * grafted and pruned procs * wip updateScores * wip * fix logging issue * pubsubpeer, chronicles explicit override * fix internal gossip tests * wip * tables troubleshooting * score wip * score wip * fixes * fix test utils generateNodes * don't delete while iterating in score update * fix grafted defect * add a handleConnect in subscribeTopic * pruning improvements * wip * score fixes * post merge - builds gossip tests * further merge fixes * rebalance improvements and opportunistic grafting * fix test for now * restore explicit peering * implement peer exchange graft message * add an hard cap to PX * backoff time management * IWANT cap/budget * Adaptive gossip dissemination * outbound mesh quota, internal tests fixing * oversub prune score based, finish outbound quota * finishup with score and ihave budget * use go daemon 0.3.0 * import fixes * byScore cleanup score sorting * remove pointless scaling in `/` Duration operator * revert using libp2p org for daemon * interop fixes * fixes and cleanup * remove heartbeat assertion, minor debug fixes * logging improvements and cleaning up * (to revert) add some traces * add explicit topic to gossip rpcs * pubsub merge fixes and type fix in switch * Revert "(to revert) add some traces" This reverts commit 4663eaab6cc336c81cee50bc54025cf0b7bcbd99. * cleanup some now irrelevant todo * shuffle peers anyway as score might be disabled * add missing shuffle * old merge fix * more merge fixes * debug improvements * re-enable gossip internal tests * add gossip10 fallback (dormant but tested) * split gossipsub internal tests into 1.0 and 1.1 Co-authored-by: Dmitriy Ryajov <dryajov@gmail.com>
641 lines
18 KiB
Nim
641 lines
18 KiB
Nim
## Nim-Libp2p
|
|
## Copyright (c) 2018 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.
|
|
|
|
{.used.}
|
|
|
|
import unittest, sequtils, options, tables, sets
|
|
import chronos, stew/byteutils
|
|
import chronicles
|
|
import utils, ../../libp2p/[errors,
|
|
peerid,
|
|
peerinfo,
|
|
stream/connection,
|
|
stream/bufferstream,
|
|
crypto/crypto,
|
|
protocols/pubsub/pubsub,
|
|
protocols/pubsub/pubsubpeer,
|
|
protocols/pubsub/peertable,
|
|
protocols/pubsub/rpc/messages]
|
|
|
|
when defined(fallback_gossipsub_10):
|
|
import ../../libp2p/protocols/pubsub/gossipsub10
|
|
else:
|
|
import ../../libp2p/protocols/pubsub/gossipsub
|
|
|
|
import ../helpers
|
|
|
|
proc waitSub(sender, receiver: auto; key: string) {.async, gcsafe.} =
|
|
if sender == receiver:
|
|
return
|
|
# turn things deterministic
|
|
# this is for testing purposes only
|
|
# peers can be inside `mesh` and `fanout`, not just `gossipsub`
|
|
var ceil = 15
|
|
let fsub = GossipSub(sender)
|
|
let ev = newAsyncEvent()
|
|
fsub.heartbeatEvents.add(ev)
|
|
|
|
# await first heartbeat
|
|
await ev.wait()
|
|
ev.clear()
|
|
|
|
while (not fsub.gossipsub.hasKey(key) or
|
|
not fsub.gossipsub.hasPeerID(key, receiver.peerInfo.peerId)) and
|
|
(not fsub.mesh.hasKey(key) or
|
|
not fsub.mesh.hasPeerID(key, receiver.peerInfo.peerId)) and
|
|
(not fsub.fanout.hasKey(key) or
|
|
not fsub.fanout.hasPeerID(key , receiver.peerInfo.peerId)):
|
|
trace "waitSub sleeping..."
|
|
|
|
# await more heartbeats
|
|
await ev.wait()
|
|
ev.clear()
|
|
|
|
dec ceil
|
|
doAssert(ceil > 0, "waitSub timeout!")
|
|
|
|
template tryPublish(call: untyped, require: int, wait: Duration = 1.seconds, times: int = 10): untyped =
|
|
var
|
|
limit = times
|
|
pubs = 0
|
|
while pubs < require and limit > 0:
|
|
pubs = pubs + call
|
|
await sleepAsync(wait)
|
|
limit.dec()
|
|
if limit == 0:
|
|
doAssert(false, "Failed to publish!")
|
|
|
|
suite "GossipSub":
|
|
teardown:
|
|
for tracker in testTrackers():
|
|
# echo tracker.dump()
|
|
check tracker.isLeaked() == false
|
|
|
|
test "GossipSub validation should succeed":
|
|
proc runTests() {.async.} =
|
|
var handlerFut = newFuture[bool]()
|
|
proc handler(topic: string, data: seq[byte]) {.async, gcsafe.} =
|
|
check topic == "foobar"
|
|
handlerFut.complete(true)
|
|
|
|
let
|
|
nodes = generateNodes(2, gossip = true)
|
|
|
|
# start switches
|
|
nodesFut = await allFinished(
|
|
nodes[0].switch.start(),
|
|
nodes[1].switch.start(),
|
|
)
|
|
|
|
# start pubsub
|
|
await allFuturesThrowing(
|
|
allFinished(
|
|
nodes[0].start(),
|
|
nodes[1].start(),
|
|
))
|
|
|
|
await subscribeNodes(nodes)
|
|
|
|
await nodes[0].subscribe("foobar", handler)
|
|
await nodes[1].subscribe("foobar", handler)
|
|
|
|
var subs: seq[Future[void]]
|
|
subs &= waitSub(nodes[1], nodes[0], "foobar")
|
|
subs &= waitSub(nodes[0], nodes[1], "foobar")
|
|
|
|
await allFuturesThrowing(subs)
|
|
|
|
var validatorFut = newFuture[bool]()
|
|
proc validator(topic: string,
|
|
message: Message):
|
|
Future[bool] {.async.} =
|
|
check topic == "foobar"
|
|
validatorFut.complete(true)
|
|
result = true
|
|
|
|
nodes[1].addValidator("foobar", validator)
|
|
tryPublish await nodes[0].publish("foobar", "Hello!".toBytes()), 1
|
|
|
|
check (await validatorFut) and (await handlerFut)
|
|
|
|
await allFuturesThrowing(
|
|
nodes[0].switch.stop(),
|
|
nodes[1].switch.stop()
|
|
)
|
|
|
|
await allFuturesThrowing(
|
|
nodes[0].stop(),
|
|
nodes[1].stop()
|
|
)
|
|
|
|
await allFuturesThrowing(nodesFut.concat())
|
|
|
|
waitFor(runTests())
|
|
|
|
test "GossipSub validation should fail":
|
|
proc runTests() {.async.} =
|
|
proc handler(topic: string, data: seq[byte]) {.async, gcsafe.} =
|
|
check false # if we get here, it should fail
|
|
|
|
let
|
|
nodes = generateNodes(2, gossip = true)
|
|
|
|
# start switches
|
|
nodesFut = await allFinished(
|
|
nodes[0].switch.start(),
|
|
nodes[1].switch.start(),
|
|
)
|
|
|
|
# start pubsub
|
|
await allFuturesThrowing(
|
|
allFinished(
|
|
nodes[0].start(),
|
|
nodes[1].start(),
|
|
))
|
|
|
|
await subscribeNodes(nodes)
|
|
|
|
await nodes[0].subscribe("foobar", handler)
|
|
await nodes[1].subscribe("foobar", handler)
|
|
|
|
var subs: seq[Future[void]]
|
|
subs &= waitSub(nodes[1], nodes[0], "foobar")
|
|
subs &= waitSub(nodes[0], nodes[1], "foobar")
|
|
|
|
await allFuturesThrowing(subs)
|
|
|
|
let gossip1 = GossipSub(nodes[0])
|
|
let gossip2 = GossipSub(nodes[1])
|
|
|
|
check:
|
|
gossip1.mesh["foobar"].len == 1 and "foobar" notin gossip1.fanout
|
|
gossip2.mesh["foobar"].len == 1 and "foobar" notin gossip2.fanout
|
|
|
|
var validatorFut = newFuture[bool]()
|
|
proc validator(topic: string,
|
|
message: Message):
|
|
Future[bool] {.async.} =
|
|
result = false
|
|
validatorFut.complete(true)
|
|
|
|
nodes[1].addValidator("foobar", validator)
|
|
tryPublish await nodes[0].publish("foobar", "Hello!".toBytes()), 1
|
|
|
|
check (await validatorFut) == true
|
|
|
|
await allFuturesThrowing(
|
|
nodes[0].switch.stop(),
|
|
nodes[1].switch.stop()
|
|
)
|
|
|
|
await allFuturesThrowing(
|
|
nodes[0].stop(),
|
|
nodes[1].stop()
|
|
)
|
|
|
|
await allFuturesThrowing(nodesFut.concat())
|
|
|
|
waitFor(runTests())
|
|
|
|
test "GossipSub validation one fails and one succeeds":
|
|
proc runTests() {.async.} =
|
|
var handlerFut = newFuture[bool]()
|
|
proc handler(topic: string, data: seq[byte]) {.async, gcsafe.} =
|
|
check topic == "foo"
|
|
handlerFut.complete(true)
|
|
|
|
let
|
|
nodes = generateNodes(2, gossip = true)
|
|
|
|
# start switches
|
|
nodesFut = await allFinished(
|
|
nodes[0].switch.start(),
|
|
nodes[1].switch.start(),
|
|
)
|
|
|
|
# start pubsub
|
|
await allFuturesThrowing(
|
|
allFinished(
|
|
nodes[0].start(),
|
|
nodes[1].start(),
|
|
))
|
|
|
|
await subscribeNodes(nodes)
|
|
|
|
await nodes[1].subscribe("foo", handler)
|
|
await nodes[1].subscribe("bar", handler)
|
|
|
|
var passed, failed: Future[bool] = newFuture[bool]()
|
|
proc validator(topic: string,
|
|
message: Message):
|
|
Future[bool] {.async.} =
|
|
result = if topic == "foo":
|
|
passed.complete(true)
|
|
true
|
|
else:
|
|
failed.complete(true)
|
|
false
|
|
|
|
nodes[1].addValidator("foo", "bar", validator)
|
|
tryPublish await nodes[0].publish("foo", "Hello!".toBytes()), 1
|
|
tryPublish await nodes[0].publish("bar", "Hello!".toBytes()), 1
|
|
|
|
check ((await passed) and (await failed) and (await handlerFut))
|
|
|
|
let gossip1 = GossipSub(nodes[0])
|
|
let gossip2 = GossipSub(nodes[1])
|
|
|
|
check:
|
|
"foo" notin gossip1.mesh and gossip1.fanout["foo"].len == 1
|
|
"foo" notin gossip2.mesh and "foo" notin gossip2.fanout
|
|
"bar" notin gossip1.mesh and gossip1.fanout["bar"].len == 1
|
|
"bar" notin gossip2.mesh and "bar" notin gossip2.fanout
|
|
|
|
await allFuturesThrowing(
|
|
nodes[0].switch.stop(),
|
|
nodes[1].switch.stop()
|
|
)
|
|
|
|
await allFuturesThrowing(
|
|
nodes[0].stop(),
|
|
nodes[1].stop()
|
|
)
|
|
|
|
await allFuturesThrowing(nodesFut.concat())
|
|
|
|
waitFor(runTests())
|
|
|
|
test "e2e - GossipSub should add remote peer topic subscriptions":
|
|
proc testBasicGossipSub() {.async.} =
|
|
proc handler(topic: string, data: seq[byte]) {.async, gcsafe.} =
|
|
discard
|
|
|
|
let
|
|
nodes = generateNodes(
|
|
2,
|
|
gossip = true,
|
|
secureManagers = [SecureProtocol.Noise])
|
|
|
|
# start switches
|
|
nodesFut = await allFinished(
|
|
nodes[0].switch.start(),
|
|
nodes[1].switch.start(),
|
|
)
|
|
|
|
# start pubsub
|
|
await allFuturesThrowing(
|
|
allFinished(
|
|
nodes[0].start(),
|
|
nodes[1].start(),
|
|
))
|
|
|
|
await subscribeNodes(nodes)
|
|
|
|
await nodes[1].subscribe("foobar", handler)
|
|
await sleepAsync(10.seconds)
|
|
|
|
let gossip1 = GossipSub(nodes[0])
|
|
let gossip2 = GossipSub(nodes[1])
|
|
|
|
check:
|
|
"foobar" in gossip2.topics
|
|
"foobar" in gossip1.gossipsub
|
|
gossip1.gossipsub.hasPeerID("foobar", gossip2.peerInfo.peerId)
|
|
|
|
await allFuturesThrowing(
|
|
nodes[0].switch.stop(),
|
|
nodes[1].switch.stop()
|
|
)
|
|
|
|
await allFuturesThrowing(
|
|
nodes[0].stop(),
|
|
nodes[1].stop()
|
|
)
|
|
|
|
await allFuturesThrowing(nodesFut.concat())
|
|
|
|
waitFor(testBasicGossipSub())
|
|
|
|
test "e2e - GossipSub should add remote peer topic subscriptions if both peers are subscribed":
|
|
proc testBasicGossipSub() {.async.} =
|
|
proc handler(topic: string, data: seq[byte]) {.async, gcsafe.} =
|
|
discard
|
|
|
|
let
|
|
nodes = generateNodes(
|
|
2,
|
|
gossip = true,
|
|
secureManagers = [SecureProtocol.Secio])
|
|
|
|
# start switches
|
|
nodesFut = await allFinished(
|
|
nodes[0].switch.start(),
|
|
nodes[1].switch.start(),
|
|
)
|
|
|
|
# start pubsub
|
|
await allFuturesThrowing(
|
|
allFinished(
|
|
nodes[0].start(),
|
|
nodes[1].start(),
|
|
))
|
|
|
|
await subscribeNodes(nodes)
|
|
|
|
await nodes[0].subscribe("foobar", handler)
|
|
await nodes[1].subscribe("foobar", handler)
|
|
|
|
var subs: seq[Future[void]]
|
|
subs &= waitSub(nodes[1], nodes[0], "foobar")
|
|
subs &= waitSub(nodes[0], nodes[1], "foobar")
|
|
|
|
await allFuturesThrowing(subs)
|
|
|
|
let
|
|
gossip1 = GossipSub(nodes[0])
|
|
gossip2 = GossipSub(nodes[1])
|
|
|
|
check:
|
|
"foobar" in gossip1.topics
|
|
"foobar" in gossip2.topics
|
|
|
|
"foobar" in gossip1.gossipsub
|
|
"foobar" in gossip2.gossipsub
|
|
|
|
gossip1.gossipsub.hasPeerID("foobar", gossip2.peerInfo.peerId) or
|
|
gossip1.mesh.hasPeerID("foobar", gossip2.peerInfo.peerId)
|
|
|
|
gossip2.gossipsub.hasPeerID("foobar", gossip1.peerInfo.peerId) or
|
|
gossip2.mesh.hasPeerID("foobar", gossip1.peerInfo.peerId)
|
|
|
|
await allFuturesThrowing(
|
|
nodes[0].switch.stop(),
|
|
nodes[1].switch.stop()
|
|
)
|
|
|
|
await allFuturesThrowing(
|
|
nodes[0].stop(),
|
|
nodes[1].stop()
|
|
)
|
|
|
|
await allFuturesThrowing(nodesFut.concat())
|
|
|
|
waitFor(testBasicGossipSub())
|
|
|
|
test "e2e - GossipSub send over fanout A -> B":
|
|
proc runTests() {.async.} =
|
|
var passed = newFuture[void]()
|
|
proc handler(topic: string, data: seq[byte]) {.async, gcsafe.} =
|
|
check topic == "foobar"
|
|
passed.complete()
|
|
|
|
let
|
|
nodes = generateNodes(
|
|
2,
|
|
gossip = true,
|
|
secureManagers = [SecureProtocol.Secio])
|
|
|
|
# start switches
|
|
nodesFut = await allFinished(
|
|
nodes[0].switch.start(),
|
|
nodes[1].switch.start(),
|
|
)
|
|
|
|
# start pubsub
|
|
await allFuturesThrowing(
|
|
allFinished(
|
|
nodes[0].start(),
|
|
nodes[1].start(),
|
|
))
|
|
|
|
await subscribeNodes(nodes)
|
|
|
|
await nodes[1].subscribe("foobar", handler)
|
|
await waitSub(nodes[0], nodes[1], "foobar")
|
|
|
|
var observed = 0
|
|
let
|
|
obs1 = PubSubObserver(onRecv: proc(peer: PubSubPeer; msgs: var RPCMsg) =
|
|
inc observed
|
|
)
|
|
obs2 = PubSubObserver(onSend: proc(peer: PubSubPeer; msgs: var RPCMsg) =
|
|
inc observed
|
|
)
|
|
|
|
# nodes[1].addObserver(obs1)
|
|
# nodes[0].addObserver(obs2)
|
|
|
|
tryPublish await nodes[0].publish("foobar", "Hello!".toBytes()), 1
|
|
|
|
var gossip1: GossipSub = GossipSub(nodes[0])
|
|
var gossip2: GossipSub = GossipSub(nodes[1])
|
|
|
|
check:
|
|
"foobar" in gossip1.gossipsub
|
|
gossip1.fanout.hasPeerID("foobar", gossip2.peerInfo.peerId)
|
|
not gossip1.mesh.hasPeerID("foobar", gossip2.peerInfo.peerId)
|
|
|
|
await passed.wait(2.seconds)
|
|
|
|
trace "test done, stopping..."
|
|
|
|
await nodes[0].stop()
|
|
await nodes[1].stop()
|
|
|
|
await allFuturesThrowing(
|
|
nodes[0].switch.stop(),
|
|
nodes[1].switch.stop()
|
|
)
|
|
|
|
await allFuturesThrowing(
|
|
nodes[0].stop(),
|
|
nodes[1].stop()
|
|
)
|
|
|
|
await allFuturesThrowing(nodesFut.concat())
|
|
# check observed == 2
|
|
|
|
waitFor(runTests())
|
|
|
|
test "e2e - GossipSub send over mesh A -> B":
|
|
proc runTests(): Future[bool] {.async.} =
|
|
var passed: Future[bool] = newFuture[bool]()
|
|
proc handler(topic: string, data: seq[byte]) {.async, gcsafe.} =
|
|
check topic == "foobar"
|
|
passed.complete(true)
|
|
|
|
let
|
|
nodes = generateNodes(
|
|
2,
|
|
gossip = true,
|
|
secureManagers = [SecureProtocol.Secio])
|
|
|
|
# start switches
|
|
nodesFut = await allFinished(
|
|
nodes[0].switch.start(),
|
|
nodes[1].switch.start(),
|
|
)
|
|
|
|
# start pubsub
|
|
await allFuturesThrowing(
|
|
allFinished(
|
|
nodes[0].start(),
|
|
nodes[1].start(),
|
|
))
|
|
|
|
await subscribeNodes(nodes)
|
|
|
|
await nodes[0].subscribe("foobar", handler)
|
|
await nodes[1].subscribe("foobar", handler)
|
|
await waitSub(nodes[0], nodes[1], "foobar")
|
|
|
|
tryPublish await nodes[0].publish("foobar", "Hello!".toBytes()), 1
|
|
|
|
result = await passed
|
|
|
|
var gossip1: GossipSub = GossipSub(nodes[0])
|
|
var gossip2: GossipSub = GossipSub(nodes[1])
|
|
|
|
check:
|
|
"foobar" in gossip1.gossipsub
|
|
"foobar" in gossip2.gossipsub
|
|
gossip1.mesh.hasPeerID("foobar", gossip2.peerInfo.peerId)
|
|
not gossip1.fanout.hasPeerID("foobar", gossip2.peerInfo.peerId)
|
|
gossip2.mesh.hasPeerID("foobar", gossip1.peerInfo.peerId)
|
|
not gossip2.fanout.hasPeerID("foobar", gossip1.peerInfo.peerId)
|
|
|
|
await allFuturesThrowing(
|
|
nodes[0].switch.stop(),
|
|
nodes[1].switch.stop()
|
|
)
|
|
|
|
await allFuturesThrowing(
|
|
nodes[0].stop(),
|
|
nodes[1].stop()
|
|
)
|
|
|
|
await allFuturesThrowing(nodesFut.concat())
|
|
|
|
check:
|
|
waitFor(runTests()) == true
|
|
|
|
test "e2e - GossipSub with multiple peers":
|
|
proc runTests() {.async.} =
|
|
var runs = 10
|
|
|
|
let
|
|
nodes = generateNodes(runs, gossip = true, triggerSelf = true)
|
|
nodesFut = nodes.mapIt(it.switch.start())
|
|
|
|
await allFuturesThrowing(nodes.mapIt(it.start()))
|
|
await subscribeNodes(nodes)
|
|
|
|
var seen: Table[string, int]
|
|
var subs: seq[Future[void]]
|
|
var seenFut = newFuture[void]()
|
|
for dialer in nodes:
|
|
var handler: TopicHandler
|
|
closureScope:
|
|
var dialerNode = dialer
|
|
handler = proc(topic: string, data: seq[byte]) {.async, gcsafe, closure.} =
|
|
if $dialerNode.peerInfo.peerId notin seen:
|
|
seen[$dialerNode.peerInfo.peerId] = 0
|
|
seen[$dialerNode.peerInfo.peerId].inc
|
|
check topic == "foobar"
|
|
if not seenFut.finished() and seen.len >= runs:
|
|
seenFut.complete()
|
|
|
|
subs &= dialer.subscribe("foobar", handler)
|
|
|
|
await allFuturesThrowing(subs).wait(30.seconds)
|
|
|
|
tryPublish await wait(nodes[0].publish("foobar",
|
|
toBytes("from node " &
|
|
$nodes[1].peerInfo.peerId)),
|
|
1.minutes), runs, 5.seconds
|
|
|
|
await wait(seenFut, 2.minutes)
|
|
check: seen.len >= runs
|
|
for k, v in seen.pairs:
|
|
check: v >= 1
|
|
|
|
for node in nodes:
|
|
var gossip = GossipSub(node)
|
|
|
|
check:
|
|
"foobar" in gossip.gossipsub
|
|
|
|
await allFuturesThrowing(
|
|
nodes.mapIt(
|
|
allFutures(
|
|
it.stop(),
|
|
it.switch.stop())))
|
|
|
|
await allFuturesThrowing(nodesFut)
|
|
|
|
waitFor(runTests())
|
|
|
|
test "e2e - GossipSub with multiple peers (sparse)":
|
|
proc runTests() {.async.} =
|
|
var runs = 10
|
|
|
|
let
|
|
nodes = generateNodes(runs, gossip = true, triggerSelf = true)
|
|
nodesFut = nodes.mapIt(it.switch.start())
|
|
|
|
await allFuturesThrowing(nodes.mapIt(it.start()))
|
|
await subscribeNodes(nodes)
|
|
|
|
var seen: Table[PeerID, int]
|
|
var subs: seq[Future[void]]
|
|
var seenFut = newFuture[void]()
|
|
for dialer in nodes:
|
|
var handler: TopicHandler
|
|
closureScope:
|
|
var dialerNode = dialer
|
|
handler = proc(topic: string, data: seq[byte])
|
|
{.async, gcsafe, closure.} =
|
|
if dialerNode.peerInfo.peerId notin seen:
|
|
seen[dialerNode.peerInfo.peerId] = 0
|
|
seen[dialerNode.peerInfo.peerId].inc
|
|
check topic == "foobar"
|
|
if not seenFut.finished() and seen.len >= runs:
|
|
seenFut.complete()
|
|
|
|
subs &= dialer.subscribe("foobar", handler)
|
|
subs &= waitSub(nodes[0], dialer, "foobar")
|
|
|
|
await allFuturesThrowing(subs)
|
|
tryPublish await wait(nodes[0].publish("foobar",
|
|
toBytes("from node " &
|
|
$nodes[1].peerInfo.peerId)),
|
|
1.minutes), 2, 5.seconds
|
|
|
|
await wait(seenFut, 5.minutes)
|
|
check: seen.len >= runs
|
|
for k, v in seen.pairs:
|
|
check: v >= 1
|
|
|
|
for node in nodes:
|
|
var gossip = GossipSub(node)
|
|
check:
|
|
"foobar" in gossip.gossipsub
|
|
gossip.fanout.len == 0
|
|
gossip.mesh["foobar"].len > 0
|
|
|
|
await allFuturesThrowing(
|
|
nodes.mapIt(
|
|
allFutures(
|
|
it.stop(),
|
|
it.switch.stop())))
|
|
|
|
await allFuturesThrowing(nodesFut)
|
|
|
|
waitFor(runTests())
|