chore: move node-ethereum to ethereum/node (#8076)

This commit is contained in:
Matthias Seitz
2024-05-03 14:52:00 +02:00
committed by GitHub
parent f20e4cbad8
commit 43599f983c
15 changed files with 2 additions and 2 deletions

View File

@ -0,0 +1,41 @@
[package]
name = "reth-node-ethereum"
version.workspace = true
edition.workspace = true
rust-version.workspace = true
license.workspace = true
homepage.workspace = true
repository.workspace = true
[lints]
workspace = true
[dependencies]
# reth
reth-payload-builder.workspace = true
reth-ethereum-engine-primitives.workspace = true
reth-basic-payload-builder.workspace = true
reth-ethereum-payload-builder.workspace = true
reth-node-builder.workspace = true
reth-tracing.workspace = true
reth-provider.workspace = true
reth-transaction-pool.workspace = true
reth-network.workspace = true
reth-evm-ethereum.workspace = true
# misc
eyre.workspace = true
[dev-dependencies]
reth.workspace = true
reth-db.workspace = true
reth-exex.workspace = true
reth-node-api.workspace = true
reth-node-core.workspace = true
reth-primitives.workspace = true
reth-e2e-test-utils.workspace = true
futures.workspace = true
tokio.workspace = true
futures-util.workspace = true
serde_json.workspace = true

View File

@ -0,0 +1,6 @@
//! Ethereum EVM support
#[doc(inline)]
pub use reth_evm_ethereum::execute::EthExecutorProvider;
#[doc(inline)]
pub use reth_evm_ethereum::EthEvmConfig;

View File

@ -0,0 +1,17 @@
//! Standalone crate for ethereum-specific Reth configuration and builder types.
#![doc(
html_logo_url = "https://raw.githubusercontent.com/paradigmxyz/reth/main/assets/reth-docs.png",
html_favicon_url = "https://avatars0.githubusercontent.com/u/97369466?s=256",
issue_tracker_base_url = "https://github.com/paradigmxyz/reth/issues/"
)]
#![cfg_attr(not(test), warn(unused_crate_dependencies))]
#![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))]
pub use reth_ethereum_engine_primitives::EthEngineTypes;
pub mod evm;
pub use evm::{EthEvmConfig, EthExecutorProvider};
pub mod node;
pub use node::EthereumNode;

View File

