mod bucket; mod bucket_entry; mod debug; mod node_ref; mod node_ref_filter; mod privacy; mod route_spec_store; mod routing_domain_editor; mod routing_domains; mod routing_table_inner; mod stats_accounting; mod tasks; use crate::*; use crate::crypto::*; use crate::network_manager::*; use crate::rpc_processor::*; use bucket::*; pub use bucket_entry::*; pub use debug::*; use hashlink::LruCache; pub use node_ref::*; pub use node_ref_filter::*; pub use privacy::*; pub use route_spec_store::*; pub use routing_domain_editor::*; pub use routing_domains::*; pub use routing_table_inner::*; pub use stats_accounting::*; ////////////////////////////////////////////////////////////////////////// /// How frequently we tick the relay management routine pub const RELAY_MANAGEMENT_INTERVAL_SECS: u32 = 1; /// How frequently we tick the private route management routine pub const PRIVATE_ROUTE_MANAGEMENT_INTERVAL_SECS: u32 = 1; // Connectionless protocols like UDP are dependent on a NAT translation timeout // We should ping them with some frequency and 30 seconds is typical timeout pub const CONNECTIONLESS_TIMEOUT_SECS: u32 = 29; pub type LowLevelProtocolPorts = BTreeSet<(LowLevelProtocolType, AddressType, u16)>; pub type ProtocolToPortMapping = BTreeMap<(ProtocolType, AddressType), (LowLevelProtocolType, u16)>; #[derive(Clone, Debug)] pub struct LowLevelPortInfo { pub low_level_protocol_ports: LowLevelProtocolPorts, pub protocol_to_port: ProtocolToPortMapping, } pub type RoutingTableEntryFilter<'t> = Box>) -> bool + Send + 't>; #[derive(Clone, Debug, Default, Eq, PartialEq)] pub struct RoutingTableHealth { /// Number of reliable (responsive) entries in the routing table pub reliable_entry_count: usize, /// Number of unreliable (occasionally unresponsive) entries in the routing table pub unreliable_entry_count: usize, /// Number of dead (always unresponsive) entries in the routing table pub dead_entry_count: usize, /// If PublicInternet network class is valid yet pub public_internet_ready: bool, /// If LocalNetwork network class is valid yet pub local_network_ready: bool, } pub(super) struct RoutingTableUnlockedInner { // Accessors config: VeilidConfig, network_manager: NetworkManager, /// The current node's public DHT keys and secrets node_id_keypairs: BTreeMap, /// Buckets to kick on our next kick task kick_queue: Mutex>, /// Background process for computing statistics rolling_transfers_task: TickTask, /// Background process to purge dead routing table entries when necessary kick_buckets_task: TickTask, /// Background process to get our initial routing table bootstrap_task: TickTask, /// Background process to ensure we have enough nodes in our routing table peer_minimum_refresh_task: TickTask, /// Background process to check nodes to see if they are still alive and for reliability ping_validator_task: TickTask, /// Background process to keep relays up relay_management_task: TickTask, /// Background process to keep private routes up private_route_management_task: TickTask, } impl RoutingTableUnlockedInner { pub fn network_manager(&self) -> NetworkManager { self.network_manager.clone() } pub fn crypto(&self) -> Crypto { self.network_manager().crypto() } pub fn rpc_processor(&self) -> RPCProcessor { self.network_manager().rpc_processor() } pub fn update_callback(&self) -> UpdateCallback { self.network_manager().update_callback() } pub fn with_config(&self, f: F) -> R where F: FnOnce(&VeilidConfigInner) -> R, { f(&*self.config.get()) } pub fn node_id(&self, kind: CryptoKind) -> TypedKey { TypedKey::new(kind, self.node_id_keypairs.get(&kind).unwrap().key) } pub fn node_ids(&self) -> TypedKeySet { let mut tks = TypedKeySet::new(); for x in &self.node_id_keypairs { tks.add(TypedKey::new(*x.0, x.1.key)); } tks } pub fn node_id_typed_key_pairs(&self) -> Vec { let mut tkps = Vec::new(); for (ck, v) in &self.node_id_keypairs { tkps.push(TypedKeyPair::new(*ck, v.key, v.secret)); } tkps } pub fn node_id_secret(&self, kind: CryptoKind) -> SecretKey { self.node_id_keypairs.get(&kind).unwrap().secret } pub fn matches_own_node_id(&self, node_ids: &[TypedKey]) -> bool { for ni in node_ids { if let Some(v) = self.node_id_keypairs.get(&ni.kind) { if v.key == ni.key { return true; } } } false } pub fn matches_own_node_id_key(&self, node_id_key: &PublicKey) -> bool { for (ck, v) in &self.node_id_keypairs { if v.key == *node_id_key { return true; } } false } pub fn calculate_bucket_index(&self, node_id: &TypedKey) -> (CryptoKind, usize) { let crypto = self.crypto(); let self_node_id = self.node_id_keypairs.get(&node_id.kind).unwrap().key; let vcrypto = crypto.get(node_id.kind).unwrap(); ( node_id.kind, vcrypto .distance(&node_id.key, &self_node_id) .first_nonzero_bit() .unwrap(), ) } } #[derive(Clone)] pub struct RoutingTable { inner: Arc>, unlocked_inner: Arc, } impl RoutingTable { fn new_unlocked_inner( config: VeilidConfig, network_manager: NetworkManager, ) -> RoutingTableUnlockedInner { let c = config.get(); RoutingTableUnlockedInner { config: config.clone(), network_manager, node_id_keypairs: c .network .routing_table .node_ids .iter() .map(|(k, v)| { ( *k, KeyPair::new(v.node_id.unwrap(), v.node_id_secret.unwrap()), ) }) .collect(), kick_queue: Mutex::new(BTreeSet::default()), rolling_transfers_task: TickTask::new(ROLLING_TRANSFERS_INTERVAL_SECS), kick_buckets_task: TickTask::new(1), bootstrap_task: TickTask::new(1), peer_minimum_refresh_task: TickTask::new_ms(c.network.dht.min_peer_refresh_time_ms), ping_validator_task: TickTask::new(1), relay_management_task: TickTask::new(RELAY_MANAGEMENT_INTERVAL_SECS), private_route_management_task: TickTask::new(PRIVATE_ROUTE_MANAGEMENT_INTERVAL_SECS), } } pub fn new(network_manager: NetworkManager) -> Self { let config = network_manager.config(); let unlocked_inner = Arc::new(Self::new_unlocked_inner(config, network_manager)); let inner = Arc::new(RwLock::new(RoutingTableInner::new(unlocked_inner.clone()))); let this = Self { inner, unlocked_inner, }; this.start_tasks(); this } ///////////////////////////////////// /// Initialization /// Called to initialize the routing table after it is created pub async fn init(&self) -> EyreResult<()> { debug!("starting routing table init"); // Set up routing buckets { let mut inner = self.inner.write(); inner.init_buckets(self.clone()); } // Load bucket entries from table db if possible debug!("loading routing table entries"); if let Err(e) = self.load_buckets().await { log_rtab!(debug "Error loading buckets from storage: {:#?}. Resetting.", e); let mut inner = self.inner.write(); inner.init_buckets(self.clone()); } // Set up routespecstore debug!("starting route spec store init"); let route_spec_store = match RouteSpecStore::load(self.clone()).await { Ok(v) => v, Err(e) => { log_rtab!(debug "Error loading route spec store: {:#?}. Resetting.", e); RouteSpecStore::new(self.clone()) } }; debug!("finished route spec store init"); { let mut inner = self.inner.write(); inner.route_spec_store = Some(route_spec_store); } debug!("finished routing table init"); Ok(()) } /// Called to shut down the routing table pub async fn terminate(&self) { debug!("starting routing table terminate"); // Stop tasks self.stop_tasks().await; // Load bucket entries from table db if possible debug!("saving routing table entries"); if let Err(e) = self.save_buckets().await { error!("failed to save routing table entries: {}", e); } debug!("saving route spec store"); let rss = { let mut inner = self.inner.write(); inner.route_spec_store.take() }; if let Some(rss) = rss { if let Err(e) = rss.save().await { error!("couldn't save route spec store: {}", e); } } debug!("shutting down routing table"); let mut inner = self.inner.write(); *inner = RoutingTableInner::new(self.unlocked_inner.clone()); debug!("finished routing table terminate"); } /// Serialize routing table to table store async fn save_buckets(&self) -> EyreResult<()> { // Since entries are shared by multiple buckets per cryptokind // we need to get the list of all unique entries when serializing let mut all_entries: Vec> = Vec::new(); // Serialize all buckets and get map of entries let mut serialized_bucket_map: BTreeMap>> = BTreeMap::new(); { let mut entry_map: HashMap<*const BucketEntry, u32> = HashMap::new(); let inner = &*self.inner.read(); for ck in VALID_CRYPTO_KINDS { let buckets = inner.buckets.get(&ck).unwrap(); let mut serialized_buckets = Vec::new(); for bucket in buckets.iter() { serialized_buckets.push(bucket.save_bucket(&mut all_entries, &mut entry_map)?) } serialized_bucket_map.insert(ck, serialized_buckets); } } // Serialize all the entries let mut all_entry_bytes = Vec::with_capacity(all_entries.len()); for entry in all_entries { // Serialize entry let entry_bytes = entry.with_inner(|e| to_rkyv(e))?; all_entry_bytes.push(entry_bytes); } let table_store = self.unlocked_inner.network_manager().table_store(); let tdb = table_store.open("routing_table", 1).await?; let dbx = tdb.transact(); if let Err(e) = dbx.store_rkyv(0, b"serialized_bucket_map", &serialized_bucket_map) { dbx.rollback(); return Err(e); } if let Err(e) = dbx.store_rkyv(0, b"all_entry_bytes", &all_entry_bytes) { dbx.rollback(); return Err(e); } dbx.commit().await?; Ok(()) } /// Deserialize routing table from table store async fn load_buckets(&self) -> EyreResult<()> { // Deserialize bucket map and all entries from the table store let tstore = self.unlocked_inner.network_manager().table_store(); let tdb = tstore.open("routing_table", 1).await?; let Some(serialized_bucket_map): Option>>> = tdb.load_rkyv(0, b"serialized_bucket_map")? else { log_rtab!(debug "no bucket map in saved routing table"); return Ok(()); }; let Some(all_entry_bytes): Option>> = tdb.load_rkyv(0, b"all_entry_bytes")? else { log_rtab!(debug "no all_entry_bytes in saved routing table"); return Ok(()); }; // Reconstruct all entries let mut all_entries: Vec> = Vec::with_capacity(all_entry_bytes.len()); for entry_bytes in all_entry_bytes { let entryinner = from_rkyv(entry_bytes).wrap_err("failed to deserialize bucket entry")?; all_entries.push(Arc::new(BucketEntry::new_with_inner(entryinner))); } // Validate serialized bucket map for (k, v) in &serialized_bucket_map { if !VALID_CRYPTO_KINDS.contains(k) { warn!("crypto kind is not valid, not loading routing table"); return Ok(()); } if v.len() != PUBLIC_KEY_LENGTH * 8 { warn!("bucket count is different, not loading routing table"); return Ok(()); } } // Recreate buckets let inner = &mut *self.inner.write(); for (k, v) in serialized_bucket_map { let buckets = inner.buckets.get_mut(&k).unwrap(); for n in 0..v.len() { buckets[n].load_bucket(v[n].clone(), &all_entries)?; } } Ok(()) } /// Set up the local network routing domain with our local routing table configuration pub fn configure_local_network_routing_domain(&self, local_networks: Vec<(IpAddr, IpAddr)>) { log_net!(debug "configure_local_network_routing_domain: {:#?}", local_networks); self.inner .write() .configure_local_network_routing_domain(local_networks); } ///////////////////////////////////// /// Locked operations pub fn routing_domain_for_address(&self, address: Address) -> Option { self.inner.read().routing_domain_for_address(address) } pub fn route_spec_store(&self) -> RouteSpecStore { self.inner.read().route_spec_store.as_ref().unwrap().clone() } pub fn relay_node(&self, domain: RoutingDomain) -> Option { self.inner.read().relay_node(domain) } pub fn has_dial_info(&self, domain: RoutingDomain) -> bool { self.inner.read().has_dial_info(domain) } pub fn dial_info_details(&self, domain: RoutingDomain) -> Vec { self.inner.read().dial_info_details(domain) } pub fn first_filtered_dial_info_detail( &self, routing_domain_set: RoutingDomainSet, filter: &DialInfoFilter, ) -> Option { self.inner .read() .first_filtered_dial_info_detail(routing_domain_set, filter) } pub fn all_filtered_dial_info_details( &self, routing_domain_set: RoutingDomainSet, filter: &DialInfoFilter, ) -> Vec { self.inner .read() .all_filtered_dial_info_details(routing_domain_set, filter) } pub fn ensure_dial_info_is_valid(&self, domain: RoutingDomain, dial_info: &DialInfo) -> bool { self.inner .read() .ensure_dial_info_is_valid(domain, dial_info) } pub fn node_info_is_valid_in_routing_domain( &self, routing_domain: RoutingDomain, node_info: &NodeInfo, ) -> bool { self.inner .read() .node_info_is_valid_in_routing_domain(routing_domain, node_info) } pub fn signed_node_info_is_valid_in_routing_domain( &self, routing_domain: RoutingDomain, signed_node_info: &SignedNodeInfo, ) -> bool { self.inner .read() .signed_node_info_is_valid_in_routing_domain(routing_domain, signed_node_info) } /// Look up the best way for two nodes to reach each other over a specific routing domain #[instrument(level = "trace", skip(self), ret)] pub fn get_contact_method( &self, routing_domain: RoutingDomain, peer_a: &PeerInfo, peer_b: &PeerInfo, dial_info_filter: DialInfoFilter, sequencing: Sequencing, ) -> ContactMethod { self.inner.read().get_contact_method( routing_domain, peer_a, peer_b, dial_info_filter, sequencing, ) } #[instrument(level = "debug", skip(self))] pub fn edit_routing_domain(&self, domain: RoutingDomain) -> RoutingDomainEditor { RoutingDomainEditor::new(self.clone(), domain) } /// Return a copy of our node's peerinfo pub fn get_own_peer_info(&self, routing_domain: RoutingDomain) -> Option { self.inner.read().get_own_peer_info(routing_domain) } /// Return the best effort copy of our node's peerinfo /// This may be invalid and should not be passed to other nodes, /// but may be used for contact method calculation pub fn get_best_effort_own_peer_info(&self, routing_domain: RoutingDomain) -> PeerInfo { self.inner .read() .get_best_effort_own_peer_info(routing_domain) } /// If we have a valid network class in this routing domain, then our 'NodeInfo' is valid /// If this is true, we can get our final peer info, otherwise we only have a 'best effort' peer info pub fn has_valid_own_node_info(&self, routing_domain: RoutingDomain) -> bool { self.inner.read().has_valid_own_node_info(routing_domain) } /// Return our current node info timestamp pub fn get_own_node_info_ts(&self, routing_domain: RoutingDomain) -> Option { self.inner.read().get_own_node_info_ts(routing_domain) } /// Return the domain's currently registered network class pub fn get_network_class(&self, routing_domain: RoutingDomain) -> Option { self.inner.read().get_network_class(routing_domain) } /// Return the domain's filter for what we can receivein the form of a dial info filter pub fn get_inbound_dial_info_filter(&self, routing_domain: RoutingDomain) -> DialInfoFilter { self.inner .read() .get_inbound_dial_info_filter(routing_domain) } /// Return the domain's filter for what we can receive in the form of a node ref filter pub fn get_inbound_node_ref_filter(&self, routing_domain: RoutingDomain) -> NodeRefFilter { self.inner .read() .get_inbound_node_ref_filter(routing_domain) } /// Return the domain's filter for what we can send out in the form of a dial info filter pub fn get_outbound_dial_info_filter(&self, routing_domain: RoutingDomain) -> DialInfoFilter { self.inner .read() .get_outbound_dial_info_filter(routing_domain) } /// Return the domain's filter for what we can receive in the form of a node ref filter pub fn get_outbound_node_ref_filter(&self, routing_domain: RoutingDomain) -> NodeRefFilter { self.inner .read() .get_outbound_node_ref_filter(routing_domain) } /// Attempt to empty the routing table /// should only be performed when there are no node_refs (detached) pub fn purge_buckets(&self) { self.inner.write().purge_buckets(); } /// Attempt to remove last_connections from entries pub fn purge_last_connections(&self) { self.inner.write().purge_last_connections(); } pub fn get_entry_count( &self, routing_domain_set: RoutingDomainSet, min_state: BucketEntryState, crypto_kinds: &[CryptoKind], ) -> usize { self.inner .read() .get_entry_count(routing_domain_set, min_state, crypto_kinds) } pub fn get_entry_count_per_crypto_kind( &self, routing_domain_set: RoutingDomainSet, min_state: BucketEntryState, ) -> BTreeMap { self.inner .read() .get_entry_count_per_crypto_kind(routing_domain_set, min_state) } pub fn get_nodes_needing_ping( &self, routing_domain: RoutingDomain, cur_ts: Timestamp, ) -> Vec { self.inner .read() .get_nodes_needing_ping(self.clone(), routing_domain, cur_ts) } pub fn get_all_nodes(&self, cur_ts: Timestamp) -> Vec { let inner = self.inner.read(); inner.get_all_nodes(self.clone(), cur_ts) } fn queue_bucket_kicks(&self, node_ids: TypedKeySet) { for node_id in node_ids.iter() { let x = self.unlocked_inner.calculate_bucket_index(node_id); self.unlocked_inner.kick_queue.lock().insert(x); } } /// Resolve an existing routing table entry using any crypto kind and return a reference to it pub fn lookup_any_node_ref(&self, node_id_key: PublicKey) -> Option { self.inner .read() .lookup_any_node_ref(self.clone(), node_id_key) } /// Resolve an existing routing table entry and return a reference to it pub fn lookup_node_ref(&self, node_id: TypedKey) -> Option { self.inner.read().lookup_node_ref(self.clone(), node_id) } /// Resolve an existing routing table entry and return a filtered reference to it pub fn lookup_and_filter_noderef( &self, node_id: TypedKey, routing_domain_set: RoutingDomainSet, dial_info_filter: DialInfoFilter, ) -> Option { self.inner.read().lookup_and_filter_noderef( self.clone(), node_id, routing_domain_set, dial_info_filter, ) } /// Shortcut function to add a node to our routing table if it doesn't exist /// and add the dial info we have for it. Returns a noderef filtered to /// the routing domain in which this node was registered for convenience. pub fn register_node_with_peer_info( &self, routing_domain: RoutingDomain, peer_info: PeerInfo, allow_invalid: bool, ) -> Option { self.inner.write().register_node_with_peer_info( self.clone(), routing_domain, peer_info, allow_invalid, ) } /// Shortcut function to add a node to our routing table if it doesn't exist /// and add the last peer address we have for it, since that's pretty common pub fn register_node_with_existing_connection( &self, node_id: TypedKey, descriptor: ConnectionDescriptor, timestamp: Timestamp, ) -> Option { self.inner.write().register_node_with_existing_connection( self.clone(), node_id, descriptor, timestamp, ) } ////////////////////////////////////////////////////////////////////// // Routing Table Health Metrics pub fn get_routing_table_health(&self) -> RoutingTableHealth { self.inner.read().get_routing_table_health() } pub fn get_recent_peers(&self) -> Vec<(TypedKey, RecentPeersEntry)> { let mut recent_peers = Vec::new(); let mut dead_peers = Vec::new(); let mut out = Vec::new(); // collect all recent peers { let inner = self.inner.read(); for (k, _v) in &inner.recent_peers { recent_peers.push(*k); } } // look up each node and make sure the connection is still live // (uses same logic as send_data, ensuring last_connection works for UDP) for e in &recent_peers { let mut dead = true; if let Some(nr) = self.lookup_node_ref(*e) { if let Some(last_connection) = nr.last_connection() { out.push((*e, RecentPeersEntry { last_connection })); dead = false; } } if dead { dead_peers.push(e); } } // purge dead recent peers { let mut inner = self.inner.write(); for d in dead_peers { inner.recent_peers.remove(d); } } out } pub fn touch_recent_peer(&self, node_id: TypedKey, last_connection: ConnectionDescriptor) { self.inner .write() .touch_recent_peer(node_id, last_connection) } ////////////////////////////////////////////////////////////////////// // Find Nodes /// Build a map of protocols to low level ports /// This way we can get the set of protocols required to keep our NAT mapping alive for keepalive pings /// Only one protocol per low level protocol/port combination is required /// For example, if WS/WSS and TCP protocols are on the same low-level TCP port, only TCP keepalives will be required /// and we do not need to do WS/WSS keepalive as well. If they are on different ports, then we will need WS/WSS keepalives too. pub fn get_low_level_port_info(&self) -> LowLevelPortInfo { let mut low_level_protocol_ports = BTreeSet::<(LowLevelProtocolType, AddressType, u16)>::new(); let mut protocol_to_port = BTreeMap::<(ProtocolType, AddressType), (LowLevelProtocolType, u16)>::new(); let our_dids = self.all_filtered_dial_info_details( RoutingDomain::PublicInternet.into(), &DialInfoFilter::all(), ); for did in our_dids { low_level_protocol_ports.insert(( did.dial_info.protocol_type().low_level_protocol_type(), did.dial_info.address_type(), did.dial_info.socket_address().port(), )); protocol_to_port.insert( (did.dial_info.protocol_type(), did.dial_info.address_type()), ( did.dial_info.protocol_type().low_level_protocol_type(), did.dial_info.socket_address().port(), ), ); } LowLevelPortInfo { low_level_protocol_ports, protocol_to_port, } } /// Makes a filter that finds nodes with a matching inbound dialinfo pub fn make_inbound_dial_info_entry_filter<'a>( routing_domain: RoutingDomain, dial_info_filter: DialInfoFilter, ) -> RoutingTableEntryFilter<'a> { // does it have matching public dial info? Box::new(move |rti, e| { if let Some(e) = e { e.with(rti, |_rti, e| { if let Some(ni) = e.node_info(routing_domain) { if ni .first_filtered_dial_info_detail(DialInfoDetail::NO_SORT, |did| { did.matches_filter(&dial_info_filter) }) .is_some() { return true; } } false }) } else { rti.first_filtered_dial_info_detail(routing_domain.into(), &dial_info_filter) .is_some() } }) } /// Makes a filter that finds nodes capable of dialing a particular outbound dialinfo pub fn make_outbound_dial_info_entry_filter<'a>( routing_domain: RoutingDomain, dial_info: DialInfo, ) -> RoutingTableEntryFilter<'a> { // does the node's outbound capabilities match the dialinfo? Box::new(move |rti, e| { if let Some(e) = e { e.with(rti, |_rti, e| { if let Some(ni) = e.node_info(routing_domain) { let dif = DialInfoFilter::all() .with_protocol_type_set(ni.outbound_protocols) .with_address_type_set(ni.address_types); if dial_info.matches_filter(&dif) { return true; } } false }) } else { let dif = rti.get_outbound_dial_info_filter(routing_domain); dial_info.matches_filter(&dif) } }) } pub fn find_fast_public_nodes_filtered( &self, node_count: usize, filters: VecDeque, ) -> Vec { self.inner .read() .find_fast_public_nodes_filtered(self.clone(), node_count, filters) } /// Retrieve up to N of each type of protocol capable nodes for a single crypto kind fn find_bootstrap_nodes_filtered_per_crypto_kind( &self, crypto_kind: CryptoKind, max_per_type: usize, ) -> Vec { let protocol_types = vec![ ProtocolType::UDP, ProtocolType::TCP, ProtocolType::WS, ProtocolType::WSS, ]; let protocol_types_len = protocol_types.len(); let mut nodes_proto_v4 = vec![0usize, 0usize, 0usize, 0usize]; let mut nodes_proto_v6 = vec![0usize, 0usize, 0usize, 0usize]; let filter = Box::new( move |rti: &RoutingTableInner, entry: Option>| { let entry = entry.unwrap(); entry.with(rti, |_rti, e| { // skip nodes on our local network here if e.has_node_info(RoutingDomain::LocalNetwork.into()) { return false; } // Ensure crypto kind is supported if !e.crypto_kinds().contains(&crypto_kind) { return false; } // does it have some dial info we need? let filter = |n: &NodeInfo| { let mut keep = false; for did in &n.dial_info_detail_list { if matches!(did.dial_info.address_type(), AddressType::IPV4) { for (n, protocol_type) in protocol_types.iter().enumerate() { if nodes_proto_v4[n] < max_per_type && did.dial_info.protocol_type() == *protocol_type { nodes_proto_v4[n] += 1; keep = true; } } } else if matches!(did.dial_info.address_type(), AddressType::IPV6) { for (n, protocol_type) in protocol_types.iter().enumerate() { if nodes_proto_v6[n] < max_per_type && did.dial_info.protocol_type() == *protocol_type { nodes_proto_v6[n] += 1; keep = true; } } } } keep }; e.node_info(RoutingDomain::PublicInternet) .map(filter) .unwrap_or(false) }) }, ) as RoutingTableEntryFilter; let filters = VecDeque::from([filter]); self.find_fastest_nodes( protocol_types_len * 2 * max_per_type, filters, |_rti, entry: Option>| { NodeRef::new(self.clone(), entry.unwrap().clone(), None) }, ) } /// Retrieve up to N of each type of protocol capable nodes for all crypto kinds pub fn find_bootstrap_nodes_filtered(&self, max_per_type: usize) -> Vec { let mut out = self.find_bootstrap_nodes_filtered_per_crypto_kind(VALID_CRYPTO_KINDS[0], max_per_type); // Merge list of nodes so we don't have duplicates for crypto_kind in &VALID_CRYPTO_KINDS[1..] { let nrs = self.find_bootstrap_nodes_filtered_per_crypto_kind(*crypto_kind, max_per_type); 'nrloop: for nr in nrs { for nro in out { if nro.same_entry(&nr) { continue 'nrloop; } } out.push(nr); } } out } pub fn find_peers_with_sort_and_filter( &self, node_count: usize, cur_ts: Timestamp, filters: VecDeque, compare: C, transform: T, ) -> Vec where C: for<'a, 'b> FnMut( &'a RoutingTableInner, &'b Option>, &'b Option>, ) -> core::cmp::Ordering, T: for<'r> FnMut(&'r RoutingTableInner, Option>) -> O + Send, { self.inner .read() .find_peers_with_sort_and_filter(node_count, cur_ts, filters, compare, transform) } pub fn find_fastest_nodes<'a, T, O>( &self, node_count: usize, filters: VecDeque, transform: T, ) -> Vec where T: for<'r> FnMut(&'r RoutingTableInner, Option>) -> O + Send, { self.inner .read() .find_fastest_nodes(node_count, filters, transform) } pub fn find_closest_nodes<'a, T, O>( &self, node_count: usize, node_id: TypedKey, filters: VecDeque, transform: T, ) -> Vec where T: for<'r> FnMut(&'r RoutingTableInner, Option>) -> O + Send, { self.inner .read() .find_closest_nodes(node_count, node_id, filters, transform) } #[instrument(level = "trace", skip(self), ret)] pub fn register_find_node_answer( &self, crypto_kind: CryptoKind, peers: Vec, ) -> Vec { // Register nodes we'd found let mut out = Vec::::with_capacity(peers.len()); for p in peers { // Ensure we're getting back nodes we asked for if !p.node_ids.kinds().contains(&crypto_kind) { continue; } // Register the node if it's new if let Some(nr) = self.register_node_with_peer_info(RoutingDomain::PublicInternet, p, false) { out.push(nr); } } out } #[instrument(level = "trace", skip(self), ret, err)] pub async fn find_node( &self, node_ref: NodeRef, node_id: TypedKey, ) -> EyreResult>> { let rpc_processor = self.rpc_processor(); let res = network_result_try!( rpc_processor .clone() .rpc_call_find_node(Destination::direct(node_ref), node_id) .await? ); // register nodes we'd found Ok(NetworkResult::value( self.register_find_node_answer(node_id.kind, res.answer), )) } /// Ask a remote node to list the nodes it has around the current node #[instrument(level = "trace", skip(self), ret, err)] pub async fn find_self( &self, crypto_kind: CryptoKind, node_ref: NodeRef, ) -> EyreResult>> { let self_node_id = self.node_id(crypto_kind); self.find_node(node_ref, self_node_id).await } /// Ask a remote node to list the nodes it has around itself #[instrument(level = "trace", skip(self), ret, err)] pub async fn find_target( &self, crypto_kind: CryptoKind, node_ref: NodeRef, ) -> EyreResult>> { let Some(target_node_id) = node_ref.node_ids().get(crypto_kind) else { bail!("no target node ids for this crypto kind"); }; self.find_node(node_ref, target_node_id).await } #[instrument(level = "trace", skip(self))] pub async fn reverse_find_node(&self, crypto_kind: CryptoKind, node_ref: NodeRef, wide: bool) { // Ask node to 'find node' on own node so we can get some more nodes near ourselves // and then contact those nodes to inform -them- that we exist // Ask node for nodes closest to our own node let closest_nodes = network_result_value_or_log!(match self.find_self(crypto_kind, node_ref.clone()).await { Err(e) => { log_rtab!(error "find_self failed for {:?}: {:?}", &node_ref, e ); return; } Ok(v) => v, } => { return; }); // Ask each node near us to find us as well if wide { for closest_nr in closest_nodes { network_result_value_or_log!(match self.find_self(crypto_kind, closest_nr.clone()).await { Err(e) => { log_rtab!(error "find_self failed for {:?}: {:?}", &closest_nr, e ); continue; } Ok(v) => v, } => { // Do nothing with non-values continue; }); } } } pub fn make_public_internet_relay_node_filter(&self) -> impl Fn(&BucketEntryInner) -> bool { // Get all our outbound protocol/address types let outbound_dif = self.get_outbound_dial_info_filter(RoutingDomain::PublicInternet); let mapped_port_info = self.get_low_level_port_info(); move |e: &BucketEntryInner| { // Ensure this node is not on the local network if e.has_node_info(RoutingDomain::LocalNetwork.into()) { return false; } // Disqualify nodes that don't cover all our inbound ports for tcp and udp // as we need to be able to use the relay for keepalives for all nat mappings let mut low_level_protocol_ports = mapped_port_info.low_level_protocol_ports.clone(); let can_serve_as_relay = e .node_info(RoutingDomain::PublicInternet) .map(|n| { let dids = n.all_filtered_dial_info_details( Some(DialInfoDetail::ordered_sequencing_sort), // By default, choose connection-oriented protocol for relay |did| did.matches_filter(&outbound_dif), ); for did in &dids { let pt = did.dial_info.protocol_type(); let at = did.dial_info.address_type(); if let Some((llpt, port)) = mapped_port_info.protocol_to_port.get(&(pt, at)) { low_level_protocol_ports.remove(&(*llpt, at, *port)); } } low_level_protocol_ports.is_empty() }) .unwrap_or(false); if !can_serve_as_relay { return false; } true } } #[instrument(level = "trace", skip(self), ret)] pub fn find_inbound_relay( &self, routing_domain: RoutingDomain, cur_ts: Timestamp, ) -> Option { // Get relay filter function let relay_node_filter = match routing_domain { RoutingDomain::PublicInternet => self.make_public_internet_relay_node_filter(), RoutingDomain::LocalNetwork => { unimplemented!(); } }; // Go through all entries and find fastest entry that matches filter function let inner = self.inner.read(); let inner = &*inner; let mut best_inbound_relay: Option> = None; // Iterate all known nodes for candidates inner.with_entries(cur_ts, BucketEntryState::Unreliable, |rti, entry| { let entry2 = entry.clone(); entry.with(rti, |rti, e| { // Ensure we have the node's status if let Some(node_status) = e.node_status(routing_domain) { // Ensure the node will relay if node_status.will_relay() { // Compare against previous candidate if let Some(best_inbound_relay) = best_inbound_relay.as_mut() { // Less is faster let better = best_inbound_relay.with(rti, |_rti, best| { // choose low latency stability for relays BucketEntryInner::cmp_fastest_reliable(cur_ts, e, best) == std::cmp::Ordering::Less }); // Now apply filter function and see if this node should be included if better && relay_node_filter(e) { *best_inbound_relay = entry2; } } else if relay_node_filter(e) { // Always store the first candidate best_inbound_relay = Some(entry2); } } } }); // Don't end early, iterate through all entries Option::<()>::None }); // Return the best inbound relay noderef best_inbound_relay.map(|e| NodeRef::new(self.clone(), e, None)) } } impl core::ops::Deref for RoutingTable { type Target = RoutingTableUnlockedInner; fn deref(&self) -> &Self::Target { &self.unlocked_inner } }