nwaku/tests/v2/test_jsonrpc_waku.nim

675 lines
22 KiB
Nim
Raw Normal View History

{.used.}
import
std/[options, sets, tables, os, strutils, sequtils, times],
2022-01-14 09:25:01 +00:00
chronicles,
testutils/unittests, stew/shims/net as stewNet,
json_rpc/[rpcserver, rpcclient],
eth/keys, eth/common/eth_types,
2021-06-09 14:37:08 +00:00
libp2p/[builders, switch, multiaddress],
libp2p/protobuf/minprotobuf,
libp2p/stream/[bufferstream, connection],
libp2p/crypto/crypto,
libp2p/protocols/pubsub/pubsub,
libp2p/protocols/pubsub/rpc/message
import
../../waku/v1/node/rpc/hexstrings,
../../waku/v2/node/message_store/waku_store_queue,
../../waku/v2/node/waku_node,
../../waku/v2/node/jsonrpc/[store_api,
relay_api,
debug_api,
filter_api,
admin_api,
private_api],
../../waku/v2/protocol/waku_message,
../../waku/v2/protocol/waku_relay,
../../waku/v2/protocol/waku_store,
../../waku/v2/protocol/waku_swap/waku_swap,
../../waku/v2/protocol/waku_filter,
../../waku/v2/protocol/waku_filter/client,
../../waku/v2/utils/peers,
../../waku/v2/utils/time,
./testlib/common
template sourceDir*: string = currentSourcePath.rsplit(DirSep, 1)[0]
const sigPath = sourceDir / ParDir / ParDir / "waku" / "v2" / "node" / "jsonrpc" / "jsonrpc_callsigs.nim"
createRpcSigs(RpcHttpClient, sigPath)
procSuite "Waku v2 JSON-RPC API":
let
rng = crypto.newRng()
privkey = crypto.PrivateKey.random(Secp256k1, rng[]).tryGet()
bindIp = ValidIpAddress.init("0.0.0.0")
extIp = ValidIpAddress.init("127.0.0.1")
port = Port(9000)
node = WakuNode.new(privkey, bindIp, port, some(extIp), some(port))
asyncTest "Debug API: get node info":
await node.start()
await node.mountRelay()
# RPC server setup
let
rpcPort = Port(8546)
ta = initTAddress(bindIp, rpcPort)
server = newRpcHttpServer([ta])
installDebugApiHandlers(node, server)
server.start()
let client = newRpcHttpClient()
2022-01-14 09:25:01 +00:00
await client.connect("127.0.0.1", rpcPort, false)
let response = await client.get_waku_v2_debug_v1_info()
check:
2022-01-10 15:07:35 +00:00
response.listenAddresses == @[$node.switch.peerInfo.addrs[^1] & "/p2p/" & $node.switch.peerInfo.peerId]
2022-01-14 09:25:01 +00:00
await server.stop()
await server.closeWait()
await node.stop()
asyncTest "Relay API: publish and subscribe/unsubscribe":
await node.start()
await node.mountRelay()
# RPC server setup
let
rpcPort = Port(8547)
ta = initTAddress(bindIp, rpcPort)
server = newRpcHttpServer([ta])
installRelayApiHandlers(node, server, newTable[string, seq[WakuMessage]]())
server.start()
let client = newRpcHttpClient()
2022-01-14 09:25:01 +00:00
await client.connect("127.0.0.1", rpcPort, false)
check:
# At this stage the node is only subscribed to the default topic
PubSub(node.wakuRelay).topics.len == 1
# Subscribe to new topics
let newTopics = @["1","2","3"]
var response = await client.post_waku_v2_relay_v1_subscriptions(newTopics)
check:
# Node is now subscribed to default + new topics
PubSub(node.wakuRelay).topics.len == 1 + newTopics.len
response == true
# Publish a message on the default topic
response = await client.post_waku_v2_relay_v1_message(DefaultPubsubTopic, WakuRelayMessage(payload: @[byte 1], contentTopic: some(DefaultContentTopic), timestamp: some(getNanosecondTime(epochTime()))))
check:
# @TODO poll topic to verify message has been published
response == true
# Unsubscribe from new topics
response = await client.delete_waku_v2_relay_v1_subscriptions(newTopics)
check:
# Node is now unsubscribed from new topics
PubSub(node.wakuRelay).topics.len == 1
response == true
2022-01-14 09:25:01 +00:00
await server.stop()
await server.closeWait()
await node.stop()
asyncTest "Relay API: get latest messages":
let
nodeKey1 = crypto.PrivateKey.random(Secp256k1, rng[])[]
node1 = WakuNode.new(nodeKey1, bindIp, Port(60300))
nodeKey2 = crypto.PrivateKey.random(Secp256k1, rng[])[]
node2 = WakuNode.new(nodeKey2, bindIp, Port(60302))
nodeKey3 = crypto.PrivateKey.random(Secp256k1, rng[])[]
node3 = WakuNode.new(nodeKey3, bindIp, Port(60303), some(extIp), some(port))
pubSubTopic = "polling"
contentTopic = DefaultContentTopic
payload1 = @[byte 9]
message1 = WakuMessage(payload: payload1, contentTopic: contentTopic)
payload2 = @[byte 8]
message2 = WakuMessage(payload: payload2, contentTopic: contentTopic)
await node1.start()
await node1.mountRelay(@[pubSubTopic])
await node2.start()
await node2.mountRelay(@[pubSubTopic])
await node3.start()
await node3.mountRelay(@[pubSubTopic])
2022-01-10 15:07:35 +00:00
await node1.connectToNodes(@[node2.switch.peerInfo.toRemotePeerInfo()])
await node3.connectToNodes(@[node2.switch.peerInfo.toRemotePeerInfo()])
# RPC server setup
let
rpcPort = Port(8548)
ta = initTAddress(bindIp, rpcPort)
server = newRpcHttpServer([ta])
# Let's connect to node 3 via the API
installRelayApiHandlers(node3, server, newTable[string, seq[WakuMessage]]())
server.start()
let client = newRpcHttpClient()
2022-01-14 09:25:01 +00:00
await client.connect("127.0.0.1", rpcPort, false)
# First see if we can retrieve messages published on the default topic (node is already subscribed)
await node2.publish(DefaultPubsubTopic, message1)
await sleepAsync(100.millis)
var messages = await client.get_waku_v2_relay_v1_messages(DefaultPubsubTopic)
check:
messages.len == 1
messages[0].contentTopic == contentTopic
messages[0].payload == payload1
# Ensure that read messages are cleared from cache
messages = await client.get_waku_v2_relay_v1_messages(pubSubTopic)
check:
messages.len == 0
# Now try to subscribe using API
var response = await client.post_waku_v2_relay_v1_subscriptions(@[pubSubTopic])
await sleepAsync(100.millis)
check:
# Node is now subscribed to pubSubTopic
response == true
# Now publish a message on node1 and see if we receive it on node3
await node1.publish(pubSubTopic, message2)
await sleepAsync(100.millis)
messages = await client.get_waku_v2_relay_v1_messages(pubSubTopic)
check:
messages.len == 1
messages[0].contentTopic == contentTopic
messages[0].payload == payload2
# Ensure that read messages are cleared from cache
messages = await client.get_waku_v2_relay_v1_messages(pubSubTopic)
check:
messages.len == 0
2022-01-14 09:25:01 +00:00
await server.stop()
await server.closeWait()
await node1.stop()
await node2.stop()
await node3.stop()
asyncTest "Store API: retrieve historical messages":
await node.start()
await node.mountRelay()
# RPC server setup
let
rpcPort = Port(8549)
ta = initTAddress(bindIp, rpcPort)
server = newRpcHttpServer([ta])
installStoreApiHandlers(node, server)
server.start()
# WakuStore setup
let
key = crypto.PrivateKey.random(ECDSA, rng[]).get()
2021-11-04 14:46:38 +00:00
peer = PeerInfo.new(key)
let store = StoreQueueRef.new()
await node.mountStore(store=store)
node.mountStoreClient(store=store)
var listenSwitch = newStandardSwitch(some(key))
await listenSwitch.start()
node.setStorePeer(listenSwitch.peerInfo.toRemotePeerInfo())
listenSwitch.mount(node.wakuRelay)
listenSwitch.mount(node.wakuStore)
# Now prime it with some history before tests
var
msgList = @[WakuMessage(payload: @[byte 0], contentTopic: ContentTopic("2"), timestamp: 0),
WakuMessage(payload: @[byte 1], contentTopic: DefaultContentTopic, timestamp: 1),
WakuMessage(payload: @[byte 2], contentTopic: DefaultContentTopic, timestamp: 2),
WakuMessage(payload: @[byte 3], contentTopic: DefaultContentTopic, timestamp: 3),
WakuMessage(payload: @[byte 4], contentTopic: DefaultContentTopic, timestamp: 4),
WakuMessage(payload: @[byte 5], contentTopic: DefaultContentTopic, timestamp: 5),
WakuMessage(payload: @[byte 6], contentTopic: DefaultContentTopic, timestamp: 6),
WakuMessage(payload: @[byte 7], contentTopic: DefaultContentTopic, timestamp: 7),
WakuMessage(payload: @[byte 8], contentTopic: DefaultContentTopic, timestamp: 8),
WakuMessage(payload: @[byte 9], contentTopic: ContentTopic("2"), timestamp: 9)]
for wakuMsg in msgList:
require node.wakuStore.store.put(DefaultPubsubTopic, wakuMsg).isOk()
let client = newRpcHttpClient()
2022-01-14 09:25:01 +00:00
await client.connect("127.0.0.1", rpcPort, false)
let response = await client.get_waku_v2_store_v1_messages(some(DefaultPubsubTopic), some(@[HistoryContentFilter(contentTopic: DefaultContentTopic)]), some(Timestamp(0)), some(Timestamp(9)), some(StorePagingOptions()))
check:
response.messages.len() == 8
response.pagingOptions.isNone()
2022-01-14 09:25:01 +00:00
await server.stop()
await server.closeWait()
await node.stop()
asyncTest "Filter API: subscribe/unsubscribe":
let
nodeKey1 = crypto.PrivateKey.random(Secp256k1, rng[])[]
node1 = WakuNode.new(nodeKey1, bindIp, Port(60390))
nodeKey2 = crypto.PrivateKey.random(Secp256k1, rng[])[]
node2 = WakuNode.new(nodeKey2, bindIp, Port(60392))
await allFutures(node1.start(), node2.start())
await node1.mountFilter()
await node2.mountFilterClient()
node2.setFilterPeer(node1.peerInfo.toRemotePeerInfo())
# RPC server setup
let
rpcPort = Port(8550)
ta = initTAddress(bindIp, rpcPort)
server = newRpcHttpServer([ta])
installFilterApiHandlers(node2, server, newTable[ContentTopic, seq[WakuMessage]]())
server.start()
let client = newRpcHttpClient()
2022-01-14 09:25:01 +00:00
await client.connect("127.0.0.1", rpcPort, false)
check:
# Light node has not yet subscribed to any filters
node2.wakuFilterClient.getSubscriptionsCount() == 0
let contentFilters = @[
ContentFilter(contentTopic: DefaultContentTopic),
ContentFilter(contentTopic: ContentTopic("2")),
ContentFilter(contentTopic: ContentTopic("3")),
ContentFilter(contentTopic: ContentTopic("4")),
]
var response = await client.post_waku_v2_filter_v1_subscription(contentFilters=contentFilters, topic=some(DefaultPubsubTopic))
check:
response == true
# Light node has successfully subscribed to 4 content topics
node2.wakuFilterClient.getSubscriptionsCount() == 4
response = await client.delete_waku_v2_filter_v1_subscription(contentFilters=contentFilters, topic=some(DefaultPubsubTopic))
check:
response == true
# Light node has successfully unsubscribed from all filters
node2.wakuFilterClient.getSubscriptionsCount() == 0
## Cleanup
2022-01-14 09:25:01 +00:00
await server.stop()
await server.closeWait()
await allFutures(node1.stop(), node2.stop())
asyncTest "Admin API: connect to ad-hoc peers":
# Create a couple of nodes
let
nodeKey1 = crypto.PrivateKey.random(Secp256k1, rng[])[]
node1 = WakuNode.new(nodeKey1, ValidIpAddress.init("0.0.0.0"), Port(60600))
nodeKey2 = crypto.PrivateKey.random(Secp256k1, rng[])[]
node2 = WakuNode.new(nodeKey2, ValidIpAddress.init("0.0.0.0"), Port(60602))
2022-01-10 15:07:35 +00:00
peerInfo2 = node2.switch.peerInfo
nodeKey3 = crypto.PrivateKey.random(Secp256k1, rng[])[]
node3 = WakuNode.new(nodeKey3, ValidIpAddress.init("0.0.0.0"), Port(60604))
2022-01-10 15:07:35 +00:00
peerInfo3 = node3.switch.peerInfo
await allFutures([node1.start(), node2.start(), node3.start()])
await node1.mountRelay()
await node2.mountRelay()
await node3.mountRelay()
# RPC server setup
let
rpcPort = Port(8551)
ta = initTAddress(bindIp, rpcPort)
server = newRpcHttpServer([ta])
installAdminApiHandlers(node1, server)
server.start()
let client = newRpcHttpClient()
2022-01-14 09:25:01 +00:00
await client.connect("127.0.0.1", rpcPort, false)
# Connect to nodes 2 and 3 using the Admin API
let postRes = await client.post_waku_v2_admin_v1_peers(@[constructMultiaddrStr(peerInfo2),
constructMultiaddrStr(peerInfo3)])
check:
postRes
# Verify that newly connected peers are being managed
let getRes = await client.get_waku_v2_admin_v1_peers()
check:
getRes.len == 2
# Check peer 2
getRes.anyIt(it.protocol == WakuRelayCodec and
it.multiaddr == constructMultiaddrStr(peerInfo2))
# Check peer 3
getRes.anyIt(it.protocol == WakuRelayCodec and
it.multiaddr == constructMultiaddrStr(peerInfo3))
2022-01-14 09:25:01 +00:00
await server.stop()
await server.closeWait()
await allFutures([node1.stop(), node2.stop(), node3.stop()])
asyncTest "Admin API: get managed peer information":
# Create a couple of nodes
let
nodeKey1 = crypto.PrivateKey.random(Secp256k1, rng[])[]
node1 = WakuNode.new(nodeKey1, ValidIpAddress.init("0.0.0.0"), Port(60220))
nodeKey2 = crypto.PrivateKey.random(Secp256k1, rng[])[]
node2 = WakuNode.new(nodeKey2, ValidIpAddress.init("0.0.0.0"), Port(60222))
peerInfo2 = node2.peerInfo
nodeKey3 = crypto.PrivateKey.random(Secp256k1, rng[])[]
node3 = WakuNode.new(nodeKey3, ValidIpAddress.init("0.0.0.0"), Port(60224))
peerInfo3 = node3.peerInfo
await allFutures([node1.start(), node2.start(), node3.start()])
await node1.mountRelay()
await node2.mountRelay()
await node3.mountRelay()
# Dial nodes 2 and 3 from node1
await node1.connectToNodes(@[constructMultiaddrStr(peerInfo2)])
await node1.connectToNodes(@[constructMultiaddrStr(peerInfo3)])
# RPC server setup
let
rpcPort = Port(8552)
ta = initTAddress(bindIp, rpcPort)
server = newRpcHttpServer([ta])
installAdminApiHandlers(node1, server)
server.start()
let client = newRpcHttpClient()
2022-01-14 09:25:01 +00:00
await client.connect("127.0.0.1", rpcPort, false)
let response = await client.get_waku_v2_admin_v1_peers()
check:
response.len == 2
# Check peer 2
response.anyIt(it.protocol == WakuRelayCodec and
it.multiaddr == constructMultiaddrStr(peerInfo2))
# Check peer 3
response.anyIt(it.protocol == WakuRelayCodec and
it.multiaddr == constructMultiaddrStr(peerInfo3))
2022-01-14 09:25:01 +00:00
await server.stop()
await server.closeWait()
await allFutures([node1.stop(), node2.stop(), node3.stop()])
asyncTest "Admin API: get unmanaged peer information":
let
nodeKey = crypto.PrivateKey.random(Secp256k1, rng[])[]
node = WakuNode.new(nodeKey, ValidIpAddress.init("0.0.0.0"), Port(60523))
await node.start()
# RPC server setup
let
rpcPort = Port(8553)
ta = initTAddress(bindIp, rpcPort)
server = newRpcHttpServer([ta])
installAdminApiHandlers(node, server)
server.start()
let client = newRpcHttpClient()
2022-01-14 09:25:01 +00:00
await client.connect("127.0.0.1", rpcPort, false)
await node.mountFilter()
await node.mountFilterClient()
await node.mountSwap()
let store = StoreQueueRef.new()
await node.mountStore(store=store)
node.mountStoreClient(store=store)
# Create and set some peers
let
locationAddr = MultiAddress.init("/ip4/127.0.0.1/tcp/0").tryGet()
filterKey = crypto.PrivateKey.random(ECDSA, rng[]).get()
2021-11-04 14:46:38 +00:00
filterPeer = PeerInfo.new(filterKey, @[locationAddr])
swapKey = crypto.PrivateKey.random(ECDSA, rng[]).get()
2021-11-04 14:46:38 +00:00
swapPeer = PeerInfo.new(swapKey, @[locationAddr])
storeKey = crypto.PrivateKey.random(ECDSA, rng[]).get()
2021-11-04 14:46:38 +00:00
storePeer = PeerInfo.new(storeKey, @[locationAddr])
node.wakuSwap.setPeer(swapPeer.toRemotePeerInfo())
node.setStorePeer(storePeer.toRemotePeerInfo())
node.setFilterPeer(filterPeer.toRemotePeerInfo())
let response = await client.get_waku_v2_admin_v1_peers()
## Then
check:
response.len == 3
# Check filter peer
(response.filterIt(it.protocol == WakuFilterCodec)[0]).multiaddr == constructMultiaddrStr(filterPeer)
# Check swap peer
(response.filterIt(it.protocol == WakuSwapCodec)[0]).multiaddr == constructMultiaddrStr(swapPeer)
# Check store peer
(response.filterIt(it.protocol == WakuStoreCodec)[0]).multiaddr == constructMultiaddrStr(storePeer)
## Cleanup
2022-01-14 09:25:01 +00:00
await server.stop()
await server.closeWait()
await node.stop()
asyncTest "Private API: generate asymmetric keys and encrypt/decrypt communication":
let
nodeKey1 = crypto.PrivateKey.random(Secp256k1, rng[])[]
node1 = WakuNode.new(nodeKey1, bindIp, Port(62001))
nodeKey2 = crypto.PrivateKey.random(Secp256k1, rng[])[]
node2 = WakuNode.new(nodeKey2, bindIp, Port(62002))
nodeKey3 = crypto.PrivateKey.random(Secp256k1, rng[])[]
node3 = WakuNode.new(nodeKey3, bindIp, Port(62003), some(extIp), some(port))
pubSubTopic = "polling"
contentTopic = DefaultContentTopic
payload = @[byte 9]
Refactoring timestamps (#842) * Refactor timestamps type from float64 to int64 (milliseconds resolution) * Revert epochs to float64 * Update 00002_addSenderTimeStamp.up.sql * Update quicksim2.nim * Add files via upload * Delete 00003_convertTimestampsToInts.up.sql * Add files via upload * Rename 00003_convertTimestampsToInts.up.sql to 00003_addTimestampsToInts.up.sql * Delete 00003_addTimestampsToInts.up.sql * Rln-relay integration into chat2 (#835) * adds ProofMetadata * adds EPOCH_INTERVAL * adds messageLog field * adds updateLog, toEpoch, fromEpoch, getEpoch, compareTo * adds unit test for toEpoch and fromEpoch * adds unit test for Epoch comparison * adds result codes for updateLog * adds unit test for update log * renames epoch related consts * modifies updateLog with new return type and new logic of spam detection * adds unit text for the modified updateLog * changes max epoch gap type size * splits updateLog into two procs isSpam and updateLog * updates unittests * fixes a bug, returns false when the message is not spam * renames messageLog to nullifierLog * renames isSpam to hasDuplicate * updates the rln validator, adds comments * adds appendRLNProof proc plus some code beatification * unit test for validate message * adds unhappy test to validateMessage unit test * renames EPOCH_UNIT_SECONDS * renames MAX_CLOCK_GAP_SECONDS * WIP: integration test * fixes compile errors * sets a real epoch value * updates on old unittests * adds comments to the rln relay tests * adds more comments * makes rln import conditional * adds todos * adds more todos * adds rln-relay mount process into chat2 * further todos * logs contentTopic * introduces rln relay configs * changes default pubsub topic * adds contentTopic config * imports rln relay dependencies * consolidates imports * removes module identifier from ContentTopic * adds contentTopic field * adds contentTopic argument to mountRlnRelay calls * appends rln proof to chat2 messages * changes the default chat2 contentTopic * adds missing content topic fields * fixes a bug * adds a new logic about empty content topics * appends proof only when rln flag is active * removes unnecessary todos * fixes an indentation issue * adds log messages * verifies the proof against the concatenation of msg payload and content topic * a bug fix * removes duplicate epoch time calculation * updates log level to trace * updates default rln-relay content topic * adds support for empty content topics * updates changelog * changelog updates * removes a commented code block * updates addRLNRelayValidator string doc * Squashed commit of the following: commit bc36c99ab202d07baa0a5f0100bd10d1d76fdfa1 Merge: dc2b2946 5a77d6e2 Author: G <28568419+s1fr0@users.noreply.github.com> Date: Sat Feb 5 01:10:06 2022 +0100 Merge branch 'master' into int64-timestamps-ns commit dc2b294667bb5770cc32b93cc560638cf5ce7087 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Sat Feb 5 00:24:45 2022 +0100 Fix commit f97b95a036a197938df38a5adaea46fca778016d Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Sat Feb 5 00:13:18 2022 +0100 Missing import commit 060c4f8d64e1b6e7c0593540fa8fa7f4cadf6df7 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Sat Feb 5 00:10:36 2022 +0100 Fixed typo commit 08ca99b6f692d3df6d4c7c2312c7cada05fc0041 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 23:59:20 2022 +0100 Time util file commit 2b5c360746990936dec256e90d08dae3c3e35a94 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 23:33:20 2022 +0100 Moved time utility functions to utils/time commit fdaf121f089aa011855303cc8dd1ce52aec506ad Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 23:10:25 2022 +0100 Fix comment commit c7e06ab4e7618d9a3fe8aa744dd48bf3f7d8754c Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 23:04:13 2022 +0100 Restore previous migration script commit 80282db1d79df676255d4b8e6e09d9f8a2b00fd3 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 22:54:15 2022 +0100 Typo commit b9d67f89b0eea11a8362dbb10b5f9d6894343352 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 22:49:29 2022 +0100 Added utilities to get int64 nanosecond, microsecond, millisecond time resolution from float commit 0130d496e694a01cfc9eeb90b7cbc77764490bf9 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 22:36:35 2022 +0100 Switched to nanoseconds support. * Update CHANGELOG.md * Create 00003_convertTimestampsToInt64.up.sql Migration script * Moved migration script to right location * Update waku_rln_relay_utils.nim * Update waku_rln_relay_utils.nim * Addressed reviewers' comments * Update default fleet metrics dashboard (#844) * Fix * No need for float * Aligning master to changes in PR * Further fixes Co-authored-by: Sanaz Taheri Boshrooyeh <35961250+staheri14@users.noreply.github.com> Co-authored-by: Hanno Cornelius <68783915+jm-clius@users.noreply.github.com>
2022-02-17 15:00:15 +00:00
message = WakuRelayMessage(payload: payload, contentTopic: some(contentTopic), timestamp: some(getNanosecondTime(epochTime())))
topicCache = newTable[string, seq[WakuMessage]]()
await node1.start()
await node1.mountRelay(@[pubSubTopic])
await node2.start()
await node2.mountRelay(@[pubSubTopic])
await node3.start()
await node3.mountRelay(@[pubSubTopic])
2022-01-10 15:07:35 +00:00
await node1.connectToNodes(@[node2.switch.peerInfo.toRemotePeerInfo()])
await node3.connectToNodes(@[node2.switch.peerInfo.toRemotePeerInfo()])
# Setup two servers so we can see both sides of encrypted communication
let
rpcPort1 = Port(8554)
ta1 = initTAddress(bindIp, rpcPort1)
server1 = newRpcHttpServer([ta1])
rpcPort3 = Port(8555)
ta3 = initTAddress(bindIp, rpcPort3)
server3 = newRpcHttpServer([ta3])
# Let's connect to nodes 1 and 3 via the API
installPrivateApiHandlers(node1, server1, newTable[string, seq[WakuMessage]]())
installPrivateApiHandlers(node3, server3, topicCache)
installRelayApiHandlers(node3, server3, topicCache)
server1.start()
server3.start()
let client1 = newRpcHttpClient()
2022-01-14 09:25:01 +00:00
await client1.connect("127.0.0.1", rpcPort1, false)
let client3 = newRpcHttpClient()
2022-01-14 09:25:01 +00:00
await client3.connect("127.0.0.1", rpcPort3, false)
# Let's get a keypair for node3
let keypair = await client3.get_waku_v2_private_v1_asymmetric_keypair()
# Now try to subscribe on node3 using API
let sub = await client3.post_waku_v2_relay_v1_subscriptions(@[pubSubTopic])
await sleepAsync(100.millis)
check:
# node3 is now subscribed to pubSubTopic
sub
# Now publish and encrypt a message on node1 using node3's public key
let posted = await client1.post_waku_v2_private_v1_asymmetric_message(pubSubTopic, message, publicKey = (%keypair.pubkey).getStr())
check:
posted
await sleepAsync(100.millis)
# Let's see if we can receive, and decrypt, this message on node3
var messages = await client3.get_waku_v2_private_v1_asymmetric_messages(pubSubTopic, privateKey = (%keypair.seckey).getStr())
check:
messages.len == 1
messages[0].contentTopic.get == contentTopic
messages[0].payload == payload
# Ensure that read messages are cleared from cache
messages = await client3.get_waku_v2_private_v1_asymmetric_messages(pubSubTopic, privateKey = (%keypair.seckey).getStr())
check:
messages.len == 0
2022-01-14 09:25:01 +00:00
await server1.stop()
await server1.closeWait()
await server3.stop()
await server3.closeWait()
await node1.stop()
await node2.stop()
await node3.stop()
asyncTest "Private API: generate symmetric keys and encrypt/decrypt communication":
let
nodeKey1 = crypto.PrivateKey.random(Secp256k1, rng[])[]
node1 = WakuNode.new(nodeKey1, bindIp, Port(62100))
nodeKey2 = crypto.PrivateKey.random(Secp256k1, rng[])[]
node2 = WakuNode.new(nodeKey2, bindIp, Port(62102))
nodeKey3 = crypto.PrivateKey.random(Secp256k1, rng[])[]
node3 = WakuNode.new(nodeKey3, bindIp, Port(62103), some(extIp), some(port))
pubSubTopic = "polling"
contentTopic = DefaultContentTopic
payload = @[byte 9]
Refactoring timestamps (#842) * Refactor timestamps type from float64 to int64 (milliseconds resolution) * Revert epochs to float64 * Update 00002_addSenderTimeStamp.up.sql * Update quicksim2.nim * Add files via upload * Delete 00003_convertTimestampsToInts.up.sql * Add files via upload * Rename 00003_convertTimestampsToInts.up.sql to 00003_addTimestampsToInts.up.sql * Delete 00003_addTimestampsToInts.up.sql * Rln-relay integration into chat2 (#835) * adds ProofMetadata * adds EPOCH_INTERVAL * adds messageLog field * adds updateLog, toEpoch, fromEpoch, getEpoch, compareTo * adds unit test for toEpoch and fromEpoch * adds unit test for Epoch comparison * adds result codes for updateLog * adds unit test for update log * renames epoch related consts * modifies updateLog with new return type and new logic of spam detection * adds unit text for the modified updateLog * changes max epoch gap type size * splits updateLog into two procs isSpam and updateLog * updates unittests * fixes a bug, returns false when the message is not spam * renames messageLog to nullifierLog * renames isSpam to hasDuplicate * updates the rln validator, adds comments * adds appendRLNProof proc plus some code beatification * unit test for validate message * adds unhappy test to validateMessage unit test * renames EPOCH_UNIT_SECONDS * renames MAX_CLOCK_GAP_SECONDS * WIP: integration test * fixes compile errors * sets a real epoch value * updates on old unittests * adds comments to the rln relay tests * adds more comments * makes rln import conditional * adds todos * adds more todos * adds rln-relay mount process into chat2 * further todos * logs contentTopic * introduces rln relay configs * changes default pubsub topic * adds contentTopic config * imports rln relay dependencies * consolidates imports * removes module identifier from ContentTopic * adds contentTopic field * adds contentTopic argument to mountRlnRelay calls * appends rln proof to chat2 messages * changes the default chat2 contentTopic * adds missing content topic fields * fixes a bug * adds a new logic about empty content topics * appends proof only when rln flag is active * removes unnecessary todos * fixes an indentation issue * adds log messages * verifies the proof against the concatenation of msg payload and content topic * a bug fix * removes duplicate epoch time calculation * updates log level to trace * updates default rln-relay content topic * adds support for empty content topics * updates changelog * changelog updates * removes a commented code block * updates addRLNRelayValidator string doc * Squashed commit of the following: commit bc36c99ab202d07baa0a5f0100bd10d1d76fdfa1 Merge: dc2b2946 5a77d6e2 Author: G <28568419+s1fr0@users.noreply.github.com> Date: Sat Feb 5 01:10:06 2022 +0100 Merge branch 'master' into int64-timestamps-ns commit dc2b294667bb5770cc32b93cc560638cf5ce7087 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Sat Feb 5 00:24:45 2022 +0100 Fix commit f97b95a036a197938df38a5adaea46fca778016d Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Sat Feb 5 00:13:18 2022 +0100 Missing import commit 060c4f8d64e1b6e7c0593540fa8fa7f4cadf6df7 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Sat Feb 5 00:10:36 2022 +0100 Fixed typo commit 08ca99b6f692d3df6d4c7c2312c7cada05fc0041 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 23:59:20 2022 +0100 Time util file commit 2b5c360746990936dec256e90d08dae3c3e35a94 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 23:33:20 2022 +0100 Moved time utility functions to utils/time commit fdaf121f089aa011855303cc8dd1ce52aec506ad Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 23:10:25 2022 +0100 Fix comment commit c7e06ab4e7618d9a3fe8aa744dd48bf3f7d8754c Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 23:04:13 2022 +0100 Restore previous migration script commit 80282db1d79df676255d4b8e6e09d9f8a2b00fd3 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 22:54:15 2022 +0100 Typo commit b9d67f89b0eea11a8362dbb10b5f9d6894343352 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 22:49:29 2022 +0100 Added utilities to get int64 nanosecond, microsecond, millisecond time resolution from float commit 0130d496e694a01cfc9eeb90b7cbc77764490bf9 Author: s1fr0 <28568419+s1fr0@users.noreply.github.com> Date: Fri Feb 4 22:36:35 2022 +0100 Switched to nanoseconds support. * Update CHANGELOG.md * Create 00003_convertTimestampsToInt64.up.sql Migration script * Moved migration script to right location * Update waku_rln_relay_utils.nim * Update waku_rln_relay_utils.nim * Addressed reviewers' comments * Update default fleet metrics dashboard (#844) * Fix * No need for float * Aligning master to changes in PR * Further fixes Co-authored-by: Sanaz Taheri Boshrooyeh <35961250+staheri14@users.noreply.github.com> Co-authored-by: Hanno Cornelius <68783915+jm-clius@users.noreply.github.com>
2022-02-17 15:00:15 +00:00
message = WakuRelayMessage(payload: payload, contentTopic: some(contentTopic), timestamp: some(getNanosecondTime(epochTime())))
topicCache = newTable[string, seq[WakuMessage]]()
await node1.start()
await node1.mountRelay(@[pubSubTopic])
await node2.start()
await node2.mountRelay(@[pubSubTopic])
await node3.start()
await node3.mountRelay(@[pubSubTopic])
2022-01-10 15:07:35 +00:00
await node1.connectToNodes(@[node2.switch.peerInfo.toRemotePeerInfo()])
await node3.connectToNodes(@[node2.switch.peerInfo.toRemotePeerInfo()])
# Setup two servers so we can see both sides of encrypted communication
let
rpcPort1 = Port(8556)
ta1 = initTAddress(bindIp, rpcPort1)
server1 = newRpcHttpServer([ta1])
rpcPort3 = Port(8557)
ta3 = initTAddress(bindIp, rpcPort3)
server3 = newRpcHttpServer([ta3])
# Let's connect to nodes 1 and 3 via the API
installPrivateApiHandlers(node1, server1, newTable[string, seq[WakuMessage]]())
installPrivateApiHandlers(node3, server3, topicCache)
installRelayApiHandlers(node3, server3, topicCache)
server1.start()
server3.start()
let client1 = newRpcHttpClient()
2022-01-14 09:25:01 +00:00
await client1.connect("127.0.0.1", rpcPort1, false)
let client3 = newRpcHttpClient()
2022-01-14 09:25:01 +00:00
await client3.connect("127.0.0.1", rpcPort3, false)
# Let's get a symkey for node3
let symkey = await client3.get_waku_v2_private_v1_symmetric_key()
# Now try to subscribe on node3 using API
let sub = await client3.post_waku_v2_relay_v1_subscriptions(@[pubSubTopic])
await sleepAsync(100.millis)
check:
# node3 is now subscribed to pubSubTopic
sub
# Now publish and encrypt a message on node1 using node3's symkey
let posted = await client1.post_waku_v2_private_v1_symmetric_message(pubSubTopic, message, symkey = (%symkey).getStr())
check:
posted
await sleepAsync(100.millis)
# Let's see if we can receive, and decrypt, this message on node3
var messages = await client3.get_waku_v2_private_v1_symmetric_messages(pubSubTopic, symkey = (%symkey).getStr())
check:
messages.len == 1
messages[0].contentTopic.get == contentTopic
messages[0].payload == payload
# Ensure that read messages are cleared from cache
messages = await client3.get_waku_v2_private_v1_symmetric_messages(pubSubTopic, symkey = (%symkey).getStr())
check:
messages.len == 0
2022-01-14 09:25:01 +00:00
await server1.stop()
await server1.closeWait()
await server3.stop()
await server3.closeWait()
await node1.stop()
await node2.stop()
Fix redundant use of content topics (#528) made resulting changes to waku_filter Made changes to wakunode2, filter_api and waku_filter Update waku v2 test scripts referencing Content Topics Update ContentFilter in chat example Remove unneccesary loops from filter api closes #496 Apply keep-alive for chat2 (#525) Makes the arguments of the store jsonrpc api optional (#526) * makes pubsubTopic filter optional * makes contentFilter optional * append Option Co-authored-by: Oskar Thorén <ot@oskarthoren.com> Update Changelog with changes to ContentFilter Fix indentation and code semantics Enables perssist-message flag in the store protocol for wakunode2 (#519) * enables perssistmessage flag * disables in memory storage when persist-messages is false * adds the persistMessages input to the mountStore * defaults the store flag to true * adds the missing argument * persists messages in memory conditioned to the persistMessages flag * adds persistmessages flag to the config_bridge * defaults persistmessages to true * defaults the store flag to true and persist-messages to false * updates store.md * updates chat2 instructions about --store flag * removes --store flag from chat2 command execution Co-authored-by: Oskar Thorén <ot@oskarthoren.com> Fix: light-mode relay for all light protocols (#529) * Fix: light-mode relay for all light protocols * Clear up confusing use of overloaded concepts Fix ContentFilter Schema in wakunode test script Enables perssist-message flag in the store protocol for wakunode2 (#519) * enables perssistmessage flag * disables in memory storage when persist-messages is false * adds the persistMessages input to the mountStore * defaults the store flag to true * adds the missing argument * persists messages in memory conditioned to the persistMessages flag * adds persistmessages flag to the config_bridge * defaults persistmessages to true * defaults the store flag to true and persist-messages to false * updates store.md * updates chat2 instructions about --store flag * removes --store flag from chat2 command execution Co-authored-by: Oskar Thorén <ot@oskarthoren.com> Fix: light-mode relay for all light protocols (#529) * Fix: light-mode relay for all light protocols * Clear up confusing use of overloaded concepts Fix resulting issues after merge
2021-05-05 08:34:40 +00:00
await node3.stop()