feat(log): log the state tip height as part of sync progress logs (#3437)
* feat(log): log the state tip height as part of sync progress logs * fix(log): downgrade some verbose state logs to debug * feat(log): log successful gossiped block verification at info level These logs help us diagnose slow progress near the tip. There won't be very many of these logs, because they only happen near the tip. * fix(log): spawn top-level tasks within the global Zebra tracing span * fix(log): spawn blocking top-level tasks within the global Zebra tracing span Co-authored-by: teor <teor@riseup.net>
This commit is contained in:
parent
ea01b92848
commit
e5b5ea5889
|
@ -24,7 +24,7 @@ use futures::{FutureExt, TryFutureExt};
|
||||||
use thiserror::Error;
|
use thiserror::Error;
|
||||||
use tokio::task::{spawn_blocking, JoinHandle};
|
use tokio::task::{spawn_blocking, JoinHandle};
|
||||||
use tower::{buffer::Buffer, util::BoxService, Service, ServiceExt};
|
use tower::{buffer::Buffer, util::BoxService, Service, ServiceExt};
|
||||||
use tracing::instrument;
|
use tracing::{instrument, Span};
|
||||||
|
|
||||||
use zebra_chain::{
|
use zebra_chain::{
|
||||||
block::{self, Block},
|
block::{self, Block},
|
||||||
|
@ -215,13 +215,16 @@ where
|
||||||
|
|
||||||
// The parameter download thread must be launched before initializing any verifiers.
|
// The parameter download thread must be launched before initializing any verifiers.
|
||||||
// Otherwise, the download might happen on the startup thread.
|
// Otherwise, the download might happen on the startup thread.
|
||||||
|
let span = Span::current();
|
||||||
let groth16_download_handle = spawn_blocking(move || {
|
let groth16_download_handle = spawn_blocking(move || {
|
||||||
if !debug_skip_parameter_preload {
|
span.in_scope(|| {
|
||||||
// The lazy static initializer does the download, if needed,
|
if !debug_skip_parameter_preload {
|
||||||
// and the file hash checks.
|
// The lazy static initializer does the download, if needed,
|
||||||
lazy_static::initialize(&crate::groth16::GROTH16_PARAMETERS);
|
// and the file hash checks.
|
||||||
tracing::info!("Groth16 pre-download and check task finished");
|
lazy_static::initialize(&crate::groth16::GROTH16_PARAMETERS);
|
||||||
}
|
tracing::info!("Groth16 pre-download and check task finished");
|
||||||
|
}
|
||||||
|
})
|
||||||
});
|
});
|
||||||
|
|
||||||
// transaction verification
|
// transaction verification
|
||||||
|
|
|
@ -7,6 +7,7 @@ use tokio::{
|
||||||
sync::{mpsc, watch},
|
sync::{mpsc, watch},
|
||||||
task::JoinHandle,
|
task::JoinHandle,
|
||||||
};
|
};
|
||||||
|
use tracing::Span;
|
||||||
|
|
||||||
use crate::{
|
use crate::{
|
||||||
address_book::AddressMetrics, meta_addr::MetaAddrChange, AddressBook, BoxError, Config,
|
address_book::AddressMetrics, meta_addr::MetaAddrChange, AddressBook, BoxError, Config,
|
||||||
|
@ -54,25 +55,28 @@ impl AddressBookUpdater {
|
||||||
let address_book = Arc::new(std::sync::Mutex::new(address_book));
|
let address_book = Arc::new(std::sync::Mutex::new(address_book));
|
||||||
|
|
||||||
let worker_address_book = address_book.clone();
|
let worker_address_book = address_book.clone();
|
||||||
|
let span = Span::current();
|
||||||
let worker = move || {
|
let worker = move || {
|
||||||
info!("starting the address book updater");
|
span.in_scope(|| {
|
||||||
|
info!("starting the address book updater");
|
||||||
|
|
||||||
while let Some(event) = worker_rx.blocking_recv() {
|
while let Some(event) = worker_rx.blocking_recv() {
|
||||||
trace!(?event, "got address book change");
|
trace!(?event, "got address book change");
|
||||||
|
|
||||||
// # Correctness
|
// # Correctness
|
||||||
//
|
//
|
||||||
// Briefly hold the address book threaded mutex, to update the
|
// Briefly hold the address book threaded mutex, to update the
|
||||||
// state for a single address.
|
// state for a single address.
|
||||||
worker_address_book
|
worker_address_book
|
||||||
.lock()
|
.lock()
|
||||||
.expect("mutex should be unpoisoned")
|
.expect("mutex should be unpoisoned")
|
||||||
.update(event);
|
.update(event);
|
||||||
}
|
}
|
||||||
|
|
||||||
let error = Err(AllAddressBookUpdaterSendersClosed.into());
|
let error = Err(AllAddressBookUpdaterSendersClosed.into());
|
||||||
info!(?error, "stopping address book updater");
|
info!(?error, "stopping address book updater");
|
||||||
error
|
error
|
||||||
|
})
|
||||||
};
|
};
|
||||||
|
|
||||||
// Correctness: spawn address book accesses on a blocking thread,
|
// Correctness: spawn address book accesses on a blocking thread,
|
||||||
|
|
|
@ -682,7 +682,7 @@ impl Service<Request> for StateService {
|
||||||
.checked_sub(new_len)
|
.checked_sub(new_len)
|
||||||
.expect("prune does not add any utxo requests");
|
.expect("prune does not add any utxo requests");
|
||||||
if prune_count > 0 {
|
if prune_count > 0 {
|
||||||
tracing::info!(
|
tracing::debug!(
|
||||||
?old_len,
|
?old_len,
|
||||||
?new_len,
|
?new_len,
|
||||||
?prune_count,
|
?prune_count,
|
||||||
|
|
|
@ -18,7 +18,7 @@ pub fn block_locator_heights(tip_height: block::Height) -> Vec<block::Height> {
|
||||||
.map(block::Height);
|
.map(block::Height);
|
||||||
|
|
||||||
let locators = locators.collect();
|
let locators = locators.collect();
|
||||||
tracing::info!(
|
tracing::debug!(
|
||||||
?tip_height,
|
?tip_height,
|
||||||
?min_locator_height,
|
?min_locator_height,
|
||||||
?locators,
|
?locators,
|
||||||
|
|
|
@ -57,6 +57,7 @@ use color_eyre::eyre::{eyre, Report};
|
||||||
use futures::FutureExt;
|
use futures::FutureExt;
|
||||||
use tokio::{pin, select, sync::oneshot};
|
use tokio::{pin, select, sync::oneshot};
|
||||||
use tower::{builder::ServiceBuilder, util::BoxService};
|
use tower::{builder::ServiceBuilder, util::BoxService};
|
||||||
|
use tracing_futures::Instrument;
|
||||||
|
|
||||||
use crate::{
|
use crate::{
|
||||||
components::{
|
components::{
|
||||||
|
@ -145,13 +146,16 @@ impl StartCmd {
|
||||||
.send(setup_data)
|
.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_task_handle = tokio::spawn(syncer.sync());
|
let syncer_task_handle = tokio::spawn(syncer.sync().in_current_span());
|
||||||
|
|
||||||
let mut block_gossip_task_handle = tokio::spawn(sync::gossip_best_tip_block_hashes(
|
let mut block_gossip_task_handle = tokio::spawn(
|
||||||
sync_status.clone(),
|
sync::gossip_best_tip_block_hashes(
|
||||||
chain_tip_change.clone(),
|
sync_status.clone(),
|
||||||
peer_set.clone(),
|
chain_tip_change.clone(),
|
||||||
));
|
peer_set.clone(),
|
||||||
|
)
|
||||||
|
.in_current_span(),
|
||||||
|
);
|
||||||
|
|
||||||
let mempool_crawler_task_handle = mempool::Crawler::spawn(
|
let mempool_crawler_task_handle = mempool::Crawler::spawn(
|
||||||
&config.mempool,
|
&config.mempool,
|
||||||
|
@ -163,10 +167,10 @@ impl StartCmd {
|
||||||
|
|
||||||
let mempool_queue_checker_task_handle = mempool::QueueChecker::spawn(mempool);
|
let mempool_queue_checker_task_handle = mempool::QueueChecker::spawn(mempool);
|
||||||
|
|
||||||
let tx_gossip_task_handle = tokio::spawn(mempool::gossip_mempool_transaction_id(
|
let tx_gossip_task_handle = tokio::spawn(
|
||||||
mempool_transaction_receiver,
|
mempool::gossip_mempool_transaction_id(mempool_transaction_receiver, peer_set)
|
||||||
peer_set,
|
.in_current_span(),
|
||||||
));
|
);
|
||||||
|
|
||||||
info!("spawned initial Zebra tasks");
|
info!("spawned initial Zebra tasks");
|
||||||
|
|
||||||
|
|
|
@ -275,45 +275,49 @@ where
|
||||||
})
|
})
|
||||||
.unwrap_or(block::Height(0));
|
.unwrap_or(block::Height(0));
|
||||||
|
|
||||||
if let Some(block_height) = block.coinbase_height() {
|
let block_height = block.coinbase_height().ok_or_else(|| {
|
||||||
if block_height > max_lookahead_height {
|
|
||||||
debug!(
|
|
||||||
?hash,
|
|
||||||
?block_height,
|
|
||||||
?tip_height,
|
|
||||||
?max_lookahead_height,
|
|
||||||
lookahead_limit = ?MAX_INBOUND_CONCURRENCY,
|
|
||||||
"gossiped block height too far ahead of the tip: dropped downloaded block"
|
|
||||||
);
|
|
||||||
metrics::counter!("gossip.max.height.limit.dropped.block.count", 1);
|
|
||||||
|
|
||||||
Err("gossiped block height too far ahead")?;
|
|
||||||
} else if block_height < min_accepted_height {
|
|
||||||
debug!(
|
|
||||||
?hash,
|
|
||||||
?block_height,
|
|
||||||
?tip_height,
|
|
||||||
?min_accepted_height,
|
|
||||||
behind_tip_limit = ?zs::MAX_BLOCK_REORG_HEIGHT,
|
|
||||||
"gossiped block height behind the finalized tip: dropped downloaded block"
|
|
||||||
);
|
|
||||||
metrics::counter!("gossip.min.height.limit.dropped.block.count", 1);
|
|
||||||
|
|
||||||
Err("gossiped block height behind the finalized tip")?;
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
debug!(
|
debug!(
|
||||||
?hash,
|
?hash,
|
||||||
"gossiped block with no height: dropped downloaded block"
|
"gossiped block with no height: dropped downloaded block"
|
||||||
);
|
);
|
||||||
metrics::counter!("gossip.no.height.dropped.block.count", 1);
|
metrics::counter!("gossip.no.height.dropped.block.count", 1);
|
||||||
|
|
||||||
Err("gossiped block with no height")?;
|
BoxError::from("gossiped block with no height")
|
||||||
|
})?;
|
||||||
|
|
||||||
|
if block_height > max_lookahead_height {
|
||||||
|
debug!(
|
||||||
|
?hash,
|
||||||
|
?block_height,
|
||||||
|
?tip_height,
|
||||||
|
?max_lookahead_height,
|
||||||
|
lookahead_limit = ?MAX_INBOUND_CONCURRENCY,
|
||||||
|
"gossiped block height too far ahead of the tip: dropped downloaded block"
|
||||||
|
);
|
||||||
|
metrics::counter!("gossip.max.height.limit.dropped.block.count", 1);
|
||||||
|
|
||||||
|
Err("gossiped block height too far ahead")?;
|
||||||
|
} else if block_height < min_accepted_height {
|
||||||
|
debug!(
|
||||||
|
?hash,
|
||||||
|
?block_height,
|
||||||
|
?tip_height,
|
||||||
|
?min_accepted_height,
|
||||||
|
behind_tip_limit = ?zs::MAX_BLOCK_REORG_HEIGHT,
|
||||||
|
"gossiped block height behind the finalized tip: dropped downloaded block"
|
||||||
|
);
|
||||||
|
metrics::counter!("gossip.min.height.limit.dropped.block.count", 1);
|
||||||
|
|
||||||
|
Err("gossiped block height behind the finalized tip")?;
|
||||||
}
|
}
|
||||||
|
|
||||||
verifier.oneshot(block).await
|
verifier
|
||||||
|
.oneshot(block)
|
||||||
|
.await
|
||||||
|
.map(|hash| (hash, block_height))
|
||||||
}
|
}
|
||||||
.map_ok(|hash| {
|
.map_ok(|(hash, height)| {
|
||||||
|
info!(?height, "downloaded and verified gossiped block");
|
||||||
metrics::counter!("gossip.verified.block.count", 1);
|
metrics::counter!("gossip.verified.block.count", 1);
|
||||||
hash
|
hash
|
||||||
})
|
})
|
||||||
|
|
|
@ -52,6 +52,7 @@ use std::{collections::HashSet, time::Duration};
|
||||||
use futures::{future, pin_mut, stream::FuturesUnordered, StreamExt};
|
use futures::{future, pin_mut, stream::FuturesUnordered, StreamExt};
|
||||||
use tokio::{sync::watch, task::JoinHandle, time::sleep};
|
use tokio::{sync::watch, task::JoinHandle, time::sleep};
|
||||||
use tower::{timeout::Timeout, BoxError, Service, ServiceExt};
|
use tower::{timeout::Timeout, BoxError, Service, ServiceExt};
|
||||||
|
use tracing_futures::Instrument;
|
||||||
|
|
||||||
use zebra_chain::{block::Height, transaction::UnminedTxId};
|
use zebra_chain::{block::Height, transaction::UnminedTxId};
|
||||||
use zebra_network as zn;
|
use zebra_network as zn;
|
||||||
|
@ -129,7 +130,7 @@ where
|
||||||
debug_enable_at_height: config.debug_enable_at_height.map(Height),
|
debug_enable_at_height: config.debug_enable_at_height.map(Height),
|
||||||
};
|
};
|
||||||
|
|
||||||
tokio::spawn(crawler.run())
|
tokio::spawn(crawler.run().in_current_span())
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Waits until the mempool crawler is enabled by a debug config option.
|
/// Waits until the mempool crawler is enabled by a debug config option.
|
||||||
|
|
|
@ -15,6 +15,7 @@ use std::time::Duration;
|
||||||
|
|
||||||
use tokio::{task::JoinHandle, time::sleep};
|
use tokio::{task::JoinHandle, time::sleep};
|
||||||
use tower::{BoxError, Service, ServiceExt};
|
use tower::{BoxError, Service, ServiceExt};
|
||||||
|
use tracing_futures::Instrument;
|
||||||
|
|
||||||
use crate::components::mempool;
|
use crate::components::mempool;
|
||||||
|
|
||||||
|
@ -45,7 +46,7 @@ where
|
||||||
pub fn spawn(mempool: Mempool) -> JoinHandle<Result<(), BoxError>> {
|
pub fn spawn(mempool: Mempool) -> JoinHandle<Result<(), BoxError>> {
|
||||||
let queue_checker = QueueChecker { mempool };
|
let queue_checker = QueueChecker { mempool };
|
||||||
|
|
||||||
tokio::spawn(queue_checker.run())
|
tokio::spawn(queue_checker.run().in_current_span())
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Periodically check if the mempool has newly verified transactions.
|
/// Periodically check if the mempool has newly verified transactions.
|
||||||
|
|
|
@ -236,6 +236,9 @@ where
|
||||||
/// The cached block chain state.
|
/// The cached block chain state.
|
||||||
state: ZS,
|
state: ZS,
|
||||||
|
|
||||||
|
/// Allows efficient access to the best tip of the blockchain.
|
||||||
|
latest_chain_tip: ZSTip,
|
||||||
|
|
||||||
// Internal sync state
|
// Internal sync state
|
||||||
/// The tips that the syncer is currently following.
|
/// The tips that the syncer is currently following.
|
||||||
prospective_tips: HashSet<CheckedTip>,
|
prospective_tips: HashSet<CheckedTip>,
|
||||||
|
@ -331,10 +334,11 @@ where
|
||||||
downloads: Box::pin(Downloads::new(
|
downloads: Box::pin(Downloads::new(
|
||||||
block_network,
|
block_network,
|
||||||
verifier,
|
verifier,
|
||||||
latest_chain_tip,
|
latest_chain_tip.clone(),
|
||||||
config.sync.lookahead_limit,
|
config.sync.lookahead_limit,
|
||||||
)),
|
)),
|
||||||
state,
|
state,
|
||||||
|
latest_chain_tip,
|
||||||
prospective_tips: HashSet::new(),
|
prospective_tips: HashSet::new(),
|
||||||
recent_syncs,
|
recent_syncs,
|
||||||
};
|
};
|
||||||
|
@ -354,7 +358,11 @@ where
|
||||||
|
|
||||||
'sync: loop {
|
'sync: loop {
|
||||||
if started_once {
|
if started_once {
|
||||||
info!(timeout = ?SYNC_RESTART_DELAY, "waiting to restart sync");
|
info!(
|
||||||
|
timeout = ?SYNC_RESTART_DELAY,
|
||||||
|
state_tip = ?self.latest_chain_tip.best_tip_height(),
|
||||||
|
"waiting to restart sync"
|
||||||
|
);
|
||||||
self.prospective_tips = HashSet::new();
|
self.prospective_tips = HashSet::new();
|
||||||
self.downloads.cancel_all();
|
self.downloads.cancel_all();
|
||||||
self.update_metrics();
|
self.update_metrics();
|
||||||
|
@ -363,7 +371,10 @@ where
|
||||||
started_once = true;
|
started_once = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
info!("starting sync, obtaining new tips");
|
info!(
|
||||||
|
state_tip = ?self.latest_chain_tip.best_tip_height(),
|
||||||
|
"starting sync, obtaining new tips"
|
||||||
|
);
|
||||||
if let Err(e) = self.obtain_tips().await {
|
if let Err(e) = self.obtain_tips().await {
|
||||||
warn!(?e, "error obtaining tips");
|
warn!(?e, "error obtaining tips");
|
||||||
continue 'sync;
|
continue 'sync;
|
||||||
|
@ -403,6 +414,7 @@ where
|
||||||
tips.len = self.prospective_tips.len(),
|
tips.len = self.prospective_tips.len(),
|
||||||
in_flight = self.downloads.in_flight(),
|
in_flight = self.downloads.in_flight(),
|
||||||
lookahead_limit = self.lookahead_limit,
|
lookahead_limit = self.lookahead_limit,
|
||||||
|
state_tip = ?self.latest_chain_tip.best_tip_height(),
|
||||||
"waiting for pending blocks",
|
"waiting for pending blocks",
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -425,6 +437,7 @@ where
|
||||||
tips.len = self.prospective_tips.len(),
|
tips.len = self.prospective_tips.len(),
|
||||||
in_flight = self.downloads.in_flight(),
|
in_flight = self.downloads.in_flight(),
|
||||||
lookahead_limit = self.lookahead_limit,
|
lookahead_limit = self.lookahead_limit,
|
||||||
|
state_tip = ?self.latest_chain_tip.best_tip_height(),
|
||||||
"extending tips",
|
"extending tips",
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue