From 3165b12c75c1796995e7ee3793c56f8e8e05f3d9 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Tue, 9 May 2023 04:50:00 +0200 Subject: [PATCH] feat(kad): remove deprecated public modules This patch removes the 3 out of 4 deprecated public modules. I've left `store` for now because we made some mistakes in declaring that. The items within `store` still need to be publicly visible but I haven't yet figured out a good way of exporting / naming them. Thus, I've left that to a follow-up PR. Related: #3647. Pull-Request: #3896. --- misc/metrics/src/identify.rs | 2 +- protocols/kad/CHANGELOG.md | 4 + protocols/kad/src/behaviour.rs | 134 +++++++++--------- protocols/kad/src/behaviour/test.rs | 22 +-- .../kad/src/{handler_priv.rs => handler.rs} | 0 .../kad/src/{kbucket_priv.rs => kbucket.rs} | 27 ++-- .../src/{kbucket_priv => kbucket}/bucket.rs | 11 +- .../src/{kbucket_priv => kbucket}/entry.rs | 39 +++-- .../kad/src/{kbucket_priv => kbucket}/key.rs | 0 protocols/kad/src/lib.rs | 32 +---- .../kad/src/{protocol_priv.rs => protocol.rs} | 14 +- protocols/kad/src/query.rs | 2 +- protocols/kad/src/query/peers/closest.rs | 2 +- .../kad/src/query/peers/closest/disjoint.rs | 2 +- protocols/kad/src/record_priv/store/memory.rs | 18 +-- 15 files changed, 144 insertions(+), 165 deletions(-) rename protocols/kad/src/{handler_priv.rs => handler.rs} (100%) rename protocols/kad/src/{kbucket_priv.rs => kbucket.rs} (97%) rename protocols/kad/src/{kbucket_priv => kbucket}/bucket.rs (99%) rename protocols/kad/src/{kbucket_priv => kbucket}/entry.rs (88%) rename protocols/kad/src/{kbucket_priv => kbucket}/key.rs (100%) rename protocols/kad/src/{protocol_priv.rs => protocol.rs} (98%) diff --git a/misc/metrics/src/identify.rs b/misc/metrics/src/identify.rs index 81e2ace6279..ffd0cdb9fc2 100644 --- a/misc/metrics/src/identify.rs +++ b/misc/metrics/src/identify.rs @@ -148,7 +148,7 @@ impl super::Recorder for Metrics { libp2p_identify::PROTOCOL_NAME, libp2p_identify::PUSH_PROTOCOL_NAME, #[cfg(feature = "kad")] - libp2p_kad::protocol::DEFAULT_PROTO_NAME, + libp2p_kad::PROTOCOL_NAME, #[cfg(feature = "ping")] libp2p_ping::PROTOCOL_NAME, #[cfg(feature = "relay")] diff --git a/protocols/kad/CHANGELOG.md b/protocols/kad/CHANGELOG.md index 525be910308..d7a35b1b8fb 100644 --- a/protocols/kad/CHANGELOG.md +++ b/protocols/kad/CHANGELOG.md @@ -3,7 +3,11 @@ - Raise MSRV to 1.65. See [PR 3715]. +- Remove deprecated public modules `handler`, `protocol` and `kbucket`. + See [PR 3896]. + [PR 3715]: https://github.com/libp2p/rust-libp2p/pull/3715 +[PR 3896]: https://github.com/libp2p/rust-libp2p/pull/3896 ## 0.43.3 diff --git a/protocols/kad/src/behaviour.rs b/protocols/kad/src/behaviour.rs index 856b8379e2c..9193770e09a 100644 --- a/protocols/kad/src/behaviour.rs +++ b/protocols/kad/src/behaviour.rs @@ -23,12 +23,12 @@ mod test; use crate::addresses::Addresses; -use crate::handler_priv::{ +use crate::handler::{ KademliaHandler, KademliaHandlerConfig, KademliaHandlerEvent, KademliaHandlerIn, KademliaRequestId, }; use crate::jobs::*; -use crate::kbucket_priv::{self, Distance, KBucketsTable, NodeStatus}; +use crate::kbucket::{self, Distance, KBucketsTable, NodeStatus}; use crate::protocol::{KadConnectionType, KadPeer, KademliaProtocolConfig}; use crate::query::{Query, QueryConfig, QueryId, QueryPool, QueryPoolState}; use crate::record_priv::{ @@ -66,7 +66,7 @@ pub use crate::query::QueryStats; /// Kademlia protocol. pub struct Kademlia { /// The Kademlia routing table. - kbuckets: KBucketsTable, Addresses>, + kbuckets: KBucketsTable, Addresses>, /// The k-bucket insertion strategy. kbucket_inserts: KademliaBucketInserts, @@ -417,7 +417,7 @@ where /// Creates a new `Kademlia` network behaviour with the given configuration. pub fn with_config(id: PeerId, store: TStore, config: KademliaConfig) -> Self { - let local_key = kbucket_priv::Key::from(id); + let local_key = kbucket::Key::from(id); let put_record_job = config .record_replication_interval @@ -518,9 +518,9 @@ where /// If the routing table has been updated as a result of this operation, /// a [`KademliaEvent::RoutingUpdated`] event is emitted. pub fn add_address(&mut self, peer: &PeerId, address: Multiaddr) -> RoutingUpdate { - let key = kbucket_priv::Key::from(*peer); + let key = kbucket::Key::from(*peer); match self.kbuckets.entry(&key) { - kbucket_priv::Entry::Present(mut entry, _) => { + kbucket::Entry::Present(mut entry, _) => { if entry.value().insert(address) { self.queued_events.push_back(ToSwarm::GenerateEvent( KademliaEvent::RoutingUpdated { @@ -538,11 +538,11 @@ where } RoutingUpdate::Success } - kbucket_priv::Entry::Pending(mut entry, _) => { + kbucket::Entry::Pending(mut entry, _) => { entry.value().insert(address); RoutingUpdate::Pending } - kbucket_priv::Entry::Absent(entry) => { + kbucket::Entry::Absent(entry) => { let addresses = Addresses::new(address); let status = if self.connected_peers.contains(peer) { NodeStatus::Connected @@ -550,7 +550,7 @@ where NodeStatus::Disconnected }; match entry.insert(addresses.clone(), status) { - kbucket_priv::InsertResult::Inserted => { + kbucket::InsertResult::Inserted => { self.queued_events.push_back(ToSwarm::GenerateEvent( KademliaEvent::RoutingUpdated { peer: *peer, @@ -566,11 +566,11 @@ where )); RoutingUpdate::Success } - kbucket_priv::InsertResult::Full => { + kbucket::InsertResult::Full => { debug!("Bucket full. Peer not added to routing table: {}", peer); RoutingUpdate::Failed } - kbucket_priv::InsertResult::Pending { disconnected } => { + kbucket::InsertResult::Pending { disconnected } => { self.queued_events.push_back(ToSwarm::Dial { opts: DialOpts::peer_id(disconnected.into_preimage()).build(), }); @@ -578,7 +578,7 @@ where } } } - kbucket_priv::Entry::SelfEntry => RoutingUpdate::Failed, + kbucket::Entry::SelfEntry => RoutingUpdate::Failed, } } @@ -596,24 +596,24 @@ where &mut self, peer: &PeerId, address: &Multiaddr, - ) -> Option, Addresses>> { - let key = kbucket_priv::Key::from(*peer); + ) -> Option, Addresses>> { + let key = kbucket::Key::from(*peer); match self.kbuckets.entry(&key) { - kbucket_priv::Entry::Present(mut entry, _) => { + kbucket::Entry::Present(mut entry, _) => { if entry.value().remove(address).is_err() { Some(entry.remove()) // it is the last address, thus remove the peer. } else { None } } - kbucket_priv::Entry::Pending(mut entry, _) => { + kbucket::Entry::Pending(mut entry, _) => { if entry.value().remove(address).is_err() { Some(entry.remove()) // it is the last address, thus remove the peer. } else { None } } - kbucket_priv::Entry::Absent(..) | kbucket_priv::Entry::SelfEntry => None, + kbucket::Entry::Absent(..) | kbucket::Entry::SelfEntry => None, } } @@ -624,20 +624,19 @@ where pub fn remove_peer( &mut self, peer: &PeerId, - ) -> Option, Addresses>> { - let key = kbucket_priv::Key::from(*peer); + ) -> Option, Addresses>> { + let key = kbucket::Key::from(*peer); match self.kbuckets.entry(&key) { - kbucket_priv::Entry::Present(entry, _) => Some(entry.remove()), - kbucket_priv::Entry::Pending(entry, _) => Some(entry.remove()), - kbucket_priv::Entry::Absent(..) | kbucket_priv::Entry::SelfEntry => None, + kbucket::Entry::Present(entry, _) => Some(entry.remove()), + kbucket::Entry::Pending(entry, _) => Some(entry.remove()), + kbucket::Entry::Absent(..) | kbucket::Entry::SelfEntry => None, } } /// Returns an iterator over all non-empty buckets in the routing table. pub fn kbuckets( &mut self, - ) -> impl Iterator, Addresses>> - { + ) -> impl Iterator, Addresses>> { self.kbuckets.iter().filter(|b| !b.is_empty()) } @@ -647,9 +646,9 @@ where pub fn kbucket( &mut self, key: K, - ) -> Option, Addresses>> + ) -> Option, Addresses>> where - K: Into> + Clone, + K: Into> + Clone, { self.kbuckets.bucket(&key.into()) } @@ -660,16 +659,15 @@ where /// [`KademliaEvent::OutboundQueryCompleted{QueryResult::GetClosestPeers}`]. pub fn get_closest_peers(&mut self, key: K) -> QueryId where - K: Into> + Into> + Clone, + K: Into> + Into> + Clone, { - let target: kbucket_priv::Key = key.clone().into(); + let target: kbucket::Key = key.clone().into(); let key: Vec = key.into(); let info = QueryInfo::GetClosestPeers { key, step: ProgressStep::first(), }; - let peer_keys: Vec> = - self.kbuckets.closest_keys(&target).collect(); + let peer_keys: Vec> = self.kbuckets.closest_keys(&target).collect(); let inner = QueryInner::new(info); self.queries.add_iter_closest(target, peer_keys, inner) } @@ -677,8 +675,8 @@ where /// Returns closest peers to the given key; takes peers from local routing table only. pub fn get_closest_local_peers<'a, K: Clone>( &'a mut self, - key: &'a kbucket_priv::Key, - ) -> impl Iterator> + 'a { + key: &'a kbucket::Key, + ) -> impl Iterator> + 'a { self.kbuckets.closest_keys(key) } @@ -703,7 +701,7 @@ where let step = ProgressStep::first(); - let target = kbucket_priv::Key::new(key.clone()); + let target = kbucket::Key::new(key.clone()); let info = if record.is_some() { QueryInfo::GetRecord { key, @@ -769,7 +767,7 @@ where .expires .or_else(|| self.record_ttl.map(|ttl| Instant::now() + ttl)); let quorum = quorum.eval(self.queries.config().replication_factor); - let target = kbucket_priv::Key::new(record.key.clone()); + let target = kbucket::Key::new(record.key.clone()); let peers = self.kbuckets.closest_keys(&target); let context = PutRecordContext::Publish; let info = QueryInfo::PutRecord { @@ -918,7 +916,7 @@ where local_addrs, ); self.store.add_provider(record)?; - let target = kbucket_priv::Key::new(key.clone()); + let target = kbucket::Key::new(key.clone()); let peers = self.kbuckets.closest_keys(&target); let context = AddProviderContext::Publish; let info = QueryInfo::AddProvider { @@ -965,7 +963,7 @@ where }, }; - let target = kbucket_priv::Key::new(key.clone()); + let target = kbucket::Key::new(key.clone()); let peers = self.kbuckets.closest_keys(&target); let inner = QueryInner::new(info); let id = self.queries.add_iter_closest(target.clone(), peers, inner); @@ -1017,7 +1015,7 @@ where /// result. fn find_closest( &mut self, - target: &kbucket_priv::Key, + target: &kbucket::Key, source: &PeerId, ) -> Vec { if target == self.kbuckets.local_key() { @@ -1067,7 +1065,7 @@ where .collect::>(), ) } else { - let key = kbucket_priv::Key::from(node_id); + let key = kbucket::Key::from(node_id); kbuckets .entry(&key) .view() @@ -1096,7 +1094,7 @@ where key: key.clone(), phase: AddProviderPhase::GetClosestPeers, }; - let target = kbucket_priv::Key::new(key); + let target = kbucket::Key::new(key); let peers = self.kbuckets.closest_keys(&target); let inner = QueryInner::new(info); self.queries.add_iter_closest(target.clone(), peers, inner); @@ -1105,7 +1103,7 @@ where /// Starts an iterative `PUT_VALUE` query for the given record. fn start_put_record(&mut self, record: Record, quorum: Quorum, context: PutRecordContext) { let quorum = quorum.eval(self.queries.config().replication_factor); - let target = kbucket_priv::Key::new(record.key.clone()); + let target = kbucket::Key::new(record.key.clone()); let peers = self.kbuckets.closest_keys(&target); let info = QueryInfo::PutRecord { record, @@ -1124,9 +1122,9 @@ where address: Option, new_status: NodeStatus, ) { - let key = kbucket_priv::Key::from(peer); + let key = kbucket::Key::from(peer); match self.kbuckets.entry(&key) { - kbucket_priv::Entry::Present(mut entry, old_status) => { + kbucket::Entry::Present(mut entry, old_status) => { if old_status != new_status { entry.update(new_status) } @@ -1149,7 +1147,7 @@ where } } - kbucket_priv::Entry::Pending(mut entry, old_status) => { + kbucket::Entry::Pending(mut entry, old_status) => { if let Some(address) = address { entry.value().insert(address); } @@ -1158,7 +1156,7 @@ where } } - kbucket_priv::Entry::Absent(entry) => { + kbucket::Entry::Absent(entry) => { // Only connected nodes with a known address are newly inserted. if new_status != NodeStatus::Connected { return; @@ -1177,7 +1175,7 @@ where (Some(a), KademliaBucketInserts::OnConnected) => { let addresses = Addresses::new(a); match entry.insert(addresses.clone(), new_status) { - kbucket_priv::InsertResult::Inserted => { + kbucket::InsertResult::Inserted => { let event = KademliaEvent::RoutingUpdated { peer, is_new_peer: true, @@ -1191,14 +1189,14 @@ where }; self.queued_events.push_back(ToSwarm::GenerateEvent(event)); } - kbucket_priv::InsertResult::Full => { + kbucket::InsertResult::Full => { debug!("Bucket full. Peer not added to routing table: {}", peer); let address = addresses.first().clone(); self.queued_events.push_back(ToSwarm::GenerateEvent( KademliaEvent::RoutablePeer { peer, address }, )); } - kbucket_priv::InsertResult::Pending { disconnected } => { + kbucket::InsertResult::Pending { disconnected } => { let address = addresses.first().clone(); self.queued_events.push_back(ToSwarm::GenerateEvent( KademliaEvent::PendingRoutablePeer { peer, address }, @@ -1261,13 +1259,13 @@ where // Pr(bucket-253) = 1 - (7/8)^16 ~= 0.88 // Pr(bucket-252) = 1 - (15/16)^16 ~= 0.64 // ... - let mut target = kbucket_priv::Key::from(PeerId::random()); + let mut target = kbucket::Key::from(PeerId::random()); for _ in 0..16 { let d = local_key.distance(&target); if b.contains(&d) { break; } - target = kbucket_priv::Key::from(PeerId::random()); + target = kbucket::Key::from(PeerId::random()); } target }) @@ -1632,7 +1630,7 @@ where // number of nodes between the local node and the closest node to the key // (beyond the replication factor). This ensures avoiding over-caching // outside of the k closest nodes to a key. - let target = kbucket_priv::Key::new(record.key.clone()); + let target = kbucket::Key::new(record.key.clone()); let num_between = self.kbuckets.count_nodes_between(&target); let k = self.queries.config().replication_factor.get(); let num_beyond_k = (usize::max(k, num_between) - k) as u32; @@ -1762,7 +1760,7 @@ where } fn address_failed(&mut self, peer_id: PeerId, address: &Multiaddr) { - let key = kbucket_priv::Key::from(peer_id); + let key = kbucket::Key::from(peer_id); if let Some(addrs) = self.kbuckets.entry(&key).value() { // TODO: Ideally, the address should only be removed if the error can @@ -1851,7 +1849,7 @@ where let (old, new) = (old.get_remote_address(), new.get_remote_address()); // Update routing table. - if let Some(addrs) = self.kbuckets.entry(&kbucket_priv::Key::from(peer)).value() { + if let Some(addrs) = self.kbuckets.entry(&kbucket::Key::from(peer)).value() { if addrs.replace(old, new) { debug!( "Address '{}' replaced with '{}' for peer '{}'.", @@ -2020,9 +2018,9 @@ where // We should order addresses from decreasing likelyhood of connectivity, so start with // the addresses of that peer in the k-buckets. - let key = kbucket_priv::Key::from(peer_id); + let key = kbucket::Key::from(peer_id); let mut peer_addrs = - if let kbucket_priv::Entry::Present(mut entry, _) = self.kbuckets.entry(&key) { + if let kbucket::Entry::Present(mut entry, _) = self.kbuckets.entry(&key) { let addrs = entry.value().iter().cloned().collect::>(); debug_assert!(!addrs.is_empty(), "Empty peer addresses in routing table."); addrs @@ -2061,7 +2059,7 @@ where } KademliaHandlerEvent::FindNodeReq { key, request_id } => { - let closer_peers = self.find_closest(&kbucket_priv::Key::new(key), &source); + let closer_peers = self.find_closest(&kbucket::Key::new(key), &source); self.queued_events.push_back(ToSwarm::GenerateEvent( KademliaEvent::InboundRequest { @@ -2090,7 +2088,7 @@ where KademliaHandlerEvent::GetProvidersReq { key, request_id } => { let provider_peers = self.provider_peers(&key, &source); - let closer_peers = self.find_closest(&kbucket_priv::Key::new(key), &source); + let closer_peers = self.find_closest(&kbucket::Key::new(key), &source); self.queued_events.push_back(ToSwarm::GenerateEvent( KademliaEvent::InboundRequest { @@ -2186,7 +2184,7 @@ where None => None, }; - let closer_peers = self.find_closest(&kbucket_priv::Key::new(key), &source); + let closer_peers = self.find_closest(&kbucket::Key::new(key), &source); self.queued_events.push_back(ToSwarm::GenerateEvent( KademliaEvent::InboundRequest { @@ -2244,8 +2242,8 @@ where } else { log::trace!("Record with key {:?} not found at {}", key, source); if let KademliaCaching::Enabled { max_peers } = self.caching { - let source_key = kbucket_priv::Key::from(source); - let target_key = kbucket_priv::Key::from(key.clone()); + let source_key = kbucket::Key::from(source); + let target_key = kbucket::Key::from(key.clone()); let distance = source_key.distance(&target_key); cache_candidates.insert(distance, source); if cache_candidates.len() > max_peers as usize { @@ -2350,7 +2348,7 @@ where // Drain applied pending entries from the routing table. if let Some(entry) = self.kbuckets.take_applied_pending() { - let kbucket_priv::Node { key, value } = entry.inserted; + let kbucket::Node { key, value } = entry.inserted; let event = KademliaEvent::RoutingUpdated { bucket_range: self .kbuckets @@ -2515,7 +2513,7 @@ pub enum KademliaEvent { is_new_peer: bool, /// The full list of known addresses of `peer`. addresses: Addresses, - /// Returns the minimum inclusive and maximum inclusive [`Distance`] for + /// Returns the minimum inclusive and maximum inclusive distance for /// the bucket of the peer. bucket_range: (Distance, Distance), /// The ID of the peer that was evicted from the routing table to make @@ -2598,7 +2596,7 @@ pub enum InboundRequest { num_closer_peers: usize, num_provider_peers: usize, }, - /// A peer sent a [`KademliaHandlerIn::AddProvider`] request. + /// A peer sent an add provider request. /// If filtering [`KademliaStoreInserts::FilterBoth`] is enabled, the [`ProviderRecord`] is /// included. /// @@ -2609,7 +2607,7 @@ pub enum InboundRequest { num_closer_peers: usize, present_locally: bool, }, - /// A peer sent a [`KademliaHandlerIn::PutRecord`] request. + /// A peer sent a put record request. /// If filtering [`KademliaStoreInserts::FilterBoth`] is enabled, the [`Record`] is included. /// /// See [`KademliaStoreInserts`] and [`KademliaConfig::set_record_filtering`]. @@ -2665,7 +2663,7 @@ pub enum GetRecordOk { /// If the lookup used a quorum > 1, you may wish to use these /// candidates with [`Kademlia::put_record_to`] after selecting /// one of the returned records. - cache_candidates: BTreeMap, + cache_candidates: BTreeMap, }, } @@ -2884,8 +2882,8 @@ impl AddProviderError { } } -impl From, Addresses>> for KadPeer { - fn from(e: kbucket_priv::EntryView, Addresses>) -> KadPeer { +impl From, Addresses>> for KadPeer { + fn from(e: kbucket::EntryView, Addresses>) -> KadPeer { KadPeer { node_id: e.node.key.into_preimage(), multiaddrs: e.node.value.into_vec(), @@ -2961,7 +2959,7 @@ pub enum QueryInfo { /// This is `None` if the initial self-lookup has not /// yet completed and `Some` with an exhausted iterator /// if bootstrapping is complete. - remaining: Option>>, + remaining: Option>>, step: ProgressStep, }, @@ -3014,7 +3012,7 @@ pub enum QueryInfo { found_a_record: bool, /// The peers closest to the `key` that were queried but did not return a record, /// i.e. the peers that are candidates for caching the record. - cache_candidates: BTreeMap, + cache_candidates: BTreeMap, }, } diff --git a/protocols/kad/src/behaviour/test.rs b/protocols/kad/src/behaviour/test.rs index 3f03842aff5..43145f0c79e 100644 --- a/protocols/kad/src/behaviour/test.rs +++ b/protocols/kad/src/behaviour/test.rs @@ -22,7 +22,7 @@ use super::*; -use crate::kbucket_priv::Distance; +use crate::kbucket::Distance; use crate::record_priv::{store::MemoryStore, Key}; use crate::{K_VALUE, SHA_256_MH}; use futures::{executor::block_on, future::poll_fn, prelude::*}; @@ -234,10 +234,10 @@ fn bootstrap() { #[test] fn query_iter() { - fn distances(key: &kbucket_priv::Key, peers: Vec) -> Vec { + fn distances(key: &kbucket::Key, peers: Vec) -> Vec { peers .into_iter() - .map(kbucket_priv::Key::from) + .map(kbucket::Key::from) .map(|k| k.distance(key)) .collect() } @@ -253,7 +253,7 @@ fn query_iter() { // Ask the first peer in the list to search a random peer. The search should // propagate forwards through the list of peers. let search_target = PeerId::random(); - let search_target_key = kbucket_priv::Key::from(search_target); + let search_target_key = kbucket::Key::from(search_target); let qid = swarms[0].behaviour_mut().get_closest_peers(search_target); match swarms[0].behaviour_mut().query(&qid) { @@ -290,7 +290,7 @@ fn query_iter() { assert_eq!(swarm_ids[i], expected_swarm_id); assert_eq!(swarm.behaviour_mut().queries.size(), 0); assert!(expected_peer_ids.iter().all(|p| ok.peers.contains(p))); - let key = kbucket_priv::Key::new(ok.key); + let key = kbucket::Key::new(ok.key); assert_eq!(expected_distances, distances(&key, ok.peers)); return Poll::Ready(()); } @@ -653,7 +653,7 @@ fn put_record() { assert_eq!(r.expires, expected.expires); assert_eq!(r.publisher, Some(*swarms[0].local_peer_id())); - let key = kbucket_priv::Key::new(r.key.clone()); + let key = kbucket::Key::new(r.key.clone()); let mut expected = swarms .iter() .skip(1) @@ -661,9 +661,9 @@ fn put_record() { .cloned() .collect::>(); expected.sort_by(|id1, id2| { - kbucket_priv::Key::from(*id1) + kbucket::Key::from(*id1) .distance(&key) - .cmp(&kbucket_priv::Key::from(*id2).distance(&key)) + .cmp(&kbucket::Key::from(*id2).distance(&key)) }); let expected = expected @@ -992,11 +992,11 @@ fn add_provider() { .map(Swarm::local_peer_id) .cloned() .collect::>(); - let kbucket_key = kbucket_priv::Key::new(key); + let kbucket_key = kbucket::Key::new(key); expected.sort_by(|id1, id2| { - kbucket_priv::Key::from(*id1) + kbucket::Key::from(*id1) .distance(&kbucket_key) - .cmp(&kbucket_priv::Key::from(*id2).distance(&kbucket_key)) + .cmp(&kbucket::Key::from(*id2).distance(&kbucket_key)) }); let expected = expected diff --git a/protocols/kad/src/handler_priv.rs b/protocols/kad/src/handler.rs similarity index 100% rename from protocols/kad/src/handler_priv.rs rename to protocols/kad/src/handler.rs diff --git a/protocols/kad/src/kbucket_priv.rs b/protocols/kad/src/kbucket.rs similarity index 97% rename from protocols/kad/src/kbucket_priv.rs rename to protocols/kad/src/kbucket.rs index 5288be44786..08dd3ed4560 100644 --- a/protocols/kad/src/kbucket_priv.rs +++ b/protocols/kad/src/kbucket.rs @@ -84,7 +84,7 @@ const NUM_BUCKETS: usize = 256; /// A `KBucketsTable` represents a Kademlia routing table. #[derive(Debug, Clone)] -pub struct KBucketsTable { +pub(crate) struct KBucketsTable { /// The key identifying the local peer that owns the routing table. local_key: TKey, /// The buckets comprising the routing table. @@ -154,7 +154,7 @@ where /// The given `pending_timeout` specifies the duration after creation of /// a [`PendingEntry`] after which it becomes eligible for insertion into /// a full bucket, replacing the least-recently (dis)connected node. - pub fn new(local_key: TKey, pending_timeout: Duration) -> Self { + pub(crate) fn new(local_key: TKey, pending_timeout: Duration) -> Self { KBucketsTable { local_key, buckets: (0..NUM_BUCKETS) @@ -165,13 +165,13 @@ where } /// Returns the local key. - pub fn local_key(&self) -> &TKey { + pub(crate) fn local_key(&self) -> &TKey { &self.local_key } /// Returns an `Entry` for the given key, representing the state of the entry /// in the routing table. - pub fn entry<'a>(&'a mut self, key: &'a TKey) -> Entry<'a, TKey, TVal> { + pub(crate) fn entry<'a>(&'a mut self, key: &'a TKey) -> Entry<'a, TKey, TVal> { let index = BucketIndex::new(&self.local_key.as_ref().distance(key)); if let Some(i) = index { let bucket = &mut self.buckets[i.get()]; @@ -188,7 +188,7 @@ where /// /// The buckets are ordered by proximity to the `local_key`, i.e. the first /// bucket is the closest bucket (containing at most one key). - pub fn iter(&mut self) -> impl Iterator> + '_ { + pub(crate) fn iter(&mut self) -> impl Iterator> + '_ { let applied_pending = &mut self.applied_pending; self.buckets.iter_mut().enumerate().map(move |(i, b)| { if let Some(applied) = b.apply_pending() { @@ -204,7 +204,7 @@ where /// Returns the bucket for the distance to the given key. /// /// Returns `None` if the given key refers to the local key. - pub fn bucket(&mut self, key: &K) -> Option> + pub(crate) fn bucket(&mut self, key: &K) -> Option> where K: AsRef, { @@ -232,13 +232,16 @@ where /// buckets are updated accordingly. The fact that a pending entry was applied is /// recorded in the `KBucketsTable` in the form of `AppliedPending` results, which must be /// consumed by calling this function. - pub fn take_applied_pending(&mut self) -> Option> { + pub(crate) fn take_applied_pending(&mut self) -> Option> { self.applied_pending.pop_front() } /// Returns an iterator over the keys closest to `target`, ordered by /// increasing distance. - pub fn closest_keys<'a, T>(&'a mut self, target: &'a T) -> impl Iterator + 'a + pub(crate) fn closest_keys<'a, T>( + &'a mut self, + target: &'a T, + ) -> impl Iterator + 'a where T: AsRef, { @@ -256,7 +259,7 @@ where /// Returns an iterator over the nodes closest to the `target` key, ordered by /// increasing distance. - pub fn closest<'a, T>( + pub(crate) fn closest<'a, T>( &'a mut self, target: &'a T, ) -> impl Iterator> + 'a @@ -286,7 +289,7 @@ where /// /// The number of nodes between the local node and the target are /// calculated by backtracking from the target towards the local key. - pub fn count_nodes_between(&mut self, target: &T) -> usize + pub(crate) fn count_nodes_between(&mut self, target: &T) -> usize where T: AsRef, { @@ -460,7 +463,7 @@ where } } -/// A reference to a bucket in a [`KBucketsTable`]. +/// A reference to a bucket. pub struct KBucketRef<'a, TKey, TVal> { index: BucketIndex, bucket: &'a mut KBucket, @@ -471,7 +474,7 @@ where TKey: Clone + AsRef, TVal: Clone, { - /// Returns the minimum inclusive and maximum inclusive [`Distance`] for + /// Returns the minimum inclusive and maximum inclusive distance for /// this bucket. pub fn range(&self) -> (Distance, Distance) { self.index.range() diff --git a/protocols/kad/src/kbucket_priv/bucket.rs b/protocols/kad/src/kbucket/bucket.rs similarity index 99% rename from protocols/kad/src/kbucket_priv/bucket.rs rename to protocols/kad/src/kbucket/bucket.rs index 78bcd95261b..bd0c5903a4a 100644 --- a/protocols/kad/src/kbucket_priv/bucket.rs +++ b/protocols/kad/src/kbucket/bucket.rs @@ -26,8 +26,7 @@ //! > of the `KBucketsTable` and in particular the public `Entry` API. use super::*; -pub use crate::K_VALUE; - +pub(crate) use crate::K_VALUE; /// A `PendingNode` is a `Node` that is pending insertion into a `KBucket`. #[derive(Debug, Clone)] pub(crate) struct PendingNode { @@ -130,7 +129,7 @@ pub(crate) struct KBucket { /// The result of inserting an entry into a bucket. #[must_use] #[derive(Debug, Clone, PartialEq, Eq)] -pub enum InsertResult { +pub(crate) enum InsertResult { /// The entry has been successfully inserted. Inserted, /// The entry is pending insertion because the relevant bucket is currently full. @@ -152,12 +151,12 @@ pub enum InsertResult { /// The result of applying a pending node to a bucket, possibly /// replacing an existing node. #[derive(Debug, Clone, PartialEq, Eq)] -pub struct AppliedPending { +pub(crate) struct AppliedPending { /// The key of the inserted pending node. - pub inserted: Node, + pub(crate) inserted: Node, /// The node that has been evicted from the bucket to make room for the /// pending node, if any. - pub evicted: Option>, + pub(crate) evicted: Option>, } impl KBucket diff --git a/protocols/kad/src/kbucket_priv/entry.rs b/protocols/kad/src/kbucket/entry.rs similarity index 88% rename from protocols/kad/src/kbucket_priv/entry.rs rename to protocols/kad/src/kbucket/entry.rs index 7ccf2017e99..0794ace4202 100644 --- a/protocols/kad/src/kbucket_priv/entry.rs +++ b/protocols/kad/src/kbucket/entry.rs @@ -21,7 +21,7 @@ //! The `Entry` API for quering and modifying the entries of a `KBucketsTable` //! representing the nodes participating in the Kademlia DHT. -pub use super::bucket::{AppliedPending, InsertResult, Node, NodeStatus, K_VALUE}; +pub(crate) use super::bucket::{AppliedPending, InsertResult, Node, NodeStatus, K_VALUE}; pub use super::key::*; use super::*; @@ -74,7 +74,7 @@ impl, TVal> AsRef for EntryView { /// A reference into a single entry of a `KBucketsTable`. #[derive(Debug)] -pub enum Entry<'a, TPeerId, TVal> { +pub(crate) enum Entry<'a, TPeerId, TVal> { /// The entry is present in a bucket. Present(PresentEntry<'a, TPeerId, TVal>, NodeStatus), /// The entry is pending insertion in a bucket. @@ -115,7 +115,7 @@ where /// /// Returns `None` if the entry is neither present in a bucket nor /// pending insertion into a bucket. - pub fn view(&'a mut self) -> Option> { + pub(crate) fn view(&'a mut self) -> Option> { match self { Entry::Present(entry, status) => Some(EntryRefView { node: NodeRefView { @@ -140,7 +140,7 @@ where /// Returns `None` if the `Key` used to construct this `Entry` is not a valid /// key for an entry in a bucket, which is the case for the `local_key` of /// the `KBucketsTable` referring to the local node. - pub fn key(&self) -> Option<&TKey> { + pub(crate) fn key(&self) -> Option<&TKey> { match self { Entry::Present(entry, _) => Some(entry.key()), Entry::Pending(entry, _) => Some(entry.key()), @@ -153,7 +153,7 @@ where /// /// Returns `None` if the entry is absent from any bucket or refers to the /// local node. - pub fn value(&mut self) -> Option<&mut TVal> { + pub(crate) fn value(&mut self) -> Option<&mut TVal> { match self { Entry::Present(entry, _) => Some(entry.value()), Entry::Pending(entry, _) => Some(entry.value()), @@ -165,8 +165,7 @@ where /// An entry present in a bucket. #[derive(Debug)] -pub struct PresentEntry<'a, TKey, TVal>(EntryRef<'a, TKey, TVal>); - +pub(crate) struct PresentEntry<'a, TKey, TVal>(EntryRef<'a, TKey, TVal>); impl<'a, TKey, TVal> PresentEntry<'a, TKey, TVal> where TKey: Clone + AsRef, @@ -177,12 +176,12 @@ where } /// Returns the key of the entry. - pub fn key(&self) -> &TKey { + pub(crate) fn key(&self) -> &TKey { self.0.key } /// Returns the value associated with the key. - pub fn value(&mut self) -> &mut TVal { + pub(crate) fn value(&mut self) -> &mut TVal { &mut self .0 .bucket @@ -192,12 +191,12 @@ where } /// Sets the status of the entry to the provided [`NodeStatus`]. - pub fn update(&mut self, status: NodeStatus) { + pub(crate) fn update(&mut self, status: NodeStatus) { self.0.bucket.update(self.0.key, status); } /// Removes the entry from the bucket. - pub fn remove(self) -> EntryView { + pub(crate) fn remove(self) -> EntryView { let (node, status, _pos) = self .0 .bucket @@ -209,8 +208,7 @@ where /// An entry waiting for a slot to be available in a bucket. #[derive(Debug)] -pub struct PendingEntry<'a, TKey, TVal>(EntryRef<'a, TKey, TVal>); - +pub(crate) struct PendingEntry<'a, TKey, TVal>(EntryRef<'a, TKey, TVal>); impl<'a, TKey, TVal> PendingEntry<'a, TKey, TVal> where TKey: Clone + AsRef, @@ -221,12 +219,12 @@ where } /// Returns the key of the entry. - pub fn key(&self) -> &TKey { + pub(crate) fn key(&self) -> &TKey { self.0.key } /// Returns the value associated with the key. - pub fn value(&mut self) -> &mut TVal { + pub(crate) fn value(&mut self) -> &mut TVal { self.0 .bucket .pending_mut() @@ -235,13 +233,13 @@ where } /// Updates the status of the pending entry. - pub fn update(self, status: NodeStatus) -> PendingEntry<'a, TKey, TVal> { + pub(crate) fn update(self, status: NodeStatus) -> PendingEntry<'a, TKey, TVal> { self.0.bucket.update_pending(status); PendingEntry::new(self.0.bucket, self.0.key) } /// Removes the pending entry from the bucket. - pub fn remove(self) -> EntryView { + pub(crate) fn remove(self) -> EntryView { let pending = self.0.bucket.remove_pending().expect( "We can only build a PendingEntry if the entry is pending insertion into the bucket; QED", @@ -254,8 +252,7 @@ where /// An entry that is not present in any bucket. #[derive(Debug)] -pub struct AbsentEntry<'a, TKey, TVal>(EntryRef<'a, TKey, TVal>); - +pub(crate) struct AbsentEntry<'a, TKey, TVal>(EntryRef<'a, TKey, TVal>); impl<'a, TKey, TVal> AbsentEntry<'a, TKey, TVal> where TKey: Clone + AsRef, @@ -266,12 +263,12 @@ where } /// Returns the key of the entry. - pub fn key(&self) -> &TKey { + pub(crate) fn key(&self) -> &TKey { self.0.key } /// Attempts to insert the entry into a bucket. - pub fn insert(self, value: TVal, status: NodeStatus) -> InsertResult { + pub(crate) fn insert(self, value: TVal, status: NodeStatus) -> InsertResult { self.0.bucket.insert( Node { key: self.0.key.clone(), diff --git a/protocols/kad/src/kbucket_priv/key.rs b/protocols/kad/src/kbucket/key.rs similarity index 100% rename from protocols/kad/src/kbucket_priv/key.rs rename to protocols/kad/src/kbucket/key.rs diff --git a/protocols/kad/src/lib.rs b/protocols/kad/src/lib.rs index c3a705900d8..7e51f8b9908 100644 --- a/protocols/kad/src/lib.rs +++ b/protocols/kad/src/lib.rs @@ -38,30 +38,6 @@ #![allow(dead_code)] #![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))] -mod handler_priv; -#[deprecated( - note = "The `handler` module will be made private in the future and should not be depended on." -)] -pub mod handler { - pub use super::handler_priv::*; -} - -mod kbucket_priv; -#[deprecated( - note = "The `kbucket` module will be made private in the future and should not be depended on." -)] -pub mod kbucket { - pub use super::kbucket_priv::*; -} - -mod protocol_priv; -#[deprecated( - note = "The `protocol` module will be made private in the future and should not be depended on." -)] -pub mod protocol { - pub use super::protocol_priv::*; -} - mod record_priv; #[deprecated( note = "The `record` module will be made private in the future and should not be depended on." @@ -72,7 +48,10 @@ pub mod record { mod addresses; mod behaviour; +mod handler; mod jobs; +mod kbucket; +mod protocol; mod query; mod proto { @@ -97,11 +76,12 @@ pub use behaviour::{ Kademlia, KademliaBucketInserts, KademliaCaching, KademliaConfig, KademliaEvent, KademliaStoreInserts, ProgressStep, Quorum, }; -pub use kbucket_priv::{EntryView, KBucketRef, Key as KBucketKey}; +pub use kbucket::{EntryView, KBucketRef, Key as KBucketKey}; pub use protocol::KadConnectionType; pub use query::QueryId; pub use record_priv::{store, Key as RecordKey, ProviderRecord, Record}; +use libp2p_swarm::StreamProtocol; use std::num::NonZeroUsize; /// The `k` parameter of the Kademlia specification. @@ -130,6 +110,8 @@ pub const K_VALUE: NonZeroUsize = unsafe { NonZeroUsize::new_unchecked(20) }; /// The current value is `3`. pub const ALPHA_VALUE: NonZeroUsize = unsafe { NonZeroUsize::new_unchecked(3) }; +pub const PROTOCOL_NAME: StreamProtocol = protocol::DEFAULT_PROTO_NAME; + /// Constant shared across tests for the [`Multihash`](libp2p_core::multihash::Multihash) type. #[cfg(test)] const SHA_256_MH: u64 = 0x12; diff --git a/protocols/kad/src/protocol_priv.rs b/protocols/kad/src/protocol.rs similarity index 98% rename from protocols/kad/src/protocol_priv.rs rename to protocols/kad/src/protocol.rs index 1801ce935fd..d960e6508d1 100644 --- a/protocols/kad/src/protocol_priv.rs +++ b/protocols/kad/src/protocol.rs @@ -43,11 +43,9 @@ use std::{io, iter}; use unsigned_varint::codec; /// The protocol name used for negotiating with multistream-select. -pub const DEFAULT_PROTO_NAME: StreamProtocol = StreamProtocol::new("/ipfs/kad/1.0.0"); - +pub(crate) const DEFAULT_PROTO_NAME: StreamProtocol = StreamProtocol::new("/ipfs/kad/1.0.0"); /// The default maximum size for a varint length-delimited packet. -pub const DEFAULT_MAX_PACKET_SIZE: usize = 16 * 1024; - +pub(crate) const DEFAULT_MAX_PACKET_SIZE: usize = 16 * 1024; /// Status of our connection to a node reported by the Kademlia protocol. #[derive(Copy, Clone, PartialEq, Eq, Debug, Hash)] pub enum KadConnectionType { @@ -258,12 +256,10 @@ where } /// Sink of responses and stream of requests. -pub type KadInStreamSink = KadStreamSink; - +pub(crate) type KadInStreamSink = KadStreamSink; /// Sink of requests and stream of responses. -pub type KadOutStreamSink = KadStreamSink; - -pub type KadStreamSink = stream::AndThen< +pub(crate) type KadOutStreamSink = KadStreamSink; +pub(crate) type KadStreamSink = stream::AndThen< sink::With< stream::ErrInto>>>, io::Error>, io::Cursor>, diff --git a/protocols/kad/src/query.rs b/protocols/kad/src/query.rs index d06b4920404..6b0a42a0b74 100644 --- a/protocols/kad/src/query.rs +++ b/protocols/kad/src/query.rs @@ -26,7 +26,7 @@ use peers::closest::{ use peers::fixed::FixedPeersIter; use peers::PeersIterState; -use crate::kbucket_priv::{Key, KeyBytes}; +use crate::kbucket::{Key, KeyBytes}; use crate::{ALPHA_VALUE, K_VALUE}; use either::Either; use fnv::FnvHashMap; diff --git a/protocols/kad/src/query/peers/closest.rs b/protocols/kad/src/query/peers/closest.rs index 66ea9d9ce52..a9011803e73 100644 --- a/protocols/kad/src/query/peers/closest.rs +++ b/protocols/kad/src/query/peers/closest.rs @@ -20,7 +20,7 @@ use super::*; -use crate::kbucket_priv::{Distance, Key, KeyBytes}; +use crate::kbucket::{Distance, Key, KeyBytes}; use crate::{ALPHA_VALUE, K_VALUE}; use instant::Instant; use libp2p_identity::PeerId; diff --git a/protocols/kad/src/query/peers/closest/disjoint.rs b/protocols/kad/src/query/peers/closest/disjoint.rs index 2ea484ed43c..3906b65b0af 100644 --- a/protocols/kad/src/query/peers/closest/disjoint.rs +++ b/protocols/kad/src/query/peers/closest/disjoint.rs @@ -19,7 +19,7 @@ // DEALINGS IN THE SOFTWARE. use super::*; -use crate::kbucket_priv::{Key, KeyBytes}; +use crate::kbucket::{Key, KeyBytes}; use instant::Instant; use libp2p_identity::PeerId; use std::{ diff --git a/protocols/kad/src/record_priv/store/memory.rs b/protocols/kad/src/record_priv/store/memory.rs index 40ad4405873..1d4caab3bd7 100644 --- a/protocols/kad/src/record_priv/store/memory.rs +++ b/protocols/kad/src/record_priv/store/memory.rs @@ -20,7 +20,7 @@ use super::*; -use crate::kbucket_priv; +use crate::kbucket; use libp2p_identity::PeerId; use smallvec::SmallVec; use std::borrow::Cow; @@ -30,7 +30,7 @@ use std::iter; /// In-memory implementation of a `RecordStore`. pub struct MemoryStore { /// The identity of the peer owning the store. - local_key: kbucket_priv::Key, + local_key: kbucket::Key, /// The configuration of the store. config: MemoryStoreConfig, /// The stored (regular) records. @@ -79,7 +79,7 @@ impl MemoryStore { /// Creates a new `MemoryRecordStore` with the given configuration. pub fn with_config(local_id: PeerId, config: MemoryStoreConfig) -> Self { MemoryStore { - local_key: kbucket_priv::Key::from(local_id), + local_key: kbucket::Key::from(local_id), config, records: HashMap::default(), provided: HashSet::default(), @@ -160,10 +160,10 @@ impl RecordStore for MemoryStore { } else { // It is a new provider record for that key. let local_key = self.local_key.clone(); - let key = kbucket_priv::Key::new(record.key.clone()); - let provider = kbucket_priv::Key::from(record.provider); + let key = kbucket::Key::new(record.key.clone()); + let provider = kbucket::Key::from(record.provider); if let Some(i) = providers.iter().position(|p| { - let pk = kbucket_priv::Key::from(p.provider); + let pk = kbucket::Key::from(p.provider); provider.distance(&key) < pk.distance(&key) }) { // Insert the new provider. @@ -225,8 +225,8 @@ mod tests { Multihash::wrap(SHA_256_MH, &rand::thread_rng().gen::<[u8; 32]>()).unwrap() } - fn distance(r: &ProviderRecord) -> kbucket_priv::Distance { - kbucket_priv::Key::new(r.key.clone()).distance(&kbucket_priv::Key::from(r.provider)) + fn distance(r: &ProviderRecord) -> kbucket::Distance { + kbucket::Key::new(r.key.clone()).distance(&kbucket::Key::from(r.provider)) } #[test] @@ -255,7 +255,7 @@ mod tests { #[test] fn providers_ordered_by_distance_to_key() { - fn prop(providers: Vec>) -> bool { + fn prop(providers: Vec>) -> bool { let mut store = MemoryStore::new(PeerId::random()); let key = Key::from(random_multihash());