Store the transaction fee in the mempool storage (#2885)

* Create a new VerifiedUnminedTx containing the miner fee

* Use VerifiedUnminedTx in mempool verification responses

And do a bunch of other cleanups.

* Use VerifiedUnminedTx in mempool download and verifier

* Use VerifiedUnminedTx in mempool storage and verified set

* Impl Display for VerifiedUnminedTx, and some convenience methods

* Use VerifiedUnminedTx in existing tests
This commit is contained in:
teor 2021-10-18 11:24:37 +10:00 committed by GitHub
parent 4648f8fc70
commit 2d129414e0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 343 additions and 142 deletions

View File

@ -25,7 +25,7 @@ pub use lock_time::LockTime;
pub use memo::Memo; pub use memo::Memo;
pub use sapling::FieldNotPresent; pub use sapling::FieldNotPresent;
pub use sighash::{HashType, SigHash}; pub use sighash::{HashType, SigHash};
pub use unmined::{UnminedTx, UnminedTxId}; pub use unmined::{UnminedTx, UnminedTxId, VerifiedUnminedTx};
use crate::{ use crate::{
amount::{Amount, Error as AmountError, NegativeAllowed, NonNegative}, amount::{Amount, Error as AmountError, NegativeAllowed, NonNegative},

View File

@ -17,12 +17,14 @@ use std::{fmt, sync::Arc};
#[cfg(any(test, feature = "proptest-impl"))] #[cfg(any(test, feature = "proptest-impl"))]
use proptest_derive::Arbitrary; use proptest_derive::Arbitrary;
use crate::serialization::ZcashSerialize; use crate::{
amount::{Amount, NonNegative},
use super::{ serialization::ZcashSerialize,
AuthDigest, Hash, transaction::{
Transaction::{self, *}, AuthDigest, Hash,
WtxId, Transaction::{self, *},
WtxId,
},
}; };
use UnminedTxId::*; use UnminedTxId::*;
@ -164,6 +166,9 @@ impl UnminedTxId {
} }
/// An unmined transaction, and its pre-calculated unique identifying ID. /// An unmined transaction, and its pre-calculated unique identifying ID.
///
/// This transaction has been structurally verified.
/// (But it might still need semantic or contextual verification.)
#[derive(Clone, Debug, Eq, PartialEq)] #[derive(Clone, Debug, Eq, PartialEq)]
pub struct UnminedTx { pub struct UnminedTx {
/// A unique identifier for this unmined transaction. /// A unique identifier for this unmined transaction.
@ -176,6 +181,15 @@ pub struct UnminedTx {
pub size: usize, pub size: usize,
} }
impl fmt::Display for UnminedTx {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
f.debug_struct("UnminedTx")
.field("transaction", &self.transaction)
.field("serialized_size", &self.size)
.finish()
}
}
// Each of these conversions is implemented slightly differently, // Each of these conversions is implemented slightly differently,
// to avoid cloning the transaction where possible. // to avoid cloning the transaction where possible.
@ -226,3 +240,35 @@ impl From<&Arc<Transaction>> for UnminedTx {
} }
} }
} }
/// A verified unmined transaction, and the corresponding transaction fee.
///
/// This transaction has been fully verified, in the context of the mempool.
#[derive(Clone, Debug, Eq, PartialEq)]
#[cfg_attr(any(test, feature = "proptest-impl"), derive(Arbitrary))]
pub struct VerifiedUnminedTx {
/// The unmined transaction.
pub transaction: UnminedTx,
/// The transaction fee for this unmined transaction.
pub miner_fee: Amount<NonNegative>,
}
impl fmt::Display for VerifiedUnminedTx {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
f.debug_struct("VerifiedUnminedTx")
.field("transaction", &self.transaction)
.field("miner_fee", &self.miner_fee)
.finish()
}
}
impl VerifiedUnminedTx {
/// Create a new verified unmined transaction from a transaction and its fee.
pub fn new(transaction: UnminedTx, miner_fee: Amount<NonNegative>) -> Self {
Self {
transaction,
miner_fee,
}
}
}

View File