@ -0,0 +1,227 @@
//! Ethereum Node types config.
use crate::{EthEngineTypes, EthEvmConfig};
use reth_basic_payload_builder::{BasicPayloadJobGenerator, BasicPayloadJobGeneratorConfig};
use reth_evm_ethereum::execute::EthExecutorProvider;
use reth_network::NetworkHandle;
use reth_node_builder::{
components::{
ComponentsBuilder, ExecutorBuilder, NetworkBuilder, PayloadServiceBuilder, PoolBuilder,
},
node::{FullNodeTypes, NodeTypes},
BuilderContext, Node, PayloadBuilderConfig,
};
use reth_payload_builder::{PayloadBuilderHandle, PayloadBuilderService};
use reth_provider::CanonStateSubscriptions;
use reth_tracing::tracing::{debug, info};
use reth_transaction_pool::{
blobstore::DiskFileBlobStore, EthTransactionPool, TransactionPool,
TransactionValidationTaskExecutor,
};
/// Type configuration for a regular Ethereum node.
#[derive(Debug, Default, Clone, Copy)]
#[non_exhaustive]
pub struct EthereumNode;
impl EthereumNode {
/// Returns a [ComponentsBuilder] configured for a regular Ethereum node.
pub fn components<Node>() -> ComponentsBuilder<
Node,
EthereumPoolBuilder,
EthereumPayloadBuilder,
EthereumNetworkBuilder,
EthereumExecutorBuilder,
>
where
Node: FullNodeTypes<Engine = EthEngineTypes>,
{
ComponentsBuilder::default()
.node_types::<Node>()
.pool(EthereumPoolBuilder::default())
.payload(EthereumPayloadBuilder::default())
.network(EthereumNetworkBuilder::default())
.executor(EthereumExecutorBuilder::default())
}
}
impl NodeTypes for EthereumNode {
type Primitives = ();
type Engine = EthEngineTypes;
}
impl<N> Node<N> for EthereumNode
where
N: FullNodeTypes<Engine = EthEngineTypes>,
{
type ComponentsBuilder = ComponentsBuilder<
N,
EthereumPoolBuilder,
EthereumPayloadBuilder,
EthereumNetworkBuilder,
EthereumExecutorBuilder,
>;
fn components_builder(self) -> Self::ComponentsBuilder {
Self::components()
}
}
/// A regular ethereum evm and executor builder.
#[derive(Debug, Default, Clone, Copy)]
#[non_exhaustive]
pub struct EthereumExecutorBuilder;
impl<Node> ExecutorBuilder<Node> for EthereumExecutorBuilder
where
Node: FullNodeTypes,
{
type EVM = EthEvmConfig;
type Executor = EthExecutorProvider<Self::EVM>;
async fn build_evm(
self,
ctx: &BuilderContext<Node>,
) -> eyre::Result<(Self::EVM, Self::Executor)> {
let chain_spec = ctx.chain_spec();
let evm_config = EthEvmConfig::default();
let executor = EthExecutorProvider::new(chain_spec, evm_config);
Ok((evm_config, executor))
}
}
/// A basic ethereum transaction pool.
///
/// This contains various settings that can be configured and take precedence over the node's
/// config.
#[derive(Debug, Default, Clone, Copy)]
#[non_exhaustive]
pub struct EthereumPoolBuilder {
// TODO add options for txpool args
}
impl<Node> PoolBuilder<Node> for EthereumPoolBuilder
where
Node: FullNodeTypes,
{
type Pool = EthTransactionPool<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(), Default::default())?;
let validator = TransactionValidationTaskExecutor::eth_builder(ctx.chain_spec())
.with_head_timestamp(ctx.head().timestamp)
.kzg_settings(ctx.kzg_settings()?)
.with_additional_tasks(1)
.build_with_tasks(
ctx.provider().clone(),
ctx.task_executor().clone(),
blob_store.clone(),
);
let transaction_pool =
reth_transaction_pool::Pool::eth_pool(validator, blob_store, ctx.pool_config());
info!(target: "reth::cli", "Transaction pool initialized");
let transactions_path = data_dir.txpool_transactions();
// spawn txpool maintenance task
{
let pool = transaction_pool.clone();
let chain_events = ctx.provider().canonical_state_stream();
let client = ctx.provider().clone();
let transactions_backup_config =
reth_transaction_pool::maintain::LocalTransactionBackupConfig::with_local_txs_backup(transactions_path);
ctx.task_executor().spawn_critical_with_graceful_shutdown_signal(
"local transactions backup task",
|shutdown| {
reth_transaction_pool::maintain::backup_local_transactions_task(
shutdown,
pool.clone(),
transactions_backup_config,
)
},
);
// spawn the maintenance task
ctx.task_executor().spawn_critical(
"txpool maintenance task",
reth_transaction_pool::maintain::maintain_transaction_pool_future(
client,
pool,
chain_events,
ctx.task_executor().clone(),
Default::default(),
),
);
debug!(target: "reth::cli", "Spawned txpool maintenance task");
}
Ok(transaction_pool)
}
}
/// A basic ethereum payload service.
#[derive(Debug, Default, Clone)]
#[non_exhaustive]
pub struct EthereumPayloadBuilder;
impl<Node, Pool> PayloadServiceBuilder<Node, Pool> for EthereumPayloadBuilder
where
Node: FullNodeTypes<Engine = EthEngineTypes>,
Pool: TransactionPool + Unpin + 'static,
{
async fn spawn_payload_service(
self,
ctx: &BuilderContext<Node>,
pool: Pool,
) -> eyre::Result<PayloadBuilderHandle<Node::Engine>> {
let payload_builder = reth_ethereum_payload_builder::EthereumPayloadBuilder::default();
let conf = ctx.payload_builder_config();
let payload_job_config = BasicPayloadJobGeneratorConfig::default()
.interval(conf.interval())
.deadline(conf.deadline())
.max_payload_tasks(conf.max_payload_tasks())
.extradata(conf.extradata_bytes());
let payload_generator = BasicPayloadJobGenerator::with_builder(
ctx.provider().clone(),
pool,
ctx.task_executor().clone(),
payload_job_config,
ctx.chain_spec(),
payload_builder,
);
let (payload_service, payload_builder) =
PayloadBuilderService::new(payload_generator, ctx.provider().canonical_state_stream());
ctx.task_executor().spawn_critical("payload builder service", Box::pin(payload_service));
Ok(payload_builder)
}
}
/// A basic ethereum payload service.
#[derive(Debug, Default, Clone, Copy)]
pub struct EthereumNetworkBuilder {
// TODO add closure to modify network
}
impl<Node, Pool> NetworkBuilder<Node, Pool> for EthereumNetworkBuilder
where
Node: FullNodeTypes,
Pool: TransactionPool + Unpin + 'static,
{
async fn build_network(
self,
ctx: &BuilderContext<Node>,
pool: Pool,
) -> eyre::Result<NetworkHandle> {
let network = ctx.network_builder().await?;
let handle = ctx.start_network(network, pool);
Ok(handle)
}
}

