mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 19:09:54 +00:00
feat(engine): new payload execution (#631)
* feat(engine): new payload execution * address PR comments * rm unused dev deps * add comment about lru * remove par_iter
This commit is contained in:
2
Cargo.lock
generated
2
Cargo.lock
generated
@ -3460,6 +3460,7 @@ dependencies = [
|
|||||||
"async-trait",
|
"async-trait",
|
||||||
"auto_impl",
|
"auto_impl",
|
||||||
"futures",
|
"futures",
|
||||||
|
"reth-executor",
|
||||||
"reth-interfaces",
|
"reth-interfaces",
|
||||||
"reth-primitives",
|
"reth-primitives",
|
||||||
"reth-provider",
|
"reth-provider",
|
||||||
@ -3817,6 +3818,7 @@ dependencies = [
|
|||||||
"hex-literal",
|
"hex-literal",
|
||||||
"modular-bitfield",
|
"modular-bitfield",
|
||||||
"parity-scale-codec",
|
"parity-scale-codec",
|
||||||
|
"parking_lot 0.12.1",
|
||||||
"postcard",
|
"postcard",
|
||||||
"rand 0.8.5",
|
"rand 0.8.5",
|
||||||
"reth-codecs",
|
"reth-codecs",
|
||||||
|
|||||||
@ -13,6 +13,7 @@ reth-interfaces = { path = "../interfaces" }
|
|||||||
reth-provider = { path = "../storage/provider" }
|
reth-provider = { path = "../storage/provider" }
|
||||||
reth-rlp = { path = "../common/rlp" }
|
reth-rlp = { path = "../common/rlp" }
|
||||||
reth-rpc-types = { path = "../net/rpc-types" }
|
reth-rpc-types = { path = "../net/rpc-types" }
|
||||||
|
reth-executor = { path = "../executor" }
|
||||||
|
|
||||||
# async
|
# async
|
||||||
futures = "0.3"
|
futures = "0.3"
|
||||||
@ -20,11 +21,9 @@ async-trait = "0.1.57"
|
|||||||
tokio = { version = "1", features = ["sync"] }
|
tokio = { version = "1", features = ["sync"] }
|
||||||
tokio-stream = "0.1"
|
tokio-stream = "0.1"
|
||||||
|
|
||||||
# common
|
# misc
|
||||||
thiserror = "1.0.37"
|
thiserror = "1.0.37"
|
||||||
auto_impl = "1.0"
|
auto_impl = "1.0"
|
||||||
|
|
||||||
# io
|
|
||||||
serde = { version = "1.0", optional = true }
|
serde = { version = "1.0", optional = true }
|
||||||
|
|
||||||
[features]
|
[features]
|
||||||
|
|||||||
@ -1,4 +1,5 @@
|
|||||||
//! Reth block execution/validation configuration and constants
|
//! Reth block execution/validation configuration and constants
|
||||||
|
use reth_executor::{Config as ExecutorConfig, SpecUpgrades};
|
||||||
use reth_primitives::BlockNumber;
|
use reth_primitives::BlockNumber;
|
||||||
|
|
||||||
#[cfg(feature = "serde")]
|
#[cfg(feature = "serde")]
|
||||||
@ -75,3 +76,24 @@ impl Default for Config {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
impl From<&Config> for ExecutorConfig {
|
||||||
|
fn from(value: &Config) -> Self {
|
||||||
|
Self {
|
||||||
|
chain_id: value.chain_id.into(),
|
||||||
|
spec_upgrades: SpecUpgrades {
|
||||||
|
frontier: 0,
|
||||||
|
homestead: value.homestead_block,
|
||||||
|
tangerine_whistle: value.eip_150_block,
|
||||||
|
spurious_dragon: value.eip_158_block,
|
||||||
|
byzantium: value.byzantium_block,
|
||||||
|
petersburg: value.petersburg_block,
|
||||||
|
istanbul: value.istanbul_block,
|
||||||
|
berlin: value.berlin_block,
|
||||||
|
london: value.london_block,
|
||||||
|
paris: value.paris_block,
|
||||||
|
shanghai: u64::MAX, // TODO: change once known
|
||||||
|
},
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|||||||
@ -29,6 +29,12 @@ pub enum EngineApiError {
|
|||||||
/// Latest available timestamp.
|
/// Latest available timestamp.
|
||||||
latest: u64,
|
latest: u64,
|
||||||
},
|
},
|
||||||
|
/// Failed to recover transaction signer.
|
||||||
|
#[error("Failed to recover signer for payload transaction: {hash:?}")]
|
||||||
|
PayloadSignerRecovery {
|
||||||
|
/// The hash of the failed transaction
|
||||||
|
hash: H256,
|
||||||
|
},
|
||||||
/// Received pre-merge payload.
|
/// Received pre-merge payload.
|
||||||
#[error("Received pre-merge payload.")]
|
#[error("Received pre-merge payload.")]
|
||||||
PayloadPreMerge,
|
PayloadPreMerge,
|
||||||
@ -47,7 +53,7 @@ pub enum EngineApiError {
|
|||||||
},
|
},
|
||||||
/// Terminal block hash mismatch during transition configuration exchange.
|
/// Terminal block hash mismatch during transition configuration exchange.
|
||||||
#[error(
|
#[error(
|
||||||
"Invalid transition terminal block hash. Execution: {execution:?}. Consensus: {consensus}"
|
"Invalid transition terminal block hash. Execution: {execution:?}. Consensus: {consensus:?}"
|
||||||
)]
|
)]
|
||||||
TerminalBlockHash {
|
TerminalBlockHash {
|
||||||
/// Execution terminal block hash. `None` if block number is not found in the database.
|
/// Execution terminal block hash. `None` if block number is not found in the database.
|
||||||
|
|||||||
@ -1,11 +1,15 @@
|
|||||||
use futures::StreamExt;
|
use futures::StreamExt;
|
||||||
|
use reth_executor::{
|
||||||
|
executor,
|
||||||
|
revm_wrap::{State, SubState},
|
||||||
|
};
|
||||||
use reth_interfaces::consensus::ForkchoiceState;
|
use reth_interfaces::consensus::ForkchoiceState;
|
||||||
use reth_primitives::{
|
use reth_primitives::{
|
||||||
proofs::{self, EMPTY_LIST_HASH},
|
proofs::{self, EMPTY_LIST_HASH},
|
||||||
rpc::BlockId,
|
rpc::BlockId,
|
||||||
Header, SealedBlock, TransactionSigned, H64,
|
Header, SealedBlock, TransactionSigned, H64,
|
||||||
};
|
};
|
||||||
use reth_provider::{BlockProvider, HeaderProvider};
|
use reth_provider::{BlockProvider, HeaderProvider, StateProvider};
|
||||||
use reth_rlp::Decodable;
|
use reth_rlp::Decodable;
|
||||||
use reth_rpc_types::engine::{
|
use reth_rpc_types::engine::{
|
||||||
ExecutionPayload, ForkchoiceUpdated, PayloadAttributes, PayloadStatus, PayloadStatusEnum,
|
ExecutionPayload, ForkchoiceUpdated, PayloadAttributes, PayloadStatus, PayloadStatusEnum,
|
||||||
@ -21,8 +25,9 @@ use std::{
|
|||||||
use tokio::sync::oneshot;
|
use tokio::sync::oneshot;
|
||||||
use tokio_stream::wrappers::UnboundedReceiverStream;
|
use tokio_stream::wrappers::UnboundedReceiverStream;
|
||||||
|
|
||||||
mod error;
|
|
||||||
use crate::Config;
|
use crate::Config;
|
||||||
|
|
||||||
|
mod error;
|
||||||
pub use error::{EngineApiError, EngineApiResult};
|
pub use error::{EngineApiError, EngineApiResult};
|
||||||
|
|
||||||
/// The Engine API response sender
|
/// The Engine API response sender
|
||||||
@ -51,6 +56,7 @@ pub trait ConsensusEngine {
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// Message type for communicating with [EthConsensusEngine]
|
/// Message type for communicating with [EthConsensusEngine]
|
||||||
|
#[derive(Debug)]
|
||||||
pub enum EngineMessage {
|
pub enum EngineMessage {
|
||||||
/// New payload message
|
/// New payload message
|
||||||
NewPayload(ExecutionPayload, EngineApiSender<PayloadStatus>),
|
NewPayload(ExecutionPayload, EngineApiSender<PayloadStatus>),
|
||||||
@ -72,16 +78,17 @@ pub enum EngineMessage {
|
|||||||
/// The consensus engine API implementation
|
/// The consensus engine API implementation
|
||||||
#[must_use = "EthConsensusEngine does nothing unless polled."]
|
#[must_use = "EthConsensusEngine does nothing unless polled."]
|
||||||
pub struct EthConsensusEngine<Client> {
|
pub struct EthConsensusEngine<Client> {
|
||||||
|
client: Arc<Client>,
|
||||||
/// Consensus configuration
|
/// Consensus configuration
|
||||||
config: Config,
|
config: Config,
|
||||||
client: Arc<Client>,
|
|
||||||
/// Placeholder for storing future blocks
|
|
||||||
local_store: HashMap<H64, ExecutionPayload>, // TODO: bound
|
|
||||||
// remote_store: HashMap<H64, ExecutionPayload>,
|
|
||||||
rx: UnboundedReceiverStream<EngineMessage>,
|
rx: UnboundedReceiverStream<EngineMessage>,
|
||||||
|
// TODO: Placeholder for storing future blocks. Make cache bounded.
|
||||||
|
// Use [lru](https://crates.io/crates/lru) crate
|
||||||
|
local_store: HashMap<H64, ExecutionPayload>,
|
||||||
|
// remote_store: HashMap<H64, ExecutionPayload>,
|
||||||
}
|
}
|
||||||
|
|
||||||
impl<Client: HeaderProvider + BlockProvider> EthConsensusEngine<Client> {
|
impl<Client: HeaderProvider + BlockProvider + StateProvider> EthConsensusEngine<Client> {
|
||||||
fn on_message(&mut self, msg: EngineMessage) {
|
fn on_message(&mut self, msg: EngineMessage) {
|
||||||
match msg {
|
match msg {
|
||||||
EngineMessage::GetPayload(payload_id, tx) => {
|
EngineMessage::GetPayload(payload_id, tx) => {
|
||||||
@ -105,8 +112,8 @@ impl<Client: HeaderProvider + BlockProvider> EthConsensusEngine<Client> {
|
|||||||
/// `base_fee_per_gas` fields.
|
/// `base_fee_per_gas` fields.
|
||||||
///
|
///
|
||||||
/// NOTE: The log bloom is assumed to be validated during serialization.
|
/// NOTE: The log bloom is assumed to be validated during serialization.
|
||||||
/// NOTE: Ommers hash is validated upon computing block hash and comparing the value with
|
/// NOTE: Empty ommers, nonce and difficulty values are validated upon computing block hash and
|
||||||
/// `payload.block_hash`.
|
/// comparing the value with `payload.block_hash`.
|
||||||
/// Ref: https://github.com/ethereum/go-ethereum/blob/79a478bb6176425c2400e949890e668a3d9a3d05/core/beacon/types.go#L145
|
/// Ref: https://github.com/ethereum/go-ethereum/blob/79a478bb6176425c2400e949890e668a3d9a3d05/core/beacon/types.go#L145
|
||||||
fn try_construct_block(&self, payload: ExecutionPayload) -> EngineApiResult<SealedBlock> {
|
fn try_construct_block(&self, payload: ExecutionPayload) -> EngineApiResult<SealedBlock> {
|
||||||
if payload.extra_data.len() > 32 {
|
if payload.extra_data.len() > 32 {
|
||||||
@ -155,7 +162,9 @@ impl<Client: HeaderProvider + BlockProvider> EthConsensusEngine<Client> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
impl<Client: HeaderProvider + BlockProvider> ConsensusEngine for EthConsensusEngine<Client> {
|
impl<Client: HeaderProvider + BlockProvider + StateProvider> ConsensusEngine
|
||||||
|
for EthConsensusEngine<Client>
|
||||||
|
{
|
||||||
fn get_payload(&self, payload_id: H64) -> Option<ExecutionPayload> {
|
fn get_payload(&self, payload_id: H64) -> Option<ExecutionPayload> {
|
||||||
self.local_store.get(&payload_id).cloned()
|
self.local_store.get(&payload_id).cloned()
|
||||||
}
|
}
|
||||||
@ -194,9 +203,29 @@ impl<Client: HeaderProvider + BlockProvider> ConsensusEngine for EthConsensusEng
|
|||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO: execute block
|
let (header, body, _) = block.split();
|
||||||
|
let transactions = body
|
||||||
Ok(PayloadStatus::new(PayloadStatusEnum::Valid, block.hash()))
|
.into_iter()
|
||||||
|
.map(|tx| {
|
||||||
|
let tx_hash = tx.hash;
|
||||||
|
tx.into_ecrecovered().ok_or(EngineApiError::PayloadSignerRecovery { hash: tx_hash })
|
||||||
|
})
|
||||||
|
.collect::<Result<Vec<_>, EngineApiError>>()?;
|
||||||
|
let state_provider = SubState::new(State::new(&*self.client));
|
||||||
|
let config = (&self.config).into();
|
||||||
|
match executor::execute_and_verify_receipt(
|
||||||
|
&header,
|
||||||
|
&transactions,
|
||||||
|
&[],
|
||||||
|
&config,
|
||||||
|
state_provider,
|
||||||
|
) {
|
||||||
|
Ok(_) => Ok(PayloadStatus::new(PayloadStatusEnum::Valid, header.hash())),
|
||||||
|
Err(err) => Ok(PayloadStatus::new(
|
||||||
|
PayloadStatusEnum::Invalid { validation_error: err.to_string() },
|
||||||
|
header.parent_hash, // The parent hash is already in our database hence it is valid
|
||||||
|
)),
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
fn fork_choice_updated(
|
fn fork_choice_updated(
|
||||||
@ -222,6 +251,8 @@ impl<Client: HeaderProvider + BlockProvider> ConsensusEngine for EthConsensusEng
|
|||||||
return Ok(ForkchoiceUpdated::from_status(PayloadStatusEnum::Syncing))
|
return Ok(ForkchoiceUpdated::from_status(PayloadStatusEnum::Syncing))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// TODO: update tip
|
||||||
|
|
||||||
let chain_info = self.client.chain_info()?;
|
let chain_info = self.client.chain_info()?;
|
||||||
Ok(ForkchoiceUpdated::from_status(PayloadStatusEnum::Valid)
|
Ok(ForkchoiceUpdated::from_status(PayloadStatusEnum::Valid)
|
||||||
.with_latest_valid_hash(chain_info.best_hash))
|
.with_latest_valid_hash(chain_info.best_hash))
|
||||||
@ -274,7 +305,7 @@ impl<Client: HeaderProvider + BlockProvider> ConsensusEngine for EthConsensusEng
|
|||||||
|
|
||||||
impl<Client> Future for EthConsensusEngine<Client>
|
impl<Client> Future for EthConsensusEngine<Client>
|
||||||
where
|
where
|
||||||
Client: HeaderProvider + BlockProvider + Unpin,
|
Client: HeaderProvider + BlockProvider + StateProvider + Unpin,
|
||||||
{
|
{
|
||||||
type Output = ();
|
type Output = ();
|
||||||
|
|
||||||
|
|||||||
@ -482,7 +482,7 @@ mod tests {
|
|||||||
hex_literal::hex, keccak256, Account, Address, Bytes, SealedBlock, StorageKey, H160, H256,
|
hex_literal::hex, keccak256, Account, Address, Bytes, SealedBlock, StorageKey, H160, H256,
|
||||||
U256,
|
U256,
|
||||||
};
|
};
|
||||||
use reth_provider::{AccountProvider, StateProvider};
|
use reth_provider::{AccountProvider, BlockHashProvider, StateProvider};
|
||||||
use reth_rlp::Decodable;
|
use reth_rlp::Decodable;
|
||||||
|
|
||||||
use super::*;
|
use super::*;
|
||||||
@ -519,6 +519,12 @@ mod tests {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
impl BlockHashProvider for StateProviderTest {
|
||||||
|
fn block_hash(&self, number: U256) -> reth_interfaces::Result<Option<H256>> {
|
||||||
|
Ok(self.block_hash.get(&number).cloned())
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
impl StateProvider for StateProviderTest {
|
impl StateProvider for StateProviderTest {
|
||||||
fn storage(
|
fn storage(
|
||||||
&self,
|
&self,
|
||||||
@ -534,10 +540,6 @@ mod tests {
|
|||||||
fn bytecode_by_hash(&self, code_hash: H256) -> reth_interfaces::Result<Option<Bytes>> {
|
fn bytecode_by_hash(&self, code_hash: H256) -> reth_interfaces::Result<Option<Bytes>> {
|
||||||
Ok(self.contracts.get(&code_hash).cloned())
|
Ok(self.contracts.get(&code_hash).cloned())
|
||||||
}
|
}
|
||||||
|
|
||||||
fn block_hash(&self, number: U256) -> reth_interfaces::Result<Option<H256>> {
|
|
||||||
Ok(self.block_hash.get(&number).cloned())
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
#[test]
|
#[test]
|
||||||
|
|||||||
@ -58,10 +58,10 @@
|
|||||||
//! use std::sync::Arc;
|
//! use std::sync::Arc;
|
||||||
//! use reth_network::config::{rng_secret_key, mainnet_nodes};
|
//! use reth_network::config::{rng_secret_key, mainnet_nodes};
|
||||||
//! use reth_network::{NetworkConfig, NetworkManager};
|
//! use reth_network::{NetworkConfig, NetworkManager};
|
||||||
//! use reth_provider::test_utils::TestApi;
|
//! use reth_provider::test_utils::NoopProvider;
|
||||||
//!
|
//!
|
||||||
//! // This block provider implementation is used for testing purposes.
|
//! // This block provider implementation is used for testing purposes.
|
||||||
//! let client = Arc::new(TestApi::default());
|
//! let client = Arc::new(NoopProvider::default());
|
||||||
//!
|
//!
|
||||||
//! // The key that's used for encrypting sessions and to identify our node.
|
//! // The key that's used for encrypting sessions and to identify our node.
|
||||||
//! let local_key = rng_secret_key();
|
//! let local_key = rng_secret_key();
|
||||||
@ -83,7 +83,7 @@
|
|||||||
//! ### Configure all components of the Network with the [`NetworkBuilder`]
|
//! ### Configure all components of the Network with the [`NetworkBuilder`]
|
||||||
//!
|
//!
|
||||||
//! ```
|
//! ```
|
||||||
//! use reth_provider::test_utils::TestApi;
|
//! use reth_provider::test_utils::NoopProvider;
|
||||||
//! use reth_transaction_pool::TransactionPool;
|
//! use reth_transaction_pool::TransactionPool;
|
||||||
//! use std::sync::Arc;
|
//! use std::sync::Arc;
|
||||||
//! use reth_discv4::bootnodes::mainnet_nodes;
|
//! use reth_discv4::bootnodes::mainnet_nodes;
|
||||||
@ -91,7 +91,7 @@
|
|||||||
//! use reth_network::{NetworkConfig, NetworkManager};
|
//! use reth_network::{NetworkConfig, NetworkManager};
|
||||||
//! async fn launch<Pool: TransactionPool>(pool: Pool) {
|
//! async fn launch<Pool: TransactionPool>(pool: Pool) {
|
||||||
//! // This block provider implementation is used for testing purposes.
|
//! // This block provider implementation is used for testing purposes.
|
||||||
//! let client = Arc::new(TestApi::default());
|
//! let client = Arc::new(NoopProvider::default());
|
||||||
//!
|
//!
|
||||||
//! // The key that's used for encrypting sessions and to identify our node.
|
//! // The key that's used for encrypting sessions and to identify our node.
|
||||||
//! let local_key = rng_secret_key();
|
//! let local_key = rng_secret_key();
|
||||||
|
|||||||
@ -208,7 +208,7 @@ where
|
|||||||
/// components of the network
|
/// components of the network
|
||||||
///
|
///
|
||||||
/// ```
|
/// ```
|
||||||
/// use reth_provider::test_utils::TestApi;
|
/// use reth_provider::test_utils::NoopProvider;
|
||||||
/// use reth_transaction_pool::TransactionPool;
|
/// use reth_transaction_pool::TransactionPool;
|
||||||
/// use std::sync::Arc;
|
/// use std::sync::Arc;
|
||||||
/// use reth_discv4::bootnodes::mainnet_nodes;
|
/// use reth_discv4::bootnodes::mainnet_nodes;
|
||||||
@ -216,7 +216,7 @@ where
|
|||||||
/// use reth_network::{NetworkConfig, NetworkManager};
|
/// use reth_network::{NetworkConfig, NetworkManager};
|
||||||
/// async fn launch<Pool: TransactionPool>(pool: Pool) {
|
/// async fn launch<Pool: TransactionPool>(pool: Pool) {
|
||||||
/// // This block provider implementation is used for testing purposes.
|
/// // This block provider implementation is used for testing purposes.
|
||||||
/// let client = Arc::new(TestApi::default());
|
/// let client = Arc::new(NoopProvider::default());
|
||||||
///
|
///
|
||||||
/// // The key that's used for encrypting sessions and to identify our node.
|
/// // The key that's used for encrypting sessions and to identify our node.
|
||||||
/// let local_key = rng_secret_key();
|
/// let local_key = rng_secret_key();
|
||||||
|
|||||||
@ -12,7 +12,7 @@ use reth_eth_wire::DisconnectReason;
|
|||||||
use reth_net_common::ban_list::BanList;
|
use reth_net_common::ban_list::BanList;
|
||||||
use reth_network::{NetworkConfig, NetworkEvent, NetworkManager, PeersConfig};
|
use reth_network::{NetworkConfig, NetworkEvent, NetworkManager, PeersConfig};
|
||||||
use reth_primitives::{NodeRecord, PeerId};
|
use reth_primitives::{NodeRecord, PeerId};
|
||||||
use reth_provider::test_utils::TestApi;
|
use reth_provider::test_utils::NoopProvider;
|
||||||
use secp256k1::SecretKey;
|
use secp256k1::SecretKey;
|
||||||
use std::{collections::HashSet, net::SocketAddr, sync::Arc, time::Duration};
|
use std::{collections::HashSet, net::SocketAddr, sync::Arc, time::Duration};
|
||||||
use tokio::task;
|
use tokio::task;
|
||||||
@ -94,7 +94,7 @@ async fn test_already_connected() {
|
|||||||
let mut net = Testnet::default();
|
let mut net = Testnet::default();
|
||||||
|
|
||||||
let secret_key = SecretKey::new(&mut rand::thread_rng());
|
let secret_key = SecretKey::new(&mut rand::thread_rng());
|
||||||
let client = Arc::new(TestApi::default());
|
let client = Arc::new(NoopProvider::default());
|
||||||
let p1 = PeerConfig::default();
|
let p1 = PeerConfig::default();
|
||||||
|
|
||||||
// initialize two peers with the same identifier
|
// initialize two peers with the same identifier
|
||||||
@ -140,8 +140,9 @@ async fn test_connect_with_boot_nodes() {
|
|||||||
let mut discv4 = Discv4Config::builder();
|
let mut discv4 = Discv4Config::builder();
|
||||||
discv4.add_boot_nodes(mainnet_nodes());
|
discv4.add_boot_nodes(mainnet_nodes());
|
||||||
|
|
||||||
let config =
|
let config = NetworkConfig::builder(Arc::new(NoopProvider::default()), secret_key)
|
||||||
NetworkConfig::builder(Arc::new(TestApi::default()), secret_key).discovery(discv4).build();
|
.discovery(discv4)
|
||||||
|
.build();
|
||||||
let network = NetworkManager::new(config).await.unwrap();
|
let network = NetworkManager::new(config).await.unwrap();
|
||||||
|
|
||||||
let handle = network.handle().clone();
|
let handle = network.handle().clone();
|
||||||
@ -161,7 +162,7 @@ async fn test_connect_with_builder() {
|
|||||||
let mut discv4 = Discv4Config::builder();
|
let mut discv4 = Discv4Config::builder();
|
||||||
discv4.add_boot_nodes(mainnet_nodes());
|
discv4.add_boot_nodes(mainnet_nodes());
|
||||||
|
|
||||||
let client = Arc::new(TestApi::default());
|
let client = Arc::new(NoopProvider::default());
|
||||||
let config = NetworkConfig::builder(Arc::clone(&client), secret_key).discovery(discv4).build();
|
let config = NetworkConfig::builder(Arc::clone(&client), secret_key).discovery(discv4).build();
|
||||||
let (handle, network, _, requests) = NetworkManager::new(config)
|
let (handle, network, _, requests) = NetworkManager::new(config)
|
||||||
.await
|
.await
|
||||||
@ -209,7 +210,7 @@ async fn test_incoming_node_id_blacklist() {
|
|||||||
|
|
||||||
let reth_p2p_socket = SocketAddr::new([127, 0, 0, 1].into(), 30303);
|
let reth_p2p_socket = SocketAddr::new([127, 0, 0, 1].into(), 30303);
|
||||||
let reth_disc_socket = SocketAddr::new([127, 0, 0, 1].into(), 30304);
|
let reth_disc_socket = SocketAddr::new([127, 0, 0, 1].into(), 30304);
|
||||||
let config = NetworkConfig::builder(Arc::new(TestApi::default()), secret_key)
|
let config = NetworkConfig::builder(Arc::new(NoopProvider::default()), secret_key)
|
||||||
.listener_addr(reth_p2p_socket)
|
.listener_addr(reth_p2p_socket)
|
||||||
.discovery_addr(reth_disc_socket)
|
.discovery_addr(reth_disc_socket)
|
||||||
.peer_config(peer_config)
|
.peer_config(peer_config)
|
||||||
@ -259,7 +260,7 @@ async fn test_incoming_connect_with_single_geth() {
|
|||||||
|
|
||||||
let reth_p2p_socket = SocketAddr::new([127, 0, 0, 1].into(), 30305);
|
let reth_p2p_socket = SocketAddr::new([127, 0, 0, 1].into(), 30305);
|
||||||
let reth_disc_socket = SocketAddr::new([127, 0, 0, 1].into(), 30306);
|
let reth_disc_socket = SocketAddr::new([127, 0, 0, 1].into(), 30306);
|
||||||
let config = NetworkConfig::builder(Arc::new(TestApi::default()), secret_key)
|
let config = NetworkConfig::builder(Arc::new(NoopProvider::default()), secret_key)
|
||||||
.listener_addr(reth_p2p_socket)
|
.listener_addr(reth_p2p_socket)
|
||||||
.discovery_addr(reth_disc_socket)
|
.discovery_addr(reth_disc_socket)
|
||||||
.build();
|
.build();
|
||||||
@ -294,7 +295,7 @@ async fn test_outgoing_connect_with_single_geth() {
|
|||||||
|
|
||||||
let reth_p2p_socket = SocketAddr::new([127, 0, 0, 1].into(), 30307);
|
let reth_p2p_socket = SocketAddr::new([127, 0, 0, 1].into(), 30307);
|
||||||
let reth_disc_socket = SocketAddr::new([127, 0, 0, 1].into(), 30308);
|
let reth_disc_socket = SocketAddr::new([127, 0, 0, 1].into(), 30308);
|
||||||
let config = NetworkConfig::builder(Arc::new(TestApi::default()), secret_key)
|
let config = NetworkConfig::builder(Arc::new(NoopProvider::default()), secret_key)
|
||||||
.listener_addr(reth_p2p_socket)
|
.listener_addr(reth_p2p_socket)
|
||||||
.discovery_addr(reth_disc_socket)
|
.discovery_addr(reth_disc_socket)
|
||||||
.build();
|
.build();
|
||||||
@ -341,7 +342,7 @@ async fn test_geth_disconnect() {
|
|||||||
|
|
||||||
let reth_p2p_socket = SocketAddr::new([127, 0, 0, 1].into(), 30309);
|
let reth_p2p_socket = SocketAddr::new([127, 0, 0, 1].into(), 30309);
|
||||||
let reth_disc_socket = SocketAddr::new([127, 0, 0, 1].into(), 30310);
|
let reth_disc_socket = SocketAddr::new([127, 0, 0, 1].into(), 30310);
|
||||||
let config = NetworkConfig::builder(Arc::new(TestApi::default()), secret_key)
|
let config = NetworkConfig::builder(Arc::new(NoopProvider::default()), secret_key)
|
||||||
.listener_addr(reth_p2p_socket)
|
.listener_addr(reth_p2p_socket)
|
||||||
.discovery_addr(reth_disc_socket)
|
.discovery_addr(reth_disc_socket)
|
||||||
.build();
|
.build();
|
||||||
|
|||||||
@ -1,7 +1,7 @@
|
|||||||
//! Tests for eth related requests
|
//! Tests for eth related requests
|
||||||
|
|
||||||
use super::testnet::Testnet;
|
use super::testnet::Testnet;
|
||||||
use crate::{MockEthProvider, NetworkEventStream};
|
use crate::NetworkEventStream;
|
||||||
use rand::Rng;
|
use rand::Rng;
|
||||||
use reth_eth_wire::BlockBody;
|
use reth_eth_wire::BlockBody;
|
||||||
use reth_interfaces::p2p::{
|
use reth_interfaces::p2p::{
|
||||||
@ -12,6 +12,7 @@ use reth_primitives::{
|
|||||||
Block, Bytes, Header, HeadersDirection, Signature, Transaction, TransactionKind,
|
Block, Bytes, Header, HeadersDirection, Signature, Transaction, TransactionKind,
|
||||||
TransactionSigned, TxEip2930, H256, U256,
|
TransactionSigned, TxEip2930, H256, U256,
|
||||||
};
|
};
|
||||||
|
use reth_provider::test_utils::MockEthProvider;
|
||||||
use std::sync::Arc;
|
use std::sync::Arc;
|
||||||
|
|
||||||
/// Returns a new [`TransactionSigned`] with some random parameters
|
/// Returns a new [`TransactionSigned`] with some random parameters
|
||||||
|
|||||||
@ -1,21 +1,16 @@
|
|||||||
//! A network implementation for testing purposes.
|
//! A network implementation for testing purposes.
|
||||||
|
|
||||||
use futures::{FutureExt, StreamExt};
|
use futures::{FutureExt, StreamExt};
|
||||||
use parking_lot::Mutex;
|
|
||||||
use pin_project::pin_project;
|
use pin_project::pin_project;
|
||||||
use reth_eth_wire::DisconnectReason;
|
use reth_eth_wire::DisconnectReason;
|
||||||
use reth_network::{
|
use reth_network::{
|
||||||
error::NetworkError, eth_requests::EthRequestHandler, NetworkConfig, NetworkEvent,
|
error::NetworkError, eth_requests::EthRequestHandler, NetworkConfig, NetworkEvent,
|
||||||
NetworkHandle, NetworkManager,
|
NetworkHandle, NetworkManager,
|
||||||
};
|
};
|
||||||
use reth_primitives::{
|
use reth_primitives::PeerId;
|
||||||
rpc::{BlockId, BlockNumber},
|
use reth_provider::{test_utils::NoopProvider, BlockProvider, HeaderProvider};
|
||||||
Block, BlockHash, Header, PeerId, H256, U256,
|
|
||||||
};
|
|
||||||
use reth_provider::{test_utils::TestApi, BlockProvider, ChainInfo, HeaderProvider};
|
|
||||||
use secp256k1::SecretKey;
|
use secp256k1::SecretKey;
|
||||||
use std::{
|
use std::{
|
||||||
collections::HashMap,
|
|
||||||
fmt,
|
fmt,
|
||||||
future::Future,
|
future::Future,
|
||||||
net::{Ipv4Addr, SocketAddr, SocketAddrV4},
|
net::{Ipv4Addr, SocketAddr, SocketAddrV4},
|
||||||
@ -145,7 +140,7 @@ where
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
impl Testnet<TestApi> {
|
impl Testnet<NoopProvider> {
|
||||||
/// Same as [`Self::try_create`] but panics on error
|
/// Same as [`Self::try_create`] but panics on error
|
||||||
pub async fn create(num_peers: usize) -> Self {
|
pub async fn create(num_peers: usize) -> Self {
|
||||||
Self::try_create(num_peers).await.unwrap()
|
Self::try_create(num_peers).await.unwrap()
|
||||||
@ -258,7 +253,7 @@ where
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
pub struct PeerConfig<C = TestApi> {
|
pub struct PeerConfig<C = NoopProvider> {
|
||||||
config: NetworkConfig<C>,
|
config: NetworkConfig<C>,
|
||||||
client: Arc<C>,
|
client: Arc<C>,
|
||||||
secret_key: SecretKey,
|
secret_key: SecretKey,
|
||||||
@ -286,7 +281,7 @@ where
|
|||||||
|
|
||||||
impl Default for PeerConfig {
|
impl Default for PeerConfig {
|
||||||
fn default() -> Self {
|
fn default() -> Self {
|
||||||
Self::new(Arc::new(TestApi::default()))
|
Self::new(Arc::new(NoopProvider::default()))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -324,92 +319,3 @@ impl NetworkEventStream {
|
|||||||
None
|
None
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/// A mock implementation for Provider interfaces.
|
|
||||||
#[derive(Debug, Clone, Default)]
|
|
||||||
pub struct MockEthProvider {
|
|
||||||
pub blocks: Arc<Mutex<HashMap<H256, Block>>>,
|
|
||||||
pub headers: Arc<Mutex<HashMap<H256, Header>>>,
|
|
||||||
}
|
|
||||||
|
|
||||||
impl MockEthProvider {
|
|
||||||
pub fn add_block(&self, hash: H256, block: Block) {
|
|
||||||
self.blocks.lock().insert(hash, block);
|
|
||||||
}
|
|
||||||
|
|
||||||
pub fn extend_blocks(&self, iter: impl IntoIterator<Item = (H256, Block)>) {
|
|
||||||
for (hash, block) in iter.into_iter() {
|
|
||||||
self.add_block(hash, block)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
pub fn add_header(&self, hash: H256, header: Header) {
|
|
||||||
self.headers.lock().insert(hash, header);
|
|
||||||
}
|
|
||||||
|
|
||||||
pub fn extend_headers(&self, iter: impl IntoIterator<Item = (H256, Header)>) {
|
|
||||||
for (hash, header) in iter.into_iter() {
|
|
||||||
self.add_header(hash, header)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
impl HeaderProvider for MockEthProvider {
|
|
||||||
fn header(&self, block_hash: &BlockHash) -> reth_interfaces::Result<Option<Header>> {
|
|
||||||
let lock = self.headers.lock();
|
|
||||||
Ok(lock.get(block_hash).cloned())
|
|
||||||
}
|
|
||||||
|
|
||||||
fn header_by_number(&self, num: u64) -> reth_interfaces::Result<Option<Header>> {
|
|
||||||
let lock = self.headers.lock();
|
|
||||||
Ok(lock.values().find(|h| h.number == num).cloned())
|
|
||||||
}
|
|
||||||
|
|
||||||
fn header_td(&self, _hash: &BlockHash) -> reth_interfaces::Result<Option<U256>> {
|
|
||||||
todo!()
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
impl BlockProvider for MockEthProvider {
|
|
||||||
fn chain_info(&self) -> reth_interfaces::Result<ChainInfo> {
|
|
||||||
todo!()
|
|
||||||
}
|
|
||||||
|
|
||||||
fn block(&self, id: BlockId) -> reth_interfaces::Result<Option<Block>> {
|
|
||||||
let lock = self.blocks.lock();
|
|
||||||
match id {
|
|
||||||
BlockId::Hash(hash) => Ok(lock.get(&hash).cloned()),
|
|
||||||
BlockId::Number(BlockNumber::Number(num)) => {
|
|
||||||
Ok(lock.values().find(|b| b.number == num.as_u64()).cloned())
|
|
||||||
}
|
|
||||||
_ => {
|
|
||||||
unreachable!("unused in network tests")
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
fn block_number(
|
|
||||||
&self,
|
|
||||||
hash: H256,
|
|
||||||
) -> reth_interfaces::Result<Option<reth_primitives::BlockNumber>> {
|
|
||||||
let lock = self.blocks.lock();
|
|
||||||
let num = lock.iter().find_map(|(h, b)| if *h == hash { Some(b.number) } else { None });
|
|
||||||
Ok(num)
|
|
||||||
}
|
|
||||||
|
|
||||||
fn block_hash(&self, number: U256) -> reth_interfaces::Result<Option<H256>> {
|
|
||||||
let lock = self.blocks.lock();
|
|
||||||
|
|
||||||
let hash =
|
|
||||||
lock.iter().find_map(
|
|
||||||
|(hash, b)| {
|
|
||||||
if b.number == number.as_u64() {
|
|
||||||
Some(*hash)
|
|
||||||
} else {
|
|
||||||
None
|
|
||||||
}
|
|
||||||
},
|
|
||||||
);
|
|
||||||
Ok(hash)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|||||||
@ -37,12 +37,26 @@ impl SealedBlock {
|
|||||||
pub fn hash(&self) -> H256 {
|
pub fn hash(&self) -> H256 {
|
||||||
self.header.hash()
|
self.header.hash()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Splits the sealed block into underlying components
|
||||||
|
pub fn split(self) -> (SealedHeader, Vec<TransactionSigned>, Vec<SealedHeader>) {
|
||||||
|
(self.header, self.body, self.ommers)
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Unseal the block
|
||||||
|
pub fn unseal(self) -> Block {
|
||||||
|
Block {
|
||||||
|
header: self.header.unseal(),
|
||||||
|
body: self.body,
|
||||||
|
ommers: self.ommers.into_iter().map(|o| o.unseal()).collect(),
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
impl Deref for SealedBlock {
|
impl Deref for SealedBlock {
|
||||||
type Target = Header;
|
type Target = SealedHeader;
|
||||||
fn deref(&self) -> &Self::Target {
|
fn deref(&self) -> &Self::Target {
|
||||||
self.header.as_ref()
|
&self.header
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@ -45,7 +45,7 @@ reth-db = { path = ".", features = ["test-utils","bench"]}
|
|||||||
# needed for test-fuzz to work properly, see https://github.com/paradigmxyz/reth/pull/177#discussion_r1021172198
|
# needed for test-fuzz to work properly, see https://github.com/paradigmxyz/reth/pull/177#discussion_r1021172198
|
||||||
secp256k1 = "0.24.2"
|
secp256k1 = "0.24.2"
|
||||||
|
|
||||||
reth-interfaces = { path = "../../interfaces",features=["bench"] }
|
reth-interfaces = { path = "../../interfaces", features=["bench"] }
|
||||||
async-trait = "0.1.58"
|
async-trait = "0.1.58"
|
||||||
|
|
||||||
[features]
|
[features]
|
||||||
|
|||||||
@ -30,6 +30,7 @@ rand = "0.8.5"
|
|||||||
arbitrary = { version = "1.1.7", features = ["derive"], optional = true }
|
arbitrary = { version = "1.1.7", features = ["derive"], optional = true }
|
||||||
secp256k1 = { version = "0.24.2", default-features = false, features = ["alloc", "recovery", "rand"], optional = true }
|
secp256k1 = { version = "0.24.2", default-features = false, features = ["alloc", "recovery", "rand"], optional = true }
|
||||||
modular-bitfield = "0.11.2"
|
modular-bitfield = "0.11.2"
|
||||||
|
parking_lot = { version = "0.12", optional = true }
|
||||||
|
|
||||||
[dev-dependencies]
|
[dev-dependencies]
|
||||||
reth-db = { path = "../db", features = ["test-utils"] }
|
reth-db = { path = "../db", features = ["test-utils"] }
|
||||||
@ -39,7 +40,8 @@ tokio-stream = { version = "0.1.11", features = ["sync"] }
|
|||||||
arbitrary = { version = "1.1.7", features = ["derive"]}
|
arbitrary = { version = "1.1.7", features = ["derive"]}
|
||||||
hex-literal = "0.3"
|
hex-literal = "0.3"
|
||||||
secp256k1 = { version = "0.24.2", default-features = false, features = ["alloc", "recovery", "rand"] }
|
secp256k1 = { version = "0.24.2", default-features = false, features = ["alloc", "recovery", "rand"] }
|
||||||
|
parking_lot = "0.12"
|
||||||
|
|
||||||
[features]
|
[features]
|
||||||
bench = []
|
bench = []
|
||||||
test-utils = ["tokio-stream/sync", "secp256k1"]
|
test-utils = ["tokio-stream/sync", "secp256k1", "parking_lot"]
|
||||||
|
|||||||
@ -10,6 +10,14 @@ use reth_primitives::{
|
|||||||
Block, BlockHash, BlockHashOrNumber, Header, SealedBlock, H256, U256,
|
Block, BlockHash, BlockHashOrNumber, Header, SealedBlock, H256, U256,
|
||||||
};
|
};
|
||||||
|
|
||||||
|
/// Client trait for fetching block hashes by number.
|
||||||
|
#[auto_impl(&)]
|
||||||
|
pub trait BlockHashProvider: Send + Sync {
|
||||||
|
/// Get the hash of the block with the given number. Returns `None` if no block with this number
|
||||||
|
/// exists.
|
||||||
|
fn block_hash(&self, number: U256) -> Result<Option<H256>>;
|
||||||
|
}
|
||||||
|
|
||||||
/// Client trait for fetching `Header` related data.
|
/// Client trait for fetching `Header` related data.
|
||||||
#[auto_impl(&)]
|
#[auto_impl(&)]
|
||||||
pub trait HeaderProvider: Send + Sync {
|
pub trait HeaderProvider: Send + Sync {
|
||||||
@ -37,7 +45,7 @@ pub trait HeaderProvider: Send + Sync {
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// Api trait for fetching `Block` related data.
|
/// Api trait for fetching `Block` related data.
|
||||||
pub trait BlockProvider: Send + Sync {
|
pub trait BlockProvider: BlockHashProvider + Send + Sync {
|
||||||
/// Returns the current info for the chain.
|
/// Returns the current info for the chain.
|
||||||
fn chain_info(&self) -> Result<ChainInfo>;
|
fn chain_info(&self) -> Result<ChainInfo>;
|
||||||
|
|
||||||
@ -89,10 +97,6 @@ pub trait BlockProvider: Send + Sync {
|
|||||||
|
|
||||||
/// Gets the `Block` for the given hash. Returns `None` if no block with this hash exists.
|
/// Gets the `Block` for the given hash. Returns `None` if no block with this hash exists.
|
||||||
fn block_number(&self, hash: H256) -> Result<Option<reth_primitives::BlockNumber>>;
|
fn block_number(&self, hash: H256) -> Result<Option<reth_primitives::BlockNumber>>;
|
||||||
|
|
||||||
/// Get the hash of the block with the given number. Returns `None` if no block with this number
|
|
||||||
/// exists.
|
|
||||||
fn block_hash(&self, number: U256) -> Result<Option<H256>>;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Current status of the blockchain's head.
|
/// Current status of the blockchain's head.
|
||||||
|
|||||||
@ -1,4 +1,4 @@
|
|||||||
use crate::{BlockProvider, ChainInfo, HeaderProvider, ProviderImpl};
|
use crate::{block::BlockHashProvider, BlockProvider, ChainInfo, HeaderProvider, ProviderImpl};
|
||||||
use reth_db::{database::Database, tables, transaction::DbTx};
|
use reth_db::{database::Database, tables, transaction::DbTx};
|
||||||
use reth_interfaces::Result;
|
use reth_interfaces::Result;
|
||||||
use reth_primitives::{rpc::BlockId, Block, BlockHash, BlockNumber, Header, H256, U256};
|
use reth_primitives::{rpc::BlockId, Block, BlockHash, BlockNumber, Header, H256, U256};
|
||||||
@ -26,6 +26,15 @@ impl<DB: Database> HeaderProvider for ProviderImpl<DB> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
impl<DB: Database> BlockHashProvider for ProviderImpl<DB> {
|
||||||
|
fn block_hash(&self, number: U256) -> Result<Option<H256>> {
|
||||||
|
// TODO: This unwrap is potentially unsafe
|
||||||
|
self.db
|
||||||
|
.view(|tx| tx.get::<tables::CanonicalHeaders>(number.try_into().unwrap()))?
|
||||||
|
.map_err(Into::into)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
impl<DB: Database> BlockProvider for ProviderImpl<DB> {
|
impl<DB: Database> BlockProvider for ProviderImpl<DB> {
|
||||||
fn chain_info(&self) -> Result<ChainInfo> {
|
fn chain_info(&self) -> Result<ChainInfo> {
|
||||||
Ok(ChainInfo {
|
Ok(ChainInfo {
|
||||||
@ -44,11 +53,4 @@ impl<DB: Database> BlockProvider for ProviderImpl<DB> {
|
|||||||
fn block_number(&self, hash: H256) -> Result<Option<BlockNumber>> {
|
fn block_number(&self, hash: H256) -> Result<Option<BlockNumber>> {
|
||||||
self.db.view(|tx| tx.get::<tables::HeaderNumbers>(hash))?.map_err(Into::into)
|
self.db.view(|tx| tx.get::<tables::HeaderNumbers>(hash))?.map_err(Into::into)
|
||||||
}
|
}
|
||||||
|
|
||||||
fn block_hash(&self, number: U256) -> Result<Option<H256>> {
|
|
||||||
// TODO: This unwrap is potentially unsafe
|
|
||||||
self.db
|
|
||||||
.view(|tx| tx.get::<tables::CanonicalHeaders>(number.try_into().unwrap()))?
|
|
||||||
.map_err(Into::into)
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
@ -1,5 +1,7 @@
|
|||||||
use super::ProviderImpl;
|
use super::ProviderImpl;
|
||||||
use crate::{AccountProvider, Error, StateProvider, StateProviderFactory};
|
use crate::{
|
||||||
|
block::BlockHashProvider, AccountProvider, Error, StateProvider, StateProviderFactory,
|
||||||
|
};
|
||||||
use reth_db::{
|
use reth_db::{
|
||||||
cursor::{DbCursorRO, DbDupCursorRO},
|
cursor::{DbCursorRO, DbDupCursorRO},
|
||||||
database::{Database, DatabaseGAT},
|
database::{Database, DatabaseGAT},
|
||||||
@ -78,6 +80,12 @@ impl<'a, TX: DbTx<'a>> AccountProvider for StateProviderImplHistory<'a, TX> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
impl<'a, TX: DbTx<'a>> BlockHashProvider for StateProviderImplHistory<'a, TX> {
|
||||||
|
fn block_hash(&self, number: U256) -> Result<Option<H256>> {
|
||||||
|
StateProviderImplRefHistory::new(&self.tx, self.transition).block_hash(number)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
impl<'a, TX: DbTx<'a>> StateProvider for StateProviderImplHistory<'a, TX> {
|
impl<'a, TX: DbTx<'a>> StateProvider for StateProviderImplHistory<'a, TX> {
|
||||||
fn storage(&self, account: Address, storage_key: StorageKey) -> Result<Option<StorageValue>> {
|
fn storage(&self, account: Address, storage_key: StorageKey) -> Result<Option<StorageValue>> {
|
||||||
StateProviderImplRefHistory::new(&self.tx, self.transition).storage(account, storage_key)
|
StateProviderImplRefHistory::new(&self.tx, self.transition).storage(account, storage_key)
|
||||||
@ -86,10 +94,6 @@ impl<'a, TX: DbTx<'a>> StateProvider for StateProviderImplHistory<'a, TX> {
|
|||||||
fn bytecode_by_hash(&self, code_hash: H256) -> Result<Option<Bytes>> {
|
fn bytecode_by_hash(&self, code_hash: H256) -> Result<Option<Bytes>> {
|
||||||
StateProviderImplRefHistory::new(&self.tx, self.transition).bytecode_by_hash(code_hash)
|
StateProviderImplRefHistory::new(&self.tx, self.transition).bytecode_by_hash(code_hash)
|
||||||
}
|
}
|
||||||
|
|
||||||
fn block_hash(&self, number: U256) -> Result<Option<H256>> {
|
|
||||||
StateProviderImplRefHistory::new(&self.tx, self.transition).block_hash(number)
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
/// State provider with given hash
|
/// State provider with given hash
|
||||||
///
|
///
|
||||||
@ -123,6 +127,13 @@ impl<'a, 'b, TX: DbTx<'a>> AccountProvider for StateProviderImplRefHistory<'a, '
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
impl<'a, 'b, TX: DbTx<'a>> BlockHashProvider for StateProviderImplRefHistory<'a, 'b, TX> {
|
||||||
|
/// Get block hash by number.
|
||||||
|
fn block_hash(&self, number: U256) -> Result<Option<H256>> {
|
||||||
|
self.tx.get::<tables::CanonicalHeaders>(number.as_u64()).map_err(Into::into)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
impl<'a, 'b, TX: DbTx<'a>> StateProvider for StateProviderImplRefHistory<'a, 'b, TX> {
|
impl<'a, 'b, TX: DbTx<'a>> StateProvider for StateProviderImplRefHistory<'a, 'b, TX> {
|
||||||
/// Get storage.
|
/// Get storage.
|
||||||
fn storage(&self, account: Address, storage_key: StorageKey) -> Result<Option<StorageValue>> {
|
fn storage(&self, account: Address, storage_key: StorageKey) -> Result<Option<StorageValue>> {
|
||||||
@ -156,11 +167,6 @@ impl<'a, 'b, TX: DbTx<'a>> StateProvider for StateProviderImplRefHistory<'a, 'b,
|
|||||||
fn bytecode_by_hash(&self, code_hash: H256) -> Result<Option<Bytes>> {
|
fn bytecode_by_hash(&self, code_hash: H256) -> Result<Option<Bytes>> {
|
||||||
self.tx.get::<tables::Bytecodes>(code_hash).map_err(Into::into).map(|r| r.map(Bytes::from))
|
self.tx.get::<tables::Bytecodes>(code_hash).map_err(Into::into).map(|r| r.map(Bytes::from))
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Get block hash by number.
|
|
||||||
fn block_hash(&self, number: U256) -> Result<Option<H256>> {
|
|
||||||
self.tx.get::<tables::CanonicalHeaders>(number.as_u64()).map_err(Into::into)
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/// State provider for latests state
|
/// State provider for latests state
|
||||||
@ -185,6 +191,12 @@ impl<'a, TX: DbTx<'a>> AccountProvider for StateProviderImplLatest<'a, TX> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
impl<'a, TX: DbTx<'a>> BlockHashProvider for StateProviderImplLatest<'a, TX> {
|
||||||
|
fn block_hash(&self, number: U256) -> Result<Option<H256>> {
|
||||||
|
StateProviderImplRefLatest::new(&self.db).block_hash(number)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
impl<'a, TX: DbTx<'a>> StateProvider for StateProviderImplLatest<'a, TX> {
|
impl<'a, TX: DbTx<'a>> StateProvider for StateProviderImplLatest<'a, TX> {
|
||||||
fn storage(&self, account: Address, storage_key: StorageKey) -> Result<Option<StorageValue>> {
|
fn storage(&self, account: Address, storage_key: StorageKey) -> Result<Option<StorageValue>> {
|
||||||
StateProviderImplRefLatest::new(&self.db).storage(account, storage_key)
|
StateProviderImplRefLatest::new(&self.db).storage(account, storage_key)
|
||||||
@ -193,10 +205,6 @@ impl<'a, TX: DbTx<'a>> StateProvider for StateProviderImplLatest<'a, TX> {
|
|||||||
fn bytecode_by_hash(&self, code_hash: H256) -> Result<Option<Bytes>> {
|
fn bytecode_by_hash(&self, code_hash: H256) -> Result<Option<Bytes>> {
|
||||||
StateProviderImplRefLatest::new(&self.db).bytecode_by_hash(code_hash)
|
StateProviderImplRefLatest::new(&self.db).bytecode_by_hash(code_hash)
|
||||||
}
|
}
|
||||||
|
|
||||||
fn block_hash(&self, number: U256) -> Result<Option<H256>> {
|
|
||||||
StateProviderImplRefLatest::new(&self.db).block_hash(number)
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/// State Provider over latest state that takes tx reference
|
/// State Provider over latest state that takes tx reference
|
||||||
@ -221,6 +229,13 @@ impl<'a, 'b, TX: DbTx<'a>> AccountProvider for StateProviderImplRefLatest<'a, 'b
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
impl<'a, 'b, TX: DbTx<'a>> BlockHashProvider for StateProviderImplRefLatest<'a, 'b, TX> {
|
||||||
|
/// Get block hash by number.
|
||||||
|
fn block_hash(&self, number: U256) -> Result<Option<H256>> {
|
||||||
|
self.db.get::<tables::CanonicalHeaders>(number.as_u64()).map_err(Into::into)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
impl<'a, 'b, TX: DbTx<'a>> StateProvider for StateProviderImplRefLatest<'a, 'b, TX> {
|
impl<'a, 'b, TX: DbTx<'a>> StateProvider for StateProviderImplRefLatest<'a, 'b, TX> {
|
||||||
/// Get storage.
|
/// Get storage.
|
||||||
fn storage(&self, account: Address, storage_key: StorageKey) -> Result<Option<StorageValue>> {
|
fn storage(&self, account: Address, storage_key: StorageKey) -> Result<Option<StorageValue>> {
|
||||||
@ -237,9 +252,4 @@ impl<'a, 'b, TX: DbTx<'a>> StateProvider for StateProviderImplRefLatest<'a, 'b,
|
|||||||
fn bytecode_by_hash(&self, code_hash: H256) -> Result<Option<Bytes>> {
|
fn bytecode_by_hash(&self, code_hash: H256) -> Result<Option<Bytes>> {
|
||||||
self.db.get::<tables::Bytecodes>(code_hash).map_err(Into::into).map(|r| r.map(Bytes::from))
|
self.db.get::<tables::Bytecodes>(code_hash).map_err(Into::into).map(|r| r.map(Bytes::from))
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Get block hash by number.
|
|
||||||
fn block_hash(&self, number: U256) -> Result<Option<H256>> {
|
|
||||||
self.db.get::<tables::CanonicalHeaders>(number.as_u64()).map_err(Into::into)
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
@ -16,7 +16,9 @@ mod state;
|
|||||||
/// Common test helpers for mocking the Provider.
|
/// Common test helpers for mocking the Provider.
|
||||||
pub mod test_utils;
|
pub mod test_utils;
|
||||||
|
|
||||||
pub use block::{insert_canonical_block, BlockProvider, ChainInfo, HeaderProvider};
|
pub use block::{
|
||||||
|
insert_canonical_block, BlockHashProvider, BlockProvider, ChainInfo, HeaderProvider,
|
||||||
|
};
|
||||||
pub use db_provider::{
|
pub use db_provider::{
|
||||||
self as db, ProviderImpl, StateProviderImplHistory, StateProviderImplLatest,
|
self as db, ProviderImpl, StateProviderImplHistory, StateProviderImplLatest,
|
||||||
StateProviderImplRefHistory, StateProviderImplRefLatest,
|
StateProviderImplRefHistory, StateProviderImplRefLatest,
|
||||||
|
|||||||
@ -1,24 +1,26 @@
|
|||||||
|
use auto_impl::auto_impl;
|
||||||
use reth_interfaces::Result;
|
use reth_interfaces::Result;
|
||||||
use reth_primitives::{
|
use reth_primitives::{
|
||||||
Account, Address, BlockHash, BlockNumber, Bytes, StorageKey, StorageValue, H256, U256,
|
Account, Address, BlockHash, BlockNumber, Bytes, StorageKey, StorageValue, H256,
|
||||||
};
|
};
|
||||||
|
|
||||||
|
use crate::block::BlockHashProvider;
|
||||||
|
|
||||||
/// Account provider
|
/// Account provider
|
||||||
|
#[auto_impl(&)]
|
||||||
pub trait AccountProvider: Send + Sync {
|
pub trait AccountProvider: Send + Sync {
|
||||||
/// Get basic account information.
|
/// Get basic account information.
|
||||||
fn basic_account(&self, address: Address) -> Result<Option<Account>>;
|
fn basic_account(&self, address: Address) -> Result<Option<Account>>;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Function needed for executor.
|
/// Function needed for executor.
|
||||||
pub trait StateProvider: AccountProvider + Send + Sync {
|
#[auto_impl(&)]
|
||||||
|
pub trait StateProvider: BlockHashProvider + AccountProvider + Send + Sync {
|
||||||
/// Get storage.
|
/// Get storage.
|
||||||
fn storage(&self, account: Address, storage_key: StorageKey) -> Result<Option<StorageValue>>;
|
fn storage(&self, account: Address, storage_key: StorageKey) -> Result<Option<StorageValue>>;
|
||||||
|
|
||||||
/// Get account code by its hash
|
/// Get account code by its hash
|
||||||
fn bytecode_by_hash(&self, code_hash: H256) -> Result<Option<Bytes>>;
|
fn bytecode_by_hash(&self, code_hash: H256) -> Result<Option<Bytes>>;
|
||||||
|
|
||||||
/// Get block hash by number.
|
|
||||||
fn block_hash(&self, number: U256) -> Result<Option<H256>>;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Light wrapper that creates StateProvider.
|
/// Light wrapper that creates StateProvider.
|
||||||
|
|||||||
177
crates/storage/provider/src/test_utils/mock.rs
Normal file
177
crates/storage/provider/src/test_utils/mock.rs
Normal file
@ -0,0 +1,177 @@
|
|||||||
|
use parking_lot::Mutex;
|
||||||
|
use reth_interfaces::Result;
|
||||||
|
use reth_primitives::{
|
||||||
|
keccak256,
|
||||||
|
rpc::{BlockId, BlockNumber},
|
||||||
|
Account, Address, Block, BlockHash, Bytes, Header, StorageKey, StorageValue, H256, U256,
|
||||||
|
};
|
||||||
|
use std::{collections::HashMap, sync::Arc};
|
||||||
|
|
||||||
|
use crate::{
|
||||||
|
AccountProvider, BlockHashProvider, BlockProvider, ChainInfo, HeaderProvider, StateProvider,
|
||||||
|
};
|
||||||
|
|
||||||
|
/// A mock implementation for Provider interfaces.
|
||||||
|
#[derive(Debug, Clone, Default)]
|
||||||
|
pub struct MockEthProvider {
|
||||||
|
/// Local block store
|
||||||
|
pub blocks: Arc<Mutex<HashMap<H256, Block>>>,
|
||||||
|
/// Local header store
|
||||||
|
pub headers: Arc<Mutex<HashMap<H256, Header>>>,
|
||||||
|
/// Local account store
|
||||||
|
pub accounts: Arc<Mutex<HashMap<Address, ExtendedAccount>>>,
|
||||||
|
}
|
||||||
|
|
||||||
|
/// An extended account for local store
|
||||||
|
#[derive(Debug, Clone)]
|
||||||
|
pub struct ExtendedAccount {
|
||||||
|
account: Account,
|
||||||
|
bytecode: Option<Bytes>,
|
||||||
|
storage: HashMap<StorageKey, StorageValue>,
|
||||||
|
}
|
||||||
|
|
||||||
|
impl ExtendedAccount {
|
||||||
|
/// Create new instance of extended account
|
||||||
|
pub fn new(nonce: u64, balance: U256) -> Self {
|
||||||
|
Self {
|
||||||
|
account: Account { nonce, balance, bytecode_hash: None },
|
||||||
|
bytecode: None,
|
||||||
|
storage: Default::default(),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Set bytecode and bytecode hash on the extended account
|
||||||
|
pub fn with_bytecode(mut self, bytecode: Bytes) -> Self {
|
||||||
|
let hash = keccak256(&bytecode);
|
||||||
|
self.account.bytecode_hash = Some(hash);
|
||||||
|
self.bytecode = Some(bytecode);
|
||||||
|
self
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
impl MockEthProvider {
|
||||||
|
/// Add block to local block store
|
||||||
|
pub fn add_block(&self, hash: H256, block: Block) {
|
||||||
|
self.blocks.lock().insert(hash, block);
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Add multiple blocks to local block store
|
||||||
|
pub fn extend_blocks(&self, iter: impl IntoIterator<Item = (H256, Block)>) {
|
||||||
|
for (hash, block) in iter.into_iter() {
|
||||||
|
self.add_block(hash, block)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Add header to local header store
|
||||||
|
pub fn add_header(&self, hash: H256, header: Header) {
|
||||||
|
self.headers.lock().insert(hash, header);
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Add multiple headers to local header store
|
||||||
|
pub fn extend_headers(&self, iter: impl IntoIterator<Item = (H256, Header)>) {
|
||||||
|
for (hash, header) in iter.into_iter() {
|
||||||
|
self.add_header(hash, header)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Add account to local account store
|
||||||
|
pub fn add_account(&self, address: Address, account: ExtendedAccount) {
|
||||||
|
self.accounts.lock().insert(address, account);
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Add account to local account store
|
||||||
|
pub fn extend_accounts(&self, iter: impl IntoIterator<Item = (Address, ExtendedAccount)>) {
|
||||||
|
for (address, account) in iter.into_iter() {
|
||||||
|
self.add_account(address, account)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
impl HeaderProvider for MockEthProvider {
|
||||||
|
fn header(&self, block_hash: &BlockHash) -> Result<Option<Header>> {
|
||||||
|
let lock = self.headers.lock();
|
||||||
|
Ok(lock.get(block_hash).cloned())
|
||||||
|
}
|
||||||
|
|
||||||
|
fn header_by_number(&self, num: u64) -> Result<Option<Header>> {
|
||||||
|
let lock = self.headers.lock();
|
||||||
|
Ok(lock.values().find(|h| h.number == num).cloned())
|
||||||
|
}
|
||||||
|
|
||||||
|
fn header_td(&self, hash: &BlockHash) -> Result<Option<U256>> {
|
||||||
|
let lock = self.headers.lock();
|
||||||
|
Ok(lock.get(hash).map(|target| {
|
||||||
|
lock.values()
|
||||||
|
.filter(|h| h.number < target.number)
|
||||||
|
.fold(target.difficulty, |td, h| td + h.difficulty)
|
||||||
|
}))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
impl BlockHashProvider for MockEthProvider {
|
||||||
|
fn block_hash(&self, number: U256) -> Result<Option<H256>> {
|
||||||
|
let lock = self.blocks.lock();
|
||||||
|
|
||||||
|
let hash =
|
||||||
|
lock.iter().find_map(
|
||||||
|
|(hash, b)| {
|
||||||
|
if b.number == number.as_u64() {
|
||||||
|
Some(*hash)
|
||||||
|
} else {
|
||||||
|
None
|
||||||
|
}
|
||||||
|
},
|
||||||
|
);
|
||||||
|
Ok(hash)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
impl BlockProvider for MockEthProvider {
|
||||||
|
fn chain_info(&self) -> Result<ChainInfo> {
|
||||||
|
todo!()
|
||||||
|
}
|
||||||
|
|
||||||
|
fn block(&self, id: BlockId) -> Result<Option<Block>> {
|
||||||
|
let lock = self.blocks.lock();
|
||||||
|
match id {
|
||||||
|
BlockId::Hash(hash) => Ok(lock.get(&hash).cloned()),
|
||||||
|
BlockId::Number(BlockNumber::Number(num)) => {
|
||||||
|
Ok(lock.values().find(|b| b.number == num.as_u64()).cloned())
|
||||||
|
}
|
||||||
|
_ => {
|
||||||
|
unreachable!("unused in network tests")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
fn block_number(&self, hash: H256) -> Result<Option<reth_primitives::BlockNumber>> {
|
||||||
|
let lock = self.blocks.lock();
|
||||||
|
let num = lock.iter().find_map(|(h, b)| if *h == hash { Some(b.number) } else { None });
|
||||||
|
Ok(num)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
impl AccountProvider for MockEthProvider {
|
||||||
|
fn basic_account(&self, address: Address) -> Result<Option<Account>> {
|
||||||
|
Ok(self.accounts.lock().get(&address).cloned().map(|a| a.account))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
impl StateProvider for MockEthProvider {
|
||||||
|
fn bytecode_by_hash(&self, code_hash: H256) -> Result<Option<Bytes>> {
|
||||||
|
let lock = self.accounts.lock();
|
||||||
|
Ok(lock.values().find_map(|account| {
|
||||||
|
match (account.account.bytecode_hash.as_ref(), account.bytecode.as_ref()) {
|
||||||
|
(Some(bytecode_hash), Some(bytecode)) if *bytecode_hash == code_hash => {
|
||||||
|
Some(bytecode.clone())
|
||||||
|
}
|
||||||
|
_ => None,
|
||||||
|
}
|
||||||
|
}))
|
||||||
|
}
|
||||||
|
|
||||||
|
fn storage(&self, account: Address, storage_key: StorageKey) -> Result<Option<StorageValue>> {
|
||||||
|
let lock = self.accounts.lock();
|
||||||
|
Ok(lock.get(&account).and_then(|account| account.storage.get(&storage_key)).cloned())
|
||||||
|
}
|
||||||
|
}
|
||||||
@ -1,3 +1,5 @@
|
|||||||
mod api;
|
mod mock;
|
||||||
|
mod noop;
|
||||||
|
|
||||||
pub use api::TestApi;
|
pub use mock::{ExtendedAccount, MockEthProvider};
|
||||||
|
pub use noop::NoopProvider;
|
||||||
|
|||||||
@ -1,14 +1,20 @@
|
|||||||
use crate::{BlockProvider, ChainInfo, HeaderProvider};
|
use crate::{BlockHashProvider, BlockProvider, ChainInfo, HeaderProvider};
|
||||||
use reth_interfaces::Result;
|
use reth_interfaces::Result;
|
||||||
use reth_primitives::{rpc::BlockId, Block, BlockHash, BlockNumber, Header, H256, U256};
|
use reth_primitives::{rpc::BlockId, Block, BlockHash, BlockNumber, Header, H256, U256};
|
||||||
|
|
||||||
/// Supports various api interfaces for testing purposes.
|
/// Supports various api interfaces for testing purposes.
|
||||||
#[derive(Debug, Clone, Default)]
|
#[derive(Debug, Clone, Default)]
|
||||||
#[non_exhaustive]
|
#[non_exhaustive]
|
||||||
pub struct TestApi;
|
pub struct NoopProvider;
|
||||||
|
|
||||||
/// Noop implementation for testing purposes
|
/// Noop implementation for testing purposes
|
||||||
impl BlockProvider for TestApi {
|
impl BlockHashProvider for NoopProvider {
|
||||||
|
fn block_hash(&self, _number: U256) -> Result<Option<H256>> {
|
||||||
|
Ok(None)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
impl BlockProvider for NoopProvider {
|
||||||
fn chain_info(&self) -> Result<ChainInfo> {
|
fn chain_info(&self) -> Result<ChainInfo> {
|
||||||
Ok(ChainInfo {
|
Ok(ChainInfo {
|
||||||
best_hash: Default::default(),
|
best_hash: Default::default(),
|
||||||
@ -25,13 +31,9 @@ impl BlockProvider for TestApi {
|
|||||||
fn block_number(&self, _hash: H256) -> Result<Option<BlockNumber>> {
|
fn block_number(&self, _hash: H256) -> Result<Option<BlockNumber>> {
|
||||||
Ok(None)
|
Ok(None)
|
||||||
}
|
}
|
||||||
|
|
||||||
fn block_hash(&self, _number: U256) -> Result<Option<H256>> {
|
|
||||||
Ok(None)
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
impl HeaderProvider for TestApi {
|
impl HeaderProvider for NoopProvider {
|
||||||
fn header(&self, _block_hash: &BlockHash) -> Result<Option<Header>> {
|
fn header(&self, _block_hash: &BlockHash) -> Result<Option<Header>> {
|
||||||
Ok(None)
|
Ok(None)
|
||||||
}
|
}
|
||||||
Reference in New Issue
Block a user