Skip to content

Commit

Permalink
feat: blob e2e test (paradigmxyz#7823)
Browse files Browse the repository at this point in the history
loocapro authored Apr 24, 2024

Verified

This commit was created on GitHub.com and signed with GitHub’s verified signature. The key has expired.
1 parent 9db1712 commit f372db4
Showing 16 changed files with 373 additions and 129 deletions.
2 changes: 2 additions & 0 deletions Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

2 changes: 1 addition & 1 deletion crates/e2e-test-utils/Cargo.toml
Original file line number Diff line number Diff line change
@@ -33,5 +33,5 @@ alloy-signer.workspace = true
alloy-signer-wallet = { workspace = true, features = ["mnemonic"] }
alloy-rpc-types.workspace = true
alloy-network.workspace = true
alloy-consensus.workspace = true
alloy-consensus = { workspace = true, features = ["kzg"] }
tracing.workspace = true
16 changes: 8 additions & 8 deletions crates/e2e-test-utils/src/engine_api.rs
Original file line number Diff line number Diff line change
@@ -13,13 +13,13 @@ use reth_primitives::B256;
use std::marker::PhantomData;

/// Helper for engine api operations
pub struct EngineApiHelper<E> {
pub struct EngineApiTestContext<E> {
pub canonical_stream: CanonStateNotificationStream,
pub engine_api_client: HttpClient,
pub _marker: PhantomData<E>,
}

impl<E: EngineTypes + 'static> EngineApiHelper<E> {
impl<E: EngineTypes + 'static> EngineApiTestContext<E> {
/// Retrieves a v3 payload from the engine api
pub async fn get_payload_v3(
&self,
@@ -34,6 +34,7 @@ impl<E: EngineTypes + 'static> EngineApiHelper<E> {
payload: E::BuiltPayload,
payload_builder_attributes: E::PayloadBuilderAttributes,
expected_status: PayloadStatusEnum,
versioned_hashes: Vec<B256>,
) -> eyre::Result<B256>
where
E::ExecutionPayloadV3: From<E::BuiltPayload> + PayloadEnvelopeExt,
@@ -45,7 +46,7 @@ impl<E: EngineTypes + 'static> EngineApiHelper<E> {
let submission = EngineApiClient::<E>::new_payload_v3(
&self.engine_api_client,
envelope_v3.execution_payload(),
vec![],
versioned_hashes,
payload_builder_attributes.parent_beacon_block_root().unwrap(),
)
.await?;
@@ -56,18 +57,17 @@ impl<E: EngineTypes + 'static> EngineApiHelper<E> {
}

/// Sends forkchoice update to the engine api
pub async fn update_forkchoice(&self, hash: B256) -> eyre::Result<()> {
pub async fn update_forkchoice(&self, current_head: B256, new_head: B256) -> eyre::Result<()> {
EngineApiClient::<E>::fork_choice_updated_v2(
&self.engine_api_client,
ForkchoiceState {
head_block_hash: hash,
safe_block_hash: hash,
finalized_block_hash: hash,
head_block_hash: new_head,
safe_block_hash: current_head,
finalized_block_hash: current_head,
},
None,
)
.await?;

Ok(())
}

12 changes: 9 additions & 3 deletions crates/e2e-test-utils/src/lib.rs
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
use node::NodeHelper;
use node::NodeTestContext;
use reth::{
args::{DiscoveryArgs, NetworkArgs, RpcServerArgs},
builder::{NodeBuilder, NodeConfig, NodeHandle},
@@ -18,6 +18,9 @@ use wallet::Wallet;
/// Wrapper type to create test nodes
pub mod node;

/// Helper for transaction operations
pub mod transaction;

/// Helper type to yield accounts from mnemonic
pub mod wallet;

@@ -29,6 +32,8 @@ mod network;

/// Helper for engine api operations
mod engine_api;
/// Helper for rpc operations
mod rpc;

/// Helper traits
mod traits;
@@ -75,7 +80,7 @@ where
.launch()
.await?;

let mut node = NodeHelper::new(node).await?;
let mut node = NodeTestContext::new(node).await?;

// Connect each node in a chain.
if let Some(previous_node) = nodes.last_mut() {
@@ -104,4 +109,5 @@ type TmpPool<N> = <<N as reth_node_builder::Node<TmpNodeAdapter<N>>>::PoolBuilde
type TmpNodeAdapter<N> = FullNodeTypesAdapter<N, TmpDB, BlockchainProvider<TmpDB>>;

/// Type alias for a type of NodeHelper
pub type NodeHelperType<N> = NodeHelper<FullNodeComponentsAdapter<TmpNodeAdapter<N>, TmpPool<N>>>;
pub type NodeHelperType<N> =
NodeTestContext<FullNodeComponentsAdapter<TmpNodeAdapter<N>, TmpPool<N>>>;
4 changes: 2 additions & 2 deletions crates/e2e-test-utils/src/network.rs
Original file line number Diff line number Diff line change
@@ -5,12 +5,12 @@ use reth_tracing::tracing::info;
use tokio_stream::wrappers::UnboundedReceiverStream;

/// Helper for network operations
pub struct NetworkHelper {
pub struct NetworkTestContext {
network_events: UnboundedReceiverStream<NetworkEvent>,
network: NetworkHandle,
}

impl NetworkHelper {
impl NetworkTestContext {
/// Creates a new network helper
pub fn new(network: NetworkHandle) -> Self {
let network_events = network.event_listener();
106 changes: 58 additions & 48 deletions crates/e2e-test-utils/src/node.rs
Original file line number Diff line number Diff line change
@@ -1,35 +1,36 @@
use crate::{
engine_api::EngineApiHelper, network::NetworkHelper, payload::PayloadHelper,
traits::PayloadEnvelopeExt,
engine_api::EngineApiTestContext, network::NetworkTestContext, payload::PayloadTestContext,
rpc::RpcTestContext, traits::PayloadEnvelopeExt,
};

use alloy_rpc_types::BlockNumberOrTag;
use eyre::Ok;

use futures_util::Future;
use reth::{
api::{BuiltPayload, EngineTypes, FullNodeComponents, PayloadBuilderAttributes},
builder::FullNode,
providers::{BlockReader, BlockReaderIdExt, CanonStateSubscriptions, StageCheckpointReader},
rpc::{
eth::{error::EthResult, EthTransactions},
types::engine::PayloadStatusEnum,
},
rpc::types::engine::PayloadStatusEnum,
};
use reth_node_builder::NodeTypes;
use reth_primitives::{stage::StageId, BlockHash, BlockNumber, Bytes, B256};
use std::{marker::PhantomData, pin::Pin};
use tokio_stream::StreamExt;

/// An helper struct to handle node actions
pub struct NodeHelper<Node>
pub struct NodeTestContext<Node>
where
Node: FullNodeComponents,
{
pub inner: FullNode<Node>,
pub payload: PayloadHelper<Node::Engine>,
pub network: NetworkHelper,
pub engine_api: EngineApiHelper<Node::Engine>,
pub payload: PayloadTestContext<Node::Engine>,
pub network: NetworkTestContext,
pub engine_api: EngineApiTestContext<Node::Engine>,
pub rpc: RpcTestContext<Node>,
}

impl<Node> NodeHelper<Node>
impl<Node> NodeTestContext<Node>
where
Node: FullNodeComponents,
{
@@ -39,17 +40,18 @@ where

Ok(Self {
inner: node.clone(),
network: NetworkHelper::new(node.network.clone()),
payload: PayloadHelper::new(builder).await?,
engine_api: EngineApiHelper {
payload: PayloadTestContext::new(builder).await?,
network: NetworkTestContext::new(node.network.clone()),
engine_api: EngineApiTestContext {
engine_api_client: node.auth_server_handle().http_client(),
canonical_stream: node.provider.canonical_state_stream(),
_marker: PhantomData::<Node::Engine>,
},
rpc: RpcTestContext { inner: node.rpc_registry },
})
}

pub async fn connect(&mut self, node: &mut NodeHelper<Node>) {
pub async fn connect(&mut self, node: &mut NodeTestContext<Node>) {
self.network.add_peer(node.network.record()).await;
node.network.add_peer(self.network.record()).await;
node.network.expect_session().await;
@@ -62,7 +64,7 @@ where
pub async fn advance(
&mut self,
length: u64,
tx_generator: impl Fn() -> Pin<Box<dyn Future<Output = Bytes>>>,
tx_generator: impl Fn(u64) -> Pin<Box<dyn Future<Output = Bytes>>>,
attributes_generator: impl Fn(u64) -> <Node::Engine as EngineTypes>::PayloadBuilderAttributes
+ Copy,
) -> eyre::Result<
@@ -76,60 +78,74 @@ where
From<<Node::Engine as EngineTypes>::BuiltPayload> + PayloadEnvelopeExt,
{
let mut chain = Vec::with_capacity(length as usize);
for _ in 0..length {
let (payload, _) =
self.advance_block(tx_generator().await, attributes_generator).await?;
chain.push(payload);
for i in 0..length {
let raw_tx = tx_generator(i).await;
let tx_hash = self.rpc.inject_tx(raw_tx).await?;
let (payload, eth_attr) = self.advance_block(vec![], attributes_generator).await?;
let block_hash = payload.block().hash();
let block_number = payload.block().number;
self.assert_new_block(tx_hash, block_hash, block_number).await?;
chain.push((payload, eth_attr));
}
Ok(chain)
}

/// Advances the node forward one block
pub async fn advance_block(
/// Creates a new payload from given attributes generator
/// expects a payload attribute event and waits until the payload is built.
///
/// It triggers the resolve payload via engine api and expects the built payload event.
pub async fn new_payload(
&mut self,
raw_tx: Bytes,
attributes_generator: impl Fn(u64) -> <Node::Engine as EngineTypes>::PayloadBuilderAttributes,
) -> eyre::Result<(
(
<Node::Engine as EngineTypes>::BuiltPayload,
<Node::Engine as EngineTypes>::PayloadBuilderAttributes,
),
B256,
<<Node as NodeTypes>::Engine as EngineTypes>::BuiltPayload,
<<Node as NodeTypes>::Engine as EngineTypes>::PayloadBuilderAttributes,
)>
where
<Node::Engine as EngineTypes>::ExecutionPayloadV3:
From<<Node::Engine as EngineTypes>::BuiltPayload> + PayloadEnvelopeExt,
{
// push tx into pool via RPC server
let tx_hash = self.inject_tx(raw_tx).await?;

// trigger new payload building draining the pool
let eth_attr = self.payload.new_payload(attributes_generator).await.unwrap();

// first event is the payload attributes
self.payload.expect_attr_event(eth_attr.clone()).await?;

// wait for the payload builder to have finished building
self.payload.wait_for_built_payload(eth_attr.payload_id()).await;

// trigger resolve payload via engine api
self.engine_api.get_payload_v3(eth_attr.payload_id()).await?;

// ensure we're also receiving the built payload as event
let payload = self.payload.expect_built_payload().await?;
Ok((self.payload.expect_built_payload().await?, eth_attr))
}

/// Advances the node forward one block
pub async fn advance_block(
&mut self,
versioned_hashes: Vec<B256>,
attributes_generator: impl Fn(u64) -> <Node::Engine as EngineTypes>::PayloadBuilderAttributes,
) -> eyre::Result<(
<Node::Engine as EngineTypes>::BuiltPayload,
<<Node as NodeTypes>::Engine as EngineTypes>::PayloadBuilderAttributes,
)>
where
<Node::Engine as EngineTypes>::ExecutionPayloadV3:
From<<Node::Engine as EngineTypes>::BuiltPayload> + PayloadEnvelopeExt,
{
let (payload, eth_attr) = self.new_payload(attributes_generator).await?;

// submit payload via engine api
let block_hash = self
.engine_api
.submit_payload(payload.clone(), eth_attr.clone(), PayloadStatusEnum::Valid)
.submit_payload(
payload.clone(),
eth_attr.clone(),
PayloadStatusEnum::Valid,
versioned_hashes,
)
.await?;

// trigger forkchoice update via engine api to commit the block to the blockchain
self.engine_api.update_forkchoice(block_hash).await?;
self.engine_api.update_forkchoice(block_hash, block_hash).await?;

// assert the block has been committed to the blockchain
self.assert_new_block(tx_hash, block_hash, payload.block().number).await?;
Ok(((payload, eth_attr), tx_hash))
Ok((payload, eth_attr))
}

/// Waits for block to be available on node.
@@ -169,12 +185,6 @@ where
Ok(())
}

/// Injects a raw transaction into the node tx pool via RPC server
async fn inject_tx(&mut self, raw_tx: Bytes) -> EthResult<B256> {
let eth_api = self.inner.rpc_registry.eth_api();
eth_api.send_raw_transaction(raw_tx).await
}

/// Asserts that a new block has been added to the blockchain
/// and the tx has been included in the block
pub async fn assert_new_block(
4 changes: 2 additions & 2 deletions crates/e2e-test-utils/src/payload.rs
Original file line number Diff line number Diff line change
@@ -4,13 +4,13 @@ use reth_payload_builder::{Events, PayloadBuilderHandle, PayloadId};
use tokio_stream::wrappers::BroadcastStream;

/// Helper for payload operations
pub struct PayloadHelper<E: EngineTypes + 'static> {
pub struct PayloadTestContext<E: EngineTypes + 'static> {
pub payload_event_stream: BroadcastStream<Events<E>>,
payload_builder: PayloadBuilderHandle<E>,
pub timestamp: u64,
}

impl<E: EngineTypes + 'static> PayloadHelper<E> {
impl<E: EngineTypes + 'static> PayloadTestContext<E> {
/// Creates a new payload helper
pub async fn new(payload_builder: PayloadBuilderHandle<E>) -> eyre::Result<Self> {
let payload_events = payload_builder.subscribe().await?;
24 changes: 24 additions & 0 deletions crates/e2e-test-utils/src/rpc.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,24 @@
use alloy_consensus::TxEnvelope;
use alloy_network::eip2718::Decodable2718;
use reth::{api::FullNodeComponents, builder::rpc::RpcRegistry, rpc::api::DebugApiServer};
use reth_primitives::{Bytes, B256};
use reth_rpc::eth::{error::EthResult, EthTransactions};

pub struct RpcTestContext<Node: FullNodeComponents> {
pub inner: RpcRegistry<Node>,
}

impl<Node: FullNodeComponents> RpcTestContext<Node> {
/// Injects a raw transaction into the node tx pool via RPC server
pub async fn inject_tx(&mut self, raw_tx: Bytes) -> EthResult<B256> {
let eth_api = self.inner.eth_api();
eth_api.send_raw_transaction(raw_tx).await
}

/// Retrieves a transaction envelope by its hash
pub async fn envelope_by_hash(&mut self, hash: B256) -> eyre::Result<TxEnvelope> {
let tx = self.inner.debug_api().raw_transaction(hash).await?.unwrap();
let tx = tx.to_vec();
Ok(TxEnvelope::decode_2718(&mut tx.as_ref()).unwrap())
}
}
80 changes: 80 additions & 0 deletions crates/e2e-test-utils/src/transaction.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,80 @@
use alloy_consensus::{
BlobTransactionSidecar, SidecarBuilder, SimpleCoder, TxEip4844Variant, TxEnvelope,
};
use alloy_network::{eip2718::Encodable2718, EthereumSigner, TransactionBuilder};
use alloy_rpc_types::{TransactionInput, TransactionRequest};
use alloy_signer_wallet::LocalWallet;
use eyre::Ok;
use reth_primitives::{hex, Address, Bytes, U256};

use reth_primitives::{constants::eip4844::MAINNET_KZG_TRUSTED_SETUP, B256};

pub struct TransactionTestContext;

impl TransactionTestContext {
/// Creates a static transfer and signs it
pub async fn transfer_tx(chain_id: u64, wallet: LocalWallet) -> Bytes {
let tx = tx(chain_id, None, 0);
let signer = EthereumSigner::from(wallet);
tx.build(&signer).await.unwrap().encoded_2718().into()
}

/// Creates a tx with blob sidecar and sign it
pub async fn tx_with_blobs(chain_id: u64, wallet: LocalWallet) -> eyre::Result<Bytes> {
let mut tx = tx(chain_id, None, 0);

let mut builder = SidecarBuilder::<SimpleCoder>::new();
builder.ingest(b"dummy blob");
let sidecar: BlobTransactionSidecar = builder.build()?;

tx.set_blob_sidecar(sidecar);
tx.set_max_fee_per_blob_gas(15e9 as u128);

let signer = EthereumSigner::from(wallet);
let signed = tx.clone().build(&signer).await.unwrap();

Ok(signed.encoded_2718().into())
}

pub async fn optimism_l1_block_info_tx(
chain_id: u64,
wallet: LocalWallet,
nonce: u64,
) -> Bytes {
let l1_block_info = Bytes::from_static(&hex!("7ef9015aa044bae9d41b8380d781187b426c6fe43df5fb2fb57bd4466ef6a701e1f01e015694deaddeaddeaddeaddeaddeaddeaddeaddead000194420000000000000000000000000000000000001580808408f0d18001b90104015d8eb900000000000000000000000000000000000000000000000000000000008057650000000000000000000000000000000000000000000000000000000063d96d10000000000000000000000000000000000000000000000000000000000009f35273d89754a1e0387b89520d989d3be9c37c1f32495a88faf1ea05c61121ab0d1900000000000000000000000000000000000000000000000000000000000000010000000000000000000000002d679b567db6187c0c8323fa982cfb88b74dbcc7000000000000000000000000000000000000000000000000000000000000083400000000000000000000000000000000000000000000000000000000000f4240"));
let tx = tx(chain_id, Some(l1_block_info), nonce);
let signer = EthereumSigner::from(wallet);
tx.build(&signer).await.unwrap().encoded_2718().into()
}

/// Validates the sidecar of a given tx envelope and returns the versioned hashes
pub fn validate_sidecar(tx: TxEnvelope) -> Vec<B256> {
let proof_setting = MAINNET_KZG_TRUSTED_SETUP.clone();

match tx {
TxEnvelope::Eip4844(signed) => match signed.tx() {
TxEip4844Variant::TxEip4844WithSidecar(tx) => {
tx.validate_blob(&proof_setting).unwrap();
tx.sidecar.versioned_hashes().collect()
}
_ => panic!("Expected Eip4844 transaction with sidecar"),
},
_ => panic!("Expected Eip4844 transaction"),
}
}
}

/// Creates a type 2 transaction
fn tx(chain_id: u64, data: Option<Bytes>, nonce: u64) -> TransactionRequest {
TransactionRequest {
nonce: Some(nonce),
value: Some(U256::from(100)),
to: Some(Address::random()),
gas: Some(210000),
max_fee_per_gas: Some(20e9 as u128),
max_priority_fee_per_gas: Some(20e9 as u128),
chain_id: Some(chain_id),
input: TransactionInput { input: None, data },
..Default::default()
}
}
55 changes: 24 additions & 31 deletions crates/e2e-test-utils/src/wallet.rs
Original file line number Diff line number Diff line change
@@ -1,19 +1,19 @@
use alloy_network::{eip2718::Encodable2718, EthereumSigner, TransactionBuilder};
use alloy_rpc_types::{TransactionInput, TransactionRequest};
use alloy_signer::Signer;
use alloy_signer_wallet::{coins_bip39::English, LocalWallet, MnemonicBuilder};
use reth_primitives::{hex, Address, Bytes, U256};

/// One of the accounts of the genesis allocations.
pub struct Wallet {
inner: LocalWallet,
pub nonce: u64,
pub inner: LocalWallet,
chain_id: u64,
amount: usize,
derivation_path: Option<String>,
}

impl Wallet {
/// Creates a new account from one of the secret/pubkeys of the genesis allocations (test.json)
pub(crate) fn new(phrase: &str) -> Self {
let inner = MnemonicBuilder::<English>::default().phrase(phrase).build().unwrap();
Self { inner, chain_id: 1, nonce: 0 }
pub fn new(amount: usize) -> Self {
let inner = MnemonicBuilder::<English>::default().phrase(TEST_MNEMONIC).build().unwrap();
Self { inner, chain_id: 1, amount, derivation_path: None }
}

/// Sets chain id
@@ -22,38 +22,31 @@ impl Wallet {
self
}

/// Creates a static transfer and signs it
pub async fn transfer_tx(&mut self) -> Bytes {
self.tx(None).await
fn get_derivation_path(&self) -> &str {
self.derivation_path.as_deref().unwrap_or("m/44'/60'/0'/0/")
}

pub async fn optimism_l1_block_info_tx(&mut self) -> Bytes {
let l1_block_info = Bytes::from_static(&hex!("7ef9015aa044bae9d41b8380d781187b426c6fe43df5fb2fb57bd4466ef6a701e1f01e015694deaddeaddeaddeaddeaddeaddeaddeaddead000194420000000000000000000000000000000000001580808408f0d18001b90104015d8eb900000000000000000000000000000000000000000000000000000000008057650000000000000000000000000000000000000000000000000000000063d96d10000000000000000000000000000000000000000000000000000000000009f35273d89754a1e0387b89520d989d3be9c37c1f32495a88faf1ea05c61121ab0d1900000000000000000000000000000000000000000000000000000000000000010000000000000000000000002d679b567db6187c0c8323fa982cfb88b74dbcc7000000000000000000000000000000000000000000000000000000000000083400000000000000000000000000000000000000000000000000000000000f4240"));
self.tx(Some(l1_block_info)).await
}
pub fn gen(&self) -> Vec<LocalWallet> {
let builder = MnemonicBuilder::<English>::default().phrase(TEST_MNEMONIC);

// use the derivation path
let derivation_path = self.get_derivation_path();

/// Creates a transaction with data and signs it
pub async fn tx(&mut self, data: Option<Bytes>) -> Bytes {
let tx = TransactionRequest {
nonce: Some(self.nonce),
value: Some(U256::from(100)),
to: Some(Address::random()),
gas_price: Some(20e9 as u128),
gas: Some(210000),
chain_id: Some(self.chain_id),
input: TransactionInput { input: None, data },
..Default::default()
};
self.nonce += 1;
let signer = EthereumSigner::from(self.inner.clone());
tx.build(&signer).await.unwrap().encoded_2718().into()
let mut wallets = Vec::with_capacity(self.amount);
for idx in 0..self.amount {
let builder =
builder.clone().derivation_path(&format!("{derivation_path}{idx}")).unwrap();
let wallet = builder.build().unwrap().with_chain_id(Some(self.chain_id));
wallets.push(wallet)
}
wallets
}
}

const TEST_MNEMONIC: &str = "test test test test test test test test test test test junk";

impl Default for Wallet {
fn default() -> Self {
Wallet::new(TEST_MNEMONIC)
Wallet::new(1)
}
}
96 changes: 96 additions & 0 deletions crates/node-ethereum/tests/e2e/blobs.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,96 @@
use std::sync::Arc;

use reth::{
args::RpcServerArgs,
builder::{NodeBuilder, NodeConfig, NodeHandle},
rpc::types::engine::PayloadStatusEnum,
tasks::TaskManager,
};
use reth_e2e_test_utils::{
node::NodeTestContext, transaction::TransactionTestContext, wallet::Wallet,
};
use reth_node_ethereum::EthereumNode;
use reth_primitives::{b256, ChainSpecBuilder, Genesis, MAINNET};
use reth_transaction_pool::TransactionPool;

use crate::utils::eth_payload_attributes;

#[tokio::test]
async fn can_handle_blobs() -> eyre::Result<()> {
reth_tracing::init_test_tracing();
let tasks = TaskManager::current();
let exec = tasks.executor();

let genesis: Genesis = serde_json::from_str(include_str!("../assets/genesis.json")).unwrap();
let chain_spec = Arc::new(
ChainSpecBuilder::default()
.chain(MAINNET.chain)
.genesis(genesis)
.cancun_activated()
.build(),
);
let node_config = NodeConfig::test()
.with_chain(chain_spec)
.with_unused_ports()
.with_rpc(RpcServerArgs::default().with_unused_ports().with_http());
let NodeHandle { node, node_exit_future: _ } = NodeBuilder::new(node_config.clone())
.testing_node(exec.clone())
.node(EthereumNode::default())
.launch()
.await?;

let mut node = NodeTestContext::new(node).await?;

let wallets = Wallet::new(2).gen();
let blob_wallet = wallets.first().unwrap();
let second_wallet = wallets.last().unwrap();

// inject normal tx
let raw_tx = TransactionTestContext::transfer_tx(1, second_wallet.clone()).await;
let tx_hash = node.rpc.inject_tx(raw_tx).await?;
// build payload with normal tx
let (payload, attributes) = node.new_payload(eth_payload_attributes).await?;

// clean the pool
node.inner.pool.remove_transactions(vec![tx_hash]);

// build blob tx
let blob_tx = TransactionTestContext::tx_with_blobs(1, blob_wallet.clone()).await?;

// inject blob tx to the pool
let blob_tx_hash = node.rpc.inject_tx(blob_tx).await?;
// fetch it from rpc
let envelope = node.rpc.envelope_by_hash(blob_tx_hash).await?;
// validate sidecar
let versioned_hashes = TransactionTestContext::validate_sidecar(envelope);

// build a payload
let (blob_payload, blob_attr) = node.new_payload(eth_payload_attributes).await?;

// submit the blob payload
let blob_block_hash = node
.engine_api
.submit_payload(blob_payload, blob_attr, PayloadStatusEnum::Valid, versioned_hashes.clone())
.await?;

let genesis_hash = b256!("d4e56740f876aef8c010b86a40d5f56745a118d0906a34e69aec8c0db1cb8fa3");

let (_, _) = tokio::join!(
// send fcu with blob hash
node.engine_api.update_forkchoice(genesis_hash, blob_block_hash),
// send fcu with normal hash
node.engine_api.update_forkchoice(genesis_hash, payload.block().hash())
);

// submit normal payload
node.engine_api.submit_payload(payload, attributes, PayloadStatusEnum::Valid, vec![]).await?;

tokio::time::sleep(std::time::Duration::from_secs(3)).await;

// expects the blob tx to be back in the pool
let envelope = node.rpc.envelope_by_hash(blob_tx_hash).await?;
// make sure the sidecar is present
TransactionTestContext::validate_sidecar(envelope);

Ok(())
}
39 changes: 30 additions & 9 deletions crates/node-ethereum/tests/e2e/eth.rs
Original file line number Diff line number Diff line change
@@ -4,7 +4,9 @@ use reth::{
builder::{NodeBuilder, NodeConfig, NodeHandle},
tasks::TaskManager,
};
use reth_e2e_test_utils::{node::NodeHelper, setup, wallet::Wallet};
use reth_e2e_test_utils::{
node::NodeTestContext, setup, transaction::TransactionTestContext, wallet::Wallet,
};
use reth_node_ethereum::EthereumNode;
use reth_primitives::{ChainSpecBuilder, Genesis, MAINNET};
use std::sync::Arc;
@@ -13,7 +15,7 @@ use std::sync::Arc;
async fn can_run_eth_node() -> eyre::Result<()> {
reth_tracing::init_test_tracing();

let (mut nodes, _tasks, mut wallet) = setup::<EthereumNode>(
let (mut nodes, _tasks, _wallet) = setup::<EthereumNode>(
1,
Arc::new(
ChainSpecBuilder::default()
@@ -27,10 +29,20 @@ async fn can_run_eth_node() -> eyre::Result<()> {
.await?;

let mut node = nodes.pop().unwrap();
let raw_tx = wallet.transfer_tx().await;
let wallet = Wallet::default();
let raw_tx = TransactionTestContext::transfer_tx(1, wallet.inner).await;

// make the node advance
let tx_hash = node.rpc.inject_tx(raw_tx).await?;

// make the node advance
node.advance_block(raw_tx, eth_payload_attributes).await?;
let (payload, _) = node.advance_block(vec![], eth_payload_attributes).await?;

let block_hash = payload.block().hash();
let block_number = payload.block().number;

// assert the block has been committed to the blockchain
node.assert_new_block(tx_hash, block_hash, block_number).await?;

Ok(())
}
@@ -62,14 +74,23 @@ async fn can_run_eth_node_with_auth_engine_api_over_ipc() -> eyre::Result<()> {
.node(EthereumNode::default())
.launch()
.await?;
let mut node = NodeHelper::new(node).await?;
let mut node = NodeTestContext::new(node).await?;

// Configure wallet from test mnemonic and create dummy transfer tx
let mut wallet = Wallet::default();
let raw_tx = wallet.transfer_tx().await;
let wallet = Wallet::default();
let raw_tx = TransactionTestContext::transfer_tx(1, wallet.inner).await;

// make the node advance
node.advance_block(raw_tx, crate::utils::eth_payload_attributes).await?;
let tx_hash = node.rpc.inject_tx(raw_tx).await?;

// make the node advance
let (payload, _) = node.advance_block(vec![], eth_payload_attributes).await?;

let block_hash = payload.block().hash();
let block_number = payload.block().number;

// assert the block has been committed to the blockchain
node.assert_new_block(tx_hash, block_hash, block_number).await?;

Ok(())
}
@@ -99,7 +120,7 @@ async fn test_failed_run_eth_node_with_no_auth_engine_api_over_ipc_opts() -> eyr
.launch()
.await?;

let node = NodeHelper::new(node).await?;
let node = NodeTestContext::new(node).await?;

// Ensure that the engine api client is not available
let client = node.inner.engine_ipc_client().await;
1 change: 1 addition & 0 deletions crates/node-ethereum/tests/e2e/main.rs
Original file line number Diff line number Diff line change
@@ -1,3 +1,4 @@
mod blobs;
mod dev;
mod eth;
mod p2p;
19 changes: 13 additions & 6 deletions crates/node-ethereum/tests/e2e/p2p.rs
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
use crate::utils::eth_payload_attributes;
use reth_e2e_test_utils::setup;
use reth_e2e_test_utils::{setup, transaction::TransactionTestContext};
use reth_node_ethereum::EthereumNode;
use reth_primitives::{ChainSpecBuilder, MAINNET};
use std::sync::Arc;
@@ -8,7 +8,7 @@ use std::sync::Arc;
async fn can_sync() -> eyre::Result<()> {
reth_tracing::init_test_tracing();

let (mut nodes, _tasks, mut wallet) = setup::<EthereumNode>(
let (mut nodes, _tasks, wallet) = setup::<EthereumNode>(
2,
Arc::new(
ChainSpecBuilder::default()
@@ -21,17 +21,24 @@ async fn can_sync() -> eyre::Result<()> {
)
.await?;

let raw_tx = wallet.transfer_tx().await;
let raw_tx = TransactionTestContext::transfer_tx(1, wallet.inner).await;
let mut second_node = nodes.pop().unwrap();
let mut first_node = nodes.pop().unwrap();

// Make the first node advance
let ((payload, _), tx_hash) =
first_node.advance_block(raw_tx.clone(), eth_payload_attributes).await?;
let tx_hash = first_node.rpc.inject_tx(raw_tx).await?;

// make the node advance
let (payload, _) = first_node.advance_block(vec![], eth_payload_attributes).await?;

let block_hash = payload.block().hash();
let block_number = payload.block().number;

// assert the block has been committed to the blockchain
first_node.assert_new_block(tx_hash, block_hash, block_number).await?;

// only send forkchoice update to second node
second_node.engine_api.update_forkchoice(block_hash).await?;
second_node.engine_api.update_forkchoice(block_hash, block_hash).await?;

// expect second node advanced via p2p gossip
second_node.assert_new_block(tx_hash, block_hash, 1).await?;
25 changes: 19 additions & 6 deletions crates/optimism/node/tests/e2e/p2p.rs
Original file line number Diff line number Diff line change
@@ -1,27 +1,40 @@
use crate::utils::{advance_chain, setup};
use std::sync::Arc;
use tokio::sync::Mutex;
use reth::primitives::BASE_MAINNET;
use reth_e2e_test_utils::{transaction::TransactionTestContext, wallet::Wallet};
use reth_primitives::ChainId;

#[tokio::test]
async fn can_sync() -> eyre::Result<()> {
reth_tracing::init_test_tracing();

let (mut nodes, _tasks, wallet) = setup(2).await?;
let wallet = Arc::new(Mutex::new(wallet));
let chain_id: ChainId = BASE_MAINNET.chain.into();

let (mut nodes, _tasks, _wallet) = setup(2).await?;

let second_node = nodes.pop().unwrap();
let mut first_node = nodes.pop().unwrap();

let tip: usize = 300;
let tip_index: usize = tip - 1;

let wallet = Wallet::default();

// On first node, create a chain up to block number 300a
let canonical_payload_chain = advance_chain(tip, &mut first_node, wallet.clone()).await?;
let canonical_payload_chain = advance_chain(tip, &mut first_node, |nonce: u64| {
let wallet = wallet.inner.clone();
Box::pin(async move {
TransactionTestContext::optimism_l1_block_info_tx(chain_id, wallet, nonce).await
})
})
.await?;
let canonical_chain =
canonical_payload_chain.iter().map(|p| p.0.block().hash()).collect::<Vec<_>>();

// On second node, sync up to block number 300a
second_node.engine_api.update_forkchoice(canonical_chain[tip_index]).await?;
second_node
.engine_api
.update_forkchoice(canonical_chain[tip_index], canonical_chain[tip_index])
.await?;
second_node.wait_block(tip as u64, canonical_chain[tip_index], true).await?;

Ok(())
17 changes: 4 additions & 13 deletions crates/optimism/node/tests/e2e/utils.rs
Original file line number Diff line number Diff line change
@@ -1,10 +1,9 @@
use reth::{rpc::types::engine::PayloadAttributes, tasks::TaskManager};
use reth::{primitives::Bytes, rpc::types::engine::PayloadAttributes, tasks::TaskManager};
use reth_e2e_test_utils::{wallet::Wallet, NodeHelperType};
use reth_node_optimism::{OptimismBuiltPayload, OptimismNode, OptimismPayloadBuilderAttributes};
use reth_payload_builder::EthPayloadBuilderAttributes;
use reth_primitives::{Address, ChainSpecBuilder, Genesis, B256, BASE_MAINNET};
use std::sync::Arc;
use tokio::sync::Mutex;
use std::{future::Future, pin::Pin, sync::Arc};

/// Optimism Node Helper type
pub(crate) type OpNode = NodeHelperType<OptimismNode>;
@@ -28,17 +27,9 @@ pub(crate) async fn setup(num_nodes: usize) -> eyre::Result<(Vec<OpNode>, TaskMa
pub(crate) async fn advance_chain(
length: usize,
node: &mut OpNode,
wallet: Arc<Mutex<Wallet>>,
tx_generator: impl Fn(u64) -> Pin<Box<dyn Future<Output = Bytes>>>,
) -> eyre::Result<Vec<(OptimismBuiltPayload, OptimismPayloadBuilderAttributes)>> {
node.advance(
length as u64,
|| {
let wallet = wallet.clone();
Box::pin(async move { wallet.lock().await.optimism_l1_block_info_tx().await })
},
optimism_payload_attributes,
)
.await
node.advance(length as u64, tx_generator, optimism_payload_attributes).await
}

/// Helper function to create a new eth payload attributes

0 comments on commit f372db4

Please sign in to comment.