zebra/zebrad/src/components/sync.rs

892 lines
36 KiB
Rust
Raw Normal View History

//! The syncer downloads and verifies large numbers of blocks from peers to Zebra.
//!
//! It is used when Zebra is a long way behind the current chain tip.
Use `MockedClientHandle` in other tests (#3241) * Move `MockedClientHandle` to `peer` module It's more closely related to a `Client` than the `PeerSet`, and this prepares it to be used by other tests. * Rename `MockedClientHandle` to `ClientTestHarness` Reduce confusion, and clarify that the client is not mocked. Co-authored-by: teor <teor@riseup.net> * Add clarification to `mock_peers` documentation Explicitly say how the generated data is returned. * Rename method to `wants_connection_heartbeats` The `Client` service only represents one direction of a connection, so `is_connected` is not the exact term. Co-authored-by: teor <teor@riseup.net> * Mock `Client` instead of `LoadTrackedClient` Move where the conversion from mocked `Client` to mocked `LoadTrackedClient` in order to make the test helper more easily used by other tests. * Use `ClientTestHarness` in `initialize` tests Replace the boilerplate code to create a fake `Client` instance with usages of the `ClientTestHarness` constructor. * Allow receiving requests from `Client` instance Create a helper type to wrap the result, to make it easier to assert on specific events after trying to receive a request. * Allow inspecting the current error in the slot Share the `ErrorSlot` between the `Client` and the handle, so that the handle can be used to inspect the contents of the `ErrorSlot`. * Allow placing an error into the `ErrorSlot` Assuming it is initially empty. If it already has an error, the code will panic. * Allow gracefully closing the request receiver Close the endpoint with the appropriate call to the `close()` method. * Allow dropping the request receiver endpoint Forcefully closes the endpoint. * Rename field to `client_request_receiver` Also rename the related methods to include `outbound_client_request_receiver` to make it more precise. Co-authored-by: teor <teor@riseup.net> * Allow dropping the heartbeat shutdown receiver Allows the `Client` to detect that the channel has been closed. * Rename fn. to `drop_heartbeat_shutdown_receiver` Make it clear that it affects the heartbeat task. Co-authored-by: teor <teor@riseup.net> * Move `NowOrLater` into a new `now-or-later` crate Make it easily accessible to other crates. * Add `IsReady` extension trait for `Service` Simplifies checking if a service is immediately ready to be called. * Add extension method to check for readiness error Checks if the `Service` isn't immediately ready because a call to `ready` immediately returns an error. * Rename method to `is_failed` Avoid negated method names. Co-authored-by: teor <teor@riseup.net> * Add a `IsReady::is_pending` extension method Checks if a `Service` is not ready to be called. * Use `ClientTestHarness` in `Client` test vectors Reduce repeated code and try to improve readability. * Create a new `ClientTestHarnessBuilder` type A builder to create test `Client` instances using mock data which can be tracked and manipulated through a `ClientTestHarness`. * Allow configuring the `Client`'s mocked version Add a `with_version` builder method. * Use `ClientTestHarnessBuilder` in `PeerVersions` Use the builder to set the peer version, so that the `version` parameter can be removed from the constructor later. * Use a default mock version where possible Reduce noise when setting up the harness for tests that don't really care about the remote peer version. * Remove `Version` parameter from the `build` method The `with_version` builder method should be used instead. * Fix some typos and outdated info in the release checklist * Add extra client tests for zero and multiple readiness checks (#3273) And document existing tests. * Replace `NowOrLater` with `futures::poll!` (#3272) * Replace NowOrLater with the futures::poll! macro in zebrad * Replace NowOrLater with the futures::poll! macro in zebra-test * Remove the now-or-later crate * remove unused imports * rustfmt Co-authored-by: teor <teor@riseup.net> Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com>
2021-12-21 12:13:26 -08:00
use std::{collections::HashSet, pin::Pin, sync::Arc, task::Poll, time::Duration};
2020-07-08 13:33:39 -07:00
use color_eyre::eyre::{eyre, Report};
use futures::stream::{FuturesUnordered, StreamExt};
use indexmap::IndexSet;
use tokio::time::sleep;
use tower::{
builder::ServiceBuilder, hedge::Hedge, limit::ConcurrencyLimit, retry::Retry, timeout::Timeout,
Service, ServiceExt,
};
2020-07-08 13:33:39 -07:00
use zebra_chain::{
block::{self, Block},
chain_tip::ChainTip,
parameters::genesis_hash,
};
use zebra_consensus::{
chain::VerifyChainError, BlockError, VerifyBlockError, VerifyCheckpointError,
};
2020-09-09 12:17:17 -07:00
use zebra_network as zn;
use zebra_state as zs;
use crate::{
Use `MockedClientHandle` in other tests (#3241) * Move `MockedClientHandle` to `peer` module It's more closely related to a `Client` than the `PeerSet`, and this prepares it to be used by other tests. * Rename `MockedClientHandle` to `ClientTestHarness` Reduce confusion, and clarify that the client is not mocked. Co-authored-by: teor <teor@riseup.net> * Add clarification to `mock_peers` documentation Explicitly say how the generated data is returned. * Rename method to `wants_connection_heartbeats` The `Client` service only represents one direction of a connection, so `is_connected` is not the exact term. Co-authored-by: teor <teor@riseup.net> * Mock `Client` instead of `LoadTrackedClient` Move where the conversion from mocked `Client` to mocked `LoadTrackedClient` in order to make the test helper more easily used by other tests. * Use `ClientTestHarness` in `initialize` tests Replace the boilerplate code to create a fake `Client` instance with usages of the `ClientTestHarness` constructor. * Allow receiving requests from `Client` instance Create a helper type to wrap the result, to make it easier to assert on specific events after trying to receive a request. * Allow inspecting the current error in the slot Share the `ErrorSlot` between the `Client` and the handle, so that the handle can be used to inspect the contents of the `ErrorSlot`. * Allow placing an error into the `ErrorSlot` Assuming it is initially empty. If it already has an error, the code will panic. * Allow gracefully closing the request receiver Close the endpoint with the appropriate call to the `close()` method. * Allow dropping the request receiver endpoint Forcefully closes the endpoint. * Rename field to `client_request_receiver` Also rename the related methods to include `outbound_client_request_receiver` to make it more precise. Co-authored-by: teor <teor@riseup.net> * Allow dropping the heartbeat shutdown receiver Allows the `Client` to detect that the channel has been closed. * Rename fn. to `drop_heartbeat_shutdown_receiver` Make it clear that it affects the heartbeat task. Co-authored-by: teor <teor@riseup.net> * Move `NowOrLater` into a new `now-or-later` crate Make it easily accessible to other crates. * Add `IsReady` extension trait for `Service` Simplifies checking if a service is immediately ready to be called. * Add extension method to check for readiness error Checks if the `Service` isn't immediately ready because a call to `ready` immediately returns an error. * Rename method to `is_failed` Avoid negated method names. Co-authored-by: teor <teor@riseup.net> * Add a `IsReady::is_pending` extension method Checks if a `Service` is not ready to be called. * Use `ClientTestHarness` in `Client` test vectors Reduce repeated code and try to improve readability. * Create a new `ClientTestHarnessBuilder` type A builder to create test `Client` instances using mock data which can be tracked and manipulated through a `ClientTestHarness`. * Allow configuring the `Client`'s mocked version Add a `with_version` builder method. * Use `ClientTestHarnessBuilder` in `PeerVersions` Use the builder to set the peer version, so that the `version` parameter can be removed from the constructor later. * Use a default mock version where possible Reduce noise when setting up the harness for tests that don't really care about the remote peer version. * Remove `Version` parameter from the `build` method The `with_version` builder method should be used instead. * Fix some typos and outdated info in the release checklist * Add extra client tests for zero and multiple readiness checks (#3273) And document existing tests. * Replace `NowOrLater` with `futures::poll!` (#3272) * Replace NowOrLater with the futures::poll! macro in zebrad * Replace NowOrLater with the futures::poll! macro in zebra-test * Remove the now-or-later crate * remove unused imports * rustfmt Co-authored-by: teor <teor@riseup.net> Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com>
2021-12-21 12:13:26 -08:00
components::sync::downloads::BlockDownloadVerifyError, config::ZebradConfig, BoxError,
};
mod downloads;
mod gossip;
mod recent_sync_lengths;
mod status;
#[cfg(test)]
mod tests;
use downloads::{AlwaysHedge, Downloads};
pub use gossip::{gossip_best_tip_block_hashes, BlockGossipError};
pub use recent_sync_lengths::RecentSyncLengths;
pub use status::SyncStatus;
/// Controls the number of peers used for each ObtainTips and ExtendTips request.
const FANOUT: usize = 3;
/// Controls how many times we will retry each block download.
///
/// Failing block downloads is important because it defends against peers who
/// feed us bad hashes. But spurious failures of valid blocks cause the syncer to
/// restart from the previous checkpoint, potentially re-downloading blocks.
///
/// We also hedge requests, so we may retry up to twice this many times. Hedged
/// retries may be concurrent, inner retries are sequential.
const BLOCK_DOWNLOAD_RETRY_LIMIT: usize = 2;
2021-01-22 02:44:24 -08:00
/// A lower bound on the user-specified lookahead limit.
///
/// Set to two checkpoint intervals, so that we're sure that the lookahead
/// limit always contains at least one complete checkpoint.
///
/// ## Security
///
/// If a malicious node is chosen for an ObtainTips or ExtendTips request, it can
/// provide up to 500 malicious block hashes. These block hashes will be
/// distributed across all available peers. Assuming there are around 50 connected
/// peers, the malicious node will receive approximately 10 of those block requests.
///
/// Malicious deserialized blocks can take up a large amount of RAM, see
/// [`super::inbound::downloads::MAX_INBOUND_CONCURRENCY`] and #1880 for details.
/// So we want to keep the lookahead limit reasonably small.
///
/// Once these malicious blocks start failing validation, the syncer will cancel all
/// the pending download and verify tasks, drop all the blocks, and start a new
/// ObtainTips with a new set of peers.
pub const MIN_LOOKAHEAD_LIMIT: usize = zebra_consensus::MAX_CHECKPOINT_HEIGHT_GAP * 2;
/// The default for the user-specified lookahead limit.
///
/// See [`MIN_LOOKAHEAD_LIMIT`] for details.
pub const DEFAULT_LOOKAHEAD_LIMIT: usize = zebra_consensus::MAX_CHECKPOINT_HEIGHT_GAP * 5;
/// The expected maximum number of hashes in an ObtainTips or ExtendTips response.
///
/// This is used to allow block heights that are slightly beyond the lookahead limit,
/// but still limit the number of blocks in the pipeline between the downloader and
/// the state.
///
/// See [`MIN_LOOKAHEAD_LIMIT`] for details.
pub const MAX_TIPS_RESPONSE_HASH_COUNT: usize = 500;
/// Controls how long we wait for a tips response to return.
///
/// ## Correctness
///
/// If this timeout is removed (or set too high), the syncer will sometimes hang.
///
/// If this timeout is set too low, the syncer will sometimes get stuck in a
/// failure loop.
pub const TIPS_RESPONSE_TIMEOUT: Duration = Duration::from_secs(6);
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
/// Controls how long we wait for a block download request to complete.
///
/// This timeout makes sure that the syncer doesn't hang when:
/// - the lookahead queue is full, and
/// - we are waiting for a request that is stuck.
/// See [`BLOCK_VERIFY_TIMEOUT`] for details.
///
/// ## Correctness
///
/// If this timeout is removed (or set too high), the syncer will sometimes hang.
///
/// If this timeout is set too low, the syncer will sometimes get stuck in a
/// failure loop.
pub(super) const BLOCK_DOWNLOAD_TIMEOUT: Duration = Duration::from_secs(15);
/// Controls how long we wait for a block verify request to complete.
///
/// This timeout makes sure that the syncer doesn't hang when:
/// - the lookahead queue is full, and
/// - all pending verifications:
/// - are waiting on a missing download request,
/// - are waiting on a download or verify request that has failed, but we have
/// deliberately ignored the error,
/// - are for blocks a long way ahead of the current tip, or
/// - are for invalid blocks which will never verify, because they depend on
/// missing blocks or transactions.
/// These conditions can happen during normal operation - they are not bugs.
///
/// This timeout also mitigates or hides the following kinds of bugs:
/// - all pending verifications:
/// - are waiting on a download or verify request that has failed, but we have
/// accidentally dropped the error,
/// - are waiting on a download request that has hung inside Zebra,
/// - are on tokio threads that are waiting for blocked operations.
///
/// ## Correctness
///
/// If this timeout is removed (or set too high), the syncer will sometimes hang.
///
/// If this timeout is set too low, the syncer will sometimes get stuck in a
/// failure loop.
pub(super) const BLOCK_VERIFY_TIMEOUT: Duration = Duration::from_secs(180);
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
/// Controls how long we wait to restart syncing after finishing a sync run.
///
/// This delay should be long enough to:
/// - allow zcashd peers to process pending requests. If the node only has a
/// few peers, we want to clear as much peer state as possible. In
/// particular, zcashd sends "next block range" hints, based on zcashd's
/// internal model of our sync progress. But we want to discard these hints,
/// so they don't get confused with ObtainTips and ExtendTips responses, and
/// - allow in-progress downloads to time out.
///
/// This delay is particularly important on instances with slow or unreliable
/// networks, and on testnet, which has a small number of slow peers.
///
/// Using a prime number makes sure that syncer fanouts don't synchronise with other crawls.
///
/// ## Correctness
///
/// If this delay is removed (or set too low), the syncer will
/// sometimes get stuck in a failure loop, due to leftover downloads from
/// previous sync runs.
const SYNC_RESTART_DELAY: Duration = Duration::from_secs(67);
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
/// Controls how long we wait to retry a failed attempt to download
/// and verify the genesis block.
///
/// This timeout gives the crawler time to find better peers.
///
/// ## Security
///
/// If this timeout is removed (or set too low), Zebra will immediately retry
/// to download and verify the genesis block from its peers. This can cause
/// a denial of service on those peers.
const GENESIS_TIMEOUT_RETRY: Duration = Duration::from_secs(5);
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
/// Helps work around defects in the bitcoin protocol by checking whether
/// the returned hashes actually extend a chain tip.
#[derive(Copy, Clone, Debug, Hash, PartialEq, Eq)]
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
struct CheckedTip {
tip: block::Hash,
expected_next: block::Hash,
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
}
pub struct ChainSync<ZN, ZS, ZV, ZSTip>
where
ZN: Service<zn::Request, Response = zn::Response, Error = BoxError>
+ Send
+ Sync
+ Clone
+ 'static,
ZN::Future: Send,
ZS: Service<zs::Request, Response = zs::Response, Error = BoxError>
+ Send
+ Sync
+ Clone
+ 'static,
ZS::Future: Send,
ZV: Service<Arc<Block>, Response = block::Hash, Error = BoxError>
+ Send
+ Sync
+ Clone
+ 'static,
ZV::Future: Send,
ZSTip: ChainTip + Clone + Send + 'static,
{
// Configuration
/// The genesis hash for the configured network
genesis_hash: block::Hash,
/// The configured lookahead limit, after applying the minimum limit.
lookahead_limit: usize,
// Services
/// A network service which is used to perform ObtainTips and ExtendTips
/// requests.
///
/// Has no retry logic, because failover is handled using fanout.
tip_network: Timeout<ZN>,
/// A service which downloads and verifies blocks, using the provided
/// network and verifier services.
downloads: Pin<
Box<
Downloads<
Hedge<ConcurrencyLimit<Retry<zn::RetryLimit, Timeout<ZN>>>, AlwaysHedge>,
Timeout<ZV>,
ZSTip,
>,
>,
>,
/// The cached block chain state.
state: ZS,
/// Allows efficient access to the best tip of the blockchain.
latest_chain_tip: ZSTip,
// Internal sync state
/// The tips that the syncer is currently following.
prospective_tips: HashSet<CheckedTip>,
/// The lengths of recent sync responses.
recent_syncs: RecentSyncLengths,
}
/// Polls the network to determine whether further blocks are available and
/// downloads them.
///
/// This component is used for initial block sync, but the `Inbound` service is
/// responsible for participating in the gossip protocols used for block
/// diffusion.
impl<ZN, ZS, ZV, ZSTip> ChainSync<ZN, ZS, ZV, ZSTip>
where
ZN: Service<zn::Request, Response = zn::Response, Error = BoxError>
+ Send
+ Sync
+ Clone
+ 'static,
ZN::Future: Send,
ZS: Service<zs::Request, Response = zs::Response, Error = BoxError>
+ Send
+ Sync
+ Clone
+ 'static,
ZS::Future: Send,
ZV: Service<Arc<Block>, Response = block::Hash, Error = BoxError>
+ Send
+ Sync
+ Clone
+ 'static,
ZV::Future: Send,
ZSTip: ChainTip + Clone + Send + 'static,
{
/// Returns a new syncer instance, using:
/// - chain: the zebra-chain `Network` to download (Mainnet or Testnet)
/// - peers: the zebra-network peers to contact for downloads
/// - verifier: the zebra-consensus verifier that checks the chain
/// - state: the zebra-state that stores the chain
/// - latest_chain_tip: the latest chain tip from `state`
///
/// Also returns a [`SyncStatus`] to check if the syncer has likely reached the chain tip.
pub fn new(
config: &ZebradConfig,
peers: ZN,
verifier: ZV,
state: ZS,
latest_chain_tip: ZSTip,
) -> (Self, SyncStatus) {
let tip_network = Timeout::new(peers.clone(), TIPS_RESPONSE_TIMEOUT);
// The Hedge middleware is the outermost layer, hedging requests
// between two retry-wrapped networks. The innermost timeout
// layer is relatively unimportant, because slow requests will
// probably be pre-emptively hedged.
//
// The Hedge goes outside the Retry, because the Retry layer
// abstracts away spurious failures from individual peers
// making a less-fallible network service, and the Hedge layer
// tries to reduce latency of that less-fallible service.
//
// XXX add ServiceBuilder::hedge() so this becomes
// ServiceBuilder::new().hedge(...).retry(...)...
let block_network = Hedge::new(
ServiceBuilder::new()
.concurrency_limit(config.sync.max_concurrent_block_requests)
.retry(zn::RetryLimit::new(BLOCK_DOWNLOAD_RETRY_LIMIT))
.timeout(BLOCK_DOWNLOAD_TIMEOUT)
.service(peers),
AlwaysHedge,
20,
0.95,
2 * SYNC_RESTART_DELAY,
);
// We apply a timeout to the verifier to avoid hangs due to missing earlier blocks.
let verifier = Timeout::new(verifier, BLOCK_VERIFY_TIMEOUT);
assert!(
config.sync.lookahead_limit >= MIN_LOOKAHEAD_LIMIT,
"configured lookahead limit {} too low, must be at least {}",
config.sync.lookahead_limit,
MIN_LOOKAHEAD_LIMIT
);
let (sync_status, recent_syncs) = SyncStatus::new();
let new_syncer = Self {
genesis_hash: genesis_hash(config.network.network),
lookahead_limit: config.sync.lookahead_limit,
tip_network,
downloads: Box::pin(Downloads::new(
block_network,
verifier,
latest_chain_tip.clone(),
config.sync.lookahead_limit,
)),
state,
latest_chain_tip,
prospective_tips: HashSet::new(),
recent_syncs,
};
(new_syncer, sync_status)
}
2020-07-08 13:33:39 -07:00
#[instrument(skip(self))]
pub async fn sync(mut self) -> Result<(), Report> {
// We can't download the genesis block using our normal algorithm,
// due to protocol limitations
self.request_genesis().await?;
// Distinguishes a restart from a start, so we don't sleep when starting
// the sync process, but we can keep restart logic in one place.
let mut started_once = false;
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
'sync: loop {
if started_once {
info!(
timeout = ?SYNC_RESTART_DELAY,
state_tip = ?self.latest_chain_tip.best_tip_height(),
"waiting to restart sync"
);
self.prospective_tips = HashSet::new();
self.downloads.cancel_all();
self.update_metrics();
sleep(SYNC_RESTART_DELAY).await;
} else {
started_once = true;
}
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
info!(
state_tip = ?self.latest_chain_tip.best_tip_height(),
"starting sync, obtaining new tips"
);
if let Err(e) = self.obtain_tips().await {
warn!(?e, "error obtaining tips");
continue 'sync;
}
self.update_metrics();
while !self.prospective_tips.is_empty() {
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
// Check whether any block tasks are currently ready:
Use `MockedClientHandle` in other tests (#3241) * Move `MockedClientHandle` to `peer` module It's more closely related to a `Client` than the `PeerSet`, and this prepares it to be used by other tests. * Rename `MockedClientHandle` to `ClientTestHarness` Reduce confusion, and clarify that the client is not mocked. Co-authored-by: teor <teor@riseup.net> * Add clarification to `mock_peers` documentation Explicitly say how the generated data is returned. * Rename method to `wants_connection_heartbeats` The `Client` service only represents one direction of a connection, so `is_connected` is not the exact term. Co-authored-by: teor <teor@riseup.net> * Mock `Client` instead of `LoadTrackedClient` Move where the conversion from mocked `Client` to mocked `LoadTrackedClient` in order to make the test helper more easily used by other tests. * Use `ClientTestHarness` in `initialize` tests Replace the boilerplate code to create a fake `Client` instance with usages of the `ClientTestHarness` constructor. * Allow receiving requests from `Client` instance Create a helper type to wrap the result, to make it easier to assert on specific events after trying to receive a request. * Allow inspecting the current error in the slot Share the `ErrorSlot` between the `Client` and the handle, so that the handle can be used to inspect the contents of the `ErrorSlot`. * Allow placing an error into the `ErrorSlot` Assuming it is initially empty. If it already has an error, the code will panic. * Allow gracefully closing the request receiver Close the endpoint with the appropriate call to the `close()` method. * Allow dropping the request receiver endpoint Forcefully closes the endpoint. * Rename field to `client_request_receiver` Also rename the related methods to include `outbound_client_request_receiver` to make it more precise. Co-authored-by: teor <teor@riseup.net> * Allow dropping the heartbeat shutdown receiver Allows the `Client` to detect that the channel has been closed. * Rename fn. to `drop_heartbeat_shutdown_receiver` Make it clear that it affects the heartbeat task. Co-authored-by: teor <teor@riseup.net> * Move `NowOrLater` into a new `now-or-later` crate Make it easily accessible to other crates. * Add `IsReady` extension trait for `Service` Simplifies checking if a service is immediately ready to be called. * Add extension method to check for readiness error Checks if the `Service` isn't immediately ready because a call to `ready` immediately returns an error. * Rename method to `is_failed` Avoid negated method names. Co-authored-by: teor <teor@riseup.net> * Add a `IsReady::is_pending` extension method Checks if a `Service` is not ready to be called. * Use `ClientTestHarness` in `Client` test vectors Reduce repeated code and try to improve readability. * Create a new `ClientTestHarnessBuilder` type A builder to create test `Client` instances using mock data which can be tracked and manipulated through a `ClientTestHarness`. * Allow configuring the `Client`'s mocked version Add a `with_version` builder method. * Use `ClientTestHarnessBuilder` in `PeerVersions` Use the builder to set the peer version, so that the `version` parameter can be removed from the constructor later. * Use a default mock version where possible Reduce noise when setting up the harness for tests that don't really care about the remote peer version. * Remove `Version` parameter from the `build` method The `with_version` builder method should be used instead. * Fix some typos and outdated info in the release checklist * Add extra client tests for zero and multiple readiness checks (#3273) And document existing tests. * Replace `NowOrLater` with `futures::poll!` (#3272) * Replace NowOrLater with the futures::poll! macro in zebrad * Replace NowOrLater with the futures::poll! macro in zebra-test * Remove the now-or-later crate * remove unused imports * rustfmt Co-authored-by: teor <teor@riseup.net> Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com>
2021-12-21 12:13:26 -08:00
while let Poll::Ready(Some(rsp)) = futures::poll!(self.downloads.next()) {
match rsp {
Ok(hash) => {
trace!(?hash, "verified and committed block to state");
}
Err(e) => {
if Self::should_restart_sync(e) {
continue 'sync;
}
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
}
}
}
self.update_metrics();
// If we have too many pending tasks, wait for some to finish.
//
// Starting to wait is interesting, but logging each wait can be
// very verbose.
if self.downloads.in_flight() > self.lookahead_limit {
tracing::info!(
tips.len = self.prospective_tips.len(),
in_flight = self.downloads.in_flight(),
lookahead_limit = self.lookahead_limit,
"waiting for pending blocks",
);
}
while self.downloads.in_flight() > self.lookahead_limit {
trace!(
tips.len = self.prospective_tips.len(),
in_flight = self.downloads.in_flight(),
lookahead_limit = self.lookahead_limit,
state_tip = ?self.latest_chain_tip.best_tip_height(),
"waiting for pending blocks",
);
match self.downloads.next().await.expect("downloads is nonempty") {
Ok(hash) => {
trace!(?hash, "verified and committed block to state");
}
Err(e) => {
if Self::should_restart_sync(e) {
continue 'sync;
}
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
}
}
self.update_metrics();
}
// Once we're below the lookahead limit, we can keep extending the tips.
info!(
tips.len = self.prospective_tips.len(),
in_flight = self.downloads.in_flight(),
lookahead_limit = self.lookahead_limit,
state_tip = ?self.latest_chain_tip.best_tip_height(),
"extending tips",
);
if let Err(e) = self.extend_tips().await {
warn!(?e, "error extending tips");
continue 'sync;
}
2020-08-07 01:04:33 -07:00
self.update_metrics();
}
info!("exhausted prospective tip set");
}
}
/// Given a block_locator list fan out request for subsequent hashes to
/// multiple peers
2020-07-08 13:33:39 -07:00
#[instrument(skip(self))]
async fn obtain_tips(&mut self) -> Result<(), Report> {
let block_locator = self
.state
Update to Tokio 1.13.0 (#2994) * Update `tower` to version `0.4.9` Update to latest version to add support for Tokio version 1. * Replace usage of `ServiceExt::ready_and` It was deprecated in favor of `ServiceExt::ready`. * Update Tokio dependency to version `1.13.0` This will break the build because the code isn't ready for the update, but future commits will fix the issues. * Replace import of `tokio::stream::StreamExt` Use `futures::stream::StreamExt` instead, because newer versions of Tokio don't have the `stream` feature. * Use `IntervalStream` in `zebra-network` In newer versions of Tokio `Interval` doesn't implement `Stream`, so the wrapper types from `tokio-stream` have to be used instead. * Use `IntervalStream` in `inventory_registry` In newer versions of Tokio the `Interval` type doesn't implement `Stream`, so `tokio_stream::wrappers::IntervalStream` has to be used instead. * Use `BroadcastStream` in `inventory_registry` In newer versions of Tokio `broadcast::Receiver` doesn't implement `Stream`, so `tokio_stream::wrappers::BroadcastStream` instead. This also requires changing the error type that is used. * Handle `Semaphore::acquire` error in `tower-batch` Newer versions of Tokio can return an error if the semaphore is closed. This shouldn't happen in `tower-batch` because the semaphore is never closed. * Handle `Semaphore::acquire` error in `zebrad` test On newer versions of Tokio `Semaphore::acquire` can return an error if the semaphore is closed. This shouldn't happen in the test because the semaphore is never closed. * Update some `zebra-network` dependencies Use versions compatible with Tokio version 1. * Upgrade Hyper to version 0.14 Use a version that supports Tokio version 1. * Update `metrics` dependency to version 0.17 And also update the `metrics-exporter-prometheus` to version 0.6.1. These updates are to make sure Tokio 1 is supported. * Use `f64` as the histogram data type `u64` isn't supported as the histogram data type in newer versions of `metrics`. * Update the initialization of the metrics component Make it compatible with the new version of `metrics`. * Simplify build version counter Remove all constants and use the new `metrics::incement_counter!` macro. * Change metrics output line to match on The snapshot string isn't included in the newer version of `metrics-exporter-prometheus`. * Update `sentry` to version 0.23.0 Use a version compatible with Tokio version 1. * Remove usage of `TracingIntegration` This seems to not be available from `sentry-tracing` anymore, so it needs to be replaced. * Add sentry layer to tracing initialization This seems like the replacement for `TracingIntegration`. * Remove unnecessary conversion Suggested by a Clippy lint. * Update Cargo lock file Apply all of the updates to dependencies. * Ban duplicate tokio dependencies Also ban git sources for tokio dependencies. * Stop allowing sentry-tracing git repository in `deny.toml` * Allow remaining duplicates after the tokio upgrade * Use C: drive for CI build output on Windows GitHub Actions uses a Windows image with two disk drives, and the default D: drive is smaller than the C: drive. Zebra currently uses a lot of space to build, so it has to use the C: drive to avoid CI build failures because of insufficient space. Co-authored-by: teor <teor@riseup.net>
2021-11-02 11:46:57 -07:00
.ready()
.await
.map_err(|e| eyre!(e))?
2020-09-09 21:19:15 -07:00
.call(zebra_state::Request::BlockLocator)
.await
.map(|response| match response {
2020-09-09 21:19:15 -07:00
zebra_state::Response::BlockLocator(block_locator) => block_locator,
_ => unreachable!(
"GetBlockLocator request can only result in Response::BlockLocator"
),
})
.map_err(|e| eyre!(e))?;
debug!(
tip = ?block_locator.first().expect("we have at least one block locator object"),
?block_locator,
"got block locator and trying to obtain new chain tips"
);
let mut requests = FuturesUnordered::new();
for attempt in 0..FANOUT {
if attempt > 0 {
// Let other tasks run, so we're more likely to choose a different peer.
//
// TODO: move fanouts into the PeerSet, so we always choose different peers (#2214)
tokio::task::yield_now().await;
}
let ready_tip_network = self.tip_network.ready().await;
requests.push(tokio::spawn(ready_tip_network.map_err(|e| eyre!(e))?.call(
Update to Tokio 1.13.0 (#2994) * Update `tower` to version `0.4.9` Update to latest version to add support for Tokio version 1. * Replace usage of `ServiceExt::ready_and` It was deprecated in favor of `ServiceExt::ready`. * Update Tokio dependency to version `1.13.0` This will break the build because the code isn't ready for the update, but future commits will fix the issues. * Replace import of `tokio::stream::StreamExt` Use `futures::stream::StreamExt` instead, because newer versions of Tokio don't have the `stream` feature. * Use `IntervalStream` in `zebra-network` In newer versions of Tokio `Interval` doesn't implement `Stream`, so the wrapper types from `tokio-stream` have to be used instead. * Use `IntervalStream` in `inventory_registry` In newer versions of Tokio the `Interval` type doesn't implement `Stream`, so `tokio_stream::wrappers::IntervalStream` has to be used instead. * Use `BroadcastStream` in `inventory_registry` In newer versions of Tokio `broadcast::Receiver` doesn't implement `Stream`, so `tokio_stream::wrappers::BroadcastStream` instead. This also requires changing the error type that is used. * Handle `Semaphore::acquire` error in `tower-batch` Newer versions of Tokio can return an error if the semaphore is closed. This shouldn't happen in `tower-batch` because the semaphore is never closed. * Handle `Semaphore::acquire` error in `zebrad` test On newer versions of Tokio `Semaphore::acquire` can return an error if the semaphore is closed. This shouldn't happen in the test because the semaphore is never closed. * Update some `zebra-network` dependencies Use versions compatible with Tokio version 1. * Upgrade Hyper to version 0.14 Use a version that supports Tokio version 1. * Update `metrics` dependency to version 0.17 And also update the `metrics-exporter-prometheus` to version 0.6.1. These updates are to make sure Tokio 1 is supported. * Use `f64` as the histogram data type `u64` isn't supported as the histogram data type in newer versions of `metrics`. * Update the initialization of the metrics component Make it compatible with the new version of `metrics`. * Simplify build version counter Remove all constants and use the new `metrics::incement_counter!` macro. * Change metrics output line to match on The snapshot string isn't included in the newer version of `metrics-exporter-prometheus`. * Update `sentry` to version 0.23.0 Use a version compatible with Tokio version 1. * Remove usage of `TracingIntegration` This seems to not be available from `sentry-tracing` anymore, so it needs to be replaced. * Add sentry layer to tracing initialization This seems like the replacement for `TracingIntegration`. * Remove unnecessary conversion Suggested by a Clippy lint. * Update Cargo lock file Apply all of the updates to dependencies. * Ban duplicate tokio dependencies Also ban git sources for tokio dependencies. * Stop allowing sentry-tracing git repository in `deny.toml` * Allow remaining duplicates after the tokio upgrade * Use C: drive for CI build output on Windows GitHub Actions uses a Windows image with two disk drives, and the default D: drive is smaller than the C: drive. Zebra currently uses a lot of space to build, so it has to use the C: drive to avoid CI build failures because of insufficient space. Co-authored-by: teor <teor@riseup.net>
2021-11-02 11:46:57 -07:00
zn::Request::FindBlocks {
known_blocks: block_locator.clone(),
stop: None,
},
)));
}
let mut download_set = IndexSet::new();
while let Some(res) = requests.next().await {
match res
.expect("panic in spawned obtain tips request")
.map_err::<Report, _>(|e| eyre!(e))
{
Ok(zn::Response::BlockHashes(hashes)) => {
trace!(?hashes);
// zcashd sometimes appends an unrelated hash at the start
// or end of its response.
//
// We can't discard the first hash, because it might be a
// block we want to download. So we just accept any
// out-of-order first hashes.
// We use the last hash for the tip, and we want to avoid bad
// tips. So we discard the last hash. (We don't need to worry
// about missed downloads, because we will pick them up again
// in ExtendTips.)
let hashes = match hashes.as_slice() {
[] => continue,
[rest @ .., _last] => rest,
};
let mut first_unknown = None;
for (i, &hash) in hashes.iter().enumerate() {
if !self.state_contains(hash).await? {
first_unknown = Some(i);
break;
}
}
debug!(hashes.len = ?hashes.len(), ?first_unknown);
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
let unknown_hashes = if let Some(index) = first_unknown {
&hashes[index..]
} else {
continue;
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
};
trace!(?unknown_hashes);
2020-07-08 13:33:39 -07:00
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
let new_tip = if let Some(end) = unknown_hashes.rchunks_exact(2).next() {
CheckedTip {
tip: end[0],
expected_next: end[1],
}
} else {
debug!("discarding response that extends only one block");
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
continue;
};
// Make sure we get the same tips, regardless of the
// order of peer responses
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
if !download_set.contains(&new_tip.expected_next) {
debug!(?new_tip,
"adding new prospective tip, and removing existing tips in the new block hash list");
self.prospective_tips
.retain(|t| !unknown_hashes.contains(&t.expected_next));
2020-07-08 13:33:39 -07:00
self.prospective_tips.insert(new_tip);
} else {
debug!(
?new_tip,
"discarding prospective tip: already in download set"
);
2020-07-08 13:33:39 -07:00
}
// security: the first response determines our download order
//
// TODO: can we make the download order independent of response order?
2020-07-08 13:33:39 -07:00
let prev_download_len = download_set.len();
download_set.extend(unknown_hashes);
let new_download_len = download_set.len();
let new_hashes = new_download_len - prev_download_len;
debug!(new_hashes, "added hashes to download set");
Update to Tokio 1.13.0 (#2994) * Update `tower` to version `0.4.9` Update to latest version to add support for Tokio version 1. * Replace usage of `ServiceExt::ready_and` It was deprecated in favor of `ServiceExt::ready`. * Update Tokio dependency to version `1.13.0` This will break the build because the code isn't ready for the update, but future commits will fix the issues. * Replace import of `tokio::stream::StreamExt` Use `futures::stream::StreamExt` instead, because newer versions of Tokio don't have the `stream` feature. * Use `IntervalStream` in `zebra-network` In newer versions of Tokio `Interval` doesn't implement `Stream`, so the wrapper types from `tokio-stream` have to be used instead. * Use `IntervalStream` in `inventory_registry` In newer versions of Tokio the `Interval` type doesn't implement `Stream`, so `tokio_stream::wrappers::IntervalStream` has to be used instead. * Use `BroadcastStream` in `inventory_registry` In newer versions of Tokio `broadcast::Receiver` doesn't implement `Stream`, so `tokio_stream::wrappers::BroadcastStream` instead. This also requires changing the error type that is used. * Handle `Semaphore::acquire` error in `tower-batch` Newer versions of Tokio can return an error if the semaphore is closed. This shouldn't happen in `tower-batch` because the semaphore is never closed. * Handle `Semaphore::acquire` error in `zebrad` test On newer versions of Tokio `Semaphore::acquire` can return an error if the semaphore is closed. This shouldn't happen in the test because the semaphore is never closed. * Update some `zebra-network` dependencies Use versions compatible with Tokio version 1. * Upgrade Hyper to version 0.14 Use a version that supports Tokio version 1. * Update `metrics` dependency to version 0.17 And also update the `metrics-exporter-prometheus` to version 0.6.1. These updates are to make sure Tokio 1 is supported. * Use `f64` as the histogram data type `u64` isn't supported as the histogram data type in newer versions of `metrics`. * Update the initialization of the metrics component Make it compatible with the new version of `metrics`. * Simplify build version counter Remove all constants and use the new `metrics::incement_counter!` macro. * Change metrics output line to match on The snapshot string isn't included in the newer version of `metrics-exporter-prometheus`. * Update `sentry` to version 0.23.0 Use a version compatible with Tokio version 1. * Remove usage of `TracingIntegration` This seems to not be available from `sentry-tracing` anymore, so it needs to be replaced. * Add sentry layer to tracing initialization This seems like the replacement for `TracingIntegration`. * Remove unnecessary conversion Suggested by a Clippy lint. * Update Cargo lock file Apply all of the updates to dependencies. * Ban duplicate tokio dependencies Also ban git sources for tokio dependencies. * Stop allowing sentry-tracing git repository in `deny.toml` * Allow remaining duplicates after the tokio upgrade * Use C: drive for CI build output on Windows GitHub Actions uses a Windows image with two disk drives, and the default D: drive is smaller than the C: drive. Zebra currently uses a lot of space to build, so it has to use the C: drive to avoid CI build failures because of insufficient space. Co-authored-by: teor <teor@riseup.net>
2021-11-02 11:46:57 -07:00
metrics::histogram!("sync.obtain.response.hash.count", new_hashes as f64);
}
Ok(_) => unreachable!("network returned wrong response"),
// We ignore this error because we made multiple fanout requests.
Err(e) => debug!(?e),
}
}
debug!(?self.prospective_tips);
2020-08-07 01:04:33 -07:00
// Check that the new tips we got are actually unknown.
for hash in &download_set {
debug!(?hash, "checking if state contains hash");
if self.state_contains(*hash).await? {
return Err(eyre!("queued download of hash behind our chain tip"));
}
}
let new_downloads = download_set.len();
debug!(new_downloads, "queueing new downloads");
metrics::gauge!("sync.obtain.queued.hash.count", new_downloads as f64);
// security: use the actual number of new downloads from all peers,
// so the last peer to respond can't toggle our mempool
self.recent_syncs.push_obtain_tips_length(new_downloads);
self.request_blocks(download_set).await?;
Ok(())
}
2020-07-08 13:33:39 -07:00
#[instrument(skip(self))]
async fn extend_tips(&mut self) -> Result<(), Report> {
let tips = std::mem::take(&mut self.prospective_tips);
let mut download_set = IndexSet::new();
debug!(tips = ?tips.len(), "trying to extend chain tips");
for tip in tips {
debug!(?tip, "asking peers to extend chain tip");
let mut responses = FuturesUnordered::new();
for attempt in 0..FANOUT {
if attempt > 0 {
// Let other tasks run, so we're more likely to choose a different peer.
//
// TODO: move fanouts into the PeerSet, so we always choose different peers (#2214)
tokio::task::yield_now().await;
}
let ready_tip_network = self.tip_network.ready().await;
responses.push(tokio::spawn(ready_tip_network.map_err(|e| eyre!(e))?.call(
Update to Tokio 1.13.0 (#2994) * Update `tower` to version `0.4.9` Update to latest version to add support for Tokio version 1. * Replace usage of `ServiceExt::ready_and` It was deprecated in favor of `ServiceExt::ready`. * Update Tokio dependency to version `1.13.0` This will break the build because the code isn't ready for the update, but future commits will fix the issues. * Replace import of `tokio::stream::StreamExt` Use `futures::stream::StreamExt` instead, because newer versions of Tokio don't have the `stream` feature. * Use `IntervalStream` in `zebra-network` In newer versions of Tokio `Interval` doesn't implement `Stream`, so the wrapper types from `tokio-stream` have to be used instead. * Use `IntervalStream` in `inventory_registry` In newer versions of Tokio the `Interval` type doesn't implement `Stream`, so `tokio_stream::wrappers::IntervalStream` has to be used instead. * Use `BroadcastStream` in `inventory_registry` In newer versions of Tokio `broadcast::Receiver` doesn't implement `Stream`, so `tokio_stream::wrappers::BroadcastStream` instead. This also requires changing the error type that is used. * Handle `Semaphore::acquire` error in `tower-batch` Newer versions of Tokio can return an error if the semaphore is closed. This shouldn't happen in `tower-batch` because the semaphore is never closed. * Handle `Semaphore::acquire` error in `zebrad` test On newer versions of Tokio `Semaphore::acquire` can return an error if the semaphore is closed. This shouldn't happen in the test because the semaphore is never closed. * Update some `zebra-network` dependencies Use versions compatible with Tokio version 1. * Upgrade Hyper to version 0.14 Use a version that supports Tokio version 1. * Update `metrics` dependency to version 0.17 And also update the `metrics-exporter-prometheus` to version 0.6.1. These updates are to make sure Tokio 1 is supported. * Use `f64` as the histogram data type `u64` isn't supported as the histogram data type in newer versions of `metrics`. * Update the initialization of the metrics component Make it compatible with the new version of `metrics`. * Simplify build version counter Remove all constants and use the new `metrics::incement_counter!` macro. * Change metrics output line to match on The snapshot string isn't included in the newer version of `metrics-exporter-prometheus`. * Update `sentry` to version 0.23.0 Use a version compatible with Tokio version 1. * Remove usage of `TracingIntegration` This seems to not be available from `sentry-tracing` anymore, so it needs to be replaced. * Add sentry layer to tracing initialization This seems like the replacement for `TracingIntegration`. * Remove unnecessary conversion Suggested by a Clippy lint. * Update Cargo lock file Apply all of the updates to dependencies. * Ban duplicate tokio dependencies Also ban git sources for tokio dependencies. * Stop allowing sentry-tracing git repository in `deny.toml` * Allow remaining duplicates after the tokio upgrade * Use C: drive for CI build output on Windows GitHub Actions uses a Windows image with two disk drives, and the default D: drive is smaller than the C: drive. Zebra currently uses a lot of space to build, so it has to use the C: drive to avoid CI build failures because of insufficient space. Co-authored-by: teor <teor@riseup.net>
2021-11-02 11:46:57 -07:00
zn::Request::FindBlocks {
known_blocks: vec![tip.tip],
stop: None,
},
)));
2020-07-08 13:33:39 -07:00
}
while let Some(res) = responses.next().await {
match res
.expect("panic in spawned extend tips request")
.map_err::<Report, _>(|e| eyre!(e))
{
Ok(zn::Response::BlockHashes(hashes)) => {
debug!(first = ?hashes.first(), len = ?hashes.len());
trace!(?hashes);
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
// zcashd sometimes appends an unrelated hash at the
// start or end of its response. Check the first hash
// against the previous response, and discard mismatches.
let unknown_hashes = match hashes.as_slice() {
[expected_hash, rest @ ..] if expected_hash == &tip.expected_next => {
rest
}
// If the first hash doesn't match, retry with the second.
[first_hash, expected_hash, rest @ ..]
if expected_hash == &tip.expected_next =>
{
debug!(?first_hash,
?tip.expected_next,
?tip.tip,
"unexpected first hash, but the second matches: using the hashes after the match");
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
rest
}
// We ignore these responses
[] => continue,
[single_hash] => {
debug!(?single_hash,
?tip.expected_next,
?tip.tip,
"discarding response containing a single unexpected hash");
continue;
}
[first_hash, second_hash, rest @ ..] => {
debug!(?first_hash,
?second_hash,
rest_len = ?rest.len(),
?tip.expected_next,
?tip.tip,
"discarding response that starts with two unexpected hashes");
continue;
}
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
};
// We use the last hash for the tip, and we want to avoid
// bad tips. So we discard the last hash. (We don't need
// to worry about missed downloads, because we will pick
// them up again in the next ExtendTips.)
let unknown_hashes = match unknown_hashes {
[] => continue,
[rest @ .., _last] => rest,
};
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
let new_tip = if let Some(end) = unknown_hashes.rchunks_exact(2).next() {
CheckedTip {
tip: end[0],
expected_next: end[1],
}
} else {
debug!("discarding response that extends only one block");
continue;
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
};
trace!(?unknown_hashes);
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
// Make sure we get the same tips, regardless of the
// order of peer responses
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
if !download_set.contains(&new_tip.expected_next) {
debug!(?new_tip,
"adding new prospective tip, and removing any existing tips in the new block hash list");
self.prospective_tips
.retain(|t| !unknown_hashes.contains(&t.expected_next));
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
self.prospective_tips.insert(new_tip);
} else {
debug!(
?new_tip,
"discarding prospective tip: already in download set"
);
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
}
// security: the first response determines our download order
//
// TODO: can we make the download order independent of response order?
2020-08-07 01:04:33 -07:00
let prev_download_len = download_set.len();
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
download_set.extend(unknown_hashes);
2020-08-07 01:04:33 -07:00
let new_download_len = download_set.len();
let new_hashes = new_download_len - prev_download_len;
debug!(new_hashes, "added hashes to download set");
Update to Tokio 1.13.0 (#2994) * Update `tower` to version `0.4.9` Update to latest version to add support for Tokio version 1. * Replace usage of `ServiceExt::ready_and` It was deprecated in favor of `ServiceExt::ready`. * Update Tokio dependency to version `1.13.0` This will break the build because the code isn't ready for the update, but future commits will fix the issues. * Replace import of `tokio::stream::StreamExt` Use `futures::stream::StreamExt` instead, because newer versions of Tokio don't have the `stream` feature. * Use `IntervalStream` in `zebra-network` In newer versions of Tokio `Interval` doesn't implement `Stream`, so the wrapper types from `tokio-stream` have to be used instead. * Use `IntervalStream` in `inventory_registry` In newer versions of Tokio the `Interval` type doesn't implement `Stream`, so `tokio_stream::wrappers::IntervalStream` has to be used instead. * Use `BroadcastStream` in `inventory_registry` In newer versions of Tokio `broadcast::Receiver` doesn't implement `Stream`, so `tokio_stream::wrappers::BroadcastStream` instead. This also requires changing the error type that is used. * Handle `Semaphore::acquire` error in `tower-batch` Newer versions of Tokio can return an error if the semaphore is closed. This shouldn't happen in `tower-batch` because the semaphore is never closed. * Handle `Semaphore::acquire` error in `zebrad` test On newer versions of Tokio `Semaphore::acquire` can return an error if the semaphore is closed. This shouldn't happen in the test because the semaphore is never closed. * Update some `zebra-network` dependencies Use versions compatible with Tokio version 1. * Upgrade Hyper to version 0.14 Use a version that supports Tokio version 1. * Update `metrics` dependency to version 0.17 And also update the `metrics-exporter-prometheus` to version 0.6.1. These updates are to make sure Tokio 1 is supported. * Use `f64` as the histogram data type `u64` isn't supported as the histogram data type in newer versions of `metrics`. * Update the initialization of the metrics component Make it compatible with the new version of `metrics`. * Simplify build version counter Remove all constants and use the new `metrics::incement_counter!` macro. * Change metrics output line to match on The snapshot string isn't included in the newer version of `metrics-exporter-prometheus`. * Update `sentry` to version 0.23.0 Use a version compatible with Tokio version 1. * Remove usage of `TracingIntegration` This seems to not be available from `sentry-tracing` anymore, so it needs to be replaced. * Add sentry layer to tracing initialization This seems like the replacement for `TracingIntegration`. * Remove unnecessary conversion Suggested by a Clippy lint. * Update Cargo lock file Apply all of the updates to dependencies. * Ban duplicate tokio dependencies Also ban git sources for tokio dependencies. * Stop allowing sentry-tracing git repository in `deny.toml` * Allow remaining duplicates after the tokio upgrade * Use C: drive for CI build output on Windows GitHub Actions uses a Windows image with two disk drives, and the default D: drive is smaller than the C: drive. Zebra currently uses a lot of space to build, so it has to use the C: drive to avoid CI build failures because of insufficient space. Co-authored-by: teor <teor@riseup.net>
2021-11-02 11:46:57 -07:00
metrics::histogram!("sync.extend.response.hash.count", new_hashes as f64);
}
Ok(_) => unreachable!("network returned wrong response"),
// We ignore this error because we made multiple fanout requests.
Err(e) => debug!(?e),
}
}
}
let new_downloads = download_set.len();
debug!(new_downloads, "queueing new downloads");
metrics::gauge!("sync.extend.queued.hash.count", new_downloads as f64);
// security: use the actual number of new downloads from all peers,
// so the last peer to respond can't toggle our mempool
self.recent_syncs.push_extend_tips_length(new_downloads);
self.request_blocks(download_set).await?;
Ok(())
}
/// Download and verify the genesis block, if it isn't currently known to
/// our node.
async fn request_genesis(&mut self) -> Result<(), Report> {
// Due to Bitcoin protocol limitations, we can't request the genesis
// block using our standard tip-following algorithm:
// - getblocks requires at least one hash
// - responses start with the block *after* the requested block, and
// - the genesis hash is used as a placeholder for "no matches".
//
// So we just download and verify the genesis block here.
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
while !self.state_contains(self.genesis_hash).await? {
info!("starting genesis block download and verify");
self.downloads
.download_and_verify(self.genesis_hash)
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
.await
.map_err(|e| eyre!(e))?;
match self.downloads.next().await.expect("downloads is nonempty") {
Ok(hash) => trace!(?hash, "verified and committed block to state"),
Err(e) => {
warn!(?e, "could not download or verify genesis block, retrying");
tokio::time::sleep(GENESIS_TIMEOUT_RETRY).await;
}
Fix sync algorithm. (#887) * checkpoint: reject older of duplicate verification requests. If we get a duplicate block verification request, we should drop the older one in favor of the newer one, because the older request is likely to have been canceled. Previously, this code would accept up to four duplicate verification requests, then fail all subsequent ones. * sync: add a timeout layer to block requests. Note that if this timeout is too short, we'll bring down the peer set in a retry storm. * sync: restart syncing on error Restart the syncing process when an error occurs, rather than ignoring it. Restarting means we discard all tips and start over with a new block locator, so we can have another chance to "unstuck" ourselves. * sync: additional debug info * sync: handle lookahead limit correctly. Instead of extracting all the completed task results, the previous code pulled results out until there were fewer tasks than the lookahead limit, then stopped. This meant that completed tasks could be left until the limit was exceeded again. Instead, extract all completed results, and use the number of pending tasks to decide whether to extend the tip or wait for blocks to finish. * network: add debug instrumentation to retry policy * sync: instrument the spawned task * sync: streamline ObtainTips/ExtendTips logic & tracing This change does three things: 1. It aligns the implementation of ObtainTips and ExtendTips so that they use the same deduplication method. This means that when debugging we only have one deduplication algorithm to focus on. 2. It streamlines the tracing output to not include information already included in spans. Both obtain_tips and extend_tips have their own spans attached to the events, so it's not necessary to add Scope: prefixes in messages. 3. It changes the messages to be focused on reporting the actual events rather than the interpretation of the events (e.g., "got genesis hash in response" rather than "peer could not extend tip"). The motivation for this change is that when debugging, the interpretation of events is already known to be incorrect, in the sense that the mental model of the code (no bug) does not match its behavior (has bug), so presenting minimally-interpreted events forces interpretation relative to the actual code. * sync: hack to work around zcashd behavior * sync: localize debug statement in extend_tips * sync: change algorithm to define tips as pairs of hashes. This is different enough from the existing description that its comments no longer apply, so I removed them. A further chunk of work is to change the sync RFC to document this algorithm. * sync: reduce block timeout * state: add resource limits for sled Closes #888 * sync: add a restart timeout constant * sync: de-pub constants
2020-08-12 16:48:01 -07:00
}
}
Ok(())
}
/// Queue download and verify tasks for each block that isn't currently known to our node
async fn request_blocks(&mut self, hashes: IndexSet<block::Hash>) -> Result<(), Report> {
debug!(hashes.len = hashes.len(), "requesting blocks");
for hash in hashes.into_iter() {
self.downloads.download_and_verify(hash).await?;
}
Ok(())
}
2020-08-07 01:04:33 -07:00
/// Returns `true` if the hash is present in the state, and `false`
/// if the hash is not present in the state.
///
/// BUG: check if the hash is in any chain (#862)
/// Depth only checks the main chain.
async fn state_contains(&mut self, hash: block::Hash) -> Result<bool, Report> {
match self
.state
Update to Tokio 1.13.0 (#2994) * Update `tower` to version `0.4.9` Update to latest version to add support for Tokio version 1. * Replace usage of `ServiceExt::ready_and` It was deprecated in favor of `ServiceExt::ready`. * Update Tokio dependency to version `1.13.0` This will break the build because the code isn't ready for the update, but future commits will fix the issues. * Replace import of `tokio::stream::StreamExt` Use `futures::stream::StreamExt` instead, because newer versions of Tokio don't have the `stream` feature. * Use `IntervalStream` in `zebra-network` In newer versions of Tokio `Interval` doesn't implement `Stream`, so the wrapper types from `tokio-stream` have to be used instead. * Use `IntervalStream` in `inventory_registry` In newer versions of Tokio the `Interval` type doesn't implement `Stream`, so `tokio_stream::wrappers::IntervalStream` has to be used instead. * Use `BroadcastStream` in `inventory_registry` In newer versions of Tokio `broadcast::Receiver` doesn't implement `Stream`, so `tokio_stream::wrappers::BroadcastStream` instead. This also requires changing the error type that is used. * Handle `Semaphore::acquire` error in `tower-batch` Newer versions of Tokio can return an error if the semaphore is closed. This shouldn't happen in `tower-batch` because the semaphore is never closed. * Handle `Semaphore::acquire` error in `zebrad` test On newer versions of Tokio `Semaphore::acquire` can return an error if the semaphore is closed. This shouldn't happen in the test because the semaphore is never closed. * Update some `zebra-network` dependencies Use versions compatible with Tokio version 1. * Upgrade Hyper to version 0.14 Use a version that supports Tokio version 1. * Update `metrics` dependency to version 0.17 And also update the `metrics-exporter-prometheus` to version 0.6.1. These updates are to make sure Tokio 1 is supported. * Use `f64` as the histogram data type `u64` isn't supported as the histogram data type in newer versions of `metrics`. * Update the initialization of the metrics component Make it compatible with the new version of `metrics`. * Simplify build version counter Remove all constants and use the new `metrics::incement_counter!` macro. * Change metrics output line to match on The snapshot string isn't included in the newer version of `metrics-exporter-prometheus`. * Update `sentry` to version 0.23.0 Use a version compatible with Tokio version 1. * Remove usage of `TracingIntegration` This seems to not be available from `sentry-tracing` anymore, so it needs to be replaced. * Add sentry layer to tracing initialization This seems like the replacement for `TracingIntegration`. * Remove unnecessary conversion Suggested by a Clippy lint. * Update Cargo lock file Apply all of the updates to dependencies. * Ban duplicate tokio dependencies Also ban git sources for tokio dependencies. * Stop allowing sentry-tracing git repository in `deny.toml` * Allow remaining duplicates after the tokio upgrade * Use C: drive for CI build output on Windows GitHub Actions uses a Windows image with two disk drives, and the default D: drive is smaller than the C: drive. Zebra currently uses a lot of space to build, so it has to use the C: drive to avoid CI build failures because of insufficient space. Co-authored-by: teor <teor@riseup.net>
2021-11-02 11:46:57 -07:00
.ready()
.await
.map_err(|e| eyre!(e))?
2020-09-09 21:19:15 -07:00
.call(zebra_state::Request::Depth(hash))
.await
.map_err(|e| eyre!(e))?
{
zs::Response::Depth(Some(_)) => Ok(true),
zs::Response::Depth(None) => Ok(false),
_ => unreachable!("wrong response to depth request"),
}
}
fn update_metrics(&mut self) {
2020-08-07 01:04:33 -07:00
metrics::gauge!(
"sync.prospective_tips.len",
self.prospective_tips.len() as f64
2020-08-07 01:04:33 -07:00
);
metrics::gauge!(
"sync.downloads.in_flight",
self.downloads.in_flight() as f64
);
2020-08-07 01:04:33 -07:00
}
/// Return if the sync should be restarted based on the given error
/// from the block downloader and verifier stream.
fn should_restart_sync(e: BlockDownloadVerifyError) -> bool {
match e {
// Structural matches
BlockDownloadVerifyError::Invalid(VerifyChainError::Checkpoint(
VerifyCheckpointError::AlreadyVerified { .. },
)) => {
debug!(error = ?e, "block was already verified, possibly from a previous sync run, continuing");
false
}
BlockDownloadVerifyError::Invalid(VerifyChainError::Block(
VerifyBlockError::Block {
source: BlockError::AlreadyInChain(_, _),
},
)) => {
debug!(error = ?e, "block is already in chain, possibly from a previous sync run, continuing");
false
}
BlockDownloadVerifyError::CancelledDuringDownload
| BlockDownloadVerifyError::CancelledDuringVerification => {
debug!(error = ?e, "block verification was cancelled, continuing");
false
}
BlockDownloadVerifyError::BehindTipHeightLimit => {
debug!(
error = ?e,
"block height is behind the current state tip, \
assuming the syncer will eventually catch up to the state, continuing"
);
false
}
// String matches
BlockDownloadVerifyError::Invalid(VerifyChainError::Block(
VerifyBlockError::Commit(ref source),
)) if format!("{:?}", source).contains("block is already committed to the state") => {
// TODO: improve this by checking the type (#2908)
debug!(error = ?e, "block is already committed, possibly from a previous sync run, continuing");
false
}
BlockDownloadVerifyError::DownloadFailed(ref source)
if format!("{:?}", source).contains("NotFound") =>
{
// TODO: improve this by checking the type (#2908)
// restart after a certain number of NotFound errors?
debug!(error = ?e, "block was not found, possibly from a peer that doesn't have the block yet, continuing");
false
}
_ => {
// download_and_verify downcasts errors from the block verifier
// into VerifyChainError, and puts the result inside one of the
// BlockDownloadVerifyError enumerations. This downcast could
// become incorrect e.g. after some refactoring, and it is difficult
// to write a test to check it. The test below is a best-effort
// attempt to catch if that happens and log it.
// TODO: add a proper test and remove this
// https://github.com/ZcashFoundation/zebra/issues/2909
let err_str = format!("{:?}", e);
if err_str.contains("AlreadyVerified")
|| err_str.contains("AlreadyInChain")
|| err_str.contains("Cancelled")
|| err_str.contains("BehindTipHeight")
|| err_str.contains("block is already committed to the state")
|| err_str.contains("NotFound")
{
error!(?e,
"a BlockDownloadVerifyError that should have been filtered out was detected, \
which possibly indicates a programming error in the downcast inside \
zebrad::components::sync::downloads::Downloads::download_and_verify"
)
}
warn!(?e, "error downloading and verifying block");
true
}
}
}
}