@ -23,7 +23,9 @@ use zebra_chain::{
parameters::{Network, NetworkUpgrade}, parameters::{Network, NetworkUpgrade},
primitives::Groth16Proof, primitives::Groth16Proof,
sapling, sapling,
transaction::{self, HashType, SigHash, Transaction, UnminedTx, UnminedTxId}, transaction::{
self, HashType, SigHash, Transaction, UnminedTx, UnminedTxId, VerifiedUnminedTx,
},
transparent, transparent,
}; };
@ -97,13 +99,46 @@ pub enum Request {
/// The response type for the transaction verifier service. /// The response type for the transaction verifier service.
/// Responses identify the transaction that was verified. /// Responses identify the transaction that was verified.
/// #[derive(Clone, Debug, Eq, PartialEq)]
/// [`Block`] requests can be uniquely identified by [`UnminedTxId::mined_id`], pub enum Response {
/// because the block's authorizing data root will be checked during contextual validation. /// A response to a block transaction verification request.
/// Block {
/// [`Mempool`] requests are uniquely identified by the [`UnminedTxId`] /// The witnessed transaction ID for this transaction.
/// variant for their transaction version. ///
pub type Response = (zebra_chain::transaction::UnminedTxId, Amount<NonNegative>); /// [`Block`] responses can be uniquely identified by [`UnminedTxId::mined_id`],
/// because the block's authorizing data root will be checked during contextual validation.
tx_id: UnminedTxId,
/// The miner fee for this transaction.
/// `None` for coinbase transactions.
///
/// Consensus rule:
/// > The remaining value in the transparent transaction value pool
/// > of a coinbase transaction is destroyed.
///
/// https://zips.z.cash/protocol/protocol.pdf#transactions
miner_fee: Option<Amount<NonNegative>>,
},
/// A response to a mempool transaction verification request.
Mempool {
/// The full content of the verified mempool transaction.
/// Also contains the transaction fee and other associated fields.
///
/// Mempool transactions always have a transaction fee,
/// because coinbase transactions are rejected from the mempool.
///
/// [`Mempool`] responses are uniquely identified by the [`UnminedTxId`]
/// variant for their transaction version.
transaction: VerifiedUnminedTx,
},
}
impl From<VerifiedUnminedTx> for Response {
fn from(transaction: VerifiedUnminedTx) -> Self {
Response::Mempool { transaction }
}
}
impl Request { impl Request {
/// The transaction to verify that's in this request. /// The transaction to verify that's in this request.
@ -114,6 +149,14 @@ impl Request {
} }
} }
/// The unverified mempool transaction, if this is a mempool request.
pub fn into_mempool_transaction(self) -> Option<UnminedTx> {
match self {
Request::Block { .. } => None,
Request::Mempool { transaction, .. } => Some(transaction),
}
}
/// The unmined transaction ID for the transaction in this request. /// The unmined transaction ID for the transaction in this request.
pub fn tx_id(&self) -> UnminedTxId { pub fn tx_id(&self) -> UnminedTxId {
match self { match self {
@ -154,6 +197,42 @@ impl Request {
} }
} }
impl Response {
/// The verified mempool transaction, if this is a mempool response.
pub fn into_mempool_transaction(self) -> Option<VerifiedUnminedTx> {
match self {
Response::Block { .. } => None,
Response::Mempool { transaction, .. } => Some(transaction),
}
}
/// The unmined transaction ID for the transaction in this response.
pub fn tx_id(&self) -> UnminedTxId {
match self {
Response::Block { tx_id, .. } => *tx_id,
Response::Mempool { transaction, .. } => transaction.transaction.id,
}
}
/// The miner fee for the transaction in this response.
///
/// Coinbase transactions do not have a miner fee.
pub fn miner_fee(&self) -> Option<Amount<NonNegative>> {
match self {
Response::Block { miner_fee, .. } => *miner_fee,
Response::Mempool { transaction, .. } => Some(transaction.miner_fee),
}
}
/// Returns true if the request is a mempool request.
pub fn is_mempool(&self) -> bool {
match self {
Response::Block { .. } => false,
Response::Mempool { .. } => true,
}
}
}
impl<ZS> Service<Request> for Verifier<ZS> impl<ZS> Service<Request> for Verifier<ZS>
where where
ZS: Service<zs::Request, Response = zs::Response, Error = BoxError> + Send + Clone + 'static, ZS: Service<zs::Request, Response = zs::Response, Error = BoxError> + Send + Clone + 'static,
@ -174,8 +253,8 @@ where
let network = self.network; let network = self.network;
let tx = req.transaction(); let tx = req.transaction();
let id = req.tx_id(); let tx_id = req.tx_id();
let span = tracing::debug_span!("tx", ?id); let span = tracing::debug_span!("tx", ?tx_id);
async move { async move {
tracing::trace!(?req); tracing::trace!(?req);
@ -221,7 +300,7 @@ where
sapling_shielded_data, sapling_shielded_data,
.. ..
} => Self::verify_v4_transaction( } => Self::verify_v4_transaction(
req, &req,
network, network,
script_verifier, script_verifier,
inputs, inputs,
@ -235,7 +314,7 @@ where
orchard_shielded_data, orchard_shielded_data,
.. ..
} => Self::verify_v5_transaction( } => Self::verify_v5_transaction(
req, &req,
network, network,
script_verifier, script_verifier,
inputs, inputs,
@ -255,21 +334,32 @@ where
// Get the `value_balance` to calculate the transaction fee. // Get the `value_balance` to calculate the transaction fee.
let value_balance = tx.value_balance(&spent_utxos); let value_balance = tx.value_balance(&spent_utxos);
// Initialize the transaction fee to zero.
let mut tx_fee = Amount::<NonNegative>::zero();
// Calculate the fee only for non-coinbase transactions. // Calculate the fee only for non-coinbase transactions.
let mut miner_fee = None;
if !tx.has_valid_coinbase_transaction_inputs() { if !tx.has_valid_coinbase_transaction_inputs() {
tx_fee = match value_balance { // TODO: deduplicate this code with remaining_transaction_value (#TODO: open ticket)
miner_fee = match value_balance {
Ok(vb) => match vb.remaining_transaction_value() { Ok(vb) => match vb.remaining_transaction_value() {
Ok(tx_rtv) => tx_rtv, Ok(tx_rtv) => Some(tx_rtv),
Err(_) => return Err(TransactionError::IncorrectFee), Err(_) => return Err(TransactionError::IncorrectFee),
}, },
Err(_) => return Err(TransactionError::IncorrectFee), Err(_) => return Err(TransactionError::IncorrectFee),
}; };
} }
Ok((id, tx_fee)) let rsp = match req {
Request::Block { .. } => Response::Block { tx_id, miner_fee },
Request::Mempool { transaction, .. } => Response::Mempool {
transaction: VerifiedUnminedTx::new(
transaction,
miner_fee.expect(
"unexpected mempool coinbase transaction: should have already rejected",
),
),
},
};
Ok(rsp)
} }
.instrument(span) .instrument(span)
.boxed() .boxed()
@ -300,7 +390,7 @@ where
/// - the Sprout `joinsplit_data` shielded data in the transaction /// - the Sprout `joinsplit_data` shielded data in the transaction
/// - the `sapling_shielded_data` in the transaction /// - the `sapling_shielded_data` in the transaction
fn verify_v4_transaction( fn verify_v4_transaction(
request: Request, request: &Request,
network: Network, network: Network,
script_verifier: script::Verifier<ZS>, script_verifier: script::Verifier<ZS>,
inputs: &[transparent::Input], inputs: &[transparent::Input],
@ -316,7 +406,7 @@ where
let shielded_sighash = tx.sighash(upgrade, HashType::ALL, None); let shielded_sighash = tx.sighash(upgrade, HashType::ALL, None);
Ok(Self::verify_transparent_inputs_and_outputs( Ok(Self::verify_transparent_inputs_and_outputs(
&request, request,
network, network,
script_verifier, script_verifier,
inputs, inputs,
@ -382,7 +472,7 @@ where
/// - the sapling shielded data of the transaction, if any /// - the sapling shielded data of the transaction, if any
/// - the orchard shielded data of the transaction, if any /// - the orchard shielded data of the transaction, if any
fn verify_v5_transaction( fn verify_v5_transaction(
request: Request, request: &Request,
network: Network, network: Network,
script_verifier: script::Verifier<ZS>, script_verifier: script::Verifier<ZS>,
inputs: &[transparent::Input], inputs: &[transparent::Input],
@ -398,7 +488,7 @@ where
let shielded_sighash = transaction.sighash(upgrade, HashType::ALL, None); let shielded_sighash = transaction.sighash(upgrade, HashType::ALL, None);
Ok(Self::verify_transparent_inputs_and_outputs( Ok(Self::verify_transparent_inputs_and_outputs(
&request, request,
network, network,
script_verifier, script_verifier,
inputs, inputs,

View File

@ -275,7 +275,7 @@ async fn v5_transaction_is_accepted_after_nu5_activation_for_network(network: Ne
.await; .await;
assert_eq!( assert_eq!(
result.expect("expected a tx_id and tx_fee").0, result.expect("unexpected error response").tx_id(),
expected_hash expected_hash
); );
} }
@ -324,7 +324,7 @@ async fn v4_transaction_with_transparent_transfer_is_accepted() {
.await; .await;
assert_eq!( assert_eq!(
result.expect("expected a tx_id and tx_fee").0, result.expect("unexpected error response").tx_id(),
transaction_hash transaction_hash
); );
} }
@ -370,7 +370,7 @@ async fn v4_coinbase_transaction_is_accepted() {
.await; .await;
assert_eq!( assert_eq!(
result.expect("expected a tx_id and tx_fee").0, result.expect("unexpected error response").tx_id(),
transaction_hash transaction_hash
); );
} }
@ -474,7 +474,7 @@ async fn v5_transaction_with_transparent_transfer_is_accepted() {
.await; .await;
assert_eq!( assert_eq!(
result.expect("expected a tx_id and tx_fee").0, result.expect("unexpected error response").tx_id(),
transaction_hash transaction_hash
); );
} }
@ -523,7 +523,7 @@ async fn v5_coinbase_transaction_is_accepted() {
.await; .await;
assert_eq!( assert_eq!(
result.expect("expected a tx_id and tx_fee").0, result.expect("unexpected error response").tx_id(),
transaction_hash transaction_hash
); );
} }
@ -643,7 +643,7 @@ fn v4_with_signed_sprout_transfer_is_accepted() {
.await; .await;
assert_eq!( assert_eq!(
result.expect("expected a tx_id and tx_fee").0, result.expect("unexpected error response").tx_id(),
expected_hash expected_hash
); );
}); });
@ -744,7 +744,7 @@ fn v4_with_sapling_spends() {
.await; .await;
assert_eq!( assert_eq!(
result.expect("expected a tx_id and tx_fee").0, result.expect("unexpected error response").tx_id(),
expected_hash expected_hash
); );
}); });
@ -788,7 +788,7 @@ fn v4_with_sapling_outputs_and_no_spends() {
.await; .await;
assert_eq!( assert_eq!(
result.expect("expected a tx_id and tx_fee").0, result.expect("unexpected error response").tx_id(),
expected_hash expected_hash
); );
}); });
@ -835,7 +835,7 @@ fn v5_with_sapling_spends() {
.await; .await;
assert_eq!( assert_eq!(
result.expect("expected a tx_id and tx_fee").0, result.expect("unexpected error response").tx_id(),
expected_hash expected_hash
); );
}); });

View File

