Pubsub lifetime (#284)
* lifecycle hooks * tests * move trace after closed check * restore 1 second heartbeat * await close event * fix tests * print direction string * more trace logging * add pubsub monitor * add log scope * adjust idle timeout * add exc.msg to trace
This commit is contained in:
parent
ed0df74bbd
commit
f7fdf31365
|
@ -174,12 +174,12 @@ method reset*(s: LPChannel) {.base, async, gcsafe.} =
|
||||||
peer = $s.conn.peerInfo
|
peer = $s.conn.peerInfo
|
||||||
# stack = getStackTrace()
|
# stack = getStackTrace()
|
||||||
|
|
||||||
trace "resetting channel"
|
|
||||||
|
|
||||||
if s.closedLocal and s.isEof:
|
if s.closedLocal and s.isEof:
|
||||||
trace "channel already closed or reset"
|
trace "channel already closed or reset"
|
||||||
return
|
return
|
||||||
|
|
||||||
|
trace "resetting channel"
|
||||||
|
|
||||||
# we asyncCheck here because the other end
|
# we asyncCheck here because the other end
|
||||||
# might be dead already - reset is always
|
# might be dead already - reset is always
|
||||||
# optimistic
|
# optimistic
|
||||||
|
@ -227,7 +227,7 @@ method close*(s: LPChannel) {.async, gcsafe.} =
|
||||||
await s.reset()
|
await s.reset()
|
||||||
raise exc
|
raise exc
|
||||||
except CatchableError as exc:
|
except CatchableError as exc:
|
||||||
trace "exception closing channel"
|
trace "exception closing channel", exc = exc.msg
|
||||||
await s.reset()
|
await s.reset()
|
||||||
|
|
||||||
trace "lpchannel closed local"
|
trace "lpchannel closed local"
|
||||||
|
@ -244,6 +244,13 @@ proc timeoutMonitor(s: LPChannel) {.async.} =
|
||||||
## be reset
|
## be reset
|
||||||
##
|
##
|
||||||
|
|
||||||
|
logScope:
|
||||||
|
id = s.id
|
||||||
|
initiator = s.initiator
|
||||||
|
name = s.name
|
||||||
|
oid = $s.oid
|
||||||
|
peer = $s.conn.peerInfo
|
||||||
|
|
||||||
try:
|
try:
|
||||||
while true:
|
while true:
|
||||||
await sleepAsync(s.timeout)
|
await sleepAsync(s.timeout)
|
||||||
|
|
|
@ -39,7 +39,7 @@ const GossipSubHistoryGossip* = 3
|
||||||
|
|
||||||
# heartbeat interval
|
# heartbeat interval
|
||||||
const GossipSubHeartbeatInitialDelay* = 100.millis
|
const GossipSubHeartbeatInitialDelay* = 100.millis
|
||||||
const GossipSubHeartbeatInterval* = 5.seconds # TODO: per the spec it should be 1 second
|
const GossipSubHeartbeatInterval* = 1.seconds
|
||||||
|
|
||||||
# fanout ttl
|
# fanout ttl
|
||||||
const GossipSubFanoutTTL* = 1.minutes
|
const GossipSubFanoutTTL* = 1.minutes
|
||||||
|
|
|
@ -242,17 +242,19 @@ method unsubscribe*(p: PubSub,
|
||||||
if p.topics[t.topic].handler.len <= 0:
|
if p.topics[t.topic].handler.len <= 0:
|
||||||
p.topics.del(t.topic)
|
p.topics.del(t.topic)
|
||||||
# metrics
|
# metrics
|
||||||
libp2p_pubsub_topics.dec()
|
libp2p_pubsub_topics.set(p.topics.len.int64)
|
||||||
|
|
||||||
proc unsubscribe*(p: PubSub,
|
proc unsubscribe*(p: PubSub,
|
||||||
topic: string,
|
topic: string,
|
||||||
handler: TopicHandler): Future[void] =
|
handler: TopicHandler): Future[void] =
|
||||||
## unsubscribe from a ``topic`` string
|
## unsubscribe from a ``topic`` string
|
||||||
|
##
|
||||||
|
|
||||||
p.unsubscribe(@[(topic, handler)])
|
p.unsubscribe(@[(topic, handler)])
|
||||||
|
|
||||||
method unsubscribeAll*(p: PubSub, topic: string) {.base, async.} =
|
method unsubscribeAll*(p: PubSub, topic: string) {.base, async.} =
|
||||||
libp2p_pubsub_topics.dec()
|
|
||||||
p.topics.del(topic)
|
p.topics.del(topic)
|
||||||
|
libp2p_pubsub_topics.set(p.topics.len.int64)
|
||||||
|
|
||||||
method subscribe*(p: PubSub,
|
method subscribe*(p: PubSub,
|
||||||
topic: string,
|
topic: string,
|
||||||
|
@ -278,7 +280,7 @@ method subscribe*(p: PubSub,
|
||||||
checkFutures(await allFinished(sent))
|
checkFutures(await allFinished(sent))
|
||||||
|
|
||||||
# metrics
|
# metrics
|
||||||
libp2p_pubsub_topics.inc()
|
libp2p_pubsub_topics.set(p.topics.len.int64)
|
||||||
|
|
||||||
proc sendHelper*(p: PubSub,
|
proc sendHelper*(p: PubSub,
|
||||||
sendPeers: HashSet[PubSubPeer],
|
sendPeers: HashSet[PubSubPeer],
|
||||||
|
|
|
@ -38,8 +38,8 @@ proc newStandardSwitch*(privKey = none(PrivateKey),
|
||||||
transportFlags: set[ServerFlags] = {},
|
transportFlags: set[ServerFlags] = {},
|
||||||
msgIdProvider: MsgIdProvider = defaultMsgIdProvider,
|
msgIdProvider: MsgIdProvider = defaultMsgIdProvider,
|
||||||
rng = newRng(),
|
rng = newRng(),
|
||||||
inTimeout: Duration = 1.minutes,
|
inTimeout: Duration = 5.minutes,
|
||||||
outTimeout: Duration = 1.minutes): Switch =
|
outTimeout: Duration = 5.minutes): Switch =
|
||||||
proc createMplex(conn: Connection): Muxer =
|
proc createMplex(conn: Connection): Muxer =
|
||||||
Mplex.init(
|
Mplex.init(
|
||||||
conn,
|
conn,
|
||||||
|
|
|
@ -44,6 +44,9 @@ declareCounter(libp2p_dialed_peers, "dialed peers")
|
||||||
declareCounter(libp2p_failed_dials, "failed dials")
|
declareCounter(libp2p_failed_dials, "failed dials")
|
||||||
declareCounter(libp2p_failed_upgrade, "peers failed upgrade")
|
declareCounter(libp2p_failed_upgrade, "peers failed upgrade")
|
||||||
|
|
||||||
|
const
|
||||||
|
MaxPubsubReconnectAttempts* = 10
|
||||||
|
|
||||||
type
|
type
|
||||||
NoPubSubException* = object of CatchableError
|
NoPubSubException* = object of CatchableError
|
||||||
|
|
||||||
|
@ -67,6 +70,7 @@ type
|
||||||
pubSub*: Option[PubSub]
|
pubSub*: Option[PubSub]
|
||||||
dialLock: Table[string, AsyncLock]
|
dialLock: Table[string, AsyncLock]
|
||||||
hooks: Table[Lifecycle, HashSet[Hook]]
|
hooks: Table[Lifecycle, HashSet[Hook]]
|
||||||
|
pubsubMonitors: Table[PeerId, Future[void]]
|
||||||
|
|
||||||
proc newNoPubSubException(): ref NoPubSubException {.inline.} =
|
proc newNoPubSubException(): ref NoPubSubException {.inline.} =
|
||||||
result = newException(NoPubSubException, "no pubsub provided!")
|
result = newException(NoPubSubException, "no pubsub provided!")
|
||||||
|
@ -95,9 +99,18 @@ proc disconnect*(s: Switch, peer: PeerInfo) {.async, gcsafe.}
|
||||||
proc subscribePeer*(s: Switch, peerInfo: PeerInfo) {.async, gcsafe.}
|
proc subscribePeer*(s: Switch, peerInfo: PeerInfo) {.async, gcsafe.}
|
||||||
|
|
||||||
proc cleanupPubSubPeer(s: Switch, conn: Connection) {.async.} =
|
proc cleanupPubSubPeer(s: Switch, conn: Connection) {.async.} =
|
||||||
await conn.closeEvent.wait()
|
try:
|
||||||
if s.pubSub.isSome:
|
await conn.closeEvent.wait()
|
||||||
await s.pubSub.get().unsubscribePeer(conn.peerInfo)
|
if s.pubSub.isSome:
|
||||||
|
let fut = s.pubsubMonitors.getOrDefault(conn.peerInfo.peerId)
|
||||||
|
if not(isNil(fut)) and not(fut.finished):
|
||||||
|
await fut.cancelAndWait()
|
||||||
|
|
||||||
|
await s.pubSub.get().unsubscribePeer(conn.peerInfo)
|
||||||
|
except CancelledError as exc:
|
||||||
|
raise exc
|
||||||
|
except CatchableError as exc:
|
||||||
|
trace "exception cleaning pubsub peer", exc = exc.msg
|
||||||
|
|
||||||
proc isConnected*(s: Switch, peer: PeerInfo): bool =
|
proc isConnected*(s: Switch, peer: PeerInfo): bool =
|
||||||
## returns true if the peer has one or more
|
## returns true if the peer has one or more
|
||||||
|
@ -322,7 +335,9 @@ proc internalConnect(s: Switch,
|
||||||
continue
|
continue
|
||||||
break
|
break
|
||||||
else:
|
else:
|
||||||
trace "Reusing existing connection", oid = $conn.oid, direction = conn.dir
|
trace "Reusing existing connection", oid = $conn.oid,
|
||||||
|
direction = $conn.dir,
|
||||||
|
peer = $conn.peerInfo
|
||||||
finally:
|
finally:
|
||||||
if lock.locked():
|
if lock.locked():
|
||||||
lock.release()
|
lock.release()
|
||||||
|
@ -344,6 +359,9 @@ proc internalConnect(s: Switch,
|
||||||
await s.subscribePeer(peer)
|
await s.subscribePeer(peer)
|
||||||
asyncCheck s.cleanupPubSubPeer(conn)
|
asyncCheck s.cleanupPubSubPeer(conn)
|
||||||
|
|
||||||
|
trace "got connection", oid = $conn.oid,
|
||||||
|
direction = $conn.dir,
|
||||||
|
peer = $conn.peerInfo
|
||||||
return conn
|
return conn
|
||||||
|
|
||||||
proc connect*(s: Switch, peer: PeerInfo) {.async.} =
|
proc connect*(s: Switch, peer: PeerInfo) {.async.} =
|
||||||
|
@ -379,7 +397,7 @@ proc dial*(s: Switch,
|
||||||
await cleanup()
|
await cleanup()
|
||||||
raise exc
|
raise exc
|
||||||
except CatchableError as exc:
|
except CatchableError as exc:
|
||||||
trace "error dialing"
|
trace "error dialing", exc = exc.msg
|
||||||
await cleanup()
|
await cleanup()
|
||||||
raise exc
|
raise exc
|
||||||
|
|
||||||
|
@ -399,7 +417,6 @@ proc start*(s: Switch): Future[seq[Future[void]]] {.async, gcsafe.} =
|
||||||
|
|
||||||
proc handle(conn: Connection): Future[void] {.async, closure, gcsafe.} =
|
proc handle(conn: Connection): Future[void] {.async, closure, gcsafe.} =
|
||||||
try:
|
try:
|
||||||
|
|
||||||
conn.closeEvent.wait()
|
conn.closeEvent.wait()
|
||||||
.addCallback do(udata: pointer):
|
.addCallback do(udata: pointer):
|
||||||
asyncCheck s.triggerHooks(
|
asyncCheck s.triggerHooks(
|
||||||
|
@ -451,7 +468,7 @@ proc stop*(s: Switch) {.async.} =
|
||||||
|
|
||||||
trace "switch stopped"
|
trace "switch stopped"
|
||||||
|
|
||||||
proc subscribePeer*(s: Switch, peerInfo: PeerInfo) {.async, gcsafe.} =
|
proc subscribePeerInternal(s: Switch, peerInfo: PeerInfo) {.async, gcsafe.} =
|
||||||
## Subscribe to pub sub peer
|
## Subscribe to pub sub peer
|
||||||
if s.pubSub.isSome and not(s.pubSub.get().connected(peerInfo)):
|
if s.pubSub.isSome and not(s.pubSub.get().connected(peerInfo)):
|
||||||
trace "about to subscribe to pubsub peer", peer = peerInfo.shortLog()
|
trace "about to subscribe to pubsub peer", peer = peerInfo.shortLog()
|
||||||
|
@ -468,7 +485,7 @@ proc subscribePeer*(s: Switch, peerInfo: PeerInfo) {.async, gcsafe.} =
|
||||||
return
|
return
|
||||||
|
|
||||||
s.pubSub.get().subscribePeer(stream)
|
s.pubSub.get().subscribePeer(stream)
|
||||||
|
await stream.closeEvent.wait()
|
||||||
except CancelledError as exc:
|
except CancelledError as exc:
|
||||||
if not(isNil(stream)):
|
if not(isNil(stream)):
|
||||||
await stream.close()
|
await stream.close()
|
||||||
|
@ -480,6 +497,35 @@ proc subscribePeer*(s: Switch, peerInfo: PeerInfo) {.async, gcsafe.} =
|
||||||
if not(isNil(stream)):
|
if not(isNil(stream)):
|
||||||
await stream.close()
|
await stream.close()
|
||||||
|
|
||||||
|
proc pubsubMonitor(switch: Switch, peer: PeerInfo) {.async.} =
|
||||||
|
## while peer connected maintain a
|
||||||
|
## pubsub connection as well
|
||||||
|
##
|
||||||
|
|
||||||
|
var tries = 0
|
||||||
|
var backoffFactor = 5 # up to ~10 mins
|
||||||
|
var backoff = 1.seconds
|
||||||
|
while switch.isConnected(peer) and
|
||||||
|
tries < MaxPubsubReconnectAttempts:
|
||||||
|
try:
|
||||||
|
debug "subscribing to pubsub peer", peer = $peer
|
||||||
|
await switch.subscribePeerInternal(peer)
|
||||||
|
except CancelledError as exc:
|
||||||
|
raise exc
|
||||||
|
except CatchableError as exc:
|
||||||
|
trace "exception in pubsub monitor", peer = $peer, exc = exc.msg
|
||||||
|
finally:
|
||||||
|
debug "awaiting backoff period before reconnecting", peer = $peer, backoff, tries
|
||||||
|
await sleepAsync(backoff) # allow the peer to cooldown
|
||||||
|
backoff = backoff * backoffFactor
|
||||||
|
tries.inc()
|
||||||
|
|
||||||
|
trace "exiting pubsub monitor", peer = $peer
|
||||||
|
|
||||||
|
proc subscribePeer*(s: Switch, peerInfo: PeerInfo) {.async, gcsafe.} =
|
||||||
|
if peerInfo.peerId notin s.pubsubMonitors:
|
||||||
|
s.pubsubMonitors[peerInfo.peerId] = s.pubsubMonitor(peerInfo)
|
||||||
|
|
||||||
proc subscribe*(s: Switch, topic: string,
|
proc subscribe*(s: Switch, topic: string,
|
||||||
handler: TopicHandler) {.async.} =
|
handler: TopicHandler) {.async.} =
|
||||||
## subscribe to a pubsub topic
|
## subscribe to a pubsub topic
|
||||||
|
|
|
@ -54,7 +54,7 @@ suite "FloodSub":
|
||||||
nodes[1].start()
|
nodes[1].start()
|
||||||
)
|
)
|
||||||
|
|
||||||
await subscribeNodes(nodes)
|
let subscribes = await subscribeNodes(nodes)
|
||||||
|
|
||||||
await nodes[1].subscribe("foobar", handler)
|
await nodes[1].subscribe("foobar", handler)
|
||||||
await waitSub(nodes[0], nodes[1], "foobar")
|
await waitSub(nodes[0], nodes[1], "foobar")
|
||||||
|
@ -69,6 +69,7 @@ suite "FloodSub":
|
||||||
)
|
)
|
||||||
|
|
||||||
await allFuturesThrowing(nodesFut.concat())
|
await allFuturesThrowing(nodesFut.concat())
|
||||||
|
await allFuturesThrowing(subscribes)
|
||||||
|
|
||||||
check:
|
check:
|
||||||
waitFor(runTests()) == true
|
waitFor(runTests()) == true
|
||||||
|
@ -85,7 +86,7 @@ suite "FloodSub":
|
||||||
awaiters.add((await nodes[0].start()))
|
awaiters.add((await nodes[0].start()))
|
||||||
awaiters.add((await nodes[1].start()))
|
awaiters.add((await nodes[1].start()))
|
||||||
|
|
||||||
await subscribeNodes(nodes)
|
let subscribes = await subscribeNodes(nodes)
|
||||||
|
|
||||||
await nodes[0].subscribe("foobar", handler)
|
await nodes[0].subscribe("foobar", handler)
|
||||||
await waitSub(nodes[1], nodes[0], "foobar")
|
await waitSub(nodes[1], nodes[0], "foobar")
|
||||||
|
@ -95,6 +96,8 @@ suite "FloodSub":
|
||||||
result = await completionFut.wait(5.seconds)
|
result = await completionFut.wait(5.seconds)
|
||||||
|
|
||||||
await allFuturesThrowing(nodes[0].stop(), nodes[1].stop())
|
await allFuturesThrowing(nodes[0].stop(), nodes[1].stop())
|
||||||
|
|
||||||
|
await allFuturesThrowing(subscribes)
|
||||||
await allFuturesThrowing(awaiters)
|
await allFuturesThrowing(awaiters)
|
||||||
|
|
||||||
check:
|
check:
|
||||||
|
@ -112,7 +115,7 @@ suite "FloodSub":
|
||||||
awaiters.add((await nodes[0].start()))
|
awaiters.add((await nodes[0].start()))
|
||||||
awaiters.add((await nodes[1].start()))
|
awaiters.add((await nodes[1].start()))
|
||||||
|
|
||||||
await subscribeNodes(nodes)
|
let subscribes = await subscribeNodes(nodes)
|
||||||
await nodes[1].subscribe("foobar", handler)
|
await nodes[1].subscribe("foobar", handler)
|
||||||
await waitSub(nodes[0], nodes[1], "foobar")
|
await waitSub(nodes[0], nodes[1], "foobar")
|
||||||
|
|
||||||
|
@ -131,6 +134,8 @@ suite "FloodSub":
|
||||||
await allFuturesThrowing(
|
await allFuturesThrowing(
|
||||||
nodes[0].stop(),
|
nodes[0].stop(),
|
||||||
nodes[1].stop())
|
nodes[1].stop())
|
||||||
|
|
||||||
|
await allFuturesThrowing(subscribes)
|
||||||
await allFuturesThrowing(awaiters)
|
await allFuturesThrowing(awaiters)
|
||||||
result = true
|
result = true
|
||||||
|
|
||||||
|
@ -147,7 +152,7 @@ suite "FloodSub":
|
||||||
awaiters.add((await nodes[0].start()))
|
awaiters.add((await nodes[0].start()))
|
||||||
awaiters.add((await nodes[1].start()))
|
awaiters.add((await nodes[1].start()))
|
||||||
|
|
||||||
await subscribeNodes(nodes)
|
let subscribes = await subscribeNodes(nodes)
|
||||||
await nodes[1].subscribe("foobar", handler)
|
await nodes[1].subscribe("foobar", handler)
|
||||||
await waitSub(nodes[0], nodes[1], "foobar")
|
await waitSub(nodes[0], nodes[1], "foobar")
|
||||||
|
|
||||||
|
@ -164,6 +169,8 @@ suite "FloodSub":
|
||||||
await allFuturesThrowing(
|
await allFuturesThrowing(
|
||||||
nodes[0].stop(),
|
nodes[0].stop(),
|
||||||
nodes[1].stop())
|
nodes[1].stop())
|
||||||
|
|
||||||
|
await allFuturesThrowing(subscribes)
|
||||||
await allFuturesThrowing(awaiters)
|
await allFuturesThrowing(awaiters)
|
||||||
result = true
|
result = true
|
||||||
|
|
||||||
|
@ -182,7 +189,7 @@ suite "FloodSub":
|
||||||
awaiters.add((await nodes[0].start()))
|
awaiters.add((await nodes[0].start()))
|
||||||
awaiters.add((await nodes[1].start()))
|
awaiters.add((await nodes[1].start()))
|
||||||
|
|
||||||
await subscribeNodes(nodes)
|
let subscribes = await subscribeNodes(nodes)
|
||||||
await nodes[1].subscribe("foo", handler)
|
await nodes[1].subscribe("foo", handler)
|
||||||
await waitSub(nodes[0], nodes[1], "foo")
|
await waitSub(nodes[0], nodes[1], "foo")
|
||||||
await nodes[1].subscribe("bar", handler)
|
await nodes[1].subscribe("bar", handler)
|
||||||
|
@ -203,6 +210,8 @@ suite "FloodSub":
|
||||||
await allFuturesThrowing(
|
await allFuturesThrowing(
|
||||||
nodes[0].stop(),
|
nodes[0].stop(),
|
||||||
nodes[1].stop())
|
nodes[1].stop())
|
||||||
|
|
||||||
|
await allFuturesThrowing(subscribes)
|
||||||
await allFuturesThrowing(awaiters)
|
await allFuturesThrowing(awaiters)
|
||||||
result = true
|
result = true
|
||||||
|
|
||||||
|
@ -237,7 +246,7 @@ suite "FloodSub":
|
||||||
for i in 0..<runs:
|
for i in 0..<runs:
|
||||||
awaitters.add(await nodes[i].start())
|
awaitters.add(await nodes[i].start())
|
||||||
|
|
||||||
await subscribeNodes(nodes)
|
let subscribes = await subscribeNodes(nodes)
|
||||||
|
|
||||||
for i in 0..<runs:
|
for i in 0..<runs:
|
||||||
await nodes[i].subscribe("foobar", futs[i][1])
|
await nodes[i].subscribe("foobar", futs[i][1])
|
||||||
|
@ -256,6 +265,8 @@ suite "FloodSub":
|
||||||
|
|
||||||
await allFuturesThrowing(futs.mapIt(it[0]))
|
await allFuturesThrowing(futs.mapIt(it[0]))
|
||||||
await allFuturesThrowing(nodes.mapIt(it.stop()))
|
await allFuturesThrowing(nodes.mapIt(it.stop()))
|
||||||
|
|
||||||
|
await allFuturesThrowing(subscribes)
|
||||||
await allFuturesThrowing(awaitters)
|
await allFuturesThrowing(awaitters)
|
||||||
|
|
||||||
result = true
|
result = true
|
||||||
|
@ -291,7 +302,7 @@ suite "FloodSub":
|
||||||
for i in 0..<runs:
|
for i in 0..<runs:
|
||||||
awaitters.add(await nodes[i].start())
|
awaitters.add(await nodes[i].start())
|
||||||
|
|
||||||
await subscribeNodes(nodes)
|
let subscribes = await subscribeNodes(nodes)
|
||||||
|
|
||||||
for i in 0..<runs:
|
for i in 0..<runs:
|
||||||
await nodes[i].subscribe("foobar", futs[i][1])
|
await nodes[i].subscribe("foobar", futs[i][1])
|
||||||
|
@ -310,6 +321,8 @@ suite "FloodSub":
|
||||||
|
|
||||||
await allFuturesThrowing(futs.mapIt(it[0]))
|
await allFuturesThrowing(futs.mapIt(it[0]))
|
||||||
await allFuturesThrowing(nodes.mapIt(it.stop()))
|
await allFuturesThrowing(nodes.mapIt(it.stop()))
|
||||||
|
|
||||||
|
await allFuturesThrowing(subscribes)
|
||||||
await allFuturesThrowing(awaitters)
|
await allFuturesThrowing(awaitters)
|
||||||
|
|
||||||
result = true
|
result = true
|
||||||
|
|
|
@ -72,7 +72,7 @@ suite "GossipSub":
|
||||||
awaiters.add((await nodes[0].start()))
|
awaiters.add((await nodes[0].start()))
|
||||||
awaiters.add((await nodes[1].start()))
|
awaiters.add((await nodes[1].start()))
|
||||||
|
|
||||||
await subscribeNodes(nodes)
|
let subscribes = await subscribeNodes(nodes)
|
||||||
|
|
||||||
await nodes[0].subscribe("foobar", handler)
|
await nodes[0].subscribe("foobar", handler)
|
||||||
await nodes[1].subscribe("foobar", handler)
|
await nodes[1].subscribe("foobar", handler)
|
||||||
|
@ -99,6 +99,8 @@ suite "GossipSub":
|
||||||
await allFuturesThrowing(
|
await allFuturesThrowing(
|
||||||
nodes[0].stop(),
|
nodes[0].stop(),
|
||||||
nodes[1].stop())
|
nodes[1].stop())
|
||||||
|
|
||||||
|
await allFuturesThrowing(subscribes)
|
||||||
await allFuturesThrowing(awaiters)
|
await allFuturesThrowing(awaiters)
|
||||||
|
|
||||||
check:
|
check:
|
||||||
|
@ -114,7 +116,7 @@ suite "GossipSub":
|
||||||
awaiters.add((await nodes[0].start()))
|
awaiters.add((await nodes[0].start()))
|
||||||
awaiters.add((await nodes[1].start()))
|
awaiters.add((await nodes[1].start()))
|
||||||
|
|
||||||
await subscribeNodes(nodes)
|
let subscribes = await subscribeNodes(nodes)
|
||||||
|
|
||||||
await nodes[0].subscribe("foobar", handler)
|
await nodes[0].subscribe("foobar", handler)
|
||||||
await nodes[1].subscribe("foobar", handler)
|
await nodes[1].subscribe("foobar", handler)
|
||||||
|
@ -140,6 +142,8 @@ suite "GossipSub":
|
||||||
await allFuturesThrowing(
|
await allFuturesThrowing(
|
||||||
nodes[0].stop(),
|
nodes[0].stop(),
|
||||||
nodes[1].stop())
|
nodes[1].stop())
|
||||||
|
|
||||||
|
await allFuturesThrowing(subscribes)
|
||||||
await allFuturesThrowing(awaiters)
|
await allFuturesThrowing(awaiters)
|
||||||
|
|
||||||
check:
|
check:
|
||||||
|
@ -157,7 +161,7 @@ suite "GossipSub":
|
||||||
awaiters.add((await nodes[0].start()))
|
awaiters.add((await nodes[0].start()))
|
||||||
awaiters.add((await nodes[1].start()))
|
awaiters.add((await nodes[1].start()))
|
||||||
|
|
||||||
await subscribeNodes(nodes)
|
let subscribes = await subscribeNodes(nodes)
|
||||||
await nodes[1].subscribe("foo", handler)
|
await nodes[1].subscribe("foo", handler)
|
||||||
await nodes[1].subscribe("bar", handler)
|
await nodes[1].subscribe("bar", handler)
|
||||||
|
|
||||||
|
@ -189,6 +193,8 @@ suite "GossipSub":
|
||||||
await allFuturesThrowing(
|
await allFuturesThrowing(
|
||||||
nodes[0].stop(),
|
nodes[0].stop(),
|
||||||
nodes[1].stop())
|
nodes[1].stop())
|
||||||
|
|
||||||
|
await allFuturesThrowing(subscribes)
|
||||||
await allFuturesThrowing(awaiters)
|
await allFuturesThrowing(awaiters)
|
||||||
result = true
|
result = true
|
||||||
check:
|
check:
|
||||||
|
@ -208,7 +214,7 @@ suite "GossipSub":
|
||||||
for node in nodes:
|
for node in nodes:
|
||||||
awaitters.add(await node.start())
|
awaitters.add(await node.start())
|
||||||
|
|
||||||
await subscribeNodes(nodes)
|
let subscribes = await subscribeNodes(nodes)
|
||||||
await nodes[1].subscribe("foobar", handler)
|
await nodes[1].subscribe("foobar", handler)
|
||||||
await sleepAsync(10.seconds)
|
await sleepAsync(10.seconds)
|
||||||
|
|
||||||
|
@ -221,6 +227,8 @@ suite "GossipSub":
|
||||||
gossip1.gossipsub.hasPeerID("foobar", gossip2.peerInfo.id)
|
gossip1.gossipsub.hasPeerID("foobar", gossip2.peerInfo.id)
|
||||||
|
|
||||||
await allFuturesThrowing(nodes.mapIt(it.stop()))
|
await allFuturesThrowing(nodes.mapIt(it.stop()))
|
||||||
|
|
||||||
|
await allFuturesThrowing(subscribes)
|
||||||
await allFuturesThrowing(awaitters)
|
await allFuturesThrowing(awaitters)
|
||||||
|
|
||||||
result = true
|
result = true
|
||||||
|
@ -241,7 +249,7 @@ suite "GossipSub":
|
||||||
for node in nodes:
|
for node in nodes:
|
||||||
awaitters.add(await node.start())
|
awaitters.add(await node.start())
|
||||||
|
|
||||||
await subscribeNodes(nodes)
|
let subscribes = await subscribeNodes(nodes)
|
||||||
|
|
||||||
await nodes[0].subscribe("foobar", handler)
|
await nodes[0].subscribe("foobar", handler)
|
||||||
await nodes[1].subscribe("foobar", handler)
|
await nodes[1].subscribe("foobar", handler)
|
||||||
|
@ -249,6 +257,7 @@ suite "GossipSub":
|
||||||
var subs: seq[Future[void]]
|
var subs: seq[Future[void]]
|
||||||
subs &= waitSub(nodes[1], nodes[0], "foobar")
|
subs &= waitSub(nodes[1], nodes[0], "foobar")
|
||||||
subs &= waitSub(nodes[0], nodes[1], "foobar")
|
subs &= waitSub(nodes[0], nodes[1], "foobar")
|
||||||
|
|
||||||
await allFuturesThrowing(subs)
|
await allFuturesThrowing(subs)
|
||||||
|
|
||||||
let
|
let
|
||||||
|
@ -269,6 +278,8 @@ suite "GossipSub":
|
||||||
gossip2.mesh.hasPeerID("foobar", gossip1.peerInfo.id)
|
gossip2.mesh.hasPeerID("foobar", gossip1.peerInfo.id)
|
||||||
|
|
||||||
await allFuturesThrowing(nodes.mapIt(it.stop()))
|
await allFuturesThrowing(nodes.mapIt(it.stop()))
|
||||||
|
|
||||||
|
await allFuturesThrowing(subscribes)
|
||||||
await allFuturesThrowing(awaitters)
|
await allFuturesThrowing(awaitters)
|
||||||
|
|
||||||
result = true
|
result = true
|
||||||
|
@ -288,7 +299,7 @@ suite "GossipSub":
|
||||||
wait.add(await nodes[0].start())
|
wait.add(await nodes[0].start())
|
||||||
wait.add(await nodes[1].start())
|
wait.add(await nodes[1].start())
|
||||||
|
|
||||||
await subscribeNodes(nodes)
|
let subscribes = await subscribeNodes(nodes)
|
||||||
|
|
||||||
await nodes[1].subscribe("foobar", handler)
|
await nodes[1].subscribe("foobar", handler)
|
||||||
await waitSub(nodes[0], nodes[1], "foobar")
|
await waitSub(nodes[0], nodes[1], "foobar")
|
||||||
|
@ -320,6 +331,8 @@ suite "GossipSub":
|
||||||
|
|
||||||
await nodes[0].stop()
|
await nodes[0].stop()
|
||||||
await nodes[1].stop()
|
await nodes[1].stop()
|
||||||
|
|
||||||
|
await allFuturesThrowing(subscribes)
|
||||||
await allFuturesThrowing(wait)
|
await allFuturesThrowing(wait)
|
||||||
|
|
||||||
check observed == 2
|
check observed == 2
|
||||||
|
@ -340,7 +353,7 @@ suite "GossipSub":
|
||||||
wait.add(await nodes[0].start())
|
wait.add(await nodes[0].start())
|
||||||
wait.add(await nodes[1].start())
|
wait.add(await nodes[1].start())
|
||||||
|
|
||||||
await subscribeNodes(nodes)
|
let subscribes = await subscribeNodes(nodes)
|
||||||
|
|
||||||
await nodes[0].subscribe("foobar", handler)
|
await nodes[0].subscribe("foobar", handler)
|
||||||
await nodes[1].subscribe("foobar", handler)
|
await nodes[1].subscribe("foobar", handler)
|
||||||
|
@ -363,6 +376,8 @@ suite "GossipSub":
|
||||||
|
|
||||||
await nodes[0].stop()
|
await nodes[0].stop()
|
||||||
await nodes[1].stop()
|
await nodes[1].stop()
|
||||||
|
|
||||||
|
await allFuturesThrowing(subscribes)
|
||||||
await allFuturesThrowing(wait)
|
await allFuturesThrowing(wait)
|
||||||
|
|
||||||
check:
|
check:
|
||||||
|
@ -380,7 +395,7 @@ suite "GossipSub":
|
||||||
secureManagers = [SecureProtocol.Noise])
|
secureManagers = [SecureProtocol.Noise])
|
||||||
awaitters.add((await nodes[i].start()))
|
awaitters.add((await nodes[i].start()))
|
||||||
|
|
||||||
await subscribeRandom(nodes)
|
let subscribes = await subscribeRandom(nodes)
|
||||||
|
|
||||||
var seen: Table[string, int]
|
var seen: Table[string, int]
|
||||||
var subs: seq[Future[void]]
|
var subs: seq[Future[void]]
|
||||||
|
@ -419,6 +434,8 @@ suite "GossipSub":
|
||||||
gossip.mesh["foobar"].len > 0
|
gossip.mesh["foobar"].len > 0
|
||||||
|
|
||||||
await allFuturesThrowing(nodes.mapIt(it.stop()))
|
await allFuturesThrowing(nodes.mapIt(it.stop()))
|
||||||
|
|
||||||
|
await allFuturesThrowing(subscribes)
|
||||||
await allFuturesThrowing(awaitters)
|
await allFuturesThrowing(awaitters)
|
||||||
result = true
|
result = true
|
||||||
|
|
||||||
|
@ -437,7 +454,7 @@ suite "GossipSub":
|
||||||
secureManagers = [SecureProtocol.Secio])
|
secureManagers = [SecureProtocol.Secio])
|
||||||
awaitters.add((await nodes[i].start()))
|
awaitters.add((await nodes[i].start()))
|
||||||
|
|
||||||
await subscribeSparseNodes(nodes)
|
let subscribes = await subscribeSparseNodes(nodes, 1)
|
||||||
|
|
||||||
var seen: Table[string, int]
|
var seen: Table[string, int]
|
||||||
var subs: seq[Future[void]]
|
var subs: seq[Future[void]]
|
||||||
|
@ -477,6 +494,8 @@ suite "GossipSub":
|
||||||
gossip.mesh["foobar"].len > 0
|
gossip.mesh["foobar"].len > 0
|
||||||
|
|
||||||
await allFuturesThrowing(nodes.mapIt(it.stop()))
|
await allFuturesThrowing(nodes.mapIt(it.stop()))
|
||||||
|
|
||||||
|
await allFuturesThrowing(subscribes)
|
||||||
await allFuturesThrowing(awaitters)
|
await allFuturesThrowing(awaitters)
|
||||||
result = true
|
result = true
|
||||||
|
|
||||||
|
|
|
@ -9,36 +9,33 @@ proc generateNodes*(num: Natural, gossip: bool = false): seq[Switch] =
|
||||||
for i in 0..<num:
|
for i in 0..<num:
|
||||||
result.add(newStandardSwitch(gossip = gossip))
|
result.add(newStandardSwitch(gossip = gossip))
|
||||||
|
|
||||||
proc subscribeNodes*(nodes: seq[Switch]) {.async.} =
|
proc subscribeNodes*(nodes: seq[Switch]): Future[seq[Future[void]]] {.async.} =
|
||||||
var dials: seq[Future[void]]
|
|
||||||
for dialer in nodes:
|
for dialer in nodes:
|
||||||
for node in nodes:
|
for node in nodes:
|
||||||
if dialer.peerInfo.peerId != node.peerInfo.peerId:
|
if dialer.peerInfo.peerId != node.peerInfo.peerId:
|
||||||
dials.add(dialer.connect(node.peerInfo))
|
await dialer.connect(node.peerInfo)
|
||||||
await allFutures(dials)
|
result.add(dialer.subscribePeer(node.peerInfo))
|
||||||
|
|
||||||
proc subscribeSparseNodes*(nodes: seq[Switch], degree: int = 2) {.async.} =
|
proc subscribeSparseNodes*(nodes: seq[Switch], degree: int = 2): Future[seq[Future[void]]] {.async.} =
|
||||||
if nodes.len < degree:
|
if nodes.len < degree:
|
||||||
raise (ref CatchableError)(msg: "nodes count needs to be greater or equal to degree!")
|
raise (ref CatchableError)(msg: "nodes count needs to be greater or equal to degree!")
|
||||||
|
|
||||||
var dials: seq[Future[void]]
|
|
||||||
for i, dialer in nodes:
|
for i, dialer in nodes:
|
||||||
if (i mod degree) != 0:
|
if (i mod degree) != 0:
|
||||||
continue
|
continue
|
||||||
|
|
||||||
for node in nodes:
|
for node in nodes:
|
||||||
if dialer.peerInfo.peerId != node.peerInfo.peerId:
|
if dialer.peerInfo.peerId != node.peerInfo.peerId:
|
||||||
dials.add(dialer.connect(node.peerInfo))
|
await dialer.connect(node.peerInfo)
|
||||||
await allFutures(dials)
|
result.add(dialer.subscribePeer(node.peerInfo))
|
||||||
|
|
||||||
proc subscribeRandom*(nodes: seq[Switch]) {.async.} =
|
proc subscribeRandom*(nodes: seq[Switch]): Future[seq[Future[void]]] {.async.} =
|
||||||
var dials: seq[Future[void]]
|
|
||||||
for dialer in nodes:
|
for dialer in nodes:
|
||||||
var dialed: seq[string]
|
var dialed: seq[string]
|
||||||
while dialed.len < nodes.len - 1:
|
while dialed.len < nodes.len - 1:
|
||||||
let node = sample(nodes)
|
let node = sample(nodes)
|
||||||
if node.peerInfo.id notin dialed:
|
if node.peerInfo.id notin dialed:
|
||||||
if dialer.peerInfo.id != node.peerInfo.id:
|
if dialer.peerInfo.id != node.peerInfo.id:
|
||||||
dials.add(dialer.connect(node.peerInfo))
|
await dialer.connect(node.peerInfo)
|
||||||
dialed &= node.peerInfo.id
|
result.add(dialer.subscribePeer(node.peerInfo))
|
||||||
await allFutures(dials)
|
dialed.add(node.peerInfo.id)
|
||||||
|
|
|
@ -88,8 +88,12 @@ proc testPubSubDaemonPublish(gossip: bool = false,
|
||||||
if times >= count and not finished:
|
if times >= count and not finished:
|
||||||
finished = true
|
finished = true
|
||||||
|
|
||||||
await nativeNode.connect(NativePeerInfo.init(daemonPeer.peer,
|
let peer = NativePeerInfo.init(
|
||||||
daemonPeer.addresses))
|
daemonPeer.peer,
|
||||||
|
daemonPeer.addresses)
|
||||||
|
await nativeNode.connect(peer)
|
||||||
|
let subscribeHanle = nativeNode.subscribePeer(peer)
|
||||||
|
|
||||||
await sleepAsync(1.seconds)
|
await sleepAsync(1.seconds)
|
||||||
await daemonNode.connect(nativePeer.peerId, nativePeer.addrs)
|
await daemonNode.connect(nativePeer.peerId, nativePeer.addrs)
|
||||||
|
|
||||||
|
@ -113,6 +117,7 @@ proc testPubSubDaemonPublish(gossip: bool = false,
|
||||||
await nativeNode.stop()
|
await nativeNode.stop()
|
||||||
await allFutures(awaiters)
|
await allFutures(awaiters)
|
||||||
await daemonNode.close()
|
await daemonNode.close()
|
||||||
|
await subscribeHanle
|
||||||
|
|
||||||
proc testPubSubNodePublish(gossip: bool = false,
|
proc testPubSubNodePublish(gossip: bool = false,
|
||||||
count: int = 1): Future[bool] {.async.} =
|
count: int = 1): Future[bool] {.async.} =
|
||||||
|
@ -134,8 +139,11 @@ proc testPubSubNodePublish(gossip: bool = false,
|
||||||
let awaiters = nativeNode.start()
|
let awaiters = nativeNode.start()
|
||||||
let nativePeer = nativeNode.peerInfo
|
let nativePeer = nativeNode.peerInfo
|
||||||
|
|
||||||
await nativeNode.connect(NativePeerInfo.init(daemonPeer.peer,
|
let peer = NativePeerInfo.init(
|
||||||
daemonPeer.addresses))
|
daemonPeer.peer,
|
||||||
|
daemonPeer.addresses)
|
||||||
|
await nativeNode.connect(peer)
|
||||||
|
let subscribeHandle = nativeNode.subscribePeer(peer)
|
||||||
|
|
||||||
await sleepAsync(1.seconds)
|
await sleepAsync(1.seconds)
|
||||||
await daemonNode.connect(nativePeer.peerId, nativePeer.addrs)
|
await daemonNode.connect(nativePeer.peerId, nativePeer.addrs)
|
||||||
|
@ -168,6 +176,7 @@ proc testPubSubNodePublish(gossip: bool = false,
|
||||||
await nativeNode.stop()
|
await nativeNode.stop()
|
||||||
await allFutures(awaiters)
|
await allFutures(awaiters)
|
||||||
await daemonNode.close()
|
await daemonNode.close()
|
||||||
|
await subscribeHandle
|
||||||
|
|
||||||
suite "Interop":
|
suite "Interop":
|
||||||
# TODO: chronos transports are leaking,
|
# TODO: chronos transports are leaking,
|
||||||
|
|
Loading…
Reference in New Issue