start re-working p2p to work with change monero-wire

adds back some changes from #22
This commit is contained in:
Boog900 2023-07-19 17:42:58 +01:00
parent d2a94f1909
commit 8981260750
No known key found for this signature in database
GPG key ID: 5401367FB7302004
36 changed files with 4447 additions and 35 deletions

View file

@ -50,6 +50,7 @@ pub enum PruningError {
///
// Internally we use an Option<u32> to represent if a pruning seed is 0 (None)which means
// no pruning will take place.
#[derive(Debug, Clone, Copy)]
pub struct PruningSeed(Option<u32>);
impl PruningSeed {
@ -78,7 +79,7 @@ impl PruningSeed {
/// and 3 for `log_stripes`.*
///
pub fn new(stripe: u32, log_stripes: u32) -> Result<PruningSeed, PruningError> {
if !(log_stripes <= PRUNING_SEED_LOG_STRIPES_MASK) {
if log_stripes > PRUNING_SEED_LOG_STRIPES_MASK {
Err(PruningError::LogStripesOutOfRange)
} else if !(stripe > 0 && stripe <= (1 << log_stripes)) {
Err(PruningError::StripeOutOfRange)

View file

@ -27,7 +27,7 @@ use crate::{
};
/// The levin tokio-codec for decoding and encoding levin buckets
#[derive(Default)]
#[derive(Default, Debug, Clone)]
pub enum LevinCodec {
/// Waiting for the peer to send a header.
#[default]
@ -90,7 +90,7 @@ impl Encoder<Bucket> for LevinCodec {
}
}
#[derive(Default)]
#[derive(Default, Debug, Clone)]
enum MessageState {
#[default]
WaitingForBucket,
@ -99,12 +99,23 @@ enum MessageState {
/// A tokio-codec for levin messages or in other words the decoded body
/// of a levin bucket.
#[derive(Debug, Clone)]
pub struct LevinMessageCodec<T> {
message_ty: PhantomData<T>,
bucket_codec: LevinCodec,
state: MessageState,
}
impl<T> Default for LevinMessageCodec<T> {
fn default() -> Self {
Self {
message_ty: Default::default(),
bucket_codec: Default::default(),
state: Default::default(),
}
}
}
impl<T: LevinBody> Decoder for LevinMessageCodec<T> {
type Item = T;
type Error = BucketError;

View file

@ -36,7 +36,7 @@
pub mod codec;
pub mod header;
pub use codec::LevinCodec;
pub use codec::*;
pub use header::BucketHead;
use std::fmt::Debug;
@ -61,7 +61,7 @@ pub enum BucketError {
InvalidFragmentedMessage(&'static str),
/// Error decoding the body
#[error("Error decoding bucket body")]
BodyDecodingError(Box<dyn Debug>),
BodyDecodingError(Box<dyn Debug + Send + Sync>),
/// I/O error
#[error("I/O error: {0}")]
IO(#[from] std::io::Error),
@ -77,7 +77,7 @@ pub struct Bucket {
}
/// An enum representing if the message is a request, response or notification.
#[derive(Debug, Eq, PartialEq)]
#[derive(Debug, Eq, PartialEq, Clone, Copy)]
pub enum MessageType {
/// Request
Request,

View file

@ -35,4 +35,6 @@ pub mod network_address;
pub use messages::*;
pub use network_address::*;
pub use levin_cuprate::BucketError;
pub type MoneroWireCodec = levin_cuprate::codec::LevinMessageCodec<Message>;

View file

@ -26,7 +26,7 @@ pub use admin::{
HandshakeRequest, HandshakeResponse, PingResponse, SupportFlagsResponse, TimedSyncRequest,
TimedSyncResponse,
};
pub use common::{BasicNodeData, CoreSyncData, PeerListEntryBase};
pub use common::{BasicNodeData, CoreSyncData, PeerID, PeerListEntryBase};
pub use protocol::{
ChainRequest, ChainResponse, FluffyMissingTransactionsRequest, GetObjectsRequest,
GetObjectsResponse, GetTxPoolCompliment, NewBlock, NewFluffyBlock, NewTransactions,

View file

@ -20,7 +20,7 @@
use epee_encoding::EpeeObject;
use super::common::{BasicNodeData, CoreSyncData, PeerListEntryBase, PeerSupportFlags};
use super::common::{BasicNodeData, CoreSyncData, PeerID, PeerListEntryBase, PeerSupportFlags};
/// A Handshake Request
#[derive(Debug, Clone, EpeeObject, PartialEq, Eq)]
@ -68,7 +68,8 @@ pub struct PingResponse {
/// Status: should be `PING_OK_RESPONSE_STATUS_TEXT`
pub status: String,
/// Peer ID
pub peer_id: u64,
#[epee_try_from_into(u64)]
pub peer_id: PeerID,
}
/// A Support Flags Response
@ -107,7 +108,7 @@ mod tests {
network_id: [
18, 48, 241, 113, 97, 4, 65, 97, 23, 49, 0, 130, 22, 161, 161, 16,
],
peer_id: 9671405426614699871,
peer_id: 9671405426614699871.into(),
support_flags: PeerSupportFlags::from(1_u32),
rpc_port: 0,
rpc_credits_per_hash: 0,
@ -912,7 +913,7 @@ mod tests {
network_id: [
18, 48, 241, 113, 97, 4, 65, 97, 23, 49, 0, 130, 22, 161, 161, 16,
],
peer_id: 6037804360359455404,
peer_id: 6037804360359455404.into(),
support_flags: PeerSupportFlags::from(1_u32),
rpc_port: 18089,
rpc_credits_per_hash: 0,

View file

@ -21,6 +21,21 @@ use crate::NetworkAddress;
mod builders;
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
pub struct PeerID(pub u64);
impl From<u64> for PeerID {
fn from(value: u64) -> Self {
PeerID(value)
}
}
impl From<PeerID> for u64 {
fn from(value: PeerID) -> u64 {
value.0
}
}
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
pub struct PeerSupportFlags(u32);
@ -36,27 +51,22 @@ impl From<PeerSupportFlags> for u32 {
}
}
/*
impl PeerSupportFlags {
const FLUFFY_BLOCKS: u32 = 0b0000_0001;
/// checks if `self` has all the flags that `other` has
pub fn contains(&self, other: &PeerSupportFlags) -> bool {
self.0. & other.0 == other.0
}
pub fn supports_fluffy_blocks(&self) -> bool {
self.0 & Self::FLUFFY_BLOCKS == Self::FLUFFY_BLOCKS
self.0 & other.0 == other.0
}
pub fn get_support_flag_fluffy_blocks() -> Self {
PeerSupportFlags {
support_flags: Self::FLUFFY_BLOCKS,
}
PeerSupportFlags(Self::FLUFFY_BLOCKS)
}
pub fn is_empty(&self) -> bool {
self.0 == 0
}
}
*/
impl From<u8> for PeerSupportFlags {
fn from(value: u8) -> Self {
PeerSupportFlags(value.into())
@ -71,7 +81,8 @@ pub struct BasicNodeData {
/// The Network Id
pub network_id: [u8; 16],
/// Peer ID
pub peer_id: u64,
#[epee_try_from_into(u64)]
pub peer_id: PeerID,
/// The Peers Support Flags
/// (If this is not in the message the default is 0)
#[epee_try_from_into(u32)]

View file

@ -7,18 +7,16 @@ authors = ["Boog900"]
[dependencies]
chrono = "0.4.24"
thiserror = "1.0.39"
cuprate-common = {path = "../common"}
monero-wire = {path= "../net/monero-wire"}
#chrono = "0.4.24"
#thiserror = "1.0.39"
#cuprate-common = {path = "../common"}
#futures = "0.3.26"
#tower = {version = "0.4.13", features = ["util", "steer", "load", "discover", "load-shed", "buffer", "timeout"]}
#tokio = {version= "1.27", features=["rt", "time", "net"]}
#tokio-util = {version = "0.7.8", features=["compat"]}
#tokio-stream = {version="0.1.14", features=["time"]}
#async-trait = "0.1.68"
#tracing = "0.1.37"
#rand = "0.8.5"
#pin-project = "1.0.12"
futures = "0.3.26"
tower = {version = "0.4.13", features = ["util", "steer", "load", "discover", "load-shed", "buffer", "timeout"]}
tokio = {version= "1.27", features=["rt", "time", "net"]}
tokio-util = {version = "0.7.8", features=["codec"]}
tokio-stream = {version="0.1.14", features=["time"]}
async-trait = "0.1.68"
tracing = "0.1.37"
rand = "0.8.5"
pin-project = "1.0.12"

157
p2p/src/address_book.rs Normal file
View file

@ -0,0 +1,157 @@
//! Cuprate Address Book
//!
//! This module holds the logic for persistent peer storage.
//! Cuprates address book is modeled as a [`tower::Service`]
//! The request is [`AddressBookRequest`] and the response is
//! [`AddressBookResponse`].
//!
//! Cuprate, like monerod, actually has 3 address books, one
//! for each [`NetZone`]. This is to reduce the possibility of
//! clear net peers getting linked to their dark counterparts
//! and so peers will only get told about peers they can
//! connect to.
//!
mod addr_book_client;
mod address_book;
pub mod connection_handle;
use cuprate_common::PruningSeed;
use monero_wire::{messages::PeerListEntryBase, network_address::NetZone, NetworkAddress, PeerID};
use connection_handle::ConnectionAddressBookHandle;
pub use addr_book_client::start_address_book;
/// Possible errors when dealing with the address book.
/// This is boxed when returning an error in the [`tower::Service`].
#[derive(Debug, thiserror::Error)]
pub enum AddressBookError {
/// The peer is not in the address book for this zone.
#[error("Peer was not found in book")]
PeerNotFound,
/// The peer list is empty.
#[error("The peer list is empty")]
PeerListEmpty,
/// The peers pruning seed has changed.
#[error("The peers pruning seed has changed")]
PeersPruningSeedChanged,
/// The peer is banned.
#[error("The peer is banned")]
PeerIsBanned,
/// When handling a received peer list, the list contains
/// a peer in a different [`NetZone`]
#[error("Peer sent an address out of it's net-zone")]
PeerSentAnAddressOutOfZone,
/// The channel to the address book has closed unexpectedly.
#[error("The address books channel has closed.")]
AddressBooksChannelClosed,
/// The address book task has exited.
#[error("The address book task has exited.")]
AddressBookTaskExited,
/// The peer file store has failed.
#[error("Peer Store Error: {0}")]
PeerStoreError(&'static str),
}
/// A message sent to tell the address book that a peer has disconnected.
pub struct PeerConnectionClosed;
/// A request to the address book.
#[derive(Debug)]
pub enum AddressBookRequest {
/// A request to handle an incoming peer list.
HandleNewPeerList(Vec<PeerListEntryBase>, NetZone),
/// Updates the `last_seen` timestamp of this peer.
SetPeerSeen(PeerID, chrono::NaiveDateTime, NetZone),
/// Bans a peer for the specified duration. This request
/// will send disconnect signals to all peers with the same
/// [`ban_identifier`](NetworkAddress::ban_identifier).
BanPeer(PeerID, std::time::Duration, NetZone),
/// Adds a peer to the connected list
ConnectedToPeer {
/// The net zone of this connection.
zone: NetZone,
/// A handle between the connection and address book.
connection_handle: ConnectionAddressBookHandle,
/// The connection addr, None if the peer is using a
/// hidden network.
addr: Option<NetworkAddress>,
/// The peers id.
id: PeerID,
/// If the peer is reachable by our node.
reachable: bool,
/// The last seen timestamp, note: Cuprate may skip updating this
/// field on some inbound messages
last_seen: chrono::NaiveDateTime,
/// The peers pruning seed
pruning_seed: PruningSeed,
/// The peers port.
rpc_port: u16,
/// The peers rpc credits per hash
rpc_credits_per_hash: u32,
},
/// A request to get and eempty the anchor list,
/// used when starting the node.
GetAndEmptyAnchorList(NetZone),
/// Get a random Gray peer from the peer list
/// If a pruning seed is given we will select from
/// peers with that seed and peers that dont prune.
GetRandomGrayPeer(NetZone, Option<PruningSeed>),
/// Get a random White peer from the peer list
/// If a pruning seed is given we will select from
/// peers with that seed and peers that dont prune.
GetRandomWhitePeer(NetZone, Option<PruningSeed>),
/// Get a list of random peers from the white list,
/// The list will be less than or equal to the provided
/// len.
GetRandomWhitePeers(NetZone, usize),
}
impl std::fmt::Display for AddressBookRequest {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
match self {
Self::HandleNewPeerList(..) => f.write_str("HandleNewPeerList"),
Self::SetPeerSeen(..) => f.write_str("SetPeerSeen"),
Self::BanPeer(..) => f.write_str("BanPeer"),
Self::ConnectedToPeer { .. } => f.write_str("ConnectedToPeer"),
Self::GetAndEmptyAnchorList(_) => f.write_str("GetAndEmptyAnchorList"),
Self::GetRandomGrayPeer(..) => f.write_str("GetRandomGrayPeer"),
Self::GetRandomWhitePeer(..) => f.write_str("GetRandomWhitePeer"),
Self::GetRandomWhitePeers(_, len) => {
f.write_str(&format!("GetRandomWhitePeers, len: {len}"))
}
}
}
}
impl AddressBookRequest {
/// Gets the [`NetZone`] for this request so we can
/// route it to the required address book.
pub fn get_zone(&self) -> NetZone {
match self {
Self::HandleNewPeerList(_, zone) => *zone,
Self::SetPeerSeen(.., zone) => *zone,
Self::BanPeer(.., zone) => *zone,
Self::ConnectedToPeer { zone, .. } => *zone,
Self::GetAndEmptyAnchorList(zone) => *zone,
Self::GetRandomGrayPeer(zone, _) => *zone,
Self::GetRandomWhitePeer(zone, _) => *zone,
Self::GetRandomWhitePeers(zone, _) => *zone,
}
}
}
/// A response from the AddressBook.
#[derive(Debug)]
pub enum AddressBookResponse {
/// The request was handled ok.
Ok,
/// A peer.
Peer(PeerListEntryBase),
/// A list of peers.
Peers(Vec<PeerListEntryBase>),
}

View file

@ -0,0 +1,166 @@
//! This module holds the address books client and [`tower::Service`].
//!
//! To start the address book use [`start_address_book`].
// TODO: Store banned peers persistently.
use std::future::Future;
use std::pin::Pin;
use std::task::Poll;
use futures::channel::{mpsc, oneshot};
use futures::FutureExt;
use tokio::task::{spawn, JoinHandle};
use tower::steer::Steer;
use tower::BoxError;
use tracing::Instrument;
use monero_wire::network_address::NetZone;
use crate::{Config, P2PStore};
use super::address_book::{AddressBook, AddressBookClientRequest};
use super::{AddressBookError, AddressBookRequest, AddressBookResponse};
/// Start the address book.
/// Under the hood this function spawns 3 address books
/// for the 3 [`NetZone`] and combines them into a [`tower::Steer`](Steer).
pub async fn start_address_book<S>(
peer_store: S,
config: Config,
) -> Result<
impl tower::Service<
AddressBookRequest,
Response = AddressBookResponse,
Error = BoxError,
Future = Pin<
Box<dyn Future<Output = Result<AddressBookResponse, BoxError>> + Send + 'static>,
>,
>,
BoxError,
>
where
S: P2PStore,
{
let mut builder = AddressBookBuilder::new(peer_store, config);
let public = builder.build(NetZone::Public).await?;
let tor = builder.build(NetZone::Tor).await?;
let i2p = builder.build(NetZone::I2p).await?;
// This list MUST be in the same order as closuer in the `Steer` func
let books = vec![public, tor, i2p];
Ok(Steer::new(
books,
|req: &AddressBookRequest, _: &[_]| match req.get_zone() {
// This:
NetZone::Public => 0,
NetZone::Tor => 1,
NetZone::I2p => 2,
},
))
}
/// An address book builder.
/// This:
/// - starts the address book
/// - creates and returns the `AddressBookClient`
struct AddressBookBuilder<S> {
peer_store: S,
config: Config,
}
impl<S> AddressBookBuilder<S>
where
S: P2PStore,
{
fn new(peer_store: S, config: Config) -> Self {
AddressBookBuilder { peer_store, config }
}
/// Builds the address book for a specific [`NetZone`]
async fn build(&mut self, zone: NetZone) -> Result<AddressBookClient, AddressBookError> {
let (white, gray, anchor) = self
.peer_store
.load_peers(zone)
.await
.map_err(|e| AddressBookError::PeerStoreError(e))?;
let book = AddressBook::new(
self.config.clone(),
zone,
white,
gray,
anchor,
vec![],
self.peer_store.clone(),
);
let (tx, rx) = mpsc::channel(0);
let book_span = tracing::info_span!("AddressBook", book = book.book_name());
let book_handle = spawn(book.run(rx).instrument(book_span));
Ok(AddressBookClient {
book: tx,
book_handle,
})
}
}
/// The Client for an individual address book.
#[derive(Debug)]
struct AddressBookClient {
/// The channel to pass requests to the address book.
book: mpsc::Sender<AddressBookClientRequest>,
/// The address book task handle.
book_handle: JoinHandle<()>,
}
impl tower::Service<AddressBookRequest> for AddressBookClient {
type Response = AddressBookResponse;
type Error = BoxError;
type Future =
Pin<Box<dyn Future<Output = Result<Self::Response, Self::Error>> + Send + 'static>>;
fn poll_ready(&mut self, cx: &mut std::task::Context<'_>) -> Poll<Result<(), Self::Error>> {
// Check the channel
match self.book.poll_ready(cx) {
Poll::Pending => return Poll::Pending,
Poll::Ready(Ok(())) => (),
Poll::Ready(Err(_)) => {
return Poll::Ready(Err(AddressBookError::AddressBooksChannelClosed.into()))
}
}
// Check the address book task is still running
match self.book_handle.poll_unpin(cx) {
// The address book is still running
Poll::Pending => Poll::Ready(Ok(())),
// The address book task has exited
Poll::Ready(_) => Err(AddressBookError::AddressBookTaskExited)?,
}
}
fn call(&mut self, req: AddressBookRequest) -> Self::Future {
let (tx, rx) = oneshot::channel();
// get the callers span
let span = tracing::debug_span!(parent: &tracing::span::Span::current(), "AddressBook");
let req = AddressBookClientRequest { req, tx, span };
match self.book.try_send(req) {
Err(_e) => {
// I'm assuming all callers will call `poll_ready` first (which they are supposed to)
futures::future::ready(Err(AddressBookError::AddressBooksChannelClosed.into()))
.boxed()
}
Ok(()) => async move {
rx.await
.expect("Address Book will not drop requests until completed")
.map_err(Into::into)
}
.boxed(),
}
}
}

View file

@ -0,0 +1,594 @@
//! This module contains the actual address book logic.
//!
//! The address book is split into multiple [`PeerList`]:
//!
//! - A White list: For peers we have connected to ourselves.
//!
//! - A Gray list: For Peers we have been told about but
//! haven't connected to ourselves.
//!
//! - An Anchor list: This holds peers we are currently
//! connected to that are reachable if we were to
//! connect to them again. For example an inbound proxy
//! connection would not get added to this list as we cant
//! connect to this peer ourselves. Behind the scenes we
//! are just storing the key to a peer in the white list.
//!
use std::collections::{HashMap, HashSet};
use std::future::Future;
use std::pin::Pin;
use std::task::{Context, Poll};
use futures::stream::FuturesUnordered;
use futures::{
channel::{mpsc, oneshot},
FutureExt, Stream, StreamExt,
};
use pin_project::pin_project;
use rand::prelude::SliceRandom;
use cuprate_common::shutdown::is_shutting_down;
use cuprate_common::PruningSeed;
use monero_wire::{messages::PeerListEntryBase, network_address::NetZone, NetworkAddress, PeerID};
use super::{AddressBookError, AddressBookRequest, AddressBookResponse};
use crate::address_book::connection_handle::ConnectionAddressBookHandle;
use crate::{constants::ADDRESS_BOOK_SAVE_INTERVAL, Config, P2PStore};
mod peer_list;
use peer_list::PeerList;
#[cfg(test)]
mod tests;
/// A request sent to the address book task.
pub(crate) struct AddressBookClientRequest {
/// The request
pub req: AddressBookRequest,
/// A oneshot to send the result down
pub tx: oneshot::Sender<Result<AddressBookResponse, AddressBookError>>,
/// The tracing span to keep the context of the request
pub span: tracing::Span,
}
/// An entry in the connected list.
pub struct ConnectionPeerEntry {
/// A oneshot sent from the Connection when it has finished.
connection_handle: ConnectionAddressBookHandle,
/// The connection addr, None if the peer is connected through
/// a hidden network.
addr: Option<NetworkAddress>,
/// If the peer is reachable by our node.
reachable: bool,
/// The last seen timestamp, note: Cuprate may skip updating this
/// field on some inbound messages
last_seen: chrono::NaiveDateTime,
/// The peers pruning seed
pruning_seed: PruningSeed,
/// The peers port.
rpc_port: u16,
/// The peers rpc credits per hash
rpc_credits_per_hash: u32,
}
/// A future that resolves when a peer is unbanned.
#[pin_project(project = EnumProj)]
pub struct BanedPeerFut(Vec<u8>, #[pin] tokio::time::Sleep);
impl Future for BanedPeerFut {
type Output = Vec<u8>;
fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
let mut this = self.project();
match this.1.poll_unpin(cx) {
Poll::Pending => Poll::Pending,
Poll::Ready(_) => Poll::Ready(this.0.clone()),
}
}
}
/// The address book for a specific [`NetZone`]
pub struct AddressBook<PeerStore> {
/// The [`NetZone`] of this address book.
zone: NetZone,
/// A copy of the nodes configuration.
config: Config,
/// The Address books white list.
white_list: PeerList,
/// The Address books gray list.
gray_list: PeerList,
/// The Address books anchor list.
anchor_list: HashSet<NetworkAddress>,
/// The Currently connected peers.
connected_peers: HashMap<PeerID, ConnectionPeerEntry>,
/// A tuple of:
/// - A hashset of [`ban_identifier`](NetworkAddress::ban_identifier)
/// - A [`FuturesUnordered`] which contains futures for every ban_id
/// that will resolve when the ban_id should be un banned.
baned_peers: (HashSet<Vec<u8>>, FuturesUnordered<BanedPeerFut>),
/// The peer store to save the peers to persistent storage
p2p_store: PeerStore,
}
impl<PeerStore: P2PStore> AddressBook<PeerStore> {
/// Creates a new address book for a given [`NetZone`]
pub fn new(
config: Config,
zone: NetZone,
white_peers: Vec<PeerListEntryBase>,
gray_peers: Vec<PeerListEntryBase>,
anchor_peers: Vec<NetworkAddress>,
baned_peers: Vec<(NetworkAddress, chrono::NaiveDateTime)>,
p2p_store: PeerStore,
) -> Self {
let white_list = PeerList::new(white_peers);
let gray_list = PeerList::new(gray_peers);
let anchor_list = HashSet::from_iter(anchor_peers);
let baned_peers = (HashSet::new(), FuturesUnordered::new());
let connected_peers = HashMap::new();
AddressBook {
zone,
config,
white_list,
gray_list,
anchor_list,
connected_peers,
baned_peers,
p2p_store,
}
}
/// Returns the books name (Based on the [`NetZone`])
pub const fn book_name(&self) -> &'static str {
match self.zone {
NetZone::Public => "PublicAddressBook",
NetZone::Tor => "TorAddressBook",
NetZone::I2p => "I2pAddressBook",
}
}
/// Returns the length of the white list
fn len_white_list(&self) -> usize {
self.white_list.len()
}
/// Returns the length of the gray list
fn len_gray_list(&self) -> usize {
self.gray_list.len()
}
/// Returns the length of the anchor list
fn len_anchor_list(&self) -> usize {
self.anchor_list.len()
}
/// Returns the length of the banned list
fn len_banned_list(&self) -> usize {
self.baned_peers.0.len()
}
/// Returns the maximum length of the white list
/// *note this list can grow bigger if we are connected to more
/// than this amount.
fn max_white_peers(&self) -> usize {
self.config.max_white_peers()
}
/// Returns the maximum length of the gray list
fn max_gray_peers(&self) -> usize {
self.config.max_gray_peers()
}
/// Checks if a peer is banned.
fn is_peer_banned(&self, peer: &NetworkAddress) -> bool {
self.baned_peers.0.contains(&peer.ban_identifier())
}
/// Checks if banned peers should be unbanned as the duration has elapsed
fn check_unban_peers(&mut self) {
while let Some(Some(addr)) = Pin::new(&mut self.baned_peers.1).next().now_or_never() {
tracing::debug!("Unbanning peer: {addr:?}");
self.baned_peers.0.remove(&addr);
}
}
/// Checks if peers have disconnected, if they have removing them from the
/// connected and anchor list.
fn check_connected_peers(&mut self) {
let mut remove_from_anchor = vec![];
// We dont have to worry about updating our white list with the information
// before we remove the peers as that happens on every save.
self.connected_peers.retain(|_, peer| {
if !peer.connection_handle.connection_closed() {
// add the peer to the list to get removed from the anchor
if let Some(addr) = peer.addr {
remove_from_anchor.push(addr)
}
false
} else {
true
}
});
// If we are shutting down we want to keep our anchor peers for
// the next time we boot up so we dont remove disconnecting peers
// from the anchor list if we are shutting down.
if !is_shutting_down() {
for peer in remove_from_anchor {
self.anchor_list.remove(&peer);
}
}
}
// Bans the peer and tells the connection tasks of peers with the same ban id to shutdown.
fn ban_peer(
&mut self,
peer: PeerID,
time: std::time::Duration,
) -> Result<(), AddressBookError> {
tracing::debug!("Banning peer: {peer:?} for: {time:?}");
let Some(conn_entry) = self.connected_peers.get(&peer) else {
tracing::debug!("Peer is not in connected list");
return Err(AddressBookError::PeerNotFound);
};
// tell the connection task to finish.
conn_entry.connection_handle.kill_connection();
// try find the NetworkAddress of the peer
let Some(addr) = conn_entry.addr else {
tracing::debug!("Peer does not have an address we can ban");
return Ok(());
};
let ban_id = addr.ban_identifier();
self.white_list.remove_peers_with_ban_id(&ban_id);
self.gray_list.remove_peers_with_ban_id(&ban_id);
// Dont remove from anchor list or connection list as this will happen when
// the connection is closed.
// tell the connection task of peers with the same ban id to shutdown.
for conn in self.connected_peers.values() {
if let Some(addr) = conn.addr {
if addr.ban_identifier() == ban_id {
conn.connection_handle.kill_connection()
}
}
}
// add the ban identifier to the ban list
self.baned_peers.0.insert(ban_id.clone());
self.baned_peers
.1
.push(BanedPeerFut(ban_id, tokio::time::sleep(time)));
Ok(())
}
/// Update the last seen timestamp of a connected peer.
fn update_last_seen(
&mut self,
peer: PeerID,
last_seen: chrono::NaiveDateTime,
) -> Result<(), AddressBookError> {
if let Some(mut peer) = self.connected_peers.get_mut(&peer) {
peer.last_seen = last_seen;
Ok(())
} else {
Err(AddressBookError::PeerNotFound)
}
}
/// adds a peer to the gray list.
fn add_peer_to_gray_list(&mut self, mut peer: PeerListEntryBase) {
if self.white_list.contains_peer(&peer.adr) {
return;
};
if !self.gray_list.contains_peer(&peer.adr) {
peer.last_seen = 0;
self.gray_list.add_new_peer(peer);
}
}
/// handles an incoming peer list,
/// dose some basic validation on the addresses
/// appends the good peers to our book.
fn handle_new_peerlist(
&mut self,
mut peers: Vec<PeerListEntryBase>,
) -> Result<(), AddressBookError> {
let length = peers.len();
tracing::debug!("Received new peer list, length: {length}");
let mut err = None;
peers.retain(|peer| {
if err.is_some() {
false
} else if peer.adr.is_local() || peer.adr.is_loopback() {
false
} else if peer.adr.port() == peer.rpc_port {
false
} else if PruningSeed::try_from(peer.pruning_seed).is_err() {
false
} else if peer.adr.get_zone() != self.zone {
tracing::info!("Received an address from a different network zone, ignoring list.");
err = Some(AddressBookError::PeerSentAnAddressOutOfZone);
false
} else if self.is_peer_banned(&peer.adr) {
false
} else {
true
}
});
if let Some(e) = err {
return Err(e);
} else {
for peer in peers {
self.add_peer_to_gray_list(peer);
}
self.gray_list
.reduce_list(&HashSet::new(), self.max_gray_peers());
Ok(())
}
}
/// Gets a random peer from our gray list.
/// If pruning seed is set we will get a peer with that pruning seed.
fn get_random_gray_peer(
&mut self,
pruning_seed: Option<PruningSeed>,
) -> Option<PeerListEntryBase> {
self.gray_list
.get_random_peer(&mut rand::thread_rng(), pruning_seed.map(Into::into))
.map(|p| *p)
}
/// Gets a random peer from our white list.
/// If pruning seed is set we will get a peer with that pruning seed.
fn get_random_white_peer(
&mut self,
pruning_seed: Option<PruningSeed>,
) -> Option<PeerListEntryBase> {
self.white_list
.get_random_peer(&mut rand::thread_rng(), pruning_seed.map(Into::into))
.map(|p| *p)
}
/// Gets random peers from our white list.
/// will be less than or equal to `len`.
fn get_random_white_peers(&mut self, len: usize) -> Vec<PeerListEntryBase> {
let white_len = self.white_list.len();
let len = if len < white_len { len } else { white_len };
let mut white_peers: Vec<&PeerListEntryBase> = self.white_list.iter_all_peers().collect();
white_peers.shuffle(&mut rand::thread_rng());
white_peers[0..len].iter().map(|peb| **peb).collect()
}
/// Updates an entry in the white list, if the peer is not found and `reachable` is true then
/// the peer will be added to the white list.
fn update_white_list_peer_entry(
&mut self,
addr: &NetworkAddress,
id: PeerID,
conn_entry: &ConnectionPeerEntry,
) -> Result<(), AddressBookError> {
if let Some(peb) = self.white_list.get_peer_mut(addr) {
if peb.pruning_seed == conn_entry.pruning_seed.into() {
return Err(AddressBookError::PeersPruningSeedChanged);
}
peb.id = id;
peb.last_seen = conn_entry.last_seen.timestamp();
peb.rpc_port = conn_entry.rpc_port;
peb.rpc_credits_per_hash = conn_entry.rpc_credits_per_hash;
peb.pruning_seed = conn_entry.pruning_seed.into();
} else if conn_entry.reachable {
// if the peer is reachable add it to our white list
let peb = PeerListEntryBase {
id,
adr: *addr,
last_seen: conn_entry.last_seen.timestamp(),
rpc_port: conn_entry.rpc_port,
rpc_credits_per_hash: conn_entry.rpc_credits_per_hash,
pruning_seed: conn_entry.pruning_seed.into(),
};
self.white_list.add_new_peer(peb);
}
Ok(())
}
/// Handles a new connection, adding it to the white list if the
/// peer is reachable by our node.
fn handle_new_connection(
&mut self,
connection_handle: ConnectionAddressBookHandle,
addr: Option<NetworkAddress>,
id: PeerID,
reachable: bool,
last_seen: chrono::NaiveDateTime,
pruning_seed: PruningSeed,
rpc_port: u16,
rpc_credits_per_hash: u32,
) -> Result<(), AddressBookError> {
let connection_entry = ConnectionPeerEntry {
connection_handle,
addr,
reachable,
last_seen,
pruning_seed,
rpc_port,
rpc_credits_per_hash,
};
if let Some(addr) = addr {
if self.baned_peers.0.contains(&addr.ban_identifier()) {
return Err(AddressBookError::PeerIsBanned);
}
// remove the peer from the gray list as we know it's active.
let _ = self.gray_list.remove_peer(&addr);
if !reachable {
// If we can't reach the peer remove it from the white list as well
let _ = self.white_list.remove_peer(&addr);
} else {
// The peer is reachable, update our white list and add it to the anchor connections.
self.update_white_list_peer_entry(&addr, id, &connection_entry)?;
self.anchor_list.insert(addr);
}
}
self.connected_peers.insert(id, connection_entry);
self.white_list
.reduce_list(&self.anchor_list, self.max_white_peers());
Ok(())
}
/// Get and empties the anchor list, used at startup to
/// connect to some peers we were previously connected to.
fn get_and_empty_anchor_list(&mut self) -> Vec<PeerListEntryBase> {
self.anchor_list
.drain()
.map(|addr| {
self.white_list
.get_peer(&addr)
.expect("If peer is in anchor it must be in white list")
.clone()
})
.collect()
}
/// Handles an [`AddressBookClientRequest`] to the address book.
async fn handle_request(&mut self, req: AddressBookClientRequest) {
let _guard = req.span.enter();
tracing::trace!("received request: {}", req.req);
let res = match req.req {
AddressBookRequest::HandleNewPeerList(new_peers, _) => self
.handle_new_peerlist(new_peers)
.map(|_| AddressBookResponse::Ok),
AddressBookRequest::SetPeerSeen(peer, last_seen, _) => self
.update_last_seen(peer, last_seen)
.map(|_| AddressBookResponse::Ok),
AddressBookRequest::BanPeer(peer, time, _) => {
self.ban_peer(peer, time).map(|_| AddressBookResponse::Ok)
}
AddressBookRequest::ConnectedToPeer {
zone: _,
connection_handle,
addr,
id,
reachable,
last_seen,
pruning_seed,
rpc_port,
rpc_credits_per_hash,
} => self
.handle_new_connection(
connection_handle,
addr,
id,
reachable,
last_seen,
pruning_seed,
rpc_port,
rpc_credits_per_hash,
)
.map(|_| AddressBookResponse::Ok),
AddressBookRequest::GetAndEmptyAnchorList(_) => {
Ok(AddressBookResponse::Peers(self.get_and_empty_anchor_list()))
}
AddressBookRequest::GetRandomGrayPeer(_, pruning_seed) => {
match self.get_random_gray_peer(pruning_seed) {
Some(peer) => Ok(AddressBookResponse::Peer(peer)),
None => Err(AddressBookError::PeerListEmpty),
}
}
AddressBookRequest::GetRandomWhitePeer(_, pruning_seed) => {
match self.get_random_white_peer(pruning_seed) {
Some(peer) => Ok(AddressBookResponse::Peer(peer)),
None => Err(AddressBookError::PeerListEmpty),
}
}
AddressBookRequest::GetRandomWhitePeers(_, len) => {
Ok(AddressBookResponse::Peers(self.get_random_white_peers(len)))
}
};
if let Err(e) = &res {
tracing::debug!("Error when handling request, err: {e}")
}
let _ = req.tx.send(res);
}
/// Updates the white list with the information in the `connected_peers` list.
/// This only updates the `last_seen` timestamp as that's the only thing that should
/// change during connections.
fn update_white_list_with_conn_list(&mut self) {
for (_, peer) in self.connected_peers.iter() {
if peer.reachable {
if let Some(peer_eb) = self.white_list.get_peer_mut(&peer.addr.unwrap()) {
peer_eb.last_seen = peer.last_seen.timestamp();
}
}
}
}
/// Saves the address book to persistent storage.
/// TODO: save the banned peer list.
#[tracing::instrument(level="trace", skip(self), fields(name = self.book_name()) )]
async fn save(&mut self) {
self.update_white_list_with_conn_list();
tracing::trace!(
"white_len: {}, gray_len: {}, anchor_len: {}, banned_len: {}",
self.len_white_list(),
self.len_gray_list(),
self.len_anchor_list(),
self.len_banned_list()
);
let res = self
.p2p_store
.save_peers(
self.zone,
(&self.white_list).into(),
(&self.gray_list).into(),
self.anchor_list.iter().collect(),
)
.await;
match res {
Ok(()) => tracing::trace!("Complete"),
Err(e) => tracing::error!("Error saving address book: {e}"),
}
}
/// Runs the address book task
/// Should be spawned in a task.
pub(crate) async fn run(mut self, mut rx: mpsc::Receiver<AddressBookClientRequest>) {
let mut save_interval = {
let mut interval = tokio::time::interval(ADDRESS_BOOK_SAVE_INTERVAL);
interval.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Skip);
// Interval ticks at 0, interval, 2 interval, ...
// this is just to ignore the first tick
interval.tick().await;
tokio_stream::wrappers::IntervalStream::new(interval).fuse()
};
loop {
self.check_unban_peers();
self.check_connected_peers();
futures::select! {
req = rx.next() => {
if let Some(req) = req {
self.handle_request(req).await
} else {
tracing::debug!("{} req channel closed, saving and shutting down book", self.book_name());
self.save().await;
return;
}
}
_ = save_interval.next() => self.save().await
}
}
}
}

View file

@ -0,0 +1,239 @@
//! This module contains the individual address books peer lists.
//!
use std::collections::{HashMap, HashSet};
use std::hash::Hash;
use cuprate_common::CRYPTONOTE_PRUNING_LOG_STRIPES;
use monero_wire::{messages::PeerListEntryBase, NetworkAddress};
use rand::Rng;
#[cfg(test)]
mod tests;
/// A Peer list in the address book.
///
/// This could either be the white list or gray list.
pub struct PeerList {
/// The peers with their peer data.
peers: HashMap<NetworkAddress, PeerListEntryBase>,
/// An index of Pruning seed to address, so
/// can quickly grab peers with the pruning seed
/// we want.
pruning_idxs: HashMap<u32, Vec<NetworkAddress>>,
/// An index of [`ban_identifier`](NetworkAddress::ban_identifier) to Address
/// to allow us to quickly remove baned peers.
ban_id_idxs: HashMap<Vec<u8>, Vec<NetworkAddress>>,
}
impl<'a> Into<Vec<&'a PeerListEntryBase>> for &'a PeerList {
fn into(self) -> Vec<&'a PeerListEntryBase> {
self.peers.iter().map(|(_, peb)| peb).collect()
}
}
impl PeerList {
/// Creates a new peer list.
pub fn new(list: Vec<PeerListEntryBase>) -> PeerList {
let mut peers = HashMap::with_capacity(list.len());
let mut pruning_idxs = HashMap::with_capacity(2 << CRYPTONOTE_PRUNING_LOG_STRIPES);
let mut ban_id_idxs = HashMap::with_capacity(list.len()); // worse case, every peer has a different NetworkAddress and ban id
for peer in list {
peers.insert(peer.adr, peer);
pruning_idxs
.entry(peer.pruning_seed)
.or_insert_with(Vec::new)
.push(peer.adr);
ban_id_idxs
.entry(peer.adr.ban_identifier())
.or_insert_with(Vec::new)
.push(peer.adr);
}
PeerList {
peers,
pruning_idxs,
ban_id_idxs,
}
}
/// Gets the length of the peer list
pub fn len(&self) -> usize {
self.peers.len()
}
/// Gets the amount of peers with a specific seed.
pub fn len_by_seed(&self, pruning_seed: &u32) -> usize {
self.pruning_idxs
.get(pruning_seed)
.map(|indexes| indexes.len())
.unwrap_or(0)
}
/// Adds a new peer to the peer list
pub fn add_new_peer(&mut self, peer: PeerListEntryBase) {
if let None = self.peers.insert(peer.adr, peer) {
self.pruning_idxs
.entry(peer.pruning_seed)
.or_insert_with(Vec::new)
.push(peer.adr);
self.ban_id_idxs
.entry(peer.adr.ban_identifier())
.or_insert_with(Vec::new)
.push(peer.adr);
}
}
/// Gets a reference to a peer
pub fn get_peer(&self, peer: &NetworkAddress) -> Option<&PeerListEntryBase> {
self.peers.get(peer)
}
/// Returns an iterator over every peer in this peer list
pub fn iter_all_peers(&self) -> impl Iterator<Item = &PeerListEntryBase> {
self.peers.values()
}
/// Returns a random peer.
/// If the pruning seed is specified then we will get a random peer with
/// that pruning seed otherwise we will just get a random peer in the whole
/// list.
pub fn get_random_peer<R: Rng>(
&self,
r: &mut R,
pruning_seed: Option<u32>,
) -> Option<&PeerListEntryBase> {
if let Some(seed) = pruning_seed {
let mut peers = self.get_peers_with_pruning(&seed)?;
let len = self.len_by_seed(&seed);
if len == 0 {
None
} else {
let n = r.gen_range(0..len);
peers.nth(n)
}
} else {
let mut peers = self.iter_all_peers();
let len = self.len();
if len == 0 {
None
} else {
let n = r.gen_range(0..len);
peers.nth(n)
}
}
}
/// Returns a mutable reference to a peer.
pub fn get_peer_mut(&mut self, peer: &NetworkAddress) -> Option<&mut PeerListEntryBase> {
self.peers.get_mut(peer)
}
/// Returns true if the list contains this peer.
pub fn contains_peer(&self, peer: &NetworkAddress) -> bool {
self.peers.contains_key(peer)
}
/// Returns an iterator of peer info of peers with a specific pruning seed.
fn get_peers_with_pruning(
&self,
seed: &u32,
) -> Option<impl Iterator<Item = &PeerListEntryBase>> {
let addrs = self.pruning_idxs.get(seed)?;
Some(addrs.iter().map(move |addr| {
self.peers
.get(addr)
.expect("Address must be in peer list if we have an idx for it")
}))
}
/// Removes a peer from the pruning idx
///
/// MUST NOT BE USED ALONE
fn remove_peer_pruning_idx(&mut self, peer: &PeerListEntryBase) {
remove_peer_idx(&mut self.pruning_idxs, &peer.pruning_seed, &peer.adr)
}
/// Removes a peer from the ban idx
///
/// MUST NOT BE USED ALONE
fn remove_peer_ban_idx(&mut self, peer: &PeerListEntryBase) {
remove_peer_idx(&mut self.ban_id_idxs, &peer.adr.ban_identifier(), &peer.adr)
}
/// Removes a peer from all the indexes
///
/// MUST NOT BE USED ALONE
fn remove_peer_from_all_idxs(&mut self, peer: &PeerListEntryBase) {
self.remove_peer_ban_idx(peer);
self.remove_peer_pruning_idx(peer);
}
/// Removes a peer from the peer list
pub fn remove_peer(&mut self, peer: &NetworkAddress) -> Option<PeerListEntryBase> {
let peer_eb = self.peers.remove(peer)?;
self.remove_peer_from_all_idxs(&peer_eb);
Some(peer_eb)
}
/// Removes all peers with a specific ban id.
pub fn remove_peers_with_ban_id(&mut self, ban_id: &Vec<u8>) {
let Some(addresses) = self.ban_id_idxs.get(ban_id) else {
// No peers to ban
return;
};
for addr in addresses.clone() {
self.remove_peer(&addr);
}
}
/// Tries to reduce the peer list to `new_len`.
///
/// This function could keep the list bigger than `new_len` if `must_keep_peers`s length
/// is larger than new_len, in that case we will remove as much as we can.
pub fn reduce_list(&mut self, must_keep_peers: &HashSet<NetworkAddress>, new_len: usize) {
if new_len >= self.len() {
return;
}
let target_removed = self.len() - new_len;
let mut removed_count = 0;
let mut peers_to_remove: Vec<NetworkAddress> = Vec::with_capacity(target_removed);
for (peer_adr, _) in &self.peers {
if removed_count >= target_removed {
break;
}
if !must_keep_peers.contains(peer_adr) {
peers_to_remove.push(*peer_adr);
removed_count += 1;
}
}
for peer_adr in peers_to_remove {
let _ = self.remove_peer(&peer_adr);
}
}
}
/// Remove a peer from an index.
fn remove_peer_idx<T: Hash + Eq + PartialEq>(
idx_map: &mut HashMap<T, Vec<NetworkAddress>>,
idx: &T,
addr: &NetworkAddress,
) {
if let Some(peer_list) = idx_map.get_mut(idx) {
if let Some(idx) = peer_list.iter().position(|peer_adr| peer_adr == addr) {
peer_list.swap_remove(idx);
} else {
unreachable!("This function will only be called when the peer exists.");
}
} else {
unreachable!("Index must exist if a peer has that index");
}
}

View file

@ -0,0 +1,176 @@
use std::{collections::HashSet, vec};
use monero_wire::{messages::PeerListEntryBase, NetworkAddress};
use rand::Rng;
use super::PeerList;
fn make_fake_peer_list(numb_o_peers: usize) -> PeerList {
let mut peer_list = vec![PeerListEntryBase::default(); numb_o_peers];
for (idx, peer) in peer_list.iter_mut().enumerate() {
let NetworkAddress::IPv4(ip) = &mut peer.adr else {panic!("this test requires default to be ipv4")};
ip.m_ip += idx as u32;
}
PeerList::new(peer_list)
}
fn make_fake_peer_list_with_random_pruning_seeds(numb_o_peers: usize) -> PeerList {
let mut r = rand::thread_rng();
let mut peer_list = vec![PeerListEntryBase::default(); numb_o_peers];
for (idx, peer) in peer_list.iter_mut().enumerate() {
let NetworkAddress::IPv4(ip) = &mut peer.adr else {panic!("this test requires default to be ipv4")};
ip.m_ip += idx as u32;
ip.m_port += r.gen_range(0..15);
peer.pruning_seed = if r.gen_bool(0.4) {
0
} else {
r.gen_range(384..=391)
};
}
PeerList::new(peer_list)
}
#[test]
fn peer_list_reduce_length() {
let mut peer_list = make_fake_peer_list(2090);
let must_keep_peers = HashSet::new();
let target_len = 2000;
peer_list.reduce_list(&must_keep_peers, target_len);
assert_eq!(peer_list.len(), target_len);
}
#[test]
fn peer_list_reduce_length_with_peers_we_need() {
let mut peer_list = make_fake_peer_list(500);
let must_keep_peers = HashSet::from_iter(peer_list.peers.iter().map(|(adr, _)| *adr));
let target_len = 49;
peer_list.reduce_list(&must_keep_peers, target_len);
// we can't remove any of the peers we said we need them all
assert_eq!(peer_list.len(), 500);
}
#[test]
fn peer_list_get_peers_by_pruning_seed() {
let mut r = rand::thread_rng();
let peer_list = make_fake_peer_list_with_random_pruning_seeds(1000);
let seed = if r.gen_bool(0.4) {
0
} else {
r.gen_range(384..=391)
};
let peers_with_seed = peer_list
.get_peers_with_pruning(&seed)
.expect("If you hit this buy a lottery ticket");
for peer in peers_with_seed {
assert_eq!(peer.pruning_seed, seed);
}
assert_eq!(peer_list.len(), 1000);
}
#[test]
fn peer_list_remove_specific_peer() {
let mut peer_list = make_fake_peer_list_with_random_pruning_seeds(100);
let peer = peer_list
.get_random_peer(&mut rand::thread_rng(), None)
.unwrap()
.clone();
assert!(peer_list.remove_peer(&peer.adr).is_some());
let pruning_idxs = peer_list.pruning_idxs;
let peers = peer_list.peers;
for (_, addrs) in pruning_idxs {
addrs.iter().for_each(|adr| assert_ne!(adr, &peer.adr))
}
assert!(!peers.contains_key(&peer.adr));
}
#[test]
fn peer_list_pruning_idxs_are_correct() {
let peer_list = make_fake_peer_list_with_random_pruning_seeds(100);
let mut total_len = 0;
for (seed, list) in peer_list.pruning_idxs {
for peer in list.iter() {
assert_eq!(peer_list.peers.get(peer).unwrap().pruning_seed, seed);
total_len += 1;
}
}
assert_eq!(total_len, peer_list.peers.len())
}
#[test]
fn peer_list_add_new_peer() {
let mut peer_list = make_fake_peer_list(10);
let mut new_peer = PeerListEntryBase::default();
let NetworkAddress::IPv4(ip) = &mut new_peer.adr else {panic!("this test requires default to be ipv4")};
ip.m_ip += 50;
peer_list.add_new_peer(new_peer.clone());
assert_eq!(peer_list.len(), 11);
assert_eq!(peer_list.get_peer(&new_peer.adr), Some(&new_peer));
assert!(peer_list
.pruning_idxs
.get(&new_peer.pruning_seed)
.unwrap()
.contains(&new_peer.adr));
}
#[test]
fn peer_list_add_existing_peer() {
let mut peer_list = make_fake_peer_list(10);
let existing_peer = peer_list
.get_peer(&NetworkAddress::default())
.unwrap()
.clone();
peer_list.add_new_peer(existing_peer.clone());
assert_eq!(peer_list.len(), 10);
assert_eq!(peer_list.get_peer(&existing_peer.adr), Some(&existing_peer));
}
#[test]
fn peer_list_get_non_existent_peer() {
let peer_list = make_fake_peer_list(10);
let mut non_existent_peer = NetworkAddress::default();
let NetworkAddress::IPv4(ip) = &mut non_existent_peer else {panic!("this test requires default to be ipv4")};
ip.m_ip += 50;
assert_eq!(peer_list.get_peer(&non_existent_peer), None);
}
#[test]
fn peer_list_ban_peers() {
let mut peer_list = make_fake_peer_list_with_random_pruning_seeds(100);
let peer = peer_list
.get_random_peer(&mut rand::thread_rng(), None)
.unwrap();
let ban_id = peer.adr.ban_identifier();
assert!(peer_list.contains_peer(&peer.adr));
assert_ne!(peer_list.ban_id_idxs.get(&ban_id).unwrap().len(), 0);
peer_list.remove_peers_with_ban_id(&ban_id);
assert_eq!(peer_list.ban_id_idxs.get(&ban_id).unwrap().len(), 0);
for (addr, _) in peer_list.peers {
assert_ne!(addr.ban_identifier(), ban_id);
}
}

View file

@ -0,0 +1,81 @@
use super::*;
use crate::NetZoneBasicNodeData;
use monero_wire::network_address::IPv4Address;
use rand::Rng;
fn create_random_net_address<R: Rng>(r: &mut R) -> NetworkAddress {
NetworkAddress::IPv4(IPv4Address {
m_ip: r.gen(),
m_port: r.gen(),
})
}
fn create_random_net_addr_vec<R: Rng>(r: &mut R, len: usize) -> Vec<NetworkAddress> {
let mut ret = Vec::with_capacity(len);
for i in 0..len {
ret.push(create_random_net_address(r));
}
ret
}
fn create_random_peer<R: Rng>(r: &mut R) -> PeerListEntryBase {
PeerListEntryBase {
adr: create_random_net_address(r),
pruning_seed: r.gen_range(384..=391),
id: PeerID(r.gen()),
last_seen: r.gen(),
rpc_port: r.gen(),
rpc_credits_per_hash: r.gen(),
}
}
fn create_random_peer_vec<R: Rng>(r: &mut R, len: usize) -> Vec<PeerListEntryBase> {
let mut ret = Vec::with_capacity(len);
for i in 0..len {
ret.push(create_random_peer(r));
}
ret
}
#[derive(Clone)]
pub struct MockPeerStore;
#[async_trait::async_trait]
impl P2PStore for MockPeerStore {
async fn basic_node_data(&mut self) -> Result<Option<NetZoneBasicNodeData>, &'static str> {
unimplemented!()
}
async fn save_basic_node_data(
&mut self,
node_id: &NetZoneBasicNodeData,
) -> Result<(), &'static str> {
unimplemented!()
}
async fn load_peers(
&mut self,
zone: NetZone,
) -> Result<
(
Vec<PeerListEntryBase>,
Vec<PeerListEntryBase>,
Vec<NetworkAddress>,
),
&'static str,
> {
let mut r = rand::thread_rng();
Ok((
create_random_peer_vec(&mut r, 300),
create_random_peer_vec(&mut r, 1500),
create_random_net_addr_vec(&mut r, 50),
))
}
async fn save_peers(
&mut self,
zone: NetZone,
white: Vec<&PeerListEntryBase>,
gray: Vec<&PeerListEntryBase>,
anchor: Vec<&NetworkAddress>,
) -> Result<(), &'static str> {
todo!()
}
}

View file

@ -0,0 +1,110 @@
//! This module contains the address book [`Connection`](crate::peer::connection::Connection) handle
//!
//! # Why do we need a handle between the address book and connection task
//!
//! When banning a peer we need to tell the connection task to close and
//! when we close a connection we need to remove it from our connection
//! and anchor list.
//!
//!
use futures::channel::oneshot;
use tokio_util::sync::CancellationToken;
/// A message sent to tell the address book that a peer has disconnected.
pub struct PeerConnectionClosed;
/// The connection side of the address book to connection
/// communication.
#[derive(Debug)]
pub struct AddressBookConnectionHandle {
connection_closed: Option<oneshot::Sender<PeerConnectionClosed>>,
close: CancellationToken,
}
impl AddressBookConnectionHandle {
/// Returns true if the address book has told us to kill the
/// connection.
pub fn is_canceled(&self) -> bool {
self.close.is_cancelled()
}
}
impl Drop for AddressBookConnectionHandle {
fn drop(&mut self) {
let connection_closed = std::mem::replace(&mut self.connection_closed, None).unwrap();
let _ = connection_closed.send(PeerConnectionClosed);
}
}
/// The address book side of the address book to connection
/// communication.
#[derive(Debug)]
pub struct ConnectionAddressBookHandle {
connection_closed: oneshot::Receiver<PeerConnectionClosed>,
killer: CancellationToken,
}
impl ConnectionAddressBookHandle {
/// Checks if the connection task has closed, returns
/// true if the task has closed
pub fn connection_closed(&mut self) -> bool {
let Ok(mes) = self.connection_closed.try_recv() else {
panic!("This must not be called again after returning true and the connection task must tell us if a connection is closed")
};
match mes {
None => false,
Some(_) => true,
}
}
/// Ends the connection task, the caller of this function should
/// wait to be told the connection has closed by [`check_if_connection_closed`](Self::check_if_connection_closed)
/// before acting on the closed connection.
pub fn kill_connection(&self) {
self.killer.cancel()
}
}
/// Creates a new handle pair that can be given to the connection task and
/// address book respectively.
pub fn new_address_book_connection_handle(
) -> (AddressBookConnectionHandle, ConnectionAddressBookHandle) {
let (tx, rx) = oneshot::channel();
let token = CancellationToken::new();
let ab_c_h = AddressBookConnectionHandle {
connection_closed: Some(tx),
close: token.clone(),
};
let c_ab_h = ConnectionAddressBookHandle {
connection_closed: rx,
killer: token,
};
(ab_c_h, c_ab_h)
}
#[cfg(test)]
mod tests {
use crate::address_book::connection_handle::new_address_book_connection_handle;
#[test]
fn close_connection_from_address_book() {
let (conn_side, mut addr_side) = new_address_book_connection_handle();
assert!(!conn_side.is_canceled());
assert!(!addr_side.connection_closed());
addr_side.kill_connection();
assert!(conn_side.is_canceled());
}
#[test]
fn close_connection_from_connection() {
let (conn_side, mut addr_side) = new_address_book_connection_handle();
assert!(!conn_side.is_canceled());
assert!(!addr_side.connection_closed());
drop(conn_side);
assert!(addr_side.connection_closed());
}
}

78
p2p/src/config.rs Normal file
View file

@ -0,0 +1,78 @@
use cuprate_common::Network;
use monero_wire::messages::{common::PeerSupportFlags, BasicNodeData, PeerID};
use crate::{
constants::{
CUPRATE_SUPPORT_FLAGS, DEFAULT_IN_PEERS, DEFAULT_LOAD_OUT_PEERS_MULTIPLIER,
DEFAULT_TARGET_OUT_PEERS, MAX_GRAY_LIST_PEERS, MAX_WHITE_LIST_PEERS,
},
NodeID,
};
#[derive(Debug, Clone, Copy)]
pub struct Config {
/// Port
my_port: u32,
/// The Network
network: Network,
/// RPC Port
rpc_port: u16,
target_out_peers: usize,
out_peers_load_multiplier: usize,
max_in_peers: usize,
max_white_peers: usize,
max_gray_peers: usize,
}
impl Default for Config {
fn default() -> Self {
Config {
my_port: 18080,
network: Network::MainNet,
rpc_port: 18081,
target_out_peers: DEFAULT_TARGET_OUT_PEERS,
out_peers_load_multiplier: DEFAULT_LOAD_OUT_PEERS_MULTIPLIER,
max_in_peers: DEFAULT_IN_PEERS,
max_white_peers: MAX_WHITE_LIST_PEERS,
max_gray_peers: MAX_GRAY_LIST_PEERS,
}
}
}
impl Config {
pub fn basic_node_data(&self, peer_id: PeerID) -> BasicNodeData {
BasicNodeData {
my_port: self.my_port,
network_id: self.network.network_id(),
peer_id,
support_flags: CUPRATE_SUPPORT_FLAGS,
rpc_port: self.rpc_port,
rpc_credits_per_hash: 0,
}
}
pub fn peerset_total_connection_limit(&self) -> usize {
self.target_out_peers * self.out_peers_load_multiplier + self.max_in_peers
}
pub fn network(&self) -> Network {
self.network
}
pub fn max_white_peers(&self) -> usize {
self.max_white_peers
}
pub fn max_gray_peers(&self) -> usize {
self.max_gray_peers
}
pub fn public_port(&self) -> u32 {
self.my_port
}
pub fn public_rpc_port(&self) -> u16 {
self.rpc_port
}
}

View file

@ -0,0 +1,132 @@
//! This module contains the address book [`Connection`](crate::peer::connection::Connection) handle
//!
//! # Why do we need a handle between the address book and connection task
//!
//! When banning a peer we need to tell the connection task to close and
//! when we close a connection we need to remove it from our connection
//! and anchor list.
//!
//!
use futures::channel::{mpsc, oneshot};
use futures::{FutureExt, SinkExt, StreamExt};
/// A message sent to tell the address book that a peer has disconnected.
pub struct PeerConnectionClosed(Option<std::time::Duration>);
pub enum ConnectionClosed {
Closed(Option<std::time::Duration>),
Running,
}
/// The connection side of the address book to connection
/// communication.
#[derive(Debug)]
pub struct ConnectionHandleAddressBookSide {
connection_closed_rx: oneshot::Receiver<PeerConnectionClosed>,
ban_tx: mpsc::Sender<std::time::Duration>,
}
impl ConnectionHandleAddressBookSide {
pub fn ban_peer(&mut self, duration: std::time::Duration) {
let _ = self.ban_tx.send(duration);
}
pub fn check_connection_closed(&mut self) -> ConnectionClosed {
let connection_closed = self
.connection_closed_rx
.try_recv()
.expect("Will not be cancelled");
match connection_closed {
Some(closed) => return ConnectionClosed::Closed(closed.0),
None => ConnectionClosed::Running,
}
}
}
/// The address book side of the address book to connection
/// communication.
#[derive(Debug)]
pub struct ConnectionHandleConnectionSide {
connection_closed_tx: Option<oneshot::Sender<PeerConnectionClosed>>,
ban_rx: mpsc::Receiver<std::time::Duration>,
ban_holder: Option<std::time::Duration>,
}
impl ConnectionHandleConnectionSide {
pub fn been_banned(&mut self) -> bool {
let ban_time =
self.ban_rx.next().now_or_never().and_then(|inner| {
Some(inner.expect("Handles to the connection task wont be dropped"))
});
let ret = ban_time.is_some();
self.ban_holder = ban_time;
ret
}
}
impl Drop for ConnectionHandleConnectionSide {
fn drop(&mut self) {
let tx = std::mem::replace(&mut self.connection_closed_tx, None)
.expect("Will only be dropped once");
let _ = tx.send(PeerConnectionClosed(self.ban_holder));
}
}
pub struct ConnectionHandleClientSide {
ban_tx: mpsc::Sender<std::time::Duration>,
}
impl ConnectionHandleClientSide {
pub fn ban_peer(&mut self, duration: std::time::Duration) {
let _ = self.ban_tx.send(duration);
}
}
/// Creates a new handle pair that can be given to the connection task and
/// address book respectively.
pub fn new_address_book_connection_handle() -> (
ConnectionHandleConnectionSide,
ConnectionHandleAddressBookSide,
ConnectionHandleClientSide,
) {
let (tx_closed, rx_closed) = oneshot::channel();
let (tx_ban, rx_ban) = mpsc::channel(0);
let c_h_c_s = ConnectionHandleConnectionSide {
connection_closed_tx: Some(tx_closed),
ban_rx: rx_ban,
ban_holder: None,
};
let c_h_a_s = ConnectionHandleAddressBookSide {
connection_closed_rx: rx_closed,
ban_tx: tx_ban.clone(),
};
let c_h_cl_s = ConnectionHandleClientSide { ban_tx: tx_ban };
(c_h_c_s, c_h_a_s, c_h_cl_s)
}
#[cfg(test)]
mod tests {
use super::new_address_book_connection_handle;
#[test]
fn close_connection_from_address_book() {
let (conn_side, mut addr_side) = new_address_book_connection_handle();
assert!(!conn_side.is_canceled());
assert!(!addr_side.connection_closed());
addr_side.kill_connection();
assert!(conn_side.is_canceled());
}
#[test]
fn close_connection_from_connection() {
let (conn_side, mut addr_side) = new_address_book_connection_handle();
assert!(!conn_side.is_canceled());
assert!(!addr_side.connection_closed());
drop(conn_side);
assert!(addr_side.connection_closed());
}
}

View file

@ -0,0 +1,174 @@
//! Counting active connections used by Cuprate.
//!
//! These types can be used to count any kind of active resource.
//! But they are currently used to track the number of open connections.
//!
//! This file was originally from Zebra
use std::{fmt, sync::Arc};
use std::sync::mpsc;
/// A signal sent by a [`Connection`][1] when it closes.
///
/// Used to count the number of open connections.
///
/// [1]: crate::peer::Connection
#[derive(Copy, Clone, Debug, PartialEq, Eq, Hash)]
pub struct ConnectionClosed;
/// A counter for active connections.
///
/// Creates a [`ConnectionTracker`] to track each active connection.
/// When these trackers are dropped, the counter gets notified.
pub struct ActiveConnectionCounter {
/// The number of active peers tracked using this counter.
count: usize,
/// The limit for this type of connection, for diagnostics only.
/// The caller must enforce the limit by ignoring, delaying, or dropping connections.
limit: usize,
/// The label for this connection counter, typically its type.
label: Arc<str>,
/// The channel used to send closed connection notifications.
close_notification_tx: mpsc::Sender<ConnectionClosed>,
/// The channel used to receive closed connection notifications.
close_notification_rx: mpsc::Receiver<ConnectionClosed>,
}
impl fmt::Debug for ActiveConnectionCounter {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
f.debug_struct("ActiveConnectionCounter")
.field("label", &self.label)
.field("count", &self.count)
.field("limit", &self.limit)
.finish()
}
}
impl ActiveConnectionCounter {
/// Create and return a new active connection counter.
pub fn new_counter() -> Self {
Self::new_counter_with(usize::MAX, "Active Connections")
}
/// Create and return a new active connection counter with `limit` and `label`.
/// The caller must check and enforce limits using [`update_count()`](Self::update_count).
pub fn new_counter_with<S: ToString>(limit: usize, label: S) -> Self {
// The number of items in this channel is bounded by the connection limit.
let (close_notification_tx, close_notification_rx) = mpsc::channel();
let label = label.to_string();
Self {
count: 0,
limit,
label: label.into(),
close_notification_rx,
close_notification_tx,
}
}
/// Create and return a new [`ConnectionTracker`], and add 1 to this counter.
///
/// When the returned tracker is dropped, this counter will be notified, and decreased by 1.
pub fn track_connection(&mut self) -> ConnectionTracker {
ConnectionTracker::new(self)
}
/// Check for closed connection notifications, and return the current connection count.
pub fn update_count(&mut self) -> usize {
let previous_connections = self.count;
// We ignore errors here:
// - TryRecvError::Empty means that there are no pending close notifications
// - TryRecvError::Closed is unreachable, because we hold a sender
while let Ok(ConnectionClosed) = self.close_notification_rx.try_recv() {
self.count -= 1;
tracing::debug!(
open_connections = ?self.count,
?previous_connections,
limit = ?self.limit,
label = ?self.label,
"a peer connection was closed",
);
}
tracing::trace!(
open_connections = ?self.count,
?previous_connections,
limit = ?self.limit,
label = ?self.label,
"updated active connection count",
);
#[cfg(feature = "progress-bar")]
self.connection_bar
.set_pos(u64::try_from(self.count).expect("fits in u64"))
.set_len(u64::try_from(self.limit).expect("fits in u64"));
self.count
}
}
impl Drop for ActiveConnectionCounter {
fn drop(&mut self) {}
}
/// A per-connection tracker.
///
/// [`ActiveConnectionCounter`] creates a tracker instance for each active connection.
/// When these trackers are dropped, the counter gets notified.
pub struct ConnectionTracker {
/// The channel used to send closed connection notifications on drop.
close_notification_tx: mpsc::Sender<ConnectionClosed>,
/// The label for this connection counter, typically its type.
label: Arc<str>,
}
impl fmt::Debug for ConnectionTracker {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
f.debug_tuple("ConnectionTracker")
.field(&self.label)
.finish()
}
}
impl ConnectionTracker {
/// Create and return a new active connection tracker, and add 1 to `counter`.
/// All connection trackers share a label with their connection counter.
///
/// When the returned tracker is dropped, `counter` will be notified, and decreased by 1.
fn new(counter: &mut ActiveConnectionCounter) -> Self {
counter.count += 1;
tracing::debug!(
open_connections = ?counter.count,
limit = ?counter.limit,
label = ?counter.label,
"opening a new peer connection",
);
Self {
close_notification_tx: counter.close_notification_tx.clone(),
label: counter.label.clone(),
}
}
}
impl Drop for ConnectionTracker {
/// Notifies the corresponding connection counter that the connection has closed.
fn drop(&mut self) {
tracing::debug!(label = ?self.label, "closing a peer connection");
// We ignore disconnected errors, because the receiver can be dropped
// before some connections are dropped.
//
let _ = self.close_notification_tx.send(ConnectionClosed);
}
}

58
p2p/src/constants.rs Normal file
View file

@ -0,0 +1,58 @@
use core::time::Duration;
use monero_wire::messages::common::PeerSupportFlags;
pub const CUPRATE_SUPPORT_FLAGS: PeerSupportFlags =
PeerSupportFlags::get_support_flag_fluffy_blocks();
pub const CUPRATE_MINIMUM_SUPPORT_FLAGS: PeerSupportFlags =
PeerSupportFlags::get_support_flag_fluffy_blocks();
pub const DEFAULT_TARGET_OUT_PEERS: usize = 20;
pub const DEFAULT_LOAD_OUT_PEERS_MULTIPLIER: usize = 3;
pub const DEFAULT_IN_PEERS: usize = 20;
pub const HANDSHAKE_TIMEOUT: Duration = Duration::from_secs(5);
pub const ADDRESS_BOOK_SAVE_INTERVAL: Duration = Duration::from_secs(60);
pub const ADDRESS_BOOK_BUFFER_SIZE: usize = 3;
pub const PEERSET_BUFFER_SIZE: usize = 3;
/// The maximum size of the address books white list.
/// This number is copied from monerod.
pub const MAX_WHITE_LIST_PEERS: usize = 1000;
/// The maximum size of the address books gray list.
/// This number is copied from monerod.
pub const MAX_GRAY_LIST_PEERS: usize = 5000;
/// The max amount of peers that can be sent in one
/// message.
pub const P2P_MAX_PEERS_IN_HANDSHAKE: usize = 250;
/// The timeout for sending a message to a remote peer,
/// and receiving a response from a remote peer.
pub const REQUEST_TIMEOUT: Duration = Duration::from_secs(20);
/// The default RTT estimate for peer responses.
///
/// We choose a high value for the default RTT, so that new peers must prove they
/// are fast, before we prefer them to other peers. This is particularly
/// important on testnet, which has a small number of peers, which are often
/// slow.
///
/// Make the default RTT slightly higher than the request timeout.
pub const EWMA_DEFAULT_RTT: Duration = Duration::from_secs(REQUEST_TIMEOUT.as_secs() + 1);
/// The decay time for the EWMA response time metric used for load balancing.
///
/// This should be much larger than the `SYNC_RESTART_TIMEOUT`, so we choose
/// better peers when we restart the sync.
pub const EWMA_DECAY_TIME_NANOS: f64 = 200.0 * NANOS_PER_SECOND;
/// The number of nanoseconds in one second.
const NANOS_PER_SECOND: f64 = 1_000_000_000.0;

View file

@ -1 +1,99 @@
pub mod address_book;
pub mod config;
pub mod connection_tracker;
mod constants;
pub mod peer;
pub mod peer_set;
mod protocol;
pub use config::Config;
use rand::Rng;
#[derive(Debug, Clone)]
pub struct NetZoneBasicNodeData {
public: monero_wire::BasicNodeData,
tor: monero_wire::BasicNodeData,
i2p: monero_wire::BasicNodeData,
}
impl NetZoneBasicNodeData {
pub fn basic_node_data(&self, net_zone: &monero_wire::NetZone) -> monero_wire::BasicNodeData {
match net_zone {
monero_wire::NetZone::Public => self.public.clone(),
_ => todo!(),
}
}
pub fn new(config: &Config, node_id: &NodeID) -> Self {
let bnd = monero_wire::BasicNodeData {
my_port: config.public_port(),
network_id: config.network().network_id(),
peer_id: node_id.public,
support_flags: constants::CUPRATE_SUPPORT_FLAGS,
rpc_port: config.public_rpc_port(),
rpc_credits_per_hash: 0,
};
// obviously this is wrong, i will change when i add tor support
NetZoneBasicNodeData {
public: bnd.clone(),
tor: bnd.clone(),
i2p: bnd,
}
}
}
#[derive(Debug, Clone)]
pub struct NodeID {
public: monero_wire::PeerID,
tor: monero_wire::PeerID,
i2p: monero_wire::PeerID,
}
impl NodeID {
pub fn generate() -> NodeID {
let mut rng = rand::thread_rng();
NodeID {
public: monero_wire::PeerID(rng.gen()),
tor: monero_wire::PeerID(rng.gen()),
i2p: monero_wire::PeerID(rng.gen()),
}
}
}
#[async_trait::async_trait]
pub trait P2PStore: Clone + Send + 'static {
/// Loads the peers from the peer store.
/// returns (in order):
/// the white list,
/// the gray list,
/// the anchor list,
/// the ban list
async fn load_peers(
&mut self,
zone: monero_wire::NetZone,
) -> Result<
(
Vec<monero_wire::PeerListEntryBase>, // white list
Vec<monero_wire::PeerListEntryBase>, // gray list
Vec<monero_wire::NetworkAddress>, // anchor list
// Vec<(monero_wire::NetworkAddress, chrono::NaiveDateTime)>, // ban list
),
&'static str,
>;
async fn save_peers(
&mut self,
zone: monero_wire::NetZone,
white: Vec<&monero_wire::PeerListEntryBase>,
gray: Vec<&monero_wire::PeerListEntryBase>,
anchor: Vec<&monero_wire::NetworkAddress>,
// bans: Vec<(&monero_wire::NetworkAddress, &chrono::NaiveDateTime)>, // ban lists
) -> Result<(), &'static str>;
async fn basic_node_data(&mut self) -> Result<Option<NetZoneBasicNodeData>, &'static str>;
async fn save_basic_node_data(
&mut self,
node_id: &NetZoneBasicNodeData,
) -> Result<(), &'static str>;
}

27
p2p/src/peer.rs Normal file
View file

@ -0,0 +1,27 @@
pub mod client;
pub mod connection;
pub mod connector;
pub mod handshaker;
pub mod load_tracked_client;
#[cfg(test)]
mod tests;
use monero_wire::levin::BucketError;
use thiserror::Error;
#[derive(Debug, Error, Clone, Copy)]
pub enum RequestServiceError {}
#[derive(Debug, Error, Clone, Copy)]
pub enum PeerError {
#[error("The connection task has closed.")]
ConnectionTaskClosed,
}
pub use client::Client;
pub use client::ConnectionInfo;
pub use connection::Connection;
pub use connector::{Connector, OutboundConnectorRequest};
pub use handshaker::Handshaker;
pub use load_tracked_client::LoadTrackedClient;

172
p2p/src/peer/client.rs Normal file
View file

@ -0,0 +1,172 @@
use std::pin::Pin;
use std::sync::atomic::AtomicU64;
use std::task::{Context, Poll};
use std::{future::Future, sync::Arc};
use futures::{
channel::{mpsc, oneshot},
FutureExt,
};
use tokio::task::JoinHandle;
use tower::BoxError;
use crate::peer::handshaker::ConnectionAddr;
use cuprate_common::shutdown::set_shutting_down;
use cuprate_common::PruningSeed;
use monero_wire::messages::PeerID;
use monero_wire::{messages::common::PeerSupportFlags, NetworkAddress};
use super::{connection::ClientRequest, PeerError};
use crate::protocol::{InternalMessageRequest, InternalMessageResponse};
pub struct ConnectionInfo {
pub addr: ConnectionAddr,
pub support_flags: PeerSupportFlags,
pub pruning_seed: PruningSeed,
/// Peer ID
pub peer_id: PeerID,
pub rpc_port: u16,
pub rpc_credits_per_hash: u32,
}
pub struct Client {
pub connection_info: Arc<ConnectionInfo>,
/// Used to shut down the corresponding heartbeat.
/// This is always Some except when we take it on drop.
heartbeat_shutdown_tx: Option<oneshot::Sender<()>>,
server_tx: mpsc::Sender<ClientRequest>,
connection_task: JoinHandle<()>,
heartbeat_task: JoinHandle<()>,
}
impl Client {
pub fn new(
connection_info: Arc<ConnectionInfo>,
heartbeat_shutdown_tx: oneshot::Sender<()>,
server_tx: mpsc::Sender<ClientRequest>,
connection_task: JoinHandle<()>,
heartbeat_task: JoinHandle<()>,
) -> Self {
Client {
connection_info,
heartbeat_shutdown_tx: Some(heartbeat_shutdown_tx),
server_tx,
connection_task,
heartbeat_task,
}
}
/// Check if this connection's heartbeat task has exited.
#[allow(clippy::unwrap_in_result)]
fn check_heartbeat(&mut self, cx: &mut Context<'_>) -> Result<(), SharedPeerError> {
let is_canceled = self
.heartbeat_shutdown_tx
.as_mut()
.expect("only taken on drop")
.poll_canceled(cx)
.is_ready();
if is_canceled {
return self.set_task_exited_error(
"heartbeat",
PeerError::HeartbeatTaskExited("Task was cancelled".to_string()),
);
}
match self.heartbeat_task.poll_unpin(cx) {
Poll::Pending => {
// Heartbeat task is still running.
Ok(())
}
Poll::Ready(Ok(Ok(_))) => {
// Heartbeat task stopped unexpectedly, without panic or error.
self.set_task_exited_error(
"heartbeat",
PeerError::HeartbeatTaskExited(
"Heartbeat task stopped unexpectedly".to_string(),
),
)
}
Poll::Ready(Ok(Err(error))) => {
// Heartbeat task stopped unexpectedly, with error.
self.set_task_exited_error(
"heartbeat",
PeerError::HeartbeatTaskExited(error.to_string()),
)
}
Poll::Ready(Err(error)) => {
// Heartbeat task was cancelled.
if error.is_cancelled() {
self.set_task_exited_error(
"heartbeat",
PeerError::HeartbeatTaskExited("Task was cancelled".to_string()),
)
}
// Heartbeat task stopped with panic.
else if error.is_panic() {
panic!("heartbeat task has panicked: {error}");
}
// Heartbeat task stopped with error.
else {
self.set_task_exited_error(
"heartbeat",
PeerError::HeartbeatTaskExited(error.to_string()),
)
}
}
}
}
/// Check if the connection's task has exited.
fn check_connection(&mut self, context: &mut Context<'_>) -> Result<(), PeerError> {
match self.connection_task.poll_unpin(context) {
Poll::Pending => {
// Connection task is still running.
Ok(())
}
Poll::Ready(Ok(())) => {
// Connection task stopped unexpectedly, without panicking.
return Err(PeerError::ConnectionTaskClosed);
}
Poll::Ready(Err(error)) => {
// Connection task stopped unexpectedly with a panic. shut the node down.
tracing::error!("Peer Connection task panicked: {error}, shutting the node down!");
set_shutting_down();
return Err(PeerError::ConnectionTaskClosed);
}
}
}
}
impl tower::Service<InternalMessageRequest> for Client {
type Response = InternalMessageResponse;
type Error = PeerError;
type Future =
Pin<Box<dyn Future<Output = Result<Self::Response, Self::Error>> + Send + 'static>>;
fn poll_ready(
&mut self,
cx: &mut std::task::Context<'_>,
) -> std::task::Poll<Result<(), Self::Error>> {
self.server_tx
.poll_ready(cx)
.map_err(|e| PeerError::ClientChannelClosed.into())
}
fn call(&mut self, req: InternalMessageRequest) -> Self::Future {
let (tx, rx) = oneshot::channel();
match self.server_tx.try_send(ClientRequest { req, tx }) {
Ok(()) => rx
.map(|recv_result| {
recv_result
.expect("ClientRequest oneshot sender must not be dropped before send")
.map_err(|e| e.into())
})
.boxed(),
Err(_e) => {
// TODO: better error handling
futures::future::ready(Err(PeerError::ClientChannelClosed.into())).boxed()
}
}
}
}

206
p2p/src/peer/connection.rs Normal file
View file

@ -0,0 +1,206 @@
use std::collections::HashSet;
use std::sync::atomic::AtomicU64;
use futures::channel::{mpsc, oneshot};
use futures::stream::Fuse;
use futures::{AsyncRead, AsyncWrite, SinkExt, StreamExt};
use crate::address_book::connection_handle::AddressBookConnectionHandle;
use levin::{MessageSink, MessageStream};
use monero_wire::messages::CoreSyncData;
use monero_wire::{levin, Message, NetworkAddress};
use tower::{BoxError, Service, ServiceExt};
use crate::connection_tracker::{self, ConnectionTracker};
use crate::peer::handshaker::ConnectionAddr;
use crate::protocol::{InternalMessageRequest, InternalMessageResponse};
use super::PeerError;
pub struct ClientRequest {
pub req: InternalMessageRequest,
pub tx: oneshot::Sender<Result<InternalMessageResponse, PeerError>>,
}
pub enum State {
WaitingForRequest,
WaitingForResponse {
request: InternalMessageRequest,
tx: oneshot::Sender<Result<InternalMessageResponse, PeerError>>,
},
}
impl State {
pub fn expected_response_id(&self) -> Option<u32> {
match self {
Self::WaitingForRequest => None,
Self::WaitingForResponse { request, tx: _ } => request.expected_id(),
}
}
}
pub struct Connection<Svc, Aw> {
address: ConnectionAddr,
state: State,
sink: MessageSink<Aw, Message>,
client_rx: mpsc::Receiver<ClientRequest>,
/// A connection tracker that reduces the open connection count when dropped.
/// Used to limit the number of open connections in Zebra.
///
/// This field does nothing until it is dropped.
///
/// # Security
///
/// If this connection tracker or `Connection`s are leaked,
/// the number of active connections will appear higher than it actually is.
/// If enough connections leak, Cuprate will stop making new connections.
#[allow(dead_code)]
connection_tracker: ConnectionTracker,
/// A handle to our slot in the address book so we can tell the address
/// book when we disconnect and the address book can tell us to disconnect.
address_book_handle: AddressBookConnectionHandle,
svc: Svc,
}
impl<Svc, Aw> Connection<Svc, Aw>
where
Svc: Service<InternalMessageRequest, Response = InternalMessageResponse, Error = BoxError>,
Aw: AsyncWrite + std::marker::Unpin,
{
pub fn new(
address: ConnectionAddr,
sink: MessageSink<Aw, Message>,
client_rx: mpsc::Receiver<ClientRequest>,
connection_tracker: ConnectionTracker,
address_book_handle: AddressBookConnectionHandle,
svc: Svc,
) -> Connection<Svc, Aw> {
Connection {
address,
state: State::WaitingForRequest,
sink,
client_rx,
connection_tracker,
address_book_handle,
svc,
}
}
async fn handle_response(&mut self, res: InternalMessageResponse) -> Result<(), PeerError> {
let state = std::mem::replace(&mut self.state, State::WaitingForRequest);
if let State::WaitingForResponse { request, tx } = state {
match (request, &res) {
(InternalMessageRequest::Handshake(_), InternalMessageResponse::Handshake(_)) => {}
(
InternalMessageRequest::SupportFlags(_),
InternalMessageResponse::SupportFlags(_),
) => {}
(InternalMessageRequest::TimedSync(_), InternalMessageResponse::TimedSync(res)) => {
}
(
InternalMessageRequest::GetObjectsRequest(req),
InternalMessageResponse::GetObjectsResponse(res),
) => {}
(
InternalMessageRequest::ChainRequest(_),
InternalMessageResponse::ChainResponse(res),
) => {}
(
InternalMessageRequest::FluffyMissingTransactionsRequest(req),
InternalMessageResponse::NewFluffyBlock(blk),
) => {}
(
InternalMessageRequest::GetTxPoolCompliment(_),
InternalMessageResponse::NewTransactions(_),
) => {
// we could check we received no transactions that we said we knew about but thats going to happen later anyway when they get added to our
// mempool
}
_ => return Err(PeerError::ResponseError("Peer sent incorrect response")),
}
// response passed our tests we can send it to the requestor
let _ = tx.send(Ok(res));
Ok(())
} else {
unreachable!("This will only be called when in state WaitingForResponse");
}
}
async fn send_message_to_peer(&mut self, mes: impl Into<Message>) -> Result<(), PeerError> {
Ok(self.sink.send(mes.into()).await?)
}
async fn handle_peer_request(&mut self, req: InternalMessageRequest) -> Result<(), PeerError> {
// we should check contents of peer requests for obvious errors like we do with responses
todo!()
/*
let ready_svc = self.svc.ready().await?;
let res = ready_svc.call(req).await?;
self.send_message_to_peer(res).await
*/
}
async fn handle_client_request(&mut self, req: ClientRequest) -> Result<(), PeerError> {
// check we need a response
if let Some(_) = req.req.expected_id() {
self.state = State::WaitingForResponse {
request: req.req.clone(),
tx: req.tx,
};
}
self.send_message_to_peer(req.req).await
}
async fn state_waiting_for_request(&mut self) -> Result<(), PeerError> {
futures::select! {
peer_message = self.stream.next() => {
match peer_message.expect("MessageStream will never return None") {
Ok(message) => {
self.handle_peer_request(message.try_into().map_err(|_| PeerError::PeerSentUnexpectedResponse)?).await
},
Err(e) => Err(e.into()),
}
},
client_req = self.client_rx.next() => {
self.handle_client_request(client_req.ok_or(PeerError::ClientChannelClosed)?).await
},
}
}
async fn state_waiting_for_response(&mut self) -> Result<(), PeerError> {
// put a timeout on this
let peer_message = self
.stream
.next()
.await
.expect("MessageStream will never return None")?;
if !peer_message.is_request()
&& self.state.expected_response_id() == Some(peer_message.id())
{
if let Ok(res) = peer_message.try_into() {
Ok(self.handle_response(res).await?)
} else {
// im almost certain this is impossible to hit, but im not certain enough to use unreachable!()
Err(PeerError::ResponseError("Peer sent incorrect response"))
}
} else {
if let Ok(req) = peer_message.try_into() {
self.handle_peer_request(req).await
} else {
// this can be hit if the peer sends a protocol response with the wrong id
Err(PeerError::ResponseError("Peer sent incorrect response"))
}
}
}
pub async fn run(mut self) {
loop {
let _res = match self.state {
State::WaitingForRequest => self.state_waiting_for_request().await,
State::WaitingForResponse { request: _, tx: _ } => {
self.state_waiting_for_response().await
}
};
}
}
}

159
p2p/src/peer/connector.rs Normal file
View file

@ -0,0 +1,159 @@
//! Wrapper around handshake logic that also opens a TCP connection.
use std::{
future::Future,
net::SocketAddr,
pin::Pin,
task::{Context, Poll},
};
use futures::{AsyncRead, AsyncWrite, FutureExt};
use monero_wire::{network_address::NetZone, NetworkAddress};
use tokio_util::compat::{TokioAsyncReadCompatExt, TokioAsyncWriteCompatExt};
use tower::{BoxError, Service, ServiceExt};
use tracing::Instrument;
use crate::peer::handshaker::ConnectionAddr;
use crate::{
address_book::{AddressBookRequest, AddressBookResponse},
connection_tracker::ConnectionTracker,
protocol::{
CoreSyncDataRequest, CoreSyncDataResponse, InternalMessageRequest, InternalMessageResponse,
},
};
use super::{
handshaker::{DoHandshakeRequest, Handshaker},
Client,
};
async fn connect(addr: &NetworkAddress) -> Result<(impl AsyncRead, impl AsyncWrite), BoxError> {
match addr.get_zone() {
NetZone::Public => {
let stream =
tokio::net::TcpStream::connect(SocketAddr::try_from(*addr).unwrap()).await?;
let (read, write) = stream.into_split();
Ok((read.compat(), write.compat_write()))
}
_ => unimplemented!(),
}
}
/// A wrapper around [`Handshake`] that opens a connection before
/// forwarding to the inner handshake service. Writing this as its own
/// [`tower::Service`] lets us apply unified timeout policies, etc.
#[derive(Debug, Clone)]
pub struct Connector<Svc, CoreSync, AdrBook>
where
CoreSync: Service<CoreSyncDataRequest, Response = CoreSyncDataResponse, Error = BoxError>
+ Clone
+ Send
+ 'static,
CoreSync::Future: Send,
Svc: Service<InternalMessageRequest, Response = InternalMessageResponse, Error = BoxError>
+ Clone
+ Send
+ 'static,
Svc::Future: Send,
AdrBook: Service<AddressBookRequest, Response = AddressBookResponse, Error = BoxError>
+ Clone
+ Send
+ 'static,
AdrBook::Future: Send,
{
handshaker: Handshaker<Svc, CoreSync, AdrBook>,
}
impl<Svc, CoreSync, AdrBook> Connector<Svc, CoreSync, AdrBook>
where
CoreSync: Service<CoreSyncDataRequest, Response = CoreSyncDataResponse, Error = BoxError>
+ Clone
+ Send
+ 'static,
CoreSync::Future: Send,
Svc: Service<InternalMessageRequest, Response = InternalMessageResponse, Error = BoxError>
+ Clone
+ Send
+ 'static,
Svc::Future: Send,
AdrBook: Service<AddressBookRequest, Response = AddressBookResponse, Error = BoxError>
+ Clone
+ Send
+ 'static,
AdrBook::Future: Send,
{
pub fn new(handshaker: Handshaker<Svc, CoreSync, AdrBook>) -> Self {
Connector { handshaker }
}
}
/// A connector request.
/// Contains the information needed to make an outbound connection to the peer.
pub struct OutboundConnectorRequest {
/// The Monero listener address of the peer.
pub addr: NetworkAddress,
/// A connection tracker that reduces the open connection count when dropped.
///
/// Used to limit the number of open connections in Cuprate.
pub connection_tracker: ConnectionTracker,
}
impl<Svc, CoreSync, AdrBook> Service<OutboundConnectorRequest> for Connector<Svc, CoreSync, AdrBook>
where
CoreSync: Service<CoreSyncDataRequest, Response = CoreSyncDataResponse, Error = BoxError>
+ Clone
+ Send
+ 'static,
CoreSync::Future: Send,
Svc: Service<InternalMessageRequest, Response = InternalMessageResponse, Error = BoxError>
+ Clone
+ Send
+ 'static,
Svc::Future: Send,
AdrBook: Service<AddressBookRequest, Response = AddressBookResponse, Error = BoxError>
+ Clone
+ Send
+ 'static,
AdrBook::Future: Send,
{
type Response = (NetworkAddress, Client);
type Error = BoxError;
type Future =
Pin<Box<dyn Future<Output = Result<Self::Response, Self::Error>> + Send + 'static>>;
fn poll_ready(&mut self, _cx: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
Poll::Ready(Ok(()))
}
fn call(&mut self, req: OutboundConnectorRequest) -> Self::Future {
let OutboundConnectorRequest {
addr: address,
connection_tracker,
}: OutboundConnectorRequest = req;
let hs = self.handshaker.clone();
let connector_span = tracing::info_span!("connector", peer = ?address);
async move {
let (read, write) = connect(&address).await?;
let client = hs
.oneshot(DoHandshakeRequest {
read,
write,
addr: ConnectionAddr::OutBound { address },
connection_tracker,
})
.await?;
Ok((address, client))
}
.instrument(connector_span)
.boxed()
}
}

627
p2p/src/peer/handshaker.rs Normal file
View file

@ -0,0 +1,627 @@
/// This module contains the logic for turning [`AsyncRead`] and [`AsyncWrite`]
/// into [`Client`] and [`Connection`].
///
/// The main entry point is modeled as a [`tower::Service`] the struct being
/// [`Handshaker`]. The [`Handshaker`] accepts handshake requests: [`DoHandshakeRequest`]
/// and creates a state machine that's drives the handshake forward: [`HandshakeSM`] and
/// eventually outputs a [`Client`] and [`Connection`].
///
use std::future::Future;
use std::net::SocketAddr;
use std::pin::Pin;
use futures::{channel::mpsc, sink::Sink, SinkExt, Stream};
use futures::{FutureExt, StreamExt};
use thiserror::Error;
use tokio::{
io::{AsyncRead, AsyncWrite},
time,
};
use tokio_util::codec::{FramedRead, FramedWrite};
use tower::{BoxError, Service, ServiceExt};
use tracing::Instrument;
use cuprate_common::{Network, PruningSeed};
use monero_wire::messages::admin::SupportFlagsResponse;
use monero_wire::{
messages::{
admin::{HandshakeRequest, HandshakeResponse},
common::PeerSupportFlags,
BasicNodeData, CoreSyncData, PeerID, PeerListEntryBase,
},
BucketError, Message, MoneroWireCodec, NetZone, NetworkAddress, RequestMessage,
ResponseMessage,
};
use super::{
client::{Client, ConnectionInfo},
connection::Connection,
PeerError,
};
use crate::address_book::connection_handle::new_address_book_connection_handle;
use crate::address_book::{AddressBookRequest, AddressBookResponse};
use crate::connection_tracker::ConnectionTracker;
use crate::constants::{
CUPRATE_MINIMUM_SUPPORT_FLAGS, HANDSHAKE_TIMEOUT, P2P_MAX_PEERS_IN_HANDSHAKE,
};
use crate::protocol::{
CoreSyncDataRequest, CoreSyncDataResponse, Direction, InternalMessageRequest,
InternalMessageResponse,
};
use crate::NetZoneBasicNodeData;
/// Possible handshake errors
#[derive(Debug, Error)]
pub enum HandShakeError {
/// The peer did not complete the handshake fast enough.
#[error("The peer did not complete the handshake fast enough")]
PeerTimedOut,
/// The Peer has non-standard pruning.
#[error("The peer has a weird pruning scheme")]
PeerClaimedWeirdPruning,
/// The peer does not have the minimum support flags
#[error("The peer does not have the minimum support flags")]
PeerDoesNotHaveTheMinimumSupportFlags,
/// The peer is not on the network we are on (MAINNET|TESTNET|STAGENET)
#[error("The peer is on a different network")]
PeerIsOnADifferentNetwork,
/// The peer sent us too many peers, more than [`P2P_MAX_PEERS_IN_HANDSHAKE`]
#[error("The peer sent too many peers, considered spamming")]
PeerSentTooManyPeers,
/// The peer sent an incorrect response
#[error("The peer sent a wrong response to our handshake")]
PeerSentWrongResponse,
/// Error communicating with peer
#[error("Bucket error while communicating with peer: {0}")]
BucketError(#[from] BucketError),
}
/// An address used to connect to a peer.
#[derive(Debug, Copy, Clone)]
pub enum ConnectionAddr {
/// Outbound connection to another peer.
OutBound { address: NetworkAddress },
/// An inbound direct connection to our node.
InBoundDirect { transient_address: SocketAddr },
/// An inbound connection through a hidden network
/// like Tor/ I2p
InBoundProxy { net_zone: NetZone },
}
impl ConnectionAddr {
/// Gets the [`NetworkAddress`] of this connection.
pub fn get_network_address(&self, port: u16) -> Option<NetworkAddress> {
match self {
ConnectionAddr::OutBound { address } => Some(*address),
_ => None,
}
}
/// Gets the [`NetZone`] of this connection.
pub fn get_zone(&self) -> NetZone {
match self {
ConnectionAddr::OutBound { address } => address.get_zone(),
ConnectionAddr::InBoundDirect { .. } => NetZone::Public,
ConnectionAddr::InBoundProxy { net_zone } => *net_zone,
}
}
/// Gets the [`Direction`] of this connection.
pub fn direction(&self) -> Direction {
match self {
ConnectionAddr::OutBound { .. } => Direction::Outbound,
ConnectionAddr::InBoundDirect { .. } | ConnectionAddr::InBoundProxy { .. } => {
Direction::Inbound
}
}
}
}
/// A request to handshake with a peer.
pub struct DoHandshakeRequest<W, R> {
/// The read-half of the connection.
pub read: R,
/// The write-half of the connection.
pub write: W,
/// The [`ConnectionAddr`] of this connection.
pub addr: ConnectionAddr,
/// The [`ConnectionTracker`] of this connection.
pub connection_tracker: ConnectionTracker,
}
/// A [`Service`] that accepts [`DoHandshakeRequest`] and
/// produces a [`Client`] and [`Connection`].
#[derive(Debug, Clone)]
pub struct Handshaker<Svc, CoreSync, AdrBook> {
/// A collection of our [`BasicNodeData`] for each [`NetZone`]
/// for more info see: [`NetZoneBasicNodeData`]
basic_node_data: NetZoneBasicNodeData,
/// The [`Network`] our node is using
network: Network,
/// The span [`Connection`] tasks will be [`tracing::instrument`]ed with
parent_span: tracing::Span,
/// The address book [`Service`]
address_book: AdrBook,
/// A [`Service`] to handle incoming [`CoreSyncData`] and to get
/// our [`CoreSyncData`].
core_sync_svc: CoreSync,
/// A service given to the [`Connection`] task to answer incoming
/// requests to our node.
peer_request_service: Svc,
}
impl<Svc, CoreSync, AdrBook> Handshaker<Svc, CoreSync, AdrBook> {
pub fn new(
basic_node_data: NetZoneBasicNodeData,
network: Network,
address_book: AdrBook,
core_sync_svc: CoreSync,
peer_request_service: Svc,
) -> Self {
Handshaker {
basic_node_data,
network,
parent_span: tracing::Span::current(),
address_book,
core_sync_svc,
peer_request_service,
}
}
}
impl<Svc, CoreSync, AdrBook, W, R> Service<DoHandshakeRequest<W, R>>
for Handshaker<Svc, CoreSync, AdrBook>
where
CoreSync: Service<CoreSyncDataRequest, Response = CoreSyncDataResponse, Error = BoxError>
+ Clone
+ Send
+ 'static,
CoreSync::Future: Send,
Svc: Service<InternalMessageRequest, Response = InternalMessageResponse, Error = BoxError>
+ Clone
+ Send
+ 'static,
Svc::Future: Send,
AdrBook: Service<AddressBookRequest, Response = AddressBookResponse, Error = BoxError>
+ Clone
+ Send
+ 'static,
AdrBook::Future: Send,
W: AsyncWrite + Unpin + Send + 'static,
R: AsyncRead + Unpin + Send + 'static,
{
type Response = Client;
type Error = BoxError;
type Future =
Pin<Box<dyn Future<Output = Result<Self::Response, Self::Error>> + Send + 'static>>;
fn poll_ready(
&mut self,
_cx: &mut std::task::Context<'_>,
) -> std::task::Poll<Result<(), Self::Error>> {
// We are always ready.
std::task::Poll::Ready(Ok(()))
}
fn call(&mut self, req: DoHandshakeRequest<W, R>) -> Self::Future {
let DoHandshakeRequest {
read,
write,
addr,
connection_tracker,
} = req;
// create the levin message stream/ sink.
let peer_stream = FramedRead::new(read, MoneroWireCodec::default());
let peer_sink = FramedWrite::new(write, MoneroWireCodec::default());
// The span the handshake state machine will use
let span = tracing::debug_span!("Handshaker");
// The span the connection task will use.
let connection_span = tracing::debug_span!(parent: &self.parent_span, "Connection");
// clone the services that the handshake state machine will need.
let core_sync_svc = self.core_sync_svc.clone();
let address_book = self.address_book.clone();
let peer_request_service = self.peer_request_service.clone();
let state_machine = HandshakeSM {
peer_sink,
peer_stream,
addr,
network: self.network,
basic_node_data: self.basic_node_data.basic_node_data(&addr.get_zone()),
address_book,
core_sync_svc,
peer_request_service,
connection_span,
connection_tracker,
state: HandshakeState::Start,
};
// although callers should use a timeout do one here as well just to be safe.
let ret = time::timeout(HANDSHAKE_TIMEOUT, state_machine.do_handshake());
async move {
match ret.await {
Ok(handshake) => handshake,
Err(_) => Err(HandShakeError::PeerTimedOut.into()),
}
}
.instrument(span)
.boxed()
}
}
/// The states a handshake can be in.
enum HandshakeState {
/// The initial state.
/// if this is an inbound handshake then this state means we
/// are waiting for a [`HandshakeRequest`].
Start,
/// Waiting for a [`HandshakeResponse`].
WaitingForHandshakeResponse,
/// Waiting for a [`SupportFlagsResponse`]
/// This contains the peers node data.
WaitingForSupportFlagResponse(BasicNodeData, CoreSyncData),
/// The handshake is complete.
/// This contains the peers node data.
Complete(BasicNodeData, CoreSyncData),
}
impl HandshakeState {
/// Returns true if the handshake is completed.
pub fn is_complete(&self) -> bool {
matches!(self, Self::Complete(..))
}
/// returns the peers [`BasicNodeData`] and [`CoreSyncData`] if the peer
/// is in state [`HandshakeState::Complete`].
pub fn peer_data(self) -> Option<(BasicNodeData, CoreSyncData)> {
match self {
HandshakeState::Complete(bnd, coresync) => Some((bnd, coresync)),
_ => None,
}
}
}
/// The state machine that drives a handshake forward and
/// accepts requests (that can happen during a handshake)
/// from a peer.
struct HandshakeSM<Svc, CoreSync, AdrBook, W, R> {
/// The levin [`FramedWrite`] for the peer.
peer_sink: W,
/// The levin [`FramedRead`] for the peer.
peer_stream: R,
/// The [`ConnectionAddr`] for the peer.
addr: ConnectionAddr,
/// The [`Network`] we are on.
network: Network,
/// Our [`BasicNodeData`].
basic_node_data: BasicNodeData,
/// The address book [`Service`]
address_book: AdrBook,
/// The core sync [`Service`] to handle incoming
/// [`CoreSyncData`] and to retrieve ours.
core_sync_svc: CoreSync,
/// The [`Service`] passed to the [`Connection`]
/// task to handle incoming peer requests.
peer_request_service: Svc,
/// The [`tracing::Span`] the [`Connection`] task
/// will be [`tracing::instrument`]ed with.
connection_span: tracing::Span,
/// A connection tracker to keep track of the
/// number of connections Cuprate is making.
connection_tracker: ConnectionTracker,
state: HandshakeState,
}
impl<Svc, CoreSync, AdrBook, W, R> HandshakeSM<Svc, CoreSync, AdrBook, W, R>
where
CoreSync: Service<CoreSyncDataRequest, Response = CoreSyncDataResponse, Error = BoxError>
+ Clone
+ Send
+ 'static,
CoreSync::Future: Send,
Svc: Service<InternalMessageRequest, Response = InternalMessageResponse, Error = BoxError>
+ Clone
+ Send
+ 'static,
Svc::Future: Send,
AdrBook: Service<AddressBookRequest, Response = AddressBookResponse, Error = BoxError>
+ Clone
+ Send
+ 'static,
AdrBook::Future: Send,
W: Sink<Message, Error = BucketError> + Unpin,
R: Stream<Item = Result<Message, BucketError>> + Unpin,
{
/// Gets our [`CoreSyncData`] from the `core_sync_svc`.
async fn get_our_core_sync(&mut self) -> Result<CoreSyncData, BoxError> {
let core_sync_svc = self.core_sync_svc.ready().await?;
let CoreSyncDataResponse::Ours(core_sync) = core_sync_svc.call(CoreSyncDataRequest::GetOurs).await? else {
unreachable!("The Service must give correct responses");
};
tracing::trace!("Got core sync data: {core_sync:?}");
Ok(core_sync)
}
/// Sends a [`HandshakeRequest`] to the peer.
async fn send_handshake_req(
&mut self,
node_data: BasicNodeData,
payload_data: CoreSyncData,
) -> Result<(), HandShakeError> {
let handshake_req = HandshakeRequest {
node_data,
payload_data,
};
tracing::trace!("Sending handshake request: {handshake_req:?}");
let message: Message = Message::Request(RequestMessage::Handshake(handshake_req));
self.peer_sink.send(message).await?;
Ok(())
}
/// Sends a [`SupportFlagsRequest`] to the peer.
/// This is done when a peer sends no support flags in their
/// [`HandshakeRequest`] or [`HandshakeResponse`].
///
/// *note because Cuprate has minimum required support flags this won't
/// happeen but is included here just in case this changes.
async fn send_support_flag_req(&mut self) -> Result<(), HandShakeError> {
tracing::trace!("Peer sent no support flags, sending request");
let message: Message = Message::Request(RequestMessage::SupportFlags);
self.peer_sink.send(message).await?;
Ok(())
}
/// Handles an incoming [`HandshakeResponse`].
async fn handle_handshake_response(&mut self, res: HandshakeResponse) -> Result<(), BoxError> {
let HandshakeResponse {
node_data: peer_node_data,
payload_data: peer_core_sync,
local_peerlist_new,
} = res;
// Check the peer is on the correct network.
if peer_node_data.network_id != self.network.network_id() {
tracing::debug!("Handshake failed: peer is on a different network");
return Err(HandShakeError::PeerIsOnADifferentNetwork.into());
}
// Check the peer meets the minimum support flags.
if !peer_node_data
.support_flags
.contains(&CUPRATE_MINIMUM_SUPPORT_FLAGS)
{
tracing::debug!("Handshake failed: peer does not have minimum required support flags");
return Err(HandShakeError::PeerDoesNotHaveTheMinimumSupportFlags.into());
}
// Check the peer didn't send too many peers.
if local_peerlist_new.len() > P2P_MAX_PEERS_IN_HANDSHAKE {
tracing::debug!("Handshake failed: peer sent too many peers in response");
return Err(HandShakeError::PeerSentTooManyPeers.into());
}
// Tell the sync mgr about the new incoming core sync data.
self.core_sync_svc
.ready()
.await?
.call(CoreSyncDataRequest::NewIncoming(peer_core_sync.clone()))
.await?;
// Tell the address book about the new peers
self.address_book
.ready()
.await?
.call(AddressBookRequest::HandleNewPeerList(
local_peerlist_new,
self.addr.get_zone(),
))
.await?;
// This won't actually happen (as long as we have a none 0 minimum support flags)
// it's just included here for completeness.
if peer_node_data.support_flags.is_empty() {
self.send_support_flag_req().await?;
self.state =
HandshakeState::WaitingForSupportFlagResponse(peer_node_data, peer_core_sync);
} else {
// this will always happen.
self.state = HandshakeState::Complete(peer_node_data, peer_core_sync);
}
Ok(())
}
/// Handles a [`MessageResponse`].
async fn handle_message_response(&mut self, response: ResponseMessage) -> Result<(), BoxError> {
// The functions called here will change the state of the HandshakeSM so `HandshakeState::Start`
// is just used as a place holder.
//
// doing this allows us to not clone the BasicNodeData and CoreSyncData for WaitingForSupportFlagResponse.
let prv_state = std::mem::replace(&mut self.state, HandshakeState::Start);
match (prv_state, response) {
(
HandshakeState::WaitingForHandshakeResponse,
ResponseMessage::Handshake(handshake),
) => self.handle_handshake_response(handshake).await,
(
HandshakeState::WaitingForSupportFlagResponse(mut bnd, coresync),
ResponseMessage::SupportFlags(support_flags),
) => {
bnd.support_flags = support_flags.support_flags;
self.state = HandshakeState::Complete(bnd, coresync);
Ok(())
}
_ => Err(HandShakeError::PeerSentWrongResponse.into()),
}
}
/// Sends our [`PeerSupportFlags`] to the peer.
async fn send_support_flags(
&mut self,
support_flags: PeerSupportFlags,
) -> Result<(), HandShakeError> {
let message = Message::Response(ResponseMessage::SupportFlags(SupportFlagsResponse {
support_flags,
}));
self.peer_sink.send(message).await?;
Ok(())
}
/// Attempts an outbound handshake with the peer.
async fn do_outbound_handshake(&mut self) -> Result<(), BoxError> {
// Get the data needed for the handshake request.
let core_sync = self.get_our_core_sync().await?;
// send the handshake request.
self.send_handshake_req(self.basic_node_data.clone(), core_sync)
.await?;
// set the state to waiting for a response.
self.state = HandshakeState::WaitingForHandshakeResponse;
while !self.state.is_complete() {
match self.peer_stream.next().await {
Some(mes) => {
let mes = mes?;
match mes {
Message::Request(RequestMessage::SupportFlags) => {
// The only request we should be getting during an outbound handshake
// is a support flag request.
self.send_support_flags(self.basic_node_data.support_flags)
.await?
}
Message::Response(response) => {
// This could be a handshake response or a support flags response.
self.handle_message_response(response).await?
}
_ => return Err(HandShakeError::PeerSentWrongResponse.into()),
}
}
None => unreachable!("peer_stream wont return None"),
}
}
Ok(())
}
/// Completes a handshake with a peer.
async fn do_handshake(mut self) -> Result<Client, BoxError> {
let mut peer_reachable = false;
match self.addr.direction() {
Direction::Outbound => {
self.do_outbound_handshake().await?;
// If this is an outbound handshake then obviously the peer
// is reachable.
peer_reachable = true
}
Direction::Inbound => todo!(),
}
let (server_tx, server_rx) = mpsc::channel(0);
let (peer_node_data, coresync) = self
.state
.peer_data()
.expect("We must be in state complete to be here");
let pruning_seed = PruningSeed::try_from(coresync.pruning_seed).map_err(|e| Box::new(e))?;
// create the handle between the Address book and the connection task to
// allow the address book to shutdown the connection task and to update
// the address book when the connection is closed.
let (book_connection_side_handle, connection_book_side_handle) =
new_address_book_connection_handle();
// tell the address book about the new connection.
self.address_book
.ready()
.await?
.call(AddressBookRequest::ConnectedToPeer {
zone: self.addr.get_zone(),
connection_handle: connection_book_side_handle,
addr: self.addr.get_network_address(
peer_node_data
.my_port
.try_into()
.map_err(|_| "Peer sent a port that does not fit into a u16")?,
),
id: peer_node_data.peer_id,
reachable: peer_reachable,
last_seen: chrono::Utc::now().naive_utc(),
pruning_seed: pruning_seed.clone(),
rpc_port: peer_node_data.rpc_port,
rpc_credits_per_hash: peer_node_data.rpc_credits_per_hash,
})
.await?;
// This block below is for keeping the last seen times in the address book
// upto date. We only update the last seen times on timed syncs to reduce
// the load on the address book.
//
// first clone the items needed
let mut address_book = self.address_book.clone();
let peer_id = peer_node_data.peer_id;
let net_zone = self.addr.get_zone();
/*
let peer_stream = self.peer_stream.then(|mes| async move {
if let Ok(mes) = &mes {
if mes.id() == TimedSync::ID {
if let Ok(ready_book) = address_book.ready().await {
// we dont care about address book errors here, If there is a problem
// with the address book the node will get shutdown.
let _ = ready_book
.call(AddressBookRequest::SetPeerSeen(
peer_id,
chrono::Utc::now().naive_utc(),
net_zone,
))
.await;
}
}
}
// return the message
mes
});
*/
let connection = Connection::new(
self.addr,
self.peer_sink,
server_rx,
self.connection_tracker,
book_connection_side_handle,
self.peer_request_service,
);
let connection_task = tokio::task::spawn(connection.run().instrument(self.connection_span));
let connection_info = ConnectionInfo {
addr: self.addr,
support_flags: peer_node_data.support_flags,
pruning_seed,
peer_id: peer_node_data.peer_id,
rpc_port: peer_node_data.rpc_port,
rpc_credits_per_hash: peer_node_data.rpc_credits_per_hash,
};
let client = Client::new(connection_info.into(), /* futures::futures_channel::oneshot::Sender<()> */, server_tx, connection_task, /* tokio::task::JoinHandle<()> */);
Ok(client)
}
}

View file

@ -0,0 +1,96 @@
//! A peer connection service wrapper type to handle load tracking and provide access to the
//! reported protocol version.
use std::sync::atomic::Ordering;
use std::{
sync::Arc,
task::{Context, Poll},
};
use cuprate_common::PruningSeed;
use tower::{
load::{Load, PeakEwma},
Service,
};
use crate::{
constants::{EWMA_DECAY_TIME_NANOS, EWMA_DEFAULT_RTT},
peer::{Client, ConnectionInfo},
};
/// A client service wrapper that keeps track of its load.
///
/// It also keeps track of the peer's reported protocol version.
pub struct LoadTrackedClient {
/// A service representing a connected peer, wrapped in a load tracker.
service: PeakEwma<Client>,
/// The metadata for the connected peer `service`.
connection_info: Arc<ConnectionInfo>,
}
impl LoadTrackedClient {
pub fn supports_fluffy_blocks(&self) -> bool {
self.connection_info.support_flags.supports_fluffy_blocks()
}
pub fn current_height(&self) -> u64 {
self.connection_info.peer_height.load(Ordering::SeqCst)
}
pub fn pruning_seed(&self) -> PruningSeed {
self.connection_info.pruning_seed
}
pub fn has_full_block(&self, block_height: u64) -> bool {
let seed = self.pruning_seed();
let Some(log_stripes) = seed.get_log_stripes() else {
return true;
};
seed.will_have_complete_block(block_height, self.current_height(), log_stripes)
}
}
/// Create a new [`LoadTrackedClient`] wrapping the provided `client` service.
impl From<Client> for LoadTrackedClient {
fn from(client: Client) -> Self {
let connection_info = client.connection_info.clone();
let service = PeakEwma::new(
client,
EWMA_DEFAULT_RTT,
EWMA_DECAY_TIME_NANOS,
tower::load::CompleteOnResponse::default(),
);
LoadTrackedClient {
service,
connection_info,
}
}
}
impl<Request> Service<Request> for LoadTrackedClient
where
Client: Service<Request>,
{
type Response = <Client as Service<Request>>::Response;
type Error = <Client as Service<Request>>::Error;
type Future = <PeakEwma<Client> as Service<Request>>::Future;
fn poll_ready(&mut self, context: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
self.service.poll_ready(context)
}
fn call(&mut self, request: Request) -> Self::Future {
self.service.call(request)
}
}
impl Load for LoadTrackedClient {
type Metric = <PeakEwma<Client> as Load>::Metric;
fn load(&self) -> Self::Metric {
self.service.load()
}
}

1
p2p/src/peer/tests.rs Normal file
View file

@ -0,0 +1 @@
mod handshake;

View file

@ -0,0 +1 @@
pub use crate::peer::handshaker::Handshaker;

7
p2p/src/peer_set.rs Normal file
View file

@ -0,0 +1,7 @@
pub mod initialize;
pub mod services;
pub mod set;
mod unready_service;
use unready_service::UnreadyService;

View file

@ -0,0 +1,94 @@
use futures::TryStreamExt;
use futures::{future, StreamExt};
use tower::buffer::Buffer;
use tower::discover::Change;
use tower::util::BoxService;
use tower::{BoxError, Layer, Service};
use monero_wire::NetworkAddress;
use crate::address_book::{start_address_book, AddressBookRequest, AddressBookResponse};
use crate::constants;
use crate::protocol::{
CoreSyncDataRequest, CoreSyncDataResponse, InternalMessageRequest, InternalMessageResponse,
};
use crate::{peer, Config, NetZoneBasicNodeData, P2PStore};
use super::set::{MorePeers, PeerSet};
type DiscoveredPeer = Result<(NetworkAddress, peer::Client), BoxError>;
/*
pub async fn init<Svc, CoreSync, P2PS>(
config: Config,
inbound_service: Svc,
core_sync_svc: CoreSync,
mut p2p_store: P2PS,
) -> Result<
Buffer<BoxService<AddressBookRequest, AddressBookResponse, BoxError>, AddressBookRequest>,
BoxError,
>
where
Svc: Service<InternalMessageRequest, Response = InternalMessageResponse, Error = BoxError>
+ Clone
+ Send
+ 'static,
Svc::Future: Send,
CoreSync: Service<CoreSyncDataRequest, Response = CoreSyncDataResponse, Error = BoxError>
+ Clone
+ Send
+ 'static,
CoreSync::Future: Send,
P2PS: P2PStore,
{
let basic_node_data: NetZoneBasicNodeData = match p2p_store.basic_node_data().await? {
Some(bnd) => bnd,
None => {
let node_id = crate::NodeID::generate();
let bnd = NetZoneBasicNodeData::new(&config, &node_id);
p2p_store.save_basic_node_data(&bnd).await?;
bnd
}
};
let address_book = Buffer::new(
BoxService::new(start_address_book(p2p_store, config).await?),
constants::ADDRESS_BOOK_BUFFER_SIZE,
);
let outbound_connector = {
use tower::timeout::TimeoutLayer;
let hs_timeout = TimeoutLayer::new(constants::HANDSHAKE_TIMEOUT);
let hs = peer::Handshaker::new(
basic_node_data,
config.network(),
address_book.clone(),
core_sync_svc,
inbound_service,
);
hs_timeout.layer(hs)
};
let (peerset_tx, peerset_rx) =
futures::channel::mpsc::channel::<DiscoveredPeer>(config.peerset_total_connection_limit());
let discovered_peers = peerset_rx
// Discover interprets an error as stream termination,
// so discard any errored connections...
.filter(|result| future::ready(result.is_ok()))
.map_ok(|(address, client)| Change::Insert(address, client.into()));
// Create an mpsc channel for peerset demand signaling,
// based on the maximum number of outbound peers.
let (mut demand_tx, demand_rx) =
futures::channel::mpsc::channel::<MorePeers>(config.peerset_total_connection_limit());
// Create a oneshot to send background task JoinHandles to the peer set
let (handle_tx, handle_rx) = tokio::sync::oneshot::channel();
let peer_set = PeerSet::new(&config, discovered_peers, demand_tx, handle_rx);
let peer_set = Buffer::new(BoxService::new(peer_set), constants::PEERSET_BUFFER_SIZE);
Ok(address_book)
}
*/

View file

@ -0,0 +1,9 @@
use monero_wire::NetworkAddress;
use tower::BoxError;
pub mod block_broadcaster;
pub mod block_download;
pub(crate) type DiscoveredPeer = Result<(NetworkAddress, crate::peer::Client), BoxError>;
pub use block_download::{BlockGetterRequest, BlockGetterResponse, BlockGetterService};

View file

@ -0,0 +1,116 @@
// TODO: Investigate tor/i2p block broadcasting; should we do it? randomise delay?
use std::future::Future;
use std::pin::Pin;
use std::task::{Context, Poll};
use futures::stream::FuturesOrdered;
use futures::{FutureExt, StreamExt};
use tokio::sync::Mutex;
use tower::discover::Discover;
use tower::BoxError;
use monero_wire::messages::common::{BlockCompleteEntry, TransactionBlobs};
use monero_wire::messages::{NewBlock, NewFluffyBlock};
use monero_wire::{NetworkAddress, PeerID};
use crate::peer::LoadTrackedClient;
use crate::peer_set::set::PeerSet;
pub enum BlockBroadCasterRequest {
/// A request to broadcast a block to all ready peers, Cuprate
/// only supports broadcasting by fluffy blocks.
BroadCastBlock { block: Vec<u8>, block_height: u64 },
}
pub enum BlockBroadCasterResponse {
BlockBroadCasted,
}
pub struct BlockBroadCaster<D>
where
D: Discover<Key = PeerID, Service = LoadTrackedClient> + Unpin,
D::Error: Into<BoxError>,
{
peer_set: std::sync::Arc<Mutex<PeerSet<D>>>,
/// The proportion of peers that need to be ready for `poll_ready` to return ready.
///
/// monerod will remove peers that do not broadcast every block to it, this is a problem
/// for us as we need peers to be ready for us to broadcast to them so we compromise and
/// only broadcast to ready peers and take the hit on the other peers.
wanted_ready_peers: f64,
}
impl<D> tower::Service<BlockBroadCasterRequest> for BlockBroadCaster<D>
where
D: Discover<Key = PeerID, Service = LoadTrackedClient> + Unpin + Send + 'static,
D::Error: Into<BoxError>,
{
type Response = BlockBroadCasterResponse;
type Error = BoxError;
type Future =
Pin<Box<dyn Future<Output = Result<Self::Response, Self::Error>> + Send + 'static>>;
fn poll_ready(&mut self, cx: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
let mutex = self.peer_set.clone();
let ret = match Box::pin(mutex.lock()).poll_unpin(cx) {
Poll::Pending => Poll::Pending,
Poll::Ready(mut peer_set) => {
peer_set.poll_all(cx)?;
peer_set.poll_ready(cx);
if self.wanted_ready_peers <= peer_set.proportion_ready() {
Poll::Ready(Ok(()))
} else {
Poll::Pending
}
}
};
ret
}
fn call(&mut self, req: BlockBroadCasterRequest) -> Self::Future {
match req {
BlockBroadCasterRequest::BroadCastBlock {
block,
block_height,
} => {
let empty_txs = TransactionBlobs::new_unpruned(vec![]);
let fluffy_complete_entry = BlockCompleteEntry {
block: block.clone(),
block_weight: 0,
txs: empty_txs,
pruned: false,
};
let new_fluffy_block = NewFluffyBlock {
b: fluffy_complete_entry,
current_blockchain_height: block_height + 1,
};
let mutex = self.peer_set.clone();
async move {
let mut peer_set = mutex.lock().await;
let all_ready_peers = peer_set.all_ready();
let mut fut = FuturesOrdered::new();
for (_, svc) in all_ready_peers {
if svc.supports_fluffy_blocks() {
fut.push_back(svc.call(new_fluffy_block.clone().into()));
} else {
tracing::error!(
"Peer which doesn't support fluffy blocks is in the PeerSet"
)
}
}
peer_set.push_all_unready();
while let Some(_) = fut.next().await {}
Ok(BlockBroadCasterResponse::BlockBroadCasted)
}
.boxed()
}
}
}
}

View file

@ -0,0 +1,173 @@
use futures::{FutureExt, Sink};
use std::future::Future;
use std::pin::Pin;
use std::sync::Arc;
use std::task::{Context, Poll};
use tokio::sync::Mutex;
use tower::discover::Discover;
use tower::BoxError;
use monero_wire::messages::GetObjectsRequest;
use monero_wire::messages::{GetObjectsResponse, MessageNotification};
use monero_wire::{Message, NetworkAddress, PeerID};
use crate::peer::LoadTrackedClient;
use crate::peer_set::set::PeerSet;
use crate::protocol::InternalMessageResponse;
pub enum BlockGetterRequest {
/// A request for blocks, used when syncing.
///
/// start_height is used to determine the peer for the next request,
/// you should use [`BlockGetterRequest::SetHeight`] before calling
/// this for the first time.
GetBlocks {
blocks: Vec<[u8; 32]>,
pruned: bool,
start_height: u64,
},
SetHeight(u64),
}
pub enum BlockGetterResponse {
Blocks(GetObjectsResponse),
HeightSet,
}
pub struct BlockGetterService<D>
where
D: Discover<Key = PeerID, Service = LoadTrackedClient> + Unpin,
D::Error: Into<BoxError>,
{
peer_set: Arc<Mutex<PeerSet<D>>>,
next_start_height: Option<u64>,
p2c_peer: Option<(D::Key, D::Service)>,
}
impl<D> tower::Service<BlockGetterRequest> for BlockGetterService<D>
where
D: Discover<Key = PeerID, Service = LoadTrackedClient> + Unpin + Send + 'static,
D::Error: Into<BoxError>,
{
type Response = BlockGetterResponse;
type Error = BoxError;
type Future =
Pin<Box<dyn Future<Output = Result<Self::Response, Self::Error>> + Send + 'static>>;
fn poll_ready(&mut self, cx: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
let span = tracing::trace_span!(parent: &tracing::span::Span::current(), "BlockGetter");
match self.next_start_height {
// If we don't know the next batch start height we must have not received
// any requests yet. The first request has to be [`SetHeight`] so thats
// what the next request will be.
None => {
tracing::trace!(parent: &span, "next height not known");
Poll::Ready(Ok(()))
}
Some(height) => {
tracing::trace!(parent: &span, next_height = height);
let mut peer_no_longer_ready = false;
if let Some((addr, svc)) = &mut self.p2c_peer {
tracing::trace!(parent: &span, preselected_peer = ?addr);
match svc.poll_ready(cx) {
Poll::Ready(Ok(())) => {
tracing::trace!(
parent: &span,
"Pre-selected peer still ready, keeping it selected"
);
return Poll::Ready(Ok(()));
}
Poll::Pending => {
tracing::trace!(
"preselected service is no longer ready, moving to unready list"
);
peer_no_longer_ready = true;
}
Poll::Ready(Err(e)) => {
tracing::trace!(parent: &span, %e, "preselected service failed, dropping it");
self.p2c_peer = None;
}
};
}
tracing::trace!(
parent: &span,
"preselected service was not ready, preselecting another ready service"
);
let mutex = self.peer_set.clone();
let ret = match Box::pin(mutex.lock()).poll_unpin(cx) {
Poll::Pending => Poll::Pending,
Poll::Ready(mut peer_set) => {
peer_set.poll_all(cx)?;
if peer_no_longer_ready {
let (key, svc) = std::mem::replace(&mut self.p2c_peer, None)
.expect("Peer must exist for it to not be ready");
peer_set.push_unready(key, svc);
}
let p2c_peer = match peer_set.preselect_p2c_peer_with_full_block(height) {
None => {
tracing::trace!(
parent: &span,
"no ready services, sending demand signal"
);
peer_set.demand_more_peers();
return Poll::Pending;
}
Some(peer) => {
tracing::trace!(parent: &span, preselected_peer = ?peer);
peer
}
};
self.p2c_peer = peer_set
.take_ready_service(&p2c_peer)
.and_then(|svc| Some((p2c_peer, svc)));
Poll::Ready(Ok(()))
}
};
ret
}
}
}
fn call(&mut self, req: BlockGetterRequest) -> Self::Future {
match req {
BlockGetterRequest::SetHeight(height) => {
self.next_start_height = Some(height);
async { Ok(BlockGetterResponse::HeightSet) }.boxed()
}
BlockGetterRequest::GetBlocks {
blocks,
pruned,
start_height,
} => {
self.next_start_height = Some(start_height + blocks.len() as u64);
let obj_req = GetObjectsRequest { blocks, pruned };
let peer_set = self.peer_set.clone();
let (addr, mut svc) = std::mem::replace(&mut self.p2c_peer, None).expect(
"A peer is always selected in poll_ready and poll_ready must be called first",
);
async move {
let fut = svc.call(obj_req.into());
let mut set = peer_set.lock().await;
set.push_unready(addr, svc);
fut.await.map(|res| {
let InternalMessageResponse::GetObjectsResponse(res) = res else {
unreachable!("Peer connection must return correct response")
};
BlockGetterResponse::Blocks(res)
})
}
.boxed()
}
}
}
}

516
p2p/src/peer_set/set.rs Normal file
View file

@ -0,0 +1,516 @@
use std::future::Future;
use std::ops::Div;
use std::{
collections::{HashMap, HashSet},
convert,
marker::PhantomData,
pin::Pin,
task::{Context, Poll},
};
use futures::future::BoxFuture;
use futures::TryFutureExt;
use futures::{
channel::{mpsc, oneshot},
stream::FuturesUnordered,
Stream,
};
use futures::{FutureExt, SinkExt};
use tokio::{sync::oneshot::error::TryRecvError, task::JoinHandle};
use tower::{
discover::{Change, Discover},
load::Load,
BoxError, Service,
};
use monero_wire::{NetworkAddress, PeerID};
use super::{unready_service::UnreadyError, UnreadyService};
use crate::{
peer::LoadTrackedClient,
protocol::{InternalMessageRequest, InternalMessageResponse},
Config,
};
/// A signal sent by the [`PeerSet`] when it has no ready peers, and gets a request from Zebra.
///
/// In response to this signal, the crawler tries to open more peer connections.
#[derive(Copy, Clone, Debug, PartialEq, Eq, Hash)]
pub struct MorePeers;
/// A signal sent by the [`PeerSet`] to cancel a [`Client`][1]'s current request
/// or response.
///
/// When it receives this signal, the [`Client`][1] stops processing and exits.
///
/// [1]: crate::peer::Client
#[derive(Copy, Clone, Debug, PartialEq, Eq, Hash)]
pub struct CancelClientWork;
/// A [`tower::Service`] that abstractly represents "the rest of the network".
///
/// # Security
///
/// The `Discover::Key` must be the transient remote address of each peer. This
/// address may only be valid for the duration of a single connection. (For
/// example, inbound connections have an ephemeral remote port, and proxy
/// connections have an ephemeral local or proxy port.)
///
/// Otherwise, malicious peers could interfere with other peers' `PeerSet` state.
pub struct PeerSet<D>
where
D: Discover<Key = PeerID, Service = LoadTrackedClient> + Unpin,
D::Error: Into<BoxError>,
{
/// Peer Tracking: New Peers
///
/// Provides new and deleted peer [`Change`]s to the peer set,
/// via the [`Discover`] trait implementation.
discover: D,
/// A channel that asks the peer crawler task to connect to more peers.
demand_signal: mpsc::Sender<MorePeers>,
/// Peer Tracking: Ready Peers
///
/// Connected peers that are ready to receive requests from Zebra,
/// or send requests to Zebra.
ready_services: HashMap<D::Key, D::Service>,
/// Peer Tracking: Busy Peers
///
/// Connected peers that are handling a Zebra request,
/// or Zebra is handling one of their requests.
unready_services: FuturesUnordered<UnreadyService<D::Key, D::Service, InternalMessageRequest>>,
/// Channels used to cancel the request that an unready service is doing.
cancel_handles: HashMap<D::Key, oneshot::Sender<CancelClientWork>>,
/// The configured limit for inbound and outbound connections.
///
/// The peer set panics if this size is exceeded.
/// If that happens, our connection limit code has a bug.
peerset_total_connection_limit: usize,
// Background Tasks
//
/// Channel for passing ownership of tokio JoinHandles from PeerSet's background tasks
///
/// The join handles passed into the PeerSet are used populate the `guards` member
handle_rx: tokio::sync::oneshot::Receiver<Vec<JoinHandle<Result<(), BoxError>>>>,
/// Unordered set of handles to background tasks associated with the `PeerSet`
///
/// These guards are checked for errors as part of `poll_ready` which lets
/// the `PeerSet` propagate errors from background tasks back to the user
guards: FuturesUnordered<JoinHandle<Result<(), BoxError>>>,
}
impl<D> PeerSet<D>
where
D: Discover<Key = PeerID, Service = LoadTrackedClient> + Unpin,
D::Error: Into<BoxError>,
{
/// Construct a peerset which uses `discover` to manage peer connections.
///
/// Arguments:
/// - `config`: configures the peer set connection limit;
/// - `discover`: handles peer connects and disconnects;
/// - `demand_signal`: requests more peers when all peers are busy (unready);
/// - `handle_rx`: receives background task handles,
/// monitors them to make sure they're still running,
/// and shuts down all the tasks as soon as one task exits;
pub fn new(
config: &Config,
discover: D,
demand_signal: mpsc::Sender<MorePeers>,
handle_rx: tokio::sync::oneshot::Receiver<Vec<JoinHandle<Result<(), BoxError>>>>,
) -> Self {
Self {
// New peers
discover,
demand_signal,
// Ready peers
ready_services: HashMap::new(),
// Busy peers
unready_services: FuturesUnordered::new(),
cancel_handles: HashMap::new(),
peerset_total_connection_limit: config.peerset_total_connection_limit(),
// Background tasks
handle_rx,
guards: futures::stream::FuturesUnordered::new(),
}
}
/// Receive background tasks, if they've been sent on the channel,
/// but not consumed yet.
///
/// Returns a result representing the current task state,
/// or `None` if the background tasks should be polled to check their state.
fn receive_tasks_if_needed(&mut self) -> Option<Result<(), BoxError>> {
if self.guards.is_empty() {
match self.handle_rx.try_recv() {
// The tasks haven't been sent yet.
Err(TryRecvError::Empty) => Some(Ok(())),
// The tasks have been sent, but not consumed.
Ok(handles) => {
// Currently, the peer set treats an empty background task set as an error.
//
// TODO: refactor `handle_rx` and `guards` into an enum
// for the background task state: Waiting/Running/Shutdown.
assert!(
!handles.is_empty(),
"the peer set requires at least one background task"
);
self.guards.extend(handles);
None
}
// The tasks have been sent and consumed, but then they exited.
//
// Correctness: the peer set must receive at least one task.
//
// TODO: refactor `handle_rx` and `guards` into an enum
// for the background task state: Waiting/Running/Shutdown.
Err(TryRecvError::Closed) => {
Some(Err("all peer set background tasks have exited".into()))
}
}
} else {
None
}
}
/// Check background task handles to make sure they're still running.
///
/// If any background task exits, shuts down all other background tasks,
/// and returns an error.
fn poll_background_errors(&mut self, cx: &mut Context) -> Result<(), BoxError> {
if let Some(result) = self.receive_tasks_if_needed() {
return result;
}
match Pin::new(&mut self.guards).poll_next(cx) {
// All background tasks are still running.
Poll::Pending => Ok(()),
Poll::Ready(Some(res)) => {
tracing::info!(
background_tasks = %self.guards.len(),
"a peer set background task exited, shutting down other peer set tasks"
);
self.shut_down_tasks_and_channels();
// Flatten the join result and inner result,
// then turn Ok() task exits into errors.
res.map_err(Into::into)
// TODO: replace with Result::flatten when it stabilises (#70142)
.and_then(convert::identity)
.and(Err("a peer set background task exited".into()))
}
Poll::Ready(None) => {
self.shut_down_tasks_and_channels();
Err("all peer set background tasks have exited".into())
}
}
}
/// Shut down:
/// - services by dropping the service lists
/// - background tasks via their join handles or cancel handles
/// - channels by closing the channel
fn shut_down_tasks_and_channels(&mut self) {
// Drop services and cancel their background tasks.
self.ready_services = HashMap::new();
for (_peer_key, handle) in self.cancel_handles.drain() {
let _ = handle.send(CancelClientWork);
}
self.unready_services = FuturesUnordered::new();
// Close the MorePeers channel for all senders,
// so we don't add more peers to a shut down peer set.
self.demand_signal.close_channel();
// Shut down background tasks.
self.handle_rx.close();
self.receive_tasks_if_needed();
for guard in self.guards.iter() {
guard.abort();
}
// TODO: implement graceful shutdown for InventoryRegistry (#1678)
}
/// Check busy peer services for request completion or errors.
///
/// Move newly ready services to the ready list if they are for peers with supported protocol
/// versions, otherwise they are dropped. Also drop failed services.
fn poll_unready(&mut self, cx: &mut Context<'_>) {
loop {
match Pin::new(&mut self.unready_services).poll_next(cx) {
// No unready service changes, or empty unready services
Poll::Pending | Poll::Ready(None) => return,
// Unready -> Ready
Poll::Ready(Some(Ok((key, svc)))) => {
tracing::trace!(?key, "service became ready");
let cancel = self.cancel_handles.remove(&key);
assert!(cancel.is_some(), "missing cancel handle");
}
// Unready -> Canceled
Poll::Ready(Some(Err((key, UnreadyError::Canceled)))) => {
// A service be canceled because we've connected to the same service twice.
// In that case, there is a cancel handle for the peer address,
// but it belongs to the service for the newer connection.
tracing::trace!(
?key,
duplicate_connection = self.cancel_handles.contains_key(&key),
"service was canceled, dropping service"
);
}
Poll::Ready(Some(Err((key, UnreadyError::CancelHandleDropped(_))))) => {
// Similarly, services with dropped cancel handes can have duplicates.
tracing::trace!(
?key,
duplicate_connection = self.cancel_handles.contains_key(&key),
"cancel handle was dropped, dropping service"
);
}
// Unready -> Errored
Poll::Ready(Some(Err((key, UnreadyError::Inner(error))))) => {
tracing::debug!(%error, "service failed while unready, dropping service");
let cancel = self.cancel_handles.remove(&key);
assert!(cancel.is_some(), "missing cancel handle");
}
}
}
}
/// Checks for newly inserted or removed services.
///
/// Puts inserted services in the unready list.
/// Drops removed services, after cancelling any pending requests.
fn poll_discover(&mut self, cx: &mut Context<'_>) -> Poll<Result<(), BoxError>> {
use futures::ready;
loop {
match ready!(Pin::new(&mut self.discover).poll_discover(cx))
.ok_or("discovery stream closed")?
.map_err(Into::into)?
{
Change::Remove(key) => {
tracing::trace!(?key, "got Change::Remove from Discover");
self.remove(&key);
}
Change::Insert(key, svc) => {
// We add peers as unready, so that we:
// - always do the same checks on every ready peer, and
// - check for any errors that happened right after the handshake
tracing::trace!(?key, "got Change::Insert from Discover");
self.remove(&key);
self.push_unready(key, svc);
}
}
}
}
/// Calls the poll functions used at the start of all `poll_ready`s
pub fn poll_all(&mut self, cx: &mut Context<'_>) -> Result<(), BoxError> {
self.poll_background_errors(cx)?;
// Update peer statuses
let _ = self.poll_discover(cx)?;
self.poll_unready(cx);
Ok(())
}
pub fn poll_ready(&mut self, cx: &mut Context<'_>) {
let mut ready_services = HashMap::with_capacity(self.ready_services.len());
let mut pending_services = vec![];
for (key, mut svc) in self.ready_services.drain() {
match svc.poll_ready(cx) {
Poll::Pending => {
pending_services.push((key, svc));
}
Poll::Ready(Ok(())) => {
ready_services.insert(key, svc);
}
Poll::Ready(Err(e)) => {
tracing::trace!("Peer poll_ready returned error: {}", e);
// peer svc will get dropped at the start of next loop
}
}
}
for (key, svc) in pending_services {
self.push_unready(key, svc);
}
self.ready_services = ready_services;
}
pub fn proportion_ready(&self) -> f64 {
let total_services = self.ready_services.len() + self.unready_services.len();
if total_services == 0 {
return 1.0;
}
self.ready_services.len() as f64 / total_services as f64
}
/// Takes a ready service by key.
pub fn take_ready_service(&mut self, key: &D::Key) -> Option<D::Service> {
if let Some(svc) = self.ready_services.remove(key) {
assert!(
!self.cancel_handles.contains_key(key),
"cancel handles are only used for unready service work"
);
Some(svc)
} else {
None
}
}
/// Remove the service corresponding to `key` from the peer set.
///
/// Drops the service, cancelling any pending request or response to that peer.
/// If the peer does not exist, does nothing.
fn remove(&mut self, key: &D::Key) {
if let Some(ready_service) = self.take_ready_service(key) {
// A ready service has no work to cancel, so just drop it.
std::mem::drop(ready_service);
} else if let Some(handle) = self.cancel_handles.remove(key) {
// Cancel the work, implicitly dropping the cancel handle.
// The service future returns a `Canceled` error,
// making `poll_unready` drop the service.
let _ = handle.send(CancelClientWork);
}
}
/// Adds a busy service to the unready list if it's for a peer with a supported version,
/// and adds a cancel handle for the service's current request.
///
/// If the service is for a connection to an outdated peer, the request is cancelled and the
/// service is dropped.
pub fn push_unready(&mut self, key: D::Key, svc: D::Service) {
let (tx, rx) = oneshot::channel();
self.unready_services.push(UnreadyService {
key: Some(key),
service: Some(svc),
cancel: rx,
_req: PhantomData,
});
self.cancel_handles.insert(key, tx);
}
pub fn preselect_p2c_peer_with_full_block(&self, block_height: u64) -> Option<D::Key> {
self.select_p2c_peer_from_list(
&self
.ready_services
.iter()
.filter_map(|(key, serv)| {
if serv.has_full_block(block_height) {
Some(key)
} else {
None
}
})
.collect(),
)
}
/// Performs P2C on `self.ready_services` to randomly select a less-loaded ready service.
pub fn preselect_p2c_peer(&self) -> Option<D::Key> {
self.select_p2c_peer_from_list(&self.ready_services.keys().collect())
}
/// Accesses a ready endpoint by `key` and returns its current load.
///
/// Returns `None` if the service is not in the ready service list.
fn query_load(&self, key: &D::Key) -> Option<<D::Service as Load>::Metric> {
let svc = self.ready_services.get(key);
svc.map(|svc| svc.load())
}
// Performs P2C on `ready_service_list` to randomly select a less-loaded ready service.
#[allow(clippy::unwrap_in_result)]
pub fn select_p2c_peer_from_list(
&self,
ready_service_list: &HashSet<&D::Key>,
) -> Option<D::Key> {
match ready_service_list.len() {
0 => None,
1 => Some(
**ready_service_list
.iter()
.next()
.expect("just checked there is one service"),
),
len => {
// If there are only 2 peers, randomise their order.
// Otherwise, choose 2 random peers in a random order.
let (a, b) = {
let idxs = rand::seq::index::sample(&mut rand::thread_rng(), len, 2);
let a = idxs.index(0);
let b = idxs.index(1);
let a = **ready_service_list
.iter()
.nth(a)
.expect("sample returns valid indexes");
let b = **ready_service_list
.iter()
.nth(b)
.expect("sample returns valid indexes");
(a, b)
};
let a_load = self.query_load(&a).expect("supplied services are ready");
let b_load = self.query_load(&b).expect("supplied services are ready");
let selected = if a_load <= b_load { a } else { b };
tracing::trace!(
a.key = ?a,
a.load = ?a_load,
b.key = ?b,
b.load = ?b_load,
selected = ?selected,
?len,
"selected service by p2c"
);
Some(selected)
}
}
}
pub fn all_ready(&mut self) -> &mut HashMap<PeerID, LoadTrackedClient> {
&mut self.ready_services
}
pub fn push_all_unready(&mut self) {
let all_ready: Vec<(_, _)> = self.ready_services.drain().collect();
for (key, svc) in all_ready {
self.push_unready(key, svc)
}
}
pub fn demand_more_peers(&mut self) {
let _ = self.demand_signal.try_send(MorePeers);
}
}

View file

@ -0,0 +1,92 @@
/// Services that are busy or newly created.
///
/// Adapted from zebra who themselves adapted this from tower-balance.
use std::{
future::Future,
marker::PhantomData,
pin::Pin,
task::{Context, Poll},
};
use futures::{channel::oneshot, ready};
use pin_project::pin_project;
use tower::Service;
use crate::peer_set::set::CancelClientWork;
/// A Future that becomes satisfied when an `S`-typed service is ready.
///
/// May fail due to cancellation, i.e. if the service is removed from discovery.
#[pin_project]
#[derive(Debug)]
pub(super) struct UnreadyService<K, S, Req> {
/// The key used to lookup `service`.
pub(super) key: Option<K>,
/// A oneshot used to cancel the request the `service` is currently working on, if any.
#[pin]
pub(super) cancel: oneshot::Receiver<CancelClientWork>,
/// The `service` that is busy (or newly created).
pub(super) service: Option<S>,
/// Dropping `service` might drop a request.
/// This [`PhantomData`] tells the Rust compiler to do a drop check for `Req`.
pub(super) _req: PhantomData<Req>,
}
#[derive(Debug, Eq, PartialEq)]
pub(super) enum UnreadyError<E> {
Inner(E),
Canceled,
CancelHandleDropped(oneshot::Canceled),
}
impl<K, S: Service<Req>, Req> Future for UnreadyService<K, S, Req> {
type Output = Result<(K, S), (K, UnreadyError<S::Error>)>;
fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
let this = self.project();
if let Poll::Ready(oneshot_result) = this.cancel.poll(cx) {
let key = this.key.take().expect("polled after ready");
// # Correctness
//
// Return an error if the service is explicitly canceled,
// or its cancel handle is dropped, implicitly cancelling it.
match oneshot_result {
Ok(CancelClientWork) => return Poll::Ready(Err((key, UnreadyError::Canceled))),
Err(canceled_error) => {
return Poll::Ready(Err((
key,
UnreadyError::CancelHandleDropped(canceled_error),
)))
}
}
}
// # Correctness
//
// The current task must be scheduled for wakeup every time we return
// `Poll::Pending`.
//
//`ready!` returns `Poll::Pending` when the service is unready, and
// the inner `poll_ready` schedules this task for wakeup.
//
// `cancel.poll` also schedules this task for wakeup if it is canceled.
let res = ready!(this
.service
.as_mut()
.expect("polled after ready")
.poll_ready(cx));
let key = this.key.take().expect("polled after ready");
let svc = this.service.take().expect("polled after ready");
match res {
Ok(()) => Poll::Ready(Ok((key, svc))),
Err(e) => Poll::Ready(Err((key, UnreadyError::Inner(e)))),
}
}
}

29
p2p/src/protocol.rs Normal file
View file

@ -0,0 +1,29 @@
pub mod internal_network;
pub use internal_network::{InternalMessageRequest, InternalMessageResponse};
use monero_wire::messages::CoreSyncData;
/// A request to a [`tower::Service`] that handles sync states.
pub enum CoreSyncDataRequest {
/// Get our [`CoreSyncData`].
GetOurs,
/// Handle an incoming [`CoreSyncData`].
NewIncoming(CoreSyncData),
}
/// A response from a [`tower::Service`] that handles sync states.
pub enum CoreSyncDataResponse {
/// Our [`CoreSyncData`]
Ours(CoreSyncData),
/// The incoming [`CoreSyncData`] is ok.
Ok,
}
/// The direction of a connection.
pub enum Direction {
/// An inbound connection.
Inbound,
/// An outbound connection.
Outbound,
}