//! Abstractions that represent "the rest of the network". //! //! # Implementation //! //! The [`PeerSet`] implementation is adapted from the one in the [Tower Balance][tower-balance] crate. //! As described in that crate's documentation, it: //! //! > Distributes requests across inner services using the [Power of Two Choices][p2c]. //! > //! > As described in the [Finagle Guide][finagle]: //! > //! > > The algorithm randomly picks two services from the set of ready endpoints and //! > > selects the least loaded of the two. By repeatedly using this strategy, we can //! > > expect a manageable upper bound on the maximum load of any server. //! > > //! > > The maximum load variance between any two servers is bound by `ln(ln(n))` where //! > > `n` is the number of servers in the cluster. //! //! The Power of Two Choices should work well for many network requests, but not all of them. //! Some requests should only be made to a subset of connected peers. //! For example, a request for a particular inventory item //! should be made to a peer that has recently advertised that inventory hash. //! Other requests require broadcasts, such as transaction diffusion. //! //! Implementing this specialized routing logic inside the `PeerSet` -- so that //! it continues to abstract away "the rest of the network" into one endpoint -- //! is not a problem, as the `PeerSet` can simply maintain more information on //! its peers and route requests appropriately. However, there is a problem with //! maintaining accurate backpressure information, because the `Service` trait //! requires that service readiness is independent of the data in the request. //! //! For this reason, in the future, this code will probably be refactored to //! address this backpressure mismatch. One possibility is to refactor the code //! so that one entity holds and maintains the peer set and metadata on the //! peers, and each "backpressure category" of request is assigned to different //! `Service` impls with specialized `poll_ready()` implementations. Another //! less-elegant solution (which might be useful as an intermediate step for the //! inventory case) is to provide a way to borrow a particular backing service, //! say by address. //! //! [finagle]: https://twitter.github.io/finagle/guide/Clients.html#power-of-two-choices-p2c-least-loaded //! [p2c]: http://www.eecs.harvard.edu/~michaelm/postscripts/handbook2001.pdf //! [tower-balance]: https://crates.io/crates/tower-balance use std::{ collections::{HashMap, HashSet}, convert, fmt::Debug, future::Future, marker::PhantomData, net::SocketAddr, pin::Pin, sync::Arc, task::{Context, Poll}, time::Instant, }; use chrono::Utc; use futures::{ channel::{mpsc, oneshot}, future::{FutureExt, TryFutureExt}, prelude::*, stream::FuturesUnordered, }; use tokio::{ sync::{broadcast, oneshot::error::TryRecvError}, task::JoinHandle, }; use tower::{ discover::{Change, Discover}, load::Load, Service, }; use crate::{ peer_set::{ unready_service::{Error as UnreadyError, UnreadyService}, InventoryRegistry, }, protocol::{ external::InventoryHash, internal::{Request, Response}, }, AddressBook, BoxError, 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`]'s current request or response. /// /// When it receives this signal, the [`Client`] stops processing and exits. #[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 where D: Discover + Unpin, D::Service: Service + Load, D::Error: Into, >::Error: Into + 'static, >::Future: Send + 'static, ::Metric: Debug, { /// Provides new and deleted peer [`Change`]s to the peer set, /// via the [`Discover`] trait implementation. discover: D, /// Connected peers that are ready to receive requests from Zebra, /// or send requests to Zebra. ready_services: HashMap, /// A preselected ready service. /// /// # Correctness /// /// If this is `Some(addr)`, `addr` must be a key for a peer in `ready_services`. /// If that peer is removed from `ready_services`, we must set the preselected peer to `None`. /// /// This is handled by [`PeerSet::take_ready_service`]. preselected_p2c_peer: Option, /// Stores gossiped inventory hashes from connected peers. /// /// Used to route inventory requests to peers that are likely to have it. inventory_registry: InventoryRegistry, /// Connected peers that are handling a Zebra request, /// or Zebra is handling one of their requests. unready_services: FuturesUnordered>, /// Channels used to cancel the request that an unready service is doing. cancel_handles: HashMap>, /// A channel that asks the peer crawler task to connect to more peers. demand_signal: mpsc::Sender, /// 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>>>, /// 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: futures::stream::FuturesUnordered>>, /// A shared list of peer addresses. /// /// Used for logging diagnostics. address_book: Arc>, /// The last time we logged a message about the peer set size last_peer_log: Option, /// 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, } impl Drop for PeerSet where D: Discover + Unpin, D::Service: Service + Load, D::Error: Into, >::Error: Into + 'static, >::Future: Send + 'static, ::Metric: Debug, { fn drop(&mut self) { self.shut_down_tasks_and_channels() } } impl PeerSet where D: Discover + Unpin, D::Service: Service + Load, D::Error: Into, >::Error: Into + 'static, >::Future: Send + 'static, ::Metric: Debug, { /// 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; /// - `inv_stream`: receives inventory advertisements for peers, /// allowing the peer set to direct inventory requests; /// - `address_book`: when peer set is busy, it logs address book diagnostics. pub fn new( config: &Config, discover: D, demand_signal: mpsc::Sender, handle_rx: tokio::sync::oneshot::Receiver>>>, inv_stream: broadcast::Receiver<(InventoryHash, SocketAddr)>, address_book: Arc>, ) -> Self { Self { // Ready peers discover, ready_services: HashMap::new(), preselected_p2c_peer: None, inventory_registry: InventoryRegistry::new(inv_stream), // Unready peers unready_services: FuturesUnordered::new(), cancel_handles: HashMap::new(), demand_signal, // Background tasks handle_rx, guards: futures::stream::FuturesUnordered::new(), // Metrics last_peer_log: None, address_book, peerset_total_connection_limit: config.peerset_total_connection_limit(), } } /// 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)) => { 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()) } } } /// 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> { 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 } } /// 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.preselected_p2c_peer = None; 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, and 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)))) => { trace!(?key, "service became ready"); let cancel = self.cancel_handles.remove(&key); assert!(cancel.is_some(), "missing cancel handle"); self.ready_services.insert(key, svc); } // 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. trace!( ?key, duplicate_connection = self.cancel_handles.contains_key(&key), "service was canceled, dropping service" ); } // Unready -> Errored Poll::Ready(Some(Err((key, UnreadyError::Inner(e))))) => { let error = e.into(); 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> { 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) => { trace!(?key, "got Change::Remove from Discover"); self.remove(&key); } Change::Insert(key, svc) => { trace!(?key, "got Change::Insert from Discover"); self.remove(&key); self.push_unready(key, svc); } } } } /// Takes a ready service by key, invalidating `preselected_p2c_peer` if needed. fn take_ready_service(&mut self, key: &D::Key) -> Option { if let Some(svc) = self.ready_services.remove(key) { if Some(*key) == self.preselected_p2c_peer { self.preselected_p2c_peer = None; } 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, /// and adds a cancel handle for the service's current request. fn push_unready(&mut self, key: D::Key, svc: D::Service) { let (tx, rx) = oneshot::channel(); self.cancel_handles.insert(key, tx); self.unready_services.push(UnreadyService { key: Some(key), service: Some(svc), cancel: rx, _req: PhantomData, }); } /// Performs P2C on `self.ready_services` to randomly select a less-loaded ready service. fn preselect_p2c_peer(&self) -> Option { self.select_p2c_peer_from_list(&self.ready_services.keys().copied().collect()) } /// Performs P2C on `ready_service_list` to randomly select a less-loaded ready service. fn select_p2c_peer_from_list(&self, ready_service_list: &HashSet) -> Option { 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 }; trace!( a.key = ?a, a.load = ?a_load, b.key = ?b, b.load = ?b_load, selected = ?selected, ?len, "selected service by p2c" ); Some(selected) } } } /// Randomly chooses `max_peers` ready services, ignoring service load. /// /// The chosen peers are unique, but their order is not fully random. fn select_random_ready_peers(&self, max_peers: usize) -> Vec { use rand::seq::IteratorRandom; self.ready_services .keys() .copied() .choose_multiple(&mut rand::thread_rng(), max_peers) } /// 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<::Metric> { let svc = self.ready_services.get(key); svc.map(|svc| svc.load()) } /// Routes a request using P2C load-balancing. fn route_p2c(&mut self, req: Request) -> >::Future { let preselected_key = self .preselected_p2c_peer .expect("ready peer service must have a preselected peer"); tracing::trace!(?preselected_key, "routing based on p2c"); let mut svc = self .take_ready_service(&preselected_key) .expect("ready peer set must have preselected a ready peer"); let fut = svc.call(req); self.push_unready(preselected_key, svc); fut.map_err(Into::into).boxed() } /// Tries to route a request to a peer that advertised that inventory, /// falling back to P2C if there is no ready peer. fn route_inv( &mut self, req: Request, hash: InventoryHash, ) -> >::Future { let inventory_peer_list = self .inventory_registry .peers(&hash) .filter(|&key| self.ready_services.contains_key(key)) .copied() .collect(); // # Security // // Choose a random, less-loaded peer with the inventory. // // If we chose the first peer in HashMap order, // peers would be able to influence our choice by switching addresses. // But we need the choice to be random, // so that a peer can't provide all our inventory responses. let peer = self.select_p2c_peer_from_list(&inventory_peer_list); match peer.and_then(|key| self.take_ready_service(&key)) { Some(mut svc) => { let peer = peer.expect("just checked peer is Some"); tracing::trace!(?hash, ?peer, "routing based on inventory"); let fut = svc.call(req); self.push_unready(peer, svc); fut.map_err(Into::into).boxed() } None => { tracing::trace!(?hash, "no ready peer for inventory, falling back to p2c"); self.route_p2c(req) } } } /// Routes the same request to up to `max_peers` ready peers, ignoring return values. /// /// `max_peers` must be at least one, and at most the number of ready peers. fn route_multiple( &mut self, req: Request, max_peers: usize, ) -> >::Future { assert!( max_peers > 0, "requests must be routed to at least one peer" ); assert!( max_peers <= self.ready_services.len(), "requests can only be routed to ready peers" ); // # Security // // We choose peers randomly, ignoring load. // This avoids favouring malicious peers, because peers can influence their own load. // // The order of peers isn't completely random, // but peer request order is not security-sensitive. let futs = FuturesUnordered::new(); for key in self.select_random_ready_peers(max_peers) { let mut svc = self .take_ready_service(&key) .expect("selected peers are ready"); futs.push(svc.call(req.clone()).map_err(|_| ())); self.push_unready(key, svc); } async move { let results = futs.collect::>>().await; tracing::debug!( ok.len = results.iter().filter(|r| r.is_ok()).count(), err.len = results.iter().filter(|r| r.is_err()).count(), "sent peer request to multiple peers" ); Ok(Response::Nil) } .boxed() } /// Broadcasts the same request to lots of ready peers, ignoring return values. fn route_broadcast(&mut self, req: Request) -> >::Future { // Round up, so that if we have one ready peer, it gets the request let half_ready_peers = (self.ready_services.len() + 1) / 2; // Broadcasts ignore the response self.route_multiple(req, half_ready_peers) } /// Logs the peer set size. fn log_peer_set_size(&mut self) { let ready_services_len = self.ready_services.len(); let unready_services_len = self.unready_services.len(); trace!(ready_peers = ?ready_services_len, unready_peers = ?unready_services_len); if ready_services_len > 0 { return; } // These logs are designed to be human-readable in a terminal, at the // default Zebra log level. If you need to know the peer set size for // every request, use the trace-level logs, or the metrics exporter. if let Some(last_peer_log) = self.last_peer_log { // Avoid duplicate peer set logs if Instant::now().duration_since(last_peer_log).as_secs() < 60 { return; } } else { // Suppress initial logs until the peer set has started up. // There can be multiple initial requests before the first peer is // ready. self.last_peer_log = Some(Instant::now()); return; } self.last_peer_log = Some(Instant::now()); // # Correctness // // Only log address metrics in exceptional circumstances, to avoid lock contention. // // Get the current time after acquiring the address book lock. // // TODO: replace with a watch channel that is updated in `AddressBook::update_metrics()`, // or turn the address book into a service (#1976) let address_metrics = self .address_book .lock() .unwrap() .address_metrics(Utc::now()); if unready_services_len == 0 { warn!( ?address_metrics, "network request with no peer connections. Hint: check your network connection" ); } else { info!(?address_metrics, "network request with no ready peers: finding more peers, waiting for {} peers to answer requests", unready_services_len); } } /// Updates the peer set metrics. /// /// # Panics /// /// If the peer set size exceeds the connection limit. fn update_metrics(&self) { let num_ready = self.ready_services.len(); let num_unready = self.unready_services.len(); let num_peers = num_ready + num_unready; metrics::gauge!("pool.num_ready", num_ready as f64); metrics::gauge!("pool.num_unready", num_unready as f64); metrics::gauge!("zcash.net.peers", num_peers as f64); // Security: make sure we haven't exceeded the connection limit if num_peers > self.peerset_total_connection_limit { // Correctness: Get the current time after acquiring the address book lock. let address_metrics = self .address_book .lock() .unwrap() .address_metrics(Utc::now()); panic!( "unexpectedly exceeded configured peer set connection limit: \n\ peers: {:?}, ready: {:?}, unready: {:?}, \n\ address_metrics: {:?}", num_peers, num_ready, num_unready, address_metrics, ); } } } impl Service for PeerSet where D: Discover + Unpin, D::Service: Service + Load, D::Error: Into, >::Error: Into + 'static, >::Future: Send + 'static, ::Metric: Debug, { type Response = Response; type Error = BoxError; type Future = Pin> + Send + 'static>>; fn poll_ready(&mut self, cx: &mut Context<'_>) -> Poll> { self.poll_background_errors(cx)?; // Update peer statuses let _ = self.poll_discover(cx)?; self.inventory_registry.poll_inventory(cx)?; self.poll_unready(cx); self.log_peer_set_size(); self.update_metrics(); loop { // Re-check that the pre-selected service is ready, in case // something has happened since (e.g., it failed, peer closed // connection, ...) if let Some(key) = self.preselected_p2c_peer { trace!(preselected_key = ?key); let mut service = self .take_ready_service(&key) .expect("preselected peer must be in the ready list"); match service.poll_ready(cx) { Poll::Ready(Ok(())) => { trace!("preselected service is still ready, keeping it selected"); self.preselected_p2c_peer = Some(key); self.ready_services.insert(key, service); return Poll::Ready(Ok(())); } Poll::Pending => { trace!("preselected service is no longer ready, moving to unready list"); self.push_unready(key, service); } Poll::Ready(Err(e)) => { let error = e.into(); trace!(%error, "preselected service failed, dropping it"); std::mem::drop(service); } } } trace!("preselected service was not ready, preselecting another ready service"); self.preselected_p2c_peer = self.preselect_p2c_peer(); self.update_metrics(); if self.preselected_p2c_peer.is_none() { // CORRECTNESS // // If the channel is full, drop the demand signal rather than waiting. // If we waited here, the crawler could deadlock sending a request to // fetch more peers, because it also empties the channel. trace!("no ready services, sending demand signal"); let _ = self.demand_signal.try_send(MorePeers); // CORRECTNESS // // The current task must be scheduled for wakeup every time we // return `Poll::Pending`. // // As long as there are unready or new peers, this task will run, // because: // - `poll_discover` schedules this task for wakeup when new // peers arrive. // - if there are unready peers, `poll_unready` schedules this // task for wakeup when peer services become ready. // - if the preselected peer is not ready, `service.poll_ready` // schedules this task for wakeup when that service becomes // ready. // // To avoid peers blocking on a full background error channel: // - if no background tasks have exited since the last poll, // `poll_background_errors` schedules this task for wakeup when // the next task exits. return Poll::Pending; } } } fn call(&mut self, req: Request) -> Self::Future { let fut = match req { // Only do inventory-aware routing on individual items. Request::BlocksByHash(ref hashes) if hashes.len() == 1 => { let hash = InventoryHash::from(*hashes.iter().next().unwrap()); self.route_inv(req, hash) } Request::TransactionsById(ref hashes) if hashes.len() == 1 => { let hash = InventoryHash::from(*hashes.iter().next().unwrap()); self.route_inv(req, hash) } // Broadcast advertisements to lots of peers Request::AdvertiseTransactionIds(_) => self.route_broadcast(req), Request::AdvertiseBlock(_) => self.route_broadcast(req), // Choose a random less-loaded peer for all other requests _ => self.route_p2c(req), }; self.update_metrics(); fut } }