From 1cf3a80acc966faeb49625ca93c783b820c1e3b5 Mon Sep 17 00:00:00 2001 From: SionoiS Date: Mon, 23 Oct 2023 15:55:28 -0400 Subject: [PATCH 01/16] sharded peer manager Fix possible out of bound & logic error Filter peers per protocol & rename proc Fix out of bound & refactor dialling Fix catching raise VS timeout & tests fixes Fix test to connect to all peer per proto Fix test Div target per shard count Logging & stuff Fixes Log peer count More logs Remove protobook override & clean up Fix relay peer management & logs Mics Fixes Fixes --- tests/test_peer_manager.nim | 68 +++-- tests/testlib/wakunode.nim | 4 +- waku/node/peer_manager/peer_manager.nim | 320 ++++++++++++--------- waku/node/peer_manager/waku_peer_store.nim | 13 +- waku/waku_core/peers.nim | 19 +- waku/waku_metadata/protocol.nim | 2 +- 6 files changed, 269 insertions(+), 157 deletions(-) diff --git a/tests/test_peer_manager.nim b/tests/test_peer_manager.nim index 8576f2725a..d1f8347d03 100644 --- a/tests/test_peer_manager.nim +++ b/tests/test_peer_manager.nim @@ -1,7 +1,7 @@ {.used.} import - std/[options, sequtils, times], + std/[options, sequtils, times, sugar], stew/shims/net as stewNet, testutils/unittests, chronos, @@ -21,6 +21,7 @@ import ../../waku/node/peer_manager/peer_manager, ../../waku/node/peer_manager/peer_store/waku_peer_storage, ../../waku/waku_node, + ../../waku/waku_core/topics, ../../waku/waku_relay, ../../waku/waku_store, ../../waku/waku_filter, @@ -128,7 +129,6 @@ procSuite "Peer Manager": await node.stop() - asyncTest "Peer manager keeps track of connections": # Create 2 nodes let nodes = toSeq(0..<2).mapIt(newTestWakuNode(generateSecp256k1Key(), ValidIpAddress.init("0.0.0.0"), Port(0))) @@ -225,18 +225,36 @@ procSuite "Peer Manager": let database = SqliteDatabase.new(":memory:")[] storage = WakuPeerStorage.new(database)[] - node1 = newTestWakuNode(generateSecp256k1Key(), ValidIpAddress.init("0.0.0.0"), Port(0), peerStorage = storage) - node2 = newTestWakuNode(generateSecp256k1Key(), ValidIpAddress.init("0.0.0.0"), Port(0)) - peerInfo2 = node2.switch.peerInfo + node1 = newTestWakuNode( + generateSecp256k1Key(), + ValidIpAddress.init("127.0.0.1"), + Port(44048), + peerStorage = storage + ) + node2 = newTestWakuNode(generateSecp256k1Key(), ValidIpAddress.init("127.0.0.1"), Port(34023)) + + node1.mountMetadata(0).expect("Mounted Waku Metadata") + node2.mountMetadata(0).expect("Mounted Waku Metadata") await node1.start() await node2.start() await node1.mountRelay() await node2.mountRelay() + + let peerInfo2 = node2.switch.peerInfo + var remotePeerInfo2 = peerInfo2.toRemotePeerInfo() + remotePeerInfo2.enr = some(node2.enr) - require: - (await node1.peerManager.connectRelay(peerInfo2.toRemotePeerInfo())) == true + let is12Connected = await node1.peerManager.connectRelay(remotePeerInfo2) + assert is12Connected == true, "Node 1 and 2 not connected" + + # When node use 0.0.0.0 and port 0 + # After connecting the peer store is updated with the wrong address + check: + node1.peerManager.peerStore[AddressBook][remotePeerInfo2.peerId] == remotePeerInfo2.addrs + + # wait for the peer store update await sleepAsync(chronos.milliseconds(500)) check: @@ -246,10 +264,17 @@ procSuite "Peer Manager": node1.peerManager.peerStore.connectedness(peerInfo2.peerId) == Connected # Simulate restart by initialising a new node using the same storage - let - node3 = newTestWakuNode(generateSecp256k1Key(), ValidIpAddress.init("0.0.0.0"), Port(0), peerStorage = storage) + let node3 = newTestWakuNode( + generateSecp256k1Key(), + ValidIpAddress.init("127.0.0.1"), + Port(56037), + peerStorage = storage + ) + + node3.mountMetadata(0).expect("Mounted Waku Metadata") await node3.start() + check: # Node2 has been loaded after "restart", but we have not yet reconnected node3.peerManager.peerStore.peers().len == 1 @@ -257,13 +282,16 @@ procSuite "Peer Manager": node3.peerManager.peerStore.connectedness(peerInfo2.peerId) == NotConnected await node3.mountRelay() - await node3.peerManager.connectToRelayPeers() + + await node3.peerManager.manageRelayPeers() + + await sleepAsync(chronos.milliseconds(500)) check: # Reconnected to node2 after "restart" node3.peerManager.peerStore.peers().len == 1 node3.peerManager.peerStore.peers().anyIt(it.peerId == peerInfo2.peerId) - node3.peerManager.peerStore.connectedness(peerInfo2.peerId) == Connected + node3.peerManager.peerStore.connectedness(peerInfo2.peerId) == Connected await allFutures([node1.stop(), node2.stop(), node3.stop()]) @@ -297,9 +325,9 @@ procSuite "Peer Manager": topics = @["/waku/2/rs/4/0"], ) - discard node1.mountMetadata(clusterId3) - discard node2.mountMetadata(clusterId4) - discard node3.mountMetadata(clusterId4) + node1.mountMetadata(clusterId3).expect("Mounted Waku Metadata") + node2.mountMetadata(clusterId4).expect("Mounted Waku Metadata") + node3.mountMetadata(clusterId4).expect("Mounted Waku Metadata") # Start nodes await allFutures([node1.start(), node2.start(), node3.start()]) @@ -318,7 +346,6 @@ procSuite "Peer Manager": conn2.isNone conn3.isSome - # TODO: nwaku/issues/1377 xasyncTest "Peer manager support multiple protocol IDs when reconnecting to peers": let @@ -380,11 +407,16 @@ procSuite "Peer Manager": let nodes = toSeq(0..<4).mapIt(newTestWakuNode(generateSecp256k1Key(), ValidIpAddress.init("0.0.0.0"), Port(0))) # Start them - await allFutures(nodes.mapIt(it.start())) + discard nodes.mapIt(it.mountMetadata(0)) await allFutures(nodes.mapIt(it.mountRelay())) + await allFutures(nodes.mapIt(it.start())) # Get all peer infos - let peerInfos = nodes.mapIt(it.switch.peerInfo.toRemotePeerInfo()) + let peerInfos = collect: + for i in 0..nodes.high: + let peerInfo = nodes[i].switch.peerInfo.toRemotePeerInfo() + peerInfo.enr = some(nodes[i].enr) + peerInfo # Add all peers (but self) to node 0 nodes[0].peerManager.addPeer(peerInfos[1]) @@ -392,7 +424,7 @@ procSuite "Peer Manager": nodes[0].peerManager.addPeer(peerInfos[3]) # Connect to relay peers - await nodes[0].peerManager.connectToRelayPeers() + await nodes[0].peerManager.manageRelayPeers() check: # Peerstore track all three peers diff --git a/tests/testlib/wakunode.nim b/tests/testlib/wakunode.nim index f614b272b5..ee44e74b85 100644 --- a/tests/testlib/wakunode.nim +++ b/tests/testlib/wakunode.nim @@ -66,7 +66,9 @@ proc newTestWakuNode*(nodeKey: crypto.PrivateKey, if (extIp.isSome() or dns4DomainName.isSome()) and extPort.isNone(): some(Port(60000)) else: extPort - let conf = defaultTestWakuNodeConf() + var conf = defaultTestWakuNodeConf() + + conf.topics = topics if dns4DomainName.isSome() and extIp.isNone(): # If there's an error resolving the IP, an exception is thrown and test fails diff --git a/waku/node/peer_manager/peer_manager.nim b/waku/node/peer_manager/peer_manager.nim index 69db53eed5..04503233df 100644 --- a/waku/node/peer_manager/peer_manager.nim +++ b/waku/node/peer_manager/peer_manager.nim @@ -49,16 +49,16 @@ const BackoffFactor = 4 # Limit the amount of paralel dials - MaxParalelDials = 10 + MaxParallelDials = 10 # Delay between consecutive relayConnectivityLoop runs - ConnectivityLoopInterval = chronos.seconds(15) + ConnectivityLoopInterval = chronos.minutes(1) # How often the peer store is pruned PrunePeerStoreInterval = chronos.minutes(10) # How often metrics and logs are shown/updated - LogAndMetricsInterval = chronos.minutes(3) + LogAndMetricsInterval = chronos.minutes(1) # Max peers that we allow from the same IP ColocationLimit = 5 @@ -115,21 +115,25 @@ proc addPeer*(pm: PeerManager, remotePeerInfo: RemotePeerInfo, origin = UnknownO # Do not attempt to manage our unmanageable self return - # ...public key - var publicKey: PublicKey - discard remotePeerInfo.peerId.extractPublicKey(publicKey) - if pm.peerStore[AddressBook][remotePeerInfo.peerId] == remotePeerInfo.addrs and - pm.peerStore[KeyBook][remotePeerInfo.peerId] == publicKey and + pm.peerStore[KeyBook][remotePeerInfo.peerId] == remotePeerInfo.publicKey and pm.peerStore[ENRBook][remotePeerInfo.peerId].raw.len > 0: # Peer already managed and ENR info is already saved return - trace "Adding peer to manager", peerId = remotePeerInfo.peerId, addresses = remotePeerInfo.addrs - + trace "Adding peer to manager", + peerId = remotePeerInfo.peerId, + addresses = remotePeerInfo.addrs, + protocols = remotePeerInfo.protocols, + origin = origin, + enr = remotePeerInfo.enr + pm.peerStore[AddressBook][remotePeerInfo.peerId] = remotePeerInfo.addrs - pm.peerStore[KeyBook][remotePeerInfo.peerId] = publicKey + pm.peerStore[KeyBook][remotePeerInfo.peerId] = remotePeerInfo.publicKey pm.peerStore[SourceBook][remotePeerInfo.peerId] = origin + + if remotePeerInfo.protocols.len > 0: + pm.peerStore[ProtoBook][remotePeerInfo.peerId] = remotePeerInfo.protocols if remotePeerInfo.enr.isSome(): pm.peerStore[ENRBook][remotePeerInfo.peerId] = remotePeerInfo.enr.get() @@ -158,27 +162,30 @@ proc connectRelay*(pm: PeerManager, pm.addPeer(peer) let failedAttempts = pm.peerStore[NumberFailedConnBook][peerId] - debug "Connecting to relay peer", wireAddr=peer.addrs, peerId=peerId, failedAttempts=failedAttempts + debug "Connecting to relay peer", + wireAddr=peer.addrs, peerId=peerId, failedAttempts=failedAttempts var deadline = sleepAsync(dialTimeout) - var workfut = pm.switch.connect(peerId, peer.addrs) - var reasonFailed = "" + let workfut = pm.switch.connect(peerId, peer.addrs) + + # Can't use catch: with .withTimeout() in this case + let res = catch: await workfut or deadline - try: - await workfut or deadline + let reasonFailed = if workfut.finished(): - if not deadline.finished(): - deadline.cancel() - waku_peers_dials.inc(labelValues = ["successful"]) - waku_node_conns_initiated.inc(labelValues = [source]) - pm.peerStore[NumberFailedConnBook][peerId] = 0 - return true + if res.isOk(): + if not deadline.finished(): + await deadline.cancelAndWait() + + waku_peers_dials.inc(labelValues = ["successful"]) + waku_node_conns_initiated.inc(labelValues = [source]) + pm.peerStore[NumberFailedConnBook][peerId] = 0 + return true + else: res.error.msg else: - reasonFailed = "timed out" - await cancelAndWait(workfut) - except CatchableError as exc: - reasonFailed = "remote peer failed" - + await workfut.cancelAndWait() + "timed out" + # Dial failed pm.peerStore[NumberFailedConnBook][peerId] = pm.peerStore[NumberFailedConnBook][peerId] + 1 pm.peerStore[LastFailedConnBook][peerId] = Moment.init(getTime().toUnix, Second) @@ -214,15 +221,15 @@ proc dialPeer(pm: PeerManager, # Dial Peer let dialFut = pm.switch.dial(peerId, addrs, proto) - var reasonFailed = "" - try: - if (await dialFut.withTimeout(dialTimeout)): + + let res = catch: + if await dialFut.withTimeout(dialTimeout): return some(dialFut.read()) - else: - reasonFailed = "timeout" - await cancelAndWait(dialFut) - except CatchableError as exc: - reasonFailed = "failed" + else: await cancelAndWait(dialFut) + + let reasonFailed = + if res.isOk: "timed out" + else: res.error.msg debug "Dialing peer failed", peerId=peerId, reason=reasonFailed, proto=proto @@ -293,105 +300,111 @@ proc canBeConnected*(pm: PeerManager, let now = Moment.init(getTime().toUnix, Second) let lastFailed = pm.peerStore[LastFailedConnBook][peerId] let backoff = calculateBackoff(pm.initialBackoffInSec, pm.backoffFactor, failedAttempts) - if now >= (lastFailed + backoff): - return true - return false + + return now >= (lastFailed + backoff) ################## # Initialisation # ################## proc getPeerIp(pm: PeerManager, peerId: PeerId): Option[string] = - if pm.switch.connManager.getConnections().hasKey(peerId): - let conns = pm.switch.connManager.getConnections().getOrDefault(peerId) - if conns.len != 0: - let observedAddr = conns[0].connection.observedAddr - let ip = observedAddr.get.getHostname() - if observedAddr.isSome: - # TODO: think if circuit relay ips should be handled differently - let ip = observedAddr.get.getHostname() - return some(ip) - return none(string) + if not pm.switch.connManager.getConnections().hasKey(peerId): + return none(string) + + let conns = pm.switch.connManager.getConnections().getOrDefault(peerId) + if conns.len == 0: + return none(string) + + let obAddr = conns[0].connection.observedAddr.valueOr: + return none(string) + + # TODO: think if circuit relay ips should be handled differently + + return some(obAddr.getHostname()) # called when a connection i) is created or ii) is closed proc onConnEvent(pm: PeerManager, peerId: PeerID, event: ConnEvent) {.async.} = case event.kind of ConnEventKind.Connected: - let direction = if event.incoming: Inbound else: Outbound + #let direction = if event.incoming: Inbound else: Outbound discard of ConnEventKind.Disconnected: discard +proc onPeerMetadata(pm: PeerManager, peerId: PeerId) {.async.} = + # To prevent metadata protocol from breaking prev nodes, by now we only + # disconnect if the clusterid is specified. + if pm.wakuMetadata.clusterId == 0: + return + + let res = catch: await pm.switch.dial(peerId, WakuMetadataCodec) + let conn = res.valueOr: + info "disconnecting from peer", peerId=peerId, reason="dial failed: " & error.msg + asyncSpawn(pm.switch.disconnect(peerId)) + pm.peerStore.delete(peerId) + return + + let metadata = (await pm.wakuMetadata.request(conn)).valueOr: + info "disconnecting from peer", peerId=peerId, reason="waku metatdata request failed: " & error + asyncSpawn(pm.switch.disconnect(peerId)) + pm.peerStore.delete(peerId) + return + + let clusterId = metadata.clusterId.valueOr: + info "disconnecting from peer", peerId=peerId, reason="empty clusterId reported" + asyncSpawn(pm.switch.disconnect(peerId)) + pm.peerStore.delete(peerId) + return + + if pm.wakuMetadata.clusterId != clusterId: + info "disconnecting from peer", + peerId=peerId, + reason="different clusterId reported: " & $pm.wakuMetadata.clusterId & " vs " & $clusterId + asyncSpawn(pm.switch.disconnect(peerId)) + pm.peerStore.delete(peerId) + return + + if not metadata.shards.anyIt(pm.wakuMetadata.shards.contains(it)): + info "disconnecting from peer", peerId=peerId, reason="no shard in common" + asyncSpawn(pm.switch.disconnect(peerId)) + pm.peerStore.delete(peerId) + return + # called when a peer i) first connects to us ii) disconnects all connections from us proc onPeerEvent(pm: PeerManager, peerId: PeerId, event: PeerEvent) {.async.} = + if not pm.wakuMetadata.isNil() and event.kind == PeerEventKind.Joined: + await pm.onPeerMetadata(peerId) + var direction: PeerDirection var connectedness: Connectedness - if event.kind == PeerEventKind.Joined: - direction = if event.initiator: Outbound else: Inbound - connectedness = Connected - - var clusterOk = false - var reason = "" - # To prevent metadata protocol from breaking prev nodes, by now we only - # disconnect if the clusterid is specified. - if not pm.wakuMetadata.isNil() and pm.wakuMetadata.clusterId != 0: - block wakuMetadata: - var conn: Connection - try: - conn = await pm.switch.dial(peerId, WakuMetadataCodec) - except CatchableError: - reason = "waku metadata codec not supported: " & getCurrentExceptionMsg() - break wakuMetadata - - # request metadata from connecting peer - let metadata = (await pm.wakuMetadata.request(conn)).valueOr: - reason = "failed waku metadata codec request" - break wakuMetadata - - # does not report any clusterId - let clusterId = metadata.clusterId.valueOr: - reason = "empty clusterId reported" - break wakuMetadata - - # drop it if it doesnt match our network id - if pm.wakuMetadata.clusterId != clusterId: - reason = "different clusterId reported: " & $pm.wakuMetadata.clusterId & " vs " & $clusterId - break wakuMetadata - - # reaching here means the clusterId matches - clusterOk = true - - if not pm.wakuMetadata.isNil() and pm.wakuMetadata.clusterId != 0 and not clusterOk: - info "disconnecting from peer", peerId=peerId, reason=reason - asyncSpawn(pm.switch.disconnect(peerId)) - pm.peerStore.delete(peerId) - - # TODO: Take action depending on the supported shards as reported by metadata - - let ip = pm.getPeerIp(peerId) - if ip.isSome: - pm.ipTable.mgetOrPut(ip.get, newSeq[PeerId]()).add(peerId) - - let peersBehindIp = pm.ipTable[ip.get] - if peersBehindIp.len > pm.colocationLimit: + case event.kind: + of Joined: + direction = if event.initiator: Outbound else: Inbound + connectedness = Connected + + if (let ip = pm.getPeerIp(peerId); ip.isSome()): + pm.ipTable.mgetOrPut(ip.get, newSeq[PeerId]()).add(peerId) + # in theory this should always be one, but just in case - for peerId in peersBehindIp[0..<(peersBehindIp.len - pm.colocationLimit)]: + let peersBehindIp = pm.ipTable[ip.get] + + let idx = max((peersBehindIp.len - pm.colocationLimit), 0) + for peerId in peersBehindIp[0.. pm.inRelayPeersTarget: - # await pm.pruneInRelayConns(inRelayPeers.len - pm.inRelayPeersTarget) + # Calculate in/out target number of peers for each shards + let inTarget = pm.inRelayPeersTarget div pm.wakuMetadata.shards.len + let outTarget = pm.outRelayPeersTarget div pm.wakuMetadata.shards.len - if outRelayPeers.len >= pm.outRelayPeersTarget: + for shard in pm.wakuMetadata.shards.items: + # Filter out peer not on this shard + let connectedInPeers = inPeers.filterIt( + pm.peerStore.hasShard(it, uint16(pm.wakuMetadata.clusterId), uint16(shard))) + + let connectedOutPeers = outPeers.filterIt( + pm.peerStore.hasShard(it, uint16(pm.wakuMetadata.clusterId), uint16(shard))) + + # Calculate the difference between current values and targets + let inPeerDiff = connectedInPeers.len - inTarget + let outPeerDiff = outTarget - connectedOutPeers.len + + if inPeerDiff > 0: + peersToDisconnect += inPeerDiff + + if outPeerDiff <= 0: + continue + + # Get all connectable relay peers for this shard + var connectablePeers = pm.peerStore.getPeersByShard( + uint16(pm.wakuMetadata.clusterId), uint16(shard)) + + debug "Unfiltered Connectable peers", + ShardNumber = shard, + Peers = connectablePeers.len + + connectablePeers.keepItIf( + it.protocols.contains(WakuRelayCodec) and + not pm.peerStore.isConnected(it.peerId) and + pm.canBeConnected(it.peerId)) + + debug "Target Peer Count per Shard", + ShardNumber = shard, + Inbound = $connectedInPeers.len & "/" & $inTarget, + Outbound = $connectedOutPeers.len & "/" & $outTarget, + Connectable = connectablePeers.len + + let length = min(outPeerDiff, connectablePeers.len) + peersToConnect.add(connectablePeers[0.. Date: Wed, 29 Nov 2023 07:56:04 -0500 Subject: [PATCH 02/16] Add proto to peers from discv5 --- apps/wakunode2/app.nim | 2 +- tests/test_peer_manager.nim | 10 +++--- tests/test_waku_lightpush.nim | 1 + tests/test_wakunode_lightpush.nim | 8 ++--- tests/wakunode_rest/test_rest_lightpush.nim | 3 +- waku/node/peer_manager/peer_manager.nim | 35 ++++++++++++++++++--- waku/node/waku_node.nim | 3 +- waku/waku_api/rest/admin/handlers.nim | 2 +- waku/waku_api/rest/lightpush/handlers.nim | 4 +-- waku/waku_lightpush/client.nim | 2 +- waku/waku_lightpush/common.nim | 21 +++++++++++++ waku/waku_lightpush/protocol.nim | 17 +++------- 12 files changed, 71 insertions(+), 37 deletions(-) create mode 100644 waku/waku_lightpush/common.nim diff --git a/apps/wakunode2/app.nim b/apps/wakunode2/app.nim index 6308ea71c0..a8516a7b0a 100644 --- a/apps/wakunode2/app.nim +++ b/apps/wakunode2/app.nim @@ -53,7 +53,7 @@ import ../../waku/waku_peer_exchange, ../../waku/waku_rln_relay, ../../waku/waku_store, - ../../waku/waku_lightpush, + ../../waku/waku_lightpush/common, ../../waku/waku_filter, ../../waku/waku_filter_v2, ./wakunode2_validator_signed, diff --git a/tests/test_peer_manager.nim b/tests/test_peer_manager.nim index d1f8347d03..7df6bf024f 100644 --- a/tests/test_peer_manager.nim +++ b/tests/test_peer_manager.nim @@ -21,11 +21,11 @@ import ../../waku/node/peer_manager/peer_manager, ../../waku/node/peer_manager/peer_store/waku_peer_storage, ../../waku/waku_node, - ../../waku/waku_core/topics, - ../../waku/waku_relay, - ../../waku/waku_store, - ../../waku/waku_filter, - ../../waku/waku_lightpush, + ../../waku/waku_core, + ../../waku/waku_relay/protocol, + ../../waku/waku_store/common, + ../../waku/waku_filter/protocol, + ../../waku/waku_lightpush/common, ../../waku/waku_peer_exchange, ../../waku/waku_metadata, ./testlib/common, diff --git a/tests/test_waku_lightpush.nim b/tests/test_waku_lightpush.nim index de125ac974..76312272ff 100644 --- a/tests/test_waku_lightpush.nim +++ b/tests/test_waku_lightpush.nim @@ -11,6 +11,7 @@ import ../../waku/node/peer_manager, ../../waku/waku_core, ../../waku/waku_lightpush, + ../../waku/waku_lightpush/common, ../../waku/waku_lightpush/client, ../../waku/waku_lightpush/protocol_metrics, ../../waku/waku_lightpush/rpc, diff --git a/tests/test_wakunode_lightpush.nim b/tests/test_wakunode_lightpush.nim index 7208c587ba..4daa122838 100644 --- a/tests/test_wakunode_lightpush.nim +++ b/tests/test_wakunode_lightpush.nim @@ -4,16 +4,12 @@ import std/options, stew/shims/net as stewNet, testutils/unittests, - chronicles, - chronos, - libp2p/crypto/crypto, - libp2p/switch + chronos import ../../waku/waku_core, - ../../waku/waku_lightpush, + ../../waku/waku_lightpush/common, ../../waku/node/peer_manager, ../../waku/waku_node, - ./testlib/common, ./testlib/wakucore, ./testlib/wakunode diff --git a/tests/wakunode_rest/test_rest_lightpush.nim b/tests/wakunode_rest/test_rest_lightpush.nim index 5581da5c71..72fb4c7b86 100644 --- a/tests/wakunode_rest/test_rest_lightpush.nim +++ b/tests/wakunode_rest/test_rest_lightpush.nim @@ -10,11 +10,10 @@ import import ../../waku/waku_api/message_cache, - ../../waku/common/base64, ../../waku/waku_core, ../../waku/waku_node, ../../waku/node/peer_manager, - ../../waku/waku_lightpush, + ../../waku/waku_lightpush/common, ../../waku/waku_api/rest/server, ../../waku/waku_api/rest/client, ../../waku/waku_api/rest/responses, diff --git a/waku/node/peer_manager/peer_manager.nim b/waku/node/peer_manager/peer_manager.nim index 04503233df..84a0ab383f 100644 --- a/waku/node/peer_manager/peer_manager.nim +++ b/waku/node/peer_manager/peer_manager.nim @@ -17,6 +17,10 @@ import ../../waku_core, ../../waku_relay, ../../waku_enr/sharding, + ../../waku_enr/capabilities, + ../../waku_store/common, + ../../waku_filter_v2/common, + ../../waku_lightpush/common, ../../waku_metadata, ./peer_store/peer_storage, ./waku_peer_store @@ -134,6 +138,24 @@ proc addPeer*(pm: PeerManager, remotePeerInfo: RemotePeerInfo, origin = UnknownO if remotePeerInfo.protocols.len > 0: pm.peerStore[ProtoBook][remotePeerInfo.peerId] = remotePeerInfo.protocols + elif remotePeerInfo.enr.isSome(): + # RemotePeerInfo from discv5 always have empty protocol list + let caps = remotePeerInfo.enr.get().getCapabilities() + + var protos: seq[string] + for cap in caps: + case cap: + of Relay: + protos.add(WakuRelayCodec) + of Store: + protos.add(WakuStoreCodec) + of Filter: + protos.add(WakuFilterSubscribeCodec) + protos.add(WakuFilterPushCodec) + of LightPush: + protos.add(WakuLightPushCodec) + + pm.peerStore[ProtoBook][remotePeerInfo.peerId] = protos if remotePeerInfo.enr.isSome(): pm.peerStore[ENRBook][remotePeerInfo.peerId] = remotePeerInfo.enr.get() @@ -689,20 +711,25 @@ proc manageRelayPeers*(pm: PeerManager) {.async.} = var connectablePeers = pm.peerStore.getPeersByShard( uint16(pm.wakuMetadata.clusterId), uint16(shard)) - debug "Unfiltered Connectable peers", + debug "Peers on this shard", ShardNumber = shard, Peers = connectablePeers.len connectablePeers.keepItIf( - it.protocols.contains(WakuRelayCodec) and not pm.peerStore.isConnected(it.peerId) and pm.canBeConnected(it.peerId)) - debug "Target Peer Count per Shard", + debug "Connectable Peers", + ShardNumber = shard, + Peers = connectablePeers.len + + connectablePeers.keepItIf(it.protocols.contains(WakuRelayCodec)) + + debug "Target Peer Count", ShardNumber = shard, Inbound = $connectedInPeers.len & "/" & $inTarget, Outbound = $connectedOutPeers.len & "/" & $outTarget, - Connectable = connectablePeers.len + RelayConnectable = connectablePeers.len let length = min(outPeerDiff, connectablePeers.len) peersToConnect.add(connectablePeers[0.. Date: Wed, 29 Nov 2023 10:16:18 -0500 Subject: [PATCH 03/16] Replace seq with hash set & loggging --- waku/node/peer_manager/peer_manager.nim | 29 ++++++++++++------------- 1 file changed, 14 insertions(+), 15 deletions(-) diff --git a/waku/node/peer_manager/peer_manager.nim b/waku/node/peer_manager/peer_manager.nim index 84a0ab383f..2fbdbdb2e4 100644 --- a/waku/node/peer_manager/peer_manager.nim +++ b/waku/node/peer_manager/peer_manager.nim @@ -679,7 +679,7 @@ proc pruneInRelayConns(pm: PeerManager, amount: int) {.async.} = asyncSpawn(pm.switch.disconnect(p)) proc manageRelayPeers*(pm: PeerManager) {.async.} = - var peersToConnect: seq[RemotePeerInfo] + var peersToConnect: HashSet[RemotePeerInfo] var peersToDisconnect: int # Get all connected peers for Waku Relay @@ -707,32 +707,31 @@ proc manageRelayPeers*(pm: PeerManager) {.async.} = if outPeerDiff <= 0: continue - # Get all connectable relay peers for this shard + # Get all peers for this shard var connectablePeers = pm.peerStore.getPeersByShard( uint16(pm.wakuMetadata.clusterId), uint16(shard)) - debug "Peers on this shard", - ShardNumber = shard, - Peers = connectablePeers.len + let shardCount = connectablePeers.len connectablePeers.keepItIf( not pm.peerStore.isConnected(it.peerId) and pm.canBeConnected(it.peerId)) - debug "Connectable Peers", - ShardNumber = shard, - Peers = connectablePeers.len + let connectableCount = connectablePeers.len connectablePeers.keepItIf(it.protocols.contains(WakuRelayCodec)) - debug "Target Peer Count", - ShardNumber = shard, - Inbound = $connectedInPeers.len & "/" & $inTarget, - Outbound = $connectedOutPeers.len & "/" & $outTarget, - RelayConnectable = connectablePeers.len + let relayCount = connectablePeers.len + debug "Sharded Peer Management", + Shard = shard, + Connectable = $connectableCount & "/" & $shardCount, + RelayConnectable = $relayCount & "/" & $shardCount, + RelayInboundTarget = $connectedInPeers.len & "/" & $inTarget, + RelayOutboundTarget = $connectedOutPeers.len & "/" & $outTarget + let length = min(outPeerDiff, connectablePeers.len) - peersToConnect.add(connectablePeers[0.. Date: Wed, 29 Nov 2023 10:54:52 -0500 Subject: [PATCH 04/16] Impl. hash for RemotePeerInfo --- waku/waku_core/peers.nim | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/waku/waku_core/peers.nim b/waku/waku_core/peers.nim index 9acfd315c3..5aaea3cbd9 100644 --- a/waku/waku_core/peers.nim +++ b/waku/waku_core/peers.nim @@ -4,7 +4,7 @@ else: {.push raises: [].} import - std/[options, sequtils, strutils, uri], + std/[options, sequtils, strutils, uri, hashes], stew/results, stew/shims/net, chronos, @@ -84,6 +84,8 @@ proc init*(T: typedesc[RemotePeerInfo], let peerId = PeerID.init(peerId).tryGet() RemotePeerInfo(peerId: peerId, addrs: addrs, enr: enr, protocols: protocols) +template hash*(remotePeerInfo: RemotePeerInfo): Hash = + hash(remotePeerInfo.peerId) ## Parse From 487e93221e5e57da90fd03b3baf17baf00c0b735 Mon Sep 17 00:00:00 2001 From: SionoiS Date: Wed, 29 Nov 2023 11:40:09 -0500 Subject: [PATCH 05/16] Remove order when adding peers --- waku/node/peer_manager/peer_manager.nim | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/waku/node/peer_manager/peer_manager.nim b/waku/node/peer_manager/peer_manager.nim index 2fbdbdb2e4..d8d6f78d9c 100644 --- a/waku/node/peer_manager/peer_manager.nim +++ b/waku/node/peer_manager/peer_manager.nim @@ -731,7 +731,11 @@ proc manageRelayPeers*(pm: PeerManager) {.async.} = RelayOutboundTarget = $connectedOutPeers.len & "/" & $outTarget let length = min(outPeerDiff, connectablePeers.len) - peersToConnect = peersToConnect + toHashSet(connectablePeers[0..= target: + break await pm.pruneInRelayConns(peersToDisconnect) From 455fe8ee7e65acbe746881a2fd554b7bf0a92afd Mon Sep 17 00:00:00 2001 From: SionoiS Date: Thu, 30 Nov 2023 08:31:56 -0500 Subject: [PATCH 06/16] Fix chat2 & elevated logging --- apps/chat2/chat2.nim | 6 ++---- waku/node/peer_manager/peer_manager.nim | 9 ++++++++- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/apps/chat2/chat2.nim b/apps/chat2/chat2.nim index 47cb2aea5e..e80d004e66 100644 --- a/apps/chat2/chat2.nim +++ b/apps/chat2/chat2.nim @@ -9,10 +9,9 @@ when (NimMajor, NimMinor) < (1, 4): else: {.push raises: [].} -import std/[strformat, strutils, times, json, options, random] +import std/[strformat, strutils, times, options, random] import confutils, chronicles, chronos, stew/shims/net as stewNet, 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 @@ -22,11 +21,10 @@ import libp2p/[switch, # manage transports, a single entry poi peerinfo, # manage the information of a peer, such as peer ID and public / private key peerid, # Implement how peers interact 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 nameresolving/dnsresolver]# define DNS resolution import ../../waku/waku_core, - ../../waku/waku_lightpush, + ../../waku/waku_lightpush/common, ../../waku/waku_lightpush/rpc, ../../waku/waku_filter, ../../waku/waku_enr, diff --git a/waku/node/peer_manager/peer_manager.nim b/waku/node/peer_manager/peer_manager.nim index d8d6f78d9c..cd29080e2d 100644 --- a/waku/node/peer_manager/peer_manager.nim +++ b/waku/node/peer_manager/peer_manager.nim @@ -676,6 +676,7 @@ proc pruneInRelayConns(pm: PeerManager, amount: int) {.async.} = let connsToPrune = min(amount, inRelayPeers.len) for p in inRelayPeers[0..= target: break @@ -748,6 +754,7 @@ proc manageRelayPeers*(pm: PeerManager) {.async.} = # Connect to all nodes for i in countup(0, uniquePeers.len, MaxParallelDials): let stop = min(i + MaxParallelDials, uniquePeers.len) + info "Connecting to Peers", Peers=$uniquePeers[i.. Date: Thu, 30 Nov 2023 08:39:17 -0500 Subject: [PATCH 07/16] Fix logs --- waku/node/peer_manager/peer_manager.nim | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/waku/node/peer_manager/peer_manager.nim b/waku/node/peer_manager/peer_manager.nim index cd29080e2d..80813c92d8 100644 --- a/waku/node/peer_manager/peer_manager.nim +++ b/waku/node/peer_manager/peer_manager.nim @@ -676,7 +676,9 @@ proc pruneInRelayConns(pm: PeerManager, amount: int) {.async.} = let connsToPrune = min(amount, inRelayPeers.len) for p in inRelayPeers[0..= target: break @@ -754,7 +759,10 @@ proc manageRelayPeers*(pm: PeerManager) {.async.} = # Connect to all nodes for i in countup(0, uniquePeers.len, MaxParallelDials): let stop = min(i + MaxParallelDials, uniquePeers.len) - info "Connecting to Peers", Peers=$uniquePeers[i.. Date: Thu, 30 Nov 2023 09:07:27 -0500 Subject: [PATCH 08/16] Fix duplicate --- waku/node/peer_manager/peer_manager.nim | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/waku/node/peer_manager/peer_manager.nim b/waku/node/peer_manager/peer_manager.nim index 80813c92d8..f19aa31d28 100644 --- a/waku/node/peer_manager/peer_manager.nim +++ b/waku/node/peer_manager/peer_manager.nim @@ -5,7 +5,7 @@ else: import - std/[options, sets, sequtils, times, strutils, math], + std/[options, sugar, sets, sequtils, times, strutils, math], chronos, chronicles, metrics, @@ -753,17 +753,19 @@ proc manageRelayPeers*(pm: PeerManager) {.async.} = if peersToConnect.len == 0: return - # Even with duplicates, after a couple of iteration the target will be reached - let uniquePeers = toSeq(peersToConnect) - # Connect to all nodes - for i in countup(0, uniquePeers.len, MaxParallelDials): - let stop = min(i + MaxParallelDials, uniquePeers.len) + for i in countup(0, peersToConnect.len, MaxParallelDials): + var stop = min(i + MaxParallelDials, peersToConnect.len) + var peers: seq[RemotePeerInfo] + while stop > 0: + peers.add(peersToConnect.pop()) + dec(stop) + info "Connecting to Peers", - Peers = $(uniquePeers[i.. Date: Thu, 30 Nov 2023 09:24:02 -0500 Subject: [PATCH 09/16] Fix test --- tests/testlib/wakunode.nim | 8 +++++--- waku/node/peer_manager/peer_manager.nim | 3 +++ 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/tests/testlib/wakunode.nim b/tests/testlib/wakunode.nim index ee44e74b85..9ba791204f 100644 --- a/tests/testlib/wakunode.nim +++ b/tests/testlib/wakunode.nim @@ -32,7 +32,8 @@ proc defaultTestWakuNodeConf*(): WakuNodeConf = dnsAddrsNameServers: @[ValidIpAddress.init("1.1.1.1"), ValidIpAddress.init("1.0.0.1")], nat: "any", maxConnections: 50, - topics: @[], + clusterId: 1.uint32, + topics: @["/waku/2/rs/1/0"], relay: true ) @@ -55,8 +56,8 @@ proc newTestWakuNode*(nodeKey: crypto.PrivateKey, dns4DomainName = none(string), discv5UdpPort = none(Port), agentString = none(string), - clusterId: uint32 = 2.uint32, - topics: seq[string] = @["/waku/2/rs/2/0"], + clusterId: uint32 = 1.uint32, + topics: seq[string] = @["/waku/2/rs/1/0"], peerStoreCapacity = none(int)): WakuNode = var resolvedExtIp = extIp @@ -68,6 +69,7 @@ proc newTestWakuNode*(nodeKey: crypto.PrivateKey, var conf = defaultTestWakuNodeConf() + conf.clusterId = clusterId conf.topics = topics if dns4DomainName.isSome() and extIp.isNone(): diff --git a/waku/node/peer_manager/peer_manager.nim b/waku/node/peer_manager/peer_manager.nim index f19aa31d28..2fcf70ee57 100644 --- a/waku/node/peer_manager/peer_manager.nim +++ b/waku/node/peer_manager/peer_manager.nim @@ -682,6 +682,9 @@ proc pruneInRelayConns(pm: PeerManager, amount: int) {.async.} = asyncSpawn(pm.switch.disconnect(p)) proc manageRelayPeers*(pm: PeerManager) {.async.} = + if pm.wakuMetadata.shards.len == 0: + return + var peersToConnect: HashSet[RemotePeerInfo] var peersToDisconnect: int From 74f4fe64764d0ccd880e5e5510bcd3a1f0b9c480 Mon Sep 17 00:00:00 2001 From: SionoiS Date: Thu, 30 Nov 2023 15:15:47 -0500 Subject: [PATCH 10/16] Fix duplicate for real --- waku/node/peer_manager/peer_manager.nim | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/waku/node/peer_manager/peer_manager.nim b/waku/node/peer_manager/peer_manager.nim index 2fcf70ee57..e361cd3349 100644 --- a/waku/node/peer_manager/peer_manager.nim +++ b/waku/node/peer_manager/peer_manager.nim @@ -685,7 +685,7 @@ proc manageRelayPeers*(pm: PeerManager) {.async.} = if pm.wakuMetadata.shards.len == 0: return - var peersToConnect: HashSet[RemotePeerInfo] + var peersToConnect: HashSet[PeerId] # Can't use RemotePeerInfo as they are ref object var peersToDisconnect: int # Get all connected peers for Waku Relay @@ -747,7 +747,7 @@ proc manageRelayPeers*(pm: PeerManager) {.async.} = info "Peer To Connect To", Peer = $peer.peerId - peersToConnect.incl(peer) + peersToConnect.incl(peer.peerId) if peersToConnect.len >= target: break @@ -756,19 +756,16 @@ proc manageRelayPeers*(pm: PeerManager) {.async.} = if peersToConnect.len == 0: return + let uniquePeers = toSeq(peersToConnect).mapIt(pm.peerStore.get(it)) + # Connect to all nodes for i in countup(0, peersToConnect.len, MaxParallelDials): var stop = min(i + MaxParallelDials, peersToConnect.len) - var peers: seq[RemotePeerInfo] - while stop > 0: - peers.add(peersToConnect.pop()) - dec(stop) - info "Connecting to Peers", - Peers = $(peers.mapIt(it.peerId)) + Peers = $uniquePeers[i.. Date: Thu, 30 Nov 2023 16:05:28 -0500 Subject: [PATCH 11/16] Remove extra peer to connect --- waku/node/peer_manager/peer_manager.nim | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/waku/node/peer_manager/peer_manager.nim b/waku/node/peer_manager/peer_manager.nim index e361cd3349..39f87554e9 100644 --- a/waku/node/peer_manager/peer_manager.nim +++ b/waku/node/peer_manager/peer_manager.nim @@ -685,7 +685,7 @@ proc manageRelayPeers*(pm: PeerManager) {.async.} = if pm.wakuMetadata.shards.len == 0: return - var peersToConnect: HashSet[PeerId] # Can't use RemotePeerInfo as they are ref object + var peersToConnect: HashSet[PeerId] # Can't use RemotePeerInfo as they are ref objects var peersToDisconnect: int # Get all connected peers for Waku Relay @@ -741,15 +741,12 @@ proc manageRelayPeers*(pm: PeerManager) {.async.} = RelayOutboundTarget = $connectedOutPeers.len & "/" & $outTarget let length = min(outPeerDiff, connectablePeers.len) - let target = peersToConnect.len + length - for peer in connectablePeers: + for peer in connectablePeers[0..length]: info "Peer To Connect To", Peer = $peer.peerId peersToConnect.incl(peer.peerId) - if peersToConnect.len >= target: - break await pm.pruneInRelayConns(peersToDisconnect) From f676d67699e3fee3be8764c956ad9bc464aa6795 Mon Sep 17 00:00:00 2001 From: SionoiS Date: Thu, 30 Nov 2023 16:14:32 -0500 Subject: [PATCH 12/16] Fix --- waku/node/peer_manager/peer_manager.nim | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/waku/node/peer_manager/peer_manager.nim b/waku/node/peer_manager/peer_manager.nim index 39f87554e9..a0fbec4c69 100644 --- a/waku/node/peer_manager/peer_manager.nim +++ b/waku/node/peer_manager/peer_manager.nim @@ -756,8 +756,8 @@ proc manageRelayPeers*(pm: PeerManager) {.async.} = let uniquePeers = toSeq(peersToConnect).mapIt(pm.peerStore.get(it)) # Connect to all nodes - for i in countup(0, peersToConnect.len, MaxParallelDials): - var stop = min(i + MaxParallelDials, peersToConnect.len) + for i in countup(0, uniquePeers.len, MaxParallelDials): + var stop = min(i + MaxParallelDials, uniquePeers.len) info "Connecting to Peers", Peers = $uniquePeers[i.. Date: Thu, 30 Nov 2023 16:33:24 -0500 Subject: [PATCH 13/16] Yet moar fixes --- waku/node/peer_manager/peer_manager.nim | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/waku/node/peer_manager/peer_manager.nim b/waku/node/peer_manager/peer_manager.nim index a0fbec4c69..73b19c0ff7 100644 --- a/waku/node/peer_manager/peer_manager.nim +++ b/waku/node/peer_manager/peer_manager.nim @@ -741,7 +741,7 @@ proc manageRelayPeers*(pm: PeerManager) {.async.} = RelayOutboundTarget = $connectedOutPeers.len & "/" & $outTarget let length = min(outPeerDiff, connectablePeers.len) - for peer in connectablePeers[0..length]: + for peer in connectablePeers[0.. Date: Fri, 1 Dec 2023 08:35:54 -0500 Subject: [PATCH 14/16] Clean-up --- tests/test_peer_manager.nim | 4 +- waku/node/peer_manager/peer_manager.nim | 64 +++++++--------------- waku/node/peer_manager/waku_peer_store.nim | 2 +- waku/waku_core/peers.nim | 10 +--- 4 files changed, 26 insertions(+), 54 deletions(-) diff --git a/tests/test_peer_manager.nim b/tests/test_peer_manager.nim index 7df6bf024f..c2de9d3239 100644 --- a/tests/test_peer_manager.nim +++ b/tests/test_peer_manager.nim @@ -249,8 +249,6 @@ procSuite "Peer Manager": let is12Connected = await node1.peerManager.connectRelay(remotePeerInfo2) assert is12Connected == true, "Node 1 and 2 not connected" - # When node use 0.0.0.0 and port 0 - # After connecting the peer store is updated with the wrong address check: node1.peerManager.peerStore[AddressBook][remotePeerInfo2.peerId] == remotePeerInfo2.addrs @@ -291,7 +289,7 @@ procSuite "Peer Manager": # Reconnected to node2 after "restart" node3.peerManager.peerStore.peers().len == 1 node3.peerManager.peerStore.peers().anyIt(it.peerId == peerInfo2.peerId) - node3.peerManager.peerStore.connectedness(peerInfo2.peerId) == Connected + node3.peerManager.peerStore.connectedness(peerInfo2.peerId) == Connected await allFutures([node1.stop(), node2.stop(), node3.stop()]) diff --git a/waku/node/peer_manager/peer_manager.nim b/waku/node/peer_manager/peer_manager.nim index 73b19c0ff7..6286eee938 100644 --- a/waku/node/peer_manager/peer_manager.nim +++ b/waku/node/peer_manager/peer_manager.nim @@ -62,7 +62,7 @@ const PrunePeerStoreInterval = chronos.minutes(10) # How often metrics and logs are shown/updated - LogAndMetricsInterval = chronos.minutes(1) + LogAndMetricsInterval = chronos.minutes(3) # Max peers that we allow from the same IP ColocationLimit = 5 @@ -125,12 +125,7 @@ proc addPeer*(pm: PeerManager, remotePeerInfo: RemotePeerInfo, origin = UnknownO # Peer already managed and ENR info is already saved return - trace "Adding peer to manager", - peerId = remotePeerInfo.peerId, - addresses = remotePeerInfo.addrs, - protocols = remotePeerInfo.protocols, - origin = origin, - enr = remotePeerInfo.enr + trace "Adding peer to manager", peerId = remotePeerInfo.peerId, addresses = remotePeerInfo.addrs pm.peerStore[AddressBook][remotePeerInfo.peerId] = remotePeerInfo.addrs pm.peerStore[KeyBook][remotePeerInfo.peerId] = remotePeerInfo.publicKey @@ -194,19 +189,20 @@ proc connectRelay*(pm: PeerManager, let res = catch: await workfut or deadline let reasonFailed = - if workfut.finished(): - if res.isOk(): - if not deadline.finished(): - await deadline.cancelAndWait() - - waku_peers_dials.inc(labelValues = ["successful"]) - waku_node_conns_initiated.inc(labelValues = [source]) - pm.peerStore[NumberFailedConnBook][peerId] = 0 - return true - else: res.error.msg - else: + if not workfut.finished(): await workfut.cancelAndWait() "timed out" + elif res.isErr(): res.error.msg + else: + if not deadline.finished(): + await deadline.cancelAndWait() + + waku_peers_dials.inc(labelValues = ["successful"]) + waku_node_conns_initiated.inc(labelValues = [source]) + + pm.peerStore[NumberFailedConnBook][peerId] = 0 + + return true # Dial failed pm.peerStore[NumberFailedConnBook][peerId] = pm.peerStore[NumberFailedConnBook][peerId] + 1 @@ -346,12 +342,7 @@ proc getPeerIp(pm: PeerManager, peerId: PeerId): Option[string] = # called when a connection i) is created or ii) is closed proc onConnEvent(pm: PeerManager, peerId: PeerID, event: ConnEvent) {.async.} = - case event.kind - of ConnEventKind.Connected: - #let direction = if event.incoming: Inbound else: Outbound - discard - of ConnEventKind.Disconnected: - discard + discard proc onPeerMetadata(pm: PeerManager, peerId: PeerId) {.async.} = # To prevent metadata protocol from breaking prev nodes, by now we only @@ -373,7 +364,7 @@ proc onPeerMetadata(pm: PeerManager, peerId: PeerId) {.async.} = return let clusterId = metadata.clusterId.valueOr: - info "disconnecting from peer", peerId=peerId, reason="empty clusterId reported" + info "disconnecting from peer", peerId=peerId, reason="empty cluster-id reported" asyncSpawn(pm.switch.disconnect(peerId)) pm.peerStore.delete(peerId) return @@ -387,7 +378,7 @@ proc onPeerMetadata(pm: PeerManager, peerId: PeerId) {.async.} = return if not metadata.shards.anyIt(pm.wakuMetadata.shards.contains(it)): - info "disconnecting from peer", peerId=peerId, reason="no shard in common" + info "disconnecting from peer", peerId=peerId, reason="no shards in common" asyncSpawn(pm.switch.disconnect(peerId)) pm.peerStore.delete(peerId) return @@ -676,9 +667,7 @@ proc pruneInRelayConns(pm: PeerManager, amount: int) {.async.} = let connsToPrune = min(amount, inRelayPeers.len) for p in inRelayPeers[0.. Date: Wed, 6 Dec 2023 10:40:43 -0500 Subject: [PATCH 15/16] Clean-up --- log.txt | 201 ++++++++++++++++++++++++ waku/node/peer_manager/peer_manager.nim | 72 ++++----- 2 files changed, 238 insertions(+), 35 deletions(-) create mode 100644 log.txt diff --git a/log.txt b/log.txt new file mode 100644 index 0000000000..56399e71d6 --- /dev/null +++ b/log.txt @@ -0,0 +1,201 @@ +Building: install/usr/lib/libbacktracenim.a +Building: build/wakunode2 +[NimScript] exec: nim c --out:build/wakunode2 --verbosity:0 --hints:off -d:chronicles_log_level=TRACE -d:git_version="v0.22.0-rc.0-42-ga4deb2" -d:release --passL:librln_v0.3.4.a --passL:-lm apps/wakunode2/wakunode2.nim +/home/sionois/Github/nwaku/vendor/nim-json-rpc/json_rpc/client.nim(33, 11) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] +/home/sionois/Github/nwaku/vendor/nim-json-rpc/json_rpc/client.nim(37, 11) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] +/home/sionois/Github/nwaku/vendor/nim-websock/websock/http/client.nim(173, 5) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] +/home/sionois/Github/nwaku/vendor/nim-websock/websock/websock.nim(257, 5) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] +/home/sionois/Github/nwaku/vendor/nim-libp2p/libp2p/nameresolving/nameresolver.nim(55, 5) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] +/home/sionois/Github/nwaku/waku/waku_rln_relay/conversion_utils.nim(10, 7) Warning: imported and not used: 'results' [UnusedImport] +/home/sionois/Github/nwaku/waku/waku_rln_relay/rln/wrappers.nim(9, 12) Warning: imported and not used: 'utils' [UnusedImport] +/home/sionois/Github/nwaku/waku/waku_rln_relay/rln/wrappers.nim(8, 6) Warning: imported and not used: 'strformat' [UnusedImport] +/home/sionois/Github/nwaku/waku/waku_rln_relay/group_manager/on_chain/group_manager.nim(101, 82) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/waku/waku_rln_relay/group_manager/on_chain/group_manager.nim(98, 8) Warning: use {.base.} for base methods; baseless methods are deprecated [UseBase] +/home/sionois/Github/nwaku/waku/common/logging.nim(4, 6) Warning: imported and not used: 'strutils' [UnusedImport] +/home/sionois/Github/nwaku/vendor/nim-libp2p/libp2p/protocols/pubsub/pubsubpeer.nim(237, 64) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] +/home/sionois/Github/nwaku/vendor/nim-libp2p/libp2p/protocols/pubsub/gossipsub.nim(388, 134) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] +/home/sionois/Github/nwaku/vendor/nim-json-rpc/json_rpc/router.nim(90, 64) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/vendor/nim-json-rpc/json_rpc/router.nim(107, 36) Warning: catch a more precise Exception deriving from CatchableError or Defect. [BareExcept] +/home/sionois/Github/nwaku/waku/common/databases/dburl.nim(10, 15) Warning: use re2(static string) instead; re is deprecated [Deprecated] +/home/sionois/Github/nwaku/waku/common/databases/dburl.nim(12, 67) Warning: use match(string, Regex2) instead; match is deprecated [Deprecated] +/home/sionois/Github/nwaku/waku/waku_archive/driver/queue_driver/queue_driver.nim(293, 63) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/waku/waku_archive/driver/queue_driver/queue_driver.nim(292, 8) Warning: use {.base.} for base methods; baseless methods are deprecated [UseBase] +/home/sionois/Github/nwaku/waku/waku_archive/retention_policy/retention_policy_size.nim(7, 6) Warning: imported and not used: 'times' [UnusedImport] +/home/sionois/Github/nwaku/waku/waku_archive/retention_policy/retention_policy_size.nim(11, 3) Warning: imported and not used: 'os' [UnusedImport] +/home/sionois/Github/nwaku/vendor/nim-libp2p/libp2p/protocols/connectivity/relay/rtransport.nim(32, 62) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/vendor/nim-libp2p/libp2p/protocols/connectivity/relay/rtransport.nim(40, 25) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] +/home/sionois/Github/nwaku/vendor/nim-libp2p/libp2p/builders.nim(230, 32) template/generic instantiation of `new` from here +/home/sionois/Github/nwaku/vendor/nim-libp2p/libp2p/upgrademngrs/muxedupgrade.nim(102, 7) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] +/home/sionois/Github/nwaku/waku/common/utils/sequence.nim(6, 11) Warning: imported and not used: 'sequtils' [UnusedImport] +/home/sionois/Github/nwaku/waku/node/peer_manager/peer_manager.nim(8, 6) Warning: imported and not used: 'sugar' [UnusedImport] +/home/sionois/Github/nwaku/waku/waku_archive/archive.nim(18, 21) Warning: imported and not used: 'retention_policy_capacity' [UnusedImport] +/home/sionois/Github/nwaku/waku/waku_archive/archive.nim(19, 21) Warning: imported and not used: 'retention_policy_time' [UnusedImport] +/home/sionois/Github/nwaku/waku/waku_archive/archive.nim(14, 22) Warning: imported and not used: 'dburl' [UnusedImport] +/home/sionois/Github/nwaku/waku/waku_archive/archive.nim(15, 22) Warning: imported and not used: 'db_sqlite' [UnusedImport] +/home/sionois/Github/nwaku/waku/waku_archive/archive.nim(11, 3) Warning: imported and not used: 'regex' [UnusedImport] +/home/sionois/Github/nwaku/waku/waku_store/protocol.nim(49, 51) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/waku/waku_store/protocol.nim(52, 31) template/generic instantiation of `decode` from here +/home/sionois/Github/nwaku/waku/waku_store/rpc_codec.nim(250, 39) template/generic instantiation of `decode` from here +/home/sionois/Github/nwaku/waku/waku_store/rpc_codec.nim(217, 40) template/generic instantiation of `parse` from here +/home/sionois/Github/nwaku/waku/waku_store/rpc.nim(88, 28) Warning: conversion to enum with holes is unsafe: HistoryResponseErrorRPC(kind) [HoleEnumConv] +/home/sionois/Github/nwaku/waku/waku_filter_v2/client.nim(39, 27) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/waku/waku_filter_v2/client.nim(74, 10) template/generic instantiation of `setResult` from here +/home/sionois/Github/nwaku/waku/waku_filter_v2/client.nim(72, 70) template/generic instantiation of `setResult` from here +/home/sionois/Github/nwaku/waku/waku_filter_v2/common.nim(67, 38) template/generic instantiation of `parse` from here +/home/sionois/Github/nwaku/waku/waku_filter_v2/common.nim(62, 29) Warning: conversion to enum with holes is unsafe: FilterSubscribeErrorKind(kind) [HoleEnumConv] +/home/sionois/Github/nwaku/vendor/nim-dnsdisc/discovery/dnsdisc/client.nim(67, 121) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] +/home/sionois/Github/nwaku/waku/waku_rln_relay/rln_relay.nim(82, 8) Warning: use {.base.} for base methods; baseless methods are deprecated [UseBase] +/home/sionois/Github/nwaku/waku/waku_rln_relay/rln_relay.nim(257, 7) Warning: Special variable 'result' is shadowed. [ResultShadowed] +/home/sionois/Github/nwaku/waku/waku_rln_relay/rln_relay.nim(27, 11) Warning: imported and not used: 'collector' [UnusedImport] +/home/sionois/Github/nwaku/waku/waku_rln_relay/rln_relay.nim(8, 40) Warning: imported and not used: 'ratelimit' [UnusedImport] +/home/sionois/Github/nwaku/waku/waku_rln_relay/rln_relay.nim(7, 6) Warning: imported and not used: 'os' [UnusedImport] +/home/sionois/Github/nwaku/waku/node/waku_node.nim(407, 20) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] +/home/sionois/Github/nwaku/waku/node/waku_node.nim(432, 43) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] +/home/sionois/Github/nwaku/waku/node/waku_node.nim(452, 31) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] +/home/sionois/Github/nwaku/waku/node/waku_node.nim(533, 19) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] +/home/sionois/Github/nwaku/waku/node/waku_node.nim(594, 33) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] +/home/sionois/Github/nwaku/waku/node/waku_node.nim(651, 19) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] +/home/sionois/Github/nwaku/waku/node/waku_node.nim(714, 19) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] +/home/sionois/Github/nwaku/waku/node/waku_node.nim(811, 36) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] +/home/sionois/Github/nwaku/waku/node/waku_node.nim(992, 43) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] +/home/sionois/Github/nwaku/waku/node/waku_node.nim(1002, 64) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] +/home/sionois/Github/nwaku/waku/node/waku_node.nim(1040, 41) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] +/home/sionois/Github/nwaku/waku/node/waku_node.nim(1082, 41) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] +/home/sionois/Github/nwaku/waku/node/waku_switch.nim(8, 6) Warning: imported and not used: 'math' [UnusedImport] +/home/sionois/Github/nwaku/waku/waku_api/handlers.nim(8, 3) Warning: imported and not used: 'chronicles' [UnusedImport] +/home/sionois/Github/nwaku/waku/waku_api/rest/filter/handlers.nim(113, 5) Warning: unreachable else, all cases are already covered [UnreachableElse] +/home/sionois/Github/nwaku/waku/waku_api/rest/lightpush/types.nim(11, 9) Warning: imported and not used: 'common' [UnusedImport] +/home/sionois/Github/nwaku/waku/waku_api/rest/lightpush/types.nim(14, 18) Warning: imported and not used: 'base64' [UnusedImport] +/home/sionois/Github/nwaku/waku/waku_api/rest/store/types.nim(8, 7) Warning: imported and not used: 'byteutils' [UnusedImport] +/home/sionois/Github/nwaku/waku/waku_api/rest/health/handlers.nim(13, 3) Warning: imported and not used: 'serdes' [UnusedImport] +/home/sionois/Github/nwaku/waku/waku_api/rest/health/handlers.nim(8, 3) Warning: imported and not used: 'json_serialization' [UnusedImport] +/home/sionois/Github/nwaku/waku/waku_api/rest/admin/handlers.nim(9, 7) Warning: imported and not used: 'byteutils' [UnusedImport] +/home/sionois/Github/nwaku/waku/waku_api/jsonrpc/filter/handlers.nim(14, 23) Warning: imported and not used: 'client' [UnusedImport] +/home/sionois/Github/nwaku/waku/waku_api/jsonrpc/relay/handlers.nim(17, 30) Warning: imported and not used: 'wrappers' [UnusedImport] +/home/sionois/Github/nwaku/waku/waku_api/jsonrpc/relay/handlers.nim(11, 12) Warning: imported and not used: 'sysrand' [UnusedImport] +/home/sionois/Github/nwaku/apps/wakunode2/wakunode2_validator_signed.nim(7, 6) Warning: imported and not used: 'math' [UnusedImport] +/home/sionois/Github/nwaku/apps/wakunode2/wakunode2_validator_signed.nim(14, 30) Warning: imported and not used: 'messages' [UnusedImport] +/home/sionois/Github/nwaku/apps/wakunode2/app.nim(405, 47) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/apps/wakunode2/app.nim(570, 10) template/generic instantiation of `setResult` from here +/home/sionois/Github/nwaku/apps/wakunode2/app.nim(499, 39) template/generic instantiation of `new` from here +/home/sionois/Github/nwaku/waku/waku_archive/retention_policy/builder.nim(25, 44) Warning: use re2(static string) instead; re is deprecated [Deprecated] +/home/sionois/Github/nwaku/apps/wakunode2/app.nim(405, 47) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/apps/wakunode2/app.nim(570, 10) template/generic instantiation of `setResult` from here +/home/sionois/Github/nwaku/apps/wakunode2/app.nim(499, 39) template/generic instantiation of `new` from here +/home/sionois/Github/nwaku/waku/waku_archive/retention_policy/builder.nim(26, 20) Warning: use match(string, Regex2) instead; match is deprecated [Deprecated] +/home/sionois/Github/nwaku/apps/wakunode2/app.nim(658, 59) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/apps/wakunode2/app.nim(675, 5) Warning: unreachable code after 'return' statement or '{.noReturn.}' proc [UnreachableCode] +/home/sionois/Github/nwaku/apps/wakunode2/app.nim(658, 59) template/generic instantiation of `async` from here +/home/sionois/Github/nwaku/apps/wakunode2/app.nim(675, 5) Warning: unreachable code after 'return' statement or '{.noReturn.}' proc [UnreachableCode] +/home/sionois/Github/nwaku/apps/wakunode2/app.nim(23, 26) Warning: imported and not used: 'nat' [UnusedImport] +/home/sionois/Github/nwaku/apps/wakunode2/wakunode2.nim(37, 29) template/generic instantiation of `load` from here +/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(578, 28) template/generic instantiation of `load` from here +/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(585, 18) template/generic instantiation of `addConfigFile` from here +/home/sionois/Github/nwaku/vendor/nim-confutils/confutils.nim(860, 39) template/generic instantiation of `loadFile` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization.nim(156, 13) template/generic instantiation of `readValue` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization.nim(31, 9) template/generic instantiation of `readValue` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(499, 8) template/generic instantiation of `decodeInlineTable` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(377, 19) template/generic instantiation of `fieldReadersTable` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(256, 34) template/generic instantiation of `makeFieldReadersTable` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(218, 26) template/generic instantiation of `enumAllSerializedFields` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(152, 32) template/generic instantiation of `enumAllSerializedFieldsImpl` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(235, 52) template/generic instantiation of `readFieldIMPL` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(203, 13) template/generic instantiation of `readValue` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization.nim(31, 9) template/generic instantiation of `readValue` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(445, 19) template/generic instantiation of `readValue` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization.nim(31, 9) template/generic instantiation of `readValue` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(481, 6) template/generic instantiation of `parseList` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(483, 12) Warning: setLen can potentially expand the sequence, but the element type 'ProtectedTopic' doesn't have a valid default value [UnsafeSetLen] +/home/sionois/Github/nwaku/apps/wakunode2/wakunode2.nim(37, 29) template/generic instantiation of `load` from here +/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(578, 28) template/generic instantiation of `load` from here +/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(585, 18) template/generic instantiation of `addConfigFile` from here +/home/sionois/Github/nwaku/vendor/nim-confutils/confutils.nim(860, 39) template/generic instantiation of `loadFile` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization.nim(156, 13) template/generic instantiation of `readValue` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization.nim(31, 9) template/generic instantiation of `readValue` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(499, 8) template/generic instantiation of `decodeInlineTable` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(377, 19) template/generic instantiation of `fieldReadersTable` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(256, 34) template/generic instantiation of `makeFieldReadersTable` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(218, 26) template/generic instantiation of `enumAllSerializedFields` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(152, 32) template/generic instantiation of `enumAllSerializedFieldsImpl` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(235, 52) template/generic instantiation of `readFieldIMPL` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(203, 13) template/generic instantiation of `readValue` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization.nim(31, 9) template/generic instantiation of `readValue` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(445, 19) template/generic instantiation of `readValue` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization.nim(30, 19) Warning: The 'ValidIpAddress' type doesn't have a valid default value [UnsafeDefault] +/home/sionois/Github/nwaku/apps/wakunode2/wakunode2.nim(37, 29) template/generic instantiation of `load` from here +/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(578, 28) template/generic instantiation of `load` from here +/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(585, 18) template/generic instantiation of `addConfigFile` from here +/home/sionois/Github/nwaku/vendor/nim-confutils/confutils.nim(860, 39) template/generic instantiation of `loadFile` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization.nim(156, 13) template/generic instantiation of `readValue` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization.nim(31, 9) template/generic instantiation of `readValue` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(499, 8) template/generic instantiation of `decodeInlineTable` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(377, 19) template/generic instantiation of `fieldReadersTable` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(256, 34) template/generic instantiation of `makeFieldReadersTable` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(218, 26) template/generic instantiation of `enumAllSerializedFields` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(152, 32) template/generic instantiation of `enumAllSerializedFieldsImpl` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(235, 52) template/generic instantiation of `readFieldIMPL` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(203, 13) template/generic instantiation of `readValue` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization.nim(31, 9) template/generic instantiation of `readValue` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(445, 19) template/generic instantiation of `readValue` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization.nim(31, 9) template/generic instantiation of `readValue` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(481, 6) template/generic instantiation of `parseList` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(483, 12) Warning: setLen can potentially expand the sequence, but the element type 'ValidIpAddress' doesn't have a valid default value [UnsafeSetLen] +/home/sionois/Github/nwaku/apps/wakunode2/wakunode2.nim(37, 29) template/generic instantiation of `load` from here +/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(578, 28) template/generic instantiation of `load` from here +/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(585, 18) template/generic instantiation of `addConfigFile` from here +/home/sionois/Github/nwaku/vendor/nim-confutils/confutils.nim(860, 39) template/generic instantiation of `loadFile` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization.nim(156, 13) template/generic instantiation of `readValue` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization.nim(31, 9) template/generic instantiation of `readValue` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(501, 8) template/generic instantiation of `decodeRecord` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(285, 27) template/generic instantiation of `arrayReadersTable` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/private/array_reader.nim(39, 26) template/generic instantiation of `enumAllSerializedFields` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(152, 32) template/generic instantiation of `enumAllSerializedFieldsImpl` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/private/array_reader.nim(53, 21) template/generic instantiation of `readValue` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(269, 14) template/generic instantiation of `readValue` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(262, 10) Warning: setLen can potentially expand the sequence, but the element type 'ProtectedTopic' doesn't have a valid default value [UnsafeSetLen] +/home/sionois/Github/nwaku/apps/wakunode2/wakunode2.nim(37, 29) template/generic instantiation of `load` from here +/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(578, 28) template/generic instantiation of `load` from here +/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(585, 18) template/generic instantiation of `addConfigFile` from here +/home/sionois/Github/nwaku/vendor/nim-confutils/confutils.nim(860, 39) template/generic instantiation of `loadFile` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization.nim(156, 13) template/generic instantiation of `readValue` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization.nim(31, 9) template/generic instantiation of `readValue` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(501, 8) template/generic instantiation of `decodeRecord` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(285, 27) template/generic instantiation of `arrayReadersTable` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/private/array_reader.nim(39, 26) template/generic instantiation of `enumAllSerializedFields` from here +/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(152, 32) template/generic instantiation of `enumAllSerializedFieldsImpl` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/private/array_reader.nim(53, 21) template/generic instantiation of `readValue` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(269, 14) template/generic instantiation of `readValue` from here +/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(262, 10) Warning: setLen can potentially expand the sequence, but the element type 'ValidIpAddress' doesn't have a valid default value [UnsafeSetLen] +/home/sionois/Github/nwaku/apps/wakunode2/wakunode2.nim(37, 29) template/generic instantiation of `load` from here +/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(578, 28) template/generic instantiation of `load` from here +/home/sionois/Github/nwaku/vendor/nim-confutils/confutils.nim(1181, 13) template/generic instantiation of `loadImpl` from here +/home/sionois/Github/nwaku/vendor/nim-confutils/confutils.nim(912, 50) template/generic instantiation of `configurationRtti` from here +/home/sionois/Github/nwaku/vendor/nim-confutils/confutils.nim(709, 19) template/generic instantiation of `setField` from here +/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(94, 29) template/generic instantiation of `newSeq` from here +/home/sionois/Github/nwaku/vendor/nimbus-build-system/vendor/Nim/lib/system.nim(680, 10) Warning: Cannot prove that 'result' is initialized. This will become a compile time error in the future. [ProveInit] +/home/sionois/Github/nwaku/apps/wakunode2/wakunode2.nim(37, 29) template/generic instantiation of `load` from here +/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(578, 28) template/generic instantiation of `load` from here +/home/sionois/Github/nwaku/vendor/nim-confutils/confutils.nim(1181, 13) template/generic instantiation of `loadImpl` from here +/home/sionois/Github/nwaku/vendor/nim-confutils/confutils.nim(937, 23) Warning: Cannot prove that 'result' is initialized. This will become a compile time error in the future. [ProveInit] diff --git a/waku/node/peer_manager/peer_manager.nim b/waku/node/peer_manager/peer_manager.nim index 6286eee938..f108e2442c 100644 --- a/waku/node/peer_manager/peer_manager.nim +++ b/waku/node/peer_manager/peer_manager.nim @@ -342,7 +342,12 @@ proc getPeerIp(pm: PeerManager, peerId: PeerId): Option[string] = # called when a connection i) is created or ii) is closed proc onConnEvent(pm: PeerManager, peerId: PeerID, event: ConnEvent) {.async.} = - discard + case event.kind + of ConnEventKind.Connected: + #let direction = if event.incoming: Inbound else: Outbound + discard + of ConnEventKind.Disconnected: + discard proc onPeerMetadata(pm: PeerManager, peerId: PeerId) {.async.} = # To prevent metadata protocol from breaking prev nodes, by now we only @@ -351,38 +356,35 @@ proc onPeerMetadata(pm: PeerManager, peerId: PeerId) {.async.} = return let res = catch: await pm.switch.dial(peerId, WakuMetadataCodec) - let conn = res.valueOr: - info "disconnecting from peer", peerId=peerId, reason="dial failed: " & error.msg - asyncSpawn(pm.switch.disconnect(peerId)) - pm.peerStore.delete(peerId) - return - - let metadata = (await pm.wakuMetadata.request(conn)).valueOr: - info "disconnecting from peer", peerId=peerId, reason="waku metatdata request failed: " & error - asyncSpawn(pm.switch.disconnect(peerId)) - pm.peerStore.delete(peerId) - return - let clusterId = metadata.clusterId.valueOr: - info "disconnecting from peer", peerId=peerId, reason="empty cluster-id reported" - asyncSpawn(pm.switch.disconnect(peerId)) - pm.peerStore.delete(peerId) - return + var reason: string + block guardClauses: + let conn = res.valueOr: + reason = "dial failed: " & error.msg + break guardClauses + + let metadata = (await pm.wakuMetadata.request(conn)).valueOr: + reason = "waku metatdata request failed: " & error + break guardClauses - if pm.wakuMetadata.clusterId != clusterId: - info "disconnecting from peer", - peerId=peerId, - reason="different clusterId reported: " & $pm.wakuMetadata.clusterId & " vs " & $clusterId - asyncSpawn(pm.switch.disconnect(peerId)) - pm.peerStore.delete(peerId) - return + let clusterId = metadata.clusterId.valueOr: + reason = "empty cluster-id reported" + break guardClauses + + if pm.wakuMetadata.clusterId != clusterId: + reason = "different clusterId reported: " & $pm.wakuMetadata.clusterId & " vs " & $clusterId + break guardClauses + + if not metadata.shards.anyIt(pm.wakuMetadata.shards.contains(it)): + reason = "no shards in common" + break guardClauses - if not metadata.shards.anyIt(pm.wakuMetadata.shards.contains(it)): - info "disconnecting from peer", peerId=peerId, reason="no shards in common" - asyncSpawn(pm.switch.disconnect(peerId)) - pm.peerStore.delete(peerId) return + info "disconnecting from peer", peerId=peerId, reason=reason + asyncSpawn(pm.switch.disconnect(peerId)) + pm.peerStore.delete(peerId) + # called when a peer i) first connects to us ii) disconnects all connections from us proc onPeerEvent(pm: PeerManager, peerId: PeerId, event: PeerEvent) {.async.} = if not pm.wakuMetadata.isNil() and event.kind == PeerEventKind.Joined: @@ -719,15 +721,15 @@ proc manageRelayPeers*(pm: PeerManager) {.async.} = let relayCount = connectablePeers.len debug "Sharded Peer Management", - Shard = shard, - Connectable = $connectableCount & "/" & $shardCount, - RelayConnectable = $relayCount & "/" & $shardCount, - RelayInboundTarget = $connectedInPeers.len & "/" & $inTarget, - RelayOutboundTarget = $connectedOutPeers.len & "/" & $outTarget + shard = shard, + connectable = $connectableCount & "/" & $shardCount, + relayConnectable = $relayCount & "/" & $shardCount, + relayInboundTarget = $connectedInPeers.len & "/" & $inTarget, + relayOutboundTarget = $connectedOutPeers.len & "/" & $outTarget let length = min(outPeerDiff, connectablePeers.len) for peer in connectablePeers[0.. Date: Wed, 6 Dec 2023 16:09:59 -0500 Subject: [PATCH 16/16] Don't store enr cap in protobook --- log.txt | 201 --------------------- tests/test_peer_manager.nim | 12 +- waku/node/peer_manager/peer_manager.nim | 22 +-- waku/node/peer_manager/waku_peer_store.nim | 10 +- 4 files changed, 18 insertions(+), 227 deletions(-) delete mode 100644 log.txt diff --git a/log.txt b/log.txt deleted file mode 100644 index 56399e71d6..0000000000 --- a/log.txt +++ /dev/null @@ -1,201 +0,0 @@ -Building: install/usr/lib/libbacktracenim.a -Building: build/wakunode2 -[NimScript] exec: nim c --out:build/wakunode2 --verbosity:0 --hints:off -d:chronicles_log_level=TRACE -d:git_version="v0.22.0-rc.0-42-ga4deb2" -d:release --passL:librln_v0.3.4.a --passL:-lm apps/wakunode2/wakunode2.nim -/home/sionois/Github/nwaku/vendor/nim-json-rpc/json_rpc/client.nim(33, 11) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] -/home/sionois/Github/nwaku/vendor/nim-json-rpc/json_rpc/client.nim(37, 11) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] -/home/sionois/Github/nwaku/vendor/nim-websock/websock/http/client.nim(173, 5) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] -/home/sionois/Github/nwaku/vendor/nim-websock/websock/websock.nim(257, 5) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] -/home/sionois/Github/nwaku/vendor/nim-libp2p/libp2p/nameresolving/nameresolver.nim(55, 5) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] -/home/sionois/Github/nwaku/waku/waku_rln_relay/conversion_utils.nim(10, 7) Warning: imported and not used: 'results' [UnusedImport] -/home/sionois/Github/nwaku/waku/waku_rln_relay/rln/wrappers.nim(9, 12) Warning: imported and not used: 'utils' [UnusedImport] -/home/sionois/Github/nwaku/waku/waku_rln_relay/rln/wrappers.nim(8, 6) Warning: imported and not used: 'strformat' [UnusedImport] -/home/sionois/Github/nwaku/waku/waku_rln_relay/group_manager/on_chain/group_manager.nim(101, 82) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/waku/waku_rln_relay/group_manager/on_chain/group_manager.nim(98, 8) Warning: use {.base.} for base methods; baseless methods are deprecated [UseBase] -/home/sionois/Github/nwaku/waku/common/logging.nim(4, 6) Warning: imported and not used: 'strutils' [UnusedImport] -/home/sionois/Github/nwaku/vendor/nim-libp2p/libp2p/protocols/pubsub/pubsubpeer.nim(237, 64) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] -/home/sionois/Github/nwaku/vendor/nim-libp2p/libp2p/protocols/pubsub/gossipsub.nim(388, 134) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] -/home/sionois/Github/nwaku/vendor/nim-json-rpc/json_rpc/router.nim(90, 64) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/vendor/nim-json-rpc/json_rpc/router.nim(107, 36) Warning: catch a more precise Exception deriving from CatchableError or Defect. [BareExcept] -/home/sionois/Github/nwaku/waku/common/databases/dburl.nim(10, 15) Warning: use re2(static string) instead; re is deprecated [Deprecated] -/home/sionois/Github/nwaku/waku/common/databases/dburl.nim(12, 67) Warning: use match(string, Regex2) instead; match is deprecated [Deprecated] -/home/sionois/Github/nwaku/waku/waku_archive/driver/queue_driver/queue_driver.nim(293, 63) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/waku/waku_archive/driver/queue_driver/queue_driver.nim(292, 8) Warning: use {.base.} for base methods; baseless methods are deprecated [UseBase] -/home/sionois/Github/nwaku/waku/waku_archive/retention_policy/retention_policy_size.nim(7, 6) Warning: imported and not used: 'times' [UnusedImport] -/home/sionois/Github/nwaku/waku/waku_archive/retention_policy/retention_policy_size.nim(11, 3) Warning: imported and not used: 'os' [UnusedImport] -/home/sionois/Github/nwaku/vendor/nim-libp2p/libp2p/protocols/connectivity/relay/rtransport.nim(32, 62) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/vendor/nim-libp2p/libp2p/protocols/connectivity/relay/rtransport.nim(40, 25) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] -/home/sionois/Github/nwaku/vendor/nim-libp2p/libp2p/builders.nim(230, 32) template/generic instantiation of `new` from here -/home/sionois/Github/nwaku/vendor/nim-libp2p/libp2p/upgrademngrs/muxedupgrade.nim(102, 7) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] -/home/sionois/Github/nwaku/waku/common/utils/sequence.nim(6, 11) Warning: imported and not used: 'sequtils' [UnusedImport] -/home/sionois/Github/nwaku/waku/node/peer_manager/peer_manager.nim(8, 6) Warning: imported and not used: 'sugar' [UnusedImport] -/home/sionois/Github/nwaku/waku/waku_archive/archive.nim(18, 21) Warning: imported and not used: 'retention_policy_capacity' [UnusedImport] -/home/sionois/Github/nwaku/waku/waku_archive/archive.nim(19, 21) Warning: imported and not used: 'retention_policy_time' [UnusedImport] -/home/sionois/Github/nwaku/waku/waku_archive/archive.nim(14, 22) Warning: imported and not used: 'dburl' [UnusedImport] -/home/sionois/Github/nwaku/waku/waku_archive/archive.nim(15, 22) Warning: imported and not used: 'db_sqlite' [UnusedImport] -/home/sionois/Github/nwaku/waku/waku_archive/archive.nim(11, 3) Warning: imported and not used: 'regex' [UnusedImport] -/home/sionois/Github/nwaku/waku/waku_store/protocol.nim(49, 51) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/waku/waku_store/protocol.nim(52, 31) template/generic instantiation of `decode` from here -/home/sionois/Github/nwaku/waku/waku_store/rpc_codec.nim(250, 39) template/generic instantiation of `decode` from here -/home/sionois/Github/nwaku/waku/waku_store/rpc_codec.nim(217, 40) template/generic instantiation of `parse` from here -/home/sionois/Github/nwaku/waku/waku_store/rpc.nim(88, 28) Warning: conversion to enum with holes is unsafe: HistoryResponseErrorRPC(kind) [HoleEnumConv] -/home/sionois/Github/nwaku/waku/waku_filter_v2/client.nim(39, 27) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/waku/waku_filter_v2/client.nim(74, 10) template/generic instantiation of `setResult` from here -/home/sionois/Github/nwaku/waku/waku_filter_v2/client.nim(72, 70) template/generic instantiation of `setResult` from here -/home/sionois/Github/nwaku/waku/waku_filter_v2/common.nim(67, 38) template/generic instantiation of `parse` from here -/home/sionois/Github/nwaku/waku/waku_filter_v2/common.nim(62, 29) Warning: conversion to enum with holes is unsafe: FilterSubscribeErrorKind(kind) [HoleEnumConv] -/home/sionois/Github/nwaku/vendor/nim-dnsdisc/discovery/dnsdisc/client.nim(67, 121) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] -/home/sionois/Github/nwaku/waku/waku_rln_relay/rln_relay.nim(82, 8) Warning: use {.base.} for base methods; baseless methods are deprecated [UseBase] -/home/sionois/Github/nwaku/waku/waku_rln_relay/rln_relay.nim(257, 7) Warning: Special variable 'result' is shadowed. [ResultShadowed] -/home/sionois/Github/nwaku/waku/waku_rln_relay/rln_relay.nim(27, 11) Warning: imported and not used: 'collector' [UnusedImport] -/home/sionois/Github/nwaku/waku/waku_rln_relay/rln_relay.nim(8, 40) Warning: imported and not used: 'ratelimit' [UnusedImport] -/home/sionois/Github/nwaku/waku/waku_rln_relay/rln_relay.nim(7, 6) Warning: imported and not used: 'os' [UnusedImport] -/home/sionois/Github/nwaku/waku/node/waku_node.nim(407, 20) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] -/home/sionois/Github/nwaku/waku/node/waku_node.nim(432, 43) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] -/home/sionois/Github/nwaku/waku/node/waku_node.nim(452, 31) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] -/home/sionois/Github/nwaku/waku/node/waku_node.nim(533, 19) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] -/home/sionois/Github/nwaku/waku/node/waku_node.nim(594, 33) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] -/home/sionois/Github/nwaku/waku/node/waku_node.nim(651, 19) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] -/home/sionois/Github/nwaku/waku/node/waku_node.nim(714, 19) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] -/home/sionois/Github/nwaku/waku/node/waku_node.nim(811, 36) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] -/home/sionois/Github/nwaku/waku/node/waku_node.nim(992, 43) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] -/home/sionois/Github/nwaku/waku/node/waku_node.nim(1002, 64) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] -/home/sionois/Github/nwaku/waku/node/waku_node.nim(1040, 41) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] -/home/sionois/Github/nwaku/waku/node/waku_node.nim(1082, 41) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/vendor/nim-chronos/chronos/internal/asyncmacro.nim(198, 14) Warning: The raises pragma doesn't work on async procedure. Please remove it or use asyncraises instead [User] -/home/sionois/Github/nwaku/waku/node/waku_switch.nim(8, 6) Warning: imported and not used: 'math' [UnusedImport] -/home/sionois/Github/nwaku/waku/waku_api/handlers.nim(8, 3) Warning: imported and not used: 'chronicles' [UnusedImport] -/home/sionois/Github/nwaku/waku/waku_api/rest/filter/handlers.nim(113, 5) Warning: unreachable else, all cases are already covered [UnreachableElse] -/home/sionois/Github/nwaku/waku/waku_api/rest/lightpush/types.nim(11, 9) Warning: imported and not used: 'common' [UnusedImport] -/home/sionois/Github/nwaku/waku/waku_api/rest/lightpush/types.nim(14, 18) Warning: imported and not used: 'base64' [UnusedImport] -/home/sionois/Github/nwaku/waku/waku_api/rest/store/types.nim(8, 7) Warning: imported and not used: 'byteutils' [UnusedImport] -/home/sionois/Github/nwaku/waku/waku_api/rest/health/handlers.nim(13, 3) Warning: imported and not used: 'serdes' [UnusedImport] -/home/sionois/Github/nwaku/waku/waku_api/rest/health/handlers.nim(8, 3) Warning: imported and not used: 'json_serialization' [UnusedImport] -/home/sionois/Github/nwaku/waku/waku_api/rest/admin/handlers.nim(9, 7) Warning: imported and not used: 'byteutils' [UnusedImport] -/home/sionois/Github/nwaku/waku/waku_api/jsonrpc/filter/handlers.nim(14, 23) Warning: imported and not used: 'client' [UnusedImport] -/home/sionois/Github/nwaku/waku/waku_api/jsonrpc/relay/handlers.nim(17, 30) Warning: imported and not used: 'wrappers' [UnusedImport] -/home/sionois/Github/nwaku/waku/waku_api/jsonrpc/relay/handlers.nim(11, 12) Warning: imported and not used: 'sysrand' [UnusedImport] -/home/sionois/Github/nwaku/apps/wakunode2/wakunode2_validator_signed.nim(7, 6) Warning: imported and not used: 'math' [UnusedImport] -/home/sionois/Github/nwaku/apps/wakunode2/wakunode2_validator_signed.nim(14, 30) Warning: imported and not used: 'messages' [UnusedImport] -/home/sionois/Github/nwaku/apps/wakunode2/app.nim(405, 47) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/apps/wakunode2/app.nim(570, 10) template/generic instantiation of `setResult` from here -/home/sionois/Github/nwaku/apps/wakunode2/app.nim(499, 39) template/generic instantiation of `new` from here -/home/sionois/Github/nwaku/waku/waku_archive/retention_policy/builder.nim(25, 44) Warning: use re2(static string) instead; re is deprecated [Deprecated] -/home/sionois/Github/nwaku/apps/wakunode2/app.nim(405, 47) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/apps/wakunode2/app.nim(570, 10) template/generic instantiation of `setResult` from here -/home/sionois/Github/nwaku/apps/wakunode2/app.nim(499, 39) template/generic instantiation of `new` from here -/home/sionois/Github/nwaku/waku/waku_archive/retention_policy/builder.nim(26, 20) Warning: use match(string, Regex2) instead; match is deprecated [Deprecated] -/home/sionois/Github/nwaku/apps/wakunode2/app.nim(658, 59) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/apps/wakunode2/app.nim(675, 5) Warning: unreachable code after 'return' statement or '{.noReturn.}' proc [UnreachableCode] -/home/sionois/Github/nwaku/apps/wakunode2/app.nim(658, 59) template/generic instantiation of `async` from here -/home/sionois/Github/nwaku/apps/wakunode2/app.nim(675, 5) Warning: unreachable code after 'return' statement or '{.noReturn.}' proc [UnreachableCode] -/home/sionois/Github/nwaku/apps/wakunode2/app.nim(23, 26) Warning: imported and not used: 'nat' [UnusedImport] -/home/sionois/Github/nwaku/apps/wakunode2/wakunode2.nim(37, 29) template/generic instantiation of `load` from here -/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(578, 28) template/generic instantiation of `load` from here -/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(585, 18) template/generic instantiation of `addConfigFile` from here -/home/sionois/Github/nwaku/vendor/nim-confutils/confutils.nim(860, 39) template/generic instantiation of `loadFile` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization.nim(156, 13) template/generic instantiation of `readValue` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization.nim(31, 9) template/generic instantiation of `readValue` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(499, 8) template/generic instantiation of `decodeInlineTable` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(377, 19) template/generic instantiation of `fieldReadersTable` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(256, 34) template/generic instantiation of `makeFieldReadersTable` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(218, 26) template/generic instantiation of `enumAllSerializedFields` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(152, 32) template/generic instantiation of `enumAllSerializedFieldsImpl` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(235, 52) template/generic instantiation of `readFieldIMPL` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(203, 13) template/generic instantiation of `readValue` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization.nim(31, 9) template/generic instantiation of `readValue` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(445, 19) template/generic instantiation of `readValue` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization.nim(31, 9) template/generic instantiation of `readValue` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(481, 6) template/generic instantiation of `parseList` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(483, 12) Warning: setLen can potentially expand the sequence, but the element type 'ProtectedTopic' doesn't have a valid default value [UnsafeSetLen] -/home/sionois/Github/nwaku/apps/wakunode2/wakunode2.nim(37, 29) template/generic instantiation of `load` from here -/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(578, 28) template/generic instantiation of `load` from here -/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(585, 18) template/generic instantiation of `addConfigFile` from here -/home/sionois/Github/nwaku/vendor/nim-confutils/confutils.nim(860, 39) template/generic instantiation of `loadFile` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization.nim(156, 13) template/generic instantiation of `readValue` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization.nim(31, 9) template/generic instantiation of `readValue` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(499, 8) template/generic instantiation of `decodeInlineTable` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(377, 19) template/generic instantiation of `fieldReadersTable` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(256, 34) template/generic instantiation of `makeFieldReadersTable` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(218, 26) template/generic instantiation of `enumAllSerializedFields` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(152, 32) template/generic instantiation of `enumAllSerializedFieldsImpl` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(235, 52) template/generic instantiation of `readFieldIMPL` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(203, 13) template/generic instantiation of `readValue` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization.nim(31, 9) template/generic instantiation of `readValue` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(445, 19) template/generic instantiation of `readValue` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization.nim(30, 19) Warning: The 'ValidIpAddress' type doesn't have a valid default value [UnsafeDefault] -/home/sionois/Github/nwaku/apps/wakunode2/wakunode2.nim(37, 29) template/generic instantiation of `load` from here -/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(578, 28) template/generic instantiation of `load` from here -/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(585, 18) template/generic instantiation of `addConfigFile` from here -/home/sionois/Github/nwaku/vendor/nim-confutils/confutils.nim(860, 39) template/generic instantiation of `loadFile` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization.nim(156, 13) template/generic instantiation of `readValue` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization.nim(31, 9) template/generic instantiation of `readValue` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(499, 8) template/generic instantiation of `decodeInlineTable` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(377, 19) template/generic instantiation of `fieldReadersTable` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(256, 34) template/generic instantiation of `makeFieldReadersTable` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(218, 26) template/generic instantiation of `enumAllSerializedFields` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(152, 32) template/generic instantiation of `enumAllSerializedFieldsImpl` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(235, 52) template/generic instantiation of `readFieldIMPL` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(203, 13) template/generic instantiation of `readValue` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization.nim(31, 9) template/generic instantiation of `readValue` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(445, 19) template/generic instantiation of `readValue` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization.nim(31, 9) template/generic instantiation of `readValue` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(481, 6) template/generic instantiation of `parseList` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(483, 12) Warning: setLen can potentially expand the sequence, but the element type 'ValidIpAddress' doesn't have a valid default value [UnsafeSetLen] -/home/sionois/Github/nwaku/apps/wakunode2/wakunode2.nim(37, 29) template/generic instantiation of `load` from here -/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(578, 28) template/generic instantiation of `load` from here -/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(585, 18) template/generic instantiation of `addConfigFile` from here -/home/sionois/Github/nwaku/vendor/nim-confutils/confutils.nim(860, 39) template/generic instantiation of `loadFile` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization.nim(156, 13) template/generic instantiation of `readValue` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization.nim(31, 9) template/generic instantiation of `readValue` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(501, 8) template/generic instantiation of `decodeRecord` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(285, 27) template/generic instantiation of `arrayReadersTable` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/private/array_reader.nim(39, 26) template/generic instantiation of `enumAllSerializedFields` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(152, 32) template/generic instantiation of `enumAllSerializedFieldsImpl` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/private/array_reader.nim(53, 21) template/generic instantiation of `readValue` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(269, 14) template/generic instantiation of `readValue` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(262, 10) Warning: setLen can potentially expand the sequence, but the element type 'ProtectedTopic' doesn't have a valid default value [UnsafeSetLen] -/home/sionois/Github/nwaku/apps/wakunode2/wakunode2.nim(37, 29) template/generic instantiation of `load` from here -/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(578, 28) template/generic instantiation of `load` from here -/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(585, 18) template/generic instantiation of `addConfigFile` from here -/home/sionois/Github/nwaku/vendor/nim-confutils/confutils.nim(860, 39) template/generic instantiation of `loadFile` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization.nim(156, 13) template/generic instantiation of `readValue` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization.nim(31, 9) template/generic instantiation of `readValue` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(501, 8) template/generic instantiation of `decodeRecord` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(285, 27) template/generic instantiation of `arrayReadersTable` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/private/array_reader.nim(39, 26) template/generic instantiation of `enumAllSerializedFields` from here -/home/sionois/Github/nwaku/vendor/nim-serialization/serialization/object_serialization.nim(152, 32) template/generic instantiation of `enumAllSerializedFieldsImpl` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/private/array_reader.nim(53, 21) template/generic instantiation of `readValue` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(269, 14) template/generic instantiation of `readValue` from here -/home/sionois/Github/nwaku/vendor/nim-toml-serialization/toml_serialization/reader.nim(262, 10) Warning: setLen can potentially expand the sequence, but the element type 'ValidIpAddress' doesn't have a valid default value [UnsafeSetLen] -/home/sionois/Github/nwaku/apps/wakunode2/wakunode2.nim(37, 29) template/generic instantiation of `load` from here -/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(578, 28) template/generic instantiation of `load` from here -/home/sionois/Github/nwaku/vendor/nim-confutils/confutils.nim(1181, 13) template/generic instantiation of `loadImpl` from here -/home/sionois/Github/nwaku/vendor/nim-confutils/confutils.nim(912, 50) template/generic instantiation of `configurationRtti` from here -/home/sionois/Github/nwaku/vendor/nim-confutils/confutils.nim(709, 19) template/generic instantiation of `setField` from here -/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(94, 29) template/generic instantiation of `newSeq` from here -/home/sionois/Github/nwaku/vendor/nimbus-build-system/vendor/Nim/lib/system.nim(680, 10) Warning: Cannot prove that 'result' is initialized. This will become a compile time error in the future. [ProveInit] -/home/sionois/Github/nwaku/apps/wakunode2/wakunode2.nim(37, 29) template/generic instantiation of `load` from here -/home/sionois/Github/nwaku/apps/wakunode2/external_config.nim(578, 28) template/generic instantiation of `load` from here -/home/sionois/Github/nwaku/vendor/nim-confutils/confutils.nim(1181, 13) template/generic instantiation of `loadImpl` from here -/home/sionois/Github/nwaku/vendor/nim-confutils/confutils.nim(937, 23) Warning: Cannot prove that 'result' is initialized. This will become a compile time error in the future. [ProveInit] diff --git a/tests/test_peer_manager.nim b/tests/test_peer_manager.nim index c2de9d3239..a480adc500 100644 --- a/tests/test_peer_manager.nim +++ b/tests/test_peer_manager.nim @@ -22,6 +22,7 @@ import ../../waku/node/peer_manager/peer_store/waku_peer_storage, ../../waku/waku_node, ../../waku/waku_core, + ../../waku/waku_enr/capabilities, ../../waku/waku_relay/protocol, ../../waku/waku_store/common, ../../waku/waku_filter/protocol, @@ -402,7 +403,16 @@ procSuite "Peer Manager": asyncTest "Peer manager connects to all peers supporting a given protocol": # Create 4 nodes - let nodes = toSeq(0..<4).mapIt(newTestWakuNode(generateSecp256k1Key(), ValidIpAddress.init("0.0.0.0"), Port(0))) + let nodes = + toSeq(0..<4) + .mapIt( + newTestWakuNode( + nodeKey = generateSecp256k1Key(), + bindIp = ValidIpAddress.init("0.0.0.0"), + bindPort = Port(0), + wakuFlags = some(CapabilitiesBitfield.init(@[Relay])) + ) + ) # Start them discard nodes.mapIt(it.mountMetadata(0)) diff --git a/waku/node/peer_manager/peer_manager.nim b/waku/node/peer_manager/peer_manager.nim index f108e2442c..dd77aefb60 100644 --- a/waku/node/peer_manager/peer_manager.nim +++ b/waku/node/peer_manager/peer_manager.nim @@ -133,25 +133,7 @@ proc addPeer*(pm: PeerManager, remotePeerInfo: RemotePeerInfo, origin = UnknownO if remotePeerInfo.protocols.len > 0: pm.peerStore[ProtoBook][remotePeerInfo.peerId] = remotePeerInfo.protocols - elif remotePeerInfo.enr.isSome(): - # RemotePeerInfo from discv5 always have empty protocol list - let caps = remotePeerInfo.enr.get().getCapabilities() - - var protos: seq[string] - for cap in caps: - case cap: - of Relay: - protos.add(WakuRelayCodec) - of Store: - protos.add(WakuStoreCodec) - of Filter: - protos.add(WakuFilterSubscribeCodec) - protos.add(WakuFilterPushCodec) - of LightPush: - protos.add(WakuLightPushCodec) - - pm.peerStore[ProtoBook][remotePeerInfo.peerId] = protos - + if remotePeerInfo.enr.isSome(): pm.peerStore[ENRBook][remotePeerInfo.peerId] = remotePeerInfo.enr.get() @@ -716,7 +698,7 @@ proc manageRelayPeers*(pm: PeerManager) {.async.} = let connectableCount = connectablePeers.len - connectablePeers.keepItIf(it.protocols.contains(WakuRelayCodec)) + connectablePeers.keepItIf(pm.peerStore.hasCapability(it.peerId, Relay)) let relayCount = connectablePeers.len diff --git a/waku/node/peer_manager/waku_peer_store.nim b/waku/node/peer_manager/waku_peer_store.nim index 6c426fce48..579ae395a8 100644 --- a/waku/node/peer_manager/waku_peer_store.nim +++ b/waku/node/peer_manager/waku_peer_store.nim @@ -97,13 +97,13 @@ proc peers*(peerStore: PeerStore, protocolMatcher: Matcher): seq[RemotePeerInfo] peerStore.peers.filterIt(it.protocols.anyIt(protocolMatcher(it))) proc connectedness*(peerStore: PeerStore, peerId: PeerID): Connectedness = - # Return the connection state of the given, managed peer - # TODO: the PeerManager should keep and update local connectedness state for peers, redial on disconnect, etc. - # TODO: richer return than just bool, e.g. add enum "CanConnect", "CannotConnect", etc. based on recent connection attempts - return peerStore[ConnectionBook].book.getOrDefault(peerId, NotConnected) + peerStore[ConnectionBook].book.getOrDefault(peerId, NotConnected) proc hasShard*(peerStore: PeerStore, peerId: PeerID, cluster, shard: uint16): bool = - peerStore[ENRBook][peerId].containsShard(cluster, shard) + peerStore[ENRBook].book.getOrDefault(peerId).containsShard(cluster, shard) + +proc hasCapability*(peerStore: PeerStore, peerId: PeerID, cap: Capabilities): bool = + peerStore[ENRBook].book.getOrDefault(peerId).supportsCapability(cap) proc isConnected*(peerStore: PeerStore, peerId: PeerID): bool = # Returns `true` if the peer is connected