View File

@ -0,0 +1,96 @@
{
"config": {
"chainId": 1,
"homesteadBlock": 0,
"daoForkSupport": true,
"eip150Block": 0,
"eip155Block": 0,
"eip158Block": 0,
"byzantiumBlock": 0,
"constantinopleBlock": 0,
"petersburgBlock": 0,
"istanbulBlock": 0,
"muirGlacierBlock": 0,
"berlinBlock": 0,
"londonBlock": 0,
"arrowGlacierBlock": 0,
"grayGlacierBlock": 0,
"shanghaiTime": 0,
"cancunTime": 0,
"terminalTotalDifficulty": "0x0",
"terminalTotalDifficultyPassed": true
},
"nonce": "0x0",
"timestamp": "0x0",
"extraData": "0x00",
"gasLimit": "0x1c9c380",
"difficulty": "0x0",
"mixHash": "0x0000000000000000000000000000000000000000000000000000000000000000",
"coinbase": "0x0000000000000000000000000000000000000000",
"alloc": {
"0x14dc79964da2c08b23698b3d3cc7ca32193d9955": {
"balance": "0xd3c21bcecceda1000000"
},
"0x15d34aaf54267db7d7c367839aaf71a00a2c6a65": {
"balance": "0xd3c21bcecceda1000000"
},
"0x1cbd3b2770909d4e10f157cabc84c7264073c9ec": {
"balance": "0xd3c21bcecceda1000000"
},
"0x23618e81e3f5cdf7f54c3d65f7fbc0abf5b21e8f": {
"balance": "0xd3c21bcecceda1000000"
},
"0x2546bcd3c84621e976d8185a91a922ae77ecec30": {
"balance": "0xd3c21bcecceda1000000"
},
"0x3c44cdddb6a900fa2b585dd299e03d12fa4293bc": {
"balance": "0xd3c21bcecceda1000000"
},
"0x70997970c51812dc3a010c7d01b50e0d17dc79c8": {
"balance": "0xd3c21bcecceda1000000"
},
"0x71be63f3384f5fb98995898a86b02fb2426c5788": {
"balance": "0xd3c21bcecceda1000000"
},
"0x8626f6940e2eb28930efb4cef49b2d1f2c9c1199": {
"balance": "0xd3c21bcecceda1000000"
},
"0x90f79bf6eb2c4f870365e785982e1f101e93b906": {
"balance": "0xd3c21bcecceda1000000"
},
"0x976ea74026e726554db657fa54763abd0c3a0aa9": {
"balance": "0xd3c21bcecceda1000000"
},
"0x9965507d1a55bcc2695c58ba16fb37d819b0a4dc": {
"balance": "0xd3c21bcecceda1000000"
},
"0x9c41de96b2088cdc640c6182dfcf5491dc574a57": {
"balance": "0xd3c21bcecceda1000000"
},
"0xa0ee7a142d267c1f36714e4a8f75612f20a79720": {
"balance": "0xd3c21bcecceda1000000"
},
"0xbcd4042de499d14e55001ccbb24a551f3b954096": {
"balance": "0xd3c21bcecceda1000000"
},
"0xbda5747bfd65f08deb54cb465eb87d40e51b197e": {
"balance": "0xd3c21bcecceda1000000"
},
"0xcd3b766ccdd6ae721141f452c550ca635964ce71": {
"balance": "0xd3c21bcecceda1000000"
},
"0xdd2fd4581271e230360230f9337d5c0430bf44c0": {
"balance": "0xd3c21bcecceda1000000"
},
"0xdf3e18d64bc6a983f673ab319ccae4f1a57c7097": {
"balance": "0xd3c21bcecceda1000000"
},
"0xf39fd6e51aad88f6f4ce6ab8827279cfffb92266": {
"balance": "0xd3c21bcecceda1000000"
},
"0xfabb0ac9d68b0b445fb7357272ff202c5651694a": {
"balance": "0xd3c21bcecceda1000000"
}
},
"number": "0x0"
}

