mirror of https://github.com/vacp2p/nim-libp2p.git
Gossipsub interop (#189)
* interop fixes * add custom messageid provider and fix seqno * use ECDSA for speed * adding messageid tests * breakout from publish loop * addressing review comments * remove unneded var * dont stop broadcasting on failed peers
This commit is contained in:
parent
536555138c
commit
7b6e1c0688
|
@ -33,8 +33,8 @@ type
|
|||
method subscribeTopic*(f: FloodSub,
|
||||
topic: string,
|
||||
subscribe: bool,
|
||||
peerId: string) {.gcsafe.} =
|
||||
procCall PubSub(f).subscribeTopic(topic, subscribe, peerId)
|
||||
peerId: string) {.gcsafe, async.} =
|
||||
await procCall PubSub(f).subscribeTopic(topic, subscribe, peerId)
|
||||
|
||||
if topic notin f.floodsub:
|
||||
f.floodsub[topic] = initHashSet[string]()
|
||||
|
|
|
@ -53,26 +53,8 @@ type
|
|||
gossip*: Table[string, seq[ControlIHave]] # pending gossip
|
||||
control*: Table[string, ControlMessage] # pending control messages
|
||||
mcache*: MCache # messages cache
|
||||
heartbeatCancel*: Future[void] # cancelation future for heartbeat interval
|
||||
heartbeatLock: AsyncLock # hearbeat lock to prevent two concecutive concurent hearbeats
|
||||
|
||||
# TODO: This belong in chronos, temporary left here until chronos is updated
|
||||
proc addInterval(every: Duration, cb: CallbackFunc,
|
||||
udata: pointer = nil): Future[void] =
|
||||
## Arrange the callback ``cb`` to be called on every ``Duration`` window
|
||||
|
||||
var retFuture = newFuture[void]("gossipsub.addInterval(Duration)")
|
||||
proc interval(arg: pointer = nil) {.gcsafe.}
|
||||
proc scheduleNext() =
|
||||
if not retFuture.finished():
|
||||
addTimer(Moment.fromNow(every), interval)
|
||||
|
||||
proc interval(arg: pointer = nil) {.gcsafe.} =
|
||||
cb(udata)
|
||||
scheduleNext()
|
||||
|
||||
scheduleNext()
|
||||
return retFuture
|
||||
heartbeatCancel*: Future[void] # cancellation future for heartbeat interval
|
||||
heartbeatLock: AsyncLock # heartbeat lock to prevent two consecutive concurrent heartbeats
|
||||
|
||||
method init(g: GossipSub) =
|
||||
proc handler(conn: Connection, proto: string) {.async.} =
|
||||
|
@ -145,9 +127,9 @@ proc rebalanceMesh(g: GossipSub, topic: string) {.async.} =
|
|||
# send a graft message to the peer
|
||||
await p.sendPrune(@[topic])
|
||||
|
||||
trace "mesh balanced, got peers", peers = g.mesh[topic].len
|
||||
trace "mesh balanced, got peers", peers = g.mesh[topic].len, topicId = topic
|
||||
except CatchableError as exc:
|
||||
trace "exception occured rebalancing mes", exc = exc.msg
|
||||
trace "exception occurred re-balancing mesh", exc = exc.msg
|
||||
|
||||
proc dropFanoutPeers(g: GossipSub) {.async.} =
|
||||
# drop peers that we haven't published to in
|
||||
|
@ -199,25 +181,28 @@ proc getGossipPeers(g: GossipSub): Table[string, ControlMessage] {.gcsafe.} =
|
|||
result[id].ihave.add(ihave)
|
||||
|
||||
proc heartbeat(g: GossipSub) {.async.} =
|
||||
try:
|
||||
await g.heartbeatLock.acquire()
|
||||
trace "running heartbeat"
|
||||
while true:
|
||||
try:
|
||||
await g.heartbeatLock.acquire()
|
||||
trace "running heartbeat"
|
||||
|
||||
await sleepAsync(GossipSubHeartbeatInitialDelay)
|
||||
for t in g.mesh.keys:
|
||||
await g.rebalanceMesh(t)
|
||||
|
||||
for t in g.mesh.keys:
|
||||
await g.rebalanceMesh(t)
|
||||
await g.dropFanoutPeers()
|
||||
let peers = g.getGossipPeers()
|
||||
var sent: seq[Future[void]]
|
||||
for peer in peers.keys:
|
||||
sent &= g.peers[peer].send(@[RPCMsg(control: some(peers[peer]))])
|
||||
checkFutures(await allFinished(sent))
|
||||
|
||||
await g.dropFanoutPeers()
|
||||
let peers = g.getGossipPeers()
|
||||
for peer in peers.keys:
|
||||
await g.peers[peer].send(@[RPCMsg(control: some(peers[peer]))])
|
||||
g.mcache.shift() # shift the cache
|
||||
except CatchableError as exc:
|
||||
trace "exception ocurred in gossipsub heartbeat", exc = exc.msg
|
||||
finally:
|
||||
g.heartbeatLock.release()
|
||||
|
||||
g.mcache.shift() # shift the cache
|
||||
except CatchableError as exc:
|
||||
trace "exception ocurred in gossipsub heartbeat", exc = exc.msg
|
||||
finally:
|
||||
g.heartbeatLock.release()
|
||||
await sleepAsync(1.seconds)
|
||||
|
||||
method handleDisconnect(g: GossipSub, peer: PubSubPeer) {.async.} =
|
||||
## handle peer disconnects
|
||||
|
@ -242,21 +227,24 @@ method subscribeToPeer*(p: GossipSub,
|
|||
method subscribeTopic*(g: GossipSub,
|
||||
topic: string,
|
||||
subscribe: bool,
|
||||
peerId: string) {.gcsafe.} =
|
||||
procCall PubSub(g).subscribeTopic(topic, subscribe, peerId)
|
||||
peerId: string) {.gcsafe, async.} =
|
||||
await procCall PubSub(g).subscribeTopic(topic, subscribe, peerId)
|
||||
|
||||
if topic notin g.gossipsub:
|
||||
g.gossipsub[topic] = initHashSet[string]()
|
||||
|
||||
if subscribe:
|
||||
trace "adding subscription for topic", peer = peerId, name = topic
|
||||
# subscribe the peer to the topic
|
||||
# subscribe remote peer to the topic
|
||||
g.gossipsub[topic].incl(peerId)
|
||||
else:
|
||||
trace "removing subscription for topic", peer = peerId, name = topic
|
||||
# unsubscribe the peer from the topic
|
||||
# unsubscribe remote peer from the topic
|
||||
g.gossipsub[topic].excl(peerId)
|
||||
|
||||
if topic in g.topics:
|
||||
await g.rebalanceMesh(topic)
|
||||
|
||||
proc handleGraft(g: GossipSub,
|
||||
peer: PubSubPeer,
|
||||
grafts: seq[ControlGraft],
|
||||
|
@ -410,20 +398,27 @@ method publish*(g: GossipSub,
|
|||
topic: string,
|
||||
data: seq[byte]) {.async.} =
|
||||
await procCall PubSub(g).publish(topic, data)
|
||||
|
||||
trace "about to publish message on topic", name = topic,
|
||||
data = data.shortLog
|
||||
|
||||
var peers: HashSet[string]
|
||||
if data.len > 0 and topic.len > 0:
|
||||
var peers: HashSet[string]
|
||||
if topic in g.topics: # if we're subscribed to the topic attempt to build a mesh
|
||||
await g.rebalanceMesh(topic)
|
||||
peers = g.mesh[topic]
|
||||
else: # send to fanout peers
|
||||
await g.replenishFanout(topic)
|
||||
if topic in g.fanout:
|
||||
peers = g.fanout[topic]
|
||||
# set the fanout expiery time
|
||||
g.lastFanoutPubSub[topic] = Moment.fromNow(GossipSubFanoutTTL)
|
||||
for _ in 0..<5: # try to get peers up to 5 times
|
||||
if peers.len > 0:
|
||||
break
|
||||
|
||||
if topic in g.topics: # if we're subscribed to the topic attempt to build a mesh
|
||||
await g.rebalanceMesh(topic)
|
||||
peers = g.mesh[topic]
|
||||
else: # send to fanout peers
|
||||
await g.replenishFanout(topic)
|
||||
if topic in g.fanout:
|
||||
peers = g.fanout[topic]
|
||||
# set the fanout expiry time
|
||||
g.lastFanoutPubSub[topic] = Moment.fromNow(GossipSubFanoutTTL)
|
||||
|
||||
# wait a second between tries
|
||||
await sleepAsync(1.seconds)
|
||||
|
||||
let msg = newMessage(g.peerInfo, data, topic, g.sign)
|
||||
var sent: seq[Future[void]]
|
||||
|
@ -434,6 +429,7 @@ method publish*(g: GossipSub,
|
|||
trace "publishing on topic", name = topic
|
||||
g.mcache.put(msg)
|
||||
sent.add(g.peers[p].send(@[RPCMsg(messages: @[msg])]))
|
||||
|
||||
sent = await allFinished(sent)
|
||||
checkFutures(sent)
|
||||
|
||||
|
@ -442,10 +438,7 @@ method start*(g: GossipSub) {.async.} =
|
|||
## start long running/repeating procedures
|
||||
|
||||
# setup the heartbeat interval
|
||||
g.heartbeatCancel = addInterval(GossipSubHeartbeatInterval,
|
||||
proc (arg: pointer = nil)
|
||||
{.gcsafe, locks: 0.} =
|
||||
asyncCheck g.heartbeat)
|
||||
g.heartbeatCancel = g.heartbeat()
|
||||
|
||||
method stop*(g: GossipSub) {.async.} =
|
||||
## stopt pubsub
|
||||
|
|
|
@ -66,7 +66,7 @@ proc sendSubs*(p: PubSub,
|
|||
method subscribeTopic*(p: PubSub,
|
||||
topic: string,
|
||||
subscribe: bool,
|
||||
peerId: string) {.base.} =
|
||||
peerId: string) {.base, async.} =
|
||||
discard
|
||||
|
||||
method rpcHandler*(p: PubSub,
|
||||
|
@ -79,7 +79,8 @@ method rpcHandler*(p: PubSub,
|
|||
trace "processing messages", msg = m.shortLog
|
||||
if m.subscriptions.len > 0: # if there are any subscriptions
|
||||
for s in m.subscriptions: # subscribe/unsubscribe the peer for each topic
|
||||
p.subscribeTopic(s.topic, s.subscribe, peer.id)
|
||||
trace "about to subscribe to topic", topicId = s.topic
|
||||
await p.subscribeTopic(s.topic, s.subscribe, peer.id)
|
||||
|
||||
method handleDisconnect*(p: PubSub, peer: PubSubPeer) {.async, base.} =
|
||||
## handle peer disconnects
|
||||
|
@ -110,7 +111,7 @@ proc getPeer(p: PubSub,
|
|||
peer.observers = p.observers
|
||||
result = peer
|
||||
|
||||
proc internalClenaup(p: PubSub, conn: Connection) {.async.} =
|
||||
proc internalCleanup(p: PubSub, conn: Connection) {.async.} =
|
||||
# handle connection close
|
||||
if conn.closed:
|
||||
return
|
||||
|
@ -151,7 +152,7 @@ method handleConn*(p: PubSub,
|
|||
peer.handler = handler
|
||||
await peer.handle(conn) # spawn peer read loop
|
||||
trace "pubsub peer handler ended, cleaning up"
|
||||
await p.internalClenaup(conn)
|
||||
await p.internalCleanup(conn)
|
||||
|
||||
method subscribeToPeer*(p: PubSub,
|
||||
conn: Connection) {.base, async.} =
|
||||
|
@ -160,7 +161,7 @@ method subscribeToPeer*(p: PubSub,
|
|||
if not peer.isConnected:
|
||||
peer.conn = conn
|
||||
|
||||
asyncCheck p.internalClenaup(conn)
|
||||
asyncCheck p.internalCleanup(conn)
|
||||
|
||||
method unsubscribe*(p: PubSub,
|
||||
topics: seq[TopicPair]) {.base, async.} =
|
||||
|
@ -199,7 +200,7 @@ method subscribe*(p: PubSub,
|
|||
method publish*(p: PubSub,
|
||||
topic: string,
|
||||
data: seq[byte]) {.base, async.} =
|
||||
# TODO: Should return bool indicating success/failure
|
||||
# TODO: Should throw indicating success/failure
|
||||
## publish to a ``topic``
|
||||
if p.triggerSelf and topic in p.topics:
|
||||
for h in p.topics[topic].handler:
|
||||
|
@ -215,7 +216,7 @@ method publish*(p: PubSub,
|
|||
debug "Could not write to pubsub connection", msg = exc.msg
|
||||
|
||||
method initPubSub*(p: PubSub) {.base.} =
|
||||
## perform pubsub initializaion
|
||||
## perform pubsub initialization
|
||||
p.observers = new(seq[PubSubObserver])
|
||||
|
||||
method start*(p: PubSub) {.async, base.} =
|
||||
|
|
|
@ -21,9 +21,17 @@ logScope:
|
|||
|
||||
const PubSubPrefix = "libp2p-pubsub:"
|
||||
|
||||
proc msgId*(m: Message): string =
|
||||
proc msgIdProvider(m: Message): string =
|
||||
## default msg id provider
|
||||
m.seqno.toHex() & PeerID.init(m.fromPeer).pretty
|
||||
|
||||
template msgId*(m: Message): string =
|
||||
## calls the ``msgIdProvider`` from
|
||||
## the instantiation scope
|
||||
##
|
||||
mixin msgIdProvider
|
||||
m.msgIdProvider()
|
||||
|
||||
proc fromPeerId*(m: Message): PeerId =
|
||||
PeerID.init(m.fromPeer)
|
||||
|
||||
|
@ -55,15 +63,16 @@ proc newMessage*(p: PeerInfo,
|
|||
data: seq[byte],
|
||||
topic: string,
|
||||
sign: bool = true): Message {.gcsafe.} =
|
||||
var seqno: seq[byte] = newSeq[byte](20)
|
||||
if p.publicKey.isSome and randomBytes(addr seqno[0], 20) > 0:
|
||||
var key: seq[byte] = p.publicKey.get().getBytes().tryGet()
|
||||
var seqno: seq[byte] = newSeq[byte](8)
|
||||
if randomBytes(addr seqno[0], 8) > 0:
|
||||
if p.publicKey.isSome:
|
||||
var key: seq[byte] = p.publicKey.get().getBytes().tryGet()
|
||||
|
||||
result = Message(fromPeer: p.peerId.getBytes(),
|
||||
data: data,
|
||||
seqno: seqno,
|
||||
topicIDs: @[topic])
|
||||
if sign:
|
||||
result = result.sign(p)
|
||||
result = Message(fromPeer: p.peerId.getBytes(),
|
||||
data: data,
|
||||
seqno: seqno,
|
||||
topicIDs: @[topic])
|
||||
if sign:
|
||||
result = result.sign(p)
|
||||
|
||||
result.key = key
|
||||
result.key = key
|
||||
|
|
|
@ -214,13 +214,12 @@ proc encodeRpcMsg*(msg: RPCMsg): ProtoBuffer {.gcsafe.} =
|
|||
trace "encoding msg: ", msg = msg.shortLog
|
||||
|
||||
if msg.subscriptions.len > 0:
|
||||
var subs = initProtoBuffer()
|
||||
for s in msg.subscriptions:
|
||||
var subs = initProtoBuffer()
|
||||
encodeSubs(s, subs)
|
||||
|
||||
# write subscriptions to protobuf
|
||||
subs.finish()
|
||||
result.write(initProtoField(1, subs))
|
||||
# write subscriptions to protobuf
|
||||
subs.finish()
|
||||
result.write(initProtoField(1, subs))
|
||||
|
||||
if msg.messages.len > 0:
|
||||
var messages = initProtoBuffer()
|
||||
|
@ -255,9 +254,9 @@ proc decodeRpcMsg*(msg: seq[byte]): RPCMsg {.gcsafe.} =
|
|||
trace "no submessage found in RPC msg"
|
||||
break
|
||||
of 1:
|
||||
result.subscriptions = pb.decodeSubs()
|
||||
result.subscriptions &= pb.decodeSubs()
|
||||
of 2:
|
||||
result.messages = pb.decodeMessages()
|
||||
result.messages &= pb.decodeMessages()
|
||||
of 3:
|
||||
result.control = pb.decodeControl()
|
||||
else:
|
||||
|
|
|
@ -303,7 +303,7 @@ proc mount*[T: LPProtocol](s: Switch, proto: T) {.gcsafe.} =
|
|||
s.ms.addHandler(proto.codec, proto)
|
||||
|
||||
proc start*(s: Switch): Future[seq[Future[void]]] {.async, gcsafe.} =
|
||||
trace "starting switch"
|
||||
trace "starting switch for peer", peerInfo = $s.peerInfo
|
||||
|
||||
proc handle(conn: Connection): Future[void] {.async, closure, gcsafe.} =
|
||||
try:
|
||||
|
|
|
@ -38,7 +38,7 @@ proc waitSub(sender, receiver: auto; key: string) {.async, gcsafe.} =
|
|||
not fsub.mesh[key].contains(receiver.peerInfo.id)) and
|
||||
(not fsub.fanout.hasKey(key) or
|
||||
not fsub.fanout[key].contains(receiver.peerInfo.id)):
|
||||
trace "waitSub sleeping...", peers=fsub.gossipsub[key]
|
||||
trace "waitSub sleeping..."
|
||||
await sleepAsync(100.millis)
|
||||
dec ceil
|
||||
doAssert(ceil > 0, "waitSub timeout!")
|
||||
|
@ -111,6 +111,7 @@ suite "GossipSub":
|
|||
await nodes[0].publish("foobar", cast[seq[byte]]("Hello!"))
|
||||
|
||||
result = await validatorFut
|
||||
|
||||
await allFuturesThrowing(nodes[0].stop(), nodes[1].stop())
|
||||
await allFuturesThrowing(awaiters)
|
||||
|
||||
|
@ -345,7 +346,6 @@ suite "GossipSub":
|
|||
waitSub(nodes[0], dialer, "foobar")))
|
||||
|
||||
await allFuturesThrowing(subs)
|
||||
|
||||
await wait(nodes[0].publish("foobar",
|
||||
cast[seq[byte]]("from node " &
|
||||
nodes[1].peerInfo.id)),
|
||||
|
|
|
@ -11,7 +11,7 @@ import ../../libp2p/[peer,
|
|||
suite "MCache":
|
||||
test "put/get":
|
||||
var mCache = newMCache(3, 5)
|
||||
var msg = Message(fromPeer: PeerID.init(PrivateKey.random(RSA).get()).data,
|
||||
var msg = Message(fromPeer: PeerID.init(PrivateKey.random(ECDSA).get()).data,
|
||||
seqno: cast[seq[byte]]("12345"))
|
||||
mCache.put(msg)
|
||||
check mCache.get(msg.msgId).isSome and mCache.get(msg.msgId).get() == msg
|
||||
|
@ -20,13 +20,13 @@ suite "MCache":
|
|||
var mCache = newMCache(3, 5)
|
||||
|
||||
for i in 0..<3:
|
||||
var msg = Message(fromPeer: PeerID.init(PrivateKey.random(RSA).get()).data,
|
||||
var msg = Message(fromPeer: PeerID.init(PrivateKey.random(ECDSA).get()).data,
|
||||
seqno: cast[seq[byte]]("12345"),
|
||||
topicIDs: @["foo"])
|
||||
mCache.put(msg)
|
||||
|
||||
for i in 0..<5:
|
||||
var msg = Message(fromPeer: PeerID.init(PrivateKey.random(RSA).get()).data,
|
||||
var msg = Message(fromPeer: PeerID.init(PrivateKey.random(ECDSA).get()).data,
|
||||
seqno: cast[seq[byte]]("12345"),
|
||||
topicIDs: @["bar"])
|
||||
mCache.put(msg)
|
||||
|
@ -41,7 +41,7 @@ suite "MCache":
|
|||
var mCache = newMCache(1, 5)
|
||||
|
||||
for i in 0..<3:
|
||||
var msg = Message(fromPeer: PeerID.init(PrivateKey.random(RSA).get()).data,
|
||||
var msg = Message(fromPeer: PeerID.init(PrivateKey.random(ECDSA).get()).data,
|
||||
seqno: cast[seq[byte]]("12345"),
|
||||
topicIDs: @["foo"])
|
||||
mCache.put(msg)
|
||||
|
@ -50,7 +50,7 @@ suite "MCache":
|
|||
check mCache.window("foo").len == 0
|
||||
|
||||
for i in 0..<3:
|
||||
var msg = Message(fromPeer: PeerID.init(PrivateKey.random(RSA).get()).data,
|
||||
var msg = Message(fromPeer: PeerID.init(PrivateKey.random(ECDSA).get()).data,
|
||||
seqno: cast[seq[byte]]("12345"),
|
||||
topicIDs: @["bar"])
|
||||
mCache.put(msg)
|
||||
|
@ -59,7 +59,7 @@ suite "MCache":
|
|||
check mCache.window("bar").len == 0
|
||||
|
||||
for i in 0..<3:
|
||||
var msg = Message(fromPeer: PeerID.init(PrivateKey.random(RSA).get()).data,
|
||||
var msg = Message(fromPeer: PeerID.init(PrivateKey.random(ECDSA).get()).data,
|
||||
seqno: cast[seq[byte]]("12345"),
|
||||
topicIDs: @["baz"])
|
||||
mCache.put(msg)
|
||||
|
@ -71,19 +71,19 @@ suite "MCache":
|
|||
var mCache = newMCache(1, 5)
|
||||
|
||||
for i in 0..<3:
|
||||
var msg = Message(fromPeer: PeerID.init(PrivateKey.random(RSA).get()).data,
|
||||
var msg = Message(fromPeer: PeerID.init(PrivateKey.random(ECDSA).get()).data,
|
||||
seqno: cast[seq[byte]]("12345"),
|
||||
topicIDs: @["foo"])
|
||||
mCache.put(msg)
|
||||
|
||||
for i in 0..<3:
|
||||
var msg = Message(fromPeer: PeerID.init(PrivateKey.random(RSA).get()).data,
|
||||
var msg = Message(fromPeer: PeerID.init(PrivateKey.random(ECDSA).get()).data,
|
||||
seqno: cast[seq[byte]]("12345"),
|
||||
topicIDs: @["bar"])
|
||||
mCache.put(msg)
|
||||
|
||||
for i in 0..<3:
|
||||
var msg = Message(fromPeer: PeerID.init(PrivateKey.random(RSA).get()).data,
|
||||
var msg = Message(fromPeer: PeerID.init(PrivateKey.random(ECDSA).get()).data,
|
||||
seqno: cast[seq[byte]]("12345"),
|
||||
topicIDs: @["baz"])
|
||||
mCache.put(msg)
|
||||
|
|
|
@ -0,0 +1,33 @@
|
|||
import unittest
|
||||
import nimcrypto/sha2,
|
||||
stew/[base64, byteutils]
|
||||
import ../../libp2p/[peer,
|
||||
crypto/crypto,
|
||||
protocols/pubsub/rpc/message,
|
||||
protocols/pubsub/rpc/messages]
|
||||
|
||||
suite "Message":
|
||||
test "default message id":
|
||||
let msg = Message(fromPeer: PeerID.init(PrivateKey.random(ECDSA).get()).data,
|
||||
seqno: ("12345").toBytes())
|
||||
|
||||
check msg.msgId == byteutils.toHex(msg.seqno) & PeerID.init(msg.fromPeer).pretty
|
||||
|
||||
test "sha256 message id":
|
||||
let msg = Message(fromPeer: PeerID.init(PrivateKey.random(ECDSA).get()).data,
|
||||
seqno: ("12345").toBytes(),
|
||||
data: ("12345").toBytes())
|
||||
|
||||
proc msgIdProvider(m: Message): string =
|
||||
Base64Url.encode(
|
||||
sha256.
|
||||
digest(m.data).
|
||||
data.
|
||||
toOpenArray(0, sha256.sizeDigest() - 1))
|
||||
|
||||
check msg.msgId == Base64Url.encode(
|
||||
sha256.
|
||||
digest(msg.data).
|
||||
data.
|
||||
toOpenArray(0, sha256.sizeDigest() - 1))
|
||||
|
|
@ -3,4 +3,5 @@
|
|||
import testgossipinternal,
|
||||
testfloodsub,
|
||||
testgossipsub,
|
||||
testmcache
|
||||
testmcache,
|
||||
testmessage
|
||||
|
|
|
@ -21,7 +21,7 @@ suite "Identify":
|
|||
test "handle identify message":
|
||||
proc testHandle(): Future[bool] {.async.} =
|
||||
let ma: MultiAddress = Multiaddress.init("/ip4/0.0.0.0/tcp/0")
|
||||
let remoteSecKey = PrivateKey.random(RSA).get()
|
||||
let remoteSecKey = PrivateKey.random(ECDSA).get()
|
||||
let remotePeerInfo = PeerInfo.init(remoteSecKey,
|
||||
[ma],
|
||||
["/test/proto1/1.0.0",
|
||||
|
@ -41,7 +41,7 @@ suite "Identify":
|
|||
let transport2: TcpTransport = TcpTransport.init()
|
||||
let conn = await transport2.dial(transport1.ma)
|
||||
|
||||
var peerInfo = PeerInfo.init(PrivateKey.random(RSA).get(), [ma])
|
||||
var peerInfo = PeerInfo.init(PrivateKey.random(ECDSA).get(), [ma])
|
||||
let identifyProto2 = newIdentify(peerInfo)
|
||||
discard await msDial.select(conn, IdentifyCodec)
|
||||
let id = await identifyProto2.identify(conn, remotePeerInfo)
|
||||
|
@ -66,7 +66,7 @@ suite "Identify":
|
|||
test "handle failed identify":
|
||||
proc testHandleError() {.async.} =
|
||||
let ma: MultiAddress = Multiaddress.init("/ip4/0.0.0.0/tcp/0")
|
||||
var remotePeerInfo = PeerInfo.init(PrivateKey.random(RSA).get(), [ma])
|
||||
var remotePeerInfo = PeerInfo.init(PrivateKey.random(ECDSA).get(), [ma])
|
||||
let identifyProto1 = newIdentify(remotePeerInfo)
|
||||
let msListen = newMultistream()
|
||||
|
||||
|
@ -85,12 +85,12 @@ suite "Identify":
|
|||
let transport2: TcpTransport = TcpTransport.init()
|
||||
let conn = await transport2.dial(transport1.ma)
|
||||
|
||||
var localPeerInfo = PeerInfo.init(PrivateKey.random(RSA).get(), [ma])
|
||||
var localPeerInfo = PeerInfo.init(PrivateKey.random(ECDSA).get(), [ma])
|
||||
let identifyProto2 = newIdentify(localPeerInfo)
|
||||
|
||||
try:
|
||||
discard await msDial.select(conn, IdentifyCodec)
|
||||
discard await identifyProto2.identify(conn, PeerInfo.init(PrivateKey.random(RSA).get()))
|
||||
discard await identifyProto2.identify(conn, PeerInfo.init(PrivateKey.random(ECDSA).get()))
|
||||
finally:
|
||||
await done.wait(5000.millis) # when no issues will not wait that long!
|
||||
await conn.close()
|
||||
|
|
|
@ -50,7 +50,7 @@ method init(p: TestProto) {.gcsafe.} =
|
|||
p.handler = handle
|
||||
|
||||
proc createSwitch(ma: MultiAddress; outgoing: bool): (Switch, PeerInfo) =
|
||||
var peerInfo: PeerInfo = PeerInfo.init(PrivateKey.random(RSA).get())
|
||||
var peerInfo: PeerInfo = PeerInfo.init(PrivateKey.random(ECDSA).get())
|
||||
peerInfo.addrs.add(ma)
|
||||
let identify = newIdentify(peerInfo)
|
||||
|
||||
|
@ -78,7 +78,7 @@ suite "Noise":
|
|||
proc testListenerDialer(): Future[bool] {.async.} =
|
||||
let
|
||||
server: MultiAddress = Multiaddress.init("/ip4/0.0.0.0/tcp/0")
|
||||
serverInfo = PeerInfo.init(PrivateKey.random(RSA).get(), [server])
|
||||
serverInfo = PeerInfo.init(PrivateKey.random(ECDSA).get(), [server])
|
||||
serverNoise = newNoise(serverInfo.privateKey, outgoing = false)
|
||||
|
||||
proc connHandler(conn: Connection) {.async, gcsafe.} =
|
||||
|
@ -94,7 +94,7 @@ suite "Noise":
|
|||
|
||||
let
|
||||
transport2: TcpTransport = TcpTransport.init()
|
||||
clientInfo = PeerInfo.init(PrivateKey.random(RSA).get(), [transport1.ma])
|
||||
clientInfo = PeerInfo.init(PrivateKey.random(ECDSA).get(), [transport1.ma])
|
||||
clientNoise = newNoise(clientInfo.privateKey, outgoing = true)
|
||||
conn = await transport2.dial(transport1.ma)
|
||||
sconn = await clientNoise.secure(conn, true)
|
||||
|
@ -116,7 +116,7 @@ suite "Noise":
|
|||
proc testListenerDialer(): Future[bool] {.async.} =
|
||||
let
|
||||
server: MultiAddress = Multiaddress.init("/ip4/0.0.0.0/tcp/0")
|
||||
serverInfo = PeerInfo.init(PrivateKey.random(RSA).get(), [server])
|
||||
serverInfo = PeerInfo.init(PrivateKey.random(ECDSA).get(), [server])
|
||||
serverNoise = newNoise(serverInfo.privateKey, outgoing = false)
|
||||
readTask = newFuture[void]()
|
||||
|
||||
|
@ -136,7 +136,7 @@ suite "Noise":
|
|||
|
||||
let
|
||||
transport2: TcpTransport = TcpTransport.init()
|
||||
clientInfo = PeerInfo.init(PrivateKey.random(RSA).get(), [transport1.ma])
|
||||
clientInfo = PeerInfo.init(PrivateKey.random(ECDSA).get(), [transport1.ma])
|
||||
clientNoise = newNoise(clientInfo.privateKey, outgoing = true)
|
||||
conn = await transport2.dial(transport1.ma)
|
||||
sconn = await clientNoise.secure(conn, true)
|
||||
|
@ -157,7 +157,7 @@ suite "Noise":
|
|||
proc testListenerDialer(): Future[bool] {.async.} =
|
||||
let
|
||||
server: MultiAddress = Multiaddress.init("/ip4/0.0.0.0/tcp/0")
|
||||
serverInfo = PeerInfo.init(PrivateKey.random(RSA).get(), [server])
|
||||
serverInfo = PeerInfo.init(PrivateKey.random(ECDSA).get(), [server])
|
||||
serverNoise = newNoise(serverInfo.privateKey, outgoing = false)
|
||||
readTask = newFuture[void]()
|
||||
|
||||
|
@ -179,7 +179,7 @@ suite "Noise":
|
|||
|
||||
let
|
||||
transport2: TcpTransport = TcpTransport.init()
|
||||
clientInfo = PeerInfo.init(PrivateKey.random(RSA).get(), [transport1.ma])
|
||||
clientInfo = PeerInfo.init(PrivateKey.random(ECDSA).get(), [transport1.ma])
|
||||
clientNoise = newNoise(clientInfo.privateKey, outgoing = true)
|
||||
conn = await transport2.dial(transport1.ma)
|
||||
sconn = await clientNoise.secure(conn, true)
|
||||
|
@ -235,7 +235,7 @@ suite "Noise":
|
|||
|
||||
# let
|
||||
# local = Multiaddress.init("/ip4/0.0.0.0/tcp/23456")
|
||||
# info = PeerInfo.init(PrivateKey.random(RSA), [local])
|
||||
# info = PeerInfo.init(PrivateKey.random(ECDSA), [local])
|
||||
# noise = newNoise(info.privateKey)
|
||||
# ms = newMultistream()
|
||||
# transport = TcpTransport.newTransport()
|
||||
|
@ -269,7 +269,7 @@ suite "Noise":
|
|||
# let
|
||||
# local = Multiaddress.init("/ip4/0.0.0.0/tcp/0")
|
||||
# remote = Multiaddress.init("/ip4/127.0.0.1/tcp/23456")
|
||||
# info = PeerInfo.init(PrivateKey.random(RSA), [local])
|
||||
# info = PeerInfo.init(PrivateKey.random(ECDSA), [local])
|
||||
# noise = newNoise(info.privateKey)
|
||||
# ms = newMultistream()
|
||||
# transport = TcpTransport.newTransport()
|
||||
|
@ -292,7 +292,7 @@ suite "Noise":
|
|||
# proc testListenerDialer(): Future[bool] {.async.} =
|
||||
# let
|
||||
# local = Multiaddress.init("/ip4/0.0.0.0/tcp/23456")
|
||||
# info = PeerInfo.init(PrivateKey.random(RSA), [local])
|
||||
# info = PeerInfo.init(PrivateKey.random(ECDSA), [local])
|
||||
# noise = newNoise(info.privateKey)
|
||||
# ms = newMultistream()
|
||||
# transport = TcpTransport.newTransport()
|
||||
|
|
|
@ -13,7 +13,7 @@ suite "PeerInfo":
|
|||
check tracker.isLeaked() == false
|
||||
|
||||
test "Should init with private key":
|
||||
let seckey = PrivateKey.random(RSA).get()
|
||||
let seckey = PrivateKey.random(ECDSA).get()
|
||||
var peerInfo = PeerInfo.init(seckey)
|
||||
var peerId = PeerID.init(seckey)
|
||||
|
||||
|
@ -22,7 +22,7 @@ suite "PeerInfo":
|
|||
check seckey.getKey().get() == peerInfo.publicKey.get()
|
||||
|
||||
test "Should init with public key":
|
||||
let seckey = PrivateKey.random(RSA).get()
|
||||
let seckey = PrivateKey.random(ECDSA).get()
|
||||
var peerInfo = PeerInfo.init(seckey.getKey().get())
|
||||
var peerId = PeerID.init(seckey.getKey().get())
|
||||
|
||||
|
@ -52,7 +52,7 @@ suite "PeerInfo":
|
|||
# PeerID.init("bafzbeie5745rpv2m6tjyuugywy4d5ewrqgqqhfnf445he3omzpjbx5xqxe") == peerInfo.peerId
|
||||
|
||||
test "Should return none if pubkey is missing from id":
|
||||
let peerInfo = PeerInfo.init(PeerID.init(PrivateKey.random(RSA).get()))
|
||||
let peerInfo = PeerInfo.init(PeerID.init(PrivateKey.random(ECDSA).get()))
|
||||
check peerInfo.publicKey.isNone
|
||||
|
||||
test "Should return some if pubkey is present in id":
|
||||
|
|
|
@ -31,7 +31,7 @@ type
|
|||
TestProto = ref object of LPProtocol
|
||||
|
||||
proc createSwitch(ma: MultiAddress): (Switch, PeerInfo) =
|
||||
var peerInfo: PeerInfo = PeerInfo.init(PrivateKey.random(RSA).tryGet())
|
||||
var peerInfo: PeerInfo = PeerInfo.init(PrivateKey.random(ECDSA).tryGet())
|
||||
peerInfo.addrs.add(ma)
|
||||
let identify = newIdentify(peerInfo)
|
||||
|
||||
|
@ -152,7 +152,7 @@ suite "Switch":
|
|||
# proc testListenerDialer(): Future[bool] {.async.} =
|
||||
# let
|
||||
# server: MultiAddress = Multiaddress.init("/ip4/0.0.0.0/tcp/0")
|
||||
# serverInfo = PeerInfo.init(PrivateKey.random(RSA), [server])
|
||||
# serverInfo = PeerInfo.init(PrivateKey.random(ECDSA), [server])
|
||||
# serverNoise = newSecio(serverInfo.privateKey)
|
||||
# readTask = newFuture[void]()
|
||||
|
||||
|
@ -174,7 +174,7 @@ suite "Switch":
|
|||
|
||||
# let
|
||||
# transport2: TcpTransport = TcpTransport.init()
|
||||
# clientInfo = PeerInfo.init(PrivateKey.random(RSA), [transport1.ma])
|
||||
# clientInfo = PeerInfo.init(PrivateKey.random(ECDSA), [transport1.ma])
|
||||
# clientNoise = newSecio(clientInfo.privateKey)
|
||||
# conn = await transport2.dial(transport1.ma)
|
||||
# sconn = await clientNoise.secure(conn)
|
||||
|
|
Loading…
Reference in New Issue