nim-libp2p/libp2p/multistream.nim

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

283 lines
8.9 KiB
Nim
Raw Normal View History

2022-07-01 18:19:57 +00:00
# Nim-LibP2P
# Copyright (c) 2023-2024 Status Research & Development GmbH
2022-07-01 18:19:57 +00:00
# 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.
2019-08-30 05:16:55 +00:00
2023-06-07 11:12:49 +00:00
{.push raises: [].}
import std/[strutils, sequtils, tables]
2020-06-03 02:21:11 +00:00
import chronos, chronicles, stew/byteutils
import stream/connection, protocols/protocol
2019-08-30 05:16:55 +00:00
2019-09-11 21:06:10 +00:00
logScope:
2020-12-01 17:34:27 +00:00
topics = "libp2p multistream"
2019-09-11 21:06:10 +00:00
2020-01-07 08:02:37 +00:00
const
2023-03-08 11:30:19 +00:00
MsgSize = 1024
Codec = "/multistream/1.0.0"
2019-10-04 16:04:21 +00:00
2023-03-08 11:30:19 +00:00
Na = "na\n"
Ls = "ls\n"
2019-08-30 05:16:55 +00:00
type
2023-06-07 11:12:49 +00:00
Matcher* = proc(proto: string): bool {.gcsafe, raises: [].}
2019-08-30 05:16:55 +00:00
MultiStreamError* = object of LPError
HandlerHolder* = ref object
Gossip one one (#240) * 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>
2020-09-21 09:16:29 +00:00
protos*: seq[string]
2019-09-06 00:16:49 +00:00
protocol*: LPProtocol
match*: Matcher
openedStreams: CountTable[PeerId]
2019-08-30 05:16:55 +00:00
2020-02-04 16:16:21 +00:00
MultistreamSelect* = ref object of RootObj
2019-08-30 05:16:55 +00:00
handlers*: seq[HandlerHolder]
codec*: string
proc new*(T: typedesc[MultistreamSelect]): T =
2023-03-08 11:30:19 +00:00
T(codec: Codec)
template validateSuffix(str: string): untyped =
if str.endsWith("\n"):
str.removeSuffix("\n")
else:
raise (ref MultiStreamError)(msg: "MultistreamSelect failed, malformed message")
proc select*(
_: MultistreamSelect | type MultistreamSelect, conn: Connection, proto: seq[string]
): Future[string] {.async: (raises: [CancelledError, LPStreamError, MultiStreamError]).} =
2023-03-08 11:30:19 +00:00
trace "initiating handshake", conn, codec = Codec
2019-08-30 05:16:55 +00:00
## select a remote protocol
2023-03-08 11:30:19 +00:00
await conn.writeLp(Codec & "\n") # write handshake
2019-08-30 05:16:55 +00:00
if proto.len() > 0:
trace "selecting proto", conn, proto = proto[0]
2019-09-05 15:19:15 +00:00
await conn.writeLp((proto[0] & "\n")) # select proto
2019-08-30 05:16:55 +00:00
var s = string.fromBytes((await conn.readLp(MsgSize))) # read ms header
validateSuffix(s)
if s != Codec:
notice "handshake failed", conn, codec = s
raise (ref MultiStreamError)(msg: "MultistreamSelect handshake failed")
else:
trace "multistream handshake success", conn
2019-08-30 05:16:55 +00:00
if proto.len() == 0: # no protocols, must be a handshake call
return Codec
else:
s = string.fromBytes(await conn.readLp(MsgSize)) # read the first proto
validateSuffix(s)
trace "reading first requested proto", conn
if s == proto[0]:
trace "successfully selected ", conn, proto = proto[0]
2022-08-01 12:31:22 +00:00
conn.protocol = proto[0]
return proto[0]
elif proto.len > 1:
# Try to negotiate alternatives
let protos = proto[1 ..< proto.len()]
trace "selecting one of several protos", conn, protos = protos
for p in protos:
trace "selecting proto", conn, proto = p
await conn.writeLp((p & "\n")) # select proto
s = string.fromBytes(await conn.readLp(MsgSize)) # read the first proto
validateSuffix(s)
if s == p:
trace "selected protocol", conn, protocol = s
2022-08-01 12:31:22 +00:00
conn.protocol = s
return s
return ""
else:
# No alternatives, fail
return ""
proc select*(
_: MultistreamSelect | type MultistreamSelect, conn: Connection, proto: string
): Future[bool] {.async: (raises: [CancelledError, LPStreamError, MultiStreamError]).} =
2020-01-07 08:02:37 +00:00
if proto.len > 0:
(await MultistreamSelect.select(conn, @[proto])) == proto
2020-01-07 08:02:37 +00:00
else:
(await MultistreamSelect.select(conn, @[])) == Codec
proc select*(
m: MultistreamSelect, conn: Connection
): Future[bool] {.
async: (raises: [CancelledError, LPStreamError, MultiStreamError], raw: true)
.} =
2019-09-08 07:43:33 +00:00
m.select(conn, "")
2019-08-30 05:16:55 +00:00
proc list*(
m: MultistreamSelect, conn: Connection
): Future[seq[string]] {.
async: (raises: [CancelledError, LPStreamError, MultiStreamError])
.} =
2019-08-30 05:16:55 +00:00
## list remote protos requests on connection
if not await m.select(conn):
2019-08-30 05:16:55 +00:00
return
2023-03-08 11:30:19 +00:00
await conn.writeLp(Ls) # send ls
2019-08-30 05:16:55 +00:00
var list = newSeq[string]()
let ms = string.fromBytes(await conn.readLp(MsgSize))
2019-08-30 05:16:55 +00:00
for s in ms.split("\n"):
if s.len() > 0:
list.add(s)
result = list
2023-03-08 11:30:19 +00:00
proc handle*(
_: type MultistreamSelect,
conn: Connection,
protos: seq[string],
matchers = newSeq[Matcher](),
active: bool = false,
): Future[string] {.async: (raises: [CancelledError, LPStreamError, MultiStreamError]).} =
2023-03-08 11:30:19 +00:00
trace "Starting multistream negotiation", conn, handshaked = active
var handshaked = active
while not conn.atEof:
var ms = string.fromBytes(await conn.readLp(MsgSize))
validateSuffix(ms)
if not handshaked and ms != Codec:
debug "expected handshake message", conn, instead = ms
raise (ref MultiStreamError)(
msg: "MultistreamSelect handling failed, invalid first message"
)
2023-03-08 11:30:19 +00:00
trace "handle: got request", conn, ms
if ms.len() <= 0:
trace "handle: invalid proto", conn
await conn.writeLp(Na)
case ms
of "ls":
trace "handle: listing protos", conn
#TODO this doens't seem to follow spec, each protocol
# should be length prefixed. Not very important
# since LS is getting deprecated
await conn.writeLp(protos.join("\n") & "\n")
of Codec:
if not handshaked:
await conn.writeLp(Codec & "\n")
handshaked = true
else:
trace "handle: sending `na` for duplicate handshake while handshaked", conn
await conn.writeLp(Na)
elif ms in protos or matchers.anyIt(it(ms)):
trace "found handler", conn, protocol = ms
await conn.writeLp(ms & "\n")
conn.protocol = ms
return ms
else:
trace "no handlers", conn, protocol = ms
await conn.writeLp(Na)
proc handle*(
m: MultistreamSelect, conn: Connection, active: bool = false
) {.async: (raises: [CancelledError]).} =
trace "Starting multistream handler", conn, handshaked = active
2023-03-08 11:30:19 +00:00
var
protos: seq[string]
matchers: seq[Matcher]
for h in m.handlers:
if h.match != nil:
2023-03-08 11:30:19 +00:00
matchers.add(h.match)
for proto in h.protos:
protos.add(proto)
2020-05-23 17:10:22 +00:00
try:
2023-03-08 11:30:19 +00:00
let ms = await MultistreamSelect.handle(conn, protos, matchers, active)
for h in m.handlers:
if (h.match != nil and h.match(ms)) or h.protos.contains(ms):
2023-03-08 11:30:19 +00:00
trace "found handler", conn, protocol = ms
var protocolHolder = h
let maxIncomingStreams = protocolHolder.protocol.maxIncomingStreams
if protocolHolder.openedStreams.getOrDefault(conn.peerId) >= maxIncomingStreams:
2023-03-08 11:30:19 +00:00
debug "Max streams for protocol reached, blocking new stream",
conn, protocol = ms, maxIncomingStreams
return
protocolHolder.openedStreams.inc(conn.peerId)
try:
await protocolHolder.protocol.handler(conn, ms)
finally:
protocolHolder.openedStreams.inc(conn.peerId, -1)
if protocolHolder.openedStreams[conn.peerId] == 0:
protocolHolder.openedStreams.del(conn.peerId)
return
debug "no handlers", conn, ms
except CancelledError as exc:
raise exc
2020-05-23 17:10:22 +00:00
except CatchableError as exc:
trace "Exception in multistream", conn, description = exc.msg
2020-05-23 17:10:22 +00:00
finally:
await conn.close()
trace "Stopped multistream handler", conn
2019-08-30 05:16:55 +00:00
proc addHandler*(
m: MultistreamSelect,
Gossip one one (#240) * 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>
2020-09-21 09:16:29 +00:00
codecs: seq[string],
protocol: LPProtocol,
matcher: Matcher = nil,
) =
Gossip one one (#240) * 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>
2020-09-21 09:16:29 +00:00
trace "registering protocols", protos = codecs
m.handlers.add(HandlerHolder(protos: codecs, protocol: protocol, match: matcher))
Gossip one one (#240) * 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>
2020-09-21 09:16:29 +00:00
proc addHandler*(
m: MultistreamSelect, codec: string, protocol: LPProtocol, matcher: Matcher = nil
) =
addHandler(m, @[codec], protocol, matcher)
proc addHandler*[E](
m: MultistreamSelect,
codec: string,
handler:
LPProtoHandler |
proc(conn: Connection, proto: string): InternalRaisesFuture[void, E],
matcher: Matcher = nil,
) =
2019-09-14 13:55:52 +00:00
## helper to allow registering pure handlers
Gossip one one (#240) * 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>
2020-09-21 09:16:29 +00:00
trace "registering proto handler", proto = codec
2019-09-14 13:55:52 +00:00
let protocol = new LPProtocol
protocol.codec = codec
protocol.handler = handler
Gossip one one (#240) * 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>
2020-09-21 09:16:29 +00:00
m.handlers.add(HandlerHolder(protos: @[codec], protocol: protocol, match: matcher))
proc start*(m: MultistreamSelect) {.async: (raises: [CancelledError]).} =
# Nim 1.6.18: Using `mapIt` results in a seq of `.Raising([])`
# TODO https://github.com/nim-lang/Nim/issues/23445
var futs = newSeqOfCap[Future[void].Raising([CancelledError])](m.handlers.len)
for it in m.handlers:
futs.add it.protocol.start()
try:
await allFutures(futs)
for fut in futs:
await fut
except CancelledError as exc:
var pending: seq[Future[void].Raising([])]
doAssert m.handlers.len == futs.len, "Handlers modified while starting"
for i, fut in futs:
if not fut.finished:
pending.add fut.cancelAndWait()
elif fut.completed:
pending.add m.handlers[i].protocol.stop()
else:
static:
doAssert typeof(fut).E is (CancelledError,)
await noCancel allFutures(pending)
raise exc
proc stop*(m: MultistreamSelect) {.async: (raises: []).} =
# Nim 1.6.18: Using `mapIt` results in a seq of `.Raising([CancelledError])`
var futs = newSeqOfCap[Future[void].Raising([])](m.handlers.len)
for it in m.handlers:
futs.add it.protocol.stop()
await noCancel allFutures(futs)
for fut in futs:
await fut