2022-07-01 18:19:57 +00:00
|
|
|
# Nim-LibP2P
|
|
|
|
# Copyright (c) 2022 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.
|
2021-02-06 00:13:04 +00:00
|
|
|
|
2022-08-03 11:33:19 +00:00
|
|
|
when (NimMajor, NimMinor) < (1, 4):
|
|
|
|
{.push raises: [Defect].}
|
|
|
|
else:
|
|
|
|
{.push raises: [].}
|
2021-03-09 12:22:52 +00:00
|
|
|
|
|
|
|
import std/[tables, sets, options]
|
2021-02-06 00:13:04 +00:00
|
|
|
import chronos, chronicles, metrics
|
|
|
|
import "."/[types]
|
|
|
|
import ".."/[pubsubpeer]
|
2022-05-11 08:38:43 +00:00
|
|
|
import "../../.."/[peerid, multiaddress, utility, switch, utils/heartbeat]
|
2021-02-06 00:13:04 +00:00
|
|
|
|
|
|
|
declareGauge(libp2p_gossipsub_peers_scores, "the scores of the peers in gossipsub", labels = ["agent"])
|
|
|
|
declareCounter(libp2p_gossipsub_bad_score_disconnection, "the number of peers disconnected by gossipsub", labels = ["agent"])
|
|
|
|
declareGauge(libp2p_gossipsub_peers_score_firstMessageDeliveries, "Detailed gossipsub scoring metric", labels = ["agent"])
|
|
|
|
declareGauge(libp2p_gossipsub_peers_score_meshMessageDeliveries, "Detailed gossipsub scoring metric", labels = ["agent"])
|
|
|
|
declareGauge(libp2p_gossipsub_peers_score_meshFailurePenalty, "Detailed gossipsub scoring metric", labels = ["agent"])
|
|
|
|
declareGauge(libp2p_gossipsub_peers_score_invalidMessageDeliveries, "Detailed gossipsub scoring metric", labels = ["agent"])
|
|
|
|
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"])
|
|
|
|
|
2021-04-18 08:08:33 +00:00
|
|
|
proc init*(_: type[TopicParams]): TopicParams =
|
|
|
|
TopicParams(
|
|
|
|
topicWeight: 0.0, # disabled by default
|
|
|
|
timeInMeshWeight: 0.01,
|
|
|
|
timeInMeshQuantum: 1.seconds,
|
|
|
|
timeInMeshCap: 10.0,
|
|
|
|
firstMessageDeliveriesWeight: 1.0,
|
|
|
|
firstMessageDeliveriesDecay: 0.5,
|
|
|
|
firstMessageDeliveriesCap: 10.0,
|
|
|
|
meshMessageDeliveriesWeight: -1.0,
|
|
|
|
meshMessageDeliveriesDecay: 0.5,
|
|
|
|
meshMessageDeliveriesCap: 10,
|
|
|
|
meshMessageDeliveriesThreshold: 1,
|
|
|
|
meshMessageDeliveriesWindow: 5.milliseconds,
|
|
|
|
meshMessageDeliveriesActivation: 10.seconds,
|
|
|
|
meshFailurePenaltyWeight: -1.0,
|
|
|
|
meshFailurePenaltyDecay: 0.5,
|
|
|
|
invalidMessageDeliveriesWeight: -1.0,
|
|
|
|
invalidMessageDeliveriesDecay: 0.5
|
|
|
|
)
|
|
|
|
|
2021-03-09 12:22:52 +00:00
|
|
|
proc withPeerStats*(
|
2021-05-25 17:05:33 +00:00
|
|
|
g: GossipSub,
|
|
|
|
peerId: PeerId,
|
2021-03-09 12:22:52 +00:00
|
|
|
action: proc (stats: var PeerStats) {.gcsafe, raises: [Defect].}) =
|
|
|
|
## Add or update peer statistics for a particular peer id - the statistics
|
|
|
|
## are retained across multiple connections until they expire
|
|
|
|
g.peerStats.withValue(peerId, stats) do:
|
|
|
|
action(stats[])
|
|
|
|
do:
|
|
|
|
action(g.peerStats.mgetOrPut(peerId, PeerStats(
|
|
|
|
expire: Moment.now() + g.parameters.retainScore
|
|
|
|
)))
|
2021-02-06 00:13:04 +00:00
|
|
|
|
|
|
|
func `/`(a, b: Duration): float64 =
|
|
|
|
let
|
|
|
|
fa = float64(a.nanoseconds)
|
|
|
|
fb = float64(b.nanoseconds)
|
|
|
|
fa / fb
|
|
|
|
|
|
|
|
func byScore*(x,y: PubSubPeer): int = system.cmp(x.score, y.score)
|
|
|
|
|
|
|
|
proc colocationFactor(g: GossipSub, peer: PubSubPeer): float64 =
|
2021-02-27 14:49:56 +00:00
|
|
|
if peer.address.isNone():
|
2021-02-06 00:13:04 +00:00
|
|
|
0.0
|
|
|
|
else:
|
|
|
|
let
|
2021-02-27 14:49:56 +00:00
|
|
|
address = peer.address.get()
|
2021-12-16 10:05:20 +00:00
|
|
|
g.peersInIP.mgetOrPut(address, initHashSet[PeerId]()).incl(peer.peerId)
|
2021-02-06 00:13:04 +00:00
|
|
|
let
|
2021-03-09 12:22:52 +00:00
|
|
|
ipPeers = g.peersInIP.getOrDefault(address).len().float64
|
|
|
|
if ipPeers > g.parameters.ipColocationFactorThreshold:
|
|
|
|
trace "colocationFactor over threshold", peer, address, ipPeers
|
|
|
|
let over = ipPeers - g.parameters.ipColocationFactorThreshold
|
2021-02-06 00:13:04 +00:00
|
|
|
over * over
|
|
|
|
else:
|
|
|
|
0.0
|
|
|
|
|
2021-03-09 12:22:52 +00:00
|
|
|
{.pop.}
|
|
|
|
|
2021-02-06 00:13:04 +00:00
|
|
|
proc disconnectPeer(g: GossipSub, peer: PubSubPeer) {.async.} =
|
2021-09-08 09:07:46 +00:00
|
|
|
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])
|
2021-02-06 00:13:04 +00:00
|
|
|
|
2021-02-22 01:04:02 +00:00
|
|
|
try:
|
|
|
|
await g.switch.disconnect(peer.peerId)
|
2021-03-09 12:22:52 +00:00
|
|
|
except CatchableError as exc: # Never cancelled
|
2021-02-22 01:04:02 +00:00
|
|
|
trace "Failed to close connection", peer, error = exc.name, msg = exc.msg
|
2021-02-06 00:13:04 +00:00
|
|
|
|
2021-03-09 12:22:52 +00:00
|
|
|
|
2021-02-06 00:13:04 +00:00
|
|
|
proc updateScores*(g: GossipSub) = # avoid async
|
2021-03-09 12:22:52 +00:00
|
|
|
## https://github.com/libp2p/specs/blob/master/pubsub/gossipsub/gossipsub-v1.1.md#the-score-function
|
|
|
|
##
|
2021-02-06 00:13:04 +00:00
|
|
|
trace "updating scores", peers = g.peers.len
|
|
|
|
|
|
|
|
let now = Moment.now()
|
2021-12-16 10:05:20 +00:00
|
|
|
var evicting: seq[PeerId]
|
2021-02-06 00:13:04 +00:00
|
|
|
|
|
|
|
for peerId, stats in g.peerStats.mpairs:
|
|
|
|
let peer = g.peers.getOrDefault(peerId)
|
|
|
|
if isNil(peer) or not(peer.connected):
|
|
|
|
if now > stats.expire:
|
|
|
|
evicting.add(peerId)
|
|
|
|
trace "evicted peer from memory", peer = peerId
|
|
|
|
continue
|
|
|
|
|
|
|
|
trace "updating peer score", peer
|
|
|
|
|
|
|
|
var
|
|
|
|
n_topics = 0
|
|
|
|
is_grafted = 0
|
2021-03-09 12:22:52 +00:00
|
|
|
score = 0.0
|
2021-02-06 00:13:04 +00:00
|
|
|
|
|
|
|
# Per topic
|
|
|
|
for topic, topicParams in g.topicParams:
|
|
|
|
var info = stats.topicInfos.getOrDefault(topic)
|
|
|
|
inc n_topics
|
|
|
|
|
|
|
|
# if weight is 0.0 avoid wasting time
|
|
|
|
if topicParams.topicWeight != 0.0:
|
|
|
|
# Scoring
|
|
|
|
var topicScore = 0'f64
|
|
|
|
|
|
|
|
if info.inMesh:
|
|
|
|
inc is_grafted
|
|
|
|
info.meshTime = now - info.graftTime
|
|
|
|
if info.meshTime > topicParams.meshMessageDeliveriesActivation:
|
|
|
|
info.meshMessageDeliveriesActive = true
|
|
|
|
|
|
|
|
var p1 = info.meshTime / topicParams.timeInMeshQuantum
|
|
|
|
if p1 > topicParams.timeInMeshCap:
|
|
|
|
p1 = topicParams.timeInMeshCap
|
|
|
|
trace "p1", peer, p1, topic, topicScore
|
|
|
|
topicScore += p1 * topicParams.timeInMeshWeight
|
|
|
|
else:
|
|
|
|
info.meshMessageDeliveriesActive = false
|
|
|
|
|
|
|
|
topicScore += info.firstMessageDeliveries * topicParams.firstMessageDeliveriesWeight
|
|
|
|
trace "p2", peer, p2 = info.firstMessageDeliveries, topic, topicScore
|
|
|
|
|
|
|
|
if info.meshMessageDeliveriesActive:
|
|
|
|
if info.meshMessageDeliveries < topicParams.meshMessageDeliveriesThreshold:
|
|
|
|
let deficit = topicParams.meshMessageDeliveriesThreshold - info.meshMessageDeliveries
|
|
|
|
let p3 = deficit * deficit
|
|
|
|
trace "p3", peer, p3, topic, topicScore
|
|
|
|
topicScore += p3 * topicParams.meshMessageDeliveriesWeight
|
|
|
|
|
|
|
|
topicScore += info.meshFailurePenalty * topicParams.meshFailurePenaltyWeight
|
|
|
|
trace "p3b", peer, p3b = info.meshFailurePenalty, topic, topicScore
|
|
|
|
|
|
|
|
topicScore += info.invalidMessageDeliveries * info.invalidMessageDeliveries * topicParams.invalidMessageDeliveriesWeight
|
|
|
|
trace "p4", p4 = info.invalidMessageDeliveries * info.invalidMessageDeliveries, topic, topicScore
|
|
|
|
|
|
|
|
trace "updated peer topic's scores", peer, topic, info, topicScore
|
|
|
|
|
2021-03-09 12:22:52 +00:00
|
|
|
score += topicScore * topicParams.topicWeight
|
2021-02-06 00:13:04 +00:00
|
|
|
|
|
|
|
# Score metrics
|
2021-09-08 09:07:46 +00:00
|
|
|
let agent =
|
|
|
|
when defined(libp2p_agents_metrics):
|
|
|
|
if peer.shortAgent.len > 0:
|
|
|
|
peer.shortAgent
|
|
|
|
else:
|
|
|
|
"unknown"
|
|
|
|
else:
|
|
|
|
"unknown"
|
|
|
|
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])
|
|
|
|
libp2p_gossipsub_peers_score_invalidMessageDeliveries.inc(info.invalidMessageDeliveries, labelValues = [agent])
|
2021-02-06 00:13:04 +00:00
|
|
|
|
|
|
|
# Score decay
|
|
|
|
info.firstMessageDeliveries *= topicParams.firstMessageDeliveriesDecay
|
|
|
|
if info.firstMessageDeliveries < g.parameters.decayToZero:
|
|
|
|
info.firstMessageDeliveries = 0
|
|
|
|
|
|
|
|
info.meshMessageDeliveries *= topicParams.meshMessageDeliveriesDecay
|
|
|
|
if info.meshMessageDeliveries < g.parameters.decayToZero:
|
|
|
|
info.meshMessageDeliveries = 0
|
|
|
|
|
|
|
|
info.meshFailurePenalty *= topicParams.meshFailurePenaltyDecay
|
|
|
|
if info.meshFailurePenalty < g.parameters.decayToZero:
|
|
|
|
info.meshFailurePenalty = 0
|
|
|
|
|
|
|
|
info.invalidMessageDeliveries *= topicParams.invalidMessageDeliveriesDecay
|
|
|
|
if info.invalidMessageDeliveries < g.parameters.decayToZero:
|
|
|
|
info.invalidMessageDeliveries = 0
|
|
|
|
|
|
|
|
# Wrap up
|
|
|
|
# commit our changes, mgetOrPut does NOT work as wanted with value types (lent?)
|
|
|
|
stats.topicInfos[topic] = info
|
|
|
|
|
2021-03-09 12:22:52 +00:00
|
|
|
score += peer.appScore * g.parameters.appSpecificWeight
|
2021-02-06 00:13:04 +00:00
|
|
|
|
2021-03-09 12:22:52 +00:00
|
|
|
|
|
|
|
# The value of the parameter is the square of the counter and is mixed with a negative weight.
|
|
|
|
score += peer.behaviourPenalty * peer.behaviourPenalty * g.parameters.behaviourPenaltyWeight
|
2021-02-06 00:13:04 +00:00
|
|
|
|
|
|
|
let colocationFactor = g.colocationFactor(peer)
|
2021-03-09 12:22:52 +00:00
|
|
|
score += colocationFactor * g.parameters.ipColocationFactorWeight
|
2021-02-06 00:13:04 +00:00
|
|
|
|
|
|
|
# Score metrics
|
2021-09-08 09:07:46 +00:00
|
|
|
let agent =
|
|
|
|
when defined(libp2p_agents_metrics):
|
|
|
|
if peer.shortAgent.len > 0:
|
|
|
|
peer.shortAgent
|
|
|
|
else:
|
|
|
|
"unknown"
|
|
|
|
else:
|
|
|
|
"unknown"
|
|
|
|
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])
|
2021-02-06 00:13:04 +00:00
|
|
|
|
|
|
|
# decay behaviourPenalty
|
|
|
|
peer.behaviourPenalty *= g.parameters.behaviourPenaltyDecay
|
|
|
|
if peer.behaviourPenalty < g.parameters.decayToZero:
|
|
|
|
peer.behaviourPenalty = 0
|
|
|
|
|
2021-03-09 12:22:52 +00:00
|
|
|
peer.score = score
|
|
|
|
|
2021-02-06 00:13:04 +00:00
|
|
|
# copy into stats the score to keep until expired
|
|
|
|
stats.score = peer.score
|
|
|
|
stats.appScore = peer.appScore
|
|
|
|
stats.behaviourPenalty = peer.behaviourPenalty
|
2021-03-09 12:22:52 +00:00
|
|
|
stats.expire = now + g.parameters.retainScore # refresh expiration
|
|
|
|
|
2021-02-06 00:13:04 +00:00
|
|
|
trace "updated peer's score", peer, score = peer.score, n_topics, is_grafted
|
|
|
|
|
2022-04-12 14:03:31 +00:00
|
|
|
if g.parameters.disconnectBadPeers and stats.score < g.parameters.graylistThreshold and
|
|
|
|
peer.peerId notin g.parameters.directPeers:
|
2021-02-06 00:13:04 +00:00
|
|
|
debug "disconnecting bad score peer", peer, score = peer.score
|
2021-03-09 12:22:52 +00:00
|
|
|
asyncSpawn(try: g.disconnectPeer(peer) except Exception as exc: raiseAssert exc.msg)
|
2021-02-06 00:13:04 +00:00
|
|
|
|
2021-09-08 09:07:46 +00:00
|
|
|
libp2p_gossipsub_peers_scores.inc(peer.score, labelValues = [agent])
|
2021-02-06 00:13:04 +00:00
|
|
|
|
|
|
|
for peer in evicting:
|
|
|
|
g.peerStats.del(peer)
|
|
|
|
|
|
|
|
trace "updated scores", peers = g.peers.len
|
|
|
|
|
2022-05-11 08:38:43 +00:00
|
|
|
proc scoringHeartbeat*(g: GossipSub) {.async.} =
|
|
|
|
heartbeat "Gossipsub scoring", g.parameters.decayInterval:
|
|
|
|
trace "running scoring heartbeat", instance = cast[int](g)
|
|
|
|
g.updateScores()
|
|
|
|
|
2021-02-06 00:13:04 +00:00
|
|
|
proc punishInvalidMessage*(g: GossipSub, peer: PubSubPeer, topics: seq[string]) =
|
2021-06-02 00:06:08 +00:00
|
|
|
for tt in topics:
|
|
|
|
let t = tt
|
2021-02-06 00:13:04 +00:00
|
|
|
if t notin g.topics:
|
|
|
|
continue
|
|
|
|
|
2021-05-25 17:05:33 +00:00
|
|
|
let tt = t
|
2021-02-06 00:13:04 +00:00
|
|
|
# update stats
|
2021-03-09 12:22:52 +00:00
|
|
|
g.withPeerStats(peer.peerId) do (stats: var PeerStats):
|
2021-05-25 17:05:33 +00:00
|
|
|
stats.topicInfos.mgetOrPut(tt, TopicInfo()).invalidMessageDeliveries += 1
|
2021-04-18 08:08:33 +00:00
|
|
|
|
|
|
|
proc addCapped*[T](stat: var T, diff, cap: T) =
|
|
|
|
stat += min(diff, cap - stat)
|
|
|
|
|
|
|
|
proc rewardDelivered*(
|
2022-05-11 08:38:43 +00:00
|
|
|
g: GossipSub, peer: PubSubPeer, topics: openArray[string], first: bool, delay = ZeroDuration) =
|
2021-06-02 00:06:08 +00:00
|
|
|
for tt in topics:
|
|
|
|
let t = tt
|
2021-04-18 08:08:33 +00:00
|
|
|
if t notin g.topics:
|
|
|
|
continue
|
2021-05-25 17:05:33 +00:00
|
|
|
|
|
|
|
let tt = t
|
2021-04-18 08:08:33 +00:00
|
|
|
let topicParams = g.topicParams.mgetOrPut(t, TopicParams.init())
|
|
|
|
# if in mesh add more delivery score
|
|
|
|
|
2022-05-11 08:38:43 +00:00
|
|
|
if delay > topicParams.meshMessageDeliveriesWindow:
|
|
|
|
# Too old
|
|
|
|
continue
|
|
|
|
|
2021-04-18 08:08:33 +00:00
|
|
|
g.withPeerStats(peer.peerId) do (stats: var PeerStats):
|
2021-05-25 17:05:33 +00:00
|
|
|
stats.topicInfos.withValue(tt, tstats):
|
2021-04-18 08:08:33 +00:00
|
|
|
if tstats[].inMesh:
|
|
|
|
if first:
|
|
|
|
tstats[].firstMessageDeliveries.addCapped(
|
|
|
|
1, topicParams.firstMessageDeliveriesCap)
|
|
|
|
|
|
|
|
tstats[].meshMessageDeliveries.addCapped(
|
|
|
|
1, topicParams.meshMessageDeliveriesCap)
|
|
|
|
do: # make sure we don't loose this information
|
2021-05-25 17:05:33 +00:00
|
|
|
stats.topicInfos[tt] = TopicInfo(meshMessageDeliveries: 1)
|