Dmitriy Ryajov e623e70e7b
PubSub (Gossip & Flood) Implementation (#36)
This adds gossipsub and floodsub, as well as basic interop testing with the go libp2p daemon. 

* add close event

* wip: gossipsub

* splitting rpc message

* making message handling more consistent

* initial gossipsub implementation

* feat: nim 1.0 cleanup

* wip: gossipsub protobuf

* adding encoding/decoding of gossipsub messages

* add disconnect handler

* add proper gossipsub msg handling

* misc: cleanup for nim 1.0

* splitting floodsub and gossipsub tests

* feat: add mesh rebalansing

* test pubsub

* add mesh rebalansing tests

* testing mesh maintenance

* finishing mcache implementatin

* wip: commenting out broken tests

* wip: don't run heartbeat for now

* switchout debug for trace logging

* testing gossip peer selection algorithm

* test stream piping

* more work around message amplification

* get the peerid from message

* use timed cache as backing store

* allow setting timeout in constructor

* several changes to improve performance

* more through testing of msg amplification

* prevent gc issues

* allow piping to self and prevent deadlocks

* improove floodsub

* allow running hook on cache eviction

* prevent race conditions

* prevent race conditions and improove tests

* use hashes as cache keys

* removing useless file

* don't create a new seq

* re-enable pubsub tests

* fix imports

* reduce number of runs to speed up tests

* break out control message processing

* normalize sleeps between steps

* implement proper transport filtering

* initial interop testing

* clean up floodsub publish logic

* allow dialing without a protocol

* adding multiple reads/writes

* use protobuf varint in mplex

* don't loose conn's peerInfo

* initial interop pubsub tests

* don't duplicate connections/peers

* bring back interop tests

* wip: interop

* re-enable interop and daemon tests

* add multiple read write tests from handlers

* don't cleanup channel prematurely

* use correct channel to send/receive msgs

* adjust tests with latest changes

* include interop tests

* remove temp logging output

* fix ci

* use correct public key serialization

* additional tests for pubsub interop
2019-12-05 20:16:18 -06:00

125 lines
4.5 KiB
Nim

## 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.
import sequtils, tables, options, sets, strutils
import chronos, chronicles
import pubsub,
pubsubpeer,
timedcache,
rpc/[messages, message],
../../crypto/crypto,
../../connection,
../../peerinfo,
../../peer
logScope:
topic = "FloodSub"
const FloodSubCodec* = "/floodsub/1.0.0"
type
FloodSub* = ref object of PubSub
floodsub*: Table[string, HashSet[string]] # topic to remote peer map
seen*: TimedCache[string] # list of messages forwarded to peers
method subscribeTopic*(f: FloodSub,
topic: string,
subscribe: bool,
peerId: string) {.gcsafe.} =
procCall PubSub(f).subscribeTopic(topic, subscribe, peerId)
if topic notin f.floodsub:
f.floodsub[topic] = initHashSet[string]()
if subscribe:
trace "adding subscription for topic", peer = peerId, name = topic
# subscribe the peer to the topic
f.floodsub[topic].incl(peerId)
else:
trace "removing subscription for topic", peer = peerId, name = topic
# unsubscribe the peer from the topic
f.floodsub[topic].excl(peerId)
method handleDisconnect*(f: FloodSub, peer: PubSubPeer) {.async, gcsafe.} =
## handle peer disconnects
for t in f.floodsub.keys:
f.floodsub[t].excl(peer.id)
method rpcHandler*(f: FloodSub,
peer: PubSubPeer,
rpcMsgs: seq[RPCMsg]) {.async, gcsafe.} =
trace "processing RPC message", peer = peer.id, msg = rpcMsgs
for m in rpcMsgs: # for all RPC messages
trace "processing message", msg = rpcMsgs
if m.subscriptions.len > 0: # if there are any subscriptions
for s in m.subscriptions: # subscribe/unsubscribe the peer for each topic
f.subscribeTopic(s.topic, s.subscribe, peer.id)
if m.messages.len > 0: # if there are any messages
var toSendPeers: HashSet[string] = initHashSet[string]()
for msg in m.messages: # for every message
if msg.msgId notin f.seen:
f.seen.put(msg.msgId) # add the message to the seen cache
for t in msg.topicIDs: # for every topic in the message
if t in f.floodsub:
toSendPeers.incl(f.floodsub[t]) # get all the peers interested in this topic
if t in f.topics: # check that we're subscribed to it
for h in f.topics[t].handler:
await h(t, msg.data) # trigger user provided handler
# forward the message to all peers interested in it
for p in toSendPeers:
if p in f.peers and f.peers[p].id != peer.id:
await f.peers[p].send(@[RPCMsg(messages: m.messages)])
method init(f: FloodSub) =
proc handler(conn: Connection, proto: string) {.async, gcsafe.} =
## main protocol handler that gets triggered on every
## connection for a protocol string
## e.g. ``/floodsub/1.0.0``, etc...
##
await f.handleConn(conn, proto)
f.handler = handler
f.codec = FloodSubCodec
method publish*(f: FloodSub,
topic: string,
data: seq[byte]) {.async, gcsafe.} =
await procCall PubSub(f).publish(topic, data)
if data.len <= 0 or topic.len <= 0:
trace "topic or data missing, skipping publish"
return
if topic notin f.floodsub:
trace "missing peers for topic, skipping publish"
return
trace "publishing on topic", name = topic
let msg = newMessage(f.peerInfo.peerId.get(), data, topic)
for p in f.floodsub[topic]:
trace "publishing message", name = topic, peer = p, data = data
await f.peers[p].send(@[RPCMsg(messages: @[msg])])
method unsubscribe*(f: FloodSub,
topics: seq[TopicPair]) {.async, gcsafe.} =
await procCall PubSub(f).unsubscribe(topics)
for p in f.peers.values:
await f.sendSubs(p, topics.mapIt(it.topic).deduplicate(), false)
method initPubSub*(f: FloodSub) =
f.peers = initTable[string, PubSubPeer]()
f.topics = initTable[string, Topic]()
f.floodsub = initTable[string, HashSet[string]]()
f.seen = newTimedCache[string](2.minutes)
f.init()