@ -12,7 +12,7 @@ use zebra_chain::{
block::Block, block::Block,
parameters::Network, parameters::Network,
serialization::ZcashDeserializeInto, serialization::ZcashDeserializeInto,
transaction::{UnminedTx, UnminedTxId}, transaction::{UnminedTx, UnminedTxId, VerifiedUnminedTx},
}; };
use zebra_consensus::{error::TransactionError, transaction, Config as ConsensusConfig}; use zebra_consensus::{error::TransactionError, transaction, Config as ConsensusConfig};
use zebra_network::{AddressBook, Request, Response}; use zebra_network::{AddressBook, Request, Response};
@ -41,6 +41,10 @@ async fn mempool_requests_for_transactions() {
tx_gossip_task_handle, tx_gossip_task_handle,
) = setup(true).await; ) = setup(true).await;
let added_transactions: Vec<UnminedTx> = added_transactions
.iter()
.map(|t| t.transaction.clone())
.collect();
let added_transaction_ids: Vec<UnminedTxId> = added_transactions.iter().map(|t| t.id).collect(); let added_transaction_ids: Vec<UnminedTxId> = added_transactions.iter().map(|t| t.id).collect();
// Test `Request::MempoolTransactionIds` // Test `Request::MempoolTransactionIds`
@ -117,11 +121,17 @@ async fn mempool_push_transaction() -> Result<(), crate::BoxError> {
.oneshot(Request::PushTransaction(tx.clone().into())); .oneshot(Request::PushTransaction(tx.clone().into()));
// Simulate a successful transaction verification // Simulate a successful transaction verification
let verification = tx_verifier.expect_request_that(|_| true).map(|responder| { let verification = tx_verifier.expect_request_that(|_| true).map(|responder| {
let txid = responder.request().tx_id(); let transaction = responder
.request()
.clone()
.into_mempool_transaction()
.expect("unexpected non-mempool request");
// Set a dummy fee. // Set a dummy fee.
let tx_fee = Amount::zero(); responder.respond(transaction::Response::from(VerifiedUnminedTx::new(
responder.respond((txid, tx_fee)); transaction,
Amount::zero(),
)));
}); });
let (response, _) = futures::join!(request, verification); let (response, _) = futures::join!(request, verification);
match response { match response {
@ -208,11 +218,17 @@ async fn mempool_advertise_transaction_ids() -> Result<(), crate::BoxError> {
}); });
// Simulate a successful transaction verification // Simulate a successful transaction verification
let verification = tx_verifier.expect_request_that(|_| true).map(|responder| { let verification = tx_verifier.expect_request_that(|_| true).map(|responder| {
let txid = responder.request().tx_id(); let transaction = responder
.request()
.clone()
.into_mempool_transaction()
.expect("unexpected non-mempool request");
// Set a dummy fee. // Set a dummy fee.
let tx_fee = Amount::zero(); responder.respond(transaction::Response::from(VerifiedUnminedTx::new(
responder.respond((txid, tx_fee)); transaction,
Amount::zero(),
)));
}); });
let (response, _, _) = futures::join!(request, peer_set_responder, verification); let (response, _, _) = futures::join!(request, peer_set_responder, verification);
@ -299,10 +315,17 @@ async fn mempool_transaction_expiration() -> Result<(), crate::BoxError> {
// Simulate a successful transaction verification // Simulate a successful transaction verification
let verification = tx_verifier.expect_request_that(|_| true).map(|responder| { let verification = tx_verifier.expect_request_that(|_| true).map(|responder| {
tx1_id = responder.request().tx_id(); tx1_id = responder.request().tx_id();
let transaction = responder
.request()
.clone()
.into_mempool_transaction()
.expect("unexpected non-mempool request");
// Set a dummy fee. // Set a dummy fee.
let tx_fee = Amount::zero(); responder.respond(transaction::Response::from(VerifiedUnminedTx::new(
responder.respond((tx1_id, tx_fee)); transaction,
Amount::zero(),
)));
}); });
let (response, _) = futures::join!(request, verification); let (response, _) = futures::join!(request, verification);
match response { match response {
@ -391,10 +414,17 @@ async fn mempool_transaction_expiration() -> Result<(), crate::BoxError> {
// Simulate a successful transaction verification // Simulate a successful transaction verification
let verification = tx_verifier.expect_request_that(|_| true).map(|responder| { let verification = tx_verifier.expect_request_that(|_| true).map(|responder| {
tx2_id = responder.request().tx_id(); tx2_id = responder.request().tx_id();
let transaction = responder
.request()
.clone()
.into_mempool_transaction()
.expect("unexpected non-mempool request");
// Set a dummy fee. // Set a dummy fee.
let tx_fee = Amount::zero(); responder.respond(transaction::Response::from(VerifiedUnminedTx::new(
responder.respond((tx2_id, tx_fee)); transaction,
Amount::zero(),
)));
}); });
let (response, _) = futures::join!(request, verification); let (response, _) = futures::join!(request, verification);
match response { match response {
@ -529,7 +559,7 @@ async fn setup(
>, >,
Buffer<BoxService<mempool::Request, mempool::Response, BoxError>, mempool::Request>, Buffer<BoxService<mempool::Request, mempool::Response, BoxError>, mempool::Request>,
Vec<Arc<Block>>, Vec<Arc<Block>>,
Vec<UnminedTx>, Vec<VerifiedUnminedTx>,
MockService<transaction::Request, transaction::Response, PanicAssertion, TransactionError>, MockService<transaction::Request, transaction::Response, PanicAssertion, TransactionError>,
MockService<Request, Response, PanicAssertion>, MockService<Request, Response, PanicAssertion>,
Buffer<BoxService<zebra_state::Request, zebra_state::Response, BoxError>, zebra_state::Request>, Buffer<BoxService<zebra_state::Request, zebra_state::Response, BoxError>, zebra_state::Request>,
@ -667,7 +697,10 @@ async fn setup(
/// Manually add a transaction to the mempool storage. /// Manually add a transaction to the mempool storage.
/// ///
/// Skips some mempool functionality, like transaction verification and peer broadcasts. /// Skips some mempool functionality, like transaction verification and peer broadcasts.
fn add_some_stuff_to_mempool(mempool_service: &mut Mempool, network: Network) -> Vec<UnminedTx> { fn add_some_stuff_to_mempool(
mempool_service: &mut Mempool,
network: Network,
) -> Vec<VerifiedUnminedTx> {
// get the genesis block coinbase transaction from the Zcash blockchain. // get the genesis block coinbase transaction from the Zcash blockchain.
let genesis_transactions: Vec<_> = unmined_transactions_in_blocks(..=0, network) let genesis_transactions: Vec<_> = unmined_transactions_in_blocks(..=0, network)
.take(1) .take(1)

View File

@ -16,7 +16,7 @@ use tokio::{sync::oneshot, task::JoinHandle};
use tower::{Service, ServiceExt}; use tower::{Service, ServiceExt};
use tracing_futures::Instrument; use tracing_futures::Instrument;
use zebra_chain::transaction::{self, UnminedTx, UnminedTxId}; use zebra_chain::transaction::{self, UnminedTx, UnminedTxId, VerifiedUnminedTx};
use zebra_consensus::transaction as tx; use zebra_consensus::transaction as tx;
use zebra_network as zn; use zebra_network as zn;
use zebra_state as zs; use zebra_state as zs;
@ -144,7 +144,7 @@ where
/// A list of pending transaction download and verify tasks. /// A list of pending transaction download and verify tasks.
#[pin] #[pin]
pending: FuturesUnordered< pending: FuturesUnordered<
JoinHandle<Result<UnminedTx, (TransactionDownloadVerifyError, UnminedTxId)>>, JoinHandle<Result<VerifiedUnminedTx, (TransactionDownloadVerifyError, UnminedTxId)>>,
>, >,
/// A list of channels that can be used to cancel pending transaction download and /// A list of channels that can be used to cancel pending transaction download and
@ -161,7 +161,7 @@ where
ZS: Service<zs::Request, Response = zs::Response, Error = BoxError> + Send + Clone + 'static, ZS: Service<zs::Request, Response = zs::Response, Error = BoxError> + Send + Clone + 'static,
ZS::Future: Send, ZS::Future: Send,
{ {
type Item = Result<UnminedTx, (UnminedTxId, TransactionDownloadVerifyError)>; type Item = Result<VerifiedUnminedTx, (UnminedTxId, TransactionDownloadVerifyError)>;
fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll<Option<Self::Item>> { fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll<Option<Self::Item>> {
let this = self.project(); let this = self.project();
@ -177,7 +177,7 @@ where
if let Some(join_result) = ready!(this.pending.poll_next(cx)) { if let Some(join_result) = ready!(this.pending.poll_next(cx)) {
match join_result.expect("transaction download and verify tasks must not panic") { match join_result.expect("transaction download and verify tasks must not panic") {
Ok(tx) => { Ok(tx) => {
this.cancel_handles.remove(&tx.id); this.cancel_handles.remove(&tx.transaction.id);
Poll::Ready(Some(Ok(tx))) Poll::Ready(Some(Ok(tx)))
} }
Err((e, hash)) => { Err((e, hash)) => {
@ -300,7 +300,10 @@ where
transaction: tx.clone(), transaction: tx.clone(),
height, height,
}) })
.map_ok(|(_tx_id, _tx_fee)| tx) .map_ok(|rsp| {
rsp.into_mempool_transaction()
.expect("unexpected non-mempool response to mempool request")
})
.await; .await;
tracing::debug!(?txid, ?result, "verified transaction for the mempool"); tracing::debug!(?txid, ?result, "verified transaction for the mempool");

View File

@ -2,7 +2,7 @@ use std::collections::{HashMap, HashSet};
use thiserror::Error; use thiserror::Error;
use zebra_chain::transaction::{self, UnminedTx, UnminedTxId}; use zebra_chain::transaction::{self, UnminedTx, UnminedTxId, VerifiedUnminedTx};
use self::verified_set::VerifiedSet; use self::verified_set::VerifiedSet;
use super::{downloads::TransactionDownloadVerifyError, MempoolError}; use super::{downloads::TransactionDownloadVerifyError, MempoolError};
@ -117,7 +117,7 @@ pub struct Storage {
} }
impl Storage { impl Storage {
/// Insert a [`UnminedTx`] into the mempool, caching any rejections. /// Insert a [`VerifiedUnminedTx`] into the mempool, caching any rejections.
/// ///
/// Returns an error if the mempool's verified transactions or rejection caches /// Returns an error if the mempool's verified transactions or rejection caches
/// prevent this transaction from being inserted. /// prevent this transaction from being inserted.
@ -125,14 +125,14 @@ impl Storage {
/// ///
/// If inserting this transaction evicts other transactions, they will be tracked /// If inserting this transaction evicts other transactions, they will be tracked
/// as [`StorageRejectionError::RandomlyEvicted`]. /// as [`StorageRejectionError::RandomlyEvicted`].
pub fn insert(&mut self, tx: UnminedTx) -> Result<UnminedTxId, MempoolError> { pub fn insert(&mut self, tx: VerifiedUnminedTx) -> Result<UnminedTxId, MempoolError> {
// # Security // # Security
// //
// This method must call `reject`, rather than modifying the rejection lists directly. // This method must call `reject`, rather than modifying the rejection lists directly.
let tx_id = tx.id; let tx_id = tx.transaction.id;
// First, check if we have a cached rejection for this transaction. // First, check if we have a cached rejection for this transaction.
if let Some(error) = self.rejection_error(&tx.id) { if let Some(error) = self.rejection_error(&tx_id) {
return Err(error); return Err(error);
} }
@ -140,7 +140,7 @@ impl Storage {
// //
// Security: transactions must not get refreshed by new queries, // Security: transactions must not get refreshed by new queries,
// because that allows malicious peers to keep transactions live forever. // because that allows malicious peers to keep transactions live forever.
if self.verified.contains(&tx.id) { if self.verified.contains(&tx_id) {
return Err(MempoolError::InMempool); return Err(MempoolError::InMempool);
} }
@ -160,13 +160,13 @@ impl Storage {
.expect("mempool is empty, but was expected to be full"); .expect("mempool is empty, but was expected to be full");
self.reject( self.reject(
evicted_tx.id, evicted_tx.transaction.id,
SameEffectsChainRejectionError::RandomlyEvicted.into(), SameEffectsChainRejectionError::RandomlyEvicted.into(),
); );
// If this transaction gets evicted, set its result to the same error // If this transaction gets evicted, set its result to the same error
// (we could return here, but we still want to check the mempool size) // (we could return here, but we still want to check the mempool size)
if evicted_tx.id == tx_id { if evicted_tx.transaction.id == tx_id {
result = Err(SameEffectsChainRejectionError::RandomlyEvicted.into()); result = Err(SameEffectsChainRejectionError::RandomlyEvicted.into());
} }
} }
@ -176,7 +176,7 @@ impl Storage {
result result
} }
/// Remove [`UnminedTx`]es from the mempool via exact [`UnminedTxId`]. /// Remove transactions from the mempool via exact [`UnminedTxId`].
/// ///
/// For v5 transactions, transactions are matched by WTXID, using both the: /// For v5 transactions, transactions are matched by WTXID, using both the:
/// - non-malleable transaction ID, and /// - non-malleable transaction ID, and
@ -193,10 +193,10 @@ impl Storage {
#[allow(dead_code)] #[allow(dead_code)]
pub fn remove_exact(&mut self, exact_wtxids: &HashSet<UnminedTxId>) -> usize { pub fn remove_exact(&mut self, exact_wtxids: &HashSet<UnminedTxId>) -> usize {
self.verified self.verified
.remove_all_that(|tx| exact_wtxids.contains(&tx.id)) .remove_all_that(|tx| exact_wtxids.contains(&tx.transaction.id))
} }
/// Remove [`UnminedTx`]es from the mempool via non-malleable [`transaction::Hash`]. /// Remove transactions from the mempool via non-malleable [`transaction::Hash`].
/// ///
/// For v5 transactions, transactions are matched by TXID, /// For v5 transactions, transactions are matched by TXID,
/// using only the non-malleable transaction ID. /// using only the non-malleable transaction ID.
@ -211,7 +211,7 @@ impl Storage {
/// Does not add or remove from the 'rejected' tracking set. /// Does not add or remove from the 'rejected' tracking set.
pub fn remove_same_effects(&mut self, mined_ids: &HashSet<transaction::Hash>) -> usize { pub fn remove_same_effects(&mut self, mined_ids: &HashSet<transaction::Hash>) -> usize {
self.verified self.verified
.remove_all_that(|tx| mined_ids.contains(&tx.id.mined_id())) .remove_all_that(|tx| mined_ids.contains(&tx.transaction.id.mined_id()))
} }
/// Clears the whole mempool storage. /// Clears the whole mempool storage.
@ -254,7 +254,7 @@ impl Storage {
self.verified.transactions().map(|tx| tx.id) self.verified.transactions().map(|tx| tx.id)
} }
/// Returns the set of [`Transaction`][transaction::Transaction]s in the mempool. /// Returns the set of [`UnminedTx`]es in the mempool.
pub fn transactions(&self) -> impl Iterator<Item = &UnminedTx> { pub fn transactions(&self) -> impl Iterator<Item = &UnminedTx> {
self.verified.transactions() self.verified.transactions()
} }
@ -265,7 +265,7 @@ impl Storage {
self.verified.transaction_count() self.verified.transaction_count()
} }
/// Returns the set of [`Transaction`][transaction::Transaction]s with exactly matching /// Returns the set of [`UnminedTx`]es with exactly matching
/// `tx_ids` in the mempool. /// `tx_ids` in the mempool.
/// ///
/// This matches the exact transaction, with identical blockchain effects, signatures, and proofs. /// This matches the exact transaction, with identical blockchain effects, signatures, and proofs.
@ -278,7 +278,7 @@ impl Storage {
.filter(move |tx| tx_ids.contains(&tx.id)) .filter(move |tx| tx_ids.contains(&tx.id))
} }
/// Returns `true` if a [`UnminedTx`] exactly matching an [`UnminedTxId`] is in /// Returns `true` if a transaction exactly matching an [`UnminedTxId`] is in
/// the mempool. /// the mempool.
/// ///
/// This matches the exact transaction, with identical blockchain effects, signatures, and proofs. /// This matches the exact transaction, with identical blockchain effects, signatures, and proofs.
@ -319,7 +319,7 @@ impl Storage {
self.limit_rejection_list_memory(); self.limit_rejection_list_memory();
} }
/// Returns the rejection error if a [`UnminedTx`] matching an [`UnminedTxId`] /// Returns the rejection error if a transaction matching an [`UnminedTxId`]
/// is in any mempool rejected list. /// is in any mempool rejected list.
/// ///
/// This matches transactions based on each rejection list's matching rule. /// This matches transactions based on each rejection list's matching rule.
@ -355,7 +355,7 @@ impl Storage {
.filter(move |txid| self.contains_rejected(txid)) .filter(move |txid| self.contains_rejected(txid))
} }
/// Returns `true` if a [`UnminedTx`] matching the supplied [`UnminedTxId`] is in /// Returns `true` if a transaction matching the supplied [`UnminedTxId`] is in
/// the mempool rejected list. /// the mempool rejected list.
/// ///
/// This matches transactions based on each rejection list's matching rule. /// This matches transactions based on each rejection list's matching rule.

View File

@ -1,7 +1,11 @@
use std::ops::RangeBounds; use std::ops::RangeBounds;
use zebra_chain::{ use zebra_chain::{
block::Block, parameters::Network, serialization::ZcashDeserializeInto, transaction::UnminedTx, amount::Amount,
block::Block,
parameters::Network,
serialization::ZcashDeserializeInto,
transaction::{UnminedTx, VerifiedUnminedTx},
}; };
mod prop; mod prop;
@ -10,7 +14,7 @@ mod vectors;
pub fn unmined_transactions_in_blocks( pub fn unmined_transactions_in_blocks(
block_height_range: impl RangeBounds<u32>, block_height_range: impl RangeBounds<u32>,
network: Network, network: Network,
) -> impl DoubleEndedIterator<Item = UnminedTx> { ) -> impl DoubleEndedIterator<Item = VerifiedUnminedTx> {
let blocks = match network { let blocks = match network {
Network::Mainnet => zebra_test::vectors::MAINNET_BLOCKS.iter(), Network::Mainnet => zebra_test::vectors::MAINNET_BLOCKS.iter(),
Network::Testnet => zebra_test::vectors::TESTNET_BLOCKS.iter(), Network::Testnet => zebra_test::vectors::TESTNET_BLOCKS.iter(),
@ -25,8 +29,10 @@ pub fn unmined_transactions_in_blocks(
.expect("block test vector is structurally valid") .expect("block test vector is structurally valid")
}); });
// Extract the transactions from the blocks and warp each one as an unmined transaction. // Extract the transactions from the blocks and wrap each one as an unmined transaction.
// Use a fake zero miner fee, because we don't have the UTXOs to calculate the correct fee.
selected_blocks selected_blocks
.flat_map(|block| block.transactions) .flat_map(|block| block.transactions)
.map(UnminedTx::from) .map(UnminedTx::from)
.map(|transaction| VerifiedUnminedTx::new(transaction, Amount::zero()))
} }

View File

@ -4,6 +4,7 @@ use proptest::{collection::vec, prelude::*};
use proptest_derive::Arbitrary; use proptest_derive::Arbitrary;
use zebra_chain::{ use zebra_chain::{
amount::Amount,
at_least_one, at_least_one,
fmt::{DisplayToDebug, SummaryDebug}, fmt::{DisplayToDebug, SummaryDebug},
orchard, orchard,
@ -11,7 +12,7 @@ use zebra_chain::{
sapling, sapling,
serialization::AtLeastOne, serialization::AtLeastOne,
sprout, sprout,
transaction::{self, JoinSplitData, Transaction, UnminedTx, UnminedTxId}, transaction::{self, JoinSplitData, Transaction, UnminedTxId, VerifiedUnminedTx},
transparent, LedgerState, transparent, LedgerState,
}; };
@ -23,7 +24,7 @@ use crate::components::mempool::{
SameEffectsChainRejectionError, SameEffectsChainRejectionError,
}; };
use self::MultipleTransactionRemovalTestInput::*; use MultipleTransactionRemovalTestInput::*;
/// The mempool list limit tests can run for a long time. /// The mempool list limit tests can run for a long time.
/// ///
@ -54,7 +55,7 @@ proptest! {
]; ];
for (transaction_to_accept, transaction_to_reject) in input_permutations { for (transaction_to_accept, transaction_to_reject) in input_permutations {
let id_to_accept = transaction_to_accept.id; let id_to_accept = transaction_to_accept.transaction.id;
prop_assert_eq!(storage.insert(transaction_to_accept), Ok(id_to_accept)); prop_assert_eq!(storage.insert(transaction_to_accept), Ok(id_to_accept));
@ -95,7 +96,7 @@ proptest! {
/// Test that the reject list length limits are applied when evicting transactions. /// Test that the reject list length limits are applied when evicting transactions.
#[test] #[test]
fn reject_lists_are_limited_insert_eviction( fn reject_lists_are_limited_insert_eviction(
transactions in vec(any::<UnminedTx>(), MEMPOOL_SIZE + 1).prop_map(SummaryDebug), transactions in vec(any::<VerifiedUnminedTx>(), MEMPOOL_SIZE + 1).prop_map(SummaryDebug),
mut rejection_template in any::<UnminedTxId>() mut rejection_template in any::<UnminedTxId>()
) { ) {
let mut storage = Storage::default(); let mut storage = Storage::default();
@ -119,7 +120,7 @@ proptest! {
prop_assert_eq!(storage.rejected_transaction_count(), MAX_EVICTION_MEMORY_ENTRIES); prop_assert_eq!(storage.rejected_transaction_count(), MAX_EVICTION_MEMORY_ENTRIES);
for transaction in transactions { for transaction in transactions {
let tx_id = transaction.id; let tx_id = transaction.transaction.id;
if storage.transaction_count() < MEMPOOL_SIZE { if storage.transaction_count() < MEMPOOL_SIZE {
// The initial transactions should be successful // The initial transactions should be successful
@ -203,8 +204,8 @@ proptest! {
]; ];
for (transaction_to_accept, transaction_to_reject) in input_permutations { for (transaction_to_accept, transaction_to_reject) in input_permutations {
let id_to_accept = transaction_to_accept.id; let id_to_accept = transaction_to_accept.transaction.id;
let id_to_reject = transaction_to_reject.id; let id_to_reject = transaction_to_reject.transaction.id;
prop_assert_eq!(storage.insert(transaction_to_accept), Ok(id_to_accept)); prop_assert_eq!(storage.insert(transaction_to_accept), Ok(id_to_accept));
@ -248,9 +249,9 @@ proptest! {
for (first_transaction_to_accept, transaction_to_reject, second_transaction_to_accept) in for (first_transaction_to_accept, transaction_to_reject, second_transaction_to_accept) in
input_permutations input_permutations
{ {
let first_id_to_accept = first_transaction_to_accept.id; let first_id_to_accept = first_transaction_to_accept.transaction.id;
let second_id_to_accept = second_transaction_to_accept.id; let second_id_to_accept = second_transaction_to_accept.transaction.id;
let id_to_reject = transaction_to_reject.id; let id_to_reject = transaction_to_reject.transaction.id;
prop_assert_eq!( prop_assert_eq!(
storage.insert(first_transaction_to_accept), storage.insert(first_transaction_to_accept),
@ -284,7 +285,7 @@ proptest! {
// Insert all input transactions, and keep track of the IDs of the one that were actually // Insert all input transactions, and keep track of the IDs of the one that were actually
// inserted. // inserted.
let inserted_transactions: HashSet<_> = input.transactions().filter_map(|transaction| { let inserted_transactions: HashSet<_> = input.transactions().filter_map(|transaction| {
let id = transaction.id; let id = transaction.transaction.id;
storage.insert(transaction.clone()).ok().map(|_| id)}).collect(); storage.insert(transaction.clone()).ok().map(|_| id)}).collect();
@ -354,7 +355,7 @@ enum SpendConflictTestInput {
impl SpendConflictTestInput { impl SpendConflictTestInput {
/// Return two transactions that have a spend conflict. /// Return two transactions that have a spend conflict.
pub fn conflicting_transactions(self) -> (UnminedTx, UnminedTx) { pub fn conflicting_transactions(self) -> (VerifiedUnminedTx, VerifiedUnminedTx) {
let (first, second) = match self { let (first, second) = match self {
SpendConflictTestInput::V4 { SpendConflictTestInput::V4 {
mut first, mut first,
@ -378,11 +379,14 @@ impl SpendConflictTestInput {
} }
}; };
(first.0.into(), second.0.into()) (
VerifiedUnminedTx::new(first.0.into(), Amount::zero()),
VerifiedUnminedTx::new(second.0.into(), Amount::zero()),
)
} }
/// Return two transactions that have no spend conflicts. /// Return two transactions that have no spend conflicts.
pub fn unconflicting_transactions(self) -> (UnminedTx, UnminedTx) { pub fn unconflicting_transactions(self) -> (VerifiedUnminedTx, VerifiedUnminedTx) {
let (mut first, mut second) = match self { let (mut first, mut second) = match self {
SpendConflictTestInput::V4 { first, second, .. } => (first, second), SpendConflictTestInput::V4 { first, second, .. } => (first, second),
SpendConflictTestInput::V5 { first, second, .. } => (first, second), SpendConflictTestInput::V5 { first, second, .. } => (first, second),
@ -393,7 +397,10 @@ impl SpendConflictTestInput {
Self::remove_sapling_conflicts(&mut first, &mut second); Self::remove_sapling_conflicts(&mut first, &mut second);
Self::remove_orchard_conflicts(&mut first, &mut second); Self::remove_orchard_conflicts(&mut first, &mut second);
(first.0.into(), second.0.into()) (
VerifiedUnminedTx::new(first.0.into(), Amount::zero()),
VerifiedUnminedTx::new(second.0.into(), Amount::zero()),
)
} }
/// Find transparent outpoint spends shared by two transactions, then remove them from the /// Find transparent outpoint spends shared by two transactions, then remove them from the
@ -820,12 +827,12 @@ impl OrchardSpendConflict {
#[derive(Clone, Debug)] #[derive(Clone, Debug)]
pub enum MultipleTransactionRemovalTestInput { pub enum MultipleTransactionRemovalTestInput {
RemoveExact { RemoveExact {
transactions: SummaryDebug<Vec<UnminedTx>>, transactions: SummaryDebug<Vec<VerifiedUnminedTx>>,
wtx_ids_to_remove: SummaryDebug<HashSet<UnminedTxId>>, wtx_ids_to_remove: SummaryDebug<HashSet<UnminedTxId>>,
}, },
RemoveSameEffects { RemoveSameEffects {
transactions: SummaryDebug<Vec<UnminedTx>>, transactions: SummaryDebug<Vec<VerifiedUnminedTx>>,
mined_ids_to_remove: SummaryDebug<HashSet<transaction::Hash>>, mined_ids_to_remove: SummaryDebug<HashSet<transaction::Hash>>,
}, },
} }
@ -834,7 +841,7 @@ impl Arbitrary for MultipleTransactionRemovalTestInput {
type Parameters = (); type Parameters = ();
fn arbitrary_with(_: Self::Parameters) -> Self::Strategy { fn arbitrary_with(_: Self::Parameters) -> Self::Strategy {
vec(any::<UnminedTx>(), 1..MEMPOOL_SIZE) vec(any::<VerifiedUnminedTx>(), 1..MEMPOOL_SIZE)
.prop_flat_map(|transactions| { .prop_flat_map(|transactions| {
let indices_to_remove = let indices_to_remove =
vec(any::<bool>(), 1..=transactions.len()).prop_map(|removal_markers| { vec(any::<bool>(), 1..=transactions.len()).prop_map(|removal_markers| {
@ -851,7 +858,7 @@ impl Arbitrary for MultipleTransactionRemovalTestInput {
.prop_flat_map(|(transactions, indices_to_remove)| { .prop_flat_map(|(transactions, indices_to_remove)| {
let wtx_ids_to_remove: HashSet<_> = indices_to_remove let wtx_ids_to_remove: HashSet<_> = indices_to_remove
.iter() .iter()
.map(|&index| transactions[index].id) .map(|&index| transactions[index].transaction.id)
.collect(); .collect();
let mined_ids_to_remove: HashSet<transaction::Hash> = wtx_ids_to_remove let mined_ids_to_remove: HashSet<transaction::Hash> = wtx_ids_to_remove
@ -878,7 +885,7 @@ impl Arbitrary for MultipleTransactionRemovalTestInput {
impl MultipleTransactionRemovalTestInput { impl MultipleTransactionRemovalTestInput {
/// Iterate over all transactions generated as input. /// Iterate over all transactions generated as input.
pub fn transactions(&self) -> impl Iterator<Item = &UnminedTx> + '_ { pub fn transactions(&self) -> impl Iterator<Item = &VerifiedUnminedTx> + '_ {
match self { match self {
RemoveExact { transactions, .. } | RemoveSameEffects { transactions, .. } => { RemoveExact { transactions, .. } | RemoveSameEffects { transactions, .. } => {
transactions.iter() transactions.iter()
@ -898,7 +905,7 @@ impl MultipleTransactionRemovalTestInput {
mined_ids_to_remove, mined_ids_to_remove,
} => transactions } => transactions
.iter() .iter()
.map(|transaction| transaction.id) .map(|transaction| transaction.transaction.id)
.filter(|id| mined_ids_to_remove.contains(&id.mined_id())) .filter(|id| mined_ids_to_remove.contains(&id.mined_id()))
.collect(), .collect(),
} }

View File

@ -1,17 +1,18 @@
use std::iter; use std::iter;
use crate::components::mempool::Mempool; use color_eyre::eyre::Result;
use super::{super::*, unmined_transactions_in_blocks};
use zebra_chain::{ use zebra_chain::{
amount::Amount,
block::{Block, Height}, block::{Block, Height},
parameters::Network, parameters::Network,
serialization::ZcashDeserializeInto, serialization::ZcashDeserializeInto,
transaction::UnminedTxId, transaction::{UnminedTxId, VerifiedUnminedTx},
}; };
use color_eyre::eyre::Result; use crate::components::mempool::{
storage::tests::unmined_transactions_in_blocks, storage::*, Mempool,
};
#[test] #[test]
fn mempool_storage_crud_exact_mainnet() { fn mempool_storage_crud_exact_mainnet() {
@ -31,14 +32,14 @@ fn mempool_storage_crud_exact_mainnet() {
let _ = storage.insert(unmined_tx.clone()); let _ = storage.insert(unmined_tx.clone());
// Check that it is in the mempool, and not rejected. // Check that it is in the mempool, and not rejected.
assert!(storage.contains_transaction_exact(&unmined_tx.id)); assert!(storage.contains_transaction_exact(&unmined_tx.transaction.id));
// Remove tx // Remove tx
let removal_count = storage.remove_exact(&iter::once(unmined_tx.id).collect()); let removal_count = storage.remove_exact(&iter::once(unmined_tx.transaction.id).collect());
// Check that it is /not/ in the mempool. // Check that it is /not/ in the mempool.
assert_eq!(removal_count, 1); assert_eq!(removal_count, 1);
assert!(!storage.contains_transaction_exact(&unmined_tx.id)); assert!(!storage.contains_transaction_exact(&unmined_tx.transaction.id));
} }
#[test] #[test]
@ -59,15 +60,15 @@ fn mempool_storage_crud_same_effects_mainnet() {
let _ = storage.insert(unmined_tx.clone()); let _ = storage.insert(unmined_tx.clone());
// Check that it is in the mempool, and not rejected. // Check that it is in the mempool, and not rejected.
assert!(storage.contains_transaction_exact(&unmined_tx.id)); assert!(storage.contains_transaction_exact(&unmined_tx.transaction.id));
// Remove tx // Remove tx
let removal_count = let removal_count =
storage.remove_same_effects(&iter::once(unmined_tx.id.mined_id()).collect()); storage.remove_same_effects(&iter::once(unmined_tx.transaction.id.mined_id()).collect());
// Check that it is /not/ in the mempool. // Check that it is /not/ in the mempool.
assert_eq!(removal_count, 1); assert_eq!(removal_count, 1);
assert!(!storage.contains_transaction_exact(&unmined_tx.id)); assert!(!storage.contains_transaction_exact(&unmined_tx.transaction.id));
} }
#[test] #[test]
@ -110,29 +111,35 @@ fn mempool_storage_basic_for_network(network: Network) -> Result<()> {
// Make sure the last MEMPOOL_SIZE transactions we sent are in the verified // Make sure the last MEMPOOL_SIZE transactions we sent are in the verified
for tx in expected_in_mempool { for tx in expected_in_mempool {
assert!(storage.contains_transaction_exact(&tx.id)); assert!(storage.contains_transaction_exact(&tx.transaction.id));
} }
// Anything greater should not be in the verified // Anything greater should not be in the verified
for tx in expected_to_be_rejected { for tx in expected_to_be_rejected {
assert!(!storage.contains_transaction_exact(&tx.id)); assert!(!storage.contains_transaction_exact(&tx.transaction.id));
} }
// Query all the ids we have for rejected, get back `total - MEMPOOL_SIZE` // Query all the ids we have for rejected, get back `total - MEMPOOL_SIZE`
let all_ids: HashSet<UnminedTxId> = unmined_transactions.iter().map(|tx| tx.id).collect(); let all_ids: HashSet<UnminedTxId> = unmined_transactions
.iter()
.map(|tx| tx.transaction.id)
.collect();
// Convert response to a `HashSet`, because the order of the response doesn't matter. // Convert response to a `HashSet`, because the order of the response doesn't matter.
let rejected_response: HashSet<UnminedTxId> = let rejected_response: HashSet<UnminedTxId> =
storage.rejected_transactions(all_ids).into_iter().collect(); storage.rejected_transactions(all_ids).into_iter().collect();
let rejected_ids = expected_to_be_rejected.iter().map(|tx| tx.id).collect(); let rejected_ids = expected_to_be_rejected
.iter()
.map(|tx| tx.transaction.id)
.collect();
assert_eq!(rejected_response, rejected_ids); assert_eq!(rejected_response, rejected_ids);
// Make sure the first id stored is now rejected // Make sure the first id stored is now rejected
assert!(storage.contains_rejected(&expected_to_be_rejected[0].id)); assert!(storage.contains_rejected(&expected_to_be_rejected[0].transaction.id));
// Make sure the last id stored is not rejected // Make sure the last id stored is not rejected
assert!(!storage.contains_rejected(&expected_in_mempool[0].id)); assert!(!storage.contains_rejected(&expected_in_mempool[0].transaction.id));
Ok(()) Ok(())
} }
@ -169,8 +176,8 @@ fn mempool_expired_basic_for_network(network: Network) -> Result<()> {
let tx_id = tx.unmined_id(); let tx_id = tx.unmined_id();
// Insert the transaction into the mempool // Insert the transaction into the mempool, with a fake zero miner fee
storage.insert(UnminedTx::from(tx))?; storage.insert(VerifiedUnminedTx::new(tx.into(), Amount::zero()))?;
assert_eq!(storage.transaction_count(), 1); assert_eq!(storage.transaction_count(), 1);

View File

@ -6,7 +6,7 @@ use std::{
use zebra_chain::{ use zebra_chain::{
orchard, sapling, sprout, orchard, sapling, sprout,
transaction::{Transaction, UnminedTx, UnminedTxId}, transaction::{Transaction, UnminedTx, UnminedTxId, VerifiedUnminedTx},
transparent, transparent,
}; };
@ -24,7 +24,7 @@ use super::super::SameEffectsTipRejectionError;
#[derive(Default)] #[derive(Default)]
pub struct VerifiedSet { pub struct VerifiedSet {
/// The set of verified transactions in the mempool. /// The set of verified transactions in the mempool.
transactions: VecDeque<UnminedTx>, transactions: VecDeque<VerifiedUnminedTx>,
/// The total size of the transactions in the mempool if they were /// The total size of the transactions in the mempool if they were
/// serialized. /// serialized.
@ -53,7 +53,7 @@ impl Drop for VerifiedSet {
impl VerifiedSet { impl VerifiedSet {
/// Returns an iterator over the transactions in the set. /// Returns an iterator over the transactions in the set.
pub fn transactions(&self) -> impl Iterator<Item = &UnminedTx> + '_ { pub fn transactions(&self) -> impl Iterator<Item = &UnminedTx> + '_ {
self.transactions.iter() self.transactions.iter().map(|tx| &tx.transaction)
} }
/// Returns the number of verified transactions in the set. /// Returns the number of verified transactions in the set.
@ -64,7 +64,7 @@ impl VerifiedSet {
/// Returns `true` if the set of verified transactions contains the transaction with the /// Returns `true` if the set of verified transactions contains the transaction with the
/// specified `id. /// specified `id.
pub fn contains(&self, id: &UnminedTxId) -> bool { pub fn contains(&self, id: &UnminedTxId) -> bool {
self.transactions.iter().any(|tx| &tx.id == id) self.transactions.iter().any(|tx| &tx.transaction.id == id)
} }
/// Clear the set of verified transactions. /// Clear the set of verified transactions.
@ -87,13 +87,16 @@ impl VerifiedSet {
/// ///
/// Two transactions have a spend conflict if they spend the same UTXO or if they reveal the /// Two transactions have a spend conflict if they spend the same UTXO or if they reveal the
/// same nullifier. /// same nullifier.
pub fn insert(&mut self, transaction: UnminedTx) -> Result<(), SameEffectsTipRejectionError> { pub fn insert(
if self.has_spend_conflicts(&transaction) { &mut self,
transaction: VerifiedUnminedTx,
) -> Result<(), SameEffectsTipRejectionError> {
if self.has_spend_conflicts(&transaction.transaction) {
return Err(SameEffectsTipRejectionError::SpendConflict); return Err(SameEffectsTipRejectionError::SpendConflict);
} }
self.cache_outputs_from(&transaction.transaction); self.cache_outputs_from(&transaction.transaction.transaction);
self.transactions_serialized_size += transaction.size; self.transactions_serialized_size += transaction.transaction.size;
self.transactions.push_front(transaction); self.transactions.push_front(transaction);
self.update_metrics(); self.update_metrics();
@ -102,7 +105,7 @@ impl VerifiedSet {
} }
/// Evict one transaction from the set to open space for another transaction. /// Evict one transaction from the set to open space for another transaction.
pub fn evict_one(&mut self) -> Option<UnminedTx> { pub fn evict_one(&mut self) -> Option<VerifiedUnminedTx> {
if self.transactions.is_empty() { if self.transactions.is_empty() {
None None
} else { } else {
@ -116,7 +119,7 @@ impl VerifiedSet {
/// Removes all transactions in the set that match the `predicate`. /// Removes all transactions in the set that match the `predicate`.
/// ///
/// Returns the amount of transactions removed. /// Returns the amount of transactions removed.
pub fn remove_all_that(&mut self, predicate: impl Fn(&UnminedTx) -> bool) -> usize { pub fn remove_all_that(&mut self, predicate: impl Fn(&VerifiedUnminedTx) -> bool) -> usize {
// Clippy suggests to remove the `collect` and the `into_iter` further down. However, it is // Clippy suggests to remove the `collect` and the `into_iter` further down. However, it is
// unable to detect that when that is done, there is a borrow conflict. What happens is the // unable to detect that when that is done, there is a borrow conflict. What happens is the
// iterator borrows `self.transactions` immutably, but it also need to be borrowed mutably // iterator borrows `self.transactions` immutably, but it also need to be borrowed mutably
@ -144,14 +147,14 @@ impl VerifiedSet {
/// Removes a transaction from the set. /// Removes a transaction from the set.
/// ///
/// Also removes its outputs from the internal caches. /// Also removes its outputs from the internal caches.
fn remove(&mut self, transaction_index: usize) -> UnminedTx { fn remove(&mut self, transaction_index: usize) -> VerifiedUnminedTx {
let removed_tx = self let removed_tx = self
.transactions .transactions
.remove(transaction_index) .remove(transaction_index)
.expect("invalid transaction index"); .expect("invalid transaction index");
self.transactions_serialized_size -= removed_tx.size; self.transactions_serialized_size -= removed_tx.transaction.size;
self.remove_outputs(&removed_tx); self.remove_outputs(&removed_tx.transaction);
self.update_metrics(); self.update_metrics();

View File

@ -4,7 +4,7 @@ use proptest::prelude::*;
use tokio::time; use tokio::time;
use tower::{buffer::Buffer, util::BoxService}; use tower::{buffer::Buffer, util::BoxService};
use zebra_chain::{parameters::Network, transaction::UnminedTx}; use zebra_chain::{parameters::Network, transaction::VerifiedUnminedTx};
use zebra_consensus::{error::TransactionError, transaction as tx}; use zebra_consensus::{error::TransactionError, transaction as tx};
use zebra_network as zn; use zebra_network as zn;
use zebra_state::{self as zs, ChainTipBlock, ChainTipSender}; use zebra_state::{self as zs, ChainTipBlock, ChainTipSender};
@ -29,7 +29,7 @@ proptest! {
#[test] #[test]
fn storage_is_cleared_on_chain_reset( fn storage_is_cleared_on_chain_reset(
network in any::<Network>(), network in any::<Network>(),
transaction in any::<UnminedTx>(), transaction in any::<VerifiedUnminedTx>(),
chain_tip in any::<ChainTipBlock>(), chain_tip in any::<ChainTipBlock>(),
) { ) {
let runtime = tokio::runtime::Builder::new_current_thread() let runtime = tokio::runtime::Builder::new_current_thread()
@ -83,7 +83,7 @@ proptest! {
#[test] #[test]
fn storage_is_cleared_if_syncer_falls_behind( fn storage_is_cleared_if_syncer_falls_behind(
network in any::<Network>(), network in any::<Network>(),
transaction in any::<UnminedTx>(), transaction in any::<VerifiedUnminedTx>(),
) { ) {
let runtime = tokio::runtime::Builder::new_current_thread() let runtime = tokio::runtime::Builder::new_current_thread()
.enable_all() .enable_all()

View File

@ -81,7 +81,7 @@ async fn mempool_service_basic() -> Result<(), Report> {
// Make sure the transaction from the blockchain test vector is the same as the // Make sure the transaction from the blockchain test vector is the same as the
// response of `Request::TransactionsById` // response of `Request::TransactionsById`
assert_eq!(genesis_transaction, transactions[0]); assert_eq!(genesis_transaction.transaction, transactions[0]);
// Insert more transactions into the mempool storage. // Insert more transactions into the mempool storage.
// This will cause the genesis transaction to be moved into rejected. // This will cause the genesis transaction to be moved into rejected.
@ -113,7 +113,7 @@ async fn mempool_service_basic() -> Result<(), Report> {
.ready_and() .ready_and()
.await .await
.unwrap() .unwrap()
.call(Request::Queue(vec![last_transaction.id.into()])) .call(Request::Queue(vec![last_transaction.transaction.id.into()]))
.await .await
.unwrap(); .unwrap();
let queued_responses = match response { let queued_responses = match response {
@ -167,7 +167,7 @@ async fn mempool_queue() -> Result<(), Report> {
.ready_and() .ready_and()
.await .await
.unwrap() .unwrap()
.call(Request::Queue(vec![new_tx.id.into()])) .call(Request::Queue(vec![new_tx.transaction.id.into()]))
.await .await
.unwrap(); .unwrap();
let queued_responses = match response { let queued_responses = match response {
@ -182,7 +182,7 @@ async fn mempool_queue() -> Result<(), Report> {
.ready_and() .ready_and()
.await .await
.unwrap() .unwrap()
.call(Request::Queue(vec![stored_tx.id.into()])) .call(Request::Queue(vec![stored_tx.transaction.id.into()]))
.await .await
.unwrap(); .unwrap();
let queued_responses = match response { let queued_responses = match response {
@ -197,7 +197,7 @@ async fn mempool_queue() -> Result<(), Report> {
.ready_and() .ready_and()
.await .await
.unwrap() .unwrap()
.call(Request::Queue(vec![rejected_tx.id.into()])) .call(Request::Queue(vec![rejected_tx.transaction.id.into()]))
.await .await
.unwrap(); .unwrap();
let queued_responses = match response { let queued_responses = match response {
@ -256,7 +256,7 @@ async fn mempool_service_disabled() -> Result<(), Report> {
// Queue a transaction for download // Queue a transaction for download
// Use the ID of the last transaction in the list // Use the ID of the last transaction in the list
let txid = more_transactions.last().unwrap().id; let txid = more_transactions.last().unwrap().transaction.id;
let response = service let response = service
.ready_and() .ready_and()
.await .await
@ -522,7 +522,7 @@ async fn mempool_failed_verification_is_rejected() -> Result<(), Report> {
.ready_and() .ready_and()
.await .await
.unwrap() .unwrap()
.call(Request::Queue(vec![rejected_tx.clone().into()])); .call(Request::Queue(vec![rejected_tx.transaction.clone().into()]));
// Make the mock verifier return that the transaction is invalid. // Make the mock verifier return that the transaction is invalid.
let verification = tx_verifier.expect_request_that(|_| true).map(|responder| { let verification = tx_verifier.expect_request_that(|_| true).map(|responder| {
responder.respond(Err(TransactionError::BadBalance)); responder.respond(Err(TransactionError::BadBalance));
@ -549,7 +549,7 @@ async fn mempool_failed_verification_is_rejected() -> Result<(), Report> {
.ready_and() .ready_and()
.await .await
.unwrap() .unwrap()
.call(Request::Queue(vec![rejected_tx.id.into()])) .call(Request::Queue(vec![rejected_tx.transaction.id.into()]))
.await .await
.unwrap(); .unwrap();
let queued_responses = match response { let queued_responses = match response {
@ -604,7 +604,10 @@ async fn mempool_failed_download_is_not_rejected() -> Result<(), Report> {
.ready_and() .ready_and()
.await .await
.unwrap() .unwrap()
.call(Request::Queue(vec![rejected_valid_tx.id.into()])); .call(Request::Queue(vec![rejected_valid_tx
.transaction
.id
.into()]));
// Make the mock peer set return that the download failed. // Make the mock peer set return that the download failed.
let verification = peer_set let verification = peer_set
.expect_request_that(|r| matches!(r, zn::Request::TransactionsById(_))) .expect_request_that(|r| matches!(r, zn::Request::TransactionsById(_)))
@ -633,7 +636,10 @@ async fn mempool_failed_download_is_not_rejected() -> Result<(), Report> {
.ready_and() .ready_and()
.await .await
.unwrap() .unwrap()
.call(Request::Queue(vec![rejected_valid_tx.id.into()])) .call(Request::Queue(vec![rejected_valid_tx
.transaction
.id
.into()]))
.await .await
.unwrap(); .unwrap();
let queued_responses = match response { let queued_responses = match response {