mirror of
https://github.com/fluencelabs/rust-libp2p
synced 2025-05-12 02:47:15 +00:00
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:
parent
43c6d21c06
commit
3165b12c75
@ -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")]
|
||||
|
@ -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
|
||||
|
||||
|
@ -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>,
|
||||
},
|
||||
}
|
||||
|
||||
|
@ -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
|
||||
|
@ -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()
|
@ -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>
|
@ -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(),
|
@ -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;
|
||||
|
@ -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>>,
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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::{
|
||||
|
@ -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());
|
||||
|
||||
|
Loading…
x
Reference in New Issue
Block a user