mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
chore: move node-ethereum to ethereum/node (#8076)
This commit is contained in:
41
crates/ethereum/node/Cargo.toml
Normal file
41
crates/ethereum/node/Cargo.toml
Normal 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
|
||||
|
||||
6
crates/ethereum/node/src/evm.rs
Normal file
6
crates/ethereum/node/src/evm.rs
Normal 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;
|
||||
17
crates/ethereum/node/src/lib.rs
Normal file
17
crates/ethereum/node/src/lib.rs
Normal 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;
|
||||
227
crates/ethereum/node/src/node.rs
Normal file
227
crates/ethereum/node/src/node.rs
Normal 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)
|
||||
}
|
||||
}
|
||||
96
crates/ethereum/node/tests/assets/genesis.json
Normal file
96
crates/ethereum/node/tests/assets/genesis.json
Normal 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"
|
||||
}
|
||||
96
crates/ethereum/node/tests/e2e/blobs.rs
Normal file
96
crates/ethereum/node/tests/e2e/blobs.rs
Normal 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(())
|
||||
}
|
||||
80
crates/ethereum/node/tests/e2e/dev.rs
Normal file
80
crates/ethereum/node/tests/e2e/dev.rs
Normal 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())
|
||||
}
|
||||
130
crates/ethereum/node/tests/e2e/eth.rs
Normal file
130
crates/ethereum/node/tests/e2e/eth.rs
Normal 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(())
|
||||
}
|
||||
7
crates/ethereum/node/tests/e2e/main.rs
Normal file
7
crates/ethereum/node/tests/e2e/main.rs
Normal file
@ -0,0 +1,7 @@
|
||||
mod blobs;
|
||||
mod dev;
|
||||
mod eth;
|
||||
mod p2p;
|
||||
mod utils;
|
||||
|
||||
fn main() {}
|
||||
47
crates/ethereum/node/tests/e2e/p2p.rs
Normal file
47
crates/ethereum/node/tests/e2e/p2p.rs
Normal 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(())
|
||||
}
|
||||
20
crates/ethereum/node/tests/e2e/utils.rs
Normal file
20
crates/ethereum/node/tests/e2e/utils.rs
Normal 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)
|
||||
}
|
||||
43
crates/ethereum/node/tests/it/builder.rs
Normal file
43
crates/ethereum/node/tests/it/builder.rs
Normal 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();
|
||||
}
|
||||
38
crates/ethereum/node/tests/it/exex.rs
Normal file
38
crates/ethereum/node/tests/it/exex.rs
Normal 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();
|
||||
}
|
||||
4
crates/ethereum/node/tests/it/main.rs
Normal file
4
crates/ethereum/node/tests/it/main.rs
Normal file
@ -0,0 +1,4 @@
|
||||
mod builder;
|
||||
mod exex;
|
||||
|
||||
fn main() {}
|
||||
Reference in New Issue
Block a user