veilid/veilid-core/src/routing_table/mod.rs

904 lines
32 KiB
Rust
Raw Normal View History

2021-11-22 16:28:30 +00:00
mod bucket;
mod bucket_entry;
2021-12-14 14:48:33 +00:00
mod debug;
2021-11-22 16:28:30 +00:00
mod node_ref;
2022-10-09 18:59:01 +00:00
mod route_spec_store;
2022-09-04 18:17:28 +00:00
mod routing_domain_editor;
2022-09-03 17:57:25 +00:00
mod routing_domains;
2022-10-19 01:53:45 +00:00
mod routing_table_inner;
mod stats_accounting;
2022-06-25 14:57:33 +00:00
mod tasks;
2021-11-22 16:28:30 +00:00
2022-10-30 23:29:31 +00:00
use crate::crypto::*;
2021-11-22 16:28:30 +00:00
use crate::network_manager::*;
use crate::rpc_processor::*;
use crate::xx::*;
use crate::*;
use bucket::*;
pub use bucket_entry::*;
2021-12-14 14:48:33 +00:00
pub use debug::*;
2022-08-31 01:21:16 +00:00
use hashlink::LruCache;
pub use node_ref::*;
2022-10-09 18:59:01 +00:00
pub use route_spec_store::*;
2022-09-04 18:17:28 +00:00
pub use routing_domain_editor::*;
2022-09-03 17:57:25 +00:00
pub use routing_domains::*;
2022-10-19 01:53:45 +00:00
pub use routing_table_inner::*;
pub use stats_accounting::*;
2021-11-22 16:28:30 +00:00
2022-08-31 01:21:16 +00:00
const RECENT_PEERS_TABLE_SIZE: usize = 64;
2021-11-22 16:28:30 +00:00
//////////////////////////////////////////////////////////////////////////
2022-10-19 01:53:45 +00:00
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,
2021-11-22 16:28:30 +00:00
}
2022-03-24 14:14:50 +00:00
#[derive(Clone, Debug, Default)]
pub struct RoutingTableHealth {
2022-09-23 15:23:33 +00:00
/// Number of reliable (responsive) entries in the routing table
2022-03-24 14:14:50 +00:00
pub reliable_entry_count: usize,
2022-09-23 15:23:33 +00:00
/// Number of unreliable (occasionally unresponsive) entries in the routing table
2022-03-24 14:14:50 +00:00
pub unreliable_entry_count: usize,
2022-09-23 15:23:33 +00:00
/// Number of dead (always unresponsive) entries in the routing table
2022-03-24 14:14:50 +00:00
pub dead_entry_count: usize,
}
2021-11-22 16:28:30 +00:00
struct RoutingTableUnlockedInner {
2022-09-23 15:23:33 +00:00
// Accessors
config: VeilidConfig,
2022-09-08 01:52:08 +00:00
network_manager: NetworkManager,
2022-09-23 15:23:33 +00:00
/// The current node's public DHT key
node_id: DHTKey,
/// The current node's DHT key secret
node_id_secret: DHTKeySecret,
/// Buckets to kick on our next kick task
kick_queue: Mutex<BTreeSet<usize>>,
/// Background process for computing statistics
2022-07-10 21:36:50 +00:00
rolling_transfers_task: TickTask<EyreReport>,
2022-09-23 15:23:33 +00:00
/// Backgroup process to purge dead routing table entries when necessary
2022-07-10 21:36:50 +00:00
kick_buckets_task: TickTask<EyreReport>,
2021-11-22 16:28:30 +00:00
}
#[derive(Clone)]
pub struct RoutingTable {
2022-06-25 14:57:33 +00:00
inner: Arc<RwLock<RoutingTableInner>>,
2021-11-22 16:28:30 +00:00
unlocked_inner: Arc<RoutingTableUnlockedInner>,
}
impl RoutingTable {
2022-09-08 01:52:08 +00:00
fn new_unlocked_inner(
2022-09-23 15:23:33 +00:00
config: VeilidConfig,
2022-09-08 01:52:08 +00:00
network_manager: NetworkManager,
) -> RoutingTableUnlockedInner {
2022-09-23 15:23:33 +00:00
let c = config.get();
2021-11-22 16:28:30 +00:00
RoutingTableUnlockedInner {
2022-09-23 15:23:33 +00:00
config: config.clone(),
2022-09-08 01:52:08 +00:00
network_manager,
2022-09-23 15:23:33 +00:00
node_id: c.network.node_id,
node_id_secret: c.network.node_id_secret,
kick_queue: Mutex::new(BTreeSet::default()),
rolling_transfers_task: TickTask::new(ROLLING_TRANSFERS_INTERVAL_SECS),
2022-06-25 14:57:33 +00:00
kick_buckets_task: TickTask::new(1),
2021-11-22 16:28:30 +00:00
}
}
pub fn new(network_manager: NetworkManager) -> Self {
let config = network_manager.config();
2022-10-19 01:53:45 +00:00
let unlocked_inner = Arc::new(Self::new_unlocked_inner(config, network_manager));
let inner = Arc::new(RwLock::new(RoutingTableInner::new(unlocked_inner.clone())));
2021-11-22 16:28:30 +00:00
let this = Self {
2022-10-19 01:53:45 +00:00
inner,
unlocked_inner,
2021-11-22 16:28:30 +00:00
};
2022-10-19 01:53:45 +00:00
2021-11-22 16:28:30 +00:00
// Set rolling transfers tick task
{
let this2 = this.clone();
this.unlocked_inner
.rolling_transfers_task
2022-06-13 00:58:02 +00:00
.set_routine(move |s, l, t| {
2022-10-04 17:09:03 +00:00
Box::pin(
this2
.clone()
.rolling_transfers_task_routine(s, l, t)
.instrument(trace_span!(
parent: None,
"RoutingTable rolling transfers task routine"
)),
)
2021-11-22 16:28:30 +00:00
});
}
2022-07-22 17:05:28 +00:00
2022-06-25 14:57:33 +00:00
// Set kick buckets tick task
{
let this2 = this.clone();
this.unlocked_inner
.kick_buckets_task
.set_routine(move |s, l, t| {
2022-10-04 17:09:03 +00:00
Box::pin(
this2
.clone()
.kick_buckets_task_routine(s, l, t)
.instrument(trace_span!(parent: None, "kick buckets task routine")),
)
2022-06-25 14:57:33 +00:00
});
}
2021-11-22 16:28:30 +00:00
this
}
pub fn network_manager(&self) -> NetworkManager {
2022-09-08 01:52:08 +00:00
self.unlocked_inner.network_manager.clone()
2021-11-22 16:28:30 +00:00
}
pub fn rpc_processor(&self) -> RPCProcessor {
self.network_manager().rpc_processor()
}
pub fn node_id(&self) -> DHTKey {
2022-09-23 15:23:33 +00:00
self.unlocked_inner.node_id
2021-11-22 16:28:30 +00:00
}
pub fn node_id_secret(&self) -> DHTKeySecret {
2022-09-23 15:23:33 +00:00
self.unlocked_inner.node_id_secret
2021-11-22 16:28:30 +00:00
}
2022-10-19 01:53:45 +00:00
/////////////////////////////////////
/// Initialization
2022-09-03 17:57:25 +00:00
2022-10-19 01:53:45 +00:00
/// Called to initialize the routing table after it is created
pub async fn init(&self) -> EyreResult<()> {
let mut inner = self.inner.write();
inner.init(self.clone());
Ok(())
2022-09-04 18:17:28 +00:00
}
2022-10-19 01:53:45 +00:00
/// Called to shut down the routing table
pub async fn terminate(&self) {
debug!("starting routing table terminate");
// Cancel all tasks being ticked
debug!("stopping rolling transfers task");
if let Err(e) = self.unlocked_inner.rolling_transfers_task.stop().await {
error!("rolling_transfers_task not stopped: {}", e);
}
debug!("stopping kick buckets task");
if let Err(e) = self.unlocked_inner.kick_buckets_task.stop().await {
error!("kick_buckets_task not stopped: {}", e);
2022-04-23 01:30:09 +00:00
}
2022-10-19 01:53:45 +00:00
let mut inner = self.inner.write();
inner.terminate();
*inner = RoutingTableInner::new(self.unlocked_inner.clone());
debug!("finished routing table terminate");
2021-12-24 23:02:53 +00:00
}
2022-10-19 01:53:45 +00:00
/// 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<RoutingDomain> {
self.inner.read().routing_domain_for_address(address)
2021-11-22 16:28:30 +00:00
}
2022-10-31 03:23:12 +00:00
pub fn route_spec_store(&self) -> RouteSpecStore {
self.inner.read().route_spec_store.clone()
2022-10-14 02:05:43 +00:00
}
2022-08-31 01:21:16 +00:00
pub fn relay_node(&self, domain: RoutingDomain) -> Option<NodeRef> {
2022-10-19 01:53:45 +00:00
self.inner.read().relay_node(domain)
2022-08-31 01:21:16 +00:00
}
2022-04-23 01:30:09 +00:00
pub fn has_dial_info(&self, domain: RoutingDomain) -> bool {
2022-10-19 01:53:45 +00:00
self.inner.read().has_dial_info(domain)
2021-12-24 01:34:52 +00:00
}
2022-04-16 15:18:54 +00:00
2022-04-23 01:30:09 +00:00
pub fn dial_info_details(&self, domain: RoutingDomain) -> Vec<DialInfoDetail> {
2022-10-19 01:53:45 +00:00
self.inner.read().dial_info_details(domain)
2021-11-22 16:28:30 +00:00
}
2022-04-23 01:30:09 +00:00
pub fn first_filtered_dial_info_detail(
2022-04-16 15:18:54 +00:00
&self,
2022-09-03 17:57:25 +00:00
routing_domain_set: RoutingDomainSet,
2022-04-16 15:18:54 +00:00
filter: &DialInfoFilter,
) -> Option<DialInfoDetail> {
2022-10-19 01:53:45 +00:00
self.inner
.read()
.first_filtered_dial_info_detail(routing_domain_set, filter)
2022-04-16 15:18:54 +00:00
}
2022-04-23 01:30:09 +00:00
pub fn all_filtered_dial_info_details(
2022-04-16 15:18:54 +00:00
&self,
2022-09-03 17:57:25 +00:00
routing_domain_set: RoutingDomainSet,
2022-04-16 15:18:54 +00:00
filter: &DialInfoFilter,
) -> Vec<DialInfoDetail> {
2022-10-19 01:53:45 +00:00
self.inner
.read()
.all_filtered_dial_info_details(routing_domain_set, filter)
2022-04-16 15:18:54 +00:00
}
2022-08-28 17:13:09 +00:00
pub fn ensure_dial_info_is_valid(&self, domain: RoutingDomain, dial_info: &DialInfo) -> bool {
2022-10-19 01:53:45 +00:00
self.inner
.read()
.ensure_dial_info_is_valid(domain, dial_info)
2022-08-28 17:13:09 +00:00
}
2022-09-04 18:17:28 +00:00
pub fn node_info_is_valid_in_routing_domain(
&self,
routing_domain: RoutingDomain,
node_info: &NodeInfo,
) -> bool {
2022-10-19 01:53:45 +00:00
self.inner
.read()
.node_info_is_valid_in_routing_domain(routing_domain, node_info)
2022-10-13 02:53:40 +00:00
}
2022-10-16 23:59:59 +00:00
/// 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,
node_a_id: &DHTKey,
node_a: &NodeInfo,
node_b_id: &DHTKey,
node_b: &NodeInfo,
dial_info_filter: DialInfoFilter,
2022-10-21 14:35:03 +00:00
sequencing: Sequencing,
2022-10-16 23:59:59 +00:00
) -> ContactMethod {
2022-10-19 01:53:45 +00:00
self.inner.read().get_contact_method(
2022-10-16 23:59:59 +00:00
routing_domain,
node_a_id,
node_a,
node_b_id,
node_b,
dial_info_filter,
2022-10-21 14:35:03 +00:00
sequencing,
2022-10-16 23:59:59 +00:00
)
}
2022-09-04 18:17:28 +00:00
#[instrument(level = "debug", skip(self))]
pub fn edit_routing_domain(&self, domain: RoutingDomain) -> RoutingDomainEditor {
RoutingDomainEditor::new(self.clone(), domain)
}
2022-10-09 18:59:01 +00:00
/// Return a copy of our node's peerinfo
2022-09-03 17:57:25 +00:00
pub fn get_own_peer_info(&self, routing_domain: RoutingDomain) -> PeerInfo {
2022-10-19 01:53:45 +00:00
self.inner.read().get_own_peer_info(routing_domain)
2022-09-03 17:57:25 +00:00
}
2022-10-09 18:59:01 +00:00
/// Return a copy of our node's signednodeinfo
2022-09-03 17:57:25 +00:00
pub fn get_own_signed_node_info(&self, routing_domain: RoutingDomain) -> SignedNodeInfo {
2022-10-19 01:53:45 +00:00
self.inner.read().get_own_signed_node_info(routing_domain)
2022-09-03 17:57:25 +00:00
}
2022-10-09 18:59:01 +00:00
/// Return a copy of our node's nodeinfo
2022-10-16 23:59:59 +00:00
pub fn get_own_node_info(&self, routing_domain: RoutingDomain) -> NodeInfo {
2022-10-19 01:53:45 +00:00
self.inner.read().get_own_node_info(routing_domain)
2022-10-16 23:59:59 +00:00
}
2021-11-22 16:28:30 +00:00
2022-10-19 01:53:45 +00:00
/// If we have a valid network class in this routing domain, then our 'NodeInfo' is valid
2022-09-03 17:57:25 +00:00
pub fn has_valid_own_node_info(&self, routing_domain: RoutingDomain) -> bool {
2022-10-19 01:53:45 +00:00
self.inner.read().has_valid_own_node_info(routing_domain)
2022-10-09 18:59:01 +00:00
}
/// Return the domain's currently registered network class
pub fn get_network_class(&self, routing_domain: RoutingDomain) -> Option<NetworkClass> {
2022-10-19 01:53:45 +00:00
self.inner.read().get_network_class(routing_domain)
2022-10-09 18:59:01 +00:00
}
/// 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 {
2022-10-19 01:53:45 +00:00
self.inner
.read()
.get_inbound_dial_info_filter(routing_domain)
2022-10-09 18:59:01 +00:00
}
/// 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 {
2022-10-19 01:53:45 +00:00
self.inner
.read()
.get_inbound_node_ref_filter(routing_domain)
2022-10-09 18:59:01 +00:00
}
/// 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 {
2022-10-19 01:53:45 +00:00
self.inner
.read()
.get_outbound_dial_info_filter(routing_domain)
2022-10-09 18:59:01 +00:00
}
/// 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 {
2022-10-19 01:53:45 +00:00
self.inner
.read()
.get_outbound_node_ref_filter(routing_domain)
2022-09-03 17:57:25 +00:00
}
2022-10-13 02:53:40 +00:00
/// Attempt to empty the routing table
/// should only be performed when there are no node_refs (detached)
2022-08-06 00:34:00 +00:00
pub fn purge_buckets(&self) {
2022-10-19 01:53:45 +00:00
self.inner.write().purge_buckets();
2022-08-06 00:34:00 +00:00
}
2022-10-13 02:53:40 +00:00
/// Attempt to remove last_connections from entries
2022-08-06 00:34:00 +00:00
pub fn purge_last_connections(&self) {
2022-10-19 01:53:45 +00:00
self.inner.write().purge_last_connections();
2021-11-22 16:28:30 +00:00
}
2022-09-23 15:23:33 +00:00
fn find_bucket_index(&self, node_id: DHTKey) -> usize {
distance(&node_id, &self.unlocked_inner.node_id)
2021-11-22 16:28:30 +00:00
.first_nonzero_bit()
.unwrap()
}
2022-09-03 17:57:25 +00:00
pub fn get_entry_count(
&self,
routing_domain_set: RoutingDomainSet,
min_state: BucketEntryState,
) -> usize {
2022-10-19 01:53:45 +00:00
self.inner
.read()
.get_entry_count(routing_domain_set, min_state)
}
2022-08-31 01:21:16 +00:00
pub fn get_nodes_needing_updates(
&self,
routing_domain: RoutingDomain,
cur_ts: u64,
all: bool,
) -> Vec<NodeRef> {
2022-10-19 01:53:45 +00:00
self.inner
.read()
.get_nodes_needing_updates(self.clone(), routing_domain, cur_ts, all)
2022-07-22 17:05:28 +00:00
}
2022-09-03 17:57:25 +00:00
pub fn get_nodes_needing_ping(
&self,
routing_domain: RoutingDomain,
cur_ts: u64,
) -> Vec<NodeRef> {
2022-10-19 01:53:45 +00:00
self.inner
.read()
.get_nodes_needing_ping(self.clone(), routing_domain, cur_ts)
2022-07-22 17:05:28 +00:00
}
pub fn get_all_nodes(&self, cur_ts: u64) -> Vec<NodeRef> {
let inner = self.inner.read();
2022-10-19 01:53:45 +00:00
inner.get_all_nodes(self.clone(), cur_ts)
2022-07-22 17:05:28 +00:00
}
2022-06-25 14:57:33 +00:00
fn queue_bucket_kick(&self, node_id: DHTKey) {
2022-09-23 15:23:33 +00:00
let idx = self.find_bucket_index(node_id);
self.unlocked_inner.kick_queue.lock().insert(idx);
2021-11-22 16:28:30 +00:00
}
2022-10-13 02:53:40 +00:00
/// Create a node reference, possibly creating a bucket entry
/// the 'update_func' closure is called on the node, and, if created,
/// in a locked fashion as to ensure the bucket entry state is always valid
2022-07-20 13:39:38 +00:00
pub fn create_node_ref<F>(&self, node_id: DHTKey, update_func: F) -> Option<NodeRef>
2022-05-03 20:43:15 +00:00
where
2022-10-09 18:59:01 +00:00
F: FnOnce(&mut RoutingTableInner, &mut BucketEntryInner),
2022-05-03 20:43:15 +00:00
{
2022-10-19 01:53:45 +00:00
self.inner
.write()
.create_node_ref(self.clone(), node_id, update_func)
2021-11-22 16:28:30 +00:00
}
2022-10-13 02:53:40 +00:00
/// Resolve an existing routing table entry and return a reference to it
2021-11-22 16:28:30 +00:00
pub fn lookup_node_ref(&self, node_id: DHTKey) -> Option<NodeRef> {
2022-10-19 01:53:45 +00:00
self.inner.read().lookup_node_ref(self.clone(), node_id)
2021-11-22 16:28:30 +00:00
}
2022-10-13 02:53:40 +00:00
/// Resolve an existing routing table entry and return a filtered reference to it
pub fn lookup_and_filter_noderef(
&self,
node_id: DHTKey,
routing_domain_set: RoutingDomainSet,
dial_info_filter: DialInfoFilter,
) -> Option<NodeRef> {
2022-10-19 01:53:45 +00:00
self.inner.read().lookup_and_filter_noderef(
self.clone(),
node_id,
routing_domain_set,
dial_info_filter,
2022-10-13 02:53:40 +00:00
)
}
/// 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.
2022-05-11 01:49:42 +00:00
pub fn register_node_with_signed_node_info(
2021-11-22 16:28:30 +00:00
&self,
2022-09-04 18:17:28 +00:00
routing_domain: RoutingDomain,
2021-11-22 16:28:30 +00:00
node_id: DHTKey,
2022-05-11 01:49:42 +00:00
signed_node_info: SignedNodeInfo,
2022-09-04 18:17:28 +00:00
allow_invalid: bool,
2022-07-20 13:39:38 +00:00
) -> Option<NodeRef> {
2022-10-19 01:53:45 +00:00
self.inner.write().register_node_with_signed_node_info(
self.clone(),
routing_domain,
node_id,
signed_node_info,
allow_invalid,
)
2021-11-22 16:28:30 +00:00
}
2022-10-13 02:53:40 +00:00
/// 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
2021-11-22 16:28:30 +00:00
pub fn register_node_with_existing_connection(
&self,
node_id: DHTKey,
descriptor: ConnectionDescriptor,
timestamp: u64,
2022-07-20 13:39:38 +00:00
) -> Option<NodeRef> {
2022-10-19 01:53:45 +00:00
self.inner.write().register_node_with_existing_connection(
self.clone(),
node_id,
descriptor,
timestamp,
)
2021-11-22 16:28:30 +00:00
}
2022-10-13 02:53:40 +00:00
/// Ticks about once per second
/// to run tick tasks which may run at slower tick rates as configured
2022-07-10 21:36:50 +00:00
pub async fn tick(&self) -> EyreResult<()> {
2021-11-22 16:28:30 +00:00
// Do rolling transfers every ROLLING_TRANSFERS_INTERVAL_SECS secs
self.unlocked_inner.rolling_transfers_task.tick().await?;
2022-06-25 14:57:33 +00:00
// Kick buckets task
2022-09-23 15:23:33 +00:00
let kick_bucket_queue_count = self.unlocked_inner.kick_queue.lock().len();
2022-06-25 14:57:33 +00:00
if kick_bucket_queue_count > 0 {
self.unlocked_inner.kick_buckets_task.tick().await?;
}
2022-04-07 13:55:09 +00:00
2021-11-22 16:28:30 +00:00
Ok(())
}
2022-03-24 14:14:50 +00:00
//////////////////////////////////////////////////////////////////////
// Routing Table Health Metrics
pub fn get_routing_table_health(&self) -> RoutingTableHealth {
2022-10-19 01:53:45 +00:00
self.inner.read().get_routing_table_health()
}
pub fn get_recent_peers(&self) -> Vec<(DHTKey, RecentPeersEntry)> {
self.inner.write().get_recent_peers(self.clone())
}
pub fn touch_recent_peer(&self, node_id: DHTKey, 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(
routing_domain: RoutingDomain,
dial_info_filter: DialInfoFilter,
) -> impl FnMut(&RoutingTableInner, &BucketEntryInner) -> bool {
// does it have matching public dial info?
move |_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;
2022-03-24 14:14:50 +00:00
}
}
2022-10-19 01:53:45 +00:00
false
2022-03-24 14:14:50 +00:00
}
}
2022-08-31 01:21:16 +00:00
2022-10-19 01:53:45 +00:00
/// Makes a filter that finds nodes capable of dialing a particular outbound dialinfo
pub fn make_outbound_dial_info_entry_filter<'s>(
routing_domain: RoutingDomain,
dial_info: DialInfo,
) -> impl FnMut(&RoutingTableInner, &'s BucketEntryInner) -> bool {
// does the node's outbound capabilities match the dialinfo?
move |_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
}
}
2022-09-07 15:30:43 +00:00
2022-10-19 01:53:45 +00:00
/// Make a filter that wraps another filter
pub fn combine_entry_filters<'a, 'b, F, G>(
mut f1: F,
mut f2: G,
) -> impl FnMut(&'a RoutingTableInner, &'b BucketEntryInner) -> bool
where
F: FnMut(&'a RoutingTableInner, &'b BucketEntryInner) -> bool,
G: FnMut(&'a RoutingTableInner, &'b BucketEntryInner) -> bool,
{
move |rti, e| {
if !f1(rti, e) {
return false;
}
if !f2(rti, e) {
return false;
2022-09-07 15:30:43 +00:00
}
2022-10-19 01:53:45 +00:00
true
2022-09-07 15:30:43 +00:00
}
2022-10-19 01:53:45 +00:00
}
pub fn find_fast_public_nodes_filtered<'a, 'b, F>(
&self,
node_count: usize,
mut entry_filter: F,
) -> Vec<NodeRef>
where
F: FnMut(&'a RoutingTableInner, &'b BucketEntryInner) -> bool,
{
self.inner
.read()
.find_fast_public_nodes_filtered(self.clone(), node_count, entry_filter)
}
/// Retrieve up to N of each type of protocol capable nodes
pub fn find_bootstrap_nodes_filtered(&self, max_per_type: usize) -> Vec<NodeRef> {
let protocol_types = vec![
ProtocolType::UDP,
ProtocolType::TCP,
ProtocolType::WS,
ProtocolType::WSS,
];
let mut nodes_proto_v4 = vec![0usize, 0usize, 0usize, 0usize];
let mut nodes_proto_v6 = vec![0usize, 0usize, 0usize, 0usize];
self.find_fastest_nodes(
// count
protocol_types.len() * 2 * max_per_type,
// filter
move |rti, _k: DHTKey, v: Option<Arc<BucketEntry>>| {
let entry = v.unwrap();
entry.with(rti, |_rti, e| {
// skip nodes on our local network here
if e.has_node_info(RoutingDomain::LocalNetwork.into()) {
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)
})
},
// transform
|_rti, k: DHTKey, v: Option<Arc<BucketEntry>>| {
NodeRef::new(self.clone(), k, v.unwrap().clone(), None)
},
)
}
pub fn find_peers_with_sort_and_filter<'a, 'b, F, C, T, O>(
&self,
node_count: usize,
cur_ts: u64,
mut filter: F,
compare: C,
mut transform: T,
) -> Vec<O>
where
F: FnMut(&'a RoutingTableInner, DHTKey, Option<Arc<BucketEntry>>) -> bool,
C: FnMut(
&'a RoutingTableInner,
&'b (DHTKey, Option<Arc<BucketEntry>>),
&'b (DHTKey, Option<Arc<BucketEntry>>),
) -> core::cmp::Ordering,
T: FnMut(&'a RoutingTableInner, DHTKey, Option<Arc<BucketEntry>>) -> O,
{
self.inner
.read()
.find_peers_with_sort_and_filter(node_count, cur_ts, filter, compare, transform)
}
pub fn find_fastest_nodes<'a, T, F, O>(
&self,
node_count: usize,
mut filter: F,
transform: T,
) -> Vec<O>
where
F: FnMut(&'a RoutingTableInner, DHTKey, Option<Arc<BucketEntry>>) -> bool,
T: FnMut(&'a RoutingTableInner, DHTKey, Option<Arc<BucketEntry>>) -> O,
{
self.inner
.read()
.find_fastest_nodes(node_count, filter, transform)
}
pub fn find_closest_nodes<'a, F, T, O>(
&self,
node_id: DHTKey,
filter: F,
mut transform: T,
) -> Vec<O>
where
F: FnMut(&'a RoutingTableInner, DHTKey, Option<Arc<BucketEntry>>) -> bool,
T: FnMut(&'a RoutingTableInner, DHTKey, Option<Arc<BucketEntry>>) -> O,
{
self.inner
.read()
.find_closest_nodes(node_id, filter, transform)
}
#[instrument(level = "trace", skip(self), ret)]
pub fn register_find_node_answer(&self, peers: Vec<PeerInfo>) -> Vec<NodeRef> {
let node_id = self.node_id();
// register nodes we'd found
let mut out = Vec::<NodeRef>::with_capacity(peers.len());
for p in peers {
// if our own node if is in the list then ignore it, as we don't add ourselves to our own routing table
if p.node_id.key == node_id {
continue;
}
2022-09-07 15:30:43 +00:00
2022-10-19 01:53:45 +00:00
// node can not be its own relay
if let Some(rpi) = &p.signed_node_info.node_info.relay_peer_info {
if rpi.node_id == p.node_id {
continue;
2022-09-07 15:30:43 +00:00
}
}
2022-10-19 01:53:45 +00:00
// register the node if it's new
if let Some(nr) = self.register_node_with_signed_node_info(
RoutingDomain::PublicInternet,
p.node_id.key,
p.signed_node_info.clone(),
false,
) {
out.push(nr);
2022-09-07 15:30:43 +00:00
}
}
2022-10-19 01:53:45 +00:00
out
}
2022-09-07 15:30:43 +00:00
2022-10-19 01:53:45 +00:00
#[instrument(level = "trace", skip(self), ret, err)]
pub async fn find_node(
&self,
node_ref: NodeRef,
node_id: DHTKey,
) -> EyreResult<NetworkResult<Vec<NodeRef>>> {
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(res.answer),
))
}
#[instrument(level = "trace", skip(self), ret, err)]
pub async fn find_self(&self, node_ref: NodeRef) -> EyreResult<NetworkResult<Vec<NodeRef>>> {
let node_id = self.node_id();
self.find_node(node_ref, node_id).await
}
#[instrument(level = "trace", skip(self), ret, err)]
pub async fn find_target(&self, node_ref: NodeRef) -> EyreResult<NetworkResult<Vec<NodeRef>>> {
let node_id = node_ref.node_id();
self.find_node(node_ref, node_id).await
}
#[instrument(level = "trace", skip(self))]
pub async fn reverse_find_node(&self, node_ref: NodeRef, wide: bool) {
// Ask bootstrap node to 'find' our own node so we can get some more nodes near ourselves
// and then contact those nodes to inform -them- that we exist
// Ask bootstrap server for nodes closest to our own node
let closest_nodes = network_result_value_or_log!(debug match self.find_self(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!(debug match self.find_self(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;
});
2022-09-07 15:30:43 +00:00
}
}
2022-10-19 01:53:45 +00:00
}
2022-09-07 15:30:43 +00:00
2022-10-19 01:53:45 +00:00
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(
2022-10-21 14:35:03 +00:00
Some(DialInfoDetail::ordered_sequencing_sort), // By default, choose connection-oriented protocol for relay
2022-10-19 01:53:45 +00:00
|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
}
2022-08-31 01:21:16 +00:00
}
2022-10-19 01:53:45 +00:00
#[instrument(level = "trace", skip(self), ret)]
pub fn find_inbound_relay(
&self,
routing_domain: RoutingDomain,
cur_ts: u64,
) -> Option<NodeRef> {
// 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<(DHTKey, Arc<BucketEntry>)> = None;
// Iterate all known nodes for candidates
inner.with_entries(cur_ts, BucketEntryState::Unreliable, |rti, k, v| {
let v2 = v.clone();
v.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.1.with(rti, |_rti, best| {
2022-10-21 14:35:03 +00:00
// choose low latency stability for relays
2022-10-19 01:53:45 +00:00
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 = (k, v2);
}
} else if relay_node_filter(e) {
// Always store the first candidate
best_inbound_relay = Some((k, v2));
}
}
}
});
// Don't end early, iterate through all entries
Option::<()>::None
});
// Return the best inbound relay noderef
best_inbound_relay.map(|(k, e)| NodeRef::new(self.clone(), k, e, None))
2022-08-31 01:21:16 +00:00
}
2021-11-22 16:28:30 +00:00
}