View File

@ -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(())
}

View File

@ -0,0 +1,80 @@
use crate::utils::EthNode;
use futures::StreamExt;
use reth::rpc::eth::EthTransactions;
use reth_e2e_test_utils::setup;
use reth_primitives::{b256, hex, ChainSpec, Genesis};
use reth_provider::CanonStateSubscriptions;
use std::sync::Arc;
#[tokio::test]
async fn can_run_dev_node() -> eyre::Result<()> {
reth_tracing::init_test_tracing();
let (mut nodes, _tasks, _) = setup(1, custom_chain(), true).await?;
assert_chain_advances(nodes.pop().unwrap()).await;
Ok(())
}
async fn assert_chain_advances(mut node: EthNode) {
let mut notifications = node.inner.provider.canonical_state_stream();
// submit tx through rpc
let raw_tx = hex!("02f876820a28808477359400847735940082520894ab0840c0e43688012c1adb0f5e3fc665188f83d28a029d394a5d630544000080c080a0a044076b7e67b5deecc63f61a8d7913fab86ca365b344b5759d1fe3563b4c39ea019eab979dd000da04dfc72bb0377c092d30fd9e1cab5ae487de49586cc8b0090");
let eth_api = node.inner.rpc_registry.eth_api();
let hash = eth_api.send_raw_transaction(raw_tx.into()).await.unwrap();
let expected = b256!("b1c6512f4fc202c04355fbda66755e0e344b152e633010e8fd75ecec09b63398");
assert_eq!(hash, expected);
println!("submitted transaction: {hash}");
let head = notifications.next().await.unwrap();
let tx = head.tip().transactions().next().unwrap();
assert_eq!(tx.hash(), hash);
println!("mined transaction: {hash}");
}
fn custom_chain() -> Arc<ChainSpec> {
let custom_genesis = r#"
{
"nonce": "0x42",
"timestamp": "0x0",
"extraData": "0x5343",
"gasLimit": "0x1388",
"difficulty": "0x400000000",
"mixHash": "0x0000000000000000000000000000000000000000000000000000000000000000",
"coinbase": "0x0000000000000000000000000000000000000000",
"alloc": {
"0x6Be02d1d3665660d22FF9624b7BE0551ee1Ac91b": {
"balance": "0x4a47e3c12448f4ad000000"
}
},
"number": "0x0",
"gasUsed": "0x0",
"parentHash": "0x0000000000000000000000000000000000000000000000000000000000000000",
"config": {
"ethash": {},
"chainId": 2600,
"homesteadBlock": 0,
"eip150Block": 0,
"eip155Block": 0,
"eip158Block": 0,
"byzantiumBlock": 0,
"constantinopleBlock": 0,
"petersburgBlock": 0,
"istanbulBlock": 0,
"berlinBlock": 0,
"londonBlock": 0,
"terminalTotalDifficulty": 0,
"terminalTotalDifficultyPassed": true,
"shanghaiTime": 0
}
}
"#;
let genesis: Genesis = serde_json::from_str(custom_genesis).unwrap();
Arc::new(genesis.into())
}

