mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
refactor(interfaces): Result -> RethResult, Error -> RethError (#4695)
This commit is contained in:
@ -5,6 +5,7 @@ use reth_db::{
|
||||
tables,
|
||||
transaction::{DbTx, DbTxMut},
|
||||
};
|
||||
use reth_interfaces::{db::DatabaseError, RethError};
|
||||
use reth_primitives::{stage::StageId, Account, Bytecode, ChainSpec, StorageEntry, H256, U256};
|
||||
use reth_provider::{
|
||||
bundle_state::{BundleStateInit, RevertsInit},
|
||||
@ -32,11 +33,11 @@ pub enum InitDatabaseError {
|
||||
|
||||
/// Low-level database error.
|
||||
#[error(transparent)]
|
||||
DBError(#[from] reth_db::DatabaseError),
|
||||
DBError(#[from] DatabaseError),
|
||||
|
||||
/// Internal error.
|
||||
#[error(transparent)]
|
||||
InternalError(#[from] reth_interfaces::Error),
|
||||
InternalError(#[from] RethError),
|
||||
}
|
||||
|
||||
/// Write the genesis block if it has not already been written
|
||||
|
||||
@ -46,6 +46,7 @@ use reth_interfaces::{
|
||||
either::EitherDownloader,
|
||||
headers::{client::HeadersClient, downloader::HeaderDownloader},
|
||||
},
|
||||
RethResult,
|
||||
};
|
||||
use reth_network::{error::NetworkError, NetworkConfig, NetworkHandle, NetworkManager};
|
||||
use reth_network_api::NetworkInfo;
|
||||
@ -664,7 +665,7 @@ impl<Ext: RethCliExt> NodeCommand<Ext> {
|
||||
Ok(handle)
|
||||
}
|
||||
|
||||
fn lookup_head(&self, db: Arc<DatabaseEnv>) -> Result<Head, reth_interfaces::Error> {
|
||||
fn lookup_head(&self, db: Arc<DatabaseEnv>) -> RethResult<Head> {
|
||||
let factory = ProviderFactory::new(db, self.chain.clone());
|
||||
let provider = factory.provider()?;
|
||||
|
||||
@ -700,7 +701,7 @@ impl<Ext: RethCliExt> NodeCommand<Ext> {
|
||||
db: DB,
|
||||
client: Client,
|
||||
tip: H256,
|
||||
) -> Result<u64, reth_interfaces::Error>
|
||||
) -> RethResult<u64>
|
||||
where
|
||||
DB: Database,
|
||||
Client: HeadersClient,
|
||||
@ -716,7 +717,7 @@ impl<Ext: RethCliExt> NodeCommand<Ext> {
|
||||
db: DB,
|
||||
client: Client,
|
||||
tip: BlockHashOrNumber,
|
||||
) -> Result<SealedHeader, reth_interfaces::Error>
|
||||
) -> RethResult<SealedHeader>
|
||||
where
|
||||
DB: Database,
|
||||
Client: HeadersClient,
|
||||
|
||||
@ -14,7 +14,7 @@ use reth_interfaces::{
|
||||
},
|
||||
consensus::{Consensus, ConsensusError},
|
||||
executor::{BlockExecutionError, BlockValidationError},
|
||||
Error,
|
||||
RethResult,
|
||||
};
|
||||
use reth_primitives::{
|
||||
BlockHash, BlockNumHash, BlockNumber, ForkBlock, Hardfork, PruneModes, Receipt, SealedBlock,
|
||||
@ -112,7 +112,7 @@ impl<DB: Database, C: Consensus, EF: ExecutorFactory> BlockchainTree<DB, C, EF>
|
||||
canon_state_notification_sender: CanonStateNotificationSender,
|
||||
config: BlockchainTreeConfig,
|
||||
prune_modes: Option<PruneModes>,
|
||||
) -> Result<Self, Error> {
|
||||
) -> RethResult<Self> {
|
||||
let max_reorg_depth = config.max_reorg_depth();
|
||||
|
||||
let last_canonical_hashes = externals
|
||||
@ -757,7 +757,7 @@ impl<DB: Database, C: Consensus, EF: ExecutorFactory> BlockchainTree<DB, C, EF>
|
||||
pub fn connect_buffered_blocks_to_canonical_hashes_and_finalize(
|
||||
&mut self,
|
||||
last_finalized_block: BlockNumber,
|
||||
) -> Result<(), Error> {
|
||||
) -> RethResult<()> {
|
||||
self.finalize_block(last_finalized_block);
|
||||
|
||||
let num_of_canonical_hashes =
|
||||
@ -792,7 +792,7 @@ impl<DB: Database, C: Consensus, EF: ExecutorFactory> BlockchainTree<DB, C, EF>
|
||||
///
|
||||
/// `N` is the `max_reorg_depth` plus the number of block hashes needed to satisfy the
|
||||
/// `BLOCKHASH` opcode in the EVM.
|
||||
pub fn connect_buffered_blocks_to_canonical_hashes(&mut self) -> Result<(), Error> {
|
||||
pub fn connect_buffered_blocks_to_canonical_hashes(&mut self) -> RethResult<()> {
|
||||
let num_of_canonical_hashes =
|
||||
self.config.max_reorg_depth() + self.config.num_of_additional_canonical_block_hashes();
|
||||
|
||||
@ -813,7 +813,7 @@ impl<DB: Database, C: Consensus, EF: ExecutorFactory> BlockchainTree<DB, C, EF>
|
||||
fn connect_buffered_blocks_to_hashes(
|
||||
&mut self,
|
||||
hashes: impl IntoIterator<Item = impl Into<BlockNumHash>>,
|
||||
) -> Result<(), Error> {
|
||||
) -> RethResult<()> {
|
||||
// check unconnected block buffer for childs of the canonical hashes
|
||||
for added_block in hashes.into_iter() {
|
||||
self.try_connect_buffered_blocks(added_block.into())
|
||||
@ -888,7 +888,7 @@ impl<DB: Database, C: Consensus, EF: ExecutorFactory> BlockchainTree<DB, C, EF>
|
||||
///
|
||||
/// Returns `Ok(None)` if the block hash is not canonical (block hash does not exist, or is
|
||||
/// included in a sidechain).
|
||||
pub fn find_canonical_header(&self, hash: &BlockHash) -> Result<Option<SealedHeader>, Error> {
|
||||
pub fn find_canonical_header(&self, hash: &BlockHash) -> RethResult<Option<SealedHeader>> {
|
||||
// if the indices show that the block hash is not canonical, it's either in a sidechain or
|
||||
// canonical, but in the db. If it is in a sidechain, it is not canonical. If it is not in
|
||||
// the db, then it is not canonical.
|
||||
@ -913,7 +913,7 @@ impl<DB: Database, C: Consensus, EF: ExecutorFactory> BlockchainTree<DB, C, EF>
|
||||
}
|
||||
|
||||
/// Determines whether or not a block is canonical, checking the db if necessary.
|
||||
pub fn is_block_hash_canonical(&self, hash: &BlockHash) -> Result<bool, Error> {
|
||||
pub fn is_block_hash_canonical(&self, hash: &BlockHash) -> RethResult<bool> {
|
||||
self.find_canonical_header(hash).map(|header| header.is_some())
|
||||
}
|
||||
|
||||
@ -929,7 +929,7 @@ impl<DB: Database, C: Consensus, EF: ExecutorFactory> BlockchainTree<DB, C, EF>
|
||||
/// Returns `Ok` if the blocks were canonicalized, or if the blocks were already canonical.
|
||||
#[track_caller]
|
||||
#[instrument(skip(self), target = "blockchain_tree")]
|
||||
pub fn make_canonical(&mut self, block_hash: &BlockHash) -> Result<CanonicalOutcome, Error> {
|
||||
pub fn make_canonical(&mut self, block_hash: &BlockHash) -> RethResult<CanonicalOutcome> {
|
||||
let old_block_indices = self.block_indices.clone();
|
||||
let old_buffered_blocks = self.buffered_blocks.parent_to_child.clone();
|
||||
|
||||
@ -1080,7 +1080,7 @@ impl<DB: Database, C: Consensus, EF: ExecutorFactory> BlockchainTree<DB, C, EF>
|
||||
}
|
||||
|
||||
/// Canonicalize the given chain and commit it to the database.
|
||||
fn commit_canonical(&self, chain: Chain) -> Result<(), Error> {
|
||||
fn commit_canonical(&self, chain: Chain) -> RethResult<()> {
|
||||
let provider = DatabaseProvider::new_rw(
|
||||
self.externals.db.tx_mut()?,
|
||||
self.externals.chain_spec.clone(),
|
||||
@ -1102,7 +1102,7 @@ impl<DB: Database, C: Consensus, EF: ExecutorFactory> BlockchainTree<DB, C, EF>
|
||||
}
|
||||
|
||||
/// Unwind tables and put it inside state
|
||||
pub fn unwind(&mut self, unwind_to: BlockNumber) -> Result<(), Error> {
|
||||
pub fn unwind(&mut self, unwind_to: BlockNumber) -> RethResult<()> {
|
||||
// nothing to be done if unwind_to is higher then the tip
|
||||
if self.block_indices.canonical_tip().number <= unwind_to {
|
||||
return Ok(())
|
||||
@ -1123,7 +1123,7 @@ impl<DB: Database, C: Consensus, EF: ExecutorFactory> BlockchainTree<DB, C, EF>
|
||||
/// Revert canonical blocks from the database and return them.
|
||||
///
|
||||
/// The block, `revert_until`, is non-inclusive, i.e. `revert_until` stays in the database.
|
||||
fn revert_canonical(&mut self, revert_until: BlockNumber) -> Result<Option<Chain>, Error> {
|
||||
fn revert_canonical(&mut self, revert_until: BlockNumber) -> RethResult<Option<Chain>> {
|
||||
// read data that is needed for new sidechain
|
||||
|
||||
let provider = DatabaseProvider::new_rw(
|
||||
|
||||
@ -8,7 +8,7 @@ use reth_db::database::Database;
|
||||
use reth_interfaces::{
|
||||
blockchain_tree::error::{BlockchainTreeError, InsertBlockError},
|
||||
consensus::{Consensus, ConsensusError},
|
||||
Error,
|
||||
RethResult,
|
||||
};
|
||||
use reth_primitives::{
|
||||
BlockHash, BlockNumber, ForkBlock, SealedBlockWithSenders, SealedHeader, U256,
|
||||
@ -183,7 +183,7 @@ impl AppendableChain {
|
||||
post_state_data_provider: BSDP,
|
||||
externals: &TreeExternals<DB, C, EF>,
|
||||
block_kind: BlockKind,
|
||||
) -> Result<BundleStateWithReceipts, Error>
|
||||
) -> RethResult<BundleStateWithReceipts>
|
||||
where
|
||||
BSDP: BundleStateDataProvider,
|
||||
DB: Database,
|
||||
@ -230,7 +230,7 @@ impl AppendableChain {
|
||||
parent_block: &SealedHeader,
|
||||
post_state_data_provider: BSDP,
|
||||
externals: &TreeExternals<DB, C, EF>,
|
||||
) -> Result<BundleStateWithReceipts, Error>
|
||||
) -> RethResult<BundleStateWithReceipts>
|
||||
where
|
||||
BSDP: BundleStateDataProvider,
|
||||
DB: Database,
|
||||
@ -252,7 +252,7 @@ impl AppendableChain {
|
||||
parent_block: &SealedHeader,
|
||||
post_state_data_provider: BSDP,
|
||||
externals: &TreeExternals<DB, C, EF>,
|
||||
) -> Result<BundleStateWithReceipts, Error>
|
||||
) -> RethResult<BundleStateWithReceipts>
|
||||
where
|
||||
BSDP: BundleStateDataProvider,
|
||||
DB: Database,
|
||||
|
||||
@ -8,7 +8,7 @@ use reth_interfaces::{
|
||||
InsertPayloadOk,
|
||||
},
|
||||
consensus::Consensus,
|
||||
Error,
|
||||
RethResult,
|
||||
};
|
||||
use reth_primitives::{
|
||||
BlockHash, BlockNumHash, BlockNumber, Receipt, SealedBlock, SealedBlockWithSenders,
|
||||
@ -69,7 +69,7 @@ impl<DB: Database, C: Consensus, EF: ExecutorFactory> BlockchainTreeEngine
|
||||
fn connect_buffered_blocks_to_canonical_hashes_and_finalize(
|
||||
&self,
|
||||
last_finalized_block: BlockNumber,
|
||||
) -> Result<(), Error> {
|
||||
) -> RethResult<()> {
|
||||
trace!(target: "blockchain_tree", ?last_finalized_block, "Connecting buffered blocks to canonical hashes and finalizing the tree");
|
||||
let mut tree = self.tree.write();
|
||||
let res =
|
||||
@ -78,7 +78,7 @@ impl<DB: Database, C: Consensus, EF: ExecutorFactory> BlockchainTreeEngine
|
||||
res
|
||||
}
|
||||
|
||||
fn connect_buffered_blocks_to_canonical_hashes(&self) -> Result<(), Error> {
|
||||
fn connect_buffered_blocks_to_canonical_hashes(&self) -> RethResult<()> {
|
||||
trace!(target: "blockchain_tree", "Connecting buffered blocks to canonical hashes");
|
||||
let mut tree = self.tree.write();
|
||||
let res = tree.connect_buffered_blocks_to_canonical_hashes();
|
||||
@ -86,7 +86,7 @@ impl<DB: Database, C: Consensus, EF: ExecutorFactory> BlockchainTreeEngine
|
||||
res
|
||||
}
|
||||
|
||||
fn make_canonical(&self, block_hash: &BlockHash) -> Result<CanonicalOutcome, Error> {
|
||||
fn make_canonical(&self, block_hash: &BlockHash) -> RethResult<CanonicalOutcome> {
|
||||
trace!(target: "blockchain_tree", ?block_hash, "Making block canonical");
|
||||
let mut tree = self.tree.write();
|
||||
let res = tree.make_canonical(block_hash);
|
||||
@ -94,7 +94,7 @@ impl<DB: Database, C: Consensus, EF: ExecutorFactory> BlockchainTreeEngine
|
||||
res
|
||||
}
|
||||
|
||||
fn unwind(&self, unwind_to: BlockNumber) -> Result<(), Error> {
|
||||
fn unwind(&self, unwind_to: BlockNumber) -> RethResult<()> {
|
||||
trace!(target: "blockchain_tree", ?unwind_to, "Unwinding to block number");
|
||||
let mut tree = self.tree.write();
|
||||
let res = tree.unwind(unwind_to);
|
||||
@ -159,7 +159,7 @@ impl<DB: Database, C: Consensus, EF: ExecutorFactory> BlockchainTreeViewer
|
||||
self.tree.read().block_indices().canonical_tip()
|
||||
}
|
||||
|
||||
fn is_canonical(&self, hash: BlockHash) -> Result<bool, Error> {
|
||||
fn is_canonical(&self, hash: BlockHash) -> RethResult<bool> {
|
||||
trace!(target: "blockchain_tree", ?hash, "Checking if block is canonical");
|
||||
self.tree.read().is_block_hash_canonical(&hash)
|
||||
}
|
||||
|
||||
@ -1,4 +1,5 @@
|
||||
use crate::engine::hooks::EngineHookError;
|
||||
use reth_interfaces::RethError;
|
||||
use reth_rpc_types::engine::ForkchoiceUpdateError;
|
||||
use reth_stages::PipelineError;
|
||||
|
||||
@ -23,9 +24,9 @@ pub enum BeaconConsensusEngineError {
|
||||
/// Hook error.
|
||||
#[error(transparent)]
|
||||
Hook(#[from] EngineHookError),
|
||||
/// Common error. Wrapper around [reth_interfaces::Error].
|
||||
/// Common error. Wrapper around [RethError].
|
||||
#[error(transparent)]
|
||||
Common(#[from] reth_interfaces::Error),
|
||||
Common(#[from] RethError),
|
||||
}
|
||||
|
||||
// box the pipeline error as it is a large enum.
|
||||
@ -53,14 +54,14 @@ pub enum BeaconForkChoiceUpdateError {
|
||||
ForkchoiceUpdateError(#[from] ForkchoiceUpdateError),
|
||||
/// Internal errors, for example, error while reading from the database.
|
||||
#[error(transparent)]
|
||||
Internal(Box<reth_interfaces::Error>),
|
||||
Internal(Box<RethError>),
|
||||
/// Thrown when the engine task is unavailable/stopped.
|
||||
#[error("beacon consensus engine task stopped")]
|
||||
EngineUnavailable,
|
||||
}
|
||||
|
||||
impl From<reth_interfaces::Error> for BeaconForkChoiceUpdateError {
|
||||
fn from(e: reth_interfaces::Error) -> Self {
|
||||
impl From<RethError> for BeaconForkChoiceUpdateError {
|
||||
fn from(e: RethError) -> Self {
|
||||
Self::Internal(Box::new(e))
|
||||
}
|
||||
}
|
||||
|
||||
@ -5,6 +5,7 @@ use crate::{
|
||||
BeaconForkChoiceUpdateError, BeaconOnNewPayloadError,
|
||||
};
|
||||
use futures::TryFutureExt;
|
||||
use reth_interfaces::RethResult;
|
||||
use reth_rpc_types::engine::{
|
||||
CancunPayloadFields, ExecutionPayload, ForkchoiceState, ForkchoiceUpdated, PayloadAttributes,
|
||||
PayloadStatus,
|
||||
@ -63,7 +64,7 @@ impl BeaconConsensusEngineHandle {
|
||||
&self,
|
||||
state: ForkchoiceState,
|
||||
payload_attrs: Option<PayloadAttributes>,
|
||||
) -> oneshot::Receiver<Result<OnForkChoiceUpdated, reth_interfaces::Error>> {
|
||||
) -> oneshot::Receiver<RethResult<OnForkChoiceUpdated>> {
|
||||
let (tx, rx) = oneshot::channel();
|
||||
let _ = self.to_engine.send(BeaconEngineMessage::ForkchoiceUpdated {
|
||||
state,
|
||||
|
||||
@ -10,6 +10,7 @@ pub(crate) use controller::{EngineHooksController, PolledHook};
|
||||
|
||||
mod prune;
|
||||
pub use prune::PruneHook;
|
||||
use reth_interfaces::RethError;
|
||||
|
||||
/// Collection of [engine hooks][`EngineHook`].
|
||||
#[derive(Default)]
|
||||
@ -96,9 +97,9 @@ pub enum EngineHookError {
|
||||
/// Hook channel closed.
|
||||
#[error("Hook channel closed")]
|
||||
ChannelClosed,
|
||||
/// Common error. Wrapper around [reth_interfaces::Error].
|
||||
/// Common error. Wrapper around [RethError].
|
||||
#[error(transparent)]
|
||||
Common(#[from] reth_interfaces::Error),
|
||||
Common(#[from] RethError),
|
||||
/// An internal error occurred.
|
||||
#[error("Internal hook error occurred.")]
|
||||
Internal(#[from] Box<dyn std::error::Error + Send + Sync>),
|
||||
|
||||
@ -9,7 +9,7 @@ use crate::{
|
||||
use futures::FutureExt;
|
||||
use metrics::Counter;
|
||||
use reth_db::database::Database;
|
||||
use reth_interfaces::sync::SyncState;
|
||||
use reth_interfaces::{sync::SyncState, RethError};
|
||||
use reth_primitives::BlockNumber;
|
||||
use reth_prune::{Pruner, PrunerError, PrunerWithResult};
|
||||
use reth_tasks::TaskSpawner;
|
||||
@ -62,8 +62,8 @@ impl<DB: Database + 'static> PruneHook<DB> {
|
||||
EngineHookError::Internal(Box::new(err))
|
||||
}
|
||||
PrunerError::Interface(err) => err.into(),
|
||||
PrunerError::Database(err) => reth_interfaces::Error::Database(err).into(),
|
||||
PrunerError::Provider(err) => reth_interfaces::Error::Provider(err).into(),
|
||||
PrunerError::Database(err) => RethError::Database(err).into(),
|
||||
PrunerError::Provider(err) => RethError::Provider(err).into(),
|
||||
})),
|
||||
}
|
||||
}
|
||||
|
||||
@ -3,7 +3,7 @@ use crate::{
|
||||
BeaconConsensusEngineEvent,
|
||||
};
|
||||
use futures::{future::Either, FutureExt};
|
||||
use reth_interfaces::consensus::ForkchoiceState;
|
||||
use reth_interfaces::{consensus::ForkchoiceState, RethResult};
|
||||
use reth_payload_builder::error::PayloadBuilderError;
|
||||
use reth_rpc_types::engine::{
|
||||
CancunPayloadFields, ExecutionPayload, ForkChoiceUpdateResult, ForkchoiceUpdateError,
|
||||
@ -158,7 +158,7 @@ pub enum BeaconEngineMessage {
|
||||
/// The payload attributes for block building.
|
||||
payload_attrs: Option<PayloadAttributes>,
|
||||
/// The sender for returning forkchoice updated result.
|
||||
tx: oneshot::Sender<Result<OnForkChoiceUpdated, reth_interfaces::Error>>,
|
||||
tx: oneshot::Sender<RethResult<OnForkChoiceUpdated>>,
|
||||
},
|
||||
/// Message with exchanged transition configuration.
|
||||
TransitionConfigurationExchanged,
|
||||
|
||||
@ -18,7 +18,7 @@ use reth_interfaces::{
|
||||
executor::{BlockExecutionError, BlockValidationError},
|
||||
p2p::{bodies::client::BodiesClient, headers::client::HeadersClient},
|
||||
sync::{NetworkSyncUpdater, SyncState},
|
||||
Error,
|
||||
RethError, RethResult,
|
||||
};
|
||||
use reth_payload_builder::{PayloadBuilderAttributes, PayloadBuilderHandle};
|
||||
use reth_primitives::{
|
||||
@ -232,7 +232,7 @@ where
|
||||
target: Option<H256>,
|
||||
pipeline_run_threshold: u64,
|
||||
hooks: EngineHooks,
|
||||
) -> Result<(Self, BeaconConsensusEngineHandle), Error> {
|
||||
) -> RethResult<(Self, BeaconConsensusEngineHandle)> {
|
||||
let (to_engine, rx) = mpsc::unbounded_channel();
|
||||
Self::with_channel(
|
||||
client,
|
||||
@ -278,7 +278,7 @@ where
|
||||
to_engine: UnboundedSender<BeaconEngineMessage>,
|
||||
rx: UnboundedReceiver<BeaconEngineMessage>,
|
||||
hooks: EngineHooks,
|
||||
) -> Result<(Self, BeaconConsensusEngineHandle), Error> {
|
||||
) -> RethResult<(Self, BeaconConsensusEngineHandle)> {
|
||||
let handle = BeaconConsensusEngineHandle { to_engine };
|
||||
let sync = EngineSyncController::new(
|
||||
pipeline,
|
||||
@ -327,7 +327,7 @@ where
|
||||
/// # Returns
|
||||
///
|
||||
/// A target block hash if the pipeline is inconsistent, otherwise `None`.
|
||||
fn check_pipeline_consistency(&self) -> Result<Option<H256>, Error> {
|
||||
fn check_pipeline_consistency(&self) -> RethResult<Option<H256>> {
|
||||
// If no target was provided, check if the stages are congruent - check if the
|
||||
// checkpoint of the last stage matches the checkpoint of the first.
|
||||
let first_stage_checkpoint = self
|
||||
@ -555,7 +555,7 @@ where
|
||||
&mut self,
|
||||
state: ForkchoiceState,
|
||||
attrs: Option<PayloadAttributes>,
|
||||
tx: oneshot::Sender<Result<OnForkChoiceUpdated, Error>>,
|
||||
tx: oneshot::Sender<Result<OnForkChoiceUpdated, RethError>>,
|
||||
) -> OnForkchoiceUpdateOutcome {
|
||||
self.metrics.forkchoice_updated_messages.increment(1);
|
||||
self.blockchain.on_forkchoice_update_received(&state);
|
||||
@ -563,7 +563,7 @@ where
|
||||
let on_updated = match self.forkchoice_updated(state, attrs) {
|
||||
Ok(response) => response,
|
||||
Err(error) => {
|
||||
if let Error::Execution(ref err) = error {
|
||||
if let RethError::Execution(ref err) = error {
|
||||
if err.is_fatal() {
|
||||
// FCU resulted in a fatal error from which we can't recover
|
||||
let err = err.clone();
|
||||
@ -623,7 +623,7 @@ where
|
||||
&mut self,
|
||||
state: ForkchoiceState,
|
||||
attrs: Option<PayloadAttributes>,
|
||||
) -> Result<OnForkChoiceUpdated, Error> {
|
||||
) -> RethResult<OnForkChoiceUpdated> {
|
||||
trace!(target: "consensus::engine", ?state, "Received new forkchoice state update");
|
||||
if state.head_block_hash.is_zero() {
|
||||
return Ok(OnForkChoiceUpdated::invalid_state())
|
||||
@ -690,7 +690,7 @@ where
|
||||
PayloadStatus::new(PayloadStatusEnum::Valid, Some(state.head_block_hash))
|
||||
}
|
||||
Err(error) => {
|
||||
if let Error::Canonical(ref err) = error {
|
||||
if let RethError::Canonical(ref err) = error {
|
||||
if err.is_fatal() {
|
||||
tracing::error!(target: "consensus::engine", ?err, "Encountered fatal error");
|
||||
return Err(error)
|
||||
@ -719,7 +719,7 @@ where
|
||||
fn record_make_canonical_latency(
|
||||
&self,
|
||||
start: Instant,
|
||||
outcome: &Result<CanonicalOutcome, Error>,
|
||||
outcome: &Result<CanonicalOutcome, RethError>,
|
||||
) {
|
||||
let elapsed = start.elapsed();
|
||||
self.metrics.make_canonical_latency.record(elapsed);
|
||||
@ -747,7 +747,7 @@ where
|
||||
&mut self,
|
||||
state: ForkchoiceState,
|
||||
status: &PayloadStatus,
|
||||
) -> Result<Option<OnForkChoiceUpdated>, reth_interfaces::Error> {
|
||||
) -> RethResult<Option<OnForkChoiceUpdated>> {
|
||||
// We only perform consistency checks if the status is VALID because if the status is
|
||||
// INVALID, we want to return the correct _type_ of error to the CL so we can properly
|
||||
// describe the reason it is invalid. For example, it's possible that the status is invalid
|
||||
@ -776,7 +776,7 @@ where
|
||||
fn ensure_consistent_state(
|
||||
&mut self,
|
||||
state: ForkchoiceState,
|
||||
) -> Result<Option<OnForkChoiceUpdated>, reth_interfaces::Error> {
|
||||
) -> RethResult<Option<OnForkChoiceUpdated>> {
|
||||
// Ensure that the finalized block, if not zero, is known and in the canonical chain
|
||||
// after the head block is canonicalized.
|
||||
//
|
||||
@ -816,11 +816,7 @@ where
|
||||
///
|
||||
/// This also updates the tracked safe and finalized blocks, and should be called before
|
||||
/// returning a VALID forkchoice update response
|
||||
fn update_canon_chain(
|
||||
&self,
|
||||
head: SealedHeader,
|
||||
update: &ForkchoiceState,
|
||||
) -> Result<(), Error> {
|
||||
fn update_canon_chain(&self, head: SealedHeader, update: &ForkchoiceState) -> RethResult<()> {
|
||||
self.update_head(head)?;
|
||||
self.update_finalized_block(update.finalized_block_hash)?;
|
||||
self.update_safe_block(update.safe_block_hash)?;
|
||||
@ -835,7 +831,7 @@ where
|
||||
///
|
||||
/// This should be called before returning a VALID forkchoice update response
|
||||
#[inline]
|
||||
fn update_head(&self, head: SealedHeader) -> Result<(), reth_interfaces::Error> {
|
||||
fn update_head(&self, head: SealedHeader) -> RethResult<()> {
|
||||
let mut head_block = Head {
|
||||
number: head.number,
|
||||
hash: head.hash,
|
||||
@ -850,7 +846,7 @@ where
|
||||
|
||||
head_block.total_difficulty =
|
||||
self.blockchain.header_td_by_number(head_block.number)?.ok_or_else(|| {
|
||||
Error::Provider(ProviderError::TotalDifficultyNotFound {
|
||||
RethError::Provider(ProviderError::TotalDifficultyNotFound {
|
||||
number: head_block.number,
|
||||
})
|
||||
})?;
|
||||
@ -863,17 +859,17 @@ where
|
||||
///
|
||||
/// Returns an error if the block is not found.
|
||||
#[inline]
|
||||
fn update_safe_block(&self, safe_block_hash: H256) -> Result<(), reth_interfaces::Error> {
|
||||
fn update_safe_block(&self, safe_block_hash: H256) -> RethResult<()> {
|
||||
if !safe_block_hash.is_zero() {
|
||||
if self.blockchain.safe_block_hash()? == Some(safe_block_hash) {
|
||||
// nothing to update
|
||||
return Ok(())
|
||||
}
|
||||
|
||||
let safe = self
|
||||
.blockchain
|
||||
.find_block_by_hash(safe_block_hash, BlockSource::Any)?
|
||||
.ok_or_else(|| Error::Provider(ProviderError::UnknownBlockHash(safe_block_hash)))?;
|
||||
let safe =
|
||||
self.blockchain.find_block_by_hash(safe_block_hash, BlockSource::Any)?.ok_or_else(
|
||||
|| RethError::Provider(ProviderError::UnknownBlockHash(safe_block_hash)),
|
||||
)?;
|
||||
self.blockchain.set_safe(safe.header.seal(safe_block_hash));
|
||||
}
|
||||
Ok(())
|
||||
@ -883,10 +879,7 @@ where
|
||||
///
|
||||
/// Returns an error if the block is not found.
|
||||
#[inline]
|
||||
fn update_finalized_block(
|
||||
&self,
|
||||
finalized_block_hash: H256,
|
||||
) -> Result<(), reth_interfaces::Error> {
|
||||
fn update_finalized_block(&self, finalized_block_hash: H256) -> RethResult<()> {
|
||||
if !finalized_block_hash.is_zero() {
|
||||
if self.blockchain.finalized_block_hash()? == Some(finalized_block_hash) {
|
||||
// nothing to update
|
||||
@ -897,7 +890,7 @@ where
|
||||
.blockchain
|
||||
.find_block_by_hash(finalized_block_hash, BlockSource::Any)?
|
||||
.ok_or_else(|| {
|
||||
Error::Provider(ProviderError::UnknownBlockHash(finalized_block_hash))
|
||||
RethError::Provider(ProviderError::UnknownBlockHash(finalized_block_hash))
|
||||
})?;
|
||||
self.blockchain.finalize_block(finalized.number);
|
||||
self.blockchain.set_finalized(finalized.header.seal(finalized_block_hash));
|
||||
@ -915,7 +908,7 @@ where
|
||||
fn on_failed_canonical_forkchoice_update(
|
||||
&mut self,
|
||||
state: &ForkchoiceState,
|
||||
error: Error,
|
||||
error: RethError,
|
||||
) -> PayloadStatus {
|
||||
debug_assert!(self.sync.is_pipeline_idle(), "pipeline must be idle");
|
||||
|
||||
@ -929,7 +922,7 @@ where
|
||||
|
||||
#[allow(clippy::single_match)]
|
||||
match &error {
|
||||
Error::Canonical(
|
||||
RethError::Canonical(
|
||||
error @ CanonicalError::Validation(BlockValidationError::BlockPreMerge { .. }),
|
||||
) => {
|
||||
warn!(target: "consensus::engine", ?error, ?state, "Failed to canonicalize the head hash");
|
||||
@ -938,7 +931,7 @@ where
|
||||
})
|
||||
.with_latest_valid_hash(H256::zero())
|
||||
}
|
||||
Error::Execution(BlockExecutionError::BlockHashNotFoundInChain { .. }) => {
|
||||
RethError::Execution(BlockExecutionError::BlockHashNotFoundInChain { .. }) => {
|
||||
// This just means we couldn't find the block when attempting to make it canonical,
|
||||
// so we should not warn the user, since this will result in us attempting to sync
|
||||
// to a new target and is considered normal operation during sync
|
||||
@ -1330,7 +1323,7 @@ where
|
||||
///
|
||||
/// If the given block is missing from the database, this will return `false`. Otherwise, `true`
|
||||
/// is returned: the database contains the hash and the tree was updated.
|
||||
fn update_tree_on_finished_pipeline(&mut self, block_hash: H256) -> Result<bool, Error> {
|
||||
fn update_tree_on_finished_pipeline(&mut self, block_hash: H256) -> RethResult<bool> {
|
||||
let synced_to_finalized = match self.blockchain.block_number(block_hash)? {
|
||||
Some(number) => {
|
||||
// Attempt to restore the tree.
|
||||
@ -1495,7 +1488,7 @@ where
|
||||
// it's part of the canonical chain: if it's the safe or the finalized block
|
||||
if matches!(
|
||||
err,
|
||||
Error::Canonical(CanonicalError::BlockchainTree(
|
||||
RethError::Canonical(CanonicalError::BlockchainTree(
|
||||
BlockchainTreeError::BlockHashNotFoundInChain { .. }
|
||||
))
|
||||
) {
|
||||
@ -1580,9 +1573,9 @@ where
|
||||
Ok(header) => match header {
|
||||
Some(header) => header,
|
||||
None => {
|
||||
return Some(Err(Error::Provider(ProviderError::HeaderNotFound(
|
||||
max_block.into(),
|
||||
))
|
||||
return Some(Err(RethError::Provider(
|
||||
ProviderError::HeaderNotFound(max_block.into()),
|
||||
)
|
||||
.into()))
|
||||
}
|
||||
},
|
||||
@ -1755,7 +1748,7 @@ where
|
||||
}
|
||||
OnForkchoiceUpdateOutcome::Fatal(err) => {
|
||||
// fatal error, we can terminate the future
|
||||
return Poll::Ready(Err(Error::Execution(err).into()))
|
||||
return Poll::Ready(Err(RethError::Execution(err).into()))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -1,5 +1,5 @@
|
||||
//! Collection of methods for block validation.
|
||||
use reth_interfaces::{consensus::ConsensusError, Result as RethResult};
|
||||
use reth_interfaces::{consensus::ConsensusError, RethResult};
|
||||
use reth_primitives::{
|
||||
constants::{
|
||||
self,
|
||||
@ -504,7 +504,7 @@ mod tests {
|
||||
use super::*;
|
||||
use assert_matches::assert_matches;
|
||||
use mockall::mock;
|
||||
use reth_interfaces::{Error::Consensus, Result};
|
||||
use reth_interfaces::{RethError::Consensus, RethResult};
|
||||
use reth_primitives::{
|
||||
constants::eip4844::DATA_GAS_PER_BLOB, hex_literal::hex, proofs, Account, Address,
|
||||
BlockBody, BlockHash, BlockHashOrNumber, Bytes, ChainSpecBuilder, Header, Signature,
|
||||
@ -516,7 +516,7 @@ mod tests {
|
||||
WithdrawalsProvider {}
|
||||
|
||||
impl WithdrawalsProvider for WithdrawalsProvider {
|
||||
fn latest_withdrawal(&self) -> Result<Option<Withdrawal>> ;
|
||||
fn latest_withdrawal(&self) -> RethResult<Option<Withdrawal>> ;
|
||||
|
||||
fn withdrawals_by_block(
|
||||
&self,
|
||||
@ -555,44 +555,44 @@ mod tests {
|
||||
}
|
||||
|
||||
impl AccountReader for Provider {
|
||||
fn basic_account(&self, _address: Address) -> Result<Option<Account>> {
|
||||
fn basic_account(&self, _address: Address) -> RethResult<Option<Account>> {
|
||||
Ok(self.account)
|
||||
}
|
||||
}
|
||||
|
||||
impl HeaderProvider for Provider {
|
||||
fn is_known(&self, _block_hash: &BlockHash) -> Result<bool> {
|
||||
fn is_known(&self, _block_hash: &BlockHash) -> RethResult<bool> {
|
||||
Ok(self.is_known)
|
||||
}
|
||||
|
||||
fn header(&self, _block_number: &BlockHash) -> Result<Option<Header>> {
|
||||
fn header(&self, _block_number: &BlockHash) -> RethResult<Option<Header>> {
|
||||
Ok(self.parent.clone())
|
||||
}
|
||||
|
||||
fn header_by_number(&self, _num: u64) -> Result<Option<Header>> {
|
||||
fn header_by_number(&self, _num: u64) -> RethResult<Option<Header>> {
|
||||
Ok(self.parent.clone())
|
||||
}
|
||||
|
||||
fn header_td(&self, _hash: &BlockHash) -> Result<Option<U256>> {
|
||||
fn header_td(&self, _hash: &BlockHash) -> RethResult<Option<U256>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn header_td_by_number(&self, _number: BlockNumber) -> Result<Option<U256>> {
|
||||
fn header_td_by_number(&self, _number: BlockNumber) -> RethResult<Option<U256>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn headers_range(&self, _range: impl RangeBounds<BlockNumber>) -> Result<Vec<Header>> {
|
||||
fn headers_range(&self, _range: impl RangeBounds<BlockNumber>) -> RethResult<Vec<Header>> {
|
||||
Ok(vec![])
|
||||
}
|
||||
|
||||
fn sealed_headers_range(
|
||||
&self,
|
||||
_range: impl RangeBounds<BlockNumber>,
|
||||
) -> Result<Vec<SealedHeader>> {
|
||||
) -> RethResult<Vec<SealedHeader>> {
|
||||
Ok(vec![])
|
||||
}
|
||||
|
||||
fn sealed_header(&self, _block_number: BlockNumber) -> Result<Option<SealedHeader>> {
|
||||
fn sealed_header(&self, _block_number: BlockNumber) -> RethResult<Option<SealedHeader>> {
|
||||
Ok(None)
|
||||
}
|
||||
}
|
||||
@ -606,7 +606,7 @@ mod tests {
|
||||
self.withdrawals_provider.withdrawals_by_block(_id, _timestamp)
|
||||
}
|
||||
|
||||
fn latest_withdrawal(&self) -> Result<Option<Withdrawal>> {
|
||||
fn latest_withdrawal(&self) -> RethResult<Option<Withdrawal>> {
|
||||
self.withdrawals_provider.latest_withdrawal()
|
||||
}
|
||||
}
|
||||
|
||||
@ -300,18 +300,18 @@ impl InsertBlockErrorKind {
|
||||
}
|
||||
|
||||
// This is a convenience impl to convert from crate::Error to InsertBlockErrorKind, most
|
||||
impl From<crate::Error> for InsertBlockErrorKind {
|
||||
fn from(err: crate::Error) -> Self {
|
||||
use crate::Error;
|
||||
impl From<crate::RethError> for InsertBlockErrorKind {
|
||||
fn from(err: crate::RethError) -> Self {
|
||||
use crate::RethError;
|
||||
|
||||
match err {
|
||||
Error::Execution(err) => InsertBlockErrorKind::Execution(err),
|
||||
Error::Consensus(err) => InsertBlockErrorKind::Consensus(err),
|
||||
Error::Database(err) => InsertBlockErrorKind::Internal(Box::new(err)),
|
||||
Error::Provider(err) => InsertBlockErrorKind::Internal(Box::new(err)),
|
||||
Error::Network(err) => InsertBlockErrorKind::Internal(Box::new(err)),
|
||||
Error::Custom(err) => InsertBlockErrorKind::Internal(err.into()),
|
||||
Error::Canonical(err) => InsertBlockErrorKind::Canonical(err),
|
||||
RethError::Execution(err) => InsertBlockErrorKind::Execution(err),
|
||||
RethError::Consensus(err) => InsertBlockErrorKind::Consensus(err),
|
||||
RethError::Database(err) => InsertBlockErrorKind::Internal(Box::new(err)),
|
||||
RethError::Provider(err) => InsertBlockErrorKind::Internal(Box::new(err)),
|
||||
RethError::Network(err) => InsertBlockErrorKind::Internal(Box::new(err)),
|
||||
RethError::Custom(err) => InsertBlockErrorKind::Internal(err.into()),
|
||||
RethError::Canonical(err) => InsertBlockErrorKind::Canonical(err),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -1,4 +1,4 @@
|
||||
use crate::{blockchain_tree::error::InsertBlockError, Error};
|
||||
use crate::{blockchain_tree::error::InsertBlockError, RethResult};
|
||||
use reth_primitives::{
|
||||
BlockHash, BlockNumHash, BlockNumber, Receipt, SealedBlock, SealedBlockWithSenders,
|
||||
SealedHeader,
|
||||
@ -66,14 +66,14 @@ pub trait BlockchainTreeEngine: BlockchainTreeViewer + Send + Sync {
|
||||
fn connect_buffered_blocks_to_canonical_hashes_and_finalize(
|
||||
&self,
|
||||
last_finalized_block: BlockNumber,
|
||||
) -> Result<(), Error>;
|
||||
) -> RethResult<()>;
|
||||
|
||||
/// Reads the last `N` canonical hashes from the database and updates the block indices of the
|
||||
/// tree by attempting to connect the buffered blocks to canonical hashes.
|
||||
///
|
||||
/// `N` is the `max_reorg_depth` plus the number of block hashes needed to satisfy the
|
||||
/// `BLOCKHASH` opcode in the EVM.
|
||||
fn connect_buffered_blocks_to_canonical_hashes(&self) -> Result<(), Error>;
|
||||
fn connect_buffered_blocks_to_canonical_hashes(&self) -> RethResult<()>;
|
||||
|
||||
/// Make a block and its parent chain part of the canonical chain by committing it to the
|
||||
/// database.
|
||||
@ -86,10 +86,10 @@ pub trait BlockchainTreeEngine: BlockchainTreeViewer + Send + Sync {
|
||||
/// # Returns
|
||||
///
|
||||
/// Returns `Ok` if the blocks were canonicalized, or if the blocks were already canonical.
|
||||
fn make_canonical(&self, block_hash: &BlockHash) -> Result<CanonicalOutcome, Error>;
|
||||
fn make_canonical(&self, block_hash: &BlockHash) -> RethResult<CanonicalOutcome>;
|
||||
|
||||
/// Unwind tables and put it inside state
|
||||
fn unwind(&self, unwind_to: BlockNumber) -> Result<(), Error>;
|
||||
fn unwind(&self, unwind_to: BlockNumber) -> RethResult<()>;
|
||||
}
|
||||
|
||||
/// All possible outcomes of a canonicalization attempt of [BlockchainTreeEngine::make_canonical].
|
||||
@ -219,7 +219,7 @@ pub trait BlockchainTreeViewer: Send + Sync {
|
||||
fn find_canonical_ancestor(&self, parent_hash: BlockHash) -> Option<BlockHash>;
|
||||
|
||||
/// Return whether or not the block is known and in the canonical chain.
|
||||
fn is_canonical(&self, hash: BlockHash) -> Result<bool, Error>;
|
||||
fn is_canonical(&self, hash: BlockHash) -> RethResult<bool>;
|
||||
|
||||
/// Given the hash of a block, this checks the buffered blocks for the lowest ancestor in the
|
||||
/// buffer.
|
||||
|
||||
@ -1,10 +1,10 @@
|
||||
/// Result alias for `Error`
|
||||
pub type Result<T> = std::result::Result<T, Error>;
|
||||
/// Result alias for [`RethError`]
|
||||
pub type RethResult<T> = Result<T, RethError>;
|
||||
|
||||
/// Core error variants possible when interacting with the blockchain
|
||||
#[derive(Debug, thiserror::Error, Clone, PartialEq, Eq)]
|
||||
#[allow(missing_docs)]
|
||||
pub enum Error {
|
||||
pub enum RethError {
|
||||
#[error(transparent)]
|
||||
Execution(#[from] crate::executor::BlockExecutionError),
|
||||
|
||||
@ -24,5 +24,5 @@ pub enum Error {
|
||||
Canonical(#[from] crate::blockchain_tree::error::CanonicalError),
|
||||
|
||||
#[error("{0}")]
|
||||
Custom(std::string::String),
|
||||
Custom(String),
|
||||
}
|
||||
|
||||
@ -28,7 +28,7 @@ pub mod executor;
|
||||
|
||||
/// Possible errors when interacting with the chain.
|
||||
mod error;
|
||||
pub use error::{Error, Result};
|
||||
pub use error::{RethError, RethResult};
|
||||
|
||||
/// P2P traits.
|
||||
pub mod p2p;
|
||||
|
||||
@ -16,7 +16,7 @@
|
||||
use crate::metrics::PayloadBuilderMetrics;
|
||||
use futures_core::ready;
|
||||
use futures_util::FutureExt;
|
||||
use reth_interfaces::Error;
|
||||
use reth_interfaces::{RethError, RethResult};
|
||||
use reth_payload_builder::{
|
||||
database::CachedReads, error::PayloadBuilderError, BuiltPayload, KeepPayloadJobAlive,
|
||||
PayloadBuilderAttributes, PayloadJob, PayloadJobGenerator,
|
||||
@ -964,12 +964,12 @@ impl WithdrawalsOutcome {
|
||||
/// Returns the withdrawals root.
|
||||
///
|
||||
/// Returns `None` values pre shanghai
|
||||
fn commit_withdrawals<DB: Database<Error = Error>>(
|
||||
fn commit_withdrawals<DB: Database<Error = RethError>>(
|
||||
db: &mut State<DB>,
|
||||
chain_spec: &ChainSpec,
|
||||
timestamp: u64,
|
||||
withdrawals: Vec<Withdrawal>,
|
||||
) -> Result<WithdrawalsOutcome, Error> {
|
||||
) -> RethResult<WithdrawalsOutcome> {
|
||||
if !chain_spec.is_shanghai_activated_at_timestamp(timestamp) {
|
||||
return Ok(WithdrawalsOutcome::pre_shanghai())
|
||||
}
|
||||
|
||||
@ -1,5 +1,6 @@
|
||||
//! Error types emitted by types or implementations of this crate.
|
||||
|
||||
use reth_interfaces::RethError;
|
||||
use reth_primitives::H256;
|
||||
use reth_transaction_pool::BlobStoreError;
|
||||
use revm_primitives::EVMError;
|
||||
@ -19,10 +20,10 @@ pub enum PayloadBuilderError {
|
||||
BlobStore(#[from] BlobStoreError),
|
||||
/// Other internal error
|
||||
#[error(transparent)]
|
||||
Internal(#[from] reth_interfaces::Error),
|
||||
Internal(#[from] RethError),
|
||||
/// Unrecoverable error during evm execution.
|
||||
#[error("evm execution error: {0:?}")]
|
||||
EvmExecutionError(EVMError<reth_interfaces::Error>),
|
||||
EvmExecutionError(EVMError<RethError>),
|
||||
/// Thrown if the payload requests withdrawals before Shanghai activation.
|
||||
#[error("withdrawals set before Shanghai activation")]
|
||||
WithdrawalsBeforeShanghai,
|
||||
|
||||
@ -1,17 +1,19 @@
|
||||
use reth_db::DatabaseError;
|
||||
use reth_interfaces::RethError;
|
||||
use reth_primitives::PrunePartError;
|
||||
use reth_provider::ProviderError;
|
||||
use thiserror::Error;
|
||||
|
||||
#[derive(Error, Debug)]
|
||||
pub enum PrunerError {
|
||||
#[error(transparent)]
|
||||
PrunePart(#[from] reth_primitives::PrunePartError),
|
||||
PrunePart(#[from] PrunePartError),
|
||||
|
||||
#[error("Inconsistent data: {0}")]
|
||||
InconsistentData(&'static str),
|
||||
|
||||
#[error("An interface error occurred.")]
|
||||
Interface(#[from] reth_interfaces::Error),
|
||||
Interface(#[from] RethError),
|
||||
|
||||
#[error(transparent)]
|
||||
Database(#[from] DatabaseError),
|
||||
|
||||
@ -11,6 +11,7 @@ use reth_db::{
|
||||
transaction::DbTxMut,
|
||||
BlockNumberList,
|
||||
};
|
||||
use reth_interfaces::RethResult;
|
||||
use reth_primitives::{
|
||||
BlockNumber, ChainSpec, PruneBatchSizes, PruneCheckpoint, PruneMode, PruneModes, PrunePart,
|
||||
TxNumber, MINIMUM_PRUNING_DISTANCE,
|
||||
@ -279,7 +280,7 @@ impl<DB: Database> Pruner<DB> {
|
||||
provider: &DatabaseProviderRW<'_, DB>,
|
||||
prune_part: PrunePart,
|
||||
to_block: BlockNumber,
|
||||
) -> reth_interfaces::Result<Option<RangeInclusive<BlockNumber>>> {
|
||||
) -> RethResult<Option<RangeInclusive<BlockNumber>>> {
|
||||
let from_block = provider
|
||||
.get_prune_checkpoint(prune_part)?
|
||||
.and_then(|checkpoint| checkpoint.block_number)
|
||||
@ -309,7 +310,7 @@ impl<DB: Database> Pruner<DB> {
|
||||
provider: &DatabaseProviderRW<'_, DB>,
|
||||
prune_part: PrunePart,
|
||||
to_block: BlockNumber,
|
||||
) -> reth_interfaces::Result<Option<RangeInclusive<TxNumber>>> {
|
||||
) -> RethResult<Option<RangeInclusive<TxNumber>>> {
|
||||
let from_tx_number = provider
|
||||
.get_prune_checkpoint(prune_part)?
|
||||
// Checkpoint exists, prune from the next transaction after the highest pruned one
|
||||
|
||||
@ -1,4 +1,4 @@
|
||||
use reth_interfaces::Error;
|
||||
use reth_interfaces::RethError;
|
||||
use reth_primitives::{H160, H256, KECCAK_EMPTY, U256};
|
||||
use reth_provider::StateProvider;
|
||||
use revm::{
|
||||
@ -11,7 +11,7 @@ use revm::{
|
||||
pub type SubState<DB> = CacheDB<StateProviderDatabase<DB>>;
|
||||
|
||||
/// State boxed database with reth Error.
|
||||
pub type RethStateDBBox<'a> = StateDBBox<'a, Error>;
|
||||
pub type RethStateDBBox<'a> = StateDBBox<'a, RethError>;
|
||||
|
||||
/// Wrapper around StateProvider that implements revm database trait
|
||||
#[derive(Debug, Clone)]
|
||||
@ -40,7 +40,7 @@ impl<DB: StateProvider> StateProviderDatabase<DB> {
|
||||
}
|
||||
|
||||
impl<DB: StateProvider> Database for StateProviderDatabase<DB> {
|
||||
type Error = Error;
|
||||
type Error = RethError;
|
||||
|
||||
fn basic(&mut self, address: H160) -> Result<Option<AccountInfo>, Self::Error> {
|
||||
Ok(self.0.basic_account(address)?.map(|account| AccountInfo {
|
||||
|
||||
@ -8,7 +8,7 @@ use crate::{
|
||||
};
|
||||
use reth_interfaces::{
|
||||
executor::{BlockExecutionError, BlockValidationError},
|
||||
Error,
|
||||
RethError,
|
||||
};
|
||||
use reth_primitives::{
|
||||
Address, Block, BlockNumber, Bloom, ChainSpec, Hardfork, Header, PruneMode, PruneModes,
|
||||
@ -46,7 +46,7 @@ pub struct EVMProcessor<'a> {
|
||||
/// The configured chain-spec
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
/// revm instance that contains database and env environment.
|
||||
evm: EVM<StateDBBox<'a, Error>>,
|
||||
evm: EVM<StateDBBox<'a, RethError>>,
|
||||
/// Hook and inspector stack that we want to invoke on that hook.
|
||||
stack: InspectorStack,
|
||||
/// The collection of receipts.
|
||||
@ -106,7 +106,10 @@ impl<'a> EVMProcessor<'a> {
|
||||
}
|
||||
|
||||
/// Create a new EVM processor with the given revm state.
|
||||
pub fn new_with_state(chain_spec: Arc<ChainSpec>, revm_state: StateDBBox<'a, Error>) -> Self {
|
||||
pub fn new_with_state(
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
revm_state: StateDBBox<'a, RethError>,
|
||||
) -> Self {
|
||||
let mut evm = EVM::new();
|
||||
evm.database(revm_state);
|
||||
EVMProcessor {
|
||||
@ -128,7 +131,7 @@ impl<'a> EVMProcessor<'a> {
|
||||
}
|
||||
|
||||
/// Returns a reference to the database
|
||||
pub fn db_mut(&mut self) -> &mut StateDBBox<'a, Error> {
|
||||
pub fn db_mut(&mut self) -> &mut StateDBBox<'a, RethError> {
|
||||
// Option will be removed from EVM in the future.
|
||||
// as it is always some.
|
||||
// https://github.com/bluealloy/revm/issues/697
|
||||
@ -552,6 +555,7 @@ pub fn verify_receipt<'a>(
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use reth_interfaces::RethResult;
|
||||
use reth_primitives::{
|
||||
constants::{BEACON_ROOTS_ADDRESS, SYSTEM_ADDRESS},
|
||||
keccak256, Account, Bytecode, Bytes, ChainSpecBuilder, ForkCondition, StorageKey, MAINNET,
|
||||
@ -594,14 +598,14 @@ mod tests {
|
||||
}
|
||||
|
||||
impl AccountReader for StateProviderTest {
|
||||
fn basic_account(&self, address: Address) -> reth_interfaces::Result<Option<Account>> {
|
||||
fn basic_account(&self, address: Address) -> RethResult<Option<Account>> {
|
||||
let ret = Ok(self.accounts.get(&address).map(|(_, acc)| *acc));
|
||||
ret
|
||||
}
|
||||
}
|
||||
|
||||
impl BlockHashReader for StateProviderTest {
|
||||
fn block_hash(&self, number: u64) -> reth_interfaces::Result<Option<H256>> {
|
||||
fn block_hash(&self, number: u64) -> RethResult<Option<H256>> {
|
||||
Ok(self.block_hash.get(&number).cloned())
|
||||
}
|
||||
|
||||
@ -609,7 +613,7 @@ mod tests {
|
||||
&self,
|
||||
start: BlockNumber,
|
||||
end: BlockNumber,
|
||||
) -> reth_interfaces::Result<Vec<H256>> {
|
||||
) -> RethResult<Vec<H256>> {
|
||||
let range = start..end;
|
||||
Ok(self
|
||||
.block_hash
|
||||
@ -620,10 +624,7 @@ mod tests {
|
||||
}
|
||||
|
||||
impl StateRootProvider for StateProviderTest {
|
||||
fn state_root(
|
||||
&self,
|
||||
_bundle_state: BundleStateWithReceipts,
|
||||
) -> reth_interfaces::Result<H256> {
|
||||
fn state_root(&self, _bundle_state: BundleStateWithReceipts) -> RethResult<H256> {
|
||||
todo!()
|
||||
}
|
||||
}
|
||||
@ -632,15 +633,15 @@ mod tests {
|
||||
fn storage(
|
||||
&self,
|
||||
account: Address,
|
||||
storage_key: reth_primitives::StorageKey,
|
||||
) -> reth_interfaces::Result<Option<reth_primitives::StorageValue>> {
|
||||
storage_key: StorageKey,
|
||||
) -> RethResult<Option<reth_primitives::StorageValue>> {
|
||||
Ok(self
|
||||
.accounts
|
||||
.get(&account)
|
||||
.and_then(|(storage, _)| storage.get(&storage_key).cloned()))
|
||||
}
|
||||
|
||||
fn bytecode_by_hash(&self, code_hash: H256) -> reth_interfaces::Result<Option<Bytecode>> {
|
||||
fn bytecode_by_hash(&self, code_hash: H256) -> RethResult<Option<Bytecode>> {
|
||||
Ok(self.contracts.get(&code_hash).cloned())
|
||||
}
|
||||
|
||||
@ -648,7 +649,7 @@ mod tests {
|
||||
&self,
|
||||
_address: Address,
|
||||
_keys: &[H256],
|
||||
) -> reth_interfaces::Result<(Vec<Bytes>, H256, Vec<Vec<Bytes>>)> {
|
||||
) -> RethResult<(Vec<Bytes>, H256, Vec<Vec<Bytes>>)> {
|
||||
todo!()
|
||||
}
|
||||
}
|
||||
|
||||
@ -11,7 +11,7 @@ use crate::eth::{
|
||||
signer::EthSigner,
|
||||
};
|
||||
use async_trait::async_trait;
|
||||
use reth_interfaces::Result;
|
||||
use reth_interfaces::RethResult;
|
||||
use reth_network_api::NetworkInfo;
|
||||
use reth_primitives::{
|
||||
Address, BlockId, BlockNumberOrTag, ChainInfo, SealedBlock, H256, U256, U64,
|
||||
@ -48,13 +48,13 @@ pub use transactions::{EthTransactions, TransactionSource};
|
||||
#[async_trait]
|
||||
pub trait EthApiSpec: EthTransactions + Send + Sync {
|
||||
/// Returns the current ethereum protocol version.
|
||||
async fn protocol_version(&self) -> Result<U64>;
|
||||
async fn protocol_version(&self) -> RethResult<U64>;
|
||||
|
||||
/// Returns the chain id
|
||||
fn chain_id(&self) -> U64;
|
||||
|
||||
/// Returns provider chain info
|
||||
fn chain_info(&self) -> Result<ChainInfo>;
|
||||
fn chain_info(&self) -> RethResult<ChainInfo>;
|
||||
|
||||
/// Returns a list of addresses owned by provider.
|
||||
fn accounts(&self) -> Vec<Address>;
|
||||
@ -63,7 +63,7 @@ pub trait EthApiSpec: EthTransactions + Send + Sync {
|
||||
fn is_syncing(&self) -> bool;
|
||||
|
||||
/// Returns the [SyncStatus] of the network
|
||||
fn sync_status(&self) -> Result<SyncStatus>;
|
||||
fn sync_status(&self) -> RethResult<SyncStatus>;
|
||||
}
|
||||
|
||||
/// `Eth` API implementation.
|
||||
@ -221,12 +221,12 @@ where
|
||||
}
|
||||
|
||||
/// Returns the state at the given block number
|
||||
pub fn state_at_hash(&self, block_hash: H256) -> Result<StateProviderBox<'_>> {
|
||||
pub fn state_at_hash(&self, block_hash: H256) -> RethResult<StateProviderBox<'_>> {
|
||||
self.provider().history_by_block_hash(block_hash)
|
||||
}
|
||||
|
||||
/// Returns the _latest_ state
|
||||
pub fn latest_state(&self) -> Result<StateProviderBox<'_>> {
|
||||
pub fn latest_state(&self) -> RethResult<StateProviderBox<'_>> {
|
||||
self.provider().latest()
|
||||
}
|
||||
}
|
||||
@ -341,7 +341,7 @@ where
|
||||
/// Returns the current ethereum protocol version.
|
||||
///
|
||||
/// Note: This returns an `U64`, since this should return as hex string.
|
||||
async fn protocol_version(&self) -> Result<U64> {
|
||||
async fn protocol_version(&self) -> RethResult<U64> {
|
||||
let status = self.network().network_status().await?;
|
||||
Ok(U64::from(status.protocol_version))
|
||||
}
|
||||
@ -352,7 +352,7 @@ where
|
||||
}
|
||||
|
||||
/// Returns the current info for the chain
|
||||
fn chain_info(&self) -> Result<ChainInfo> {
|
||||
fn chain_info(&self) -> RethResult<ChainInfo> {
|
||||
self.provider().chain_info()
|
||||
}
|
||||
|
||||
@ -365,7 +365,7 @@ where
|
||||
}
|
||||
|
||||
/// Returns the [SyncStatus] of the network
|
||||
fn sync_status(&self) -> Result<SyncStatus> {
|
||||
fn sync_status(&self) -> RethResult<SyncStatus> {
|
||||
let status = if self.is_syncing() {
|
||||
let current_block = U256::from(
|
||||
self.provider().chain_info().map(|info| info.best_number).unwrap_or_default(),
|
||||
|
||||
37
crates/rpc/rpc/src/eth/cache/mod.rs
vendored
37
crates/rpc/rpc/src/eth/cache/mod.rs
vendored
@ -1,7 +1,7 @@
|
||||
//! Async caching support for eth RPC
|
||||
|
||||
use futures::{future::Either, Stream, StreamExt};
|
||||
use reth_interfaces::{provider::ProviderError, Result};
|
||||
use reth_interfaces::{provider::ProviderError, RethResult};
|
||||
use reth_primitives::{Block, Receipt, SealedBlock, TransactionSigned, H256};
|
||||
use reth_provider::{
|
||||
BlockReader, BlockSource, CanonStateNotification, EvmEnvProvider, StateProviderFactory,
|
||||
@ -30,16 +30,16 @@ mod multi_consumer;
|
||||
pub use multi_consumer::MultiConsumerLruCache;
|
||||
|
||||
/// The type that can send the response to a requested [Block]
|
||||
type BlockResponseSender = oneshot::Sender<Result<Option<Block>>>;
|
||||
type BlockResponseSender = oneshot::Sender<RethResult<Option<Block>>>;
|
||||
|
||||
/// The type that can send the response to a requested [Block]
|
||||
type BlockTransactionsResponseSender = oneshot::Sender<Result<Option<Vec<TransactionSigned>>>>;
|
||||
type BlockTransactionsResponseSender = oneshot::Sender<RethResult<Option<Vec<TransactionSigned>>>>;
|
||||
|
||||
/// The type that can send the response to the requested receipts of a block.
|
||||
type ReceiptsResponseSender = oneshot::Sender<Result<Option<Vec<Receipt>>>>;
|
||||
type ReceiptsResponseSender = oneshot::Sender<RethResult<Option<Vec<Receipt>>>>;
|
||||
|
||||
/// The type that can send the response to a requested env
|
||||
type EnvResponseSender = oneshot::Sender<Result<(CfgEnv, BlockEnv)>>;
|
||||
type EnvResponseSender = oneshot::Sender<RethResult<(CfgEnv, BlockEnv)>>;
|
||||
|
||||
type BlockLruCache<L> = MultiConsumerLruCache<
|
||||
H256,
|
||||
@ -127,7 +127,7 @@ impl EthStateCache {
|
||||
/// Requests the [Block] for the block hash
|
||||
///
|
||||
/// Returns `None` if the block does not exist.
|
||||
pub(crate) async fn get_block(&self, block_hash: H256) -> Result<Option<Block>> {
|
||||
pub(crate) async fn get_block(&self, block_hash: H256) -> RethResult<Option<Block>> {
|
||||
let (response_tx, rx) = oneshot::channel();
|
||||
let _ = self.to_service.send(CacheAction::GetBlock { block_hash, response_tx });
|
||||
rx.await.map_err(|_| ProviderError::CacheServiceUnavailable)?
|
||||
@ -136,7 +136,10 @@ impl EthStateCache {
|
||||
/// Requests the [Block] for the block hash, sealed with the given block hash.
|
||||
///
|
||||
/// Returns `None` if the block does not exist.
|
||||
pub(crate) async fn get_sealed_block(&self, block_hash: H256) -> Result<Option<SealedBlock>> {
|
||||
pub(crate) async fn get_sealed_block(
|
||||
&self,
|
||||
block_hash: H256,
|
||||
) -> RethResult<Option<SealedBlock>> {
|
||||
Ok(self.get_block(block_hash).await?.map(|block| block.seal(block_hash)))
|
||||
}
|
||||
|
||||
@ -146,7 +149,7 @@ impl EthStateCache {
|
||||
pub(crate) async fn get_block_transactions(
|
||||
&self,
|
||||
block_hash: H256,
|
||||
) -> Result<Option<Vec<TransactionSigned>>> {
|
||||
) -> RethResult<Option<Vec<TransactionSigned>>> {
|
||||
let (response_tx, rx) = oneshot::channel();
|
||||
let _ = self.to_service.send(CacheAction::GetBlockTransactions { block_hash, response_tx });
|
||||
rx.await.map_err(|_| ProviderError::CacheServiceUnavailable)?
|
||||
@ -156,7 +159,7 @@ impl EthStateCache {
|
||||
pub(crate) async fn get_transactions_and_receipts(
|
||||
&self,
|
||||
block_hash: H256,
|
||||
) -> Result<Option<(Vec<TransactionSigned>, Vec<Receipt>)>> {
|
||||
) -> RethResult<Option<(Vec<TransactionSigned>, Vec<Receipt>)>> {
|
||||
let transactions = self.get_block_transactions(block_hash);
|
||||
let receipts = self.get_receipts(block_hash);
|
||||
|
||||
@ -168,7 +171,7 @@ impl EthStateCache {
|
||||
/// Requests the [Receipt] for the block hash
|
||||
///
|
||||
/// Returns `None` if the block was not found.
|
||||
pub(crate) async fn get_receipts(&self, block_hash: H256) -> Result<Option<Vec<Receipt>>> {
|
||||
pub(crate) async fn get_receipts(&self, block_hash: H256) -> RethResult<Option<Vec<Receipt>>> {
|
||||
let (response_tx, rx) = oneshot::channel();
|
||||
let _ = self.to_service.send(CacheAction::GetReceipts { block_hash, response_tx });
|
||||
rx.await.map_err(|_| ProviderError::CacheServiceUnavailable)?
|
||||
@ -178,7 +181,7 @@ impl EthStateCache {
|
||||
pub(crate) async fn get_block_and_receipts(
|
||||
&self,
|
||||
block_hash: H256,
|
||||
) -> Result<Option<(SealedBlock, Vec<Receipt>)>> {
|
||||
) -> RethResult<Option<(SealedBlock, Vec<Receipt>)>> {
|
||||
let block = self.get_sealed_block(block_hash);
|
||||
let receipts = self.get_receipts(block_hash);
|
||||
|
||||
@ -191,7 +194,7 @@ impl EthStateCache {
|
||||
///
|
||||
/// Returns an error if the corresponding header (required for populating the envs) was not
|
||||
/// found.
|
||||
pub(crate) async fn get_evm_env(&self, block_hash: H256) -> Result<(CfgEnv, BlockEnv)> {
|
||||
pub(crate) async fn get_evm_env(&self, block_hash: H256) -> RethResult<(CfgEnv, BlockEnv)> {
|
||||
let (response_tx, rx) = oneshot::channel();
|
||||
let _ = self.to_service.send(CacheAction::GetEnv { block_hash, response_tx });
|
||||
rx.await.map_err(|_| ProviderError::CacheServiceUnavailable)?
|
||||
@ -248,7 +251,7 @@ where
|
||||
Provider: StateProviderFactory + BlockReader + EvmEnvProvider + Clone + Unpin + 'static,
|
||||
Tasks: TaskSpawner + Clone + 'static,
|
||||
{
|
||||
fn on_new_block(&mut self, block_hash: H256, res: Result<Option<Block>>) {
|
||||
fn on_new_block(&mut self, block_hash: H256, res: RethResult<Option<Block>>) {
|
||||
if let Some(queued) = self.full_block_cache.remove(&block_hash) {
|
||||
// send the response to queued senders
|
||||
for tx in queued {
|
||||
@ -271,7 +274,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
fn on_new_receipts(&mut self, block_hash: H256, res: Result<Option<Vec<Receipt>>>) {
|
||||
fn on_new_receipts(&mut self, block_hash: H256, res: RethResult<Option<Vec<Receipt>>>) {
|
||||
if let Some(queued) = self.receipts_cache.remove(&block_hash) {
|
||||
// send the response to queued senders
|
||||
for tx in queued {
|
||||
@ -454,9 +457,9 @@ enum CacheAction {
|
||||
GetBlockTransactions { block_hash: H256, response_tx: BlockTransactionsResponseSender },
|
||||
GetEnv { block_hash: H256, response_tx: EnvResponseSender },
|
||||
GetReceipts { block_hash: H256, response_tx: ReceiptsResponseSender },
|
||||
BlockResult { block_hash: H256, res: Result<Option<Block>> },
|
||||
ReceiptsResult { block_hash: H256, res: Result<Option<Vec<Receipt>>> },
|
||||
EnvResult { block_hash: H256, res: Box<Result<(CfgEnv, BlockEnv)>> },
|
||||
BlockResult { block_hash: H256, res: RethResult<Option<Block>> },
|
||||
ReceiptsResult { block_hash: H256, res: RethResult<Option<Vec<Receipt>>> },
|
||||
EnvResult { block_hash: H256, res: Box<RethResult<(CfgEnv, BlockEnv)>> },
|
||||
CacheNewCanonicalChain { blocks: Vec<SealedBlock>, receipts: Vec<BlockReceipts> },
|
||||
}
|
||||
|
||||
|
||||
@ -5,6 +5,7 @@ use jsonrpsee::{
|
||||
core::Error as RpcError,
|
||||
types::{error::CALL_EXECUTION_FAILED_CODE, ErrorObject},
|
||||
};
|
||||
use reth_interfaces::RethError;
|
||||
use reth_primitives::{abi::decode_revert_reason, Address, Bytes, U256};
|
||||
use reth_revm::tracing::js::JsInspectorError;
|
||||
use reth_rpc_types::{error::EthRpcErrorCode, BlockError, CallInputError};
|
||||
@ -61,7 +62,7 @@ pub enum EthApiError {
|
||||
BothStateAndStateDiffInOverride(Address),
|
||||
/// Other internal error
|
||||
#[error(transparent)]
|
||||
Internal(reth_interfaces::Error),
|
||||
Internal(RethError),
|
||||
/// Error related to signing
|
||||
#[error(transparent)]
|
||||
Signing(#[from] SignError),
|
||||
@ -153,10 +154,10 @@ impl From<JsInspectorError> for EthApiError {
|
||||
}
|
||||
}
|
||||
|
||||
impl From<reth_interfaces::Error> for EthApiError {
|
||||
fn from(error: reth_interfaces::Error) -> Self {
|
||||
impl From<RethError> for EthApiError {
|
||||
fn from(error: RethError) -> Self {
|
||||
match error {
|
||||
reth_interfaces::Error::Provider(err) => err.into(),
|
||||
RethError::Provider(err) => err.into(),
|
||||
err => EthApiError::Internal(err),
|
||||
}
|
||||
}
|
||||
|
||||
@ -9,6 +9,7 @@ use crate::{
|
||||
};
|
||||
use async_trait::async_trait;
|
||||
use jsonrpsee::{core::RpcResult, server::IdProvider};
|
||||
use reth_interfaces::RethError;
|
||||
use reth_primitives::{BlockHashOrNumber, Receipt, SealedBlock};
|
||||
use reth_provider::{BlockIdReader, BlockReader, EvmEnvProvider};
|
||||
use reth_rpc_api::EthFilterApiServer;
|
||||
@ -458,8 +459,8 @@ impl From<FilterError> for jsonrpsee::types::error::ErrorObject<'static> {
|
||||
}
|
||||
}
|
||||
|
||||
impl From<reth_interfaces::Error> for FilterError {
|
||||
fn from(err: reth_interfaces::Error) -> Self {
|
||||
impl From<RethError> for FilterError {
|
||||
fn from(err: RethError) -> Self {
|
||||
FilterError::EthAPIError(err.into())
|
||||
}
|
||||
}
|
||||
|
||||
@ -2,7 +2,7 @@
|
||||
|
||||
use crate::eth::error::EthApiError;
|
||||
use jsonrpsee::core::RpcResult;
|
||||
use reth_interfaces::Result as RethResult;
|
||||
use reth_interfaces::RethResult;
|
||||
use reth_primitives::Block;
|
||||
use std::fmt::Display;
|
||||
|
||||
@ -101,7 +101,7 @@ macro_rules! impl_to_rpc_result {
|
||||
};
|
||||
}
|
||||
|
||||
impl_to_rpc_result!(reth_interfaces::Error);
|
||||
impl_to_rpc_result!(reth_interfaces::RethError);
|
||||
impl_to_rpc_result!(reth_network_api::NetworkError);
|
||||
|
||||
/// An extension to used to apply error conversions to various result types
|
||||
@ -174,16 +174,17 @@ pub(crate) fn rpc_err(
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
use reth_interfaces::RethError;
|
||||
|
||||
fn assert_rpc_result<Ok, Err, T: ToRpcResult<Ok, Err>>() {}
|
||||
|
||||
fn to_reth_err<Ok>(o: Ok) -> reth_interfaces::Result<Ok> {
|
||||
fn to_reth_err<Ok>(o: Ok) -> RethResult<Ok> {
|
||||
Ok(o)
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn can_convert_rpc() {
|
||||
assert_rpc_result::<(), reth_interfaces::Error, reth_interfaces::Result<()>>();
|
||||
assert_rpc_result::<(), RethError, RethResult<()>>();
|
||||
let res = to_reth_err(100);
|
||||
|
||||
let rpc_res = res.map_internal_err(|_| "This is a message");
|
||||
|
||||
@ -1,7 +1,7 @@
|
||||
use crate::eth::error::{EthApiError, EthResult};
|
||||
use async_trait::async_trait;
|
||||
use jsonrpsee::core::RpcResult;
|
||||
use reth_interfaces::Result;
|
||||
use reth_interfaces::RethResult;
|
||||
use reth_primitives::{Address, BlockId, U256};
|
||||
use reth_provider::{BlockReaderIdExt, ChangeSetReader, StateProviderFactory};
|
||||
use reth_rpc_api::RethApiServer;
|
||||
@ -70,7 +70,7 @@ where
|
||||
let accounts_before = self.provider().account_block_changeset(block_number)?;
|
||||
let hash_map = accounts_before.iter().try_fold(
|
||||
HashMap::new(),
|
||||
|mut hash_map, account_before| -> Result<_> {
|
||||
|mut hash_map, account_before| -> RethResult<_> {
|
||||
let current_balance = state.account_balance(account_before.address)?;
|
||||
let prev_balance = account_before.info.map(|info| info.balance);
|
||||
if current_balance != prev_balance {
|
||||
|
||||
@ -1,7 +1,7 @@
|
||||
use crate::pipeline::PipelineEvent;
|
||||
use reth_interfaces::{
|
||||
consensus, db::DatabaseError as DbError, executor, p2p::error::DownloadError,
|
||||
provider::ProviderError,
|
||||
provider::ProviderError, RethError,
|
||||
};
|
||||
use reth_primitives::SealedHeader;
|
||||
use thiserror::Error;
|
||||
@ -67,7 +67,7 @@ pub enum StageError {
|
||||
Download(#[from] DownloadError),
|
||||
/// Internal error
|
||||
#[error(transparent)]
|
||||
Internal(#[from] reth_interfaces::Error),
|
||||
Internal(#[from] RethError),
|
||||
/// The stage encountered a recoverable error.
|
||||
///
|
||||
/// These types of errors are caught by the [Pipeline][crate::Pipeline] and trigger a restart
|
||||
@ -107,7 +107,7 @@ pub enum PipelineError {
|
||||
Database(#[from] DbError),
|
||||
/// The pipeline encountered an irrecoverable error in one of the stages.
|
||||
#[error("An interface error occurred.")]
|
||||
Interface(#[from] reth_interfaces::Error),
|
||||
Interface(#[from] RethError),
|
||||
/// The pipeline encountered an error while trying to send an event.
|
||||
#[error("The pipeline encountered an error while trying to send an event.")]
|
||||
Channel(#[from] SendError<PipelineEvent>),
|
||||
|
||||
@ -1,6 +1,7 @@
|
||||
use super::TestTransaction;
|
||||
use crate::{ExecInput, ExecOutput, Stage, StageError, UnwindInput, UnwindOutput};
|
||||
use reth_db::DatabaseEnv;
|
||||
use reth_interfaces::{db::DatabaseError, RethError};
|
||||
use reth_primitives::MAINNET;
|
||||
use reth_provider::ProviderFactory;
|
||||
use std::{borrow::Borrow, sync::Arc};
|
||||
@ -9,11 +10,11 @@ use tokio::sync::oneshot;
|
||||
#[derive(thiserror::Error, Debug)]
|
||||
pub(crate) enum TestRunnerError {
|
||||
#[error("Database error occurred.")]
|
||||
Database(#[from] reth_interfaces::db::DatabaseError),
|
||||
Database(#[from] DatabaseError),
|
||||
#[error("Internal runner error occurred.")]
|
||||
Internal(#[from] Box<dyn std::error::Error>),
|
||||
#[error("Internal interface error occurred.")]
|
||||
Interface(#[from] reth_interfaces::Error),
|
||||
Interface(#[from] RethError),
|
||||
}
|
||||
|
||||
/// A generic test runner for stages.
|
||||
|
||||
@ -9,7 +9,7 @@ use reth_db::{
|
||||
transaction::{DbTx, DbTxGAT, DbTxMut, DbTxMutGAT},
|
||||
DatabaseEnv, DatabaseError as DbError,
|
||||
};
|
||||
use reth_interfaces::test_utils::generators::ChangeSet;
|
||||
use reth_interfaces::{test_utils::generators::ChangeSet, RethResult};
|
||||
use reth_primitives::{
|
||||
keccak256, Account, Address, BlockNumber, Receipt, SealedBlock, SealedHeader, StorageEntry,
|
||||
TxHash, TxNumber, H256, MAINNET, U256,
|
||||
@ -379,11 +379,7 @@ impl TestTransaction {
|
||||
})
|
||||
}
|
||||
|
||||
pub fn insert_history<I>(
|
||||
&self,
|
||||
changesets: I,
|
||||
block_offset: Option<u64>,
|
||||
) -> reth_interfaces::Result<()>
|
||||
pub fn insert_history<I>(&self, changesets: I, block_offset: Option<u64>) -> RethResult<()>
|
||||
where
|
||||
I: IntoIterator<Item = ChangeSet>,
|
||||
{
|
||||
|
||||
@ -1,7 +1,7 @@
|
||||
//! Contains [Chain], a chain of blocks and their final state.
|
||||
|
||||
use crate::bundle_state::BundleStateWithReceipts;
|
||||
use reth_interfaces::{executor::BlockExecutionError, Error};
|
||||
use reth_interfaces::{executor::BlockExecutionError, RethResult};
|
||||
use reth_primitives::{
|
||||
BlockHash, BlockNumHash, BlockNumber, ForkBlock, Receipt, SealedBlock, SealedBlockWithSenders,
|
||||
SealedHeader, TransactionSigned, TxHash,
|
||||
@ -163,7 +163,7 @@ impl Chain {
|
||||
/// Merge two chains by appending the given chain into the current one.
|
||||
///
|
||||
/// The state of accounts for this chain is set to the state of the newest chain.
|
||||
pub fn append_chain(&mut self, chain: Chain) -> Result<(), Error> {
|
||||
pub fn append_chain(&mut self, chain: Chain) -> RethResult<()> {
|
||||
let chain_tip = self.tip();
|
||||
if chain_tip.hash != chain.fork_block_hash() {
|
||||
return Err(BlockExecutionError::AppendChainDoesntConnect {
|
||||
|
||||
@ -2,7 +2,7 @@ use crate::{
|
||||
bundle_state::BundleStateWithReceipts, AccountReader, BlockHashReader, BundleStateDataProvider,
|
||||
StateProvider, StateRootProvider,
|
||||
};
|
||||
use reth_interfaces::{provider::ProviderError, Result};
|
||||
use reth_interfaces::{provider::ProviderError, RethResult};
|
||||
use reth_primitives::{Account, Address, BlockNumber, Bytecode, Bytes, H256};
|
||||
|
||||
/// A state provider that either resolves to data in a wrapped [`crate::BundleStateWithReceipts`],
|
||||
@ -26,7 +26,7 @@ impl<SP: StateProvider, BSDP: BundleStateDataProvider> BundleStateProvider<SP, B
|
||||
impl<SP: StateProvider, BSDP: BundleStateDataProvider> BlockHashReader
|
||||
for BundleStateProvider<SP, BSDP>
|
||||
{
|
||||
fn block_hash(&self, block_number: BlockNumber) -> Result<Option<H256>> {
|
||||
fn block_hash(&self, block_number: BlockNumber) -> RethResult<Option<H256>> {
|
||||
let block_hash = self.post_state_data_provider.block_hash(block_number);
|
||||
if block_hash.is_some() {
|
||||
return Ok(block_hash)
|
||||
@ -34,7 +34,11 @@ impl<SP: StateProvider, BSDP: BundleStateDataProvider> BlockHashReader
|
||||
self.state_provider.block_hash(block_number)
|
||||
}
|
||||
|
||||
fn canonical_hashes_range(&self, _start: BlockNumber, _end: BlockNumber) -> Result<Vec<H256>> {
|
||||
fn canonical_hashes_range(
|
||||
&self,
|
||||
_start: BlockNumber,
|
||||
_end: BlockNumber,
|
||||
) -> RethResult<Vec<H256>> {
|
||||
unimplemented!()
|
||||
}
|
||||
}
|
||||
@ -42,7 +46,7 @@ impl<SP: StateProvider, BSDP: BundleStateDataProvider> BlockHashReader
|
||||
impl<SP: StateProvider, BSDP: BundleStateDataProvider> AccountReader
|
||||
for BundleStateProvider<SP, BSDP>
|
||||
{
|
||||
fn basic_account(&self, address: Address) -> Result<Option<Account>> {
|
||||
fn basic_account(&self, address: Address) -> RethResult<Option<Account>> {
|
||||
if let Some(account) = self.post_state_data_provider.state().account(&address) {
|
||||
Ok(account)
|
||||
} else {
|
||||
@ -54,7 +58,7 @@ impl<SP: StateProvider, BSDP: BundleStateDataProvider> AccountReader
|
||||
impl<SP: StateProvider, BSDP: BundleStateDataProvider> StateRootProvider
|
||||
for BundleStateProvider<SP, BSDP>
|
||||
{
|
||||
fn state_root(&self, post_state: BundleStateWithReceipts) -> Result<H256> {
|
||||
fn state_root(&self, post_state: BundleStateWithReceipts) -> RethResult<H256> {
|
||||
let mut state = self.post_state_data_provider.state().clone();
|
||||
state.extend(post_state);
|
||||
self.state_provider.state_root(state)
|
||||
@ -68,7 +72,7 @@ impl<SP: StateProvider, BSDP: BundleStateDataProvider> StateProvider
|
||||
&self,
|
||||
account: Address,
|
||||
storage_key: reth_primitives::StorageKey,
|
||||
) -> Result<Option<reth_primitives::StorageValue>> {
|
||||
) -> RethResult<Option<reth_primitives::StorageValue>> {
|
||||
let u256_storage_key = storage_key.into();
|
||||
if let Some(value) =
|
||||
self.post_state_data_provider.state().storage(&account, u256_storage_key)
|
||||
@ -79,7 +83,7 @@ impl<SP: StateProvider, BSDP: BundleStateDataProvider> StateProvider
|
||||
self.state_provider.storage(account, storage_key)
|
||||
}
|
||||
|
||||
fn bytecode_by_hash(&self, code_hash: H256) -> Result<Option<Bytecode>> {
|
||||
fn bytecode_by_hash(&self, code_hash: H256) -> RethResult<Option<Bytecode>> {
|
||||
if let Some(bytecode) = self.post_state_data_provider.state().bytecode(&code_hash) {
|
||||
return Ok(Some(bytecode))
|
||||
}
|
||||
@ -91,7 +95,7 @@ impl<SP: StateProvider, BSDP: BundleStateDataProvider> StateProvider
|
||||
&self,
|
||||
_address: Address,
|
||||
_keys: &[H256],
|
||||
) -> Result<(Vec<Bytes>, H256, Vec<Vec<Bytes>>)> {
|
||||
) -> RethResult<(Vec<Bytes>, H256, Vec<Vec<Bytes>>)> {
|
||||
Err(ProviderError::StateRootNotAvailableForHistoricalBlock.into())
|
||||
}
|
||||
}
|
||||
|
||||
@ -6,7 +6,7 @@ use crate::{
|
||||
TransactionsProvider, WithdrawalsProvider,
|
||||
};
|
||||
use reth_db::{database::Database, init_db, models::StoredBlockBodyIndices, DatabaseEnv};
|
||||
use reth_interfaces::Result;
|
||||
use reth_interfaces::{RethError, RethResult};
|
||||
use reth_primitives::{
|
||||
stage::{StageCheckpoint, StageId},
|
||||
Address, Block, BlockHash, BlockHashOrNumber, BlockNumber, BlockWithSenders, ChainInfo,
|
||||
@ -37,7 +37,7 @@ impl<DB: Database> ProviderFactory<DB> {
|
||||
/// Returns a provider with a created `DbTx` inside, which allows fetching data from the
|
||||
/// database using different types of providers. Example: [`HeaderProvider`]
|
||||
/// [`BlockHashReader`]. This may fail if the inner read database transaction fails to open.
|
||||
pub fn provider(&self) -> Result<DatabaseProviderRO<'_, DB>> {
|
||||
pub fn provider(&self) -> RethResult<DatabaseProviderRO<'_, DB>> {
|
||||
Ok(DatabaseProvider::new(self.db.tx()?, self.chain_spec.clone()))
|
||||
}
|
||||
|
||||
@ -45,7 +45,7 @@ impl<DB: Database> ProviderFactory<DB> {
|
||||
/// data from the database using different types of providers. Example: [`HeaderProvider`]
|
||||
/// [`BlockHashReader`]. This may fail if the inner read/write database transaction fails to
|
||||
/// open.
|
||||
pub fn provider_rw(&self) -> Result<DatabaseProviderRW<'_, DB>> {
|
||||
pub fn provider_rw(&self) -> RethResult<DatabaseProviderRW<'_, DB>> {
|
||||
Ok(DatabaseProviderRW(DatabaseProvider::new_rw(self.db.tx_mut()?, self.chain_spec.clone())))
|
||||
}
|
||||
}
|
||||
@ -64,10 +64,9 @@ impl<DB: Database> ProviderFactory<DB> {
|
||||
path: P,
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
log_level: Option<LogLevel>,
|
||||
) -> Result<ProviderFactory<DatabaseEnv>> {
|
||||
) -> RethResult<ProviderFactory<DatabaseEnv>> {
|
||||
Ok(ProviderFactory::<DatabaseEnv> {
|
||||
db: init_db(path, log_level)
|
||||
.map_err(|e| reth_interfaces::Error::Custom(e.to_string()))?,
|
||||
db: init_db(path, log_level).map_err(|e| RethError::Custom(e.to_string()))?,
|
||||
chain_spec,
|
||||
})
|
||||
}
|
||||
@ -81,7 +80,7 @@ impl<DB: Clone> Clone for ProviderFactory<DB> {
|
||||
|
||||
impl<DB: Database> ProviderFactory<DB> {
|
||||
/// Storage provider for latest block
|
||||
pub fn latest(&self) -> Result<StateProviderBox<'_>> {
|
||||
pub fn latest(&self) -> RethResult<StateProviderBox<'_>> {
|
||||
trace!(target: "providers::db", "Returning latest state provider");
|
||||
Ok(Box::new(LatestStateProvider::new(self.db.tx()?)))
|
||||
}
|
||||
@ -90,7 +89,7 @@ impl<DB: Database> ProviderFactory<DB> {
|
||||
fn state_provider_by_block_number(
|
||||
&self,
|
||||
mut block_number: BlockNumber,
|
||||
) -> Result<StateProviderBox<'_>> {
|
||||
) -> RethResult<StateProviderBox<'_>> {
|
||||
let provider = self.provider()?;
|
||||
|
||||
if block_number == provider.best_block_number().unwrap_or_default() &&
|
||||
@ -133,14 +132,14 @@ impl<DB: Database> ProviderFactory<DB> {
|
||||
pub fn history_by_block_number(
|
||||
&self,
|
||||
block_number: BlockNumber,
|
||||
) -> Result<StateProviderBox<'_>> {
|
||||
) -> RethResult<StateProviderBox<'_>> {
|
||||
let state_provider = self.state_provider_by_block_number(block_number)?;
|
||||
trace!(target: "providers::db", ?block_number, "Returning historical state provider for block number");
|
||||
Ok(state_provider)
|
||||
}
|
||||
|
||||
/// Storage provider for state at that given block hash
|
||||
pub fn history_by_block_hash(&self, block_hash: BlockHash) -> Result<StateProviderBox<'_>> {
|
||||
pub fn history_by_block_hash(&self, block_hash: BlockHash) -> RethResult<StateProviderBox<'_>> {
|
||||
let block_number = self
|
||||
.provider()?
|
||||
.block_number(block_hash)?
|
||||
@ -153,164 +152,174 @@ impl<DB: Database> ProviderFactory<DB> {
|
||||
}
|
||||
|
||||
impl<DB: Database> HeaderProvider for ProviderFactory<DB> {
|
||||
fn header(&self, block_hash: &BlockHash) -> Result<Option<Header>> {
|
||||
fn header(&self, block_hash: &BlockHash) -> RethResult<Option<Header>> {
|
||||
self.provider()?.header(block_hash)
|
||||
}
|
||||
|
||||
fn header_by_number(&self, num: BlockNumber) -> Result<Option<Header>> {
|
||||
fn header_by_number(&self, num: BlockNumber) -> RethResult<Option<Header>> {
|
||||
self.provider()?.header_by_number(num)
|
||||
}
|
||||
|
||||
fn header_td(&self, hash: &BlockHash) -> Result<Option<U256>> {
|
||||
fn header_td(&self, hash: &BlockHash) -> RethResult<Option<U256>> {
|
||||
self.provider()?.header_td(hash)
|
||||
}
|
||||
|
||||
fn header_td_by_number(&self, number: BlockNumber) -> Result<Option<U256>> {
|
||||
fn header_td_by_number(&self, number: BlockNumber) -> RethResult<Option<U256>> {
|
||||
self.provider()?.header_td_by_number(number)
|
||||
}
|
||||
|
||||
fn headers_range(&self, range: impl RangeBounds<BlockNumber>) -> Result<Vec<Header>> {
|
||||
fn headers_range(&self, range: impl RangeBounds<BlockNumber>) -> RethResult<Vec<Header>> {
|
||||
self.provider()?.headers_range(range)
|
||||
}
|
||||
|
||||
fn sealed_headers_range(
|
||||
&self,
|
||||
range: impl RangeBounds<BlockNumber>,
|
||||
) -> Result<Vec<SealedHeader>> {
|
||||
) -> RethResult<Vec<SealedHeader>> {
|
||||
self.provider()?.sealed_headers_range(range)
|
||||
}
|
||||
|
||||
fn sealed_header(&self, number: BlockNumber) -> Result<Option<SealedHeader>> {
|
||||
fn sealed_header(&self, number: BlockNumber) -> RethResult<Option<SealedHeader>> {
|
||||
self.provider()?.sealed_header(number)
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database> BlockHashReader for ProviderFactory<DB> {
|
||||
fn block_hash(&self, number: u64) -> Result<Option<H256>> {
|
||||
fn block_hash(&self, number: u64) -> RethResult<Option<H256>> {
|
||||
self.provider()?.block_hash(number)
|
||||
}
|
||||
|
||||
fn canonical_hashes_range(&self, start: BlockNumber, end: BlockNumber) -> Result<Vec<H256>> {
|
||||
fn canonical_hashes_range(
|
||||
&self,
|
||||
start: BlockNumber,
|
||||
end: BlockNumber,
|
||||
) -> RethResult<Vec<H256>> {
|
||||
self.provider()?.canonical_hashes_range(start, end)
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database> BlockNumReader for ProviderFactory<DB> {
|
||||
fn chain_info(&self) -> Result<ChainInfo> {
|
||||
fn chain_info(&self) -> RethResult<ChainInfo> {
|
||||
self.provider()?.chain_info()
|
||||
}
|
||||
|
||||
fn best_block_number(&self) -> Result<BlockNumber> {
|
||||
fn best_block_number(&self) -> RethResult<BlockNumber> {
|
||||
self.provider()?.best_block_number()
|
||||
}
|
||||
|
||||
fn last_block_number(&self) -> Result<BlockNumber> {
|
||||
fn last_block_number(&self) -> RethResult<BlockNumber> {
|
||||
self.provider()?.last_block_number()
|
||||
}
|
||||
|
||||
fn block_number(&self, hash: H256) -> Result<Option<BlockNumber>> {
|
||||
fn block_number(&self, hash: H256) -> RethResult<Option<BlockNumber>> {
|
||||
self.provider()?.block_number(hash)
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database> BlockReader for ProviderFactory<DB> {
|
||||
fn find_block_by_hash(&self, hash: H256, source: BlockSource) -> Result<Option<Block>> {
|
||||
fn find_block_by_hash(&self, hash: H256, source: BlockSource) -> RethResult<Option<Block>> {
|
||||
self.provider()?.find_block_by_hash(hash, source)
|
||||
}
|
||||
|
||||
fn block(&self, id: BlockHashOrNumber) -> Result<Option<Block>> {
|
||||
fn block(&self, id: BlockHashOrNumber) -> RethResult<Option<Block>> {
|
||||
self.provider()?.block(id)
|
||||
}
|
||||
|
||||
fn pending_block(&self) -> Result<Option<SealedBlock>> {
|
||||
fn pending_block(&self) -> RethResult<Option<SealedBlock>> {
|
||||
self.provider()?.pending_block()
|
||||
}
|
||||
|
||||
fn pending_block_and_receipts(&self) -> Result<Option<(SealedBlock, Vec<Receipt>)>> {
|
||||
fn pending_block_and_receipts(&self) -> RethResult<Option<(SealedBlock, Vec<Receipt>)>> {
|
||||
self.provider()?.pending_block_and_receipts()
|
||||
}
|
||||
|
||||
fn ommers(&self, id: BlockHashOrNumber) -> Result<Option<Vec<Header>>> {
|
||||
fn ommers(&self, id: BlockHashOrNumber) -> RethResult<Option<Vec<Header>>> {
|
||||
self.provider()?.ommers(id)
|
||||
}
|
||||
|
||||
fn block_body_indices(&self, number: BlockNumber) -> Result<Option<StoredBlockBodyIndices>> {
|
||||
fn block_body_indices(
|
||||
&self,
|
||||
number: BlockNumber,
|
||||
) -> RethResult<Option<StoredBlockBodyIndices>> {
|
||||
self.provider()?.block_body_indices(number)
|
||||
}
|
||||
|
||||
fn block_with_senders(&self, number: BlockNumber) -> Result<Option<BlockWithSenders>> {
|
||||
fn block_with_senders(&self, number: BlockNumber) -> RethResult<Option<BlockWithSenders>> {
|
||||
self.provider()?.block_with_senders(number)
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database> TransactionsProvider for ProviderFactory<DB> {
|
||||
fn transaction_id(&self, tx_hash: TxHash) -> Result<Option<TxNumber>> {
|
||||
fn transaction_id(&self, tx_hash: TxHash) -> RethResult<Option<TxNumber>> {
|
||||
self.provider()?.transaction_id(tx_hash)
|
||||
}
|
||||
|
||||
fn transaction_by_id(&self, id: TxNumber) -> Result<Option<TransactionSigned>> {
|
||||
fn transaction_by_id(&self, id: TxNumber) -> RethResult<Option<TransactionSigned>> {
|
||||
self.provider()?.transaction_by_id(id)
|
||||
}
|
||||
|
||||
fn transaction_by_id_no_hash(&self, id: TxNumber) -> Result<Option<TransactionSignedNoHash>> {
|
||||
fn transaction_by_id_no_hash(
|
||||
&self,
|
||||
id: TxNumber,
|
||||
) -> RethResult<Option<TransactionSignedNoHash>> {
|
||||
self.provider()?.transaction_by_id_no_hash(id)
|
||||
}
|
||||
|
||||
fn transaction_by_hash(&self, hash: TxHash) -> Result<Option<TransactionSigned>> {
|
||||
fn transaction_by_hash(&self, hash: TxHash) -> RethResult<Option<TransactionSigned>> {
|
||||
self.provider()?.transaction_by_hash(hash)
|
||||
}
|
||||
|
||||
fn transaction_by_hash_with_meta(
|
||||
&self,
|
||||
tx_hash: TxHash,
|
||||
) -> Result<Option<(TransactionSigned, TransactionMeta)>> {
|
||||
) -> RethResult<Option<(TransactionSigned, TransactionMeta)>> {
|
||||
self.provider()?.transaction_by_hash_with_meta(tx_hash)
|
||||
}
|
||||
|
||||
fn transaction_block(&self, id: TxNumber) -> Result<Option<BlockNumber>> {
|
||||
fn transaction_block(&self, id: TxNumber) -> RethResult<Option<BlockNumber>> {
|
||||
self.provider()?.transaction_block(id)
|
||||
}
|
||||
|
||||
fn transactions_by_block(
|
||||
&self,
|
||||
id: BlockHashOrNumber,
|
||||
) -> Result<Option<Vec<TransactionSigned>>> {
|
||||
) -> RethResult<Option<Vec<TransactionSigned>>> {
|
||||
self.provider()?.transactions_by_block(id)
|
||||
}
|
||||
|
||||
fn transactions_by_block_range(
|
||||
&self,
|
||||
range: impl RangeBounds<BlockNumber>,
|
||||
) -> Result<Vec<Vec<TransactionSigned>>> {
|
||||
) -> RethResult<Vec<Vec<TransactionSigned>>> {
|
||||
self.provider()?.transactions_by_block_range(range)
|
||||
}
|
||||
|
||||
fn transactions_by_tx_range(
|
||||
&self,
|
||||
range: impl RangeBounds<TxNumber>,
|
||||
) -> Result<Vec<TransactionSignedNoHash>> {
|
||||
) -> RethResult<Vec<TransactionSignedNoHash>> {
|
||||
self.provider()?.transactions_by_tx_range(range)
|
||||
}
|
||||
|
||||
fn senders_by_tx_range(&self, range: impl RangeBounds<TxNumber>) -> Result<Vec<Address>> {
|
||||
fn senders_by_tx_range(&self, range: impl RangeBounds<TxNumber>) -> RethResult<Vec<Address>> {
|
||||
self.provider()?.senders_by_tx_range(range)
|
||||
}
|
||||
|
||||
fn transaction_sender(&self, id: TxNumber) -> Result<Option<Address>> {
|
||||
fn transaction_sender(&self, id: TxNumber) -> RethResult<Option<Address>> {
|
||||
self.provider()?.transaction_sender(id)
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database> ReceiptProvider for ProviderFactory<DB> {
|
||||
fn receipt(&self, id: TxNumber) -> Result<Option<Receipt>> {
|
||||
fn receipt(&self, id: TxNumber) -> RethResult<Option<Receipt>> {
|
||||
self.provider()?.receipt(id)
|
||||
}
|
||||
|
||||
fn receipt_by_hash(&self, hash: TxHash) -> Result<Option<Receipt>> {
|
||||
fn receipt_by_hash(&self, hash: TxHash) -> RethResult<Option<Receipt>> {
|
||||
self.provider()?.receipt_by_hash(hash)
|
||||
}
|
||||
|
||||
fn receipts_by_block(&self, block: BlockHashOrNumber) -> Result<Option<Vec<Receipt>>> {
|
||||
fn receipts_by_block(&self, block: BlockHashOrNumber) -> RethResult<Option<Vec<Receipt>>> {
|
||||
self.provider()?.receipts_by_block(block)
|
||||
}
|
||||
}
|
||||
@ -320,21 +329,21 @@ impl<DB: Database> WithdrawalsProvider for ProviderFactory<DB> {
|
||||
&self,
|
||||
id: BlockHashOrNumber,
|
||||
timestamp: u64,
|
||||
) -> Result<Option<Vec<Withdrawal>>> {
|
||||
) -> RethResult<Option<Vec<Withdrawal>>> {
|
||||
self.provider()?.withdrawals_by_block(id, timestamp)
|
||||
}
|
||||
|
||||
fn latest_withdrawal(&self) -> Result<Option<Withdrawal>> {
|
||||
fn latest_withdrawal(&self) -> RethResult<Option<Withdrawal>> {
|
||||
self.provider()?.latest_withdrawal()
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB: Database> StageCheckpointReader for ProviderFactory<DB> {
|
||||
fn get_stage_checkpoint(&self, id: StageId) -> Result<Option<StageCheckpoint>> {
|
||||
fn get_stage_checkpoint(&self, id: StageId) -> RethResult<Option<StageCheckpoint>> {
|
||||
self.provider()?.get_stage_checkpoint(id)
|
||||
}
|
||||
|
||||
fn get_stage_checkpoint_progress(&self, id: StageId) -> Result<Option<Vec<u8>>> {
|
||||
fn get_stage_checkpoint_progress(&self, id: StageId) -> RethResult<Option<Vec<u8>>> {
|
||||
self.provider()?.get_stage_checkpoint_progress(id)
|
||||
}
|
||||
}
|
||||
@ -345,7 +354,7 @@ impl<DB: Database> EvmEnvProvider for ProviderFactory<DB> {
|
||||
cfg: &mut CfgEnv,
|
||||
block_env: &mut BlockEnv,
|
||||
at: BlockHashOrNumber,
|
||||
) -> Result<()> {
|
||||
) -> RethResult<()> {
|
||||
self.provider()?.fill_env_at(cfg, block_env, at)
|
||||
}
|
||||
|
||||
@ -354,23 +363,27 @@ impl<DB: Database> EvmEnvProvider for ProviderFactory<DB> {
|
||||
cfg: &mut CfgEnv,
|
||||
block_env: &mut BlockEnv,
|
||||
header: &Header,
|
||||
) -> Result<()> {
|
||||
) -> RethResult<()> {
|
||||
self.provider()?.fill_env_with_header(cfg, block_env, header)
|
||||
}
|
||||
|
||||
fn fill_block_env_at(&self, block_env: &mut BlockEnv, at: BlockHashOrNumber) -> Result<()> {
|
||||
fn fill_block_env_at(&self, block_env: &mut BlockEnv, at: BlockHashOrNumber) -> RethResult<()> {
|
||||
self.provider()?.fill_block_env_at(block_env, at)
|
||||
}
|
||||
|
||||
fn fill_block_env_with_header(&self, block_env: &mut BlockEnv, header: &Header) -> Result<()> {
|
||||
fn fill_block_env_with_header(
|
||||
&self,
|
||||
block_env: &mut BlockEnv,
|
||||
header: &Header,
|
||||
) -> RethResult<()> {
|
||||
self.provider()?.fill_block_env_with_header(block_env, header)
|
||||
}
|
||||
|
||||
fn fill_cfg_env_at(&self, cfg: &mut CfgEnv, at: BlockHashOrNumber) -> Result<()> {
|
||||
fn fill_cfg_env_at(&self, cfg: &mut CfgEnv, at: BlockHashOrNumber) -> RethResult<()> {
|
||||
self.provider()?.fill_cfg_env_at(cfg, at)
|
||||
}
|
||||
|
||||
fn fill_cfg_env_with_header(&self, cfg: &mut CfgEnv, header: &Header) -> Result<()> {
|
||||
fn fill_cfg_env_with_header(&self, cfg: &mut CfgEnv, header: &Header) -> RethResult<()> {
|
||||
self.provider()?.fill_cfg_env_with_header(cfg, header)
|
||||
}
|
||||
}
|
||||
@ -385,7 +398,7 @@ where
|
||||
}
|
||||
|
||||
impl<DB: Database> PruneCheckpointReader for ProviderFactory<DB> {
|
||||
fn get_prune_checkpoint(&self, part: PrunePart) -> Result<Option<PruneCheckpoint>> {
|
||||
fn get_prune_checkpoint(&self, part: PrunePart) -> RethResult<Option<PruneCheckpoint>> {
|
||||
self.provider()?.get_prune_checkpoint(part)
|
||||
}
|
||||
}
|
||||
|
||||
@ -24,7 +24,7 @@ use reth_db::{
|
||||
};
|
||||
use reth_interfaces::{
|
||||
executor::{BlockExecutionError, BlockValidationError},
|
||||
Result,
|
||||
RethResult,
|
||||
};
|
||||
use reth_primitives::{
|
||||
keccak256,
|
||||
@ -77,7 +77,7 @@ impl<'this, DB: Database> DerefMut for DatabaseProviderRW<'this, DB> {
|
||||
|
||||
impl<'this, DB: Database> DatabaseProviderRW<'this, DB> {
|
||||
/// Commit database transaction
|
||||
pub fn commit(self) -> Result<bool> {
|
||||
pub fn commit(self) -> RethResult<bool> {
|
||||
self.0.commit()
|
||||
}
|
||||
|
||||
@ -125,7 +125,7 @@ fn unwind_history_shards<'a, S, T, C>(
|
||||
start_key: T::Key,
|
||||
block_number: BlockNumber,
|
||||
mut shard_belongs_to_key: impl FnMut(&T::Key) -> bool,
|
||||
) -> Result<Vec<usize>>
|
||||
) -> RethResult<Vec<usize>>
|
||||
where
|
||||
T: Table<Value = BlockNumberList>,
|
||||
T::Key: AsRef<ShardedKey<S>>,
|
||||
@ -191,7 +191,7 @@ impl<'this, TX: DbTx<'this>> DatabaseProvider<'this, TX> {
|
||||
|
||||
impl<'this, TX: DbTxMut<'this> + DbTx<'this>> DatabaseProvider<'this, TX> {
|
||||
/// Commit database transaction.
|
||||
pub fn commit(self) -> Result<bool> {
|
||||
pub fn commit(self) -> RethResult<bool> {
|
||||
Ok(self.tx.commit()?)
|
||||
}
|
||||
|
||||
@ -224,7 +224,7 @@ impl<'this, TX: DbTxMut<'this> + DbTx<'this>> DatabaseProvider<'this, TX> {
|
||||
fn unwind_or_peek_state<const UNWIND: bool>(
|
||||
&self,
|
||||
range: RangeInclusive<BlockNumber>,
|
||||
) -> Result<BundleStateWithReceipts> {
|
||||
) -> RethResult<BundleStateWithReceipts> {
|
||||
if range.is_empty() {
|
||||
return Ok(BundleStateWithReceipts::default())
|
||||
}
|
||||
@ -402,7 +402,7 @@ impl<'this, TX: DbTxMut<'this> + DbTx<'this>> DatabaseProvider<'this, TX> {
|
||||
pub(crate) fn get_take_block_transaction_range<const TAKE: bool>(
|
||||
&self,
|
||||
range: impl RangeBounds<BlockNumber> + Clone,
|
||||
) -> Result<Vec<(BlockNumber, Vec<TransactionSignedEcRecovered>)>> {
|
||||
) -> RethResult<Vec<(BlockNumber, Vec<TransactionSignedEcRecovered>)>> {
|
||||
// Raad range of block bodies to get all transactions id's of this range.
|
||||
let block_bodies = self.get_or_take::<tables::BlockBodyIndices, false>(range)?;
|
||||
|
||||
@ -540,7 +540,7 @@ impl<'this, TX: DbTxMut<'this> + DbTx<'this>> DatabaseProvider<'this, TX> {
|
||||
&self,
|
||||
chain_spec: &ChainSpec,
|
||||
range: impl RangeBounds<BlockNumber> + Clone,
|
||||
) -> Result<Vec<SealedBlockWithSenders>> {
|
||||
) -> RethResult<Vec<SealedBlockWithSenders>> {
|
||||
// For block we need Headers, Bodies, Uncles, withdrawals, Transactions, Signers
|
||||
|
||||
let block_headers = self.get_or_take::<tables::Headers, TAKE>(range.clone())?;
|
||||
@ -737,7 +737,7 @@ impl<'this, TX: DbTxMut<'this> + DbTx<'this>> DatabaseProvider<'this, TX> {
|
||||
|
||||
/// Load shard and remove it. If list is empty, last shard was full or
|
||||
/// there are no shards at all.
|
||||
fn take_shard<T>(&self, key: T::Key) -> Result<Vec<u64>>
|
||||
fn take_shard<T>(&self, key: T::Key) -> RethResult<Vec<u64>>
|
||||
where
|
||||
T: Table<Value = BlockNumberList>,
|
||||
{
|
||||
@ -763,7 +763,7 @@ impl<'this, TX: DbTxMut<'this> + DbTx<'this>> DatabaseProvider<'this, TX> {
|
||||
&self,
|
||||
index_updates: BTreeMap<P, Vec<u64>>,
|
||||
mut sharded_key_factory: impl FnMut(P, BlockNumber) -> T::Key,
|
||||
) -> Result<()>
|
||||
) -> RethResult<()>
|
||||
where
|
||||
P: Copy,
|
||||
T: Table<Value = BlockNumberList>,
|
||||
@ -797,7 +797,7 @@ impl<'this, TX: DbTxMut<'this> + DbTx<'this>> DatabaseProvider<'this, TX> {
|
||||
}
|
||||
|
||||
impl<'this, TX: DbTx<'this>> AccountReader for DatabaseProvider<'this, TX> {
|
||||
fn basic_account(&self, address: Address) -> Result<Option<Account>> {
|
||||
fn basic_account(&self, address: Address) -> RethResult<Option<Account>> {
|
||||
Ok(self.tx.get::<tables::PlainAccountState>(address)?)
|
||||
}
|
||||
}
|
||||
@ -806,7 +806,7 @@ impl<'this, TX: DbTx<'this>> AccountExtReader for DatabaseProvider<'this, TX> {
|
||||
fn changed_accounts_with_range(
|
||||
&self,
|
||||
range: impl RangeBounds<BlockNumber>,
|
||||
) -> Result<BTreeSet<Address>> {
|
||||
) -> RethResult<BTreeSet<Address>> {
|
||||
self.tx
|
||||
.cursor_read::<tables::AccountChangeSet>()?
|
||||
.walk_range(range)?
|
||||
@ -819,7 +819,7 @@ impl<'this, TX: DbTx<'this>> AccountExtReader for DatabaseProvider<'this, TX> {
|
||||
fn basic_accounts(
|
||||
&self,
|
||||
iter: impl IntoIterator<Item = Address>,
|
||||
) -> Result<Vec<(Address, Option<Account>)>> {
|
||||
) -> RethResult<Vec<(Address, Option<Account>)>> {
|
||||
let mut plain_accounts = self.tx.cursor_read::<tables::PlainAccountState>()?;
|
||||
Ok(iter
|
||||
.into_iter()
|
||||
@ -830,12 +830,12 @@ impl<'this, TX: DbTx<'this>> AccountExtReader for DatabaseProvider<'this, TX> {
|
||||
fn changed_accounts_and_blocks_with_range(
|
||||
&self,
|
||||
range: RangeInclusive<BlockNumber>,
|
||||
) -> Result<BTreeMap<Address, Vec<u64>>> {
|
||||
) -> RethResult<BTreeMap<Address, Vec<u64>>> {
|
||||
let mut changeset_cursor = self.tx.cursor_read::<tables::AccountChangeSet>()?;
|
||||
|
||||
let account_transitions = changeset_cursor.walk_range(range)?.try_fold(
|
||||
BTreeMap::new(),
|
||||
|mut accounts: BTreeMap<Address, Vec<u64>>, entry| -> Result<_> {
|
||||
|mut accounts: BTreeMap<Address, Vec<u64>>, entry| -> RethResult<_> {
|
||||
let (index, account) = entry?;
|
||||
accounts.entry(account.address).or_default().push(index);
|
||||
Ok(accounts)
|
||||
@ -847,12 +847,15 @@ impl<'this, TX: DbTx<'this>> AccountExtReader for DatabaseProvider<'this, TX> {
|
||||
}
|
||||
|
||||
impl<'this, TX: DbTx<'this>> ChangeSetReader for DatabaseProvider<'this, TX> {
|
||||
fn account_block_changeset(&self, block_number: BlockNumber) -> Result<Vec<AccountBeforeTx>> {
|
||||
fn account_block_changeset(
|
||||
&self,
|
||||
block_number: BlockNumber,
|
||||
) -> RethResult<Vec<AccountBeforeTx>> {
|
||||
let range = block_number..=block_number;
|
||||
self.tx
|
||||
.cursor_read::<tables::AccountChangeSet>()?
|
||||
.walk_range(range)?
|
||||
.map(|result| -> Result<_> {
|
||||
.map(|result| -> RethResult<_> {
|
||||
let (_, account_before) = result?;
|
||||
Ok(account_before)
|
||||
})
|
||||
@ -861,7 +864,7 @@ impl<'this, TX: DbTx<'this>> ChangeSetReader for DatabaseProvider<'this, TX> {
|
||||
}
|
||||
|
||||
impl<'this, TX: DbTx<'this>> HeaderProvider for DatabaseProvider<'this, TX> {
|
||||
fn header(&self, block_hash: &BlockHash) -> Result<Option<Header>> {
|
||||
fn header(&self, block_hash: &BlockHash) -> RethResult<Option<Header>> {
|
||||
if let Some(num) = self.block_number(*block_hash)? {
|
||||
Ok(self.header_by_number(num)?)
|
||||
} else {
|
||||
@ -869,11 +872,11 @@ impl<'this, TX: DbTx<'this>> HeaderProvider for DatabaseProvider<'this, TX> {
|
||||
}
|
||||
}
|
||||
|
||||
fn header_by_number(&self, num: BlockNumber) -> Result<Option<Header>> {
|
||||
fn header_by_number(&self, num: BlockNumber) -> RethResult<Option<Header>> {
|
||||
Ok(self.tx.get::<tables::Headers>(num)?)
|
||||
}
|
||||
|
||||
fn header_td(&self, block_hash: &BlockHash) -> Result<Option<U256>> {
|
||||
fn header_td(&self, block_hash: &BlockHash) -> RethResult<Option<U256>> {
|
||||
if let Some(num) = self.block_number(*block_hash)? {
|
||||
self.header_td_by_number(num)
|
||||
} else {
|
||||
@ -881,7 +884,7 @@ impl<'this, TX: DbTx<'this>> HeaderProvider for DatabaseProvider<'this, TX> {
|
||||
}
|
||||
}
|
||||
|
||||
fn header_td_by_number(&self, number: BlockNumber) -> Result<Option<U256>> {
|
||||
fn header_td_by_number(&self, number: BlockNumber) -> RethResult<Option<U256>> {
|
||||
if let Some(td) = self.chain_spec.final_paris_total_difficulty(number) {
|
||||
// if this block is higher than the final paris(merge) block, return the final paris
|
||||
// difficulty
|
||||
@ -891,18 +894,18 @@ impl<'this, TX: DbTx<'this>> HeaderProvider for DatabaseProvider<'this, TX> {
|
||||
Ok(self.tx.get::<tables::HeaderTD>(number)?.map(|td| td.0))
|
||||
}
|
||||
|
||||
fn headers_range(&self, range: impl RangeBounds<BlockNumber>) -> Result<Vec<Header>> {
|
||||
fn headers_range(&self, range: impl RangeBounds<BlockNumber>) -> RethResult<Vec<Header>> {
|
||||
let mut cursor = self.tx.cursor_read::<tables::Headers>()?;
|
||||
cursor
|
||||
.walk_range(range)?
|
||||
.map(|result| result.map(|(_, header)| header).map_err(Into::into))
|
||||
.collect::<Result<Vec<_>>>()
|
||||
.collect::<RethResult<Vec<_>>>()
|
||||
}
|
||||
|
||||
fn sealed_headers_range(
|
||||
&self,
|
||||
range: impl RangeBounds<BlockNumber>,
|
||||
) -> Result<Vec<SealedHeader>> {
|
||||
) -> RethResult<Vec<SealedHeader>> {
|
||||
let mut headers = vec![];
|
||||
for entry in self.tx.cursor_read::<tables::Headers>()?.walk_range(range)? {
|
||||
let (number, header) = entry?;
|
||||
@ -914,7 +917,7 @@ impl<'this, TX: DbTx<'this>> HeaderProvider for DatabaseProvider<'this, TX> {
|
||||
Ok(headers)
|
||||
}
|
||||
|
||||
fn sealed_header(&self, number: BlockNumber) -> Result<Option<SealedHeader>> {
|
||||
fn sealed_header(&self, number: BlockNumber) -> RethResult<Option<SealedHeader>> {
|
||||
if let Some(header) = self.header_by_number(number)? {
|
||||
let hash = self
|
||||
.block_hash(number)?
|
||||
@ -927,45 +930,49 @@ impl<'this, TX: DbTx<'this>> HeaderProvider for DatabaseProvider<'this, TX> {
|
||||
}
|
||||
|
||||
impl<'this, TX: DbTx<'this>> BlockHashReader for DatabaseProvider<'this, TX> {
|
||||
fn block_hash(&self, number: u64) -> Result<Option<H256>> {
|
||||
fn block_hash(&self, number: u64) -> RethResult<Option<H256>> {
|
||||
Ok(self.tx.get::<tables::CanonicalHeaders>(number)?)
|
||||
}
|
||||
|
||||
fn canonical_hashes_range(&self, start: BlockNumber, end: BlockNumber) -> Result<Vec<H256>> {
|
||||
fn canonical_hashes_range(
|
||||
&self,
|
||||
start: BlockNumber,
|
||||
end: BlockNumber,
|
||||
) -> RethResult<Vec<H256>> {
|
||||
let range = start..end;
|
||||
let mut cursor = self.tx.cursor_read::<tables::CanonicalHeaders>()?;
|
||||
cursor
|
||||
.walk_range(range)?
|
||||
.map(|result| result.map(|(_, hash)| hash).map_err(Into::into))
|
||||
.collect::<Result<Vec<_>>>()
|
||||
.collect::<RethResult<Vec<_>>>()
|
||||
}
|
||||
}
|
||||
|
||||
impl<'this, TX: DbTx<'this>> BlockNumReader for DatabaseProvider<'this, TX> {
|
||||
fn chain_info(&self) -> Result<ChainInfo> {
|
||||
fn chain_info(&self) -> RethResult<ChainInfo> {
|
||||
let best_number = self.best_block_number()?;
|
||||
let best_hash = self.block_hash(best_number)?.unwrap_or_default();
|
||||
Ok(ChainInfo { best_hash, best_number })
|
||||
}
|
||||
|
||||
fn best_block_number(&self) -> Result<BlockNumber> {
|
||||
fn best_block_number(&self) -> RethResult<BlockNumber> {
|
||||
Ok(self
|
||||
.get_stage_checkpoint(StageId::Finish)?
|
||||
.map(|checkpoint| checkpoint.block_number)
|
||||
.unwrap_or_default())
|
||||
}
|
||||
|
||||
fn last_block_number(&self) -> Result<BlockNumber> {
|
||||
fn last_block_number(&self) -> RethResult<BlockNumber> {
|
||||
Ok(self.tx.cursor_read::<tables::CanonicalHeaders>()?.last()?.unwrap_or_default().0)
|
||||
}
|
||||
|
||||
fn block_number(&self, hash: H256) -> Result<Option<BlockNumber>> {
|
||||
fn block_number(&self, hash: H256) -> RethResult<Option<BlockNumber>> {
|
||||
Ok(self.tx.get::<tables::HeaderNumbers>(hash)?)
|
||||
}
|
||||
}
|
||||
|
||||
impl<'this, TX: DbTx<'this>> BlockReader for DatabaseProvider<'this, TX> {
|
||||
fn find_block_by_hash(&self, hash: H256, source: BlockSource) -> Result<Option<Block>> {
|
||||
fn find_block_by_hash(&self, hash: H256, source: BlockSource) -> RethResult<Option<Block>> {
|
||||
if source.is_database() {
|
||||
self.block(hash.into())
|
||||
} else {
|
||||
@ -978,7 +985,7 @@ impl<'this, TX: DbTx<'this>> BlockReader for DatabaseProvider<'this, TX> {
|
||||
/// If the header for this block is not found, this returns `None`.
|
||||
/// If the header is found, but the transactions either do not exist, or are not indexed, this
|
||||
/// will return None.
|
||||
fn block(&self, id: BlockHashOrNumber) -> Result<Option<Block>> {
|
||||
fn block(&self, id: BlockHashOrNumber) -> RethResult<Option<Block>> {
|
||||
if let Some(number) = self.convert_hash_or_number(id)? {
|
||||
if let Some(header) = self.header_by_number(number)? {
|
||||
let withdrawals = self.withdrawals_by_block(number.into(), header.timestamp)?;
|
||||
@ -999,15 +1006,15 @@ impl<'this, TX: DbTx<'this>> BlockReader for DatabaseProvider<'this, TX> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn pending_block(&self) -> Result<Option<SealedBlock>> {
|
||||
fn pending_block(&self) -> RethResult<Option<SealedBlock>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn pending_block_and_receipts(&self) -> Result<Option<(SealedBlock, Vec<Receipt>)>> {
|
||||
fn pending_block_and_receipts(&self) -> RethResult<Option<(SealedBlock, Vec<Receipt>)>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn ommers(&self, id: BlockHashOrNumber) -> Result<Option<Vec<Header>>> {
|
||||
fn ommers(&self, id: BlockHashOrNumber) -> RethResult<Option<Vec<Header>>> {
|
||||
if let Some(number) = self.convert_hash_or_number(id)? {
|
||||
// If the Paris (Merge) hardfork block is known and block is after it, return empty
|
||||
// ommers.
|
||||
@ -1022,7 +1029,7 @@ impl<'this, TX: DbTx<'this>> BlockReader for DatabaseProvider<'this, TX> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn block_body_indices(&self, num: u64) -> Result<Option<StoredBlockBodyIndices>> {
|
||||
fn block_body_indices(&self, num: u64) -> RethResult<Option<StoredBlockBodyIndices>> {
|
||||
Ok(self.tx.get::<tables::BlockBodyIndices>(num)?)
|
||||
}
|
||||
|
||||
@ -1034,7 +1041,10 @@ impl<'this, TX: DbTx<'this>> BlockReader for DatabaseProvider<'this, TX> {
|
||||
/// If the header for this block is not found, this returns `None`.
|
||||
/// If the header is found, but the transactions either do not exist, or are not indexed, this
|
||||
/// will return None.
|
||||
fn block_with_senders(&self, block_number: BlockNumber) -> Result<Option<BlockWithSenders>> {
|
||||
fn block_with_senders(
|
||||
&self,
|
||||
block_number: BlockNumber,
|
||||
) -> RethResult<Option<BlockWithSenders>> {
|
||||
let header = self
|
||||
.header_by_number(block_number)?
|
||||
.ok_or_else(|| ProviderError::HeaderNotFound(block_number.into()))?;
|
||||
@ -1079,19 +1089,22 @@ impl<'this, TX: DbTx<'this>> BlockReader for DatabaseProvider<'this, TX> {
|
||||
}
|
||||
|
||||
impl<'this, TX: DbTx<'this>> TransactionsProvider for DatabaseProvider<'this, TX> {
|
||||
fn transaction_id(&self, tx_hash: TxHash) -> Result<Option<TxNumber>> {
|
||||
fn transaction_id(&self, tx_hash: TxHash) -> RethResult<Option<TxNumber>> {
|
||||
Ok(self.tx.get::<tables::TxHashNumber>(tx_hash)?)
|
||||
}
|
||||
|
||||
fn transaction_by_id(&self, id: TxNumber) -> Result<Option<TransactionSigned>> {
|
||||
fn transaction_by_id(&self, id: TxNumber) -> RethResult<Option<TransactionSigned>> {
|
||||
Ok(self.tx.get::<tables::Transactions>(id)?.map(Into::into))
|
||||
}
|
||||
|
||||
fn transaction_by_id_no_hash(&self, id: TxNumber) -> Result<Option<TransactionSignedNoHash>> {
|
||||
fn transaction_by_id_no_hash(
|
||||
&self,
|
||||
id: TxNumber,
|
||||
) -> RethResult<Option<TransactionSignedNoHash>> {
|
||||
Ok(self.tx.get::<tables::Transactions>(id)?)
|
||||
}
|
||||
|
||||
fn transaction_by_hash(&self, hash: TxHash) -> Result<Option<TransactionSigned>> {
|
||||
fn transaction_by_hash(&self, hash: TxHash) -> RethResult<Option<TransactionSigned>> {
|
||||
if let Some(id) = self.transaction_id(hash)? {
|
||||
Ok(self.transaction_by_id_no_hash(id)?.map(|tx| TransactionSigned {
|
||||
hash,
|
||||
@ -1107,7 +1120,7 @@ impl<'this, TX: DbTx<'this>> TransactionsProvider for DatabaseProvider<'this, TX
|
||||
fn transaction_by_hash_with_meta(
|
||||
&self,
|
||||
tx_hash: TxHash,
|
||||
) -> Result<Option<(TransactionSigned, TransactionMeta)>> {
|
||||
) -> RethResult<Option<(TransactionSigned, TransactionMeta)>> {
|
||||
let mut transaction_cursor = self.tx.cursor_read::<tables::TransactionBlock>()?;
|
||||
if let Some(transaction_id) = self.transaction_id(tx_hash)? {
|
||||
if let Some(tx) = self.transaction_by_id_no_hash(transaction_id)? {
|
||||
@ -1147,7 +1160,7 @@ impl<'this, TX: DbTx<'this>> TransactionsProvider for DatabaseProvider<'this, TX
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn transaction_block(&self, id: TxNumber) -> Result<Option<BlockNumber>> {
|
||||
fn transaction_block(&self, id: TxNumber) -> RethResult<Option<BlockNumber>> {
|
||||
let mut cursor = self.tx.cursor_read::<tables::TransactionBlock>()?;
|
||||
Ok(cursor.seek(id)?.map(|(_, bn)| bn))
|
||||
}
|
||||
@ -1155,7 +1168,7 @@ impl<'this, TX: DbTx<'this>> TransactionsProvider for DatabaseProvider<'this, TX
|
||||
fn transactions_by_block(
|
||||
&self,
|
||||
id: BlockHashOrNumber,
|
||||
) -> Result<Option<Vec<TransactionSigned>>> {
|
||||
) -> RethResult<Option<Vec<TransactionSigned>>> {
|
||||
let mut tx_cursor = self.tx.cursor_read::<tables::Transactions>()?;
|
||||
if let Some(block_number) = self.convert_hash_or_number(id)? {
|
||||
if let Some(body) = self.block_body_indices(block_number)? {
|
||||
@ -1177,7 +1190,7 @@ impl<'this, TX: DbTx<'this>> TransactionsProvider for DatabaseProvider<'this, TX
|
||||
fn transactions_by_block_range(
|
||||
&self,
|
||||
range: impl RangeBounds<BlockNumber>,
|
||||
) -> Result<Vec<Vec<TransactionSigned>>> {
|
||||
) -> RethResult<Vec<Vec<TransactionSigned>>> {
|
||||
let mut results = Vec::new();
|
||||
let mut body_cursor = self.tx.cursor_read::<tables::BlockBodyIndices>()?;
|
||||
let mut tx_cursor = self.tx.cursor_read::<tables::Transactions>()?;
|
||||
@ -1201,7 +1214,7 @@ impl<'this, TX: DbTx<'this>> TransactionsProvider for DatabaseProvider<'this, TX
|
||||
fn transactions_by_tx_range(
|
||||
&self,
|
||||
range: impl RangeBounds<TxNumber>,
|
||||
) -> Result<Vec<TransactionSignedNoHash>> {
|
||||
) -> RethResult<Vec<TransactionSignedNoHash>> {
|
||||
Ok(self
|
||||
.tx
|
||||
.cursor_read::<tables::Transactions>()?
|
||||
@ -1210,7 +1223,7 @@ impl<'this, TX: DbTx<'this>> TransactionsProvider for DatabaseProvider<'this, TX
|
||||
.collect::<std::result::Result<Vec<_>, _>>()?)
|
||||
}
|
||||
|
||||
fn senders_by_tx_range(&self, range: impl RangeBounds<TxNumber>) -> Result<Vec<Address>> {
|
||||
fn senders_by_tx_range(&self, range: impl RangeBounds<TxNumber>) -> RethResult<Vec<Address>> {
|
||||
Ok(self
|
||||
.tx
|
||||
.cursor_read::<tables::TxSenders>()?
|
||||
@ -1219,17 +1232,17 @@ impl<'this, TX: DbTx<'this>> TransactionsProvider for DatabaseProvider<'this, TX
|
||||
.collect::<std::result::Result<Vec<_>, _>>()?)
|
||||
}
|
||||
|
||||
fn transaction_sender(&self, id: TxNumber) -> Result<Option<Address>> {
|
||||
fn transaction_sender(&self, id: TxNumber) -> RethResult<Option<Address>> {
|
||||
Ok(self.tx.get::<tables::TxSenders>(id)?)
|
||||
}
|
||||
}
|
||||
|
||||
impl<'this, TX: DbTx<'this>> ReceiptProvider for DatabaseProvider<'this, TX> {
|
||||
fn receipt(&self, id: TxNumber) -> Result<Option<Receipt>> {
|
||||
fn receipt(&self, id: TxNumber) -> RethResult<Option<Receipt>> {
|
||||
Ok(self.tx.get::<tables::Receipts>(id)?)
|
||||
}
|
||||
|
||||
fn receipt_by_hash(&self, hash: TxHash) -> Result<Option<Receipt>> {
|
||||
fn receipt_by_hash(&self, hash: TxHash) -> RethResult<Option<Receipt>> {
|
||||
if let Some(id) = self.transaction_id(hash)? {
|
||||
self.receipt(id)
|
||||
} else {
|
||||
@ -1237,7 +1250,7 @@ impl<'this, TX: DbTx<'this>> ReceiptProvider for DatabaseProvider<'this, TX> {
|
||||
}
|
||||
}
|
||||
|
||||
fn receipts_by_block(&self, block: BlockHashOrNumber) -> Result<Option<Vec<Receipt>>> {
|
||||
fn receipts_by_block(&self, block: BlockHashOrNumber) -> RethResult<Option<Vec<Receipt>>> {
|
||||
if let Some(number) = self.convert_hash_or_number(block)? {
|
||||
if let Some(body) = self.block_body_indices(number)? {
|
||||
let tx_range = body.tx_num_range();
|
||||
@ -1262,7 +1275,7 @@ impl<'this, TX: DbTx<'this>> WithdrawalsProvider for DatabaseProvider<'this, TX>
|
||||
&self,
|
||||
id: BlockHashOrNumber,
|
||||
timestamp: u64,
|
||||
) -> Result<Option<Vec<Withdrawal>>> {
|
||||
) -> RethResult<Option<Vec<Withdrawal>>> {
|
||||
if self.chain_spec.is_shanghai_activated_at_timestamp(timestamp) {
|
||||
if let Some(number) = self.convert_hash_or_number(id)? {
|
||||
// If we are past shanghai, then all blocks should have a withdrawal list, even if
|
||||
@ -1278,7 +1291,7 @@ impl<'this, TX: DbTx<'this>> WithdrawalsProvider for DatabaseProvider<'this, TX>
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn latest_withdrawal(&self) -> Result<Option<Withdrawal>> {
|
||||
fn latest_withdrawal(&self) -> RethResult<Option<Withdrawal>> {
|
||||
let latest_block_withdrawal = self.tx.cursor_read::<tables::BlockWithdrawals>()?.last()?;
|
||||
Ok(latest_block_withdrawal
|
||||
.and_then(|(_, mut block_withdrawal)| block_withdrawal.withdrawals.pop()))
|
||||
@ -1291,7 +1304,7 @@ impl<'this, TX: DbTx<'this>> EvmEnvProvider for DatabaseProvider<'this, TX> {
|
||||
cfg: &mut CfgEnv,
|
||||
block_env: &mut BlockEnv,
|
||||
at: BlockHashOrNumber,
|
||||
) -> Result<()> {
|
||||
) -> RethResult<()> {
|
||||
let hash = self.convert_number(at)?.ok_or(ProviderError::HeaderNotFound(at))?;
|
||||
let header = self.header(&hash)?.ok_or(ProviderError::HeaderNotFound(at))?;
|
||||
self.fill_env_with_header(cfg, block_env, &header)
|
||||
@ -1302,7 +1315,7 @@ impl<'this, TX: DbTx<'this>> EvmEnvProvider for DatabaseProvider<'this, TX> {
|
||||
cfg: &mut CfgEnv,
|
||||
block_env: &mut BlockEnv,
|
||||
header: &Header,
|
||||
) -> Result<()> {
|
||||
) -> RethResult<()> {
|
||||
let total_difficulty = self
|
||||
.header_td_by_number(header.number)?
|
||||
.ok_or_else(|| ProviderError::HeaderNotFound(header.number.into()))?;
|
||||
@ -1310,14 +1323,18 @@ impl<'this, TX: DbTx<'this>> EvmEnvProvider for DatabaseProvider<'this, TX> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn fill_block_env_at(&self, block_env: &mut BlockEnv, at: BlockHashOrNumber) -> Result<()> {
|
||||
fn fill_block_env_at(&self, block_env: &mut BlockEnv, at: BlockHashOrNumber) -> RethResult<()> {
|
||||
let hash = self.convert_number(at)?.ok_or(ProviderError::HeaderNotFound(at))?;
|
||||
let header = self.header(&hash)?.ok_or(ProviderError::HeaderNotFound(at))?;
|
||||
|
||||
self.fill_block_env_with_header(block_env, &header)
|
||||
}
|
||||
|
||||
fn fill_block_env_with_header(&self, block_env: &mut BlockEnv, header: &Header) -> Result<()> {
|
||||
fn fill_block_env_with_header(
|
||||
&self,
|
||||
block_env: &mut BlockEnv,
|
||||
header: &Header,
|
||||
) -> RethResult<()> {
|
||||
let total_difficulty = self
|
||||
.header_td_by_number(header.number)?
|
||||
.ok_or_else(|| ProviderError::HeaderNotFound(header.number.into()))?;
|
||||
@ -1337,13 +1354,13 @@ impl<'this, TX: DbTx<'this>> EvmEnvProvider for DatabaseProvider<'this, TX> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn fill_cfg_env_at(&self, cfg: &mut CfgEnv, at: BlockHashOrNumber) -> Result<()> {
|
||||
fn fill_cfg_env_at(&self, cfg: &mut CfgEnv, at: BlockHashOrNumber) -> RethResult<()> {
|
||||
let hash = self.convert_number(at)?.ok_or(ProviderError::HeaderNotFound(at))?;
|
||||
let header = self.header(&hash)?.ok_or(ProviderError::HeaderNotFound(at))?;
|
||||
self.fill_cfg_env_with_header(cfg, &header)
|
||||
}
|
||||
|
||||
fn fill_cfg_env_with_header(&self, cfg: &mut CfgEnv, header: &Header) -> Result<()> {
|
||||
fn fill_cfg_env_with_header(&self, cfg: &mut CfgEnv, header: &Header) -> RethResult<()> {
|
||||
let total_difficulty = self
|
||||
.header_td_by_number(header.number)?
|
||||
.ok_or_else(|| ProviderError::HeaderNotFound(header.number.into()))?;
|
||||
@ -1353,24 +1370,24 @@ impl<'this, TX: DbTx<'this>> EvmEnvProvider for DatabaseProvider<'this, TX> {
|
||||
}
|
||||
|
||||
impl<'this, TX: DbTx<'this>> StageCheckpointReader for DatabaseProvider<'this, TX> {
|
||||
fn get_stage_checkpoint(&self, id: StageId) -> Result<Option<StageCheckpoint>> {
|
||||
fn get_stage_checkpoint(&self, id: StageId) -> RethResult<Option<StageCheckpoint>> {
|
||||
Ok(self.tx.get::<tables::SyncStage>(id.to_string())?)
|
||||
}
|
||||
|
||||
/// Get stage checkpoint progress.
|
||||
fn get_stage_checkpoint_progress(&self, id: StageId) -> Result<Option<Vec<u8>>> {
|
||||
fn get_stage_checkpoint_progress(&self, id: StageId) -> RethResult<Option<Vec<u8>>> {
|
||||
Ok(self.tx.get::<tables::SyncStageProgress>(id.to_string())?)
|
||||
}
|
||||
}
|
||||
|
||||
impl<'this, TX: DbTxMut<'this>> StageCheckpointWriter for DatabaseProvider<'this, TX> {
|
||||
/// Save stage checkpoint progress.
|
||||
fn save_stage_checkpoint_progress(&self, id: StageId, checkpoint: Vec<u8>) -> Result<()> {
|
||||
fn save_stage_checkpoint_progress(&self, id: StageId, checkpoint: Vec<u8>) -> RethResult<()> {
|
||||
Ok(self.tx.put::<tables::SyncStageProgress>(id.to_string(), checkpoint)?)
|
||||
}
|
||||
|
||||
/// Save stage checkpoint.
|
||||
fn save_stage_checkpoint(&self, id: StageId, checkpoint: StageCheckpoint) -> Result<()> {
|
||||
fn save_stage_checkpoint(&self, id: StageId, checkpoint: StageCheckpoint) -> RethResult<()> {
|
||||
Ok(self.tx.put::<tables::SyncStage>(id.to_string(), checkpoint)?)
|
||||
}
|
||||
|
||||
@ -1378,7 +1395,7 @@ impl<'this, TX: DbTxMut<'this>> StageCheckpointWriter for DatabaseProvider<'this
|
||||
&self,
|
||||
block_number: BlockNumber,
|
||||
drop_stage_checkpoint: bool,
|
||||
) -> Result<()> {
|
||||
) -> RethResult<()> {
|
||||
// iterate over all existing stages in the table and update its progress.
|
||||
let mut cursor = self.tx.cursor_write::<tables::SyncStage>()?;
|
||||
for stage_id in StageId::ALL {
|
||||
@ -1400,7 +1417,7 @@ impl<'this, TX: DbTx<'this>> StorageReader for DatabaseProvider<'this, TX> {
|
||||
fn plainstate_storages(
|
||||
&self,
|
||||
addresses_with_keys: impl IntoIterator<Item = (Address, impl IntoIterator<Item = H256>)>,
|
||||
) -> Result<Vec<(Address, Vec<StorageEntry>)>> {
|
||||
) -> RethResult<Vec<(Address, Vec<StorageEntry>)>> {
|
||||
let mut plain_storage = self.tx.cursor_dup_read::<tables::PlainStorageState>()?;
|
||||
|
||||
addresses_with_keys
|
||||
@ -1408,22 +1425,22 @@ impl<'this, TX: DbTx<'this>> StorageReader for DatabaseProvider<'this, TX> {
|
||||
.map(|(address, storage)| {
|
||||
storage
|
||||
.into_iter()
|
||||
.map(|key| -> Result<_> {
|
||||
.map(|key| -> RethResult<_> {
|
||||
Ok(plain_storage
|
||||
.seek_by_key_subkey(address, key)?
|
||||
.filter(|v| v.key == key)
|
||||
.unwrap_or_else(|| StorageEntry { key, value: Default::default() }))
|
||||
})
|
||||
.collect::<Result<Vec<_>>>()
|
||||
.collect::<RethResult<Vec<_>>>()
|
||||
.map(|storage| (address, storage))
|
||||
})
|
||||
.collect::<Result<Vec<(_, _)>>>()
|
||||
.collect::<RethResult<Vec<(_, _)>>>()
|
||||
}
|
||||
|
||||
fn changed_storages_with_range(
|
||||
&self,
|
||||
range: RangeInclusive<BlockNumber>,
|
||||
) -> Result<BTreeMap<Address, BTreeSet<H256>>> {
|
||||
) -> RethResult<BTreeMap<Address, BTreeSet<H256>>> {
|
||||
self.tx
|
||||
.cursor_read::<tables::StorageChangeSet>()?
|
||||
.walk_range(BlockNumberAddress::range(range))?
|
||||
@ -1439,13 +1456,13 @@ impl<'this, TX: DbTx<'this>> StorageReader for DatabaseProvider<'this, TX> {
|
||||
fn changed_storages_and_blocks_with_range(
|
||||
&self,
|
||||
range: RangeInclusive<BlockNumber>,
|
||||
) -> Result<BTreeMap<(Address, H256), Vec<u64>>> {
|
||||
) -> RethResult<BTreeMap<(Address, H256), Vec<u64>>> {
|
||||
let mut changeset_cursor = self.tx.cursor_read::<tables::StorageChangeSet>()?;
|
||||
|
||||
let storage_changeset_lists =
|
||||
changeset_cursor.walk_range(BlockNumberAddress::range(range))?.try_fold(
|
||||
BTreeMap::new(),
|
||||
|mut storages: BTreeMap<(Address, H256), Vec<u64>>, entry| -> Result<_> {
|
||||
|mut storages: BTreeMap<(Address, H256), Vec<u64>>, entry| -> RethResult<_> {
|
||||
let (index, storage) = entry?;
|
||||
storages
|
||||
.entry((index.address(), storage.key))
|
||||
@ -1465,7 +1482,7 @@ impl<'this, TX: DbTxMut<'this> + DbTx<'this>> HashingWriter for DatabaseProvider
|
||||
range: RangeInclusive<BlockNumber>,
|
||||
end_block_hash: H256,
|
||||
expected_state_root: H256,
|
||||
) -> Result<()> {
|
||||
) -> RethResult<()> {
|
||||
// Initialize prefix sets.
|
||||
let mut account_prefix_set = PrefixSetMut::default();
|
||||
let mut storage_prefix_set: HashMap<H256, PrefixSetMut> = HashMap::default();
|
||||
@ -1529,7 +1546,7 @@ impl<'this, TX: DbTxMut<'this> + DbTx<'this>> HashingWriter for DatabaseProvider
|
||||
fn unwind_storage_hashing(
|
||||
&self,
|
||||
range: Range<BlockNumberAddress>,
|
||||
) -> Result<HashMap<H256, BTreeSet<H256>>> {
|
||||
) -> RethResult<HashMap<H256, BTreeSet<H256>>> {
|
||||
let mut hashed_storage = self.tx.cursor_dup_write::<tables::HashedStorage>()?;
|
||||
|
||||
// Aggregate all block changesets and make list of accounts that have been changed.
|
||||
@ -1563,7 +1580,7 @@ impl<'this, TX: DbTxMut<'this> + DbTx<'this>> HashingWriter for DatabaseProvider
|
||||
hashed_storages
|
||||
.into_iter()
|
||||
// Apply values to HashedStorage (if Value is zero just remove it);
|
||||
.try_for_each(|((hashed_address, key), value)| -> Result<()> {
|
||||
.try_for_each(|((hashed_address, key), value)| -> RethResult<()> {
|
||||
if hashed_storage
|
||||
.seek_by_key_subkey(hashed_address, key)?
|
||||
.filter(|entry| entry.key == key)
|
||||
@ -1584,7 +1601,7 @@ impl<'this, TX: DbTxMut<'this> + DbTx<'this>> HashingWriter for DatabaseProvider
|
||||
fn insert_storage_for_hashing(
|
||||
&self,
|
||||
storages: impl IntoIterator<Item = (Address, impl IntoIterator<Item = StorageEntry>)>,
|
||||
) -> Result<HashMap<H256, BTreeSet<H256>>> {
|
||||
) -> RethResult<HashMap<H256, BTreeSet<H256>>> {
|
||||
// hash values
|
||||
let hashed_storages =
|
||||
storages.into_iter().fold(BTreeMap::new(), |mut map, (address, storage)| {
|
||||
@ -1605,7 +1622,7 @@ impl<'this, TX: DbTxMut<'this> + DbTx<'this>> HashingWriter for DatabaseProvider
|
||||
// Hash the address and key and apply them to HashedStorage (if Storage is None
|
||||
// just remove it);
|
||||
hashed_storages.into_iter().try_for_each(|(hashed_address, storage)| {
|
||||
storage.into_iter().try_for_each(|(key, value)| -> Result<()> {
|
||||
storage.into_iter().try_for_each(|(key, value)| -> RethResult<()> {
|
||||
if hashed_storage_cursor
|
||||
.seek_by_key_subkey(hashed_address, key)?
|
||||
.filter(|entry| entry.key == key)
|
||||
@ -1627,7 +1644,7 @@ impl<'this, TX: DbTxMut<'this> + DbTx<'this>> HashingWriter for DatabaseProvider
|
||||
fn unwind_account_hashing(
|
||||
&self,
|
||||
range: RangeInclusive<BlockNumber>,
|
||||
) -> Result<BTreeMap<H256, Option<Account>>> {
|
||||
) -> RethResult<BTreeMap<H256, Option<Account>>> {
|
||||
let mut hashed_accounts_cursor = self.tx.cursor_write::<tables::HashedAccount>()?;
|
||||
|
||||
// Aggregate all block changesets and make a list of accounts that have been changed.
|
||||
@ -1655,7 +1672,7 @@ impl<'this, TX: DbTxMut<'this> + DbTx<'this>> HashingWriter for DatabaseProvider
|
||||
hashed_accounts
|
||||
.iter()
|
||||
// Apply values to HashedState (if Account is None remove it);
|
||||
.try_for_each(|(hashed_address, account)| -> Result<()> {
|
||||
.try_for_each(|(hashed_address, account)| -> RethResult<()> {
|
||||
if let Some(account) = account {
|
||||
hashed_accounts_cursor.upsert(*hashed_address, *account)?;
|
||||
} else if hashed_accounts_cursor.seek_exact(*hashed_address)?.is_some() {
|
||||
@ -1670,7 +1687,7 @@ impl<'this, TX: DbTxMut<'this> + DbTx<'this>> HashingWriter for DatabaseProvider
|
||||
fn insert_account_for_hashing(
|
||||
&self,
|
||||
accounts: impl IntoIterator<Item = (Address, Option<Account>)>,
|
||||
) -> Result<BTreeMap<H256, Option<Account>>> {
|
||||
) -> RethResult<BTreeMap<H256, Option<Account>>> {
|
||||
let mut hashed_accounts_cursor = self.tx.cursor_write::<tables::HashedAccount>()?;
|
||||
|
||||
let hashed_accounts = accounts.into_iter().fold(
|
||||
@ -1681,7 +1698,7 @@ impl<'this, TX: DbTxMut<'this> + DbTx<'this>> HashingWriter for DatabaseProvider
|
||||
},
|
||||
);
|
||||
|
||||
hashed_accounts.iter().try_for_each(|(hashed_address, account)| -> Result<()> {
|
||||
hashed_accounts.iter().try_for_each(|(hashed_address, account)| -> RethResult<()> {
|
||||
if let Some(account) = account {
|
||||
hashed_accounts_cursor.upsert(*hashed_address, *account)?
|
||||
} else if hashed_accounts_cursor.seek_exact(*hashed_address)?.is_some() {
|
||||
@ -1695,7 +1712,7 @@ impl<'this, TX: DbTxMut<'this> + DbTx<'this>> HashingWriter for DatabaseProvider
|
||||
}
|
||||
|
||||
impl<'this, TX: DbTxMut<'this> + DbTx<'this>> HistoryWriter for DatabaseProvider<'this, TX> {
|
||||
fn calculate_history_indices(&self, range: RangeInclusive<BlockNumber>) -> Result<()> {
|
||||
fn calculate_history_indices(&self, range: RangeInclusive<BlockNumber>) -> RethResult<()> {
|
||||
// account history stage
|
||||
{
|
||||
let indices = self.changed_accounts_and_blocks_with_range(range.clone())?;
|
||||
@ -1714,7 +1731,7 @@ impl<'this, TX: DbTxMut<'this> + DbTx<'this>> HistoryWriter for DatabaseProvider
|
||||
fn insert_storage_history_index(
|
||||
&self,
|
||||
storage_transitions: BTreeMap<(Address, H256), Vec<u64>>,
|
||||
) -> Result<()> {
|
||||
) -> RethResult<()> {
|
||||
self.append_history_index::<_, tables::StorageHistory>(
|
||||
storage_transitions,
|
||||
|(address, storage_key), highest_block_number| {
|
||||
@ -1726,11 +1743,14 @@ impl<'this, TX: DbTxMut<'this> + DbTx<'this>> HistoryWriter for DatabaseProvider
|
||||
fn insert_account_history_index(
|
||||
&self,
|
||||
account_transitions: BTreeMap<Address, Vec<u64>>,
|
||||
) -> Result<()> {
|
||||
) -> RethResult<()> {
|
||||
self.append_history_index::<_, tables::AccountHistory>(account_transitions, ShardedKey::new)
|
||||
}
|
||||
|
||||
fn unwind_storage_history_indices(&self, range: Range<BlockNumberAddress>) -> Result<usize> {
|
||||
fn unwind_storage_history_indices(
|
||||
&self,
|
||||
range: Range<BlockNumberAddress>,
|
||||
) -> RethResult<usize> {
|
||||
let storage_changesets = self
|
||||
.tx
|
||||
.cursor_read::<tables::StorageChangeSet>()?
|
||||
@ -1777,7 +1797,10 @@ impl<'this, TX: DbTxMut<'this> + DbTx<'this>> HistoryWriter for DatabaseProvider
|
||||
Ok(changesets)
|
||||
}
|
||||
|
||||
fn unwind_account_history_indices(&self, range: RangeInclusive<BlockNumber>) -> Result<usize> {
|
||||
fn unwind_account_history_indices(
|
||||
&self,
|
||||
range: RangeInclusive<BlockNumber>,
|
||||
) -> RethResult<usize> {
|
||||
let account_changeset = self
|
||||
.tx
|
||||
.cursor_read::<tables::AccountChangeSet>()?
|
||||
@ -1826,7 +1849,7 @@ impl<'this, TX: DbTxMut<'this> + DbTx<'this>> BlockExecutionWriter for DatabaseP
|
||||
&self,
|
||||
chain_spec: &ChainSpec,
|
||||
range: RangeInclusive<BlockNumber>,
|
||||
) -> Result<Chain> {
|
||||
) -> RethResult<Chain> {
|
||||
if TAKE {
|
||||
let storage_range = BlockNumberAddress::range(range.clone());
|
||||
|
||||
@ -1925,7 +1948,7 @@ impl<'this, TX: DbTxMut<'this> + DbTx<'this>> BlockWriter for DatabaseProvider<'
|
||||
block: SealedBlock,
|
||||
senders: Option<Vec<Address>>,
|
||||
prune_modes: Option<&PruneModes>,
|
||||
) -> Result<StoredBlockBodyIndices> {
|
||||
) -> RethResult<StoredBlockBodyIndices> {
|
||||
let block_number = block.number;
|
||||
self.tx.put::<tables::CanonicalHeaders>(block.number, block.hash())?;
|
||||
// Put header with canonical hashes.
|
||||
@ -2019,7 +2042,7 @@ impl<'this, TX: DbTxMut<'this> + DbTx<'this>> BlockWriter for DatabaseProvider<'
|
||||
blocks: Vec<SealedBlockWithSenders>,
|
||||
state: BundleStateWithReceipts,
|
||||
prune_modes: Option<&PruneModes>,
|
||||
) -> Result<()> {
|
||||
) -> RethResult<()> {
|
||||
if blocks.is_empty() {
|
||||
return Ok(())
|
||||
}
|
||||
@ -2055,13 +2078,17 @@ impl<'this, TX: DbTxMut<'this> + DbTx<'this>> BlockWriter for DatabaseProvider<'
|
||||
}
|
||||
|
||||
impl<'this, TX: DbTx<'this>> PruneCheckpointReader for DatabaseProvider<'this, TX> {
|
||||
fn get_prune_checkpoint(&self, part: PrunePart) -> Result<Option<PruneCheckpoint>> {
|
||||
fn get_prune_checkpoint(&self, part: PrunePart) -> RethResult<Option<PruneCheckpoint>> {
|
||||
Ok(self.tx.get::<tables::PruneCheckpoints>(part)?)
|
||||
}
|
||||
}
|
||||
|
||||
impl<'this, TX: DbTxMut<'this>> PruneCheckpointWriter for DatabaseProvider<'this, TX> {
|
||||
fn save_prune_checkpoint(&self, part: PrunePart, checkpoint: PruneCheckpoint) -> Result<()> {
|
||||
fn save_prune_checkpoint(
|
||||
&self,
|
||||
part: PrunePart,
|
||||
checkpoint: PruneCheckpoint,
|
||||
) -> RethResult<()> {
|
||||
Ok(self.tx.put::<tables::PruneCheckpoints>(part, checkpoint)?)
|
||||
}
|
||||
}
|
||||
|
||||
@ -10,7 +10,7 @@ use reth_db::{database::Database, models::StoredBlockBodyIndices};
|
||||
use reth_interfaces::{
|
||||
blockchain_tree::{BlockchainTreeEngine, BlockchainTreeViewer},
|
||||
consensus::ForkchoiceState,
|
||||
Error, Result,
|
||||
RethError, RethResult,
|
||||
};
|
||||
use reth_primitives::{
|
||||
stage::{StageCheckpoint, StageId},
|
||||
@ -73,7 +73,7 @@ where
|
||||
{
|
||||
/// Create a new provider using only the database and the tree, fetching the latest header from
|
||||
/// the database to initialize the provider.
|
||||
pub fn new(database: ProviderFactory<DB>, tree: Tree) -> Result<Self> {
|
||||
pub fn new(database: ProviderFactory<DB>, tree: Tree) -> RethResult<Self> {
|
||||
let provider = database.provider()?;
|
||||
let best: ChainInfo = provider.chain_info()?;
|
||||
match provider.header_by_number(best.best_number)? {
|
||||
@ -81,7 +81,9 @@ where
|
||||
drop(provider);
|
||||
Ok(Self::with_latest(database, tree, header.seal(best.best_hash)))
|
||||
}
|
||||
None => Err(Error::Provider(ProviderError::HeaderNotFound(best.best_number.into()))),
|
||||
None => {
|
||||
Err(RethError::Provider(ProviderError::HeaderNotFound(best.best_number.into())))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -100,7 +102,7 @@ where
|
||||
/// Instead, we ensure that the `block_number` is within the range of the
|
||||
/// [Self::best_block_number] which is updated when a block is synced.
|
||||
#[inline]
|
||||
fn ensure_canonical_block(&self, block_number: BlockNumber) -> Result<()> {
|
||||
fn ensure_canonical_block(&self, block_number: BlockNumber) -> RethResult<()> {
|
||||
let latest = self.best_block_number()?;
|
||||
if block_number > latest {
|
||||
Err(ProviderError::HeaderNotFound(block_number.into()).into())
|
||||
@ -115,34 +117,34 @@ where
|
||||
DB: Database,
|
||||
Tree: Send + Sync,
|
||||
{
|
||||
fn header(&self, block_hash: &BlockHash) -> Result<Option<Header>> {
|
||||
fn header(&self, block_hash: &BlockHash) -> RethResult<Option<Header>> {
|
||||
self.database.provider()?.header(block_hash)
|
||||
}
|
||||
|
||||
fn header_by_number(&self, num: BlockNumber) -> Result<Option<Header>> {
|
||||
fn header_by_number(&self, num: BlockNumber) -> RethResult<Option<Header>> {
|
||||
self.database.provider()?.header_by_number(num)
|
||||
}
|
||||
|
||||
fn header_td(&self, hash: &BlockHash) -> Result<Option<U256>> {
|
||||
fn header_td(&self, hash: &BlockHash) -> RethResult<Option<U256>> {
|
||||
self.database.provider()?.header_td(hash)
|
||||
}
|
||||
|
||||
fn header_td_by_number(&self, number: BlockNumber) -> Result<Option<U256>> {
|
||||
fn header_td_by_number(&self, number: BlockNumber) -> RethResult<Option<U256>> {
|
||||
self.database.provider()?.header_td_by_number(number)
|
||||
}
|
||||
|
||||
fn headers_range(&self, range: impl RangeBounds<BlockNumber>) -> Result<Vec<Header>> {
|
||||
fn headers_range(&self, range: impl RangeBounds<BlockNumber>) -> RethResult<Vec<Header>> {
|
||||
self.database.provider()?.headers_range(range)
|
||||
}
|
||||
|
||||
fn sealed_headers_range(
|
||||
&self,
|
||||
range: impl RangeBounds<BlockNumber>,
|
||||
) -> Result<Vec<SealedHeader>> {
|
||||
) -> RethResult<Vec<SealedHeader>> {
|
||||
self.database.provider()?.sealed_headers_range(range)
|
||||
}
|
||||
|
||||
fn sealed_header(&self, number: BlockNumber) -> Result<Option<SealedHeader>> {
|
||||
fn sealed_header(&self, number: BlockNumber) -> RethResult<Option<SealedHeader>> {
|
||||
self.database.provider()?.sealed_header(number)
|
||||
}
|
||||
}
|
||||
@ -152,11 +154,15 @@ where
|
||||
DB: Database,
|
||||
Tree: Send + Sync,
|
||||
{
|
||||
fn block_hash(&self, number: u64) -> Result<Option<H256>> {
|
||||
fn block_hash(&self, number: u64) -> RethResult<Option<H256>> {
|
||||
self.database.provider()?.block_hash(number)
|
||||
}
|
||||
|
||||
fn canonical_hashes_range(&self, start: BlockNumber, end: BlockNumber) -> Result<Vec<H256>> {
|
||||
fn canonical_hashes_range(
|
||||
&self,
|
||||
start: BlockNumber,
|
||||
end: BlockNumber,
|
||||
) -> RethResult<Vec<H256>> {
|
||||
self.database.provider()?.canonical_hashes_range(start, end)
|
||||
}
|
||||
}
|
||||
@ -166,19 +172,19 @@ where
|
||||
DB: Database,
|
||||
Tree: BlockchainTreeViewer + Send + Sync,
|
||||
{
|
||||
fn chain_info(&self) -> Result<ChainInfo> {
|
||||
fn chain_info(&self) -> RethResult<ChainInfo> {
|
||||
Ok(self.chain_info.chain_info())
|
||||
}
|
||||
|
||||
fn best_block_number(&self) -> Result<BlockNumber> {
|
||||
fn best_block_number(&self) -> RethResult<BlockNumber> {
|
||||
Ok(self.chain_info.get_canonical_block_number())
|
||||
}
|
||||
|
||||
fn last_block_number(&self) -> Result<BlockNumber> {
|
||||
fn last_block_number(&self) -> RethResult<BlockNumber> {
|
||||
self.database.provider()?.last_block_number()
|
||||
}
|
||||
|
||||
fn block_number(&self, hash: H256) -> Result<Option<BlockNumber>> {
|
||||
fn block_number(&self, hash: H256) -> RethResult<Option<BlockNumber>> {
|
||||
self.database.provider()?.block_number(hash)
|
||||
}
|
||||
}
|
||||
@ -188,15 +194,15 @@ where
|
||||
DB: Database,
|
||||
Tree: BlockchainTreeViewer + Send + Sync,
|
||||
{
|
||||
fn pending_block_num_hash(&self) -> Result<Option<BlockNumHash>> {
|
||||
fn pending_block_num_hash(&self) -> RethResult<Option<BlockNumHash>> {
|
||||
Ok(self.tree.pending_block_num_hash())
|
||||
}
|
||||
|
||||
fn safe_block_num_hash(&self) -> Result<Option<BlockNumHash>> {
|
||||
fn safe_block_num_hash(&self) -> RethResult<Option<BlockNumHash>> {
|
||||
Ok(self.chain_info.get_safe_num_hash())
|
||||
}
|
||||
|
||||
fn finalized_block_num_hash(&self) -> Result<Option<BlockNumHash>> {
|
||||
fn finalized_block_num_hash(&self) -> RethResult<Option<BlockNumHash>> {
|
||||
Ok(self.chain_info.get_finalized_num_hash())
|
||||
}
|
||||
}
|
||||
@ -206,7 +212,7 @@ where
|
||||
DB: Database,
|
||||
Tree: BlockchainTreeViewer + Send + Sync,
|
||||
{
|
||||
fn find_block_by_hash(&self, hash: H256, source: BlockSource) -> Result<Option<Block>> {
|
||||
fn find_block_by_hash(&self, hash: H256, source: BlockSource) -> RethResult<Option<Block>> {
|
||||
let block = match source {
|
||||
BlockSource::Any => {
|
||||
// check database first
|
||||
@ -225,26 +231,29 @@ where
|
||||
Ok(block)
|
||||
}
|
||||
|
||||
fn block(&self, id: BlockHashOrNumber) -> Result<Option<Block>> {
|
||||
fn block(&self, id: BlockHashOrNumber) -> RethResult<Option<Block>> {
|
||||
match id {
|
||||
BlockHashOrNumber::Hash(hash) => self.find_block_by_hash(hash, BlockSource::Any),
|
||||
BlockHashOrNumber::Number(num) => self.database.provider()?.block_by_number(num),
|
||||
}
|
||||
}
|
||||
|
||||
fn pending_block(&self) -> Result<Option<SealedBlock>> {
|
||||
fn pending_block(&self) -> RethResult<Option<SealedBlock>> {
|
||||
Ok(self.tree.pending_block())
|
||||
}
|
||||
|
||||
fn pending_block_and_receipts(&self) -> Result<Option<(SealedBlock, Vec<Receipt>)>> {
|
||||
fn pending_block_and_receipts(&self) -> RethResult<Option<(SealedBlock, Vec<Receipt>)>> {
|
||||
Ok(self.tree.pending_block_and_receipts())
|
||||
}
|
||||
|
||||
fn ommers(&self, id: BlockHashOrNumber) -> Result<Option<Vec<Header>>> {
|
||||
fn ommers(&self, id: BlockHashOrNumber) -> RethResult<Option<Vec<Header>>> {
|
||||
self.database.provider()?.ommers(id)
|
||||
}
|
||||
|
||||
fn block_body_indices(&self, number: BlockNumber) -> Result<Option<StoredBlockBodyIndices>> {
|
||||
fn block_body_indices(
|
||||
&self,
|
||||
number: BlockNumber,
|
||||
) -> RethResult<Option<StoredBlockBodyIndices>> {
|
||||
self.database.provider()?.block_body_indices(number)
|
||||
}
|
||||
|
||||
@ -254,7 +263,7 @@ where
|
||||
/// spot, and we want fast querying.**
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn block_with_senders(&self, number: BlockNumber) -> Result<Option<BlockWithSenders>> {
|
||||
fn block_with_senders(&self, number: BlockNumber) -> RethResult<Option<BlockWithSenders>> {
|
||||
self.database.provider()?.block_with_senders(number)
|
||||
}
|
||||
}
|
||||
@ -264,59 +273,62 @@ where
|
||||
DB: Database,
|
||||
Tree: BlockchainTreeViewer + Send + Sync,
|
||||
{
|
||||
fn transaction_id(&self, tx_hash: TxHash) -> Result<Option<TxNumber>> {
|
||||
fn transaction_id(&self, tx_hash: TxHash) -> RethResult<Option<TxNumber>> {
|
||||
self.database.provider()?.transaction_id(tx_hash)
|
||||
}
|
||||
|
||||
fn transaction_by_id(&self, id: TxNumber) -> Result<Option<TransactionSigned>> {
|
||||
fn transaction_by_id(&self, id: TxNumber) -> RethResult<Option<TransactionSigned>> {
|
||||
self.database.provider()?.transaction_by_id(id)
|
||||
}
|
||||
|
||||
fn transaction_by_id_no_hash(&self, id: TxNumber) -> Result<Option<TransactionSignedNoHash>> {
|
||||
fn transaction_by_id_no_hash(
|
||||
&self,
|
||||
id: TxNumber,
|
||||
) -> RethResult<Option<TransactionSignedNoHash>> {
|
||||
self.database.provider()?.transaction_by_id_no_hash(id)
|
||||
}
|
||||
|
||||
fn transaction_by_hash(&self, hash: TxHash) -> Result<Option<TransactionSigned>> {
|
||||
fn transaction_by_hash(&self, hash: TxHash) -> RethResult<Option<TransactionSigned>> {
|
||||
self.database.provider()?.transaction_by_hash(hash)
|
||||
}
|
||||
|
||||
fn transaction_by_hash_with_meta(
|
||||
&self,
|
||||
tx_hash: TxHash,
|
||||
) -> Result<Option<(TransactionSigned, TransactionMeta)>> {
|
||||
) -> RethResult<Option<(TransactionSigned, TransactionMeta)>> {
|
||||
self.database.provider()?.transaction_by_hash_with_meta(tx_hash)
|
||||
}
|
||||
|
||||
fn transaction_block(&self, id: TxNumber) -> Result<Option<BlockNumber>> {
|
||||
fn transaction_block(&self, id: TxNumber) -> RethResult<Option<BlockNumber>> {
|
||||
self.database.provider()?.transaction_block(id)
|
||||
}
|
||||
|
||||
fn transactions_by_block(
|
||||
&self,
|
||||
id: BlockHashOrNumber,
|
||||
) -> Result<Option<Vec<TransactionSigned>>> {
|
||||
) -> RethResult<Option<Vec<TransactionSigned>>> {
|
||||
self.database.provider()?.transactions_by_block(id)
|
||||
}
|
||||
|
||||
fn transactions_by_block_range(
|
||||
&self,
|
||||
range: impl RangeBounds<BlockNumber>,
|
||||
) -> Result<Vec<Vec<TransactionSigned>>> {
|
||||
) -> RethResult<Vec<Vec<TransactionSigned>>> {
|
||||
self.database.provider()?.transactions_by_block_range(range)
|
||||
}
|
||||
|
||||
fn transactions_by_tx_range(
|
||||
&self,
|
||||
range: impl RangeBounds<TxNumber>,
|
||||
) -> Result<Vec<TransactionSignedNoHash>> {
|
||||
) -> RethResult<Vec<TransactionSignedNoHash>> {
|
||||
self.database.provider()?.transactions_by_tx_range(range)
|
||||
}
|
||||
|
||||
fn senders_by_tx_range(&self, range: impl RangeBounds<TxNumber>) -> Result<Vec<Address>> {
|
||||
fn senders_by_tx_range(&self, range: impl RangeBounds<TxNumber>) -> RethResult<Vec<Address>> {
|
||||
self.database.provider()?.senders_by_tx_range(range)
|
||||
}
|
||||
|
||||
fn transaction_sender(&self, id: TxNumber) -> Result<Option<Address>> {
|
||||
fn transaction_sender(&self, id: TxNumber) -> RethResult<Option<Address>> {
|
||||
self.database.provider()?.transaction_sender(id)
|
||||
}
|
||||
}
|
||||
@ -326,15 +338,15 @@ where
|
||||
DB: Database,
|
||||
Tree: Send + Sync,
|
||||
{
|
||||
fn receipt(&self, id: TxNumber) -> Result<Option<Receipt>> {
|
||||
fn receipt(&self, id: TxNumber) -> RethResult<Option<Receipt>> {
|
||||
self.database.provider()?.receipt(id)
|
||||
}
|
||||
|
||||
fn receipt_by_hash(&self, hash: TxHash) -> Result<Option<Receipt>> {
|
||||
fn receipt_by_hash(&self, hash: TxHash) -> RethResult<Option<Receipt>> {
|
||||
self.database.provider()?.receipt_by_hash(hash)
|
||||
}
|
||||
|
||||
fn receipts_by_block(&self, block: BlockHashOrNumber) -> Result<Option<Vec<Receipt>>> {
|
||||
fn receipts_by_block(&self, block: BlockHashOrNumber) -> RethResult<Option<Vec<Receipt>>> {
|
||||
self.database.provider()?.receipts_by_block(block)
|
||||
}
|
||||
}
|
||||
@ -343,7 +355,7 @@ where
|
||||
DB: Database,
|
||||
Tree: BlockchainTreeViewer + Send + Sync,
|
||||
{
|
||||
fn receipts_by_block_id(&self, block: BlockId) -> Result<Option<Vec<Receipt>>> {
|
||||
fn receipts_by_block_id(&self, block: BlockId) -> RethResult<Option<Vec<Receipt>>> {
|
||||
match block {
|
||||
BlockId::Hash(rpc_block_hash) => {
|
||||
let mut receipts = self.receipts_by_block(rpc_block_hash.block_hash.into())?;
|
||||
@ -375,11 +387,11 @@ where
|
||||
&self,
|
||||
id: BlockHashOrNumber,
|
||||
timestamp: u64,
|
||||
) -> Result<Option<Vec<Withdrawal>>> {
|
||||
) -> RethResult<Option<Vec<Withdrawal>>> {
|
||||
self.database.provider()?.withdrawals_by_block(id, timestamp)
|
||||
}
|
||||
|
||||
fn latest_withdrawal(&self) -> Result<Option<Withdrawal>> {
|
||||
fn latest_withdrawal(&self) -> RethResult<Option<Withdrawal>> {
|
||||
self.database.provider()?.latest_withdrawal()
|
||||
}
|
||||
}
|
||||
@ -389,11 +401,11 @@ where
|
||||
DB: Database,
|
||||
Tree: Send + Sync,
|
||||
{
|
||||
fn get_stage_checkpoint(&self, id: StageId) -> Result<Option<StageCheckpoint>> {
|
||||
fn get_stage_checkpoint(&self, id: StageId) -> RethResult<Option<StageCheckpoint>> {
|
||||
self.database.provider()?.get_stage_checkpoint(id)
|
||||
}
|
||||
|
||||
fn get_stage_checkpoint_progress(&self, id: StageId) -> Result<Option<Vec<u8>>> {
|
||||
fn get_stage_checkpoint_progress(&self, id: StageId) -> RethResult<Option<Vec<u8>>> {
|
||||
self.database.provider()?.get_stage_checkpoint_progress(id)
|
||||
}
|
||||
}
|
||||
@ -408,7 +420,7 @@ where
|
||||
cfg: &mut CfgEnv,
|
||||
block_env: &mut BlockEnv,
|
||||
at: BlockHashOrNumber,
|
||||
) -> Result<()> {
|
||||
) -> RethResult<()> {
|
||||
self.database.provider()?.fill_env_at(cfg, block_env, at)
|
||||
}
|
||||
|
||||
@ -417,23 +429,27 @@ where
|
||||
cfg: &mut CfgEnv,
|
||||
block_env: &mut BlockEnv,
|
||||
header: &Header,
|
||||
) -> Result<()> {
|
||||
) -> RethResult<()> {
|
||||
self.database.provider()?.fill_env_with_header(cfg, block_env, header)
|
||||
}
|
||||
|
||||
fn fill_block_env_at(&self, block_env: &mut BlockEnv, at: BlockHashOrNumber) -> Result<()> {
|
||||
fn fill_block_env_at(&self, block_env: &mut BlockEnv, at: BlockHashOrNumber) -> RethResult<()> {
|
||||
self.database.provider()?.fill_block_env_at(block_env, at)
|
||||
}
|
||||
|
||||
fn fill_block_env_with_header(&self, block_env: &mut BlockEnv, header: &Header) -> Result<()> {
|
||||
fn fill_block_env_with_header(
|
||||
&self,
|
||||
block_env: &mut BlockEnv,
|
||||
header: &Header,
|
||||
) -> RethResult<()> {
|
||||
self.database.provider()?.fill_block_env_with_header(block_env, header)
|
||||
}
|
||||
|
||||
fn fill_cfg_env_at(&self, cfg: &mut CfgEnv, at: BlockHashOrNumber) -> Result<()> {
|
||||
fn fill_cfg_env_at(&self, cfg: &mut CfgEnv, at: BlockHashOrNumber) -> RethResult<()> {
|
||||
self.database.provider()?.fill_cfg_env_at(cfg, at)
|
||||
}
|
||||
|
||||
fn fill_cfg_env_with_header(&self, cfg: &mut CfgEnv, header: &Header) -> Result<()> {
|
||||
fn fill_cfg_env_with_header(&self, cfg: &mut CfgEnv, header: &Header) -> RethResult<()> {
|
||||
self.database.provider()?.fill_cfg_env_with_header(cfg, header)
|
||||
}
|
||||
}
|
||||
@ -443,7 +459,7 @@ where
|
||||
DB: Database,
|
||||
Tree: Send + Sync,
|
||||
{
|
||||
fn get_prune_checkpoint(&self, part: PrunePart) -> Result<Option<PruneCheckpoint>> {
|
||||
fn get_prune_checkpoint(&self, part: PrunePart) -> RethResult<Option<PruneCheckpoint>> {
|
||||
self.database.provider()?.get_prune_checkpoint(part)
|
||||
}
|
||||
}
|
||||
@ -464,23 +480,26 @@ where
|
||||
Tree: BlockchainTreePendingStateProvider + BlockchainTreeViewer,
|
||||
{
|
||||
/// Storage provider for latest block
|
||||
fn latest(&self) -> Result<StateProviderBox<'_>> {
|
||||
fn latest(&self) -> RethResult<StateProviderBox<'_>> {
|
||||
trace!(target: "providers::blockchain", "Getting latest block state provider");
|
||||
self.database.latest()
|
||||
}
|
||||
|
||||
fn history_by_block_number(&self, block_number: BlockNumber) -> Result<StateProviderBox<'_>> {
|
||||
fn history_by_block_number(
|
||||
&self,
|
||||
block_number: BlockNumber,
|
||||
) -> RethResult<StateProviderBox<'_>> {
|
||||
trace!(target: "providers::blockchain", ?block_number, "Getting history by block number");
|
||||
self.ensure_canonical_block(block_number)?;
|
||||
self.database.history_by_block_number(block_number)
|
||||
}
|
||||
|
||||
fn history_by_block_hash(&self, block_hash: BlockHash) -> Result<StateProviderBox<'_>> {
|
||||
fn history_by_block_hash(&self, block_hash: BlockHash) -> RethResult<StateProviderBox<'_>> {
|
||||
trace!(target: "providers::blockchain", ?block_hash, "Getting history by block hash");
|
||||
self.database.history_by_block_hash(block_hash)
|
||||
}
|
||||
|
||||
fn state_by_block_hash(&self, block: BlockHash) -> Result<StateProviderBox<'_>> {
|
||||
fn state_by_block_hash(&self, block: BlockHash) -> RethResult<StateProviderBox<'_>> {
|
||||
trace!(target: "providers::blockchain", ?block, "Getting state by block hash");
|
||||
let mut state = self.history_by_block_hash(block);
|
||||
|
||||
@ -496,7 +515,7 @@ where
|
||||
}
|
||||
|
||||
/// Storage provider for pending state.
|
||||
fn pending(&self) -> Result<StateProviderBox<'_>> {
|
||||
fn pending(&self) -> RethResult<StateProviderBox<'_>> {
|
||||
trace!(target: "providers::blockchain", "Getting provider for pending state");
|
||||
|
||||
if let Some(block) = self.tree.pending_block_num_hash() {
|
||||
@ -506,7 +525,7 @@ where
|
||||
self.latest()
|
||||
}
|
||||
|
||||
fn pending_state_by_hash(&self, block_hash: H256) -> Result<Option<StateProviderBox<'_>>> {
|
||||
fn pending_state_by_hash(&self, block_hash: H256) -> RethResult<Option<StateProviderBox<'_>>> {
|
||||
if let Some(state) = self.tree.find_pending_state_provider(block_hash) {
|
||||
return Ok(Some(self.pending_with_provider(state)?))
|
||||
}
|
||||
@ -516,7 +535,7 @@ where
|
||||
fn pending_with_provider(
|
||||
&self,
|
||||
post_state_data: Box<dyn BundleStateDataProvider>,
|
||||
) -> Result<StateProviderBox<'_>> {
|
||||
) -> RethResult<StateProviderBox<'_>> {
|
||||
let canonical_fork = post_state_data.canonical_fork();
|
||||
trace!(target: "providers::blockchain", ?canonical_fork, "Returning post state provider");
|
||||
|
||||
@ -552,19 +571,19 @@ where
|
||||
fn connect_buffered_blocks_to_canonical_hashes_and_finalize(
|
||||
&self,
|
||||
last_finalized_block: BlockNumber,
|
||||
) -> Result<()> {
|
||||
) -> RethResult<()> {
|
||||
self.tree.connect_buffered_blocks_to_canonical_hashes_and_finalize(last_finalized_block)
|
||||
}
|
||||
|
||||
fn connect_buffered_blocks_to_canonical_hashes(&self) -> Result<()> {
|
||||
fn connect_buffered_blocks_to_canonical_hashes(&self) -> RethResult<()> {
|
||||
self.tree.connect_buffered_blocks_to_canonical_hashes()
|
||||
}
|
||||
|
||||
fn make_canonical(&self, block_hash: &BlockHash) -> Result<CanonicalOutcome> {
|
||||
fn make_canonical(&self, block_hash: &BlockHash) -> RethResult<CanonicalOutcome> {
|
||||
self.tree.make_canonical(block_hash)
|
||||
}
|
||||
|
||||
fn unwind(&self, unwind_to: BlockNumber) -> Result<()> {
|
||||
fn unwind(&self, unwind_to: BlockNumber) -> RethResult<()> {
|
||||
self.tree.unwind(unwind_to)
|
||||
}
|
||||
}
|
||||
@ -602,7 +621,7 @@ where
|
||||
self.tree.find_canonical_ancestor(hash)
|
||||
}
|
||||
|
||||
fn is_canonical(&self, hash: BlockHash) -> std::result::Result<bool, Error> {
|
||||
fn is_canonical(&self, hash: BlockHash) -> std::result::Result<bool, RethError> {
|
||||
self.tree.is_canonical(hash)
|
||||
}
|
||||
|
||||
@ -672,7 +691,7 @@ where
|
||||
Self: BlockReader + BlockIdReader + ReceiptProviderIdExt,
|
||||
Tree: BlockchainTreeEngine,
|
||||
{
|
||||
fn block_by_id(&self, id: BlockId) -> Result<Option<Block>> {
|
||||
fn block_by_id(&self, id: BlockId) -> RethResult<Option<Block>> {
|
||||
match id {
|
||||
BlockId::Number(num) => self.block_by_number_or_tag(num),
|
||||
BlockId::Hash(hash) => {
|
||||
@ -690,7 +709,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
fn header_by_number_or_tag(&self, id: BlockNumberOrTag) -> Result<Option<Header>> {
|
||||
fn header_by_number_or_tag(&self, id: BlockNumberOrTag) -> RethResult<Option<Header>> {
|
||||
match id {
|
||||
BlockNumberOrTag::Latest => Ok(Some(self.chain_info.get_canonical_head().unseal())),
|
||||
BlockNumberOrTag::Finalized => {
|
||||
@ -703,7 +722,10 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
fn sealed_header_by_number_or_tag(&self, id: BlockNumberOrTag) -> Result<Option<SealedHeader>> {
|
||||
fn sealed_header_by_number_or_tag(
|
||||
&self,
|
||||
id: BlockNumberOrTag,
|
||||
) -> RethResult<Option<SealedHeader>> {
|
||||
match id {
|
||||
BlockNumberOrTag::Latest => Ok(Some(self.chain_info.get_canonical_head())),
|
||||
BlockNumberOrTag::Finalized => Ok(self.chain_info.get_finalized_header()),
|
||||
@ -718,7 +740,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
fn sealed_header_by_id(&self, id: BlockId) -> Result<Option<SealedHeader>> {
|
||||
fn sealed_header_by_id(&self, id: BlockId) -> RethResult<Option<SealedHeader>> {
|
||||
match id {
|
||||
BlockId::Number(num) => self.sealed_header_by_number_or_tag(num),
|
||||
BlockId::Hash(hash) => {
|
||||
@ -727,14 +749,14 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
fn header_by_id(&self, id: BlockId) -> Result<Option<Header>> {
|
||||
fn header_by_id(&self, id: BlockId) -> RethResult<Option<Header>> {
|
||||
match id {
|
||||
BlockId::Number(num) => self.header_by_number_or_tag(num),
|
||||
BlockId::Hash(hash) => self.header(&hash.block_hash),
|
||||
}
|
||||
}
|
||||
|
||||
fn ommers_by_id(&self, id: BlockId) -> Result<Option<Vec<Header>>> {
|
||||
fn ommers_by_id(&self, id: BlockId) -> RethResult<Option<Vec<Header>>> {
|
||||
match id {
|
||||
BlockId::Number(num) => self.ommers_by_number_or_tag(num),
|
||||
BlockId::Hash(hash) => {
|
||||
@ -774,7 +796,10 @@ where
|
||||
DB: Database,
|
||||
Tree: Sync + Send,
|
||||
{
|
||||
fn account_block_changeset(&self, block_number: BlockNumber) -> Result<Vec<AccountBeforeTx>> {
|
||||
fn account_block_changeset(
|
||||
&self,
|
||||
block_number: BlockNumber,
|
||||
) -> RethResult<Vec<AccountBeforeTx>> {
|
||||
self.database.provider()?.account_block_changeset(block_number)
|
||||
}
|
||||
}
|
||||
|
||||
@ -10,7 +10,7 @@ use reth_db::{
|
||||
transaction::DbTx,
|
||||
BlockNumberList,
|
||||
};
|
||||
use reth_interfaces::Result;
|
||||
use reth_interfaces::RethResult;
|
||||
use reth_primitives::{
|
||||
Account, Address, BlockNumber, Bytecode, Bytes, StorageKey, StorageValue, H256,
|
||||
};
|
||||
@ -68,7 +68,7 @@ impl<'a, 'b, TX: DbTx<'a>> HistoricalStateProviderRef<'a, 'b, TX> {
|
||||
}
|
||||
|
||||
/// Lookup an account in the AccountHistory table
|
||||
pub fn account_history_lookup(&self, address: Address) -> Result<HistoryInfo> {
|
||||
pub fn account_history_lookup(&self, address: Address) -> RethResult<HistoryInfo> {
|
||||
if !self.lowest_available_blocks.is_account_history_available(self.block_number) {
|
||||
return Err(ProviderError::StateAtBlockPruned(self.block_number).into())
|
||||
}
|
||||
@ -87,7 +87,7 @@ impl<'a, 'b, TX: DbTx<'a>> HistoricalStateProviderRef<'a, 'b, TX> {
|
||||
&self,
|
||||
address: Address,
|
||||
storage_key: StorageKey,
|
||||
) -> Result<HistoryInfo> {
|
||||
) -> RethResult<HistoryInfo> {
|
||||
if !self.lowest_available_blocks.is_storage_history_available(self.block_number) {
|
||||
return Err(ProviderError::StateAtBlockPruned(self.block_number).into())
|
||||
}
|
||||
@ -106,7 +106,7 @@ impl<'a, 'b, TX: DbTx<'a>> HistoricalStateProviderRef<'a, 'b, TX> {
|
||||
key: K,
|
||||
key_filter: impl Fn(&K) -> bool,
|
||||
lowest_available_block_number: Option<BlockNumber>,
|
||||
) -> Result<HistoryInfo>
|
||||
) -> RethResult<HistoryInfo>
|
||||
where
|
||||
T: Table<Key = K, Value = BlockNumberList>,
|
||||
{
|
||||
@ -160,7 +160,7 @@ impl<'a, 'b, TX: DbTx<'a>> HistoricalStateProviderRef<'a, 'b, TX> {
|
||||
|
||||
impl<'a, 'b, TX: DbTx<'a>> AccountReader for HistoricalStateProviderRef<'a, 'b, TX> {
|
||||
/// Get basic account information.
|
||||
fn basic_account(&self, address: Address) -> Result<Option<Account>> {
|
||||
fn basic_account(&self, address: Address) -> RethResult<Option<Account>> {
|
||||
match self.account_history_lookup(address)? {
|
||||
HistoryInfo::NotYetWritten => Ok(None),
|
||||
HistoryInfo::InChangeset(changeset_block_number) => Ok(self
|
||||
@ -182,11 +182,15 @@ impl<'a, 'b, TX: DbTx<'a>> AccountReader for HistoricalStateProviderRef<'a, 'b,
|
||||
|
||||
impl<'a, 'b, TX: DbTx<'a>> BlockHashReader for HistoricalStateProviderRef<'a, 'b, TX> {
|
||||
/// Get block hash by number.
|
||||
fn block_hash(&self, number: u64) -> Result<Option<H256>> {
|
||||
fn block_hash(&self, number: u64) -> RethResult<Option<H256>> {
|
||||
self.tx.get::<tables::CanonicalHeaders>(number).map_err(Into::into)
|
||||
}
|
||||
|
||||
fn canonical_hashes_range(&self, start: BlockNumber, end: BlockNumber) -> Result<Vec<H256>> {
|
||||
fn canonical_hashes_range(
|
||||
&self,
|
||||
start: BlockNumber,
|
||||
end: BlockNumber,
|
||||
) -> RethResult<Vec<H256>> {
|
||||
let range = start..end;
|
||||
self.tx
|
||||
.cursor_read::<tables::CanonicalHeaders>()
|
||||
@ -194,21 +198,25 @@ impl<'a, 'b, TX: DbTx<'a>> BlockHashReader for HistoricalStateProviderRef<'a, 'b
|
||||
cursor
|
||||
.walk_range(range)?
|
||||
.map(|result| result.map(|(_, hash)| hash).map_err(Into::into))
|
||||
.collect::<Result<Vec<_>>>()
|
||||
.collect::<RethResult<Vec<_>>>()
|
||||
})?
|
||||
.map_err(Into::into)
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a, 'b, TX: DbTx<'a>> StateRootProvider for HistoricalStateProviderRef<'a, 'b, TX> {
|
||||
fn state_root(&self, _post_state: BundleStateWithReceipts) -> Result<H256> {
|
||||
fn state_root(&self, _post_state: BundleStateWithReceipts) -> RethResult<H256> {
|
||||
Err(ProviderError::StateRootNotAvailableForHistoricalBlock.into())
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a, 'b, TX: DbTx<'a>> StateProvider for HistoricalStateProviderRef<'a, 'b, TX> {
|
||||
/// Get storage.
|
||||
fn storage(&self, address: Address, storage_key: StorageKey) -> Result<Option<StorageValue>> {
|
||||
fn storage(
|
||||
&self,
|
||||
address: Address,
|
||||
storage_key: StorageKey,
|
||||
) -> RethResult<Option<StorageValue>> {
|
||||
match self.storage_history_lookup(address, storage_key)? {
|
||||
HistoryInfo::NotYetWritten => Ok(None),
|
||||
HistoryInfo::InChangeset(changeset_block_number) => Ok(Some(
|
||||
@ -234,7 +242,7 @@ impl<'a, 'b, TX: DbTx<'a>> StateProvider for HistoricalStateProviderRef<'a, 'b,
|
||||
}
|
||||
|
||||
/// Get account code by its hash
|
||||
fn bytecode_by_hash(&self, code_hash: H256) -> Result<Option<Bytecode>> {
|
||||
fn bytecode_by_hash(&self, code_hash: H256) -> RethResult<Option<Bytecode>> {
|
||||
self.tx.get::<tables::Bytecodes>(code_hash).map_err(Into::into)
|
||||
}
|
||||
|
||||
@ -243,7 +251,7 @@ impl<'a, 'b, TX: DbTx<'a>> StateProvider for HistoricalStateProviderRef<'a, 'b,
|
||||
&self,
|
||||
_address: Address,
|
||||
_keys: &[H256],
|
||||
) -> Result<(Vec<Bytes>, H256, Vec<Vec<Bytes>>)> {
|
||||
) -> RethResult<(Vec<Bytes>, H256, Vec<Vec<Bytes>>)> {
|
||||
Err(ProviderError::StateRootNotAvailableForHistoricalBlock.into())
|
||||
}
|
||||
}
|
||||
|
||||
@ -7,7 +7,7 @@ use reth_db::{
|
||||
tables,
|
||||
transaction::DbTx,
|
||||
};
|
||||
use reth_interfaces::{provider::ProviderError, Result};
|
||||
use reth_interfaces::{provider::ProviderError, RethError, RethResult};
|
||||
use reth_primitives::{
|
||||
keccak256, Account, Address, BlockNumber, Bytecode, Bytes, StorageKey, StorageValue, H256,
|
||||
};
|
||||
@ -30,18 +30,22 @@ impl<'a, 'b, TX: DbTx<'a>> LatestStateProviderRef<'a, 'b, TX> {
|
||||
|
||||
impl<'a, 'b, TX: DbTx<'a>> AccountReader for LatestStateProviderRef<'a, 'b, TX> {
|
||||
/// Get basic account information.
|
||||
fn basic_account(&self, address: Address) -> Result<Option<Account>> {
|
||||
fn basic_account(&self, address: Address) -> RethResult<Option<Account>> {
|
||||
self.db.get::<tables::PlainAccountState>(address).map_err(Into::into)
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a, 'b, TX: DbTx<'a>> BlockHashReader for LatestStateProviderRef<'a, 'b, TX> {
|
||||
/// Get block hash by number.
|
||||
fn block_hash(&self, number: u64) -> Result<Option<H256>> {
|
||||
fn block_hash(&self, number: u64) -> RethResult<Option<H256>> {
|
||||
self.db.get::<tables::CanonicalHeaders>(number).map_err(Into::into)
|
||||
}
|
||||
|
||||
fn canonical_hashes_range(&self, start: BlockNumber, end: BlockNumber) -> Result<Vec<H256>> {
|
||||
fn canonical_hashes_range(
|
||||
&self,
|
||||
start: BlockNumber,
|
||||
end: BlockNumber,
|
||||
) -> RethResult<Vec<H256>> {
|
||||
let range = start..end;
|
||||
self.db
|
||||
.cursor_read::<tables::CanonicalHeaders>()
|
||||
@ -49,23 +53,25 @@ impl<'a, 'b, TX: DbTx<'a>> BlockHashReader for LatestStateProviderRef<'a, 'b, TX
|
||||
cursor
|
||||
.walk_range(range)?
|
||||
.map(|result| result.map(|(_, hash)| hash).map_err(Into::into))
|
||||
.collect::<Result<Vec<_>>>()
|
||||
.collect::<RethResult<Vec<_>>>()
|
||||
})?
|
||||
.map_err(Into::into)
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a, 'b, TX: DbTx<'a>> StateRootProvider for LatestStateProviderRef<'a, 'b, TX> {
|
||||
fn state_root(&self, bundle_state: BundleStateWithReceipts) -> Result<H256> {
|
||||
bundle_state
|
||||
.state_root_slow(self.db)
|
||||
.map_err(|err| reth_interfaces::Error::Database(err.into()))
|
||||
fn state_root(&self, bundle_state: BundleStateWithReceipts) -> RethResult<H256> {
|
||||
bundle_state.state_root_slow(self.db).map_err(|err| RethError::Database(err.into()))
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a, 'b, TX: DbTx<'a>> StateProvider for LatestStateProviderRef<'a, 'b, TX> {
|
||||
/// Get storage.
|
||||
fn storage(&self, account: Address, storage_key: StorageKey) -> Result<Option<StorageValue>> {
|
||||
fn storage(
|
||||
&self,
|
||||
account: Address,
|
||||
storage_key: StorageKey,
|
||||
) -> RethResult<Option<StorageValue>> {
|
||||
let mut cursor = self.db.cursor_dup_read::<tables::PlainStorageState>()?;
|
||||
if let Some(entry) = cursor.seek_by_key_subkey(account, storage_key)? {
|
||||
if entry.key == storage_key {
|
||||
@ -76,7 +82,7 @@ impl<'a, 'b, TX: DbTx<'a>> StateProvider for LatestStateProviderRef<'a, 'b, TX>
|
||||
}
|
||||
|
||||
/// Get account code by its hash
|
||||
fn bytecode_by_hash(&self, code_hash: H256) -> Result<Option<Bytecode>> {
|
||||
fn bytecode_by_hash(&self, code_hash: H256) -> RethResult<Option<Bytecode>> {
|
||||
self.db.get::<tables::Bytecodes>(code_hash).map_err(Into::into)
|
||||
}
|
||||
|
||||
@ -84,7 +90,7 @@ impl<'a, 'b, TX: DbTx<'a>> StateProvider for LatestStateProviderRef<'a, 'b, TX>
|
||||
&self,
|
||||
address: Address,
|
||||
_keys: &[H256],
|
||||
) -> Result<(Vec<Bytes>, H256, Vec<Vec<Bytes>>)> {
|
||||
) -> RethResult<(Vec<Bytes>, H256, Vec<Vec<Bytes>>)> {
|
||||
let _hashed_address = keccak256(address);
|
||||
let _root = self
|
||||
.db
|
||||
|
||||
@ -31,19 +31,19 @@ macro_rules! delegate_provider_impls {
|
||||
$crate::providers::state::macros::delegate_impls_to_as_ref!(
|
||||
for $target =>
|
||||
StateRootProvider $(where [$($generics)*])? {
|
||||
fn state_root(&self, state: crate::BundleStateWithReceipts) -> reth_interfaces::Result<reth_primitives::H256>;
|
||||
fn state_root(&self, state: crate::BundleStateWithReceipts) -> reth_interfaces::RethResult<reth_primitives::H256>;
|
||||
}
|
||||
AccountReader $(where [$($generics)*])? {
|
||||
fn basic_account(&self, address: reth_primitives::Address) -> reth_interfaces::Result<Option<reth_primitives::Account>>;
|
||||
fn basic_account(&self, address: reth_primitives::Address) -> reth_interfaces::RethResult<Option<reth_primitives::Account>>;
|
||||
}
|
||||
BlockHashReader $(where [$($generics)*])? {
|
||||
fn block_hash(&self, number: u64) -> reth_interfaces::Result<Option<reth_primitives::H256>>;
|
||||
fn canonical_hashes_range(&self, start: reth_primitives::BlockNumber, end: reth_primitives::BlockNumber) -> reth_interfaces::Result<Vec<reth_primitives::H256>>;
|
||||
fn block_hash(&self, number: u64) -> reth_interfaces::RethResult<Option<reth_primitives::H256>>;
|
||||
fn canonical_hashes_range(&self, start: reth_primitives::BlockNumber, end: reth_primitives::BlockNumber) -> reth_interfaces::RethResult<Vec<reth_primitives::H256>>;
|
||||
}
|
||||
StateProvider $(where [$($generics)*])?{
|
||||
fn storage(&self, account: reth_primitives::Address, storage_key: reth_primitives::StorageKey) -> reth_interfaces::Result<Option<reth_primitives::StorageValue>>;
|
||||
fn proof(&self, address: reth_primitives::Address, keys: &[reth_primitives::H256]) -> reth_interfaces::Result<(Vec<reth_primitives::Bytes>, reth_primitives::H256, Vec<Vec<reth_primitives::Bytes>>)>;
|
||||
fn bytecode_by_hash(&self, code_hash: reth_primitives::H256) -> reth_interfaces::Result<Option<reth_primitives::Bytecode>>;
|
||||
fn storage(&self, account: reth_primitives::Address, storage_key: reth_primitives::StorageKey) -> reth_interfaces::RethResult<Option<reth_primitives::StorageValue>>;
|
||||
fn proof(&self, address: reth_primitives::Address, keys: &[reth_primitives::H256]) -> reth_interfaces::RethResult<(Vec<reth_primitives::Bytes>, reth_primitives::H256, Vec<Vec<reth_primitives::Bytes>>)>;
|
||||
fn bytecode_by_hash(&self, code_hash: reth_primitives::H256) -> reth_interfaces::RethResult<Option<reth_primitives::Bytecode>>;
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
@ -8,7 +8,7 @@ use crate::{
|
||||
};
|
||||
use parking_lot::Mutex;
|
||||
use reth_db::models::StoredBlockBodyIndices;
|
||||
use reth_interfaces::{provider::ProviderError, Result};
|
||||
use reth_interfaces::{provider::ProviderError, RethResult};
|
||||
use reth_primitives::{
|
||||
keccak256, Account, Address, Block, BlockHash, BlockHashOrNumber, BlockId, BlockNumber,
|
||||
BlockWithSenders, Bytecode, Bytes, ChainInfo, ChainSpec, Header, Receipt, SealedBlock,
|
||||
@ -124,17 +124,17 @@ impl MockEthProvider {
|
||||
}
|
||||
|
||||
impl HeaderProvider for MockEthProvider {
|
||||
fn header(&self, block_hash: &BlockHash) -> Result<Option<Header>> {
|
||||
fn header(&self, block_hash: &BlockHash) -> RethResult<Option<Header>> {
|
||||
let lock = self.headers.lock();
|
||||
Ok(lock.get(block_hash).cloned())
|
||||
}
|
||||
|
||||
fn header_by_number(&self, num: u64) -> Result<Option<Header>> {
|
||||
fn header_by_number(&self, num: u64) -> RethResult<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>> {
|
||||
fn header_td(&self, hash: &BlockHash) -> RethResult<Option<U256>> {
|
||||
let lock = self.headers.lock();
|
||||
Ok(lock.get(hash).map(|target| {
|
||||
lock.values()
|
||||
@ -143,7 +143,7 @@ impl HeaderProvider for MockEthProvider {
|
||||
}))
|
||||
}
|
||||
|
||||
fn header_td_by_number(&self, number: BlockNumber) -> Result<Option<U256>> {
|
||||
fn header_td_by_number(&self, number: BlockNumber) -> RethResult<Option<U256>> {
|
||||
let lock = self.headers.lock();
|
||||
let sum = lock
|
||||
.values()
|
||||
@ -152,7 +152,7 @@ impl HeaderProvider for MockEthProvider {
|
||||
Ok(Some(sum))
|
||||
}
|
||||
|
||||
fn headers_range(&self, range: impl RangeBounds<BlockNumber>) -> Result<Vec<Header>> {
|
||||
fn headers_range(&self, range: impl RangeBounds<BlockNumber>) -> RethResult<Vec<Header>> {
|
||||
let lock = self.headers.lock();
|
||||
|
||||
let mut headers: Vec<_> =
|
||||
@ -165,11 +165,11 @@ impl HeaderProvider for MockEthProvider {
|
||||
fn sealed_headers_range(
|
||||
&self,
|
||||
range: impl RangeBounds<BlockNumber>,
|
||||
) -> Result<Vec<SealedHeader>> {
|
||||
) -> RethResult<Vec<SealedHeader>> {
|
||||
Ok(self.headers_range(range)?.into_iter().map(|h| h.seal_slow()).collect())
|
||||
}
|
||||
|
||||
fn sealed_header(&self, number: BlockNumber) -> Result<Option<SealedHeader>> {
|
||||
fn sealed_header(&self, number: BlockNumber) -> RethResult<Option<SealedHeader>> {
|
||||
Ok(self.header_by_number(number)?.map(|h| h.seal_slow()))
|
||||
}
|
||||
}
|
||||
@ -181,7 +181,7 @@ impl ChainSpecProvider for MockEthProvider {
|
||||
}
|
||||
|
||||
impl TransactionsProvider for MockEthProvider {
|
||||
fn transaction_id(&self, tx_hash: TxHash) -> Result<Option<TxNumber>> {
|
||||
fn transaction_id(&self, tx_hash: TxHash) -> RethResult<Option<TxNumber>> {
|
||||
let lock = self.blocks.lock();
|
||||
let tx_number = lock
|
||||
.values()
|
||||
@ -192,14 +192,17 @@ impl TransactionsProvider for MockEthProvider {
|
||||
Ok(tx_number)
|
||||
}
|
||||
|
||||
fn transaction_by_id(&self, id: TxNumber) -> Result<Option<TransactionSigned>> {
|
||||
fn transaction_by_id(&self, id: TxNumber) -> RethResult<Option<TransactionSigned>> {
|
||||
let lock = self.blocks.lock();
|
||||
let transaction = lock.values().flat_map(|block| &block.body).nth(id as usize).cloned();
|
||||
|
||||
Ok(transaction)
|
||||
}
|
||||
|
||||
fn transaction_by_id_no_hash(&self, id: TxNumber) -> Result<Option<TransactionSignedNoHash>> {
|
||||
fn transaction_by_id_no_hash(
|
||||
&self,
|
||||
id: TxNumber,
|
||||
) -> RethResult<Option<TransactionSignedNoHash>> {
|
||||
let lock = self.blocks.lock();
|
||||
let transaction = lock
|
||||
.values()
|
||||
@ -210,7 +213,7 @@ impl TransactionsProvider for MockEthProvider {
|
||||
Ok(transaction)
|
||||
}
|
||||
|
||||
fn transaction_by_hash(&self, hash: TxHash) -> Result<Option<TransactionSigned>> {
|
||||
fn transaction_by_hash(&self, hash: TxHash) -> RethResult<Option<TransactionSigned>> {
|
||||
Ok(self
|
||||
.blocks
|
||||
.lock()
|
||||
@ -221,7 +224,7 @@ impl TransactionsProvider for MockEthProvider {
|
||||
fn transaction_by_hash_with_meta(
|
||||
&self,
|
||||
hash: TxHash,
|
||||
) -> Result<Option<(TransactionSigned, TransactionMeta)>> {
|
||||
) -> RethResult<Option<(TransactionSigned, TransactionMeta)>> {
|
||||
let lock = self.blocks.lock();
|
||||
for (block_hash, block) in lock.iter() {
|
||||
for (index, tx) in block.body.iter().enumerate() {
|
||||
@ -241,7 +244,7 @@ impl TransactionsProvider for MockEthProvider {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn transaction_block(&self, id: TxNumber) -> Result<Option<BlockNumber>> {
|
||||
fn transaction_block(&self, id: TxNumber) -> RethResult<Option<BlockNumber>> {
|
||||
let lock = self.blocks.lock();
|
||||
let mut current_tx_number: TxNumber = 0;
|
||||
for block in lock.values() {
|
||||
@ -256,14 +259,14 @@ impl TransactionsProvider for MockEthProvider {
|
||||
fn transactions_by_block(
|
||||
&self,
|
||||
id: BlockHashOrNumber,
|
||||
) -> Result<Option<Vec<TransactionSigned>>> {
|
||||
) -> RethResult<Option<Vec<TransactionSigned>>> {
|
||||
Ok(self.block(id)?.map(|b| b.body))
|
||||
}
|
||||
|
||||
fn transactions_by_block_range(
|
||||
&self,
|
||||
range: impl RangeBounds<reth_primitives::BlockNumber>,
|
||||
) -> Result<Vec<Vec<TransactionSigned>>> {
|
||||
) -> RethResult<Vec<Vec<TransactionSigned>>> {
|
||||
// init btreemap so we can return in order
|
||||
let mut map = BTreeMap::new();
|
||||
for (_, block) in self.blocks.lock().iter() {
|
||||
@ -278,7 +281,7 @@ impl TransactionsProvider for MockEthProvider {
|
||||
fn transactions_by_tx_range(
|
||||
&self,
|
||||
range: impl RangeBounds<TxNumber>,
|
||||
) -> Result<Vec<reth_primitives::TransactionSignedNoHash>> {
|
||||
) -> RethResult<Vec<reth_primitives::TransactionSignedNoHash>> {
|
||||
let lock = self.blocks.lock();
|
||||
let transactions = lock
|
||||
.values()
|
||||
@ -296,7 +299,7 @@ impl TransactionsProvider for MockEthProvider {
|
||||
Ok(transactions)
|
||||
}
|
||||
|
||||
fn senders_by_tx_range(&self, range: impl RangeBounds<TxNumber>) -> Result<Vec<Address>> {
|
||||
fn senders_by_tx_range(&self, range: impl RangeBounds<TxNumber>) -> RethResult<Vec<Address>> {
|
||||
let lock = self.blocks.lock();
|
||||
let transactions = lock
|
||||
.values()
|
||||
@ -314,21 +317,21 @@ impl TransactionsProvider for MockEthProvider {
|
||||
Ok(transactions)
|
||||
}
|
||||
|
||||
fn transaction_sender(&self, id: TxNumber) -> Result<Option<Address>> {
|
||||
fn transaction_sender(&self, id: TxNumber) -> RethResult<Option<Address>> {
|
||||
self.transaction_by_id(id).map(|tx_option| tx_option.map(|tx| tx.recover_signer().unwrap()))
|
||||
}
|
||||
}
|
||||
|
||||
impl ReceiptProvider for MockEthProvider {
|
||||
fn receipt(&self, _id: TxNumber) -> Result<Option<Receipt>> {
|
||||
fn receipt(&self, _id: TxNumber) -> RethResult<Option<Receipt>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn receipt_by_hash(&self, _hash: TxHash) -> Result<Option<Receipt>> {
|
||||
fn receipt_by_hash(&self, _hash: TxHash) -> RethResult<Option<Receipt>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn receipts_by_block(&self, _block: BlockHashOrNumber) -> Result<Option<Vec<Receipt>>> {
|
||||
fn receipts_by_block(&self, _block: BlockHashOrNumber) -> RethResult<Option<Vec<Receipt>>> {
|
||||
Ok(None)
|
||||
}
|
||||
}
|
||||
@ -336,14 +339,18 @@ impl ReceiptProvider for MockEthProvider {
|
||||
impl ReceiptProviderIdExt for MockEthProvider {}
|
||||
|
||||
impl BlockHashReader for MockEthProvider {
|
||||
fn block_hash(&self, number: u64) -> Result<Option<H256>> {
|
||||
fn block_hash(&self, number: u64) -> RethResult<Option<H256>> {
|
||||
let lock = self.blocks.lock();
|
||||
|
||||
let hash = lock.iter().find_map(|(hash, b)| (b.number == number).then_some(*hash));
|
||||
Ok(hash)
|
||||
}
|
||||
|
||||
fn canonical_hashes_range(&self, start: BlockNumber, end: BlockNumber) -> Result<Vec<H256>> {
|
||||
fn canonical_hashes_range(
|
||||
&self,
|
||||
start: BlockNumber,
|
||||
end: BlockNumber,
|
||||
) -> RethResult<Vec<H256>> {
|
||||
let range = start..end;
|
||||
let lock = self.blocks.lock();
|
||||
|
||||
@ -356,7 +363,7 @@ impl BlockHashReader for MockEthProvider {
|
||||
}
|
||||
|
||||
impl BlockNumReader for MockEthProvider {
|
||||
fn chain_info(&self) -> Result<ChainInfo> {
|
||||
fn chain_info(&self) -> RethResult<ChainInfo> {
|
||||
let best_block_number = self.best_block_number()?;
|
||||
let lock = self.headers.lock();
|
||||
|
||||
@ -367,7 +374,7 @@ impl BlockNumReader for MockEthProvider {
|
||||
.unwrap_or_default())
|
||||
}
|
||||
|
||||
fn best_block_number(&self) -> Result<BlockNumber> {
|
||||
fn best_block_number(&self) -> RethResult<BlockNumber> {
|
||||
let lock = self.headers.lock();
|
||||
Ok(lock
|
||||
.iter()
|
||||
@ -376,11 +383,11 @@ impl BlockNumReader for MockEthProvider {
|
||||
.ok_or(ProviderError::BestBlockNotFound)?)
|
||||
}
|
||||
|
||||
fn last_block_number(&self) -> Result<BlockNumber> {
|
||||
fn last_block_number(&self) -> RethResult<BlockNumber> {
|
||||
self.best_block_number()
|
||||
}
|
||||
|
||||
fn block_number(&self, hash: H256) -> Result<Option<reth_primitives::BlockNumber>> {
|
||||
fn block_number(&self, hash: H256) -> RethResult<Option<reth_primitives::BlockNumber>> {
|
||||
let lock = self.blocks.lock();
|
||||
let num = lock.iter().find_map(|(h, b)| (*h == hash).then_some(b.number));
|
||||
Ok(num)
|
||||
@ -388,25 +395,25 @@ impl BlockNumReader for MockEthProvider {
|
||||
}
|
||||
|
||||
impl BlockIdReader for MockEthProvider {
|
||||
fn pending_block_num_hash(&self) -> Result<Option<reth_primitives::BlockNumHash>> {
|
||||
fn pending_block_num_hash(&self) -> RethResult<Option<reth_primitives::BlockNumHash>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn safe_block_num_hash(&self) -> Result<Option<reth_primitives::BlockNumHash>> {
|
||||
fn safe_block_num_hash(&self) -> RethResult<Option<reth_primitives::BlockNumHash>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn finalized_block_num_hash(&self) -> Result<Option<reth_primitives::BlockNumHash>> {
|
||||
fn finalized_block_num_hash(&self) -> RethResult<Option<reth_primitives::BlockNumHash>> {
|
||||
Ok(None)
|
||||
}
|
||||
}
|
||||
|
||||
impl BlockReader for MockEthProvider {
|
||||
fn find_block_by_hash(&self, hash: H256, _source: BlockSource) -> Result<Option<Block>> {
|
||||
fn find_block_by_hash(&self, hash: H256, _source: BlockSource) -> RethResult<Option<Block>> {
|
||||
self.block(hash.into())
|
||||
}
|
||||
|
||||
fn block(&self, id: BlockHashOrNumber) -> Result<Option<Block>> {
|
||||
fn block(&self, id: BlockHashOrNumber) -> RethResult<Option<Block>> {
|
||||
let lock = self.blocks.lock();
|
||||
match id {
|
||||
BlockHashOrNumber::Hash(hash) => Ok(lock.get(&hash).cloned()),
|
||||
@ -414,47 +421,47 @@ impl BlockReader for MockEthProvider {
|
||||
}
|
||||
}
|
||||
|
||||
fn pending_block(&self) -> Result<Option<SealedBlock>> {
|
||||
fn pending_block(&self) -> RethResult<Option<SealedBlock>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn pending_block_and_receipts(&self) -> Result<Option<(SealedBlock, Vec<Receipt>)>> {
|
||||
fn pending_block_and_receipts(&self) -> RethResult<Option<(SealedBlock, Vec<Receipt>)>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn ommers(&self, _id: BlockHashOrNumber) -> Result<Option<Vec<Header>>> {
|
||||
fn ommers(&self, _id: BlockHashOrNumber) -> RethResult<Option<Vec<Header>>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn block_body_indices(&self, _num: u64) -> Result<Option<StoredBlockBodyIndices>> {
|
||||
fn block_body_indices(&self, _num: u64) -> RethResult<Option<StoredBlockBodyIndices>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn block_with_senders(&self, _number: BlockNumber) -> Result<Option<BlockWithSenders>> {
|
||||
fn block_with_senders(&self, _number: BlockNumber) -> RethResult<Option<BlockWithSenders>> {
|
||||
Ok(None)
|
||||
}
|
||||
}
|
||||
|
||||
impl BlockReaderIdExt for MockEthProvider {
|
||||
fn block_by_id(&self, id: BlockId) -> Result<Option<Block>> {
|
||||
fn block_by_id(&self, id: BlockId) -> RethResult<Option<Block>> {
|
||||
match id {
|
||||
BlockId::Number(num) => self.block_by_number_or_tag(num),
|
||||
BlockId::Hash(hash) => self.block_by_hash(hash.block_hash),
|
||||
}
|
||||
}
|
||||
|
||||
fn sealed_header_by_id(&self, id: BlockId) -> Result<Option<SealedHeader>> {
|
||||
fn sealed_header_by_id(&self, id: BlockId) -> RethResult<Option<SealedHeader>> {
|
||||
self.header_by_id(id)?.map_or_else(|| Ok(None), |h| Ok(Some(h.seal_slow())))
|
||||
}
|
||||
|
||||
fn header_by_id(&self, id: BlockId) -> Result<Option<Header>> {
|
||||
fn header_by_id(&self, id: BlockId) -> RethResult<Option<Header>> {
|
||||
match self.block_by_id(id)? {
|
||||
None => Ok(None),
|
||||
Some(block) => Ok(Some(block.header)),
|
||||
}
|
||||
}
|
||||
|
||||
fn ommers_by_id(&self, id: BlockId) -> Result<Option<Vec<Header>>> {
|
||||
fn ommers_by_id(&self, id: BlockId) -> RethResult<Option<Vec<Header>>> {
|
||||
match id {
|
||||
BlockId::Number(num) => self.ommers_by_number_or_tag(num),
|
||||
BlockId::Hash(hash) => self.ommers(BlockHashOrNumber::Hash(hash.block_hash)),
|
||||
@ -463,24 +470,28 @@ impl BlockReaderIdExt for MockEthProvider {
|
||||
}
|
||||
|
||||
impl AccountReader for MockEthProvider {
|
||||
fn basic_account(&self, address: Address) -> Result<Option<Account>> {
|
||||
fn basic_account(&self, address: Address) -> RethResult<Option<Account>> {
|
||||
Ok(self.accounts.lock().get(&address).cloned().map(|a| a.account))
|
||||
}
|
||||
}
|
||||
|
||||
impl StateRootProvider for MockEthProvider {
|
||||
fn state_root(&self, _state: BundleStateWithReceipts) -> Result<H256> {
|
||||
fn state_root(&self, _state: BundleStateWithReceipts) -> RethResult<H256> {
|
||||
todo!()
|
||||
}
|
||||
}
|
||||
|
||||
impl StateProvider for MockEthProvider {
|
||||
fn storage(&self, account: Address, storage_key: StorageKey) -> Result<Option<StorageValue>> {
|
||||
fn storage(
|
||||
&self,
|
||||
account: Address,
|
||||
storage_key: StorageKey,
|
||||
) -> RethResult<Option<StorageValue>> {
|
||||
let lock = self.accounts.lock();
|
||||
Ok(lock.get(&account).and_then(|account| account.storage.get(&storage_key)).cloned())
|
||||
}
|
||||
|
||||
fn bytecode_by_hash(&self, code_hash: H256) -> Result<Option<Bytecode>> {
|
||||
fn bytecode_by_hash(&self, code_hash: H256) -> RethResult<Option<Bytecode>> {
|
||||
let lock = self.accounts.lock();
|
||||
Ok(lock.values().find_map(|account| {
|
||||
match (account.account.bytecode_hash.as_ref(), account.bytecode.as_ref()) {
|
||||
@ -496,7 +507,7 @@ impl StateProvider for MockEthProvider {
|
||||
&self,
|
||||
_address: Address,
|
||||
_keys: &[H256],
|
||||
) -> Result<(Vec<Bytes>, H256, Vec<Vec<Bytes>>)> {
|
||||
) -> RethResult<(Vec<Bytes>, H256, Vec<Vec<Bytes>>)> {
|
||||
todo!()
|
||||
}
|
||||
}
|
||||
@ -507,7 +518,7 @@ impl EvmEnvProvider for MockEthProvider {
|
||||
_cfg: &mut CfgEnv,
|
||||
_block_env: &mut BlockEnv,
|
||||
_at: BlockHashOrNumber,
|
||||
) -> Result<()> {
|
||||
) -> RethResult<()> {
|
||||
unimplemented!()
|
||||
}
|
||||
|
||||
@ -516,11 +527,15 @@ impl EvmEnvProvider for MockEthProvider {
|
||||
_cfg: &mut CfgEnv,
|
||||
_block_env: &mut BlockEnv,
|
||||
_header: &Header,
|
||||
) -> Result<()> {
|
||||
) -> RethResult<()> {
|
||||
unimplemented!()
|
||||
}
|
||||
|
||||
fn fill_block_env_at(&self, _block_env: &mut BlockEnv, _at: BlockHashOrNumber) -> Result<()> {
|
||||
fn fill_block_env_at(
|
||||
&self,
|
||||
_block_env: &mut BlockEnv,
|
||||
_at: BlockHashOrNumber,
|
||||
) -> RethResult<()> {
|
||||
unimplemented!()
|
||||
}
|
||||
|
||||
@ -528,94 +543,94 @@ impl EvmEnvProvider for MockEthProvider {
|
||||
&self,
|
||||
_block_env: &mut BlockEnv,
|
||||
_header: &Header,
|
||||
) -> Result<()> {
|
||||
) -> RethResult<()> {
|
||||
unimplemented!()
|
||||
}
|
||||
|
||||
fn fill_cfg_env_at(&self, _cfg: &mut CfgEnv, _at: BlockHashOrNumber) -> Result<()> {
|
||||
fn fill_cfg_env_at(&self, _cfg: &mut CfgEnv, _at: BlockHashOrNumber) -> RethResult<()> {
|
||||
unimplemented!()
|
||||
}
|
||||
|
||||
fn fill_cfg_env_with_header(&self, _cfg: &mut CfgEnv, _header: &Header) -> Result<()> {
|
||||
fn fill_cfg_env_with_header(&self, _cfg: &mut CfgEnv, _header: &Header) -> RethResult<()> {
|
||||
unimplemented!()
|
||||
}
|
||||
}
|
||||
|
||||
impl StateProviderFactory for MockEthProvider {
|
||||
fn latest(&self) -> Result<StateProviderBox<'_>> {
|
||||
fn latest(&self) -> RethResult<StateProviderBox<'_>> {
|
||||
Ok(Box::new(self.clone()))
|
||||
}
|
||||
|
||||
fn history_by_block_number(&self, _block: BlockNumber) -> Result<StateProviderBox<'_>> {
|
||||
fn history_by_block_number(&self, _block: BlockNumber) -> RethResult<StateProviderBox<'_>> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
fn history_by_block_hash(&self, _block: BlockHash) -> Result<StateProviderBox<'_>> {
|
||||
fn history_by_block_hash(&self, _block: BlockHash) -> RethResult<StateProviderBox<'_>> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
fn state_by_block_hash(&self, _block: BlockHash) -> Result<StateProviderBox<'_>> {
|
||||
fn state_by_block_hash(&self, _block: BlockHash) -> RethResult<StateProviderBox<'_>> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
fn pending(&self) -> Result<StateProviderBox<'_>> {
|
||||
fn pending(&self) -> RethResult<StateProviderBox<'_>> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
fn pending_state_by_hash(&self, _block_hash: H256) -> Result<Option<StateProviderBox<'_>>> {
|
||||
fn pending_state_by_hash(&self, _block_hash: H256) -> RethResult<Option<StateProviderBox<'_>>> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
fn pending_with_provider<'a>(
|
||||
&'a self,
|
||||
_post_state_data: Box<dyn BundleStateDataProvider + 'a>,
|
||||
) -> Result<StateProviderBox<'a>> {
|
||||
) -> RethResult<StateProviderBox<'a>> {
|
||||
todo!()
|
||||
}
|
||||
}
|
||||
|
||||
impl StateProviderFactory for Arc<MockEthProvider> {
|
||||
fn latest(&self) -> Result<StateProviderBox<'_>> {
|
||||
fn latest(&self) -> RethResult<StateProviderBox<'_>> {
|
||||
Ok(Box::new(self.clone()))
|
||||
}
|
||||
|
||||
fn history_by_block_number(&self, _block: BlockNumber) -> Result<StateProviderBox<'_>> {
|
||||
fn history_by_block_number(&self, _block: BlockNumber) -> RethResult<StateProviderBox<'_>> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
fn history_by_block_hash(&self, _block: BlockHash) -> Result<StateProviderBox<'_>> {
|
||||
fn history_by_block_hash(&self, _block: BlockHash) -> RethResult<StateProviderBox<'_>> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
fn state_by_block_hash(&self, _block: BlockHash) -> Result<StateProviderBox<'_>> {
|
||||
fn state_by_block_hash(&self, _block: BlockHash) -> RethResult<StateProviderBox<'_>> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
fn pending(&self) -> Result<StateProviderBox<'_>> {
|
||||
fn pending(&self) -> RethResult<StateProviderBox<'_>> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
fn pending_state_by_hash(&self, _block_hash: H256) -> Result<Option<StateProviderBox<'_>>> {
|
||||
fn pending_state_by_hash(&self, _block_hash: H256) -> RethResult<Option<StateProviderBox<'_>>> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
fn pending_with_provider<'a>(
|
||||
&'a self,
|
||||
_post_state_data: Box<dyn BundleStateDataProvider + 'a>,
|
||||
) -> Result<StateProviderBox<'a>> {
|
||||
) -> RethResult<StateProviderBox<'a>> {
|
||||
todo!()
|
||||
}
|
||||
}
|
||||
|
||||
impl WithdrawalsProvider for MockEthProvider {
|
||||
fn latest_withdrawal(&self) -> Result<Option<reth_primitives::Withdrawal>> {
|
||||
fn latest_withdrawal(&self) -> RethResult<Option<reth_primitives::Withdrawal>> {
|
||||
unimplemented!()
|
||||
}
|
||||
fn withdrawals_by_block(
|
||||
&self,
|
||||
_id: BlockHashOrNumber,
|
||||
_timestamp: u64,
|
||||
) -> Result<Option<Vec<reth_primitives::Withdrawal>>> {
|
||||
) -> RethResult<Option<Vec<reth_primitives::Withdrawal>>> {
|
||||
unimplemented!()
|
||||
}
|
||||
}
|
||||
|
||||
@ -7,7 +7,7 @@ use crate::{
|
||||
StateProviderFactory, StateRootProvider, TransactionsProvider, WithdrawalsProvider,
|
||||
};
|
||||
use reth_db::models::{AccountBeforeTx, StoredBlockBodyIndices};
|
||||
use reth_interfaces::Result;
|
||||
use reth_interfaces::RethResult;
|
||||
use reth_primitives::{
|
||||
stage::{StageCheckpoint, StageId},
|
||||
Account, Address, Block, BlockHash, BlockHashOrNumber, BlockId, BlockNumber, Bytecode, Bytes,
|
||||
@ -31,166 +31,173 @@ impl ChainSpecProvider for NoopProvider {
|
||||
|
||||
/// Noop implementation for testing purposes
|
||||
impl BlockHashReader for NoopProvider {
|
||||
fn block_hash(&self, _number: u64) -> Result<Option<H256>> {
|
||||
fn block_hash(&self, _number: u64) -> RethResult<Option<H256>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn canonical_hashes_range(&self, _start: BlockNumber, _end: BlockNumber) -> Result<Vec<H256>> {
|
||||
fn canonical_hashes_range(
|
||||
&self,
|
||||
_start: BlockNumber,
|
||||
_end: BlockNumber,
|
||||
) -> RethResult<Vec<H256>> {
|
||||
Ok(vec![])
|
||||
}
|
||||
}
|
||||
|
||||
impl BlockNumReader for NoopProvider {
|
||||
fn chain_info(&self) -> Result<ChainInfo> {
|
||||
fn chain_info(&self) -> RethResult<ChainInfo> {
|
||||
Ok(ChainInfo::default())
|
||||
}
|
||||
|
||||
fn best_block_number(&self) -> Result<BlockNumber> {
|
||||
fn best_block_number(&self) -> RethResult<BlockNumber> {
|
||||
Ok(0)
|
||||
}
|
||||
|
||||
fn last_block_number(&self) -> Result<BlockNumber> {
|
||||
fn last_block_number(&self) -> RethResult<BlockNumber> {
|
||||
Ok(0)
|
||||
}
|
||||
|
||||
fn block_number(&self, _hash: H256) -> Result<Option<BlockNumber>> {
|
||||
fn block_number(&self, _hash: H256) -> RethResult<Option<BlockNumber>> {
|
||||
Ok(None)
|
||||
}
|
||||
}
|
||||
|
||||
impl BlockReader for NoopProvider {
|
||||
fn find_block_by_hash(&self, hash: H256, _source: BlockSource) -> Result<Option<Block>> {
|
||||
fn find_block_by_hash(&self, hash: H256, _source: BlockSource) -> RethResult<Option<Block>> {
|
||||
self.block(hash.into())
|
||||
}
|
||||
|
||||
fn block(&self, _id: BlockHashOrNumber) -> Result<Option<Block>> {
|
||||
fn block(&self, _id: BlockHashOrNumber) -> RethResult<Option<Block>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn pending_block(&self) -> Result<Option<SealedBlock>> {
|
||||
fn pending_block(&self) -> RethResult<Option<SealedBlock>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn pending_block_and_receipts(&self) -> Result<Option<(SealedBlock, Vec<Receipt>)>> {
|
||||
fn pending_block_and_receipts(&self) -> RethResult<Option<(SealedBlock, Vec<Receipt>)>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn ommers(&self, _id: BlockHashOrNumber) -> Result<Option<Vec<Header>>> {
|
||||
fn ommers(&self, _id: BlockHashOrNumber) -> RethResult<Option<Vec<Header>>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn block_body_indices(&self, _num: u64) -> Result<Option<StoredBlockBodyIndices>> {
|
||||
fn block_body_indices(&self, _num: u64) -> RethResult<Option<StoredBlockBodyIndices>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn block_with_senders(
|
||||
&self,
|
||||
_number: BlockNumber,
|
||||
) -> Result<Option<reth_primitives::BlockWithSenders>> {
|
||||
) -> RethResult<Option<reth_primitives::BlockWithSenders>> {
|
||||
Ok(None)
|
||||
}
|
||||
}
|
||||
|
||||
impl BlockReaderIdExt for NoopProvider {
|
||||
fn block_by_id(&self, _id: BlockId) -> Result<Option<Block>> {
|
||||
fn block_by_id(&self, _id: BlockId) -> RethResult<Option<Block>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn sealed_header_by_id(&self, _id: BlockId) -> Result<Option<SealedHeader>> {
|
||||
fn sealed_header_by_id(&self, _id: BlockId) -> RethResult<Option<SealedHeader>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn header_by_id(&self, _id: BlockId) -> Result<Option<Header>> {
|
||||
fn header_by_id(&self, _id: BlockId) -> RethResult<Option<Header>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn ommers_by_id(&self, _id: BlockId) -> Result<Option<Vec<Header>>> {
|
||||
fn ommers_by_id(&self, _id: BlockId) -> RethResult<Option<Vec<Header>>> {
|
||||
Ok(None)
|
||||
}
|
||||
}
|
||||
|
||||
impl BlockIdReader for NoopProvider {
|
||||
fn pending_block_num_hash(&self) -> Result<Option<reth_primitives::BlockNumHash>> {
|
||||
fn pending_block_num_hash(&self) -> RethResult<Option<reth_primitives::BlockNumHash>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn safe_block_num_hash(&self) -> Result<Option<reth_primitives::BlockNumHash>> {
|
||||
fn safe_block_num_hash(&self) -> RethResult<Option<reth_primitives::BlockNumHash>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn finalized_block_num_hash(&self) -> Result<Option<reth_primitives::BlockNumHash>> {
|
||||
fn finalized_block_num_hash(&self) -> RethResult<Option<reth_primitives::BlockNumHash>> {
|
||||
Ok(None)
|
||||
}
|
||||
}
|
||||
|
||||
impl TransactionsProvider for NoopProvider {
|
||||
fn transaction_id(&self, _tx_hash: TxHash) -> Result<Option<TxNumber>> {
|
||||
fn transaction_id(&self, _tx_hash: TxHash) -> RethResult<Option<TxNumber>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn transaction_by_id(&self, _id: TxNumber) -> Result<Option<TransactionSigned>> {
|
||||
fn transaction_by_id(&self, _id: TxNumber) -> RethResult<Option<TransactionSigned>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn transaction_by_id_no_hash(&self, _id: TxNumber) -> Result<Option<TransactionSignedNoHash>> {
|
||||
fn transaction_by_id_no_hash(
|
||||
&self,
|
||||
_id: TxNumber,
|
||||
) -> RethResult<Option<TransactionSignedNoHash>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn transaction_by_hash(&self, _hash: TxHash) -> Result<Option<TransactionSigned>> {
|
||||
fn transaction_by_hash(&self, _hash: TxHash) -> RethResult<Option<TransactionSigned>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn transaction_by_hash_with_meta(
|
||||
&self,
|
||||
_hash: TxHash,
|
||||
) -> Result<Option<(TransactionSigned, TransactionMeta)>> {
|
||||
) -> RethResult<Option<(TransactionSigned, TransactionMeta)>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn transaction_block(&self, _id: TxNumber) -> Result<Option<BlockNumber>> {
|
||||
fn transaction_block(&self, _id: TxNumber) -> RethResult<Option<BlockNumber>> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
fn transactions_by_block(
|
||||
&self,
|
||||
_block_id: BlockHashOrNumber,
|
||||
) -> Result<Option<Vec<TransactionSigned>>> {
|
||||
) -> RethResult<Option<Vec<TransactionSigned>>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn transactions_by_block_range(
|
||||
&self,
|
||||
_range: impl RangeBounds<BlockNumber>,
|
||||
) -> Result<Vec<Vec<TransactionSigned>>> {
|
||||
) -> RethResult<Vec<Vec<TransactionSigned>>> {
|
||||
Ok(Vec::default())
|
||||
}
|
||||
|
||||
fn senders_by_tx_range(&self, _range: impl RangeBounds<TxNumber>) -> Result<Vec<Address>> {
|
||||
fn senders_by_tx_range(&self, _range: impl RangeBounds<TxNumber>) -> RethResult<Vec<Address>> {
|
||||
Ok(Vec::default())
|
||||
}
|
||||
|
||||
fn transactions_by_tx_range(
|
||||
&self,
|
||||
_range: impl RangeBounds<TxNumber>,
|
||||
) -> Result<Vec<reth_primitives::TransactionSignedNoHash>> {
|
||||
) -> RethResult<Vec<reth_primitives::TransactionSignedNoHash>> {
|
||||
Ok(Vec::default())
|
||||
}
|
||||
|
||||
fn transaction_sender(&self, _id: TxNumber) -> Result<Option<Address>> {
|
||||
fn transaction_sender(&self, _id: TxNumber) -> RethResult<Option<Address>> {
|
||||
Ok(None)
|
||||
}
|
||||
}
|
||||
|
||||
impl ReceiptProvider for NoopProvider {
|
||||
fn receipt(&self, _id: TxNumber) -> Result<Option<Receipt>> {
|
||||
fn receipt(&self, _id: TxNumber) -> RethResult<Option<Receipt>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn receipt_by_hash(&self, _hash: TxHash) -> Result<Option<Receipt>> {
|
||||
fn receipt_by_hash(&self, _hash: TxHash) -> RethResult<Option<Receipt>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn receipts_by_block(&self, _block: BlockHashOrNumber) -> Result<Option<Vec<Receipt>>> {
|
||||
fn receipts_by_block(&self, _block: BlockHashOrNumber) -> RethResult<Option<Vec<Receipt>>> {
|
||||
Ok(None)
|
||||
}
|
||||
}
|
||||
@ -198,62 +205,69 @@ impl ReceiptProvider for NoopProvider {
|
||||
impl ReceiptProviderIdExt for NoopProvider {}
|
||||
|
||||
impl HeaderProvider for NoopProvider {
|
||||
fn header(&self, _block_hash: &BlockHash) -> Result<Option<Header>> {
|
||||
fn header(&self, _block_hash: &BlockHash) -> RethResult<Option<Header>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn header_by_number(&self, _num: u64) -> Result<Option<Header>> {
|
||||
fn header_by_number(&self, _num: u64) -> RethResult<Option<Header>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn header_td(&self, _hash: &BlockHash) -> Result<Option<U256>> {
|
||||
fn header_td(&self, _hash: &BlockHash) -> RethResult<Option<U256>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn header_td_by_number(&self, _number: BlockNumber) -> Result<Option<U256>> {
|
||||
fn header_td_by_number(&self, _number: BlockNumber) -> RethResult<Option<U256>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn headers_range(&self, _range: impl RangeBounds<BlockNumber>) -> Result<Vec<Header>> {
|
||||
fn headers_range(&self, _range: impl RangeBounds<BlockNumber>) -> RethResult<Vec<Header>> {
|
||||
Ok(vec![])
|
||||
}
|
||||
|
||||
fn sealed_headers_range(
|
||||
&self,
|
||||
_range: impl RangeBounds<BlockNumber>,
|
||||
) -> Result<Vec<SealedHeader>> {
|
||||
) -> RethResult<Vec<SealedHeader>> {
|
||||
Ok(vec![])
|
||||
}
|
||||
|
||||
fn sealed_header(&self, _number: BlockNumber) -> Result<Option<SealedHeader>> {
|
||||
fn sealed_header(&self, _number: BlockNumber) -> RethResult<Option<SealedHeader>> {
|
||||
Ok(None)
|
||||
}
|
||||
}
|
||||
|
||||
impl AccountReader for NoopProvider {
|
||||
fn basic_account(&self, _address: Address) -> Result<Option<Account>> {
|
||||
fn basic_account(&self, _address: Address) -> RethResult<Option<Account>> {
|
||||
Ok(None)
|
||||
}
|
||||
}
|
||||
|
||||
impl ChangeSetReader for NoopProvider {
|
||||
fn account_block_changeset(&self, _block_number: BlockNumber) -> Result<Vec<AccountBeforeTx>> {
|
||||
fn account_block_changeset(
|
||||
&self,
|
||||
_block_number: BlockNumber,
|
||||
) -> RethResult<Vec<AccountBeforeTx>> {
|
||||
Ok(Vec::default())
|
||||
}
|
||||
}
|
||||
|
||||
impl StateRootProvider for NoopProvider {
|
||||
fn state_root(&self, _state: BundleStateWithReceipts) -> Result<H256> {
|
||||
fn state_root(&self, _state: BundleStateWithReceipts) -> RethResult<H256> {
|
||||
todo!()
|
||||
}
|
||||
}
|
||||
|
||||
impl StateProvider for NoopProvider {
|
||||
fn storage(&self, _account: Address, _storage_key: StorageKey) -> Result<Option<StorageValue>> {
|
||||
fn storage(
|
||||
&self,
|
||||
_account: Address,
|
||||
_storage_key: StorageKey,
|
||||
) -> RethResult<Option<StorageValue>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn bytecode_by_hash(&self, _code_hash: H256) -> Result<Option<Bytecode>> {
|
||||
fn bytecode_by_hash(&self, _code_hash: H256) -> RethResult<Option<Bytecode>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
@ -261,7 +275,7 @@ impl StateProvider for NoopProvider {
|
||||
&self,
|
||||
_address: Address,
|
||||
_keys: &[H256],
|
||||
) -> Result<(Vec<Bytes>, H256, Vec<Vec<Bytes>>)> {
|
||||
) -> RethResult<(Vec<Bytes>, H256, Vec<Vec<Bytes>>)> {
|
||||
Ok((vec![], KECCAK_EMPTY, vec![]))
|
||||
}
|
||||
}
|
||||
@ -272,7 +286,7 @@ impl EvmEnvProvider for NoopProvider {
|
||||
_cfg: &mut CfgEnv,
|
||||
_block_env: &mut BlockEnv,
|
||||
_at: BlockHashOrNumber,
|
||||
) -> Result<()> {
|
||||
) -> RethResult<()> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
@ -281,11 +295,15 @@ impl EvmEnvProvider for NoopProvider {
|
||||
_cfg: &mut CfgEnv,
|
||||
_block_env: &mut BlockEnv,
|
||||
_header: &Header,
|
||||
) -> Result<()> {
|
||||
) -> RethResult<()> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn fill_block_env_at(&self, _block_env: &mut BlockEnv, _at: BlockHashOrNumber) -> Result<()> {
|
||||
fn fill_block_env_at(
|
||||
&self,
|
||||
_block_env: &mut BlockEnv,
|
||||
_at: BlockHashOrNumber,
|
||||
) -> RethResult<()> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
@ -293,77 +311,77 @@ impl EvmEnvProvider for NoopProvider {
|
||||
&self,
|
||||
_block_env: &mut BlockEnv,
|
||||
_header: &Header,
|
||||
) -> Result<()> {
|
||||
) -> RethResult<()> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn fill_cfg_env_at(&self, _cfg: &mut CfgEnv, _at: BlockHashOrNumber) -> Result<()> {
|
||||
fn fill_cfg_env_at(&self, _cfg: &mut CfgEnv, _at: BlockHashOrNumber) -> RethResult<()> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn fill_cfg_env_with_header(&self, _cfg: &mut CfgEnv, _header: &Header) -> Result<()> {
|
||||
fn fill_cfg_env_with_header(&self, _cfg: &mut CfgEnv, _header: &Header) -> RethResult<()> {
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
impl StateProviderFactory for NoopProvider {
|
||||
fn latest(&self) -> Result<StateProviderBox<'_>> {
|
||||
fn latest(&self) -> RethResult<StateProviderBox<'_>> {
|
||||
Ok(Box::new(*self))
|
||||
}
|
||||
|
||||
fn history_by_block_number(&self, _block: BlockNumber) -> Result<StateProviderBox<'_>> {
|
||||
fn history_by_block_number(&self, _block: BlockNumber) -> RethResult<StateProviderBox<'_>> {
|
||||
Ok(Box::new(*self))
|
||||
}
|
||||
|
||||
fn history_by_block_hash(&self, _block: BlockHash) -> Result<StateProviderBox<'_>> {
|
||||
fn history_by_block_hash(&self, _block: BlockHash) -> RethResult<StateProviderBox<'_>> {
|
||||
Ok(Box::new(*self))
|
||||
}
|
||||
|
||||
fn state_by_block_hash(&self, _block: BlockHash) -> Result<StateProviderBox<'_>> {
|
||||
fn state_by_block_hash(&self, _block: BlockHash) -> RethResult<StateProviderBox<'_>> {
|
||||
Ok(Box::new(*self))
|
||||
}
|
||||
|
||||
fn pending(&self) -> Result<StateProviderBox<'_>> {
|
||||
fn pending(&self) -> RethResult<StateProviderBox<'_>> {
|
||||
Ok(Box::new(*self))
|
||||
}
|
||||
|
||||
fn pending_state_by_hash(&self, _block_hash: H256) -> Result<Option<StateProviderBox<'_>>> {
|
||||
fn pending_state_by_hash(&self, _block_hash: H256) -> RethResult<Option<StateProviderBox<'_>>> {
|
||||
Ok(Some(Box::new(*self)))
|
||||
}
|
||||
|
||||
fn pending_with_provider<'a>(
|
||||
&'a self,
|
||||
_post_state_data: Box<dyn crate::BundleStateDataProvider + 'a>,
|
||||
) -> Result<StateProviderBox<'a>> {
|
||||
) -> RethResult<StateProviderBox<'a>> {
|
||||
Ok(Box::new(*self))
|
||||
}
|
||||
}
|
||||
|
||||
impl StageCheckpointReader for NoopProvider {
|
||||
fn get_stage_checkpoint(&self, _id: StageId) -> Result<Option<StageCheckpoint>> {
|
||||
fn get_stage_checkpoint(&self, _id: StageId) -> RethResult<Option<StageCheckpoint>> {
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
fn get_stage_checkpoint_progress(&self, _id: StageId) -> Result<Option<Vec<u8>>> {
|
||||
fn get_stage_checkpoint_progress(&self, _id: StageId) -> RethResult<Option<Vec<u8>>> {
|
||||
Ok(None)
|
||||
}
|
||||
}
|
||||
|
||||
impl WithdrawalsProvider for NoopProvider {
|
||||
fn latest_withdrawal(&self) -> Result<Option<reth_primitives::Withdrawal>> {
|
||||
fn latest_withdrawal(&self) -> RethResult<Option<reth_primitives::Withdrawal>> {
|
||||
Ok(None)
|
||||
}
|
||||
fn withdrawals_by_block(
|
||||
&self,
|
||||
_id: BlockHashOrNumber,
|
||||
_timestamp: u64,
|
||||
) -> Result<Option<Vec<reth_primitives::Withdrawal>>> {
|
||||
) -> RethResult<Option<Vec<reth_primitives::Withdrawal>>> {
|
||||
Ok(None)
|
||||
}
|
||||
}
|
||||
|
||||
impl PruneCheckpointReader for NoopProvider {
|
||||
fn get_prune_checkpoint(&self, _part: PrunePart) -> Result<Option<PruneCheckpoint>> {
|
||||
fn get_prune_checkpoint(&self, _part: PrunePart) -> RethResult<Option<PruneCheckpoint>> {
|
||||
Ok(None)
|
||||
}
|
||||
}
|
||||
|
||||
@ -1,6 +1,6 @@
|
||||
use auto_impl::auto_impl;
|
||||
use reth_db::models::AccountBeforeTx;
|
||||
use reth_interfaces::Result;
|
||||
use reth_interfaces::RethResult;
|
||||
use reth_primitives::{Account, Address, BlockNumber};
|
||||
use std::{
|
||||
collections::{BTreeMap, BTreeSet},
|
||||
@ -13,7 +13,7 @@ pub trait AccountReader: Send + Sync {
|
||||
/// Get basic account information.
|
||||
///
|
||||
/// Returns `None` if the account doesn't exist.
|
||||
fn basic_account(&self, address: Address) -> Result<Option<Account>>;
|
||||
fn basic_account(&self, address: Address) -> RethResult<Option<Account>>;
|
||||
}
|
||||
|
||||
/// Account reader
|
||||
@ -23,7 +23,7 @@ pub trait AccountExtReader: Send + Sync {
|
||||
fn changed_accounts_with_range(
|
||||
&self,
|
||||
_range: impl RangeBounds<BlockNumber>,
|
||||
) -> Result<BTreeSet<Address>>;
|
||||
) -> RethResult<BTreeSet<Address>>;
|
||||
|
||||
/// Get basic account information for multiple accounts. A more efficient version than calling
|
||||
/// [`AccountReader::basic_account`] repeatedly.
|
||||
@ -32,7 +32,7 @@ pub trait AccountExtReader: Send + Sync {
|
||||
fn basic_accounts(
|
||||
&self,
|
||||
_iter: impl IntoIterator<Item = Address>,
|
||||
) -> Result<Vec<(Address, Option<Account>)>>;
|
||||
) -> RethResult<Vec<(Address, Option<Account>)>>;
|
||||
|
||||
/// Iterate over account changesets and return all account addresses that were changed alongside
|
||||
/// each specific set of blocks.
|
||||
@ -41,12 +41,15 @@ pub trait AccountExtReader: Send + Sync {
|
||||
fn changed_accounts_and_blocks_with_range(
|
||||
&self,
|
||||
range: RangeInclusive<BlockNumber>,
|
||||
) -> Result<BTreeMap<Address, Vec<BlockNumber>>>;
|
||||
) -> RethResult<BTreeMap<Address, Vec<BlockNumber>>>;
|
||||
}
|
||||
|
||||
/// AccountChange reader
|
||||
#[auto_impl(&, Arc, Box)]
|
||||
pub trait ChangeSetReader: Send + Sync {
|
||||
/// Iterate over account changesets and return the account state from before this block.
|
||||
fn account_block_changeset(&self, block_number: BlockNumber) -> Result<Vec<AccountBeforeTx>>;
|
||||
fn account_block_changeset(
|
||||
&self,
|
||||
block_number: BlockNumber,
|
||||
) -> RethResult<Vec<AccountBeforeTx>>;
|
||||
}
|
||||
|
||||
@ -4,7 +4,7 @@ use crate::{
|
||||
};
|
||||
use auto_impl::auto_impl;
|
||||
use reth_db::models::StoredBlockBodyIndices;
|
||||
use reth_interfaces::Result;
|
||||
use reth_interfaces::RethResult;
|
||||
use reth_primitives::{
|
||||
Address, Block, BlockHashOrNumber, BlockId, BlockNumber, BlockNumberOrTag, BlockWithSenders,
|
||||
ChainSpec, Header, PruneModes, Receipt, SealedBlock, SealedBlockWithSenders, SealedHeader,
|
||||
@ -62,50 +62,50 @@ pub trait BlockReader:
|
||||
/// Note: this only operates on the hash because the number might be ambiguous.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn find_block_by_hash(&self, hash: H256, source: BlockSource) -> Result<Option<Block>>;
|
||||
fn find_block_by_hash(&self, hash: H256, source: BlockSource) -> RethResult<Option<Block>>;
|
||||
|
||||
/// Returns the block with given id from the database.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn block(&self, id: BlockHashOrNumber) -> Result<Option<Block>>;
|
||||
fn block(&self, id: BlockHashOrNumber) -> RethResult<Option<Block>>;
|
||||
|
||||
/// Returns the pending block if available
|
||||
///
|
||||
/// Note: This returns a [SealedBlock] because it's expected that this is sealed by the provider
|
||||
/// and the caller does not know the hash.
|
||||
fn pending_block(&self) -> Result<Option<SealedBlock>>;
|
||||
fn pending_block(&self) -> RethResult<Option<SealedBlock>>;
|
||||
|
||||
/// Returns the pending block and receipts if available.
|
||||
fn pending_block_and_receipts(&self) -> Result<Option<(SealedBlock, Vec<Receipt>)>>;
|
||||
fn pending_block_and_receipts(&self) -> RethResult<Option<(SealedBlock, Vec<Receipt>)>>;
|
||||
|
||||
/// Returns the ommers/uncle headers of the given block from the database.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn ommers(&self, id: BlockHashOrNumber) -> Result<Option<Vec<Header>>>;
|
||||
fn ommers(&self, id: BlockHashOrNumber) -> RethResult<Option<Vec<Header>>>;
|
||||
|
||||
/// Returns the block with matching hash from the database.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn block_by_hash(&self, hash: H256) -> Result<Option<Block>> {
|
||||
fn block_by_hash(&self, hash: H256) -> RethResult<Option<Block>> {
|
||||
self.block(hash.into())
|
||||
}
|
||||
|
||||
/// Returns the block with matching number from database.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn block_by_number(&self, num: u64) -> Result<Option<Block>> {
|
||||
fn block_by_number(&self, num: u64) -> RethResult<Option<Block>> {
|
||||
self.block(num.into())
|
||||
}
|
||||
|
||||
/// Returns the block body indices with matching number from database.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn block_body_indices(&self, num: u64) -> Result<Option<StoredBlockBodyIndices>>;
|
||||
fn block_body_indices(&self, num: u64) -> RethResult<Option<StoredBlockBodyIndices>>;
|
||||
|
||||
/// Returns the block with senders with matching number from database.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn block_with_senders(&self, number: BlockNumber) -> Result<Option<BlockWithSenders>>;
|
||||
fn block_with_senders(&self, number: BlockNumber) -> RethResult<Option<BlockWithSenders>>;
|
||||
}
|
||||
|
||||
/// Trait extension for `BlockReader`, for types that implement `BlockId` conversion.
|
||||
@ -123,7 +123,7 @@ pub trait BlockReaderIdExt: BlockReader + BlockIdReader + ReceiptProviderIdExt {
|
||||
/// Returns the block with matching tag from the database
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn block_by_number_or_tag(&self, id: BlockNumberOrTag) -> Result<Option<Block>> {
|
||||
fn block_by_number_or_tag(&self, id: BlockNumberOrTag) -> RethResult<Option<Block>> {
|
||||
self.convert_block_number(id)?.map_or_else(|| Ok(None), |num| self.block(num.into()))
|
||||
}
|
||||
|
||||
@ -131,7 +131,7 @@ pub trait BlockReaderIdExt: BlockReader + BlockIdReader + ReceiptProviderIdExt {
|
||||
///
|
||||
/// Note: This returns a [SealedHeader] because it's expected that this is sealed by the
|
||||
/// provider and the caller does not know the hash.
|
||||
fn pending_header(&self) -> Result<Option<SealedHeader>> {
|
||||
fn pending_header(&self) -> RethResult<Option<SealedHeader>> {
|
||||
self.sealed_header_by_id(BlockNumberOrTag::Pending.into())
|
||||
}
|
||||
|
||||
@ -139,7 +139,7 @@ pub trait BlockReaderIdExt: BlockReader + BlockIdReader + ReceiptProviderIdExt {
|
||||
///
|
||||
/// Note: This returns a [SealedHeader] because it's expected that this is sealed by the
|
||||
/// provider and the caller does not know the hash.
|
||||
fn latest_header(&self) -> Result<Option<SealedHeader>> {
|
||||
fn latest_header(&self) -> RethResult<Option<SealedHeader>> {
|
||||
self.sealed_header_by_id(BlockNumberOrTag::Latest.into())
|
||||
}
|
||||
|
||||
@ -147,7 +147,7 @@ pub trait BlockReaderIdExt: BlockReader + BlockIdReader + ReceiptProviderIdExt {
|
||||
///
|
||||
/// Note: This returns a [SealedHeader] because it's expected that this is sealed by the
|
||||
/// provider and the caller does not know the hash.
|
||||
fn safe_header(&self) -> Result<Option<SealedHeader>> {
|
||||
fn safe_header(&self) -> RethResult<Option<SealedHeader>> {
|
||||
self.sealed_header_by_id(BlockNumberOrTag::Safe.into())
|
||||
}
|
||||
|
||||
@ -155,19 +155,19 @@ pub trait BlockReaderIdExt: BlockReader + BlockIdReader + ReceiptProviderIdExt {
|
||||
///
|
||||
/// Note: This returns a [SealedHeader] because it's expected that this is sealed by the
|
||||
/// provider and the caller does not know the hash.
|
||||
fn finalized_header(&self) -> Result<Option<SealedHeader>> {
|
||||
fn finalized_header(&self) -> RethResult<Option<SealedHeader>> {
|
||||
self.sealed_header_by_id(BlockNumberOrTag::Finalized.into())
|
||||
}
|
||||
|
||||
/// Returns the block with the matching `BlockId` from the database.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn block_by_id(&self, id: BlockId) -> Result<Option<Block>>;
|
||||
fn block_by_id(&self, id: BlockId) -> RethResult<Option<Block>>;
|
||||
|
||||
/// Returns the header with matching tag from the database
|
||||
///
|
||||
/// Returns `None` if header is not found.
|
||||
fn header_by_number_or_tag(&self, id: BlockNumberOrTag) -> Result<Option<Header>> {
|
||||
fn header_by_number_or_tag(&self, id: BlockNumberOrTag) -> RethResult<Option<Header>> {
|
||||
self.convert_block_number(id)?
|
||||
.map_or_else(|| Ok(None), |num| self.header_by_hash_or_number(num.into()))
|
||||
}
|
||||
@ -175,7 +175,10 @@ pub trait BlockReaderIdExt: BlockReader + BlockIdReader + ReceiptProviderIdExt {
|
||||
/// Returns the header with matching tag from the database
|
||||
///
|
||||
/// Returns `None` if header is not found.
|
||||
fn sealed_header_by_number_or_tag(&self, id: BlockNumberOrTag) -> Result<Option<SealedHeader>> {
|
||||
fn sealed_header_by_number_or_tag(
|
||||
&self,
|
||||
id: BlockNumberOrTag,
|
||||
) -> RethResult<Option<SealedHeader>> {
|
||||
self.convert_block_number(id)?
|
||||
.map_or_else(|| Ok(None), |num| self.header_by_hash_or_number(num.into()))?
|
||||
.map_or_else(|| Ok(None), |h| Ok(Some(h.seal_slow())))
|
||||
@ -184,22 +187,22 @@ pub trait BlockReaderIdExt: BlockReader + BlockIdReader + ReceiptProviderIdExt {
|
||||
/// Returns the sealed header with the matching `BlockId` from the database.
|
||||
///
|
||||
/// Returns `None` if header is not found.
|
||||
fn sealed_header_by_id(&self, id: BlockId) -> Result<Option<SealedHeader>>;
|
||||
fn sealed_header_by_id(&self, id: BlockId) -> RethResult<Option<SealedHeader>>;
|
||||
|
||||
/// Returns the header with the matching `BlockId` from the database.
|
||||
///
|
||||
/// Returns `None` if header is not found.
|
||||
fn header_by_id(&self, id: BlockId) -> Result<Option<Header>>;
|
||||
fn header_by_id(&self, id: BlockId) -> RethResult<Option<Header>>;
|
||||
|
||||
/// Returns the ommers with the matching tag from the database.
|
||||
fn ommers_by_number_or_tag(&self, id: BlockNumberOrTag) -> Result<Option<Vec<Header>>> {
|
||||
fn ommers_by_number_or_tag(&self, id: BlockNumberOrTag) -> RethResult<Option<Vec<Header>>> {
|
||||
self.convert_block_number(id)?.map_or_else(|| Ok(None), |num| self.ommers(num.into()))
|
||||
}
|
||||
|
||||
/// Returns the ommers with the matching `BlockId` from the database.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn ommers_by_id(&self, id: BlockId) -> Result<Option<Vec<Header>>>;
|
||||
fn ommers_by_id(&self, id: BlockId) -> RethResult<Option<Vec<Header>>>;
|
||||
}
|
||||
|
||||
/// BlockExecution Writer
|
||||
@ -210,7 +213,7 @@ pub trait BlockExecutionWriter: BlockWriter + BlockReader + Send + Sync {
|
||||
&self,
|
||||
chain_spec: &ChainSpec,
|
||||
range: RangeInclusive<BlockNumber>,
|
||||
) -> Result<Chain> {
|
||||
) -> RethResult<Chain> {
|
||||
self.get_or_take_block_and_execution_range::<false>(chain_spec, range)
|
||||
}
|
||||
|
||||
@ -219,7 +222,7 @@ pub trait BlockExecutionWriter: BlockWriter + BlockReader + Send + Sync {
|
||||
&self,
|
||||
chain_spec: &ChainSpec,
|
||||
range: RangeInclusive<BlockNumber>,
|
||||
) -> Result<Chain> {
|
||||
) -> RethResult<Chain> {
|
||||
self.get_or_take_block_and_execution_range::<true>(chain_spec, range)
|
||||
}
|
||||
|
||||
@ -228,7 +231,7 @@ pub trait BlockExecutionWriter: BlockWriter + BlockReader + Send + Sync {
|
||||
&self,
|
||||
chain_spec: &ChainSpec,
|
||||
range: RangeInclusive<BlockNumber>,
|
||||
) -> Result<Chain>;
|
||||
) -> RethResult<Chain>;
|
||||
}
|
||||
|
||||
/// Block Writer
|
||||
@ -244,7 +247,7 @@ pub trait BlockWriter: Send + Sync {
|
||||
block: SealedBlock,
|
||||
senders: Option<Vec<Address>>,
|
||||
prune_modes: Option<&PruneModes>,
|
||||
) -> Result<StoredBlockBodyIndices>;
|
||||
) -> RethResult<StoredBlockBodyIndices>;
|
||||
|
||||
/// Appends a batch of sealed blocks to the blockchain, including sender information, and
|
||||
/// updates the post-state.
|
||||
@ -266,5 +269,5 @@ pub trait BlockWriter: Send + Sync {
|
||||
blocks: Vec<SealedBlockWithSenders>,
|
||||
state: BundleStateWithReceipts,
|
||||
prune_modes: Option<&PruneModes>,
|
||||
) -> Result<()>;
|
||||
) -> RethResult<()>;
|
||||
}
|
||||
|
||||
@ -1,5 +1,5 @@
|
||||
use auto_impl::auto_impl;
|
||||
use reth_interfaces::Result;
|
||||
use reth_interfaces::RethResult;
|
||||
use reth_primitives::{BlockHashOrNumber, BlockNumber, H256};
|
||||
|
||||
/// Client trait for fetching block hashes by number.
|
||||
@ -7,11 +7,11 @@ use reth_primitives::{BlockHashOrNumber, BlockNumber, H256};
|
||||
pub trait BlockHashReader: 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: BlockNumber) -> Result<Option<H256>>;
|
||||
fn block_hash(&self, number: BlockNumber) -> RethResult<Option<H256>>;
|
||||
|
||||
/// Get the hash of the block with the given number. Returns `None` if no block with this number
|
||||
/// exists.
|
||||
fn convert_block_hash(&self, hash_or_number: BlockHashOrNumber) -> Result<Option<H256>> {
|
||||
fn convert_block_hash(&self, hash_or_number: BlockHashOrNumber) -> RethResult<Option<H256>> {
|
||||
match hash_or_number {
|
||||
BlockHashOrNumber::Hash(hash) => Ok(Some(hash)),
|
||||
BlockHashOrNumber::Number(num) => self.block_hash(num),
|
||||
@ -23,5 +23,6 @@ pub trait BlockHashReader: Send + Sync {
|
||||
/// Returns the available hashes of that range.
|
||||
///
|
||||
/// Note: The range is `start..end`, so the expected result is `[start..end)`
|
||||
fn canonical_hashes_range(&self, start: BlockNumber, end: BlockNumber) -> Result<Vec<H256>>;
|
||||
fn canonical_hashes_range(&self, start: BlockNumber, end: BlockNumber)
|
||||
-> RethResult<Vec<H256>>;
|
||||
}
|
||||
|
||||
@ -1,5 +1,5 @@
|
||||
use super::BlockHashReader;
|
||||
use reth_interfaces::{provider::ProviderError, Result};
|
||||
use reth_interfaces::{provider::ProviderError, RethResult};
|
||||
use reth_primitives::{BlockHashOrNumber, BlockId, BlockNumber, BlockNumberOrTag, ChainInfo, H256};
|
||||
|
||||
/// Client trait for getting important block numbers (such as the latest block number), converting
|
||||
@ -9,20 +9,20 @@ use reth_primitives::{BlockHashOrNumber, BlockId, BlockNumber, BlockNumberOrTag,
|
||||
#[auto_impl::auto_impl(&, Arc)]
|
||||
pub trait BlockNumReader: BlockHashReader + Send + Sync {
|
||||
/// Returns the current info for the chain.
|
||||
fn chain_info(&self) -> Result<ChainInfo>;
|
||||
fn chain_info(&self) -> RethResult<ChainInfo>;
|
||||
|
||||
/// Returns the best block number in the chain.
|
||||
fn best_block_number(&self) -> Result<BlockNumber>;
|
||||
fn best_block_number(&self) -> RethResult<BlockNumber>;
|
||||
|
||||
/// Returns the last block number associated with the last canonical header in the database.
|
||||
fn last_block_number(&self) -> Result<BlockNumber>;
|
||||
fn last_block_number(&self) -> RethResult<BlockNumber>;
|
||||
|
||||
/// Gets the `BlockNumber` for the given hash. Returns `None` if no block with this hash exists.
|
||||
fn block_number(&self, hash: H256) -> Result<Option<BlockNumber>>;
|
||||
fn block_number(&self, hash: H256) -> RethResult<Option<BlockNumber>>;
|
||||
|
||||
/// Gets the block number for the given `BlockHashOrNumber`. Returns `None` if no block with
|
||||
/// this hash exists. If the `BlockHashOrNumber` is a `Number`, it is returned as is.
|
||||
fn convert_hash_or_number(&self, id: BlockHashOrNumber) -> Result<Option<BlockNumber>> {
|
||||
fn convert_hash_or_number(&self, id: BlockHashOrNumber) -> RethResult<Option<BlockNumber>> {
|
||||
match id {
|
||||
BlockHashOrNumber::Hash(hash) => self.block_number(hash),
|
||||
BlockHashOrNumber::Number(num) => Ok(Some(num)),
|
||||
@ -31,7 +31,7 @@ pub trait BlockNumReader: BlockHashReader + Send + Sync {
|
||||
|
||||
/// Gets the block hash for the given `BlockHashOrNumber`. Returns `None` if no block with this
|
||||
/// number exists. If the `BlockHashOrNumber` is a `Hash`, it is returned as is.
|
||||
fn convert_number(&self, id: BlockHashOrNumber) -> Result<Option<H256>> {
|
||||
fn convert_number(&self, id: BlockHashOrNumber) -> RethResult<Option<H256>> {
|
||||
match id {
|
||||
BlockHashOrNumber::Hash(hash) => Ok(Some(hash)),
|
||||
BlockHashOrNumber::Number(num) => self.block_hash(num),
|
||||
@ -51,7 +51,7 @@ pub trait BlockNumReader: BlockHashReader + Send + Sync {
|
||||
#[auto_impl::auto_impl(&, Arc)]
|
||||
pub trait BlockIdReader: BlockNumReader + Send + Sync {
|
||||
/// Converts the `BlockNumberOrTag` variants to a block number.
|
||||
fn convert_block_number(&self, num: BlockNumberOrTag) -> Result<Option<BlockNumber>> {
|
||||
fn convert_block_number(&self, num: BlockNumberOrTag) -> RethResult<Option<BlockNumber>> {
|
||||
let num = match num {
|
||||
BlockNumberOrTag::Latest => self.best_block_number()?,
|
||||
BlockNumberOrTag::Earliest => 0,
|
||||
@ -74,7 +74,7 @@ pub trait BlockIdReader: BlockNumReader + Send + Sync {
|
||||
}
|
||||
|
||||
/// Get the hash of the block by matching the given id.
|
||||
fn block_hash_for_id(&self, block_id: BlockId) -> Result<Option<H256>> {
|
||||
fn block_hash_for_id(&self, block_id: BlockId) -> RethResult<Option<H256>> {
|
||||
match block_id {
|
||||
BlockId::Hash(hash) => Ok(Some(hash.into())),
|
||||
BlockId::Number(num) => {
|
||||
@ -97,7 +97,7 @@ pub trait BlockIdReader: BlockNumReader + Send + Sync {
|
||||
}
|
||||
|
||||
/// Get the number of the block by matching the given id.
|
||||
fn block_number_for_id(&self, block_id: BlockId) -> Result<Option<BlockNumber>> {
|
||||
fn block_number_for_id(&self, block_id: BlockId) -> RethResult<Option<BlockNumber>> {
|
||||
match block_id {
|
||||
BlockId::Hash(hash) => self.block_number(hash.into()),
|
||||
BlockId::Number(num) => self.convert_block_number(num),
|
||||
@ -105,31 +105,31 @@ pub trait BlockIdReader: BlockNumReader + Send + Sync {
|
||||
}
|
||||
|
||||
/// Get the current pending block number and hash.
|
||||
fn pending_block_num_hash(&self) -> Result<Option<reth_primitives::BlockNumHash>>;
|
||||
fn pending_block_num_hash(&self) -> RethResult<Option<reth_primitives::BlockNumHash>>;
|
||||
|
||||
/// Get the current safe block number and hash.
|
||||
fn safe_block_num_hash(&self) -> Result<Option<reth_primitives::BlockNumHash>>;
|
||||
fn safe_block_num_hash(&self) -> RethResult<Option<reth_primitives::BlockNumHash>>;
|
||||
|
||||
/// Get the current finalized block number and hash.
|
||||
fn finalized_block_num_hash(&self) -> Result<Option<reth_primitives::BlockNumHash>>;
|
||||
fn finalized_block_num_hash(&self) -> RethResult<Option<reth_primitives::BlockNumHash>>;
|
||||
|
||||
/// Get the safe block number.
|
||||
fn safe_block_number(&self) -> Result<Option<BlockNumber>> {
|
||||
fn safe_block_number(&self) -> RethResult<Option<BlockNumber>> {
|
||||
self.safe_block_num_hash().map(|res_opt| res_opt.map(|num_hash| num_hash.number))
|
||||
}
|
||||
|
||||
/// Get the finalized block number.
|
||||
fn finalized_block_number(&self) -> Result<Option<BlockNumber>> {
|
||||
fn finalized_block_number(&self) -> RethResult<Option<BlockNumber>> {
|
||||
self.finalized_block_num_hash().map(|res_opt| res_opt.map(|num_hash| num_hash.number))
|
||||
}
|
||||
|
||||
/// Get the safe block hash.
|
||||
fn safe_block_hash(&self) -> Result<Option<H256>> {
|
||||
fn safe_block_hash(&self) -> RethResult<Option<H256>> {
|
||||
self.safe_block_num_hash().map(|res_opt| res_opt.map(|num_hash| num_hash.hash))
|
||||
}
|
||||
|
||||
/// Get the finalized block hash.
|
||||
fn finalized_block_hash(&self) -> Result<Option<H256>> {
|
||||
fn finalized_block_hash(&self) -> RethResult<Option<H256>> {
|
||||
self.finalized_block_num_hash().map(|res_opt| res_opt.map(|num_hash| num_hash.hash))
|
||||
}
|
||||
}
|
||||
|
||||
@ -1,4 +1,4 @@
|
||||
use reth_interfaces::Result;
|
||||
use reth_interfaces::RethResult;
|
||||
use reth_primitives::{BlockHashOrNumber, Header};
|
||||
use reth_revm_primitives::primitives::{BlockEnv, CfgEnv};
|
||||
|
||||
@ -15,10 +15,10 @@ pub trait EvmEnvProvider: Send + Sync {
|
||||
cfg: &mut CfgEnv,
|
||||
block_env: &mut BlockEnv,
|
||||
at: BlockHashOrNumber,
|
||||
) -> Result<()>;
|
||||
) -> RethResult<()>;
|
||||
|
||||
/// Fills the default [CfgEnv] and [BlockEnv] fields with values specific to the given [Header].
|
||||
fn env_with_header(&self, header: &Header) -> Result<(CfgEnv, BlockEnv)> {
|
||||
fn env_with_header(&self, header: &Header) -> RethResult<(CfgEnv, BlockEnv)> {
|
||||
let mut cfg = CfgEnv::default();
|
||||
let mut block_env = BlockEnv::default();
|
||||
self.fill_env_with_header(&mut cfg, &mut block_env, header)?;
|
||||
@ -31,17 +31,21 @@ pub trait EvmEnvProvider: Send + Sync {
|
||||
cfg: &mut CfgEnv,
|
||||
block_env: &mut BlockEnv,
|
||||
header: &Header,
|
||||
) -> Result<()>;
|
||||
) -> RethResult<()>;
|
||||
|
||||
/// Fills the [BlockEnv] fields with values specific to the given [BlockHashOrNumber].
|
||||
fn fill_block_env_at(&self, block_env: &mut BlockEnv, at: BlockHashOrNumber) -> Result<()>;
|
||||
fn fill_block_env_at(&self, block_env: &mut BlockEnv, at: BlockHashOrNumber) -> RethResult<()>;
|
||||
|
||||
/// Fills the [BlockEnv] fields with values specific to the given [Header].
|
||||
fn fill_block_env_with_header(&self, block_env: &mut BlockEnv, header: &Header) -> Result<()>;
|
||||
fn fill_block_env_with_header(
|
||||
&self,
|
||||
block_env: &mut BlockEnv,
|
||||
header: &Header,
|
||||
) -> RethResult<()>;
|
||||
|
||||
/// Fills the [CfgEnv] fields with values specific to the given [BlockHashOrNumber].
|
||||
fn fill_cfg_env_at(&self, cfg: &mut CfgEnv, at: BlockHashOrNumber) -> Result<()>;
|
||||
fn fill_cfg_env_at(&self, cfg: &mut CfgEnv, at: BlockHashOrNumber) -> RethResult<()>;
|
||||
|
||||
/// Fills the [CfgEnv] fields with values specific to the given [Header].
|
||||
fn fill_cfg_env_with_header(&self, cfg: &mut CfgEnv, header: &Header) -> Result<()>;
|
||||
fn fill_cfg_env_with_header(&self, cfg: &mut CfgEnv, header: &Header) -> RethResult<()>;
|
||||
}
|
||||
|
||||
@ -1,6 +1,6 @@
|
||||
use auto_impl::auto_impl;
|
||||
use reth_db::models::BlockNumberAddress;
|
||||
use reth_interfaces::Result;
|
||||
use reth_interfaces::RethResult;
|
||||
use reth_primitives::{Account, Address, BlockNumber, StorageEntry, H256};
|
||||
use std::{
|
||||
collections::{BTreeMap, BTreeSet, HashMap},
|
||||
@ -18,7 +18,7 @@ pub trait HashingWriter: Send + Sync {
|
||||
fn unwind_account_hashing(
|
||||
&self,
|
||||
range: RangeInclusive<BlockNumber>,
|
||||
) -> Result<BTreeMap<H256, Option<Account>>>;
|
||||
) -> RethResult<BTreeMap<H256, Option<Account>>>;
|
||||
|
||||
/// Inserts all accounts into [reth_db::tables::AccountHistory] table.
|
||||
///
|
||||
@ -28,7 +28,7 @@ pub trait HashingWriter: Send + Sync {
|
||||
fn insert_account_for_hashing(
|
||||
&self,
|
||||
accounts: impl IntoIterator<Item = (Address, Option<Account>)>,
|
||||
) -> Result<BTreeMap<H256, Option<Account>>>;
|
||||
) -> RethResult<BTreeMap<H256, Option<Account>>>;
|
||||
|
||||
/// Unwind and clear storage hashing
|
||||
///
|
||||
@ -38,7 +38,7 @@ pub trait HashingWriter: Send + Sync {
|
||||
fn unwind_storage_hashing(
|
||||
&self,
|
||||
range: Range<BlockNumberAddress>,
|
||||
) -> Result<HashMap<H256, BTreeSet<H256>>>;
|
||||
) -> RethResult<HashMap<H256, BTreeSet<H256>>>;
|
||||
|
||||
/// Iterates over storages and inserts them to hashing table.
|
||||
///
|
||||
@ -48,7 +48,7 @@ pub trait HashingWriter: Send + Sync {
|
||||
fn insert_storage_for_hashing(
|
||||
&self,
|
||||
storages: impl IntoIterator<Item = (Address, impl IntoIterator<Item = StorageEntry>)>,
|
||||
) -> Result<HashMap<H256, BTreeSet<H256>>>;
|
||||
) -> RethResult<HashMap<H256, BTreeSet<H256>>>;
|
||||
|
||||
/// Calculate the hashes of all changed accounts and storages, and finally calculate the state
|
||||
/// root.
|
||||
@ -61,5 +61,5 @@ pub trait HashingWriter: Send + Sync {
|
||||
range: RangeInclusive<BlockNumber>,
|
||||
end_block_hash: H256,
|
||||
expected_state_root: H256,
|
||||
) -> Result<()>;
|
||||
) -> RethResult<()>;
|
||||
}
|
||||
|
||||
@ -1,5 +1,5 @@
|
||||
use auto_impl::auto_impl;
|
||||
use reth_interfaces::Result;
|
||||
use reth_interfaces::RethResult;
|
||||
use reth_primitives::{BlockHash, BlockHashOrNumber, BlockNumber, Header, SealedHeader, U256};
|
||||
use std::ops::RangeBounds;
|
||||
|
||||
@ -7,18 +7,21 @@ use std::ops::RangeBounds;
|
||||
#[auto_impl(&, Arc)]
|
||||
pub trait HeaderProvider: Send + Sync {
|
||||
/// Check if block is known
|
||||
fn is_known(&self, block_hash: &BlockHash) -> Result<bool> {
|
||||
fn is_known(&self, block_hash: &BlockHash) -> RethResult<bool> {
|
||||
self.header(block_hash).map(|header| header.is_some())
|
||||
}
|
||||
|
||||
/// Get header by block hash
|
||||
fn header(&self, block_hash: &BlockHash) -> Result<Option<Header>>;
|
||||
fn header(&self, block_hash: &BlockHash) -> RethResult<Option<Header>>;
|
||||
|
||||
/// Get header by block number
|
||||
fn header_by_number(&self, num: u64) -> Result<Option<Header>>;
|
||||
fn header_by_number(&self, num: u64) -> RethResult<Option<Header>>;
|
||||
|
||||
/// Get header by block number or hash
|
||||
fn header_by_hash_or_number(&self, hash_or_num: BlockHashOrNumber) -> Result<Option<Header>> {
|
||||
fn header_by_hash_or_number(
|
||||
&self,
|
||||
hash_or_num: BlockHashOrNumber,
|
||||
) -> RethResult<Option<Header>> {
|
||||
match hash_or_num {
|
||||
BlockHashOrNumber::Hash(hash) => self.header(&hash),
|
||||
BlockHashOrNumber::Number(num) => self.header_by_number(num),
|
||||
@ -26,20 +29,20 @@ pub trait HeaderProvider: Send + Sync {
|
||||
}
|
||||
|
||||
/// Get total difficulty by block hash.
|
||||
fn header_td(&self, hash: &BlockHash) -> Result<Option<U256>>;
|
||||
fn header_td(&self, hash: &BlockHash) -> RethResult<Option<U256>>;
|
||||
|
||||
/// Get total difficulty by block number.
|
||||
fn header_td_by_number(&self, number: BlockNumber) -> Result<Option<U256>>;
|
||||
fn header_td_by_number(&self, number: BlockNumber) -> RethResult<Option<U256>>;
|
||||
|
||||
/// Get headers in range of block numbers
|
||||
fn headers_range(&self, range: impl RangeBounds<BlockNumber>) -> Result<Vec<Header>>;
|
||||
fn headers_range(&self, range: impl RangeBounds<BlockNumber>) -> RethResult<Vec<Header>>;
|
||||
|
||||
/// Get headers in range of block numbers
|
||||
fn sealed_headers_range(
|
||||
&self,
|
||||
range: impl RangeBounds<BlockNumber>,
|
||||
) -> Result<Vec<SealedHeader>>;
|
||||
) -> RethResult<Vec<SealedHeader>>;
|
||||
|
||||
/// Get a single sealed header by block number
|
||||
fn sealed_header(&self, number: BlockNumber) -> Result<Option<SealedHeader>>;
|
||||
fn sealed_header(&self, number: BlockNumber) -> RethResult<Option<SealedHeader>>;
|
||||
}
|
||||
|
||||
@ -1,6 +1,6 @@
|
||||
use auto_impl::auto_impl;
|
||||
use reth_db::models::BlockNumberAddress;
|
||||
use reth_interfaces::Result;
|
||||
use reth_interfaces::RethResult;
|
||||
use reth_primitives::{Address, BlockNumber, H256};
|
||||
use std::{
|
||||
collections::BTreeMap,
|
||||
@ -13,25 +13,29 @@ pub trait HistoryWriter: Send + Sync {
|
||||
/// Unwind and clear account history indices.
|
||||
///
|
||||
/// Returns number of changesets walked.
|
||||
fn unwind_account_history_indices(&self, range: RangeInclusive<BlockNumber>) -> Result<usize>;
|
||||
fn unwind_account_history_indices(
|
||||
&self,
|
||||
range: RangeInclusive<BlockNumber>,
|
||||
) -> RethResult<usize>;
|
||||
|
||||
/// Insert account change index to database. Used inside AccountHistoryIndex stage
|
||||
fn insert_account_history_index(
|
||||
&self,
|
||||
account_transitions: BTreeMap<Address, Vec<u64>>,
|
||||
) -> Result<()>;
|
||||
) -> RethResult<()>;
|
||||
|
||||
/// Unwind and clear storage history indices.
|
||||
///
|
||||
/// Returns number of changesets walked.
|
||||
fn unwind_storage_history_indices(&self, range: Range<BlockNumberAddress>) -> Result<usize>;
|
||||
fn unwind_storage_history_indices(&self, range: Range<BlockNumberAddress>)
|
||||
-> RethResult<usize>;
|
||||
|
||||
/// Insert storage change index to database. Used inside StorageHistoryIndex stage
|
||||
fn insert_storage_history_index(
|
||||
&self,
|
||||
storage_transitions: BTreeMap<(Address, H256), Vec<u64>>,
|
||||
) -> Result<()>;
|
||||
) -> RethResult<()>;
|
||||
|
||||
/// Read account/storage changesets and update account/storage history indices.
|
||||
fn calculate_history_indices(&self, range: RangeInclusive<BlockNumber>) -> Result<()>;
|
||||
fn calculate_history_indices(&self, range: RangeInclusive<BlockNumber>) -> RethResult<()>;
|
||||
}
|
||||
|
||||
@ -1,16 +1,17 @@
|
||||
use reth_interfaces::Result;
|
||||
use reth_interfaces::RethResult;
|
||||
use reth_primitives::{PruneCheckpoint, PrunePart};
|
||||
|
||||
/// The trait for fetching prune checkpoint related data.
|
||||
#[auto_impl::auto_impl(&, Arc)]
|
||||
pub trait PruneCheckpointReader: Send + Sync {
|
||||
/// Fetch the checkpoint for the given prune part.
|
||||
fn get_prune_checkpoint(&self, part: PrunePart) -> Result<Option<PruneCheckpoint>>;
|
||||
fn get_prune_checkpoint(&self, part: PrunePart) -> RethResult<Option<PruneCheckpoint>>;
|
||||
}
|
||||
|
||||
/// The trait for updating prune checkpoint related data.
|
||||
#[auto_impl::auto_impl(&, Arc)]
|
||||
pub trait PruneCheckpointWriter: Send + Sync {
|
||||
/// Save prune checkpoint.
|
||||
fn save_prune_checkpoint(&self, part: PrunePart, checkpoint: PruneCheckpoint) -> Result<()>;
|
||||
fn save_prune_checkpoint(&self, part: PrunePart, checkpoint: PruneCheckpoint)
|
||||
-> RethResult<()>;
|
||||
}
|
||||
|
||||
@ -1,4 +1,4 @@
|
||||
use reth_interfaces::Result;
|
||||
use reth_interfaces::RethResult;
|
||||
use reth_primitives::{BlockHashOrNumber, BlockId, BlockNumberOrTag, Receipt, TxHash, TxNumber};
|
||||
|
||||
use crate::BlockIdReader;
|
||||
@ -9,17 +9,17 @@ pub trait ReceiptProvider: Send + Sync {
|
||||
/// Get receipt by transaction number
|
||||
///
|
||||
/// Returns `None` if the transaction is not found.
|
||||
fn receipt(&self, id: TxNumber) -> Result<Option<Receipt>>;
|
||||
fn receipt(&self, id: TxNumber) -> RethResult<Option<Receipt>>;
|
||||
|
||||
/// Get receipt by transaction hash.
|
||||
///
|
||||
/// Returns `None` if the transaction is not found.
|
||||
fn receipt_by_hash(&self, hash: TxHash) -> Result<Option<Receipt>>;
|
||||
fn receipt_by_hash(&self, hash: TxHash) -> RethResult<Option<Receipt>>;
|
||||
|
||||
/// Get receipts by block num or hash.
|
||||
///
|
||||
/// Returns `None` if the block is not found.
|
||||
fn receipts_by_block(&self, block: BlockHashOrNumber) -> Result<Option<Vec<Receipt>>>;
|
||||
fn receipts_by_block(&self, block: BlockHashOrNumber) -> RethResult<Option<Vec<Receipt>>>;
|
||||
}
|
||||
|
||||
/// Trait extension for `ReceiptProvider`, for types that implement `BlockId` conversion.
|
||||
@ -34,7 +34,7 @@ pub trait ReceiptProvider: Send + Sync {
|
||||
/// retrieving the receipts should be done using the type's `ReceiptProvider` methods.
|
||||
pub trait ReceiptProviderIdExt: ReceiptProvider + BlockIdReader {
|
||||
/// Get receipt by block id
|
||||
fn receipts_by_block_id(&self, block: BlockId) -> Result<Option<Vec<Receipt>>> {
|
||||
fn receipts_by_block_id(&self, block: BlockId) -> RethResult<Option<Vec<Receipt>>> {
|
||||
let id = match block {
|
||||
BlockId::Hash(hash) => BlockHashOrNumber::Hash(hash.block_hash),
|
||||
BlockId::Number(num_tag) => {
|
||||
@ -55,7 +55,7 @@ pub trait ReceiptProviderIdExt: ReceiptProvider + BlockIdReader {
|
||||
fn receipts_by_number_or_tag(
|
||||
&self,
|
||||
number_or_tag: BlockNumberOrTag,
|
||||
) -> Result<Option<Vec<Receipt>>> {
|
||||
) -> RethResult<Option<Vec<Receipt>>> {
|
||||
self.receipts_by_block_id(number_or_tag.into())
|
||||
}
|
||||
}
|
||||
|
||||
@ -1,4 +1,4 @@
|
||||
use reth_interfaces::Result;
|
||||
use reth_interfaces::RethResult;
|
||||
use reth_primitives::{
|
||||
stage::{StageCheckpoint, StageId},
|
||||
BlockNumber,
|
||||
@ -8,25 +8,25 @@ use reth_primitives::{
|
||||
#[auto_impl::auto_impl(&, Arc)]
|
||||
pub trait StageCheckpointReader: Send + Sync {
|
||||
/// Fetch the checkpoint for the given stage.
|
||||
fn get_stage_checkpoint(&self, id: StageId) -> Result<Option<StageCheckpoint>>;
|
||||
fn get_stage_checkpoint(&self, id: StageId) -> RethResult<Option<StageCheckpoint>>;
|
||||
|
||||
/// Get stage checkpoint progress.
|
||||
fn get_stage_checkpoint_progress(&self, id: StageId) -> Result<Option<Vec<u8>>>;
|
||||
fn get_stage_checkpoint_progress(&self, id: StageId) -> RethResult<Option<Vec<u8>>>;
|
||||
}
|
||||
|
||||
/// The trait for updating stage checkpoint related data.
|
||||
#[auto_impl::auto_impl(&, Arc)]
|
||||
pub trait StageCheckpointWriter: Send + Sync {
|
||||
/// Save stage checkpoint.
|
||||
fn save_stage_checkpoint(&self, id: StageId, checkpoint: StageCheckpoint) -> Result<()>;
|
||||
fn save_stage_checkpoint(&self, id: StageId, checkpoint: StageCheckpoint) -> RethResult<()>;
|
||||
|
||||
/// Save stage checkpoint progress.
|
||||
fn save_stage_checkpoint_progress(&self, id: StageId, checkpoint: Vec<u8>) -> Result<()>;
|
||||
fn save_stage_checkpoint_progress(&self, id: StageId, checkpoint: Vec<u8>) -> RethResult<()>;
|
||||
|
||||
/// Update all pipeline sync stage progress.
|
||||
fn update_pipeline_stages(
|
||||
&self,
|
||||
block_number: BlockNumber,
|
||||
drop_stage_checkpoint: bool,
|
||||
) -> Result<()>;
|
||||
) -> RethResult<()>;
|
||||
}
|
||||
|
||||
@ -1,7 +1,7 @@
|
||||
use super::AccountReader;
|
||||
use crate::{BlockHashReader, BlockIdReader, BundleStateWithReceipts};
|
||||
use auto_impl::auto_impl;
|
||||
use reth_interfaces::{provider::ProviderError, Result};
|
||||
use reth_interfaces::{provider::ProviderError, RethResult};
|
||||
use reth_primitives::{
|
||||
Address, BlockHash, BlockId, BlockNumHash, BlockNumber, BlockNumberOrTag, Bytecode, Bytes,
|
||||
StorageKey, StorageValue, H256, KECCAK_EMPTY, U256,
|
||||
@ -14,19 +14,26 @@ pub type StateProviderBox<'a> = Box<dyn StateProvider + 'a>;
|
||||
#[auto_impl(&, Arc, Box)]
|
||||
pub trait StateProvider: BlockHashReader + AccountReader + StateRootProvider + Send + Sync {
|
||||
/// Get storage of given account.
|
||||
fn storage(&self, account: Address, storage_key: StorageKey) -> Result<Option<StorageValue>>;
|
||||
fn storage(
|
||||
&self,
|
||||
account: Address,
|
||||
storage_key: StorageKey,
|
||||
) -> RethResult<Option<StorageValue>>;
|
||||
|
||||
/// Get account code by its hash
|
||||
fn bytecode_by_hash(&self, code_hash: H256) -> Result<Option<Bytecode>>;
|
||||
fn bytecode_by_hash(&self, code_hash: H256) -> RethResult<Option<Bytecode>>;
|
||||
|
||||
/// Get account and storage proofs.
|
||||
fn proof(&self, address: Address, keys: &[H256])
|
||||
-> Result<(Vec<Bytes>, H256, Vec<Vec<Bytes>>)>;
|
||||
fn proof(
|
||||
&self,
|
||||
address: Address,
|
||||
keys: &[H256],
|
||||
) -> RethResult<(Vec<Bytes>, H256, Vec<Vec<Bytes>>)>;
|
||||
|
||||
/// Get account code by its address.
|
||||
///
|
||||
/// Returns `None` if the account doesn't exist or account is not a contract
|
||||
fn account_code(&self, addr: Address) -> Result<Option<Bytecode>> {
|
||||
fn account_code(&self, addr: Address) -> RethResult<Option<Bytecode>> {
|
||||
// Get basic account information
|
||||
// Returns None if acc doesn't exist
|
||||
let acc = match self.basic_account(addr)? {
|
||||
@ -49,7 +56,7 @@ pub trait StateProvider: BlockHashReader + AccountReader + StateRootProvider + S
|
||||
/// Get account balance by its address.
|
||||
///
|
||||
/// Returns `None` if the account doesn't exist
|
||||
fn account_balance(&self, addr: Address) -> Result<Option<U256>> {
|
||||
fn account_balance(&self, addr: Address) -> RethResult<Option<U256>> {
|
||||
// Get basic account information
|
||||
// Returns None if acc doesn't exist
|
||||
match self.basic_account(addr)? {
|
||||
@ -61,7 +68,7 @@ pub trait StateProvider: BlockHashReader + AccountReader + StateRootProvider + S
|
||||
/// Get account nonce by its address.
|
||||
///
|
||||
/// Returns `None` if the account doesn't exist
|
||||
fn account_nonce(&self, addr: Address) -> Result<Option<u64>> {
|
||||
fn account_nonce(&self, addr: Address) -> RethResult<Option<u64>> {
|
||||
// Get basic account information
|
||||
// Returns None if acc doesn't exist
|
||||
match self.basic_account(addr)? {
|
||||
@ -96,12 +103,12 @@ pub trait StateProvider: BlockHashReader + AccountReader + StateRootProvider + S
|
||||
/// to be used, since block `n` was executed on its parent block's state.
|
||||
pub trait StateProviderFactory: BlockIdReader + Send + Sync {
|
||||
/// Storage provider for latest block.
|
||||
fn latest(&self) -> Result<StateProviderBox<'_>>;
|
||||
fn latest(&self) -> RethResult<StateProviderBox<'_>>;
|
||||
|
||||
/// Returns a [StateProvider] indexed by the given [BlockId].
|
||||
///
|
||||
/// Note: if a number or hash is provided this will only look at historical(canonical) state.
|
||||
fn state_by_block_id(&self, block_id: BlockId) -> Result<StateProviderBox<'_>> {
|
||||
fn state_by_block_id(&self, block_id: BlockId) -> RethResult<StateProviderBox<'_>> {
|
||||
match block_id {
|
||||
BlockId::Number(block_number) => self.state_by_block_number_or_tag(block_number),
|
||||
BlockId::Hash(block_hash) => self.history_by_block_hash(block_hash.into()),
|
||||
@ -114,7 +121,7 @@ pub trait StateProviderFactory: BlockIdReader + Send + Sync {
|
||||
fn state_by_block_number_or_tag(
|
||||
&self,
|
||||
number_or_tag: BlockNumberOrTag,
|
||||
) -> Result<StateProviderBox<'_>> {
|
||||
) -> RethResult<StateProviderBox<'_>> {
|
||||
match number_or_tag {
|
||||
BlockNumberOrTag::Latest => self.latest(),
|
||||
BlockNumberOrTag::Finalized => {
|
||||
@ -148,37 +155,37 @@ pub trait StateProviderFactory: BlockIdReader + Send + Sync {
|
||||
///
|
||||
///
|
||||
/// Note: this only looks at historical blocks, not pending blocks.
|
||||
fn history_by_block_number(&self, block: BlockNumber) -> Result<StateProviderBox<'_>>;
|
||||
fn history_by_block_number(&self, block: BlockNumber) -> RethResult<StateProviderBox<'_>>;
|
||||
|
||||
/// Returns a historical [StateProvider] indexed by the given block hash.
|
||||
///
|
||||
/// Note: this only looks at historical blocks, not pending blocks.
|
||||
fn history_by_block_hash(&self, block: BlockHash) -> Result<StateProviderBox<'_>>;
|
||||
fn history_by_block_hash(&self, block: BlockHash) -> RethResult<StateProviderBox<'_>>;
|
||||
|
||||
/// Returns _any_[StateProvider] with matching block hash.
|
||||
///
|
||||
/// This will return a [StateProvider] for either a historical or pending block.
|
||||
fn state_by_block_hash(&self, block: BlockHash) -> Result<StateProviderBox<'_>>;
|
||||
fn state_by_block_hash(&self, block: BlockHash) -> RethResult<StateProviderBox<'_>>;
|
||||
|
||||
/// Storage provider for pending state.
|
||||
///
|
||||
/// Represents the state at the block that extends the canonical chain by one.
|
||||
/// If there's no `pending` block, then this is equal to [StateProviderFactory::latest]
|
||||
fn pending(&self) -> Result<StateProviderBox<'_>>;
|
||||
fn pending(&self) -> RethResult<StateProviderBox<'_>>;
|
||||
|
||||
/// Storage provider for pending state for the given block hash.
|
||||
///
|
||||
/// Represents the state at the block that extends the canonical chain.
|
||||
///
|
||||
/// If the block couldn't be found, returns `None`.
|
||||
fn pending_state_by_hash(&self, block_hash: H256) -> Result<Option<StateProviderBox<'_>>>;
|
||||
fn pending_state_by_hash(&self, block_hash: H256) -> RethResult<Option<StateProviderBox<'_>>>;
|
||||
|
||||
/// Return a [StateProvider] that contains post state data provider.
|
||||
/// Used to inspect or execute transaction on the pending state.
|
||||
fn pending_with_provider(
|
||||
&self,
|
||||
post_state_data: Box<dyn BundleStateDataProvider>,
|
||||
) -> Result<StateProviderBox<'_>>;
|
||||
) -> RethResult<StateProviderBox<'_>>;
|
||||
}
|
||||
|
||||
/// Blockchain trait provider that gives access to the blockchain state that is not yet committed
|
||||
@ -191,7 +198,7 @@ pub trait BlockchainTreePendingStateProvider: Send + Sync {
|
||||
fn pending_state_provider(
|
||||
&self,
|
||||
block_hash: BlockHash,
|
||||
) -> Result<Box<dyn BundleStateDataProvider>> {
|
||||
) -> RethResult<Box<dyn BundleStateDataProvider>> {
|
||||
Ok(self
|
||||
.find_pending_state_provider(block_hash)
|
||||
.ok_or(ProviderError::StateForHashNotFound(block_hash))?)
|
||||
@ -227,5 +234,5 @@ pub trait BundleStateDataProvider: Send + Sync {
|
||||
#[auto_impl[Box,&, Arc]]
|
||||
pub trait StateRootProvider: Send + Sync {
|
||||
/// Returns the state root of the BundleState on top of the current state.
|
||||
fn state_root(&self, post_state: BundleStateWithReceipts) -> Result<H256>;
|
||||
fn state_root(&self, post_state: BundleStateWithReceipts) -> RethResult<H256>;
|
||||
}
|
||||
|
||||
@ -4,7 +4,7 @@ use std::{
|
||||
};
|
||||
|
||||
use auto_impl::auto_impl;
|
||||
use reth_interfaces::Result;
|
||||
use reth_interfaces::RethResult;
|
||||
use reth_primitives::{Address, BlockNumber, StorageEntry, H256};
|
||||
|
||||
/// Storage reader
|
||||
@ -14,13 +14,13 @@ pub trait StorageReader: Send + Sync {
|
||||
fn plainstate_storages(
|
||||
&self,
|
||||
addresses_with_keys: impl IntoIterator<Item = (Address, impl IntoIterator<Item = H256>)>,
|
||||
) -> Result<Vec<(Address, Vec<StorageEntry>)>>;
|
||||
) -> RethResult<Vec<(Address, Vec<StorageEntry>)>>;
|
||||
|
||||
/// Iterate over storage changesets and return all storage slots that were changed.
|
||||
fn changed_storages_with_range(
|
||||
&self,
|
||||
range: RangeInclusive<BlockNumber>,
|
||||
) -> Result<BTreeMap<Address, BTreeSet<H256>>>;
|
||||
) -> RethResult<BTreeMap<Address, BTreeSet<H256>>>;
|
||||
|
||||
/// Iterate over storage changesets and return all storage slots that were changed alongside
|
||||
/// each specific set of blocks.
|
||||
@ -29,5 +29,5 @@ pub trait StorageReader: Send + Sync {
|
||||
fn changed_storages_and_blocks_with_range(
|
||||
&self,
|
||||
range: RangeInclusive<BlockNumber>,
|
||||
) -> Result<BTreeMap<(Address, H256), Vec<u64>>>;
|
||||
) -> RethResult<BTreeMap<(Address, H256), Vec<u64>>>;
|
||||
}
|
||||
|
||||
@ -1,5 +1,5 @@
|
||||
use crate::BlockNumReader;
|
||||
use reth_interfaces::Result;
|
||||
use reth_interfaces::RethResult;
|
||||
use reth_primitives::{
|
||||
Address, BlockHashOrNumber, BlockNumber, TransactionMeta, TransactionSigned,
|
||||
TransactionSignedNoHash, TxHash, TxNumber,
|
||||
@ -13,50 +13,53 @@ pub trait TransactionsProvider: BlockNumReader + Send + Sync {
|
||||
///
|
||||
/// This is the inverse of [TransactionsProvider::transaction_by_id].
|
||||
/// Returns None if the transaction is not found.
|
||||
fn transaction_id(&self, tx_hash: TxHash) -> Result<Option<TxNumber>>;
|
||||
fn transaction_id(&self, tx_hash: TxHash) -> RethResult<Option<TxNumber>>;
|
||||
|
||||
/// Get transaction by id, computes hash everytime so more expensive.
|
||||
fn transaction_by_id(&self, id: TxNumber) -> Result<Option<TransactionSigned>>;
|
||||
fn transaction_by_id(&self, id: TxNumber) -> RethResult<Option<TransactionSigned>>;
|
||||
|
||||
/// Get transaction by id without computing the hash.
|
||||
fn transaction_by_id_no_hash(&self, id: TxNumber) -> Result<Option<TransactionSignedNoHash>>;
|
||||
fn transaction_by_id_no_hash(
|
||||
&self,
|
||||
id: TxNumber,
|
||||
) -> RethResult<Option<TransactionSignedNoHash>>;
|
||||
|
||||
/// Get transaction by transaction hash.
|
||||
fn transaction_by_hash(&self, hash: TxHash) -> Result<Option<TransactionSigned>>;
|
||||
fn transaction_by_hash(&self, hash: TxHash) -> RethResult<Option<TransactionSigned>>;
|
||||
|
||||
/// Get transaction by transaction hash and additional metadata of the block the transaction was
|
||||
/// mined in
|
||||
fn transaction_by_hash_with_meta(
|
||||
&self,
|
||||
hash: TxHash,
|
||||
) -> Result<Option<(TransactionSigned, TransactionMeta)>>;
|
||||
) -> RethResult<Option<(TransactionSigned, TransactionMeta)>>;
|
||||
|
||||
/// Get transaction block number
|
||||
fn transaction_block(&self, id: TxNumber) -> Result<Option<BlockNumber>>;
|
||||
fn transaction_block(&self, id: TxNumber) -> RethResult<Option<BlockNumber>>;
|
||||
|
||||
/// Get transactions by block id.
|
||||
fn transactions_by_block(
|
||||
&self,
|
||||
block: BlockHashOrNumber,
|
||||
) -> Result<Option<Vec<TransactionSigned>>>;
|
||||
) -> RethResult<Option<Vec<TransactionSigned>>>;
|
||||
|
||||
/// Get transactions by block range.
|
||||
fn transactions_by_block_range(
|
||||
&self,
|
||||
range: impl RangeBounds<BlockNumber>,
|
||||
) -> Result<Vec<Vec<TransactionSigned>>>;
|
||||
) -> RethResult<Vec<Vec<TransactionSigned>>>;
|
||||
|
||||
/// Get transactions by tx range.
|
||||
fn transactions_by_tx_range(
|
||||
&self,
|
||||
range: impl RangeBounds<TxNumber>,
|
||||
) -> Result<Vec<TransactionSignedNoHash>>;
|
||||
) -> RethResult<Vec<TransactionSignedNoHash>>;
|
||||
|
||||
/// Get Senders from a tx range.
|
||||
fn senders_by_tx_range(&self, range: impl RangeBounds<TxNumber>) -> Result<Vec<Address>>;
|
||||
fn senders_by_tx_range(&self, range: impl RangeBounds<TxNumber>) -> RethResult<Vec<Address>>;
|
||||
|
||||
/// Get transaction sender.
|
||||
///
|
||||
/// Returns None if the transaction is not found.
|
||||
fn transaction_sender(&self, id: TxNumber) -> Result<Option<Address>>;
|
||||
fn transaction_sender(&self, id: TxNumber) -> RethResult<Option<Address>>;
|
||||
}
|
||||
|
||||
@ -1,4 +1,4 @@
|
||||
use reth_interfaces::Result;
|
||||
use reth_interfaces::RethResult;
|
||||
use reth_primitives::{BlockHashOrNumber, Withdrawal};
|
||||
|
||||
/// Client trait for fetching [Withdrawal] related data.
|
||||
@ -9,8 +9,8 @@ pub trait WithdrawalsProvider: Send + Sync {
|
||||
&self,
|
||||
id: BlockHashOrNumber,
|
||||
timestamp: u64,
|
||||
) -> Result<Option<Vec<Withdrawal>>>;
|
||||
) -> RethResult<Option<Vec<Withdrawal>>>;
|
||||
|
||||
/// Get latest withdrawal from this block or earlier .
|
||||
fn latest_withdrawal(&self) -> Result<Option<Withdrawal>>;
|
||||
fn latest_withdrawal(&self) -> RethResult<Option<Withdrawal>>;
|
||||
}
|
||||
|
||||
@ -10,6 +10,7 @@ use futures_util::{
|
||||
future::{BoxFuture, Fuse, FusedFuture},
|
||||
FutureExt, Stream, StreamExt,
|
||||
};
|
||||
use reth_interfaces::RethError;
|
||||
use reth_primitives::{
|
||||
Address, BlockHash, BlockNumber, BlockNumberOrTag, FromRecoveredTransaction,
|
||||
};
|
||||
@ -470,7 +471,7 @@ fn load_accounts<Client, I>(
|
||||
client: Client,
|
||||
at: BlockHash,
|
||||
addresses: I,
|
||||
) -> Result<LoadedAccounts, Box<(HashSet<Address>, reth_interfaces::Error)>>
|
||||
) -> Result<LoadedAccounts, Box<(HashSet<Address>, RethError)>>
|
||||
where
|
||||
I: Iterator<Item = Address>,
|
||||
|
||||
|
||||
@ -269,14 +269,12 @@ mod tests {
|
||||
use super::*;
|
||||
use crate::StateRoot;
|
||||
use reth_db::{database::Database, test_utils::create_test_rw_db};
|
||||
use reth_interfaces::RethResult;
|
||||
use reth_primitives::{ChainSpec, StorageEntry, MAINNET};
|
||||
use reth_provider::{HashingWriter, ProviderFactory};
|
||||
use std::{str::FromStr, sync::Arc};
|
||||
|
||||
fn insert_genesis<DB: Database>(
|
||||
db: DB,
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
) -> reth_interfaces::Result<()> {
|
||||
fn insert_genesis<DB: Database>(db: DB, chain_spec: Arc<ChainSpec>) -> RethResult<()> {
|
||||
let provider_factory = ProviderFactory::new(db, chain_spec.clone());
|
||||
let mut provider = provider_factory.provider_rw()?;
|
||||
|
||||
|
||||
@ -2,6 +2,7 @@
|
||||
|
||||
use crate::Case;
|
||||
use reth_db::DatabaseError;
|
||||
use reth_interfaces::RethError;
|
||||
use std::path::{Path, PathBuf};
|
||||
use thiserror::Error;
|
||||
|
||||
@ -42,7 +43,7 @@ pub enum Error {
|
||||
Assertion(String),
|
||||
/// An error internally in reth occurred.
|
||||
#[error("Test failed: {0}")]
|
||||
RethError(#[from] reth_interfaces::Error),
|
||||
RethError(#[from] RethError),
|
||||
/// An error occurred while decoding RLP.
|
||||
#[error("An error occurred deserializing RLP")]
|
||||
RlpDecodeError(#[from] reth_rlp::DecodeError),
|
||||
|
||||
Reference in New Issue
Block a user