Skip to content

Commit

Permalink
add(mempool): Verify transactions with unmined inputs in the mempool (#…
Browse files Browse the repository at this point in the history
…8857)

* Adds a parameter to `zebra_consensus::router::init()` for accepting a mempool setup argument, adds and uses an `init_test()` fn for passing a closed channel receiver in tests where no mempool service is needed in the transaction verifier.

* Adds a `mempool` argument to the transaction::Verifier constructor (and a `new_for_tests()` constructor for convenience)

* Removes `Clone` impl on `transaction::Verifier` to add mempool oneshot receiver, updates tests.

* Adds TODOs

* updates transaction verifier's poll_ready() method to setup the mempool service handle.

* Updates VerifiedSet struct used in mempool storage

* Updates mempool service and its `Storage` to use the updated `VerifiedSet` `transactions()` return type.

* updates `created_outputs` when inserting or removing a transaction from the mempool's verified set

* Adds a TODO, updates field docs

* Updates `spent_utxos()` to query the mempool for unspent outputs

* Adds `spent_mempool_outpoints` as a field on tx verifier mempool response

* Updates mempool `Downloads` to return the spent_mempool_outpoints from the tx verifier response

* Updates `Storage.insert()` to accept a list of spent mempool transaction outputs

* Adds transaction dependencies when inserting a tx in `VerifiedSet`

* polls mempool svc from tx verifier when a mempool tx that creates transparent outputs has been verified.

adds a TODO for adding a `pending_outputs` field to the mempool Storage

* Adds `pending_outputs` field on mempool Storage and responds to pending outputs requests when inserting new transactions into the mempool's verified set

* replaces `UnminedTxId` type with `transaction::Hash` in mempool's verified set

* prune pending outputs when rejecting and removing same effects.

* Remove dependent transactions from verified set when removing a tx

* updates tests

* appeases clippy.

* removes unused `len()` method

* fixes doc links

* Adds transaction dependencies to the `FullTransactions` response, let the caller handle it (required to avoid moving zip317 tx selection code to mempool)

* updates block template construction to avoid including transactions unless their dependencies have already been added.

* updates tests

* Replaces placeholder setup channel with one that sends the mempool svc to the tx verifier, adds a timeout layer, adds a TODO about a concurrency bug

* Use a single query to check for unspent outputs in the mempool

* Updates `getblocktemplate` method to consider dependencies when sorting transactions for the final template

* fixes clippy lints, removes unnecessary Option in UnspentOutput response variant

* renames type alias and method, adds a TODO to use iteration instead of recursion

* Adds mempool_removes_dependent_transactions() test

* Updates Storage and VerifiedSet clear() methods to clear pending_outputs, created_outputs, and transaction_dependencies, adds TODO to use iteration instead of recursion.

* removes outdated TODO

* Adds a TODO for reporting queued transaction verification results from the mempool from the poll_ready() method

* Adds `mempool_responds_to_await_output` test

* updates mempool_responds_to_await_output test

* Uses iteration instead of recursion in verified set's remove() method and zip317 mod's dependencies_depth() method

* Adds a mempool_request_with_mempool_output_is_accepted test for the transaction verifier

* Moves delay duration before polling the mempool to a constant, uses a shorter timeout for mempool output lookups, adds a `poll_count` to MockService, and updates `mempool_request_with_unmined_output_spends_is_accepted` to check that the transaction verifier polls the mempool after verifying a mempool transaction with transparent outputs

* adds long_poll_input_mempool_tx_ids_are_sorted test

* Adds a `excludes_tx_with_unselected_dependencies` test

* Updates a TODO

* moves `TransactionDependencies` struct to `zebra-node-services`

* Updates `FullTransactions` response variant's `transaction_dependencies` type

* updates zip317 transaction selection for block templates to include dependent transactions

* Moves and refactors zip317 tx selection test to its own module, adds an `unmined_transactions_in_blocks()` method on network

* Removes `unmined_transactions_in_blocks()`  test utility fn from mempool Storage test module and replaces calls to it with calls to the new test method on Network

* Fixes spelling mistake

* Adds `includes_tx_with_selected_dependencies` test

* fixes zip317 block construction issue

