Improve XOR metric. (#1108)

There are two issues with the current definition and use of Kademlia's
XOR metric:

  1. The distance is currently equated with the bucket index, i.e.
     `distance(a,b) - 1` is the index of the bucket into which either
     peer is put by the other. The result is a metric that is not
     unidirectional, as defined in the Kademlia paper and as implemented
     in e.g. libp2p-go and libp2p-js, which is to interpret the result
     of the XOR as an integer in its entirety.

  2. The current `KBucketsPeerId` trait and its instances allow computing
     distances between types with differing bit lengths as well as between
     types that hash all inputs again (i.e. `KadHash`) and "plain" `PeerId`s
     or `Multihash`es. This can result in computed distances that are either
     incorrect as per the requirement of the libp2p specs that all distances
     are to be computed from the XOR of the SHA256 of the input keys, or
     even fall outside of the image of the metric used for the `KBucketsTable`.
     In the latter case, such distances are not currently used as a bucket index
     - they can only occur in the context of comparing distances for the purpose
     of sorting peers - but that still seems undesirable.

These issues are addressed here as follows:

  * Unidirectionality of the XOR metric is restored by keeping the "full"
    integer representation of the bitwise XOR. The result is an XOR metric
    as defined in the paper. This also opens the door to avoiding the
    "full table scan" when searching for the keys closest to a given key -
    the ideal order in which to visit the buckets can be computed with the
    help of the distance bit string.

  * As a simplification and to make it easy to "do the right thing", the
    XOR metric is only defined on an opaque `kbucket::Key` type, partially
    derived from the current `KadHash`. `KadHash` and `KBucketsPeerId`
    are removed.
This commit is contained in:
Roman Borschel
2019-05-17 17:27:57 +02:00
committed by GitHub
parent 93d89964e1
commit c80205454a
7 changed files with 310 additions and 363 deletions

View File

@ -26,30 +26,101 @@
//! corresponding to its distance with the reference key.
use arrayvec::ArrayVec;
use bigint::{U512, U256};
use crate::kad_hash::KadHash;
use bigint::U256;
use libp2p_core::PeerId;
use multihash::Multihash;
use std::num::NonZeroUsize;
use sha2::{Digest, Sha256, digest::generic_array::{GenericArray, typenum::U32}};
use std::slice::IterMut as SliceIterMut;
use std::time::{Duration, Instant};
use std::vec::IntoIter as VecIntoIter;
/// Maximum number of nodes in a bucket.
pub const MAX_NODES_PER_BUCKET: usize = 20;
/// Maximum number of k-buckets.
const NUM_BUCKETS: usize = 256;
/// Maximum number of nodes in a bucket, i.e. `k`.
const MAX_NODES_PER_BUCKET: usize = 20;
/// Table of k-buckets.
/// A table of `KBucket`s, i.e. a Kademlia routing table.
#[derive(Debug, Clone)]
pub struct KBucketsTable<TPeerId, TVal> {
/// Peer ID of the local node.
my_id: TPeerId,
local_key: Key<TPeerId>,
/// The actual tables that store peers or values.
tables: Vec<KBucket<TPeerId, TVal>>,
/// The timeout when trying to reach the youngest node after which we consider it unresponsive.
unresponsive_timeout: Duration,
}
/// An individual table that stores peers or values.
/// A `Key` is a cryptographic hash, stored with an associated value in a `KBucket`
/// of a `KBucketsTable`.
///
/// `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,
hash: GenericArray<u8, U32>,
}
impl<T> PartialEq for Key<T> {
fn eq(&self, other: &Key<T>) -> bool {
self.hash == other.hash
}
}
impl<T> Eq for Key<T> {}
impl<T> Key<T> {
/// Construct a new `Key` by hashing the bytes of the given `preimage`.
///
/// The preimage of type `T` is preserved. See [`Key::preimage`] and
/// [`Key::into_preimage`].
pub fn new(preimage: T) -> Key<T>
where
T: AsRef<[u8]>
{
let hash = Sha256::digest(preimage.as_ref());
Key { preimage, hash }
}
/// Borrows the preimage of the key.
pub fn preimage(&self) -> &T {
&self.preimage
}
/// Converts the key into its preimage.
pub fn into_preimage(self) -> T {
self.preimage
}
/// Computes the distance of the keys according to the XOR metric.
pub fn distance<U>(&self, other: &Key<U>) -> Distance {
let a = U256::from(self.hash.as_ref());
let b = U256::from(other.hash.as_ref());
Distance(a ^ b)
}
}
impl From<Multihash> for Key<Multihash> {
fn from(h: Multihash) -> Self {
let k = Key::new(h.clone().into_bytes());
Key { preimage: h, hash: k.hash }
}
}
impl From<PeerId> for Key<PeerId> {
fn from(peer_id: PeerId) -> Self {
Key::new(peer_id)
}
}
/// A distance between two `Key`s.
#[derive(Copy, Clone, PartialEq, Eq, Default, PartialOrd, Ord, Debug)]
pub struct Distance(bigint::U256);
/// A `KBucket` is a list of up to `MAX_NODES_PER_BUCKET` `Key`s and associated values,
/// ordered from least recently used to most recently used.
#[derive(Debug, Clone)]
struct KBucket<TPeerId, TVal> {
/// Nodes are always ordered from oldest to newest. The nodes we are connected to are always
@ -79,131 +150,25 @@ struct PendingNode<TPeerId, TVal> {
replace: Instant,
}
/// A single node in a k-bucket.
/// A single node in a `KBucket`.
#[derive(Debug, Clone)]
struct Node<TPeerId, TVal> {
/// Id of the node.
id: TPeerId,
id: Key<TPeerId>,
/// Value associated to it.
value: TVal,
}
/// Trait that must be implemented on types that can be used as an identifier in a k-bucket.
///
/// If `TOther` is not the same as `Self`, it represents an entry already in the k-buckets that
/// `Self` can compare against.
pub trait KBucketsPeerId<TOther = Self>: PartialEq<TOther> {
/// Computes the XOR of this value and another one. The lower the closer.
fn distance_with(&self, other: &TOther) -> u32;
/// Returns then number of bits that are necessary to store the distance between peer IDs.
/// Used for pre-allocations.
fn max_distance() -> NonZeroUsize;
}
impl KBucketsPeerId for PeerId {
fn distance_with(&self, other: &Self) -> u32 {
<Multihash as KBucketsPeerId<Multihash>>::distance_with(self.as_ref(), other.as_ref())
}
fn max_distance() -> NonZeroUsize {
<Multihash as KBucketsPeerId>::max_distance()
}
}
impl KBucketsPeerId<PeerId> for Multihash {
fn distance_with(&self, other: &PeerId) -> u32 {
<Multihash as KBucketsPeerId<Multihash>>::distance_with(self, other.as_ref())
}
fn max_distance() -> NonZeroUsize {
<PeerId as KBucketsPeerId>::max_distance()
}
}
impl KBucketsPeerId for KadHash {
fn distance_with(&self, other: &Self) -> u32 {
// Note that we don't compare the hash functions because there's no chance of collision
// of the same value hashed with two different hash functions.
let my_hash = U256::from(self.hash());
let other_hash = U256::from(other.hash());
let xor = my_hash ^ other_hash;
256 - xor.leading_zeros()
}
fn max_distance() -> NonZeroUsize {
// Hash is SHA2256, so fixed value
NonZeroUsize::new(256).expect("256 is not zero; QED")
}
}
impl KBucketsPeerId<KadHash> for Multihash {
fn distance_with(&self, other: &KadHash) -> u32 {
let my_hash = U512::from(self.digest());
let other_hash = U512::from(U256::from(other.hash()));
let xor = my_hash ^ other_hash;
512 - xor.leading_zeros()
}
fn max_distance() -> NonZeroUsize {
NonZeroUsize::new(512).expect("512 is not zero; QED")
}
}
impl KBucketsPeerId for Multihash {
fn distance_with(&self, other: &Self) -> u32 {
// Note that we don't compare the hash functions because there's no chance of collision
// of the same value hashed with two different hash functions.
let my_hash = U512::from(self.digest());
let other_hash = U512::from(other.digest());
let xor = my_hash ^ other_hash;
512 - xor.leading_zeros()
}
fn max_distance() -> NonZeroUsize {
NonZeroUsize::new(512).expect("512 is not zero; QED")
}
}
impl<A, B> KBucketsPeerId for (A, B)
where
A: KBucketsPeerId + PartialEq,
B: KBucketsPeerId + PartialEq,
{
fn distance_with(&self, other: &(A, B)) -> u32 {
A::distance_with(&self.0, &other.0) + B::distance_with(&self.1, &other.1)
}
fn max_distance() -> NonZeroUsize {
let n = <A as KBucketsPeerId<A>>::max_distance().get()
.saturating_add(<B as KBucketsPeerId<B>>::max_distance().get());
NonZeroUsize::new(n).expect("Saturating-add of two non-zeros can't be zero; QED")
}
}
impl<'a, T> KBucketsPeerId for &'a T
where
T: KBucketsPeerId,
{
fn distance_with(&self, other: &&'a T) -> u32 {
T::distance_with(*self, *other)
}
fn max_distance() -> NonZeroUsize {
<T as KBucketsPeerId>::max_distance()
}
}
impl<TPeerId, TVal> KBucketsTable<TPeerId, TVal>
where
TPeerId: KBucketsPeerId + Clone,
TPeerId: Clone,
{
/// Builds a new routing table.
pub fn new(my_id: TPeerId, unresponsive_timeout: Duration) -> Self {
/// Builds a new routing table whose keys are distributed over `KBucket`s as
/// per the relative distance to `local_key`.
pub fn new(local_key: Key<TPeerId>, unresponsive_timeout: Duration) -> Self {
KBucketsTable {
my_id,
tables: (0..TPeerId::max_distance().get())
local_key,
tables: (0..NUM_BUCKETS)
.map(|_| KBucket {
nodes: ArrayVec::new(),
first_connected_pos: 0,
@ -214,20 +179,20 @@ where
}
}
/// Returns the ID of the local node.
pub fn my_id(&self) -> &TPeerId {
&self.my_id
/// Returns the local key.
pub fn local_key(&self) -> &Key<TPeerId> {
&self.local_key
}
/// Returns the id of the bucket that should contain the peer with the given ID.
///
/// Returns `None` if out of range, which happens if `id` is the same as the local peer id.
fn bucket_num(&self, id: &TPeerId) -> Option<usize> {
(self.my_id.distance_with(id) as usize).checked_sub(1)
fn bucket_num(&self, key: &Key<TPeerId>) -> Option<usize> {
(NUM_BUCKETS - self.local_key.distance(key).0.leading_zeros() as usize).checked_sub(1)
}
/// Returns an object containing the state of the given entry.
pub fn entry<'a>(&'a mut self, peer_id: &'a TPeerId) -> Entry<'a, TPeerId, TVal> {
pub fn entry<'a>(&'a mut self, peer_id: &'a Key<TPeerId>) -> Entry<'a, TPeerId, TVal> {
let bucket_num = if let Some(num) = self.bucket_num(peer_id) {
num
} else {
@ -293,7 +258,7 @@ where
}
/// Returns an iterator to all the peer IDs in the bucket, without the pending nodes.
pub fn entries_not_pending(&self) -> impl Iterator<Item = (&TPeerId, &TVal)> {
pub fn entries_not_pending(&self) -> impl Iterator<Item = (&Key<TPeerId>, &TVal)> {
self.tables
.iter()
.flat_map(|table| table.nodes.iter())
@ -308,10 +273,10 @@ where
BucketsIter(self.tables.iter_mut(), self.unresponsive_timeout)
}
/// Finds the nodes closest to `id`, ordered by distance.
/// Finds the keys closest to `key`, ordered by distance.
///
/// Pending nodes are ignored.
pub fn find_closest(&mut self, id: &impl KBucketsPeerId<TPeerId>) -> VecIntoIter<TPeerId> {
pub fn find_closest<T>(&mut self, key: &Key<T>) -> VecIntoIter<Key<TPeerId>> {
// TODO: optimize
let mut out = Vec::new();
for table in self.tables.iter_mut() {
@ -331,7 +296,7 @@ where
}
}
}
out.sort_by(|a, b| id.distance_with(a).cmp(&id.distance_with(b)));
out.sort_by(|a, b| key.distance(a).cmp(&key.distance(b)));
out.into_iter()
}
}
@ -354,7 +319,7 @@ pub enum Entry<'a, TPeerId, TVal> {
impl<'a, TPeerId, TVal> Entry<'a, TPeerId, TVal>
where
TPeerId: KBucketsPeerId + Clone,
TPeerId: Clone,
{
/// Returns the value associated to the entry in the bucket, including if the node is pending.
pub fn value(&mut self) -> Option<&mut TVal> {
@ -384,12 +349,12 @@ where
/// Represents an entry in a k-bucket.
pub struct EntryInKbucketConn<'a, TPeerId, TVal> {
parent: &'a mut KBucketsTable<TPeerId, TVal>,
peer_id: &'a TPeerId,
peer_id: &'a Key<TPeerId>,
}
impl<'a, TPeerId, TVal> EntryInKbucketConn<'a, TPeerId, TVal>
where
TPeerId: KBucketsPeerId + Clone,
TPeerId: Clone,
{
/// Returns the value associated to the entry in the bucket.
pub fn value(&mut self) -> &mut TVal {
@ -470,7 +435,7 @@ pub enum SetDisconnectedOutcome<'a, TPeerId, TVal> {
Replaced {
/// Node that replaced the node.
// TODO: could be a EntryInKbucketConn, but we have borrow issues with the new peer id
replacement: TPeerId,
replacement: Key<TPeerId>,
/// Value os the node that has been pushed out.
old_val: TVal,
},
@ -479,12 +444,12 @@ pub enum SetDisconnectedOutcome<'a, TPeerId, TVal> {
/// Represents an entry waiting for a slot to be available in its k-bucket.
pub struct EntryInKbucketConnPending<'a, TPeerId, TVal> {
parent: &'a mut KBucketsTable<TPeerId, TVal>,
peer_id: &'a TPeerId,
peer_id: &'a Key<TPeerId>,
}
impl<'a, TPeerId, TVal> EntryInKbucketConnPending<'a, TPeerId, TVal>
where
TPeerId: KBucketsPeerId + Clone,
TPeerId: Clone,
{
/// Returns the value associated to the entry in the bucket.
pub fn value(&mut self) -> &mut TVal {
@ -526,12 +491,12 @@ where
/// Represents an entry waiting for a slot to be available in its k-bucket.
pub struct EntryInKbucketDiscPending<'a, TPeerId, TVal> {
parent: &'a mut KBucketsTable<TPeerId, TVal>,
peer_id: &'a TPeerId,
peer_id: &'a Key<TPeerId>,
}
impl<'a, TPeerId, TVal> EntryInKbucketDiscPending<'a, TPeerId, TVal>
where
TPeerId: KBucketsPeerId + Clone,
TPeerId: Clone,
{
/// Returns the value associated to the entry in the bucket.
pub fn value(&mut self) -> &mut TVal {
@ -573,12 +538,12 @@ where
/// Represents an entry in a k-bucket.
pub struct EntryInKbucketDisc<'a, TPeerId, TVal> {
parent: &'a mut KBucketsTable<TPeerId, TVal>,
peer_id: &'a TPeerId,
peer_id: &'a Key<TPeerId>,
}
impl<'a, TPeerId, TVal> EntryInKbucketDisc<'a, TPeerId, TVal>
where
TPeerId: KBucketsPeerId + Clone,
TPeerId: Clone,
{
/// Returns the value associated to the entry in the bucket.
pub fn value(&mut self) -> &mut TVal {
@ -640,12 +605,12 @@ where
/// Represents an entry not in any k-bucket.
pub struct EntryNotInKbucket<'a, TPeerId, TVal> {
parent: &'a mut KBucketsTable<TPeerId, TVal>,
peer_id: &'a TPeerId,
peer_id: &'a Key<TPeerId>,
}
impl<'a, TPeerId, TVal> EntryNotInKbucket<'a, TPeerId, TVal>
where
TPeerId: KBucketsPeerId + Clone,
TPeerId: Clone,
{
/// Inserts the node as connected, if possible.
pub fn insert_connected(self, value: TVal) -> InsertOutcome<TPeerId> {
@ -703,14 +668,14 @@ where
/// Outcome of calling `insert`.
#[must_use]
#[derive(Debug, Copy, Clone, PartialEq, Eq)]
#[derive(Debug, Clone, PartialEq, Eq)]
pub enum InsertOutcome<TPeerId> {
/// The entry has been successfully inserted.
Inserted,
/// The entry has been inserted as a pending node.
Pending {
/// We have to try connect to the returned node.
to_ping: TPeerId,
to_ping: Key<TPeerId>,
},
/// The entry was not inserted because the bucket was full of connected nodes.
Full,
@ -760,17 +725,68 @@ impl<'a, TPeerId, TVal> Bucket<'a, TPeerId, TVal> {
#[cfg(test)]
mod tests {
use crate::kbucket::{Entry, InsertOutcome, KBucketsPeerId, KBucketsTable, MAX_NODES_PER_BUCKET};
use multihash::{Multihash, Hash};
use std::thread;
use super::*;
use quickcheck::*;
use libp2p_core::PeerId;
use crate::kbucket::{Entry, InsertOutcome, KBucketsTable, MAX_NODES_PER_BUCKET};
use std::time::Duration;
impl Arbitrary for Key<PeerId> {
fn arbitrary<G: Gen>(_: &mut G) -> Key<PeerId> {
Key::from(PeerId::random())
}
}
#[test]
fn identity() {
fn prop(a: Key<PeerId>) -> bool {
a.distance(&a) == Distance::default()
}
quickcheck(prop as fn(_) -> _)
}
#[test]
fn symmetry() {
fn prop(a: Key<PeerId>, b: Key<PeerId>) -> bool {
a.distance(&b) == b.distance(&a)
}
quickcheck(prop as fn(_,_) -> _)
}
#[test]
fn triangle_inequality() {
fn prop(a: Key<PeerId>, b: Key<PeerId>, c: Key<PeerId>) -> TestResult {
let ab = a.distance(&b);
let bc = b.distance(&c);
let (ab_plus_bc, overflow) = ab.0.overflowing_add(bc.0);
if overflow {
TestResult::discard()
} else {
TestResult::from_bool(a.distance(&c) <= Distance(ab_plus_bc))
}
}
quickcheck(prop as fn(_,_,_) -> _)
}
#[test]
fn unidirectionality() {
fn prop(a: Key<PeerId>, b: Key<PeerId>) -> bool {
let d = a.distance(&b);
(0..100).all(|_| {
let c = Key::from(PeerId::random());
a.distance(&c) != d || b == c
})
}
quickcheck(prop as fn(_,_) -> _)
}
#[test]
fn basic_closest() {
let my_id = Multihash::random(Hash::SHA2256);
let other_id = Multihash::random(Hash::SHA2256);
let my_key = Key::from(PeerId::random());
let other_id = Key::from(PeerId::random());
let mut table = KBucketsTable::<_, ()>::new(my_id, Duration::from_secs(5));
let mut table = KBucketsTable::<_, ()>::new(my_key, Duration::from_secs(5));
if let Entry::NotInKbucket(entry) = table.entry(&other_id) {
match entry.insert_connected(()) {
InsertOutcome::Inserted => (),
@ -787,10 +803,10 @@ mod tests {
#[test]
fn update_local_id_fails() {
let my_id = Multihash::random(Hash::SHA2256);
let my_key = Key::from(PeerId::random());
let mut table = KBucketsTable::<_, ()>::new(my_id.clone(), Duration::from_secs(5));
match table.entry(&my_id) {
let mut table = KBucketsTable::<_, ()>::new(my_key.clone(), Duration::from_secs(5));
match table.entry(&my_key) {
Entry::SelfEntry => (),
_ => panic!(),
}
@ -798,23 +814,32 @@ mod tests {
#[test]
fn full_kbucket() {
let my_id = Multihash::random(Hash::SHA2256);
let my_key = Key::from(PeerId::random());
let mut table = KBucketsTable::<_, ()>::new(my_key.clone(), Duration::from_secs(5));
// Step 1: Fill the most distant bucket, i.e. bucket index `NUM_BUCKETS - 1`,
// with "disconnected" peers.
// Prepare `MAX_NODES_PER_BUCKET` keys to fill the bucket, plus 2
// additional keys which will be used to test the behavior on a full
// bucket.
assert!(MAX_NODES_PER_BUCKET <= 251); // Test doesn't work otherwise.
let mut fill_ids = (0..MAX_NODES_PER_BUCKET + 3)
.map(|n| {
let mut id = my_id.clone().into_bytes();
id[2] ^= 0x80; // Flip the first bit so that we get in the most distant bucket.
id[33] = id[33].wrapping_add(n as u8);
Multihash::from_bytes(id).unwrap()
let mut id = my_key.clone();
// Flip the first bit so that we get in the most distant bucket.
id.hash[0] ^= 0x80;
// Each ID gets a unique low-order byte (i.e. counter).
id.hash[31] = id.hash[31].wrapping_add(n as u8);
id
})
.collect::<Vec<_>>();
let first_node = fill_ids[0].clone();
let second_node = fill_ids[1].clone();
let mut table = KBucketsTable::<_, ()>::new(my_id.clone(), Duration::from_secs(1));
// Fill the bucket, consuming all but the last 3 test keys.
for (num, id) in fill_ids.drain(..MAX_NODES_PER_BUCKET).enumerate() {
if let Entry::NotInKbucket(entry) = table.entry(&id) {
match entry.insert_disconnected(()) {
@ -826,12 +851,16 @@ mod tests {
}
assert_eq!(table.buckets().nth(255).unwrap().num_entries(), num + 1);
}
assert_eq!(
table.buckets().nth(255).unwrap().num_entries(),
MAX_NODES_PER_BUCKET
);
assert!(!table.buckets().nth(255).unwrap().has_pending());
// Step 2: Insert another key on the full bucket. It must be marked as
// pending and the first (i.e. "least recently used") entry scheduled
// for replacement.
if let Entry::NotInKbucket(entry) = table.entry(&fill_ids.remove(0)) {
match entry.insert_connected(()) {
InsertOutcome::Pending { ref to_ping } if *to_ping == first_node => (),
@ -840,13 +869,13 @@ mod tests {
} else {
panic!()
}
assert_eq!(
table.buckets().nth(255).unwrap().num_entries(),
MAX_NODES_PER_BUCKET
);
assert!(table.buckets().nth(255).unwrap().has_pending());
if let Entry::NotInKbucket(entry) = table.entry(&fill_ids.remove(0)) {
// Trying to insert yet another key is rejected.
if let Entry::NotInKbucket(entry) = table.entry(&Key::from(fill_ids.remove(0))) {
match entry.insert_connected(()) {
InsertOutcome::Full => (),
_ => panic!()
@ -855,7 +884,12 @@ mod tests {
panic!()
}
thread::sleep(Duration::from_secs(2));
// Step 3: Make the pending nodes eligible for replacing existing nodes.
// The pending node must be consumed and replace the first (i.e. "least
// recently used") node.
let elapsed = Instant::now() - Duration::from_secs(1);
table.tables[255].pending_node.as_mut().map(|n| n.replace = elapsed);
assert!(!table.buckets().nth(255).unwrap().has_pending());
if let Entry::NotInKbucket(entry) = table.entry(&fill_ids.remove(0)) {
match entry.insert_connected(()) {
@ -866,18 +900,4 @@ mod tests {
panic!()
}
}
#[test]
fn self_distance_zero() {
let a = Multihash::random(Hash::SHA2256);
assert_eq!(a.distance_with(&a), 0);
}
#[test]
fn distance_correct_order() {
let a = Multihash::random(Hash::SHA2256);
let b = Multihash::random(Hash::SHA2256);
assert!(a.distance_with(&a) < b.distance_with(&a));
assert!(a.distance_with(&b) > b.distance_with(&b));
}
}