mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
feat: add OP specific txpool (#6902)
This commit is contained in:
2
Cargo.lock
generated
2
Cargo.lock
generated
@ -6493,6 +6493,7 @@ version = "0.1.0-alpha.21"
|
||||
dependencies = [
|
||||
"clap",
|
||||
"eyre",
|
||||
"parking_lot 0.12.1",
|
||||
"reth-basic-payload-builder",
|
||||
"reth-db",
|
||||
"reth-network",
|
||||
@ -6502,6 +6503,7 @@ dependencies = [
|
||||
"reth-payload-builder",
|
||||
"reth-primitives",
|
||||
"reth-provider",
|
||||
"reth-revm",
|
||||
"reth-rpc",
|
||||
"reth-rpc-types",
|
||||
"reth-rpc-types-compat",
|
||||
|
||||
@ -25,12 +25,14 @@ reth-tracing.workspace = true
|
||||
reth-provider.workspace = true
|
||||
reth-transaction-pool.workspace = true
|
||||
reth-network.workspace = true
|
||||
reth-revm.workspace = true
|
||||
revm.workspace = true
|
||||
|
||||
# misc
|
||||
clap.workspace = true
|
||||
serde.workspace = true
|
||||
eyre.workspace = true
|
||||
parking_lot.workspace = true
|
||||
|
||||
[dev-dependencies]
|
||||
reth-db.workspace = true
|
||||
@ -45,4 +47,5 @@ optimism = [
|
||||
"reth-rpc-types-compat/optimism",
|
||||
"reth-rpc/optimism",
|
||||
"reth-optimism-payload-builder/optimism",
|
||||
"reth-revm/optimism",
|
||||
]
|
||||
|
||||
@ -9,6 +9,9 @@
|
||||
// The `optimism` feature must be enabled to use this crate.
|
||||
#![cfg(feature = "optimism")]
|
||||
|
||||
/// CLI argument parsing for the optimism node.
|
||||
pub mod args;
|
||||
|
||||
/// Exports optimism-specific implementations of the [EngineTypes](reth_node_api::EngineTypes)
|
||||
/// trait.
|
||||
pub mod engine;
|
||||
@ -19,6 +22,7 @@ pub use engine::OptimismEngineTypes;
|
||||
pub mod evm;
|
||||
pub use evm::OptimismEvmConfig;
|
||||
|
||||
pub mod args;
|
||||
pub mod node;
|
||||
pub use node::OptimismNode;
|
||||
|
||||
pub mod txpool;
|
||||
|
||||
@ -1,6 +1,10 @@
|
||||
//! Optimism Node types config.
|
||||
|
||||
use crate::{args::RollupArgs, OptimismEngineTypes, OptimismEvmConfig};
|
||||
use crate::{
|
||||
args::RollupArgs,
|
||||
txpool::{OpTransactionPool, OpTransactionValidator},
|
||||
OptimismEngineTypes, OptimismEvmConfig,
|
||||
};
|
||||
use reth_basic_payload_builder::{BasicPayloadJobGenerator, BasicPayloadJobGeneratorConfig};
|
||||
use reth_network::{NetworkHandle, NetworkManager};
|
||||
use reth_node_builder::{
|
||||
@ -12,7 +16,7 @@ use reth_payload_builder::{PayloadBuilderHandle, PayloadBuilderService};
|
||||
use reth_provider::CanonStateSubscriptions;
|
||||
use reth_tracing::tracing::{debug, info};
|
||||
use reth_transaction_pool::{
|
||||
blobstore::DiskFileBlobStore, EthTransactionPool, TransactionPool,
|
||||
blobstore::DiskFileBlobStore, CoinbaseTipOrdering, TransactionPool,
|
||||
TransactionValidationTaskExecutor,
|
||||
};
|
||||
|
||||
@ -84,11 +88,12 @@ impl<Node> PoolBuilder<Node> for OptimismPoolBuilder
|
||||
where
|
||||
Node: FullNodeTypes,
|
||||
{
|
||||
type Pool = EthTransactionPool<Node::Provider, DiskFileBlobStore>;
|
||||
type Pool = OpTransactionPool<Node::Provider, DiskFileBlobStore>;
|
||||
|
||||
async fn build_pool(self, ctx: &BuilderContext<Node>) -> eyre::Result<Self::Pool> {
|
||||
let data_dir = ctx.data_dir();
|
||||
let blob_store = DiskFileBlobStore::open(data_dir.blobstore_path(), Default::default())?;
|
||||
|
||||
let validator = TransactionValidationTaskExecutor::eth_builder(ctx.chain_spec())
|
||||
.with_head_timestamp(ctx.head().timestamp)
|
||||
.kzg_settings(ctx.kzg_settings()?)
|
||||
@ -97,10 +102,15 @@ where
|
||||
ctx.provider().clone(),
|
||||
ctx.task_executor().clone(),
|
||||
blob_store.clone(),
|
||||
);
|
||||
)
|
||||
.map(OpTransactionValidator::new);
|
||||
|
||||
let transaction_pool =
|
||||
reth_transaction_pool::Pool::eth_pool(validator, blob_store, ctx.pool_config());
|
||||
let transaction_pool = reth_transaction_pool::Pool::new(
|
||||
validator,
|
||||
CoinbaseTipOrdering::default(),
|
||||
blob_store,
|
||||
ctx.pool_config(),
|
||||
);
|
||||
info!(target: "reth::cli", "Transaction pool initialized");
|
||||
let transactions_path = data_dir.txpool_transactions_path();
|
||||
|
||||
|
||||
247
crates/node-optimism/src/txpool.rs
Normal file
247
crates/node-optimism/src/txpool.rs
Normal file
@ -0,0 +1,247 @@
|
||||
//! OP transaction pool types
|
||||
use parking_lot::RwLock;
|
||||
use reth_primitives::{Block, ChainSpec, GotExpected, InvalidTransactionError, SealedBlock};
|
||||
use reth_provider::{BlockReaderIdExt, StateProviderFactory};
|
||||
use reth_revm::{optimism::RethL1BlockInfo, L1BlockInfo};
|
||||
use reth_transaction_pool::{
|
||||
CoinbaseTipOrdering, EthPoolTransaction, EthPooledTransaction, EthTransactionValidator, Pool,
|
||||
TransactionOrigin, TransactionValidationOutcome, TransactionValidationTaskExecutor,
|
||||
TransactionValidator,
|
||||
};
|
||||
use std::sync::{
|
||||
atomic::{AtomicU64, Ordering},
|
||||
Arc,
|
||||
};
|
||||
|
||||
/// Type alias for default optimism transaction pool
|
||||
pub type OpTransactionPool<Client, S> = Pool<
|
||||
TransactionValidationTaskExecutor<OpTransactionValidator<Client, EthPooledTransaction>>,
|
||||
CoinbaseTipOrdering<EthPooledTransaction>,
|
||||
S,
|
||||
>;
|
||||
|
||||
/// Validator for Optimism transactions.
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct OpTransactionValidator<Client, Tx> {
|
||||
/// The type that performs the actual validation.
|
||||
inner: EthTransactionValidator<Client, Tx>,
|
||||
/// Additional block info required for validation.
|
||||
block_info: Arc<OpL1BlockInfo>,
|
||||
}
|
||||
|
||||
impl<Client, Tx> OpTransactionValidator<Client, Tx> {
|
||||
/// Returns the configured chain spec
|
||||
pub fn chain_spec(&self) -> Arc<ChainSpec> {
|
||||
self.inner.chain_spec()
|
||||
}
|
||||
|
||||
/// Returns the current block timestamp.
|
||||
fn block_timestamp(&self) -> u64 {
|
||||
self.block_info.timestamp.load(Ordering::Relaxed)
|
||||
}
|
||||
}
|
||||
|
||||
impl<Client, Tx> OpTransactionValidator<Client, Tx>
|
||||
where
|
||||
Client: StateProviderFactory + BlockReaderIdExt,
|
||||
Tx: EthPoolTransaction,
|
||||
{
|
||||
/// Create a new [OpTransactionValidator].
|
||||
pub fn new(inner: EthTransactionValidator<Client, Tx>) -> Self {
|
||||
let this = Self::with_block_info(inner, OpL1BlockInfo::default());
|
||||
if let Ok(Some(block)) =
|
||||
this.inner.client().block_by_number_or_tag(reth_primitives::BlockNumberOrTag::Latest)
|
||||
{
|
||||
this.update_l1_block_info(&block);
|
||||
}
|
||||
|
||||
this
|
||||
}
|
||||
|
||||
/// Create a new [OpTransactionValidator] with the given [OpL1BlockInfo].
|
||||
pub fn with_block_info(
|
||||
inner: EthTransactionValidator<Client, Tx>,
|
||||
block_info: OpL1BlockInfo,
|
||||
) -> Self {
|
||||
Self { inner, block_info: Arc::new(block_info) }
|
||||
}
|
||||
|
||||
/// Update the L1 block info.
|
||||
fn update_l1_block_info(&self, block: &Block) {
|
||||
self.block_info.timestamp.store(block.timestamp, Ordering::Relaxed);
|
||||
let cost_addition = reth_revm::optimism::extract_l1_info(block).ok();
|
||||
*self.block_info.l1_block_info.write() = cost_addition;
|
||||
}
|
||||
|
||||
/// Validates a single transaction.
|
||||
///
|
||||
/// See also [TransactionValidator::validate_transaction]
|
||||
///
|
||||
/// This behaves the same as [EthTransactionValidator::validate_one], but in addition, ensures
|
||||
/// that the account has enough balance to cover the L1 gas cost.
|
||||
pub fn validate_one(
|
||||
&self,
|
||||
origin: TransactionOrigin,
|
||||
transaction: Tx,
|
||||
) -> TransactionValidationOutcome<Tx> {
|
||||
if transaction.is_deposit() || transaction.is_eip4844() {
|
||||
return TransactionValidationOutcome::Invalid(
|
||||
transaction,
|
||||
InvalidTransactionError::TxTypeNotSupported.into(),
|
||||
)
|
||||
}
|
||||
|
||||
let outcome = self.inner.validate_one(origin, transaction);
|
||||
|
||||
// ensure that the account has enough balance to cover the L1 gas cost
|
||||
if let TransactionValidationOutcome::Valid {
|
||||
balance,
|
||||
state_nonce,
|
||||
transaction: valid_tx,
|
||||
propagate,
|
||||
} = outcome
|
||||
{
|
||||
let Some(l1_block_info) = self.block_info.l1_block_info.read().clone() else {
|
||||
return TransactionValidationOutcome::Error(
|
||||
*valid_tx.hash(),
|
||||
"L1BlockInfoError".into(),
|
||||
)
|
||||
};
|
||||
|
||||
let mut encoded = reth_primitives::bytes::BytesMut::default();
|
||||
valid_tx.transaction().to_recovered_transaction().encode_enveloped(&mut encoded);
|
||||
|
||||
let cost_addition = match l1_block_info.l1_tx_data_fee(
|
||||
&self.chain_spec(),
|
||||
self.block_timestamp(),
|
||||
encoded.as_ref(),
|
||||
valid_tx.transaction().is_deposit(),
|
||||
) {
|
||||
Ok(cost) => cost,
|
||||
Err(err) => {
|
||||
return TransactionValidationOutcome::Error(*valid_tx.hash(), Box::new(err))
|
||||
}
|
||||
};
|
||||
let cost = valid_tx.transaction().cost().saturating_add(cost_addition);
|
||||
|
||||
// Checks for max cost
|
||||
if cost > balance {
|
||||
return TransactionValidationOutcome::Invalid(
|
||||
valid_tx.into_transaction(),
|
||||
InvalidTransactionError::InsufficientFunds(
|
||||
GotExpected { got: balance, expected: cost }.into(),
|
||||
)
|
||||
.into(),
|
||||
)
|
||||
}
|
||||
|
||||
return TransactionValidationOutcome::Valid {
|
||||
balance,
|
||||
state_nonce,
|
||||
transaction: valid_tx,
|
||||
propagate,
|
||||
}
|
||||
}
|
||||
|
||||
outcome
|
||||
}
|
||||
|
||||
/// Validates all given transactions.
|
||||
///
|
||||
/// Returns all outcomes for the given transactions in the same order.
|
||||
///
|
||||
/// See also [Self::validate_one]
|
||||
pub fn validate_all(
|
||||
&self,
|
||||
transactions: Vec<(TransactionOrigin, Tx)>,
|
||||
) -> Vec<TransactionValidationOutcome<Tx>> {
|
||||
transactions.into_iter().map(|(origin, tx)| self.validate_one(origin, tx)).collect()
|
||||
}
|
||||
}
|
||||
|
||||
impl<Client, Tx> TransactionValidator for OpTransactionValidator<Client, Tx>
|
||||
where
|
||||
Client: StateProviderFactory + BlockReaderIdExt,
|
||||
Tx: EthPoolTransaction,
|
||||
{
|
||||
type Transaction = Tx;
|
||||
|
||||
async fn validate_transaction(
|
||||
&self,
|
||||
origin: TransactionOrigin,
|
||||
transaction: Self::Transaction,
|
||||
) -> TransactionValidationOutcome<Self::Transaction> {
|
||||
self.validate_one(origin, transaction)
|
||||
}
|
||||
|
||||
async fn validate_transactions(
|
||||
&self,
|
||||
transactions: Vec<(TransactionOrigin, Self::Transaction)>,
|
||||
) -> Vec<TransactionValidationOutcome<Self::Transaction>> {
|
||||
self.validate_all(transactions)
|
||||
}
|
||||
|
||||
fn on_new_head_block(&self, new_tip_block: &SealedBlock) {
|
||||
self.inner.on_new_head_block(new_tip_block);
|
||||
self.update_l1_block_info(&new_tip_block.clone().unseal());
|
||||
}
|
||||
}
|
||||
|
||||
/// Tracks additional infos for the current block.
|
||||
#[derive(Debug, Default)]
|
||||
pub struct OpL1BlockInfo {
|
||||
/// The current L1 block info.
|
||||
l1_block_info: RwLock<Option<L1BlockInfo>>,
|
||||
/// Current block timestamp.
|
||||
timestamp: AtomicU64,
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use crate::txpool::OpTransactionValidator;
|
||||
use reth_primitives::{
|
||||
Signature, Transaction, TransactionKind, TransactionSigned, TransactionSignedEcRecovered,
|
||||
TxDeposit, MAINNET, U256,
|
||||
};
|
||||
use reth_provider::test_utils::MockEthProvider;
|
||||
use reth_transaction_pool::{
|
||||
blobstore::InMemoryBlobStore, validate::EthTransactionValidatorBuilder,
|
||||
EthPooledTransaction, TransactionOrigin, TransactionValidationOutcome,
|
||||
};
|
||||
|
||||
#[test]
|
||||
fn validate_optimism_transaction() {
|
||||
let client = MockEthProvider::default();
|
||||
let validator = EthTransactionValidatorBuilder::new(MAINNET.clone())
|
||||
.no_shanghai()
|
||||
.no_cancun()
|
||||
.build(client, InMemoryBlobStore::default());
|
||||
let validator = OpTransactionValidator::new(validator);
|
||||
|
||||
let origin = TransactionOrigin::External;
|
||||
let signer = Default::default();
|
||||
let deposit_tx = Transaction::Deposit(TxDeposit {
|
||||
source_hash: Default::default(),
|
||||
from: signer,
|
||||
to: TransactionKind::Create,
|
||||
mint: None,
|
||||
value: U256::ZERO,
|
||||
gas_limit: 0u64,
|
||||
is_system_transaction: false,
|
||||
input: Default::default(),
|
||||
});
|
||||
let signature = Signature::default();
|
||||
let signed_tx = TransactionSigned::from_transaction_and_signature(deposit_tx, signature);
|
||||
let signed_recovered =
|
||||
TransactionSignedEcRecovered::from_signed_transaction(signed_tx, signer);
|
||||
let len = signed_recovered.length_without_header();
|
||||
let pooled_tx = EthPooledTransaction::new(signed_recovered, len);
|
||||
let outcome = validator.validate_one(origin, pooled_tx);
|
||||
|
||||
let err = match outcome {
|
||||
TransactionValidationOutcome::Invalid(_, err) => err,
|
||||
_ => panic!("Expected invalid transaction"),
|
||||
};
|
||||
assert_eq!(err.to_string(), "transaction type not supported");
|
||||
}
|
||||
}
|
||||
@ -1,5 +1,6 @@
|
||||
//! Ethereum transaction validator.
|
||||
|
||||
use super::constants::DEFAULT_MAX_TX_INPUT_BYTES;
|
||||
use crate::{
|
||||
blobstore::BlobStore,
|
||||
error::{Eip4844PoolTransactionError, InvalidPoolTransactionError},
|
||||
@ -26,11 +27,6 @@ use std::{
|
||||
};
|
||||
use tokio::sync::Mutex;
|
||||
|
||||
#[cfg(feature = "optimism")]
|
||||
use reth_revm::optimism::RethL1BlockInfo;
|
||||
|
||||
use super::constants::DEFAULT_MAX_TX_INPUT_BYTES;
|
||||
|
||||
/// Validator for Ethereum transactions.
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct EthTransactionValidator<Client, T> {
|
||||
@ -38,6 +34,18 @@ pub struct EthTransactionValidator<Client, T> {
|
||||
inner: Arc<EthTransactionValidatorInner<Client, T>>,
|
||||
}
|
||||
|
||||
impl<Client, Tx> EthTransactionValidator<Client, Tx> {
|
||||
/// Returns the configured chain spec
|
||||
pub fn chain_spec(&self) -> Arc<ChainSpec> {
|
||||
self.inner.chain_spec.clone()
|
||||
}
|
||||
|
||||
/// Returns the configured client
|
||||
pub fn client(&self) -> &Client {
|
||||
&self.inner.client
|
||||
}
|
||||
}
|
||||
|
||||
impl<Client, Tx> EthTransactionValidator<Client, Tx>
|
||||
where
|
||||
Client: StateProviderFactory + BlockReaderIdExt,
|
||||
@ -145,14 +153,6 @@ where
|
||||
origin: TransactionOrigin,
|
||||
mut transaction: Tx,
|
||||
) -> TransactionValidationOutcome<Tx> {
|
||||
#[cfg(feature = "optimism")]
|
||||
if transaction.is_deposit() || transaction.is_eip4844() {
|
||||
return TransactionValidationOutcome::Invalid(
|
||||
transaction,
|
||||
InvalidTransactionError::TxTypeNotSupported.into(),
|
||||
)
|
||||
}
|
||||
|
||||
// Checks for tx_type
|
||||
match transaction.tx_type() {
|
||||
LEGACY_TX_TYPE_ID => {
|
||||
@ -318,52 +318,8 @@ where
|
||||
)
|
||||
}
|
||||
|
||||
#[cfg(not(feature = "optimism"))]
|
||||
let cost = transaction.cost();
|
||||
|
||||
#[cfg(feature = "optimism")]
|
||||
let cost = {
|
||||
let block = match self
|
||||
.client
|
||||
.block_by_number_or_tag(reth_primitives::BlockNumberOrTag::Latest)
|
||||
{
|
||||
Ok(Some(block)) => block,
|
||||
Ok(None) => {
|
||||
return TransactionValidationOutcome::Error(
|
||||
*transaction.hash(),
|
||||
"Latest block should be found".into(),
|
||||
)
|
||||
}
|
||||
Err(err) => {
|
||||
return TransactionValidationOutcome::Error(*transaction.hash(), Box::new(err))
|
||||
}
|
||||
};
|
||||
|
||||
let mut encoded = reth_primitives::bytes::BytesMut::default();
|
||||
transaction.to_recovered_transaction().encode_enveloped(&mut encoded);
|
||||
let cost_addition = match reth_revm::optimism::extract_l1_info(&block).map(|info| {
|
||||
info.l1_tx_data_fee(
|
||||
&self.chain_spec,
|
||||
block.timestamp,
|
||||
&encoded,
|
||||
transaction.is_deposit(),
|
||||
)
|
||||
}) {
|
||||
Ok(Ok(cost)) => cost,
|
||||
Err(err) => {
|
||||
return TransactionValidationOutcome::Error(*transaction.hash(), Box::new(err))
|
||||
}
|
||||
_ => {
|
||||
return TransactionValidationOutcome::Error(
|
||||
*transaction.hash(),
|
||||
"L1BlockInfoError".into(),
|
||||
)
|
||||
}
|
||||
};
|
||||
|
||||
transaction.cost().saturating_add(cost_addition)
|
||||
};
|
||||
|
||||
// Checks for max cost
|
||||
if cost > account.balance {
|
||||
return TransactionValidationOutcome::Invalid(
|
||||
@ -800,47 +756,4 @@ mod tests {
|
||||
let tx = pool.get(transaction.hash());
|
||||
assert!(tx.is_some());
|
||||
}
|
||||
|
||||
#[cfg(feature = "optimism")]
|
||||
#[tokio::test(flavor = "multi_thread")]
|
||||
async fn validate_optimism_transaction() {
|
||||
use crate::{blobstore::InMemoryBlobStore, traits::EthPooledTransaction};
|
||||
use reth_primitives::{
|
||||
Signature, Transaction, TransactionKind, TransactionSigned,
|
||||
TransactionSignedEcRecovered, TxDeposit, MAINNET, U256,
|
||||
};
|
||||
use reth_provider::test_utils::MockEthProvider;
|
||||
use reth_tasks::TokioTaskExecutor;
|
||||
|
||||
let client = MockEthProvider::default();
|
||||
let validator =
|
||||
// EthTransactionValidator::new(client, MAINNET.clone(), TokioTaskExecutor::default());
|
||||
crate::validate::EthTransactionValidatorBuilder::new(MAINNET.clone()).no_shanghai().no_cancun().build_with_tasks(client, TokioTaskExecutor::default(), InMemoryBlobStore::default());
|
||||
let origin = crate::TransactionOrigin::External;
|
||||
let signer = Default::default();
|
||||
let deposit_tx = Transaction::Deposit(TxDeposit {
|
||||
source_hash: Default::default(),
|
||||
from: signer,
|
||||
to: TransactionKind::Create,
|
||||
mint: None,
|
||||
value: reth_primitives::U256::ZERO,
|
||||
gas_limit: 0u64,
|
||||
is_system_transaction: false,
|
||||
input: Default::default(),
|
||||
});
|
||||
let signature = Signature { r: U256::ZERO, s: U256::ZERO, odd_y_parity: false };
|
||||
let signed_tx = TransactionSigned::from_transaction_and_signature(deposit_tx, signature);
|
||||
let signed_recovered =
|
||||
TransactionSignedEcRecovered::from_signed_transaction(signed_tx, signer);
|
||||
let len = signed_recovered.length_without_header();
|
||||
let pooled_tx = EthPooledTransaction::new(signed_recovered, len);
|
||||
let outcome =
|
||||
crate::TransactionValidator::validate_transaction(&validator, origin, pooled_tx).await;
|
||||
|
||||
let err = match outcome {
|
||||
crate::TransactionValidationOutcome::Invalid(_, err) => err,
|
||||
_ => panic!("Expected invalid transaction"),
|
||||
};
|
||||
assert_eq!(err.to_string(), "transaction type not supported");
|
||||
}
|
||||
}
|
||||
|
||||
@ -115,13 +115,22 @@ impl<T> ValidTransaction<T> {
|
||||
}
|
||||
|
||||
impl<T: PoolTransaction> ValidTransaction<T> {
|
||||
/// Returns the transaction.
|
||||
#[inline]
|
||||
pub(crate) const fn transaction(&self) -> &T {
|
||||
pub const fn transaction(&self) -> &T {
|
||||
match self {
|
||||
Self::Valid(transaction) | Self::ValidWithSidecar { transaction, .. } => transaction,
|
||||
}
|
||||
}
|
||||
|
||||
/// Consumes the wrapper and returns the transaction.
|
||||
pub fn into_transaction(self) -> T {
|
||||
match self {
|
||||
Self::Valid(transaction) => transaction,
|
||||
Self::ValidWithSidecar { transaction, .. } => transaction,
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns the address of that transaction.
|
||||
#[inline]
|
||||
pub(crate) fn sender(&self) -> Address {
|
||||
@ -130,13 +139,13 @@ impl<T: PoolTransaction> ValidTransaction<T> {
|
||||
|
||||
/// Returns the hash of the transaction.
|
||||
#[inline]
|
||||
pub(crate) fn hash(&self) -> &B256 {
|
||||
pub fn hash(&self) -> &B256 {
|
||||
self.transaction().hash()
|
||||
}
|
||||
|
||||
/// Returns the nonce of the transaction.
|
||||
#[inline]
|
||||
pub(crate) fn nonce(&self) -> u64 {
|
||||
pub fn nonce(&self) -> u64 {
|
||||
self.transaction().nonce()
|
||||
}
|
||||
}
|
||||
|
||||
@ -97,6 +97,19 @@ impl TransactionValidationTaskExecutor<()> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<V> TransactionValidationTaskExecutor<V> {
|
||||
/// Maps the given validator to a new type.
|
||||
pub fn map<F, T>(self, mut f: F) -> TransactionValidationTaskExecutor<T>
|
||||
where
|
||||
F: FnMut(V) -> T,
|
||||
{
|
||||
TransactionValidationTaskExecutor {
|
||||
validator: f(self.validator),
|
||||
to_validation_task: self.to_validation_task,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<Client, Tx> TransactionValidationTaskExecutor<EthTransactionValidator<Client, Tx>>
|
||||
where
|
||||
Client: BlockReaderIdExt,
|
||||
|
||||
Reference in New Issue
Block a user