2019-09-10 02:15:52 +00:00
|
|
|
## Nim-LibP2P
|
2019-09-24 17:48:23 +00:00
|
|
|
## Copyright (c) 2019 Status Research & Development GmbH
|
2019-09-10 02:15:52 +00:00
|
|
|
## 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.
|
|
|
|
|
remove send lock (#334)
* remove send lock
When mplex receives data it will block until a reader has processed the
data. Thus, when a large message is received, such as a gossipsub
subscription table, all of mplex will be blocked until all reading is
finished.
However, if at the same time a `dial` to establish a gossipsub send
connection is ongoing, that `dial` will be blocked because mplex is no
longer reading data - specifically, it might indeed be the connection
that's processing the previous data that is waiting for a send
connection.
There are other problems with the current code:
* If an exception is raised, it is not necessarily raised for the same
connection as `p.sendConn`, so resetting `p.sendConn` in the exception
handling is wrong
* `p.isConnected` is checked before taking the lock - thus, if it
returns false, a new dial will be started. If a new task enters `send`
before dial is finished, it will also determine `p.isConnected` is
false, then get stuck on the lock - when the previous task finishes and
releases the lock, the new task will _also_ dial and thus reset
`p.sendConn` causing a leak.
* prefer existing connection
simplifies flow
2020-08-17 10:38:27 +00:00
|
|
|
import std/[hashes, options, strutils, tables]
|
2020-06-11 18:09:34 +00:00
|
|
|
import chronos, chronicles, nimcrypto/sha2, metrics
|
2019-12-10 20:50:35 +00:00
|
|
|
import rpc/[messages, message, protobuf],
|
2019-10-03 21:34:12 +00:00
|
|
|
timedcache,
|
2020-08-12 00:05:49 +00:00
|
|
|
../../switch,
|
2020-07-01 06:25:09 +00:00
|
|
|
../../peerid,
|
2019-09-24 16:16:39 +00:00
|
|
|
../../peerinfo,
|
2020-06-19 17:29:43 +00:00
|
|
|
../../stream/connection,
|
2019-09-12 05:46:08 +00:00
|
|
|
../../crypto/crypto,
|
2020-03-23 06:03:36 +00:00
|
|
|
../../protobuf/minprotobuf,
|
|
|
|
../../utility
|
2019-09-10 02:15:52 +00:00
|
|
|
|
|
|
|
logScope:
|
2020-06-10 08:48:01 +00:00
|
|
|
topics = "pubsubpeer"
|
2019-09-10 02:15:52 +00:00
|
|
|
|
2020-08-04 23:27:59 +00:00
|
|
|
when defined(libp2p_expensive_metrics):
|
|
|
|
declareCounter(libp2p_pubsub_sent_messages, "number of messages sent", labels = ["id", "topic"])
|
|
|
|
declareCounter(libp2p_pubsub_received_messages, "number of messages received", labels = ["id", "topic"])
|
|
|
|
declareCounter(libp2p_pubsub_skipped_received_messages, "number of received skipped messages", labels = ["id"])
|
|
|
|
declareCounter(libp2p_pubsub_skipped_sent_messages, "number of sent skipped messages", labels = ["id"])
|
2020-06-12 02:20:58 +00:00
|
|
|
|
2020-08-03 05:20:11 +00:00
|
|
|
const
|
|
|
|
DefaultSendTimeout* = 10.seconds
|
|
|
|
|
2019-09-10 02:15:52 +00:00
|
|
|
type
|
2020-04-30 13:22:31 +00:00
|
|
|
PubSubObserver* = ref object
|
2020-05-29 16:46:27 +00:00
|
|
|
onRecv*: proc(peer: PubSubPeer; msgs: var RPCMsg) {.gcsafe, raises: [Defect].}
|
|
|
|
onSend*: proc(peer: PubSubPeer; msgs: var RPCMsg) {.gcsafe, raises: [Defect].}
|
2019-09-10 02:15:52 +00:00
|
|
|
|
2020-04-30 13:22:31 +00:00
|
|
|
PubSubPeer* = ref object of RootObj
|
2020-08-12 00:05:49 +00:00
|
|
|
switch*: Switch # switch instance to dial peers
|
|
|
|
codec*: string # the protocol that this peer joined from
|
remove send lock (#334)
* remove send lock
When mplex receives data it will block until a reader has processed the
data. Thus, when a large message is received, such as a gossipsub
subscription table, all of mplex will be blocked until all reading is
finished.
However, if at the same time a `dial` to establish a gossipsub send
connection is ongoing, that `dial` will be blocked because mplex is no
longer reading data - specifically, it might indeed be the connection
that's processing the previous data that is waiting for a send
connection.
There are other problems with the current code:
* If an exception is raised, it is not necessarily raised for the same
connection as `p.sendConn`, so resetting `p.sendConn` in the exception
handling is wrong
* `p.isConnected` is checked before taking the lock - thus, if it
returns false, a new dial will be started. If a new task enters `send`
before dial is finished, it will also determine `p.isConnected` is
false, then get stuck on the lock - when the previous task finishes and
releases the lock, the new task will _also_ dial and thus reset
`p.sendConn` causing a leak.
* prefer existing connection
simplifies flow
2020-08-17 10:38:27 +00:00
|
|
|
sendConn: Connection # cached send connection
|
2020-08-12 00:05:49 +00:00
|
|
|
peerId*: PeerID
|
2020-04-30 13:22:31 +00:00
|
|
|
handler*: RPCHandler
|
2020-07-15 19:18:55 +00:00
|
|
|
sentRpcCache: TimedCache[string] # cache for already sent messages
|
|
|
|
recvdRpcCache: TimedCache[string] # cache for already received messages
|
2020-04-30 13:22:31 +00:00
|
|
|
observers*: ref seq[PubSubObserver] # ref as in smart_ptr
|
2020-08-12 00:05:49 +00:00
|
|
|
subscribed*: bool # are we subscribed to this peer
|
2020-04-30 13:22:31 +00:00
|
|
|
|
|
|
|
RPCHandler* = proc(peer: PubSubPeer, msg: seq[RPCMsg]): Future[void] {.gcsafe.}
|
2019-09-10 02:15:52 +00:00
|
|
|
|
2020-07-15 08:25:39 +00:00
|
|
|
func hash*(p: PubSubPeer): Hash =
|
2020-07-13 13:32:38 +00:00
|
|
|
# int is either 32/64, so intptr basically, pubsubpeer is a ref
|
|
|
|
cast[pointer](p).hash
|
|
|
|
|
2020-08-12 00:05:49 +00:00
|
|
|
proc id*(p: PubSubPeer): string =
|
|
|
|
doAssert(not p.isNil, "nil pubsubpeer")
|
|
|
|
p.peerId.pretty
|
2019-12-06 02:16:18 +00:00
|
|
|
|
2020-06-29 15:15:31 +00:00
|
|
|
proc connected*(p: PubSubPeer): bool =
|
2020-08-12 00:05:49 +00:00
|
|
|
not p.sendConn.isNil and not
|
|
|
|
(p.sendConn.closed or p.sendConn.atEof)
|
2020-07-08 00:33:05 +00:00
|
|
|
|
2020-05-29 16:46:27 +00:00
|
|
|
proc recvObservers(p: PubSubPeer, msg: var RPCMsg) =
|
|
|
|
# trigger hooks
|
|
|
|
if not(isNil(p.observers)) and p.observers[].len > 0:
|
|
|
|
for obs in p.observers[]:
|
2020-06-24 15:08:44 +00:00
|
|
|
if not(isNil(obs)): # TODO: should never be nil, but...
|
|
|
|
obs.onRecv(p, msg)
|
2020-05-29 16:46:27 +00:00
|
|
|
|
|
|
|
proc sendObservers(p: PubSubPeer, msg: var RPCMsg) =
|
|
|
|
# trigger hooks
|
|
|
|
if not(isNil(p.observers)) and p.observers[].len > 0:
|
|
|
|
for obs in p.observers[]:
|
2020-06-24 15:08:44 +00:00
|
|
|
if not(isNil(obs)): # TODO: should never be nil, but...
|
|
|
|
obs.onSend(p, msg)
|
2020-05-29 16:46:27 +00:00
|
|
|
|
2019-12-17 05:24:03 +00:00
|
|
|
proc handle*(p: PubSubPeer, conn: Connection) {.async.} =
|
2020-07-16 19:26:57 +00:00
|
|
|
logScope:
|
|
|
|
peer = p.id
|
2020-08-12 00:05:49 +00:00
|
|
|
|
2020-07-16 19:26:57 +00:00
|
|
|
debug "starting pubsub read loop for peer", closed = conn.closed
|
2019-09-10 02:15:52 +00:00
|
|
|
try:
|
2020-05-25 14:33:24 +00:00
|
|
|
try:
|
2020-08-03 05:20:11 +00:00
|
|
|
while not conn.atEof:
|
2020-07-16 19:26:57 +00:00
|
|
|
trace "waiting for data", closed = conn.closed
|
2020-08-12 00:05:49 +00:00
|
|
|
let data = await conn.readLp(64 * 1024)
|
2020-05-25 14:33:24 +00:00
|
|
|
let digest = $(sha256.digest(data))
|
2020-07-16 19:26:57 +00:00
|
|
|
trace "read data from peer", data = data.shortLog
|
2020-05-25 14:33:24 +00:00
|
|
|
if digest in p.recvdRpcCache:
|
2020-08-04 23:27:59 +00:00
|
|
|
when defined(libp2p_expensive_metrics):
|
|
|
|
libp2p_pubsub_skipped_received_messages.inc(labelValues = [p.id])
|
2020-07-16 19:26:57 +00:00
|
|
|
trace "message already received, skipping"
|
2020-05-25 14:33:24 +00:00
|
|
|
continue
|
|
|
|
|
2020-07-15 08:25:39 +00:00
|
|
|
var rmsg = decodeRpcMsg(data)
|
|
|
|
if rmsg.isErr():
|
2020-07-16 19:26:57 +00:00
|
|
|
notice "failed to decode msg from peer"
|
2020-07-15 08:25:39 +00:00
|
|
|
break
|
|
|
|
|
|
|
|
var msg = rmsg.get()
|
|
|
|
|
2020-07-16 19:26:57 +00:00
|
|
|
trace "decoded msg from peer", msg = msg.shortLog
|
2020-05-25 14:33:24 +00:00
|
|
|
# trigger hooks
|
2020-06-02 23:53:38 +00:00
|
|
|
p.recvObservers(msg)
|
|
|
|
|
2020-08-04 23:27:59 +00:00
|
|
|
when defined(libp2p_expensive_metrics):
|
|
|
|
for m in msg.messages:
|
|
|
|
for t in m.topicIDs:
|
|
|
|
# metrics
|
|
|
|
libp2p_pubsub_received_messages.inc(labelValues = [p.id, t])
|
2020-06-11 18:09:34 +00:00
|
|
|
|
2020-05-25 14:33:24 +00:00
|
|
|
await p.handler(p, @[msg])
|
|
|
|
p.recvdRpcCache.put(digest)
|
|
|
|
finally:
|
2020-07-16 19:26:57 +00:00
|
|
|
debug "exiting pubsub peer read loop"
|
2020-05-25 14:33:24 +00:00
|
|
|
await conn.close()
|
|
|
|
|
remove send lock (#334)
* remove send lock
When mplex receives data it will block until a reader has processed the
data. Thus, when a large message is received, such as a gossipsub
subscription table, all of mplex will be blocked until all reading is
finished.
However, if at the same time a `dial` to establish a gossipsub send
connection is ongoing, that `dial` will be blocked because mplex is no
longer reading data - specifically, it might indeed be the connection
that's processing the previous data that is waiting for a send
connection.
There are other problems with the current code:
* If an exception is raised, it is not necessarily raised for the same
connection as `p.sendConn`, so resetting `p.sendConn` in the exception
handling is wrong
* `p.isConnected` is checked before taking the lock - thus, if it
returns false, a new dial will be started. If a new task enters `send`
before dial is finished, it will also determine `p.isConnected` is
false, then get stuck on the lock - when the previous task finishes and
releases the lock, the new task will _also_ dial and thus reset
`p.sendConn` causing a leak.
* prefer existing connection
simplifies flow
2020-08-17 10:38:27 +00:00
|
|
|
if p.sendConn == conn:
|
|
|
|
p.sendConn = nil
|
|
|
|
|
2020-07-08 00:33:05 +00:00
|
|
|
except CancelledError as exc:
|
|
|
|
raise exc
|
2019-12-06 02:16:18 +00:00
|
|
|
except CatchableError as exc:
|
2020-03-27 14:25:52 +00:00
|
|
|
trace "Exception occurred in PubSubPeer.handle", exc = exc.msg
|
2019-09-10 02:15:52 +00:00
|
|
|
|
remove send lock (#334)
* remove send lock
When mplex receives data it will block until a reader has processed the
data. Thus, when a large message is received, such as a gossipsub
subscription table, all of mplex will be blocked until all reading is
finished.
However, if at the same time a `dial` to establish a gossipsub send
connection is ongoing, that `dial` will be blocked because mplex is no
longer reading data - specifically, it might indeed be the connection
that's processing the previous data that is waiting for a send
connection.
There are other problems with the current code:
* If an exception is raised, it is not necessarily raised for the same
connection as `p.sendConn`, so resetting `p.sendConn` in the exception
handling is wrong
* `p.isConnected` is checked before taking the lock - thus, if it
returns false, a new dial will be started. If a new task enters `send`
before dial is finished, it will also determine `p.isConnected` is
false, then get stuck on the lock - when the previous task finishes and
releases the lock, the new task will _also_ dial and thus reset
`p.sendConn` causing a leak.
* prefer existing connection
simplifies flow
2020-08-17 10:38:27 +00:00
|
|
|
proc getSendConn(p: PubSubPeer): Future[Connection] {.async.} =
|
|
|
|
# get a cached send connection or create a new one
|
|
|
|
block: # check if there's an existing connection that can be reused
|
|
|
|
let current = p.sendConn
|
|
|
|
|
|
|
|
if not current.isNil:
|
|
|
|
if not (current.closed() or current.atEof):
|
|
|
|
# The existing send connection looks like it might work - reuse it
|
|
|
|
return current
|
|
|
|
|
|
|
|
# Send connection is set but broken - get rid of it
|
|
|
|
p.sendConn = nil
|
|
|
|
|
|
|
|
# Careful, p.sendConn might change after here!
|
|
|
|
await current.close() # TODO this might be unnecessary
|
|
|
|
|
|
|
|
# Grab a new send connection
|
|
|
|
let newConn = await p.switch.dial(p.peerId, p.codec) # ...and here
|
|
|
|
if newConn == nil:
|
|
|
|
return p.sendConn # A concurrent attempt perhaps succeeded?
|
|
|
|
|
|
|
|
# Because of the awaits above, a concurrent `getSendConn` call might have
|
|
|
|
# set up a send connection already. We cannot take a lock here because
|
|
|
|
# it might block the reading of data from mplex which will cause its
|
|
|
|
# backpressure handling to stop reading from the socket and thus prevent the
|
|
|
|
# channel negotiation from finishing
|
|
|
|
if p.sendConn != nil and not(p.sendConn.closed or p.sendConn.atEof):
|
|
|
|
let current = p.sendConn
|
|
|
|
# Either the new or the old connection could potentially be closed - it's
|
|
|
|
# slightly easier to sequence the closing of the new connection because the
|
|
|
|
# old one might still be in use.
|
|
|
|
await newConn.close()
|
|
|
|
return current
|
|
|
|
|
|
|
|
p.sendConn = newConn
|
|
|
|
asyncCheck p.handle(newConn) # start a read loop on the new connection
|
|
|
|
|
|
|
|
return newConn
|
|
|
|
|
2020-08-03 05:20:11 +00:00
|
|
|
proc send*(
|
|
|
|
p: PubSubPeer,
|
|
|
|
msg: RPCMsg,
|
|
|
|
timeout: Duration = DefaultSendTimeout) {.async.} =
|
2020-08-12 00:05:49 +00:00
|
|
|
|
|
|
|
doAssert(not isNil(p), "pubsubpeer nil!")
|
|
|
|
|
2020-07-08 00:33:05 +00:00
|
|
|
logScope:
|
|
|
|
peer = p.id
|
2020-08-07 22:46:00 +00:00
|
|
|
rpcMsg = shortLog(msg)
|
2020-07-08 00:33:05 +00:00
|
|
|
|
2020-07-16 10:06:57 +00:00
|
|
|
trace "sending msg to peer"
|
2019-12-10 20:50:35 +00:00
|
|
|
|
2020-07-16 10:06:57 +00:00
|
|
|
# trigger send hooks
|
|
|
|
var mm = msg # hooks can modify the message
|
|
|
|
p.sendObservers(mm)
|
2020-05-29 16:46:27 +00:00
|
|
|
|
2020-07-16 10:06:57 +00:00
|
|
|
let encoded = encodeRpcMsg(mm)
|
|
|
|
if encoded.len <= 0:
|
2020-08-03 05:20:11 +00:00
|
|
|
info "empty message, skipping"
|
2020-07-16 10:06:57 +00:00
|
|
|
return
|
2019-12-06 02:16:18 +00:00
|
|
|
|
2020-07-16 10:06:57 +00:00
|
|
|
logScope:
|
|
|
|
encoded = shortLog(encoded)
|
|
|
|
|
|
|
|
let digest = $(sha256.digest(encoded))
|
|
|
|
if digest in p.sentRpcCache:
|
|
|
|
trace "message already sent to peer, skipping"
|
2020-08-04 23:27:59 +00:00
|
|
|
when defined(libp2p_expensive_metrics):
|
|
|
|
libp2p_pubsub_skipped_sent_messages.inc(labelValues = [p.id])
|
2020-07-16 10:06:57 +00:00
|
|
|
return
|
2020-05-23 16:50:29 +00:00
|
|
|
|
remove send lock (#334)
* remove send lock
When mplex receives data it will block until a reader has processed the
data. Thus, when a large message is received, such as a gossipsub
subscription table, all of mplex will be blocked until all reading is
finished.
However, if at the same time a `dial` to establish a gossipsub send
connection is ongoing, that `dial` will be blocked because mplex is no
longer reading data - specifically, it might indeed be the connection
that's processing the previous data that is waiting for a send
connection.
There are other problems with the current code:
* If an exception is raised, it is not necessarily raised for the same
connection as `p.sendConn`, so resetting `p.sendConn` in the exception
handling is wrong
* `p.isConnected` is checked before taking the lock - thus, if it
returns false, a new dial will be started. If a new task enters `send`
before dial is finished, it will also determine `p.isConnected` is
false, then get stuck on the lock - when the previous task finishes and
releases the lock, the new task will _also_ dial and thus reset
`p.sendConn` causing a leak.
* prefer existing connection
simplifies flow
2020-08-17 10:38:27 +00:00
|
|
|
var conn: Connection
|
2020-08-12 00:05:49 +00:00
|
|
|
try:
|
2020-07-16 10:06:57 +00:00
|
|
|
trace "about to send message"
|
remove send lock (#334)
* remove send lock
When mplex receives data it will block until a reader has processed the
data. Thus, when a large message is received, such as a gossipsub
subscription table, all of mplex will be blocked until all reading is
finished.
However, if at the same time a `dial` to establish a gossipsub send
connection is ongoing, that `dial` will be blocked because mplex is no
longer reading data - specifically, it might indeed be the connection
that's processing the previous data that is waiting for a send
connection.
There are other problems with the current code:
* If an exception is raised, it is not necessarily raised for the same
connection as `p.sendConn`, so resetting `p.sendConn` in the exception
handling is wrong
* `p.isConnected` is checked before taking the lock - thus, if it
returns false, a new dial will be started. If a new task enters `send`
before dial is finished, it will also determine `p.isConnected` is
false, then get stuck on the lock - when the previous task finishes and
releases the lock, the new task will _also_ dial and thus reset
`p.sendConn` causing a leak.
* prefer existing connection
simplifies flow
2020-08-17 10:38:27 +00:00
|
|
|
conn = await p.getSendConn()
|
2020-08-12 00:05:49 +00:00
|
|
|
|
remove send lock (#334)
* remove send lock
When mplex receives data it will block until a reader has processed the
data. Thus, when a large message is received, such as a gossipsub
subscription table, all of mplex will be blocked until all reading is
finished.
However, if at the same time a `dial` to establish a gossipsub send
connection is ongoing, that `dial` will be blocked because mplex is no
longer reading data - specifically, it might indeed be the connection
that's processing the previous data that is waiting for a send
connection.
There are other problems with the current code:
* If an exception is raised, it is not necessarily raised for the same
connection as `p.sendConn`, so resetting `p.sendConn` in the exception
handling is wrong
* `p.isConnected` is checked before taking the lock - thus, if it
returns false, a new dial will be started. If a new task enters `send`
before dial is finished, it will also determine `p.isConnected` is
false, then get stuck on the lock - when the previous task finishes and
releases the lock, the new task will _also_ dial and thus reset
`p.sendConn` causing a leak.
* prefer existing connection
simplifies flow
2020-08-17 10:38:27 +00:00
|
|
|
if conn == nil:
|
|
|
|
debug "Couldn't get send connection, dropping message"
|
|
|
|
return
|
2020-08-12 00:05:49 +00:00
|
|
|
trace "sending encoded msgs to peer"
|
remove send lock (#334)
* remove send lock
When mplex receives data it will block until a reader has processed the
data. Thus, when a large message is received, such as a gossipsub
subscription table, all of mplex will be blocked until all reading is
finished.
However, if at the same time a `dial` to establish a gossipsub send
connection is ongoing, that `dial` will be blocked because mplex is no
longer reading data - specifically, it might indeed be the connection
that's processing the previous data that is waiting for a send
connection.
There are other problems with the current code:
* If an exception is raised, it is not necessarily raised for the same
connection as `p.sendConn`, so resetting `p.sendConn` in the exception
handling is wrong
* `p.isConnected` is checked before taking the lock - thus, if it
returns false, a new dial will be started. If a new task enters `send`
before dial is finished, it will also determine `p.isConnected` is
false, then get stuck on the lock - when the previous task finishes and
releases the lock, the new task will _also_ dial and thus reset
`p.sendConn` causing a leak.
* prefer existing connection
simplifies flow
2020-08-17 10:38:27 +00:00
|
|
|
await conn.writeLp(encoded).wait(timeout)
|
|
|
|
|
2020-08-12 00:05:49 +00:00
|
|
|
p.sentRpcCache.put(digest)
|
|
|
|
trace "sent pubsub message to remote"
|
2020-08-03 05:20:11 +00:00
|
|
|
|
2020-08-12 00:05:49 +00:00
|
|
|
when defined(libp2p_expensive_metrics):
|
|
|
|
for x in mm.messages:
|
|
|
|
for t in x.topicIDs:
|
|
|
|
# metrics
|
|
|
|
libp2p_pubsub_sent_messages.inc(labelValues = [p.id, t])
|
2020-07-16 10:06:57 +00:00
|
|
|
|
2020-07-09 17:16:46 +00:00
|
|
|
except CatchableError as exc:
|
2020-07-16 10:06:57 +00:00
|
|
|
trace "unable to send to remote", exc = exc.msg
|
remove send lock (#334)
* remove send lock
When mplex receives data it will block until a reader has processed the
data. Thus, when a large message is received, such as a gossipsub
subscription table, all of mplex will be blocked until all reading is
finished.
However, if at the same time a `dial` to establish a gossipsub send
connection is ongoing, that `dial` will be blocked because mplex is no
longer reading data - specifically, it might indeed be the connection
that's processing the previous data that is waiting for a send
connection.
There are other problems with the current code:
* If an exception is raised, it is not necessarily raised for the same
connection as `p.sendConn`, so resetting `p.sendConn` in the exception
handling is wrong
* `p.isConnected` is checked before taking the lock - thus, if it
returns false, a new dial will be started. If a new task enters `send`
before dial is finished, it will also determine `p.isConnected` is
false, then get stuck on the lock - when the previous task finishes and
releases the lock, the new task will _also_ dial and thus reset
`p.sendConn` causing a leak.
* prefer existing connection
simplifies flow
2020-08-17 10:38:27 +00:00
|
|
|
# Next time sendConn is used, it will be have its close flag set and thus
|
|
|
|
# will be recycled
|
|
|
|
if not isNil(conn):
|
|
|
|
await conn.close()
|
2020-07-16 10:06:57 +00:00
|
|
|
|
2020-08-03 05:20:11 +00:00
|
|
|
raise exc
|
2019-12-06 02:16:18 +00:00
|
|
|
|
2020-07-08 00:33:05 +00:00
|
|
|
proc `$`*(p: PubSubPeer): string =
|
|
|
|
p.id
|
|
|
|
|
2020-08-12 00:05:49 +00:00
|
|
|
proc newPubSubPeer*(peerId: PeerID,
|
|
|
|
switch: Switch,
|
|
|
|
codec: string): PubSubPeer =
|
2019-09-10 02:15:52 +00:00
|
|
|
new result
|
2020-08-12 00:05:49 +00:00
|
|
|
result.switch = switch
|
|
|
|
result.codec = codec
|
|
|
|
result.peerId = peerId
|
2019-12-06 02:16:18 +00:00
|
|
|
result.sentRpcCache = newTimedCache[string](2.minutes)
|
|
|
|
result.recvdRpcCache = newTimedCache[string](2.minutes)
|