zebra/zebrad/src/components/sync.rs

1250 lines
51 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 std::{cmp::max, collections::HashSet, convert, pin::Pin, 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 serde::{Deserialize, Serialize};
use tokio::{
sync::watch,
task::JoinError,
time::{sleep, timeout},
};
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, Height, HeightDiff},
chain_tip::ChainTip,
parameters::genesis_hash,
};
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;
pub mod end_of_support;
mod gossip;
mod progress;
mod recent_sync_lengths;
mod status;
#[cfg(test)]
mod tests;
use downloads::{AlwaysHedge, Downloads};
change(state): Write non-finalized blocks to the state in a separate thread, to avoid network and RPC hangs (#5257) * Add a new block commit task and channels, that don't do anything yet * Add last_block_hash_sent to the state service, to avoid database accesses * Update last_block_hash_sent regardless of commit errors * Rename a field to StateService.max_queued_finalized_height * Commit finalized blocks to the state in a separate task * Check for panics in the block write task * Wait for the block commit task in tests, and check for errors * Always run a proptest that sleeps once * Add extra debugging to state shutdowns * Work around a RocksDB shutdown bug * Close the finalized block channel when we're finished with it * Only reset state queue once per error * Update some TODOs * Add a module doc comment * Drop channels and check for closed channels in the block commit task * Close state channels and tasks on drop * Remove some duplicate fields across StateService and ReadStateService * Try tweaking the shutdown steps * Update and clarify some comments * Clarify another comment * Don't try to cancel RocksDB background work on drop * Fix up some comments * Remove some duplicate code * Remove redundant workarounds for shutdown issues * Remode a redundant channel close in the block commit task * Remove a mistaken `!force` shutdown condition * Remove duplicate force-shutdown code and explain it better * Improve RPC error logging * Wait for chain tip updates in the RPC tests * Wait 2 seconds for chain tip updates before skipping them * Remove an unnecessary block_in_place() * Fix some test error messages that were changed by earlier fixes * Expand some comments, fix typos Co-authored-by: Marek <mail@marek.onl> * Actually drop children of failed blocks * Explain why we drop descendants of failed blocks * Clarify a comment * Wait for chain tip updates in a failing test on macOS * Clean duplicate finalized blocks when the non-finalized state activates * Send an error when receiving a duplicate finalized block * Update checkpoint block behaviour, document its consensus rule * Wait for chain tip changes in inbound_block_height_lookahead_limit test * Wait for the genesis block to commit in the fake peer set mempool tests * Disable unreliable mempool verification check in the send transaction test * Appease rustfmt * Use clear_finalized_block_queue() everywhere that blocks are dropped * Document how Finalized and NonFinalized clones are different * sends non-finalized blocks to the block write task * passes ZebraDb to commit_new_chain, commit_block, and no_duplicates_in_finalized_chain instead of FinalizedState * Update zebra-state/src/service/write.rs Co-authored-by: teor <teor@riseup.net> * updates comments, renames send_process_queued, other minor cleanup * update assert_block_can_be_validated comment * removes `mem` field from StateService * removes `disk` field from StateService and updates block_iter to use `ZebraDb` instead of the finalized state * updates tests that use the disk to use read_service.db instead * moves best_tip to a read fn and returns finalized & non-finalized states from setup instead of the state service * changes `contextual_validity` to get the network from the finalized_state instead of another param * swaps out StateService with FinalizedState and NonFinalizedState in tests * adds NotReadyToBeCommitted error and returns it from validate_and_commit when a blocks parent hash is not in any chain * removes NonFinalizedWriteCmd and calls, moves update_latest_channels above rsp_tx.send * makes parent_errors_map an indexmap * clears non-finalized block queue when the receiver is dropped and when the StateService is being dropped * sends non-finalized blocks to the block write task * passes ZebraDb to commit_new_chain, commit_block, and no_duplicates_in_finalized_chain instead of FinalizedState * updates comments, renames send_process_queued, other minor cleanup * Update zebra-state/src/service/write.rs Co-authored-by: teor <teor@riseup.net> * update assert_block_can_be_validated comment * removes `mem` field from StateService * removes `disk` field from StateService and updates block_iter to use `ZebraDb` instead of the finalized state * updates tests that use the disk to use read_service.db instead * moves best_tip to a read fn and returns finalized & non-finalized states from setup instead of the state service * changes `contextual_validity` to get the network from the finalized_state instead of another param * swaps out StateService with FinalizedState and NonFinalizedState in tests * adds NotReadyToBeCommitted error and returns it from validate_and_commit when a blocks parent hash is not in any chain * removes NonFinalizedWriteCmd and calls, moves update_latest_channels above rsp_tx.send * makes parent_errors_map an indexmap * clears non-finalized block queue when the receiver is dropped and when the StateService is being dropped * removes duplicate field definitions on StateService that were a result of a bad merge * update NotReadyToBeCommitted error message * Appear rustfmt * Fix doc links * Rename a function to initial_contextual_validity() * Do error tasks on Err, and success tasks on Ok * Simplify parent_error_map truncation * Rewrite best_tip() to use tip() * Rename latest_mem() to latest_non_finalized_state() ```sh fastmod latest_mem latest_non_finalized_state zebra* cargo fmt --all ``` * Simplify latest_non_finalized_state() using a new WatchReceiver API * Expand some error messages * Send the result after updating the channels, and document why * wait for chain_tip_update before cancelling download in mempool_cancel_mined * adds `sent_non_finalized_block_hashes` field to StateService * adds batched sent_hash insertions and checks sent hashes in queue_and_commit_non_finalized before adding a block to the queue * check that the `curr_buf` in SentHashes is not empty before pushing it to the `sent_bufs` * Apply suggestions from code review Co-authored-by: teor <teor@riseup.net> * Fix rustfmt * Check for finalized block heights using zs_contains() * adds known_utxos field to SentHashes * updates comment on SentHashes.add method * Apply suggestions from code review Co-authored-by: teor <teor@riseup.net> * return early when there's a duplicate hash in QueuedBlocks.queue instead of panicking * Make finalized UTXOs near the final checkpoint available for full block verification * Replace a checkpoint height literal with the actual config * Update mainnet and testnet checkpoints - 7 October 2022 * Fix some state service init arguments * Allow more lookahead in the downloader, but less lookahead in the syncer * Add the latest config to the tests, and fix the latest config check * Increase the number of finalized blocks checked for non-finalized block UTXO spends * fix(log): reduce verbose logs for block commits (#5348) * Remove some verbose block write channel logs * Only warn about tracing endpoint if the address is actually set * Use CloneError instead of formatting a non-cloneable error Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> * Increase block verify timeout * Work around a known block timeout bug by using a shorter timeout Co-authored-by: teor <teor@riseup.net> Co-authored-by: Marek <mail@marek.onl> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com>
2022-10-11 12:25:45 -07:00
pub use downloads::VERIFICATION_PIPELINE_SCALING_MULTIPLIER;
pub use gossip::{gossip_best_tip_block_hashes, BlockGossipError};
pub use progress::show_block_chain_progress;
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 = 3;
/// A lower bound on the user-specified checkpoint verification concurrency limit.
2021-01-22 02:44:24 -08:00
///
/// Set to the maximum checkpoint interval, so the pipeline holds around a checkpoint's
/// worth of blocks.
///
/// ## 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_CHECKPOINT_CONCURRENCY_LIMIT: usize = zebra_consensus::MAX_CHECKPOINT_HEIGHT_GAP;
/// The default for the user-specified lookahead limit.
///
/// See [`MIN_CHECKPOINT_CONCURRENCY_LIMIT`] for details.
pub const DEFAULT_CHECKPOINT_CONCURRENCY_LIMIT: usize = MAX_TIPS_RESPONSE_HASH_COUNT * 2;
/// A lower bound on the user-specified concurrency limit.
///
/// If the concurrency limit is 0, Zebra can't download or verify any blocks.
pub const MIN_CONCURRENCY_LIMIT: usize = 1;
/// 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_CHECKPOINT_CONCURRENCY_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);
/// Controls how long we wait between gossiping successive blocks or transactions.
///
/// ## Correctness
///
/// If this timeout is set too high, blocks and transactions won't propagate through
/// the network efficiently.
///
/// If this timeout is set too low, the peer set and remote peers can get overloaded.
pub const PEER_GOSSIP_DELAY: Duration = Duration::from_secs(7);
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.
change(state): Write non-finalized blocks to the state in a separate thread, to avoid network and RPC hangs (#5257) * Add a new block commit task and channels, that don't do anything yet * Add last_block_hash_sent to the state service, to avoid database accesses * Update last_block_hash_sent regardless of commit errors * Rename a field to StateService.max_queued_finalized_height * Commit finalized blocks to the state in a separate task * Check for panics in the block write task * Wait for the block commit task in tests, and check for errors * Always run a proptest that sleeps once * Add extra debugging to state shutdowns * Work around a RocksDB shutdown bug * Close the finalized block channel when we're finished with it * Only reset state queue once per error * Update some TODOs * Add a module doc comment * Drop channels and check for closed channels in the block commit task * Close state channels and tasks on drop * Remove some duplicate fields across StateService and ReadStateService * Try tweaking the shutdown steps * Update and clarify some comments * Clarify another comment * Don't try to cancel RocksDB background work on drop * Fix up some comments * Remove some duplicate code * Remove redundant workarounds for shutdown issues * Remode a redundant channel close in the block commit task * Remove a mistaken `!force` shutdown condition * Remove duplicate force-shutdown code and explain it better * Improve RPC error logging * Wait for chain tip updates in the RPC tests * Wait 2 seconds for chain tip updates before skipping them * Remove an unnecessary block_in_place() * Fix some test error messages that were changed by earlier fixes * Expand some comments, fix typos Co-authored-by: Marek <mail@marek.onl> * Actually drop children of failed blocks * Explain why we drop descendants of failed blocks * Clarify a comment * Wait for chain tip updates in a failing test on macOS * Clean duplicate finalized blocks when the non-finalized state activates * Send an error when receiving a duplicate finalized block * Update checkpoint block behaviour, document its consensus rule * Wait for chain tip changes in inbound_block_height_lookahead_limit test * Wait for the genesis block to commit in the fake peer set mempool tests * Disable unreliable mempool verification check in the send transaction test * Appease rustfmt * Use clear_finalized_block_queue() everywhere that blocks are dropped * Document how Finalized and NonFinalized clones are different * sends non-finalized blocks to the block write task * passes ZebraDb to commit_new_chain, commit_block, and no_duplicates_in_finalized_chain instead of FinalizedState * Update zebra-state/src/service/write.rs Co-authored-by: teor <teor@riseup.net> * updates comments, renames send_process_queued, other minor cleanup * update assert_block_can_be_validated comment * removes `mem` field from StateService * removes `disk` field from StateService and updates block_iter to use `ZebraDb` instead of the finalized state * updates tests that use the disk to use read_service.db instead * moves best_tip to a read fn and returns finalized & non-finalized states from setup instead of the state service * changes `contextual_validity` to get the network from the finalized_state instead of another param * swaps out StateService with FinalizedState and NonFinalizedState in tests * adds NotReadyToBeCommitted error and returns it from validate_and_commit when a blocks parent hash is not in any chain * removes NonFinalizedWriteCmd and calls, moves update_latest_channels above rsp_tx.send * makes parent_errors_map an indexmap * clears non-finalized block queue when the receiver is dropped and when the StateService is being dropped * sends non-finalized blocks to the block write task * passes ZebraDb to commit_new_chain, commit_block, and no_duplicates_in_finalized_chain instead of FinalizedState * updates comments, renames send_process_queued, other minor cleanup * Update zebra-state/src/service/write.rs Co-authored-by: teor <teor@riseup.net> * update assert_block_can_be_validated comment * removes `mem` field from StateService * removes `disk` field from StateService and updates block_iter to use `ZebraDb` instead of the finalized state * updates tests that use the disk to use read_service.db instead * moves best_tip to a read fn and returns finalized & non-finalized states from setup instead of the state service * changes `contextual_validity` to get the network from the finalized_state instead of another param * swaps out StateService with FinalizedState and NonFinalizedState in tests * adds NotReadyToBeCommitted error and returns it from validate_and_commit when a blocks parent hash is not in any chain * removes NonFinalizedWriteCmd and calls, moves update_latest_channels above rsp_tx.send * makes parent_errors_map an indexmap * clears non-finalized block queue when the receiver is dropped and when the StateService is being dropped * removes duplicate field definitions on StateService that were a result of a bad merge * update NotReadyToBeCommitted error message * Appear rustfmt * Fix doc links * Rename a function to initial_contextual_validity() * Do error tasks on Err, and success tasks on Ok * Simplify parent_error_map truncation * Rewrite best_tip() to use tip() * Rename latest_mem() to latest_non_finalized_state() ```sh fastmod latest_mem latest_non_finalized_state zebra* cargo fmt --all ``` * Simplify latest_non_finalized_state() using a new WatchReceiver API * Expand some error messages * Send the result after updating the channels, and document why * wait for chain_tip_update before cancelling download in mempool_cancel_mined * adds `sent_non_finalized_block_hashes` field to StateService * adds batched sent_hash insertions and checks sent hashes in queue_and_commit_non_finalized before adding a block to the queue * check that the `curr_buf` in SentHashes is not empty before pushing it to the `sent_bufs` * Apply suggestions from code review Co-authored-by: teor <teor@riseup.net> * Fix rustfmt * Check for finalized block heights using zs_contains() * adds known_utxos field to SentHashes * updates comment on SentHashes.add method * Apply suggestions from code review Co-authored-by: teor <teor@riseup.net> * return early when there's a duplicate hash in QueuedBlocks.queue instead of panicking * Make finalized UTXOs near the final checkpoint available for full block verification * Replace a checkpoint height literal with the actual config * Update mainnet and testnet checkpoints - 7 October 2022 * Fix some state service init arguments * Allow more lookahead in the downloader, but less lookahead in the syncer * Add the latest config to the tests, and fix the latest config check * Increase the number of finalized blocks checked for non-finalized block UTXO spends * fix(log): reduce verbose logs for block commits (#5348) * Remove some verbose block write channel logs * Only warn about tracing endpoint if the address is actually set * Use CloneError instead of formatting a non-cloneable error Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> * Increase block verify timeout * Work around a known block timeout bug by using a shorter timeout Co-authored-by: teor <teor@riseup.net> Co-authored-by: Marek <mail@marek.onl> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com>
2022-10-11 12:25:45 -07:00
///
/// We set the timeout so that it requires under 1 Mbps bandwidth for a full 2 MB block.
pub(super) const BLOCK_DOWNLOAD_TIMEOUT: Duration = Duration::from_secs(20);
/// 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.
///
change(state): Write non-finalized blocks to the state in a separate thread, to avoid network and RPC hangs (#5257) * Add a new block commit task and channels, that don't do anything yet * Add last_block_hash_sent to the state service, to avoid database accesses * Update last_block_hash_sent regardless of commit errors * Rename a field to StateService.max_queued_finalized_height * Commit finalized blocks to the state in a separate task * Check for panics in the block write task * Wait for the block commit task in tests, and check for errors * Always run a proptest that sleeps once * Add extra debugging to state shutdowns * Work around a RocksDB shutdown bug * Close the finalized block channel when we're finished with it * Only reset state queue once per error * Update some TODOs * Add a module doc comment * Drop channels and check for closed channels in the block commit task * Close state channels and tasks on drop * Remove some duplicate fields across StateService and ReadStateService * Try tweaking the shutdown steps * Update and clarify some comments * Clarify another comment * Don't try to cancel RocksDB background work on drop * Fix up some comments * Remove some duplicate code * Remove redundant workarounds for shutdown issues * Remode a redundant channel close in the block commit task * Remove a mistaken `!force` shutdown condition * Remove duplicate force-shutdown code and explain it better * Improve RPC error logging * Wait for chain tip updates in the RPC tests * Wait 2 seconds for chain tip updates before skipping them * Remove an unnecessary block_in_place() * Fix some test error messages that were changed by earlier fixes * Expand some comments, fix typos Co-authored-by: Marek <mail@marek.onl> * Actually drop children of failed blocks * Explain why we drop descendants of failed blocks * Clarify a comment * Wait for chain tip updates in a failing test on macOS * Clean duplicate finalized blocks when the non-finalized state activates * Send an error when receiving a duplicate finalized block * Update checkpoint block behaviour, document its consensus rule * Wait for chain tip changes in inbound_block_height_lookahead_limit test * Wait for the genesis block to commit in the fake peer set mempool tests * Disable unreliable mempool verification check in the send transaction test * Appease rustfmt * Use clear_finalized_block_queue() everywhere that blocks are dropped * Document how Finalized and NonFinalized clones are different * sends non-finalized blocks to the block write task * passes ZebraDb to commit_new_chain, commit_block, and no_duplicates_in_finalized_chain instead of FinalizedState * Update zebra-state/src/service/write.rs Co-authored-by: teor <teor@riseup.net> * updates comments, renames send_process_queued, other minor cleanup * update assert_block_can_be_validated comment * removes `mem` field from StateService * removes `disk` field from StateService and updates block_iter to use `ZebraDb` instead of the finalized state * updates tests that use the disk to use read_service.db instead * moves best_tip to a read fn and returns finalized & non-finalized states from setup instead of the state service * changes `contextual_validity` to get the network from the finalized_state instead of another param * swaps out StateService with FinalizedState and NonFinalizedState in tests * adds NotReadyToBeCommitted error and returns it from validate_and_commit when a blocks parent hash is not in any chain * removes NonFinalizedWriteCmd and calls, moves update_latest_channels above rsp_tx.send * makes parent_errors_map an indexmap * clears non-finalized block queue when the receiver is dropped and when the StateService is being dropped * sends non-finalized blocks to the block write task * passes ZebraDb to commit_new_chain, commit_block, and no_duplicates_in_finalized_chain instead of FinalizedState * updates comments, renames send_process_queued, other minor cleanup * Update zebra-state/src/service/write.rs Co-authored-by: teor <teor@riseup.net> * update assert_block_can_be_validated comment * removes `mem` field from StateService * removes `disk` field from StateService and updates block_iter to use `ZebraDb` instead of the finalized state * updates tests that use the disk to use read_service.db instead * moves best_tip to a read fn and returns finalized & non-finalized states from setup instead of the state service * changes `contextual_validity` to get the network from the finalized_state instead of another param * swaps out StateService with FinalizedState and NonFinalizedState in tests * adds NotReadyToBeCommitted error and returns it from validate_and_commit when a blocks parent hash is not in any chain * removes NonFinalizedWriteCmd and calls, moves update_latest_channels above rsp_tx.send * makes parent_errors_map an indexmap * clears non-finalized block queue when the receiver is dropped and when the StateService is being dropped * removes duplicate field definitions on StateService that were a result of a bad merge * update NotReadyToBeCommitted error message * Appear rustfmt * Fix doc links * Rename a function to initial_contextual_validity() * Do error tasks on Err, and success tasks on Ok * Simplify parent_error_map truncation * Rewrite best_tip() to use tip() * Rename latest_mem() to latest_non_finalized_state() ```sh fastmod latest_mem latest_non_finalized_state zebra* cargo fmt --all ``` * Simplify latest_non_finalized_state() using a new WatchReceiver API * Expand some error messages * Send the result after updating the channels, and document why * wait for chain_tip_update before cancelling download in mempool_cancel_mined * adds `sent_non_finalized_block_hashes` field to StateService * adds batched sent_hash insertions and checks sent hashes in queue_and_commit_non_finalized before adding a block to the queue * check that the `curr_buf` in SentHashes is not empty before pushing it to the `sent_bufs` * Apply suggestions from code review Co-authored-by: teor <teor@riseup.net> * Fix rustfmt * Check for finalized block heights using zs_contains() * adds known_utxos field to SentHashes * updates comment on SentHashes.add method * Apply suggestions from code review Co-authored-by: teor <teor@riseup.net> * return early when there's a duplicate hash in QueuedBlocks.queue instead of panicking * Make finalized UTXOs near the final checkpoint available for full block verification * Replace a checkpoint height literal with the actual config * Update mainnet and testnet checkpoints - 7 October 2022 * Fix some state service init arguments * Allow more lookahead in the downloader, but less lookahead in the syncer * Add the latest config to the tests, and fix the latest config check * Increase the number of finalized blocks checked for non-finalized block UTXO spends * fix(log): reduce verbose logs for block commits (#5348) * Remove some verbose block write channel logs * Only warn about tracing endpoint if the address is actually set * Use CloneError instead of formatting a non-cloneable error Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> * Increase block verify timeout * Work around a known block timeout bug by using a shorter timeout Co-authored-by: teor <teor@riseup.net> Co-authored-by: Marek <mail@marek.onl> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com>
2022-10-11 12:25:45 -07:00
/// We've observed spurious 15 minute timeouts when a lot of blocks are being committed to
/// the state. But there are also some blocks that seem to hang entirely, and never return.
///
/// So we allow about half the spurious timeout, which might cause some re-downloads.
pub(super) const BLOCK_VERIFY_TIMEOUT: Duration = Duration::from_secs(8 * 60);
change(state): Write non-finalized blocks to the state in a separate thread, to avoid network and RPC hangs (#5257) * Add a new block commit task and channels, that don't do anything yet * Add last_block_hash_sent to the state service, to avoid database accesses * Update last_block_hash_sent regardless of commit errors * Rename a field to StateService.max_queued_finalized_height * Commit finalized blocks to the state in a separate task * Check for panics in the block write task * Wait for the block commit task in tests, and check for errors * Always run a proptest that sleeps once * Add extra debugging to state shutdowns * Work around a RocksDB shutdown bug * Close the finalized block channel when we're finished with it * Only reset state queue once per error * Update some TODOs * Add a module doc comment * Drop channels and check for closed channels in the block commit task * Close state channels and tasks on drop * Remove some duplicate fields across StateService and ReadStateService * Try tweaking the shutdown steps * Update and clarify some comments * Clarify another comment * Don't try to cancel RocksDB background work on drop * Fix up some comments * Remove some duplicate code * Remove redundant workarounds for shutdown issues * Remode a redundant channel close in the block commit task * Remove a mistaken `!force` shutdown condition * Remove duplicate force-shutdown code and explain it better * Improve RPC error logging * Wait for chain tip updates in the RPC tests * Wait 2 seconds for chain tip updates before skipping them * Remove an unnecessary block_in_place() * Fix some test error messages that were changed by earlier fixes * Expand some comments, fix typos Co-authored-by: Marek <mail@marek.onl> * Actually drop children of failed blocks * Explain why we drop descendants of failed blocks * Clarify a comment * Wait for chain tip updates in a failing test on macOS * Clean duplicate finalized blocks when the non-finalized state activates * Send an error when receiving a duplicate finalized block * Update checkpoint block behaviour, document its consensus rule * Wait for chain tip changes in inbound_block_height_lookahead_limit test * Wait for the genesis block to commit in the fake peer set mempool tests * Disable unreliable mempool verification check in the send transaction test * Appease rustfmt * Use clear_finalized_block_queue() everywhere that blocks are dropped * Document how Finalized and NonFinalized clones are different * sends non-finalized blocks to the block write task * passes ZebraDb to commit_new_chain, commit_block, and no_duplicates_in_finalized_chain instead of FinalizedState * Update zebra-state/src/service/write.rs Co-authored-by: teor <teor@riseup.net> * updates comments, renames send_process_queued, other minor cleanup * update assert_block_can_be_validated comment * removes `mem` field from StateService * removes `disk` field from StateService and updates block_iter to use `ZebraDb` instead of the finalized state * updates tests that use the disk to use read_service.db instead * moves best_tip to a read fn and returns finalized & non-finalized states from setup instead of the state service * changes `contextual_validity` to get the network from the finalized_state instead of another param * swaps out StateService with FinalizedState and NonFinalizedState in tests * adds NotReadyToBeCommitted error and returns it from validate_and_commit when a blocks parent hash is not in any chain * removes NonFinalizedWriteCmd and calls, moves update_latest_channels above rsp_tx.send * makes parent_errors_map an indexmap * clears non-finalized block queue when the receiver is dropped and when the StateService is being dropped * sends non-finalized blocks to the block write task * passes ZebraDb to commit_new_chain, commit_block, and no_duplicates_in_finalized_chain instead of FinalizedState * updates comments, renames send_process_queued, other minor cleanup * Update zebra-state/src/service/write.rs Co-authored-by: teor <teor@riseup.net> * update assert_block_can_be_validated comment * removes `mem` field from StateService * removes `disk` field from StateService and updates block_iter to use `ZebraDb` instead of the finalized state * updates tests that use the disk to use read_service.db instead * moves best_tip to a read fn and returns finalized & non-finalized states from setup instead of the state service * changes `contextual_validity` to get the network from the finalized_state instead of another param * swaps out StateService with FinalizedState and NonFinalizedState in tests * adds NotReadyToBeCommitted error and returns it from validate_and_commit when a blocks parent hash is not in any chain * removes NonFinalizedWriteCmd and calls, moves update_latest_channels above rsp_tx.send * makes parent_errors_map an indexmap * clears non-finalized block queue when the receiver is dropped and when the StateService is being dropped * removes duplicate field definitions on StateService that were a result of a bad merge * update NotReadyToBeCommitted error message * Appear rustfmt * Fix doc links * Rename a function to initial_contextual_validity() * Do error tasks on Err, and success tasks on Ok * Simplify parent_error_map truncation * Rewrite best_tip() to use tip() * Rename latest_mem() to latest_non_finalized_state() ```sh fastmod latest_mem latest_non_finalized_state zebra* cargo fmt --all ``` * Simplify latest_non_finalized_state() using a new WatchReceiver API * Expand some error messages * Send the result after updating the channels, and document why * wait for chain_tip_update before cancelling download in mempool_cancel_mined * adds `sent_non_finalized_block_hashes` field to StateService * adds batched sent_hash insertions and checks sent hashes in queue_and_commit_non_finalized before adding a block to the queue * check that the `curr_buf` in SentHashes is not empty before pushing it to the `sent_bufs` * Apply suggestions from code review Co-authored-by: teor <teor@riseup.net> * Fix rustfmt * Check for finalized block heights using zs_contains() * adds known_utxos field to SentHashes * updates comment on SentHashes.add method * Apply suggestions from code review Co-authored-by: teor <teor@riseup.net> * return early when there's a duplicate hash in QueuedBlocks.queue instead of panicking * Make finalized UTXOs near the final checkpoint available for full block verification * Replace a checkpoint height literal with the actual config * Update mainnet and testnet checkpoints - 7 October 2022 * Fix some state service init arguments * Allow more lookahead in the downloader, but less lookahead in the syncer * Add the latest config to the tests, and fix the latest config check * Increase the number of finalized blocks checked for non-finalized block UTXO spends * fix(log): reduce verbose logs for block commits (#5348) * Remove some verbose block write channel logs * Only warn about tracing endpoint if the address is actually set * Use CloneError instead of formatting a non-cloneable error Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> * Increase block verify timeout * Work around a known block timeout bug by using a shorter timeout Co-authored-by: teor <teor@riseup.net> Co-authored-by: Marek <mail@marek.onl> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com>
2022-10-11 12:25:45 -07:00
/// A shorter timeout used for the first few blocks after the final checkpoint.
///
/// This is a workaround for bug #5125, where the first fully validated blocks
/// after the final checkpoint fail with a timeout, due to a UTXO race condition.
const FINAL_CHECKPOINT_BLOCK_VERIFY_TIMEOUT: Duration = Duration::from_secs(2 * 60);
change(state): Write non-finalized blocks to the state in a separate thread, to avoid network and RPC hangs (#5257) * Add a new block commit task and channels, that don't do anything yet * Add last_block_hash_sent to the state service, to avoid database accesses * Update last_block_hash_sent regardless of commit errors * Rename a field to StateService.max_queued_finalized_height * Commit finalized blocks to the state in a separate task * Check for panics in the block write task * Wait for the block commit task in tests, and check for errors * Always run a proptest that sleeps once * Add extra debugging to state shutdowns * Work around a RocksDB shutdown bug * Close the finalized block channel when we're finished with it * Only reset state queue once per error * Update some TODOs * Add a module doc comment * Drop channels and check for closed channels in the block commit task * Close state channels and tasks on drop * Remove some duplicate fields across StateService and ReadStateService * Try tweaking the shutdown steps * Update and clarify some comments * Clarify another comment * Don't try to cancel RocksDB background work on drop * Fix up some comments * Remove some duplicate code * Remove redundant workarounds for shutdown issues * Remode a redundant channel close in the block commit task * Remove a mistaken `!force` shutdown condition * Remove duplicate force-shutdown code and explain it better * Improve RPC error logging * Wait for chain tip updates in the RPC tests * Wait 2 seconds for chain tip updates before skipping them * Remove an unnecessary block_in_place() * Fix some test error messages that were changed by earlier fixes * Expand some comments, fix typos Co-authored-by: Marek <mail@marek.onl> * Actually drop children of failed blocks * Explain why we drop descendants of failed blocks * Clarify a comment * Wait for chain tip updates in a failing test on macOS * Clean duplicate finalized blocks when the non-finalized state activates * Send an error when receiving a duplicate finalized block * Update checkpoint block behaviour, document its consensus rule * Wait for chain tip changes in inbound_block_height_lookahead_limit test * Wait for the genesis block to commit in the fake peer set mempool tests * Disable unreliable mempool verification check in the send transaction test * Appease rustfmt * Use clear_finalized_block_queue() everywhere that blocks are dropped * Document how Finalized and NonFinalized clones are different * sends non-finalized blocks to the block write task * passes ZebraDb to commit_new_chain, commit_block, and no_duplicates_in_finalized_chain instead of FinalizedState * Update zebra-state/src/service/write.rs Co-authored-by: teor <teor@riseup.net> * updates comments, renames send_process_queued, other minor cleanup * update assert_block_can_be_validated comment * removes `mem` field from StateService * removes `disk` field from StateService and updates block_iter to use `ZebraDb` instead of the finalized state * updates tests that use the disk to use read_service.db instead * moves best_tip to a read fn and returns finalized & non-finalized states from setup instead of the state service * changes `contextual_validity` to get the network from the finalized_state instead of another param * swaps out StateService with FinalizedState and NonFinalizedState in tests * adds NotReadyToBeCommitted error and returns it from validate_and_commit when a blocks parent hash is not in any chain * removes NonFinalizedWriteCmd and calls, moves update_latest_channels above rsp_tx.send * makes parent_errors_map an indexmap * clears non-finalized block queue when the receiver is dropped and when the StateService is being dropped * sends non-finalized blocks to the block write task * passes ZebraDb to commit_new_chain, commit_block, and no_duplicates_in_finalized_chain instead of FinalizedState * updates comments, renames send_process_queued, other minor cleanup * Update zebra-state/src/service/write.rs Co-authored-by: teor <teor@riseup.net> * update assert_block_can_be_validated comment * removes `mem` field from StateService * removes `disk` field from StateService and updates block_iter to use `ZebraDb` instead of the finalized state * updates tests that use the disk to use read_service.db instead * moves best_tip to a read fn and returns finalized & non-finalized states from setup instead of the state service * changes `contextual_validity` to get the network from the finalized_state instead of another param * swaps out StateService with FinalizedState and NonFinalizedState in tests * adds NotReadyToBeCommitted error and returns it from validate_and_commit when a blocks parent hash is not in any chain * removes NonFinalizedWriteCmd and calls, moves update_latest_channels above rsp_tx.send * makes parent_errors_map an indexmap * clears non-finalized block queue when the receiver is dropped and when the StateService is being dropped * removes duplicate field definitions on StateService that were a result of a bad merge * update NotReadyToBeCommitted error message * Appear rustfmt * Fix doc links * Rename a function to initial_contextual_validity() * Do error tasks on Err, and success tasks on Ok * Simplify parent_error_map truncation * Rewrite best_tip() to use tip() * Rename latest_mem() to latest_non_finalized_state() ```sh fastmod latest_mem latest_non_finalized_state zebra* cargo fmt --all ``` * Simplify latest_non_finalized_state() using a new WatchReceiver API * Expand some error messages * Send the result after updating the channels, and document why * wait for chain_tip_update before cancelling download in mempool_cancel_mined * adds `sent_non_finalized_block_hashes` field to StateService * adds batched sent_hash insertions and checks sent hashes in queue_and_commit_non_finalized before adding a block to the queue * check that the `curr_buf` in SentHashes is not empty before pushing it to the `sent_bufs` * Apply suggestions from code review Co-authored-by: teor <teor@riseup.net> * Fix rustfmt * Check for finalized block heights using zs_contains() * adds known_utxos field to SentHashes * updates comment on SentHashes.add method * Apply suggestions from code review Co-authored-by: teor <teor@riseup.net> * return early when there's a duplicate hash in QueuedBlocks.queue instead of panicking * Make finalized UTXOs near the final checkpoint available for full block verification * Replace a checkpoint height literal with the actual config * Update mainnet and testnet checkpoints - 7 October 2022 * Fix some state service init arguments * Allow more lookahead in the downloader, but less lookahead in the syncer * Add the latest config to the tests, and fix the latest config check * Increase the number of finalized blocks checked for non-finalized block UTXO spends * fix(log): reduce verbose logs for block commits (#5348) * Remove some verbose block write channel logs * Only warn about tracing endpoint if the address is actually set * Use CloneError instead of formatting a non-cloneable error Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> * Increase block verify timeout * Work around a known block timeout bug by using a shorter timeout Co-authored-by: teor <teor@riseup.net> Co-authored-by: Marek <mail@marek.onl> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com>
2022-10-11 12:25:45 -07:00
/// The number of blocks after the final checkpoint that get the shorter timeout.
///
/// We've only seen this error on the first few blocks after the final checkpoint.
const FINAL_CHECKPOINT_BLOCK_VERIFY_TIMEOUT_LIMIT: HeightDiff = 100;
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.
///
/// If this timeout is too short, old or buggy nodes will keep making useless
/// network requests. If there are a lot of them, it could overwhelm the network.
const GENESIS_TIMEOUT_RETRY: Duration = Duration::from_secs(10);
/// Sync configuration section.
#[derive(Clone, Debug, Eq, PartialEq, Deserialize, Serialize)]
#[serde(deny_unknown_fields, default)]
pub struct Config {
/// The number of parallel block download requests.
///
/// This is set to a low value by default, to avoid task and
/// network contention. Increasing this value may improve
/// performance on machines with a fast network connection.
#[serde(alias = "max_concurrent_block_requests")]
pub download_concurrency_limit: usize,
/// The number of blocks submitted in parallel to the checkpoint verifier.
///
/// Increasing this limit increases the buffer size, so it reduces
/// the impact of an individual block request failing. However, it
/// also increases memory and CPU usage if block validation stalls,
/// or there are some large blocks in the pipeline.
///
/// The block size limit is 2MB, so in theory, this could represent multiple
/// gigabytes of data, if we downloaded arbitrary blocks. However,
/// because we randomly load balance outbound requests, and separate
/// block download from obtaining block hashes, an adversary would
/// have to control a significant fraction of our peers to lead us
/// astray.
///
/// For reliable checkpoint syncing, Zebra enforces a
/// [`MIN_CHECKPOINT_CONCURRENCY_LIMIT`].
///
/// This is set to a high value by default, to avoid verification pipeline stalls.
/// Decreasing this value reduces RAM usage.
#[serde(alias = "lookahead_limit")]
pub checkpoint_verify_concurrency_limit: usize,
/// The number of blocks submitted in parallel to the full verifier.
///
/// This is set to a low value by default, to avoid verification timeouts on large blocks.
/// Increasing this value may improve performance on machines with many cores.
pub full_verify_concurrency_limit: usize,
/// The number of threads used to verify signatures, proofs, and other CPU-intensive code.
///
feat(mine): Add an internal Zcash miner to Zebra (#8136) * Patch equihash to use the solver branch * Add an internal-miner feature and set up its dependencies * Remove 'Experimental' from mining RPC docs * Fix a nightly clippy::question_mark lint * Move a byte array utility function to zebra-chain * fixup! Add an internal-miner feature and set up its dependencies * Add an equihash::Solution::solve() method with difficulty checks * Check solution is valid before returning it * Add a TODO to check for peers before mining * Move config validation into GetBlockTemplateRpcImpl::new() * fixup! fixup! Add an internal-miner feature and set up its dependencies * Use the same generic constraints for GetBlockTemplateRpcImpl struct and impls * Start adding an internal miner component * Add the miner task to the start command * Add basic miner code * Split out a method to mine one block * Spawn to a blocking thread * Wait until a valid template is available * Handle shutdown * Run mining on low priority threads * Ignore some invalid solutions * Use a difference nonce for each solver thread * Update TODOs * Change the patch into a renamed dependency to simplify crate releases * Clean up instrumentation and TODOs * Make RPC instances cloneable and clean up generics * Make LongPollId Copy so it's easier to use * Add API to restart mining if there's a new block template * Actually restart mining if there's a new block template * Tidy instrumentation * fixup! Move config validation into GetBlockTemplateRpcImpl::new() * fixup! Make RPC instances cloneable and clean up generics * Run the template generator and one miner concurrently * Reduce logging * Fix a bug in getblocktemplate RPC tip change detection * Work around some watch channel change bugs * Rate-limit template changes in the receiver * Run one mining solver per available core * Use updated C code with double-free protection * Update to the latest solver branch * Return and submit all valid solutions * Document what INPUT_LENGTH means * Fix watch channel change detection * Don't return early when a mining task fails * Spawn async miner tasks to avoid cooperative blocking, deadlocks, and improve shutdown responsiveness * Make existing parallelism docs and configs consistent * Add a mining parallelism config * Use the minimum of the configured or available threads for mining * Ignore optional feature fields in tests * Downgrade some frequent logs to debug * Document new zebrad features and tasks * Describe the internal-miner feature in the CHANGELOG * Update dependency to de-duplicate equihash solutions * Use futures::StreamExt instead of TryStreamExt * Fix a panic message typo
2024-01-11 06:41:01 -08:00
/// If the number of threads is not configured or zero, Zebra uses the number of logical cores.
/// If the number of logical cores can't be detected, Zebra uses one thread.
/// For details, see [the `rayon` documentation](https://docs.rs/rayon/latest/rayon/struct.ThreadPoolBuilder.html#method.num_threads).
pub parallel_cpu_threads: usize,
}
impl Default for Config {
fn default() -> Self {
Self {
// 2/3 of the default outbound peer limit.
download_concurrency_limit: 50,
// A few max-length checkpoints.
checkpoint_verify_concurrency_limit: DEFAULT_CHECKPOINT_CONCURRENCY_LIMIT,
// This default is deliberately very low, so Zebra can verify a few large blocks in under 60 seconds,
// even on machines with only a few cores.
//
// This lets users see the committed block height changing in every progress log,
// and avoids hangs due to out-of-order verifications flooding the CPUs.
//
// TODO:
// - limit full verification concurrency based on block transaction counts?
// - move more disk work to blocking tokio threads,
// and CPU work to the rayon thread pool inside blocking tokio threads
full_verify_concurrency_limit: 20,
// Use one thread per CPU.
//
// If this causes tokio executor starvation, move CPU-intensive tasks to rayon threads,
// or reserve a few cores for tokio threads, based on `num_cpus()`.
parallel_cpu_threads: 0,
}
}
}
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,
change(rpc): Add proposal capability to getblocktemplate (#5870) * adds ValidateBlock request to state * adds `Request` enum in block verifier skips solution check for BlockProposal requests calls CheckBlockValidity instead of Commit block for BlockProposal requests * uses new Request in references to chain verifier * adds getblocktemplate proposal mode response type * makes getblocktemplate-rpcs feature in zebra-consensus select getblocktemplate-rpcs in zebra-state * Adds PR review revisions * adds info log in CheckBlockProposalValidity * Reverts replacement of match statement * adds `GetBlockTemplate::capabilities` fn * conditions calling checkpoint verifier on !request.is_proposal * updates references to validate_and_commit_non_finalized * adds snapshot test, updates test vectors * adds `should_count_metrics` to NonFinalizedState * Returns an error from chain verifier for block proposal requests below checkpoint height adds feature flags * adds "proposal" to GET_BLOCK_TEMPLATE_CAPABILITIES_FIELD * adds back block::Request to zebra-consensus lib * updates snapshots * Removes unnecessary network arg * skips req in tracing intstrument for read state * Moves out block proposal validation to its own fn * corrects `difficulty_threshold_is_valid` docs adds/fixes some comments, adds TODOs general cleanup from a self-review. * Update zebra-state/src/service.rs * Apply suggestions from code review Co-authored-by: teor <teor@riseup.net> * Update zebra-rpc/src/methods/get_block_template_rpcs.rs Co-authored-by: teor <teor@riseup.net> * check best chain tip * Update zebra-state/src/service.rs Co-authored-by: teor <teor@riseup.net> * Applies cleanup suggestions from code review Co-authored-by: teor <teor@riseup.net> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com>
2023-01-11 15:39:51 -08:00
ZV: Service<zebra_consensus::Request, 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 largest block height for the checkpoint verifier, based on the current config.
max_checkpoint_height: Height,
/// The configured checkpoint verification concurrency limit, after applying the minimum limit.
checkpoint_verify_concurrency_limit: usize,
/// The configured full verification concurrency limit, after applying the minimum limit.
full_verify_concurrency_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,
/// Receiver that is `true` when the downloader is past the lookahead limit.
/// This is based on the downloaded block height and the state tip height.
past_lookahead_limit_receiver: zs::WatchReceiver<bool>,
}
/// 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,
change(rpc): Add proposal capability to getblocktemplate (#5870) * adds ValidateBlock request to state * adds `Request` enum in block verifier skips solution check for BlockProposal requests calls CheckBlockValidity instead of Commit block for BlockProposal requests * uses new Request in references to chain verifier * adds getblocktemplate proposal mode response type * makes getblocktemplate-rpcs feature in zebra-consensus select getblocktemplate-rpcs in zebra-state * Adds PR review revisions * adds info log in CheckBlockProposalValidity * Reverts replacement of match statement * adds `GetBlockTemplate::capabilities` fn * conditions calling checkpoint verifier on !request.is_proposal * updates references to validate_and_commit_non_finalized * adds snapshot test, updates test vectors * adds `should_count_metrics` to NonFinalizedState * Returns an error from chain verifier for block proposal requests below checkpoint height adds feature flags * adds "proposal" to GET_BLOCK_TEMPLATE_CAPABILITIES_FIELD * adds back block::Request to zebra-consensus lib * updates snapshots * Removes unnecessary network arg * skips req in tracing intstrument for read state * Moves out block proposal validation to its own fn * corrects `difficulty_threshold_is_valid` docs adds/fixes some comments, adds TODOs general cleanup from a self-review. * Update zebra-state/src/service.rs * Apply suggestions from code review Co-authored-by: teor <teor@riseup.net> * Update zebra-rpc/src/methods/get_block_template_rpcs.rs Co-authored-by: teor <teor@riseup.net> * check best chain tip * Update zebra-state/src/service.rs Co-authored-by: teor <teor@riseup.net> * Applies cleanup suggestions from code review Co-authored-by: teor <teor@riseup.net> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com>
2023-01-11 15:39:51 -08:00
ZV: Service<zebra_consensus::Request, 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,
max_checkpoint_height: Height,
peers: ZN,
verifier: ZV,
state: ZS,
latest_chain_tip: ZSTip,
) -> (Self, SyncStatus) {
let mut download_concurrency_limit = config.sync.download_concurrency_limit;
let mut checkpoint_verify_concurrency_limit =
config.sync.checkpoint_verify_concurrency_limit;
let mut full_verify_concurrency_limit = config.sync.full_verify_concurrency_limit;
if download_concurrency_limit < MIN_CONCURRENCY_LIMIT {
warn!(
"configured download concurrency limit {} too low, increasing to {}",
config.sync.download_concurrency_limit, MIN_CONCURRENCY_LIMIT,
);
download_concurrency_limit = MIN_CONCURRENCY_LIMIT;
}
if checkpoint_verify_concurrency_limit < MIN_CHECKPOINT_CONCURRENCY_LIMIT {
warn!(
"configured checkpoint verify concurrency limit {} too low, increasing to {}",
config.sync.checkpoint_verify_concurrency_limit, MIN_CHECKPOINT_CONCURRENCY_LIMIT,
);
checkpoint_verify_concurrency_limit = MIN_CHECKPOINT_CONCURRENCY_LIMIT;
}
if full_verify_concurrency_limit < MIN_CONCURRENCY_LIMIT {
warn!(
"configured full verify concurrency limit {} too low, increasing to {}",
config.sync.full_verify_concurrency_limit, MIN_CONCURRENCY_LIMIT,
);
full_verify_concurrency_limit = MIN_CONCURRENCY_LIMIT;
}
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.
let block_network = Hedge::new(
ServiceBuilder::new()
.concurrency_limit(download_concurrency_limit)
.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);
let (sync_status, recent_syncs) = SyncStatus::new();
let (past_lookahead_limit_sender, past_lookahead_limit_receiver) = watch::channel(false);
let past_lookahead_limit_receiver = zs::WatchReceiver::new(past_lookahead_limit_receiver);
let downloads = Box::pin(Downloads::new(
block_network,
verifier,
latest_chain_tip.clone(),
past_lookahead_limit_sender,
max(
checkpoint_verify_concurrency_limit,
full_verify_concurrency_limit,
),
max_checkpoint_height,
));
let new_syncer = Self {
genesis_hash: genesis_hash(config.network.network),
max_checkpoint_height,
checkpoint_verify_concurrency_limit,
full_verify_concurrency_limit,
tip_network,
downloads,
state,
latest_chain_tip,
prospective_tips: HashSet::new(),
recent_syncs,
past_lookahead_limit_receiver,
};
(new_syncer, sync_status)
}
/// Runs the syncer to synchronize the chain and keep it synchronized.
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?;
loop {
if self.try_to_sync().await.is_err() {
self.downloads.cancel_all();
}
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();
info!(
timeout = ?SYNC_RESTART_DELAY,
state_tip = ?self.latest_chain_tip.best_tip_height(),
"waiting to restart sync"
);
sleep(SYNC_RESTART_DELAY).await;
}
}
/// Tries to synchronize the chain as far as it can.
///
/// Obtains some prospective tips and iteratively tries to extend them and download the missing
/// blocks.
///
/// Returns `Ok` if it was able to synchronize as much of the chain as it could, and then ran
/// out of prospective tips. This happens when synchronization finishes or if Zebra ended up
/// following a fork. Either way, Zebra should attempt to obtain some more tips.
///
/// Returns `Err` if there was an unrecoverable error and restarting the synchronization is
/// necessary. This includes outer timeouts, where an entire syncing step takes an extremely
/// long time. (These usually indicate hangs.)
#[instrument(skip(self))]
async fn try_to_sync(&mut self) -> Result<(), Report> {
self.prospective_tips = HashSet::new();
info!(
state_tip = ?self.latest_chain_tip.best_tip_height(),
"starting sync, obtaining new tips"
);
let mut extra_hashes = timeout(SYNC_RESTART_DELAY, self.obtain_tips())
.await
.map_err(Into::into)
// TODO: replace with flatten() when it stabilises (#70142)
.and_then(convert::identity)
.map_err(|e| {
info!("temporary error obtaining tips: {:#}", e);
e
})?;
self.update_metrics();
while !self.prospective_tips.is_empty() || !extra_hashes.is_empty() {
// Avoid hangs due to service readiness or other internal operations
extra_hashes = timeout(BLOCK_VERIFY_TIMEOUT, self.try_to_sync_once(extra_hashes))
.await
.map_err(Into::into)
// TODO: replace with flatten() when it stabilises (#70142)
.and_then(convert::identity)?;
}
info!("exhausted prospective tip set");
Ok(())
}
/// Tries to synchronize the chain once, using the existing `extra_hashes`.
///
/// Tries to extend the existing tips and download the missing blocks.
///
/// Returns `Ok(extra_hashes)` if it was able to extend once and synchronize sone of the chain.
/// Returns `Err` if there was an unrecoverable error and restarting the synchronization is
/// necessary.
#[instrument(skip(self))]
async fn try_to_sync_once(
&mut self,
mut extra_hashes: IndexSet<block::Hash>,
) -> Result<IndexSet<block::Hash>, Report> {
// Check whether any block tasks are currently ready.
while let Poll::Ready(Some(rsp)) = futures::poll!(self.downloads.next()) {
// Some temporary errors are ignored, and syncing continues with other blocks.
// If it turns out they were actually important, syncing will run out of blocks, and
// the syncer will reset itself.
self.handle_block_response(rsp)?;
}
self.update_metrics();
// Pause new downloads while the syncer or downloader are past their lookahead limits.
//
// To avoid a deadlock or long waits for blocks to expire, we ignore the download
// lookahead limit when there are only a small number of blocks waiting.
while self.downloads.in_flight() >= self.lookahead_limit(extra_hashes.len())
|| (self.downloads.in_flight() >= self.lookahead_limit(extra_hashes.len()) / 2
&& self.past_lookahead_limit_receiver.cloned_watch_data())
{
trace!(
tips.len = self.prospective_tips.len(),
in_flight = self.downloads.in_flight(),
extra_hashes = extra_hashes.len(),
lookahead_limit = self.lookahead_limit(extra_hashes.len()),
state_tip = ?self.latest_chain_tip.best_tip_height(),
"waiting for pending blocks",
);
let response = self.downloads.next().await.expect("downloads is nonempty");
self.handle_block_response(response)?;
self.update_metrics();
}
// Once we're below the lookahead limit, we can request more blocks or hashes.
if !extra_hashes.is_empty() {
debug!(
tips.len = self.prospective_tips.len(),
in_flight = self.downloads.in_flight(),
extra_hashes = extra_hashes.len(),
lookahead_limit = self.lookahead_limit(extra_hashes.len()),
state_tip = ?self.latest_chain_tip.best_tip_height(),
"requesting more blocks",
);
let response = self.request_blocks(extra_hashes).await;
extra_hashes = Self::handle_hash_response(response)?;
} else {
info!(
tips.len = self.prospective_tips.len(),
in_flight = self.downloads.in_flight(),
extra_hashes = extra_hashes.len(),
lookahead_limit = self.lookahead_limit(extra_hashes.len()),
state_tip = ?self.latest_chain_tip.best_tip_height(),
"extending tips",
);
extra_hashes = self.extend_tips().await.map_err(|e| {
info!("temporary error extending tips: {:#}", e);
e
})?;
}
self.update_metrics();
Ok(extra_hashes)
}
/// 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<IndexSet<block::Hash>, 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
.unwrap_or_else(|e @ JoinError { .. }| {
if e.is_panic() {
panic!("panic in obtain tips task: {e:?}");
} else {
info!(
"task error during obtain tips task: {e:?},\
is Zebra shutting down?"
);
Err(e.into())
}
})
.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");
build(deps): bump the prod group with 6 updates (#8125) * build(deps): bump the prod group with 6 updates Bumps the prod group with 6 updates: | Package | From | To | | --- | --- | --- | | [futures](https://github.com/rust-lang/futures-rs) | `0.3.29` | `0.3.30` | | [tokio](https://github.com/tokio-rs/tokio) | `1.35.0` | `1.35.1` | | [metrics](https://github.com/metrics-rs/metrics) | `0.21.1` | `0.22.0` | | [metrics-exporter-prometheus](https://github.com/metrics-rs/metrics) | `0.12.2` | `0.13.0` | | [reqwest](https://github.com/seanmonstar/reqwest) | `0.11.22` | `0.11.23` | | [owo-colors](https://github.com/jam1garner/owo-colors) | `3.5.0` | `4.0.0` | Updates `futures` from 0.3.29 to 0.3.30 - [Release notes](https://github.com/rust-lang/futures-rs/releases) - [Changelog](https://github.com/rust-lang/futures-rs/blob/master/CHANGELOG.md) - [Commits](https://github.com/rust-lang/futures-rs/compare/0.3.29...0.3.30) Updates `tokio` from 1.35.0 to 1.35.1 - [Release notes](https://github.com/tokio-rs/tokio/releases) - [Commits](https://github.com/tokio-rs/tokio/compare/tokio-1.35.0...tokio-1.35.1) Updates `metrics` from 0.21.1 to 0.22.0 - [Changelog](https://github.com/metrics-rs/metrics/blob/main/release.toml) - [Commits](https://github.com/metrics-rs/metrics/compare/metrics-v0.21.1...metrics-v0.22.0) Updates `metrics-exporter-prometheus` from 0.12.2 to 0.13.0 - [Changelog](https://github.com/metrics-rs/metrics/blob/main/release.toml) - [Commits](https://github.com/metrics-rs/metrics/compare/metrics-exporter-prometheus-v0.12.2...metrics-exporter-prometheus-v0.13.0) Updates `reqwest` from 0.11.22 to 0.11.23 - [Release notes](https://github.com/seanmonstar/reqwest/releases) - [Changelog](https://github.com/seanmonstar/reqwest/blob/master/CHANGELOG.md) - [Commits](https://github.com/seanmonstar/reqwest/compare/v0.11.22...v0.11.23) Updates `owo-colors` from 3.5.0 to 4.0.0 - [Commits](https://github.com/jam1garner/owo-colors/compare/v3.5.0...v4.0.0) --- updated-dependencies: - dependency-name: futures dependency-type: direct:production update-type: version-update:semver-patch dependency-group: prod - dependency-name: tokio dependency-type: direct:production update-type: version-update:semver-patch dependency-group: prod - dependency-name: metrics dependency-type: direct:production update-type: version-update:semver-minor dependency-group: prod - dependency-name: metrics-exporter-prometheus dependency-type: direct:production update-type: version-update:semver-minor dependency-group: prod - dependency-name: reqwest dependency-type: direct:production update-type: version-update:semver-patch dependency-group: prod - dependency-name: owo-colors dependency-type: direct:production update-type: version-update:semver-major dependency-group: prod ... Signed-off-by: dependabot[bot] <support@github.com> * update all metric macros * fix deprecated function * fix duplicated deps * Fix an incorrect gauge method call * Expand documentation and error messages for best chain length --------- Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com> Co-authored-by: teor <teor@riseup.net>
2024-01-01 17:26:54 -08:00
metrics::histogram!("sync.obtain.response.hash.count")
.record(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");
build(deps): bump the prod group with 6 updates (#8125) * build(deps): bump the prod group with 6 updates Bumps the prod group with 6 updates: | Package | From | To | | --- | --- | --- | | [futures](https://github.com/rust-lang/futures-rs) | `0.3.29` | `0.3.30` | | [tokio](https://github.com/tokio-rs/tokio) | `1.35.0` | `1.35.1` | | [metrics](https://github.com/metrics-rs/metrics) | `0.21.1` | `0.22.0` | | [metrics-exporter-prometheus](https://github.com/metrics-rs/metrics) | `0.12.2` | `0.13.0` | | [reqwest](https://github.com/seanmonstar/reqwest) | `0.11.22` | `0.11.23` | | [owo-colors](https://github.com/jam1garner/owo-colors) | `3.5.0` | `4.0.0` | Updates `futures` from 0.3.29 to 0.3.30 - [Release notes](https://github.com/rust-lang/futures-rs/releases) - [Changelog](https://github.com/rust-lang/futures-rs/blob/master/CHANGELOG.md) - [Commits](https://github.com/rust-lang/futures-rs/compare/0.3.29...0.3.30) Updates `tokio` from 1.35.0 to 1.35.1 - [Release notes](https://github.com/tokio-rs/tokio/releases) - [Commits](https://github.com/tokio-rs/tokio/compare/tokio-1.35.0...tokio-1.35.1) Updates `metrics` from 0.21.1 to 0.22.0 - [Changelog](https://github.com/metrics-rs/metrics/blob/main/release.toml) - [Commits](https://github.com/metrics-rs/metrics/compare/metrics-v0.21.1...metrics-v0.22.0) Updates `metrics-exporter-prometheus` from 0.12.2 to 0.13.0 - [Changelog](https://github.com/metrics-rs/metrics/blob/main/release.toml) - [Commits](https://github.com/metrics-rs/metrics/compare/metrics-exporter-prometheus-v0.12.2...metrics-exporter-prometheus-v0.13.0) Updates `reqwest` from 0.11.22 to 0.11.23 - [Release notes](https://github.com/seanmonstar/reqwest/releases) - [Changelog](https://github.com/seanmonstar/reqwest/blob/master/CHANGELOG.md) - [Commits](https://github.com/seanmonstar/reqwest/compare/v0.11.22...v0.11.23) Updates `owo-colors` from 3.5.0 to 4.0.0 - [Commits](https://github.com/jam1garner/owo-colors/compare/v3.5.0...v4.0.0) --- updated-dependencies: - dependency-name: futures dependency-type: direct:production update-type: version-update:semver-patch dependency-group: prod - dependency-name: tokio dependency-type: direct:production update-type: version-update:semver-patch dependency-group: prod - dependency-name: metrics dependency-type: direct:production update-type: version-update:semver-minor dependency-group: prod - dependency-name: metrics-exporter-prometheus dependency-type: direct:production update-type: version-update:semver-minor dependency-group: prod - dependency-name: reqwest dependency-type: direct:production update-type: version-update:semver-patch dependency-group: prod - dependency-name: owo-colors dependency-type: direct:production update-type: version-update:semver-major dependency-group: prod ... Signed-off-by: dependabot[bot] <support@github.com> * update all metric macros * fix deprecated function * fix duplicated deps * Fix an incorrect gauge method call * Expand documentation and error messages for best chain length --------- Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com> Co-authored-by: teor <teor@riseup.net>
2024-01-01 17:26:54 -08:00
metrics::gauge!("sync.obtain.queued.hash.count").set(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);
let response = self.request_blocks(download_set).await;
Self::handle_hash_response(response).map_err(Into::into)
}
2020-07-08 13:33:39 -07:00
#[instrument(skip(self))]
async fn extend_tips(&mut self) -> Result<IndexSet<block::Hash>, 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");
build(deps): bump the prod group with 6 updates (#8125) * build(deps): bump the prod group with 6 updates Bumps the prod group with 6 updates: | Package | From | To | | --- | --- | --- | | [futures](https://github.com/rust-lang/futures-rs) | `0.3.29` | `0.3.30` | | [tokio](https://github.com/tokio-rs/tokio) | `1.35.0` | `1.35.1` | | [metrics](https://github.com/metrics-rs/metrics) | `0.21.1` | `0.22.0` | | [metrics-exporter-prometheus](https://github.com/metrics-rs/metrics) | `0.12.2` | `0.13.0` | | [reqwest](https://github.com/seanmonstar/reqwest) | `0.11.22` | `0.11.23` | | [owo-colors](https://github.com/jam1garner/owo-colors) | `3.5.0` | `4.0.0` | Updates `futures` from 0.3.29 to 0.3.30 - [Release notes](https://github.com/rust-lang/futures-rs/releases) - [Changelog](https://github.com/rust-lang/futures-rs/blob/master/CHANGELOG.md) - [Commits](https://github.com/rust-lang/futures-rs/compare/0.3.29...0.3.30) Updates `tokio` from 1.35.0 to 1.35.1 - [Release notes](https://github.com/tokio-rs/tokio/releases) - [Commits](https://github.com/tokio-rs/tokio/compare/tokio-1.35.0...tokio-1.35.1) Updates `metrics` from 0.21.1 to 0.22.0 - [Changelog](https://github.com/metrics-rs/metrics/blob/main/release.toml) - [Commits](https://github.com/metrics-rs/metrics/compare/metrics-v0.21.1...metrics-v0.22.0) Updates `metrics-exporter-prometheus` from 0.12.2 to 0.13.0 - [Changelog](https://github.com/metrics-rs/metrics/blob/main/release.toml) - [Commits](https://github.com/metrics-rs/metrics/compare/metrics-exporter-prometheus-v0.12.2...metrics-exporter-prometheus-v0.13.0) Updates `reqwest` from 0.11.22 to 0.11.23 - [Release notes](https://github.com/seanmonstar/reqwest/releases) - [Changelog](https://github.com/seanmonstar/reqwest/blob/master/CHANGELOG.md) - [Commits](https://github.com/seanmonstar/reqwest/compare/v0.11.22...v0.11.23) Updates `owo-colors` from 3.5.0 to 4.0.0 - [Commits](https://github.com/jam1garner/owo-colors/compare/v3.5.0...v4.0.0) --- updated-dependencies: - dependency-name: futures dependency-type: direct:production update-type: version-update:semver-patch dependency-group: prod - dependency-name: tokio dependency-type: direct:production update-type: version-update:semver-patch dependency-group: prod - dependency-name: metrics dependency-type: direct:production update-type: version-update:semver-minor dependency-group: prod - dependency-name: metrics-exporter-prometheus dependency-type: direct:production update-type: version-update:semver-minor dependency-group: prod - dependency-name: reqwest dependency-type: direct:production update-type: version-update:semver-patch dependency-group: prod - dependency-name: owo-colors dependency-type: direct:production update-type: version-update:semver-major dependency-group: prod ... Signed-off-by: dependabot[bot] <support@github.com> * update all metric macros * fix deprecated function * fix duplicated deps * Fix an incorrect gauge method call * Expand documentation and error messages for best chain length --------- Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com> Co-authored-by: teor <teor@riseup.net>
2024-01-01 17:26:54 -08:00
metrics::histogram!("sync.extend.response.hash.count")
.record(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");
build(deps): bump the prod group with 6 updates (#8125) * build(deps): bump the prod group with 6 updates Bumps the prod group with 6 updates: | Package | From | To | | --- | --- | --- | | [futures](https://github.com/rust-lang/futures-rs) | `0.3.29` | `0.3.30` | | [tokio](https://github.com/tokio-rs/tokio) | `1.35.0` | `1.35.1` | | [metrics](https://github.com/metrics-rs/metrics) | `0.21.1` | `0.22.0` | | [metrics-exporter-prometheus](https://github.com/metrics-rs/metrics) | `0.12.2` | `0.13.0` | | [reqwest](https://github.com/seanmonstar/reqwest) | `0.11.22` | `0.11.23` | | [owo-colors](https://github.com/jam1garner/owo-colors) | `3.5.0` | `4.0.0` | Updates `futures` from 0.3.29 to 0.3.30 - [Release notes](https://github.com/rust-lang/futures-rs/releases) - [Changelog](https://github.com/rust-lang/futures-rs/blob/master/CHANGELOG.md) - [Commits](https://github.com/rust-lang/futures-rs/compare/0.3.29...0.3.30) Updates `tokio` from 1.35.0 to 1.35.1 - [Release notes](https://github.com/tokio-rs/tokio/releases) - [Commits](https://github.com/tokio-rs/tokio/compare/tokio-1.35.0...tokio-1.35.1) Updates `metrics` from 0.21.1 to 0.22.0 - [Changelog](https://github.com/metrics-rs/metrics/blob/main/release.toml) - [Commits](https://github.com/metrics-rs/metrics/compare/metrics-v0.21.1...metrics-v0.22.0) Updates `metrics-exporter-prometheus` from 0.12.2 to 0.13.0 - [Changelog](https://github.com/metrics-rs/metrics/blob/main/release.toml) - [Commits](https://github.com/metrics-rs/metrics/compare/metrics-exporter-prometheus-v0.12.2...metrics-exporter-prometheus-v0.13.0) Updates `reqwest` from 0.11.22 to 0.11.23 - [Release notes](https://github.com/seanmonstar/reqwest/releases) - [Changelog](https://github.com/seanmonstar/reqwest/blob/master/CHANGELOG.md) - [Commits](https://github.com/seanmonstar/reqwest/compare/v0.11.22...v0.11.23) Updates `owo-colors` from 3.5.0 to 4.0.0 - [Commits](https://github.com/jam1garner/owo-colors/compare/v3.5.0...v4.0.0) --- updated-dependencies: - dependency-name: futures dependency-type: direct:production update-type: version-update:semver-patch dependency-group: prod - dependency-name: tokio dependency-type: direct:production update-type: version-update:semver-patch dependency-group: prod - dependency-name: metrics dependency-type: direct:production update-type: version-update:semver-minor dependency-group: prod - dependency-name: metrics-exporter-prometheus dependency-type: direct:production update-type: version-update:semver-minor dependency-group: prod - dependency-name: reqwest dependency-type: direct:production update-type: version-update:semver-patch dependency-group: prod - dependency-name: owo-colors dependency-type: direct:production update-type: version-update:semver-major dependency-group: prod ... Signed-off-by: dependabot[bot] <support@github.com> * update all metric macros * fix deprecated function * fix duplicated deps * Fix an incorrect gauge method call * Expand documentation and error messages for best chain length --------- Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com> Co-authored-by: teor <teor@riseup.net>
2024-01-01 17:26:54 -08:00
metrics::gauge!("sync.extend.queued.hash.count").set(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);
let response = self.request_blocks(download_set).await;
Self::handle_hash_response(response).map_err(Into::into)
}
/// 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");
let response = timeout(SYNC_RESTART_DELAY, self.request_genesis_once())
.await
.map_err(Into::into);
// 3 layers of results is not ideal, but we need the timeout on the outside.
match response {
Ok(Ok(Ok(response))) => self
.handle_block_response(Ok(response))
.expect("never returns Err for Ok"),
// Handle fatal errors
Ok(Err(fatal_error)) => Err(fatal_error)?,
// Handle timeouts and block errors
Err(error) | Ok(Ok(Err(error))) => {
// TODO: exit syncer on permanent service errors (NetworkError, VerifierError)
if Self::should_restart_sync(&error) {
warn!(
?error,
"could not download or verify genesis block, retrying"
);
} else {
info!(
?error,
"temporary error downloading or verifying 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(())
}
/// Try to download and verify the genesis block once.
///
/// Fatal errors are returned in the outer result, temporary errors in the inner one.
async fn request_genesis_once(
&mut self,
) -> Result<Result<(Height, block::Hash), BlockDownloadVerifyError>, Report> {
let response = self.downloads.download_and_verify(self.genesis_hash).await;
Self::handle_response(response).map_err(|e| eyre!(e))?;
let response = self.downloads.next().await.expect("downloads is nonempty");
Ok(response)
}
/// Queue download and verify tasks for each block that isn't currently known to our node.
///
/// TODO: turn obtain and extend tips into a separate task, which sends hashes via a channel?
async fn request_blocks(
&mut self,
mut hashes: IndexSet<block::Hash>,
) -> Result<IndexSet<block::Hash>, BlockDownloadVerifyError> {
let lookahead_limit = self.lookahead_limit(hashes.len());
debug!(
hashes.len = hashes.len(),
?lookahead_limit,
"requesting blocks",
);
let extra_hashes = if hashes.len() > lookahead_limit {
hashes.split_off(lookahead_limit)
} else {
IndexSet::new()
};
for hash in hashes.into_iter() {
self.downloads.download_and_verify(hash).await?;
}
Ok(extra_hashes)
}
/// The configured lookahead limit, based on the currently verified height,
/// and the number of hashes we haven't queued yet.
fn lookahead_limit(&self, new_hashes: usize) -> usize {
let max_checkpoint_height: usize = self
.max_checkpoint_height
.0
.try_into()
.expect("fits in usize");
// When the state is empty, we want to verify using checkpoints
let verified_height: usize = self
.latest_chain_tip
.best_tip_height()
.unwrap_or(Height(0))
.0
.try_into()
.expect("fits in usize");
if verified_height >= max_checkpoint_height {
self.full_verify_concurrency_limit
} else if (verified_height + new_hashes) >= max_checkpoint_height {
// If we're just about to start full verification, allow enough for the remaining checkpoint,
// and also enough for a separate full verification lookahead.
let checkpoint_hashes = verified_height + new_hashes - max_checkpoint_height;
self.full_verify_concurrency_limit + checkpoint_hashes
} else {
self.checkpoint_verify_concurrency_limit
}
}
2020-08-07 01:04:33 -07:00
/// Handles a response for a requested block.
///
/// See [`Self::handle_response`] for more details.
#[allow(unknown_lints)]
fn handle_block_response(
&mut self,
response: Result<(Height, block::Hash), BlockDownloadVerifyError>,
) -> Result<(), BlockDownloadVerifyError> {
match response {
Ok((height, hash)) => {
trace!(?height, ?hash, "verified and committed block to state");
Ok(())
}
Err(_) => Self::handle_response(response),
}
}
/// Handles a response to block hash submission, passing through any extra hashes.
///
/// See [`Self::handle_response`] for more details.
#[allow(unknown_lints)]
fn handle_hash_response(
response: Result<IndexSet<block::Hash>, BlockDownloadVerifyError>,
) -> Result<IndexSet<block::Hash>, BlockDownloadVerifyError> {
match response {
Ok(extra_hashes) => Ok(extra_hashes),
Err(_) => Self::handle_response(response).map(|()| IndexSet::new()),
}
}
/// Handles a response to a syncer request.
///
/// Returns `Ok` if the request was successful, or if an expected error occurred,
/// so that the synchronization can continue normally.
///
/// Returns `Err` if an unexpected error occurred, to force the synchronizer to restart.
#[allow(unknown_lints)]
fn handle_response<T>(
response: Result<T, BlockDownloadVerifyError>,
) -> Result<(), BlockDownloadVerifyError> {
match response {
Ok(_t) => Ok(()),
Err(error) => {
// TODO: exit syncer on permanent service errors (NetworkError, VerifierError)
if Self::should_restart_sync(&error) {
Err(error)
} else {
Ok(())
}
}
}
}
/// Returns `true` if the hash is present in the state, and `false`
/// if the hash is not present in the state.
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))?
.call(zebra_state::Request::KnownBlock(hash))
.await
.map_err(|e| eyre!(e))?
{
zs::Response::KnownBlock(loc) => Ok(loc.is_some()),
_ => unreachable!("wrong response to known block request"),
}
}
fn update_metrics(&mut self) {
build(deps): bump the prod group with 6 updates (#8125) * build(deps): bump the prod group with 6 updates Bumps the prod group with 6 updates: | Package | From | To | | --- | --- | --- | | [futures](https://github.com/rust-lang/futures-rs) | `0.3.29` | `0.3.30` | | [tokio](https://github.com/tokio-rs/tokio) | `1.35.0` | `1.35.1` | | [metrics](https://github.com/metrics-rs/metrics) | `0.21.1` | `0.22.0` | | [metrics-exporter-prometheus](https://github.com/metrics-rs/metrics) | `0.12.2` | `0.13.0` | | [reqwest](https://github.com/seanmonstar/reqwest) | `0.11.22` | `0.11.23` | | [owo-colors](https://github.com/jam1garner/owo-colors) | `3.5.0` | `4.0.0` | Updates `futures` from 0.3.29 to 0.3.30 - [Release notes](https://github.com/rust-lang/futures-rs/releases) - [Changelog](https://github.com/rust-lang/futures-rs/blob/master/CHANGELOG.md) - [Commits](https://github.com/rust-lang/futures-rs/compare/0.3.29...0.3.30) Updates `tokio` from 1.35.0 to 1.35.1 - [Release notes](https://github.com/tokio-rs/tokio/releases) - [Commits](https://github.com/tokio-rs/tokio/compare/tokio-1.35.0...tokio-1.35.1) Updates `metrics` from 0.21.1 to 0.22.0 - [Changelog](https://github.com/metrics-rs/metrics/blob/main/release.toml) - [Commits](https://github.com/metrics-rs/metrics/compare/metrics-v0.21.1...metrics-v0.22.0) Updates `metrics-exporter-prometheus` from 0.12.2 to 0.13.0 - [Changelog](https://github.com/metrics-rs/metrics/blob/main/release.toml) - [Commits](https://github.com/metrics-rs/metrics/compare/metrics-exporter-prometheus-v0.12.2...metrics-exporter-prometheus-v0.13.0) Updates `reqwest` from 0.11.22 to 0.11.23 - [Release notes](https://github.com/seanmonstar/reqwest/releases) - [Changelog](https://github.com/seanmonstar/reqwest/blob/master/CHANGELOG.md) - [Commits](https://github.com/seanmonstar/reqwest/compare/v0.11.22...v0.11.23) Updates `owo-colors` from 3.5.0 to 4.0.0 - [Commits](https://github.com/jam1garner/owo-colors/compare/v3.5.0...v4.0.0) --- updated-dependencies: - dependency-name: futures dependency-type: direct:production update-type: version-update:semver-patch dependency-group: prod - dependency-name: tokio dependency-type: direct:production update-type: version-update:semver-patch dependency-group: prod - dependency-name: metrics dependency-type: direct:production update-type: version-update:semver-minor dependency-group: prod - dependency-name: metrics-exporter-prometheus dependency-type: direct:production update-type: version-update:semver-minor dependency-group: prod - dependency-name: reqwest dependency-type: direct:production update-type: version-update:semver-patch dependency-group: prod - dependency-name: owo-colors dependency-type: direct:production update-type: version-update:semver-major dependency-group: prod ... Signed-off-by: dependabot[bot] <support@github.com> * update all metric macros * fix deprecated function * fix duplicated deps * Fix an incorrect gauge method call * Expand documentation and error messages for best chain length --------- Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com> Co-authored-by: teor <teor@riseup.net>
2024-01-01 17:26:54 -08:00
metrics::gauge!("sync.prospective_tips.len",).set(self.prospective_tips.len() as f64);
metrics::gauge!("sync.downloads.in_flight",).set(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: downcasts
BlockDownloadVerifyError::Invalid { error, .. } if error.is_duplicate_request() => {
debug!(error = ?e, "block was already verified, possibly from a previous sync run, continuing");
false
}
// Structural matches: direct
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
}
BlockDownloadVerifyError::DuplicateBlockQueuedForDownload { .. } => {
debug!(
error = ?e,
"queued duplicate block hash for download, \
assuming the syncer will eventually resolve duplicates, continuing"
);
false
}
// String matches
//
// We want to match VerifyChainError::Block(VerifyBlockError::Commit(ref source)),
// but that type is boxed.
// TODO:
// - turn this check into a function on VerifyChainError, like is_duplicate_request()
BlockDownloadVerifyError::Invalid { error, .. }
if format!("{error:?}").contains("block is already committed to the state")
|| format!("{error:?}")
.contains("block has already been sent to be committed to the state") =>
{
// TODO: improve this by checking the type (#2908)
debug!(error = ?e, "block is already committed or pending a commit, possibly from a previous sync run, continuing");
false
}
BlockDownloadVerifyError::DownloadFailed { ref error, .. }
if format!("{error:?}").contains("NotFound") =>
{
// Covers these errors:
// - NotFoundResponse
// - NotFoundRegistry
4. Avoid repeated requests to peers after partial responses or errors (#3505) * fix(network): split synthetic NotFoundRegistry from message NotFoundResponse * docs(network): Improve `notfound` message documentation * refactor(network): Rename MustUseOneshotSender to MustUseClientResponseSender ``` fastmod MustUseOneshotSender MustUseClientResponseSender zebra* ``` * docs(network): fix a comment typo * refactor(network): remove generics from MustUseClientResponseSender * refactor(network): add an inventory collector to Client, but don't use it yet * feat(network): register missing peer responses as missing inventory We register this missing inventory based on peer responses, or connection errors or timeouts. Inbound message inventory tracking requires peers to send `notfound` messages. But `zcashd` skips `notfound` for blocks, so we can't rely on peer messages. This missing inventory tracking works regardless of peer `notfound` messages. * refactor(network): rename ResponseStatus to InventoryResponse ```sh fastmod ResponseStatus InventoryResponse zebra* ``` * refactor(network): rename InventoryStatus::inner() to to_inner() * fix(network): remove a redundant runtime.enter() in a test * doc(network): the exact time used to filter outbound peers doesn't matter * fix(network): handle block requests slightly more efficiently * doc(network): fix a typo * fmt(network): `cargo fmt` after rename ResponseStatus to InventoryResponse * doc(test): clarify some test comments * test(network): test synthetic notfound from connection errors and peer inventory routing * test(network): improve inbound test diagnostics * feat(network): add a proptest-impl feature to zebra-network * feat(network): add a test-only connect_isolated_with_inbound function * test(network): allow a response on the isolated peer test connection * test(network): fix failures in test synthetic notfound * test(network): Simplify SharedPeerError test assertions * test(network): test synthetic notfound from partially successful requests * test(network): MissingInventoryCollector ignores local NotFoundRegistry errors * fix(network): decrease the inventory rotation interval This stops us waiting 3-4 sync resets (4 minutes) before we retry a missing block. Now we wait 1-2 sync resets (2 minutes), which is still a reasonable rate limit. This should speed up syncing near the tip, and on testnet. * fmt(network): cargo fmt --all * cleanup(network): remove unnecessary allow(dead_code) * cleanup(network): stop importing the whole sync module into tests * doc(network): clarify syncer inventory retry constraint * doc(network): add a TODO for a fix to ensure API behaviour remains consistent * doc(network): fix a function doc typo * doc(network): clarify how we handle peers that don't send `notfound` * docs(network): clarify a test comment Co-authored-by: Janito Vaqueiro Ferreira Filho <janito.vff@gmail.com> Co-authored-by: Janito Vaqueiro Ferreira Filho <janito.vff@gmail.com> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com>
2022-02-14 17:44:33 -08:00
//
// 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("block is already committed to the state")
|| err_str.contains("block has already been sent to be 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
}
}
}
}