Re-order Zebra startup, so slow services are launched last (#3091)

* Start network before verifiers

This makes the Groth16 download task start as late as possible.

* Explain why the Groth16 download must happen first

* Speed up Zebra shutdown: skip waiting for the tokio runtime
This commit is contained in:
teor 2021-11-24 03:42:44 +10:00 committed by GitHub
parent 8e49663e31
commit 68d7198e9f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 224 additions and 220 deletions

View File

@ -197,7 +197,8 @@ where
S: Service<zs::Request, Response = zs::Response, Error = BoxError> + Send + Clone + 'static, S: Service<zs::Request, Response = zs::Response, Error = BoxError> + Send + Clone + 'static,
S::Future: Send + 'static, S::Future: Send + 'static,
{ {
// pre-download Groth16 parameters async // Pre-download Groth16 parameters in a separate thread.
// This thread must be launched first, so the download doesn't happen on the startup thread.
let groth16_download_handle = spawn_blocking(|| { let groth16_download_handle = spawn_blocking(|| {
tracing::info!("checking if Zcash Sapling and Sprout parameters have been downloaded"); tracing::info!("checking if Zcash Sapling and Sprout parameters have been downloaded");

View File

@ -56,6 +56,7 @@ use tower::{builder::ServiceBuilder, util::BoxService};
use crate::{ use crate::{
components::{ components::{
inbound::InboundSetupData,
mempool::{self, Mempool}, mempool::{self, Mempool},
sync, sync,
tokio::{RuntimeRun, TokioComponent}, tokio::{RuntimeRun, TokioComponent},
@ -83,6 +84,19 @@ impl StartCmd {
zebra_state::init(config.state.clone(), config.network.network); zebra_state::init(config.state.clone(), config.network.network);
let state = ServiceBuilder::new().buffer(20).service(state_service); let state = ServiceBuilder::new().buffer(20).service(state_service);
info!("initializing network");
// The service that our node uses to respond to requests by peers. The
// load_shed middleware ensures that we reduce the size of the peer set
// in response to excess load.
let (setup_tx, setup_rx) = oneshot::channel();
let inbound = ServiceBuilder::new()
.load_shed()
.buffer(20)
.service(Inbound::new(setup_rx));
let (peer_set, address_book) =
zebra_network::init(config.network.clone(), inbound, latest_chain_tip.clone()).await;
info!("initializing verifiers"); info!("initializing verifiers");
let (chain_verifier, tx_verifier, mut groth16_download_handle) = let (chain_verifier, tx_verifier, mut groth16_download_handle) =
zebra_consensus::chain::init( zebra_consensus::chain::init(
@ -92,32 +106,19 @@ impl StartCmd {
) )
.await; .await;
info!("initializing network");
// The service that our node uses to respond to requests by peers. The
// load_shed middleware ensures that we reduce the size of the peer set
// in response to excess load.
let (setup_tx, setup_rx) = oneshot::channel();
let inbound = ServiceBuilder::new()
.load_shed()
.buffer(20)
.service(Inbound::new(
setup_rx,
state.clone(),
chain_verifier.clone(),
));
let (peer_set, address_book) =
zebra_network::init(config.network.clone(), inbound, latest_chain_tip.clone()).await;
info!("initializing syncer"); info!("initializing syncer");
let (syncer, sync_status) = let (syncer, sync_status) = ChainSync::new(
ChainSync::new(&config, peer_set.clone(), state.clone(), chain_verifier); &config,
peer_set.clone(),
state.clone(),
chain_verifier.clone(),
);
info!("initializing mempool"); info!("initializing mempool");
let (mempool, mempool_transaction_receiver) = Mempool::new( let (mempool, mempool_transaction_receiver) = Mempool::new(
&config.mempool, &config.mempool,
peer_set.clone(), peer_set.clone(),
state, state.clone(),
tx_verifier, tx_verifier,
sync_status.clone(), sync_status.clone(),
latest_chain_tip, latest_chain_tip,
@ -126,8 +127,15 @@ impl StartCmd {
let mempool = BoxService::new(mempool); let mempool = BoxService::new(mempool);
let mempool = ServiceBuilder::new().buffer(20).service(mempool); let mempool = ServiceBuilder::new().buffer(20).service(mempool);
let setup_data = InboundSetupData {
address_book,
block_download_peer_set: peer_set.clone(),
block_verifier: chain_verifier,
mempool: mempool.clone(),
state,
};
setup_tx setup_tx
.send((peer_set.clone(), address_book, mempool.clone())) .send(setup_data)
.map_err(|_| eyre!("could not send setup data to inbound service"))?; .map_err(|_| eyre!("could not send setup data to inbound service"))?;
let syncer_error_future = syncer.sync(); let syncer_error_future = syncer.sync();

View File

@ -5,7 +5,7 @@
//! component and dependency injection models are designed to work together, but //! component and dependency injection models are designed to work together, but
//! don't fit the async context well. //! don't fit the async context well.
mod inbound; pub mod inbound;
#[allow(missing_docs)] #[allow(missing_docs)]
pub mod mempool; pub mod mempool;
pub mod metrics; pub mod metrics;

View File

@ -16,8 +16,7 @@ use futures::{
future::{FutureExt, TryFutureExt}, future::{FutureExt, TryFutureExt},
stream::Stream, stream::Stream,
}; };
use oneshot::error::TryRecvError; use tokio::sync::oneshot::{self, error::TryRecvError};
use tokio::sync::oneshot;
use tower::{buffer::Buffer, timeout::Timeout, util::BoxService, Service, ServiceExt}; use tower::{buffer::Buffer, timeout::Timeout, util::BoxService, Service, ServiceExt};
use zebra_network as zn; use zebra_network as zn;
@ -39,35 +38,50 @@ mod tests;
use downloads::Downloads as BlockDownloads; use downloads::Downloads as BlockDownloads;
type Outbound = Buffer<BoxService<zn::Request, zn::Response, zn::BoxError>, zn::Request>;
type State = Buffer<BoxService<zs::Request, zs::Response, zs::BoxError>, zs::Request>;
type Mempool = Buffer<BoxService<mp::Request, mp::Response, mp::BoxError>, mp::Request>;
type BlockVerifier = Buffer<BoxService<Arc<Block>, block::Hash, VerifyChainError>, Arc<Block>>;
type InboundBlockDownloads = BlockDownloads<Timeout<Outbound>, Timeout<BlockVerifier>, State>;
pub type NetworkSetupData = (Outbound, Arc<std::sync::Mutex<AddressBook>>, Mempool);
/// A security parameter to return only 1/3 of available addresses as a /// A security parameter to return only 1/3 of available addresses as a
/// response to a `Peers` request. /// response to a `Peers` request.
const FRAC_OF_AVAILABLE_ADDRESS: f64 = 1. / 3.; const FRAC_OF_AVAILABLE_ADDRESS: f64 = 1. / 3.;
/// Tracks the internal state of the [`Inbound`] service during network setup. type BlockDownloadPeerSet =
pub enum Setup { Buffer<BoxService<zn::Request, zn::Response, zn::BoxError>, zn::Request>;
/// Waiting for network setup to complete. type State = Buffer<BoxService<zs::Request, zs::Response, zs::BoxError>, zs::Request>;
/// type Mempool = Buffer<BoxService<mp::Request, mp::Response, mp::BoxError>, mp::Request>;
/// Requests that depend on Zebra's internal network setup are ignored. type BlockVerifier = Buffer<BoxService<Arc<Block>, block::Hash, VerifyChainError>, Arc<Block>>;
/// Other requests are answered. type GossipedBlockDownloads =
AwaitingNetwork { BlockDownloads<Timeout<BlockDownloadPeerSet>, Timeout<BlockVerifier>, State>;
/// A oneshot channel used to receive the address_book and outbound services
/// after the network is set up.
network_setup: oneshot::Receiver<NetworkSetupData>,
/// A service that verifies downloaded blocks. Given to `block_downloads` /// The services used by the [`Inbound`] service.
/// after the network is set up. pub struct InboundSetupData {
block_verifier: BlockVerifier, /// A shared list of peer addresses.
pub address_book: Arc<std::sync::Mutex<AddressBook>>,
/// A service that can be used to download gossiped blocks.
pub block_download_peer_set: BlockDownloadPeerSet,
/// A service that verifies downloaded blocks.
///
/// Given to `Inbound.block_downloads` after the required services are set up.
pub block_verifier: BlockVerifier,
/// A service that manages transactions in the memory pool.
pub mempool: Mempool,
/// A service that manages cached blockchain state.
pub state: State,
}
/// Tracks the internal state of the [`Inbound`] service during setup.
pub enum Setup {
/// Waiting for service setup to complete.
///
/// All requests are ignored.
Pending {
/// A oneshot channel used to receive required services,
/// after they are set up.
setup: oneshot::Receiver<InboundSetupData>,
}, },
/// Network setup is complete. /// Setup is complete.
/// ///
/// All requests are answered. /// All requests are answered.
Initialized { Initialized {
@ -75,24 +89,29 @@ pub enum Setup {
address_book: Arc<std::sync::Mutex<zn::AddressBook>>, address_book: Arc<std::sync::Mutex<zn::AddressBook>>,
/// A `futures::Stream` that downloads and verifies gossiped blocks. /// A `futures::Stream` that downloads and verifies gossiped blocks.
block_downloads: Pin<Box<InboundBlockDownloads>>, block_downloads: Pin<Box<GossipedBlockDownloads>>,
/// A service that manages transactions in the memory pool. /// A service that manages transactions in the memory pool.
mempool: Mempool, mempool: Mempool,
/// A service that manages cached blockchain state.
state: State,
}, },
/// Temporary state used in the service's internal network initialization /// Temporary state used in the inbound service's internal initialization code.
/// code.
/// ///
/// If this state occurs outside the service initialization code, the /// If this state occurs outside the service initialization code, the service panics.
/// service panics.
FailedInit, FailedInit,
/// Network setup failed, because the setup channel permanently failed. /// Setup failed, because the setup channel permanently failed.
/// The service keeps returning readiness errors for every request. /// The service keeps returning readiness errors for every request.
FailedRecv { error: SharedRecvError }, FailedRecv {
/// The original channel error.
error: SharedRecvError,
},
} }
/// A wrapper around `Arc<TryRecvError>` that implements `Error`.
#[derive(thiserror::Error, Debug, Clone)] #[derive(thiserror::Error, Debug, Clone)]
#[error(transparent)] #[error(transparent)]
pub struct SharedRecvError(Arc<TryRecvError>); pub struct SharedRecvError(Arc<TryRecvError>);
@ -112,51 +131,46 @@ impl From<TryRecvError> for SharedRecvError {
/// ///
/// - supplying network data like peer addresses to other nodes; /// - supplying network data like peer addresses to other nodes;
/// - supplying chain data like blocks to other nodes; /// - supplying chain data like blocks to other nodes;
/// - performing transaction diffusion; /// - supplying mempool transactions to other nodes;
/// - performing block diffusion. /// - receiving gossiped transactions; and
/// - receiving gossiped blocks.
/// ///
/// Because the `Inbound` service is responsible for participating in the gossip /// Because the `Inbound` service is responsible for participating in the gossip
/// protocols used for transaction and block diffusion, there is a potential /// protocols used for transaction and block diffusion, there is a potential
/// overlap with the `ChainSync` component. /// overlap with the `ChainSync` and `Mempool` components.
/// ///
/// The division of responsibility is that the `ChainSync` component is /// The division of responsibility is that:
/// *internally driven*, periodically polling the network to check whether it is ///
/// behind the current tip, while the `Inbound` service is *externally driven*, /// The `ChainSync` and `Mempool` components are *internally driven*,
/// responding to block gossip by attempting to download and validate advertised /// periodically polling the network to check for new blocks or transactions.
/// blocks. ///
/// The `Inbound` service is *externally driven*, responding to block gossip
/// by attempting to download and validate advertised blocks.
///
/// Gossiped transactions are forwarded to the mempool downloader,
/// which unifies polled and gossiped transactions into a single download list.
pub struct Inbound { pub struct Inbound {
/// Provides network-dependent services, if they are available. /// Provides service dependencies, if they are available.
/// ///
/// Some services are unavailable until Zebra has completed network setup. /// Some services are unavailable until Zebra has completed setup.
network_setup: Setup, setup: Setup,
/// A service that manages cached blockchain state.
state: State,
} }
impl Inbound { impl Inbound {
/// Create a new inbound service. /// Create a new inbound service.
/// ///
/// The address book and peer set use the newly created inbound service. /// Dependent services are sent via the `setup` channel after initialization.
/// So they are sent via the `network_setup` channel after initialization. pub fn new(setup: oneshot::Receiver<InboundSetupData>) -> Inbound {
pub fn new( Inbound {
network_setup: oneshot::Receiver<NetworkSetupData>, setup: Setup::Pending { setup },
state: State,
block_verifier: BlockVerifier,
) -> Self {
Self {
network_setup: Setup::AwaitingNetwork {
network_setup,
block_verifier,
},
state,
} }
} }
/// Remove `self.setup`, temporarily replacing it with an invalid state.
fn take_setup(&mut self) -> Setup { fn take_setup(&mut self) -> Setup {
let mut network_setup = Setup::FailedInit; let mut setup = Setup::FailedInit;
std::mem::swap(&mut self.network_setup, &mut network_setup); std::mem::swap(&mut self.setup, &mut setup);
network_setup setup
} }
} }
@ -167,25 +181,30 @@ impl Service<zn::Request> for Inbound {
Pin<Box<dyn Future<Output = Result<Self::Response, Self::Error>> + Send + 'static>>; Pin<Box<dyn Future<Output = Result<Self::Response, Self::Error>> + Send + 'static>>;
fn poll_ready(&mut self, cx: &mut Context<'_>) -> Poll<Result<(), Self::Error>> { fn poll_ready(&mut self, cx: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
// Check whether the network setup is finished, but don't wait for it to // Check whether the setup is finished, but don't wait for it to
// become ready before reporting readiness. We expect to get it "soon", // become ready before reporting readiness. We expect to get it "soon",
// and reporting unreadiness might cause unwanted load-shedding, since // and reporting unreadiness might cause unwanted load-shedding, since
// the load-shed middleware is unable to distinguish being unready due // the load-shed middleware is unable to distinguish being unready due
// to load from being unready while waiting on setup. // to load from being unready while waiting on setup.
// Every network_setup state handler must provide a result // Every setup variant handler must provide a result
let result; let result;
self.network_setup = match self.take_setup() { self.setup = match self.take_setup() {
Setup::AwaitingNetwork { Setup::Pending { mut setup } => match setup.try_recv() {
mut network_setup, Ok(setup_data) => {
block_verifier, let InboundSetupData {
} => match network_setup.try_recv() { address_book,
Ok((outbound, address_book, mempool)) => { block_download_peer_set,
block_verifier,
mempool,
state,
} = setup_data;
let block_downloads = Box::pin(BlockDownloads::new( let block_downloads = Box::pin(BlockDownloads::new(
Timeout::new(outbound.clone(), BLOCK_DOWNLOAD_TIMEOUT), Timeout::new(block_download_peer_set.clone(), BLOCK_DOWNLOAD_TIMEOUT),
Timeout::new(block_verifier, BLOCK_VERIFY_TIMEOUT), Timeout::new(block_verifier, BLOCK_VERIFY_TIMEOUT),
self.state.clone(), state.clone(),
)); ));
result = Ok(()); result = Ok(());
@ -193,27 +212,25 @@ impl Service<zn::Request> for Inbound {
address_book, address_book,
block_downloads, block_downloads,
mempool, mempool,
state,
} }
} }
Err(TryRecvError::Empty) => { Err(TryRecvError::Empty) => {
// There's no setup data yet, so keep waiting for it // There's no setup data yet, so keep waiting for it
result = Ok(()); result = Ok(());
Setup::AwaitingNetwork { Setup::Pending { setup }
network_setup,
block_verifier,
}
} }
Err(error @ TryRecvError::Closed) => { Err(error @ TryRecvError::Closed) => {
// Mark the service as failed, because network setup failed // Mark the service as failed, because setup failed
error!(?error, "inbound network setup failed"); error!(?error, "inbound setup failed");
let error: SharedRecvError = error.into(); let error: SharedRecvError = error.into();
result = Err(error.clone().into()); result = Err(error.clone().into());
Setup::FailedRecv { error } Setup::FailedRecv { error }
} }
}, },
// Make sure previous network setups were left in a valid state // Make sure previous setups were left in a valid state
Setup::FailedInit => unreachable!("incomplete previous Inbound initialization"), Setup::FailedInit => unreachable!("incomplete previous Inbound initialization"),
// If network setup failed, report service failure // If setup failed, report service failure
Setup::FailedRecv { error } => { Setup::FailedRecv { error } => {
result = Err(error.clone().into()); result = Err(error.clone().into());
Setup::FailedRecv { error } Setup::FailedRecv { error }
@ -223,6 +240,7 @@ impl Service<zn::Request> for Inbound {
address_book, address_book,
mut block_downloads, mut block_downloads,
mempool, mempool,
state,
} => { } => {
while let Poll::Ready(Some(_)) = block_downloads.as_mut().poll_next(cx) {} while let Poll::Ready(Some(_)) = block_downloads.as_mut().poll_next(cx) {}
@ -231,12 +249,13 @@ impl Service<zn::Request> for Inbound {
address_book, address_book,
block_downloads, block_downloads,
mempool, mempool,
state,
} }
} }
}; };
// Make sure we're leaving the network setup in a valid state // Make sure we're leaving the setup in a valid state
if matches!(self.network_setup, Setup::FailedInit) { if matches!(self.setup, Setup::FailedInit) {
unreachable!("incomplete Inbound initialization after poll_ready state handling"); unreachable!("incomplete Inbound initialization after poll_ready state handling");
} }
@ -258,38 +277,46 @@ impl Service<zn::Request> for Inbound {
/// and will cause callers to disconnect from the remote peer. /// and will cause callers to disconnect from the remote peer.
#[instrument(name = "inbound", skip(self, req))] #[instrument(name = "inbound", skip(self, req))]
fn call(&mut self, req: zn::Request) -> Self::Future { fn call(&mut self, req: zn::Request) -> Self::Future {
let (address_book, block_downloads, mempool, state) = match &mut self.setup {
Setup::Initialized {
address_book,
block_downloads,
mempool,
state,
} => (address_book, block_downloads, mempool, state),
_ => {
debug!("ignoring request from remote peer during setup");
return async { Ok(zn::Response::Nil) }.boxed();
}
};
match req { match req {
zn::Request::Peers => { zn::Request::Peers => {
if let Setup::Initialized { address_book, .. } = &self.network_setup { // # Security
// # Security //
// // We truncate the list to not reveal our entire peer set in one call.
// We truncate the list to not reveal our entire peer set in one call. // But we don't monitor repeated requests and the results are shuffled,
// But we don't monitor repeated requests and the results are shuffled, // a crawler could just send repeated queries and get the full list.
// a crawler could just send repeated queries and get the full list. //
// // # Correctness
// # Correctness //
// // Briefly hold the address book threaded mutex while
// Briefly hold the address book threaded mutex while // cloning the address book. Then sanitize after releasing
// cloning the address book. Then sanitize after releasing // the lock.
// the lock. let peers = address_book.lock().unwrap().clone();
let peers = address_book.lock().unwrap().clone();
// Send a sanitized response // Send a sanitized response
let mut peers = peers.sanitized(); let mut peers = peers.sanitized();
// Truncate the list // Truncate the list
let truncate_at = MAX_ADDRS_IN_MESSAGE let truncate_at = MAX_ADDRS_IN_MESSAGE
.min((peers.len() as f64 * FRAC_OF_AVAILABLE_ADDRESS).ceil() as usize); .min((peers.len() as f64 * FRAC_OF_AVAILABLE_ADDRESS).ceil() as usize);
peers.truncate(truncate_at); peers.truncate(truncate_at);
if !peers.is_empty() { if !peers.is_empty() {
async { Ok(zn::Response::Peers(peers)) }.boxed() async { Ok(zn::Response::Peers(peers)) }.boxed()
} else {
info!("ignoring `Peers` request from remote peer because our address book is empty");
async { Ok(zn::Response::Nil) }.boxed()
}
} else { } else {
info!("ignoring `Peers` request from remote peer during network setup"); info!("ignoring `Peers` request from remote peer because our address book is empty");
async { Ok(zn::Response::Nil) }.boxed() async { Ok(zn::Response::Nil) }.boxed()
} }
} }
@ -306,7 +333,7 @@ impl Service<zn::Request> for Inbound {
hashes hashes
.into_iter() .into_iter()
.map(|hash| zs::Request::Block(hash.into())) .map(|hash| zs::Request::Block(hash.into()))
.map(|request| self.state.clone().oneshot(request)) .map(|request| state.clone().oneshot(request))
.collect::<futures::stream::FuturesOrdered<_>>() .collect::<futures::stream::FuturesOrdered<_>>()
.try_filter_map(|response| async move { .try_filter_map(|response| async move {
Ok(match response { Ok(match response {
@ -329,102 +356,60 @@ impl Service<zn::Request> for Inbound {
.boxed() .boxed()
} }
zn::Request::TransactionsById(transactions) => { zn::Request::TransactionsById(transactions) => {
if let Setup::Initialized { mempool, .. } = &mut self.network_setup { let request = mempool::Request::TransactionsById(transactions);
let request = mempool::Request::TransactionsById(transactions); mempool.clone().oneshot(request).map_ok(|resp| match resp {
mempool.clone().oneshot(request).map_ok(|resp| match resp { mempool::Response::Transactions(transactions) if transactions.is_empty() => zn::Response::Nil,
mempool::Response::Transactions(transactions) if transactions.is_empty() => zn::Response::Nil, mempool::Response::Transactions(transactions) => zn::Response::Transactions(transactions),
mempool::Response::Transactions(transactions) => zn::Response::Transactions(transactions), _ => unreachable!("Mempool component should always respond to a `TransactionsById` request with a `Transactions` response"),
_ => unreachable!("Mempool component should always respond to a `TransactionsById` request with a `Transactions` response"), })
})
.boxed() .boxed()
} else {
info!(
transaction_hash_count = ?transactions.len(),
"ignoring `TransactionsById` request from remote peer during network setup"
);
async { Ok(zn::Response::Nil) }.boxed()
}
} }
zn::Request::FindBlocks { known_blocks, stop } => { zn::Request::FindBlocks { known_blocks, stop } => {
let request = zs::Request::FindBlockHashes { known_blocks, stop }; let request = zs::Request::FindBlockHashes { known_blocks, stop };
self.state.clone().oneshot(request).map_ok(|resp| match resp { state.clone().oneshot(request).map_ok(|resp| match resp {
zs::Response::BlockHashes(hashes) if hashes.is_empty() => zn::Response::Nil, zs::Response::BlockHashes(hashes) if hashes.is_empty() => zn::Response::Nil,
zs::Response::BlockHashes(hashes) => zn::Response::BlockHashes(hashes), zs::Response::BlockHashes(hashes) => zn::Response::BlockHashes(hashes),
_ => unreachable!("zebra-state should always respond to a `FindBlockHashes` request with a `BlockHashes` response"), _ => unreachable!("zebra-state should always respond to a `FindBlockHashes` request with a `BlockHashes` response"),
}) })
.boxed() .boxed()
} }
zn::Request::FindHeaders { known_blocks, stop } => { zn::Request::FindHeaders { known_blocks, stop } => {
let request = zs::Request::FindBlockHeaders { known_blocks, stop }; let request = zs::Request::FindBlockHeaders { known_blocks, stop };
self.state.clone().oneshot(request).map_ok(|resp| match resp { state.clone().oneshot(request).map_ok(|resp| match resp {
zs::Response::BlockHeaders(headers) if headers.is_empty() => zn::Response::Nil, zs::Response::BlockHeaders(headers) if headers.is_empty() => zn::Response::Nil,
zs::Response::BlockHeaders(headers) => zn::Response::BlockHeaders(headers), zs::Response::BlockHeaders(headers) => zn::Response::BlockHeaders(headers),
_ => unreachable!("zebra-state should always respond to a `FindBlockHeaders` request with a `BlockHeaders` response"), _ => unreachable!("zebra-state should always respond to a `FindBlockHeaders` request with a `BlockHeaders` response"),
}) })
.boxed() .boxed()
} }
zn::Request::PushTransaction(transaction) => { zn::Request::PushTransaction(transaction) => {
if let Setup::Initialized { mempool, .. } = &mut self.network_setup { mempool
mempool .clone()
.clone() .oneshot(mempool::Request::Queue(vec![transaction.into()]))
.oneshot(mempool::Request::Queue(vec![transaction.into()])) // The response just indicates if processing was queued or not; ignore it
// The response just indicates if processing was queued or not; ignore it .map_ok(|_resp| zn::Response::Nil)
.map_ok(|_resp| zn::Response::Nil) .boxed()
.boxed()
} else {
info!(
?transaction.id,
"ignoring `PushTransaction` request from remote peer during network setup"
);
async { Ok(zn::Response::Nil) }.boxed()
}
} }
zn::Request::AdvertiseTransactionIds(transactions) => { zn::Request::AdvertiseTransactionIds(transactions) => {
if let Setup::Initialized { mempool, .. } = &mut self.network_setup { let transactions = transactions.into_iter().map(Into::into).collect();
let transactions = transactions.into_iter().map(Into::into).collect(); mempool
mempool .clone()
.clone() .oneshot(mempool::Request::Queue(transactions))
.oneshot(mempool::Request::Queue(transactions)) // The response just indicates if processing was queued or not; ignore it
// The response just indicates if processing was queued or not; ignore it .map_ok(|_resp| zn::Response::Nil)
.map_ok(|_resp| zn::Response::Nil) .boxed()
.boxed()
} else {
// Peers send a lot of these when we first connect to them.
debug!(
"ignoring `AdvertiseTransactionIds` request from remote peer during network setup"
);
async { Ok(zn::Response::Nil) }.boxed()
}
} }
zn::Request::AdvertiseBlock(hash) => { zn::Request::AdvertiseBlock(hash) => {
if let Setup::Initialized { block_downloads.download_and_verify(hash);
block_downloads, ..
} = &mut self.network_setup
{
block_downloads.download_and_verify(hash);
} else {
// Peers send a lot of these when we first connect to them.
debug!(
?hash,
"ignoring `AdvertiseBlock` request from remote peer during network setup"
);
}
async { Ok(zn::Response::Nil) }.boxed() async { Ok(zn::Response::Nil) }.boxed()
} }
zn::Request::MempoolTransactionIds => { zn::Request::MempoolTransactionIds => {
if let Setup::Initialized { mempool, .. } = &mut self.network_setup { mempool.clone().oneshot(mempool::Request::TransactionIds).map_ok(|resp| match resp {
mempool.clone().oneshot(mempool::Request::TransactionIds).map_ok(|resp| match resp { mempool::Response::TransactionIds(transaction_ids) if transaction_ids.is_empty() => zn::Response::Nil,
mempool::Response::TransactionIds(transaction_ids) if transaction_ids.is_empty() => zn::Response::Nil, mempool::Response::TransactionIds(transaction_ids) => zn::Response::TransactionIds(transaction_ids.into_iter().collect()),
mempool::Response::TransactionIds(transaction_ids) => zn::Response::TransactionIds(transaction_ids.into_iter().collect()), _ => unreachable!("Mempool component should always respond to a `TransactionIds` request with a `TransactionIds` response"),
_ => unreachable!("Mempool component should always respond to a `TransactionIds` request with a `TransactionIds` response"), })
})
.boxed() .boxed()
} else {
info!(
"ignoring `MempoolTransactionIds` request from remote peer during network setup"
);
async { Ok(zn::Response::Nil) }.boxed()
}
} }
zn::Request::Ping(_) => { zn::Request::Ping(_) => {
unreachable!("ping requests are handled internally"); unreachable!("ping requests are handled internally");

View File

@ -24,6 +24,7 @@ use zebra_test::mock_service::{MockService, PanicAssertion};
use crate::{ use crate::{
components::{ components::{
inbound::InboundSetupData,
mempool::{self, gossip_mempool_transaction_id, unmined_transactions_in_blocks, Mempool}, mempool::{self, gossip_mempool_transaction_id, unmined_transactions_in_blocks, Mempool},
sync::{self, BlockGossipError, SyncStatus}, sync::{self, BlockGossipError, SyncStatus},
}, },
@ -661,15 +662,18 @@ async fn setup(
let inbound_service = ServiceBuilder::new() let inbound_service = ServiceBuilder::new()
.load_shed() .load_shed()
.service(super::Inbound::new( .service(super::Inbound::new(setup_rx));
setup_rx,
state_service.clone(),
block_verifier.clone(),
));
let inbound_service = BoxService::new(inbound_service); let inbound_service = BoxService::new(inbound_service);
let inbound_service = ServiceBuilder::new().buffer(1).service(inbound_service); let inbound_service = ServiceBuilder::new().buffer(1).service(inbound_service);
let r = setup_tx.send((buffered_peer_set, address_book, mempool_service.clone())); let setup_data = InboundSetupData {
address_book,
block_download_peer_set: buffered_peer_set,
block_verifier,
mempool: mempool_service.clone(),
state: state_service.clone(),
};
let r = setup_tx.send(setup_data);
// We can't expect or unwrap because the returned Result does not implement Debug // We can't expect or unwrap because the returned Result does not implement Debug
assert!(r.is_ok(), "unexpected setup channel send failure"); assert!(r.is_ok(), "unexpected setup channel send failure");

View File

@ -46,16 +46,22 @@ pub(crate) trait RuntimeRun {
impl RuntimeRun for Runtime { impl RuntimeRun for Runtime {
fn run(&mut self, fut: impl Future<Output = Result<(), Report>>) { fn run(&mut self, fut: impl Future<Output = Result<(), Report>>) {
let result = self.block_on(async move { let result = self.block_on(async move {
// If the run task and shutdown are both ready, select! chooses // Always poll the shutdown future first.
// one of them at random. //
// Otherwise, a busy Zebra instance could starve the shutdown future,
// and delay shutting down.
tokio::select! { tokio::select! {
result = fut => result, biased;
_ = shutdown() => Ok(()), _ = shutdown() => Ok(()),
result = fut => result,
} }
}); });
match result { match result {
Ok(()) => {} Ok(()) => {
// Don't wait for the runtime to shut down all the tasks.
app_writer().shutdown(Shutdown::Graceful);
}
Err(e) => { Err(e) => {
eprintln!("Error: {:?}", e); eprintln!("Error: {:?}", e);
app_writer().shutdown(Shutdown::Forced); app_writer().shutdown(Shutdown::Forced);