zebra/zebra-network/src/peer_set/candidate_set.rs

354 lines
16 KiB
Rust
Raw Normal View History

use std::{cmp::min, mem, sync::Arc, time::Duration};
2019-10-22 12:48:50 -07:00
use futures::stream::{FuturesUnordered, StreamExt};
Fix a deadlock between the crawler and dialer, and other hangs (#1950) * Stop ignoring inbound message errors and handshake timeouts To avoid hangs, Zebra needs to maintain the following invariants in the handshake and heartbeat code: - each handshake should run in a separate spawned task (not yet implemented) - every message, error, timeout, and shutdown must update the peer address state - every await that depends on the network must have a timeout Once the Connection is created, it should handle timeouts. But we need to handle timeouts during handshake setup. * Avoid hangs by adding a timeout to the candidate set update Also increase the fanout from 1 to 2, to increase address diversity. But only return permanent errors from `CandidateSet::update`, because the crawler task exits if `update` returns an error. Also log Peers response errors in the CandidateSet. * Use the select macro in the crawler to reduce hangs The `select` function is biased towards its first argument, risking starvation. As a side-benefit, this change also makes the code a lot easier to read and maintain. * Split CrawlerAction::Demand into separate actions This refactor makes the code a bit easier to read, at the cost of sometimes blocking the crawler on `candidates.next()`. That's ok, because `next` only has a short (< 100 ms) delay. And we're just about to spawn a separate task for each handshake. * Spawn a separate task for each handshake This change avoids deadlocks by letting each handshake make progress independently. * Move the dial task into a separate function This refactor improves readability. * Fix buggy future::select function usage And document the correctness of the new code.
2021-04-07 06:25:10 -07:00
use tokio::time::{sleep, sleep_until, timeout, Sleep};
use tower::{Service, ServiceExt};
2021-05-31 06:49:59 -07:00
use zebra_chain::serialization::DateTime32;
Fix a deadlock between the crawler and dialer, and other hangs (#1950) * Stop ignoring inbound message errors and handshake timeouts To avoid hangs, Zebra needs to maintain the following invariants in the handshake and heartbeat code: - each handshake should run in a separate spawned task (not yet implemented) - every message, error, timeout, and shutdown must update the peer address state - every await that depends on the network must have a timeout Once the Connection is created, it should handle timeouts. But we need to handle timeouts during handshake setup. * Avoid hangs by adding a timeout to the candidate set update Also increase the fanout from 1 to 2, to increase address diversity. But only return permanent errors from `CandidateSet::update`, because the crawler task exits if `update` returns an error. Also log Peers response errors in the CandidateSet. * Use the select macro in the crawler to reduce hangs The `select` function is biased towards its first argument, risking starvation. As a side-benefit, this change also makes the code a lot easier to read and maintain. * Split CrawlerAction::Demand into separate actions This refactor makes the code a bit easier to read, at the cost of sometimes blocking the crawler on `candidates.next()`. That's ok, because `next` only has a short (< 100 ms) delay. And we're just about to spawn a separate task for each handshake. * Spawn a separate task for each handshake This change avoids deadlocks by letting each handshake make progress independently. * Move the dial task into a separate function This refactor improves readability. * Fix buggy future::select function usage And document the correctness of the new code.
2021-04-07 06:25:10 -07:00
use crate::{constants, types::MetaAddr, AddressBook, BoxError, Request, Response};
/// The `CandidateSet` manages the `PeerSet`'s peer reconnection attempts.
///
/// It divides the set of all possible candidate peers into disjoint subsets,
/// using the `PeerAddrState`:
///
/// 1. `Responded` peers, which we previously had inbound or outbound connections
/// to. If we have not received any messages from a `Responded` peer within a
/// cutoff time, we assume that it has disconnected or hung, and attempt
/// reconnection;
/// 2. `NeverAttempted` peers, which we learned about from other peers or a DNS
/// seeder, but have never connected to;
/// 3. `Failed` peers, to whom we attempted to connect but were unable to;
/// 4. `AttemptPending` peers, which we've recently queued for reconnection.
///
/// ```ascii,no_run
/// ┌──────────────────┐
/// │ PeerSet │
/// │GetPeers Responses│
/// └──────────────────┘
/// │
/// │
/// │
/// │
/// ▼
/// filter by Λ
/// !contains_addr
/// ┌────────────────────────────▶▕ ▏
/// │ ╲
/// │ V
/// │ │
/// │ │
/// │ │
/// │ ┌──────────────────┐ │
/// │ │ Inbound │ │
/// │ │ Peer Connections │ │
/// │ └──────────────────┘ │
/// │ │ │
/// ├──────────┼────────────────────┼───────────────────────────────┐
/// │ PeerSet ▼ AddressBook ▼ │
/// │ ┌─────────────┐ ┌────────────────┐ ┌─────────────┐ │
/// │ │ Possibly │ │`NeverAttempted`│ │ `Failed` │ │
/// │ │Disconnected │ │ Peers │ │ Peers │◀┼┐
/// │ │ `Responded` │ │ │ │ │ ││
/// │ │ Peers │ │ │ │ │ ││
/// │ └─────────────┘ └────────────────┘ └─────────────┘ ││
/// │ │ │ │ ││
/// │ #1 oldest_first #2 newest_first #3 oldest_first ││
/// │ │ │ │ ││
/// │ ├──────────────────────┴──────────────────────┘ ││
/// │ │ disjoint `PeerAddrState`s ││
/// ├────────┼──────────────────────────────────────────────────────┘│
/// │ ▼ │
/// │ Λ │
/// │ ╲ filter by │
/// └─────▶▕ ▏!is_potentially_connected │
/// ╲ to remove live │
/// V `Responded` peers │
/// │ │
/// │ Try outbound connection │
/// ▼ │
/// ┌────────────────┐ │
/// │`AttemptPending`│ │
/// │ Peers │ │
/// │ │ │
/// └────────────────┘ │
/// │ │
/// │ │
/// ▼ │
/// Λ │
/// ╲ │
/// ▕ ▏─────────────────────────────────────────────────────┘
/// ╲ connection failed, update last_seen to now()
/// V
/// │
/// │
/// ▼
/// ┌────────────┐
/// │ send │
2019-11-27 11:27:17 -08:00
/// │peer::Client│
/// │to Discover │
/// └────────────┘
/// │
/// │
/// ▼
/// ┌───────────────────────────────────────┐
/// │ every time we receive a peer message: │
/// │ * update state to `Responded` │
/// │ * update last_seen to now() │
/// └───────────────────────────────────────┘
///
/// ```
// TODO:
// * draw arrow from the "peer message" box into the `Responded` state box
// * make the "disjoint states" box include `AttemptPending`
pub(super) struct CandidateSet<S> {
pub(super) address_book: Arc<std::sync::Mutex<AddressBook>>,
pub(super) peer_service: S,
next_peer_min_wait: Sleep,
}
impl<S> CandidateSet<S>
where
S: Service<Request, Response = Response, Error = BoxError>,
S::Future: Send + 'static,
{
/// The minimum time between successive calls to `CandidateSet::next()`.
///
/// ## Security
///
/// Zebra resists distributed denial of service attacks by making sure that new peer connections
/// are initiated at least `MIN_PEER_CONNECTION_INTERVAL` apart.
const MIN_PEER_CONNECTION_INTERVAL: Duration = Duration::from_millis(100);
/// Uses `address_book` and `peer_service` to manage a [`CandidateSet`] of peers.
pub fn new(
address_book: Arc<std::sync::Mutex<AddressBook>>,
peer_service: S,
) -> CandidateSet<S> {
CandidateSet {
address_book,
peer_service,
next_peer_min_wait: sleep(Duration::from_secs(0)),
}
}
/// Update the peer set from the network, using the default fanout limit.
///
/// See [`update_initial`][Self::update_initial] for details.
pub async fn update(&mut self) -> Result<(), BoxError> {
self.update_timeout(None).await
}
/// Update the peer set from the network, limiting the fanout to
/// `fanout_limit`.
///
/// - Ask a few live [`Responded`] peers to send us more peers.
/// - Process all completed peer responses, adding new peers in the
/// [`NeverAttemptedGossiped`] state.
///
/// ## Correctness
///
/// Pass the initial peer set size as `fanout_limit` during initialization,
/// so that Zebra does not send duplicate requests to the same peer.
///
Fix a deadlock between the crawler and dialer, and other hangs (#1950) * Stop ignoring inbound message errors and handshake timeouts To avoid hangs, Zebra needs to maintain the following invariants in the handshake and heartbeat code: - each handshake should run in a separate spawned task (not yet implemented) - every message, error, timeout, and shutdown must update the peer address state - every await that depends on the network must have a timeout Once the Connection is created, it should handle timeouts. But we need to handle timeouts during handshake setup. * Avoid hangs by adding a timeout to the candidate set update Also increase the fanout from 1 to 2, to increase address diversity. But only return permanent errors from `CandidateSet::update`, because the crawler task exits if `update` returns an error. Also log Peers response errors in the CandidateSet. * Use the select macro in the crawler to reduce hangs The `select` function is biased towards its first argument, risking starvation. As a side-benefit, this change also makes the code a lot easier to read and maintain. * Split CrawlerAction::Demand into separate actions This refactor makes the code a bit easier to read, at the cost of sometimes blocking the crawler on `candidates.next()`. That's ok, because `next` only has a short (< 100 ms) delay. And we're just about to spawn a separate task for each handshake. * Spawn a separate task for each handshake This change avoids deadlocks by letting each handshake make progress independently. * Move the dial task into a separate function This refactor improves readability. * Fix buggy future::select function usage And document the correctness of the new code.
2021-04-07 06:25:10 -07:00
/// The crawler exits when update returns an error, so it must only return
/// errors on permanent failures.
///
/// The handshaker sets up the peer message receiver so it also sends a
/// [`Responded`] peer address update.
///
/// [`report_failed`][Self::report_failed] puts peers into the [`Failed`] state.
///
/// [`next`][Self::next] puts peers into the [`AttemptPending`] state.
///
/// [`Responded`]: crate::PeerAddrState::Responded
/// [`NeverAttemptedGossiped`]: crate::PeerAddrState::NeverAttemptedGossiped
/// [`Failed`]: crate::PeerAddrState::Failed
/// [`AttemptPending`]: crate::PeerAddrState::AttemptPending
pub async fn update_initial(&mut self, fanout_limit: usize) -> Result<(), BoxError> {
self.update_timeout(Some(fanout_limit)).await
}
/// Update the peer set from the network, limiting the fanout to
/// `fanout_limit`, and imposing a timeout on the entire fanout.
///
/// See [`update_initial`][Self::update_initial] for details.
async fn update_timeout(&mut self, fanout_limit: Option<usize>) -> Result<(), BoxError> {
// CORRECTNESS
//
// Use a timeout to avoid deadlocks when there are no connected
// peers, and:
// - we're waiting on a handshake to complete so there are peers, or
// - another task that handles or adds peers is waiting on this task
// to complete.
if let Ok(fanout_result) =
timeout(constants::REQUEST_TIMEOUT, self.update_fanout(fanout_limit)).await
{
fanout_result?;
} else {
// update must only return an error for permanent failures
info!("timeout waiting for the peer service to become ready");
}
Ok(())
}
/// Update the peer set from the network, limiting the fanout to
/// `fanout_limit`.
///
/// See [`update_initial`][Self::update_initial] for details.
///
/// # Correctness
///
/// This function does not have a timeout.
/// Use [`update_timeout`][Self::update_timeout] instead.
async fn update_fanout(&mut self, fanout_limit: Option<usize>) -> Result<(), BoxError> {
// Opportunistically crawl the network on every update call to ensure
// we're actively fetching peers. Continue independently of whether we
// actually receive any peers, but always ask the network for more.
Fix a deadlock between the crawler and dialer, and other hangs (#1950) * Stop ignoring inbound message errors and handshake timeouts To avoid hangs, Zebra needs to maintain the following invariants in the handshake and heartbeat code: - each handshake should run in a separate spawned task (not yet implemented) - every message, error, timeout, and shutdown must update the peer address state - every await that depends on the network must have a timeout Once the Connection is created, it should handle timeouts. But we need to handle timeouts during handshake setup. * Avoid hangs by adding a timeout to the candidate set update Also increase the fanout from 1 to 2, to increase address diversity. But only return permanent errors from `CandidateSet::update`, because the crawler task exits if `update` returns an error. Also log Peers response errors in the CandidateSet. * Use the select macro in the crawler to reduce hangs The `select` function is biased towards its first argument, risking starvation. As a side-benefit, this change also makes the code a lot easier to read and maintain. * Split CrawlerAction::Demand into separate actions This refactor makes the code a bit easier to read, at the cost of sometimes blocking the crawler on `candidates.next()`. That's ok, because `next` only has a short (< 100 ms) delay. And we're just about to spawn a separate task for each handshake. * Spawn a separate task for each handshake This change avoids deadlocks by letting each handshake make progress independently. * Move the dial task into a separate function This refactor improves readability. * Fix buggy future::select function usage And document the correctness of the new code.
2021-04-07 06:25:10 -07:00
//
// Because requests are load-balanced across existing peers, we can make
// multiple requests concurrently, which will be randomly assigned to
// existing peers, but we don't make too many because update may be
// called while the peer set is already loaded.
let mut responses = FuturesUnordered::new();
let fanout_limit = fanout_limit
.map(|fanout_limit| min(fanout_limit, constants::GET_ADDR_FANOUT))
.unwrap_or(constants::GET_ADDR_FANOUT);
debug!(?fanout_limit, "sending GetPeers requests");
// TODO: launch each fanout in its own task (might require tokio 1.6)
for _ in 0..fanout_limit {
let peer_service = self.peer_service.ready_and().await?;
Fix a deadlock between the crawler and dialer, and other hangs (#1950) * Stop ignoring inbound message errors and handshake timeouts To avoid hangs, Zebra needs to maintain the following invariants in the handshake and heartbeat code: - each handshake should run in a separate spawned task (not yet implemented) - every message, error, timeout, and shutdown must update the peer address state - every await that depends on the network must have a timeout Once the Connection is created, it should handle timeouts. But we need to handle timeouts during handshake setup. * Avoid hangs by adding a timeout to the candidate set update Also increase the fanout from 1 to 2, to increase address diversity. But only return permanent errors from `CandidateSet::update`, because the crawler task exits if `update` returns an error. Also log Peers response errors in the CandidateSet. * Use the select macro in the crawler to reduce hangs The `select` function is biased towards its first argument, risking starvation. As a side-benefit, this change also makes the code a lot easier to read and maintain. * Split CrawlerAction::Demand into separate actions This refactor makes the code a bit easier to read, at the cost of sometimes blocking the crawler on `candidates.next()`. That's ok, because `next` only has a short (< 100 ms) delay. And we're just about to spawn a separate task for each handshake. * Spawn a separate task for each handshake This change avoids deadlocks by letting each handshake make progress independently. * Move the dial task into a separate function This refactor improves readability. * Fix buggy future::select function usage And document the correctness of the new code.
2021-04-07 06:25:10 -07:00
responses.push(peer_service.call(Request::Peers));
}
while let Some(rsp) = responses.next().await {
Fix a deadlock between the crawler and dialer, and other hangs (#1950) * Stop ignoring inbound message errors and handshake timeouts To avoid hangs, Zebra needs to maintain the following invariants in the handshake and heartbeat code: - each handshake should run in a separate spawned task (not yet implemented) - every message, error, timeout, and shutdown must update the peer address state - every await that depends on the network must have a timeout Once the Connection is created, it should handle timeouts. But we need to handle timeouts during handshake setup. * Avoid hangs by adding a timeout to the candidate set update Also increase the fanout from 1 to 2, to increase address diversity. But only return permanent errors from `CandidateSet::update`, because the crawler task exits if `update` returns an error. Also log Peers response errors in the CandidateSet. * Use the select macro in the crawler to reduce hangs The `select` function is biased towards its first argument, risking starvation. As a side-benefit, this change also makes the code a lot easier to read and maintain. * Split CrawlerAction::Demand into separate actions This refactor makes the code a bit easier to read, at the cost of sometimes blocking the crawler on `candidates.next()`. That's ok, because `next` only has a short (< 100 ms) delay. And we're just about to spawn a separate task for each handshake. * Spawn a separate task for each handshake This change avoids deadlocks by letting each handshake make progress independently. * Move the dial task into a separate function This refactor improves readability. * Fix buggy future::select function usage And document the correctness of the new code.
2021-04-07 06:25:10 -07:00
match rsp {
Ok(Response::Peers(addrs)) => {
Fix a deadlock between the crawler and dialer, and other hangs (#1950) * Stop ignoring inbound message errors and handshake timeouts To avoid hangs, Zebra needs to maintain the following invariants in the handshake and heartbeat code: - each handshake should run in a separate spawned task (not yet implemented) - every message, error, timeout, and shutdown must update the peer address state - every await that depends on the network must have a timeout Once the Connection is created, it should handle timeouts. But we need to handle timeouts during handshake setup. * Avoid hangs by adding a timeout to the candidate set update Also increase the fanout from 1 to 2, to increase address diversity. But only return permanent errors from `CandidateSet::update`, because the crawler task exits if `update` returns an error. Also log Peers response errors in the CandidateSet. * Use the select macro in the crawler to reduce hangs The `select` function is biased towards its first argument, risking starvation. As a side-benefit, this change also makes the code a lot easier to read and maintain. * Split CrawlerAction::Demand into separate actions This refactor makes the code a bit easier to read, at the cost of sometimes blocking the crawler on `candidates.next()`. That's ok, because `next` only has a short (< 100 ms) delay. And we're just about to spawn a separate task for each handshake. * Spawn a separate task for each handshake This change avoids deadlocks by letting each handshake make progress independently. * Move the dial task into a separate function This refactor improves readability. * Fix buggy future::select function usage And document the correctness of the new code.
2021-04-07 06:25:10 -07:00
trace!(
addr_count = ?addrs.len(),
?addrs,
Fix a deadlock between the crawler and dialer, and other hangs (#1950) * Stop ignoring inbound message errors and handshake timeouts To avoid hangs, Zebra needs to maintain the following invariants in the handshake and heartbeat code: - each handshake should run in a separate spawned task (not yet implemented) - every message, error, timeout, and shutdown must update the peer address state - every await that depends on the network must have a timeout Once the Connection is created, it should handle timeouts. But we need to handle timeouts during handshake setup. * Avoid hangs by adding a timeout to the candidate set update Also increase the fanout from 1 to 2, to increase address diversity. But only return permanent errors from `CandidateSet::update`, because the crawler task exits if `update` returns an error. Also log Peers response errors in the CandidateSet. * Use the select macro in the crawler to reduce hangs The `select` function is biased towards its first argument, risking starvation. As a side-benefit, this change also makes the code a lot easier to read and maintain. * Split CrawlerAction::Demand into separate actions This refactor makes the code a bit easier to read, at the cost of sometimes blocking the crawler on `candidates.next()`. That's ok, because `next` only has a short (< 100 ms) delay. And we're just about to spawn a separate task for each handshake. * Spawn a separate task for each handshake This change avoids deadlocks by letting each handshake make progress independently. * Move the dial task into a separate function This refactor improves readability. * Fix buggy future::select function usage And document the correctness of the new code.
2021-04-07 06:25:10 -07:00
"got response to GetPeers"
);
2021-05-31 06:49:59 -07:00
let addrs = validate_addrs(addrs, DateTime32::now());
self.send_addrs(addrs);
Fix a deadlock between the crawler and dialer, and other hangs (#1950) * Stop ignoring inbound message errors and handshake timeouts To avoid hangs, Zebra needs to maintain the following invariants in the handshake and heartbeat code: - each handshake should run in a separate spawned task (not yet implemented) - every message, error, timeout, and shutdown must update the peer address state - every await that depends on the network must have a timeout Once the Connection is created, it should handle timeouts. But we need to handle timeouts during handshake setup. * Avoid hangs by adding a timeout to the candidate set update Also increase the fanout from 1 to 2, to increase address diversity. But only return permanent errors from `CandidateSet::update`, because the crawler task exits if `update` returns an error. Also log Peers response errors in the CandidateSet. * Use the select macro in the crawler to reduce hangs The `select` function is biased towards its first argument, risking starvation. As a side-benefit, this change also makes the code a lot easier to read and maintain. * Split CrawlerAction::Demand into separate actions This refactor makes the code a bit easier to read, at the cost of sometimes blocking the crawler on `candidates.next()`. That's ok, because `next` only has a short (< 100 ms) delay. And we're just about to spawn a separate task for each handshake. * Spawn a separate task for each handshake This change avoids deadlocks by letting each handshake make progress independently. * Move the dial task into a separate function This refactor improves readability. * Fix buggy future::select function usage And document the correctness of the new code.
2021-04-07 06:25:10 -07:00
}
Err(e) => {
// since we do a fanout, and new updates are triggered by
// each demand, we can ignore errors in individual responses
trace!(?e, "got error in GetPeers request");
}
Ok(_) => unreachable!("Peers requests always return Peers responses"),
}
}
Ok(())
}
/// Add new `addrs` to the address book.
fn send_addrs(&self, addrs: impl IntoIterator<Item = MetaAddr>) {
// # Correctness
//
// Briefly hold the address book threaded mutex, to extend
// the address list.
//
// Extend handles duplicate addresses internally.
self.address_book.lock().unwrap().extend(addrs);
}
/// Returns the next candidate for a connection attempt, if any are available.
///
/// Returns peers in this order:
/// - oldest `Responded` that are not live
/// - newest `NeverAttempted`
/// - oldest `Failed`
///
/// Skips `AttemptPending` peers and live `Responded` peers.
///
/// ## Correctness
///
/// `AttemptPending` peers will become `Responded` if they respond, or
/// become `Failed` if they time out or provide a bad response.
///
/// Live `Responded` peers will stay live if they keep responding, or
/// become a reconnection candidate if they stop responding.
///
/// ## Security
///
/// Zebra resists distributed denial of service attacks by making sure that
/// new peer connections are initiated at least
/// `MIN_PEER_CONNECTION_INTERVAL` apart.
pub async fn next(&mut self) -> Option<MetaAddr> {
let current_deadline = self.next_peer_min_wait.deadline();
let mut sleep = sleep_until(current_deadline + Self::MIN_PEER_CONNECTION_INTERVAL);
mem::swap(&mut self.next_peer_min_wait, &mut sleep);
// # Correctness
Fix a deadlock between the crawler and dialer, and other hangs (#1950) * Stop ignoring inbound message errors and handshake timeouts To avoid hangs, Zebra needs to maintain the following invariants in the handshake and heartbeat code: - each handshake should run in a separate spawned task (not yet implemented) - every message, error, timeout, and shutdown must update the peer address state - every await that depends on the network must have a timeout Once the Connection is created, it should handle timeouts. But we need to handle timeouts during handshake setup. * Avoid hangs by adding a timeout to the candidate set update Also increase the fanout from 1 to 2, to increase address diversity. But only return permanent errors from `CandidateSet::update`, because the crawler task exits if `update` returns an error. Also log Peers response errors in the CandidateSet. * Use the select macro in the crawler to reduce hangs The `select` function is biased towards its first argument, risking starvation. As a side-benefit, this change also makes the code a lot easier to read and maintain. * Split CrawlerAction::Demand into separate actions This refactor makes the code a bit easier to read, at the cost of sometimes blocking the crawler on `candidates.next()`. That's ok, because `next` only has a short (< 100 ms) delay. And we're just about to spawn a separate task for each handshake. * Spawn a separate task for each handshake This change avoids deadlocks by letting each handshake make progress independently. * Move the dial task into a separate function This refactor improves readability. * Fix buggy future::select function usage And document the correctness of the new code.
2021-04-07 06:25:10 -07:00
//
// In this critical section, we hold the address mutex, blocking the
// current thread, and all async tasks scheduled on that thread.
Fix a deadlock between the crawler and dialer, and other hangs (#1950) * Stop ignoring inbound message errors and handshake timeouts To avoid hangs, Zebra needs to maintain the following invariants in the handshake and heartbeat code: - each handshake should run in a separate spawned task (not yet implemented) - every message, error, timeout, and shutdown must update the peer address state - every await that depends on the network must have a timeout Once the Connection is created, it should handle timeouts. But we need to handle timeouts during handshake setup. * Avoid hangs by adding a timeout to the candidate set update Also increase the fanout from 1 to 2, to increase address diversity. But only return permanent errors from `CandidateSet::update`, because the crawler task exits if `update` returns an error. Also log Peers response errors in the CandidateSet. * Use the select macro in the crawler to reduce hangs The `select` function is biased towards its first argument, risking starvation. As a side-benefit, this change also makes the code a lot easier to read and maintain. * Split CrawlerAction::Demand into separate actions This refactor makes the code a bit easier to read, at the cost of sometimes blocking the crawler on `candidates.next()`. That's ok, because `next` only has a short (< 100 ms) delay. And we're just about to spawn a separate task for each handshake. * Spawn a separate task for each handshake This change avoids deadlocks by letting each handshake make progress independently. * Move the dial task into a separate function This refactor improves readability. * Fix buggy future::select function usage And document the correctness of the new code.
2021-04-07 06:25:10 -07:00
//
// To avoid deadlocks, the critical section:
// - must not acquire any other locks
// - must not await any futures
//
// To avoid hangs, any computation in the critical section should
// be kept to a minimum.
let reconnect = {
let mut guard = self.address_book.lock().unwrap();
// It's okay to return without sleeping here, because we're returning
// `None`. We only need to sleep before yielding an address.
let reconnect = guard.reconnection_peers().next()?;
let reconnect = MetaAddr::new_reconnect(&reconnect.addr, &reconnect.services);
guard.update(reconnect);
reconnect
};
// SECURITY: rate-limit new candidate connections
sleep.await;
Some(reconnect)
}
/// Mark `addr` as a failed peer.
pub fn report_failed(&mut self, addr: &MetaAddr) {
let addr = MetaAddr::new_errored(&addr.addr, &addr.services);
// # Correctness
//
// Briefly hold the address book threaded mutex, to update the state for
// a single address.
self.address_book.lock().unwrap().update(addr);
}
}
/// Check new `addrs` before adding them to the address book.
///
/// `last_seen_limit` is the maximum permitted last seen time, typically
/// [`Utc::now`].
///
/// If the data in an address is invalid, this function can:
/// - modify the address data, or
/// - delete the address.
//
// TODO: re-enable this lint when last_seen_limit is used
#[allow(unused_variables)]
fn validate_addrs(
addrs: impl IntoIterator<Item = MetaAddr>,
2021-05-31 06:49:59 -07:00
last_seen_limit: DateTime32,
) -> impl Iterator<Item = MetaAddr> {
// Note: The address book handles duplicate addresses internally,
// so we don't need to de-duplicate addresses here.
// TODO:
// We should eventually implement these checks in this function:
// - Zebra should stop believing far-future last_seen times from peers (#1871)
// - Zebra should ignore peers that are older than 3 weeks (part of #1865)
// - Zebra should count back 3 weeks from the newest peer timestamp sent
// by the other peer, to compensate for clock skew
// - Zebra should limit the number of addresses it uses from a single Addrs
// response (#1869)
addrs.into_iter()
}