Merge commit '4e7eaba67ae1d35eb89af5882b85a7d091c6c95a' into dev/etan/zz-dbg

This commit is contained in:
Etan Kissling 2023-12-21 17:59:42 +01:00
commit 1ac8cf9010
No known key found for this signature in database
GPG Key ID: B21DA824C5A3D03D
25 changed files with 714 additions and 138 deletions

View File

@ -19,13 +19,13 @@ jobs:
- uses: jiro4989/setup-nim-action@v1
with:
nim-version: 'stable'
nim-version: '1.6.x'
- name: Generate doc
run: |
nim --version
nimble --version
nimble install_pinned -y
nimble install_pinned
# nim doc can "fail", but the doc is still generated
nim doc --git.url:https://github.com/status-im/nim-libp2p --git.commit:${GITHUB_REF##*/} --outdir:${GITHUB_REF##*/} --project libp2p || true

View File

@ -23,7 +23,7 @@ jobs:
- name: Build image
run: >
cd multidim-interop/impl/nim/v1.0 &&
cd transport-interop/impl/nim/v1.0 &&
make commitSha=$GITHUB_SHA image_name=nim-libp2p-head
- name: Create ping-version.json
@ -45,10 +45,10 @@ jobs:
]
}
EOF
) > ${{ github.workspace }}/test_head.json
- uses: libp2p/test-plans/.github/actions/run-interop-ping-test@master
- uses: libp2p/test-plans/.github/actions/run-transport-interop-test@master
with:
test-filter: nim-libp2p-head
extra-versions: ${{ github.workspace }}/test_head.json

View File

@ -227,7 +227,7 @@ proc build*(b: SwitchBuilder): Switch
identify = Identify.new(peerInfo, b.sendSignedPeerRecord)
connManager = ConnManager.new(b.maxConnsPerPeer, b.maxConnections, b.maxIn, b.maxOut)
ms = MultistreamSelect.new()
muxedUpgrade = MuxedUpgrade.new(b.muxers, secureManagerInstances, connManager, ms)
muxedUpgrade = MuxedUpgrade.new(b.muxers, secureManagerInstances, ms)
let
transports = block:

View File

@ -122,20 +122,15 @@ proc request*[T](dm: DiscoveryManager, value: T): DiscoveryQuery =
pa.add(value)
return dm.request(pa)
proc advertise*(dm: DiscoveryManager, pa: PeerAttributes) =
proc advertise*[T](dm: DiscoveryManager, value: T) =
for i in dm.interfaces:
i.toAdvertise = pa
i.toAdvertise.add(value)
if i.advertiseLoop.isNil:
i.advertisementUpdated = newAsyncEvent()
i.advertiseLoop = i.advertise()
else:
i.advertisementUpdated.fire()
proc advertise*[T](dm: DiscoveryManager, value: T) =
var pa: PeerAttributes
pa.add(value)
dm.advertise(pa)
template forEach*(query: DiscoveryQuery, code: untyped) =
## Will execute `code` for each discovered peer. The
## peer attritubtes are available through the variable

View File

@ -19,6 +19,7 @@ type
rdv*: RendezVous
timeToRequest: Duration
timeToAdvertise: Duration
ttl: Duration
RdvNamespace* = distinct string
@ -62,12 +63,16 @@ method advertise*(self: RendezVousInterface) {.async.} =
self.advertisementUpdated.clear()
for toAdv in toAdvertise:
await self.rdv.advertise(toAdv, self.timeToAdvertise)
try:
await self.rdv.advertise(toAdv, self.ttl)
except CatchableError as error:
debug "RendezVous advertise error: ", msg = error.msg
await sleepAsync(self.timeToAdvertise) or self.advertisementUpdated.wait()
proc new*(T: typedesc[RendezVousInterface],
rdv: RendezVous,
ttr: Duration = 1.minutes,
tta: Duration = MinimumDuration): RendezVousInterface =
T(rdv: rdv, timeToRequest: ttr, timeToAdvertise: tta)
tta: Duration = 1.minutes,
ttl: Duration = MinimumDuration): RendezVousInterface =
T(rdv: rdv, timeToRequest: ttr, timeToAdvertise: tta, ttl: ttl)

View File

@ -19,7 +19,8 @@ func toException*(e: string): ref LPError =
# sadly nim needs more love for hygienic templates
# so here goes the macro, its based on the proc/template version
# and uses quote do so it's quite readable
macro checkFutures*[T](futs: seq[Future[T]], exclude: untyped = []): untyped =
# TODO https://github.com/nim-lang/Nim/issues/22936
macro checkFutures*[F](futs: seq[F], exclude: untyped = []): untyped =
let nexclude = exclude.len
case nexclude
of 0:

View File

@ -15,7 +15,7 @@ import ./pubsub,
./pubsubpeer,
./timedcache,
./peertable,
./rpc/[message, messages],
./rpc/[message, messages, protobuf],
../../crypto/crypto,
../../stream/connection,
../../peerid,
@ -95,7 +95,16 @@ method unsubscribePeer*(f: FloodSub, peer: PeerId) =
method rpcHandler*(f: FloodSub,
peer: PubSubPeer,
rpcMsg: RPCMsg) {.async.} =
data: seq[byte]) {.async.} =
var rpcMsg = decodeRpcMsg(data).valueOr:
debug "failed to decode msg from peer", peer, err = error
raise newException(CatchableError, "")
trace "decoded msg from peer", peer, msg = rpcMsg.shortLog
# trigger hooks
peer.recvObservers(rpcMsg)
for i in 0..<min(f.topicsHigh, rpcMsg.subscriptions.len):
template sub: untyped = rpcMsg.subscriptions[i]
f.handleSubscribe(peer, sub.topic, sub.subscribe)

View File

