2021-07-14 17:58:46 +00:00
|
|
|
{.push raises: [Defect].}
|
|
|
|
|
2020-04-29 04:49:27 +00:00
|
|
|
import
|
2022-10-18 14:05:53 +00:00
|
|
|
std/[options, tables, strutils, sequtils, os],
|
2021-06-09 14:37:08 +00:00
|
|
|
stew/shims/net as stewNet,
|
2022-10-18 17:35:26 +00:00
|
|
|
chronicles,
|
|
|
|
chronos,
|
|
|
|
metrics,
|
|
|
|
confutils,
|
|
|
|
toml_serialization,
|
|
|
|
system/ansi_c,
|
2020-09-01 02:09:54 +00:00
|
|
|
eth/keys,
|
2021-08-12 08:51:38 +00:00
|
|
|
eth/p2p/discoveryv5/enr,
|
2022-10-18 17:35:26 +00:00
|
|
|
libp2p/[builders, multihash],
|
2020-05-15 04:11:14 +00:00
|
|
|
libp2p/crypto/crypto,
|
2021-06-15 08:55:47 +00:00
|
|
|
libp2p/protocols/ping,
|
2022-01-10 14:07:01 +00:00
|
|
|
libp2p/protocols/pubsub/[gossipsub, rpc/messages],
|
2022-10-18 17:35:26 +00:00
|
|
|
libp2p/transports/[transport, wstransport],
|
|
|
|
libp2p/nameresolving/dnsresolver
|
2022-07-25 11:01:37 +00:00
|
|
|
import
|
2022-10-18 17:35:26 +00:00
|
|
|
../../waku/v2/protocol/waku_store,
|
|
|
|
../../waku/v2/protocol/waku_filter,
|
|
|
|
../../waku/v2/protocol/waku_peer_exchange,
|
|
|
|
../../waku/v2/node/peer_manager/peer_manager,
|
|
|
|
../../waku/v2/node/dnsdisc/waku_dnsdisc,
|
|
|
|
../../waku/v2/node/discv5/waku_discv5,
|
|
|
|
../../waku/v2/node/storage/sqlite,
|
|
|
|
../../waku/v2/node/storage/peer/waku_peer_storage,
|
|
|
|
../../waku/v2/node/storage/message/waku_store_queue,
|
|
|
|
../../waku/v2/node/storage/message/dual_message_store,
|
|
|
|
../../waku/v2/node/storage/message/sqlite_store,
|
|
|
|
../../waku/v2/node/storage/message/message_retention_policy_capacity,
|
|
|
|
../../waku/v2/node/storage/message/message_retention_policy_time,
|
2022-10-21 08:33:36 +00:00
|
|
|
../../waku/v2/node/[wakuswitch, waku_node, waku_metrics],
|
|
|
|
../../waku/v2/utils/[peers, wakuenr],
|
2022-10-18 17:35:26 +00:00
|
|
|
../../waku/common/utils/nat,
|
|
|
|
./wakunode2_setup_rest,
|
|
|
|
./wakunode2_setup_rpc,
|
|
|
|
./wakunode2_setup_sql_migrations,
|
|
|
|
./config
|
|
|
|
|
|
|
|
when defined(rln) or defined(rlnzerokit):
|
|
|
|
import
|
|
|
|
../../waku/v2/protocol/waku_rln_relay/waku_rln_relay_types,
|
|
|
|
../../waku/v2/protocol/waku_rln_relay/waku_rln_relay_utils
|
|
|
|
|
|
|
|
|
|
|
|
logScope:
|
|
|
|
topics = "wakunode.setup"
|
2020-10-21 09:54:29 +00:00
|
|
|
|
2021-07-22 09:46:54 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
type SetupResult[T] = Result[T, string]
|
2022-10-18 17:35:26 +00:00
|
|
|
|
2021-07-22 09:46:54 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
proc setupStorage(conf: WakuNodeConf):
|
|
|
|
SetupResult[tuple[pStorage: WakuPeerStorage, mStorage: MessageStore]] =
|
2021-07-22 09:46:54 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
## Setup a SQLite Database for a wakunode based on a supplied
|
|
|
|
## configuration file and perform all necessary migration.
|
|
|
|
##
|
|
|
|
## If config allows, return peer storage and message store
|
|
|
|
## for use elsewhere.
|
|
|
|
|
|
|
|
var
|
|
|
|
sqliteDatabase: SqliteDatabase
|
|
|
|
storeTuple: tuple[pStorage: WakuPeerStorage, mStorage: MessageStore]
|
|
|
|
|
|
|
|
# Setup database connection
|
|
|
|
if conf.dbPath != "":
|
|
|
|
let dbRes = SqliteDatabase.init(conf.dbPath)
|
|
|
|
if dbRes.isErr():
|
|
|
|
warn "failed to init database connection", err = dbRes.error
|
|
|
|
waku_node_errors.inc(labelValues = ["init_db_failure"])
|
|
|
|
return err("failed to init database connection")
|
|
|
|
else:
|
|
|
|
sqliteDatabase = dbRes.value
|
|
|
|
|
|
|
|
|
|
|
|
if not sqliteDatabase.isNil():
|
|
|
|
## Database vacuuming
|
|
|
|
# TODO: Wrap and move this logic to the appropriate module
|
|
|
|
let
|
|
|
|
pageSize = ?sqliteDatabase.getPageSize()
|
|
|
|
pageCount = ?sqliteDatabase.getPageCount()
|
|
|
|
freelistCount = ?sqliteDatabase.getFreelistCount()
|
2022-09-20 09:39:52 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
debug "sqlite database page stats", pageSize=pageSize, pages=pageCount, freePages=freelistCount
|
2022-09-14 16:09:08 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
# TODO: Run vacuuming conditionally based on database page stats
|
|
|
|
if conf.dbVacuum and (pageCount > 0 and freelistCount > 0):
|
|
|
|
debug "starting sqlite database vacuuming"
|
2022-09-14 16:09:08 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
let resVacuum = sqliteDatabase.vacuum()
|
|
|
|
if resVacuum.isErr():
|
|
|
|
return err("failed to execute vacuum: " & resVacuum.error())
|
2022-09-14 16:09:08 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
debug "finished sqlite database vacuuming"
|
2022-09-14 16:09:08 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
sqliteDatabase.runMigrations(conf)
|
2022-09-14 16:09:08 +00:00
|
|
|
|
2022-09-20 09:39:52 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
if conf.persistPeers:
|
|
|
|
let res = WakuPeerStorage.new(sqliteDatabase)
|
|
|
|
if res.isErr():
|
|
|
|
warn "failed to init peer store", err = res.error
|
|
|
|
waku_node_errors.inc(labelValues = ["init_store_failure"])
|
|
|
|
else:
|
|
|
|
storeTuple.pStorage = res.value
|
2021-07-22 09:46:54 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
if conf.persistMessages:
|
|
|
|
if conf.sqliteStore:
|
|
|
|
debug "setting up sqlite-only store"
|
|
|
|
let res = SqliteStore.init(sqliteDatabase)
|
2022-09-20 09:39:52 +00:00
|
|
|
if res.isErr():
|
2022-10-21 13:01:01 +00:00
|
|
|
warn "failed to init message store", err = res.error
|
2022-09-20 09:39:52 +00:00
|
|
|
waku_node_errors.inc(labelValues = ["init_store_failure"])
|
|
|
|
else:
|
2022-10-21 13:01:01 +00:00
|
|
|
storeTuple.mStorage = res.value
|
|
|
|
elif not sqliteDatabase.isNil():
|
|
|
|
debug "setting up dual message store"
|
|
|
|
let res = DualMessageStore.init(sqliteDatabase, conf.storeCapacity)
|
|
|
|
if res.isErr():
|
|
|
|
warn "failed to init message store", err = res.error
|
|
|
|
waku_node_errors.inc(labelValues = ["init_store_failure"])
|
2022-10-04 13:34:05 +00:00
|
|
|
else:
|
2022-10-21 13:01:01 +00:00
|
|
|
storeTuple.mStorage = res.value
|
|
|
|
else:
|
|
|
|
debug "setting up in-memory store"
|
|
|
|
storeTuple.mStorage = StoreQueueRef.new(conf.storeCapacity)
|
2022-09-20 09:39:52 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
ok(storeTuple)
|
2021-07-22 09:46:54 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
proc retrieveDynamicBootstrapNodes(conf: WakuNodeConf): SetupResult[seq[RemotePeerInfo]] =
|
|
|
|
|
|
|
|
if conf.dnsDiscovery and conf.dnsDiscoveryUrl != "":
|
|
|
|
# DNS discovery
|
|
|
|
debug "Discovering nodes using Waku DNS discovery", url=conf.dnsDiscoveryUrl
|
|
|
|
|
|
|
|
var nameServers: seq[TransportAddress]
|
|
|
|
for ip in conf.dnsDiscoveryNameServers:
|
|
|
|
nameServers.add(initTAddress(ip, Port(53))) # Assume all servers use port 53
|
|
|
|
|
|
|
|
let dnsResolver = DnsResolver.new(nameServers)
|
|
|
|
|
|
|
|
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
|
|
|
|
|
|
|
|
var wakuDnsDiscovery = WakuDnsDiscovery.init(conf.dnsDiscoveryUrl,
|
|
|
|
resolver)
|
|
|
|
if wakuDnsDiscovery.isOk:
|
|
|
|
return wakuDnsDiscovery.get().findPeers()
|
|
|
|
.mapErr(proc (e: cstring): string = $e)
|
|
|
|
else:
|
|
|
|
warn "Failed to init Waku DNS discovery"
|
|
|
|
|
|
|
|
debug "No method for retrieving dynamic bootstrap nodes specified."
|
|
|
|
ok(newSeq[RemotePeerInfo]()) # Return an empty seq by default
|
|
|
|
|
|
|
|
proc initNode(conf: WakuNodeConf,
|
|
|
|
pStorage: WakuPeerStorage = nil,
|
|
|
|
dynamicBootstrapNodes: openArray[RemotePeerInfo] = @[]): SetupResult[WakuNode] =
|
|
|
|
|
|
|
|
## Setup a basic Waku v2 node based on a supplied configuration
|
|
|
|
## file. Optionally include persistent peer storage.
|
|
|
|
## No protocols are mounted yet.
|
|
|
|
|
|
|
|
var dnsResolver: DnsResolver
|
|
|
|
if conf.dnsAddrs:
|
|
|
|
# Support for DNS multiaddrs
|
|
|
|
var nameServers: seq[TransportAddress]
|
|
|
|
for ip in conf.dnsAddrsNameServers:
|
|
|
|
nameServers.add(initTAddress(ip, Port(53))) # Assume all servers use port 53
|
2022-06-08 09:20:18 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
dnsResolver = DnsResolver.new(nameServers)
|
|
|
|
|
|
|
|
let
|
|
|
|
## `udpPort` is only supplied to satisfy underlying APIs but is not
|
|
|
|
## actually a supported transport for libp2p traffic.
|
|
|
|
udpPort = conf.tcpPort
|
|
|
|
(extIp, extTcpPort, extUdpPort) = setupNat(conf.nat,
|
|
|
|
clientId,
|
|
|
|
Port(uint16(conf.tcpPort) + conf.portsShift),
|
|
|
|
Port(uint16(udpPort) + conf.portsShift))
|
|
|
|
|
|
|
|
dns4DomainName = if conf.dns4DomainName != "": some(conf.dns4DomainName)
|
|
|
|
else: none(string)
|
2021-07-22 09:46:54 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
discv5UdpPort = if conf.discv5Discovery: some(Port(uint16(conf.discv5UdpPort) + conf.portsShift))
|
|
|
|
else: none(Port)
|
|
|
|
|
|
|
|
## @TODO: the NAT setup assumes a manual port mapping configuration if extIp config is set. This probably
|
|
|
|
## implies adding manual config item for extPort as well. The following heuristic assumes that, in absence of manual
|
|
|
|
## config, the external port is the same as the bind port.
|
|
|
|
extPort = if (extIp.isSome() or dns4DomainName.isSome()) and extTcpPort.isNone():
|
|
|
|
some(Port(uint16(conf.tcpPort) + conf.portsShift))
|
|
|
|
else:
|
|
|
|
extTcpPort
|
2022-02-16 16:12:09 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
wakuFlags = initWakuFlags(conf.lightpush,
|
|
|
|
conf.filter,
|
|
|
|
conf.store,
|
|
|
|
conf.relay)
|
|
|
|
|
|
|
|
var node: WakuNode
|
|
|
|
try:
|
|
|
|
node = WakuNode.new(conf.nodekey,
|
|
|
|
conf.listenAddress, Port(uint16(conf.tcpPort) + conf.portsShift),
|
|
|
|
extIp, extPort,
|
|
|
|
pStorage,
|
|
|
|
conf.maxConnections.int,
|
|
|
|
Port(uint16(conf.websocketPort) + conf.portsShift),
|
|
|
|
conf.websocketSupport,
|
|
|
|
conf.websocketSecureSupport,
|
|
|
|
conf.websocketSecureKeyPath,
|
|
|
|
conf.websocketSecureCertPath,
|
|
|
|
some(wakuFlags),
|
|
|
|
dnsResolver,
|
|
|
|
conf.relayPeerExchange, # We send our own signed peer record when peer exchange enabled
|
|
|
|
dns4DomainName,
|
|
|
|
discv5UdpPort
|
|
|
|
)
|
|
|
|
except:
|
|
|
|
return err("failed to create waku node instance: " & getCurrentExceptionMsg())
|
|
|
|
|
|
|
|
if conf.discv5Discovery:
|
|
|
|
let
|
|
|
|
discoveryConfig = DiscoveryConfig.init(
|
|
|
|
conf.discv5TableIpLimit, conf.discv5BucketIpLimit, conf.discv5BitsPerHop)
|
|
|
|
|
|
|
|
# select dynamic bootstrap nodes that have an ENR containing a udp port.
|
|
|
|
# Discv5 only supports UDP https://github.com/ethereum/devp2p/blob/master/discv5/discv5-theory.md)
|
|
|
|
var discv5BootstrapEnrs: seq[enr.Record]
|
|
|
|
for n in dynamicBootstrapNodes:
|
|
|
|
if n.enr.isSome():
|
|
|
|
let
|
|
|
|
enr = n.enr.get()
|
|
|
|
tenrRes = enr.toTypedRecord()
|
|
|
|
if tenrRes.isOk() and (tenrRes.get().udp.isSome() or tenrRes.get().udp6.isSome()):
|
|
|
|
discv5BootstrapEnrs.add(enr)
|
|
|
|
|
|
|
|
# parse enrURIs from the configuration and add the resulting ENRs to the discv5BootstrapEnrs seq
|
|
|
|
for enrUri in conf.discv5BootstrapNodes:
|
|
|
|
addBootstrapNode(enrUri, discv5BootstrapEnrs)
|
|
|
|
|
|
|
|
node.wakuDiscv5 = WakuDiscoveryV5.new(
|
|
|
|
extIP, extPort, discv5UdpPort,
|
|
|
|
conf.listenAddress,
|
|
|
|
discv5UdpPort.get(),
|
|
|
|
discv5BootstrapEnrs,
|
|
|
|
conf.discv5EnrAutoUpdate,
|
|
|
|
keys.PrivateKey(conf.nodekey.skkey),
|
|
|
|
wakuFlags,
|
|
|
|
[], # Empty enr fields, for now
|
|
|
|
node.rng,
|
|
|
|
discoveryConfig
|
|
|
|
)
|
|
|
|
|
|
|
|
ok(node)
|
|
|
|
|
|
|
|
proc setupProtocols(node: WakuNode, conf: WakuNodeConf, mStorage: MessageStore): Future[SetupResult[void]] {.async.} =
|
|
|
|
## Setup configured protocols on an existing Waku v2 node.
|
|
|
|
## Optionally include persistent message storage.
|
|
|
|
## No protocols are started yet.
|
|
|
|
|
|
|
|
# Mount relay on all nodes
|
|
|
|
var peerExchangeHandler = none(RoutingRecordsHandler)
|
|
|
|
if conf.relayPeerExchange:
|
|
|
|
proc handlePeerExchange(peer: PeerId, topic: string,
|
|
|
|
peers: seq[RoutingRecordsPair]) {.gcsafe, raises: [Defect].} =
|
|
|
|
## Handle peers received via gossipsub peer exchange
|
|
|
|
# TODO: Only consider peers on pubsub topics we subscribe to
|
|
|
|
let exchangedPeers = peers.filterIt(it.record.isSome()) # only peers with populated records
|
|
|
|
.mapIt(toRemotePeerInfo(it.record.get()))
|
2021-12-06 19:51:37 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
debug "connecting to exchanged peers", src=peer, topic=topic, numPeers=exchangedPeers.len
|
2021-07-22 09:46:54 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
# asyncSpawn, as we don't want to block here
|
|
|
|
asyncSpawn node.connectToNodes(exchangedPeers, "peer exchange")
|
|
|
|
|
|
|
|
peerExchangeHandler = some(handlePeerExchange)
|
2022-03-29 08:09:48 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
if conf.relay:
|
|
|
|
try:
|
|
|
|
let pubsubTopics = conf.topics.split(" ")
|
|
|
|
await mountRelay(node, pubsubTopics, peerExchangeHandler = peerExchangeHandler)
|
|
|
|
except:
|
|
|
|
return err("failed to mount waku relay protocol: " & getCurrentExceptionMsg())
|
|
|
|
|
|
|
|
|
|
|
|
# Keepalive mounted on all nodes
|
|
|
|
try:
|
|
|
|
await mountLibp2pPing(node)
|
|
|
|
except:
|
|
|
|
return err("failed to mount libp2p ping protocol: " & getCurrentExceptionMsg())
|
|
|
|
|
|
|
|
when defined(rln) or defined(rlnzerokit):
|
|
|
|
if conf.rlnRelay:
|
|
|
|
try:
|
2022-08-30 17:59:02 +00:00
|
|
|
let res = node.mountRlnRelay(conf)
|
|
|
|
if res.isErr():
|
2022-10-21 13:01:01 +00:00
|
|
|
return err("failed to mount waku RLN relay protocol: " & res.error)
|
|
|
|
except:
|
|
|
|
return err("failed to mount waku RLN relay protocol: " & getCurrentExceptionMsg())
|
|
|
|
|
|
|
|
if conf.swap:
|
|
|
|
try:
|
|
|
|
await mountSwap(node)
|
|
|
|
# TODO: Set swap peer, for now should be same as store peer
|
|
|
|
except:
|
|
|
|
return err("failed to mount waku swap protocol: " & getCurrentExceptionMsg())
|
|
|
|
|
|
|
|
# Store setup
|
|
|
|
if (conf.storenode != "") or (conf.store):
|
|
|
|
let retentionPolicy = if conf.sqliteStore: TimeRetentionPolicy.init(conf.sqliteRetentionTime)
|
|
|
|
else: CapacityRetentionPolicy.init(conf.storeCapacity)
|
|
|
|
|
|
|
|
try:
|
|
|
|
await mountStore(node, mStorage, retentionPolicy=some(retentionPolicy))
|
|
|
|
except:
|
|
|
|
return err("failed to mount waku store protocol: " & getCurrentExceptionMsg())
|
|
|
|
|
|
|
|
executeMessageRetentionPolicy(node)
|
|
|
|
startMessageRetentionPolicyPeriodicTask(node, interval=MessageStoreDefaultRetentionPolicyInterval)
|
|
|
|
|
|
|
|
if conf.storenode != "":
|
|
|
|
try:
|
2021-07-22 09:46:54 +00:00
|
|
|
setStorePeer(node, conf.storenode)
|
2022-10-21 13:01:01 +00:00
|
|
|
except:
|
|
|
|
return err("failed to set node waku store peer: " & getCurrentExceptionMsg())
|
|
|
|
|
|
|
|
# NOTE Must be mounted after relay
|
|
|
|
if (conf.lightpushnode != "") or (conf.lightpush):
|
|
|
|
try:
|
|
|
|
await mountLightPush(node)
|
|
|
|
except:
|
|
|
|
return err("failed to mount waku lightpush protocol: " & getCurrentExceptionMsg())
|
|
|
|
|
|
|
|
if conf.lightpushnode != "":
|
|
|
|
try:
|
2021-07-22 09:46:54 +00:00
|
|
|
setLightPushPeer(node, conf.lightpushnode)
|
2022-10-21 13:01:01 +00:00
|
|
|
except:
|
|
|
|
return err("failed to set node waku lightpush peer: " & getCurrentExceptionMsg())
|
|
|
|
|
|
|
|
# Filter setup. NOTE Must be mounted after relay
|
|
|
|
if (conf.filternode != "") or (conf.filter):
|
|
|
|
try:
|
|
|
|
await mountFilter(node, filterTimeout = chronos.seconds(conf.filterTimeout))
|
|
|
|
except:
|
|
|
|
return err("failed to mount waku filter protocol: " & getCurrentExceptionMsg())
|
|
|
|
|
|
|
|
if conf.filternode != "":
|
|
|
|
try:
|
2021-07-22 09:46:54 +00:00
|
|
|
setFilterPeer(node, conf.filternode)
|
2022-10-21 13:01:01 +00:00
|
|
|
except:
|
|
|
|
return err("failed to set node waku filter peer: " & getCurrentExceptionMsg())
|
|
|
|
|
|
|
|
# waku peer exchange setup
|
|
|
|
if (conf.peerExchangeNode != "") or (conf.peerExchange):
|
|
|
|
try:
|
|
|
|
await mountWakuPeerExchange(node)
|
|
|
|
except:
|
|
|
|
return err("failed to mount waku peer-exchange protocol: " & getCurrentExceptionMsg())
|
2022-09-20 11:03:34 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
if conf.peerExchangeNode != "":
|
|
|
|
try:
|
2022-09-20 11:03:34 +00:00
|
|
|
setPeerExchangePeer(node, conf.peerExchangeNode)
|
2022-10-21 13:01:01 +00:00
|
|
|
except:
|
|
|
|
return err("failed to set node waku lightpush peer: " & getCurrentExceptionMsg())
|
2022-09-20 11:03:34 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
return ok()
|
2022-03-01 14:11:56 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
proc startNode(node: WakuNode, conf: WakuNodeConf,
|
|
|
|
dynamicBootstrapNodes: seq[RemotePeerInfo] = @[]): Future[SetupResult[void]] {.async.} =
|
|
|
|
## Start a configured node and all mounted protocols.
|
|
|
|
## Resume history, connect to static nodes and start
|
|
|
|
## keep-alive, if configured.
|
2021-11-15 13:29:18 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
# Start Waku v2 node
|
|
|
|
try:
|
|
|
|
await node.start()
|
|
|
|
except:
|
|
|
|
return err("failed to start waku node: " & getCurrentExceptionMsg())
|
|
|
|
|
|
|
|
# Start discv5 and connect to discovered nodes
|
|
|
|
if conf.discv5Discovery:
|
|
|
|
try:
|
|
|
|
if not await node.startDiscv5():
|
|
|
|
error "could not start Discovery v5"
|
|
|
|
except:
|
|
|
|
return err("failed to start waku discovery v5: " & getCurrentExceptionMsg())
|
2022-09-20 11:03:34 +00:00
|
|
|
|
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
# Resume historical messages, this has to be called after the node has been started
|
|
|
|
if conf.store and conf.persistMessages:
|
|
|
|
try:
|
|
|
|
await node.resume()
|
|
|
|
except:
|
|
|
|
return err("failed to resume messages history: " & getCurrentExceptionMsg())
|
|
|
|
|
|
|
|
# Connect to configured static nodes
|
|
|
|
if conf.staticnodes.len > 0:
|
|
|
|
try:
|
|
|
|
await connectToNodes(node, conf.staticnodes, "static")
|
|
|
|
except:
|
|
|
|
return err("failed to connect to static nodes: " & getCurrentExceptionMsg())
|
|
|
|
|
|
|
|
if dynamicBootstrapNodes.len > 0:
|
|
|
|
info "Connecting to dynamic bootstrap peers"
|
|
|
|
try:
|
|
|
|
await connectToNodes(node, dynamicBootstrapNodes, "dynamic bootstrap")
|
|
|
|
except:
|
|
|
|
return err("failed to connect to dynamic bootstrap nodes: " & getCurrentExceptionMsg())
|
2021-07-22 09:46:54 +00:00
|
|
|
|
2020-11-24 04:53:42 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
# retrieve and connect to peer exchange peers
|
|
|
|
if conf.peerExchangeNode != "":
|
|
|
|
info "Retrieving peer info via peer exchange protocol"
|
|
|
|
let desiredOutDegree = node.wakuRelay.parameters.d.uint64()
|
|
|
|
try:
|
|
|
|
discard await node.wakuPeerExchange.request(desiredOutDegree)
|
|
|
|
except:
|
|
|
|
return err("failed to retrieve peer info via peer exchange protocol: " & getCurrentExceptionMsg())
|
|
|
|
|
|
|
|
# Start keepalive, if enabled
|
|
|
|
if conf.keepAlive:
|
|
|
|
node.startKeepalive()
|
|
|
|
|
|
|
|
return ok()
|
|
|
|
|
|
|
|
proc startExternal(node: WakuNode, conf: WakuNodeConf): SetupResult[void] =
|
|
|
|
## Start configured external interfaces and monitoring tools
|
|
|
|
## on a Waku v2 node, including the RPC API, REST API and metrics
|
|
|
|
## monitoring ports.
|
|
|
|
|
|
|
|
if conf.rpc:
|
|
|
|
try:
|
2022-07-17 14:25:21 +00:00
|
|
|
startRpcServer(node, conf.rpcAddress, Port(conf.rpcPort + conf.portsShift), conf)
|
2022-10-21 13:01:01 +00:00
|
|
|
except:
|
|
|
|
return err("failed to start the json-rpc server: " & getCurrentExceptionMsg())
|
|
|
|
|
|
|
|
if conf.rest:
|
|
|
|
startRestServer(node, conf.restAddress, Port(conf.restPort + conf.portsShift), conf)
|
2021-05-13 21:21:46 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
if conf.metricsLogging:
|
|
|
|
startMetricsLog()
|
|
|
|
|
|
|
|
if conf.metricsServer:
|
|
|
|
startMetricsServer(conf.metricsServerAddress,
|
|
|
|
Port(conf.metricsServerPort + conf.portsShift))
|
|
|
|
|
|
|
|
ok()
|
2021-07-22 09:46:54 +00:00
|
|
|
|
2022-05-17 15:48:08 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
{.pop.} # @TODO confutils.nim(775, 17) Error: can raise an unlisted exception: ref IOError
|
|
|
|
when isMainModule:
|
|
|
|
## Node setup happens in 6 phases:
|
|
|
|
## 1. Set up storage
|
|
|
|
## 2. Initialize node
|
|
|
|
## 3. Mount and initialize configured protocols
|
|
|
|
## 4. Start node and mounted protocols
|
|
|
|
## 5. Start monitoring tools and external interfaces
|
|
|
|
## 6. Setup graceful shutdown hooks
|
|
|
|
|
2022-05-17 15:48:08 +00:00
|
|
|
{.push warning[ProveInit]: off.}
|
|
|
|
let conf = try:
|
|
|
|
WakuNodeConf.load(
|
|
|
|
secondarySources = proc (conf: WakuNodeConf, sources: auto) =
|
|
|
|
if conf.configFile.isSome:
|
|
|
|
sources.addConfigFile(Toml, conf.configFile.get)
|
|
|
|
)
|
2022-10-21 13:01:01 +00:00
|
|
|
except CatchableError:
|
|
|
|
error "Failure while loading the configuration: \n", error=getCurrentExceptionMsg()
|
2022-05-17 15:48:08 +00:00
|
|
|
quit 1 # if we don't leave here, the initialization of conf does not work in the success case
|
|
|
|
{.pop.}
|
|
|
|
|
2022-05-17 20:11:07 +00:00
|
|
|
# if called with --version, print the version and quit
|
|
|
|
if conf.version:
|
|
|
|
echo "version / git commit hash: ", git_version
|
|
|
|
quit(QuitSuccess)
|
2022-10-18 23:03:43 +00:00
|
|
|
|
|
|
|
# set log level
|
|
|
|
if conf.logLevel != LogLevel.NONE:
|
|
|
|
setLogLevel(conf.logLevel)
|
2022-05-17 20:11:07 +00:00
|
|
|
|
2021-07-22 09:46:54 +00:00
|
|
|
var
|
|
|
|
node: WakuNode # This is the node we're going to setup using the conf
|
2021-05-04 12:11:41 +00:00
|
|
|
|
2021-07-22 09:46:54 +00:00
|
|
|
##############
|
|
|
|
# Node setup #
|
|
|
|
##############
|
2022-03-01 14:11:56 +00:00
|
|
|
|
2022-03-17 16:33:17 +00:00
|
|
|
debug "1/7 Setting up storage"
|
2022-03-01 14:11:56 +00:00
|
|
|
|
2021-07-22 09:46:54 +00:00
|
|
|
var
|
|
|
|
pStorage: WakuPeerStorage
|
2022-09-20 09:39:52 +00:00
|
|
|
mStorage: MessageStore
|
2021-04-29 04:54:31 +00:00
|
|
|
|
2021-07-22 09:46:54 +00:00
|
|
|
let setupStorageRes = setupStorage(conf)
|
2022-10-21 13:01:01 +00:00
|
|
|
if setupStorageRes.isErr():
|
|
|
|
error "1/7 Setting up storage failed. Continuing without storage.", error=setupStorageRes.error
|
2021-07-22 09:46:54 +00:00
|
|
|
else:
|
|
|
|
(pStorage, mStorage) = setupStorageRes.get()
|
|
|
|
|
2022-03-17 16:33:17 +00:00
|
|
|
debug "2/7 Retrieve dynamic bootstrap nodes"
|
|
|
|
|
|
|
|
var dynamicBootstrapNodes: seq[RemotePeerInfo]
|
|
|
|
let dynamicBootstrapNodesRes = retrieveDynamicBootstrapNodes(conf)
|
2022-10-21 13:01:01 +00:00
|
|
|
if dynamicBootstrapNodesRes.isErr():
|
|
|
|
warn "2/7 Retrieving dynamic bootstrap nodes failed. Continuing without dynamic bootstrap nodes.", error=dynamicBootstrapNodesRes.error
|
2022-03-17 16:33:17 +00:00
|
|
|
else:
|
|
|
|
dynamicBootstrapNodes = dynamicBootstrapNodesRes.get()
|
|
|
|
|
|
|
|
debug "3/7 Initializing node"
|
2021-07-22 09:46:54 +00:00
|
|
|
|
2022-03-17 16:33:17 +00:00
|
|
|
let initNodeRes = initNode(conf, pStorage, dynamicBootstrapNodes)
|
2022-10-21 13:01:01 +00:00
|
|
|
if initNodeRes.isErr():
|
|
|
|
error "3/7 Initializing node failed. Quitting.", error=initNodeRes.error
|
2021-07-22 09:46:54 +00:00
|
|
|
quit(QuitFailure)
|
|
|
|
else:
|
|
|
|
node = initNodeRes.get()
|
|
|
|
|
2022-03-17 16:33:17 +00:00
|
|
|
debug "4/7 Mounting protocols"
|
2021-04-24 04:56:37 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
let setupProtocolsRes = waitFor setupProtocols(node, conf, mStorage)
|
|
|
|
if setupProtocolsRes.isErr():
|
|
|
|
error "4/7 Mounting protocols failed. Continuing in current state.", error=setupProtocolsRes.error
|
2020-09-01 02:09:54 +00:00
|
|
|
|
2022-03-17 16:33:17 +00:00
|
|
|
debug "5/7 Starting node and mounted protocols"
|
2021-04-15 08:18:14 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
let startNodeRes = waitFor startNode(node, conf, dynamicBootstrapNodes)
|
|
|
|
if startNodeRes.isErr():
|
|
|
|
error "5/7 Starting node and mounted protocols failed. Continuing in current state.", error=startNodeRes.error
|
2021-07-22 09:46:54 +00:00
|
|
|
|
2022-03-17 16:33:17 +00:00
|
|
|
debug "6/7 Starting monitoring and external interfaces"
|
2021-07-22 09:46:54 +00:00
|
|
|
|
|
|
|
let startExternalRes = startExternal(node, conf)
|
2022-10-21 13:01:01 +00:00
|
|
|
if startExternalRes.isErr():
|
|
|
|
error "6/7 Starting monitoring and external interfaces failed. Continuing in current state.", error=startExternalRes.error
|
2021-07-22 09:46:54 +00:00
|
|
|
|
2022-03-17 16:33:17 +00:00
|
|
|
debug "7/7 Setting up shutdown hooks"
|
2021-07-22 09:46:54 +00:00
|
|
|
|
2022-03-17 16:33:17 +00:00
|
|
|
# 7/7 Setup graceful shutdown hooks
|
2021-07-22 09:46:54 +00:00
|
|
|
## Setup shutdown hooks for this process.
|
|
|
|
## Stop node gracefully on shutdown.
|
2021-04-15 08:18:14 +00:00
|
|
|
|
|
|
|
# Handle Ctrl-C SIGINT
|
|
|
|
proc handleCtrlC() {.noconv.} =
|
|
|
|
when defined(windows):
|
|
|
|
# workaround for https://github.com/nim-lang/Nim/issues/4057
|
|
|
|
setupForeignThreadGc()
|
|
|
|
info "Shutting down after receiving SIGINT"
|
|
|
|
waitFor node.stop()
|
|
|
|
quit(QuitSuccess)
|
|
|
|
|
|
|
|
setControlCHook(handleCtrlC)
|
|
|
|
|
|
|
|
# Handle SIGTERM
|
|
|
|
when defined(posix):
|
|
|
|
proc handleSigterm(signal: cint) {.noconv.} =
|
|
|
|
info "Shutting down after receiving SIGTERM"
|
|
|
|
waitFor node.stop()
|
|
|
|
quit(QuitSuccess)
|
|
|
|
|
2022-09-07 15:31:27 +00:00
|
|
|
c_signal(ansi_c.SIGTERM, handleSigterm)
|
2021-06-02 07:53:34 +00:00
|
|
|
|
2022-10-21 13:01:01 +00:00
|
|
|
info "Node setup complete"
|
2020-09-01 02:09:54 +00:00
|
|
|
|
2021-05-28 18:41:29 +00:00
|
|
|
runForever()
|