2019-12-06 02:16:18 +00:00
|
|
|
## Nim-LibP2P
|
|
|
|
## Copyright (c) 2019 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.
|
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
import std/[tables, sets, options, sequtils, random, algorithm]
|
2020-06-07 07:15:21 +00:00
|
|
|
import chronos, chronicles, metrics
|
2020-09-04 06:10:32 +00:00
|
|
|
import ./pubsub,
|
|
|
|
./floodsub,
|
|
|
|
./pubsubpeer,
|
|
|
|
./peertable,
|
|
|
|
./mcache,
|
|
|
|
./timedcache,
|
|
|
|
./rpc/[messages, message],
|
2019-12-06 02:16:18 +00:00
|
|
|
../protocol,
|
2020-06-19 17:29:43 +00:00
|
|
|
../../stream/connection,
|
2020-09-04 06:10:32 +00:00
|
|
|
../../peerinfo,
|
2020-07-01 06:25:09 +00:00
|
|
|
../../peerid,
|
2020-05-20 00:14:15 +00:00
|
|
|
../../utility
|
2020-09-21 09:16:29 +00:00
|
|
|
import stew/results
|
|
|
|
export results
|
2019-12-06 02:16:18 +00:00
|
|
|
|
|
|
|
logScope:
|
2020-12-01 17:34:27 +00:00
|
|
|
topics = "libp2p gossipsub"
|
2019-12-06 02:16:18 +00:00
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
const
|
|
|
|
GossipSubCodec* = "/meshsub/1.1.0"
|
|
|
|
GossipSubCodec_10* = "/meshsub/1.0.0"
|
2019-12-06 02:16:18 +00:00
|
|
|
|
|
|
|
# overlay parameters
|
2020-11-23 21:02:23 +00:00
|
|
|
const
|
2020-09-21 09:16:29 +00:00
|
|
|
GossipSubD* = 6
|
|
|
|
GossipSubDlo* = 4
|
|
|
|
GossipSubDhi* = 12
|
2019-12-06 02:16:18 +00:00
|
|
|
|
|
|
|
# gossip parameters
|
2020-09-21 09:16:29 +00:00
|
|
|
const
|
|
|
|
GossipSubHistoryLength* = 5
|
|
|
|
GossipSubHistoryGossip* = 3
|
2019-12-06 02:16:18 +00:00
|
|
|
|
|
|
|
# heartbeat interval
|
2020-09-21 09:16:29 +00:00
|
|
|
GossipSubHeartbeatInterval* = 1.seconds
|
2019-12-06 02:16:18 +00:00
|
|
|
|
|
|
|
# fanout ttl
|
2020-09-22 07:05:53 +00:00
|
|
|
const
|
2020-09-21 09:16:29 +00:00
|
|
|
GossipSubFanoutTTL* = 1.minutes
|
|
|
|
|
2020-11-19 07:48:17 +00:00
|
|
|
# gossip parameters
|
|
|
|
const
|
|
|
|
GossipBackoffPeriod* = 1.minutes
|
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
const
|
|
|
|
BackoffSlackTime = 2 # seconds
|
|
|
|
IWantPeerBudget = 25 # 25 messages per second ( reset every heartbeat )
|
|
|
|
IHavePeerBudget = 10
|
2020-11-28 07:27:39 +00:00
|
|
|
# the max amount of IHave to expose, not by spec, but go as example
|
|
|
|
# rust sigp: https://github.com/sigp/rust-libp2p/blob/f53d02bc873fef2bf52cd31e3d5ce366a41d8a8c/protocols/gossipsub/src/config.rs#L572
|
|
|
|
# go: https://github.com/libp2p/go-libp2p-pubsub/blob/08c17398fb11b2ab06ca141dddc8ec97272eb772/gossipsub.go#L155
|
|
|
|
IHaveMaxLength = 5000
|
2019-12-06 02:16:18 +00:00
|
|
|
|
|
|
|
type
|
2020-09-21 09:16:29 +00:00
|
|
|
TopicInfo* = object
|
|
|
|
# gossip 1.1 related
|
|
|
|
graftTime: Moment
|
|
|
|
meshTime: Duration
|
|
|
|
inMesh: bool
|
|
|
|
meshMessageDeliveriesActive: bool
|
|
|
|
firstMessageDeliveries: float64
|
|
|
|
meshMessageDeliveries: float64
|
|
|
|
meshFailurePenalty: float64
|
|
|
|
invalidMessageDeliveries: float64
|
|
|
|
|
|
|
|
TopicParams* = object
|
|
|
|
topicWeight*: float64
|
|
|
|
|
|
|
|
# p1
|
|
|
|
timeInMeshWeight*: float64
|
|
|
|
timeInMeshQuantum*: Duration
|
|
|
|
timeInMeshCap*: float64
|
|
|
|
|
|
|
|
# p2
|
|
|
|
firstMessageDeliveriesWeight*: float64
|
|
|
|
firstMessageDeliveriesDecay*: float64
|
|
|
|
firstMessageDeliveriesCap*: float64
|
|
|
|
|
|
|
|
# p3
|
|
|
|
meshMessageDeliveriesWeight*: float64
|
|
|
|
meshMessageDeliveriesDecay*: float64
|
|
|
|
meshMessageDeliveriesThreshold*: float64
|
|
|
|
meshMessageDeliveriesCap*: float64
|
|
|
|
meshMessageDeliveriesActivation*: Duration
|
|
|
|
meshMessageDeliveriesWindow*: Duration
|
|
|
|
|
|
|
|
# p3b
|
|
|
|
meshFailurePenaltyWeight*: float64
|
|
|
|
meshFailurePenaltyDecay*: float64
|
|
|
|
|
|
|
|
# p4
|
|
|
|
invalidMessageDeliveriesWeight*: float64
|
|
|
|
invalidMessageDeliveriesDecay*: float64
|
|
|
|
|
|
|
|
PeerStats* = object
|
|
|
|
topicInfos*: Table[string, TopicInfo]
|
|
|
|
expire*: Moment # updated on disconnect, to retain scores until expire
|
|
|
|
|
|
|
|
GossipSubParams* = object
|
|
|
|
explicit: bool
|
|
|
|
pruneBackoff*: Duration
|
|
|
|
floodPublish*: bool
|
|
|
|
gossipFactor*: float64
|
2020-11-19 07:48:17 +00:00
|
|
|
d*: int
|
|
|
|
dLow*: int
|
|
|
|
dHigh*: int
|
2020-09-21 09:16:29 +00:00
|
|
|
dScore*: int
|
|
|
|
dOut*: int
|
|
|
|
dLazy*: int
|
|
|
|
|
2020-11-19 07:48:17 +00:00
|
|
|
heartbeatInterval*: Duration
|
|
|
|
|
|
|
|
historyLength*: int
|
|
|
|
historyGossip*: int
|
|
|
|
|
|
|
|
fanoutTTL*: Duration
|
2020-11-26 05:45:10 +00:00
|
|
|
seenTTL*: Duration
|
2020-11-19 07:48:17 +00:00
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
gossipThreshold*: float64
|
|
|
|
publishThreshold*: float64
|
|
|
|
graylistThreshold*: float64
|
|
|
|
acceptPXThreshold*: float64
|
|
|
|
opportunisticGraftThreshold*: float64
|
|
|
|
decayInterval*: Duration
|
|
|
|
decayToZero*: float64
|
|
|
|
retainScore*: Duration
|
|
|
|
|
|
|
|
appSpecificWeight*: float64
|
|
|
|
ipColocationFactorWeight*: float64
|
|
|
|
ipColocationFactorThreshold*: float64
|
|
|
|
behaviourPenaltyWeight*: float64
|
|
|
|
behaviourPenaltyDecay*: float64
|
|
|
|
|
|
|
|
directPeers*: seq[PeerId]
|
|
|
|
|
2019-12-06 02:16:18 +00:00
|
|
|
GossipSub* = ref object of FloodSub
|
2020-07-13 13:32:38 +00:00
|
|
|
mesh*: PeerTable # peers that we send messages to when we are subscribed to the topic
|
|
|
|
fanout*: PeerTable # peers that we send messages to when we're not subscribed to the topic
|
|
|
|
gossipsub*: PeerTable # peers that are subscribed to a topic
|
2020-09-21 09:16:29 +00:00
|
|
|
explicit*: PeerTable # directpeers that we keep alive explicitly
|
|
|
|
backingOff*: Table[PeerID, Moment] # explicit (always connected/forward) peers
|
2020-01-07 08:04:02 +00:00
|
|
|
lastFanoutPubSub*: Table[string, Moment] # last publish time for fanout topics
|
|
|
|
gossip*: Table[string, seq[ControlIHave]] # pending gossip
|
|
|
|
control*: Table[string, ControlMessage] # pending control messages
|
|
|
|
mcache*: MCache # messages cache
|
2020-06-29 15:15:31 +00:00
|
|
|
heartbeatFut: Future[void] # cancellation future for heartbeat interval
|
2020-06-20 10:56:55 +00:00
|
|
|
heartbeatRunning: bool
|
2019-12-06 02:16:18 +00:00
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
peerStats: Table[PubSubPeer, PeerStats]
|
|
|
|
parameters*: GossipSubParams
|
|
|
|
topicParams*: Table[string, TopicParams]
|
|
|
|
directPeersLoop: Future[void]
|
|
|
|
peersInIP: Table[MultiAddress, HashSet[PubSubPeer]]
|
|
|
|
|
|
|
|
heartbeatEvents*: seq[AsyncEvent]
|
|
|
|
|
|
|
|
when not defined(release):
|
|
|
|
prunedPeers: HashSet[PubSubPeer]
|
|
|
|
|
2020-08-05 12:02:26 +00:00
|
|
|
when defined(libp2p_expensive_metrics):
|
|
|
|
declareGauge(libp2p_gossipsub_peers_per_topic_mesh,
|
|
|
|
"gossipsub peers per topic in mesh",
|
|
|
|
labels = ["topic"])
|
2020-07-08 00:33:05 +00:00
|
|
|
|
2020-08-05 12:02:26 +00:00
|
|
|
declareGauge(libp2p_gossipsub_peers_per_topic_fanout,
|
|
|
|
"gossipsub peers per topic in fanout",
|
|
|
|
labels = ["topic"])
|
2020-07-08 00:33:05 +00:00
|
|
|
|
2020-08-05 12:02:26 +00:00
|
|
|
declareGauge(libp2p_gossipsub_peers_per_topic_gossipsub,
|
|
|
|
"gossipsub peers per topic in gossipsub",
|
|
|
|
labels = ["topic"])
|
2020-06-07 07:15:21 +00:00
|
|
|
|
2020-11-04 14:18:00 +00:00
|
|
|
declareGauge(libp2p_gossipsub_peers_mesh_sum, "pubsub peers in mesh table summed")
|
|
|
|
declareGauge(libp2p_gossipsub_peers_gossipsub_sum, "pubsub peers in gossipsub table summed")
|
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
proc init*(_: type[GossipSubParams]): GossipSubParams =
|
|
|
|
GossipSubParams(
|
|
|
|
explicit: true,
|
|
|
|
pruneBackoff: 1.minutes,
|
|
|
|
floodPublish: true,
|
|
|
|
gossipFactor: 0.25,
|
2020-11-19 07:48:17 +00:00
|
|
|
d: GossipSubD,
|
|
|
|
dLow: GossipSubDlo,
|
|
|
|
dHigh: GossipSubDhi,
|
|
|
|
dScore: GossipSubDlo,
|
|
|
|
dOut: GossipSubDlo - 1, # DLow - 1
|
|
|
|
dLazy: GossipSubD, # Like D
|
|
|
|
heartbeatInterval: GossipSubHeartbeatInterval,
|
|
|
|
historyLength: GossipSubHistoryLength,
|
|
|
|
historyGossip: GossipSubHistoryGossip,
|
|
|
|
fanoutTTL: GossipSubFanoutTTL,
|
2020-11-26 05:45:10 +00:00
|
|
|
seenTTL: 2.minutes,
|
2020-09-21 09:16:29 +00:00
|
|
|
gossipThreshold: -10,
|
|
|
|
publishThreshold: -100,
|
|
|
|
graylistThreshold: -10000,
|
2020-10-03 00:26:45 +00:00
|
|
|
opportunisticGraftThreshold: 0,
|
2020-09-21 09:16:29 +00:00
|
|
|
decayInterval: 1.seconds,
|
|
|
|
decayToZero: 0.01,
|
|
|
|
retainScore: 10.seconds,
|
|
|
|
appSpecificWeight: 0.0,
|
|
|
|
ipColocationFactorWeight: 0.0,
|
|
|
|
ipColocationFactorThreshold: 1.0,
|
|
|
|
behaviourPenaltyWeight: -1.0,
|
|
|
|
behaviourPenaltyDecay: 0.999,
|
|
|
|
)
|
|
|
|
|
|
|
|
proc validateParameters*(parameters: GossipSubParams): Result[void, cstring] =
|
2020-11-19 07:48:17 +00:00
|
|
|
if (parameters.dOut >= parameters.dLow) or
|
|
|
|
(parameters.dOut > (parameters.d div 2)):
|
2020-09-21 09:16:29 +00:00
|
|
|
err("gossipsub: dOut parameter error, Number of outbound connections to keep in the mesh. Must be less than D_lo and at most D/2")
|
|
|
|
elif parameters.gossipThreshold >= 0:
|
|
|
|
err("gossipsub: gossipThreshold parameter error, Must be < 0")
|
|
|
|
elif parameters.publishThreshold >= parameters.gossipThreshold:
|
|
|
|
err("gossipsub: publishThreshold parameter error, Must be < gossipThreshold")
|
|
|
|
elif parameters.graylistThreshold >= parameters.publishThreshold:
|
|
|
|
err("gossipsub: graylistThreshold parameter error, Must be < publishThreshold")
|
|
|
|
elif parameters.acceptPXThreshold < 0:
|
|
|
|
err("gossipsub: acceptPXThreshold parameter error, Must be >= 0")
|
|
|
|
elif parameters.opportunisticGraftThreshold < 0:
|
|
|
|
err("gossipsub: opportunisticGraftThreshold parameter error, Must be >= 0")
|
|
|
|
elif parameters.decayToZero > 0.5 or parameters.decayToZero <= 0.0:
|
|
|
|
err("gossipsub: decayToZero parameter error, Should be close to 0.0")
|
|
|
|
elif parameters.appSpecificWeight < 0:
|
|
|
|
err("gossipsub: appSpecificWeight parameter error, Must be positive")
|
|
|
|
elif parameters.ipColocationFactorWeight > 0:
|
|
|
|
err("gossipsub: ipColocationFactorWeight parameter error, Must be negative or 0")
|
|
|
|
elif parameters.ipColocationFactorThreshold < 1.0:
|
|
|
|
err("gossipsub: ipColocationFactorThreshold parameter error, Must be at least 1")
|
|
|
|
elif parameters.behaviourPenaltyWeight >= 0:
|
|
|
|
err("gossipsub: behaviourPenaltyWeight parameter error, Must be negative")
|
|
|
|
elif parameters.behaviourPenaltyDecay < 0 or parameters.behaviourPenaltyDecay >= 1:
|
|
|
|
err("gossipsub: behaviourPenaltyDecay parameter error, Must be between 0 and 1")
|
|
|
|
else:
|
|
|
|
ok()
|
|
|
|
|
|
|
|
proc init*(_: type[TopicParams]): TopicParams =
|
|
|
|
TopicParams(
|
|
|
|
topicWeight: 0.0, # disable score
|
|
|
|
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
|
|
|
|
)
|
|
|
|
|
|
|
|
proc validateParameters*(parameters: TopicParams): Result[void, cstring] =
|
|
|
|
if parameters.timeInMeshWeight <= 0.0 or parameters.timeInMeshWeight > 1.0:
|
|
|
|
err("gossipsub: timeInMeshWeight parameter error, Must be a small positive value")
|
|
|
|
elif parameters.timeInMeshCap <= 0.0:
|
|
|
|
err("gossipsub: timeInMeshCap parameter error, Should be a positive value")
|
|
|
|
elif parameters.firstMessageDeliveriesWeight <= 0.0:
|
|
|
|
err("gossipsub: firstMessageDeliveriesWeight parameter error, Should be a positive value")
|
|
|
|
elif parameters.meshMessageDeliveriesWeight >= 0.0:
|
|
|
|
err("gossipsub: meshMessageDeliveriesWeight parameter error, Should be a negative value")
|
|
|
|
elif parameters.meshMessageDeliveriesThreshold <= 0.0:
|
|
|
|
err("gossipsub: meshMessageDeliveriesThreshold parameter error, Should be a positive value")
|
|
|
|
elif parameters.meshMessageDeliveriesCap < parameters.meshMessageDeliveriesThreshold:
|
|
|
|
err("gossipsub: meshMessageDeliveriesCap parameter error, Should be >= meshMessageDeliveriesThreshold")
|
|
|
|
elif parameters.meshFailurePenaltyWeight >= 0.0:
|
|
|
|
err("gossipsub: meshFailurePenaltyWeight parameter error, Should be a negative value")
|
|
|
|
elif parameters.invalidMessageDeliveriesWeight >= 0.0:
|
|
|
|
err("gossipsub: invalidMessageDeliveriesWeight parameter error, Should be a negative value")
|
|
|
|
else:
|
|
|
|
ok()
|
|
|
|
|
2020-10-30 12:49:54 +00:00
|
|
|
func byScore(x,y: PubSubPeer): int = system.cmp(x.score, y.score)
|
2020-09-21 09:16:29 +00:00
|
|
|
|
2020-06-12 23:54:12 +00:00
|
|
|
method init*(g: GossipSub) =
|
2019-12-17 05:24:03 +00:00
|
|
|
proc handler(conn: Connection, proto: string) {.async.} =
|
2019-12-06 02:16:18 +00:00
|
|
|
## main protocol handler that gets triggered on every
|
|
|
|
## connection for a protocol string
|
|
|
|
## e.g. ``/floodsub/1.0.0``, etc...
|
|
|
|
##
|
2020-09-04 16:30:45 +00:00
|
|
|
try:
|
|
|
|
await g.handleConn(conn, proto)
|
|
|
|
except CancelledError:
|
|
|
|
# This is top-level procedure which will work as separate task, so it
|
|
|
|
# do not need to propogate CancelledError.
|
2020-09-06 08:31:47 +00:00
|
|
|
trace "Unexpected cancellation in gossipsub handler", conn
|
2020-09-04 16:30:45 +00:00
|
|
|
except CatchableError as exc:
|
2020-09-06 08:31:47 +00:00
|
|
|
trace "GossipSub handler leaks an error", exc = exc.msg, conn
|
2019-12-06 02:16:18 +00:00
|
|
|
|
|
|
|
g.handler = handler
|
2020-09-21 09:16:29 +00:00
|
|
|
g.codecs &= GossipSubCodec
|
|
|
|
g.codecs &= GossipSubCodec_10
|
|
|
|
|
|
|
|
method onNewPeer(g: GossipSub, peer: PubSubPeer) =
|
|
|
|
if peer notin g.peerStats:
|
|
|
|
# new peer
|
|
|
|
g.peerStats[peer] = PeerStats()
|
|
|
|
peer.iWantBudget = IWantPeerBudget
|
|
|
|
peer.iHaveBudget = IHavePeerBudget
|
|
|
|
return
|
|
|
|
else:
|
|
|
|
# we knew this peer
|
|
|
|
discard
|
|
|
|
|
|
|
|
proc grafted(g: GossipSub, p: PubSubPeer, topic: string) =
|
2020-12-15 01:25:22 +00:00
|
|
|
g.peerStats.withValue(p, stats):
|
2020-09-21 09:16:29 +00:00
|
|
|
var info = stats.topicInfos.getOrDefault(topic)
|
|
|
|
info.graftTime = Moment.now()
|
|
|
|
info.meshTime = 0.seconds
|
|
|
|
info.inMesh = true
|
|
|
|
info.meshMessageDeliveriesActive = false
|
|
|
|
|
|
|
|
# mgetOrPut does not work, so we gotta do this without referencing
|
|
|
|
stats.topicInfos[topic] = info
|
|
|
|
assert(g.peerStats[p].topicInfos[topic].inMesh == true)
|
|
|
|
|
2020-10-02 04:09:31 +00:00
|
|
|
trace "grafted", peer=p, topic
|
2020-09-21 09:16:29 +00:00
|
|
|
do:
|
|
|
|
g.onNewPeer(p)
|
|
|
|
g.grafted(p, topic)
|
|
|
|
|
|
|
|
proc pruned(g: GossipSub, p: PubSubPeer, topic: string) =
|
2020-12-15 01:25:22 +00:00
|
|
|
g.peerStats.withValue(p, stats):
|
2020-09-21 09:16:29 +00:00
|
|
|
when not defined(release):
|
|
|
|
g.prunedPeers.incl(p)
|
|
|
|
|
|
|
|
if topic in stats.topicInfos:
|
|
|
|
var info = stats.topicInfos[topic]
|
|
|
|
let topicParams = g.topicParams.mgetOrPut(topic, TopicParams.init())
|
|
|
|
|
|
|
|
# penalize a peer that delivered no message
|
|
|
|
let threshold = topicParams.meshMessageDeliveriesThreshold
|
|
|
|
if info.inMesh and info.meshMessageDeliveriesActive and info.meshMessageDeliveries < threshold:
|
|
|
|
let deficit = threshold - info.meshMessageDeliveries
|
|
|
|
info.meshFailurePenalty += deficit * deficit
|
|
|
|
|
|
|
|
info.inMesh = false
|
|
|
|
|
|
|
|
# mgetOrPut does not work, so we gotta do this without referencing
|
|
|
|
stats.topicInfos[topic] = info
|
|
|
|
|
2020-10-02 04:09:31 +00:00
|
|
|
trace "pruned", peer=p, topic
|
2020-09-21 09:16:29 +00:00
|
|
|
|
|
|
|
proc peerExchangeList(g: GossipSub, topic: string): seq[PeerInfoMsg] =
|
|
|
|
var peers = g.gossipsub.getOrDefault(topic, initHashSet[PubSubPeer]()).toSeq()
|
|
|
|
peers.keepIf do (x: PubSubPeer) -> bool:
|
|
|
|
x.score >= 0.0
|
|
|
|
# by spec, larger then Dhi, but let's put some hard caps
|
2020-11-19 07:48:17 +00:00
|
|
|
peers.setLen(min(peers.len, g.parameters.dHigh * 2))
|
2020-09-21 09:16:29 +00:00
|
|
|
peers.map do (x: PubSubPeer) -> PeerInfoMsg:
|
|
|
|
PeerInfoMsg(peerID: x.peerId.getBytes())
|
2019-12-06 02:16:18 +00:00
|
|
|
|
2020-07-08 00:33:05 +00:00
|
|
|
proc replenishFanout(g: GossipSub, topic: string) =
|
2020-05-21 20:24:20 +00:00
|
|
|
## get fanout peers for a topic
|
2020-09-06 08:31:47 +00:00
|
|
|
logScope: topic
|
2020-05-21 20:24:20 +00:00
|
|
|
trace "about to replenish fanout"
|
2020-07-09 17:16:46 +00:00
|
|
|
|
2020-11-19 07:48:17 +00:00
|
|
|
if g.fanout.peers(topic) < g.parameters.dLow:
|
2020-07-09 17:16:46 +00:00
|
|
|
trace "replenishing fanout", peers = g.fanout.peers(topic)
|
|
|
|
if topic in g.gossipsub:
|
2020-07-13 13:32:38 +00:00
|
|
|
for peer in g.gossipsub[topic]:
|
|
|
|
if g.fanout.addPeer(topic, peer):
|
2020-11-19 07:48:17 +00:00
|
|
|
if g.fanout.peers(topic) == g.parameters.d:
|
2020-05-21 20:24:20 +00:00
|
|
|
break
|
|
|
|
|
2020-08-05 12:02:26 +00:00
|
|
|
when defined(libp2p_expensive_metrics):
|
|
|
|
libp2p_gossipsub_peers_per_topic_fanout
|
|
|
|
.set(g.fanout.peers(topic).int64, labelValues = [topic])
|
2020-07-08 00:33:05 +00:00
|
|
|
|
2020-07-09 17:16:46 +00:00
|
|
|
trace "fanout replenished with peers", peers = g.fanout.peers(topic)
|
2020-05-21 20:24:20 +00:00
|
|
|
|
2020-09-22 07:05:53 +00:00
|
|
|
method onPubSubPeerEvent*(p: GossipSub, peer: PubsubPeer, event: PubSubPeerEvent) {.gcsafe.} =
|
|
|
|
case event.kind
|
|
|
|
of PubSubPeerEventKind.Connected:
|
|
|
|
discard
|
|
|
|
of PubSubPeerEventKind.Disconnected:
|
|
|
|
# If a send connection is lost, it's better to remove peer from the mesh -
|
|
|
|
# if it gets reestablished, the peer will be readded to the mesh, and if it
|
|
|
|
# doesn't, well.. then we hope the peer is going away!
|
2020-10-02 04:09:31 +00:00
|
|
|
for topic, peers in p.mesh.mpairs():
|
|
|
|
p.pruned(peer, topic)
|
2020-09-22 07:05:53 +00:00
|
|
|
peers.excl(peer)
|
|
|
|
for _, peers in p.fanout.mpairs():
|
|
|
|
peers.excl(peer)
|
|
|
|
|
|
|
|
procCall FloodSub(p).onPubSubPeerEvent(peer, event)
|
|
|
|
|
2020-12-15 01:25:22 +00:00
|
|
|
proc rebalanceMesh(g: GossipSub, topic: string) =
|
2020-07-16 19:26:57 +00:00
|
|
|
logScope:
|
|
|
|
topic
|
2020-08-15 19:50:31 +00:00
|
|
|
mesh = g.mesh.peers(topic)
|
|
|
|
gossipsub = g.gossipsub.peers(topic)
|
2020-07-16 19:26:57 +00:00
|
|
|
|
2020-08-15 19:50:31 +00:00
|
|
|
trace "rebalancing mesh"
|
2020-07-16 19:26:57 +00:00
|
|
|
|
2020-07-09 17:16:46 +00:00
|
|
|
# create a mesh topic that we're subscribing to
|
2020-07-08 00:33:05 +00:00
|
|
|
|
2020-07-09 17:16:46 +00:00
|
|
|
var
|
2020-12-01 10:44:09 +00:00
|
|
|
prunes, grafts: seq[PubSubPeer]
|
2020-07-08 00:33:05 +00:00
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
let npeers = g.mesh.peers(topic)
|
2020-11-19 07:48:17 +00:00
|
|
|
if npeers < g.parameters.dLow:
|
2020-12-01 10:44:09 +00:00
|
|
|
trace "replenishing mesh", peers = npeers
|
2020-07-16 19:26:57 +00:00
|
|
|
# replenish the mesh if we're below Dlo
|
2020-12-01 10:44:09 +00:00
|
|
|
var candidates = toSeq(
|
2020-07-13 13:32:38 +00:00
|
|
|
g.gossipsub.getOrDefault(topic, initHashSet[PubSubPeer]()) -
|
|
|
|
g.mesh.getOrDefault(topic, initHashSet[PubSubPeer]())
|
2020-12-01 10:44:09 +00:00
|
|
|
).filterIt(
|
|
|
|
it.connected and
|
2020-09-21 09:16:29 +00:00
|
|
|
# avoid negative score peers
|
2020-12-01 10:44:09 +00:00
|
|
|
it.score >= 0.0 and
|
2020-09-21 09:16:29 +00:00
|
|
|
# don't pick explicit peers
|
2020-12-01 10:44:09 +00:00
|
|
|
it.peerId notin g.parameters.directPeers and
|
2020-09-21 09:16:29 +00:00
|
|
|
# and avoid peers we are backing off
|
2020-12-01 10:44:09 +00:00
|
|
|
it.peerId notin g.backingOff
|
|
|
|
)
|
2020-09-21 09:16:29 +00:00
|
|
|
|
|
|
|
# shuffle anyway, score might be not used
|
2020-12-01 10:44:09 +00:00
|
|
|
shuffle(candidates)
|
2020-07-08 00:33:05 +00:00
|
|
|
|
2020-10-30 12:49:54 +00:00
|
|
|
# sort peers by score, high score first since we graft
|
2020-12-01 10:44:09 +00:00
|
|
|
candidates.sort(byScore, SortOrder.Descending)
|
2020-09-21 09:16:29 +00:00
|
|
|
|
2020-07-16 19:26:57 +00:00
|
|
|
# Graft peers so we reach a count of D
|
2020-12-01 10:44:09 +00:00
|
|
|
candidates.setLen(min(candidates.len, g.parameters.d - npeers))
|
2020-07-09 17:16:46 +00:00
|
|
|
|
2020-12-01 10:44:09 +00:00
|
|
|
trace "grafting", grafting = candidates.len
|
|
|
|
for peer in candidates:
|
2020-09-21 09:16:29 +00:00
|
|
|
if g.mesh.addPeer(topic, peer):
|
|
|
|
g.grafted(peer, topic)
|
|
|
|
g.fanout.removePeer(topic, peer)
|
2020-12-01 10:44:09 +00:00
|
|
|
grafts &= peer
|
2020-09-22 07:05:53 +00:00
|
|
|
|
2020-10-30 12:49:54 +00:00
|
|
|
else:
|
|
|
|
var meshPeers = toSeq(g.mesh.getOrDefault(topic, initHashSet[PubSubPeer]()))
|
|
|
|
meshPeers.keepIf do (x: PubSubPeer) -> bool: x.outbound
|
|
|
|
if meshPeers.len < g.parameters.dOut:
|
|
|
|
trace "replenishing mesh outbound quota", peers = g.mesh.peers(topic)
|
2020-11-23 21:02:23 +00:00
|
|
|
|
2020-12-01 10:44:09 +00:00
|
|
|
var candidates = toSeq(
|
2020-10-30 12:49:54 +00:00
|
|
|
g.gossipsub.getOrDefault(topic, initHashSet[PubSubPeer]()) -
|
|
|
|
g.mesh.getOrDefault(topic, initHashSet[PubSubPeer]())
|
2020-12-01 10:44:09 +00:00
|
|
|
).filterIt(
|
|
|
|
it.connected and
|
2020-10-30 12:49:54 +00:00
|
|
|
# get only outbound ones
|
2020-12-01 10:44:09 +00:00
|
|
|
it.outbound and
|
2020-10-30 12:49:54 +00:00
|
|
|
# avoid negative score peers
|
2020-12-01 10:44:09 +00:00
|
|
|
it.score >= 0.0 and
|
2020-10-30 12:49:54 +00:00
|
|
|
# don't pick explicit peers
|
2020-12-01 10:44:09 +00:00
|
|
|
it.peerId notin g.parameters.directPeers and
|
2020-10-30 12:49:54 +00:00
|
|
|
# and avoid peers we are backing off
|
2020-12-01 10:44:09 +00:00
|
|
|
it.peerId notin g.backingOff
|
|
|
|
)
|
2020-09-21 09:16:29 +00:00
|
|
|
|
2020-10-30 12:49:54 +00:00
|
|
|
# shuffle anyway, score might be not used
|
2020-12-01 10:44:09 +00:00
|
|
|
shuffle(candidates)
|
2020-09-21 09:16:29 +00:00
|
|
|
|
2020-10-30 12:49:54 +00:00
|
|
|
# sort peers by score, high score first, we are grafting
|
2020-12-01 10:44:09 +00:00
|
|
|
candidates.sort(byScore, SortOrder.Descending)
|
2020-09-21 09:16:29 +00:00
|
|
|
|
2020-10-30 12:49:54 +00:00
|
|
|
# Graft peers so we reach a count of D
|
2020-12-01 10:44:09 +00:00
|
|
|
candidates.setLen(min(candidates.len, g.parameters.dOut))
|
2020-09-21 09:16:29 +00:00
|
|
|
|
2020-12-01 10:44:09 +00:00
|
|
|
trace "grafting outbound peers", topic, peers = candidates.len
|
2020-07-08 00:33:05 +00:00
|
|
|
|
2020-12-01 10:44:09 +00:00
|
|
|
for peer in candidates:
|
2020-10-30 12:49:54 +00:00
|
|
|
if g.mesh.addPeer(topic, peer):
|
|
|
|
g.grafted(peer, topic)
|
|
|
|
g.fanout.removePeer(topic, peer)
|
2020-12-01 10:44:09 +00:00
|
|
|
grafts &= peer
|
2020-09-22 07:05:53 +00:00
|
|
|
|
2020-05-21 20:24:20 +00:00
|
|
|
|
2020-11-19 07:48:17 +00:00
|
|
|
if g.mesh.peers(topic) > g.parameters.dHigh:
|
2020-07-16 19:26:57 +00:00
|
|
|
# prune peers if we've gone over Dhi
|
|
|
|
prunes = toSeq(g.mesh[topic])
|
2020-10-30 12:49:54 +00:00
|
|
|
# avoid pruning peers we are currently grafting in this heartbeat
|
2020-12-01 10:44:09 +00:00
|
|
|
prunes.keepIf do (x: PubSubPeer) -> bool: x notin grafts
|
2020-12-15 21:46:03 +00:00
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
# shuffle anyway, score might be not used
|
|
|
|
shuffle(prunes)
|
2020-06-12 02:20:58 +00:00
|
|
|
|
2020-10-30 12:49:54 +00:00
|
|
|
# sort peers by score (inverted), pruning, so low score peers are on top
|
|
|
|
prunes.sort(byScore, SortOrder.Ascending)
|
2020-09-21 09:16:29 +00:00
|
|
|
|
|
|
|
# keep high score peers
|
|
|
|
if prunes.len > g.parameters.dScore:
|
|
|
|
prunes.setLen(prunes.len - g.parameters.dScore)
|
2020-10-30 12:49:54 +00:00
|
|
|
|
|
|
|
# collect inbound/outbound info
|
2020-09-21 09:16:29 +00:00
|
|
|
var outbound: seq[PubSubPeer]
|
|
|
|
var inbound: seq[PubSubPeer]
|
|
|
|
for peer in prunes:
|
|
|
|
if peer.outbound:
|
|
|
|
outbound &= peer
|
|
|
|
else:
|
|
|
|
inbound &= peer
|
2020-09-22 07:05:53 +00:00
|
|
|
|
2020-12-01 10:44:09 +00:00
|
|
|
let
|
|
|
|
meshOutbound = prunes.countIt(it.outbound)
|
|
|
|
maxOutboundPrunes = meshOutbound - g.parameters.dOut
|
|
|
|
|
2020-11-19 07:48:17 +00:00
|
|
|
# ensure that there are at least D_out peers first and rebalance to g.d after that
|
2020-10-30 12:49:54 +00:00
|
|
|
outbound.setLen(min(outbound.len, max(0, maxOutboundPrunes)))
|
|
|
|
|
|
|
|
# concat remaining outbound peers
|
2020-12-01 10:44:09 +00:00
|
|
|
prunes = inbound & outbound
|
2020-10-30 12:49:54 +00:00
|
|
|
|
2020-12-01 10:44:09 +00:00
|
|
|
let pruneLen = prunes.len - g.parameters.d
|
2020-09-21 09:16:29 +00:00
|
|
|
if pruneLen > 0:
|
|
|
|
# Ok we got some peers to prune,
|
|
|
|
# for this heartbeat let's prune those
|
2020-12-01 10:44:09 +00:00
|
|
|
shuffle(prunes)
|
|
|
|
prunes.setLen(pruneLen)
|
2020-09-21 09:16:29 +00:00
|
|
|
|
2020-12-01 10:44:09 +00:00
|
|
|
trace "pruning", prunes = prunes.len
|
|
|
|
for peer in prunes:
|
2020-12-15 01:25:22 +00:00
|
|
|
trace "pruning peer on rebalance", peer, score = peer.score
|
2020-09-21 09:16:29 +00:00
|
|
|
g.pruned(peer, topic)
|
|
|
|
g.mesh.removePeer(topic, peer)
|
|
|
|
|
|
|
|
# opportunistic grafting, by spec mesh should not be empty...
|
|
|
|
if g.mesh.peers(topic) > 1:
|
|
|
|
var peers = toSeq(g.mesh[topic])
|
2020-10-30 12:49:54 +00:00
|
|
|
# grafting so high score has priority
|
|
|
|
peers.sort(byScore, SortOrder.Descending)
|
2020-09-21 09:16:29 +00:00
|
|
|
let medianIdx = peers.len div 2
|
|
|
|
let median = peers[medianIdx]
|
|
|
|
if median.score < g.parameters.opportunisticGraftThreshold:
|
|
|
|
trace "median score below opportunistic threshold", score = median.score
|
|
|
|
var avail = toSeq(
|
|
|
|
g.gossipsub.getOrDefault(topic, initHashSet[PubSubPeer]()) -
|
|
|
|
g.mesh.getOrDefault(topic, initHashSet[PubSubPeer]())
|
|
|
|
)
|
|
|
|
|
|
|
|
avail.keepIf do (x: PubSubPeer) -> bool:
|
|
|
|
# avoid negative score peers
|
|
|
|
x.score >= median.score and
|
|
|
|
# don't pick explicit peers
|
|
|
|
x.peerId notin g.parameters.directPeers and
|
|
|
|
# and avoid peers we are backing off
|
|
|
|
x.peerId notin g.backingOff
|
2020-09-22 07:05:53 +00:00
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
# by spec, grab only 2
|
|
|
|
if avail.len > 2:
|
|
|
|
avail.setLen(2)
|
2020-09-22 07:05:53 +00:00
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
for peer in avail:
|
|
|
|
if g.mesh.addPeer(topic, peer):
|
|
|
|
g.grafted(peer, topic)
|
2020-12-01 10:44:09 +00:00
|
|
|
grafts &= peer
|
2020-09-24 16:43:20 +00:00
|
|
|
trace "opportunistic grafting", peer
|
2020-09-22 07:05:53 +00:00
|
|
|
|
2020-08-05 12:02:26 +00:00
|
|
|
when defined(libp2p_expensive_metrics):
|
|
|
|
libp2p_gossipsub_peers_per_topic_gossipsub
|
|
|
|
.set(g.gossipsub.peers(topic).int64, labelValues = [topic])
|
2020-05-21 20:24:20 +00:00
|
|
|
|
2020-08-05 12:02:26 +00:00
|
|
|
libp2p_gossipsub_peers_per_topic_fanout
|
|
|
|
.set(g.fanout.peers(topic).int64, labelValues = [topic])
|
2020-07-09 17:16:46 +00:00
|
|
|
|
2020-08-05 12:02:26 +00:00
|
|
|
libp2p_gossipsub_peers_per_topic_mesh
|
|
|
|
.set(g.mesh.peers(topic).int64, labelValues = [topic])
|
2020-07-09 17:16:46 +00:00
|
|
|
|
2020-08-15 19:50:31 +00:00
|
|
|
trace "mesh balanced"
|
2020-07-09 17:16:46 +00:00
|
|
|
|
2020-08-15 19:50:31 +00:00
|
|
|
# Send changes to peers after table updates to avoid stale state
|
|
|
|
if grafts.len > 0:
|
|
|
|
let graft = RPCMsg(control: some(ControlMessage(graft: @[ControlGraft(topicID: topic)])))
|
2020-09-01 07:33:03 +00:00
|
|
|
g.broadcast(grafts, graft)
|
2020-08-15 19:50:31 +00:00
|
|
|
if prunes.len > 0:
|
2020-09-21 09:16:29 +00:00
|
|
|
let prune = RPCMsg(control: some(ControlMessage(
|
|
|
|
prune: @[ControlPrune(
|
|
|
|
topicID: topic,
|
2020-09-22 07:05:53 +00:00
|
|
|
peers: g.peerExchangeList(topic),
|
2020-09-21 09:16:29 +00:00
|
|
|
backoff: g.parameters.pruneBackoff.seconds.uint64)])))
|
2020-09-01 07:33:03 +00:00
|
|
|
g.broadcast(prunes, prune)
|
2020-07-09 17:16:46 +00:00
|
|
|
|
|
|
|
proc dropFanoutPeers(g: GossipSub) =
|
2020-05-21 20:24:20 +00:00
|
|
|
# drop peers that we haven't published to in
|
|
|
|
# GossipSubFanoutTTL seconds
|
2020-07-09 17:16:46 +00:00
|
|
|
let now = Moment.now()
|
2020-07-09 23:54:16 +00:00
|
|
|
for topic in toSeq(g.lastFanoutPubSub.keys):
|
|
|
|
let val = g.lastFanoutPubSub[topic]
|
2020-07-09 17:16:46 +00:00
|
|
|
if now > val:
|
2020-05-21 20:24:20 +00:00
|
|
|
g.fanout.del(topic)
|
2020-07-09 23:54:16 +00:00
|
|
|
g.lastFanoutPubSub.del(topic)
|
2020-07-08 00:33:05 +00:00
|
|
|
trace "dropping fanout topic", topic
|
2020-06-12 02:20:58 +00:00
|
|
|
|
2020-08-05 12:02:26 +00:00
|
|
|
when defined(libp2p_expensive_metrics):
|
|
|
|
libp2p_gossipsub_peers_per_topic_fanout
|
|
|
|
.set(g.fanout.peers(topic).int64, labelValues = [topic])
|
2020-05-21 20:24:20 +00:00
|
|
|
|
2020-08-12 00:05:49 +00:00
|
|
|
proc getGossipPeers(g: GossipSub): Table[PubSubPeer, ControlMessage] {.gcsafe.} =
|
2020-05-21 20:24:20 +00:00
|
|
|
## gossip iHave messages to peers
|
2020-07-08 00:33:05 +00:00
|
|
|
##
|
|
|
|
|
|
|
|
trace "getting gossip peers (iHave)"
|
2020-05-21 20:24:20 +00:00
|
|
|
let topics = toHashSet(toSeq(g.mesh.keys)) + toHashSet(toSeq(g.fanout.keys))
|
|
|
|
for topic in topics:
|
2020-09-21 09:16:29 +00:00
|
|
|
if topic notin g.gossipsub:
|
|
|
|
trace "topic not in gossip array, skipping", topicID = topic
|
|
|
|
continue
|
2020-05-21 20:24:20 +00:00
|
|
|
|
|
|
|
let mids = g.mcache.window(topic)
|
2020-12-01 14:55:51 +00:00
|
|
|
if not(mids.len > 0):
|
2020-07-08 00:33:05 +00:00
|
|
|
continue
|
2020-05-21 20:24:20 +00:00
|
|
|
|
2020-11-28 07:27:39 +00:00
|
|
|
var midsSeq = toSeq(mids)
|
|
|
|
# not in spec
|
|
|
|
# similar to rust: https://github.com/sigp/rust-libp2p/blob/f53d02bc873fef2bf52cd31e3d5ce366a41d8a8c/protocols/gossipsub/src/behaviour.rs#L2101
|
|
|
|
# and go https://github.com/libp2p/go-libp2p-pubsub/blob/08c17398fb11b2ab06ca141dddc8ec97272eb772/gossipsub.go#L582
|
|
|
|
if midsSeq.len > IHaveMaxLength:
|
|
|
|
shuffle(midsSeq)
|
|
|
|
midsSeq.setLen(IHaveMaxLength)
|
|
|
|
let ihave = ControlIHave(topicID: topic, messageIDs: midsSeq)
|
2020-09-21 09:16:29 +00:00
|
|
|
|
|
|
|
let mesh = g.mesh.getOrDefault(topic)
|
|
|
|
let fanout = g.fanout.getOrDefault(topic)
|
|
|
|
let gossipPeers = mesh + fanout
|
|
|
|
var allPeers = toSeq(g.gossipsub.getOrDefault(topic))
|
|
|
|
|
|
|
|
allPeers.keepIf do (x: PubSubPeer) -> bool:
|
|
|
|
x.peerId notin g.parameters.directPeers and
|
|
|
|
x notin gossipPeers and
|
|
|
|
x.score >= g.parameters.gossipThreshold
|
|
|
|
|
|
|
|
var target = g.parameters.dLazy
|
|
|
|
let factor = (g.parameters.gossipFactor.float * allPeers.len.float).int
|
|
|
|
if factor > target:
|
|
|
|
target = min(factor, allPeers.len)
|
|
|
|
|
|
|
|
if target < allPeers.len:
|
|
|
|
shuffle(allPeers)
|
|
|
|
allPeers.setLen(target)
|
|
|
|
|
2020-07-13 13:32:38 +00:00
|
|
|
for peer in allPeers:
|
2020-09-21 09:16:29 +00:00
|
|
|
if peer notin result:
|
|
|
|
result[peer] = ControlMessage()
|
|
|
|
result[peer].ihave.add(ihave)
|
2020-05-21 20:24:20 +00:00
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
func `/`(a, b: Duration): float64 =
|
|
|
|
let
|
|
|
|
fa = float64(a.nanoseconds)
|
|
|
|
fb = float64(b.nanoseconds)
|
|
|
|
fa / fb
|
|
|
|
|
|
|
|
proc colocationFactor(g: GossipSub, peer: PubSubPeer): float64 =
|
|
|
|
if peer.connections.len == 0:
|
2020-12-15 01:25:22 +00:00
|
|
|
debug "colocationFactor, no connections", peer
|
2020-09-21 09:16:29 +00:00
|
|
|
0.0
|
|
|
|
else:
|
|
|
|
let
|
|
|
|
address = peer.connections[0].observedAddr
|
|
|
|
ipPeers = g.peersInIP.getOrDefault(address)
|
|
|
|
len = ipPeers.len.float64
|
|
|
|
if len > g.parameters.ipColocationFactorThreshold:
|
2020-12-15 01:25:22 +00:00
|
|
|
debug "colocationFactor over threshold", peer, address, len
|
2020-09-21 09:16:29 +00:00
|
|
|
let over = len - g.parameters.ipColocationFactorThreshold
|
|
|
|
over * over
|
|
|
|
else:
|
|
|
|
# lazy update peersInIP
|
|
|
|
if address notin g.peersInIP:
|
|
|
|
g.peersInIP[address] = initHashSet[PubSubPeer]()
|
|
|
|
g.peersInIP[address].incl(peer)
|
|
|
|
0.0
|
|
|
|
|
|
|
|
proc updateScores(g: GossipSub) = # avoid async
|
|
|
|
trace "updating scores", peers = g.peers.len
|
2020-09-22 07:05:53 +00:00
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
let now = Moment.now()
|
|
|
|
var evicting: seq[PubSubPeer]
|
|
|
|
|
|
|
|
for peer, stats in g.peerStats.mpairs:
|
|
|
|
trace "updating peer score", peer
|
|
|
|
var n_topics = 0
|
|
|
|
var is_grafted = 0
|
|
|
|
|
|
|
|
if not peer.connected:
|
|
|
|
if now > stats.expire:
|
|
|
|
evicting.add(peer)
|
|
|
|
trace "evicted peer from memory", peer
|
2020-07-08 00:33:05 +00:00
|
|
|
continue
|
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
# Per topic
|
|
|
|
for topic, topicParams in g.topicParams:
|
|
|
|
var info = stats.topicInfos.getOrDefault(topic)
|
|
|
|
inc n_topics
|
|
|
|
|
2020-12-15 01:25:22 +00:00
|
|
|
# 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
|
2020-07-08 00:33:05 +00:00
|
|
|
|
2020-12-15 01:25:22 +00:00
|
|
|
topicScore += info.firstMessageDeliveries * topicParams.firstMessageDeliveriesWeight
|
|
|
|
trace "p2", peer, p2 = info.firstMessageDeliveries, topic, topicScore
|
2020-09-21 09:16:29 +00:00
|
|
|
|
2020-12-15 01:25:22 +00:00
|
|
|
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
|
2020-09-21 09:16:29 +00:00
|
|
|
|
2020-12-15 01:25:22 +00:00
|
|
|
topicScore += info.meshFailurePenalty * topicParams.meshFailurePenaltyWeight
|
|
|
|
trace "p3b", peer, p3b = info.meshFailurePenalty, topic, topicScore
|
2020-09-21 09:16:29 +00:00
|
|
|
|
2020-12-15 01:25:22 +00:00
|
|
|
topicScore += info.invalidMessageDeliveries * info.invalidMessageDeliveries * topicParams.invalidMessageDeliveriesWeight
|
|
|
|
trace "p4", p4 = info.invalidMessageDeliveries * info.invalidMessageDeliveries, topic, topicScore
|
2020-09-21 09:16:29 +00:00
|
|
|
|
2020-12-15 01:25:22 +00:00
|
|
|
trace "updated peer topic's scores", peer, topic, info, topicScore
|
2020-09-21 09:16:29 +00:00
|
|
|
|
2020-12-15 01:25:22 +00:00
|
|
|
peer.score += topicScore * topicParams.topicWeight
|
2020-09-21 09:16:29 +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
|
2020-09-22 07:05:53 +00:00
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
peer.score += peer.appScore * g.parameters.appSpecificWeight
|
|
|
|
|
|
|
|
peer.score += peer.behaviourPenalty * peer.behaviourPenalty * g.parameters.behaviourPenaltyWeight
|
|
|
|
|
|
|
|
peer.score += g.colocationFactor(peer) * g.parameters.ipColocationFactorWeight
|
|
|
|
|
|
|
|
# decay behaviourPenalty
|
|
|
|
peer.behaviourPenalty *= g.parameters.behaviourPenaltyDecay
|
|
|
|
if peer.behaviourPenalty < g.parameters.decayToZero:
|
|
|
|
peer.behaviourPenalty = 0
|
|
|
|
|
2020-10-30 12:49:54 +00:00
|
|
|
trace "updated peer's score", peer, score = peer.score, n_topics, is_grafted
|
2020-09-22 07:05:53 +00:00
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
for peer in evicting:
|
|
|
|
g.peerStats.del(peer)
|
2020-09-22 07:05:53 +00:00
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
trace "updated scores", peers = g.peers.len
|
2020-06-12 02:20:58 +00:00
|
|
|
|
2020-05-21 20:24:20 +00:00
|
|
|
proc heartbeat(g: GossipSub) {.async.} =
|
2020-06-20 10:56:55 +00:00
|
|
|
while g.heartbeatRunning:
|
2020-05-27 18:33:49 +00:00
|
|
|
try:
|
2020-09-21 09:16:29 +00:00
|
|
|
trace "running heartbeat", instance = cast[int](g)
|
|
|
|
|
|
|
|
# remove expired backoffs
|
|
|
|
block:
|
|
|
|
let now = Moment.now()
|
|
|
|
var expired = toSeq(g.backingOff.pairs())
|
|
|
|
expired.keepIf do (pair: tuple[peer: PeerID, expire: Moment]) -> bool:
|
|
|
|
now >= pair.expire
|
|
|
|
for (peer, _) in expired:
|
|
|
|
g.backingOff.del(peer)
|
|
|
|
|
|
|
|
# reset IWANT budget
|
|
|
|
# reset IHAVE cap
|
|
|
|
block:
|
|
|
|
for peer in g.peers.values:
|
|
|
|
peer.iWantBudget = IWantPeerBudget
|
|
|
|
peer.iHaveBudget = IHavePeerBudget
|
|
|
|
|
|
|
|
g.updateScores()
|
2020-05-21 20:24:20 +00:00
|
|
|
|
2020-11-04 14:18:00 +00:00
|
|
|
var
|
|
|
|
totalMeshPeers = 0
|
|
|
|
totalGossipPeers = 0
|
2020-06-15 18:30:09 +00:00
|
|
|
for t in toSeq(g.topics.keys):
|
2020-09-21 09:16:29 +00:00
|
|
|
# prune every negative score peer
|
|
|
|
# do this before relance
|
|
|
|
# in order to avoid grafted -> pruned in the same cycle
|
|
|
|
let meshPeers = g.mesh.getOrDefault(t)
|
2020-11-04 14:18:00 +00:00
|
|
|
let gossipPeers = g.gossipsub.getOrDefault(t)
|
|
|
|
# this will be changed by rebalance but does not matter
|
|
|
|
totalMeshPeers += meshPeers.len
|
2020-12-15 21:46:03 +00:00
|
|
|
totalGossipPeers += g.gossipsub.peers(t)
|
2020-09-21 09:16:29 +00:00
|
|
|
var prunes: seq[PubSubPeer]
|
|
|
|
for peer in meshPeers:
|
|
|
|
if peer.score < 0.0:
|
2020-12-15 01:25:22 +00:00
|
|
|
trace "pruning negative score peer", peer, score = peer.score
|
2020-09-21 09:16:29 +00:00
|
|
|
g.pruned(peer, t)
|
|
|
|
g.mesh.removePeer(t, peer)
|
|
|
|
prunes &= peer
|
2020-12-15 21:46:03 +00:00
|
|
|
if prunes.len > 0:
|
|
|
|
let prune = RPCMsg(control: some(ControlMessage(
|
|
|
|
prune: @[ControlPrune(
|
|
|
|
topicID: t,
|
|
|
|
peers: g.peerExchangeList(t),
|
|
|
|
backoff: g.parameters.pruneBackoff.seconds.uint64)])))
|
|
|
|
g.broadcast(prunes, prune)
|
2020-09-21 09:16:29 +00:00
|
|
|
|
2020-12-15 01:25:22 +00:00
|
|
|
g.rebalanceMesh(t)
|
2020-11-23 21:02:23 +00:00
|
|
|
|
2020-11-04 14:18:00 +00:00
|
|
|
libp2p_gossipsub_peers_mesh_sum.set(totalMeshPeers.int64)
|
|
|
|
libp2p_gossipsub_peers_gossipsub_sum.set(totalGossipPeers.int64)
|
2020-05-21 20:24:20 +00:00
|
|
|
|
2020-07-09 17:16:46 +00:00
|
|
|
g.dropFanoutPeers()
|
2020-07-08 00:33:05 +00:00
|
|
|
|
|
|
|
# replenish known topics to the fanout
|
|
|
|
for t in toSeq(g.fanout.keys):
|
|
|
|
g.replenishFanout(t)
|
|
|
|
|
2020-05-27 18:33:49 +00:00
|
|
|
let peers = g.getGossipPeers()
|
2020-08-02 10:22:49 +00:00
|
|
|
for peer, control in peers:
|
2020-12-15 01:25:22 +00:00
|
|
|
g.peers.withValue(peer.peerId, pubsubPeer):
|
2020-09-01 07:33:03 +00:00
|
|
|
g.send(
|
2020-08-12 00:05:49 +00:00
|
|
|
pubsubPeer[],
|
2020-09-01 07:33:03 +00:00
|
|
|
RPCMsg(control: some(control)))
|
2020-05-21 20:24:20 +00:00
|
|
|
|
2020-05-27 18:33:49 +00:00
|
|
|
g.mcache.shift() # shift the cache
|
2020-06-29 15:15:31 +00:00
|
|
|
except CancelledError as exc:
|
|
|
|
raise exc
|
2020-05-27 18:33:49 +00:00
|
|
|
except CatchableError as exc:
|
2020-11-23 21:02:23 +00:00
|
|
|
warn "exception ocurred in gossipsub heartbeat", exc = exc.msg,
|
|
|
|
trace = exc.getStackTrace()
|
2020-09-21 09:16:29 +00:00
|
|
|
|
|
|
|
for trigger in g.heartbeatEvents:
|
|
|
|
trace "firing heartbeat event", instance = cast[int](g)
|
|
|
|
trigger.fire()
|
2020-05-21 20:24:20 +00:00
|
|
|
|
2020-11-19 07:48:17 +00:00
|
|
|
await sleepAsync(g.parameters.heartbeatInterval)
|
2020-05-21 20:24:20 +00:00
|
|
|
|
2020-08-12 00:05:49 +00:00
|
|
|
method unsubscribePeer*(g: GossipSub, peer: PeerID) =
|
2019-12-06 02:16:18 +00:00
|
|
|
## handle peer disconnects
|
2020-08-12 00:05:49 +00:00
|
|
|
##
|
|
|
|
|
2020-09-06 08:31:47 +00:00
|
|
|
trace "unsubscribing gossipsub peer", peer
|
2020-08-12 00:05:49 +00:00
|
|
|
let pubSubPeer = g.peers.getOrDefault(peer)
|
|
|
|
if pubSubPeer.isNil:
|
2020-09-06 08:31:47 +00:00
|
|
|
trace "no peer to unsubscribe", peer
|
2020-08-12 00:05:49 +00:00
|
|
|
return
|
2020-06-19 21:19:07 +00:00
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
# remove from peer IPs collection too
|
|
|
|
if pubSubPeer.connections.len > 0:
|
2020-12-15 01:25:22 +00:00
|
|
|
g.peersInIP.withValue(pubSubPeer.connections[0].observedAddr, s):
|
2020-09-21 09:16:29 +00:00
|
|
|
s[].excl(pubSubPeer)
|
|
|
|
|
2020-08-12 00:05:49 +00:00
|
|
|
for t in toSeq(g.gossipsub.keys):
|
|
|
|
g.gossipsub.removePeer(t, pubSubPeer)
|
2020-09-21 09:16:29 +00:00
|
|
|
# also try to remove from explicit table here
|
|
|
|
g.explicit.removePeer(t, pubSubPeer)
|
2020-06-12 02:20:58 +00:00
|
|
|
|
2020-08-12 00:05:49 +00:00
|
|
|
when defined(libp2p_expensive_metrics):
|
|
|
|
libp2p_gossipsub_peers_per_topic_gossipsub
|
|
|
|
.set(g.gossipsub.peers(t).int64, labelValues = [t])
|
2020-06-19 21:19:07 +00:00
|
|
|
|
2020-08-12 00:05:49 +00:00
|
|
|
for t in toSeq(g.mesh.keys):
|
2020-12-15 01:25:22 +00:00
|
|
|
trace "pruning unsubscribing peer", pubSubPeer, score = pubSubPeer.score
|
2020-09-21 09:16:29 +00:00
|
|
|
g.pruned(pubSubPeer, t)
|
2020-08-12 00:05:49 +00:00
|
|
|
g.mesh.removePeer(t, pubSubPeer)
|
2019-12-06 02:16:18 +00:00
|
|
|
|
2020-08-12 00:05:49 +00:00
|
|
|
when defined(libp2p_expensive_metrics):
|
|
|
|
libp2p_gossipsub_peers_per_topic_mesh
|
|
|
|
.set(g.mesh.peers(t).int64, labelValues = [t])
|
2020-06-19 21:19:07 +00:00
|
|
|
|
2020-08-12 00:05:49 +00:00
|
|
|
for t in toSeq(g.fanout.keys):
|
|
|
|
g.fanout.removePeer(t, pubSubPeer)
|
2020-08-06 02:12:52 +00:00
|
|
|
|
2020-08-12 00:05:49 +00:00
|
|
|
when defined(libp2p_expensive_metrics):
|
|
|
|
libp2p_gossipsub_peers_per_topic_fanout
|
|
|
|
.set(g.fanout.peers(t).int64, labelValues = [t])
|
2020-09-22 07:05:53 +00:00
|
|
|
|
2020-12-15 01:25:22 +00:00
|
|
|
g.peerStats.withValue(pubSubPeer, stats):
|
|
|
|
stats[].expire = Moment.now() + g.parameters.retainScore
|
|
|
|
for topic, info in stats[].topicInfos.mpairs:
|
2020-09-21 09:16:29 +00:00
|
|
|
info.firstMessageDeliveries = 0
|
2019-12-06 02:16:18 +00:00
|
|
|
|
2020-08-12 00:05:49 +00:00
|
|
|
procCall FloodSub(g).unsubscribePeer(peer)
|
2020-05-21 17:33:48 +00:00
|
|
|
|
2019-12-06 02:16:18 +00:00
|
|
|
method subscribeTopic*(g: GossipSub,
|
|
|
|
topic: string,
|
|
|
|
subscribe: bool,
|
2020-08-17 10:10:22 +00:00
|
|
|
peer: PubSubPeer) {.gcsafe.} =
|
2020-07-16 19:26:57 +00:00
|
|
|
logScope:
|
2020-09-06 08:31:47 +00:00
|
|
|
peer
|
2020-07-16 19:26:57 +00:00
|
|
|
topic
|
2020-09-22 07:05:53 +00:00
|
|
|
|
2020-12-15 18:32:11 +00:00
|
|
|
# this is a workaround for a race condition
|
|
|
|
# that can happen if we disconnect the peer very early
|
|
|
|
# in the future we might use this as a test case
|
|
|
|
# and eventually remove this workaround
|
|
|
|
if subscribe and peer.peerId notin g.peers:
|
|
|
|
trace "ignoring unknown peer"
|
|
|
|
return
|
|
|
|
|
|
|
|
# Skip floodsub - we don't want it to add the peer to `g.floodsub`
|
|
|
|
procCall PubSub(g).subscribeTopic(topic, subscribe, peer)
|
2020-07-16 19:26:57 +00:00
|
|
|
|
2019-12-10 20:50:35 +00:00
|
|
|
if subscribe:
|
2020-07-16 19:26:57 +00:00
|
|
|
trace "peer subscribed to topic"
|
2020-12-15 18:32:11 +00:00
|
|
|
# populate scoring structs and such
|
|
|
|
g.onNewPeer(peer)
|
2020-05-27 18:33:49 +00:00
|
|
|
# subscribe remote peer to the topic
|
2020-07-13 13:32:38 +00:00
|
|
|
discard g.gossipsub.addPeer(topic, peer)
|
2020-09-21 09:16:29 +00:00
|
|
|
if peer.peerId in g.parameters.directPeers:
|
|
|
|
discard g.explicit.addPeer(topic, peer)
|
2019-12-10 20:50:35 +00:00
|
|
|
else:
|
2020-07-16 19:26:57 +00:00
|
|
|
trace "peer unsubscribed from topic"
|
2020-05-27 18:33:49 +00:00
|
|
|
# unsubscribe remote peer from the topic
|
2020-07-13 13:32:38 +00:00
|
|
|
g.gossipsub.removePeer(topic, peer)
|
|
|
|
g.mesh.removePeer(topic, peer)
|
|
|
|
g.fanout.removePeer(topic, peer)
|
2020-09-21 09:16:29 +00:00
|
|
|
if peer.peerId in g.parameters.directPeers:
|
|
|
|
g.explicit.removePeer(topic, peer)
|
2020-07-09 17:16:46 +00:00
|
|
|
|
2020-08-05 12:02:26 +00:00
|
|
|
when defined(libp2p_expensive_metrics):
|
|
|
|
libp2p_gossipsub_peers_per_topic_mesh
|
|
|
|
.set(g.mesh.peers(topic).int64, labelValues = [topic])
|
|
|
|
libp2p_gossipsub_peers_per_topic_fanout
|
|
|
|
.set(g.fanout.peers(topic).int64, labelValues = [topic])
|
2020-06-12 02:20:58 +00:00
|
|
|
|
2020-08-05 12:02:26 +00:00
|
|
|
when defined(libp2p_expensive_metrics):
|
|
|
|
libp2p_gossipsub_peers_per_topic_gossipsub
|
|
|
|
.set(g.gossipsub.peers(topic).int64, labelValues = [topic])
|
2019-12-06 02:16:18 +00:00
|
|
|
|
2020-07-09 17:16:46 +00:00
|
|
|
trace "gossip peers", peers = g.gossipsub.peers(topic), topic
|
2020-07-08 00:33:05 +00:00
|
|
|
|
2020-10-30 12:49:54 +00:00
|
|
|
proc punishPeer(g: GossipSub, peer: PubSubPeer, topics: seq[string]) =
|
|
|
|
for t in topics:
|
|
|
|
# ensure we init a new topic if unknown
|
|
|
|
let _ = g.topicParams.mgetOrPut(t, TopicParams.init())
|
|
|
|
# update stats
|
2020-12-15 01:25:22 +00:00
|
|
|
g.peerStats.withValue(peer, stats):
|
|
|
|
stats[].topicInfos.withValue(t, tstats):
|
|
|
|
tstats[].invalidMessageDeliveries += 1
|
|
|
|
do: # if we have no stats populate!
|
|
|
|
stats[].topicInfos[t] = TopicInfo(invalidMessageDeliveries: 1)
|
|
|
|
do: # if we have no stats populate!
|
|
|
|
g.peerStats[peer] =
|
|
|
|
block:
|
|
|
|
var stats = PeerStats()
|
|
|
|
stats.topicInfos[t] = TopicInfo(invalidMessageDeliveries: 1)
|
|
|
|
stats
|
|
|
|
|
2020-10-30 12:49:54 +00:00
|
|
|
|
2019-12-10 20:50:35 +00:00
|
|
|
proc handleGraft(g: GossipSub,
|
2019-12-06 02:16:18 +00:00
|
|
|
peer: PubSubPeer,
|
2020-07-09 23:54:16 +00:00
|
|
|
grafts: seq[ControlGraft]): seq[ControlPrune] =
|
2019-12-06 02:16:18 +00:00
|
|
|
for graft in grafts:
|
2020-07-09 17:16:46 +00:00
|
|
|
let topic = graft.topicID
|
2020-07-16 19:26:57 +00:00
|
|
|
logScope:
|
2020-09-06 08:31:47 +00:00
|
|
|
peer
|
2020-07-16 19:26:57 +00:00
|
|
|
topic
|
|
|
|
|
|
|
|
trace "peer grafted topic"
|
2020-07-09 17:16:46 +00:00
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
# It is an error to GRAFT on a explicit peer
|
|
|
|
if peer.peerId in g.parameters.directPeers:
|
2020-10-30 12:49:54 +00:00
|
|
|
# receiving a graft from a direct peer should yield a more prominent warning (protocol violation)
|
|
|
|
warn "attempt to graft an explicit peer", peer=peer.id,
|
2020-09-21 09:16:29 +00:00
|
|
|
topicID=graft.topicID
|
|
|
|
# and such an attempt should be logged and rejected with a PRUNE
|
|
|
|
result.add(ControlPrune(
|
|
|
|
topicID: graft.topicID,
|
|
|
|
peers: @[], # omitting heavy computation here as the remote did something illegal
|
|
|
|
backoff: g.parameters.pruneBackoff.seconds.uint64))
|
2020-10-30 12:49:54 +00:00
|
|
|
|
|
|
|
g.punishPeer(peer, @[topic])
|
2020-11-23 21:02:23 +00:00
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
continue
|
2020-07-09 17:16:46 +00:00
|
|
|
|
2020-10-30 12:49:54 +00:00
|
|
|
if peer.peerId in g.backingOff and g.backingOff[peer.peerId] > Moment.now():
|
|
|
|
trace "attempt to graft a backingOff peer", peer=peer.id,
|
2020-09-21 09:16:29 +00:00
|
|
|
topicID=graft.topicID,
|
|
|
|
expire=g.backingOff[peer.peerId]
|
|
|
|
# and such an attempt should be logged and rejected with a PRUNE
|
|
|
|
result.add(ControlPrune(
|
|
|
|
topicID: graft.topicID,
|
|
|
|
peers: @[], # omitting heavy computation here as the remote did something illegal
|
|
|
|
backoff: g.parameters.pruneBackoff.seconds.uint64))
|
2020-11-23 21:02:23 +00:00
|
|
|
|
2020-10-30 12:49:54 +00:00
|
|
|
g.punishPeer(peer, @[topic])
|
2020-11-23 21:02:23 +00:00
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
continue
|
2020-09-22 07:05:53 +00:00
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
if peer notin g.peerStats:
|
2020-10-30 12:49:54 +00:00
|
|
|
g.onNewPeer(peer)
|
|
|
|
|
|
|
|
# not in the spec exactly, but let's avoid way too low score peers
|
|
|
|
# other clients do it too also was an audit recommendation
|
|
|
|
if peer.score < g.parameters.publishThreshold:
|
|
|
|
continue
|
2020-09-22 07:05:53 +00:00
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
# If they send us a graft before they send us a subscribe, what should
|
|
|
|
# we do? For now, we add them to mesh but don't add them to gossipsub.
|
2020-07-09 17:16:46 +00:00
|
|
|
if topic in g.topics:
|
2020-11-19 07:48:17 +00:00
|
|
|
if g.mesh.peers(topic) < g.parameters.dHigh or peer.outbound:
|
2020-07-09 17:16:46 +00:00
|
|
|
# In the spec, there's no mention of DHi here, but implicitly, a
|
|
|
|
# peer will be removed from the mesh on next rebalance, so we don't want
|
|
|
|
# this peer to push someone else out
|
2020-07-13 13:32:38 +00:00
|
|
|
if g.mesh.addPeer(topic, peer):
|
2020-09-21 09:16:29 +00:00
|
|
|
g.grafted(peer, topic)
|
2020-07-13 13:32:38 +00:00
|
|
|
g.fanout.removePeer(topic, peer)
|
2020-07-09 17:16:46 +00:00
|
|
|
else:
|
2020-07-16 19:26:57 +00:00
|
|
|
trace "peer already in mesh"
|
2019-12-06 02:16:18 +00:00
|
|
|
else:
|
2020-12-15 01:25:22 +00:00
|
|
|
trace "pruning grafting peer, mesh full", peer, score = peer.score, mesh = g.mesh.peers(topic)
|
2020-09-21 09:16:29 +00:00
|
|
|
result.add(ControlPrune(
|
2020-09-22 07:05:53 +00:00
|
|
|
topicID: topic,
|
2020-09-21 09:16:29 +00:00
|
|
|
peers: g.peerExchangeList(topic),
|
|
|
|
backoff: g.parameters.pruneBackoff.seconds.uint64))
|
2019-12-06 02:16:18 +00:00
|
|
|
else:
|
2020-09-21 09:16:29 +00:00
|
|
|
trace "peer grafting topic we're not interested in", topic
|
|
|
|
# gossip 1.1, we do not send a control message prune anymore
|
2019-12-06 02:16:18 +00:00
|
|
|
|
2020-08-05 12:02:26 +00:00
|
|
|
when defined(libp2p_expensive_metrics):
|
|
|
|
libp2p_gossipsub_peers_per_topic_mesh
|
|
|
|
.set(g.mesh.peers(topic).int64, labelValues = [topic])
|
|
|
|
libp2p_gossipsub_peers_per_topic_fanout
|
|
|
|
.set(g.fanout.peers(topic).int64, labelValues = [topic])
|
2020-06-12 02:20:58 +00:00
|
|
|
|
2019-12-06 02:16:18 +00:00
|
|
|
proc handlePrune(g: GossipSub, peer: PubSubPeer, prunes: seq[ControlPrune]) =
|
|
|
|
for prune in prunes:
|
2020-09-06 08:31:47 +00:00
|
|
|
trace "peer pruned topic", peer, topic = prune.topicID
|
2019-12-06 02:16:18 +00:00
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
# add peer backoff
|
|
|
|
if prune.backoff > 0:
|
|
|
|
let backoff = Moment.fromNow((prune.backoff + BackoffSlackTime).int64.seconds)
|
|
|
|
let current = g.backingOff.getOrDefault(peer.peerId)
|
|
|
|
if backoff > current:
|
|
|
|
g.backingOff[peer.peerId] = backoff
|
2020-09-22 07:05:53 +00:00
|
|
|
|
2020-12-15 01:25:22 +00:00
|
|
|
trace "pruning rpc received peer", peer, score = peer.score
|
2020-09-21 09:16:29 +00:00
|
|
|
g.pruned(peer, prune.topicID)
|
2020-07-13 13:32:38 +00:00
|
|
|
g.mesh.removePeer(prune.topicID, peer)
|
2020-09-22 07:05:53 +00:00
|
|
|
|
2020-10-30 12:49:54 +00:00
|
|
|
# TODO peer exchange, we miss ambient peer discovery in libp2p, so we are blocked by that
|
|
|
|
# another option could be to implement signed peer records
|
|
|
|
## if peer.score > g.parameters.gossipThreshold and prunes.peers.len > 0:
|
|
|
|
|
2020-08-05 12:02:26 +00:00
|
|
|
when defined(libp2p_expensive_metrics):
|
|
|
|
libp2p_gossipsub_peers_per_topic_mesh
|
|
|
|
.set(g.mesh.peers(prune.topicID).int64, labelValues = [prune.topicID])
|
2019-12-06 02:16:18 +00:00
|
|
|
|
2020-05-21 20:24:20 +00:00
|
|
|
proc handleIHave(g: GossipSub,
|
|
|
|
peer: PubSubPeer,
|
|
|
|
ihaves: seq[ControlIHave]): ControlIWant =
|
2020-09-21 09:16:29 +00:00
|
|
|
if peer.score < g.parameters.gossipThreshold:
|
2020-09-24 16:43:20 +00:00
|
|
|
trace "ihave: ignoring low score peer", peer, score = peer.score
|
2020-10-30 12:49:54 +00:00
|
|
|
elif peer.iHaveBudget <= 0:
|
2020-09-24 16:43:20 +00:00
|
|
|
trace "ihave: ignoring out of budget peer", peer, score = peer.score
|
2020-09-21 09:16:29 +00:00
|
|
|
else:
|
2020-10-30 12:49:54 +00:00
|
|
|
var deIhaves = ihaves.deduplicate()
|
|
|
|
for ihave in deIhaves.mitems:
|
2020-09-21 09:16:29 +00:00
|
|
|
trace "peer sent ihave",
|
|
|
|
peer, topic = ihave.topicID, msgs = ihave.messageIDs
|
|
|
|
if ihave.topicID in g.mesh:
|
|
|
|
for m in ihave.messageIDs:
|
|
|
|
if m notin g.seen:
|
2020-10-30 12:49:54 +00:00
|
|
|
if peer.iHaveBudget > 0:
|
|
|
|
result.messageIDs.add(m)
|
|
|
|
dec peer.iHaveBudget
|
|
|
|
else:
|
|
|
|
return
|
2020-11-23 21:02:23 +00:00
|
|
|
|
|
|
|
# shuffling result.messageIDs before sending it out to increase the likelihood
|
2020-10-30 12:49:54 +00:00
|
|
|
# of getting an answer if the peer truncates the list due to internal size restrictions.
|
|
|
|
shuffle(result.messageIDs)
|
2019-12-06 02:16:18 +00:00
|
|
|
|
2020-05-21 20:24:20 +00:00
|
|
|
proc handleIWant(g: GossipSub,
|
|
|
|
peer: PubSubPeer,
|
|
|
|
iwants: seq[ControlIWant]): seq[Message] =
|
2020-09-21 09:16:29 +00:00
|
|
|
if peer.score < g.parameters.gossipThreshold:
|
2020-09-24 16:43:20 +00:00
|
|
|
trace "iwant: ignoring low score peer", peer, score = peer.score
|
2020-10-30 12:49:54 +00:00
|
|
|
elif peer.iWantBudget <= 0:
|
|
|
|
trace "iwant: ignoring out of budget peer", peer, score = peer.score
|
2020-09-21 09:16:29 +00:00
|
|
|
else:
|
2020-10-30 12:49:54 +00:00
|
|
|
var deIwants = iwants.deduplicate()
|
|
|
|
for iwant in deIwants:
|
2020-09-21 09:16:29 +00:00
|
|
|
for mid in iwant.messageIDs:
|
|
|
|
trace "peer sent iwant", peer, messageID = mid
|
|
|
|
let msg = g.mcache.get(mid)
|
|
|
|
if msg.isSome:
|
|
|
|
# avoid spam
|
|
|
|
if peer.iWantBudget > 0:
|
|
|
|
result.add(msg.get())
|
|
|
|
dec peer.iWantBudget
|
|
|
|
else:
|
|
|
|
return
|
|
|
|
|
2020-06-12 23:54:12 +00:00
|
|
|
method rpcHandler*(g: GossipSub,
|
2019-12-06 02:16:18 +00:00
|
|
|
peer: PubSubPeer,
|
2020-09-01 07:33:03 +00:00
|
|
|
rpcMsg: RPCMsg) {.async.} =
|
|
|
|
await procCall PubSub(g).rpcHandler(peer, rpcMsg)
|
|
|
|
|
2020-09-04 06:10:32 +00:00
|
|
|
for msg in rpcMsg.messages: # for every message
|
|
|
|
let msgId = g.msgIdProvider(msg)
|
2020-09-01 07:33:03 +00:00
|
|
|
|
2020-09-04 06:10:32 +00:00
|
|
|
if g.seen.put(msgId):
|
2020-09-06 08:31:47 +00:00
|
|
|
trace "Dropping already-seen message", msgId, peer
|
2020-09-21 09:16:29 +00:00
|
|
|
|
|
|
|
# make sure to update score tho before continuing
|
|
|
|
for t in msg.topicIDs: # for every topic in the message
|
|
|
|
let topicParams = g.topicParams.mgetOrPut(t, TopicParams.init())
|
|
|
|
# if in mesh add more delivery score
|
2020-12-15 01:25:22 +00:00
|
|
|
g.peerStats.withValue(peer, pstats):
|
|
|
|
pstats[].topicInfos.withValue(t, stats):
|
|
|
|
if stats[].inMesh:
|
|
|
|
# TODO: take into account meshMessageDeliveriesWindow
|
|
|
|
# score only if messages are not too old.
|
|
|
|
stats[].meshMessageDeliveries += 1
|
|
|
|
if stats[].meshMessageDeliveries > topicParams.meshMessageDeliveriesCap:
|
|
|
|
stats[].meshMessageDeliveries = topicParams.meshMessageDeliveriesCap
|
|
|
|
do: # make sure we don't loose this information
|
2020-12-15 21:46:03 +00:00
|
|
|
pstats[].topicInfos[t] = TopicInfo(meshMessageDeliveries: 1)
|
2020-12-15 01:25:22 +00:00
|
|
|
do: # make sure we don't loose this information
|
|
|
|
g.peerStats[peer] =
|
|
|
|
block:
|
|
|
|
var stats = PeerStats()
|
|
|
|
stats.topicInfos[t] = TopicInfo(meshMessageDeliveries: 1)
|
|
|
|
stats
|
2020-12-15 21:46:03 +00:00
|
|
|
|
2020-12-15 01:25:22 +00:00
|
|
|
# onto the next message
|
2020-09-04 06:10:32 +00:00
|
|
|
continue
|
2020-09-01 07:33:03 +00:00
|
|
|
|
2020-09-23 15:56:33 +00:00
|
|
|
if (msg.signature.len > 0 or g.verifySignature) and not msg.verify():
|
|
|
|
# always validate if signature is present or required
|
2020-09-06 08:31:47 +00:00
|
|
|
debug "Dropping message due to failed signature verification", msgId, peer
|
2020-10-30 12:49:54 +00:00
|
|
|
g.punishPeer(peer, msg.topicIDs)
|
2020-09-04 06:10:32 +00:00
|
|
|
continue
|
2020-09-01 07:33:03 +00:00
|
|
|
|
2020-09-23 15:56:33 +00:00
|
|
|
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, peer
|
2020-10-30 12:49:54 +00:00
|
|
|
g.punishPeer(peer, msg.topicIDs)
|
2020-09-23 15:56:33 +00:00
|
|
|
continue
|
|
|
|
|
|
|
|
# g.anonymize needs no evaluation when receiving messages
|
|
|
|
# as we have a "lax" policy and allow signed messages
|
|
|
|
|
2020-10-21 03:25:42 +00:00
|
|
|
let validation = await g.validate(msg)
|
|
|
|
case validation
|
|
|
|
of ValidationResult.Reject:
|
2020-11-11 16:42:12 +00:00
|
|
|
debug "Dropping message after validation, reason: reject", msgId, peer
|
2020-10-30 12:49:54 +00:00
|
|
|
g.punishPeer(peer, msg.topicIDs)
|
2020-09-04 06:10:32 +00:00
|
|
|
continue
|
2020-10-21 03:25:42 +00:00
|
|
|
of ValidationResult.Ignore:
|
2020-11-11 16:42:12 +00:00
|
|
|
debug "Dropping message after validation, reason: ignore", msgId, peer
|
2020-10-21 03:25:42 +00:00
|
|
|
continue
|
|
|
|
of ValidationResult.Accept:
|
|
|
|
discard
|
2020-09-01 07:33:03 +00:00
|
|
|
|
2020-12-15 01:25:22 +00:00
|
|
|
# store in cache only after validation
|
|
|
|
g.mcache.put(msgId, msg)
|
|
|
|
|
2020-09-04 06:10:32 +00:00
|
|
|
var toSendPeers = initHashSet[PubSubPeer]()
|
2020-09-21 09:16:29 +00:00
|
|
|
for t in msg.topicIDs: # for every topic in the message
|
|
|
|
let topicParams = g.topicParams.mgetOrPut(t, TopicParams.init())
|
|
|
|
|
2020-12-15 01:25:22 +00:00
|
|
|
g.peerStats.withValue(peer, pstats):
|
|
|
|
pstats[].topicInfos.withValue(t, stats):
|
|
|
|
# contribute to peer score first delivery
|
|
|
|
stats[].firstMessageDeliveries += 1
|
|
|
|
if stats[].firstMessageDeliveries > topicParams.firstMessageDeliveriesCap:
|
|
|
|
stats[].firstMessageDeliveries = topicParams.firstMessageDeliveriesCap
|
|
|
|
|
|
|
|
# if in mesh add more delivery score
|
|
|
|
if stats[].inMesh:
|
|
|
|
stats[].meshMessageDeliveries += 1
|
|
|
|
if stats[].meshMessageDeliveries > topicParams.meshMessageDeliveriesCap:
|
|
|
|
stats[].meshMessageDeliveries = topicParams.meshMessageDeliveriesCap
|
|
|
|
do: # make sure we don't loose this information
|
2020-12-15 21:46:03 +00:00
|
|
|
pstats[].topicInfos[t] = TopicInfo(firstMessageDeliveries: 1, meshMessageDeliveries: 1)
|
2020-12-15 01:25:22 +00:00
|
|
|
do: # make sure we don't loose this information
|
|
|
|
g.peerStats[peer] =
|
|
|
|
block:
|
|
|
|
var stats = PeerStats()
|
2020-12-15 21:46:03 +00:00
|
|
|
stats.topicInfos[t] = TopicInfo(firstMessageDeliveries: 1, meshMessageDeliveries: 1)
|
2020-12-15 01:25:22 +00:00
|
|
|
stats
|
2020-09-21 09:16:29 +00:00
|
|
|
|
2020-09-04 06:10:32 +00:00
|
|
|
g.floodsub.withValue(t, peers): toSendPeers.incl(peers[])
|
|
|
|
g.mesh.withValue(t, peers): toSendPeers.incl(peers[])
|
2020-09-01 07:33:03 +00:00
|
|
|
|
2020-09-04 06:10:32 +00:00
|
|
|
await handleData(g, t, msg.data)
|
2020-09-01 07:33:03 +00:00
|
|
|
|
2020-09-04 06:10:32 +00:00
|
|
|
# In theory, if topics are the same in all messages, we could batch - we'd
|
|
|
|
# also have to be careful to only include validated messages
|
|
|
|
g.broadcast(toSeq(toSendPeers), RPCMsg(messages: @[msg]))
|
2020-09-06 08:31:47 +00:00
|
|
|
trace "forwared message to peers", peers = toSendPeers.len, msgId, peer
|
2020-11-26 07:20:34 +00:00
|
|
|
libp2p_pubsub_messages_rebroadcasted.inc()
|
2020-09-01 07:33:03 +00:00
|
|
|
|
|
|
|
if rpcMsg.control.isSome:
|
|
|
|
let control = rpcMsg.control.get()
|
|
|
|
g.handlePrune(peer, control.prune)
|
2019-12-06 02:16:18 +00:00
|
|
|
|
|
|
|
var respControl: ControlMessage
|
2020-09-01 07:33:03 +00:00
|
|
|
respControl.iwant.add(g.handleIHave(peer, control.ihave))
|
|
|
|
respControl.prune.add(g.handleGraft(peer, control.graft))
|
|
|
|
let messages = g.handleIWant(peer, control.iwant)
|
|
|
|
|
|
|
|
if respControl.graft.len > 0 or respControl.prune.len > 0 or
|
|
|
|
respControl.ihave.len > 0 or messages.len > 0:
|
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
trace "sending control message", msg = shortLog(respControl), peer
|
2020-09-01 07:33:03 +00:00
|
|
|
g.send(
|
|
|
|
peer,
|
|
|
|
RPCMsg(control: some(respControl), messages: messages))
|
2019-12-06 02:16:18 +00:00
|
|
|
|
|
|
|
method subscribe*(g: GossipSub,
|
|
|
|
topic: string,
|
2019-12-17 05:24:03 +00:00
|
|
|
handler: TopicHandler) {.async.} =
|
2019-12-06 02:16:18 +00:00
|
|
|
await procCall PubSub(g).subscribe(topic, handler)
|
2020-09-22 07:05:53 +00:00
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
# if we have a fanout on this topic break it
|
|
|
|
if topic in g.fanout:
|
|
|
|
g.fanout.del(topic)
|
2020-09-22 07:05:53 +00:00
|
|
|
|
2020-12-15 01:25:22 +00:00
|
|
|
g.rebalanceMesh(topic)
|
2019-12-06 02:16:18 +00:00
|
|
|
|
|
|
|
method unsubscribe*(g: GossipSub,
|
2019-12-17 05:24:03 +00:00
|
|
|
topics: seq[TopicPair]) {.async.} =
|
2019-12-06 02:16:18 +00:00
|
|
|
await procCall PubSub(g).unsubscribe(topics)
|
|
|
|
|
2020-07-20 16:16:13 +00:00
|
|
|
for (topic, handler) in topics:
|
|
|
|
# delete from mesh only if no handlers are left
|
2020-09-04 06:10:32 +00:00
|
|
|
if topic notin g.topics:
|
2020-07-20 16:16:13 +00:00
|
|
|
if topic in g.mesh:
|
2020-09-04 06:10:32 +00:00
|
|
|
let peers = g.mesh[topic]
|
2020-07-20 16:16:13 +00:00
|
|
|
g.mesh.del(topic)
|
2020-09-21 09:16:29 +00:00
|
|
|
g.topicParams.del(topic)
|
|
|
|
for peer in peers:
|
2020-12-15 01:25:22 +00:00
|
|
|
trace "pruning unsubscribe call peer", peer, score = peer.score
|
2020-09-21 09:16:29 +00:00
|
|
|
g.pruned(peer, topic)
|
|
|
|
let prune = RPCMsg(control: some(ControlMessage(
|
|
|
|
prune: @[ControlPrune(
|
2020-09-22 07:05:53 +00:00
|
|
|
topicID: topic,
|
2020-09-21 09:16:29 +00:00
|
|
|
peers: g.peerExchangeList(topic),
|
|
|
|
backoff: g.parameters.pruneBackoff.seconds.uint64)])))
|
2020-09-01 07:33:03 +00:00
|
|
|
g.broadcast(toSeq(peers), prune)
|
2020-07-20 16:16:13 +00:00
|
|
|
|
|
|
|
method unsubscribeAll*(g: GossipSub, topic: string) {.async.} =
|
|
|
|
await procCall PubSub(g).unsubscribeAll(topic)
|
|
|
|
|
|
|
|
if topic in g.mesh:
|
|
|
|
let peers = g.mesh.getOrDefault(topic)
|
|
|
|
g.mesh.del(topic)
|
2020-09-21 09:16:29 +00:00
|
|
|
for peer in peers:
|
2020-12-15 01:25:22 +00:00
|
|
|
trace "pruning unsubscribeAll call peer", peer, score = peer.score
|
2020-09-21 09:16:29 +00:00
|
|
|
g.pruned(peer, topic)
|
|
|
|
let prune = RPCMsg(control: some(ControlMessage(
|
|
|
|
prune: @[ControlPrune(
|
2020-09-22 07:05:53 +00:00
|
|
|
topicID: topic,
|
2020-09-21 09:16:29 +00:00
|
|
|
peers: g.peerExchangeList(topic),
|
|
|
|
backoff: g.parameters.pruneBackoff.seconds.uint64)])))
|
2020-09-01 07:33:03 +00:00
|
|
|
g.broadcast(toSeq(peers), prune)
|
2019-12-06 02:16:18 +00:00
|
|
|
|
|
|
|
method publish*(g: GossipSub,
|
|
|
|
topic: string,
|
2020-09-01 07:33:03 +00:00
|
|
|
data: seq[byte]): Future[int] {.async.} =
|
2020-07-08 00:33:05 +00:00
|
|
|
# base returns always 0
|
2020-09-01 07:33:03 +00:00
|
|
|
discard await procCall PubSub(g).publish(topic, data)
|
2020-09-04 06:10:32 +00:00
|
|
|
|
|
|
|
logScope: topic
|
|
|
|
trace "Publishing message on topic", data = data.shortLog
|
2020-07-16 19:26:57 +00:00
|
|
|
|
2020-07-08 00:33:05 +00:00
|
|
|
if topic.len <= 0: # data could be 0/empty
|
2020-09-04 06:10:32 +00:00
|
|
|
debug "Empty topic, skipping publish"
|
2020-07-08 00:33:05 +00:00
|
|
|
return 0
|
2020-09-22 07:05:53 +00:00
|
|
|
|
2020-08-15 19:50:31 +00:00
|
|
|
var peers: HashSet[PubSubPeer]
|
2020-09-21 09:16:29 +00:00
|
|
|
|
|
|
|
if g.parameters.floodPublish:
|
2020-09-22 07:05:53 +00:00
|
|
|
# With flood publishing enabled, the mesh is used when propagating messages from other peers,
|
2020-09-21 09:16:29 +00:00
|
|
|
# but a peer's own messages will always be published to all known peers in the topic.
|
|
|
|
for peer in g.gossipsub.getOrDefault(topic):
|
|
|
|
if peer.score >= g.parameters.publishThreshold:
|
2020-09-24 16:43:20 +00:00
|
|
|
trace "publish: including flood/high score peer", peer
|
2020-09-21 09:16:29 +00:00
|
|
|
peers.incl(peer)
|
|
|
|
|
|
|
|
# add always direct peers
|
|
|
|
peers.incl(g.explicit.getOrDefault(topic))
|
|
|
|
|
2020-07-08 00:33:05 +00:00
|
|
|
if topic in g.topics: # if we're subscribed use the mesh
|
2020-09-21 09:16:29 +00:00
|
|
|
peers.incl(g.mesh.getOrDefault(topic))
|
2020-07-08 00:33:05 +00:00
|
|
|
else: # not subscribed, send to fanout peers
|
|
|
|
# try optimistically
|
2020-09-21 09:16:29 +00:00
|
|
|
peers.incl(g.fanout.getOrDefault(topic))
|
2020-07-08 00:33:05 +00:00
|
|
|
if peers.len == 0:
|
|
|
|
# ok we had nothing.. let's try replenish inline
|
|
|
|
g.replenishFanout(topic)
|
2020-09-21 09:16:29 +00:00
|
|
|
peers.incl(g.fanout.getOrDefault(topic))
|
2020-07-08 00:33:05 +00:00
|
|
|
|
2020-07-09 20:21:47 +00:00
|
|
|
# even if we couldn't publish,
|
|
|
|
# we still attempted to publish
|
|
|
|
# on the topic, so it makes sense
|
|
|
|
# to update the last topic publish
|
|
|
|
# time
|
2020-11-19 07:48:17 +00:00
|
|
|
g.lastFanoutPubSub[topic] = Moment.fromNow(g.parameters.fanoutTTL)
|
2020-07-09 20:21:47 +00:00
|
|
|
|
2020-09-04 06:10:32 +00:00
|
|
|
if peers.len == 0:
|
|
|
|
debug "No peers for topic, skipping publish"
|
|
|
|
return 0
|
|
|
|
|
2020-07-15 03:51:33 +00:00
|
|
|
inc g.msgSeqno
|
2020-07-08 00:33:05 +00:00
|
|
|
let
|
2020-09-23 15:56:33 +00:00
|
|
|
msg =
|
|
|
|
if g.anonymize:
|
|
|
|
Message.init(none(PeerInfo), data, topic, none(uint64), false)
|
|
|
|
else:
|
|
|
|
Message.init(some(g.peerInfo), data, topic, some(g.msgSeqno), g.sign)
|
2020-07-08 00:33:05 +00:00
|
|
|
msgId = g.msgIdProvider(msg)
|
|
|
|
|
2020-09-04 06:10:32 +00:00
|
|
|
logScope: msgId
|
2020-07-08 00:33:05 +00:00
|
|
|
|
2020-09-04 06:10:32 +00:00
|
|
|
trace "Created new message", msg = shortLog(msg), peers = peers.len
|
2020-06-17 04:14:02 +00:00
|
|
|
|
2020-09-04 06:10:32 +00:00
|
|
|
if g.seen.put(msgId):
|
|
|
|
# custom msgid providers might cause this
|
|
|
|
trace "Dropping already-seen message"
|
2020-08-15 19:50:31 +00:00
|
|
|
return 0
|
2020-07-08 00:33:05 +00:00
|
|
|
|
2020-09-04 06:10:32 +00:00
|
|
|
g.mcache.put(msgId, msg)
|
|
|
|
|
|
|
|
g.broadcast(toSeq(peers), RPCMsg(messages: @[msg]))
|
|
|
|
when defined(libp2p_expensive_metrics):
|
|
|
|
if peers.len > 0:
|
|
|
|
libp2p_pubsub_messages_published.inc(labelValues = [topic])
|
2020-11-26 07:20:34 +00:00
|
|
|
else:
|
|
|
|
if peers.len > 0:
|
|
|
|
libp2p_pubsub_messages_published.inc()
|
2020-09-04 06:10:32 +00:00
|
|
|
|
|
|
|
trace "Published message to peers"
|
|
|
|
|
|
|
|
return peers.len
|
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
proc maintainDirectPeers(g: GossipSub) {.async.} =
|
|
|
|
while g.heartbeatRunning:
|
|
|
|
for id in g.parameters.directPeers:
|
|
|
|
let peer = g.peers.getOrDefault(id)
|
|
|
|
if peer == nil:
|
|
|
|
# this creates a new peer and assigns the current switch to it
|
|
|
|
# as a result the next time we try to Send we will as well try to open a connection
|
|
|
|
# see pubsubpeer.nim send and such
|
|
|
|
discard g.getOrCreatePeer(id, g.codecs)
|
|
|
|
|
|
|
|
await sleepAsync(1.minutes)
|
|
|
|
|
2019-12-06 02:16:18 +00:00
|
|
|
method start*(g: GossipSub) {.async.} =
|
2020-07-08 00:33:05 +00:00
|
|
|
trace "gossipsub start"
|
2020-06-28 15:56:38 +00:00
|
|
|
|
2020-09-04 16:31:43 +00:00
|
|
|
if not g.heartbeatFut.isNil:
|
|
|
|
warn "Starting gossipsub twice"
|
|
|
|
return
|
2019-12-06 02:16:18 +00:00
|
|
|
|
2020-06-20 10:56:55 +00:00
|
|
|
g.heartbeatRunning = true
|
|
|
|
g.heartbeatFut = g.heartbeat()
|
2020-09-21 09:16:29 +00:00
|
|
|
g.directPeersLoop = g.maintainDirectPeers()
|
2020-06-20 10:56:55 +00:00
|
|
|
|
2019-12-06 02:16:18 +00:00
|
|
|
method stop*(g: GossipSub) {.async.} =
|
2020-07-08 00:33:05 +00:00
|
|
|
trace "gossipsub stop"
|
2020-09-04 16:31:43 +00:00
|
|
|
if g.heartbeatFut.isNil:
|
|
|
|
warn "Stopping gossipsub without starting it"
|
|
|
|
return
|
2019-12-06 02:16:18 +00:00
|
|
|
|
|
|
|
# stop heartbeat interval
|
2020-06-20 10:56:55 +00:00
|
|
|
g.heartbeatRunning = false
|
2020-09-21 09:16:29 +00:00
|
|
|
g.directPeersLoop.cancel()
|
2020-06-20 10:56:55 +00:00
|
|
|
if not g.heartbeatFut.finished:
|
2020-07-08 00:33:05 +00:00
|
|
|
trace "awaiting last heartbeat"
|
2020-06-20 10:56:55 +00:00
|
|
|
await g.heartbeatFut
|
2020-07-08 00:33:05 +00:00
|
|
|
trace "heartbeat stopped"
|
2020-09-04 16:31:43 +00:00
|
|
|
g.heartbeatFut = nil
|
2019-12-06 02:16:18 +00:00
|
|
|
|
2020-05-29 15:35:03 +00:00
|
|
|
method initPubSub*(g: GossipSub) =
|
2019-12-06 02:16:18 +00:00
|
|
|
procCall FloodSub(g).initPubSub()
|
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
if not g.parameters.explicit:
|
|
|
|
g.parameters = GossipSubParams.init()
|
2020-09-22 07:05:53 +00:00
|
|
|
|
2020-09-21 09:16:29 +00:00
|
|
|
g.parameters.validateParameters().tryGet()
|
|
|
|
|
2020-05-21 20:24:20 +00:00
|
|
|
randomize()
|
2020-11-26 05:45:10 +00:00
|
|
|
|
|
|
|
# init the floodsub stuff here, we customize timedcache in gossip!
|
|
|
|
g.floodsub = initTable[string, HashSet[PubSubPeer]]()
|
|
|
|
g.seen = TimedCache[MessageID].init(g.parameters.seenTTL)
|
|
|
|
|
|
|
|
# init gossip stuff
|
2020-11-19 07:48:17 +00:00
|
|
|
g.mcache = MCache.init(g.parameters.historyGossip, g.parameters.historyLength)
|
2020-07-13 13:32:38 +00:00
|
|
|
g.mesh = initTable[string, HashSet[PubSubPeer]]() # meshes - topic to peer
|
|
|
|
g.fanout = initTable[string, HashSet[PubSubPeer]]() # fanout - topic to peer
|
|
|
|
g.gossipsub = initTable[string, HashSet[PubSubPeer]]()# topic to peer map of all gossipsub peers
|
2020-06-11 18:09:34 +00:00
|
|
|
g.lastFanoutPubSub = initTable[string, Moment]() # last publish time for fanout topics
|
2019-12-06 02:16:18 +00:00
|
|
|
g.gossip = initTable[string, seq[ControlIHave]]() # pending gossip
|
2020-06-11 18:09:34 +00:00
|
|
|
g.control = initTable[string, ControlMessage]() # pending control messages
|