* Fixes vectors test

* Update zebra-node-services/src/mempool.rs

* restores `tip_rejected_exact` type

* updates affected tests

* Documents the new argument in `Storage::insert()`, updates outdated comment

* Update zebrad/src/components/mempool/storage/verified_set.rs

* fixes potential issue with calling buffered mempool's poll_ready() method without calling it.

* Avoids removing dependent transactions of transactions that have been mined onto the best chain.

* Updates `spent_utxos()` method documentation

* Avoids sorting getblocktemplate transactions in non-test compilations

* documents PendingOutputs struct

* Apply suggestions from code review

Co-authored-by: Marek <[email protected]>

* cargo fmt

* Applies suggestions from code review

Avoids unnecessarily rejecting dependent transactions of randomly evicted mempool transactions.

Updates `TransactionDependencies::remove_all()` to omit provided transaction id from the list of removed transaction ids.

* Applies suggestions from code review.

* Adds minor comments

* Update zebrad/src/components/mempool/storage/verified_set.rs

Co-authored-by: Marek <[email protected]>

* Remove an outdated comment (#9013)

---------

Co-authored-by: Marek <[email protected]>
  • Loading branch information
arya2 and upbqdn authored Nov 18, 2024
1 parent 77f1460 commit 1dfac40
Show file tree
Hide file tree
Showing 34 changed files with 1,742 additions and 419 deletions.
43 changes: 41 additions & 2 deletions zebra-chain/src/tests/vectors.rs
Original file line number Diff line number Diff line change
@@ -1,9 +1,15 @@
//! Network methods for fetching blockchain vectors.
//!

use std::collections::BTreeMap;
use std::{collections::BTreeMap, ops::RangeBounds};

use crate::{block::Block, parameters::Network, serialization::ZcashDeserializeInto};
use crate::{
amount::Amount,
block::Block,
parameters::Network,
serialization::ZcashDeserializeInto,
transaction::{UnminedTx, VerifiedUnminedTx},
};

use zebra_test::vectors::{
BLOCK_MAINNET_1046400_BYTES, BLOCK_MAINNET_653599_BYTES, BLOCK_MAINNET_982681_BYTES,
Expand All @@ -30,6 +36,39 @@ impl Network {
}
}

/// Returns iterator over verified unmined transactions in the provided block height range.
pub fn unmined_transactions_in_blocks(
&self,
block_height_range: impl RangeBounds<u32>,
) -> impl DoubleEndedIterator<Item = VerifiedUnminedTx> {
let blocks = self.block_iter();

// Deserialize the blocks that are selected based on the specified `block_height_range`.
let selected_blocks = blocks
.filter(move |(&height, _)| block_height_range.contains(&height))
.map(|(_, block)| {
block
.zcash_deserialize_into::<Block>()
.expect("block test vector is structurally valid")
});

// Extract the transactions from the blocks and wrap each one as an unmined transaction.
// Use a fake zero miner fee and sigops, because we don't have the UTXOs to calculate
// the correct fee.
selected_blocks
.flat_map(|block| block.transactions)
.map(UnminedTx::from)
// Skip transactions that fail ZIP-317 mempool checks
.filter_map(|transaction| {
VerifiedUnminedTx::new(
transaction,
Amount::try_from(1_000_000).expect("invalid value"),
0,
)
.ok()
})
}

/// Returns blocks indexed by height in a [`BTreeMap`].
///
/// Returns Mainnet blocks if `self` is set to Mainnet, and Testnet blocks otherwise.
Expand Down
2 changes: 1 addition & 1 deletion zebra-consensus/src/block/tests.rs
Original file line number Diff line number Diff line change
Expand Up @@ -137,7 +137,7 @@ async fn check_transcripts() -> Result<(), Report> {
let network = Network::Mainnet;
let state_service = zebra_state::init_test(&network);

let transaction = transaction::Verifier::new(&network, state_service.clone());
let transaction = transaction::Verifier::new_for_tests(&network, state_service.clone());
let transaction = Buffer::new(BoxService::new(transaction), 1);
let block_verifier = Buffer::new(
SemanticBlockVerifier::new(&network, state_service.clone(), transaction),
Expand Down
48 changes: 44 additions & 4 deletions zebra-consensus/src/router.rs
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ use std::{

use futures::{FutureExt, TryFutureExt};
use thiserror::Error;
use tokio::task::JoinHandle;
use tokio::{sync::oneshot, task::JoinHandle};
use tower::{buffer::Buffer, util::BoxService, Service, ServiceExt};
use tracing::{instrument, Instrument, Span};

Expand All @@ -30,6 +30,7 @@ use zebra_chain::{
parameters::Network,
};

use zebra_node_services::mempool;
use zebra_state as zs;

use crate::{
Expand Down Expand Up @@ -230,11 +231,12 @@ where
/// Block and transaction verification requests should be wrapped in a timeout,
/// so that out-of-order and invalid requests do not hang indefinitely.
/// See the [`router`](`crate::router`) module documentation for details.
#[instrument(skip(state_service))]
pub async fn init<S>(
#[instrument(skip(state_service, mempool))]
pub async fn init<S, Mempool>(
config: Config,
network: &Network,
mut state_service: S,
mempool: oneshot::Receiver<Mempool>,
) -> (
Buffer<BoxService<Request, block::Hash, RouterError>, Request>,
Buffer<
Expand All @@ -247,6 +249,11 @@ pub async fn init<S>(
where
S: Service<zs::Request, Response = zs::Response, Error = BoxError> + Send + Clone + 'static,
S::Future: Send + 'static,
Mempool: Service<mempool::Request, Response = mempool::Response, Error = BoxError>
+ Send
+ Clone
+ 'static,
Mempool::Future: Send + 'static,
{
// Give other tasks priority before spawning the checkpoint task.
tokio::task::yield_now().await;
Expand Down Expand Up @@ -333,7 +340,7 @@ where

// transaction verification

let transaction = transaction::Verifier::new(network, state_service.clone());
let transaction = transaction::Verifier::new(network, state_service.clone(), mempool);
let transaction = Buffer::new(BoxService::new(transaction), VERIFIER_BUFFER_BOUND);

// block verification
Expand Down Expand Up @@ -397,3 +404,36 @@ pub struct BackgroundTaskHandles {
/// Finishes when all the checkpoints are verified, or when the state tip is reached.
pub state_checkpoint_verify_handle: JoinHandle<()>,
}

/// Calls [`init`] with a closed mempool setup channel for conciseness in tests.
///
/// See [`init`] for more details.
#[cfg(any(test, feature = "proptest-impl"))]
pub async fn init_test<S>(
config: Config,
network: &Network,
state_service: S,
) -> (
Buffer<BoxService<Request, block::Hash, RouterError>, Request>,
Buffer<
BoxService<transaction::Request, transaction::Response, TransactionError>,
transaction::Request,
>,
BackgroundTaskHandles,
Height,
)
where
S: Service<zs::Request, Response = zs::Response, Error = BoxError> + Send + Clone + 'static,
S::Future: Send + 'static,
{
init(
config.clone(),
network,
state_service.clone(),
oneshot::channel::<
Buffer<BoxService<mempool::Request, mempool::Response, BoxError>, mempool::Request>,
>()
.1,
)
.await
}
4 changes: 2 additions & 2 deletions zebra-consensus/src/router/tests.rs
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,7 @@ async fn verifiers_from_network(
_transaction_verifier,
_groth16_download_handle,
_max_checkpoint_height,
) = crate::router::init(Config::default(), &network, state_service.clone()).await;
) = crate::router::init_test(Config::default(), &network, state_service.clone()).await;

// We can drop the download task handle here, because:
// - if the download task fails, the tests will panic, and
Expand Down Expand Up @@ -169,7 +169,7 @@ async fn verify_checkpoint(config: Config) -> Result<(), Report> {
_transaction_verifier,
_groth16_download_handle,
_max_checkpoint_height,
) = super::init(config.clone(), &network, zs::init_test(&network)).await;
) = super::init_test(config.clone(), &network, zs::init_test(&network)).await;

// Add a timeout layer
let block_verifier_router =
Expand Down
Loading

0 comments on commit 1dfac40

Please sign in to comment.