nwaku/apps/chat2/chat2.nim

580 lines
20 KiB
Nim
Raw Normal View History

## chat2 is an example of usage of Waku v2. For suggested usage options, please
## see dingpu tutorial in docs folder.
when not(compileOption("threads")):
{.fatal: "Please, compile this program with the --threads:on option!".}
when (NimMajor, NimMinor) < (1, 4):
{.push raises: [Defect].}
else:
{.push raises: [].}
import std/[strformat, strutils, times, json, options, random]
import confutils, chronicles, chronos, stew/shims/net as stewNet,
2022-11-04 09:52:08 +00:00
eth/keys, bearssl, stew/[byteutils, results],
nimcrypto/pbkdf2,
metrics,
metrics/chronos_httpserver
import libp2p/[switch, # manage transports, a single entry point for dialing and listening
crypto/crypto, # cryptographic functions
stream/connection, # create and close stream read / write connections
multiaddress, # encode different addressing schemes. For example, /ip4/7.7.7.7/tcp/6543 means it is using IPv4 protocol and TCP
peerinfo, # manage the information of a peer, such as peer ID and public / private key
peerid, # Implement how peers interact
2021-03-09 07:23:53 +00:00
protobuf/minprotobuf, # message serialisation/deserialisation from and to protobufs
protocols/secure/secio, # define the protocol of secure input / output, allows encrypted communication that uses public keys to validate signed messages instead of a certificate authority like in TLS
2022-11-04 09:52:08 +00:00
nameresolving/dnsresolver]# define DNS resolution
import
../../waku/waku_core,
../../waku/waku_lightpush,
../../waku/waku_lightpush/rpc,
../../waku/waku_filter,
2023-08-15 00:43:10 +00:00
../../waku/waku_enr,
../../waku/waku_store,
../../waku/waku_dnsdisc,
../../waku/waku_node,
../../waku/node/waku_metrics,
../../waku/node/peer_manager,
../../waku/common/utils/nat,
./config_chat2
import
libp2p/protocols/pubsub/rpc/messages,
libp2p/protocols/pubsub/pubsub
import
../../waku/waku_rln_relay
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
2022-02-04 23:58:27 +00:00
const Help = """
Commands: /[?|help|connect|nick|exit]
help: Prints this help
connect: dials a remote peer
2021-03-04 07:19:21 +00:00
nick: change nickname for current chat session
exit: exits chat session
"""
# XXX Connected is a bit annoying, because incoming connections don't trigger state change
# Could poll connection pool or something here, I suppose
# TODO Ensure connected turns true on incoming connections, or get rid of it
type Chat = ref object
node: WakuNode # waku node for publishing, subscribing, etc
transp: StreamTransport # transport streams between read & write file descriptor
subscribed: bool # indicates if a node is subscribed or not to a topic
connected: bool # if the node is connected to another peer
started: bool # if the node has started
2021-03-03 08:40:19 +00:00
nick: string # nickname for this chat session
2021-03-08 07:45:10 +00:00
prompt: bool # chat prompt is showing
contentTopic: string # default content topic for chat messages
type
PrivateKey* = crypto.PrivateKey
Topic* = waku_core.PubsubTopic
2021-03-09 07:23:53 +00:00
#####################
## chat2 protobufs ##
#####################
type
SelectResult*[T] = Result[T, string]
Chat2Message* = object
timestamp*: int64
nick*: string
payload*: seq[byte]
2021-03-09 07:23:53 +00:00
proc init*(T: type Chat2Message, buffer: seq[byte]): ProtoResult[T] =
var msg = Chat2Message()
let pb = initProtoBuffer(buffer)
var timestamp: uint64
discard ? pb.getField(1, timestamp)
msg.timestamp = int64(timestamp)
discard ? pb.getField(2, msg.nick)
discard ? pb.getField(3, msg.payload)
ok(msg)
proc encode*(message: Chat2Message): ProtoBuffer =
var serialised = initProtoBuffer()
serialised.write(1, uint64(message.timestamp))
serialised.write(2, message.nick)
serialised.write(3, message.payload)
return serialised
proc toString*(message: Chat2Message): string =
# Get message date and timestamp in local time
let time = message.timestamp.fromUnix().local().format("'<'MMM' 'dd,' 'HH:mm'>'")
return time & " " & message.nick & ": " & string.fromBytes(message.payload)
#####################
proc connectToNodes(c: Chat, nodes: seq[string]) {.async.} =
echo "Connecting to nodes"
await c.node.connectToNodes(nodes)
c.connected = true
2021-03-08 07:45:10 +00:00
proc showChatPrompt(c: Chat) =
if not c.prompt:
try:
stdout.write(">> ")
stdout.flushFile()
c.prompt = true
except IOError:
discard
Updates rln-chat2 interface (#846) * 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 * WIP * removes duplicate epoch time calculation * converts echo to log * invokes handler * bug fix * prints calculated epoch * changes the format of printed epoch * updates log levels * logs the input buffer supplied to the generate_proof * replaces echos with logs * changes log level to trace * resets the log level of chat2 to INFO * upgrades log level to debug * exports toRLNSignal and adds a doc string * updates log level * enables all test2 * removes an echo statement * modifies a comment * further updates on the log level * a minor update * invokes the spam handler when provided * checks for payload version * deletes a redundant check * deletes a rendant check * updates default rln-relay cht2 content topic * adds a todo and log * changes the case of testnet content topic * removes a flaky check
2022-02-16 22:52:21 +00:00
proc getChatLine(c: Chat, msg:WakuMessage): Result[string, string]=
# No payload encoding/encryption from Waku
let
pb = Chat2Message.init(msg.payload)
chatLine = if pb.isOk: pb[].toString()
else: string.fromBytes(msg.payload)
return ok(chatline)
Updates rln-chat2 interface (#846) * 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 * WIP * removes duplicate epoch time calculation * converts echo to log * invokes handler * bug fix * prints calculated epoch * changes the format of printed epoch * updates log levels * logs the input buffer supplied to the generate_proof * replaces echos with logs * changes log level to trace * resets the log level of chat2 to INFO * upgrades log level to debug * exports toRLNSignal and adds a doc string * updates log level * enables all test2 * removes an echo statement * modifies a comment * further updates on the log level * a minor update * invokes the spam handler when provided * checks for payload version * deletes a redundant check * deletes a rendant check * updates default rln-relay cht2 content topic * adds a todo and log * changes the case of testnet content topic * removes a flaky check
2022-02-16 22:52:21 +00:00
proc printReceivedMessage(c: Chat, msg: WakuMessage) =
let
pb = Chat2Message.init(msg.payload)
chatLine = if pb.isOk: pb[].toString()
else: string.fromBytes(msg.payload)
try:
echo &"{chatLine}"
except ValueError:
# Formatting fail. Print chat line in any case.
echo chatLine
c.prompt = false
showChatPrompt(c)
trace "Printing message", topic=DefaultPubsubTopic, chatLine,
contentTopic = msg.contentTopic
2021-03-03 08:40:19 +00:00
proc readNick(transp: StreamTransport): Future[string] {.async.} =
# Chat prompt
stdout.write("Choose a nickname >> ")
stdout.flushFile()
return await transp.readLine()
proc startMetricsServer(serverIp: ValidIpAddress, serverPort: Port): Result[MetricsHttpServerRef, string] =
info "Starting metrics HTTP server", serverIp= $serverIp, serverPort= $serverPort
let metricsServerRes = MetricsHttpServerRef.new($serverIp, serverPort)
if metricsServerRes.isErr():
return err("metrics HTTP server start failed: " & $metricsServerRes.error)
let server = metricsServerRes.value
try:
waitFor server.start()
except CatchableError:
return err("metrics HTTP server start failed: " & getCurrentExceptionMsg())
info "Metrics HTTP server started", serverIp= $serverIp, serverPort= $serverPort
ok(metricsServerRes.value)
proc publish(c: Chat, line: string) =
2021-03-09 07:23:53 +00:00
# First create a Chat2Message protobuf with this line of text
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
2022-02-04 23:58:27 +00:00
let time = getTime().toUnix()
let chat2pb = Chat2Message(timestamp: time,
2021-03-09 07:23:53 +00:00
nick: c.nick,
payload: line.toBytes()).encode()
## @TODO: error handling on failure
proc handler(response: PushResponse) {.gcsafe, closure.} =
trace "lightpush response received", response=response
var message = WakuMessage(payload: chat2pb.buffer,
contentTopic: c.contentTopic, version: 0, timestamp: getNanosecondTime(time))
if not isNil(c.node.wakuRlnRelay):
# for future version when we support more than one rln protected content topic,
# we should check the message content topic as well
let success = c.node.wakuRlnRelay.appendRLNProof(message, float64(time))
if not success:
debug "could not append rate limit proof to the message", success=success
else:
debug "rate limit proof is appended to the message", success=success
let decodeRes = RateLimitProof.init(message.proof)
if decodeRes.isErr():
error "could not decode the RLN proof"
let proof = decodeRes.get()
# TODO move it to log after dogfooding
let msgEpoch = fromEpoch(proof.epoch)
if fromEpoch(c.node.wakuRlnRelay.lastEpoch) == msgEpoch:
echo "--rln epoch: ", msgEpoch, " ⚠️ message rate violation! you are spamming the network!"
else:
echo "--rln epoch: ", msgEpoch
# update the last epoch
c.node.wakuRlnRelay.lastEpoch = proof.epoch
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
2022-02-04 23:58:27 +00:00
if not c.node.wakuLightPush.isNil():
# Attempt lightpush
asyncSpawn c.node.lightpushPublish(some(DefaultPubsubTopic), message)
else:
asyncSpawn c.node.publish(some(DefaultPubsubTopic), message)
# TODO This should read or be subscribe handler subscribe
proc readAndPrint(c: Chat) {.async.} =
while true:
# while p.connected:
# # TODO: echo &"{p.id} -> "
#
# echo cast[string](await p.conn.readLp(1024))
#echo "readAndPrint subscribe NYI"
await sleepAsync(100.millis)
# TODO Implement
proc writeAndPrint(c: Chat) {.async.} =
while true:
# Connect state not updated on incoming WakuRelay connections
# if not c.connected:
# echo "type an address or wait for a connection:"
# echo "type /[help|?] for help"
2021-03-03 08:40:19 +00:00
# Chat prompt
2021-03-08 07:45:10 +00:00
showChatPrompt(c)
2021-03-03 08:40:19 +00:00
let line = await c.transp.readLine()
if line.startsWith("/help") or line.startsWith("/?") or not c.started:
echo Help
continue
# if line.startsWith("/disconnect"):
# echo "Ending current session"
# if p.connected and p.conn.closed.not:
# await p.conn.close()
# p.connected = false
elif line.startsWith("/connect"):
# TODO Should be able to connect to multiple peers for Waku chat
if c.connected:
echo "already connected to at least one peer"
continue
echo "enter address of remote peer"
let address = await c.transp.readLine()
if address.len > 0:
await c.connectToNodes(@[address])
2021-03-03 08:40:19 +00:00
elif line.startsWith("/nick"):
# Set a new nickname
c.nick = await readNick(c.transp)
echo "You are now known as " & c.nick
elif line.startsWith("/exit"):
if not c.node.wakuFilterLegacy.isNil():
echo "unsubscribing from content filters..."
let peerOpt = c.node.peerManager.selectPeer(WakuLegacyFilterCodec)
if peerOpt.isSome():
await c.node.legacyFilterUnsubscribe(pubsubTopic=some(DefaultPubsubTopic), contentTopics=c.contentTopic, peer=peerOpt.get())
echo "quitting..."
await c.node.stop()
quit(QuitSuccess)
else:
# XXX connected state problematic
if c.started:
2021-03-09 07:23:53 +00:00
c.publish(line)
# TODO Connect to peer logic?
else:
try:
if line.startsWith("/") and "p2p" in line:
await c.connectToNodes(@[line])
except:
echo &"unable to dial remote peer {line}"
echo getCurrentExceptionMsg()
proc readWriteLoop(c: Chat) {.async.} =
asyncSpawn c.writeAndPrint() # execute the async function but does not block
asyncSpawn c.readAndPrint()
proc readInput(wfd: AsyncFD) {.thread, raises: [Defect, CatchableError].} =
## This procedure performs reading from `stdin` and sends data over
## pipe to main thread.
let transp = fromPipe(wfd)
while true:
let line = stdin.readLine()
discard waitFor transp.write(line & "\r\n")
{.pop.} # @TODO confutils.nim(775, 17) Error: can raise an unlisted exception: ref IOError
proc processInput(rfd: AsyncFD, rng: ref HmacDrbgContext) {.async.} =
let
2022-10-18 23:03:43 +00:00
transp = fromPipe(rfd)
conf = Chat2Conf.load()
nodekey = if conf.nodekey.isSome(): conf.nodekey.get()
else: PrivateKey.random(Secp256k1, rng[]).tryGet()
2022-10-18 23:03:43 +00:00
# set log level
if conf.logLevel != LogLevel.NONE:
setLogLevel(conf.logLevel)
let natRes = setupNat(conf.nat, clientId,
Port(uint16(conf.tcpPort) + conf.portsShift),
Port(uint16(conf.udpPort) + conf.portsShift))
if natRes.isErr():
raise newException(ValueError, "setupNat error " & natRes.error)
let (extIp, extTcpPort, extUdpPort) = natRes.get()
2023-08-15 00:43:10 +00:00
var enrBuilder = EnrBuilder.init(nodeKey)
let recordRes = enrBuilder.build()
let record =
if recordRes.isErr():
error "failed to create enr record", error=recordRes.error
quit(QuitFailure)
else: recordRes.get()
let node = block:
var builder = WakuNodeBuilder.init()
builder.withNodeKey(nodeKey)
2023-08-15 00:43:10 +00:00
builder.withRecord(record)
builder.withNetworkConfigurationDetails(conf.listenAddress, Port(uint16(conf.tcpPort) + conf.portsShift),
extIp, extTcpPort,
wsBindPort = Port(uint16(conf.websocketPort) + conf.portsShift),
wsEnabled = conf.websocketSupport,
wssEnabled = conf.websocketSecureSupport).tryGet()
builder.build().tryGet()
await node.start()
if conf.rlnRelayCredPath == "":
raise newException(ConfigurationError, "rln-relay-cred-path MUST be passed")
if conf.relay:
await node.mountRelay(conf.topics.split(" "))
await node.mountLibp2pPing()
2021-03-03 08:40:19 +00:00
let nick = await readNick(transp)
echo "Welcome, " & nick & "!"
var chat = Chat(node: node,
transp: transp,
subscribed: true,
connected: false,
started: true,
nick: nick,
prompt: false,
contentTopic: conf.contentTopic)
2022-02-22 12:36:38 +00:00
if conf.staticnodes.len > 0:
echo "Connecting to static peers..."
await connectToNodes(chat, conf.staticnodes)
2022-02-22 12:36:38 +00:00
var dnsDiscoveryUrl = none(string)
if conf.fleet != Fleet.none:
# Use DNS discovery to connect to selected fleet
echo "Connecting to " & $conf.fleet & " fleet using DNS discovery..."
2022-02-22 12:36:38 +00:00
if conf.fleet == Fleet.test:
dnsDiscoveryUrl = some("enrtree://AO47IDOLBKH72HIZZOXQP6NMRESAN7CHYWIBNXDXWRJRZWLODKII6@test.wakuv2.nodes.status.im")
2022-02-22 12:36:38 +00:00
else:
# Connect to prod by default
dnsDiscoveryUrl = some("enrtree://ANEDLO25QVUGJOUTQFRYKWX6P4Z4GKVESBMHML7DZ6YK4LGS5FC5O@prod.wakuv2.nodes.status.im")
2022-02-22 12:36:38 +00:00
elif conf.dnsDiscovery and conf.dnsDiscoveryUrl != "":
2022-02-22 12:36:38 +00:00
# No pre-selected fleet. Discover nodes via DNS using user config
debug "Discovering nodes using Waku DNS discovery", url=conf.dnsDiscoveryUrl
dnsDiscoveryUrl = some(conf.dnsDiscoveryUrl)
2022-02-22 12:36:38 +00:00
var discoveredNodes: seq[RemotePeerInfo]
2022-02-22 12:36:38 +00:00
if dnsDiscoveryUrl.isSome:
var nameServers: seq[TransportAddress]
for ip in conf.dnsDiscoveryNameServers:
nameServers.add(initTAddress(ip, Port(53))) # Assume all servers use port 53
2022-02-22 12:36:38 +00:00
let dnsResolver = DnsResolver.new(nameServers)
2021-03-03 08:40:19 +00:00
2022-02-22 12:36:38 +00:00
proc resolver(domain: string): Future[string] {.async, gcsafe.} =
trace "resolving", domain=domain
let resolved = await dnsResolver.resolveTxt(domain)
return resolved[0] # Use only first answer
2022-02-22 12:36:38 +00:00
var wakuDnsDiscovery = WakuDnsDiscovery.init(dnsDiscoveryUrl.get(),
resolver)
if wakuDnsDiscovery.isOk:
let discoveredPeers = wakuDnsDiscovery.get().findPeers()
if discoveredPeers.isOk:
info "Connecting to discovered peers"
discoveredNodes = discoveredPeers.get()
echo "Discovered and connecting to " & $discoveredNodes
waitFor chat.node.connectToNodes(discoveredNodes)
else:
2022-02-22 12:36:38 +00:00
warn "Failed to init Waku DNS discovery"
2022-01-10 15:07:35 +00:00
let peerInfo = node.switch.peerInfo
let listenStr = $peerInfo.addrs[0] & "/p2p/" & $peerInfo.peerId
echo &"Listening on\n {listenStr}"
2021-03-04 07:19:21 +00:00
if (conf.storenode != "") or (conf.store == true):
await node.mountStore()
2022-02-22 12:36:38 +00:00
var storenode: Option[RemotePeerInfo]
2021-03-04 07:19:21 +00:00
if conf.storenode != "":
let peerInfo = parsePeerInfo(conf.storenode)
if peerInfo.isOk():
storenode = some(peerInfo.value)
else:
error "Incorrect conf.storenode", error = peerInfo.error
2022-02-22 12:36:38 +00:00
elif discoveredNodes.len > 0:
echo "Store enabled, but no store nodes configured. Choosing one at random from discovered peers"
storenode = some(discoveredNodes[rand(0..len(discoveredNodes) - 1)])
if storenode.isSome():
# We have a viable storenode. Let's query it for historical messages.
2022-02-22 12:36:38 +00:00
echo "Connecting to storenode: " & $(storenode.get())
node.mountStoreClient()
node.peerManager.addServicePeer(storenode.get(), WakuStoreCodec)
proc storeHandler(response: HistoryResponse) {.gcsafe.} =
for msg in response.messages:
let
pb = Chat2Message.init(msg.payload)
chatLine = if pb.isOk: pb[].toString()
else: string.fromBytes(msg.payload)
echo &"{chatLine}"
info "Hit store handler"
let queryRes = await node.query(HistoryQuery(contentTopics: @[chat.contentTopic]))
if queryRes.isOk():
storeHandler(queryRes.value)
# NOTE Must be mounted after relay
if conf.lightpushnode != "":
let peerInfo = parsePeerInfo(conf.lightpushnode)
if peerInfo.isOk():
await mountLightPush(node)
node.mountLightPushClient()
node.peerManager.addServicePeer(peerInfo.value, WakuLightpushCodec)
else:
error "LightPush not mounted. Couldn't parse conf.lightpushnode",
error = peerInfo.error
if conf.filternode != "":
let peerInfo = parsePeerInfo(conf.filternode)
if peerInfo.isOk():
await node.mountFilter()
await node.mountFilterClient()
node.peerManager.addServicePeer(peerInfo.value, WakuLegacyFilterCodec)
proc filterHandler(pubsubTopic: PubsubTopic, msg: WakuMessage) {.async, gcsafe, closure.} =
trace "Hit filter handler", contentTopic=msg.contentTopic
chat.printReceivedMessage(msg)
await node.legacyFilterSubscribe(pubsubTopic=some(DefaultPubsubTopic),
contentTopics=chat.contentTopic,
filterHandler,
peerInfo.value)
# TODO: Here to support FilterV2 relevant subscription, but still
# Legacy Filter is concurrent to V2 untill legacy filter will be removed
else:
error "Filter not mounted. Couldn't parse conf.filternode",
error = peerInfo.error
# Subscribe to a topic, if relay is mounted
if conf.relay:
proc handler(topic: PubsubTopic, msg: WakuMessage): Future[void] {.async, gcsafe.} =
trace "Hit subscribe handler", topic
if msg.contentTopic == chat.contentTopic:
chat.printReceivedMessage(msg)
node.subscribe((kind: PubsubSub, topic: DefaultPubsubTopic), some(handler))
if conf.rlnRelay:
info "WakuRLNRelay is enabled"
proc spamHandler(wakuMessage: WakuMessage) {.gcsafe, closure.} =
debug "spam handler is called"
let chatLineResult = chat.getChatLine(wakuMessage)
if chatLineResult.isOk():
echo "A spam message is found and discarded : ", chatLineResult.value
else:
echo "A spam message is found and discarded"
chat.prompt = false
showChatPrompt(chat)
echo "rln-relay preparation is in progress..."
let rlnConf = WakuRlnConfig(
rlnRelayDynamic: conf.rlnRelayDynamic,
rlnRelayCredIndex: conf.rlnRelayCredIndex,
rlnRelayEthContractAddress: conf.rlnRelayEthContractAddress,
rlnRelayEthClientAddress: conf.rlnRelayEthClientAddress,
rlnRelayCredPath: conf.rlnRelayCredPath,
rlnRelayCredPassword: conf.rlnRelayCredPassword
)
waitFor node.mountRlnRelay(rlnConf,
spamHandler=some(spamHandler))
let membershipIndex = node.wakuRlnRelay.groupManager.membershipIndex.get()
let identityCredential = node.wakuRlnRelay.groupManager.idCredentials.get()
echo "your membership index is: ", membershipIndex
echo "your rln identity commitment key is: ", identityCredential.idCommitment.inHex()
else:
info "WakuRLNRelay is disabled"
echo "WakuRLNRelay is disabled, please enable it by passing in the --rln-relay flag"
if conf.metricsLogging:
startMetricsLog()
if conf.metricsServer:
let metricsServer = startMetricsServer(
conf.metricsServerAddress,
Port(conf.metricsServerPort + conf.portsShift)
)
await chat.readWriteLoop()
if conf.keepAlive:
node.startKeepalive()
runForever()
proc main(rng: ref HmacDrbgContext) {.async.} =
let (rfd, wfd) = createAsyncPipe()
if rfd == asyncInvalidPipe or wfd == asyncInvalidPipe:
raise newException(ValueError, "Could not initialize pipe!")
var thread: Thread[AsyncFD]
thread.createThread(readInput, wfd)
try:
await processInput(rfd, rng)
# Handle only ConfigurationError for now
# TODO: Throw other errors from the mounting procedure
except ConfigurationError as e:
raise e
when isMainModule: # isMainModule = true when the module is compiled as the main file
let rng = crypto.newRng()
try:
waitFor(main(rng))
except CatchableError as e:
raise e
## Dump of things that can be improved:
##
## - Incoming dialed peer does not change connected state (not relying on it for now)
## - Unclear if staticnode argument works (can enter manually)
## - Don't trigger self / double publish own messages
## - Integrate store protocol (fetch messages in beginning)
## - Integrate filter protocol (default/option to be light node, connect to filter node)
## - Test/default to cluster node connection (diff protocol version)
## - Redirect logs to separate file
## - Expose basic publish/subscribe etc commands with /syntax
## - Show part of peerid to know who sent message
## - Deal with protobuf messages (e.g. other chat protocol, or encrypted)