Kademlia: Somewhat complete the records implementation. (#1189)

* Somewhat complete the implementation of Kademlia records.

This commit relates to [libp2p-146] and [libp2p-1089].

  * All records expire (by default, configurable).
  * Provider records are also stored in the RecordStore, and the RecordStore
    API extended.
  * Background jobs for periodic (re-)replication and (re-)publication
    of records. Regular (value-)records are subject to re-replication and
    re-publication as per standard Kademlia. Provider records are only
    subject to re-publication.
  * For standard Kademlia value lookups (quorum = 1), the record is cached
    at the closest peer to the key that did not return the value, as per
    standard Kademlia.
  * Expiration times of regular (value-)records is computed exponentially
    inversely proportional to the number of nodes between the local node
    and the closest node known to the key (beyond the k closest), as per
    standard Kademlia.

The protobuf messages are extended with two fields: `ttl` and `publisher`
in order to implement the different semantics of re-replication (by any
of the k closest peers to the key, not affecting expiry) and re-publication
(by the original publisher, resetting the expiry). This is not done yet in
other libp2p Kademlia implementations, see e.g. [libp2p-go-323]. The new protobuf fields
have been given somewhat unique identifiers to prevent future collision.

Similarly, periodic re-publication of provider records does not seem to
be done yet in other implementations, see e.g. [libp2p-js-98].

[libp2p-146]: https://github.com/libp2p/rust-libp2p/issues/146
[libp2p-1089]: https://github.com/libp2p/rust-libp2p/issues/1089
[libp2p-go-323]: https://github.com/libp2p/go-libp2p-kad-dht/issues/323
[libp2p-js-98]: https://github.com/libp2p/js-libp2p-kad-dht/issues/98

* Tweak kad-ipfs example.

* Add missing files.

* Ensure new delays are polled immediately.

To ensure task notification, since `NotReady` is returned right after.

* Fix ipfs-kad example and use wasm_timer.

* Small cleanup.

* Incorporate some feedback.

* Adjustments after rebase.

* Distinguish events further.

In order for a user to easily distinguish the result of e.g.
a `put_record` operation from the result of a later republication,
different event constructors are used. Furthermore, for now,
re-replication and "caching" of records (at the closest peer to
the key that did not return a value during a successful lookup)
do not yield events for now as they are less interesting.

* Speed up tests for CI.

* Small refinements and more documentation.

  * Guard a node against overriding records for which it considers
    itself to be the publisher.

  * Document the jobs module more extensively.

* More inline docs around removal of "unreachable" addresses.

* Remove wildcard re-exports.

* Use NonZeroUsize for the constants.

* Re-add method lost on merge.

* Add missing 'pub'.

* Further increase the timeout in the ipfs-kad example.

* Readd log dependency to libp2p-kad.

* Simplify RecordStore API slightly.

* Some more commentary.

* Change Addresses::remove to return Result<(),()>.

Change the semantics of `Addresses::remove` so that the error case
is unambiguous, instead of the success case. Use the `Result` for
clearer semantics to that effect.

* Add some documentation to .
This commit is contained in:
Roman Borschel 2019-07-17 14:40:48 +02:00 committed by GitHub
parent 01bce16d09
commit cde93f5432
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
21 changed files with 2715 additions and 947 deletions

View File

@ -31,6 +31,7 @@ use libp2p::{
build_development_transport
};
use libp2p::kad::{Kademlia, KademliaConfig, KademliaEvent, GetClosestPeersError};
use libp2p::kad::record::store::MemoryStore;
use std::env;
use std::time::Duration;
@ -51,9 +52,10 @@ fn main() {
// to insert our local node in the DHT. However here we use `without_init` because this
// example is very ephemeral and we don't want to pollute the DHT. In a real world
// application, you want to use `new` instead.
let mut cfg = KademliaConfig::new(local_peer_id.clone());
let mut cfg = KademliaConfig::default();
cfg.set_query_timeout(Duration::from_secs(5 * 60));
let mut behaviour: Kademlia<_> = Kademlia::new(cfg);
let store = MemoryStore::new(local_peer_id.clone());
let mut behaviour = Kademlia::with_config(local_peer_id.clone(), store, cfg);
// TODO: the /dnsaddr/ scheme is not supported (https://github.com/libp2p/rust-libp2p/issues/967)
/*behaviour.add_address(&"QmNnooDu7bfjPFoTZYxMNLWUQJyrVwtbZg5gBMjTezGAJN".parse().unwrap(), "/dnsaddr/bootstrap.libp2p.io".parse().unwrap());

View File

@ -83,7 +83,7 @@ fn three_fields() {
struct Foo<TSubstream> {
ping: libp2p::ping::Ping<TSubstream>,
identify: libp2p::identify::Identify<TSubstream>,
kad: libp2p::kad::Kademlia<TSubstream>,
kad: libp2p::kad::Kademlia<TSubstream, libp2p::kad::record::store::MemoryStore>,
#[behaviour(ignore)]
foo: String,
}

View File

@ -16,6 +16,7 @@ bytes = "0.4"
either = "1.5"
fnv = "1.0"
futures = "0.1"
log = "0.4"
libp2p-core = { version = "0.10.0", path = "../../core" }
libp2p-swarm = { version = "0.1.0", path = "../../swarm" }
multiaddr = { package = "parity-multiaddr", version = "0.5.0", path = "../../misc/multiaddr" }

View File

@ -18,6 +18,14 @@ message Record {
// Time the record was received, set by receiver
string timeReceived = 5;
// The original publisher of the record.
// Currently specific to rust-libp2p.
bytes publisher = 666;
// The remaining TTL of the record, in seconds.
// Currently specific to rust-libp2p.
uint32 ttl = 777;
};
message Message {

View File

@ -22,7 +22,7 @@ use libp2p_core::Multiaddr;
use smallvec::SmallVec;
use std::fmt;
/// List of addresses of a peer.
/// A non-empty list of (unique) addresses of a peer in the routing table.
#[derive(Clone)]
pub struct Addresses {
addrs: SmallVec<[Multiaddr; 6]>,
@ -30,59 +30,69 @@ pub struct Addresses {
impl Addresses {
/// Creates a new list of addresses.
pub fn new() -> Addresses {
Addresses {
addrs: SmallVec::new(),
}
pub fn new(addr: Multiaddr) -> Addresses {
let mut addrs = SmallVec::new();
addrs.push(addr);
Addresses { addrs }
}
/// Returns an iterator over the list of addresses.
/// Gets a reference to the first address in the list.
pub fn first(&self) -> &Multiaddr {
&self.addrs[0]
}
/// Returns an iterator over the addresses.
pub fn iter(&self) -> impl Iterator<Item = &Multiaddr> {
self.addrs.iter()
}
/// Returns the number of addresses in the list.
pub fn len(&self) -> usize {
self.addrs.len()
}
/// Converts the addresses into a `Vec`.
pub fn into_vec(self) -> Vec<Multiaddr> {
self.addrs.into_vec()
}
/// Returns true if the list of addresses is empty.
pub fn is_empty(&self) -> bool {
self.addrs.is_empty()
/// Removes the given address from the list.
///
/// Returns `Ok(())` if the address is either not in the list or was found and
/// removed. Returns `Err(())` if the address is the last remaining address,
/// which cannot be removed.
///
/// An address should only be removed if is determined to be invalid or
/// otherwise unreachable.
pub fn remove(&mut self, addr: &Multiaddr) -> Result<(),()> {
if self.addrs.len() == 1 {
return Err(())
}
/// Removes the given address from the list. Typically called if an address is determined to
/// be invalid or unreachable.
pub fn remove(&mut self, addr: &Multiaddr) {
if let Some(pos) = self.addrs.iter().position(|a| a == addr) {
self.addrs.remove(pos);
}
if self.addrs.len() <= self.addrs.inline_size() {
self.addrs.shrink_to_fit();
}
}
/// Clears the list. It is empty afterwards.
pub fn clear(&mut self) {
self.addrs.clear();
self.addrs.shrink_to_fit();
Ok(())
}
/// Inserts an address in the list. No effect if the address was already in the list.
pub fn insert(&mut self, addr: Multiaddr) {
/// Adds a new address to the end of the list.
///
/// Returns true if the address was added, false otherwise (i.e. if the
/// address is already in the list).
pub fn insert(&mut self, addr: Multiaddr) -> bool {
if self.addrs.iter().all(|a| *a != addr) {
self.addrs.push(addr);
true
} else {
false
}
}
}
impl Default for Addresses {
fn default() -> Self {
Addresses::new()
}
}
impl fmt::Debug for Addresses {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
f.debug_list()

File diff suppressed because it is too large Load Diff

View File

@ -22,7 +22,9 @@
use super::*;
use crate::K_VALUE;
use crate::kbucket::Distance;
use crate::record::store::MemoryStore;
use futures::future;
use libp2p_core::{
PeerId,
@ -37,18 +39,24 @@ use libp2p_core::{
use libp2p_secio::SecioConfig;
use libp2p_swarm::Swarm;
use libp2p_yamux as yamux;
use quickcheck::*;
use rand::{Rng, random, thread_rng};
use std::{collections::HashSet, iter::FromIterator, io, num::NonZeroU8, u64};
use std::{collections::{HashSet, HashMap}, io, num::NonZeroUsize, u64};
use tokio::runtime::current_thread;
use multihash::Hash::SHA2256;
type TestSwarm = Swarm<
Boxed<(PeerId, StreamMuxerBox), io::Error>,
Kademlia<Substream<StreamMuxerBox>>
Kademlia<Substream<StreamMuxerBox>, MemoryStore>
>;
/// Builds swarms, each listening on a port. Does *not* connect the nodes together.
fn build_nodes(num: usize) -> (u64, Vec<TestSwarm>) {
build_nodes_with_config(num, Default::default())
}
/// Builds swarms, each listening on a port. Does *not* connect the nodes together.
fn build_nodes_with_config(num: usize, cfg: KademliaConfig) -> (u64, Vec<TestSwarm>) {
let port_base = 1 + random::<u64>() % (u64::MAX - num as u64);
let mut result: Vec<Swarm<_, _>> = Vec::with_capacity(num);
@ -68,22 +76,27 @@ fn build_nodes(num: usize) -> (u64, Vec<TestSwarm>) {
.map_err(|e| panic!("Failed to create transport: {:?}", e))
.boxed();
let cfg = KademliaConfig::new(local_public_key.clone().into_peer_id());
let kad = Kademlia::new(cfg);
result.push(Swarm::new(transport, kad, local_public_key.into_peer_id()));
let local_id = local_public_key.clone().into_peer_id();
let store = MemoryStore::new(local_id.clone());
let behaviour = Kademlia::with_config(local_id.clone(), store, cfg.clone());
result.push(Swarm::new(transport, behaviour, local_id));
}
let mut i = 0;
for s in result.iter_mut() {
Swarm::listen_on(s, Protocol::Memory(port_base + i).into()).unwrap();
i += 1
for (i, s) in result.iter_mut().enumerate() {
Swarm::listen_on(s, Protocol::Memory(port_base + i as u64).into()).unwrap();
}
(port_base, result)
}
fn build_connected_nodes(total: usize, step: usize) -> (Vec<PeerId>, Vec<TestSwarm>) {
let (port_base, mut swarms) = build_nodes(total);
build_connected_nodes_with_config(total, step, Default::default())
}
fn build_connected_nodes_with_config(total: usize, step: usize, cfg: KademliaConfig)
-> (Vec<PeerId>, Vec<TestSwarm>)
{
let (port_base, mut swarms) = build_nodes_with_config(total, cfg);
let swarm_ids: Vec<_> = swarms.iter().map(Swarm::local_peer_id).cloned().collect();
let mut i = 0;
@ -101,7 +114,7 @@ fn build_connected_nodes(total: usize, step: usize) -> (Vec<PeerId>, Vec<TestSwa
#[test]
fn bootstrap() {
fn run<G: rand::Rng>(rng: &mut G) {
fn run(rng: &mut impl Rng) {
let num_total = rng.gen_range(2, 20);
let num_group = rng.gen_range(1, num_total);
let (swarm_ids, mut swarms) = build_connected_nodes(num_total, num_group);
@ -150,7 +163,7 @@ fn query_iter() {
.collect()
}
fn run<G: Rng>(rng: &mut G) {
fn run(rng: &mut impl Rng) {
let num_total = rng.gen_range(2, 20);
let (swarm_ids, mut swarms) = build_connected_nodes(num_total, 1);
@ -242,10 +255,7 @@ fn unresponsive_not_returned_indirect() {
// Add fake addresses to first.
let first_peer_id = Swarm::local_peer_id(&swarms[0]).clone();
for _ in 0 .. 10 {
swarms[0].add_address(
&PeerId::random(),
multiaddr![Udp(10u16)]
);
swarms[0].add_address(&PeerId::random(), multiaddr![Udp(10u16)]);
}
// Connect second to first.
@ -315,35 +325,52 @@ fn get_record_not_found() {
}
#[test]
fn put_value() {
fn run<G: rand::Rng>(rng: &mut G) {
let num_total = rng.gen_range(21, 40);
let num_group = rng.gen_range(1, usize::min(num_total, kbucket::K_VALUE));
let (swarm_ids, mut swarms) = build_connected_nodes(num_total, num_group);
fn put_record() {
fn prop(replication_factor: usize, records: Vec<Record>) {
let replication_factor = NonZeroUsize::new(replication_factor % (K_VALUE.get() / 2) + 1).unwrap();
let num_total = replication_factor.get() * 2;
let num_group = replication_factor.get();
let key = multihash::encode(SHA2256, &vec![1,2,3]).unwrap();
let bucket_key = kbucket::Key::from(key.clone());
let mut config = KademliaConfig::default();
config.set_replication_factor(replication_factor);
let (swarm_ids, mut swarms) = build_connected_nodes_with_config(num_total, num_group, config);
let mut sorted_peer_ids: Vec<_> = swarm_ids
.iter()
.map(|id| (id.clone(), kbucket::Key::from(id.clone()).distance(&bucket_key)))
.collect();
let records = records.into_iter()
.take(num_total)
.map(|mut r| {
// We don't want records to expire prematurely, as they would
// be removed from storage and no longer replicated, but we still
// want to check that an explicitly set expiration is preserved.
r.expires = r.expires.map(|t| t + Duration::from_secs(60));
(r.key.clone(), r)
})
.collect::<HashMap<_,_>>();
sorted_peer_ids.sort_by(|(_, d1), (_, d2)| d1.cmp(d2));
for r in records.values() {
swarms[0].put_record(r.clone(), Quorum::All);
}
let closest = HashSet::from_iter(sorted_peer_ids.into_iter().map(|(id, _)| id));
let record = Record { key: key.clone(), value: vec![4,5,6] };
swarms[0].put_record(record, Quorum::All);
// Each test run republishes all records once.
let mut republished = false;
// The accumulated results for one round of publishing.
let mut results = Vec::new();
current_thread::run(
future::poll_fn(move || {
let mut check_results = false;
future::poll_fn(move || loop {
// Poll all swarms until they are "NotReady".
for swarm in &mut swarms {
loop {
match swarm.poll().unwrap() {
Async::Ready(Some(KademliaEvent::PutRecordResult(Ok(_)))) => {
check_results = true;
Async::Ready(Some(KademliaEvent::PutRecordResult(res))) |
Async::Ready(Some(KademliaEvent::RepublishRecordResult(res))) => {
match res {
Err(e) => panic!(e),
Ok(ok) => {
assert!(records.contains_key(&ok.key));
let record = swarm.store.get(&ok.key).unwrap();
results.push(record.into_owned());
}
}
}
Async::Ready(_) => (),
Async::NotReady => break,
@ -351,31 +378,64 @@ fn put_value() {
}
}
if check_results {
let mut have: HashSet<_> = Default::default();
for (i, swarm) in swarms.iter().skip(1).enumerate() {
if swarm.records.get(&key).is_some() {
have.insert(swarm_ids[i].clone());
}
// All swarms are NotReady and not enough results have been collected
// so far, thus wait to be polled again for further progress.
if results.len() != records.len() {
return Ok(Async::NotReady)
}
let intersection: HashSet<_> = have.intersection(&closest).collect();
// Consume the results, checking that each record was replicated
// correctly to the closest peers to the key.
while let Some(r) = results.pop() {
let expected = records.get(&r.key).unwrap();
assert_eq!(have.len(), kbucket::K_VALUE);
assert_eq!(intersection.len(), kbucket::K_VALUE);
assert_eq!(r.key, expected.key);
assert_eq!(r.value, expected.value);
assert_eq!(r.expires, expected.expires);
assert_eq!(r.publisher.as_ref(), Some(&swarm_ids[0]));
let key = kbucket::Key::new(r.key.clone());
let mut expected = swarm_ids.clone().split_off(1);
expected.sort_by(|id1, id2|
kbucket::Key::new(id1).distance(&key).cmp(
&kbucket::Key::new(id2).distance(&key)));
let expected = expected
.into_iter()
.take(replication_factor.get())
.collect::<HashSet<_>>();
let actual = swarms.iter().enumerate().skip(1)
.filter_map(|(i, s)|
if s.store.get(key.preimage()).is_some() {
Some(swarm_ids[i].clone())
} else {
None
})
.collect::<HashSet<_>>();
assert_eq!(actual.len(), replication_factor.get());
assert_eq!(actual, expected);
}
if republished {
assert_eq!(swarms[0].store.records().count(), records.len());
for k in records.keys() {
swarms[0].store.remove(&k);
}
assert_eq!(swarms[0].store.records().count(), 0);
// All records have been republished, thus the test is complete.
return Ok(Async::Ready(()));
}
Ok(Async::NotReady)
}))
// Tell the replication job to republish asap.
swarms[0].put_record_job.as_mut().unwrap().asap(true);
republished = true;
})
)
}
let mut rng = thread_rng();
for _ in 0 .. 10 {
run(&mut rng);
}
QuickCheck::new().tests(3).quickcheck(prop as fn(_,_))
}
#[test]
@ -387,12 +447,9 @@ fn get_value() {
swarms[0].add_address(&swarm_ids[1], Protocol::Memory(port_base + 1).into());
swarms[1].add_address(&swarm_ids[2], Protocol::Memory(port_base + 2).into());
let record = Record {
key: multihash::encode(SHA2256, &vec![1,2,3]).unwrap(),
value: vec![4,5,6]
};
let record = Record::new(multihash::encode(SHA2256, &vec![1,2,3]).unwrap(), vec![4,5,6]);
swarms[1].records.put(record.clone()).unwrap();
swarms[1].store.put(record.clone()).unwrap();
swarms[0].get_record(&record.key, Quorum::One);
current_thread::run(
@ -416,23 +473,19 @@ fn get_value() {
}
#[test]
fn get_value_multiple() {
// Check that if we have responses from multiple peers, a correct number of
// results is returned.
fn get_value_many() {
// TODO: Randomise
let num_nodes = 12;
let (_swarm_ids, mut swarms) = build_connected_nodes(num_nodes, num_nodes);
let (_, mut swarms) = build_connected_nodes(num_nodes, num_nodes);
let num_results = 10;
let record = Record {
key: multihash::encode(SHA2256, &vec![1,2,3]).unwrap(),
value: vec![4,5,6],
};
let record = Record::new(multihash::encode(SHA2256, &vec![1,2,3]).unwrap(), vec![4,5,6]);
for i in 0 .. num_nodes {
swarms[i].records.put(record.clone()).unwrap();
swarms[i].store.put(record.clone()).unwrap();
}
let quorum = Quorum::N(NonZeroU8::new(num_results as u8).unwrap());
let quorum = Quorum::N(NonZeroUsize::new(num_results).unwrap());
swarms[0].get_record(&record.key, quorum);
current_thread::run(
@ -453,3 +506,123 @@ fn get_value_multiple() {
Ok(Async::NotReady)
}))
}
#[test]
fn add_provider() {
fn prop(replication_factor: usize, keys: Vec<kbucket::Key<Multihash>>) {
let replication_factor = NonZeroUsize::new(replication_factor % (K_VALUE.get() / 2) + 1).unwrap();
let num_total = replication_factor.get() * 2;
let num_group = replication_factor.get();
let mut config = KademliaConfig::default();
config.set_replication_factor(replication_factor);
let (swarm_ids, mut swarms) = build_connected_nodes_with_config(num_total, num_group, config);
let keys: HashSet<_> = keys.into_iter().take(num_total).collect();
// Each test run publishes all records twice.
let mut published = false;
let mut republished = false;
// The accumulated results for one round of publishing.
let mut results = Vec::new();
// Initiate the first round of publishing.
for k in &keys {
swarms[0].start_providing(k.preimage().clone());
}
current_thread::run(
future::poll_fn(move || loop {
// Poll all swarms until they are "NotReady".
for swarm in &mut swarms {
loop {
match swarm.poll().unwrap() {
Async::Ready(Some(KademliaEvent::StartProvidingResult(res))) |
Async::Ready(Some(KademliaEvent::RepublishProviderResult(res))) => {
match res {
Err(e) => panic!(e),
Ok(ok) => {
let key = kbucket::Key::new(ok.key.clone());
assert!(keys.contains(&key));
results.push(key);
}
}
}
Async::Ready(_) => (),
Async::NotReady => break,
}
}
}
if results.len() == keys.len() {
// All requests have been sent for one round of publishing.
published = true
}
if !published {
// Still waiting for all requests to be sent for one round
// of publishing.
return Ok(Async::NotReady)
}
// A round of publishing is complete. Consume the results, checking that
// each key was published to the `replication_factor` closest peers.
while let Some(key) = results.pop() {
// Collect the nodes that have a provider record for `key`.
let actual = swarms.iter().enumerate().skip(1)
.filter_map(|(i, s)|
if s.store.providers(key.preimage()).len() == 1 {
Some(swarm_ids[i].clone())
} else {
None
})
.collect::<HashSet<_>>();
if actual.len() != replication_factor.get() {
// Still waiting for some nodes to process the request.
results.push(key);
return Ok(Async::NotReady)
}
let mut expected = swarm_ids.clone().split_off(1);
expected.sort_by(|id1, id2|
kbucket::Key::new(id1).distance(&key).cmp(
&kbucket::Key::new(id2).distance(&key)));
let expected = expected
.into_iter()
.take(replication_factor.get())
.collect::<HashSet<_>>();
assert_eq!(actual, expected);
}
// One round of publishing is complete.
assert!(results.is_empty());
for s in &swarms {
assert_eq!(s.queries.size(), 0);
}
if republished {
assert_eq!(swarms[0].store.provided().count(), keys.len());
for k in &keys {
swarms[0].stop_providing(k.preimage());
}
assert_eq!(swarms[0].store.provided().count(), 0);
// All records have been republished, thus the test is complete.
return Ok(Async::Ready(()));
}
// Initiate the second round of publishing by telling the
// periodic provider job to run asap.
swarms[0].add_provider_job.as_mut().unwrap().asap();
published = false;
republished = true;
})
)
}
QuickCheck::new().tests(3).quickcheck(prop as fn(_,_))
}

View File

@ -179,26 +179,26 @@ pub enum KademliaHandlerEvent<TUserData> {
user_data: TUserData,
},
/// The remote indicates that this list of providers is known for this key.
/// The peer announced itself as a provider of a key.
AddProvider {
/// Key for which we should add providers.
/// The key for which the peer is a provider of the associated value.
key: Multihash,
/// Known provider for this key.
provider_peer: KadPeer,
/// The peer that is the provider of the value for `key`.
provider: KadPeer,
},
/// Request to get a value from the dht records
GetValue {
GetRecord {
/// Key for which we should look in the dht
key: Multihash,
/// Identifier of the request. Needs to be passed back when answering.
request_id: KademliaRequestId,
},
/// Response to a `KademliaHandlerIn::GetValue`.
GetValueRes {
/// Response to a `KademliaHandlerIn::GetRecord`.
GetRecordRes {
/// The result is present if the key has been found
result: Option<Record>,
record: Option<Record>,
/// Nodes closest to the key.
closer_peers: Vec<KadPeer>,
/// The user data passed to the `GetValue`.
@ -206,20 +206,19 @@ pub enum KademliaHandlerEvent<TUserData> {
},
/// Request to put a value in the dht records
PutValue {
/// The key of the record
key: Multihash,
/// The value of the record
value: Vec<u8>,
PutRecord {
record: Record,
/// Identifier of the request. Needs to be passed back when answering.
request_id: KademliaRequestId,
},
/// Response to a request to put a value
PutValueRes {
/// The key we were putting in
/// Response to a request to store a record.
PutRecordRes {
/// The key of the stored record.
key: Multihash,
/// The user data passed to the `GetValue`.
/// The value of the stored record.
value: Vec<u8>,
/// The user data passed to the `PutValue`.
user_data: TUserData,
}
}
@ -271,6 +270,15 @@ impl From<ProtocolsHandlerUpgrErr<io::Error>> for KademliaHandlerQueryErr {
/// Event to send to the handler.
#[derive(Debug)]
pub enum KademliaHandlerIn<TUserData> {
/// Resets the (sub)stream associated with the given request ID,
/// thus signaling an error to the remote.
///
/// Explicitly resetting the (sub)stream associated with a request
/// can be used as an alternative to letting requests simply time
/// out on the remote peer, thus potentially avoiding some delay
/// for the query on the remote.
Reset(KademliaRequestId),
/// Request for the list of nodes whose IDs are the closest to `key`. The number of nodes
/// returned is not specified, but should be around 20.
FindNodeReq {
@ -319,21 +327,21 @@ pub enum KademliaHandlerIn<TUserData> {
/// Key for which we should add providers.
key: Multihash,
/// Known provider for this key.
provider_peer: KadPeer,
provider: KadPeer,
},
/// Request to get a node from the dht
GetValue {
/// The key of the value we are looking for
/// Request to retrieve a record from the DHT.
GetRecord {
/// The key of the record.
key: Multihash,
/// Custom data. Passed back in the out event when the results arrive.
user_data: TUserData,
},
/// Response to a `GetValue`.
GetValueRes {
/// Response to a `GetRecord` request.
GetRecordRes {
/// The value that might have been found in our storage.
result: Option<Record>,
record: Option<Record>,
/// Nodes that are closer to the key we were searching for.
closer_peers: Vec<KadPeer>,
/// Identifier of the request that was made by the remote.
@ -341,17 +349,14 @@ pub enum KademliaHandlerIn<TUserData> {
},
/// Put a value into the dht records.
PutValue {
/// The key of the record.
key: Multihash,
/// The value of the record.
value: Vec<u8>,
PutRecord {
record: Record,
/// Custom data. Passed back in the out event when the results arrive.
user_data: TUserData,
},
/// Response to a `PutValue`.
PutValueRes {
/// Response to a `PutRecord`.
PutRecordRes {
/// Key of the value that was put.
key: Multihash,
/// Value that was put.
@ -472,9 +477,18 @@ where
.push(SubstreamState::InWaitingMessage(connec_unique_id, protocol));
}
#[inline]
fn inject_event(&mut self, message: KademliaHandlerIn<TUserData>) {
match message {
KademliaHandlerIn::Reset(request_id) => {
let pos = self.substreams.iter().position(|state| match state {
SubstreamState::InWaitingUser(conn_id, _) =>
conn_id == &request_id.connec_unique_id,
_ => false,
});
if let Some(pos) = pos {
let _ = self.substreams.remove(pos).try_close();
}
}
KademliaHandlerIn::FindNodeReq { key, user_data } => {
let msg = KadRequestMsg::FindNode { key };
self.substreams.push(SubstreamState::OutPendingOpen(msg, Some(user_data.clone())));
@ -535,31 +549,27 @@ where
.push(SubstreamState::InPendingSend(conn_id, substream, msg));
}
}
KademliaHandlerIn::AddProvider { key, provider_peer } => {
KademliaHandlerIn::AddProvider { key, provider } => {
let msg = KadRequestMsg::AddProvider {
key: key.clone(),
provider_peer: provider_peer.clone(),
provider: provider.clone(),
};
self.substreams
.push(SubstreamState::OutPendingOpen(msg, None));
}
KademliaHandlerIn::GetValue { key, user_data } => {
KademliaHandlerIn::GetRecord { key, user_data } => {
let msg = KadRequestMsg::GetValue { key };
self.substreams
.push(SubstreamState::OutPendingOpen(msg, Some(user_data)));
}
KademliaHandlerIn::PutValue { key, value, user_data } => {
let msg = KadRequestMsg::PutValue {
key,
value,
};
KademliaHandlerIn::PutRecord { record, user_data } => {
let msg = KadRequestMsg::PutValue { record };
self.substreams
.push(SubstreamState::OutPendingOpen(msg, Some(user_data)));
}
KademliaHandlerIn::GetValueRes {
result,
KademliaHandlerIn::GetRecordRes {
record,
closer_peers,
request_id,
} => {
@ -576,14 +586,14 @@ where
};
let msg = KadResponseMsg::GetValue {
result,
record,
closer_peers: closer_peers.clone(),
};
self.substreams
.push(SubstreamState::InPendingSend(conn_id, substream, msg));
}
}
KademliaHandlerIn::PutValueRes {
KademliaHandlerIn::PutRecordRes {
key,
request_id,
value,
@ -883,16 +893,15 @@ fn process_kad_request<TUserData>(
key,
request_id: KademliaRequestId { connec_unique_id },
}),
KadRequestMsg::AddProvider { key, provider_peer } => {
Ok(KademliaHandlerEvent::AddProvider { key, provider_peer })
KadRequestMsg::AddProvider { key, provider } => {
Ok(KademliaHandlerEvent::AddProvider { key, provider })
}
KadRequestMsg::GetValue { key } => Ok(KademliaHandlerEvent::GetValue {
KadRequestMsg::GetValue { key } => Ok(KademliaHandlerEvent::GetRecord {
key,
request_id: KademliaRequestId { connec_unique_id },
}),
KadRequestMsg::PutValue { key, value } => Ok(KademliaHandlerEvent::PutValue {
key,
value,
KadRequestMsg::PutValue { record } => Ok(KademliaHandlerEvent::PutRecord {
record,
request_id: KademliaRequestId { connec_unique_id },
})
}
@ -927,16 +936,17 @@ fn process_kad_response<TUserData>(
user_data,
},
KadResponseMsg::GetValue {
result,
record,
closer_peers,
} => KademliaHandlerEvent::GetValueRes {
result,
} => KademliaHandlerEvent::GetRecordRes {
record,
closer_peers,
user_data,
},
KadResponseMsg::PutValue { key, .. } => {
KademliaHandlerEvent::PutValueRes {
KadResponseMsg::PutValue { key, value, .. } => {
KademliaHandlerEvent::PutRecordRes {
key,
value,
user_data,
}
}

406
protocols/kad/src/jobs.rs Normal file
View File

@ -0,0 +1,406 @@
// Copyright 2019 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
//! Periodic (background) jobs.
//!
//! ## Record Persistence & Expiry
//!
//! To ensure persistence of records in the DHT, a Kademlia node
//! must periodically (re-)publish and (re-)replicate its records:
//!
//! 1. (Re-)publishing: The original publisher or provider of a record
//! must regularly re-publish in order to prolong the expiration.
//!
//! 2. (Re-)replication: Every node storing a replica of a record must
//! regularly re-replicate it to the closest nodes to the key in
//! order to ensure the record is present at these nodes.
//!
//! Re-publishing primarily ensures persistence of the record beyond its
//! initial TTL, for as long as the publisher stores (or provides) the record,
//! whilst (re-)replication primarily ensures persistence for the duration
//! of the TTL in the light of topology changes. Consequently, replication
//! intervals should be shorter than publication intervals and
//! publication intervals should be shorter than the TTL.
//!
//! This module implements two periodic jobs:
//!
//! * [`jobs::PutRecordJob`]: For (re-)publication and (re-)replication of
//! regular (value-)records.
//!
//! * [`jobs::AddProviderJob`]: For (re-)publication of provider records.
//! Provider records currently have no separate replication mechanism.
//!
//! A periodic job is driven like a `Future` or `Stream` by `poll`ing it.
//! Once a job starts running it emits records to send to the `k` closest
//! nodes to the key, where `k` is the replication factor.
//!
//! Furthermore, these jobs perform double-duty by removing expired records
//! from the `RecordStore` on every run. Expired records are never emitted
//! by the jobs.
//!
//! > **Note**: The current implementation takes a snapshot of the records
//! > to replicate from the `RecordStore` when it starts and thus, to account
//! > for the worst case, it temporarily requires additional memory proportional
//! > to the size of all stored records. As a job runs, the records are moved
//! > out of the job to the consumer, where they can be dropped after being sent.
use crate::record::{Record, ProviderRecord, store::RecordStore};
use libp2p_core::PeerId;
use futures::prelude::*;
use multihash::Multihash;
use std::collections::HashSet;
use std::time::Duration;
use std::vec;
use wasm_timer::{Instant, Delay};
/// The maximum number of queries towards which background jobs
/// are allowed to start new queries on an invocation of
/// `Kademlia::poll`.
pub const JOBS_MAX_QUERIES: usize = 100;
/// The maximum number of new queries started by a background job
/// per invocation of `Kademlia::poll`.
pub const JOBS_MAX_NEW_QUERIES: usize = 10;
/// A background job run periodically.
#[derive(Debug)]
struct PeriodicJob<T> {
interval: Duration,
state: PeriodicJobState<T>,
}
impl<T> PeriodicJob<T> {
fn is_running(&self) -> bool {
match self.state {
PeriodicJobState::Running(..) => true,
PeriodicJobState::Waiting(..) => false,
}
}
/// Cuts short the remaining delay, if the job is currently waiting
/// for the delay to expire.
fn asap(&mut self) {
if let PeriodicJobState::Waiting(delay) = &mut self.state {
delay.reset(Instant::now() - Duration::from_secs(1))
}
}
/// Returns `true` if the job is currently not running but ready
/// to be run, `false` otherwise.
fn is_ready(&mut self, now: Instant) -> bool {
if let PeriodicJobState::Waiting(delay) = &mut self.state {
if now >= delay.deadline() || delay.poll().map(|a| a.is_ready()).unwrap_or(false) {
return true
}
}
false
}
}
/// The state of a background job run periodically.
#[derive(Debug)]
enum PeriodicJobState<T> {
Running(T),
Waiting(Delay)
}
//////////////////////////////////////////////////////////////////////////////
// PutRecordJob
/// Periodic job for replicating / publishing records.
pub struct PutRecordJob {
local_id: PeerId,
next_publish: Option<Instant>,
publish_interval: Option<Duration>,
record_ttl: Option<Duration>,
skipped: HashSet<Multihash>,
inner: PeriodicJob<vec::IntoIter<Record>>,
}
impl PutRecordJob {
/// Creates a new periodic job for replicating and re-publishing
/// locally stored records.
pub fn new(
local_id: PeerId,
replicate_interval: Duration,
publish_interval: Option<Duration>,
record_ttl: Option<Duration>,
) -> Self {
let now = Instant::now();
let delay = Delay::new(now + replicate_interval);
let next_publish = publish_interval.map(|i| now + i);
Self {
local_id,
next_publish,
publish_interval,
record_ttl,
skipped: HashSet::new(),
inner: PeriodicJob {
interval: replicate_interval,
state: PeriodicJobState::Waiting(delay)
}
}
}
/// Adds the key of a record that is ignored on the current or
/// next run of the job.
pub fn skip(&mut self, key: Multihash) {
self.skipped.insert(key);
}
/// Checks whether the job is currently running.
pub fn is_running(&self) -> bool {
self.inner.is_running()
}
/// Cuts short the remaining delay, if the job is currently waiting
/// for the delay to expire.
///
/// The job is guaranteed to run on the next invocation of `poll`.
pub fn asap(&mut self, publish: bool) {
if publish {
self.next_publish = Some(Instant::now() - Duration::from_secs(1))
}
self.inner.asap()
}
/// Polls the job for records to replicate.
///
/// Must be called in the context of a task. When `NotReady` is returned,
/// the current task is registered to be notified when the job is ready
/// to be run.
pub fn poll<T>(&mut self, store: &mut T, now: Instant) -> Async<Record>
where
for<'a> T: RecordStore<'a>
{
if self.inner.is_ready(now) {
let publish = self.next_publish.map_or(false, |t_pub| now >= t_pub);
let records = store.records()
.filter_map(|r| {
let is_publisher = r.publisher.as_ref() == Some(&self.local_id);
if self.skipped.contains(&r.key) || (!publish && is_publisher) {
None
} else {
let mut record = r.into_owned();
if publish && is_publisher {
record.expires = record.expires.or_else(||
self.record_ttl.map(|ttl| now + ttl));
}
Some(record)
}
})
.collect::<Vec<_>>()
.into_iter();
// Schedule the next publishing run.
if publish {
self.next_publish = self.publish_interval.map(|i| now + i);
}
self.skipped.clear();
self.inner.state = PeriodicJobState::Running(records);
}
if let PeriodicJobState::Running(records) = &mut self.inner.state {
loop {
if let Some(r) = records.next() {
if r.is_expired(now) {
store.remove(&r.key)
} else {
return Async::Ready(r)
}
} else {
break
}
}
// Wait for the next run.
let delay = Delay::new(now + self.inner.interval);
self.inner.state = PeriodicJobState::Waiting(delay);
assert!(!self.inner.is_ready(now));
}
Async::NotReady
}
}
//////////////////////////////////////////////////////////////////////////////
// AddProviderJob
/// Periodic job for replicating provider records.
pub struct AddProviderJob {
inner: PeriodicJob<vec::IntoIter<ProviderRecord>>
}
impl AddProviderJob {
/// Creates a new periodic job for provider announcements.
pub fn new(interval: Duration) -> Self {
let now = Instant::now();
Self {
inner: PeriodicJob {
interval,
state: PeriodicJobState::Waiting(Delay::new(now + interval))
}
}
}
/// Checks whether the job is currently running.
pub fn is_running(&self) -> bool {
self.inner.is_running()
}
/// Cuts short the remaining delay, if the job is currently waiting
/// for the delay to expire.
///
/// The job is guaranteed to run on the next invocation of `poll`.
pub fn asap(&mut self) {
self.inner.asap()
}
/// Polls the job for provider records to replicate.
///
/// Must be called in the context of a task. When `NotReady` is returned,
/// the current task is registered to be notified when the job is ready
/// to be run.
pub fn poll<T>(&mut self, store: &mut T, now: Instant) -> Async<ProviderRecord>
where
for<'a> T: RecordStore<'a>
{
if self.inner.is_ready(now) {
let records = store.provided()
.map(|r| r.into_owned())
.collect::<Vec<_>>()
.into_iter();
self.inner.state = PeriodicJobState::Running(records);
}
if let PeriodicJobState::Running(keys) = &mut self.inner.state {
loop {
if let Some(r) = keys.next() {
if r.is_expired(now) {
store.remove_provider(&r.key, &r.provider)
} else {
return Async::Ready(r)
}
} else {
break
}
}
let delay = Delay::new(now + self.inner.interval);
self.inner.state = PeriodicJobState::Waiting(delay);
assert!(!self.inner.is_ready(now));
}
Async::NotReady
}
}
#[cfg(test)]
mod tests {
use crate::record::store::MemoryStore;
use quickcheck::*;
use rand::Rng;
use super::*;
fn rand_put_record_job() -> PutRecordJob {
let mut rng = rand::thread_rng();
let id = PeerId::random();
let replicate_interval = Duration::from_secs(rng.gen_range(1, 60));
let publish_interval = Some(replicate_interval * rng.gen_range(1, 10));
let record_ttl = Some(Duration::from_secs(rng.gen_range(1, 600)));
PutRecordJob::new(id.clone(), replicate_interval, publish_interval, record_ttl)
}
fn rand_add_provider_job() -> AddProviderJob {
let mut rng = rand::thread_rng();
let interval = Duration::from_secs(rng.gen_range(1, 60));
AddProviderJob::new(interval)
}
#[test]
fn new_job_not_running() {
let job = rand_put_record_job();
assert!(!job.is_running());
let job = rand_add_provider_job();
assert!(!job.is_running());
}
#[test]
fn run_put_record_job() {
fn prop(records: Vec<Record>) {
let mut job = rand_put_record_job();
// Fill a record store.
let mut store = MemoryStore::new(job.local_id.clone());
for r in records {
let _ = store.put(r);
}
// Polling with an instant beyond the deadline for the next run
// is guaranteed to run the job, without the job needing to poll the `Delay`
// and thus without needing to run `poll` in the context of a task
// for testing purposes.
let now = Instant::now() + job.inner.interval;
// All (non-expired) records in the store must be yielded by the job.
for r in store.records().map(|r| r.into_owned()).collect::<Vec<_>>() {
if !r.is_expired(now) {
assert_eq!(job.poll(&mut store, now), Async::Ready(r));
assert!(job.is_running());
}
}
assert_eq!(job.poll(&mut store, now), Async::NotReady);
assert!(!job.is_running());
}
quickcheck(prop as fn(_))
}
#[test]
fn run_add_provider_job() {
fn prop(records: Vec<ProviderRecord>) {
let mut job = rand_add_provider_job();
let id = PeerId::random();
// Fill a record store.
let mut store = MemoryStore::new(id.clone());
for mut r in records {
r.provider = id.clone();
let _ = store.add_provider(r);
}
// Polling with an instant beyond the deadline for the next run
// is guaranteed to run the job, without the job needing to poll the `Delay`
// and thus without needing to run `poll` in the context of a task
// for testing purposes.
let now = Instant::now() + job.inner.interval;
// All (non-expired) records in the store must be yielded by the job.
for r in store.provided().map(|r| r.into_owned()).collect::<Vec<_>>() {
if !r.is_expired(now) {
assert_eq!(job.poll(&mut store, now), Async::Ready(r));
assert!(job.is_running());
}
}
assert_eq!(job.poll(&mut store, now), Async::NotReady);
assert!(!job.is_running());
}
quickcheck(prop as fn(_))
}
}

View File

@ -83,19 +83,19 @@ const NUM_BUCKETS: usize = 256;
/// A `KBucketsTable` represents a Kademlia routing table.
#[derive(Debug, Clone)]
pub struct KBucketsTable<TPeerId, TVal> {
pub struct KBucketsTable<TKey, TVal> {
/// The key identifying the local peer that owns the routing table.
local_key: Key<TPeerId>,
local_key: TKey,
/// The buckets comprising the routing table.
buckets: Vec<KBucket<TPeerId, TVal>>,
buckets: Vec<KBucket<TKey, TVal>>,
/// The list of evicted entries that have been replaced with pending
/// entries since the last call to [`KBucketsTable::take_applied_pending`].
applied_pending: VecDeque<AppliedPending<TPeerId, TVal>>
applied_pending: VecDeque<AppliedPending<TKey, TVal>>
}
/// A (type-safe) index into a `KBucketsTable`, i.e. a non-negative integer in the
/// interval `[0, NUM_BUCKETS)`.
#[derive(Copy, Clone, PartialEq, Eq)]
#[derive(Debug, Copy, Clone, PartialEq, Eq)]
struct BucketIndex(usize);
impl BucketIndex {
@ -124,17 +124,18 @@ impl BucketIndex {
for i in 0 .. quot {
bytes[31 - i] = rng.gen();
}
let rem = self.0 % 8;
let lower = usize::pow(2, rem as u32);
let upper = usize::pow(2, rem as u32 + 1);
let rem = (self.0 % 8) as u32;
let lower = usize::pow(2, rem);
let upper = usize::pow(2, rem + 1);
bytes[31 - quot] = rng.gen_range(lower, upper) as u8;
Distance(bigint::U256::from(bytes))
}
}
impl<TPeerId, TVal> KBucketsTable<TPeerId, TVal>
impl<TKey, TVal> KBucketsTable<TKey, TVal>
where
TPeerId: Clone,
TKey: Clone + AsRef<KeyBytes>,
TVal: Clone
{
/// Creates a new, empty Kademlia routing table with entries partitioned
/// into buckets as per the Kademlia protocol.
@ -142,7 +143,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: Key<TPeerId>, pending_timeout: Duration) -> Self {
pub fn new(local_key: TKey, pending_timeout: Duration) -> Self {
KBucketsTable {
local_key,
buckets: (0 .. NUM_BUCKETS).map(|_| KBucket::new(pending_timeout)).collect(),
@ -151,14 +152,14 @@ where
}
/// Returns the local key.
pub fn local_key(&self) -> &Key<TPeerId> {
pub 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 Key<TPeerId>) -> Entry<'a, TPeerId, TVal> {
let index = BucketIndex::new(&self.local_key.distance(key));
pub 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()];
if let Some(applied) = bucket.apply_pending() {
@ -171,7 +172,7 @@ where
}
/// Returns an iterator over all the entries in the routing table.
pub fn iter<'a>(&'a mut self) -> impl Iterator<Item = EntryRefView<'a, TPeerId, TVal>> {
pub fn iter<'a>(&'a mut self) -> impl Iterator<Item = EntryRefView<'a, TKey, TVal>> {
let applied_pending = &mut self.applied_pending;
self.buckets.iter_mut().flat_map(move |table| {
if let Some(applied) = table.apply_pending() {
@ -194,7 +195,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 buckets<'a>(&'a mut self) -> impl Iterator<Item = KBucketRef<'a, TPeerId, TVal>> + 'a {
pub fn buckets<'a>(&'a mut self) -> impl Iterator<Item = KBucketRef<'a, TKey, TVal>> + 'a {
let applied_pending = &mut self.applied_pending;
self.buckets.iter_mut().enumerate().map(move |(i, b)| {
if let Some(applied) = b.apply_pending() {
@ -219,24 +220,24 @@ 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<TPeerId, TVal>> {
pub 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 Key<T>)
-> impl Iterator<Item = Key<TPeerId>> + 'a
pub fn closest_keys<'a, T>(&'a mut self, target: &'a T)
-> impl Iterator<Item = TKey> + 'a
where
T: Clone
T: Clone + AsRef<KeyBytes>
{
let distance = self.local_key.distance(target);
let distance = self.local_key.as_ref().distance(target);
ClosestIter {
target,
iter: None,
table: self,
buckets_iter: ClosestBucketsIter::new(distance),
fmap: |b: &KBucket<_, _>| -> ArrayVec<_> {
fmap: |b: &KBucket<TKey, _>| -> ArrayVec<_> {
b.iter().map(|(n,_)| n.key.clone()).collect()
}
}
@ -244,13 +245,13 @@ where
/// Returns an iterator over the nodes closest to the `target` key, ordered by
/// increasing distance.
pub fn closest<'a, T>(&'a mut self, target: &'a Key<T>)
-> impl Iterator<Item = EntryView<TPeerId, TVal>> + 'a
pub fn closest<'a, T>(&'a mut self, target: &'a T)
-> impl Iterator<Item = EntryView<TKey, TVal>> + 'a
where
T: Clone,
T: Clone + AsRef<KeyBytes>,
TVal: Clone
{
let distance = self.local_key.distance(target);
let distance = self.local_key.as_ref().distance(target);
ClosestIter {
target,
iter: None,
@ -264,23 +265,46 @@ where
}
}
}
/// Counts the number of nodes between the local node and the node
/// closest to `target`.
///
/// 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
where
T: AsRef<KeyBytes>
{
let local_key = self.local_key.clone();
let distance = target.as_ref().distance(&local_key);
let mut iter = ClosestBucketsIter::new(distance).take_while(|i| i.get() != 0);
if let Some(i) = iter.next() {
let num_first = self.buckets[i.get()].iter()
.filter(|(n,_)| n.key.as_ref().distance(&local_key) <= distance)
.count();
let num_rest: usize = iter.map(|i| self.buckets[i.get()].num_entries()).sum();
num_first + num_rest
} else {
0
}
}
}
/// An iterator over (some projection of) the closest entries in a
/// `KBucketsTable` w.r.t. some target `Key`.
struct ClosestIter<'a, TTarget, TPeerId, TVal, TMap, TOut> {
struct ClosestIter<'a, TTarget, TKey, TVal, TMap, TOut> {
/// A reference to the target key whose distance to the local key determines
/// the order in which the buckets are traversed. The resulting
/// array from projecting the entries of each bucket using `fmap` is
/// sorted according to the distance to the target.
target: &'a Key<TTarget>,
target: &'a TTarget,
/// A reference to all buckets of the `KBucketsTable`.
table: &'a mut KBucketsTable<TPeerId, TVal>,
table: &'a mut KBucketsTable<TKey, TVal>,
/// The iterator over the bucket indices in the order determined by the
/// distance of the local key to the target.
buckets_iter: ClosestBucketsIter,
/// The iterator over the entries in the currently traversed bucket.
iter: Option<arrayvec::IntoIter<[TOut; K_VALUE]>>,
iter: Option<arrayvec::IntoIter<[TOut; K_VALUE.get()]>>,
/// The projection function / mapping applied on each bucket as
/// it is encountered, producing the next `iter`ator.
fmap: TMap
@ -376,12 +400,14 @@ impl Iterator for ClosestBucketsIter {
}
}
impl<TTarget, TPeerId, TVal, TMap, TOut> Iterator
for ClosestIter<'_, TTarget, TPeerId, TVal, TMap, TOut>
impl<TTarget, TKey, TVal, TMap, TOut> Iterator
for ClosestIter<'_, TTarget, TKey, TVal, TMap, TOut>
where
TPeerId: Clone,
TMap: Fn(&KBucket<TPeerId, TVal>) -> ArrayVec<[TOut; K_VALUE]>,
TOut: AsRef<Key<TPeerId>>
TTarget: AsRef<KeyBytes>,
TKey: Clone + AsRef<KeyBytes>,
TVal: Clone,
TMap: Fn(&KBucket<TKey, TVal>) -> ArrayVec<[TOut; K_VALUE.get()]>,
TOut: AsRef<KeyBytes>
{
type Item = TOut;
@ -400,8 +426,8 @@ where
}
let mut v = (self.fmap)(bucket);
v.sort_by(|a, b|
self.target.distance(a.as_ref())
.cmp(&self.target.distance(b.as_ref())));
self.target.as_ref().distance(a.as_ref())
.cmp(&self.target.as_ref().distance(b.as_ref())));
self.iter = Some(v.into_iter());
} else {
return None
@ -418,9 +444,10 @@ pub struct KBucketRef<'a, TPeerId, TVal> {
bucket: &'a mut KBucket<TPeerId, TVal>
}
impl<TPeerId, TVal> KBucketRef<'_, TPeerId, TVal>
impl<TKey, TVal> KBucketRef<'_, TKey, TVal>
where
TPeerId: Clone
TKey: Clone + AsRef<KeyBytes>,
TVal: Clone
{
/// Returns the number of entries in the bucket.
pub fn num_entries(&self) -> usize {
@ -432,6 +459,7 @@ where
self.bucket.pending().map_or(false, |n| !n.is_ready())
}
/// Tests whether the given distance falls into this bucket.
pub fn contains(&self, d: &Distance) -> bool {
BucketIndex::new(d).map_or(false, |i| i == self.index)
}
@ -453,6 +481,34 @@ mod tests {
use super::*;
use libp2p_core::PeerId;
use quickcheck::*;
use rand::Rng;
type TestTable = KBucketsTable<KeyBytes, ()>;
impl Arbitrary for TestTable {
fn arbitrary<G: Gen>(g: &mut G) -> TestTable {
let local_key = Key::from(PeerId::random());
let timeout = Duration::from_secs(g.gen_range(1, 360));
let mut table = TestTable::new(local_key.clone().into(), timeout);
let mut num_total = g.gen_range(0, 100);
for (i, b) in &mut table.buckets.iter_mut().enumerate().rev() {
let ix = BucketIndex(i);
let num = g.gen_range(0, usize::min(K_VALUE.get(), num_total) + 1);
num_total -= num;
for _ in 0 .. num {
let distance = ix.rand_distance(g);
let key = local_key.for_distance(distance);
let node = Node { key: key.clone(), value: () };
let status = NodeStatus::arbitrary(g);
match b.insert(node, status) {
InsertResult::Inserted => {}
_ => panic!()
}
}
}
table
}
}
#[test]
fn rand_distance() {
@ -469,7 +525,7 @@ mod tests {
}
#[test]
fn basic_closest() {
fn entry_inserted() {
let local_key = Key::from(PeerId::random());
let other_id = Key::from(PeerId::random());
@ -489,7 +545,7 @@ mod tests {
}
#[test]
fn update_local_id_fails() {
fn entry_self() {
let local_key = Key::from(PeerId::random());
let mut table = KBucketsTable::<_, ()>::new(local_key.clone(), Duration::from_secs(5));
match table.entry(&local_key) {
@ -545,7 +601,7 @@ mod tests {
match e.insert((), NodeStatus::Connected) {
InsertResult::Pending { disconnected } => {
expected_applied = AppliedPending {
inserted: key.clone(),
inserted: Node { key: key.clone(), value: () },
evicted: Some(Node { key: disconnected, value: () })
};
full_bucket_index = BucketIndex::new(&key.distance(&local_key));
@ -569,7 +625,7 @@ mod tests {
let elapsed = Instant::now() - Duration::from_secs(1);
full_bucket.pending_mut().unwrap().set_ready_at(elapsed);
match table.entry(&expected_applied.inserted) {
match table.entry(&expected_applied.inserted.key) {
Entry::Present(_, NodeStatus::Connected) => {}
x => panic!("Unexpected entry: {:?}", x)
}
@ -582,4 +638,28 @@ mod tests {
assert_eq!(Some(expected_applied), table.take_applied_pending());
assert_eq!(None, table.take_applied_pending());
}
#[test]
fn count_nodes_between() {
fn prop(mut table: TestTable, target: Key<PeerId>) -> bool {
let num_to_target = table.count_nodes_between(&target);
let distance = table.local_key.distance(&target);
let base2 = U256::from(2);
let mut iter = ClosestBucketsIter::new(distance);
iter.all(|i| {
// Flip the distance bit related to the bucket.
let d = Distance(distance.0 ^ (base2.pow(U256::from(i.get()))));
let k = table.local_key.for_distance(d);
if distance.0.bit(i.get()) {
// Bit flip `1` -> `0`, the key must be closer than `target`.
d < distance && table.count_nodes_between(&k) <= num_to_target
} else {
// Bit flip `0` -> `1`, the key must be farther than `target`.
d > distance && table.count_nodes_between(&k) >= num_to_target
}
})
}
QuickCheck::new().tests(10).quickcheck(prop as fn(_,_) -> _)
}
}

View File

@ -30,9 +30,9 @@ use super::*;
/// A `PendingNode` is a `Node` that is pending insertion into a `KBucket`.
#[derive(Debug, Clone)]
pub struct PendingNode<TPeerId, TVal> {
pub struct PendingNode<TKey, TVal> {
/// The pending node to insert.
node: Node<TPeerId, TVal>,
node: Node<TKey, TVal>,
/// The status of the pending node.
status: NodeStatus,
@ -54,8 +54,8 @@ pub enum NodeStatus {
Disconnected
}
impl<TPeerId, TVal> PendingNode<TPeerId, TVal> {
pub fn key(&self) -> &Key<TPeerId> {
impl<TKey, TVal> PendingNode<TKey, TVal> {
pub fn key(&self) -> &TKey {
&self.node.key
}
@ -80,9 +80,9 @@ impl<TPeerId, TVal> PendingNode<TPeerId, TVal> {
/// in the Kademlia DHT together with an associated value (e.g. contact
/// information).
#[derive(Debug, Clone, PartialEq, Eq)]
pub struct Node<TPeerId, TVal> {
pub struct Node<TKey, TVal> {
/// The key of the node, identifying the peer.
pub key: Key<TPeerId>,
pub key: TKey,
/// The associated value.
pub value: TVal,
}
@ -92,12 +92,12 @@ pub struct Node<TPeerId, TVal> {
#[derive(Copy, Clone, Debug, PartialEq, Eq, PartialOrd, Ord)]
pub struct Position(usize);
/// A `KBucket` is a list of up to `K_VALUE` `Key`s and associated values,
/// A `KBucket` is a list of up to `K_VALUE` keys and associated values,
/// ordered from least-recently connected to most-recently connected.
#[derive(Debug, Clone)]
pub struct KBucket<TPeerId, TVal> {
pub struct KBucket<TKey, TVal> {
/// The nodes contained in the bucket.
nodes: ArrayVec<[Node<TPeerId, TVal>; K_VALUE]>,
nodes: ArrayVec<[Node<TKey, TVal>; K_VALUE.get()]>,
/// The position (index) in `nodes` that marks the first connected node.
///
@ -116,7 +116,7 @@ pub struct KBucket<TPeerId, TVal> {
/// A node that is pending to be inserted into a full bucket, should the
/// least-recently connected (and currently disconnected) node not be
/// marked as connected within `unresponsive_timeout`.
pending: Option<PendingNode<TPeerId, TVal>>,
pending: Option<PendingNode<TKey, TVal>>,
/// The timeout window before a new pending node is eligible for insertion,
/// if the least-recently connected node is not updated as being connected
@ -127,7 +127,7 @@ pub struct KBucket<TPeerId, TVal> {
/// The result of inserting an entry into a bucket.
#[must_use]
#[derive(Debug, Clone, PartialEq, Eq)]
pub enum InsertResult<TPeerId> {
pub enum InsertResult<TKey> {
/// The entry has been successfully inserted.
Inserted,
/// The entry is pending insertion because the relevant bucket is currently full.
@ -140,7 +140,7 @@ pub enum InsertResult<TPeerId> {
/// in order to prevent it from being evicted. If connectivity to the peer is
/// re-established, the corresponding entry should be updated with
/// [`NodeStatus::Connected`].
disconnected: Key<TPeerId>
disconnected: TKey
},
/// The entry was not inserted because the relevant bucket is full.
Full
@ -149,17 +149,18 @@ pub enum InsertResult<TPeerId> {
/// The result of applying a pending node to a bucket, possibly
/// replacing an existing node.
#[derive(Debug, Clone, PartialEq, Eq)]
pub struct AppliedPending<TPeerId, TVal> {
pub struct AppliedPending<TKey, TVal> {
/// The key of the inserted pending node.
pub inserted: Key<TPeerId>,
pub 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<TPeerId, TVal>>
pub evicted: Option<Node<TKey, TVal>>
}
impl<TPeerId, TVal> KBucket<TPeerId, TVal>
impl<TKey, TVal> KBucket<TKey, TVal>
where
TPeerId: Clone
TKey: Clone + AsRef<KeyBytes>,
TVal: Clone
{
/// Creates a new `KBucket` with the given timeout for pending entries.
pub fn new(pending_timeout: Duration) -> Self {
@ -172,28 +173,28 @@ where
}
/// Returns a reference to the pending node of the bucket, if there is any.
pub fn pending(&self) -> Option<&PendingNode<TPeerId, TVal>> {
pub fn pending(&self) -> Option<&PendingNode<TKey, TVal>> {
self.pending.as_ref()
}
/// Returns a mutable reference to the pending node of the bucket, if there is any.
pub fn pending_mut(&mut self) -> Option<&mut PendingNode<TPeerId, TVal>> {
pub fn pending_mut(&mut self) -> Option<&mut PendingNode<TKey, TVal>> {
self.pending.as_mut()
}
/// Returns a reference to the pending node of the bucket, if there is any
/// with a matching key.
pub fn as_pending(&self, key: &Key<TPeerId>) -> Option<&PendingNode<TPeerId, TVal>> {
self.pending().filter(|p| &p.node.key == key)
pub fn as_pending(&self, key: &TKey) -> Option<&PendingNode<TKey, TVal>> {
self.pending().filter(|p| p.node.key.as_ref() == key.as_ref())
}
/// Returns a reference to a node in the bucket.
pub fn get(&self, key: &Key<TPeerId>) -> Option<&Node<TPeerId, TVal>> {
pub fn get(&self, key: &TKey) -> Option<&Node<TKey, TVal>> {
self.position(key).map(|p| &self.nodes[p.0])
}
/// Returns an iterator over the nodes in the bucket, together with their status.
pub fn iter(&self) -> impl Iterator<Item = (&Node<TPeerId, TVal>, NodeStatus)> {
pub fn iter(&self) -> impl Iterator<Item = (&Node<TKey, TVal>, NodeStatus)> {
self.nodes.iter().enumerate().map(move |(p, n)| (n, self.status(Position(p))))
}
@ -203,7 +204,7 @@ where
/// If a pending node has been inserted, its key is returned together with
/// the node that was replaced. `None` indicates that the nodes in the
/// bucket remained unchanged.
pub fn apply_pending(&mut self) -> Option<AppliedPending<TPeerId, TVal>> {
pub fn apply_pending(&mut self) -> Option<AppliedPending<TKey, TVal>> {
if let Some(pending) = self.pending.take() {
if pending.replace <= Instant::now() {
if self.nodes.is_full() {
@ -212,7 +213,7 @@ where
return None
}
// The pending node will be inserted.
let inserted = pending.node.key.clone();
let inserted = pending.node.clone();
// A connected pending node goes at the end of the list for
// the connected peers, removing the least-recently connected.
if pending.status == NodeStatus::Connected {
@ -241,7 +242,7 @@ where
}
} else {
// There is room in the bucket, so just insert the pending node.
let inserted = pending.node.key.clone();
let inserted = pending.node.clone();
match self.insert(pending.node, pending.status) {
InsertResult::Inserted =>
return Some(AppliedPending { inserted, evicted: None }),
@ -265,7 +266,7 @@ where
/// Updates the status of the node referred to by the given key, if it is
/// in the bucket.
pub fn update(&mut self, key: &Key<TPeerId>, status: NodeStatus) {
pub fn update(&mut self, key: &TKey, status: NodeStatus) {
// Remove the node from its current position and then reinsert it
// with the desired status, which puts it at the end of either the
// prefix list of disconnected nodes or the suffix list of connected
@ -318,7 +319,7 @@ where
/// i.e. as the most-recently disconnected node. If there are no connected nodes,
/// the new node is added as the last element of the bucket.
///
pub fn insert(&mut self, node: Node<TPeerId, TVal>, status: NodeStatus) -> InsertResult<TPeerId> {
pub fn insert(&mut self, node: Node<TKey, TVal>, status: NodeStatus) -> InsertResult<TKey> {
match status {
NodeStatus::Connected => {
if self.nodes.is_full() {
@ -385,16 +386,16 @@ where
}
/// Gets the position of an node in the bucket.
pub fn position(&self, key: &Key<TPeerId>) -> Option<Position> {
self.nodes.iter().position(|p| &p.key == key).map(Position)
pub fn position(&self, key: &TKey) -> Option<Position> {
self.nodes.iter().position(|p| p.key.as_ref() == key.as_ref()).map(Position)
}
/// Gets a mutable reference to the node identified by the given key.
///
/// Returns `None` if the given key does not refer to an node in the
/// bucket.
pub fn get_mut(&mut self, key: &Key<TPeerId>) -> Option<&mut Node<TPeerId, TVal>> {
self.nodes.iter_mut().find(move |p| &p.key == key)
pub fn get_mut(&mut self, key: &TKey) -> Option<&mut Node<TKey, TVal>> {
self.nodes.iter_mut().find(move |p| p.key.as_ref() == key.as_ref())
}
}
@ -406,11 +407,11 @@ mod tests {
use super::*;
use quickcheck::*;
impl Arbitrary for KBucket<PeerId, ()> {
fn arbitrary<G: Gen>(g: &mut G) -> KBucket<PeerId, ()> {
impl Arbitrary for KBucket<Key<PeerId>, ()> {
fn arbitrary<G: Gen>(g: &mut G) -> KBucket<Key<PeerId>, ()> {
let timeout = Duration::from_secs(g.gen_range(1, g.size() as u64));
let mut bucket = KBucket::<PeerId, ()>::new(timeout);
let num_nodes = g.gen_range(1, K_VALUE + 1);
let mut bucket = KBucket::<Key<PeerId>, ()>::new(timeout);
let num_nodes = g.gen_range(1, K_VALUE.get() + 1);
for _ in 0 .. num_nodes {
let key = Key::new(PeerId::random());
let node = Node { key: key.clone(), value: () };
@ -436,14 +437,14 @@ mod tests {
impl Arbitrary for Position {
fn arbitrary<G: Gen>(g: &mut G) -> Position {
Position(g.gen_range(0, K_VALUE))
Position(g.gen_range(0, K_VALUE.get()))
}
}
// Fill a bucket with random nodes with the given status.
fn fill_bucket(bucket: &mut KBucket<PeerId, ()>, status: NodeStatus) {
fn fill_bucket(bucket: &mut KBucket<Key<PeerId>, ()>, status: NodeStatus) {
let num_entries_start = bucket.num_entries();
for i in 0 .. K_VALUE - num_entries_start {
for i in 0 .. K_VALUE.get() - num_entries_start {
let key = Key::new(PeerId::random());
let node = Node { key, value: () };
assert_eq!(InsertResult::Inserted, bucket.insert(node, status));
@ -454,7 +455,7 @@ mod tests {
#[test]
fn ordering() {
fn prop(status: Vec<NodeStatus>) -> bool {
let mut bucket = KBucket::<PeerId, ()>::new(Duration::from_secs(1));
let mut bucket = KBucket::<Key<PeerId>, ()>::new(Duration::from_secs(1));
// The expected lists of connected and disconnected nodes.
let mut connected = VecDeque::new();
@ -464,7 +465,7 @@ mod tests {
for status in status {
let key = Key::new(PeerId::random());
let node = Node { key: key.clone(), value: () };
let full = bucket.num_entries() == K_VALUE;
let full = bucket.num_entries() == K_VALUE.get();
match bucket.insert(node, status) {
InsertResult::Inserted => {
let vec = match status {
@ -503,7 +504,7 @@ mod tests {
#[test]
fn full_bucket() {
let mut bucket = KBucket::<PeerId, ()>::new(Duration::from_secs(1));
let mut bucket = KBucket::<Key<PeerId>, ()>::new(Duration::from_secs(1));
// Fill the bucket with disconnected nodes.
fill_bucket(&mut bucket, NodeStatus::Disconnected);
@ -517,7 +518,7 @@ mod tests {
}
// One-by-one fill the bucket with connected nodes, replacing the disconnected ones.
for i in 0 .. K_VALUE {
for i in 0 .. K_VALUE.get() {
let (first, first_status) = bucket.iter().next().unwrap();
let first_disconnected = first.clone();
assert_eq!(first_status, NodeStatus::Disconnected);
@ -545,16 +546,16 @@ mod tests {
pending.set_ready_at(Instant::now() - Duration::from_secs(1));
let result = bucket.apply_pending();
assert_eq!(result, Some(AppliedPending {
inserted: key.clone(),
inserted: node.clone(),
evicted: Some(first_disconnected)
}));
assert_eq!(Some((&node, NodeStatus::Connected)), bucket.iter().last());
assert!(bucket.pending().is_none());
assert_eq!(Some(K_VALUE - (i + 1)), bucket.first_connected_pos);
assert_eq!(Some(K_VALUE.get() - (i + 1)), bucket.first_connected_pos);
}
assert!(bucket.pending().is_none());
assert_eq!(K_VALUE, bucket.num_entries());
assert_eq!(K_VALUE.get(), bucket.num_entries());
// Trying to insert another connected node fails.
let key = Key::new(PeerId::random());
@ -567,7 +568,7 @@ mod tests {
#[test]
fn full_bucket_discard_pending() {
let mut bucket = KBucket::<PeerId, ()>::new(Duration::from_secs(1));
let mut bucket = KBucket::<Key<PeerId>, ()>::new(Duration::from_secs(1));
fill_bucket(&mut bucket, NodeStatus::Disconnected);
let (first, _) = bucket.iter().next().unwrap();
let first_disconnected = first.clone();
@ -593,13 +594,13 @@ mod tests {
assert_eq!(Some((&first_disconnected, NodeStatus::Connected)), bucket.iter().last());
assert_eq!(bucket.position(&first_disconnected.key).map(|p| p.0), bucket.first_connected_pos);
assert_eq!(1, bucket.num_connected());
assert_eq!(K_VALUE - 1, bucket.num_disconnected());
assert_eq!(K_VALUE.get() - 1, bucket.num_disconnected());
}
#[test]
fn bucket_update() {
fn prop(mut bucket: KBucket<PeerId, ()>, pos: Position, status: NodeStatus) -> bool {
fn prop(mut bucket: KBucket<Key<PeerId>, ()>, pos: Position, status: NodeStatus) -> bool {
let num_nodes = bucket.num_entries();
// Capture position and key of the random node to update.

View File

@ -35,15 +35,15 @@ pub struct EntryRefView<'a, TPeerId, TVal> {
}
/// An immutable by-reference view of a `Node`.
pub struct NodeRefView<'a, TPeerId, TVal> {
pub key: &'a Key<TPeerId>,
pub struct NodeRefView<'a, TKey, TVal> {
pub key: &'a TKey,
pub value: &'a TVal
}
impl<TPeerId, TVal> EntryRefView<'_, TPeerId, TVal> {
pub fn to_owned(&self) -> EntryView<TPeerId, TVal>
impl<TKey, TVal> EntryRefView<'_, TKey, TVal> {
pub fn to_owned(&self) -> EntryView<TKey, TVal>
where
TPeerId: Clone,
TKey: Clone,
TVal: Clone
{
EntryView {
@ -59,16 +59,16 @@ impl<TPeerId, TVal> EntryRefView<'_, TPeerId, TVal> {
/// A cloned, immutable view of an entry that is either present in a bucket
/// or pending insertion.
#[derive(Clone, Debug)]
pub struct EntryView<TPeerId, TVal> {
pub struct EntryView<TKey, TVal> {
/// The node represented by the entry.
pub node: Node<TPeerId, TVal>,
pub node: Node<TKey, TVal>,
/// The status of the node.
pub status: NodeStatus
}
impl<TPeerId, TVal> AsRef<Key<TPeerId>> for EntryView<TPeerId, TVal> {
fn as_ref(&self) -> &Key<TPeerId> {
&self.node.key
impl<TKey: AsRef<KeyBytes>, TVal> AsRef<KeyBytes> for EntryView<TKey, TVal> {
fn as_ref(&self) -> &KeyBytes {
self.node.key.as_ref()
}
}
@ -88,17 +88,18 @@ pub enum Entry<'a, TPeerId, TVal> {
/// The internal representation of the different states of an `Entry`,
/// referencing the associated key and bucket.
#[derive(Debug)]
struct EntryRef<'a, TPeerId, TVal> {
bucket: &'a mut KBucket<TPeerId, TVal>,
key: &'a Key<TPeerId>,
struct EntryRef<'a, TKey, TVal> {
bucket: &'a mut KBucket<TKey, TVal>,
key: &'a TKey,
}
impl<'a, TPeerId, TVal> Entry<'a, TPeerId, TVal>
impl<'a, TKey, TVal> Entry<'a, TKey, TVal>
where
TPeerId: Clone,
TKey: Clone + AsRef<KeyBytes>,
TVal: Clone
{
/// Creates a new `Entry` for a `Key`, encapsulating access to a bucket.
pub(super) fn new(bucket: &'a mut KBucket<TPeerId, TVal>, key: &'a Key<TPeerId>) -> Self {
pub(super) fn new(bucket: &'a mut KBucket<TKey, TVal>, key: &'a TKey) -> Self {
if let Some(pos) = bucket.position(key) {
let status = bucket.status(pos);
Entry::Present(PresentEntry::new(bucket, key), status)
@ -114,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, TPeerId, TVal>> {
pub fn view(&'a mut self) -> Option<EntryRefView<'a, TKey, TVal>> {
match self {
Entry::Present(entry, status) => Some(EntryRefView {
node: NodeRefView {
@ -139,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<&Key<TPeerId>> {
pub fn key(&self) -> Option<&TKey> {
match self {
Entry::Present(entry, _) => Some(entry.key()),
Entry::Pending(entry, _) => Some(entry.key()),
@ -150,7 +151,7 @@ where
/// Returns the value associated with the entry.
///
/// Returns `None` if the entry absent from any bucket or refers to the
/// Returns `None` if the entry is absent from any bucket or refers to the
/// local node.
pub fn value(&mut self) -> Option<&mut TVal> {
match self {
@ -164,18 +165,19 @@ where
/// An entry present in a bucket.
#[derive(Debug)]
pub struct PresentEntry<'a, TPeerId, TVal>(EntryRef<'a, TPeerId, TVal>);
pub struct PresentEntry<'a, TKey, TVal>(EntryRef<'a, TKey, TVal>);
impl<'a, TPeerId, TVal> PresentEntry<'a, TPeerId, TVal>
impl<'a, TKey, TVal> PresentEntry<'a, TKey, TVal>
where
TPeerId: Clone,
TKey: Clone + AsRef<KeyBytes>,
TVal: Clone
{
fn new(bucket: &'a mut KBucket<TPeerId, TVal>, key: &'a Key<TPeerId>) -> Self {
fn new(bucket: &'a mut KBucket<TKey, TVal>, key: &'a TKey) -> Self {
PresentEntry(EntryRef { bucket, key })
}
/// Returns the key of the entry.
pub fn key(&self) -> &Key<TPeerId> {
pub fn key(&self) -> &TKey {
self.0.key
}
@ -196,18 +198,19 @@ where
/// An entry waiting for a slot to be available in a bucket.
#[derive(Debug)]
pub struct PendingEntry<'a, TPeerId, TVal>(EntryRef<'a, TPeerId, TVal>);
pub struct PendingEntry<'a, TKey, TVal>(EntryRef<'a, TKey, TVal>);
impl<'a, TPeerId, TVal> PendingEntry<'a, TPeerId, TVal>
impl<'a, TKey, TVal> PendingEntry<'a, TKey, TVal>
where
TPeerId: Clone,
TKey: Clone + AsRef<KeyBytes>,
TVal: Clone
{
fn new(bucket: &'a mut KBucket<TPeerId, TVal>, key: &'a Key<TPeerId>) -> Self {
fn new(bucket: &'a mut KBucket<TKey, TVal>, key: &'a TKey) -> Self {
PendingEntry(EntryRef { bucket, key })
}
/// Returns the key of the entry.
pub fn key(&self) -> &Key<TPeerId> {
pub fn key(&self) -> &TKey {
self.0.key
}
@ -220,7 +223,7 @@ where
}
/// Updates the status of the pending entry.
pub fn update(self, status: NodeStatus) -> PendingEntry<'a, TPeerId, TVal> {
pub fn update(self, status: NodeStatus) -> PendingEntry<'a, TKey, TVal> {
self.0.bucket.update_pending(status);
PendingEntry::new(self.0.bucket, self.0.key)
}
@ -228,23 +231,24 @@ where
/// An entry that is not present in any bucket.
#[derive(Debug)]
pub struct AbsentEntry<'a, TPeerId, TVal>(EntryRef<'a, TPeerId, TVal>);
pub struct AbsentEntry<'a, TKey, TVal>(EntryRef<'a, TKey, TVal>);
impl<'a, TPeerId, TVal> AbsentEntry<'a, TPeerId, TVal>
impl<'a, TKey, TVal> AbsentEntry<'a, TKey, TVal>
where
TPeerId: Clone,
TKey: Clone + AsRef<KeyBytes>,
TVal: Clone
{
fn new(bucket: &'a mut KBucket<TPeerId, TVal>, key: &'a Key<TPeerId>) -> Self {
fn new(bucket: &'a mut KBucket<TKey, TVal>, key: &'a TKey) -> Self {
AbsentEntry(EntryRef { bucket, key })
}
/// Returns the key of the entry.
pub fn key(&self) -> &Key<TPeerId> {
pub fn key(&self) -> &TKey {
self.0.key
}
/// Attempts to insert the entry into a bucket.
pub fn insert(self, value: TVal, status: NodeStatus) -> InsertResult<TPeerId> {
pub fn insert(self, value: TVal, status: NodeStatus) -> InsertResult<TKey> {
self.0.bucket.insert(Node {
key: self.0.key.clone(),
value

View File

@ -21,67 +21,26 @@
use bigint::U256;
use libp2p_core::PeerId;
use multihash::Multihash;
use sha2::{Digest, Sha256, digest::generic_array::{GenericArray, typenum::U32}};
use sha2::{Digest, Sha256};
use sha2::digest::generic_array::{GenericArray, typenum::U32};
use std::hash::{Hash, Hasher};
/// A `Key` identifies both the nodes participating in the Kademlia DHT, as well as
/// records stored in the DHT.
/// A `Key` in the DHT keyspace with preserved preimage.
///
/// The set of all `Key`s defines the Kademlia keyspace.
/// Keys in the DHT keyspace identify both the participating nodes, as well as
/// the records stored in the DHT.
///
/// `Key`s have an XOR metric as defined in the Kademlia paper, i.e. the bitwise XOR of
/// the hash digests, interpreted as an integer. See [`Key::distance`].
///
/// A `Key` preserves the preimage of type `T` of the hash function. See [`Key::preimage`].
#[derive(Clone, Debug)]
pub struct Key<T> {
preimage: T,
bytes: KeyBytes,
}
/// The raw bytes of a key in the DHT keyspace.
#[derive(PartialEq, Eq, Clone, Debug)]
pub struct KeyBytes(GenericArray<u8, U32>);
impl KeyBytes {
/// Computes the distance of the keys according to the XOR metric.
pub fn distance<U>(&self, other: &U) -> Distance
where
U: AsRef<KeyBytes>
{
let a = U256::from(self.0.as_ref());
let b = U256::from(other.as_ref().0.as_ref());
Distance(a ^ b)
}
}
impl AsRef<KeyBytes> for KeyBytes {
fn as_ref(&self) -> &KeyBytes {
self
}
}
impl<T> AsRef<KeyBytes> for Key<T> {
fn as_ref(&self) -> &KeyBytes {
&self.bytes
}
}
impl<T, U> PartialEq<Key<U>> for Key<T> {
fn eq(&self, other: &Key<U>) -> bool {
self.bytes == other.bytes
}
}
impl<T> Eq for Key<T> {}
impl<TPeerId> AsRef<Key<TPeerId>> for Key<TPeerId> {
fn as_ref(&self) -> &Key<TPeerId> {
self
}
}
impl<T> Key<T> {
/// Construct a new `Key` by hashing the bytes of the given `preimage`.
/// Constructs a new `Key` by running the given value through a random
/// oracle.
///
/// The preimage of type `T` is preserved. See [`Key::preimage`] and
/// [`Key::into_preimage`].
@ -89,20 +48,10 @@ impl<T> Key<T> {
where
T: AsRef<[u8]>
{
let bytes = KeyBytes(Sha256::digest(preimage.as_ref()));
let bytes = KeyBytes::new(&preimage);
Key { preimage, bytes }
}
/// Returns the uniquely determined key with the given distance to `self`.
///
/// This implements the following equivalence:
///
/// `self xor other = distance <==> other = self xor distance`
pub fn for_distance(&self, d: Distance) -> KeyBytes {
let key_int = U256::from(self.bytes.0.as_ref()) ^ d.0;
KeyBytes(GenericArray::from(<[u8; 32]>::from(key_int)))
}
/// Borrows the preimage of the key.
pub fn preimage(&self) -> &T {
&self.preimage
@ -120,6 +69,15 @@ impl<T> Key<T> {
{
self.bytes.distance(other)
}
/// Returns the uniquely determined key with the given distance to `self`.
///
/// This implements the following equivalence:
///
/// `self xor other = distance <==> other = self xor distance`
pub fn for_distance(&self, d: Distance) -> KeyBytes {
self.bytes.for_distance(d)
}
}
impl<T> Into<KeyBytes> for Key<T> {
@ -140,7 +98,68 @@ impl From<PeerId> for Key<PeerId> {
}
}
/// A distance between two `Key`s.
impl<T> AsRef<KeyBytes> for Key<T> {
fn as_ref(&self) -> &KeyBytes {
&self.bytes
}
}
impl<T, U> PartialEq<Key<U>> for Key<T> {
fn eq(&self, other: &Key<U>) -> bool {
self.bytes == other.bytes
}
}
impl<T> Eq for Key<T> {}
impl<T> Hash for Key<T> {
fn hash<H: Hasher>(&self, state: &mut H) {
self.bytes.0.hash(state);
}
}
/// The raw bytes of a key in the DHT keyspace.
#[derive(PartialEq, Eq, Clone, Debug)]
pub struct KeyBytes(GenericArray<u8, U32>);
impl KeyBytes {
/// Creates a new key in the DHT keyspace by running the given
/// value through a random oracle.
pub fn new<T>(value: T) -> Self
where
T: AsRef<[u8]>
{
KeyBytes(Sha256::digest(value.as_ref()))
}
/// Computes the distance of the keys according to the XOR metric.
pub fn distance<U>(&self, other: &U) -> Distance
where
U: AsRef<KeyBytes>
{
let a = U256::from(self.0.as_ref());
let b = U256::from(other.as_ref().0.as_ref());
Distance(a ^ b)
}
/// Returns the uniquely determined key with the given distance to `self`.
///
/// This implements the following equivalence:
///
/// `self xor other = distance <==> other = self xor distance`
pub fn for_distance(&self, d: Distance) -> KeyBytes {
let key_int = U256::from(self.0.as_ref()) ^ d.0;
KeyBytes(GenericArray::from(<[u8; 32]>::from(key_int)))
}
}
impl AsRef<KeyBytes> for KeyBytes {
fn as_ref(&self) -> &KeyBytes {
self
}
}
/// A distance between two keys in the DHT keyspace.
#[derive(Copy, Clone, PartialEq, Eq, Default, PartialOrd, Ord, Debug)]
pub struct Distance(pub(super) bigint::U256);
@ -148,6 +167,7 @@ pub struct Distance(pub(super) bigint::U256);
mod tests {
use super::*;
use quickcheck::*;
use multihash::Hash::SHA2256;
impl Arbitrary for Key<PeerId> {
fn arbitrary<G: Gen>(_: &mut G) -> Key<PeerId> {
@ -155,6 +175,12 @@ mod tests {
}
}
impl Arbitrary for Key<Multihash> {
fn arbitrary<G: Gen>(_: &mut G) -> Key<Multihash> {
Key::from(Multihash::random(SHA2256))
}
}
#[test]
fn identity() {
fn prop(a: Key<PeerId>) -> bool {

View File

@ -31,9 +31,11 @@ pub mod record;
mod addresses;
mod behaviour;
mod jobs;
mod protobuf_structs;
mod query;
pub use addresses::Addresses;
pub use behaviour::{Kademlia, KademliaConfig, KademliaEvent, Quorum};
pub use behaviour::{
BootstrapResult,
@ -61,9 +63,9 @@ pub use behaviour::{
GetProvidersError,
};
pub use protocol::KadConnectionType;
pub use record::{RecordStore, RecordStorageError, MemoryRecordStorage};
pub use record::{store, Record, ProviderRecord};
use std::time::Duration;
use std::num::NonZeroUsize;
/// The `k` parameter of the Kademlia specification.
///
@ -79,7 +81,7 @@ use std::time::Duration;
/// DHT should agree on the choices made for (1) and (2).
///
/// The current value is `20`.
pub const K_VALUE: usize = 20;
pub const K_VALUE: NonZeroUsize = unsafe { NonZeroUsize::new_unchecked(20) };
/// The `α` parameter of the Kademlia specification.
///
@ -89,8 +91,5 @@ pub const K_VALUE: usize = 20;
/// locating the closest peers to a key.
///
/// The current value is `3`.
pub const ALPHA_VALUE: usize = 3;
const KBUCKET_PENDING_TIMEOUT: Duration = Duration::from_secs(60);
const ADD_PROVIDER_INTERVAL: Duration = Duration::from_secs(60);
pub const ALPHA_VALUE: NonZeroUsize = unsafe { NonZeroUsize::new_unchecked(3) };

View File

@ -27,6 +27,8 @@ pub struct Record {
pub key: ::std::vec::Vec<u8>,
pub value: ::std::vec::Vec<u8>,
pub timeReceived: ::std::string::String,
pub publisher: ::std::vec::Vec<u8>,
pub ttl: u32,
// special fields
pub unknown_fields: ::protobuf::UnknownFields,
pub cached_size: ::protobuf::CachedSize,
@ -120,6 +122,47 @@ impl Record {
pub fn take_timeReceived(&mut self) -> ::std::string::String {
::std::mem::replace(&mut self.timeReceived, ::std::string::String::new())
}
// bytes publisher = 666;
pub fn get_publisher(&self) -> &[u8] {
&self.publisher
}
pub fn clear_publisher(&mut self) {
self.publisher.clear();
}
// Param is passed by value, moved
pub fn set_publisher(&mut self, v: ::std::vec::Vec<u8>) {
self.publisher = v;
}
// Mutable pointer to the field.
// If field is not initialized, it is initialized with default value first.
pub fn mut_publisher(&mut self) -> &mut ::std::vec::Vec<u8> {
&mut self.publisher
}
// Take field
pub fn take_publisher(&mut self) -> ::std::vec::Vec<u8> {
::std::mem::replace(&mut self.publisher, ::std::vec::Vec::new())
}
// uint32 ttl = 777;
pub fn get_ttl(&self) -> u32 {
self.ttl
}
pub fn clear_ttl(&mut self) {
self.ttl = 0;
}
// Param is passed by value, moved
pub fn set_ttl(&mut self, v: u32) {
self.ttl = v;
}
}
impl ::protobuf::Message for Record {
@ -140,6 +183,16 @@ impl ::protobuf::Message for Record {
5 => {
::protobuf::rt::read_singular_proto3_string_into(wire_type, is, &mut self.timeReceived)?;
},
666 => {
::protobuf::rt::read_singular_proto3_bytes_into(wire_type, is, &mut self.publisher)?;
},
777 => {
if wire_type != ::protobuf::wire_format::WireTypeVarint {
return ::std::result::Result::Err(::protobuf::rt::unexpected_wire_type(wire_type));
}
let tmp = is.read_uint32()?;
self.ttl = tmp;
},
_ => {
::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?;
},
@ -161,6 +214,12 @@ impl ::protobuf::Message for Record {
if !self.timeReceived.is_empty() {
my_size += ::protobuf::rt::string_size(5, &self.timeReceived);
}
if !self.publisher.is_empty() {
my_size += ::protobuf::rt::bytes_size(666, &self.publisher);
}
if self.ttl != 0 {
my_size += ::protobuf::rt::value_size(777, self.ttl, ::protobuf::wire_format::WireTypeVarint);
}
my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields());
self.cached_size.set(my_size);
my_size
@ -176,6 +235,12 @@ impl ::protobuf::Message for Record {
if !self.timeReceived.is_empty() {
os.write_string(5, &self.timeReceived)?;
}
if !self.publisher.is_empty() {
os.write_bytes(666, &self.publisher)?;
}
if self.ttl != 0 {
os.write_uint32(777, self.ttl)?;
}
os.write_unknown_fields(self.get_unknown_fields())?;
::std::result::Result::Ok(())
}
@ -233,6 +298,16 @@ impl ::protobuf::Message for Record {
|m: &Record| { &m.timeReceived },
|m: &mut Record| { &mut m.timeReceived },
));
fields.push(::protobuf::reflect::accessor::make_simple_field_accessor::<_, ::protobuf::types::ProtobufTypeBytes>(
"publisher",
|m: &Record| { &m.publisher },
|m: &mut Record| { &mut m.publisher },
));
fields.push(::protobuf::reflect::accessor::make_simple_field_accessor::<_, ::protobuf::types::ProtobufTypeUint32>(
"ttl",
|m: &Record| { &m.ttl },
|m: &mut Record| { &mut m.ttl },
));
::protobuf::reflect::MessageDescriptor::new::<Record>(
"Record",
fields,
@ -258,6 +333,8 @@ impl ::protobuf::Clear for Record {
self.key.clear();
self.value.clear();
self.timeReceived.clear();
self.publisher.clear();
self.ttl = 0;
self.unknown_fields.clear();
}
}
@ -1034,123 +1111,134 @@ impl ::protobuf::reflect::ProtobufValue for Message_ConnectionType {
}
static file_descriptor_proto_data: &'static [u8] = b"\
\n\tdht.proto\x12\x06dht.pb\"T\n\x06Record\x12\x10\n\x03key\x18\x01\x20\
\x01(\x0cR\x03key\x12\x14\n\x05value\x18\x02\x20\x01(\x0cR\x05value\x12\
\"\n\x0ctimeReceived\x18\x05\x20\x01(\tR\x0ctimeReceived\"\xc4\x04\n\x07\
Message\x12/\n\x04type\x18\x01\x20\x01(\x0e2\x1b.dht.pb.Message.MessageT\
ypeR\x04type\x12(\n\x0fclusterLevelRaw\x18\n\x20\x01(\x05R\x0fclusterLev\
elRaw\x12\x10\n\x03key\x18\x02\x20\x01(\x0cR\x03key\x12&\n\x06record\x18\
\x03\x20\x01(\x0b2\x0e.dht.pb.RecordR\x06record\x126\n\x0bcloserPeers\
\x18\x08\x20\x03(\x0b2\x14.dht.pb.Message.PeerR\x0bcloserPeers\x12:\n\rp\
roviderPeers\x18\t\x20\x03(\x0b2\x14.dht.pb.Message.PeerR\rproviderPeers\
\x1al\n\x04Peer\x12\x0e\n\x02id\x18\x01\x20\x01(\x0cR\x02id\x12\x14\n\
\x05addrs\x18\x02\x20\x03(\x0cR\x05addrs\x12>\n\nconnection\x18\x03\x20\
\x01(\x0e2\x1e.dht.pb.Message.ConnectionTypeR\nconnection\"i\n\x0bMessag\
eType\x12\r\n\tPUT_VALUE\x10\0\x12\r\n\tGET_VALUE\x10\x01\x12\x10\n\x0cA\
DD_PROVIDER\x10\x02\x12\x11\n\rGET_PROVIDERS\x10\x03\x12\r\n\tFIND_NODE\
\x10\x04\x12\x08\n\x04PING\x10\x05\"W\n\x0eConnectionType\x12\x11\n\rNOT\
_CONNECTED\x10\0\x12\r\n\tCONNECTED\x10\x01\x12\x0f\n\x0bCAN_CONNECT\x10\
\x02\x12\x12\n\x0eCANNOT_CONNECT\x10\x03J\x91\x16\n\x06\x12\x04\0\0P\x01\
\n\x08\n\x01\x0c\x12\x03\0\0\x12\n\x08\n\x01\x02\x12\x03\x01\x08\x0e\nX\
\n\x02\x04\0\x12\x04\x05\0\x14\x01\x1aL\x20Record\x20represents\x20a\x20\
dht\x20record\x20that\x20contains\x20a\x20value\n\x20for\x20a\x20key\x20\
value\x20pair\n\n\n\n\x03\x04\0\x01\x12\x03\x05\x08\x0e\n2\n\x04\x04\0\
\x02\0\x12\x03\x07\x08\x16\x1a%\x20The\x20key\x20that\x20references\x20t\
his\x20record\n\n\r\n\x05\x04\0\x02\0\x04\x12\x04\x07\x08\x05\x10\n\x0c\
\n\x05\x04\0\x02\0\x05\x12\x03\x07\x08\r\n\x0c\n\x05\x04\0\x02\0\x01\x12\
\x03\x07\x0e\x11\n\x0c\n\x05\x04\0\x02\0\x03\x12\x03\x07\x14\x15\n6\n\
\x04\x04\0\x02\x01\x12\x03\n\x08\x18\x1a)\x20The\x20actual\x20value\x20t\
his\x20record\x20is\x20storing\n\n\r\n\x05\x04\0\x02\x01\x04\x12\x04\n\
\x08\x07\x16\n\x0c\n\x05\x04\0\x02\x01\x05\x12\x03\n\x08\r\n\x0c\n\x05\
\x04\0\x02\x01\x01\x12\x03\n\x0e\x13\n\x0c\n\x05\x04\0\x02\x01\x03\x12\
\x03\n\x16\x17\n\xfc\x01\n\x04\x04\0\x02\x02\x12\x03\x13\x08\x20\x1a/\
\x20Time\x20the\x20record\x20was\x20received,\x20set\x20by\x20receiver\n\
2\xbd\x01\x20Note:\x20These\x20fields\x20were\x20removed\x20from\x20the\
\x20Record\x20message\n\x20hash\x20of\x20the\x20authors\x20public\x20key\
\noptional\x20string\x20author\x20=\x203;\n\x20A\x20PKI\x20signature\x20\
for\x20the\x20key+value+author\noptional\x20bytes\x20signature\x20=\x204\
;\n\n\r\n\x05\x04\0\x02\x02\x04\x12\x04\x13\x08\n\x18\n\x0c\n\x05\x04\0\
\x02\x02\x05\x12\x03\x13\x08\x0e\n\x0c\n\x05\x04\0\x02\x02\x01\x12\x03\
\x13\x0f\x1b\n\x0c\n\x05\x04\0\x02\x02\x03\x12\x03\x13\x1e\x1f\n\n\n\x02\
\x04\x01\x12\x04\x16\0P\x01\n\n\n\x03\x04\x01\x01\x12\x03\x16\x08\x0f\n\
\x0c\n\x04\x04\x01\x04\0\x12\x04\x17\x08\x1e\t\n\x0c\n\x05\x04\x01\x04\0\
\x01\x12\x03\x17\r\x18\n\r\n\x06\x04\x01\x04\0\x02\0\x12\x03\x18\x10\x1e\
\n\x0e\n\x07\x04\x01\x04\0\x02\0\x01\x12\x03\x18\x10\x19\n\x0e\n\x07\x04\
\x01\x04\0\x02\0\x02\x12\x03\x18\x1c\x1d\n\r\n\x06\x04\x01\x04\0\x02\x01\
\x12\x03\x19\x10\x1e\n\x0e\n\x07\x04\x01\x04\0\x02\x01\x01\x12\x03\x19\
\x10\x19\n\x0e\n\x07\x04\x01\x04\0\x02\x01\x02\x12\x03\x19\x1c\x1d\n\r\n\
\x06\x04\x01\x04\0\x02\x02\x12\x03\x1a\x10!\n\x0e\n\x07\x04\x01\x04\0\
\x02\x02\x01\x12\x03\x1a\x10\x1c\n\x0e\n\x07\x04\x01\x04\0\x02\x02\x02\
\x12\x03\x1a\x1f\x20\n\r\n\x06\x04\x01\x04\0\x02\x03\x12\x03\x1b\x10\"\n\
\x0e\n\x07\x04\x01\x04\0\x02\x03\x01\x12\x03\x1b\x10\x1d\n\x0e\n\x07\x04\
\x01\x04\0\x02\x03\x02\x12\x03\x1b\x20!\n\r\n\x06\x04\x01\x04\0\x02\x04\
\x12\x03\x1c\x10\x1e\n\x0e\n\x07\x04\x01\x04\0\x02\x04\x01\x12\x03\x1c\
\x10\x19\n\x0e\n\x07\x04\x01\x04\0\x02\x04\x02\x12\x03\x1c\x1c\x1d\n\r\n\
\x06\x04\x01\x04\0\x02\x05\x12\x03\x1d\x10\x19\n\x0e\n\x07\x04\x01\x04\0\
\x02\x05\x01\x12\x03\x1d\x10\x14\n\x0e\n\x07\x04\x01\x04\0\x02\x05\x02\
\x12\x03\x1d\x17\x18\n\x0c\n\x04\x04\x01\x04\x01\x12\x04\x20\x08-\t\n\
\x0c\n\x05\x04\x01\x04\x01\x01\x12\x03\x20\r\x1b\n^\n\x06\x04\x01\x04\
\x01\x02\0\x12\x03\"\x10\"\x1aO\x20sender\x20does\x20not\x20have\x20a\
\x20connection\x20to\x20peer,\x20and\x20no\x20extra\x20information\x20(d\
efault)\n\n\x0e\n\x07\x04\x01\x04\x01\x02\0\x01\x12\x03\"\x10\x1d\n\x0e\
\n\x07\x04\x01\x04\x01\x02\0\x02\x12\x03\"\x20!\n5\n\x06\x04\x01\x04\x01\
\x02\x01\x12\x03%\x10\x1e\x1a&\x20sender\x20has\x20a\x20live\x20connecti\
on\x20to\x20peer\n\n\x0e\n\x07\x04\x01\x04\x01\x02\x01\x01\x12\x03%\x10\
\x19\n\x0e\n\x07\x04\x01\x04\x01\x02\x01\x02\x12\x03%\x1c\x1d\n2\n\x06\
\x04\x01\x04\x01\x02\x02\x12\x03(\x10\x20\x1a#\x20sender\x20recently\x20\
connected\x20to\x20peer\n\n\x0e\n\x07\x04\x01\x04\x01\x02\x02\x01\x12\
\x03(\x10\x1b\n\x0e\n\x07\x04\x01\x04\x01\x02\x02\x02\x12\x03(\x1e\x1f\n\
\xa7\x01\n\x06\x04\x01\x04\x01\x02\x03\x12\x03,\x10#\x1a\x97\x01\x20send\
er\x20recently\x20tried\x20to\x20connect\x20to\x20peer\x20repeatedly\x20\
but\x20failed\x20to\x20connect\n\x20(\"try\"\x20here\x20is\x20loose,\x20\
but\x20this\x20should\x20signal\x20\"made\x20strong\x20effort,\x20failed\
\")\n\n\x0e\n\x07\x04\x01\x04\x01\x02\x03\x01\x12\x03,\x10\x1e\n\x0e\n\
\x07\x04\x01\x04\x01\x02\x03\x02\x12\x03,!\"\n\x0c\n\x04\x04\x01\x03\0\
\x12\x04/\x088\t\n\x0c\n\x05\x04\x01\x03\0\x01\x12\x03/\x10\x14\n$\n\x06\
\x04\x01\x03\0\x02\0\x12\x031\x10\x1d\x1a\x15\x20ID\x20of\x20a\x20given\
\x20peer.\n\n\x0f\n\x07\x04\x01\x03\0\x02\0\x04\x12\x041\x10/\x16\n\x0e\
\n\x07\x04\x01\x03\0\x02\0\x05\x12\x031\x10\x15\n\x0e\n\x07\x04\x01\x03\
\0\x02\0\x01\x12\x031\x16\x18\n\x0e\n\x07\x04\x01\x03\0\x02\0\x03\x12\
\x031\x1b\x1c\n,\n\x06\x04\x01\x03\0\x02\x01\x12\x034\x10)\x1a\x1d\x20mu\
ltiaddrs\x20for\x20a\x20given\x20peer\n\n\x0e\n\x07\x04\x01\x03\0\x02\
\x01\x04\x12\x034\x10\x18\n\x0e\n\x07\x04\x01\x03\0\x02\x01\x05\x12\x034\
\x19\x1e\n\x0e\n\x07\x04\x01\x03\0\x02\x01\x01\x12\x034\x1f$\n\x0e\n\x07\
\x04\x01\x03\0\x02\x01\x03\x12\x034'(\nP\n\x06\x04\x01\x03\0\x02\x02\x12\
\x037\x10.\x1aA\x20used\x20to\x20signal\x20the\x20sender's\x20connection\
\x20capabilities\x20to\x20the\x20peer\n\n\x0f\n\x07\x04\x01\x03\0\x02\
\x02\x04\x12\x047\x104)\n\x0e\n\x07\x04\x01\x03\0\x02\x02\x06\x12\x037\
\x10\x1e\n\x0e\n\x07\x04\x01\x03\0\x02\x02\x01\x12\x037\x1f)\n\x0e\n\x07\
\x04\x01\x03\0\x02\x02\x03\x12\x037,-\n2\n\x04\x04\x01\x02\0\x12\x03;\
\x08\x1d\x1a%\x20defines\x20what\x20type\x20of\x20message\x20it\x20is.\n\
\n\r\n\x05\x04\x01\x02\0\x04\x12\x04;\x088\t\n\x0c\n\x05\x04\x01\x02\0\
\x06\x12\x03;\x08\x13\n\x0c\n\x05\x04\x01\x02\0\x01\x12\x03;\x14\x18\n\
\x0c\n\x05\x04\x01\x02\0\x03\x12\x03;\x1b\x1c\n\x9f\x01\n\x04\x04\x01\
\x02\x01\x12\x03?\x08#\x1a\x85\x01\x20defines\x20what\x20coral\x20cluste\
r\x20level\x20this\x20query/response\x20belongs\x20to.\n\x20in\x20case\
\x20we\x20want\x20to\x20implement\x20coral's\x20cluster\x20rings\x20in\
\n\tdht.proto\x12\x06dht.pb\"\x86\x01\n\x06Record\x12\x10\n\x03key\x18\
\x01\x20\x01(\x0cR\x03key\x12\x14\n\x05value\x18\x02\x20\x01(\x0cR\x05va\
lue\x12\"\n\x0ctimeReceived\x18\x05\x20\x01(\tR\x0ctimeReceived\x12\x1d\
\n\tpublisher\x18\x9a\x05\x20\x01(\x0cR\tpublisher\x12\x11\n\x03ttl\x18\
\x89\x06\x20\x01(\rR\x03ttl\"\xc4\x04\n\x07Message\x12/\n\x04type\x18\
\x01\x20\x01(\x0e2\x1b.dht.pb.Message.MessageTypeR\x04type\x12(\n\x0fclu\
sterLevelRaw\x18\n\x20\x01(\x05R\x0fclusterLevelRaw\x12\x10\n\x03key\x18\
\x02\x20\x01(\x0cR\x03key\x12&\n\x06record\x18\x03\x20\x01(\x0b2\x0e.dht\
.pb.RecordR\x06record\x126\n\x0bcloserPeers\x18\x08\x20\x03(\x0b2\x14.dh\
t.pb.Message.PeerR\x0bcloserPeers\x12:\n\rproviderPeers\x18\t\x20\x03(\
\x0b2\x14.dht.pb.Message.PeerR\rproviderPeers\x1al\n\x04Peer\x12\x0e\n\
\x02id\x18\x01\x20\x01(\x0cR\x02id\x12\x14\n\x05addrs\x18\x02\x20\x03(\
\x0cR\x05addrs\x12>\n\nconnection\x18\x03\x20\x01(\x0e2\x1e.dht.pb.Messa\
ge.ConnectionTypeR\nconnection\"i\n\x0bMessageType\x12\r\n\tPUT_VALUE\
\x10\0\x12\r\n\tGET_VALUE\x10\x01\x12\x10\n\x0cADD_PROVIDER\x10\x02\x12\
\x11\n\rGET_PROVIDERS\x10\x03\x12\r\n\tFIND_NODE\x10\x04\x12\x08\n\x04PI\
NG\x10\x05\"W\n\x0eConnectionType\x12\x11\n\rNOT_CONNECTED\x10\0\x12\r\n\
\tCONNECTED\x10\x01\x12\x0f\n\x0bCAN_CONNECT\x10\x02\x12\x12\n\x0eCANNOT\
_CONNECT\x10\x03J\xbe\x18\n\x06\x12\x04\0\0X\x01\n\x08\n\x01\x0c\x12\x03\
\0\0\x12\n\x08\n\x01\x02\x12\x03\x01\x08\x0e\nX\n\x02\x04\0\x12\x04\x05\
\0\x1c\x01\x1aL\x20Record\x20represents\x20a\x20dht\x20record\x20that\
\x20contains\x20a\x20value\n\x20for\x20a\x20key\x20value\x20pair\n\n\n\n\
\x03\x04\0\x01\x12\x03\x05\x08\x0e\n2\n\x04\x04\0\x02\0\x12\x03\x07\x08\
\x16\x1a%\x20The\x20key\x20that\x20references\x20this\x20record\n\n\r\n\
\x05\x04\0\x02\0\x04\x12\x04\x07\x08\x05\x10\n\x0c\n\x05\x04\0\x02\0\x05\
\x12\x03\x07\x08\r\n\x0c\n\x05\x04\0\x02\0\x01\x12\x03\x07\x0e\x11\n\x0c\
\n\x05\x04\0\x02\0\x03\x12\x03\x07\x14\x15\n6\n\x04\x04\0\x02\x01\x12\
\x03\n\x08\x18\x1a)\x20The\x20actual\x20value\x20this\x20record\x20is\
\x20storing\n\n\r\n\x05\x04\0\x02\x01\x04\x12\x04\n\x08\x07\x16\n\x0c\n\
\x05\x04\0\x02\x01\x05\x12\x03\n\x08\r\n\x0c\n\x05\x04\0\x02\x01\x01\x12\
\x03\n\x0e\x13\n\x0c\n\x05\x04\0\x02\x01\x03\x12\x03\n\x16\x17\n\xfc\x01\
\n\x04\x04\0\x02\x02\x12\x03\x13\x08\x20\x1a/\x20Time\x20the\x20record\
\x20was\x20received,\x20set\x20by\x20receiver\n2\xbd\x01\x20Note:\x20The\
se\x20fields\x20were\x20removed\x20from\x20the\x20Record\x20message\n\
\x20hash\x20of\x20the\x20authors\x20public\x20key\noptional\x20string\
\x20author\x20=\x203;\n\x20A\x20PKI\x20signature\x20for\x20the\x20key+va\
lue+author\noptional\x20bytes\x20signature\x20=\x204;\n\n\r\n\x05\x04\0\
\x02\x02\x04\x12\x04\x13\x08\n\x18\n\x0c\n\x05\x04\0\x02\x02\x05\x12\x03\
\x13\x08\x0e\n\x0c\n\x05\x04\0\x02\x02\x01\x12\x03\x13\x0f\x1b\n\x0c\n\
\x05\x04\0\x02\x02\x03\x12\x03\x13\x1e\x1f\nX\n\x04\x04\0\x02\x03\x12\
\x03\x17\x04\x1a\x1aK\x20The\x20original\x20publisher\x20of\x20the\x20re\
cord.\n\x20Currently\x20specific\x20to\x20rust-libp2p.\n\n\r\n\x05\x04\0\
\x02\x03\x04\x12\x04\x17\x04\x13\x20\n\x0c\n\x05\x04\0\x02\x03\x05\x12\
\x03\x17\x04\t\n\x0c\n\x05\x04\0\x02\x03\x01\x12\x03\x17\n\x13\n\x0c\n\
\x05\x04\0\x02\x03\x03\x12\x03\x17\x16\x19\n_\n\x04\x04\0\x02\x04\x12\
\x03\x1b\x04\x15\x1aR\x20The\x20remaining\x20TTL\x20of\x20the\x20record,\
\x20in\x20seconds.\n\x20Currently\x20specific\x20to\x20rust-libp2p.\n\n\
\r\n\x05\x04\0\x02\x04\x04\x12\x04\x1b\x04\x17\x1a\n\x0c\n\x05\x04\0\x02\
\x04\x05\x12\x03\x1b\x04\n\n\x0c\n\x05\x04\0\x02\x04\x01\x12\x03\x1b\x0b\
\x0e\n\x0c\n\x05\x04\0\x02\x04\x03\x12\x03\x1b\x11\x14\n\n\n\x02\x04\x01\
\x12\x04\x1e\0X\x01\n\n\n\x03\x04\x01\x01\x12\x03\x1e\x08\x0f\n\x0c\n\
\x04\x04\x01\x04\0\x12\x04\x1f\x08&\t\n\x0c\n\x05\x04\x01\x04\0\x01\x12\
\x03\x1f\r\x18\n\r\n\x06\x04\x01\x04\0\x02\0\x12\x03\x20\x10\x1e\n\x0e\n\
\x07\x04\x01\x04\0\x02\0\x01\x12\x03\x20\x10\x19\n\x0e\n\x07\x04\x01\x04\
\0\x02\0\x02\x12\x03\x20\x1c\x1d\n\r\n\x06\x04\x01\x04\0\x02\x01\x12\x03\
!\x10\x1e\n\x0e\n\x07\x04\x01\x04\0\x02\x01\x01\x12\x03!\x10\x19\n\x0e\n\
\x07\x04\x01\x04\0\x02\x01\x02\x12\x03!\x1c\x1d\n\r\n\x06\x04\x01\x04\0\
\x02\x02\x12\x03\"\x10!\n\x0e\n\x07\x04\x01\x04\0\x02\x02\x01\x12\x03\"\
\x10\x1c\n\x0e\n\x07\x04\x01\x04\0\x02\x02\x02\x12\x03\"\x1f\x20\n\r\n\
\x06\x04\x01\x04\0\x02\x03\x12\x03#\x10\"\n\x0e\n\x07\x04\x01\x04\0\x02\
\x03\x01\x12\x03#\x10\x1d\n\x0e\n\x07\x04\x01\x04\0\x02\x03\x02\x12\x03#\
\x20!\n\r\n\x06\x04\x01\x04\0\x02\x04\x12\x03$\x10\x1e\n\x0e\n\x07\x04\
\x01\x04\0\x02\x04\x01\x12\x03$\x10\x19\n\x0e\n\x07\x04\x01\x04\0\x02\
\x04\x02\x12\x03$\x1c\x1d\n\r\n\x06\x04\x01\x04\0\x02\x05\x12\x03%\x10\
\x19\n\x0e\n\x07\x04\x01\x04\0\x02\x05\x01\x12\x03%\x10\x14\n\x0e\n\x07\
\x04\x01\x04\0\x02\x05\x02\x12\x03%\x17\x18\n\x0c\n\x04\x04\x01\x04\x01\
\x12\x04(\x085\t\n\x0c\n\x05\x04\x01\x04\x01\x01\x12\x03(\r\x1b\n^\n\x06\
\x04\x01\x04\x01\x02\0\x12\x03*\x10\"\x1aO\x20sender\x20does\x20not\x20h\
ave\x20a\x20connection\x20to\x20peer,\x20and\x20no\x20extra\x20informati\
on\x20(default)\n\n\x0e\n\x07\x04\x01\x04\x01\x02\0\x01\x12\x03*\x10\x1d\
\n\x0e\n\x07\x04\x01\x04\x01\x02\0\x02\x12\x03*\x20!\n5\n\x06\x04\x01\
\x04\x01\x02\x01\x12\x03-\x10\x1e\x1a&\x20sender\x20has\x20a\x20live\x20\
connection\x20to\x20peer\n\n\x0e\n\x07\x04\x01\x04\x01\x02\x01\x01\x12\
\x03-\x10\x19\n\x0e\n\x07\x04\x01\x04\x01\x02\x01\x02\x12\x03-\x1c\x1d\n\
2\n\x06\x04\x01\x04\x01\x02\x02\x12\x030\x10\x20\x1a#\x20sender\x20recen\
tly\x20connected\x20to\x20peer\n\n\x0e\n\x07\x04\x01\x04\x01\x02\x02\x01\
\x12\x030\x10\x1b\n\x0e\n\x07\x04\x01\x04\x01\x02\x02\x02\x12\x030\x1e\
\x1f\n\xa7\x01\n\x06\x04\x01\x04\x01\x02\x03\x12\x034\x10#\x1a\x97\x01\
\x20sender\x20recently\x20tried\x20to\x20connect\x20to\x20peer\x20repeat\
edly\x20but\x20failed\x20to\x20connect\n\x20(\"try\"\x20here\x20is\x20lo\
ose,\x20but\x20this\x20should\x20signal\x20\"made\x20strong\x20effort,\
\x20failed\")\n\n\x0e\n\x07\x04\x01\x04\x01\x02\x03\x01\x12\x034\x10\x1e\
\n\x0e\n\x07\x04\x01\x04\x01\x02\x03\x02\x12\x034!\"\n\x0c\n\x04\x04\x01\
\x03\0\x12\x047\x08@\t\n\x0c\n\x05\x04\x01\x03\0\x01\x12\x037\x10\x14\n$\
\n\x06\x04\x01\x03\0\x02\0\x12\x039\x10\x1d\x1a\x15\x20ID\x20of\x20a\x20\
given\x20peer.\n\n\x0f\n\x07\x04\x01\x03\0\x02\0\x04\x12\x049\x107\x16\n\
\x0e\n\x07\x04\x01\x03\0\x02\0\x05\x12\x039\x10\x15\n\x0e\n\x07\x04\x01\
\x03\0\x02\0\x01\x12\x039\x16\x18\n\x0e\n\x07\x04\x01\x03\0\x02\0\x03\
\x12\x039\x1b\x1c\n,\n\x06\x04\x01\x03\0\x02\x01\x12\x03<\x10)\x1a\x1d\
\x20multiaddrs\x20for\x20a\x20given\x20peer\n\n\x0e\n\x07\x04\x01\x03\0\
\x02\x01\x04\x12\x03<\x10\x18\n\x0e\n\x07\x04\x01\x03\0\x02\x01\x05\x12\
\x03<\x19\x1e\n\x0e\n\x07\x04\x01\x03\0\x02\x01\x01\x12\x03<\x1f$\n\x0e\
\n\x07\x04\x01\x03\0\x02\x01\x03\x12\x03<'(\nP\n\x06\x04\x01\x03\0\x02\
\x02\x12\x03?\x10.\x1aA\x20used\x20to\x20signal\x20the\x20sender's\x20co\
nnection\x20capabilities\x20to\x20the\x20peer\n\n\x0f\n\x07\x04\x01\x03\
\0\x02\x02\x04\x12\x04?\x10<)\n\x0e\n\x07\x04\x01\x03\0\x02\x02\x06\x12\
\x03?\x10\x1e\n\x0e\n\x07\x04\x01\x03\0\x02\x02\x01\x12\x03?\x1f)\n\x0e\
\n\x07\x04\x01\x03\0\x02\x02\x03\x12\x03?,-\n2\n\x04\x04\x01\x02\0\x12\
\x03C\x08\x1d\x1a%\x20defines\x20what\x20type\x20of\x20message\x20it\x20\
is.\n\n\r\n\x05\x04\x01\x02\0\x04\x12\x04C\x08@\t\n\x0c\n\x05\x04\x01\
\x02\0\x06\x12\x03C\x08\x13\n\x0c\n\x05\x04\x01\x02\0\x01\x12\x03C\x14\
\x18\n\x0c\n\x05\x04\x01\x02\0\x03\x12\x03C\x1b\x1c\n\x9f\x01\n\x04\x04\
\x01\x02\x01\x12\x03G\x08#\x1a\x85\x01\x20defines\x20what\x20coral\x20cl\
uster\x20level\x20this\x20query/response\x20belongs\x20to.\n\x20in\x20ca\
se\x20we\x20want\x20to\x20implement\x20coral's\x20cluster\x20rings\x20in\
\x20the\x20future.\n\"\n\x20NOT\x20USED\n\n\r\n\x05\x04\x01\x02\x01\x04\
\x12\x04?\x08;\x1d\n\x0c\n\x05\x04\x01\x02\x01\x05\x12\x03?\x08\r\n\x0c\
\n\x05\x04\x01\x02\x01\x01\x12\x03?\x0e\x1d\n\x0c\n\x05\x04\x01\x02\x01\
\x03\x12\x03?\x20\"\nw\n\x04\x04\x01\x02\x02\x12\x03C\x08\x16\x1aj\x20Us\
\x12\x04G\x08C\x1d\n\x0c\n\x05\x04\x01\x02\x01\x05\x12\x03G\x08\r\n\x0c\
\n\x05\x04\x01\x02\x01\x01\x12\x03G\x0e\x1d\n\x0c\n\x05\x04\x01\x02\x01\
\x03\x12\x03G\x20\"\nw\n\x04\x04\x01\x02\x02\x12\x03K\x08\x16\x1aj\x20Us\
ed\x20to\x20specify\x20the\x20key\x20associated\x20with\x20this\x20messa\
ge.\n\x20PUT_VALUE,\x20GET_VALUE,\x20ADD_PROVIDER,\x20GET_PROVIDERS\n\n\
\r\n\x05\x04\x01\x02\x02\x04\x12\x04C\x08?#\n\x0c\n\x05\x04\x01\x02\x02\
\x05\x12\x03C\x08\r\n\x0c\n\x05\x04\x01\x02\x02\x01\x12\x03C\x0e\x11\n\
\x0c\n\x05\x04\x01\x02\x02\x03\x12\x03C\x14\x15\n;\n\x04\x04\x01\x02\x03\
\x12\x03G\x08\x1a\x1a.\x20Used\x20to\x20return\x20a\x20value\n\x20PUT_VA\
LUE,\x20GET_VALUE\n\n\r\n\x05\x04\x01\x02\x03\x04\x12\x04G\x08C\x16\n\
\x0c\n\x05\x04\x01\x02\x03\x06\x12\x03G\x08\x0e\n\x0c\n\x05\x04\x01\x02\
\x03\x01\x12\x03G\x0f\x15\n\x0c\n\x05\x04\x01\x02\x03\x03\x12\x03G\x18\
\x19\nc\n\x04\x04\x01\x02\x04\x12\x03K\x08&\x1aV\x20Used\x20to\x20return\
\r\n\x05\x04\x01\x02\x02\x04\x12\x04K\x08G#\n\x0c\n\x05\x04\x01\x02\x02\
\x05\x12\x03K\x08\r\n\x0c\n\x05\x04\x01\x02\x02\x01\x12\x03K\x0e\x11\n\
\x0c\n\x05\x04\x01\x02\x02\x03\x12\x03K\x14\x15\n;\n\x04\x04\x01\x02\x03\
\x12\x03O\x08\x1a\x1a.\x20Used\x20to\x20return\x20a\x20value\n\x20PUT_VA\
LUE,\x20GET_VALUE\n\n\r\n\x05\x04\x01\x02\x03\x04\x12\x04O\x08K\x16\n\
\x0c\n\x05\x04\x01\x02\x03\x06\x12\x03O\x08\x0e\n\x0c\n\x05\x04\x01\x02\
\x03\x01\x12\x03O\x0f\x15\n\x0c\n\x05\x04\x01\x02\x03\x03\x12\x03O\x18\
\x19\nc\n\x04\x04\x01\x02\x04\x12\x03S\x08&\x1aV\x20Used\x20to\x20return\
\x20peers\x20closer\x20to\x20a\x20key\x20in\x20a\x20query\n\x20GET_VALUE\
,\x20GET_PROVIDERS,\x20FIND_NODE\n\n\x0c\n\x05\x04\x01\x02\x04\x04\x12\
\x03K\x08\x10\n\x0c\n\x05\x04\x01\x02\x04\x06\x12\x03K\x11\x15\n\x0c\n\
\x05\x04\x01\x02\x04\x01\x12\x03K\x16!\n\x0c\n\x05\x04\x01\x02\x04\x03\
\x12\x03K$%\nO\n\x04\x04\x01\x02\x05\x12\x03O\x08(\x1aB\x20Used\x20to\
\x03S\x08\x10\n\x0c\n\x05\x04\x01\x02\x04\x06\x12\x03S\x11\x15\n\x0c\n\
\x05\x04\x01\x02\x04\x01\x12\x03S\x16!\n\x0c\n\x05\x04\x01\x02\x04\x03\
\x12\x03S$%\nO\n\x04\x04\x01\x02\x05\x12\x03W\x08(\x1aB\x20Used\x20to\
\x20return\x20Providers\n\x20GET_VALUE,\x20ADD_PROVIDER,\x20GET_PROVIDER\
S\n\n\x0c\n\x05\x04\x01\x02\x05\x04\x12\x03O\x08\x10\n\x0c\n\x05\x04\x01\
\x02\x05\x06\x12\x03O\x11\x15\n\x0c\n\x05\x04\x01\x02\x05\x01\x12\x03O\
\x16#\n\x0c\n\x05\x04\x01\x02\x05\x03\x12\x03O&'b\x06proto3\
S\n\n\x0c\n\x05\x04\x01\x02\x05\x04\x12\x03W\x08\x10\n\x0c\n\x05\x04\x01\
\x02\x05\x06\x12\x03W\x11\x15\n\x0c\n\x05\x04\x01\x02\x05\x01\x12\x03W\
\x16#\n\x0c\n\x05\x04\x01\x02\x05\x03\x12\x03W&'b\x06proto3\
";
static mut file_descriptor_proto_lazy: ::protobuf::lazy::Lazy<::protobuf::descriptor::FileDescriptorProto> = ::protobuf::lazy::Lazy {

View File

@ -39,11 +39,12 @@ use libp2p_core::{Multiaddr, PeerId};
use libp2p_core::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo, Negotiated};
use multihash::Multihash;
use protobuf::{self, Message};
use std::{borrow::Cow, convert::TryFrom};
use std::{borrow::Cow, convert::TryFrom, time::Duration};
use std::{io, iter};
use tokio_codec::Framed;
use tokio_io::{AsyncRead, AsyncWrite};
use unsigned_varint::codec;
use wasm_timer::Instant;
/// Status of our connection to a node reported by the Kademlia protocol.
#[derive(Copy, Clone, PartialEq, Eq, Debug, Hash)]
@ -272,7 +273,7 @@ pub enum KadRequestMsg {
/// Key for which we should add providers.
key: Multihash,
/// Known provider for this key.
provider_peer: KadPeer,
provider: KadPeer,
},
/// Request to get a value from the dht records.
@ -283,10 +284,7 @@ pub enum KadRequestMsg {
/// Request to put a value into the dht records.
PutValue {
/// The key of the record.
key: Multihash,
/// The value of the record.
value: Vec<u8>,
record: Record,
}
}
@ -313,7 +311,7 @@ pub enum KadResponseMsg {
/// Response to a `GetValue`.
GetValue {
/// Result that might have been found
result: Option<Record>,
record: Option<Record>,
/// Nodes closest to the key
closer_peers: Vec<KadPeer>,
},
@ -349,12 +347,12 @@ fn req_msg_to_proto(kad_msg: KadRequestMsg) -> proto::Message {
msg.set_clusterLevelRaw(10);
msg
}
KadRequestMsg::AddProvider { key, provider_peer } => {
KadRequestMsg::AddProvider { key, provider } => {
let mut msg = proto::Message::new();
msg.set_field_type(proto::Message_MessageType::ADD_PROVIDER);
msg.set_clusterLevelRaw(10);
msg.set_key(key.into_bytes());
msg.mut_providerPeers().push(provider_peer.into());
msg.mut_providerPeers().push(provider.into());
msg
}
KadRequestMsg::GetValue { key } => {
@ -365,14 +363,10 @@ fn req_msg_to_proto(kad_msg: KadRequestMsg) -> proto::Message {
msg
}
KadRequestMsg::PutValue { key, value} => {
KadRequestMsg::PutValue { record } => {
let mut msg = proto::Message::new();
msg.set_field_type(proto::Message_MessageType::PUT_VALUE);
let mut record = proto::Record::new();
record.set_value(value);
record.set_key(key.into_bytes());
msg.set_record(record);
msg.set_record(record_to_proto(record));
msg
}
}
@ -411,7 +405,7 @@ fn resp_msg_to_proto(kad_msg: KadResponseMsg) -> proto::Message {
msg
}
KadResponseMsg::GetValue {
result,
record,
closer_peers,
} => {
let mut msg = proto::Message::new();
@ -420,12 +414,8 @@ fn resp_msg_to_proto(kad_msg: KadResponseMsg) -> proto::Message {
for peer in closer_peers {
msg.mut_closerPeers().push(peer.into());
}
if let Some(Record{ key, value }) = result {
let mut record = proto::Record::new();
record.set_key(key.into_bytes());
record.set_value(value);
msg.set_record(record);
if let Some(record) = record {
msg.set_record(record_to_proto(record));
}
msg
@ -456,9 +446,8 @@ fn proto_to_req_msg(mut message: proto::Message) -> Result<KadRequestMsg, io::Er
proto::Message_MessageType::PING => Ok(KadRequestMsg::Ping),
proto::Message_MessageType::PUT_VALUE => {
let record = message.mut_record();
let key = Multihash::from_bytes(record.take_key()).map_err(invalid_data)?;
Ok(KadRequestMsg::PutValue { key, value: record.take_value() })
let record = record_from_proto(message.take_record())?;
Ok(KadRequestMsg::PutValue { record })
}
proto::Message_MessageType::GET_VALUE => {
@ -481,14 +470,14 @@ fn proto_to_req_msg(mut message: proto::Message) -> Result<KadRequestMsg, io::Er
// TODO: for now we don't parse the peer properly, so it is possible that we get
// parsing errors for peers even when they are valid; we ignore these
// errors for now, but ultimately we should just error altogether
let provider_peer = message
let provider = message
.mut_providerPeers()
.iter_mut()
.find_map(|peer| KadPeer::try_from(peer).ok());
if let Some(provider_peer) = provider_peer {
if let Some(provider) = provider {
let key = Multihash::from_bytes(message.take_key()).map_err(invalid_data)?;
Ok(KadRequestMsg::AddProvider { key, provider_peer })
Ok(KadRequestMsg::AddProvider { key, provider })
} else {
Err(invalid_data("ADD_PROVIDER message with no valid peer."))
}
@ -504,13 +493,11 @@ fn proto_to_resp_msg(mut message: proto::Message) -> Result<KadResponseMsg, io::
proto::Message_MessageType::PING => Ok(KadResponseMsg::Pong),
proto::Message_MessageType::GET_VALUE => {
let result = match message.has_record() {
true => {
let mut record = message.take_record();
let key = Multihash::from_bytes(record.take_key()).map_err(invalid_data)?;
Some(Record { key, value: record.take_value() })
}
false => None,
let record =
if message.has_record() {
Some(record_from_proto(message.take_record())?)
} else {
None
};
let closer_peers = message
@ -519,7 +506,7 @@ fn proto_to_resp_msg(mut message: proto::Message) -> Result<KadResponseMsg, io::
.filter_map(|peer| KadPeer::try_from(peer).ok())
.collect::<Vec<_>>();
Ok(KadResponseMsg::GetValue { result, closer_peers })
Ok(KadResponseMsg::GetValue { record, closer_peers })
},
proto::Message_MessageType::FIND_NODE => {
@ -569,6 +556,48 @@ fn proto_to_resp_msg(mut message: proto::Message) -> Result<KadResponseMsg, io::
}
}
fn record_from_proto(mut record: proto::Record) -> Result<Record, io::Error> {
let key = Multihash::from_bytes(record.take_key()).map_err(invalid_data)?;
let value = record.take_value();
let publisher =
if record.publisher.len() > 0 {
PeerId::from_bytes(record.take_publisher())
.map(Some)
.map_err(|_| invalid_data("Invalid publisher peer ID."))?
} else {
None
};
let expires =
if record.ttl > 0 {
Some(Instant::now() + Duration::from_secs(record.ttl as u64))
} else {
None
};
Ok(Record { key, value, publisher, expires })
}
fn record_to_proto(record: Record) -> proto::Record {
let mut pb_record = proto::Record::new();
pb_record.key = record.key.into_bytes();
pb_record.value = record.value;
if let Some(p) = record.publisher {
pb_record.publisher = p.into_bytes();
}
if let Some(t) = record.expires {
let now = Instant::now();
if t > now {
pb_record.ttl = (t - now).as_secs() as u32;
} else {
pb_record.ttl = 1; // because 0 means "does not expire"
}
}
pb_record
}
/// Creates an `io::Error` with `io::ErrorKind::InvalidData`.
fn invalid_data<E>(e: E) -> io::Error
where

View File

@ -26,11 +26,10 @@ use peers::fixed::FixedPeersIter;
use crate::K_VALUE;
use crate::kbucket::{Key, KeyBytes};
use crate::handler::KademliaHandlerIn;
use either::Either;
use fnv::FnvHashMap;
use libp2p_core::PeerId;
use std::time::Duration;
use std::{time::Duration, num::NonZeroUsize};
use wasm_timer::Instant;
/// A `QueryPool` provides an aggregate state machine for driving `Query`s to completion.
@ -50,7 +49,7 @@ pub enum QueryPoolState<'a, TInner> {
Idle,
/// At least one query is waiting for results. `Some(request)` indicates
/// that a new request is now being waited on.
Waiting(Option<(&'a Query<TInner>, PeerId)>),
Waiting(Option<(&'a mut Query<TInner>, PeerId)>),
/// A query has finished.
Finished(Query<TInner>),
/// A query has timed out.
@ -77,6 +76,11 @@ impl<TInner> QueryPool<TInner> {
self.queries.values()
}
/// Gets the current size of the pool, i.e. the number of running queries.
pub fn size(&self) -> usize {
self.queries.len()
}
/// Returns an iterator that allows modifying each query in the pool.
pub fn iter_mut(&mut self) -> impl Iterator<Item = &mut Query<TInner>> {
self.queries.values_mut()
@ -88,7 +92,7 @@ impl<TInner> QueryPool<TInner> {
I: IntoIterator<Item = Key<PeerId>>
{
let peers = peers.into_iter().map(|k| k.into_preimage()).collect::<Vec<_>>();
let parallelism = self.config.replication_factor;
let parallelism = self.config.replication_factor.get();
let peer_iter = QueryPeerIter::Fixed(FixedPeersIter::new(peers, parallelism));
self.add(peer_iter, inner)
}
@ -100,7 +104,7 @@ impl<TInner> QueryPool<TInner> {
I: IntoIterator<Item = Key<PeerId>>
{
let cfg = ClosestPeersIterConfig {
num_results: self.config.replication_factor,
num_results: self.config.replication_factor.get(),
.. ClosestPeersIterConfig::default()
};
let peer_iter = QueryPeerIter::Closest(ClosestPeersIter::with_config(cfg, target, peers));
@ -154,7 +158,7 @@ impl<TInner> QueryPool<TInner> {
}
if let Some((query_id, peer_id)) = waiting {
let query = self.queries.get(&query_id).expect("s.a.");
let query = self.queries.get_mut(&query_id).expect("s.a.");
return QueryPoolState::Waiting(Some((query, peer_id)))
}
@ -181,17 +185,17 @@ impl<TInner> QueryPool<TInner> {
pub struct QueryId(usize);
/// The configuration for queries in a `QueryPool`.
#[derive(Clone)]
#[derive(Debug, Clone)]
pub struct QueryConfig {
pub timeout: Duration,
pub replication_factor: usize,
pub replication_factor: NonZeroUsize,
}
impl Default for QueryConfig {
fn default() -> Self {
QueryConfig {
timeout: Duration::from_secs(60),
replication_factor: K_VALUE
replication_factor: NonZeroUsize::new(K_VALUE.get()).expect("K_VALUE > 0")
}
}
}

View File

@ -75,8 +75,8 @@ pub struct ClosestPeersIterConfig {
impl Default for ClosestPeersIterConfig {
fn default() -> Self {
ClosestPeersIterConfig {
parallelism: ALPHA_VALUE as usize,
num_results: K_VALUE as usize,
parallelism: ALPHA_VALUE.get(),
num_results: K_VALUE.get(),
peer_timeout: Duration::from_secs(10),
}
}

View File

@ -20,92 +20,111 @@
//! Records and record storage abstraction of the libp2p Kademlia DHT.
use fnv::FnvHashMap;
use multihash::Multihash;
use std::borrow::Cow;
pub mod store;
/// The records that are kept in the dht.
use libp2p_core::PeerId;
use multihash::Multihash;
use std::hash::{Hash, Hasher};
use wasm_timer::Instant;
/// A record stored in the DHT.
#[derive(Clone, Debug, Eq, PartialEq)]
pub struct Record {
/// Key of the record.
pub key: Multihash,
/// Value of the record.
pub value: Vec<u8>,
/// The (original) publisher of the record.
pub publisher: Option<PeerId>,
/// The expiration time as measured by a local, monotonic clock.
pub expires: Option<Instant>,
}
/// Trait for a record store.
pub trait RecordStore {
fn get(&self, k: &Multihash) -> Option<Cow<Record>>;
fn put(&mut self, r: Record) -> Result<(), RecordStorageError>;
impl Record {
/// Creates a new record for insertion into the DHT.
pub fn new(key: Multihash, value: Vec<u8>) -> Self {
Record {
key,
value,
publisher: None,
expires: None,
}
}
/// The error record store may return
#[derive(Clone, Debug, PartialEq)]
pub enum RecordStorageError {
/// Store reached the capacity limit.
AtCapacity,
/// Value being put is larger than the limit.
ValueTooLarge,
/// Checks whether the record is expired w.r.t. the given `Instant`.
pub fn is_expired(&self, now: Instant) -> bool {
self.expires.map_or(false, |t| now >= t)
}
}
/// In-memory implementation of the record store.
pub struct MemoryRecordStorage {
/// Maximum number of records we will store.
max_records: usize,
/// Maximum size of the record we will store.
max_record_size: usize,
/// The records.
records: FnvHashMap<Multihash, Record>
/// A record stored in the DHT whose value is the ID of a peer
/// who can provide the value on-demand.
#[derive(Clone, Debug, PartialEq, Eq)]
pub struct ProviderRecord {
/// The key whose value is provided by the provider.
pub key: Multihash,
/// The provider of the value for the key.
pub provider: PeerId,
/// The expiration time as measured by a local, monotonic clock.
pub expires: Option<Instant>,
}
impl MemoryRecordStorage {
const MAX_RECORDS: usize = 1024;
const MAX_RECORD_SIZE: usize = 65535;
impl Hash for ProviderRecord {
fn hash<H: Hasher>(&self, state: &mut H) {
self.key.hash(state);
self.provider.hash(state);
}
}
/// Creates a new `MemoryRecordStorage`.
pub fn new(max_records: usize, max_record_size: usize) -> Self {
MemoryRecordStorage{
max_records,
max_record_size,
records: FnvHashMap::default()
impl ProviderRecord {
/// Creates a new provider record for insertion into a `RecordStore`.
pub fn new(key: Multihash, provider: PeerId) -> Self {
ProviderRecord {
key, provider, expires: None
}
}
/// Checks whether the provider record is expired w.r.t. the given `Instant`.
pub fn is_expired(&self, now: Instant) -> bool {
self.expires.map_or(false, |t| now >= t)
}
}
#[cfg(test)]
mod tests {
use super::*;
use quickcheck::*;
use multihash::Hash::SHA2256;
use rand::Rng;
use std::time::Duration;
impl Arbitrary for Record {
fn arbitrary<G: Gen>(g: &mut G) -> Record {
Record {
key: Multihash::random(SHA2256),
value: Vec::arbitrary(g),
publisher: if g.gen() { Some(PeerId::random()) } else { None },
expires: if g.gen() {
Some(Instant::now() + Duration::from_secs(g.gen_range(0, 60)))
} else {
None
},
}
}
}
impl Default for MemoryRecordStorage {
fn default() -> Self {
MemoryRecordStorage::new(Self::MAX_RECORDS, Self::MAX_RECORD_SIZE)
impl Arbitrary for ProviderRecord {
fn arbitrary<G: Gen>(g: &mut G) -> ProviderRecord {
ProviderRecord {
key: Multihash::random(SHA2256),
provider: PeerId::random(),
expires: if g.gen() {
Some(Instant::now() + Duration::from_secs(g.gen_range(0, 60)))
} else {
None
},
}
}
}
}
impl MemoryRecordStorage {
/// Retain the elements by a predicate.
pub fn retain<F>(&mut self, f: F)
where F: FnMut(&Multihash, &mut Record) -> bool
{
self.records.retain(f);
}
}
impl RecordStore for MemoryRecordStorage {
fn get(&self, k: &Multihash) -> Option<Cow<Record>> {
match self.records.get(k) {
Some(rec) => Some(Cow::Borrowed(rec)),
None => None,
}
}
fn put(&mut self, r: Record) -> Result<(), RecordStorageError> {
if self.records.len() >= self.max_records {
return Err(RecordStorageError::AtCapacity);
}
if r.value.len() >= self.max_record_size {
return Err(RecordStorageError::ValueTooLarge)
}
self.records.insert(r.key.clone(), r);
Ok(())
}
}

View File

@ -0,0 +1,95 @@
// Copyright 2019 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
mod memory;
pub use memory::MemoryStore;
use crate::K_VALUE;
use super::*;
use std::borrow::Cow;
/// The result of an operation on a `RecordStore`.
pub type Result<T> = std::result::Result<T, Error>;
/// The possible errors of a `RecordStore` operation.
#[derive(Debug)]
pub enum Error {
/// The store is at capacity w.r.t. the total number of stored records.
MaxRecords,
/// The store is at capacity w.r.t. the total number of stored keys for
/// provider records.
MaxProvidedKeys,
/// The value of a record to be stored is too large.
ValueTooLarge,
}
/// Trait for types implementing a record store.
///
/// There are two types of records managed by a `RecordStore`:
///
/// 1. Regular (value-)records. These records store an arbitrary value
/// associated with a key which is distributed to the closest nodes
/// to the key in the Kademlia DHT as per the standard Kademlia "push-model".
/// These records are subject to re-replication and re-publication as
/// per the standard Kademlia protocol.
///
/// 2. Provider records. These records associate the ID of a peer with a key
/// who can supposedly provide the associated value. These records are
/// mere "pointers" to the data which may be followed by contacting these
/// providers to obtain the value. These records are specific to the
/// libp2p Kademlia specification and realise a "pull-model" for distributed
/// content. Just like a regular record, a provider record is distributed
/// to the closest nodes to the key.
///
pub trait RecordStore<'a> {
type RecordsIter: Iterator<Item = Cow<'a, Record>>;
type ProvidedIter: Iterator<Item = Cow<'a, ProviderRecord>>;
/// Gets a record from the store, given its key.
fn get(&'a self, k: &Multihash) -> Option<Cow<Record>>;
/// Puts a record into the store.
fn put(&'a mut self, r: Record) -> Result<()>;
/// Removes the record with the given key from the store.
fn remove(&'a mut self, k: &Multihash);
/// Gets an iterator over all (value-) records currently stored.
fn records(&'a self) -> Self::RecordsIter;
/// Adds a provider record to the store.
///
/// A record store only needs to store a number of provider records
/// for a key corresponding to the replication factor and should
/// store those records whose providers are closest to the key.
fn add_provider(&'a mut self, record: ProviderRecord) -> Result<()>;
/// Gets a copy of the stored provider records for the given key.
fn providers(&'a self, key: &Multihash) -> Vec<ProviderRecord>;
/// Gets an iterator over all stored provider records for which the
/// node owning the store is itself the provider.
fn provided(&'a self) -> Self::ProvidedIter;
/// Removes a provider record from the store.
fn remove_provider(&'a mut self, k: &Multihash, p: &PeerId);
}

View File

@ -0,0 +1,317 @@
// Copyright 2019 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use super::*;
use crate::kbucket;
use libp2p_core::PeerId;
use multihash::Multihash;
use smallvec::SmallVec;
use std::borrow::Cow;
use std::collections::{hash_map, hash_set, HashMap, HashSet};
use std::iter;
/// In-memory implementation of a `RecordStore`.
pub struct MemoryStore {
/// The identity of the peer owning the store.
local_key: kbucket::Key<PeerId>,
/// The configuration of the store.
config: MemoryStoreConfig,
/// The stored (regular) records.
records: HashMap<Multihash, Record>,
/// The stored provider records.
providers: HashMap<Multihash, SmallVec<[ProviderRecord; K_VALUE.get()]>>,
/// The set of all provider records for the node identified by `local_key`.
///
/// Must be kept in sync with `providers`.
provided: HashSet<ProviderRecord>,
}
/// Configuration for a `MemoryStore`.
pub struct MemoryStoreConfig {
/// The maximum number of records.
pub max_records: usize,
/// The maximum size of record values, in bytes.
pub max_value_bytes: usize,
/// The maximum number of providers stored for a key.
///
/// This should match up with the chosen replication factor.
pub max_providers_per_key: usize,
/// The maximum number of provider records for which the
/// local node is the provider.
pub max_provided_keys: usize,
}
impl Default for MemoryStoreConfig {
fn default() -> Self {
Self {
max_records: 1024,
max_value_bytes: 65 * 1024,
max_provided_keys: 1024,
max_providers_per_key: K_VALUE.get(),
}
}
}
impl MemoryStore {
/// Creates a new `MemoryRecordStore` with a default configuration.
pub fn new(local_id: PeerId) -> Self {
Self::with_config(local_id, Default::default())
}
/// Creates a new `MemoryRecordStore` with the given configuration.
pub fn with_config(local_id: PeerId, config: MemoryStoreConfig) -> Self {
MemoryStore {
local_key: kbucket::Key::new(local_id),
config,
records: HashMap::default(),
provided: HashSet::default(),
providers: HashMap::default(),
}
}
/// Retains the records satisfying a predicate.
pub fn retain<F>(&mut self, f: F)
where
F: FnMut(&Multihash, &mut Record) -> bool
{
self.records.retain(f);
}
}
impl<'a> RecordStore<'a> for MemoryStore {
type RecordsIter = iter::Map<
hash_map::Values<'a, Multihash, Record>,
fn(&'a Record) -> Cow<'a, Record>
>;
type ProvidedIter = iter::Map<
hash_set::Iter<'a, ProviderRecord>,
fn(&'a ProviderRecord) -> Cow<'a, ProviderRecord>
>;
fn get(&'a self, k: &Multihash) -> Option<Cow<Record>> {
self.records.get(k).map(Cow::Borrowed)
}
fn put(&'a mut self, r: Record) -> Result<()> {
if r.value.len() >= self.config.max_value_bytes {
return Err(Error::ValueTooLarge)
}
let num_records = self.records.len();
match self.records.entry(r.key.clone()) {
hash_map::Entry::Occupied(mut e) => {
e.insert(r);
}
hash_map::Entry::Vacant(e) => {
if num_records >= self.config.max_records {
return Err(Error::MaxRecords)
}
e.insert(r);
}
}
Ok(())
}
fn remove(&'a mut self, k: &Multihash) {
self.records.remove(k);
}
fn records(&'a self) -> Self::RecordsIter {
self.records.values().map(Cow::Borrowed)
}
fn add_provider(&'a mut self, record: ProviderRecord) -> Result<()> {
let num_keys = self.providers.len();
// Obtain the entry
let providers = match self.providers.entry(record.key.clone()) {
e@hash_map::Entry::Occupied(_) => e,
e@hash_map::Entry::Vacant(_) => {
if self.config.max_provided_keys == num_keys {
return Err(Error::MaxProvidedKeys)
}
e
}
}.or_insert_with(Default::default);
if let Some(i) = providers.iter().position(|p| p.provider == record.provider) {
// In-place update of an existing provider record.
providers.as_mut()[i] = record;
} else {
// It is a new provider record for that key.
let local_key = self.local_key.clone();
let key = kbucket::Key::new(record.key.clone());
let provider = kbucket::Key::new(record.provider.clone());
if let Some(i) = providers.iter().position(|p| {
let pk = kbucket::Key::new(p.provider.clone());
provider.distance(&key) < pk.distance(&key)
}) {
// Insert the new provider.
if local_key.preimage() == &record.provider {
self.provided.insert(record.clone());
}
providers.insert(i, record);
// Remove the excess provider, if any.
if providers.len() > self.config.max_providers_per_key {
if let Some(p) = providers.pop() {
self.provided.remove(&p);
}
}
}
else if providers.len() < self.config.max_providers_per_key {
// The distance of the new provider to the key is larger than
// the distance of any existing provider, but there is still room.
if local_key.preimage() == &record.provider {
self.provided.insert(record.clone());
}
providers.push(record);
}
}
Ok(())
}
fn providers(&'a self, key: &Multihash) -> Vec<ProviderRecord> {
self.providers.get(&key).map_or_else(Vec::new, |ps| ps.clone().into_vec())
}
fn provided(&'a self) -> Self::ProvidedIter {
self.provided.iter().map(Cow::Borrowed)
}
fn remove_provider(&'a mut self, key: &Multihash, provider: &PeerId) {
if let hash_map::Entry::Occupied(mut e) = self.providers.entry(key.clone()) {
let providers = e.get_mut();
if let Some(i) = providers.iter().position(|p| &p.provider == provider) {
let p = providers.remove(i);
self.provided.remove(&p);
}
if providers.len() == 0 {
e.remove();
}
}
}
}
#[cfg(test)]
mod tests {
use super::*;
use multihash::Hash::SHA2256;
use quickcheck::*;
fn distance(r: &ProviderRecord) -> kbucket::Distance {
kbucket::Key::new(r.key.clone())
.distance(&kbucket::Key::new(r.provider.clone()))
}
#[test]
fn put_get_remove_record() {
fn prop(r: Record) {
let mut store = MemoryStore::new(PeerId::random());
assert!(store.put(r.clone()).is_ok());
assert_eq!(Some(Cow::Borrowed(&r)), store.get(&r.key));
store.remove(&r.key);
assert!(store.get(&r.key).is_none());
}
quickcheck(prop as fn(_))
}
#[test]
fn add_get_remove_provider() {
fn prop(r: ProviderRecord) {
let mut store = MemoryStore::new(PeerId::random());
assert!(store.add_provider(r.clone()).is_ok());
assert!(store.providers(&r.key).contains(&r));
store.remove_provider(&r.key, &r.provider);
assert!(!store.providers(&r.key).contains(&r));
}
quickcheck(prop as fn(_))
}
#[test]
fn providers_ordered_by_distance_to_key() {
fn prop(providers: Vec<kbucket::Key<PeerId>>) -> bool {
let mut store = MemoryStore::new(PeerId::random());
let key = Multihash::random(SHA2256);
let mut records = providers.into_iter().map(|p| {
ProviderRecord::new(key.clone(), p.into_preimage())
}).collect::<Vec<_>>();
for r in &records {
assert!(store.add_provider(r.clone()).is_ok());
}
records.sort_by(|r1, r2| distance(r1).cmp(&distance(r2)));
records.truncate(store.config.max_providers_per_key);
records == store.providers(&key).to_vec()
}
quickcheck(prop as fn(_) -> _)
}
#[test]
fn provided() {
let id = PeerId::random();
let mut store = MemoryStore::new(id.clone());
let key = Multihash::random(SHA2256);
let rec = ProviderRecord::new(key, id.clone());
assert!(store.add_provider(rec.clone()).is_ok());
assert_eq!(vec![Cow::Borrowed(&rec)], store.provided().collect::<Vec<_>>());
store.remove_provider(&rec.key, &id);
assert_eq!(store.provided().count(), 0);
}
#[test]
fn update_provider() {
let mut store = MemoryStore::new(PeerId::random());
let key = Multihash::random(SHA2256);
let prv = PeerId::random();
let mut rec = ProviderRecord::new(key, prv);
assert!(store.add_provider(rec.clone()).is_ok());
assert_eq!(vec![rec.clone()], store.providers(&rec.key).to_vec());
rec.expires = Some(Instant::now());
assert!(store.add_provider(rec.clone()).is_ok());
assert_eq!(vec![rec.clone()], store.providers(&rec.key).to_vec());
}
#[test]
fn max_provided_keys() {
let mut store = MemoryStore::new(PeerId::random());
for _ in 0 .. store.config.max_provided_keys {
let key = Multihash::random(SHA2256);
let prv = PeerId::random();
let rec = ProviderRecord::new(key, prv);
let _ = store.add_provider(rec);
}
let key = Multihash::random(SHA2256);
let prv = PeerId::random();
let rec = ProviderRecord::new(key, prv);
match store.add_provider(rec) {
Err(Error::MaxProvidedKeys) => {}
_ => panic!("Unexpected result"),
}
}
}