Skip to content

Commit

Permalink
GossipSub Traffic scoring (#920)
Browse files Browse the repository at this point in the history
  • Loading branch information
diegomrsantos authored Sep 22, 2023
1 parent b2eac7e commit 56599f5
Show file tree
Hide file tree
Showing 9 changed files with 194 additions and 80 deletions.
13 changes: 11 additions & 2 deletions libp2p/protocols/pubsub/floodsub.nim
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@ import ./pubsub,
./pubsubpeer,
./timedcache,
./peertable,
./rpc/[message, messages],
./rpc/[message, messages, protobuf],
../../crypto/crypto,
../../stream/connection,
../../peerid,
Expand Down Expand Up @@ -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)
Expand Down
74 changes: 67 additions & 7 deletions libp2p/protocols/pubsub/gossipsub.nim
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down Expand Up @@ -78,7 +79,8 @@ proc init*(_: type[GossipSubParams]): GossipSubParams =
disconnectBadPeers: false,
enablePX: false,
bandwidthEstimatebps: 100_000_000, # 100 Mbps or 12.5 MBps
iwantTimeout: 3 * GossipSubHeartbeatInterval
iwantTimeout: 3 * GossipSubHeartbeatInterval,
overheadRateLimit: Opt.none(tuple[bytes: int, interval: Duration])
)

proc validateParameters*(parameters: GossipSubParams): Result[void, cstring] =
Expand Down Expand Up @@ -160,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
Expand Down Expand Up @@ -316,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)
g.punishInvalidMessage(peer, msg)
return
of ValidationResult.Ignore:
debug "Dropping message after validation, reason: ignore",
Expand Down Expand Up @@ -379,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].} =
# 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_disconnections.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
# discard g.disconnectPeer(peer)
# debug "Peer disconnected", peer, uselessAppBytesNum = msgSize
# raise newException(PeerRateLimitError, "Peer sent a msg that couldn't be decoded and 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_disconnections.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
# discard g.disconnectPeer(peer)
# debug "Peer disconnected", peer, msgSize, uselessAppBytesNum
# raise newException(PeerRateLimitError, "Peer sent too much useless application data and 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
rateLimit(g, peer, Opt.none(RPCMsg), msgSize)
return

trace "decoded msg from peer", peer, msg = rpcMsg.shortLog
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
Expand Down Expand Up @@ -445,14 +495,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)
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)
g.punishInvalidMessage(peer, msg)
continue

# g.anonymize needs no evaluation when receiving messages
Expand Down Expand Up @@ -676,3 +726,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
54 changes: 22 additions & 32 deletions libp2p/protocols/pubsub/gossipsub/scoring.nim
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand All @@ -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_disconnections, "The number of peer disconnections by gossipsub because of rate limit", labels = ["agent"])

proc init*(_: type[TopicParams]): TopicParams =
TopicParams(
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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])
Expand Down Expand Up @@ -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])
Expand All @@ -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:
Expand All @@ -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) =
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_disconnections.inc(labelValues = [peer.getAgent()]) # let's just measure at the beginning for test purposes.
# discard g.disconnectPeer(peer)
# debug "Peer disconnected", peer, uselessAppBytesNum
# raise newException(PeerRateLimitError, "Peer sent invalid message and it's above rate limit")


for tt in msg.topicIds:
let t = tt
if t notin g.topics:
continue
Expand Down
2 changes: 2 additions & 0 deletions libp2p/protocols/pubsub/gossipsub/types.nim
Original file line number Diff line number Diff line change
Expand Up @@ -145,6 +145,8 @@ type
bandwidthEstimatebps*: int # This is currently used only for limting flood publishing. 0 disables flood-limiting completely
iwantTimeout*: Duration

overheadRateLimit*: Opt[tuple[bytes: int, interval: Duration]]

BackoffTable* = Table[string, Table[PeerId, Moment]]
ValidationSeenTable* = Table[MessageId, HashSet[PubSubPeer]]

Expand Down
12 changes: 7 additions & 5 deletions libp2p/protocols/pubsub/pubsub.nim
Original file line number Diff line number Diff line change
Expand Up @@ -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],
Expand Down Expand Up @@ -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"

Expand All @@ -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[]

Expand Down Expand Up @@ -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])

Expand Down
Loading

0 comments on commit 56599f5

Please sign in to comment.