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.
This commit is contained in:
Thomas Eizinger 2023-05-09 04:50:00 +02:00 committed by GitHub
parent 43c6d21c06
commit 3165b12c75
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
15 changed files with 144 additions and 165 deletions

View File

@ -148,7 +148,7 @@ impl super::Recorder<libp2p_identify::Event> 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")]

View File

@ -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

View File

@ -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<TStore> {
/// The Kademlia routing table.
kbuckets: KBucketsTable<kbucket_priv::Key<PeerId>, Addresses>,
kbuckets: KBucketsTable<kbucket::Key<PeerId>, 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<kbucket_priv::EntryView<kbucket_priv::Key<PeerId>, Addresses>> {
let key = kbucket_priv::Key::from(*peer);
) -> Option<kbucket::EntryView<kbucket::Key<PeerId>, 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<kbucket_priv::EntryView<kbucket_priv::Key<PeerId>, Addresses>> {
let key = kbucket_priv::Key::from(*peer);
) -> Option<kbucket::EntryView<kbucket::Key<PeerId>, 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<Item = kbucket_priv::KBucketRef<'_, kbucket_priv::Key<PeerId>, Addresses>>
{
) -> impl Iterator<Item = kbucket::KBucketRef<'_, kbucket::Key<PeerId>, Addresses>> {
self.kbuckets.iter().filter(|b| !b.is_empty())
}
@ -647,9 +646,9 @@ where
pub fn kbucket<K>(
&mut self,
key: K,
) -> Option<kbucket_priv::KBucketRef<'_, kbucket_priv::Key<PeerId>, Addresses>>
) -> Option<kbucket::KBucketRef<'_, kbucket::Key<PeerId>, Addresses>>
where
K: Into<kbucket_priv::Key<K>> + Clone,
K: Into<kbucket::Key<K>> + Clone,
{
self.kbuckets.bucket(&key.into())
}
@ -660,16 +659,15 @@ where
/// [`KademliaEvent::OutboundQueryCompleted{QueryResult::GetClosestPeers}`].
pub fn get_closest_peers<K>(&mut self, key: K) -> QueryId
where
K: Into<kbucket_priv::Key<K>> + Into<Vec<u8>> + Clone,
K: Into<kbucket::Key<K>> + Into<Vec<u8>> + Clone,
{
let target: kbucket_priv::Key<K> = key.clone().into();
let target: kbucket::Key<K> = key.clone().into();
let key: Vec<u8> = key.into();
let info = QueryInfo::GetClosestPeers {
key,
step: ProgressStep::first(),
};
let peer_keys: Vec<kbucket_priv::Key<PeerId>> =
self.kbuckets.closest_keys(&target).collect();
let peer_keys: Vec<kbucket::Key<PeerId>> = 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<K>,
) -> impl Iterator<Item = kbucket_priv::Key<PeerId>> + 'a {
key: &'a kbucket::Key<K>,
) -> impl Iterator<Item = kbucket::Key<PeerId>> + '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<T: Clone>(
&mut self,
target: &kbucket_priv::Key<T>,
target: &kbucket::Key<T>,
source: &PeerId,
) -> Vec<KadPeer> {
if target == self.kbuckets.local_key() {
@ -1067,7 +1065,7 @@ where
.collect::<Vec<_>>(),
)
} 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<Multiaddr>,
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::<Vec<_>>();
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<kbucket_priv::Distance, PeerId>,
cache_candidates: BTreeMap<kbucket::Distance, PeerId>,
},
}
@ -2884,8 +2882,8 @@ impl AddProviderError {
}
}
impl From<kbucket_priv::EntryView<kbucket_priv::Key<PeerId>, Addresses>> for KadPeer {
fn from(e: kbucket_priv::EntryView<kbucket_priv::Key<PeerId>, Addresses>) -> KadPeer {
impl From<kbucket::EntryView<kbucket::Key<PeerId>, Addresses>> for KadPeer {
fn from(e: kbucket::EntryView<kbucket::Key<PeerId>, 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<vec::IntoIter<kbucket_priv::Key<PeerId>>>,
remaining: Option<vec::IntoIter<kbucket::Key<PeerId>>>,
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<kbucket_priv::Distance, PeerId>,
cache_candidates: BTreeMap<kbucket::Distance, PeerId>,
},
}

View File

@ -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<K>(key: &kbucket_priv::Key<K>, peers: Vec<PeerId>) -> Vec<Distance> {
fn distances<K>(key: &kbucket::Key<K>, peers: Vec<PeerId>) -> Vec<Distance> {
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::<Vec<_>>();
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::<Vec<_>>();
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

View File

@ -84,7 +84,7 @@ const NUM_BUCKETS: usize = 256;
/// A `KBucketsTable` represents a Kademlia routing table.
#[derive(Debug, Clone)]
pub struct KBucketsTable<TKey, TVal> {
pub(crate) struct KBucketsTable<TKey, TVal> {
/// 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<Item = KBucketRef<'_, TKey, TVal>> + '_ {
pub(crate) fn iter(&mut self) -> impl Iterator<Item = KBucketRef<'_, TKey, TVal>> + '_ {
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<K>(&mut self, key: &K) -> Option<KBucketRef<'_, TKey, TVal>>
pub(crate) fn bucket<K>(&mut self, key: &K) -> Option<KBucketRef<'_, TKey, TVal>>
where
K: AsRef<KeyBytes>,
{
@ -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<AppliedPending<TKey, TVal>> {
pub(crate) fn take_applied_pending(&mut self) -> Option<AppliedPending<TKey, TVal>> {
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<Item = TKey> + 'a
pub(crate) fn closest_keys<'a, T>(
&'a mut self,
target: &'a T,
) -> impl Iterator<Item = TKey> + 'a
where
T: AsRef<KeyBytes>,
{
@ -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<Item = EntryView<TKey, TVal>> + '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<T>(&mut self, target: &T) -> usize
pub(crate) fn count_nodes_between<T>(&mut self, target: &T) -> usize
where
T: AsRef<KeyBytes>,
{
@ -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<TKey, TVal>,
@ -471,7 +474,7 @@ where
TKey: Clone + AsRef<KeyBytes>,
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()

View File

@ -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<TKey, TVal> {
@ -130,7 +129,7 @@ pub(crate) struct KBucket<TKey, TVal> {
/// The result of inserting an entry into a bucket.
#[must_use]
#[derive(Debug, Clone, PartialEq, Eq)]
pub enum InsertResult<TKey> {
pub(crate) enum InsertResult<TKey> {
/// 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<TKey> {
/// The result of applying a pending node to a bucket, possibly
/// replacing an existing node.
#[derive(Debug, Clone, PartialEq, Eq)]
pub struct AppliedPending<TKey, TVal> {
pub(crate) struct AppliedPending<TKey, TVal> {
/// The key of the inserted pending node.
pub inserted: Node<TKey, TVal>,
pub(crate) inserted: Node<TKey, TVal>,
/// The node that has been evicted from the bucket to make room for the
/// pending node, if any.
pub evicted: Option<Node<TKey, TVal>>,
pub(crate) evicted: Option<Node<TKey, TVal>>,
}
impl<TKey, TVal> KBucket<TKey, TVal>

View File

@ -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<TKey: AsRef<KeyBytes>, TVal> AsRef<KeyBytes> for EntryView<TKey, TVal> {
/// 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<EntryRefView<'a, TKey, TVal>> {
pub(crate) fn view(&'a mut self) -> Option<EntryRefView<'a, TKey, TVal>> {
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<KeyBytes>,
@ -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<TKey, TVal> {
pub(crate) fn remove(self) -> EntryView<TKey, TVal> {
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<KeyBytes>,
@ -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<TKey, TVal> {
pub(crate) fn remove(self) -> EntryView<TKey, TVal> {
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<KeyBytes>,
@ -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<TKey> {
pub(crate) fn insert(self, value: TVal, status: NodeStatus) -> InsertResult<TKey> {
self.0.bucket.insert(
Node {
key: self.0.key.clone(),

View File

@ -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;

View File

@ -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<S> = KadStreamSink<S, KadResponseMsg, KadRequestMsg>;
pub(crate) type KadInStreamSink<S> = KadStreamSink<S, KadResponseMsg, KadRequestMsg>;
/// Sink of requests and stream of responses.
pub type KadOutStreamSink<S> = KadStreamSink<S, KadRequestMsg, KadResponseMsg>;
pub type KadStreamSink<S, A, B> = stream::AndThen<
pub(crate) type KadOutStreamSink<S> = KadStreamSink<S, KadRequestMsg, KadResponseMsg>;
pub(crate) type KadStreamSink<S, A, B> = stream::AndThen<
sink::With<
stream::ErrInto<Framed<S, UviBytes<io::Cursor<Vec<u8>>>>, io::Error>,
io::Cursor<Vec<u8>>,

View File

@ -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;

View File

@ -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;

View File

@ -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::{

View File

@ -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<PeerId>,
local_key: kbucket::Key<PeerId>,
/// 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<kbucket_priv::Key<PeerId>>) -> bool {
fn prop(providers: Vec<kbucket::Key<PeerId>>) -> bool {
let mut store = MemoryStore::new(PeerId::random());
let key = Key::from(random_multihash());