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:
parent
4648f8fc70
commit
2d129414e0
|
@ -25,7 +25,7 @@ pub use lock_time::LockTime;
|
|||
pub use memo::Memo;
|
||||
pub use sapling::FieldNotPresent;
|
||||
pub use sighash::{HashType, SigHash};
|
||||
pub use unmined::{UnminedTx, UnminedTxId};
|
||||
pub use unmined::{UnminedTx, UnminedTxId, VerifiedUnminedTx};
|
||||
|
||||
use crate::{
|
||||
amount::{Amount, Error as AmountError, NegativeAllowed, NonNegative},
|
||||
|
|
|
@ -17,12 +17,14 @@ use std::{fmt, sync::Arc};
|
|||
#[cfg(any(test, feature = "proptest-impl"))]
|
||||
use proptest_derive::Arbitrary;
|
||||
|
||||
use crate::serialization::ZcashSerialize;
|
||||
|
||||
use super::{
|
||||
AuthDigest, Hash,
|
||||
Transaction::{self, *},
|
||||
WtxId,
|
||||
use crate::{
|
||||
amount::{Amount, NonNegative},
|
||||
serialization::ZcashSerialize,
|
||||
transaction::{
|
||||
AuthDigest, Hash,
|
||||
Transaction::{self, *},
|
||||
WtxId,
|
||||
},
|
||||
};
|
||||
|
||||
use UnminedTxId::*;
|
||||
|
@ -164,6 +166,9 @@ impl UnminedTxId {
|
|||
}
|
||||
|
||||
/// 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)]
|
||||
pub struct UnminedTx {
|
||||
/// A unique identifier for this unmined transaction.
|
||||
|
@ -176,6 +181,15 @@ pub struct UnminedTx {
|
|||
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,
|
||||
// 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,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,7 +23,9 @@ use zebra_chain::{
|
|||
parameters::{Network, NetworkUpgrade},
|
||||
primitives::Groth16Proof,
|
||||
sapling,
|
||||
transaction::{self, HashType, SigHash, Transaction, UnminedTx, UnminedTxId},
|
||||
transaction::{
|
||||
self, HashType, SigHash, Transaction, UnminedTx, UnminedTxId, VerifiedUnminedTx,
|
||||
},
|
||||
transparent,
|
||||
};
|
||||
|
||||
|
@ -97,13 +99,46 @@ pub enum Request {
|
|||
|
||||
/// The response type for the transaction verifier service.
|
||||
/// Responses identify the transaction that was verified.
|
||||
///
|
||||
/// [`Block`] requests can be uniquely identified by [`UnminedTxId::mined_id`],
|
||||
/// because the block's authorizing data root will be checked during contextual validation.
|
||||
///
|
||||
/// [`Mempool`] requests are uniquely identified by the [`UnminedTxId`]
|
||||
/// variant for their transaction version.
|
||||
pub type Response = (zebra_chain::transaction::UnminedTxId, Amount<NonNegative>);
|
||||
#[derive(Clone, Debug, Eq, PartialEq)]
|
||||
pub enum Response {
|
||||
/// A response to a block transaction verification request.
|
||||
Block {
|
||||
/// The witnessed transaction ID for this transaction.
|
||||
///
|
||||
/// [`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 {
|
||||
/// 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.
|
||||
pub fn tx_id(&self) -> UnminedTxId {
|
||||
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>
|
||||
where
|
||||
ZS: Service<zs::Request, Response = zs::Response, Error = BoxError> + Send + Clone + 'static,
|
||||
|
@ -174,8 +253,8 @@ where
|
|||
let network = self.network;
|
||||
|
||||
let tx = req.transaction();
|
||||
let id = req.tx_id();
|
||||
let span = tracing::debug_span!("tx", ?id);
|
||||
let tx_id = req.tx_id();
|
||||
let span = tracing::debug_span!("tx", ?tx_id);
|
||||
|
||||
async move {
|
||||
tracing::trace!(?req);
|
||||
|
@ -221,7 +300,7 @@ where
|
|||
sapling_shielded_data,
|
||||
..
|
||||
} => Self::verify_v4_transaction(
|
||||
req,
|
||||
&req,
|
||||
network,
|
||||
script_verifier,
|
||||
inputs,
|
||||
|
@ -235,7 +314,7 @@ where
|
|||
orchard_shielded_data,
|
||||
..
|
||||
} => Self::verify_v5_transaction(
|
||||
req,
|
||||
&req,
|
||||
network,
|
||||
script_verifier,
|
||||
inputs,
|
||||
|
@ -255,21 +334,32 @@ where
|
|||
// Get the `value_balance` to calculate the transaction fee.
|
||||
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.
|
||||
let mut miner_fee = None;
|
||||
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(tx_rtv) => tx_rtv,
|
||||
Ok(tx_rtv) => Some(tx_rtv),
|
||||
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)
|
||||
.boxed()
|
||||
|
@ -300,7 +390,7 @@ where
|
|||
/// - the Sprout `joinsplit_data` shielded data in the transaction
|
||||
/// - the `sapling_shielded_data` in the transaction
|
||||
fn verify_v4_transaction(
|
||||
request: Request,
|
||||
request: &Request,
|
||||
network: Network,
|
||||
script_verifier: script::Verifier<ZS>,
|
||||
inputs: &[transparent::Input],
|
||||
|
@ -316,7 +406,7 @@ where
|
|||
let shielded_sighash = tx.sighash(upgrade, HashType::ALL, None);
|
||||
|
||||
Ok(Self::verify_transparent_inputs_and_outputs(
|
||||
&request,
|
||||
request,
|
||||
network,
|
||||
script_verifier,
|
||||
inputs,
|
||||
|
@ -382,7 +472,7 @@ where
|
|||
/// - the sapling shielded data of the transaction, if any
|
||||
/// - the orchard shielded data of the transaction, if any
|
||||
fn verify_v5_transaction(
|
||||
request: Request,
|
||||
request: &Request,
|
||||
network: Network,
|
||||
script_verifier: script::Verifier<ZS>,
|
||||
inputs: &[transparent::Input],
|
||||
|
@ -398,7 +488,7 @@ where
|
|||
let shielded_sighash = transaction.sighash(upgrade, HashType::ALL, None);
|
||||
|
||||
Ok(Self::verify_transparent_inputs_and_outputs(
|
||||
&request,
|
||||
request,
|
||||
network,
|
||||
script_verifier,
|
||||
inputs,
|
||||
|
|
|
@ -275,7 +275,7 @@ async fn v5_transaction_is_accepted_after_nu5_activation_for_network(network: Ne
|
|||
.await;
|
||||
|
||||
assert_eq!(
|
||||
result.expect("expected a tx_id and tx_fee").0,
|
||||
result.expect("unexpected error response").tx_id(),
|
||||
expected_hash
|
||||
);
|
||||
}
|
||||
|
@ -324,7 +324,7 @@ async fn v4_transaction_with_transparent_transfer_is_accepted() {
|
|||
.await;
|
||||
|
||||
assert_eq!(
|
||||
result.expect("expected a tx_id and tx_fee").0,
|
||||
result.expect("unexpected error response").tx_id(),
|
||||
transaction_hash
|
||||
);
|
||||
}
|
||||
|
@ -370,7 +370,7 @@ async fn v4_coinbase_transaction_is_accepted() {
|
|||
.await;
|
||||
|
||||
assert_eq!(
|
||||
result.expect("expected a tx_id and tx_fee").0,
|
||||
result.expect("unexpected error response").tx_id(),
|
||||
transaction_hash
|
||||
);
|
||||
}
|
||||
|
@ -474,7 +474,7 @@ async fn v5_transaction_with_transparent_transfer_is_accepted() {
|
|||
.await;
|
||||
|
||||
assert_eq!(
|
||||
result.expect("expected a tx_id and tx_fee").0,
|
||||
result.expect("unexpected error response").tx_id(),
|
||||
transaction_hash
|
||||
);
|
||||
}
|
||||
|
@ -523,7 +523,7 @@ async fn v5_coinbase_transaction_is_accepted() {
|
|||
.await;
|
||||
|
||||
assert_eq!(
|
||||
result.expect("expected a tx_id and tx_fee").0,
|
||||
result.expect("unexpected error response").tx_id(),
|
||||
transaction_hash
|
||||
);
|
||||
}
|
||||
|
@ -643,7 +643,7 @@ fn v4_with_signed_sprout_transfer_is_accepted() {
|
|||
.await;
|
||||
|
||||
assert_eq!(
|
||||
result.expect("expected a tx_id and tx_fee").0,
|
||||
result.expect("unexpected error response").tx_id(),
|
||||
expected_hash
|
||||
);
|
||||
});
|
||||
|
@ -744,7 +744,7 @@ fn v4_with_sapling_spends() {
|
|||
.await;
|
||||
|
||||
assert_eq!(
|
||||
result.expect("expected a tx_id and tx_fee").0,
|
||||
result.expect("unexpected error response").tx_id(),
|
||||
expected_hash
|
||||
);
|
||||
});
|
||||
|
@ -788,7 +788,7 @@ fn v4_with_sapling_outputs_and_no_spends() {
|
|||
.await;
|
||||
|
||||
assert_eq!(
|
||||
result.expect("expected a tx_id and tx_fee").0,
|
||||
result.expect("unexpected error response").tx_id(),
|
||||
expected_hash
|
||||
);
|
||||
});
|
||||
|
@ -835,7 +835,7 @@ fn v5_with_sapling_spends() {
|
|||
.await;
|
||||
|
||||
assert_eq!(
|
||||
result.expect("expected a tx_id and tx_fee").0,
|
||||
result.expect("unexpected error response").tx_id(),
|
||||
expected_hash
|
||||
);
|
||||
});
|
||||
|
|
|
@ -12,7 +12,7 @@ use zebra_chain::{
|
|||
block::Block,
|
||||
parameters::Network,
|
||||
serialization::ZcashDeserializeInto,
|
||||
transaction::{UnminedTx, UnminedTxId},
|
||||
transaction::{UnminedTx, UnminedTxId, VerifiedUnminedTx},
|
||||
};
|
||||
use zebra_consensus::{error::TransactionError, transaction, Config as ConsensusConfig};
|
||||
use zebra_network::{AddressBook, Request, Response};
|
||||
|
@ -41,6 +41,10 @@ async fn mempool_requests_for_transactions() {
|
|||
tx_gossip_task_handle,
|
||||
) = 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();
|
||||
|
||||
// Test `Request::MempoolTransactionIds`
|
||||
|
@ -117,11 +121,17 @@ async fn mempool_push_transaction() -> Result<(), crate::BoxError> {
|
|||
.oneshot(Request::PushTransaction(tx.clone().into()));
|
||||
// Simulate a successful transaction verification
|
||||
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.
|
||||
let tx_fee = Amount::zero();
|
||||
responder.respond((txid, tx_fee));
|
||||
responder.respond(transaction::Response::from(VerifiedUnminedTx::new(
|
||||
transaction,
|
||||
Amount::zero(),
|
||||
)));
|
||||
});
|
||||
let (response, _) = futures::join!(request, verification);
|
||||
match response {
|
||||
|
@ -208,11 +218,17 @@ async fn mempool_advertise_transaction_ids() -> Result<(), crate::BoxError> {
|
|||
});
|
||||
// Simulate a successful transaction verification
|
||||
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.
|
||||
let tx_fee = Amount::zero();
|
||||
responder.respond((txid, tx_fee));
|
||||
responder.respond(transaction::Response::from(VerifiedUnminedTx::new(
|
||||
transaction,
|
||||
Amount::zero(),
|
||||
)));
|
||||
});
|
||||
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
|
||||
let verification = tx_verifier.expect_request_that(|_| true).map(|responder| {
|
||||
tx1_id = responder.request().tx_id();
|
||||
let transaction = responder
|
||||
.request()
|
||||
.clone()
|
||||
.into_mempool_transaction()
|
||||
.expect("unexpected non-mempool request");
|
||||
|
||||
// Set a dummy fee.
|
||||
let tx_fee = Amount::zero();
|
||||
responder.respond((tx1_id, tx_fee));
|
||||
responder.respond(transaction::Response::from(VerifiedUnminedTx::new(
|
||||
transaction,
|
||||
Amount::zero(),
|
||||
)));
|
||||
});
|
||||
let (response, _) = futures::join!(request, verification);
|
||||
match response {
|
||||
|
@ -391,10 +414,17 @@ async fn mempool_transaction_expiration() -> Result<(), crate::BoxError> {
|
|||
// Simulate a successful transaction verification
|
||||
let verification = tx_verifier.expect_request_that(|_| true).map(|responder| {
|
||||
tx2_id = responder.request().tx_id();
|
||||
let transaction = responder
|
||||
.request()
|
||||
.clone()
|
||||
.into_mempool_transaction()
|
||||
.expect("unexpected non-mempool request");
|
||||
|
||||
// Set a dummy fee.
|
||||
let tx_fee = Amount::zero();
|
||||
responder.respond((tx2_id, tx_fee));
|
||||
responder.respond(transaction::Response::from(VerifiedUnminedTx::new(
|
||||
transaction,
|
||||
Amount::zero(),
|
||||
)));
|
||||
});
|
||||
let (response, _) = futures::join!(request, verification);
|
||||
match response {
|
||||
|
@ -529,7 +559,7 @@ async fn setup(
|
|||
>,
|
||||
Buffer<BoxService<mempool::Request, mempool::Response, BoxError>, mempool::Request>,
|
||||
Vec<Arc<Block>>,
|
||||
Vec<UnminedTx>,
|
||||
Vec<VerifiedUnminedTx>,
|
||||
MockService<transaction::Request, transaction::Response, PanicAssertion, TransactionError>,
|
||||
MockService<Request, Response, PanicAssertion>,
|
||||
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.
|
||||
///
|
||||
/// 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.
|
||||
let genesis_transactions: Vec<_> = unmined_transactions_in_blocks(..=0, network)
|
||||
.take(1)
|
||||
|
|
|
@ -16,7 +16,7 @@ use tokio::{sync::oneshot, task::JoinHandle};
|
|||
use tower::{Service, ServiceExt};
|
||||
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_network as zn;
|
||||
use zebra_state as zs;
|
||||
|
@ -144,7 +144,7 @@ where
|
|||
/// A list of pending transaction download and verify tasks.
|
||||
#[pin]
|
||||
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
|
||||
|
@ -161,7 +161,7 @@ where
|
|||
ZS: Service<zs::Request, Response = zs::Response, Error = BoxError> + Send + Clone + 'static,
|
||||
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>> {
|
||||
let this = self.project();
|
||||
|
@ -177,7 +177,7 @@ where
|
|||
if let Some(join_result) = ready!(this.pending.poll_next(cx)) {
|
||||
match join_result.expect("transaction download and verify tasks must not panic") {
|
||||
Ok(tx) => {
|
||||
this.cancel_handles.remove(&tx.id);
|
||||
this.cancel_handles.remove(&tx.transaction.id);
|
||||
Poll::Ready(Some(Ok(tx)))
|
||||
}
|
||||
Err((e, hash)) => {
|
||||
|
@ -300,7 +300,10 @@ where
|
|||
transaction: tx.clone(),
|
||||
height,
|
||||
})
|
||||
.map_ok(|(_tx_id, _tx_fee)| tx)
|
||||
.map_ok(|rsp| {
|
||||
rsp.into_mempool_transaction()
|
||||
.expect("unexpected non-mempool response to mempool request")
|
||||
})
|
||||
.await;
|
||||
|
||||
tracing::debug!(?txid, ?result, "verified transaction for the mempool");
|
||||
|
|
|
@ -2,7 +2,7 @@ use std::collections::{HashMap, HashSet};
|
|||
|
||||
use thiserror::Error;
|
||||
|
||||
use zebra_chain::transaction::{self, UnminedTx, UnminedTxId};
|
||||
use zebra_chain::transaction::{self, UnminedTx, UnminedTxId, VerifiedUnminedTx};
|
||||
|
||||
use self::verified_set::VerifiedSet;
|
||||
use super::{downloads::TransactionDownloadVerifyError, MempoolError};
|
||||
|
@ -117,7 +117,7 @@ pub struct 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
|
||||
/// prevent this transaction from being inserted.
|
||||
|
@ -125,14 +125,14 @@ impl Storage {
|
|||
///
|
||||
/// If inserting this transaction evicts other transactions, they will be tracked
|
||||
/// as [`StorageRejectionError::RandomlyEvicted`].
|
||||
pub fn insert(&mut self, tx: UnminedTx) -> Result<UnminedTxId, MempoolError> {
|
||||
pub fn insert(&mut self, tx: VerifiedUnminedTx) -> Result<UnminedTxId, MempoolError> {
|
||||
// # Security
|
||||
//
|
||||
// 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.
|
||||
if let Some(error) = self.rejection_error(&tx.id) {
|
||||
if let Some(error) = self.rejection_error(&tx_id) {
|
||||
return Err(error);
|
||||
}
|
||||
|
||||
|
@ -140,7 +140,7 @@ impl Storage {
|
|||
//
|
||||
// Security: transactions must not get refreshed by new queries,
|
||||
// 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);
|
||||
}
|
||||
|
||||
|
@ -160,13 +160,13 @@ impl Storage {
|
|||
.expect("mempool is empty, but was expected to be full");
|
||||
|
||||
self.reject(
|
||||
evicted_tx.id,
|
||||
evicted_tx.transaction.id,
|
||||
SameEffectsChainRejectionError::RandomlyEvicted.into(),
|
||||
);
|
||||
|
||||
// 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)
|
||||
if evicted_tx.id == tx_id {
|
||||
if evicted_tx.transaction.id == tx_id {
|
||||
result = Err(SameEffectsChainRejectionError::RandomlyEvicted.into());
|
||||
}
|
||||
}
|
||||
|
@ -176,7 +176,7 @@ impl Storage {
|
|||
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:
|
||||
/// - non-malleable transaction ID, and
|
||||
|
@ -193,10 +193,10 @@ impl Storage {
|
|||
#[allow(dead_code)]
|
||||
pub fn remove_exact(&mut self, exact_wtxids: &HashSet<UnminedTxId>) -> usize {
|
||||
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,
|
||||
/// using only the non-malleable transaction ID.
|
||||
|
@ -211,7 +211,7 @@ impl Storage {
|
|||
/// Does not add or remove from the 'rejected' tracking set.
|
||||
pub fn remove_same_effects(&mut self, mined_ids: &HashSet<transaction::Hash>) -> usize {
|
||||
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.
|
||||
|
@ -254,7 +254,7 @@ impl Storage {
|
|||
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> {
|
||||
self.verified.transactions()
|
||||
}
|
||||
|
@ -265,7 +265,7 @@ impl Storage {
|
|||
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.
|
||||
///
|
||||
/// 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))
|
||||
}
|
||||
|
||||
/// Returns `true` if a [`UnminedTx`] exactly matching an [`UnminedTxId`] is in
|
||||
/// Returns `true` if a transaction exactly matching an [`UnminedTxId`] is in
|
||||
/// the mempool.
|
||||
///
|
||||
/// This matches the exact transaction, with identical blockchain effects, signatures, and proofs.
|
||||
|
@ -319,7 +319,7 @@ impl Storage {
|
|||
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.
|
||||
///
|
||||
/// This matches transactions based on each rejection list's matching rule.
|
||||
|
@ -355,7 +355,7 @@ impl Storage {
|
|||
.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.
|
||||
///
|
||||
/// This matches transactions based on each rejection list's matching rule.
|
||||
|
|
|
@ -1,7 +1,11 @@
|
|||
use std::ops::RangeBounds;
|
||||
|
||||
use zebra_chain::{
|
||||
block::Block, parameters::Network, serialization::ZcashDeserializeInto, transaction::UnminedTx,
|
||||
amount::Amount,
|
||||
block::Block,
|
||||
parameters::Network,
|
||||
serialization::ZcashDeserializeInto,
|
||||
transaction::{UnminedTx, VerifiedUnminedTx},
|
||||
};
|
||||
|
||||
mod prop;
|
||||
|
@ -10,7 +14,7 @@ mod vectors;
|
|||
pub fn unmined_transactions_in_blocks(
|
||||
block_height_range: impl RangeBounds<u32>,
|
||||
network: Network,
|
||||
) -> impl DoubleEndedIterator<Item = UnminedTx> {
|
||||
) -> impl DoubleEndedIterator<Item = VerifiedUnminedTx> {
|
||||
let blocks = match network {
|
||||
Network::Mainnet => zebra_test::vectors::MAINNET_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")
|
||||
});
|
||||
|
||||
// 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
|
||||
.flat_map(|block| block.transactions)
|
||||
.map(UnminedTx::from)
|
||||
.map(|transaction| VerifiedUnminedTx::new(transaction, Amount::zero()))
|
||||
}
|
||||
|
|
|
@ -4,6 +4,7 @@ use proptest::{collection::vec, prelude::*};
|
|||
use proptest_derive::Arbitrary;
|
||||
|
||||
use zebra_chain::{
|
||||
amount::Amount,
|
||||
at_least_one,
|
||||
fmt::{DisplayToDebug, SummaryDebug},
|
||||
orchard,
|
||||
|
@ -11,7 +12,7 @@ use zebra_chain::{
|
|||
sapling,
|
||||
serialization::AtLeastOne,
|
||||
sprout,
|
||||
transaction::{self, JoinSplitData, Transaction, UnminedTx, UnminedTxId},
|
||||
transaction::{self, JoinSplitData, Transaction, UnminedTxId, VerifiedUnminedTx},
|
||||
transparent, LedgerState,
|
||||
};
|
||||
|
||||
|
@ -23,7 +24,7 @@ use crate::components::mempool::{
|
|||
SameEffectsChainRejectionError,
|
||||
};
|
||||
|
||||
use self::MultipleTransactionRemovalTestInput::*;
|
||||
use MultipleTransactionRemovalTestInput::*;
|
||||
|
||||
/// 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 {
|
||||
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));
|
||||
|
||||
|
@ -95,7 +96,7 @@ proptest! {
|
|||
/// Test that the reject list length limits are applied when evicting transactions.
|
||||
#[test]
|
||||
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>()
|
||||
) {
|
||||
let mut storage = Storage::default();
|
||||
|
@ -119,7 +120,7 @@ proptest! {
|
|||
prop_assert_eq!(storage.rejected_transaction_count(), MAX_EVICTION_MEMORY_ENTRIES);
|
||||
|
||||
for transaction in transactions {
|
||||
let tx_id = transaction.id;
|
||||
let tx_id = transaction.transaction.id;
|
||||
|
||||
if storage.transaction_count() < MEMPOOL_SIZE {
|
||||
// The initial transactions should be successful
|
||||
|
@ -203,8 +204,8 @@ proptest! {
|
|||
];
|
||||
|
||||
for (transaction_to_accept, transaction_to_reject) in input_permutations {
|
||||
let id_to_accept = transaction_to_accept.id;
|
||||
let id_to_reject = transaction_to_reject.id;
|
||||
let id_to_accept = transaction_to_accept.transaction.id;
|
||||
let id_to_reject = transaction_to_reject.transaction.id;
|
||||
|
||||
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
|
||||
input_permutations
|
||||
{
|
||||
let first_id_to_accept = first_transaction_to_accept.id;
|
||||
let second_id_to_accept = second_transaction_to_accept.id;
|
||||
let id_to_reject = transaction_to_reject.id;
|
||||
let first_id_to_accept = first_transaction_to_accept.transaction.id;
|
||||
let second_id_to_accept = second_transaction_to_accept.transaction.id;
|
||||
let id_to_reject = transaction_to_reject.transaction.id;
|
||||
|
||||
prop_assert_eq!(
|
||||
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
|
||||
// inserted.
|
||||
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();
|
||||
|
||||
|
@ -354,7 +355,7 @@ enum SpendConflictTestInput {
|
|||
|
||||
impl SpendConflictTestInput {
|
||||
/// 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 {
|
||||
SpendConflictTestInput::V4 {
|
||||
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.
|
||||
pub fn unconflicting_transactions(self) -> (UnminedTx, UnminedTx) {
|
||||
pub fn unconflicting_transactions(self) -> (VerifiedUnminedTx, VerifiedUnminedTx) {
|
||||
let (mut first, mut second) = match self {
|
||||
SpendConflictTestInput::V4 { 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_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
|
||||
|
@ -820,12 +827,12 @@ impl OrchardSpendConflict {
|
|||
#[derive(Clone, Debug)]
|
||||
pub enum MultipleTransactionRemovalTestInput {
|
||||
RemoveExact {
|
||||
transactions: SummaryDebug<Vec<UnminedTx>>,
|
||||
transactions: SummaryDebug<Vec<VerifiedUnminedTx>>,
|
||||
wtx_ids_to_remove: SummaryDebug<HashSet<UnminedTxId>>,
|
||||
},
|
||||
|
||||
RemoveSameEffects {
|
||||
transactions: SummaryDebug<Vec<UnminedTx>>,
|
||||
transactions: SummaryDebug<Vec<VerifiedUnminedTx>>,
|
||||
mined_ids_to_remove: SummaryDebug<HashSet<transaction::Hash>>,
|
||||
},
|
||||
}
|
||||
|
@ -834,7 +841,7 @@ impl Arbitrary for MultipleTransactionRemovalTestInput {
|
|||
type Parameters = ();
|
||||
|
||||
fn arbitrary_with(_: Self::Parameters) -> Self::Strategy {
|
||||
vec(any::<UnminedTx>(), 1..MEMPOOL_SIZE)
|
||||
vec(any::<VerifiedUnminedTx>(), 1..MEMPOOL_SIZE)
|
||||
.prop_flat_map(|transactions| {
|
||||
let indices_to_remove =
|
||||
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)| {
|
||||
let wtx_ids_to_remove: HashSet<_> = indices_to_remove
|
||||
.iter()
|
||||
.map(|&index| transactions[index].id)
|
||||
.map(|&index| transactions[index].transaction.id)
|
||||
.collect();
|
||||
|
||||
let mined_ids_to_remove: HashSet<transaction::Hash> = wtx_ids_to_remove
|
||||
|
@ -878,7 +885,7 @@ impl Arbitrary for MultipleTransactionRemovalTestInput {
|
|||
|
||||
impl MultipleTransactionRemovalTestInput {
|
||||
/// 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 {
|
||||
RemoveExact { transactions, .. } | RemoveSameEffects { transactions, .. } => {
|
||||
transactions.iter()
|
||||
|
@ -898,7 +905,7 @@ impl MultipleTransactionRemovalTestInput {
|
|||
mined_ids_to_remove,
|
||||
} => transactions
|
||||
.iter()
|
||||
.map(|transaction| transaction.id)
|
||||
.map(|transaction| transaction.transaction.id)
|
||||
.filter(|id| mined_ids_to_remove.contains(&id.mined_id()))
|
||||
.collect(),
|
||||
}
|
||||
|
|
|
@ -1,17 +1,18 @@
|
|||
use std::iter;
|
||||
|
||||
use crate::components::mempool::Mempool;
|
||||
|
||||
use super::{super::*, unmined_transactions_in_blocks};
|
||||
use color_eyre::eyre::Result;
|
||||
|
||||
use zebra_chain::{
|
||||
amount::Amount,
|
||||
block::{Block, Height},
|
||||
parameters::Network,
|
||||
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]
|
||||
fn mempool_storage_crud_exact_mainnet() {
|
||||
|
@ -31,14 +32,14 @@ fn mempool_storage_crud_exact_mainnet() {
|
|||
let _ = storage.insert(unmined_tx.clone());
|
||||
|
||||
// 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
|
||||
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.
|
||||
assert_eq!(removal_count, 1);
|
||||
assert!(!storage.contains_transaction_exact(&unmined_tx.id));
|
||||
assert!(!storage.contains_transaction_exact(&unmined_tx.transaction.id));
|
||||
}
|
||||
|
||||
#[test]
|
||||
|
@ -59,15 +60,15 @@ fn mempool_storage_crud_same_effects_mainnet() {
|
|||
let _ = storage.insert(unmined_tx.clone());
|
||||
|
||||
// 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
|
||||
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.
|
||||
assert_eq!(removal_count, 1);
|
||||
assert!(!storage.contains_transaction_exact(&unmined_tx.id));
|
||||
assert!(!storage.contains_transaction_exact(&unmined_tx.transaction.id));
|
||||
}
|
||||
|
||||
#[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
|
||||
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
|
||||
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`
|
||||
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.
|
||||
let rejected_response: HashSet<UnminedTxId> =
|
||||
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);
|
||||
|
||||
// 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
|
||||
assert!(!storage.contains_rejected(&expected_in_mempool[0].id));
|
||||
assert!(!storage.contains_rejected(&expected_in_mempool[0].transaction.id));
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
@ -169,8 +176,8 @@ fn mempool_expired_basic_for_network(network: Network) -> Result<()> {
|
|||
|
||||
let tx_id = tx.unmined_id();
|
||||
|
||||
// Insert the transaction into the mempool
|
||||
storage.insert(UnminedTx::from(tx))?;
|
||||
// Insert the transaction into the mempool, with a fake zero miner fee
|
||||
storage.insert(VerifiedUnminedTx::new(tx.into(), Amount::zero()))?;
|
||||
|
||||
assert_eq!(storage.transaction_count(), 1);
|
||||
|
||||
|
|
|
@ -6,7 +6,7 @@ use std::{
|
|||
|
||||
use zebra_chain::{
|
||||
orchard, sapling, sprout,
|
||||
transaction::{Transaction, UnminedTx, UnminedTxId},
|
||||
transaction::{Transaction, UnminedTx, UnminedTxId, VerifiedUnminedTx},
|
||||
transparent,
|
||||
};
|
||||
|
||||
|
@ -24,7 +24,7 @@ use super::super::SameEffectsTipRejectionError;
|
|||
#[derive(Default)]
|
||||
pub struct VerifiedSet {
|
||||
/// 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
|
||||
/// serialized.
|
||||
|
@ -53,7 +53,7 @@ impl Drop for VerifiedSet {
|
|||
impl VerifiedSet {
|
||||
/// Returns an iterator over the transactions in the set.
|
||||
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.
|
||||
|
@ -64,7 +64,7 @@ impl VerifiedSet {
|
|||
/// Returns `true` if the set of verified transactions contains the transaction with the
|
||||
/// specified `id.
|
||||
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.
|
||||
|
@ -87,13 +87,16 @@ impl VerifiedSet {
|
|||
///
|
||||
/// Two transactions have a spend conflict if they spend the same UTXO or if they reveal the
|
||||
/// same nullifier.
|
||||
pub fn insert(&mut self, transaction: UnminedTx) -> Result<(), SameEffectsTipRejectionError> {
|
||||
if self.has_spend_conflicts(&transaction) {
|
||||
pub fn insert(
|
||||
&mut self,
|
||||
transaction: VerifiedUnminedTx,
|
||||
) -> Result<(), SameEffectsTipRejectionError> {
|
||||
if self.has_spend_conflicts(&transaction.transaction) {
|
||||
return Err(SameEffectsTipRejectionError::SpendConflict);
|
||||
}
|
||||
|
||||
self.cache_outputs_from(&transaction.transaction);
|
||||
self.transactions_serialized_size += transaction.size;
|
||||
self.cache_outputs_from(&transaction.transaction.transaction);
|
||||
self.transactions_serialized_size += transaction.transaction.size;
|
||||
self.transactions.push_front(transaction);
|
||||
|
||||
self.update_metrics();
|
||||
|
@ -102,7 +105,7 @@ impl VerifiedSet {
|
|||
}
|
||||
|
||||
/// 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() {
|
||||
None
|
||||
} else {
|
||||
|
@ -116,7 +119,7 @@ impl VerifiedSet {
|
|||
/// Removes all transactions in the set that match the `predicate`.
|
||||
///
|
||||
/// 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
|
||||
// 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
|
||||
|
@ -144,14 +147,14 @@ impl VerifiedSet {
|
|||
/// Removes a transaction from the set.
|
||||
///
|
||||
/// 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
|
||||
.transactions
|
||||
.remove(transaction_index)
|
||||
.expect("invalid transaction index");
|
||||
|
||||
self.transactions_serialized_size -= removed_tx.size;
|
||||
self.remove_outputs(&removed_tx);
|
||||
self.transactions_serialized_size -= removed_tx.transaction.size;
|
||||
self.remove_outputs(&removed_tx.transaction);
|
||||
|
||||
self.update_metrics();
|
||||
|
||||
|
|
|
@ -4,7 +4,7 @@ use proptest::prelude::*;
|
|||
use tokio::time;
|
||||
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_network as zn;
|
||||
use zebra_state::{self as zs, ChainTipBlock, ChainTipSender};
|
||||
|
@ -29,7 +29,7 @@ proptest! {
|
|||
#[test]
|
||||
fn storage_is_cleared_on_chain_reset(
|
||||
network in any::<Network>(),
|
||||
transaction in any::<UnminedTx>(),
|
||||
transaction in any::<VerifiedUnminedTx>(),
|
||||
chain_tip in any::<ChainTipBlock>(),
|
||||
) {
|
||||
let runtime = tokio::runtime::Builder::new_current_thread()
|
||||
|
@ -83,7 +83,7 @@ proptest! {
|
|||
#[test]
|
||||
fn storage_is_cleared_if_syncer_falls_behind(
|
||||
network in any::<Network>(),
|
||||
transaction in any::<UnminedTx>(),
|
||||
transaction in any::<VerifiedUnminedTx>(),
|
||||
) {
|
||||
let runtime = tokio::runtime::Builder::new_current_thread()
|
||||
.enable_all()
|
||||
|
|
|
@ -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
|
||||
// response of `Request::TransactionsById`
|
||||
assert_eq!(genesis_transaction, transactions[0]);
|
||||
assert_eq!(genesis_transaction.transaction, transactions[0]);
|
||||
|
||||
// Insert more transactions into the mempool storage.
|
||||
// This will cause the genesis transaction to be moved into rejected.
|
||||
|
@ -113,7 +113,7 @@ async fn mempool_service_basic() -> Result<(), Report> {
|
|||
.ready_and()
|
||||
.await
|
||||
.unwrap()
|
||||
.call(Request::Queue(vec![last_transaction.id.into()]))
|
||||
.call(Request::Queue(vec![last_transaction.transaction.id.into()]))
|
||||
.await
|
||||
.unwrap();
|
||||
let queued_responses = match response {
|
||||
|
@ -167,7 +167,7 @@ async fn mempool_queue() -> Result<(), Report> {
|
|||
.ready_and()
|
||||
.await
|
||||
.unwrap()
|
||||
.call(Request::Queue(vec![new_tx.id.into()]))
|
||||
.call(Request::Queue(vec![new_tx.transaction.id.into()]))
|
||||
.await
|
||||
.unwrap();
|
||||
let queued_responses = match response {
|
||||
|
@ -182,7 +182,7 @@ async fn mempool_queue() -> Result<(), Report> {
|
|||
.ready_and()
|
||||
.await
|
||||
.unwrap()
|
||||
.call(Request::Queue(vec![stored_tx.id.into()]))
|
||||
.call(Request::Queue(vec![stored_tx.transaction.id.into()]))
|
||||
.await
|
||||
.unwrap();
|
||||
let queued_responses = match response {
|
||||
|
@ -197,7 +197,7 @@ async fn mempool_queue() -> Result<(), Report> {
|
|||
.ready_and()
|
||||
.await
|
||||
.unwrap()
|
||||
.call(Request::Queue(vec![rejected_tx.id.into()]))
|
||||
.call(Request::Queue(vec![rejected_tx.transaction.id.into()]))
|
||||
.await
|
||||
.unwrap();
|
||||
let queued_responses = match response {
|
||||
|
@ -256,7 +256,7 @@ async fn mempool_service_disabled() -> Result<(), Report> {
|
|||
|
||||
// Queue a transaction for download
|
||||
// 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
|
||||
.ready_and()
|
||||
.await
|
||||
|
@ -522,7 +522,7 @@ async fn mempool_failed_verification_is_rejected() -> Result<(), Report> {
|
|||
.ready_and()
|
||||
.await
|
||||
.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.
|
||||
let verification = tx_verifier.expect_request_that(|_| true).map(|responder| {
|
||||
responder.respond(Err(TransactionError::BadBalance));
|
||||
|
@ -549,7 +549,7 @@ async fn mempool_failed_verification_is_rejected() -> Result<(), Report> {
|
|||
.ready_and()
|
||||
.await
|
||||
.unwrap()
|
||||
.call(Request::Queue(vec![rejected_tx.id.into()]))
|
||||
.call(Request::Queue(vec![rejected_tx.transaction.id.into()]))
|
||||
.await
|
||||
.unwrap();
|
||||
let queued_responses = match response {
|
||||
|
@ -604,7 +604,10 @@ async fn mempool_failed_download_is_not_rejected() -> Result<(), Report> {
|
|||
.ready_and()
|
||||
.await
|
||||
.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.
|
||||
let verification = peer_set
|
||||
.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()
|
||||
.await
|
||||
.unwrap()
|
||||
.call(Request::Queue(vec![rejected_valid_tx.id.into()]))
|
||||
.call(Request::Queue(vec![rejected_valid_tx
|
||||
.transaction
|
||||
.id
|
||||
.into()]))
|
||||
.await
|
||||
.unwrap();
|
||||
let queued_responses = match response {
|
||||
|
|
Loading…
Reference in New Issue