@ -13,13 +13,14 @@
import std/[sets, sequtils]
import chronos, chronicles, metrics
import chronos/ratelimit
import ./pubsub,
./floodsub,
./pubsubpeer,
./peertable,
./mcache,
./timedcache,
./rpc/[messages, message],
./rpc/[messages, message, protobuf],
../protocol,
../../stream/connection,
../../peerinfo,
@ -77,7 +78,9 @@ proc init*(_: type[GossipSubParams]): GossipSubParams =
behaviourPenaltyDecay: 0.999,
disconnectBadPeers: false,
enablePX: false,
bandwidthEstimatebps: 100_000_000 # 100 Mbps or 12.5 MBps
bandwidthEstimatebps: 100_000_000, # 100 Mbps or 12.5 MBps
overheadRateLimit: Opt.none(tuple[bytes: int, interval: Duration]),
disconnectPeerAboveRateLimit: false
)
proc validateParameters*(parameters: GossipSubParams): Result[void, cstring] =
@ -150,7 +153,7 @@ method init*(g: GossipSub) =
g.codecs &= GossipSubCodec
g.codecs &= GossipSubCodec_10
method onNewPeer(g: GossipSub, peer: PubSubPeer) =
method onNewPeer*(g: GossipSub, peer: PubSubPeer) =
g.withPeerStats(peer.peerId) do (stats: var PeerStats):
# Make sure stats and peer information match, even when reloading peer stats
# from a previous connection
@ -159,7 +162,7 @@ method onNewPeer(g: GossipSub, peer: PubSubPeer) =
peer.behaviourPenalty = stats.behaviourPenalty
# Check if the score is below the threshold and disconnect the peer if necessary
g.disconnectBadPeerCheck(peer, stats.score)
g.disconnectIfBadScorePeer(peer, stats.score)
peer.iHaveBudget = IHavePeerBudget
peer.pingBudget = PingsPeerBudget
@ -315,7 +318,7 @@ proc validateAndRelay(g: GossipSub,
of ValidationResult.Reject:
debug "Dropping message after validation, reason: reject",
msgId = shortLog(msgId), peer
g.punishInvalidMessage(peer, msg.topicIds)
await g.punishInvalidMessage(peer, msg)
return
of ValidationResult.Ignore:
debug "Dropping message after validation, reason: ignore",
@ -378,9 +381,57 @@ proc validateAndRelay(g: GossipSub,
except CatchableError as exc:
info "validateAndRelay failed", msg=exc.msg
proc dataAndTopicsIdSize(msgs: seq[Message]): int =
msgs.mapIt(it.data.len + it.topicIds.mapIt(it.len).foldl(a + b, 0)).foldl(a + b, 0)
proc rateLimit*(g: GossipSub, peer: PubSubPeer, rpcMsgOpt: Opt[RPCMsg], msgSize: int) {.raises:[PeerRateLimitError, CatchableError], async.} =
# In this way we count even ignored fields by protobuf
var rmsg = rpcMsgOpt.valueOr:
peer.overheadRateLimitOpt.withValue(overheadRateLimit):
if not overheadRateLimit.tryConsume(msgSize):
libp2p_gossipsub_peers_rate_limit_hits.inc(labelValues = [peer.getAgent()]) # let's just measure at the beginning for test purposes.
debug "Peer sent a msg that couldn't be decoded and it's above rate limit.", peer, uselessAppBytesNum = msgSize
if g.parameters.disconnectPeerAboveRateLimit:
await g.disconnectPeer(peer)
raise newException(PeerRateLimitError, "Peer disconnected because it's above rate limit.")
raise newException(CatchableError, "Peer msg couldn't be decoded")
let usefulMsgBytesNum =
if g.verifySignature:
byteSize(rmsg.messages)
else:
dataAndTopicsIdSize(rmsg.messages)
var uselessAppBytesNum = msgSize - usefulMsgBytesNum
rmsg.control.withValue(control):
uselessAppBytesNum -= (byteSize(control.ihave) + byteSize(control.iwant))
peer.overheadRateLimitOpt.withValue(overheadRateLimit):
if not overheadRateLimit.tryConsume(uselessAppBytesNum):
libp2p_gossipsub_peers_rate_limit_hits.inc(labelValues = [peer.getAgent()]) # let's just measure at the beginning for test purposes.
debug "Peer sent too much useless application data and it's above rate limit.", peer, msgSize, uselessAppBytesNum, rmsg
if g.parameters.disconnectPeerAboveRateLimit:
await g.disconnectPeer(peer)
raise newException(PeerRateLimitError, "Peer disconnected because it's above rate limit.")
method rpcHandler*(g: GossipSub,
peer: PubSubPeer,
rpcMsg: RPCMsg) {.async.} =
data: seq[byte]) {.async.} =
let msgSize = data.len
var rpcMsg = decodeRpcMsg(data).valueOr:
debug "failed to decode msg from peer", peer, err = error
await rateLimit(g, peer, Opt.none(RPCMsg), msgSize)
return
trace "decoded msg from peer", peer, msg = rpcMsg.shortLog
await rateLimit(g, peer, Opt.some(rpcMsg), msgSize)
# trigger hooks
peer.recvObservers(rpcMsg)
if rpcMsg.ping.len in 1..<64 and peer.pingBudget > 0:
g.send(peer, RPCMsg(pong: rpcMsg.ping))
peer.pingBudget.dec
@ -441,14 +492,14 @@ method rpcHandler*(g: GossipSub,
# always validate if signature is present or required
debug "Dropping message due to failed signature verification",
msgId = shortLog(msgId), peer
g.punishInvalidMessage(peer, msg.topicIds)
await g.punishInvalidMessage(peer, msg)
continue
if msg.seqno.len > 0 and msg.seqno.len != 8:
# if we have seqno should be 8 bytes long
debug "Dropping message due to invalid seqno length",
msgId = shortLog(msgId), peer
g.punishInvalidMessage(peer, msg.topicIds)
await g.punishInvalidMessage(peer, msg)
continue
# g.anonymize needs no evaluation when receiving messages
@ -672,3 +723,13 @@ method initPubSub*(g: GossipSub)
# init gossip stuff
g.mcache = MCache.init(g.parameters.historyGossip, g.parameters.historyLength)
method getOrCreatePeer*(
g: GossipSub,
peerId: PeerId,
protos: seq[string]): PubSubPeer =
let peer = procCall PubSub(g).getOrCreatePeer(peerId, protos)
g.parameters.overheadRateLimit.withValue(overheadRateLimit):
peer.overheadRateLimitOpt = Opt.some(TokenBucket.new(overheadRateLimit.bytes, overheadRateLimit.interval))
return peer

View File

@ -11,9 +11,12 @@
import std/[tables, sets]
import chronos, chronicles, metrics
import chronos/ratelimit
import "."/[types]
import ".."/[pubsubpeer]
import ../rpc/messages
import "../../.."/[peerid, multiaddress, switch, utils/heartbeat]
import ../pubsub
logScope:
topics = "libp2p gossipsub"
@ -27,6 +30,7 @@ declareGauge(libp2p_gossipsub_peers_score_invalidMessageDeliveries, "Detailed go
declareGauge(libp2p_gossipsub_peers_score_appScore, "Detailed gossipsub scoring metric", labels = ["agent"])
declareGauge(libp2p_gossipsub_peers_score_behaviourPenalty, "Detailed gossipsub scoring metric", labels = ["agent"])
declareGauge(libp2p_gossipsub_peers_score_colocationFactor, "Detailed gossipsub scoring metric", labels = ["agent"])
declarePublicCounter(libp2p_gossipsub_peers_rate_limit_hits, "The number of times peers were above their rate limit", labels = ["agent"])
proc init*(_: type[TopicParams]): TopicParams =
TopicParams(
@ -85,27 +89,18 @@ proc colocationFactor(g: GossipSub, peer: PubSubPeer): float64 =
{.pop.}
proc disconnectPeer(g: GossipSub, peer: PubSubPeer) {.async.} =
let agent =
when defined(libp2p_agents_metrics):
if peer.shortAgent.len > 0:
peer.shortAgent
else:
"unknown"
else:
"unknown"
libp2p_gossipsub_bad_score_disconnection.inc(labelValues = [agent])
proc disconnectPeer*(g: GossipSub, peer: PubSubPeer) {.async.} =
try:
await g.switch.disconnect(peer.peerId)
except CatchableError as exc: # Never cancelled
trace "Failed to close connection", peer, error = exc.name, msg = exc.msg
proc disconnectBadPeerCheck*(g: GossipSub, peer: PubSubPeer, score: float64) =
proc disconnectIfBadScorePeer*(g: GossipSub, peer: PubSubPeer, score: float64) =
if g.parameters.disconnectBadPeers and score < g.parameters.graylistThreshold and
peer.peerId notin g.parameters.directPeers:
debug "disconnecting bad score peer", peer, score = peer.score
asyncSpawn(g.disconnectPeer(peer))
libp2p_gossipsub_bad_score_disconnection.inc(labelValues = [peer.getAgent()])
proc updateScores*(g: GossipSub) = # avoid async
## https://github.com/libp2p/specs/blob/master/pubsub/gossipsub/gossipsub-v1.1.md#the-score-function
@ -175,14 +170,7 @@ proc updateScores*(g: GossipSub) = # avoid async
score += topicScore * topicParams.topicWeight
# Score metrics
let agent =
when defined(libp2p_agents_metrics):
if peer.shortAgent.len > 0:
peer.shortAgent
else:
"unknown"
else:
"unknown"
let agent = peer.getAgent()
libp2p_gossipsub_peers_score_firstMessageDeliveries.inc(info.firstMessageDeliveries, labelValues = [agent])
libp2p_gossipsub_peers_score_meshMessageDeliveries.inc(info.meshMessageDeliveries, labelValues = [agent])
libp2p_gossipsub_peers_score_meshFailurePenalty.inc(info.meshFailurePenalty, labelValues = [agent])
@ -219,14 +207,7 @@ proc updateScores*(g: GossipSub) = # avoid async
score += colocationFactor * g.parameters.ipColocationFactorWeight
# Score metrics
let agent =
when defined(libp2p_agents_metrics):
if peer.shortAgent.len > 0:
peer.shortAgent
else:
"unknown"
else:
"unknown"
let agent = peer.getAgent()
libp2p_gossipsub_peers_score_appScore.inc(peer.appScore, labelValues = [agent])
libp2p_gossipsub_peers_score_behaviourPenalty.inc(peer.behaviourPenalty, labelValues = [agent])
libp2p_gossipsub_peers_score_colocationFactor.inc(colocationFactor, labelValues = [agent])
@ -246,8 +227,7 @@ proc updateScores*(g: GossipSub) = # avoid async
trace "updated peer's score", peer, score = peer.score, n_topics, is_grafted
g.disconnectBadPeerCheck(peer, stats.score)
g.disconnectIfBadScorePeer(peer, stats.score)
libp2p_gossipsub_peers_scores.inc(peer.score, labelValues = [agent])
for peer in evicting:
@ -260,8 +240,18 @@ proc scoringHeartbeat*(g: GossipSub) {.async.} =
trace "running scoring heartbeat", instance = cast[int](g)
g.updateScores()
proc punishInvalidMessage*(g: GossipSub, peer: PubSubPeer, topics: seq[string]) =
for tt in topics:
proc punishInvalidMessage*(g: GossipSub, peer: PubSubPeer, msg: Message) {.async.} =
let uselessAppBytesNum = msg.data.len
peer.overheadRateLimitOpt.withValue(overheadRateLimit):
if not overheadRateLimit.tryConsume(uselessAppBytesNum):
debug "Peer sent invalid message and it's above rate limit", peer, uselessAppBytesNum
libp2p_gossipsub_peers_rate_limit_hits.inc(labelValues = [peer.getAgent()]) # let's just measure at the beginning for test purposes.
if g.parameters.disconnectPeerAboveRateLimit:
await g.disconnectPeer(peer)
raise newException(PeerRateLimitError, "Peer disconnected because it's above rate limit.")
for tt in msg.topicIds:
let t = tt
if t notin g.topics:
continue

View File

@ -144,6 +144,9 @@ type
bandwidthEstimatebps*: int # This is currently used only for limting flood publishing. 0 disables flood-limiting completely
overheadRateLimit*: Opt[tuple[bytes: int, interval: Duration]]
disconnectPeerAboveRateLimit*: bool
BackoffTable* = Table[string, Table[PeerId, Moment]]
ValidationSeenTable* = Table[MessageId, HashSet[PubSubPeer]]

View File

@ -17,6 +17,7 @@
import std/[tables, sequtils, sets, strutils]
import chronos, chronicles, metrics
import chronos/ratelimit
import ./errors as pubsub_errors,
./pubsubpeer,
./rpc/[message, messages, protobuf],
@ -263,7 +264,7 @@ proc updateMetrics*(p: PubSub, rpcMsg: RPCMsg) =
method rpcHandler*(p: PubSub,
peer: PubSubPeer,
rpcMsg: RPCMsg): Future[void] {.base, async.} =
data: seq[byte]): Future[void] {.base, async.} =
## Handler that must be overridden by concrete implementation
raiseAssert "Unimplemented"
@ -278,10 +279,11 @@ method onPubSubPeerEvent*(p: PubSub, peer: PubSubPeer, event: PubSubPeerEvent) {
of PubSubPeerEventKind.Disconnected:
discard
proc getOrCreatePeer*(
method getOrCreatePeer*(
p: PubSub,
peerId: PeerId,
protos: seq[string]): PubSubPeer =
protos: seq[string]): PubSubPeer {.base, gcsafe.} =
p.peers.withValue(peerId, peer):
return peer[]
@ -354,9 +356,9 @@ method handleConn*(p: PubSub,
## that we're interested in
##
proc handler(peer: PubSubPeer, msg: RPCMsg): Future[void] =
proc handler(peer: PubSubPeer, data: seq[byte]): Future[void] =
# call pubsub rpc handler
p.rpcHandler(peer, msg)
p.rpcHandler(peer, data)
let peer = p.getOrCreatePeer(conn.peerId, @[proto])

View File

@ -12,6 +12,7 @@
import std/[sequtils, strutils, tables, hashes, options, sets, deques]
import stew/results
import chronos, chronicles, nimcrypto/sha2, metrics
import chronos/ratelimit
import rpc/[messages, message, protobuf],
../../peerid,
../../peerinfo,
@ -32,6 +33,8 @@ when defined(libp2p_expensive_metrics):
declareCounter(libp2p_pubsub_skipped_sent_messages, "number of sent skipped messages", labels = ["id"])
type
PeerRateLimitError* = object of CatchableError
PubSubObserver* = ref object
onRecv*: proc(peer: PubSubPeer; msgs: var RPCMsg) {.gcsafe, raises: [].}
onSend*: proc(peer: PubSubPeer; msgs: var RPCMsg) {.gcsafe, raises: [].}
@ -66,8 +69,9 @@ type
maxMessageSize: int
appScore*: float64 # application specific score
behaviourPenalty*: float64 # the eventual penalty score
overheadRateLimitOpt*: Opt[TokenBucket]
RPCHandler* = proc(peer: PubSubPeer, msg: RPCMsg): Future[void]
RPCHandler* = proc(peer: PubSubPeer, data: seq[byte]): Future[void]
{.gcsafe, raises: [].}
when defined(libp2p_agents_metrics):
@ -107,7 +111,7 @@ func outbound*(p: PubSubPeer): bool =
else:
false
proc recvObservers(p: PubSubPeer, msg: var RPCMsg) =
proc recvObservers*(p: PubSubPeer, msg: var RPCMsg) =
# trigger hooks
if not(isNil(p.observers)) and p.observers[].len > 0:
for obs in p.observers[]:
@ -134,26 +138,19 @@ proc handle*(p: PubSubPeer, conn: Connection) {.async.} =
conn, peer = p, closed = conn.closed,
data = data.shortLog
var rmsg = decodeRpcMsg(data).valueOr:
debug "failed to decode msg from peer",
conn, peer = p, closed = conn.closed,
err = error
break
data = newSeq[byte]() # Release memory
trace "decoded msg from peer",
conn, peer = p, closed = conn.closed,
msg = rmsg.shortLog
# trigger hooks
p.recvObservers(rmsg)
when defined(libp2p_expensive_metrics):
for m in rmsg.messages:
for t in m.topicIDs:
# metrics
libp2p_pubsub_received_messages.inc(labelValues = [$p.peerId, t])
await p.handler(p, rmsg)
await p.handler(p, data)
data = newSeq[byte]() # Release memory
except PeerRateLimitError as exc:
debug "Peer rate limit exceeded, exiting read while", conn, peer = p, error = exc.msg
except CatchableError as exc:
debug "Exception occurred in PubSubPeer.handle",
conn, peer = p, closed = conn.closed, exc = exc.msg
finally:
await conn.close()
except CancelledError:
@ -245,7 +242,7 @@ proc sendEncoded*(p: PubSubPeer, msg: seq[byte]) {.raises: [], async.} =
return
if msg.len > p.maxMessageSize:
info "trying to send a too big for pubsub", maxSize=p.maxMessageSize, msgSize=msg.len
info "trying to send a msg too big for pubsub", maxSize=p.maxMessageSize, msgSize=msg.len
return
if p.sendConn == nil:
@ -272,9 +269,42 @@ proc sendEncoded*(p: PubSubPeer, msg: seq[byte]) {.raises: [], async.} =
await conn.close() # This will clean up the send connection
proc send*(p: PubSubPeer, msg: RPCMsg, anonymize: bool) {.raises: [].} =
trace "sending msg to peer", peer = p, rpcMsg = shortLog(msg)
iterator splitRPCMsg(peer: PubSubPeer, rpcMsg: RPCMsg, maxSize: int, anonymize: bool): seq[byte] =
## This iterator takes an `RPCMsg` and sequentially repackages its Messages into new `RPCMsg` instances.
## Each new `RPCMsg` accumulates Messages until reaching the specified `maxSize`. If a single Message
## exceeds the `maxSize` when trying to fit into an empty `RPCMsg`, the latter is skipped as too large to send.
## Every constructed `RPCMsg` is then encoded, optionally anonymized, and yielded as a sequence of bytes.
var currentRPCMsg = rpcMsg
currentRPCMsg.messages = newSeq[Message]()
var currentSize = byteSize(currentRPCMsg)
for msg in rpcMsg.messages:
let msgSize = byteSize(msg)
# Check if adding the next message will exceed maxSize
if float(currentSize + msgSize) * 1.1 > float(maxSize): # Guessing 10% protobuf overhead
if currentRPCMsg.messages.len == 0:
trace "message too big to sent", peer, rpcMsg = shortLog(currentRPCMsg)
continue # Skip this message
trace "sending msg to peer", peer, rpcMsg = shortLog(currentRPCMsg)
yield encodeRpcMsg(currentRPCMsg, anonymize)
currentRPCMsg = RPCMsg()
currentSize = 0
currentRPCMsg.messages.add(msg)
currentSize += msgSize
# Check if there is a non-empty currentRPCMsg left to be added
if currentSize > 0 and currentRPCMsg.messages.len > 0:
trace "sending msg to peer", peer, rpcMsg = shortLog(currentRPCMsg)
yield encodeRpcMsg(currentRPCMsg, anonymize)
else:
trace "message too big to sent", peer, rpcMsg = shortLog(currentRPCMsg)
proc send*(p: PubSubPeer, msg: RPCMsg, anonymize: bool) {.raises: [].} =
# When sending messages, we take care to re-encode them with the right
# anonymization flag to ensure that we're not penalized for sending invalid
# or malicious data on the wire - in particular, re-encoding protects against
@ -292,7 +322,13 @@ proc send*(p: PubSubPeer, msg: RPCMsg, anonymize: bool) {.raises: [].} =
sendMetrics(msg)
encodeRpcMsg(msg, anonymize)
asyncSpawn p.sendEncoded(encoded)
if encoded.len > p.maxMessageSize and msg.messages.len > 1:
for encodedSplitMsg in splitRPCMsg(p, msg, p.maxMessageSize, anonymize):
asyncSpawn p.sendEncoded(encodedSplitMsg)
else:
# If the message size is within limits, send it as is
trace "sending msg to peer", peer = p, rpcMsg = shortLog(msg)
asyncSpawn p.sendEncoded(encoded)
proc canAskIWant*(p: PubSubPeer, msgId: MessageId): bool =
for sentIHave in p.sentIHaves.mitems():
@ -307,7 +343,8 @@ proc new*(
getConn: GetConn,
onEvent: OnEvent,
codec: string,
maxMessageSize: int): T =
maxMessageSize: int,
overheadRateLimitOpt: Opt[TokenBucket] = Opt.none(TokenBucket)): T =
result = T(
getConn: getConn,
@ -315,7 +352,18 @@ proc new*(
codec: codec,
peerId: peerId,
connectedFut: newFuture[void](),
maxMessageSize: maxMessageSize
maxMessageSize: maxMessageSize,
overheadRateLimitOpt: overheadRateLimitOpt
)
result.sentIHaves.addFirst(default(HashSet[MessageId]))
result.heDontWants.addFirst(default(HashSet[MessageId]))
proc getAgent*(peer: PubSubPeer): string =
return
when defined(libp2p_agents_metrics):
if peer.shortAgent.len > 0:
peer.shortAgent
else:
"unknown"
else:
"unknown"

View File

@ -9,7 +9,7 @@
{.push raises: [].}
import options, sequtils
import options, sequtils, sugar
import "../../.."/[
peerid,
routing_record,
@ -18,6 +18,14 @@ import "../../.."/[
export options
proc expectedFields[T](t: typedesc[T], existingFieldNames: seq[string]) {.raises: [CatchableError].} =
var fieldNames: seq[string]
for name, _ in fieldPairs(T()):
fieldNames &= name
if fieldNames != existingFieldNames:
fieldNames.keepIf(proc(it: string): bool = it notin existingFieldNames)
raise newException(CatchableError, $T & " fields changed, please search for and revise all relevant procs. New fields: " & $fieldNames)
type
PeerInfoMsg* = object
peerId*: PeerId
@ -116,3 +124,54 @@ func shortLog*(m: RPCMsg): auto =
messages: mapIt(m.messages, it.shortLog),
control: m.control.get(ControlMessage()).shortLog
)
static: expectedFields(PeerInfoMsg, @["peerId", "signedPeerRecord"])
proc byteSize(peerInfo: PeerInfoMsg): int =
peerInfo.peerId.len + peerInfo.signedPeerRecord.len
static: expectedFields(SubOpts, @["subscribe", "topic"])
proc byteSize(subOpts: SubOpts): int =
1 + subOpts.topic.len # 1 byte for the bool
static: expectedFields(Message, @["fromPeer", "data", "seqno", "topicIds", "signature", "key"])
proc byteSize*(msg: Message): int =
msg.fromPeer.len + msg.data.len + msg.seqno.len +
msg.signature.len + msg.key.len + msg.topicIds.foldl(a + b.len, 0)
proc byteSize*(msgs: seq[Message]): int =
msgs.foldl(a + b.byteSize, 0)
static: expectedFields(ControlIHave, @["topicId", "messageIds"])
proc byteSize(controlIHave: ControlIHave): int =
controlIHave.topicId.len + controlIHave.messageIds.foldl(a + b.len, 0)
proc byteSize*(ihaves: seq[ControlIHave]): int =
ihaves.foldl(a + b.byteSize, 0)
static: expectedFields(ControlIWant, @["messageIds"])
proc byteSize(controlIWant: ControlIWant): int =
controlIWant.messageIds.foldl(a + b.len, 0)
proc byteSize*(iwants: seq[ControlIWant]): int =
iwants.foldl(a + b.byteSize, 0)
static: expectedFields(ControlGraft, @["topicId"])
proc byteSize(controlGraft: ControlGraft): int =
controlGraft.topicId.len
static: expectedFields(ControlPrune, @["topicId", "peers", "backoff"])
proc byteSize(controlPrune: ControlPrune): int =
controlPrune.topicId.len + controlPrune.peers.foldl(a + b.byteSize, 0) + 8 # 8 bytes for uint64
static: expectedFields(ControlMessage, @["ihave", "iwant", "graft", "prune", "idontwant"])
proc byteSize(control: ControlMessage): int =
control.ihave.foldl(a + b.byteSize, 0) + control.iwant.foldl(a + b.byteSize, 0) +
control.graft.foldl(a + b.byteSize, 0) + control.prune.foldl(a + b.byteSize, 0) +
control.idontwant.foldl(a + b.byteSize, 0)
static: expectedFields(RPCMsg, @["subscriptions", "messages", "control", "ping", "pong"])
proc byteSize*(rpc: RPCMsg): int =
result = rpc.subscriptions.foldl(a + b.byteSize, 0) + byteSize(rpc.messages) +
rpc.ping.len + rpc.pong.len
rpc.control.withValue(ctrl):
result += ctrl.byteSize

View File

@ -469,6 +469,8 @@ proc advertisePeer(rdv: RendezVous,
trace "Unexpected register response", peer, msgType = msgRecv.msgType
elif msgRecv.registerResponse.tryGet().status != ResponseStatus.Ok:
trace "Refuse to register", peer, response = msgRecv.registerResponse
else:
trace "Successfully registered", peer, response = msgRecv.registerResponse
except CatchableError as exc:
trace "exception in the advertise", error = exc.msg
finally:
@ -476,9 +478,9 @@ proc advertisePeer(rdv: RendezVous,
await rdv.sema.acquire()
discard await advertiseWrap().withTimeout(5.seconds)
proc advertise*(rdv: RendezVous,
method advertise*(rdv: RendezVous,
ns: string,
ttl: Duration = MinimumDuration) {.async.} =
ttl: Duration = MinimumDuration) {.async, base.} =
let sprBuff = rdv.switch.peerInfo.signedPeerRecord.encode().valueOr:
raise newException(RendezVousError, "Wrong Signed Peer Record")
if ns.len notin 1..255:

View File

@ -219,7 +219,7 @@ method accept*(self: TcpTransport): Future[Connection] {.async, gcsafe.} =
try:
if self.acceptFuts.len <= 0:
self.acceptFuts = self.servers.mapIt(it.accept())
self.acceptFuts = self.servers.mapIt(Future[StreamTransport](it.accept()))
if self.acceptFuts.len <= 0:
return

View File

@ -276,6 +276,8 @@ method accept*(self: WsTransport): Future[Connection] {.async, gcsafe.} =
debug "AsyncStream Error", exc = exc.msg
except TransportTooManyError as exc:
debug "Too many files opened", exc = exc.msg
except TransportAbortedError as exc:
debug "Connection aborted", exc = exc.msg
except AsyncTimeoutError as exc:
debug "Timed out", exc = exc.msg
except TransportUseClosedError as exc:

View File

@ -91,13 +91,11 @@ proc new*(
T: type MuxedUpgrade,
muxers: seq[MuxerProvider],
secureManagers: openArray[Secure] = [],
connManager: ConnManager,
ms: MultistreamSelect): T =
let upgrader = T(
muxers: muxers,
secureManagers: @secureManagers,
connManager: connManager,
ms: ms)
upgrader.streamHandler = proc(conn: Connection)

View File

@ -35,7 +35,6 @@ type
Upgrade* = ref object of RootObj
ms*: MultistreamSelect
connManager*: ConnManager
secureManagers*: seq[Secure]
method upgrade*(

View File

@ -1,41 +1,31 @@
include ../../libp2p/protocols/pubsub/gossipsub
# Nim-LibP2P
# Copyright (c) 2023 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 std/[options, deques]
import std/[options, deques, sequtils, enumerate, algorithm]
import stew/byteutils
import ../../libp2p/builders
import ../../libp2p/errors
import ../../libp2p/crypto/crypto
import ../../libp2p/stream/bufferstream
import ../../libp2p/protocols/pubsub/[pubsub, gossipsub, mcache, mcache, peertable]
import ../../libp2p/protocols/pubsub/rpc/[message, messages]
import ../../libp2p/switch
import ../../libp2p/muxers/muxer
import ../../libp2p/protocols/pubsub/rpc/protobuf
import utils
import ../helpers
type
TestGossipSub = ref object of GossipSub
proc noop(data: seq[byte]) {.async, gcsafe.} = discard
proc getPubSubPeer(p: TestGossipSub, peerId: PeerId): PubSubPeer =
proc getConn(): Future[Connection] =
p.switch.dial(peerId, GossipSubCodec)
let pubSubPeer = PubSubPeer.new(peerId, getConn, nil, GossipSubCodec, 1024 * 1024)
debug "created new pubsub peer", peerId
p.peers[peerId] = pubSubPeer
onNewPeer(p, pubSubPeer)
pubSubPeer
proc randomPeerId(): PeerId =
try:
PeerId.init(PrivateKey.random(ECDSA, rng[]).get()).tryGet()
except CatchableError as exc:
raise newException(Defect, exc.msg)
const MsgIdSuccess = "msg id gen success"
suite "GossipSub internal":
@ -170,7 +160,7 @@ suite "GossipSub internal":
asyncTest "`replenishFanout` Degree Lo":
let gossipSub = TestGossipSub.init(newStandardSwitch())
proc handler(peer: PubSubPeer, msg: RPCMsg) {.async.} =
proc handler(peer: PubSubPeer, data: seq[byte]) {.async.} =
discard
let topic = "foobar"
@ -197,7 +187,7 @@ suite "GossipSub internal":
asyncTest "`dropFanoutPeers` drop expired fanout topics":
let gossipSub = TestGossipSub.init(newStandardSwitch())
proc handler(peer: PubSubPeer, msg: RPCMsg) {.async.} =
proc handler(peer: PubSubPeer, data: seq[byte]) {.async.} =
discard
let topic = "foobar"
@ -227,7 +217,7 @@ suite "GossipSub internal":
asyncTest "`dropFanoutPeers` leave unexpired fanout topics":
let gossipSub = TestGossipSub.init(newStandardSwitch())
proc handler(peer: PubSubPeer, msg: RPCMsg) {.async.} =
proc handler(peer: PubSubPeer, data: seq[byte]) {.async.} =
discard
let topic1 = "foobar1"
@ -264,7 +254,7 @@ suite "GossipSub internal":
asyncTest "`getGossipPeers` - should gather up to degree D non intersecting peers":
let gossipSub = TestGossipSub.init(newStandardSwitch())
proc handler(peer: PubSubPeer, msg: RPCMsg) {.async.} =
proc handler(peer: PubSubPeer, data: seq[byte]) {.async.} =
discard
let topic = "foobar"
@ -325,7 +315,7 @@ suite "GossipSub internal":
asyncTest "`getGossipPeers` - should not crash on missing topics in mesh":
let gossipSub = TestGossipSub.init(newStandardSwitch())
proc handler(peer: PubSubPeer, msg: RPCMsg) {.async.} =
proc handler(peer: PubSubPeer, data: seq[byte]) {.async.} =
discard
let topic = "foobar"
@ -365,7 +355,7 @@ suite "GossipSub internal":
asyncTest "`getGossipPeers` - should not crash on missing topics in fanout":
let gossipSub = TestGossipSub.init(newStandardSwitch())
proc handler(peer: PubSubPeer, msg: RPCMsg) {.async.} =
proc handler(peer: PubSubPeer, data: seq[byte]) {.async.} =
discard
let topic = "foobar"
@ -406,7 +396,7 @@ suite "GossipSub internal":
asyncTest "`getGossipPeers` - should not crash on missing topics in gossip":
let gossipSub = TestGossipSub.init(newStandardSwitch())
proc handler(peer: PubSubPeer, msg: RPCMsg) {.async.} =
proc handler(peer: PubSubPeer, data: seq[byte]) {.async.} =
discard
let topic = "foobar"
@ -447,7 +437,7 @@ suite "GossipSub internal":
asyncTest "Drop messages of topics without subscription":
let gossipSub = TestGossipSub.init(newStandardSwitch())
proc handler(peer: PubSubPeer, msg: RPCMsg) {.async.} =
proc handler(peer: PubSubPeer, data: seq[byte]) {.async.} =
check false
let topic = "foobar"
@ -470,7 +460,7 @@ suite "GossipSub internal":
let peer = gossipSub.getPubSubPeer(peerId)
inc seqno
let msg = Message.init(peerId, ("bar" & $i).toBytes(), topic, some(seqno))
await gossipSub.rpcHandler(peer, RPCMsg(messages: @[msg]))
await gossipSub.rpcHandler(peer, encodeRpcMsg(RPCMsg(messages: @[msg]), false))
check gossipSub.mcache.msgs.len == 0
@ -481,7 +471,7 @@ suite "GossipSub internal":
let gossipSub = TestGossipSub.init(newStandardSwitch())
gossipSub.parameters.disconnectBadPeers = true
gossipSub.parameters.appSpecificWeight = 1.0
proc handler(peer: PubSubPeer, msg: RPCMsg) {.async.} =
proc handler(peer: PubSubPeer, data: seq[byte]) {.async.} =
check false
let topic = "foobar"
@ -525,7 +515,7 @@ suite "GossipSub internal":
conn.peerId = peerId
let peer = gossipSub.getPubSubPeer(peerId)
await gossipSub.rpcHandler(peer, lotOfSubs)
await gossipSub.rpcHandler(peer, encodeRpcMsg(lotOfSubs, false))
check:
gossipSub.gossipsub.len == gossipSub.topicsHigh
@ -656,7 +646,7 @@ suite "GossipSub internal":
asyncTest "handleIHave/Iwant tests":
let gossipSub = TestGossipSub.init(newStandardSwitch())
proc handler(peer: PubSubPeer, msg: RPCMsg) {.async.} =
proc handler(peer: PubSubPeer, data: seq[byte]) {.async.} =
check false
proc handler2(topic: string, data: seq[byte]) {.async.} = discard
@ -727,3 +717,130 @@ suite "GossipSub internal":
await allFuturesThrowing(conns.mapIt(it.close()))
await gossipSub.switch.stop()
proc setupTest(): Future[tuple[gossip0: GossipSub, gossip1: GossipSub, receivedMessages: ref HashSet[seq[byte]]]] {.async.} =
let
nodes = generateNodes(2, gossip = true, verifySignature = false)
discard await allFinished(
nodes[0].switch.start(),
nodes[1].switch.start()
)
await nodes[1].switch.connect(nodes[0].switch.peerInfo.peerId, nodes[0].switch.peerInfo.addrs)
var receivedMessages = new(HashSet[seq[byte]])
proc handlerA(topic: string, data: seq[byte]) {.async, gcsafe.} =
receivedMessages[].incl(data)
proc handlerB(topic: string, data: seq[byte]) {.async, gcsafe.} =
discard
nodes[0].subscribe("foobar", handlerA)
nodes[1].subscribe("foobar", handlerB)
await waitSubGraph(nodes, "foobar")
var gossip0: GossipSub = GossipSub(nodes[0])
var gossip1: GossipSub = GossipSub(nodes[1])
return (gossip0, gossip1, receivedMessages)
proc teardownTest(gossip0: GossipSub, gossip1: GossipSub) {.async.} =
await allFuturesThrowing(
gossip0.switch.stop(),
gossip1.switch.stop()
)
proc createMessages(gossip0: GossipSub, gossip1: GossipSub, size1: int, size2: int): tuple[iwantMessageIds: seq[MessageId], sentMessages: HashSet[seq[byte]]] =
var iwantMessageIds = newSeq[MessageId]()
var sentMessages = initHashSet[seq[byte]]()
for i, size in enumerate([size1, size2]):
let data = newSeqWith[byte](size, i.byte)
sentMessages.incl(data)
let msg = Message.init(gossip1.peerInfo.peerId, data, "foobar", some(uint64(i + 1)))
let iwantMessageId = gossip1.msgIdProvider(msg).expect(MsgIdSuccess)
iwantMessageIds.add(iwantMessageId)
gossip1.mcache.put(iwantMessageId, msg)
let peer = gossip1.peers[(gossip0.peerInfo.peerId)]
peer.sentIHaves[^1].incl(iwantMessageId)
return (iwantMessageIds, sentMessages)
asyncTest "e2e - Split IWANT replies when individual messages are below maxSize but combined exceed maxSize":
# This test checks if two messages, each below the maxSize, are correctly split when their combined size exceeds maxSize.
# Expected: Both messages should be received.
let (gossip0, gossip1, receivedMessages) = await setupTest()
let messageSize = gossip1.maxMessageSize div 2 + 1
let (iwantMessageIds, sentMessages) = createMessages(gossip0, gossip1, messageSize, messageSize)
gossip1.broadcast(gossip1.mesh["foobar"], RPCMsg(control: some(ControlMessage(
ihave: @[ControlIHave(topicId: "foobar", messageIds: iwantMessageIds)]
))))
checkExpiring: receivedMessages[] == sentMessages
check receivedMessages[].len == 2
await teardownTest(gossip0, gossip1)
asyncTest "e2e - Discard IWANT replies when both messages individually exceed maxSize":
# This test checks if two messages, each exceeding the maxSize, are discarded and not sent.
# Expected: No messages should be received.
let (gossip0, gossip1, receivedMessages) = await setupTest()
let messageSize = gossip1.maxMessageSize + 10
let (bigIWantMessageIds, sentMessages) = createMessages(gossip0, gossip1, messageSize, messageSize)
gossip1.broadcast(gossip1.mesh["foobar"], RPCMsg(control: some(ControlMessage(
ihave: @[ControlIHave(topicId: "foobar", messageIds: bigIWantMessageIds)]
))))
await sleepAsync(300.milliseconds)
checkExpiring: receivedMessages[].len == 0
await teardownTest(gossip0, gossip1)
asyncTest "e2e - Process IWANT replies when both messages are below maxSize":
# This test checks if two messages, both below the maxSize, are correctly processed and sent.
# Expected: Both messages should be received.
let (gossip0, gossip1, receivedMessages) = await setupTest()
let size1 = gossip1.maxMessageSize div 2
let size2 = gossip1.maxMessageSize div 3
let (bigIWantMessageIds, sentMessages) = createMessages(gossip0, gossip1, size1, size2)
gossip1.broadcast(gossip1.mesh["foobar"], RPCMsg(control: some(ControlMessage(
ihave: @[ControlIHave(topicId: "foobar", messageIds: bigIWantMessageIds)]
))))
checkExpiring: receivedMessages[] == sentMessages
check receivedMessages[].len == 2
await teardownTest(gossip0, gossip1)
asyncTest "e2e - Split IWANT replies when one message is below maxSize and the other exceeds maxSize":
# This test checks if, when given two messages where one is below maxSize and the other exceeds it, only the smaller message is processed and sent.
# Expected: Only the smaller message should be received.
let (gossip0, gossip1, receivedMessages) = await setupTest()
let maxSize = gossip1.maxMessageSize
let size1 = maxSize div 2
let size2 = maxSize + 10
let (bigIWantMessageIds, sentMessages) = createMessages(gossip0, gossip1, size1, size2)
gossip1.broadcast(gossip1.mesh["foobar"], RPCMsg(control: some(ControlMessage(
ihave: @[ControlIHave(topicId: "foobar", messageIds: bigIWantMessageIds)]
))))
var smallestSet: HashSet[seq[byte]]
let seqs = toSeq(sentMessages)
if seqs[0] < seqs[1]:
smallestSet.incl(seqs[0])
else:
smallestSet.incl(seqs[1])
checkExpiring: receivedMessages[] == smallestSet
check receivedMessages[].len == 1
await teardownTest(gossip0, gossip1)

View File

@ -10,8 +10,9 @@
{.used.}
import sequtils, options, tables, sets, sugar
import chronos, stew/byteutils
import chronos, stew/byteutils, chronos/ratelimit
import chronicles
import metrics
import utils, ../../libp2p/[errors,
peerid,
peerinfo,
@ -20,6 +21,7 @@ import utils, ../../libp2p/[errors,
crypto/crypto,
protocols/pubsub/pubsub,
protocols/pubsub/gossipsub,
protocols/pubsub/gossipsub/scoring,
protocols/pubsub/pubsubpeer,
protocols/pubsub/peertable,
protocols/pubsub/timedcache,
@ -928,3 +930,136 @@ suite "GossipSub":
await allFuturesThrowing(nodesFut.concat())
proc initializeGossipTest(): Future[(seq[PubSub], GossipSub, GossipSub)] {.async.} =
let nodes = generateNodes(
2,
gossip = true,
overheadRateLimit = Opt.some((20, 1.millis)))
discard await allFinished(
nodes[0].switch.start(),
nodes[1].switch.start(),
)
await subscribeNodes(nodes)
proc handle(topic: string, data: seq[byte]) {.async, gcsafe.} = discard
let gossip0 = GossipSub(nodes[0])
let gossip1 = GossipSub(nodes[1])
gossip0.subscribe("foobar", handle)
gossip1.subscribe("foobar", handle)
await waitSubGraph(nodes, "foobar")
# Avoid being disconnected by failing signature verification
gossip0.verifySignature = false
gossip1.verifySignature = false
return (nodes, gossip0, gossip1)
proc currentRateLimitHits(): float64 =
try:
libp2p_gossipsub_peers_rate_limit_hits.valueByName("libp2p_gossipsub_peers_rate_limit_hits_total", @["nim-libp2p"])
except KeyError:
0
asyncTest "e2e - GossipSub should not rate limit decodable messages below the size allowed":
let rateLimitHits = currentRateLimitHits()
let (nodes, gossip0, gossip1) = await initializeGossipTest()
gossip0.broadcast(gossip0.mesh["foobar"], RPCMsg(messages: @[Message(topicIDs: @["foobar"], data: newSeq[byte](10))]))
await sleepAsync(300.millis)
check currentRateLimitHits() == rateLimitHits
check gossip1.switch.isConnected(gossip0.switch.peerInfo.peerId) == true
# Disconnect peer when rate limiting is enabled
gossip1.parameters.disconnectPeerAboveRateLimit = true
gossip0.broadcast(gossip0.mesh["foobar"], RPCMsg(messages: @[Message(topicIDs: @["foobar"], data: newSeq[byte](12))]))
await sleepAsync(300.millis)
check gossip1.switch.isConnected(gossip0.switch.peerInfo.peerId) == true
check currentRateLimitHits() == rateLimitHits
await stopNodes(nodes)
asyncTest "e2e - GossipSub should rate limit undecodable messages above the size allowed":
let rateLimitHits = currentRateLimitHits()
let (nodes, gossip0, gossip1) = await initializeGossipTest()
# Simulate sending an undecodable message
await gossip1.peers[gossip0.switch.peerInfo.peerId].sendEncoded(newSeqWith[byte](33, 1.byte))
await sleepAsync(300.millis)
check currentRateLimitHits() == rateLimitHits + 1
check gossip1.switch.isConnected(gossip0.switch.peerInfo.peerId) == true
# Disconnect peer when rate limiting is enabled
gossip1.parameters.disconnectPeerAboveRateLimit = true
await gossip0.peers[gossip1.switch.peerInfo.peerId].sendEncoded(newSeqWith[byte](35, 1.byte))
checkExpiring gossip1.switch.isConnected(gossip0.switch.peerInfo.peerId) == false
check currentRateLimitHits() == rateLimitHits + 2
await stopNodes(nodes)
asyncTest "e2e - GossipSub should rate limit decodable messages above the size allowed":
let rateLimitHits = currentRateLimitHits()
let (nodes, gossip0, gossip1) = await initializeGossipTest()
let msg = RPCMsg(control: some(ControlMessage(prune: @[
ControlPrune(topicID: "foobar", peers: @[
PeerInfoMsg(peerId: PeerId(data: newSeq[byte](33)))
], backoff: 123'u64)
])))
gossip0.broadcast(gossip0.mesh["foobar"], msg)
await sleepAsync(300.millis)
check currentRateLimitHits() == rateLimitHits + 1
check gossip1.switch.isConnected(gossip0.switch.peerInfo.peerId) == true
# Disconnect peer when rate limiting is enabled
gossip1.parameters.disconnectPeerAboveRateLimit = true
let msg2 = RPCMsg(control: some(ControlMessage(prune: @[
ControlPrune(topicID: "foobar", peers: @[
PeerInfoMsg(peerId: PeerId(data: newSeq[byte](35)))
], backoff: 123'u64)
])))
gossip0.broadcast(gossip0.mesh["foobar"], msg2)
checkExpiring gossip1.switch.isConnected(gossip0.switch.peerInfo.peerId) == false
check currentRateLimitHits() == rateLimitHits + 2
await stopNodes(nodes)
asyncTest "e2e - GossipSub should rate limit invalid messages above the size allowed":
let rateLimitHits = currentRateLimitHits()
let (nodes, gossip0, gossip1) = await initializeGossipTest()
let topic = "foobar"
proc execValidator(topic: string, message: messages.Message): Future[ValidationResult] {.raises: [].} =
let res = newFuture[ValidationResult]()
res.complete(ValidationResult.Reject)
res
gossip0.addValidator(topic, execValidator)
gossip1.addValidator(topic, execValidator)
let msg = RPCMsg(messages: @[Message(topicIDs: @[topic], data: newSeq[byte](40))])
gossip0.broadcast(gossip0.mesh[topic], msg)
await sleepAsync(300.millis)
check currentRateLimitHits() == rateLimitHits + 1
check gossip1.switch.isConnected(gossip0.switch.peerInfo.peerId) == true
# Disconnect peer when rate limiting is enabled
gossip1.parameters.disconnectPeerAboveRateLimit = true
gossip0.broadcast(gossip0.mesh[topic], RPCMsg(messages: @[Message(topicIDs: @[topic], data: newSeq[byte](35))]))
checkExpiring gossip1.switch.isConnected(gossip0.switch.peerInfo.peerId) == false
check currentRateLimitHits() == rateLimitHits + 2
await stopNodes(nodes)

View File

@ -2,10 +2,10 @@ import unittest2
{.used.}
import options
import options, strutils
import stew/byteutils
import ../../libp2p/[peerid, peerinfo,
crypto/crypto,
crypto/crypto as crypto,
protocols/pubsub/errors,
protocols/pubsub/rpc/message,
protocols/pubsub/rpc/messages]
@ -28,7 +28,7 @@ suite "Message":
"""08011240B9EA7F0357B5C1247E4FCB5AD09C46818ECB07318CA84711875F4C6C
E6B946186A4EB44E0D714B2A2D48263D75CF52D30BEF9D9AE2A9FEB7DAF1775F
E731065A"""
seckey = PrivateKey.init(fromHex(stripSpaces(pkHex)))
seckey = PrivateKey.init(crypto.fromHex(stripSpaces(pkHex)))
.expect("valid private key bytes")
peer = PeerInfo.new(seckey)
msg = Message.init(some(peer), @[], "topic", some(seqno), sign = true)
@ -46,7 +46,7 @@ suite "Message":
"""08011240B9EA7F0357B5C1247E4FCB5AD09C46818ECB07318CA84711875F4C6C
E6B946186A4EB44E0D714B2A2D48263D75CF52D30BEF9D9AE2A9FEB7DAF1775F
E731065A"""
seckey = PrivateKey.init(fromHex(stripSpaces(pkHex)))
seckey = PrivateKey.init(crypto.fromHex(stripSpaces(pkHex)))
.expect("valid private key bytes")
peer = PeerInfo.new(seckey)
@ -64,7 +64,7 @@ suite "Message":
"""08011240B9EA7F0357B5C1247E4FCB5AD09C46818ECB07318CA84711875F4C6C
E6B946186A4EB44E0D714B2A2D48263D75CF52D30BEF9D9AE2A9FEB7DAF1775F
E731065A"""
seckey = PrivateKey.init(fromHex(stripSpaces(pkHex)))
seckey = PrivateKey.init(crypto.fromHex(stripSpaces(pkHex)))
.expect("valid private key bytes")
peer = PeerInfo.new(seckey)
msg = Message.init(some(peer), @[], "topic", uint64.none, sign = true)
@ -73,3 +73,55 @@ suite "Message":
check:
msgIdResult.isErr
msgIdResult.error == ValidationResult.Reject
test "byteSize for RPCMsg":
var msg = Message(
fromPeer: PeerId(data: @['a'.byte, 'b'.byte]), # 2 bytes
data: @[1'u8, 2, 3], # 3 bytes
seqno: @[4'u8, 5], # 2 bytes
signature: @['c'.byte, 'd'.byte], # 2 bytes
key: @[6'u8, 7], # 2 bytes
topicIds: @["abc", "defgh"] # 3 + 5 = 8 bytes
)
var peerInfo = PeerInfoMsg(
peerId: PeerId(data: @['e'.byte]), # 1 byte
signedPeerRecord: @['f'.byte, 'g'.byte] # 2 bytes
)
var controlIHave = ControlIHave(
topicId: "ijk", # 3 bytes
messageIds: @[ @['l'.byte], @['m'.byte, 'n'.byte] ] # 1 + 2 = 3 bytes
)
var controlIWant = ControlIWant(
messageIds: @[ @['o'.byte, 'p'.byte], @['q'.byte] ] # 2 + 1 = 3 bytes
)
var controlGraft = ControlGraft(
topicId: "rst" # 3 bytes
)
var controlPrune = ControlPrune(
topicId: "uvw", # 3 bytes
peers: @[peerInfo, peerInfo], # (1 + 2) * 2 = 6 bytes
backoff: 12345678 # 8 bytes for uint64
)
var control = ControlMessage(
ihave: @[controlIHave, controlIHave], # (3 + 3) * 2 = 12 bytes
iwant: @[controlIWant], # 3 bytes
graft: @[controlGraft], # 3 bytes
prune: @[controlPrune], # 3 + 6 + 8 = 17 bytes
idontwant: @[controlIWant] # 3 bytes
)
var rpcMsg = RPCMsg(
subscriptions: @[SubOpts(subscribe: true, topic: "a".repeat(12)), SubOpts(subscribe: false, topic: "b".repeat(14))], # 1 + 12 + 1 + 14 = 28 bytes
messages: @[msg, msg], # 19 * 2 = 38 bytes
ping: @[1'u8, 2], # 2 bytes
pong: @[3'u8, 4], # 2 bytes
control: some(control) # 12 + 3 + 3 + 17 + 3 = 38 bytes
)
check byteSize(rpcMsg) == 28 + 38 + 2 + 2 + 38 # Total: 108 bytes

View File

@ -5,20 +5,43 @@ const
libp2p_pubsub_anonymize {.booldefine.} = false
import hashes, random, tables, sets, sequtils
import chronos, stew/[byteutils, results]
import chronos, stew/[byteutils, results], chronos/ratelimit
import ../../libp2p/[builders,
protocols/pubsub/errors,
protocols/pubsub/pubsub,
protocols/pubsub/pubsubpeer,
protocols/pubsub/gossipsub,
protocols/pubsub/floodsub,
protocols/pubsub/rpc/messages,
protocols/secure/secure]
import ../helpers
import chronicles
export builders
randomize()
type
TestGossipSub* = ref object of GossipSub
proc getPubSubPeer*(p: TestGossipSub, peerId: PeerId): PubSubPeer =
proc getConn(): Future[Connection] =
p.switch.dial(peerId, GossipSubCodec)
let pubSubPeer = PubSubPeer.new(peerId, getConn, nil, GossipSubCodec, 1024 * 1024)
debug "created new pubsub peer", peerId
p.peers[peerId] = pubSubPeer
onNewPeer(p, pubSubPeer)
pubSubPeer
proc randomPeerId*(): PeerId =
try:
PeerId.init(PrivateKey.random(ECDSA, rng[]).get()).tryGet()
except CatchableError as exc:
raise newException(Defect, exc.msg)
func defaultMsgIdProvider*(m: Message): Result[MessageId, ValidationResult] =
let mid =
if m.seqno.len > 0 and m.fromPeer.data.len > 0:
@ -44,7 +67,8 @@ proc generateNodes*(
sendSignedPeerRecord = false,
unsubscribeBackoff = 1.seconds,
maxMessageSize: int = 1024 * 1024,
enablePX: bool = false): seq[PubSub] =
enablePX: bool = false,
overheadRateLimit: Opt[tuple[bytes: int, interval: Duration]] = Opt.none(tuple[bytes: int, interval: Duration])): seq[PubSub] =
for i in 0..<num:
let switch = newStandardSwitch(secureManagers = secureManagers, sendSignedPeerRecord = sendSignedPeerRecord)
@ -57,7 +81,7 @@ proc generateNodes*(
msgIdProvider = msgIdProvider,
anonymize = anonymize,
maxMessageSize = maxMessageSize,
parameters = (var p = GossipSubParams.init(); p.floodPublish = false; p.historyLength = 20; p.historyGossip = 20; p.unsubscribeBackoff = unsubscribeBackoff; p.enablePX = enablePX; p))
parameters = (var p = GossipSubParams.init(); p.floodPublish = false; p.historyLength = 20; p.historyGossip = 20; p.unsubscribeBackoff = unsubscribeBackoff; p.enablePX = enablePX; p.overheadRateLimit = overheadRateLimit; p))
# set some testing params, to enable scores
g.topicParams.mgetOrPut("foobar", TopicParams.init()).topicWeight = 1.0
g.topicParams.mgetOrPut("foo", TopicParams.init()).topicWeight = 1.0

View File

@ -72,7 +72,7 @@ proc createSwitch(ma: MultiAddress; outgoing: bool, secio: bool = false): (Switc
[Secure(Noise.new(rng, privateKey, outgoing = outgoing))]
connManager = ConnManager.new()
ms = MultistreamSelect.new()
muxedUpgrade = MuxedUpgrade.new(muxers, secureManagers, connManager, ms)
muxedUpgrade = MuxedUpgrade.new(muxers, secureManagers, ms)
transports = @[Transport(TcpTransport.new(upgrade = muxedUpgrade))]
let switch = newSwitch(

View File

@ -14,6 +14,7 @@ import chronos
import ../libp2p/[protocols/rendezvous,
switch,
builders,]
import ../libp2p/discovery/[rendezvousinterface, discoverymngr]
import ./helpers
proc createSwitch(rdv: RendezVous = RendezVous.new()): Switch =

View File

@ -0,0 +1,73 @@
{.used.}
# Nim-Libp2p
# Copyright (c) 2023 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 sequtils, strutils
import chronos
import ../libp2p/[protocols/rendezvous,
switch,
builders,]
import ../libp2p/discovery/[rendezvousinterface, discoverymngr]
import ./helpers
proc createSwitch(rdv: RendezVous = RendezVous.new()): Switch =
SwitchBuilder.new()
.withRng(newRng())
.withAddresses(@[ MultiAddress.init("/ip4/0.0.0.0/tcp/0").tryGet() ])
.withTcpTransport()
.withMplex()
.withNoise()
.withRendezVous(rdv)
.build()
type
MockRendezVous = ref object of RendezVous
numAdvertiseNs1: int
numAdvertiseNs2: int
MockErrorRendezVous = ref object of MockRendezVous
method advertise*(self: MockRendezVous, namespace: string, ttl: Duration) {.async.} =
if namespace == "ns1":
self.numAdvertiseNs1 += 1
elif namespace == "ns2":
self.numAdvertiseNs2 += 1
# Forward the call to the actual implementation
await procCall RendezVous(self).advertise(namespace, ttl)
method advertise*(self: MockErrorRendezVous, namespace: string, ttl: Duration) {.async.} =
await procCall MockRendezVous(self).advertise(namespace, ttl)
raise newException(CatchableError, "MockErrorRendezVous.advertise")
suite "RendezVous Interface":
teardown:
checkTrackers()
proc baseTimeToAdvertiseTest(rdv: MockRendezVous) {.async.} =
let
tta = 100.milliseconds
ttl = 2.hours
client = createSwitch(rdv)
dm = DiscoveryManager()
await client.start()
dm.add(RendezVousInterface.new(rdv = rdv, tta = tta, ttl = ttl))
dm.advertise(RdvNamespace("ns1"))
dm.advertise(RdvNamespace("ns2"))
checkExpiring: rdv.numAdvertiseNs1 >= 5
checkExpiring: rdv.numAdvertiseNs2 >= 5
await client.stop()
asyncTest "Check timeToAdvertise interval":
await baseTimeToAdvertiseTest(MockRendezVous.new(newRng()))
asyncTest "Check timeToAdvertise interval when there is an error":
await baseTimeToAdvertiseTest(MockErrorRendezVous.new(newRng()))