View File

@ -0,0 +1,130 @@
use crate::utils::eth_payload_attributes;
use reth::{
args::RpcServerArgs,
builder::{NodeBuilder, NodeConfig, NodeHandle},
tasks::TaskManager,
};
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;
#[tokio::test]
async fn can_run_eth_node() -> eyre::Result<()> {
reth_tracing::init_test_tracing();
let (mut nodes, _tasks, _wallet) = setup::<EthereumNode>(
1,
Arc::new(
ChainSpecBuilder::default()
.chain(MAINNET.chain)
.genesis(serde_json::from_str(include_str!("../assets/genesis.json")).unwrap())
.cancun_activated()
.build(),
),
false,
)
.await?;
let mut node = nodes.pop().unwrap();
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
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(())
}
#[tokio::test]
#[cfg(unix)]
async fn can_run_eth_node_with_auth_engine_api_over_ipc() -> eyre::Result<()> {
reth_tracing::init_test_tracing();
let exec = TaskManager::current();
let exec = exec.executor();
// Chain spec with test allocs
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(),
);
// Node setup
let node_config = NodeConfig::test()
.with_chain(chain_spec)
.with_rpc(RpcServerArgs::default().with_unused_ports().with_http().with_auth_ipc());
let NodeHandle { node, node_exit_future: _ } = NodeBuilder::new(node_config)
.testing_node(exec)
.node(EthereumNode::default())
.launch()
.await?;
let mut node = NodeTestContext::new(node).await?;
// Configure wallet from test mnemonic and create dummy transfer tx
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
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(())
}
#[tokio::test]
#[cfg(unix)]
async fn test_failed_run_eth_node_with_no_auth_engine_api_over_ipc_opts() -> eyre::Result<()> {
reth_tracing::init_test_tracing();
let exec = TaskManager::current();
let exec = exec.executor();
// Chain spec with test allocs
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(),
);
// Node setup
let node_config = NodeConfig::test().with_chain(chain_spec);
let NodeHandle { node, node_exit_future: _ } = NodeBuilder::new(node_config)
.testing_node(exec)
.node(EthereumNode::default())
.launch()
.await?;
let node = NodeTestContext::new(node).await?;
// Ensure that the engine api client is not available
let client = node.inner.engine_ipc_client().await;
assert!(client.is_none(), "ipc auth should be disabled by default");
Ok(())
}

View File

@ -0,0 +1,7 @@
mod blobs;
mod dev;
mod eth;
mod p2p;
mod utils;
fn main() {}

View File

