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

1154 lines
40 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;
2023-05-29 19:24:57 +00:00
mod find_peers;
2021-11-22 16:28:30 +00:00
mod node_ref;
2022-11-04 02:02:40 +00:00
mod node_ref_filter;
2022-11-22 23:26:39 +00:00
mod privacy;
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;
2023-05-29 19:24:57 +00:00
mod types;
2021-11-22 16:28:30 +00:00
2023-05-29 19:24:57 +00:00
pub mod tests;
use super::*;
2022-11-26 21:17: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 bucket::*;
2023-05-29 19:24:57 +00:00
use hashlink::LruCache;
pub use bucket_entry::*;
2021-12-14 14:48:33 +00:00
pub use debug::*;
2023-05-29 19:24:57 +00:00
pub use find_peers::*;
pub use node_ref::*;
2022-11-04 02:02:40 +00:00
pub use node_ref_filter::*;
2022-11-22 23:26:39 +00:00
pub use privacy::*;
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::*;
2023-05-29 19:24:57 +00:00
pub use types::*;
2021-11-22 16:28:30 +00:00
//////////////////////////////////////////////////////////////////////////
2022-12-09 01:30:42 +00:00
/// How frequently we tick the relay management routine
2022-11-25 01:17:54 +00:00
pub const RELAY_MANAGEMENT_INTERVAL_SECS: u32 = 1;
2022-12-09 01:30:42 +00:00
/// How frequently we tick the private route management routine
2022-11-25 01:17:54 +00:00
pub const PRIVATE_ROUTE_MANAGEMENT_INTERVAL_SECS: u32 = 1;
2021-11-22 16:28:30 +00:00
2022-12-09 01:30:42 +00:00
// 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;
2023-07-15 20:18:13 +00:00
// Table store keys
const ALL_ENTRY_BYTES: &[u8] = b"all_entry_bytes";
const ROUTING_TABLE: &str = "routing_table";
const SERIALIZED_BUCKET_MAP: &[u8] = b"serialized_bucket_map";
const CACHE_VALIDITY_KEY: &[u8] = b"cache_validity_key";
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-11-02 20:29:29 +00:00
pub type RoutingTableEntryFilter<'t> =
2023-02-11 20:54:55 +00:00
Box<dyn FnMut(&RoutingTableInner, Option<Arc<BucketEntry>>) -> bool + Send + 't>;
2023-05-29 19:24:57 +00:00
pub type SerializedBuckets = Vec<Vec<u8>>;
pub type SerializedBucketMap = BTreeMap<CryptoKind, SerializedBuckets>;
2021-11-22 16:28:30 +00:00
2022-12-26 21:33:48 +00:00
#[derive(Clone, Debug, Default, Eq, PartialEq)]
2022-03-24 14:14:50 +00:00
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,
2022-12-26 02:28:45 +00:00
/// If PublicInternet network class is valid yet
2022-12-26 21:33:48 +00:00
pub public_internet_ready: bool,
2022-12-26 02:28:45 +00:00
/// If LocalNetwork network class is valid yet
2022-12-26 21:33:48 +00:00
pub local_network_ready: bool,
2022-03-24 14:14:50 +00:00
}
2023-03-01 02:11:26 +00:00
pub type BucketIndex = (CryptoKind, usize);
pub struct RoutingTableUnlockedInner {
2022-09-23 15:23:33 +00:00
// Accessors
config: VeilidConfig,
2022-09-08 01:52:08 +00:00
network_manager: NetworkManager,
2023-03-03 15:55:31 +00:00
/// The current node's public DHT keys
node_id: TypedKeyGroup,
2023-03-03 15:55:31 +00:00
/// The current node's public DHT secrets
node_id_secret: TypedSecretGroup,
2022-09-23 15:23:33 +00:00
/// Buckets to kick on our next kick task
2023-03-01 02:11:26 +00:00
kick_queue: Mutex<BTreeSet<BucketIndex>>,
2022-09-23 15:23:33 +00:00
/// Background process for computing statistics
2022-07-10 21:36:50 +00:00
rolling_transfers_task: TickTask<EyreReport>,
2022-12-26 21:33:48 +00:00
/// Background process to purge dead routing table entries when necessary
2022-07-10 21:36:50 +00:00
kick_buckets_task: TickTask<EyreReport>,
2022-11-25 01:17:54 +00:00
/// Background process to get our initial routing table
bootstrap_task: TickTask<EyreReport>,
/// Background process to ensure we have enough nodes in our routing table
peer_minimum_refresh_task: TickTask<EyreReport>,
/// Background process to check nodes to see if they are still alive and for reliability
ping_validator_task: TickTask<EyreReport>,
/// Background process to keep relays up
relay_management_task: TickTask<EyreReport>,
/// Background process to keep private routes up
private_route_management_task: TickTask<EyreReport>,
2021-11-22 16:28:30 +00:00
}
2023-02-10 02:01:04 +00:00
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<F, R>(&self, f: F) -> R
where
F: FnOnce(&VeilidConfigInner) -> R,
{
f(&*self.config.get())
}
2023-02-11 20:54:55 +00:00
pub fn node_id(&self, kind: CryptoKind) -> TypedKey {
2023-03-03 15:55:31 +00:00
self.node_id.get(kind).unwrap()
}
pub fn node_id_secret_key(&self, kind: CryptoKind) -> SecretKey {
self.node_id_secret.get(kind).unwrap().value
2023-02-13 21:12:46 +00:00
}
pub fn node_ids(&self) -> TypedKeyGroup {
2023-03-03 15:55:31 +00:00
self.node_id.clone()
2023-02-10 02:01:04 +00:00
}
2023-02-22 23:29:07 +00:00
pub fn node_id_typed_key_pairs(&self) -> Vec<TypedKeyPair> {
let mut tkps = Vec::new();
2023-03-03 15:55:31 +00:00
for ck in VALID_CRYPTO_KINDS {
tkps.push(TypedKeyPair::new(
ck,
KeyPair::new(self.node_id(ck).value, self.node_id_secret_key(ck)),
));
2023-02-22 23:29:07 +00:00
}
tkps
}
2023-02-10 02:01:04 +00:00
pub fn matches_own_node_id(&self, node_ids: &[TypedKey]) -> bool {
for ni in node_ids {
2023-03-03 15:55:31 +00:00
if let Some(v) = self.node_id.get(ni.kind) {
if v.value == ni.value {
2023-02-10 02:01:04 +00:00
return true;
}
}
}
false
}
2023-02-23 02:47:00 +00:00
pub fn matches_own_node_id_key(&self, node_id_key: &PublicKey) -> bool {
2023-03-03 15:55:31 +00:00
for tk in self.node_id.iter() {
if tk.value == *node_id_key {
2023-02-23 02:47:00 +00:00
return true;
}
}
false
}
2023-03-01 02:11:26 +00:00
pub fn calculate_bucket_index(&self, node_id: &TypedKey) -> BucketIndex {
2023-02-10 02:01:04 +00:00
let crypto = self.crypto();
2023-03-03 15:55:31 +00:00
let self_node_id_key = self.node_id(node_id.kind).value;
2023-02-17 00:54:29 +00:00
let vcrypto = crypto.get(node_id.kind).unwrap();
(
node_id.kind,
vcrypto
2023-03-03 15:55:31 +00:00
.distance(&node_id.value, &self_node_id_key)
2023-02-17 00:54:29 +00:00
.first_nonzero_bit()
.unwrap(),
)
2023-02-10 02:01:04 +00:00
}
}
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();
2023-03-03 15:55:31 +00:00
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,
2023-03-03 15:55:31 +00:00
node_id: c.network.routing_table.node_id.clone(),
node_id_secret: c.network.routing_table.node_id_secret.clone(),
2022-09-23 15:23:33 +00:00
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),
2022-11-25 01:17:54 +00:00
bootstrap_task: TickTask::new(1),
2023-07-19 14:07:51 +00:00
peer_minimum_refresh_task: TickTask::new(1),
2022-11-25 01:17:54 +00:00
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),
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
2023-05-29 19:24:57 +00:00
this.setup_tasks();
2022-07-22 17:05:28 +00:00
2021-11-22 16:28:30 +00:00
this
}
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<()> {
2022-11-01 02:03:05 +00:00
debug!("starting routing table init");
2022-11-06 21:07:56 +00:00
// Set up routing buckets
{
let mut inner = self.inner.write();
2023-03-01 02:11:26 +00:00
inner.init_buckets();
2022-11-06 21:07:56 +00:00
}
// Load bucket entries from table db if possible
debug!("loading routing table entries");
if let Err(e) = self.load_buckets().await {
2022-11-09 22:11:35 +00:00
log_rtab!(debug "Error loading buckets from storage: {:#?}. Resetting.", e);
2022-11-06 21:07:56 +00:00
let mut inner = self.inner.write();
2023-03-01 02:11:26 +00:00
inner.init_buckets();
2022-11-06 21:07:56 +00:00
}
2022-11-01 02:03:05 +00:00
// Set up routespecstore
debug!("starting route spec store init");
let route_spec_store = match RouteSpecStore::load(self.clone()).await {
Ok(v) => v,
Err(e) => {
2022-11-09 22:11:35 +00:00
log_rtab!(debug "Error loading route spec store: {:#?}. Resetting.", e);
2022-11-01 02:03:05 +00:00
RouteSpecStore::new(self.clone())
}
};
debug!("finished route spec store init");
2022-11-06 21:07:56 +00:00
{
let mut inner = self.inner.write();
inner.route_spec_store = Some(route_spec_store);
}
2022-11-01 02:03:05 +00:00
debug!("finished routing table init");
2022-10-19 01:53:45 +00:00
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");
2022-11-25 01:17:54 +00:00
// Stop tasks
2023-05-29 19:24:57 +00:00
self.cancel_tasks().await;
2022-10-19 01:53:45 +00:00
2022-11-06 21:07:56 +00:00
// 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);
}
2022-11-01 02:03:05 +00:00
debug!("saving route spec store");
let rss = {
let mut inner = self.inner.write();
inner.route_spec_store.take()
};
if let Some(rss) = rss {
2022-11-02 01:05:48 +00:00
if let Err(e) = rss.save().await {
error!("couldn't save route spec store: {}", e);
}
2022-11-01 02:03:05 +00:00
}
debug!("shutting down routing table");
2022-10-19 01:53:45 +00:00
let mut inner = self.inner.write();
*inner = RoutingTableInner::new(self.unlocked_inner.clone());
debug!("finished routing table terminate");
2021-12-24 23:02:53 +00:00
}
2023-05-29 19:24:57 +00:00
/// Serialize the routing table.
2023-07-15 20:18:13 +00:00
fn serialized_buckets(&self) -> (SerializedBucketMap, SerializedBuckets) {
2023-02-10 02:01:04 +00:00
// 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<Arc<BucketEntry>> = Vec::new();
// Serialize all buckets and get map of entries
2023-05-29 19:24:57 +00:00
let mut serialized_bucket_map: SerializedBucketMap = BTreeMap::new();
2022-11-06 21:07:56 +00:00
{
2023-02-10 02:01:04 +00:00
let mut entry_map: HashMap<*const BucketEntry, u32> = HashMap::new();
2022-11-06 21:07:56 +00:00
let inner = &*self.inner.read();
2023-02-10 02:01:04 +00:00
for ck in VALID_CRYPTO_KINDS {
let buckets = inner.buckets.get(&ck).unwrap();
let mut serialized_buckets = Vec::new();
for bucket in buckets.iter() {
2023-07-15 20:18:13 +00:00
serialized_buckets.push(bucket.save_bucket(&mut all_entries, &mut entry_map))
2023-02-10 02:01:04 +00:00
}
serialized_bucket_map.insert(ck, serialized_buckets);
2022-11-06 21:07:56 +00:00
}
}
2023-02-10 02:01:04 +00:00
// Serialize all the entries
let mut all_entry_bytes = Vec::with_capacity(all_entries.len());
for entry in all_entries {
// Serialize entry
2023-07-15 20:18:13 +00:00
let entry_bytes = entry.with_inner(|e| serialize_json_bytes(e));
2023-02-10 02:01:04 +00:00
all_entry_bytes.push(entry_bytes);
}
2023-07-15 20:18:13 +00:00
(serialized_bucket_map, all_entry_bytes)
2023-05-29 19:24:57 +00:00
}
/// Write the serialized routing table to the table store.
async fn save_buckets(&self) -> EyreResult<()> {
2023-07-15 20:18:13 +00:00
let (serialized_bucket_map, all_entry_bytes) = self.serialized_buckets();
2023-05-29 19:24:57 +00:00
2023-02-10 02:01:04 +00:00
let table_store = self.unlocked_inner.network_manager().table_store();
2023-07-15 20:18:13 +00:00
let tdb = table_store.open(ROUTING_TABLE, 1).await?;
2022-12-28 17:12:04 +00:00
let dbx = tdb.transact();
2023-07-15 20:18:13 +00:00
if let Err(e) = dbx.store_json(0, SERIALIZED_BUCKET_MAP, &serialized_bucket_map) {
2022-11-06 21:07:56 +00:00
dbx.rollback();
2023-05-29 19:24:57 +00:00
return Err(e.into());
2022-11-06 21:07:56 +00:00
}
2023-07-15 20:18:13 +00:00
if let Err(e) = dbx.store_json(0, ALL_ENTRY_BYTES, &all_entry_bytes) {
2023-02-10 02:01:04 +00:00
dbx.rollback();
2023-05-29 19:24:57 +00:00
return Err(e.into());
2022-11-06 21:07:56 +00:00
}
2022-12-28 17:12:04 +00:00
dbx.commit().await?;
2022-11-06 21:07:56 +00:00
Ok(())
}
2023-02-10 02:01:04 +00:00
/// Deserialize routing table from table store
2022-11-06 21:07:56 +00:00
async fn load_buckets(&self) -> EyreResult<()> {
2023-06-22 21:42:34 +00:00
// Make a cache validity key of all our node ids and our bootstrap choice
let mut cache_validity_key: Vec<u8> = Vec::new();
{
let c = self.unlocked_inner.config.get();
for ck in VALID_CRYPTO_KINDS {
2023-06-22 22:31:31 +00:00
if let Some(nid) = c.network.routing_table.node_id.get(ck) {
cache_validity_key.append(&mut nid.value.bytes.to_vec());
}
2023-06-22 21:42:34 +00:00
}
for b in &c.network.routing_table.bootstrap {
cache_validity_key.append(&mut b.as_bytes().to_vec());
}
};
2023-02-10 02:01:04 +00:00
// Deserialize bucket map and all entries from the table store
2023-06-22 21:42:34 +00:00
let table_store = self.unlocked_inner.network_manager().table_store();
2023-07-15 20:18:13 +00:00
let db = table_store.open(ROUTING_TABLE, 1).await?;
2023-06-22 21:42:34 +00:00
2023-07-15 20:18:13 +00:00
let caches_valid = match db.load(0, CACHE_VALIDITY_KEY).await? {
2023-06-22 21:42:34 +00:00
Some(v) => v == cache_validity_key,
None => false,
};
if !caches_valid {
// Caches not valid, start over
log_rtab!(debug "cache validity key changed, emptying routing table");
drop(db);
2023-07-15 20:18:13 +00:00
table_store.delete(ROUTING_TABLE).await?;
let db = table_store.open(ROUTING_TABLE, 1).await?;
db.store(0, CACHE_VALIDITY_KEY, &cache_validity_key).await?;
2023-06-22 21:42:34 +00:00
return Ok(());
}
// Caches valid, load saved routing table
2023-07-15 20:18:13 +00:00
let Some(serialized_bucket_map): Option<SerializedBucketMap> = db.load_json(0, SERIALIZED_BUCKET_MAP).await? else {
2023-02-10 02:01:04 +00:00
log_rtab!(debug "no bucket map in saved routing table");
2022-11-06 21:07:56 +00:00
return Ok(());
};
2023-07-15 20:18:13 +00:00
let Some(all_entry_bytes): Option<SerializedBuckets> = db.load_json(0, ALL_ENTRY_BYTES).await? else {
2023-02-10 02:01:04 +00:00
log_rtab!(debug "no all_entry_bytes in saved routing table");
2022-11-06 21:07:56 +00:00
return Ok(());
2023-02-10 02:01:04 +00:00
};
// Reconstruct all entries
2023-03-03 22:17:23 +00:00
let inner = &mut *self.inner.write();
2023-05-29 19:24:57 +00:00
self.populate_routing_table(inner, serialized_bucket_map, all_entry_bytes)?;
2023-03-03 22:17:23 +00:00
2023-05-29 19:24:57 +00:00
Ok(())
}
/// Write the deserialized table store data to the routing table.
pub fn populate_routing_table(
&self,
inner: &mut RoutingTableInner,
serialized_bucket_map: SerializedBucketMap,
all_entry_bytes: SerializedBuckets,
) -> EyreResult<()> {
2023-02-10 02:01:04 +00:00
let mut all_entries: Vec<Arc<BucketEntry>> = Vec::with_capacity(all_entry_bytes.len());
for entry_bytes in all_entry_bytes {
2023-07-15 20:18:13 +00:00
let entryinner = deserialize_json_bytes(&entry_bytes)
.wrap_err("failed to deserialize bucket entry")?;
2023-03-03 22:17:23 +00:00
let entry = Arc::new(BucketEntry::new_with_inner(entryinner));
// Keep strong reference in table
all_entries.push(entry.clone());
// Keep all entries in weak table too
inner.all_entries.insert(entry);
2022-11-06 21:07:56 +00:00
}
2023-02-10 02:01:04 +00:00
// 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(());
}
2022-11-06 21:07:56 +00:00
}
2023-02-10 02:01:04 +00:00
// Recreate buckets
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)?;
}
2022-11-06 21:07:56 +00:00
}
Ok(())
}
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 {
2022-11-01 02:03:05 +00:00
self.inner.read().route_spec_store.as_ref().unwrap().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
}
2023-07-14 18:21:00 +00:00
pub fn relay_node_last_keepalive(&self, domain: RoutingDomain) -> Option<Timestamp> {
self.inner.read().relay_node_last_keepalive(domain)
}
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-11-10 03:27:37 +00:00
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)
}
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
pub fn get_contact_method(
&self,
routing_domain: RoutingDomain,
2022-11-10 03:27:37 +00:00
peer_a: &PeerInfo,
peer_b: &PeerInfo,
2022-10-16 23:59:59 +00:00
dial_info_filter: DialInfoFilter,
2022-10-21 14:35:03 +00:00
sequencing: Sequencing,
2023-08-22 01:04:21 +00:00
dif_sort: Option<Arc<dyn Fn(&DialInfoDetail, &DialInfoDetail) -> core::cmp::Ordering>>,
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,
2022-11-10 03:27:37 +00:00
peer_a,
peer_b,
2022-10-16 23:59:59 +00:00
dial_info_filter,
2022-10-21 14:35:03 +00:00
sequencing,
2023-08-22 01:04:21 +00:00
dif_sort,
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
2023-07-14 23:14:12 +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-19 01:53:45 +00:00
/// If we have a valid network class in this routing domain, then our 'NodeInfo' is valid
2022-12-08 15:24:33 +00:00
/// If this is true, we can get our final peer info, otherwise we only have a 'best effort' peer info
2023-07-14 23:14:12 +00:00
pub fn has_valid_network_class(&self, routing_domain: RoutingDomain) -> bool {
self.inner.read().has_valid_network_class(routing_domain)
2022-10-09 18:59:01 +00:00
}
2022-12-08 15:24:33 +00:00
/// Return our current node info timestamp
2023-07-14 23:14:12 +00:00
pub fn get_own_node_info_ts(&self, routing_domain: RoutingDomain) -> Timestamp {
2022-12-08 15:24:33 +00:00
self.inner.read().get_own_node_info_ts(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
2023-03-12 16:24:21 +00:00
/// May not empty buckets completely if there are existing node_refs
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-03 17:57:25 +00:00
pub fn get_entry_count(
&self,
routing_domain_set: RoutingDomainSet,
min_state: BucketEntryState,
2023-02-22 16:15:26 +00:00
crypto_kinds: &[CryptoKind],
2022-09-03 17:57:25 +00:00
) -> usize {
2022-10-19 01:53:45 +00:00
self.inner
.read()
2023-02-22 16:15:26 +00:00
.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<CryptoKind, usize> {
self.inner
.read()
.get_entry_count_per_crypto_kind(routing_domain_set, min_state)
}
2022-09-03 17:57:25 +00:00
pub fn get_nodes_needing_ping(
&self,
routing_domain: RoutingDomain,
2022-12-17 01:07:28 +00:00
cur_ts: Timestamp,
2022-09-03 17:57:25 +00:00
) -> 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
}
2022-12-17 01:07:28 +00:00
pub fn get_all_nodes(&self, cur_ts: Timestamp) -> Vec<NodeRef> {
2022-07-22 17:05:28 +00:00
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
}
fn queue_bucket_kicks(&self, node_ids: TypedKeyGroup) {
2023-02-12 04:16:32 +00:00
for node_id in node_ids.iter() {
2023-03-14 18:52:02 +00:00
// Skip node ids we didn't add to buckets
if !VALID_CRYPTO_KINDS.contains(&node_id.kind) {
continue;
}
// Put it in the kick queue
2023-02-17 00:54:29 +00:00
let x = self.unlocked_inner.calculate_bucket_index(node_id);
2023-02-12 04:16:32 +00:00
self.unlocked_inner.kick_queue.lock().insert(x);
}
2021-11-22 16:28:30 +00:00
}
2023-02-23 02:47:00 +00:00
/// Resolve an existing routing table entry using any crypto kind and return a reference to it
2023-06-16 00:22:54 +00:00
pub fn lookup_any_node_ref(&self, node_id_key: PublicKey) -> EyreResult<Option<NodeRef>> {
2023-02-23 02:47:00 +00:00
self.inner
.read()
.lookup_any_node_ref(self.clone(), node_id_key)
}
2022-10-13 02:53:40 +00:00
/// Resolve an existing routing table entry and return a reference to it
2023-06-16 00:22:54 +00:00
pub fn lookup_node_ref(&self, node_id: TypedKey) -> EyreResult<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,
2023-02-11 20:54:55 +00:00
node_id: TypedKey,
2022-10-13 02:53:40 +00:00
routing_domain_set: RoutingDomainSet,
dial_info_filter: DialInfoFilter,
2023-06-16 00:22:54 +00:00
) -> EyreResult<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.
2023-02-10 02:01:04 +00:00
pub fn register_node_with_peer_info(
2021-11-22 16:28:30 +00:00
&self,
2022-09-04 18:17:28 +00:00
routing_domain: RoutingDomain,
2023-02-10 02:01:04 +00:00
peer_info: PeerInfo,
2022-09-04 18:17:28 +00:00
allow_invalid: bool,
2023-06-16 00:22:54 +00:00
) -> EyreResult<NodeRef> {
2023-02-10 02:01:04 +00:00
self.inner.write().register_node_with_peer_info(
2022-10-19 01:53:45 +00:00
self.clone(),
routing_domain,
2023-02-10 02:01:04 +00:00
peer_info,
2022-10-19 01:53:45 +00:00
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,
2023-02-11 20:54:55 +00:00
node_id: TypedKey,
2021-11-22 16:28:30 +00:00
descriptor: ConnectionDescriptor,
2022-12-17 01:07:28 +00:00
timestamp: Timestamp,
2023-06-16 00:22:54 +00:00
) -> EyreResult<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-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()
}
2023-02-11 20:54:55 +00:00
pub fn get_recent_peers(&self) -> Vec<(TypedKey, RecentPeersEntry)> {
2022-11-03 02:21:22 +00:00
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;
2023-06-16 00:22:54 +00:00
if let Ok(Some(nr)) = self.lookup_node_ref(*e) {
2022-11-03 02:21:22 +00:00
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
2022-10-19 01:53:45 +00:00
}
2023-02-11 20:54:55 +00:00
pub fn touch_recent_peer(&self, node_id: TypedKey, last_connection: ConnectionDescriptor) {
2022-10-19 01:53:45 +00:00
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
2022-11-02 01:05:48 +00:00
pub fn make_inbound_dial_info_entry_filter<'a>(
2022-10-19 01:53:45 +00:00
routing_domain: RoutingDomain,
dial_info_filter: DialInfoFilter,
2022-11-02 20:29:29 +00:00
) -> RoutingTableEntryFilter<'a> {
2022-10-19 01:53:45 +00:00
// does it have matching public dial info?
2023-02-11 20:54:55 +00:00
Box::new(move |rti, e| {
2022-11-02 01:05:48 +00:00
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)
2022-10-19 01:53:45 +00:00
.is_some()
2022-03-24 14:14:50 +00:00
}
2022-11-01 02:03:05 +00:00
})
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
2022-11-02 20:29:29 +00:00
pub fn make_outbound_dial_info_entry_filter<'a>(
2022-10-19 01:53:45 +00:00
routing_domain: RoutingDomain,
dial_info: DialInfo,
2022-11-02 20:29:29 +00:00
) -> RoutingTableEntryFilter<'a> {
2022-10-19 01:53:45 +00:00
// does the node's outbound capabilities match the dialinfo?
2023-02-11 20:54:55 +00:00
Box::new(move |rti, e| {
2022-11-02 01:05:48 +00:00
if let Some(e) = e {
e.with(rti, |_rti, e| {
if let Some(ni) = e.node_info(routing_domain) {
let dif = DialInfoFilter::all()
2023-05-29 19:24:57 +00:00
.with_protocol_type_set(ni.outbound_protocols())
.with_address_type_set(ni.address_types());
2022-11-02 01:05:48 +00:00
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)
2022-09-07 15:30:43 +00:00
}
2022-11-01 02:03:05 +00:00
})
2022-10-19 01:53:45 +00:00
}
2022-11-02 01:05:48 +00:00
pub fn find_fast_public_nodes_filtered(
2022-10-19 01:53:45 +00:00
&self,
node_count: usize,
2022-11-02 01:05:48 +00:00
filters: VecDeque<RoutingTableEntryFilter>,
) -> Vec<NodeRef> {
2022-10-19 01:53:45 +00:00
self.inner
.read()
2022-11-02 01:05:48 +00:00
.find_fast_public_nodes_filtered(self.clone(), node_count, filters)
2022-10-19 01:53:45 +00:00
}
2023-02-22 16:15:26 +00:00
/// 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<NodeRef> {
2022-10-19 01:53:45 +00:00
let protocol_types = vec![
ProtocolType::UDP,
ProtocolType::TCP,
ProtocolType::WS,
ProtocolType::WSS,
];
2023-02-22 16:15:26 +00:00
2022-11-02 01:05:48 +00:00
let protocol_types_len = protocol_types.len();
2022-10-19 01:53:45 +00:00
let mut nodes_proto_v4 = vec![0usize, 0usize, 0usize, 0usize];
let mut nodes_proto_v6 = vec![0usize, 0usize, 0usize, 0usize];
2022-11-02 01:05:48 +00:00
let filter = Box::new(
2023-02-11 20:54:55 +00:00
move |rti: &RoutingTableInner, entry: Option<Arc<BucketEntry>>| {
let entry = entry.unwrap();
2022-10-19 01:53:45 +00:00
entry.with(rti, |_rti, e| {
// skip nodes on our local network here
if e.has_node_info(RoutingDomain::LocalNetwork.into()) {
return false;
}
2023-02-22 16:15:26 +00:00
// Ensure crypto kind is supported
if !e.crypto_kinds().contains(&crypto_kind) {
return false;
}
2022-10-19 01:53:45 +00:00
// does it have some dial info we need?
let filter = |n: &NodeInfo| {
let mut keep = false;
2023-05-29 19:24:57 +00:00
for did in n.dial_info_detail_list() {
2022-10-19 01:53:45 +00:00
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)
})
},
2022-11-02 01:05:48 +00:00
) as RoutingTableEntryFilter;
let filters = VecDeque::from([filter]);
2023-09-09 00:38:31 +00:00
self.find_preferred_fastest_nodes(
2022-11-02 01:05:48 +00:00
protocol_types_len * 2 * max_per_type,
filters,
2023-02-11 20:54:55 +00:00
|_rti, entry: Option<Arc<BucketEntry>>| {
NodeRef::new(self.clone(), entry.unwrap().clone(), None)
2022-10-19 01:53:45 +00:00
},
)
}
2023-02-22 16:15:26 +00:00
/// 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<NodeRef> {
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 {
2023-03-01 02:11:26 +00:00
for nro in &out {
2023-02-22 16:15:26 +00:00
if nro.same_entry(&nr) {
continue 'nrloop;
}
}
out.push(nr);
}
}
out
}
2022-11-02 01:05:48 +00:00
pub fn find_peers_with_sort_and_filter<C, T, O>(
2022-10-19 01:53:45 +00:00
&self,
node_count: usize,
2022-12-17 01:07:28 +00:00
cur_ts: Timestamp,
2022-11-02 01:05:48 +00:00
filters: VecDeque<RoutingTableEntryFilter>,
2022-10-19 01:53:45 +00:00
compare: C,
2022-11-02 01:05:48 +00:00
transform: T,
2022-10-19 01:53:45 +00:00
) -> Vec<O>
where
2022-11-02 01:05:48 +00:00
C: for<'a, 'b> FnMut(
2022-10-19 01:53:45 +00:00
&'a RoutingTableInner,
2023-02-11 20:54:55 +00:00
&'b Option<Arc<BucketEntry>>,
&'b Option<Arc<BucketEntry>>,
2022-10-19 01:53:45 +00:00
) -> core::cmp::Ordering,
2023-02-11 20:54:55 +00:00
T: for<'r> FnMut(&'r RoutingTableInner, Option<Arc<BucketEntry>>) -> O + Send,
2022-10-19 01:53:45 +00:00
{
self.inner
.read()
2022-11-02 01:05:48 +00:00
.find_peers_with_sort_and_filter(node_count, cur_ts, filters, compare, transform)
2022-10-19 01:53:45 +00:00
}
2023-09-09 00:38:31 +00:00
pub fn find_preferred_fastest_nodes<'a, T, O>(
2022-10-19 01:53:45 +00:00
&self,
node_count: usize,
2022-11-02 01:05:48 +00:00
filters: VecDeque<RoutingTableEntryFilter>,
2022-10-19 01:53:45 +00:00
transform: T,
) -> Vec<O>
where
2023-02-11 20:54:55 +00:00
T: for<'r> FnMut(&'r RoutingTableInner, Option<Arc<BucketEntry>>) -> O + Send,
2022-10-19 01:53:45 +00:00
{
self.inner
.read()
2023-09-09 00:38:31 +00:00
.find_preferred_fastest_nodes(node_count, filters, transform)
2022-10-19 01:53:45 +00:00
}
2023-09-09 00:38:31 +00:00
pub fn find_preferred_closest_nodes<'a, T, O>(
2022-10-19 01:53:45 +00:00
&self,
2023-02-13 21:12:46 +00:00
node_count: usize,
2023-02-11 20:54:55 +00:00
node_id: TypedKey,
2022-11-02 01:05:48 +00:00
filters: VecDeque<RoutingTableEntryFilter>,
transform: T,
2023-08-05 15:33:27 +00:00
) -> VeilidAPIResult<Vec<O>>
2022-10-19 01:53:45 +00:00
where
2023-02-11 20:54:55 +00:00
T: for<'r> FnMut(&'r RoutingTableInner, Option<Arc<BucketEntry>>) -> O + Send,
2022-10-19 01:53:45 +00:00
{
self.inner
.read()
2023-09-09 00:38:31 +00:00
.find_preferred_closest_nodes(node_count, node_id, filters, transform)
2022-10-19 01:53:45 +00:00
}
2022-11-02 01:05:48 +00:00
2023-05-29 19:24:57 +00:00
pub fn sort_and_clean_closest_noderefs(
&self,
node_id: TypedKey,
2023-09-09 00:38:31 +00:00
closest_nodes: &[NodeRef],
) -> Vec<NodeRef> {
2023-05-29 19:24:57 +00:00
self.inner
.read()
.sort_and_clean_closest_noderefs(node_id, closest_nodes)
}
2023-07-14 01:12:59 +00:00
#[instrument(level = "trace", skip(self, peers))]
2023-02-21 01:37:52 +00:00
pub fn register_find_node_answer(
&self,
crypto_kind: CryptoKind,
peers: Vec<PeerInfo>,
) -> Vec<NodeRef> {
// Register nodes we'd found
2022-10-19 01:53:45 +00:00
let mut out = Vec::<NodeRef>::with_capacity(peers.len());
for p in peers {
2023-02-21 01:37:52 +00:00
// Ensure we're getting back nodes we asked for
2023-05-29 19:24:57 +00:00
if !p.node_ids().kinds().contains(&crypto_kind) {
2023-02-21 01:37:52 +00:00
continue;
}
2023-03-05 19:31:11 +00:00
// Don't register our own node
2023-05-29 19:24:57 +00:00
if self.matches_own_node_id(p.node_ids()) {
2023-03-05 19:31:11 +00:00
continue;
}
2023-02-21 01:37:52 +00:00
// Register the node if it's new
2023-06-16 00:22:54 +00:00
match self.register_node_with_peer_info(RoutingDomain::PublicInternet, p, false) {
Ok(nr) => out.push(nr),
Err(e) => {
log_rtab!(debug "failed to register node with peer info from find node answer: {}", e);
}
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
2023-07-14 01:12:59 +00:00
#[instrument(level = "trace", skip(self), err)]
2022-10-19 01:53:45 +00:00
pub async fn find_node(
&self,
node_ref: NodeRef,
2023-02-11 20:54:55 +00:00
node_id: TypedKey,
2022-10-19 01:53:45 +00:00
) -> EyreResult<NetworkResult<Vec<NodeRef>>> {
let rpc_processor = self.rpc_processor();
let res = network_result_try!(
rpc_processor
.clone()
2023-07-04 16:35:48 +00:00
.rpc_call_find_node(Destination::direct(node_ref), node_id, vec![])
2022-10-19 01:53:45 +00:00
.await?
);
// register nodes we'd found
Ok(NetworkResult::value(
2023-02-21 01:37:52 +00:00
self.register_find_node_answer(node_id.kind, res.answer),
2022-10-19 01:53:45 +00:00
))
}
2023-02-21 01:37:52 +00:00
/// Ask a remote node to list the nodes it has around the current node
2023-07-14 01:12:59 +00:00
#[instrument(level = "trace", skip(self), err)]
2023-02-21 01:37:52 +00:00
pub async fn find_self(
&self,
crypto_kind: CryptoKind,
node_ref: NodeRef,
) -> EyreResult<NetworkResult<Vec<NodeRef>>> {
let self_node_id = self.node_id(crypto_kind);
2023-02-13 21:12:46 +00:00
self.find_node(node_ref, self_node_id).await
2022-10-19 01:53:45 +00:00
}
2023-02-21 01:37:52 +00:00
/// Ask a remote node to list the nodes it has around itself
2023-07-14 01:12:59 +00:00
#[instrument(level = "trace", skip(self), err)]
2023-02-21 01:37:52 +00:00
pub async fn find_target(
&self,
crypto_kind: CryptoKind,
node_ref: NodeRef,
) -> EyreResult<NetworkResult<Vec<NodeRef>>> {
let Some(target_node_id) = node_ref.node_ids().get(crypto_kind) else {
bail!("no target node ids for this crypto kind");
};
2023-02-13 21:12:46 +00:00
self.find_node(node_ref, target_node_id).await
2022-10-19 01:53:45 +00:00
}
#[instrument(level = "trace", skip(self))]
2023-02-21 01:37:52 +00:00
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
2022-10-19 01:53:45 +00:00
// and then contact those nodes to inform -them- that we exist
2023-02-21 01:37:52 +00:00
// 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 {
2022-10-19 01:53:45 +00:00
Err(e) => {
log_rtab!(error
"find_self failed for {:?}: {:?}",
&node_ref, e
);
return;
}
Ok(v) => v,
2023-06-25 18:09:22 +00:00
} => [ format!(": crypto_kind={} node_ref={} wide={}", crypto_kind, node_ref, wide) ] {
2022-10-19 01:53:45 +00:00
return;
});
// Ask each node near us to find us as well
if wide {
for closest_nr in closest_nodes {
2023-02-21 01:37:52 +00:00
network_result_value_or_log!(match self.find_self(crypto_kind, closest_nr.clone()).await {
2022-10-19 01:53:45 +00:00
Err(e) => {
log_rtab!(error
"find_self failed for {:?}: {:?}",
&closest_nr, e
);
continue;
}
Ok(v) => v,
2023-06-25 18:09:22 +00:00
} => [ format!(": crypto_kind={} closest_nr={} wide={}", crypto_kind, closest_nr, wide) ] {
2022-10-19 01:53:45 +00:00
// Do nothing with non-values
continue;
});
2022-09-07 15:30:43 +00:00
}
}
2022-10-19 01:53:45 +00:00
}
2021-11-22 16:28:30 +00:00
}
2023-02-14 02:12:27 +00:00
impl core::ops::Deref for RoutingTable {
type Target = RoutingTableUnlockedInner;
fn deref(&self) -> &Self::Target {
&self.unlocked_inner
}
}