mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 19:09:54 +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 = [
|
dependencies = [
|
||||||
"clap",
|
"clap",
|
||||||
"eyre",
|
"eyre",
|
||||||
|
"parking_lot 0.12.1",
|
||||||
"reth-basic-payload-builder",
|
"reth-basic-payload-builder",
|
||||||
"reth-db",
|
"reth-db",
|
||||||
"reth-network",
|
"reth-network",
|
||||||
@ -6502,6 +6503,7 @@ dependencies = [
|
|||||||
"reth-payload-builder",
|
"reth-payload-builder",
|
||||||
"reth-primitives",
|
"reth-primitives",
|
||||||
"reth-provider",
|
"reth-provider",
|
||||||
|
"reth-revm",
|
||||||
"reth-rpc",
|
"reth-rpc",
|
||||||
"reth-rpc-types",
|
"reth-rpc-types",
|
||||||
"reth-rpc-types-compat",
|
"reth-rpc-types-compat",
|
||||||
|
|||||||
@ -25,12 +25,14 @@ reth-tracing.workspace = true
|
|||||||
reth-provider.workspace = true
|
reth-provider.workspace = true
|
||||||
reth-transaction-pool.workspace = true
|
reth-transaction-pool.workspace = true
|
||||||
reth-network.workspace = true
|
reth-network.workspace = true
|
||||||
|
reth-revm.workspace = true
|
||||||
revm.workspace = true
|
revm.workspace = true
|
||||||
|
|
||||||
# misc
|
# misc
|
||||||
clap.workspace = true
|
clap.workspace = true
|
||||||
serde.workspace = true
|
serde.workspace = true
|
||||||
eyre.workspace = true
|
eyre.workspace = true
|
||||||
|
parking_lot.workspace = true
|
||||||
|
|
||||||
[dev-dependencies]
|
[dev-dependencies]
|
||||||
reth-db.workspace = true
|
reth-db.workspace = true
|
||||||
@ -45,4 +47,5 @@ optimism = [
|
|||||||
"reth-rpc-types-compat/optimism",
|
"reth-rpc-types-compat/optimism",
|
||||||
"reth-rpc/optimism",
|
"reth-rpc/optimism",
|
||||||
"reth-optimism-payload-builder/optimism",
|
"reth-optimism-payload-builder/optimism",
|
||||||
|
"reth-revm/optimism",
|
||||||
]
|
]
|
||||||
|
|||||||
@ -9,6 +9,9 @@
|
|||||||
// The `optimism` feature must be enabled to use this crate.
|
// The `optimism` feature must be enabled to use this crate.
|
||||||
#![cfg(feature = "optimism")]
|
#![cfg(feature = "optimism")]
|
||||||
|
|
||||||
|
/// CLI argument parsing for the optimism node.
|
||||||
|
pub mod args;
|
||||||
|
|
||||||
/// Exports optimism-specific implementations of the [EngineTypes](reth_node_api::EngineTypes)
|
/// Exports optimism-specific implementations of the [EngineTypes](reth_node_api::EngineTypes)
|
||||||
/// trait.
|
/// trait.
|
||||||
pub mod engine;
|
pub mod engine;
|
||||||
@ -19,6 +22,7 @@ pub use engine::OptimismEngineTypes;
|
|||||||
pub mod evm;
|
pub mod evm;
|
||||||
pub use evm::OptimismEvmConfig;
|
pub use evm::OptimismEvmConfig;
|
||||||
|
|
||||||
pub mod args;
|
|
||||||
pub mod node;
|
pub mod node;
|
||||||
pub use node::OptimismNode;
|
pub use node::OptimismNode;
|
||||||
|
|
||||||
|
pub mod txpool;
|
||||||
|
|||||||
@ -1,6 +1,10 @@
|
|||||||
//! Optimism Node types config.
|
//! 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_basic_payload_builder::{BasicPayloadJobGenerator, BasicPayloadJobGeneratorConfig};
|
||||||
use reth_network::{NetworkHandle, NetworkManager};
|
use reth_network::{NetworkHandle, NetworkManager};
|
||||||
use reth_node_builder::{
|
use reth_node_builder::{
|
||||||
@ -12,7 +16,7 @@ use reth_payload_builder::{PayloadBuilderHandle, PayloadBuilderService};
|
|||||||
use reth_provider::CanonStateSubscriptions;
|
use reth_provider::CanonStateSubscriptions;
|
||||||
use reth_tracing::tracing::{debug, info};
|
use reth_tracing::tracing::{debug, info};
|
||||||
use reth_transaction_pool::{
|
use reth_transaction_pool::{
|
||||||
blobstore::DiskFileBlobStore, EthTransactionPool, TransactionPool,
|
blobstore::DiskFileBlobStore, CoinbaseTipOrdering, TransactionPool,
|
||||||
TransactionValidationTaskExecutor,
|
TransactionValidationTaskExecutor,
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -84,11 +88,12 @@ impl<Node> PoolBuilder<Node> for OptimismPoolBuilder
|
|||||||
where
|
where
|
||||||
Node: FullNodeTypes,
|
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> {
|
async fn build_pool(self, ctx: &BuilderContext<Node>) -> eyre::Result<Self::Pool> {
|
||||||
let data_dir = ctx.data_dir();
|
let data_dir = ctx.data_dir();
|
||||||
let blob_store = DiskFileBlobStore::open(data_dir.blobstore_path(), Default::default())?;
|
let blob_store = DiskFileBlobStore::open(data_dir.blobstore_path(), Default::default())?;
|
||||||
|
|
||||||
let validator = TransactionValidationTaskExecutor::eth_builder(ctx.chain_spec())
|
let validator = TransactionValidationTaskExecutor::eth_builder(ctx.chain_spec())
|
||||||
.with_head_timestamp(ctx.head().timestamp)
|
.with_head_timestamp(ctx.head().timestamp)
|
||||||
.kzg_settings(ctx.kzg_settings()?)
|
.kzg_settings(ctx.kzg_settings()?)
|
||||||
@ -97,10 +102,15 @@ where
|
|||||||
ctx.provider().clone(),
|
ctx.provider().clone(),
|
||||||
ctx.task_executor().clone(),
|
ctx.task_executor().clone(),
|
||||||
blob_store.clone(),
|
blob_store.clone(),
|
||||||
);
|
)
|
||||||
|
.map(OpTransactionValidator::new);
|
||||||
|
|
||||||
let transaction_pool =
|
let transaction_pool = reth_transaction_pool::Pool::new(
|
||||||
reth_transaction_pool::Pool::eth_pool(validator, blob_store, ctx.pool_config());
|
validator,
|
||||||
|
CoinbaseTipOrdering::default(),
|
||||||
|
blob_store,
|
||||||
|
ctx.pool_config(),
|
||||||
|
);
|
||||||
info!(target: "reth::cli", "Transaction pool initialized");
|
info!(target: "reth::cli", "Transaction pool initialized");
|
||||||
let transactions_path = data_dir.txpool_transactions_path();
|
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.
|
//! Ethereum transaction validator.
|
||||||
|
|
||||||
|
use super::constants::DEFAULT_MAX_TX_INPUT_BYTES;
|
||||||
use crate::{
|
use crate::{
|
||||||
blobstore::BlobStore,
|
blobstore::BlobStore,
|
||||||
error::{Eip4844PoolTransactionError, InvalidPoolTransactionError},
|
error::{Eip4844PoolTransactionError, InvalidPoolTransactionError},
|
||||||
@ -26,11 +27,6 @@ use std::{
|
|||||||
};
|
};
|
||||||
use tokio::sync::Mutex;
|
use tokio::sync::Mutex;
|
||||||
|
|
||||||
#[cfg(feature = "optimism")]
|
|
||||||
use reth_revm::optimism::RethL1BlockInfo;
|
|
||||||
|
|
||||||
use super::constants::DEFAULT_MAX_TX_INPUT_BYTES;
|
|
||||||
|
|
||||||
/// Validator for Ethereum transactions.
|
/// Validator for Ethereum transactions.
|
||||||
#[derive(Debug, Clone)]
|
#[derive(Debug, Clone)]
|
||||||
pub struct EthTransactionValidator<Client, T> {
|
pub struct EthTransactionValidator<Client, T> {
|
||||||
@ -38,6 +34,18 @@ pub struct EthTransactionValidator<Client, T> {
|
|||||||
inner: Arc<EthTransactionValidatorInner<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>
|
impl<Client, Tx> EthTransactionValidator<Client, Tx>
|
||||||
where
|
where
|
||||||
Client: StateProviderFactory + BlockReaderIdExt,
|
Client: StateProviderFactory + BlockReaderIdExt,
|
||||||
@ -145,14 +153,6 @@ where
|
|||||||
origin: TransactionOrigin,
|
origin: TransactionOrigin,
|
||||||
mut transaction: Tx,
|
mut transaction: Tx,
|
||||||
) -> TransactionValidationOutcome<Tx> {
|
) -> TransactionValidationOutcome<Tx> {
|
||||||
#[cfg(feature = "optimism")]
|
|
||||||
if transaction.is_deposit() || transaction.is_eip4844() {
|
|
||||||
return TransactionValidationOutcome::Invalid(
|
|
||||||
transaction,
|
|
||||||
InvalidTransactionError::TxTypeNotSupported.into(),
|
|
||||||
)
|
|
||||||
}
|
|
||||||
|
|
||||||
// Checks for tx_type
|
// Checks for tx_type
|
||||||
match transaction.tx_type() {
|
match transaction.tx_type() {
|
||||||
LEGACY_TX_TYPE_ID => {
|
LEGACY_TX_TYPE_ID => {
|
||||||
@ -318,52 +318,8 @@ where
|
|||||||
)
|
)
|
||||||
}
|
}
|
||||||
|
|
||||||
#[cfg(not(feature = "optimism"))]
|
|
||||||
let cost = transaction.cost();
|
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
|
// Checks for max cost
|
||||||
if cost > account.balance {
|
if cost > account.balance {
|
||||||
return TransactionValidationOutcome::Invalid(
|
return TransactionValidationOutcome::Invalid(
|
||||||
@ -800,47 +756,4 @@ mod tests {
|
|||||||
let tx = pool.get(transaction.hash());
|
let tx = pool.get(transaction.hash());
|
||||||
assert!(tx.is_some());
|
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> {
|
impl<T: PoolTransaction> ValidTransaction<T> {
|
||||||
|
/// Returns the transaction.
|
||||||
#[inline]
|
#[inline]
|
||||||
pub(crate) const fn transaction(&self) -> &T {
|
pub const fn transaction(&self) -> &T {
|
||||||
match self {
|
match self {
|
||||||
Self::Valid(transaction) | Self::ValidWithSidecar { transaction, .. } => transaction,
|
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.
|
/// Returns the address of that transaction.
|
||||||
#[inline]
|
#[inline]
|
||||||
pub(crate) fn sender(&self) -> Address {
|
pub(crate) fn sender(&self) -> Address {
|
||||||
@ -130,13 +139,13 @@ impl<T: PoolTransaction> ValidTransaction<T> {
|
|||||||
|
|
||||||
/// Returns the hash of the transaction.
|
/// Returns the hash of the transaction.
|
||||||
#[inline]
|
#[inline]
|
||||||
pub(crate) fn hash(&self) -> &B256 {
|
pub fn hash(&self) -> &B256 {
|
||||||
self.transaction().hash()
|
self.transaction().hash()
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Returns the nonce of the transaction.
|
/// Returns the nonce of the transaction.
|
||||||
#[inline]
|
#[inline]
|
||||||
pub(crate) fn nonce(&self) -> u64 {
|
pub fn nonce(&self) -> u64 {
|
||||||
self.transaction().nonce()
|
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>>
|
impl<Client, Tx> TransactionValidationTaskExecutor<EthTransactionValidator<Client, Tx>>
|
||||||
where
|
where
|
||||||
Client: BlockReaderIdExt,
|
Client: BlockReaderIdExt,
|
||||||
|
|||||||
Reference in New Issue
Block a user