@ -0,0 +1,47 @@
use crate::utils::eth_payload_attributes;
use reth_e2e_test_utils::{setup, transaction::TransactionTestContext};
use reth_node_ethereum::EthereumNode;
use reth_primitives::{ChainSpecBuilder, MAINNET};
use std::sync::Arc;
#[tokio::test]
async fn can_sync() -> eyre::Result<()> {
reth_tracing::init_test_tracing();
let (mut nodes, _tasks, wallet) = setup::<EthereumNode>(
2,
Arc::new(
ChainSpecBuilder::default()
.chain(MAINNET.chain)
.genesis(serde_json::from_str(include_str!("../assets/genesis.json")).unwrap())
.cancun_activated()
.build(),
),
false,
)
.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 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, block_hash).await?;
// expect second node advanced via p2p gossip
second_node.assert_new_block(tx_hash, block_hash, 1).await?;
Ok(())
}

View File

@ -0,0 +1,20 @@
use reth::rpc::types::engine::PayloadAttributes;
use reth_e2e_test_utils::NodeHelperType;
use reth_node_ethereum::EthereumNode;
use reth_payload_builder::EthPayloadBuilderAttributes;
use reth_primitives::{Address, B256};
/// Ethereum Node Helper type
pub(crate) type EthNode = NodeHelperType<EthereumNode>;
/// Helper function to create a new eth payload attributes
pub(crate) fn eth_payload_attributes(timestamp: u64) -> EthPayloadBuilderAttributes {
let attributes = PayloadAttributes {
timestamp,
prev_randao: B256::ZERO,
suggested_fee_recipient: Address::ZERO,
withdrawals: Some(vec![]),
parent_beacon_block_root: Some(B256::ZERO),
};
EthPayloadBuilderAttributes::new(B256::ZERO, attributes)
}

View File

@ -0,0 +1,43 @@
//! Node builder setup tests.
use reth_db::test_utils::create_test_rw_db;
use reth_node_api::FullNodeComponents;
use reth_node_builder::{NodeBuilder, NodeConfig};
use reth_node_ethereum::node::EthereumNode;
#[test]
fn test_basic_setup() {
// parse CLI -> config
let config = NodeConfig::test();
let db = create_test_rw_db();
let msg = "On components".to_string();
let _builder = NodeBuilder::new(config)
.with_database(db)
.with_types::<EthereumNode>()
.with_components(EthereumNode::components())
.on_component_initialized(move |ctx| {
let _provider = ctx.provider();
println!("{msg}");
Ok(())
})
.on_node_started(|_full_node| Ok(()))
.on_rpc_started(|_ctx, handles| {
let _client = handles.rpc.http_client();
Ok(())
})
.extend_rpc_modules(|ctx| {
let _ = ctx.config();
let _ = ctx.node().provider();
Ok(())
})
.check_launch();
}
#[test]
fn test_node_setup() {
let config = NodeConfig::test();
let db = create_test_rw_db();
let _builder =
NodeBuilder::new(config).with_database(db).node(EthereumNode::default()).check_launch();
}

View File

@ -0,0 +1,38 @@
use futures::future;
use reth_db::test_utils::create_test_rw_db;
use reth_exex::ExExContext;
use reth_node_api::FullNodeComponents;
use reth_node_builder::{NodeBuilder, NodeConfig};
use reth_node_ethereum::EthereumNode;
use std::{
future::Future,
pin::Pin,
task::{Context, Poll},
};
struct DummyExEx<Node: FullNodeComponents> {
_ctx: ExExContext<Node>,
}
impl<Node> Future for DummyExEx<Node>
where
Node: FullNodeComponents,
{
type Output = eyre::Result<()>;
fn poll(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll<Self::Output> {
Poll::Pending
}
}
#[test]
fn basic_exex() {
let config = NodeConfig::test();
let db = create_test_rw_db();
let _builder = NodeBuilder::new(config)
.with_database(db)
.with_types::<EthereumNode>()
.with_components(EthereumNode::components())
.install_exex("dummy", move |ctx| future::ok(DummyExEx { _ctx: ctx }))
.check_launch();
}

View File

@ -0,0 +1,4 @@
mod builder;
mod exex;
fn main() {}