From 20707565c2fd79f51d1ff68d440b943e37d14e96 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Tue, 7 May 2024 18:34:41 +0100 Subject: [PATCH 01/61] add StaticFileProvider::check_consistency --- crates/primitives/src/static_file/segment.rs | 5 + .../src/providers/static_file/manager.rs | 162 +++++++++++++++++- 2 files changed, 164 insertions(+), 3 deletions(-) diff --git a/crates/primitives/src/static_file/segment.rs b/crates/primitives/src/static_file/segment.rs index a9ad2a075f9..bf5798a244e 100644 --- a/crates/primitives/src/static_file/segment.rs +++ b/crates/primitives/src/static_file/segment.rs @@ -138,6 +138,11 @@ impl StaticFileSegment { pub fn is_headers(&self) -> bool { matches!(self, StaticFileSegment::Headers) } + + /// Returns `true` if the segment is `StaticFileSegment::Receipts`. + pub fn is_receipts(&self) -> bool { + matches!(self, StaticFileSegment::Receipts) + } } /// A segment header that contains information common to all segments. Used for storage. diff --git a/crates/storage/provider/src/providers/static_file/manager.rs b/crates/storage/provider/src/providers/static_file/manager.rs index 7814a709768..5da3724d134 100644 --- a/crates/storage/provider/src/providers/static_file/manager.rs +++ b/crates/storage/provider/src/providers/static_file/manager.rs @@ -3,23 +3,26 @@ use super::{ StaticFileProviderRWRefMut, BLOCKS_PER_STATIC_FILE, }; use crate::{ - to_range, BlockHashReader, BlockNumReader, BlockReader, BlockSource, HeaderProvider, - ReceiptProvider, StatsReader, TransactionVariant, TransactionsProvider, - TransactionsProviderExt, WithdrawalsProvider, + to_range, BlockHashReader, BlockNumReader, BlockReader, BlockSource, DatabaseProvider, + HeaderProvider, ReceiptProvider, StageCheckpointReader, StatsReader, TransactionVariant, + TransactionsProvider, TransactionsProviderExt, WithdrawalsProvider, }; use dashmap::{mapref::entry::Entry as DashMapEntry, DashMap}; use parking_lot::RwLock; use reth_db::{ codecs::CompactU256, + cursor::DbCursorRO, models::StoredBlockBodyIndices, static_file::{iter_static_files, HeaderMask, ReceiptMask, StaticFileCursor, TransactionMask}, table::Table, tables, + transaction::DbTx, }; use reth_interfaces::provider::{ProviderError, ProviderResult}; use reth_nippy_jar::NippyJar; use reth_primitives::{ keccak256, + stage::StageId, static_file::{find_fixed_range, HighestStaticFiles, SegmentHeader, SegmentRangeInclusive}, Address, Block, BlockHash, BlockHashOrNumber, BlockNumber, BlockWithSenders, ChainInfo, Header, Receipt, SealedBlock, SealedBlockWithSenders, SealedHeader, StaticFileSegment, TransactionMeta, @@ -32,6 +35,7 @@ use std::{ path::{Path, PathBuf}, sync::{mpsc, Arc}, }; +use strum::IntoEnumIterator; use tracing::warn; /// Alias type for a map that can be queried for block ranges from a transaction @@ -448,6 +452,158 @@ impl StaticFileProvider { Ok(()) } + /// Makes consistency checks across all static file segments. + pub fn check_consistency( + &self, + provider: &DatabaseProvider, + ) -> ProviderResult> { + let mut unwind_target: Option = None; + let mut update_unwind_target = |new_target: Option| { + let new_target = new_target.unwrap_or_default(); + if let Some(target) = unwind_target.as_mut() { + *target = (*target).min(new_target); + } else { + unwind_target = Some(new_target); + } + }; + + // Since Header Stage is the first stage, a mismatch with Finish Stage means that we had an + // interrupted pipeline run/unwind. + let interrupted_pipeline = provider.get_stage_checkpoint(StageId::Finish)? != + provider.get_stage_checkpoint(StageId::Headers)?; + + // TODO: this check is not good + let is_pruned_node = + self.get_highest_static_file_block(StaticFileSegment::Receipts).is_none() && + provider.get_stage_checkpoint(StageId::Execution)?.is_some() && + provider.tx_ref().entries::()? > 0; + + for segment in StaticFileSegment::iter() { + if is_pruned_node && segment.is_receipts() { + // Pruned nodes (including full node) do not store receipts as static files. + continue + } + + let initial_highest_block = self.get_highest_static_file_block(segment); + + // By calling a writer, an inner consistency check will be triggered on + // [`NippyJarWriter::check_consistency_and_heal`] + // + // * If appending data was interrupted before a config commit, then data file will be + // truncated according to the config. + // + // * If pruning data was interrupted before a config commit, then we have deleted data + // that we are expected to still have. We need to check the Database and unwind + // everything accordingly. + let mut writer = self.latest_writer(segment)?; + + // This will commit any inconsistencies found and update the reader index + writer.commit()?; + + let mut highest_block = self.get_highest_static_file_block(segment); + if initial_highest_block != highest_block { + // The updated `highest_block` is a consistency heal as a result from some pruning + // behavior being interrupted. + update_unwind_target(highest_block); + } + + // Make sure the last transaction matches the last block from its indices. + // Only applies to Transaction & Receipt segments + let highest_tx = self.get_highest_static_file_tx(segment); + if let Some(highest_tx) = highest_tx { + let mut last_block = highest_block.unwrap_or_default(); + // To be extra safe, we make sure that the last tx num matches the last block from + // its indices. If not, get it and create an unwind target, since we + // are missing data we should have. + loop { + if let Some(indices) = provider.block_body_indices(last_block)? { + if indices.last_tx_num() <= highest_tx { + break + } + } + if last_block == 0 { + break + } + last_block -= 1; + + highest_block = Some(last_block); + update_unwind_target(highest_block); + } + } + + // There should be no gap between static file & database elements. + if let Some(unwind) = if segment.is_headers() { + self.ensure_header_continuity(provider, highest_block)? + } else { + self.ensure_tx_continuity(segment, provider, highest_tx, highest_block)? + } { + update_unwind_target(Some(unwind)); + } + } + + Ok(unwind_target) + } + + /// Ensures that there are no gaps between static file headers and database headers. + /// + /// If a gap is found (eg. static block 5 and first db block 7), return a PipelineTarget::Unwind + /// to the given static file block as to restore missing data. + fn ensure_header_continuity( + &self, + provider: &DatabaseProvider, + highest_static: Option, + ) -> ProviderResult> { + let highest_static = highest_static.unwrap_or_default(); + let mut db_cursor = provider.tx_ref().cursor_read::()?; + + if let Some((db_header_num, _)) = db_cursor.first()? { + // If there is a gap between the header found in static file and + // database, then we have most likely lost static file data and need to load it again + if !(db_header_num <= highest_static || highest_static + 1 == db_header_num) { + return Ok(Some(highest_static)) + } + } + + Ok(None) + } + + /// Ensures that there are no gaps between static file transactions/receipts and database + /// transactions/receipts. + /// + /// If a gap is found (eg. static tx 5 and first db tx 7), return a PipelineTarget::Unwind to + /// the given static file block as to restore missing data. + fn ensure_tx_continuity( + &self, + segment: StaticFileSegment, + provider: &DatabaseProvider, + highest_static_tx: Option, + highest_static_block: Option, + ) -> ProviderResult> { + let highest_static_tx = highest_static_tx.unwrap_or_default(); + let first_db_tx_num = match segment { + StaticFileSegment::Headers => unreachable!(), + StaticFileSegment::Transactions => { + let mut cursor = provider.tx_ref().cursor_read::()?; + cursor.first()?.map(|(tx_num, _)| tx_num) + } + StaticFileSegment::Receipts => { + let mut cursor = provider.tx_ref().cursor_read::()?; + cursor.first()?.map(|(tx_num, _)| tx_num) + } + }; + + if let Some(first_db_tx_num) = first_db_tx_num { + // If there is a gap more than one element between the header found in static file and + // database, then we have most likely lost static file data and need to load it again + if !(first_db_tx_num <= highest_static_tx || highest_static_tx + 1 == first_db_tx_num) { + // TODO is this unwrap or default safe + return Ok(Some(highest_static_block.unwrap_or_default())) + } + } + + Ok(None) + } + /// Gets the highest static file block if it exists for a static file segment. pub fn get_highest_static_file_block(&self, segment: StaticFileSegment) -> Option { self.static_files_max_block.read().get(&segment).copied() From 65495fb637642b6855f7516011ac21b7dd99e369 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Wed, 8 May 2024 15:06:37 +0100 Subject: [PATCH 02/61] use PipelineTarget::Unwind --- .../src/providers/static_file/manager.rs | 21 ++++++++++++------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/crates/storage/provider/src/providers/static_file/manager.rs b/crates/storage/provider/src/providers/static_file/manager.rs index 5da3724d134..94687ac2db8 100644 --- a/crates/storage/provider/src/providers/static_file/manager.rs +++ b/crates/storage/provider/src/providers/static_file/manager.rs @@ -22,7 +22,7 @@ use reth_interfaces::provider::{ProviderError, ProviderResult}; use reth_nippy_jar::NippyJar; use reth_primitives::{ keccak256, - stage::StageId, + stage::{PipelineTarget, StageId}, static_file::{find_fixed_range, HighestStaticFiles, SegmentHeader, SegmentRangeInclusive}, Address, Block, BlockHash, BlockHashOrNumber, BlockNumber, BlockWithSenders, ChainInfo, Header, Receipt, SealedBlock, SealedBlockWithSenders, SealedHeader, StaticFileSegment, TransactionMeta, @@ -453,10 +453,15 @@ impl StaticFileProvider { } /// Makes consistency checks across all static file segments. + /// + /// invariant: static files and database should have continuity + /// invariant: if there are no database entries, stage checkpoints should match static file + /// + /// Returns a [`Option::Some`] of [`PipelineTarget::Unwind`] if any healing was done. pub fn check_consistency( &self, provider: &DatabaseProvider, - ) -> ProviderResult> { + ) -> ProviderResult> { let mut unwind_target: Option = None; let mut update_unwind_target = |new_target: Option| { let new_target = new_target.unwrap_or_default(); @@ -472,7 +477,7 @@ impl StaticFileProvider { let interrupted_pipeline = provider.get_stage_checkpoint(StageId::Finish)? != provider.get_stage_checkpoint(StageId::Headers)?; - // TODO: this check is not good + // TODO: this check is not good enough let is_pruned_node = self.get_highest_static_file_block(StaticFileSegment::Receipts).is_none() && provider.get_stage_checkpoint(StageId::Execution)?.is_some() && @@ -541,13 +546,13 @@ impl StaticFileProvider { } } - Ok(unwind_target) + Ok(unwind_target.map(PipelineTarget::Unwind)) } /// Ensures that there are no gaps between static file headers and database headers. /// - /// If a gap is found (eg. static block 5 and first db block 7), return a PipelineTarget::Unwind - /// to the given static file block as to restore missing data. + /// If a gap is found (eg. static block 5 and first db block 7), return a block number to unwind + /// to as to restore missing data. fn ensure_header_continuity( &self, provider: &DatabaseProvider, @@ -570,8 +575,8 @@ impl StaticFileProvider { /// Ensures that there are no gaps between static file transactions/receipts and database /// transactions/receipts. /// - /// If a gap is found (eg. static tx 5 and first db tx 7), return a PipelineTarget::Unwind to - /// the given static file block as to restore missing data. + /// If a gap is found (eg. static tx 5 and first db tx 7), return a block number to unwind to as + /// to restore missing data. fn ensure_tx_continuity( &self, segment: StaticFileSegment, From d8a44ddb589e1b90ae476d57f0926c501ce7eb65 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Sat, 11 May 2024 15:39:41 +0100 Subject: [PATCH 03/61] generalize check_invariants and add docs --- .../src/providers/static_file/manager.rs | 145 +++++++++--------- 1 file changed, 74 insertions(+), 71 deletions(-) diff --git a/crates/storage/provider/src/providers/static_file/manager.rs b/crates/storage/provider/src/providers/static_file/manager.rs index 94687ac2db8..6c8b6f0ece3 100644 --- a/crates/storage/provider/src/providers/static_file/manager.rs +++ b/crates/storage/provider/src/providers/static_file/manager.rs @@ -452,12 +452,23 @@ impl StaticFileProvider { Ok(()) } - /// Makes consistency checks across all static file segments. + /// Ensures that any broken invariants return a pipeline target to unwind to. /// - /// invariant: static files and database should have continuity - /// invariant: if there are no database entries, stage checkpoints should match static file + /// Two types of consistency checks are done for: /// - /// Returns a [`Option::Some`] of [`PipelineTarget::Unwind`] if any healing was done. + /// 1) When a static file fails to commit but the underlying data was changed. + /// 2) When a static file was committed, but the required database transaction was not. + /// + /// For 1) it can self-heal. + /// For 2) the invariants below are checked, and if broken, require a pipeline unwind to heal. + /// + /// For each static file segment: + /// * the corresponding database table should overlap or have continuity in their keys + /// ([TxNumber] or [BlockNumber]). + /// * its highest block should match the stage checkpoint block number if it's equal or higher + /// than the corresponding database table last entry. + /// + /// Returns a [`Option`] of [`PipelineTarget::Unwind`] if any healing is required. pub fn check_consistency( &self, provider: &DatabaseProvider, @@ -472,12 +483,7 @@ impl StaticFileProvider { } }; - // Since Header Stage is the first stage, a mismatch with Finish Stage means that we had an - // interrupted pipeline run/unwind. - let interrupted_pipeline = provider.get_stage_checkpoint(StageId::Finish)? != - provider.get_stage_checkpoint(StageId::Headers)?; - - // TODO: this check is not good enough + // TODO: this check might not be good enough let is_pruned_node = self.get_highest_static_file_block(StaticFileSegment::Receipts).is_none() && provider.get_stage_checkpoint(StageId::Execution)?.is_some() && @@ -502,24 +508,27 @@ impl StaticFileProvider { // everything accordingly. let mut writer = self.latest_writer(segment)?; - // This will commit any inconsistencies found and update the reader index + // This will commit any inconsistencies found and update the reader index. If it + // happens, it means that the highest block of the segement has decreased. writer.commit()?; + // Only applies to block-based static files. (Headers) + // + // The updated `highest_block` may have decreased if we healed from a pruning + // interruption. let mut highest_block = self.get_highest_static_file_block(segment); if initial_highest_block != highest_block { - // The updated `highest_block` is a consistency heal as a result from some pruning - // behavior being interrupted. update_unwind_target(highest_block); } - // Make sure the last transaction matches the last block from its indices. - // Only applies to Transaction & Receipt segments + // Only applies to transaction-based static files. (Receipts & Transactions) + // + // Make sure the last transaction matches the last block from its indices, since a heal + // from a pruning interruption might have decreased the number of transactions without + // being able to update the last block of the static file segment. let highest_tx = self.get_highest_static_file_tx(segment); if let Some(highest_tx) = highest_tx { let mut last_block = highest_block.unwrap_or_default(); - // To be extra safe, we make sure that the last tx num matches the last block from - // its indices. If not, get it and create an unwind target, since we - // are missing data we should have. loop { if let Some(indices) = provider.block_body_indices(last_block)? { if indices.last_tx_num() <= highest_tx { @@ -536,11 +545,26 @@ impl StaticFileProvider { } } - // There should be no gap between static file & database elements. - if let Some(unwind) = if segment.is_headers() { - self.ensure_header_continuity(provider, highest_block)? - } else { - self.ensure_tx_continuity(segment, provider, highest_tx, highest_block)? + if let Some(unwind) = match segment { + StaticFileSegment::Headers => self.check_invariants::<_, tables::Headers>( + provider, + StageId::Headers, + highest_block, + highest_block, + )?, + StaticFileSegment::Transactions => self + .check_invariants::<_, tables::Transactions>( + provider, + StageId::Bodies, + highest_tx, + highest_block, + )?, + StaticFileSegment::Receipts => self.check_invariants::<_, tables::Receipts>( + provider, + StageId::Execution, + highest_tx, + highest_block, + )?, } { update_unwind_target(Some(unwind)); } @@ -549,63 +573,42 @@ impl StaticFileProvider { Ok(unwind_target.map(PipelineTarget::Unwind)) } - /// Ensures that there are no gaps between static file headers and database headers. - /// - /// If a gap is found (eg. static block 5 and first db block 7), return a block number to unwind - /// to as to restore missing data. - fn ensure_header_continuity( - &self, - provider: &DatabaseProvider, - highest_static: Option, - ) -> ProviderResult> { - let highest_static = highest_static.unwrap_or_default(); - let mut db_cursor = provider.tx_ref().cursor_read::()?; - - if let Some((db_header_num, _)) = db_cursor.first()? { - // If there is a gap between the header found in static file and - // database, then we have most likely lost static file data and need to load it again - if !(db_header_num <= highest_static || highest_static + 1 == db_header_num) { - return Ok(Some(highest_static)) - } - } - - Ok(None) - } - - /// Ensures that there are no gaps between static file transactions/receipts and database - /// transactions/receipts. - /// - /// If a gap is found (eg. static tx 5 and first db tx 7), return a block number to unwind to as - /// to restore missing data. - fn ensure_tx_continuity( + /// Check invariants for each corresponding table and static file segment. See + /// [Self::check_consistency] for more. + fn check_invariants>( &self, - segment: StaticFileSegment, provider: &DatabaseProvider, - highest_static_tx: Option, + stage_id: StageId, + static_file_entry: Option, highest_static_block: Option, ) -> ProviderResult> { - let highest_static_tx = highest_static_tx.unwrap_or_default(); - let first_db_tx_num = match segment { - StaticFileSegment::Headers => unreachable!(), - StaticFileSegment::Transactions => { - let mut cursor = provider.tx_ref().cursor_read::()?; - cursor.first()?.map(|(tx_num, _)| tx_num) - } - StaticFileSegment::Receipts => { - let mut cursor = provider.tx_ref().cursor_read::()?; - cursor.first()?.map(|(tx_num, _)| tx_num) + let static_file_entry = static_file_entry.unwrap_or_default(); + let highest_static_block = highest_static_block.unwrap_or_default(); + let mut db_cursor = provider.tx_ref().cursor_read::()?; + + if let Some((db_first_entry, _)) = db_cursor.first()? { + // If there is a gap between the entry found in static file and + // database, then we have most likely lost static file data and need to unwind so we can + // load it again + if !(db_first_entry <= static_file_entry || static_file_entry + 1 == db_first_entry) { + return Ok(Some(highest_static_block)) } - }; - if let Some(first_db_tx_num) = first_db_tx_num { - // If there is a gap more than one element between the header found in static file and - // database, then we have most likely lost static file data and need to load it again - if !(first_db_tx_num <= highest_static_tx || highest_static_tx + 1 == first_db_tx_num) { - // TODO is this unwrap or default safe - return Ok(Some(highest_static_block.unwrap_or_default())) + if let Some((db_last_entry, _)) = db_cursor.last()? { + if db_last_entry > static_file_entry { + return Ok(None) + } } } + // If static file entry is ahead of the database entries, then ensure the checkpoint block + // number matches. + let block_number = + provider.get_stage_checkpoint(stage_id)?.unwrap_or_default().block_number; + if block_number != highest_static_block { + return Ok(Some(block_number.min(highest_static_block))); + } + Ok(None) } From e06334a2ed56c4d4b74c3d4bc2eb5642e741b77c Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Sat, 11 May 2024 16:49:30 +0100 Subject: [PATCH 04/61] add NoopBlockExecutorProvider --- crates/evm/src/execute.rs | 55 +++++++++++++++++++ .../interfaces/src/blockchain_tree/error.rs | 3 +- crates/interfaces/src/executor.rs | 3 + 3 files changed, 60 insertions(+), 1 deletion(-) diff --git a/crates/evm/src/execute.rs b/crates/evm/src/execute.rs index e7ce09e7980..b72bf50089d 100644 --- a/crates/evm/src/execute.rs +++ b/crates/evm/src/execute.rs @@ -176,6 +176,61 @@ pub trait BlockExecutorProvider: Send + Sync + Clone + Unpin + 'static { DB: Database; } +/// A [BlockExecutorProvider] implementation that does nothing. +#[derive(Debug, Default, Clone)] +#[non_exhaustive] +pub struct NoopBlockExecutorProvider; + +impl BlockExecutorProvider for NoopBlockExecutorProvider { + type Executor> = Self; + + type BatchExecutor> = Self; + + fn executor(&self, _: DB) -> Self::Executor + where + DB: Database, + { + Self + } + + fn batch_executor(&self, _: DB, _: PruneModes) -> Self::BatchExecutor + where + DB: Database, + { + Self + } +} + +impl Executor for NoopBlockExecutorProvider { + type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>; + type Output = BlockExecutionOutput; + type Error = BlockExecutionError; + + fn execute(self, _: Self::Input<'_>) -> Result { + Err(BlockExecutionError::UnavailableForNoop) + } +} + +impl BatchExecutor for NoopBlockExecutorProvider { + type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>; + type Output = BatchBlockExecutionOutput; + type Error = BlockExecutionError; + + fn execute_one(&mut self, _: Self::Input<'_>) -> Result<(), Self::Error> { + Err(BlockExecutionError::UnavailableForNoop) + } + + fn finalize(self) -> Self::Output { + unreachable!() + } + + fn set_tip(&mut self, _: BlockNumber) {} + + fn size_hint(&self) -> Option { + None + } +} + #[cfg(test)] mod tests { use super::*; diff --git a/crates/interfaces/src/blockchain_tree/error.rs b/crates/interfaces/src/blockchain_tree/error.rs index a98d765014b..01e5c807842 100644 --- a/crates/interfaces/src/blockchain_tree/error.rs +++ b/crates/interfaces/src/blockchain_tree/error.rs @@ -307,7 +307,8 @@ impl InsertBlockErrorKind { BlockExecutionError::CanonicalRevert { .. } | BlockExecutionError::CanonicalCommit { .. } | BlockExecutionError::AppendChainDoesntConnect { .. } | - BlockExecutionError::UnavailableForTest => false, + BlockExecutionError::UnavailableForTest | + BlockExecutionError::UnavailableForNoop => false, BlockExecutionError::Other(_) => false, } } diff --git a/crates/interfaces/src/executor.rs b/crates/interfaces/src/executor.rs index 04b9832f092..bad87fb8498 100644 --- a/crates/interfaces/src/executor.rs +++ b/crates/interfaces/src/executor.rs @@ -115,6 +115,9 @@ pub enum BlockExecutionError { /// Note: this is not feature gated for convenience. #[error("execution unavailable for tests")] UnavailableForTest, + /// Only used for NoopBlockExecutorProvider + #[error("execution unavailable for noop")] + UnavailableForNoop, /// Error when fetching latest block state. #[error(transparent)] LatestBlock(#[from] ProviderError), From c820410528ccb7c82f0c4229c1719d186e4b866f Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Sat, 11 May 2024 17:11:59 +0100 Subject: [PATCH 05/61] check consistency on onLaunchContextWith::create_provider_factory --- crates/node/builder/src/launch/common.rs | 56 +++++++++++++++++++++--- crates/node/builder/src/launch/mod.rs | 2 +- 2 files changed, 51 insertions(+), 7 deletions(-) diff --git a/crates/node/builder/src/launch/common.rs b/crates/node/builder/src/launch/common.rs index 043b587b891..d8876d20d2d 100644 --- a/crates/node/builder/src/launch/common.rs +++ b/crates/node/builder/src/launch/common.rs @@ -4,11 +4,14 @@ use std::{cmp::max, sync::Arc, thread::available_parallelism}; use eyre::Context; use rayon::ThreadPoolBuilder; -use tokio::sync::mpsc::Receiver; +use tokio::sync::{mpsc::Receiver, oneshot}; use reth_auto_seal_consensus::MiningMode; +use reth_beacon_consensus::EthBeaconConsensus; use reth_config::{config::EtlConfig, PruneConfig}; use reth_db::{database::Database, database_metrics::DatabaseMetrics}; +use reth_downloaders::{bodies::noop::NoopBodiesDownloader, headers::noop::NoopHeaderDownloader}; +use reth_evm::execute::NoopBlockExecutorProvider; use reth_interfaces::p2p::headers::client::HeadersClient; use reth_node_core::{ cli::config::RethRpcConfig, @@ -17,9 +20,12 @@ use reth_node_core::{ node_config::NodeConfig, }; use reth_primitives::{BlockNumber, Chain, ChainSpec, Head, PruneModes, B256}; -use reth_provider::{providers::StaticFileProvider, ProviderFactory, StaticFileProviderFactory}; +use reth_provider::{ + providers::StaticFileProvider, HeaderSyncMode, ProviderFactory, StaticFileProviderFactory, +}; use reth_prune::PrunerBuilder; use reth_rpc::JwtSecret; +use reth_stages::{sets::DefaultStages, Pipeline}; use reth_static_file::StaticFileProducer; use reth_tasks::TaskExecutor; use reth_tracing::tracing::{error, info, warn}; @@ -312,10 +318,10 @@ impl LaunchContextWith> { impl LaunchContextWith> where - DB: Clone, + DB: Database + Clone + 'static, { /// Returns the [ProviderFactory] for the attached database. - pub fn create_provider_factory(&self) -> eyre::Result> { + pub async fn create_provider_factory(&self) -> eyre::Result> { let factory = ProviderFactory::new( self.right().clone(), self.chain_spec(), @@ -323,14 +329,52 @@ where )? .with_static_files_metrics(); + // Check for consistency between database and static files. If it fails, it unwinds to + // a previous block. + if let Some(unwind_target) = + factory.static_file_provider().check_consistency(&factory.provider()?)? + { + // Builds an unwind-only pipeline + let pipeline = Pipeline::builder() + .add_stages(DefaultStages::new( + factory.clone(), + HeaderSyncMode::Continuous, + Arc::new(EthBeaconConsensus::new(self.chain_spec())), + NoopHeaderDownloader::default(), + NoopBodiesDownloader::default(), + NoopBlockExecutorProvider::default(), + // TODO: add stage_conf when possible + Default::default(), + )) + .build( + factory.clone(), + StaticFileProducer::new( + factory.clone(), + factory.static_file_provider(), + self.prune_modes().unwrap_or_default(), + ), + ); + + // Unwinds to block + let (tx, rx) = oneshot::channel(); + self.task_executor().spawn_critical_blocking( + "pipeline task", + Box::pin(async move { + let (_, result) = pipeline.run_as_fut(Some(unwind_target)).await; + let _ = tx.send(result); + }), + ); + rx.await??; + } + Ok(factory) } /// Creates a new [ProviderFactory] and attaches it to the launch context. - pub fn with_provider_factory( + pub async fn with_provider_factory( self, ) -> eyre::Result>>> { - let factory = self.create_provider_factory()?; + let factory = self.create_provider_factory().await?; let ctx = LaunchContextWith { inner: self.inner, attachment: self.attachment.map_right(|_| factory), diff --git a/crates/node/builder/src/launch/mod.rs b/crates/node/builder/src/launch/mod.rs index 22143475818..cc060f39ebe 100644 --- a/crates/node/builder/src/launch/mod.rs +++ b/crates/node/builder/src/launch/mod.rs @@ -99,7 +99,7 @@ where // ensure certain settings take effect .with_adjusted_configs() // Create the provider factory - .with_provider_factory()? + .with_provider_factory().await? .inspect(|_| { info!(target: "reth::cli", "Database opened"); }) From 5737636741443fbee2e7613d37dc7dabeb8aa9ef Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Wed, 15 May 2024 09:20:28 +0100 Subject: [PATCH 06/61] pass config to unwind pipeline on create_provider_factory --- crates/node/builder/src/launch/common.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/node/builder/src/launch/common.rs b/crates/node/builder/src/launch/common.rs index d42c11eefa7..09961ab1bed 100644 --- a/crates/node/builder/src/launch/common.rs +++ b/crates/node/builder/src/launch/common.rs @@ -343,8 +343,8 @@ where NoopHeaderDownloader::default(), NoopBodiesDownloader::default(), NoopBlockExecutorProvider::default(), - // TODO: add stage_conf when possible - Default::default(), + self.toml_config().stages.clone(), + self.prune_modes().unwrap_or_default(), )) .build( factory.clone(), From 76c91f687e40408d2e13f500f0c7c68d9847dc56 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Wed, 15 May 2024 09:33:24 +0100 Subject: [PATCH 07/61] pass has_receipt_pruning to check_consistency --- crates/config/src/config.rs | 7 +++++++ crates/node/builder/src/launch/common.rs | 8 ++++++-- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/crates/config/src/config.rs b/crates/config/src/config.rs index aa8b7ee09ab..1b15d92f864 100644 --- a/crates/config/src/config.rs +++ b/crates/config/src/config.rs @@ -337,6 +337,13 @@ impl Default for PruneConfig { } } +impl PruneConfig { + /// Returns whether there is any kind of receipt pruning configuration. + pub fn has_receipts_pruning(&self) -> bool { + self.segments.receipts.is_some() || !self.segments.receipts_log_filter.is_empty() + } +} + /// Helper type to support older versions of Duration deserialization. fn deserialize_duration<'de, D>(deserializer: D) -> Result, D::Error> where diff --git a/crates/node/builder/src/launch/common.rs b/crates/node/builder/src/launch/common.rs index 09961ab1bed..7d82cfb487a 100644 --- a/crates/node/builder/src/launch/common.rs +++ b/crates/node/builder/src/launch/common.rs @@ -329,10 +329,14 @@ where )? .with_static_files_metrics(); + let has_receipt_pruning = + self.toml_config().prune.as_ref().map_or(false, |a| a.has_receipts_pruning()); + // Check for consistency between database and static files. If it fails, it unwinds to // a previous block. - if let Some(unwind_target) = - factory.static_file_provider().check_consistency(&factory.provider()?)? + if let Some(unwind_target) = factory + .static_file_provider() + .check_consistency(&factory.provider()?, has_receipt_pruning)? { // Builds an unwind-only pipeline let pipeline = Pipeline::builder() From 3aba5cca5ca4f242b34dc0130a623df34474b2a6 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Wed, 15 May 2024 09:47:21 +0100 Subject: [PATCH 08/61] missing arg --- .../provider/src/providers/static_file/manager.rs | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/crates/storage/provider/src/providers/static_file/manager.rs b/crates/storage/provider/src/providers/static_file/manager.rs index 6c8b6f0ece3..a0f0938da67 100644 --- a/crates/storage/provider/src/providers/static_file/manager.rs +++ b/crates/storage/provider/src/providers/static_file/manager.rs @@ -472,6 +472,7 @@ impl StaticFileProvider { pub fn check_consistency( &self, provider: &DatabaseProvider, + has_receipt_pruning: bool, ) -> ProviderResult> { let mut unwind_target: Option = None; let mut update_unwind_target = |new_target: Option| { @@ -483,14 +484,8 @@ impl StaticFileProvider { } }; - // TODO: this check might not be good enough - let is_pruned_node = - self.get_highest_static_file_block(StaticFileSegment::Receipts).is_none() && - provider.get_stage_checkpoint(StageId::Execution)?.is_some() && - provider.tx_ref().entries::()? > 0; - for segment in StaticFileSegment::iter() { - if is_pruned_node && segment.is_receipts() { + if has_receipt_pruning && segment.is_receipts() { // Pruned nodes (including full node) do not store receipts as static files. continue } From 0d4a1ae8350006c09fa41b2325c8209fc1ac7d44 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Wed, 15 May 2024 09:48:42 +0100 Subject: [PATCH 09/61] add docs --- crates/storage/provider/src/providers/static_file/manager.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/storage/provider/src/providers/static_file/manager.rs b/crates/storage/provider/src/providers/static_file/manager.rs index a0f0938da67..9fa7aa3e72b 100644 --- a/crates/storage/provider/src/providers/static_file/manager.rs +++ b/crates/storage/provider/src/providers/static_file/manager.rs @@ -459,7 +459,7 @@ impl StaticFileProvider { /// 1) When a static file fails to commit but the underlying data was changed. /// 2) When a static file was committed, but the required database transaction was not. /// - /// For 1) it can self-heal. + /// For 1) it can self-heal. Opening a writer to this segment will automatically do that. /// For 2) the invariants below are checked, and if broken, require a pipeline unwind to heal. /// /// For each static file segment: From 7aea4a8e0e0ea5b557ee724276fb2025788b3371 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Thu, 16 May 2024 16:56:23 +0100 Subject: [PATCH 10/61] docs and variable names --- crates/node/builder/src/launch/common.rs | 4 ++-- .../src/providers/static_file/manager.rs | 20 ++++++++++--------- 2 files changed, 13 insertions(+), 11 deletions(-) diff --git a/crates/node/builder/src/launch/common.rs b/crates/node/builder/src/launch/common.rs index 7d82cfb487a..9d0d707b54f 100644 --- a/crates/node/builder/src/launch/common.rs +++ b/crates/node/builder/src/launch/common.rs @@ -331,9 +331,9 @@ where let has_receipt_pruning = self.toml_config().prune.as_ref().map_or(false, |a| a.has_receipts_pruning()); - + // Check for consistency between database and static files. If it fails, it unwinds to - // a previous block. + // the first block that's consistent between database and static files. if let Some(unwind_target) = factory .static_file_provider() .check_consistency(&factory.provider()?, has_receipt_pruning)? diff --git a/crates/storage/provider/src/providers/static_file/manager.rs b/crates/storage/provider/src/providers/static_file/manager.rs index 9fa7aa3e72b..8511cc92084 100644 --- a/crates/storage/provider/src/providers/static_file/manager.rs +++ b/crates/storage/provider/src/providers/static_file/manager.rs @@ -574,23 +574,25 @@ impl StaticFileProvider { &self, provider: &DatabaseProvider, stage_id: StageId, - static_file_entry: Option, - highest_static_block: Option, + highest_static_file_entry: Option, + highest_static_file_block: Option, ) -> ProviderResult> { - let static_file_entry = static_file_entry.unwrap_or_default(); - let highest_static_block = highest_static_block.unwrap_or_default(); + let highest_static_file_entry = highest_static_file_entry.unwrap_or_default(); + let highest_static_file_block = highest_static_file_block.unwrap_or_default(); let mut db_cursor = provider.tx_ref().cursor_read::()?; if let Some((db_first_entry, _)) = db_cursor.first()? { // If there is a gap between the entry found in static file and // database, then we have most likely lost static file data and need to unwind so we can // load it again - if !(db_first_entry <= static_file_entry || static_file_entry + 1 == db_first_entry) { - return Ok(Some(highest_static_block)) + if !(db_first_entry <= highest_static_file_entry || + highest_static_file_entry + 1 == db_first_entry) + { + return Ok(Some(highest_static_file_block)) } if let Some((db_last_entry, _)) = db_cursor.last()? { - if db_last_entry > static_file_entry { + if db_last_entry > highest_static_file_entry { return Ok(None) } } @@ -600,8 +602,8 @@ impl StaticFileProvider { // number matches. let block_number = provider.get_stage_checkpoint(stage_id)?.unwrap_or_default().block_number; - if block_number != highest_static_block { - return Ok(Some(block_number.min(highest_static_block))); + if block_number != highest_static_file_block { + return Ok(Some(block_number.min(highest_static_file_block))); } Ok(None) From bd66959784e634273a16a46da67dfa6f957d5953 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Thu, 16 May 2024 16:57:26 +0100 Subject: [PATCH 11/61] more docs --- crates/node/builder/src/launch/common.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/crates/node/builder/src/launch/common.rs b/crates/node/builder/src/launch/common.rs index 9d0d707b54f..725ec524f44 100644 --- a/crates/node/builder/src/launch/common.rs +++ b/crates/node/builder/src/launch/common.rs @@ -361,6 +361,8 @@ where // Unwinds to block let (tx, rx) = oneshot::channel(); + + // Pipeline should be run as blocking and panic if it fails. self.task_executor().spawn_critical_blocking( "pipeline task", Box::pin(async move { From 6a8aab830c2e99505e42c442998cd0a5aa620955 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Thu, 16 May 2024 17:13:58 +0100 Subject: [PATCH 12/61] added ensure_file_consistency --- crates/node/builder/src/launch/common.rs | 2 +- crates/storage/nippy-jar/src/writer.rs | 2 +- .../src/providers/static_file/manager.rs | 24 ++++++++++--------- .../src/providers/static_file/writer.rs | 8 +++++++ 4 files changed, 23 insertions(+), 13 deletions(-) diff --git a/crates/node/builder/src/launch/common.rs b/crates/node/builder/src/launch/common.rs index 725ec524f44..db9f63cfa03 100644 --- a/crates/node/builder/src/launch/common.rs +++ b/crates/node/builder/src/launch/common.rs @@ -361,7 +361,7 @@ where // Unwinds to block let (tx, rx) = oneshot::channel(); - + // Pipeline should be run as blocking and panic if it fails. self.task_executor().spawn_critical_blocking( "pipeline task", diff --git a/crates/storage/nippy-jar/src/writer.rs b/crates/storage/nippy-jar/src/writer.rs index 6417e60076c..ba72df677a2 100644 --- a/crates/storage/nippy-jar/src/writer.rs +++ b/crates/storage/nippy-jar/src/writer.rs @@ -124,7 +124,7 @@ impl NippyJarWriter { /// /// This is based on the assumption that [`NippyJar`] configuration is **always** the last one /// to be updated when something is written, as by the `commit()` function shows. - fn check_consistency_and_heal(&mut self) -> Result<(), NippyJarError> { + pub fn check_consistency_and_heal(&mut self) -> Result<(), NippyJarError> { let reader = self.jar.open_data_reader()?; // When an offset size is smaller than the initial (8), we are dealing with immutable diff --git a/crates/storage/provider/src/providers/static_file/manager.rs b/crates/storage/provider/src/providers/static_file/manager.rs index 8511cc92084..cdd9ca91b42 100644 --- a/crates/storage/provider/src/providers/static_file/manager.rs +++ b/crates/storage/provider/src/providers/static_file/manager.rs @@ -492,20 +492,15 @@ impl StaticFileProvider { let initial_highest_block = self.get_highest_static_file_block(segment); - // By calling a writer, an inner consistency check will be triggered on - // [`NippyJarWriter::check_consistency_and_heal`] + // File consistency is broken if: // - // * If appending data was interrupted before a config commit, then data file will be + // * appending data was interrupted before a config commit, then data file will be // truncated according to the config. // - // * If pruning data was interrupted before a config commit, then we have deleted data - // that we are expected to still have. We need to check the Database and unwind - // everything accordingly. - let mut writer = self.latest_writer(segment)?; - - // This will commit any inconsistencies found and update the reader index. If it - // happens, it means that the highest block of the segement has decreased. - writer.commit()?; + // * pruning data was interrupted before a config commit, then we have deleted data that + // we are expected to still have. We need to check the Database and unwind everything + // accordingly. + self.ensure_file_consistency(segment)?; // Only applies to block-based static files. (Headers) // @@ -878,6 +873,9 @@ pub trait StaticFileWriter { /// Commits all changes of all [`StaticFileProviderRW`] of all [`StaticFileSegment`]. fn commit(&self) -> ProviderResult<()>; + + /// Checks consistency of the segment latest file and heals if necessary. + fn ensure_file_consistency(&self, segment: StaticFileSegment) -> ProviderResult<()>; } impl StaticFileWriter for StaticFileProvider { @@ -914,6 +912,10 @@ impl StaticFileWriter for StaticFileProvider { } Ok(()) } + + fn ensure_file_consistency(&self, segment: StaticFileSegment) -> ProviderResult<()> { + self.latest_writer(segment)?.ensure_file_consistency() + } } impl HeaderProvider for StaticFileProvider { diff --git a/crates/storage/provider/src/providers/static_file/writer.rs b/crates/storage/provider/src/providers/static_file/writer.rs index 3a0f2d03174..76e08afe96d 100644 --- a/crates/storage/provider/src/providers/static_file/writer.rs +++ b/crates/storage/provider/src/providers/static_file/writer.rs @@ -110,6 +110,14 @@ impl StaticFileProviderRW { Ok(result) } + /// Checks the consistency of the file and heals it if necessary. + pub fn ensure_file_consistency(&mut self) -> ProviderResult<()> { + let err = |err: NippyJarError| ProviderError::NippyJar(err.to_string()); + self.writer.check_consistency_and_heal().map_err(err)?; + self.writer.commit().map_err(err)?; + Ok(()) + } + /// Commits configuration changes to disk and updates the reader index with the new changes. pub fn commit(&mut self) -> ProviderResult<()> { let start = Instant::now(); From 1cb6009e342754eaa93e8cfa886efbbd939aa43e Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Mon, 20 May 2024 19:57:49 +0100 Subject: [PATCH 13/61] add first test_check_consistency --- crates/primitives/src/stage/mod.rs | 2 +- crates/stages/src/stages/mod.rs | 71 ++++++++++++++++++++++--- crates/stages/src/test_utils/test_db.rs | 36 +++++++++++++ 3 files changed, 102 insertions(+), 7 deletions(-) diff --git a/crates/primitives/src/stage/mod.rs b/crates/primitives/src/stage/mod.rs index 3c7c972bcf6..4b9f17d834a 100644 --- a/crates/primitives/src/stage/mod.rs +++ b/crates/primitives/src/stage/mod.rs @@ -12,7 +12,7 @@ pub use checkpoints::{ }; /// Direction and target block for pipeline operations. -#[derive(Debug, Clone, Copy)] +#[derive(Debug, Clone, Copy, PartialEq, Eq)] pub enum PipelineTarget { /// Target for forward synchronization, indicating a block hash to sync to. Sync(BlockHash), diff --git a/crates/stages/src/stages/mod.rs b/crates/stages/src/stages/mod.rs index 7bb88ff96e4..d79f83d9b5b 100644 --- a/crates/stages/src/stages/mod.rs +++ b/crates/stages/src/stages/mod.rs @@ -40,7 +40,7 @@ use utils::*; #[cfg(test)] mod tests { use super::*; - use crate::test_utils::TestStageDB; + use crate::test_utils::{StorageKind, TestStageDB}; use alloy_rlp::Decodable; use reth_db::{ cursor::DbCursorRO, @@ -52,14 +52,21 @@ mod tests { }; use reth_evm_ethereum::execute::EthExecutorProvider; use reth_exex::ExExManagerHandle; - use reth_interfaces::test_utils::generators::{self, random_block}; + use reth_interfaces::{ + provider::ProviderResult, + test_utils::generators::{self, random_block, random_block_range, random_receipt}, + }; use reth_primitives::{ - address, hex_literal::hex, keccak256, Account, Bytecode, ChainSpecBuilder, PruneMode, - PruneModes, SealedBlock, StaticFileSegment, U256, + address, + hex_literal::hex, + keccak256, + stage::{StageCheckpoint, StageId}, + Account, BlockNumber, Bytecode, ChainSpecBuilder, PruneMode, PruneModes, Receipt, + SealedBlock, StaticFileSegment, TxNumber, B256, U256, }; use reth_provider::{ - providers::StaticFileWriter, AccountExtReader, ProviderFactory, ReceiptProvider, - StorageReader, + providers::StaticFileWriter, AccountExtReader, DatabaseProviderFactory, ProviderFactory, + ReceiptProvider, StageCheckpointWriter, StaticFileProviderFactory, StorageReader, }; use reth_stages_api::{ExecInput, Stage}; use std::sync::Arc; @@ -239,4 +246,56 @@ mod tests { // The one account is the miner check_pruning(test_db.factory.clone(), prune.clone(), 0, 1, 0).await; } + + fn seed_data( + ) -> ProviderResult<(TestStageDB, Vec, Vec<(BlockNumber, Vec<(TxNumber, Receipt)>)>)> + { + let mut rng = generators::rng(); + let end_block = 100usize; + let take = 10; + let genesis_hash = B256::ZERO; + + // generate test data + let db = TestStageDB::default(); + let mut blocks = random_block_range(&mut rng, 0..=(end_block as u64), genesis_hash, 0..2); + let mut receipts = Vec::new(); + let tx_num = 0u64; + for block in &blocks { + let mut block_receipts = Vec::with_capacity(block.body.len()); + for transaction in &block.body { + block_receipts.push((tx_num, random_receipt(&mut rng, transaction, Some(0)))); + } + receipts.push((block.number, block_receipts)); + } + + // insert data respective to the first `end_block - take` blocks + let inserteable_blocks = blocks.drain(..end_block - take).collect::>(); + db.insert_blocks(inserteable_blocks.iter(), StorageKind::Static)?; + + let inserteable_receipts = receipts.drain(..end_block - take).collect::>(); + db.insert_receipts_by_block(inserteable_receipts, StorageKind::Static)?; + + // simulate pipeline by setting all checkpoints to inserted height. + let provider_rw = db.factory.provider_rw()?; + for stage in StageId::ALL { + provider_rw.save_stage_checkpoint( + stage, + StageCheckpoint::new((end_block - take - 1) as u64), + )?; + } + provider_rw.commit()?; + + Ok((db, blocks, receipts)) + } + + #[test] + fn test_check_consistency() { + let (db, _, _) = seed_data().unwrap(); + let db_provider = db.factory.database_provider_ro().unwrap(); + + assert_eq!( + db.factory.static_file_provider().check_consistency(&db_provider, false), + Ok(None) + ); + } } diff --git a/crates/stages/src/test_utils/test_db.rs b/crates/stages/src/test_utils/test_db.rs index 5fe65a73734..c67b2f3343d 100644 --- a/crates/stages/src/test_utils/test_db.rs +++ b/crates/stages/src/test_utils/test_db.rs @@ -314,6 +314,42 @@ impl TestStageDB { }) } + /// Insert collection of ([TxNumber], [Receipt]) organized by respective block numbers into the + /// corresponding table or static file segment. + pub fn insert_receipts_by_block( + &self, + receipts: I, + storage_kind: StorageKind, + ) -> ProviderResult<()> + where + I: IntoIterator, + J: IntoIterator, + { + match storage_kind { + StorageKind::Database(_) => self.commit(|tx| { + receipts.into_iter().try_for_each(|(_, receipts)| { + for (tx_num, receipt) in receipts { + tx.put::(tx_num, receipt)?; + } + Ok(()) + }) + }), + StorageKind::Static => { + let provider = self.factory.static_file_provider(); + let mut writer = provider.latest_writer(StaticFileSegment::Receipts)?; + let res = receipts.into_iter().try_for_each(|(block_num, receipts)| { + writer.increment_block(StaticFileSegment::Receipts, block_num)?; + for (tx_num, receipt) in receipts { + writer.append_receipt(tx_num, receipt)?; + } + Ok(()) + }); + writer.commit_without_sync_all()?; + res + } + } + } + pub fn insert_transaction_senders(&self, transaction_senders: I) -> ProviderResult<()> where I: IntoIterator, From d32cecb9d291580aa47c36cddff68bce5de56edf Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Mon, 20 May 2024 20:05:50 +0100 Subject: [PATCH 14/61] panic on unwind target to 0 --- crates/node/builder/src/launch/common.rs | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/crates/node/builder/src/launch/common.rs b/crates/node/builder/src/launch/common.rs index db9f63cfa03..70bfa3e36e7 100644 --- a/crates/node/builder/src/launch/common.rs +++ b/crates/node/builder/src/launch/common.rs @@ -19,7 +19,9 @@ use reth_node_core::{ init::{init_genesis, InitDatabaseError}, node_config::NodeConfig, }; -use reth_primitives::{BlockNumber, Chain, ChainSpec, Head, PruneModes, B256}; +use reth_primitives::{ + stage::PipelineTarget, BlockNumber, Chain, ChainSpec, Head, PruneModes, B256, +}; use reth_provider::{ providers::StaticFileProvider, HeaderSyncMode, ProviderFactory, StaticFileProviderFactory, }; @@ -338,6 +340,12 @@ where .static_file_provider() .check_consistency(&factory.provider()?, has_receipt_pruning)? { + // Highly unlikely to happen, and given its destructive nature, it's better to panic + // instead. + if let PipelineTarget::Unwind(0) = unwind_target { + panic!("A static file <> database inconsistency was found that would trigger an unwind to block 0.") + } + // Builds an unwind-only pipeline let pipeline = Pipeline::builder() .add_stages(DefaultStages::new( From 39a802ab822e9f8e0d08d4feffef7554288f781d Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Tue, 21 May 2024 13:38:44 +0100 Subject: [PATCH 15/61] add warning to check_consistency --- crates/storage/provider/src/providers/static_file/manager.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/crates/storage/provider/src/providers/static_file/manager.rs b/crates/storage/provider/src/providers/static_file/manager.rs index cdd9ca91b42..85980cc01de 100644 --- a/crates/storage/provider/src/providers/static_file/manager.rs +++ b/crates/storage/provider/src/providers/static_file/manager.rs @@ -469,6 +469,8 @@ impl StaticFileProvider { /// than the corresponding database table last entry. /// /// Returns a [`Option`] of [`PipelineTarget::Unwind`] if any healing is required. + /// + /// WARNING: No static file writer should be held before calling this function, otherwise it will deadlock. pub fn check_consistency( &self, provider: &DatabaseProvider, From 50cead02ea011ff2ab8241f8816b9c4014a99e19 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Tue, 21 May 2024 13:41:04 +0100 Subject: [PATCH 16/61] add read_only to check_consistency --- crates/node/builder/src/launch/common.rs | 9 +++++---- .../provider/src/providers/static_file/manager.rs | 15 ++++++++++----- 2 files changed, 15 insertions(+), 9 deletions(-) diff --git a/crates/node/builder/src/launch/common.rs b/crates/node/builder/src/launch/common.rs index 70bfa3e36e7..fd65396aa5b 100644 --- a/crates/node/builder/src/launch/common.rs +++ b/crates/node/builder/src/launch/common.rs @@ -336,10 +336,11 @@ where // Check for consistency between database and static files. If it fails, it unwinds to // the first block that's consistent between database and static files. - if let Some(unwind_target) = factory - .static_file_provider() - .check_consistency(&factory.provider()?, has_receipt_pruning)? - { + if let Some(unwind_target) = factory.static_file_provider().check_consistency( + &factory.provider()?, + has_receipt_pruning, + false, + )? { // Highly unlikely to happen, and given its destructive nature, it's better to panic // instead. if let PipelineTarget::Unwind(0) = unwind_target { diff --git a/crates/storage/provider/src/providers/static_file/manager.rs b/crates/storage/provider/src/providers/static_file/manager.rs index 85980cc01de..da17392a59d 100644 --- a/crates/storage/provider/src/providers/static_file/manager.rs +++ b/crates/storage/provider/src/providers/static_file/manager.rs @@ -459,8 +459,9 @@ impl StaticFileProvider { /// 1) When a static file fails to commit but the underlying data was changed. /// 2) When a static file was committed, but the required database transaction was not. /// - /// For 1) it can self-heal. Opening a writer to this segment will automatically do that. - /// For 2) the invariants below are checked, and if broken, require a pipeline unwind to heal. + /// For 1) it can self-heal. Opening a writer to this segment will automatically do that if + /// `read_only` is set to `false`. For 2) the invariants below are checked, and if broken, + /// require a pipeline unwind to heal. /// /// For each static file segment: /// * the corresponding database table should overlap or have continuity in their keys @@ -469,12 +470,14 @@ impl StaticFileProvider { /// than the corresponding database table last entry. /// /// Returns a [`Option`] of [`PipelineTarget::Unwind`] if any healing is required. - /// - /// WARNING: No static file writer should be held before calling this function, otherwise it will deadlock. + /// + /// WARNING: No static file writer should be held before calling this function, otherwise it + /// will deadlock. pub fn check_consistency( &self, provider: &DatabaseProvider, has_receipt_pruning: bool, + read_only: bool, ) -> ProviderResult> { let mut unwind_target: Option = None; let mut update_unwind_target = |new_target: Option| { @@ -502,7 +505,9 @@ impl StaticFileProvider { // * pruning data was interrupted before a config commit, then we have deleted data that // we are expected to still have. We need to check the Database and unwind everything // accordingly. - self.ensure_file_consistency(segment)?; + if !read_only { + self.ensure_file_consistency(segment)?; + } // Only applies to block-based static files. (Headers) // From aa865fa4dce9f904c827501b51be67669b95f4cf Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Tue, 21 May 2024 14:50:33 +0100 Subject: [PATCH 17/61] make sure SegmentHeader is updated after file heal --- .../src/providers/static_file/writer.rs | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/crates/storage/provider/src/providers/static_file/writer.rs b/crates/storage/provider/src/providers/static_file/writer.rs index 76e08afe96d..00f107f1c86 100644 --- a/crates/storage/provider/src/providers/static_file/writer.rs +++ b/crates/storage/provider/src/providers/static_file/writer.rs @@ -111,10 +111,24 @@ impl StaticFileProviderRW { } /// Checks the consistency of the file and heals it if necessary. + /// + /// Healing will update the end range on the [SegmentHeader]. However, for transaction based + /// segments, the block end range has to be found and healed externally. pub fn ensure_file_consistency(&mut self) -> ProviderResult<()> { let err = |err: NippyJarError| ProviderError::NippyJar(err.to_string()); + let initial_rows = self.writer.rows(); self.writer.check_consistency_and_heal().map_err(err)?; + + // If we have lost rows, we need to the [SegmentHeader] + let pruned_rows = initial_rows - self.writer.rows(); + if pruned_rows > 0 { + self.user_header_mut().prune(pruned_rows as u64); + } + self.writer.commit().map_err(err)?; + + // Updates the [SnapshotProvider] manager + self.update_index()?; Ok(()) } @@ -621,6 +635,11 @@ impl StaticFileProviderRW { provider.upgrade().map(StaticFileProvider).expect("StaticFileProvider is dropped") } + /// Helper function to access a mutable reference to [`SegmentHeader`]. + pub fn user_header_mut(&mut self) -> &mut SegmentHeader { + self.writer.user_header_mut() + } + #[cfg(any(test, feature = "test-utils"))] /// Helper function to override block range for testing. pub fn set_block_range(&mut self, block_range: std::ops::RangeInclusive) { @@ -632,6 +651,12 @@ impl StaticFileProviderRW { pub fn user_header(&self) -> &SegmentHeader { self.writer.user_header() } + + #[cfg(any(test, feature = "test-utils"))] + /// Helper function to override block range for testing. + pub fn inner(&mut self) -> &mut NippyJarWriter { + &mut self.writer + } } fn create_jar( From 67719b1ecbbfcf327225759972af8ff0c52affd2 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Tue, 21 May 2024 15:52:13 +0100 Subject: [PATCH 18/61] add early prune tests --- crates/stages/src/stages/mod.rs | 49 ++++++++++++++++++++++---- crates/storage/nippy-jar/src/writer.rs | 15 ++++++++ 2 files changed, 58 insertions(+), 6 deletions(-) diff --git a/crates/stages/src/stages/mod.rs b/crates/stages/src/stages/mod.rs index d79f83d9b5b..27eabba2f43 100644 --- a/crates/stages/src/stages/mod.rs +++ b/crates/stages/src/stages/mod.rs @@ -60,16 +60,17 @@ mod tests { address, hex_literal::hex, keccak256, - stage::{StageCheckpoint, StageId}, + stage::{PipelineTarget, StageCheckpoint, StageId}, Account, BlockNumber, Bytecode, ChainSpecBuilder, PruneMode, PruneModes, Receipt, SealedBlock, StaticFileSegment, TxNumber, B256, U256, }; use reth_provider::{ - providers::StaticFileWriter, AccountExtReader, DatabaseProviderFactory, ProviderFactory, + providers::{StaticFileProvider, StaticFileWriter}, + AccountExtReader, DatabaseProviderFactory, HeaderProvider, ProviderFactory, ReceiptProvider, StageCheckpointWriter, StaticFileProviderFactory, StorageReader, }; use reth_stages_api::{ExecInput, Stage}; - use std::sync::Arc; + use std::{io::Write, sync::Arc}; #[tokio::test] #[ignore] @@ -257,7 +258,7 @@ mod tests { // generate test data let db = TestStageDB::default(); - let mut blocks = random_block_range(&mut rng, 0..=(end_block as u64), genesis_hash, 0..2); + let mut blocks = random_block_range(&mut rng, 0..=(end_block as u64), genesis_hash, 2..3); let mut receipts = Vec::new(); let tx_num = 0u64; for block in &blocks { @@ -288,14 +289,50 @@ mod tests { Ok((db, blocks, receipts)) } + fn simulate_no_commit_prune( + num_rows: usize, + static_file_provider: &StaticFileProvider, + segment: StaticFileSegment, + ) { + let mut headers_writer = static_file_provider.latest_writer(segment).unwrap(); + let reader = headers_writer.inner().jar().open_data_reader().unwrap(); + let columns = headers_writer.inner().columns(); + let data_file = headers_writer.inner().data_file(); + let last_offset = reader.reverse_offset(num_rows * columns).unwrap(); + data_file.get_mut().set_len(last_offset).unwrap(); + data_file.flush().unwrap(); + data_file.get_ref().sync_all().unwrap(); + } + #[test] - fn test_check_consistency() { + fn test_consistency() { let (db, _, _) = seed_data().unwrap(); let db_provider = db.factory.database_provider_ro().unwrap(); assert_eq!( - db.factory.static_file_provider().check_consistency(&db_provider, false), + db.factory.static_file_provider().check_consistency(&db_provider, false, false), Ok(None) ); } + + #[test] + fn test_consistency_early_prune() { + let (db, _, _) = seed_data().unwrap(); + let db_provider = db.factory.database_provider_ro().unwrap(); + let static_file_provider = db.factory.static_file_provider(); + + // there are 2 to 3 transactions per block. however, if we lose one tx, we need to unwind to + // the previous block. + simulate_no_commit_prune(1, &static_file_provider, StaticFileSegment::Transactions); + assert_eq!( + static_file_provider.check_consistency(&db_provider, false, false), + Ok(Some(PipelineTarget::Unwind(88))) + ); + + simulate_no_commit_prune(3, &static_file_provider, StaticFileSegment::Headers); + assert_eq!( + static_file_provider.check_consistency(&db_provider, false, false), + Ok(Some(PipelineTarget::Unwind(86))) + ); + } } diff --git a/crates/storage/nippy-jar/src/writer.rs b/crates/storage/nippy-jar/src/writer.rs index ba72df677a2..0c2711052b7 100644 --- a/crates/storage/nippy-jar/src/writer.rs +++ b/crates/storage/nippy-jar/src/writer.rs @@ -479,4 +479,19 @@ impl NippyJarWriter { pub fn data_path(&self) -> &Path { self.jar.data_path() } + + #[cfg(any(test, feature = "test-utils"))] + pub fn data_file(&mut self) -> &mut BufWriter { + &mut self.data_file + } + + #[cfg(any(test, feature = "test-utils"))] + pub fn jar(&mut self) -> &NippyJar { + &self.jar + } + + #[cfg(any(test, feature = "test-utils"))] + pub fn columns(&self) -> usize { + self.jar.columns() + } } From f1502ddbfaa592110ad00f6c95d23c0d3efe4c78 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Tue, 21 May 2024 17:38:35 +0100 Subject: [PATCH 19/61] nippyjar writer handle partial pruned row --- crates/storage/nippy-jar/src/writer.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/crates/storage/nippy-jar/src/writer.rs b/crates/storage/nippy-jar/src/writer.rs index 0c2711052b7..d96e266ea2d 100644 --- a/crates/storage/nippy-jar/src/writer.rs +++ b/crates/storage/nippy-jar/src/writer.rs @@ -177,7 +177,8 @@ impl NippyJarWriter { // find the matching one. for index in 0..reader.offsets_count()? { let offset = reader.reverse_offset(index + 1)?; - if offset == data_file_len { + // It would only be equal if the previous row was fully pruned. + if offset <= data_file_len { let new_len = self .offsets_file .get_ref() From 98f1381e1e8d5fdf21daf03f9a94f1f1099f461e Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Tue, 21 May 2024 18:09:10 +0100 Subject: [PATCH 20/61] rename to test_consistency_no_commit_prune --- crates/stages/src/stages/mod.rs | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/crates/stages/src/stages/mod.rs b/crates/stages/src/stages/mod.rs index 27eabba2f43..d3e43713f4a 100644 --- a/crates/stages/src/stages/mod.rs +++ b/crates/stages/src/stages/mod.rs @@ -289,6 +289,7 @@ mod tests { Ok((db, blocks, receipts)) } + // Simulates a pruning job that was never committed. fn simulate_no_commit_prune( num_rows: usize, static_file_provider: &StaticFileProvider, @@ -316,22 +317,31 @@ mod tests { } #[test] - fn test_consistency_early_prune() { + fn test_consistency_no_commit_prune() { let (db, _, _) = seed_data().unwrap(); let db_provider = db.factory.database_provider_ro().unwrap(); let static_file_provider = db.factory.static_file_provider(); + let mut is_full_node = true; + // Full node does not use receipts, therefore doesn't check for consistency on receipts segment + simulate_no_commit_prune(1, &static_file_provider, StaticFileSegment::Receipts); + assert_eq!( + static_file_provider.check_consistency(&db_provider, is_full_node, false), + Ok(None) + ); + + is_full_node = false; // there are 2 to 3 transactions per block. however, if we lose one tx, we need to unwind to // the previous block. - simulate_no_commit_prune(1, &static_file_provider, StaticFileSegment::Transactions); + simulate_no_commit_prune(1, &static_file_provider, StaticFileSegment::Receipts); assert_eq!( - static_file_provider.check_consistency(&db_provider, false, false), + static_file_provider.check_consistency(&db_provider, is_full_node, false), Ok(Some(PipelineTarget::Unwind(88))) ); simulate_no_commit_prune(3, &static_file_provider, StaticFileSegment::Headers); assert_eq!( - static_file_provider.check_consistency(&db_provider, false, false), + static_file_provider.check_consistency(&db_provider, is_full_node, false), Ok(Some(PipelineTarget::Unwind(86))) ); } From a988e9ec8db6291b48af616a26735fa12abd3126 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Tue, 21 May 2024 19:43:55 +0100 Subject: [PATCH 21/61] fix usage of storage_kind on insert_blocks --- crates/stages/src/test_utils/test_db.rs | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/crates/stages/src/test_utils/test_db.rs b/crates/stages/src/test_utils/test_db.rs index c67b2f3343d..e8b57e03492 100644 --- a/crates/stages/src/test_utils/test_db.rs +++ b/crates/stages/src/test_utils/test_db.rs @@ -224,13 +224,17 @@ impl TestStageDB { let blocks = blocks.into_iter().collect::>(); { - let mut headers_writer = provider.latest_writer(StaticFileSegment::Headers)?; + let mut headers_writer = storage_kind + .is_static() + .then(|| provider.latest_writer(StaticFileSegment::Headers).unwrap()); blocks.iter().try_for_each(|block| { - Self::insert_header(Some(&mut headers_writer), &tx, &block.header, U256::ZERO) + Self::insert_header(headers_writer.as_mut(), &tx, &block.header, U256::ZERO) })?; - headers_writer.commit()?; + if let Some(mut writer) = headers_writer { + writer.commit()?; + } } { From 86946a6fd4b3e4c87826c336aad1cf290398017f Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Tue, 21 May 2024 20:07:48 +0100 Subject: [PATCH 22/61] add checkpoint and gap tests --- crates/node/builder/src/launch/common.rs | 2 +- crates/stages/src/stages/mod.rs | 167 +++++++++++++++++++---- 2 files changed, 141 insertions(+), 28 deletions(-) diff --git a/crates/node/builder/src/launch/common.rs b/crates/node/builder/src/launch/common.rs index fd65396aa5b..ca768ca9cbf 100644 --- a/crates/node/builder/src/launch/common.rs +++ b/crates/node/builder/src/launch/common.rs @@ -343,7 +343,7 @@ where )? { // Highly unlikely to happen, and given its destructive nature, it's better to panic // instead. - if let PipelineTarget::Unwind(0) = unwind_target { + if PipelineTarget::Unwind(0) == unwind_target { panic!("A static file <> database inconsistency was found that would trigger an unwind to block 0.") } diff --git a/crates/stages/src/stages/mod.rs b/crates/stages/src/stages/mod.rs index d3e43713f4a..0f374fb74fc 100644 --- a/crates/stages/src/stages/mod.rs +++ b/crates/stages/src/stages/mod.rs @@ -43,7 +43,7 @@ mod tests { use crate::test_utils::{StorageKind, TestStageDB}; use alloy_rlp::Decodable; use reth_db::{ - cursor::DbCursorRO, + cursor::{DbCursorRO, DbCursorRW}, mdbx::{cursor::Cursor, RW}, tables, test_utils::TempDatabase, @@ -61,13 +61,14 @@ mod tests { hex_literal::hex, keccak256, stage::{PipelineTarget, StageCheckpoint, StageId}, - Account, BlockNumber, Bytecode, ChainSpecBuilder, PruneMode, PruneModes, Receipt, - SealedBlock, StaticFileSegment, TxNumber, B256, U256, + Account, Bytecode, ChainSpecBuilder, PruneMode, PruneModes, + SealedBlock, StaticFileSegment, B256, U256, }; use reth_provider::{ providers::{StaticFileProvider, StaticFileWriter}, AccountExtReader, DatabaseProviderFactory, HeaderProvider, ProviderFactory, ReceiptProvider, StageCheckpointWriter, StaticFileProviderFactory, StorageReader, + TransactionsProvider, }; use reth_stages_api::{ExecInput, Stage}; use std::{io::Write, sync::Arc}; @@ -248,45 +249,38 @@ mod tests { check_pruning(test_db.factory.clone(), prune.clone(), 0, 1, 0).await; } - fn seed_data( - ) -> ProviderResult<(TestStageDB, Vec, Vec<(BlockNumber, Vec<(TxNumber, Receipt)>)>)> - { + /// It will generate `num_blocks`, push them to static files and set all stage checkpoints to + /// `num_blocks - 1`. + fn seed_data(num_blocks: usize) -> ProviderResult { + let db = TestStageDB::default(); let mut rng = generators::rng(); - let end_block = 100usize; - let take = 10; let genesis_hash = B256::ZERO; + let tip = (num_blocks - 1) as u64; + + let blocks = random_block_range(&mut rng, 0..=tip, genesis_hash, 2..3); + db.insert_blocks(blocks.iter(), StorageKind::Static)?; - // generate test data - let db = TestStageDB::default(); - let mut blocks = random_block_range(&mut rng, 0..=(end_block as u64), genesis_hash, 2..3); let mut receipts = Vec::new(); - let tx_num = 0u64; + let mut tx_num = 0u64; for block in &blocks { let mut block_receipts = Vec::with_capacity(block.body.len()); for transaction in &block.body { block_receipts.push((tx_num, random_receipt(&mut rng, transaction, Some(0)))); + tx_num += 1; } receipts.push((block.number, block_receipts)); } - - // insert data respective to the first `end_block - take` blocks - let inserteable_blocks = blocks.drain(..end_block - take).collect::>(); - db.insert_blocks(inserteable_blocks.iter(), StorageKind::Static)?; - - let inserteable_receipts = receipts.drain(..end_block - take).collect::>(); - db.insert_receipts_by_block(inserteable_receipts, StorageKind::Static)?; + db.insert_receipts_by_block(receipts, StorageKind::Static)?; // simulate pipeline by setting all checkpoints to inserted height. let provider_rw = db.factory.provider_rw()?; for stage in StageId::ALL { - provider_rw.save_stage_checkpoint( - stage, - StageCheckpoint::new((end_block - take - 1) as u64), - )?; + provider_rw + .save_stage_checkpoint(stage, StageCheckpoint::new(tip))?; } provider_rw.commit()?; - Ok((db, blocks, receipts)) + Ok(db) } // Simulates a pruning job that was never committed. @@ -307,7 +301,7 @@ mod tests { #[test] fn test_consistency() { - let (db, _, _) = seed_data().unwrap(); + let db = seed_data(90).unwrap(); let db_provider = db.factory.database_provider_ro().unwrap(); assert_eq!( @@ -318,12 +312,13 @@ mod tests { #[test] fn test_consistency_no_commit_prune() { - let (db, _, _) = seed_data().unwrap(); + let db = seed_data(90).unwrap(); let db_provider = db.factory.database_provider_ro().unwrap(); let static_file_provider = db.factory.static_file_provider(); let mut is_full_node = true; - // Full node does not use receipts, therefore doesn't check for consistency on receipts segment + // Full node does not use receipts, therefore doesn't check for consistency on receipts + // segment simulate_no_commit_prune(1, &static_file_provider, StaticFileSegment::Receipts); assert_eq!( static_file_provider.check_consistency(&db_provider, is_full_node, false), @@ -345,4 +340,122 @@ mod tests { Ok(Some(PipelineTarget::Unwind(86))) ); } + + #[test] + fn test_consistency_checkpoints() { + let db = seed_data(90).unwrap(); + let static_file_provider = db.factory.static_file_provider(); + + let provider_rw = db.factory.provider_rw().unwrap(); + provider_rw.save_stage_checkpoint(StageId::Headers, StageCheckpoint::new(91)).unwrap(); + provider_rw.commit().unwrap(); + + assert_eq!( + static_file_provider.check_consistency( + &db.factory.database_provider_ro().unwrap(), + false, + false + ), + Ok(Some(PipelineTarget::Unwind(89))) + ); + + let provider_rw = db.factory.provider_rw().unwrap(); + provider_rw.save_stage_checkpoint(StageId::Bodies, StageCheckpoint::new(87)).unwrap(); + provider_rw.commit().unwrap(); + + assert_eq!( + static_file_provider.check_consistency( + &db.factory.database_provider_ro().unwrap(), + false, + false + ), + Ok(Some(PipelineTarget::Unwind(87))) + ); + + let provider_rw = db.factory.provider_rw().unwrap(); + provider_rw.save_stage_checkpoint(StageId::Execution, StageCheckpoint::new(50)).unwrap(); + provider_rw.commit().unwrap(); + + assert_eq!( + static_file_provider.check_consistency( + &db.factory.database_provider_ro().unwrap(), + false, + false + ), + Ok(Some(PipelineTarget::Unwind(50))) + ); + } + + #[test] + fn test_consistency_headers_gap() { + let db = seed_data(90).unwrap(); + let static_file_provider = db.factory.static_file_provider(); + + // Creates a gap of one block static_file(89) db_(91) + { + let current = static_file_provider + .get_highest_static_file_block(StaticFileSegment::Headers) + .unwrap(); + let provider_rw = db.factory.provider_rw().unwrap(); + let mut cursor = provider_rw.tx_ref().cursor_write::().unwrap(); + cursor.append(current + 2, Default::default()).unwrap(); + provider_rw.commit().unwrap(); + } + + let db_provider = db.factory.database_provider_ro().unwrap(); + assert!(db_provider.header_by_number(90).unwrap().is_none()); + + assert_eq!( + static_file_provider.check_consistency(&db_provider, false, false), + Ok(Some(PipelineTarget::Unwind(89))) + ); + } + + #[test] + fn test_consistency_tx_gap() { + let db = seed_data(90).unwrap(); + let static_file_provider = db.factory.static_file_provider(); + let current = static_file_provider + .get_highest_static_file_tx(StaticFileSegment::Transactions) + .unwrap(); + + // Creates a gap of one transaction: static_file db + { + let provider_rw = db.factory.provider_rw().unwrap(); + let mut cursor = provider_rw.tx_ref().cursor_write::().unwrap(); + cursor.append(current + 2, Default::default()).unwrap(); + provider_rw.commit().unwrap(); + } + + let db_provider = db.factory.database_provider_ro().unwrap(); + assert!(db_provider.transaction_by_id(current + 1).unwrap().is_none()); + + assert_eq!( + static_file_provider.check_consistency(&db_provider, false, false), + Ok(Some(PipelineTarget::Unwind(89))) + ); + } + + #[test] + fn test_consistency_receipt_gap() { + let db = seed_data(90).unwrap(); + let static_file_provider = db.factory.static_file_provider(); + let current = + static_file_provider.get_highest_static_file_tx(StaticFileSegment::Receipts).unwrap(); + + // Creates a gap of one receipt: static_file db + { + let provider_rw = db.factory.provider_rw().unwrap(); + let mut cursor = provider_rw.tx_ref().cursor_write::().unwrap(); + cursor.append(current + 2, Default::default()).unwrap(); + provider_rw.commit().unwrap(); + } + let db_provider = db.factory.database_provider_ro().unwrap(); + assert!(db_provider.receipt(current + 1).unwrap().is_none()); + + assert_eq!( + static_file_provider.check_consistency(&db_provider, false, false), + Ok(Some(PipelineTarget::Unwind(89))) + ); + } } From 016deaaa3790e036c753e3147b753848f8b0ae29 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Wed, 22 May 2024 13:30:04 +0100 Subject: [PATCH 23/61] smol refactor on test functions --- crates/stages/src/stages/mod.rs | 242 ++++++++++++++++---------------- 1 file changed, 124 insertions(+), 118 deletions(-) diff --git a/crates/stages/src/stages/mod.rs b/crates/stages/src/stages/mod.rs index 0f374fb74fc..8f2ba9041c7 100644 --- a/crates/stages/src/stages/mod.rs +++ b/crates/stages/src/stages/mod.rs @@ -45,6 +45,7 @@ mod tests { use reth_db::{ cursor::{DbCursorRO, DbCursorRW}, mdbx::{cursor::Cursor, RW}, + table::Table, tables, test_utils::TempDatabase, transaction::{DbTx, DbTxMut}, @@ -61,8 +62,8 @@ mod tests { hex_literal::hex, keccak256, stage::{PipelineTarget, StageCheckpoint, StageId}, - Account, Bytecode, ChainSpecBuilder, PruneMode, PruneModes, - SealedBlock, StaticFileSegment, B256, U256, + Account, BlockNumber, Bytecode, ChainSpecBuilder, PruneMode, PruneModes, SealedBlock, + StaticFileSegment, B256, U256, }; use reth_provider::{ providers::{StaticFileProvider, StaticFileWriter}, @@ -275,28 +276,90 @@ mod tests { // simulate pipeline by setting all checkpoints to inserted height. let provider_rw = db.factory.provider_rw()?; for stage in StageId::ALL { - provider_rw - .save_stage_checkpoint(stage, StageCheckpoint::new(tip))?; + provider_rw.save_stage_checkpoint(stage, StageCheckpoint::new(tip))?; } provider_rw.commit()?; Ok(db) } - // Simulates a pruning job that was never committed. - fn simulate_no_commit_prune( - num_rows: usize, - static_file_provider: &StaticFileProvider, + /// Simulates a pruning job that was never committed and compare the check consistency result against the expected one. + fn simulate_no_commit_prune_and_check( + db: &TestStageDB, + prune_count: usize, segment: StaticFileSegment, + is_full_node: bool, + expected_unwind: Option, ) { - let mut headers_writer = static_file_provider.latest_writer(segment).unwrap(); - let reader = headers_writer.inner().jar().open_data_reader().unwrap(); - let columns = headers_writer.inner().columns(); - let data_file = headers_writer.inner().data_file(); - let last_offset = reader.reverse_offset(num_rows * columns).unwrap(); - data_file.get_mut().set_len(last_offset).unwrap(); - data_file.flush().unwrap(); - data_file.get_ref().sync_all().unwrap(); + let static_file_provider = db.factory.static_file_provider(); + + // Simulate pruning by removing `prune_count` rows from the data file without updating its + // offset list and configuration. + { + let mut headers_writer = static_file_provider.latest_writer(segment).unwrap(); + let reader = headers_writer.inner().jar().open_data_reader().unwrap(); + let columns = headers_writer.inner().columns(); + let data_file = headers_writer.inner().data_file(); + let last_offset = reader.reverse_offset(prune_count * columns).unwrap(); + data_file.get_mut().set_len(last_offset).unwrap(); + data_file.flush().unwrap(); + data_file.get_ref().sync_all().unwrap(); + } + + let db_provider = db.factory.database_provider_ro().unwrap(); + let consistency_check = + static_file_provider.check_consistency(&db_provider, is_full_node, false); + + assert_eq!(consistency_check, Ok(expected_unwind)); + } + + /// Saves a checkpoint with `checkpoint_block_number` and compare the check consistency result against the expected one. + fn save_checkpoint_and_check( + db: &TestStageDB, // replace DbType with your actual database type + stage_id: StageId, + checkpoint_block_number: BlockNumber, + expected_unwind: PipelineTarget, + ) { + let provider_rw = db.factory.provider_rw().unwrap(); + provider_rw + .save_stage_checkpoint(stage_id, StageCheckpoint::new(checkpoint_block_number)) + .unwrap(); + provider_rw.commit().unwrap(); + + assert_eq!( + db.factory.static_file_provider().check_consistency( + &db.factory.database_provider_ro().unwrap(), + false, + false + ), + Ok(Some(expected_unwind)) + ); + } + + /// Inserts a dummy value at key and compare the check consistency result against the expected one. + fn update_db_and_check>( + db: &TestStageDB, + key: u64, + expect_unwind: Option, + ) where + ::Value: Default, + { + { + let provider_rw = db.factory.provider_rw().unwrap(); + let mut cursor = provider_rw.tx_ref().cursor_write::().unwrap(); + cursor.insert(key, Default::default()).unwrap(); + provider_rw.commit().unwrap(); + } + + let db_provider = db.factory.database_provider_ro().unwrap(); + let consistency_check = + db.factory.static_file_provider().check_consistency(&db_provider, false, false); + + if let Some(target) = expect_unwind { + assert_eq!(consistency_check, Ok(Some(target))); + } else { + assert_eq!(consistency_check, Ok(None)); + } } #[test] @@ -313,149 +376,92 @@ mod tests { #[test] fn test_consistency_no_commit_prune() { let db = seed_data(90).unwrap(); - let db_provider = db.factory.database_provider_ro().unwrap(); - let static_file_provider = db.factory.static_file_provider(); - let mut is_full_node = true; + let full_node = true; + let archive_node = !full_node; // Full node does not use receipts, therefore doesn't check for consistency on receipts // segment - simulate_no_commit_prune(1, &static_file_provider, StaticFileSegment::Receipts); - assert_eq!( - static_file_provider.check_consistency(&db_provider, is_full_node, false), - Ok(None) - ); + simulate_no_commit_prune_and_check(&db, 1, StaticFileSegment::Receipts, full_node, None); - is_full_node = false; // there are 2 to 3 transactions per block. however, if we lose one tx, we need to unwind to // the previous block. - simulate_no_commit_prune(1, &static_file_provider, StaticFileSegment::Receipts); - assert_eq!( - static_file_provider.check_consistency(&db_provider, is_full_node, false), - Ok(Some(PipelineTarget::Unwind(88))) + simulate_no_commit_prune_and_check( + &db, + 1, + StaticFileSegment::Receipts, + archive_node, + Some(PipelineTarget::Unwind(88)), ); - simulate_no_commit_prune(3, &static_file_provider, StaticFileSegment::Headers); - assert_eq!( - static_file_provider.check_consistency(&db_provider, is_full_node, false), - Ok(Some(PipelineTarget::Unwind(86))) + simulate_no_commit_prune_and_check( + &db, + 3, + StaticFileSegment::Headers, + archive_node, + Some(PipelineTarget::Unwind(86)), ); } #[test] fn test_consistency_checkpoints() { let db = seed_data(90).unwrap(); - let static_file_provider = db.factory.static_file_provider(); - - let provider_rw = db.factory.provider_rw().unwrap(); - provider_rw.save_stage_checkpoint(StageId::Headers, StageCheckpoint::new(91)).unwrap(); - provider_rw.commit().unwrap(); - - assert_eq!( - static_file_provider.check_consistency( - &db.factory.database_provider_ro().unwrap(), - false, - false - ), - Ok(Some(PipelineTarget::Unwind(89))) - ); - let provider_rw = db.factory.provider_rw().unwrap(); - provider_rw.save_stage_checkpoint(StageId::Bodies, StageCheckpoint::new(87)).unwrap(); - provider_rw.commit().unwrap(); + save_checkpoint_and_check(&db, StageId::Headers, 91, PipelineTarget::Unwind(89)); - assert_eq!( - static_file_provider.check_consistency( - &db.factory.database_provider_ro().unwrap(), - false, - false - ), - Ok(Some(PipelineTarget::Unwind(87))) - ); - - let provider_rw = db.factory.provider_rw().unwrap(); - provider_rw.save_stage_checkpoint(StageId::Execution, StageCheckpoint::new(50)).unwrap(); - provider_rw.commit().unwrap(); + save_checkpoint_and_check(&db, StageId::Bodies, 87, PipelineTarget::Unwind(87)); - assert_eq!( - static_file_provider.check_consistency( - &db.factory.database_provider_ro().unwrap(), - false, - false - ), - Ok(Some(PipelineTarget::Unwind(50))) - ); + save_checkpoint_and_check(&db, StageId::Execution, 50, PipelineTarget::Unwind(50)); } #[test] fn test_consistency_headers_gap() { let db = seed_data(90).unwrap(); - let static_file_provider = db.factory.static_file_provider(); - - // Creates a gap of one block static_file(89) db_(91) - { - let current = static_file_provider - .get_highest_static_file_block(StaticFileSegment::Headers) - .unwrap(); - let provider_rw = db.factory.provider_rw().unwrap(); - let mut cursor = provider_rw.tx_ref().cursor_write::().unwrap(); - cursor.append(current + 2, Default::default()).unwrap(); - provider_rw.commit().unwrap(); - } + let current = db + .factory + .static_file_provider() + .get_highest_static_file_block(StaticFileSegment::Headers) + .unwrap(); - let db_provider = db.factory.database_provider_ro().unwrap(); - assert!(db_provider.header_by_number(90).unwrap().is_none()); + // Creates a gap of one header: static_file db + update_db_and_check::(&db, current + 2, Some(PipelineTarget::Unwind(89))); - assert_eq!( - static_file_provider.check_consistency(&db_provider, false, false), - Ok(Some(PipelineTarget::Unwind(89))) - ); + // Fill the gap, and ensure no unwind is necessary. + update_db_and_check::(&db, current + 1, None); } #[test] fn test_consistency_tx_gap() { let db = seed_data(90).unwrap(); - let static_file_provider = db.factory.static_file_provider(); - let current = static_file_provider + let current = db + .factory + .static_file_provider() .get_highest_static_file_tx(StaticFileSegment::Transactions) .unwrap(); // Creates a gap of one transaction: static_file db - { - let provider_rw = db.factory.provider_rw().unwrap(); - let mut cursor = provider_rw.tx_ref().cursor_write::().unwrap(); - cursor.append(current + 2, Default::default()).unwrap(); - provider_rw.commit().unwrap(); - } - - let db_provider = db.factory.database_provider_ro().unwrap(); - assert!(db_provider.transaction_by_id(current + 1).unwrap().is_none()); - - assert_eq!( - static_file_provider.check_consistency(&db_provider, false, false), - Ok(Some(PipelineTarget::Unwind(89))) + update_db_and_check::( + &db, + current + 2, + Some(PipelineTarget::Unwind(89)), ); + + // Fill the gap, and ensure no unwind is necessary. + update_db_and_check::(&db, current + 1, None); } #[test] fn test_consistency_receipt_gap() { let db = seed_data(90).unwrap(); - let static_file_provider = db.factory.static_file_provider(); - let current = - static_file_provider.get_highest_static_file_tx(StaticFileSegment::Receipts).unwrap(); + let current = db + .factory + .static_file_provider() + .get_highest_static_file_tx(StaticFileSegment::Receipts) + .unwrap(); // Creates a gap of one receipt: static_file db - { - let provider_rw = db.factory.provider_rw().unwrap(); - let mut cursor = provider_rw.tx_ref().cursor_write::().unwrap(); - cursor.append(current + 2, Default::default()).unwrap(); - provider_rw.commit().unwrap(); - } - let db_provider = db.factory.database_provider_ro().unwrap(); - assert!(db_provider.receipt(current + 1).unwrap().is_none()); + update_db_and_check::(&db, current + 2, Some(PipelineTarget::Unwind(89))); - assert_eq!( - static_file_provider.check_consistency(&db_provider, false, false), - Ok(Some(PipelineTarget::Unwind(89))) - ); + // Fill the gap, and ensure no unwind is necessary. + update_db_and_check::(&db, current + 1, None); } } From a0e3976171702aa7b8e4ccbd144991110babbebf Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Wed, 22 May 2024 13:37:47 +0100 Subject: [PATCH 24/61] clippy --- crates/stages/src/stages/mod.rs | 15 ++++++++------- crates/storage/nippy-jar/src/writer.rs | 7 +------ 2 files changed, 9 insertions(+), 13 deletions(-) diff --git a/crates/stages/src/stages/mod.rs b/crates/stages/src/stages/mod.rs index 8f2ba9041c7..c7e986e4c4e 100644 --- a/crates/stages/src/stages/mod.rs +++ b/crates/stages/src/stages/mod.rs @@ -66,10 +66,8 @@ mod tests { StaticFileSegment, B256, U256, }; use reth_provider::{ - providers::{StaticFileProvider, StaticFileWriter}, - AccountExtReader, DatabaseProviderFactory, HeaderProvider, ProviderFactory, + providers::StaticFileWriter, AccountExtReader, DatabaseProviderFactory, ProviderFactory, ReceiptProvider, StageCheckpointWriter, StaticFileProviderFactory, StorageReader, - TransactionsProvider, }; use reth_stages_api::{ExecInput, Stage}; use std::{io::Write, sync::Arc}; @@ -283,7 +281,8 @@ mod tests { Ok(db) } - /// Simulates a pruning job that was never committed and compare the check consistency result against the expected one. + /// Simulates a pruning job that was never committed and compare the check consistency result + /// against the expected one. fn simulate_no_commit_prune_and_check( db: &TestStageDB, prune_count: usize, @@ -298,7 +297,7 @@ mod tests { { let mut headers_writer = static_file_provider.latest_writer(segment).unwrap(); let reader = headers_writer.inner().jar().open_data_reader().unwrap(); - let columns = headers_writer.inner().columns(); + let columns = headers_writer.inner().jar().columns(); let data_file = headers_writer.inner().data_file(); let last_offset = reader.reverse_offset(prune_count * columns).unwrap(); data_file.get_mut().set_len(last_offset).unwrap(); @@ -313,7 +312,8 @@ mod tests { assert_eq!(consistency_check, Ok(expected_unwind)); } - /// Saves a checkpoint with `checkpoint_block_number` and compare the check consistency result against the expected one. + /// Saves a checkpoint with `checkpoint_block_number` and compare the check consistency result + /// against the expected one. fn save_checkpoint_and_check( db: &TestStageDB, // replace DbType with your actual database type stage_id: StageId, @@ -336,7 +336,8 @@ mod tests { ); } - /// Inserts a dummy value at key and compare the check consistency result against the expected one. + /// Inserts a dummy value at key and compare the check consistency result against the expected + /// one. fn update_db_and_check>( db: &TestStageDB, key: u64, diff --git a/crates/storage/nippy-jar/src/writer.rs b/crates/storage/nippy-jar/src/writer.rs index d96e266ea2d..64b07ddc77d 100644 --- a/crates/storage/nippy-jar/src/writer.rs +++ b/crates/storage/nippy-jar/src/writer.rs @@ -487,12 +487,7 @@ impl NippyJarWriter { } #[cfg(any(test, feature = "test-utils"))] - pub fn jar(&mut self) -> &NippyJar { + pub fn jar(&self) -> &NippyJar { &self.jar } - - #[cfg(any(test, feature = "test-utils"))] - pub fn columns(&self) -> usize { - self.jar.columns() - } } From ea0ef7f276084a54bebb6233a3347f38ff816874 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Wed, 22 May 2024 13:43:40 +0100 Subject: [PATCH 25/61] nit --- crates/stages/src/stages/mod.rs | 40 ++++++++++++++++----------------- 1 file changed, 20 insertions(+), 20 deletions(-) diff --git a/crates/stages/src/stages/mod.rs b/crates/stages/src/stages/mod.rs index c7e986e4c4e..cd8c1cd2600 100644 --- a/crates/stages/src/stages/mod.rs +++ b/crates/stages/src/stages/mod.rs @@ -305,11 +305,14 @@ mod tests { data_file.get_ref().sync_all().unwrap(); } - let db_provider = db.factory.database_provider_ro().unwrap(); - let consistency_check = - static_file_provider.check_consistency(&db_provider, is_full_node, false); - - assert_eq!(consistency_check, Ok(expected_unwind)); + assert_eq!( + static_file_provider.check_consistency( + &db.factory.database_provider_ro().unwrap(), + is_full_node, + false + ), + Ok(expect_unwind) + ); } /// Saves a checkpoint with `checkpoint_block_number` and compare the check consistency result @@ -345,22 +348,19 @@ mod tests { ) where ::Value: Default, { - { - let provider_rw = db.factory.provider_rw().unwrap(); - let mut cursor = provider_rw.tx_ref().cursor_write::().unwrap(); - cursor.insert(key, Default::default()).unwrap(); - provider_rw.commit().unwrap(); - } - - let db_provider = db.factory.database_provider_ro().unwrap(); - let consistency_check = - db.factory.static_file_provider().check_consistency(&db_provider, false, false); + let provider_rw = db.factory.provider_rw().unwrap(); + let mut cursor = provider_rw.tx_ref().cursor_write::().unwrap(); + cursor.insert(key, Default::default()).unwrap(); + provider_rw.commit().unwrap(); - if let Some(target) = expect_unwind { - assert_eq!(consistency_check, Ok(Some(target))); - } else { - assert_eq!(consistency_check, Ok(None)); - } + assert_eq!( + db.factory.static_file_provider().check_consistency( + &db.factory.database_provider_ro().unwrap(), + false, + false + ), + Ok(expect_unwind) + ); } #[test] From d9fb35307312703eb759210a0368dfa8e7d5e6a9 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Wed, 22 May 2024 13:54:27 +0100 Subject: [PATCH 26/61] add logs to node launch consistency check --- crates/node/builder/src/launch/common.rs | 4 ++++ crates/primitives/src/stage/mod.rs | 11 +++++++++++ 2 files changed, 15 insertions(+) diff --git a/crates/node/builder/src/launch/common.rs b/crates/node/builder/src/launch/common.rs index ca768ca9cbf..f2c7bbe6cc8 100644 --- a/crates/node/builder/src/launch/common.rs +++ b/crates/node/builder/src/launch/common.rs @@ -334,6 +334,8 @@ where let has_receipt_pruning = self.toml_config().prune.as_ref().map_or(false, |a| a.has_receipts_pruning()); + info!(target: "reth::cli", "Verifying storage consistency."); + // Check for consistency between database and static files. If it fails, it unwinds to // the first block that's consistent between database and static files. if let Some(unwind_target) = factory.static_file_provider().check_consistency( @@ -347,6 +349,8 @@ where panic!("A static file <> database inconsistency was found that would trigger an unwind to block 0.") } + info!(target: "reth::cli", unwind_target = %unwind_target, "Executing an unwind after a failed storage consistency check."); + // Builds an unwind-only pipeline let pipeline = Pipeline::builder() .add_stages(DefaultStages::new( diff --git a/crates/primitives/src/stage/mod.rs b/crates/primitives/src/stage/mod.rs index 4b9f17d834a..2b6d90b733c 100644 --- a/crates/primitives/src/stage/mod.rs +++ b/crates/primitives/src/stage/mod.rs @@ -53,3 +53,14 @@ impl From for PipelineTarget { Self::Sync(hash) } } + +impl std::fmt::Display for PipelineTarget { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + PipelineTarget::Sync(block) => { + write!(f, "Sync({block})") + } + PipelineTarget::Unwind(block) => write!(f, "Unwind({block})"), + } + } +} \ No newline at end of file From dfd2d8c6de17999cb09ec62b7127e40df50508dc Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Wed, 22 May 2024 13:55:40 +0100 Subject: [PATCH 27/61] fmt --- crates/primitives/src/stage/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/primitives/src/stage/mod.rs b/crates/primitives/src/stage/mod.rs index 2b6d90b733c..367d3abe31e 100644 --- a/crates/primitives/src/stage/mod.rs +++ b/crates/primitives/src/stage/mod.rs @@ -63,4 +63,4 @@ impl std::fmt::Display for PipelineTarget { PipelineTarget::Unwind(block) => write!(f, "Unwind({block})"), } } -} \ No newline at end of file +} From 9266ca478b36c85754f1dced83e8d2fa6b871c90 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Wed, 22 May 2024 14:04:50 +0100 Subject: [PATCH 28/61] nit --- crates/stages/src/stages/mod.rs | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/crates/stages/src/stages/mod.rs b/crates/stages/src/stages/mod.rs index cd8c1cd2600..e631fe49a70 100644 --- a/crates/stages/src/stages/mod.rs +++ b/crates/stages/src/stages/mod.rs @@ -288,7 +288,7 @@ mod tests { prune_count: usize, segment: StaticFileSegment, is_full_node: bool, - expected_unwind: Option, + expected: Option, ) { let static_file_provider = db.factory.static_file_provider(); @@ -311,7 +311,7 @@ mod tests { is_full_node, false ), - Ok(expect_unwind) + Ok(expected) ); } @@ -321,7 +321,7 @@ mod tests { db: &TestStageDB, // replace DbType with your actual database type stage_id: StageId, checkpoint_block_number: BlockNumber, - expected_unwind: PipelineTarget, + expected: PipelineTarget, ) { let provider_rw = db.factory.provider_rw().unwrap(); provider_rw @@ -335,7 +335,7 @@ mod tests { false, false ), - Ok(Some(expected_unwind)) + Ok(Some(expected)) ); } @@ -344,7 +344,7 @@ mod tests { fn update_db_and_check>( db: &TestStageDB, key: u64, - expect_unwind: Option, + expected: Option, ) where ::Value: Default, { @@ -359,7 +359,7 @@ mod tests { false, false ), - Ok(expect_unwind) + Ok(expected) ); } From 2d142f1a7bf554956a9b3e79e008e61dcd955876 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Wed, 22 May 2024 14:11:44 +0100 Subject: [PATCH 29/61] update docs --- crates/storage/provider/src/providers/static_file/manager.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/crates/storage/provider/src/providers/static_file/manager.rs b/crates/storage/provider/src/providers/static_file/manager.rs index da17392a59d..29dc85a66ed 100644 --- a/crates/storage/provider/src/providers/static_file/manager.rs +++ b/crates/storage/provider/src/providers/static_file/manager.rs @@ -459,9 +459,8 @@ impl StaticFileProvider { /// 1) When a static file fails to commit but the underlying data was changed. /// 2) When a static file was committed, but the required database transaction was not. /// - /// For 1) it can self-heal. Opening a writer to this segment will automatically do that if - /// `read_only` is set to `false`. For 2) the invariants below are checked, and if broken, - /// require a pipeline unwind to heal. + /// For 1) it can self-heal if `read_only` is set to `false`. + /// For 2) the invariants below are checked, and if broken, require a pipeline unwind to heal. /// /// For each static file segment: /// * the corresponding database table should overlap or have continuity in their keys From 8ace1e520400de715651493ecc3f4bb48e6ec422 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Mon, 27 May 2024 08:36:55 +0100 Subject: [PATCH 30/61] const fns on segments --- crates/primitives/src/static_file/segment.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/primitives/src/static_file/segment.rs b/crates/primitives/src/static_file/segment.rs index bf5798a244e..b62ea535620 100644 --- a/crates/primitives/src/static_file/segment.rs +++ b/crates/primitives/src/static_file/segment.rs @@ -135,12 +135,12 @@ impl StaticFileSegment { } /// Returns `true` if the segment is `StaticFileSegment::Headers`. - pub fn is_headers(&self) -> bool { + pub const fn is_headers(&self) -> bool { matches!(self, StaticFileSegment::Headers) } /// Returns `true` if the segment is `StaticFileSegment::Receipts`. - pub fn is_receipts(&self) -> bool { + pub const fn is_receipts(&self) -> bool { matches!(self, StaticFileSegment::Receipts) } } From 50b19bacb6f6d059ccfa214d1a0d799e8ffb940a Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Mon, 27 May 2024 08:43:51 +0100 Subject: [PATCH 31/61] noop executor --- crates/evm/src/execute.rs | 55 -------------------- crates/evm/src/lib.rs | 1 + crates/evm/src/noop.rs | 65 ++++++++++++++++++++++++ crates/node/builder/src/launch/common.rs | 2 +- 4 files changed, 67 insertions(+), 56 deletions(-) create mode 100644 crates/evm/src/noop.rs diff --git a/crates/evm/src/execute.rs b/crates/evm/src/execute.rs index b72bf50089d..e7ce09e7980 100644 --- a/crates/evm/src/execute.rs +++ b/crates/evm/src/execute.rs @@ -176,61 +176,6 @@ pub trait BlockExecutorProvider: Send + Sync + Clone + Unpin + 'static { DB: Database; } -/// A [BlockExecutorProvider] implementation that does nothing. -#[derive(Debug, Default, Clone)] -#[non_exhaustive] -pub struct NoopBlockExecutorProvider; - -impl BlockExecutorProvider for NoopBlockExecutorProvider { - type Executor> = Self; - - type BatchExecutor> = Self; - - fn executor(&self, _: DB) -> Self::Executor - where - DB: Database, - { - Self - } - - fn batch_executor(&self, _: DB, _: PruneModes) -> Self::BatchExecutor - where - DB: Database, - { - Self - } -} - -impl Executor for NoopBlockExecutorProvider { - type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>; - type Output = BlockExecutionOutput; - type Error = BlockExecutionError; - - fn execute(self, _: Self::Input<'_>) -> Result { - Err(BlockExecutionError::UnavailableForNoop) - } -} - -impl BatchExecutor for NoopBlockExecutorProvider { - type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>; - type Output = BatchBlockExecutionOutput; - type Error = BlockExecutionError; - - fn execute_one(&mut self, _: Self::Input<'_>) -> Result<(), Self::Error> { - Err(BlockExecutionError::UnavailableForNoop) - } - - fn finalize(self) -> Self::Output { - unreachable!() - } - - fn set_tip(&mut self, _: BlockNumber) {} - - fn size_hint(&self) -> Option { - None - } -} - #[cfg(test)] mod tests { use super::*; diff --git a/crates/evm/src/lib.rs b/crates/evm/src/lib.rs index 94cac8bccd4..0431af2d585 100644 --- a/crates/evm/src/lib.rs +++ b/crates/evm/src/lib.rs @@ -16,6 +16,7 @@ use revm_primitives::{BlockEnv, CfgEnvWithHandlerCfg, EnvWithHandlerCfg, SpecId, pub mod either; pub mod execute; +pub mod noop; #[cfg(any(test, feature = "test-utils"))] /// test helpers for mocking executor diff --git a/crates/evm/src/noop.rs b/crates/evm/src/noop.rs new file mode 100644 index 00000000000..e68de72f188 --- /dev/null +++ b/crates/evm/src/noop.rs @@ -0,0 +1,65 @@ +//! A no operation block executor implementation. + +use reth_interfaces::{executor::BlockExecutionError, provider::ProviderError}; +use reth_primitives::{BlockNumber, BlockWithSenders, PruneModes, Receipt}; +use revm_primitives::db::Database; + +use crate::execute::{ + BatchBlockExecutionOutput, BatchExecutor, BlockExecutionInput, BlockExecutionOutput, + BlockExecutorProvider, Executor, +}; + +/// A [BlockExecutorProvider] implementation that does nothing. +#[derive(Debug, Default, Clone)] +#[non_exhaustive] +pub struct NoopBlockExecutorProvider; + +impl BlockExecutorProvider for NoopBlockExecutorProvider { + type Executor> = Self; + + type BatchExecutor> = Self; + + fn executor(&self, _: DB) -> Self::Executor + where + DB: Database, + { + Self + } + + fn batch_executor(&self, _: DB, _: PruneModes) -> Self::BatchExecutor + where + DB: Database, + { + Self + } +} + +impl Executor for NoopBlockExecutorProvider { + type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>; + type Output = BlockExecutionOutput; + type Error = BlockExecutionError; + + fn execute(self, _: Self::Input<'_>) -> Result { + Err(BlockExecutionError::UnavailableForNoop) + } +} + +impl BatchExecutor for NoopBlockExecutorProvider { + type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>; + type Output = BatchBlockExecutionOutput; + type Error = BlockExecutionError; + + fn execute_one(&mut self, _: Self::Input<'_>) -> Result<(), Self::Error> { + Err(BlockExecutionError::UnavailableForNoop) + } + + fn finalize(self) -> Self::Output { + unreachable!() + } + + fn set_tip(&mut self, _: BlockNumber) {} + + fn size_hint(&self) -> Option { + None + } +} diff --git a/crates/node/builder/src/launch/common.rs b/crates/node/builder/src/launch/common.rs index f2c7bbe6cc8..e6eb321ec17 100644 --- a/crates/node/builder/src/launch/common.rs +++ b/crates/node/builder/src/launch/common.rs @@ -11,7 +11,7 @@ use reth_beacon_consensus::EthBeaconConsensus; use reth_config::{config::EtlConfig, PruneConfig}; use reth_db::{database::Database, database_metrics::DatabaseMetrics}; use reth_downloaders::{bodies::noop::NoopBodiesDownloader, headers::noop::NoopHeaderDownloader}; -use reth_evm::execute::NoopBlockExecutorProvider; +use reth_evm::noop::NoopBlockExecutorProvider; use reth_interfaces::p2p::headers::client::HeadersClient; use reth_node_core::{ cli::config::RethRpcConfig, From 0eaa9bd3da42deb6f079dafe0c63c225ee8f4262 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Mon, 27 May 2024 08:56:24 +0100 Subject: [PATCH 32/61] swap order --- crates/storage/provider/src/providers/static_file/writer.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/crates/storage/provider/src/providers/static_file/writer.rs b/crates/storage/provider/src/providers/static_file/writer.rs index 00f107f1c86..1d38b1ebca3 100644 --- a/crates/storage/provider/src/providers/static_file/writer.rs +++ b/crates/storage/provider/src/providers/static_file/writer.rs @@ -640,20 +640,20 @@ impl StaticFileProviderRW { self.writer.user_header_mut() } - #[cfg(any(test, feature = "test-utils"))] /// Helper function to override block range for testing. + #[cfg(any(test, feature = "test-utils"))] pub fn set_block_range(&mut self, block_range: std::ops::RangeInclusive) { self.writer.user_header_mut().set_block_range(*block_range.start(), *block_range.end()) } - #[cfg(any(test, feature = "test-utils"))] /// Helper function to access [`SegmentHeader`]. + #[cfg(any(test, feature = "test-utils"))] pub fn user_header(&self) -> &SegmentHeader { self.writer.user_header() } - #[cfg(any(test, feature = "test-utils"))] /// Helper function to override block range for testing. + #[cfg(any(test, feature = "test-utils"))] pub fn inner(&mut self) -> &mut NippyJarWriter { &mut self.writer } From 7541b52b817e069f923c0deba6ccc1bd45c2696d Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Mon, 27 May 2024 08:58:03 +0100 Subject: [PATCH 33/61] update docs on create_provider_factory --- crates/node/builder/src/launch/common.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/crates/node/builder/src/launch/common.rs b/crates/node/builder/src/launch/common.rs index e6eb321ec17..6ae790bf460 100644 --- a/crates/node/builder/src/launch/common.rs +++ b/crates/node/builder/src/launch/common.rs @@ -322,7 +322,9 @@ impl LaunchContextWith> where DB: Database + Clone + 'static, { - /// Returns the [ProviderFactory] for the attached database. + /// Returns the [ProviderFactory] for the attached storage after executing a consistent check + /// between the database and static files. **It may execute a pipeline unwind if it fails this + /// check.** pub async fn create_provider_factory(&self) -> eyre::Result> { let factory = ProviderFactory::new( self.right().clone(), From 014c694199e547b421297c63818cc6884e71d858 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Mon, 27 May 2024 12:12:42 +0100 Subject: [PATCH 34/61] fix merge --- crates/evm/src/noop.rs | 5 +++-- crates/node/builder/src/launch/common.rs | 1 + 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/crates/evm/src/noop.rs b/crates/evm/src/noop.rs index e68de72f188..5e419430fa1 100644 --- a/crates/evm/src/noop.rs +++ b/crates/evm/src/noop.rs @@ -1,7 +1,8 @@ //! A no operation block executor implementation. -use reth_interfaces::{executor::BlockExecutionError, provider::ProviderError}; +use reth_execution_errors::BlockExecutionError; use reth_primitives::{BlockNumber, BlockWithSenders, PruneModes, Receipt}; +use reth_storage_errors::provider::ProviderError; use revm_primitives::db::Database; use crate::execute::{ @@ -49,7 +50,7 @@ impl BatchExecutor for NoopBlockExecutorProvider { type Output = BatchBlockExecutionOutput; type Error = BlockExecutionError; - fn execute_one(&mut self, _: Self::Input<'_>) -> Result<(), Self::Error> { + fn execute_and_verify_one(&mut self, _: Self::Input<'_>) -> Result<(), Self::Error> { Err(BlockExecutionError::UnavailableForNoop) } diff --git a/crates/node/builder/src/launch/common.rs b/crates/node/builder/src/launch/common.rs index 53a66fb7e69..9c2d4a9e0ca 100644 --- a/crates/node/builder/src/launch/common.rs +++ b/crates/node/builder/src/launch/common.rs @@ -11,6 +11,7 @@ use reth_beacon_consensus::EthBeaconConsensus; use reth_config::{config::EtlConfig, PruneConfig}; use reth_db::{database::Database, database_metrics::DatabaseMetrics}; use reth_db_common::init::{init_genesis, InitDatabaseError}; +use reth_downloaders::{bodies::noop::NoopBodiesDownloader, headers::noop::NoopHeaderDownloader}; use reth_evm::noop::NoopBlockExecutorProvider; use reth_interfaces::p2p::headers::client::HeadersClient; use reth_node_core::{ From 51c069e60241d72c29e72363bbae2499a049de65 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Tue, 28 May 2024 20:01:31 +0200 Subject: [PATCH 35/61] feat: `StaticFileProvider` requires `StaticFileAccess` for initialization (#8405) --- bin/reth/src/commands/db/mod.rs | 19 +++-- .../src/commands/db/static_files/headers.rs | 2 +- bin/reth/src/commands/db/static_files/mod.rs | 8 +- .../src/commands/db/static_files/receipts.rs | 2 +- .../commands/db/static_files/transactions.rs | 2 +- bin/reth/src/commands/db/stats.rs | 2 +- .../src/commands/debug_cmd/build_block.rs | 11 ++- bin/reth/src/commands/debug_cmd/execution.rs | 20 +++-- .../commands/debug_cmd/in_memory_merkle.rs | 16 ++-- bin/reth/src/commands/debug_cmd/merkle.rs | 15 +++- .../src/commands/debug_cmd/replay_engine.rs | 15 ++-- bin/reth/src/commands/import.rs | 12 ++- bin/reth/src/commands/import_op.rs | 12 ++- bin/reth/src/commands/import_receipts.rs | 11 ++- bin/reth/src/commands/init_cmd.rs | 8 +- bin/reth/src/commands/init_state.rs | 8 +- bin/reth/src/commands/p2p/mod.rs | 4 +- .../src/commands/recover/storage_tries.rs | 10 ++- bin/reth/src/commands/stage/drop.rs | 12 ++- bin/reth/src/commands/stage/dump/execution.rs | 8 +- .../commands/stage/dump/hashing_account.rs | 8 +- .../commands/stage/dump/hashing_storage.rs | 8 +- bin/reth/src/commands/stage/dump/merkle.rs | 8 +- bin/reth/src/commands/stage/dump/mod.rs | 9 +- bin/reth/src/commands/stage/run.rs | 12 ++- bin/reth/src/commands/stage/unwind.rs | 11 ++- crates/consensus/beacon/src/engine/mod.rs | 85 ++++++++++++++----- crates/net/downloaders/src/bodies/bodies.rs | 44 ++++++++-- crates/node/builder/src/launch/common.rs | 11 ++- crates/prune/src/pruner.rs | 10 ++- crates/stages/src/stages/mod.rs | 25 ++---- crates/stages/src/test_utils/test_db.rs | 12 ++- crates/storage/db-common/src/init.rs | 2 +- crates/storage/errors/src/provider.rs | 3 + crates/storage/nippy-jar/src/error.rs | 2 + crates/storage/nippy-jar/src/lib.rs | 20 ++--- crates/storage/nippy-jar/src/writer.rs | 31 +++++-- crates/storage/provider/src/lib.rs | 2 +- .../provider/src/providers/database/mod.rs | 22 +++-- crates/storage/provider/src/providers/mod.rs | 4 +- .../src/providers/static_file/manager.rs | 77 ++++++++++++++--- .../provider/src/providers/static_file/mod.rs | 5 +- .../src/providers/static_file/writer.rs | 24 ++++-- crates/storage/provider/src/test_utils/mod.rs | 10 ++- examples/db-access/src/main.rs | 10 ++- examples/rpc-db/src/main.rs | 11 ++- testing/ef-tests/src/cases/blockchain_test.rs | 7 +- 47 files changed, 472 insertions(+), 198 deletions(-) diff --git a/bin/reth/src/commands/db/mod.rs b/bin/reth/src/commands/db/mod.rs index 6eedabcc771..5f69c963e51 100644 --- a/bin/reth/src/commands/db/mod.rs +++ b/bin/reth/src/commands/db/mod.rs @@ -14,7 +14,7 @@ use reth_db::{ version::{get_db_version, DatabaseVersionError, DB_VERSION}, }; use reth_primitives::ChainSpec; -use reth_provider::ProviderFactory; +use reth_provider::{providers::StaticFileProvider, ProviderFactory}; use std::{ io::{self, Write}, sync::Arc, @@ -96,7 +96,8 @@ pub enum Subcommands { macro_rules! db_ro_exec { ($chain:expr, $db_path:expr, $db_args:ident, $sfp:ident, $tool:ident, $command:block) => { let db = open_db_read_only($db_path, $db_args)?; - let provider_factory = ProviderFactory::new(db, $chain.clone(), $sfp)?; + let provider_factory = + ProviderFactory::new(db, $chain.clone(), StaticFileProvider::read_only($sfp)?)?; let $tool = DbTool::new(provider_factory, $chain.clone())?; $command; @@ -156,16 +157,22 @@ impl Command { } let db = open_db(&db_path, db_args)?; - let provider_factory = - ProviderFactory::new(db, self.chain.clone(), static_files_path.clone())?; + let provider_factory = ProviderFactory::new( + db, + self.chain.clone(), + StaticFileProvider::read_write(&static_files_path)?, + )?; let tool = DbTool::new(provider_factory, self.chain.clone())?; tool.drop(db_path, static_files_path)?; } Subcommands::Clear(command) => { let db = open_db(&db_path, db_args)?; - let provider_factory = - ProviderFactory::new(db, self.chain.clone(), static_files_path)?; + let provider_factory = ProviderFactory::new( + db, + self.chain.clone(), + StaticFileProvider::read_write(static_files_path)?, + )?; command.execute(provider_factory)?; } diff --git a/bin/reth/src/commands/db/static_files/headers.rs b/bin/reth/src/commands/db/static_files/headers.rs index 7584f614c9b..e9dd8802679 100644 --- a/bin/reth/src/commands/db/static_files/headers.rs +++ b/bin/reth/src/commands/db/static_files/headers.rs @@ -38,7 +38,7 @@ impl Command { let path: PathBuf = StaticFileSegment::Headers .filename_with_configuration(filters, compression, &block_range) .into(); - let provider = StaticFileProvider::new(PathBuf::default())?; + let provider = StaticFileProvider::read_only(PathBuf::default())?; let jar_provider = provider.get_segment_provider_from_block( StaticFileSegment::Headers, self.from, diff --git a/bin/reth/src/commands/db/static_files/mod.rs b/bin/reth/src/commands/db/static_files/mod.rs index 8f5930e1083..bf657b00d03 100644 --- a/bin/reth/src/commands/db/static_files/mod.rs +++ b/bin/reth/src/commands/db/static_files/mod.rs @@ -16,7 +16,7 @@ use reth_primitives::{ }, BlockNumber, ChainSpec, StaticFileSegment, }; -use reth_provider::{BlockNumReader, ProviderFactory}; +use reth_provider::{providers::StaticFileProvider, BlockNumReader, ProviderFactory}; use reth_static_file::{segments as static_file_segments, segments::Segment}; use std::{ path::{Path, PathBuf}, @@ -99,7 +99,11 @@ impl Command { data_dir.db().as_path(), db_args.with_max_read_transaction_duration(Some(MaxReadTransactionDuration::Unbounded)), )?; - let provider_factory = Arc::new(ProviderFactory::new(db, chain, data_dir.static_files())?); + let provider_factory = Arc::new(ProviderFactory::new( + db, + chain, + StaticFileProvider::read_only(data_dir.static_files())?, + )?); { if !self.only_bench { diff --git a/bin/reth/src/commands/db/static_files/receipts.rs b/bin/reth/src/commands/db/static_files/receipts.rs index 50ebd42caf6..5c2e8ea7aee 100644 --- a/bin/reth/src/commands/db/static_files/receipts.rs +++ b/bin/reth/src/commands/db/static_files/receipts.rs @@ -43,7 +43,7 @@ impl Command { .filename_with_configuration(filters, compression, &block_range) .into(); - let provider = StaticFileProvider::new(PathBuf::default())?; + let provider = StaticFileProvider::read_only(PathBuf::default())?; let jar_provider = provider.get_segment_provider_from_block( StaticFileSegment::Receipts, self.from, diff --git a/bin/reth/src/commands/db/static_files/transactions.rs b/bin/reth/src/commands/db/static_files/transactions.rs index a5939c0f41b..cd8b8811df7 100644 --- a/bin/reth/src/commands/db/static_files/transactions.rs +++ b/bin/reth/src/commands/db/static_files/transactions.rs @@ -42,7 +42,7 @@ impl Command { let path: PathBuf = StaticFileSegment::Transactions .filename_with_configuration(filters, compression, &block_range) .into(); - let provider = StaticFileProvider::new(PathBuf::default())?; + let provider = StaticFileProvider::read_only(PathBuf::default())?; let jar_provider = provider.get_segment_provider_from_block( StaticFileSegment::Transactions, self.from, diff --git a/bin/reth/src/commands/db/stats.rs b/bin/reth/src/commands/db/stats.rs index 03c384b2ffc..850953b6b96 100644 --- a/bin/reth/src/commands/db/stats.rs +++ b/bin/reth/src/commands/db/stats.rs @@ -176,7 +176,7 @@ impl Command { } let static_files = iter_static_files(data_dir.static_files())?; - let static_file_provider = StaticFileProvider::new(data_dir.static_files())?; + let static_file_provider = StaticFileProvider::read_only(data_dir.static_files())?; let mut total_data_size = 0; let mut total_index_size = 0; diff --git a/bin/reth/src/commands/debug_cmd/build_block.rs b/bin/reth/src/commands/debug_cmd/build_block.rs index 7914ec7829d..8c22c12deeb 100644 --- a/bin/reth/src/commands/debug_cmd/build_block.rs +++ b/bin/reth/src/commands/debug_cmd/build_block.rs @@ -35,8 +35,9 @@ use reth_primitives::{ U256, }; use reth_provider::{ - providers::BlockchainProvider, BlockHashReader, BlockReader, BlockWriter, - BundleStateWithReceipts, ProviderFactory, StageCheckpointReader, StateProviderFactory, + providers::{BlockchainProvider, StaticFileProvider}, + BlockHashReader, BlockReader, BlockWriter, BundleStateWithReceipts, ProviderFactory, + StageCheckpointReader, StateProviderFactory, }; use reth_revm::database::StateProviderDatabase; #[cfg(feature = "optimism")] @@ -115,7 +116,9 @@ impl Command { let factory = ProviderFactory::new( db, self.chain.clone(), - self.datadir.unwrap_or_chain_default(self.chain.chain).static_files(), + StaticFileProvider::read_only( + self.datadir.unwrap_or_chain_default(self.chain.chain).static_files(), + )?, )?; let provider = factory.provider()?; @@ -157,7 +160,7 @@ impl Command { let provider_factory = ProviderFactory::new( Arc::clone(&db), Arc::clone(&self.chain), - data_dir.static_files(), + StaticFileProvider::read_only(data_dir.static_files())?, )?; let consensus: Arc = diff --git a/bin/reth/src/commands/debug_cmd/execution.rs b/bin/reth/src/commands/debug_cmd/execution.rs index c07efab2b9f..944be7adf6f 100644 --- a/bin/reth/src/commands/debug_cmd/execution.rs +++ b/bin/reth/src/commands/debug_cmd/execution.rs @@ -31,8 +31,8 @@ use reth_primitives::{ stage::StageId, BlockHashOrNumber, BlockNumber, ChainSpec, PruneModes, B256, }; use reth_provider::{ - BlockExecutionWriter, HeaderSyncMode, ProviderFactory, StageCheckpointReader, - StaticFileProviderFactory, + providers::StaticFileProvider, BlockExecutionWriter, HeaderSyncMode, ProviderFactory, + StageCheckpointReader, StaticFileProviderFactory, }; use reth_stages::{ sets::DefaultStages, @@ -156,6 +156,9 @@ impl Command { default_peers_path: PathBuf, ) -> eyre::Result { let secret_key = get_secret_key(&network_secret_path)?; + let static_files = StaticFileProvider::read_only( + self.datadir.unwrap_or_chain_default(self.chain.chain).static_files(), + )?; let network = self .network .network_config(config, self.chain.clone(), secret_key, default_peers_path) @@ -165,11 +168,7 @@ impl Command { self.network.discovery.addr, self.network.discovery.port, )) - .build(ProviderFactory::new( - db, - self.chain.clone(), - self.datadir.unwrap_or_chain_default(self.chain.chain).static_files(), - )?) + .build(ProviderFactory::new(db, self.chain.clone(), static_files)?) .start_network() .await?; info!(target: "reth::cli", peer_id = %network.peer_id(), local_addr = %network.local_addr(), "Connected to P2P network"); @@ -210,8 +209,11 @@ impl Command { fs::create_dir_all(&db_path)?; let db = Arc::new(init_db(db_path, self.db.database_args())?); - let provider_factory = - ProviderFactory::new(db.clone(), self.chain.clone(), data_dir.static_files())?; + let provider_factory = ProviderFactory::new( + db.clone(), + self.chain.clone(), + StaticFileProvider::read_write(data_dir.static_files())?, + )?; debug!(target: "reth::cli", chain=%self.chain.chain, genesis=?self.chain.genesis_hash(), "Initializing genesis"); init_genesis(provider_factory.clone())?; diff --git a/bin/reth/src/commands/debug_cmd/in_memory_merkle.rs b/bin/reth/src/commands/debug_cmd/in_memory_merkle.rs index 6f7a580a4b7..c3ac033bd00 100644 --- a/bin/reth/src/commands/debug_cmd/in_memory_merkle.rs +++ b/bin/reth/src/commands/debug_cmd/in_memory_merkle.rs @@ -22,9 +22,9 @@ use reth_network::NetworkHandle; use reth_network_api::NetworkInfo; use reth_primitives::{stage::StageId, BlockHashOrNumber, ChainSpec, Receipts}; use reth_provider::{ - AccountExtReader, BundleStateWithReceipts, HashingWriter, HeaderProvider, - LatestStateProviderRef, OriginalValuesKnown, ProviderFactory, StageCheckpointReader, - StateWriter, StaticFileProviderFactory, StorageReader, + providers::StaticFileProvider, AccountExtReader, BundleStateWithReceipts, HashingWriter, + HeaderProvider, LatestStateProviderRef, OriginalValuesKnown, ProviderFactory, + StageCheckpointReader, StateWriter, StaticFileProviderFactory, StorageReader, }; use reth_revm::database::StateProviderDatabase; use reth_tasks::TaskExecutor; @@ -97,7 +97,9 @@ impl Command { .build(ProviderFactory::new( db, self.chain.clone(), - self.datadir.unwrap_or_chain_default(self.chain.chain).static_files(), + StaticFileProvider::read_only( + self.datadir.unwrap_or_chain_default(self.chain.chain).static_files(), + )?, )?) .start_network() .await?; @@ -117,7 +119,11 @@ impl Command { // initialize the database let db = Arc::new(init_db(db_path, self.db.database_args())?); - let factory = ProviderFactory::new(&db, self.chain.clone(), data_dir.static_files())?; + let factory = ProviderFactory::new( + &db, + self.chain.clone(), + StaticFileProvider::read_only(data_dir.static_files())?, + )?; let provider = factory.provider()?; // Look up merkle checkpoint diff --git a/bin/reth/src/commands/debug_cmd/merkle.rs b/bin/reth/src/commands/debug_cmd/merkle.rs index 291788bad75..5bf94fd1289 100644 --- a/bin/reth/src/commands/debug_cmd/merkle.rs +++ b/bin/reth/src/commands/debug_cmd/merkle.rs @@ -24,8 +24,9 @@ use reth_network::NetworkHandle; use reth_network_api::NetworkInfo; use reth_primitives::{stage::StageCheckpoint, BlockHashOrNumber, ChainSpec, PruneModes}; use reth_provider::{ - BlockNumReader, BlockWriter, BundleStateWithReceipts, HeaderProvider, LatestStateProviderRef, - OriginalValuesKnown, ProviderError, ProviderFactory, StateWriter, + providers::StaticFileProvider, BlockNumReader, BlockWriter, BundleStateWithReceipts, + HeaderProvider, LatestStateProviderRef, OriginalValuesKnown, ProviderError, ProviderFactory, + StateWriter, }; use reth_revm::database::StateProviderDatabase; use reth_stages::{ @@ -102,7 +103,9 @@ impl Command { .build(ProviderFactory::new( db, self.chain.clone(), - self.datadir.unwrap_or_chain_default(self.chain.chain).static_files(), + StaticFileProvider::read_only( + self.datadir.unwrap_or_chain_default(self.chain.chain).static_files(), + )?, )?) .start_network() .await?; @@ -122,7 +125,11 @@ impl Command { // initialize the database let db = Arc::new(init_db(db_path, self.db.database_args())?); - let factory = ProviderFactory::new(&db, self.chain.clone(), data_dir.static_files())?; + let factory = ProviderFactory::new( + &db, + self.chain.clone(), + StaticFileProvider::read_only(data_dir.static_files())?, + )?; let provider_rw = factory.provider_rw()?; // Configure and build network diff --git a/bin/reth/src/commands/debug_cmd/replay_engine.rs b/bin/reth/src/commands/debug_cmd/replay_engine.rs index 72031ce1b86..49d3dc08add 100644 --- a/bin/reth/src/commands/debug_cmd/replay_engine.rs +++ b/bin/reth/src/commands/debug_cmd/replay_engine.rs @@ -25,8 +25,8 @@ use reth_node_core::engine::engine_store::{EngineMessageStore, StoredEngineApiMe use reth_payload_builder::{PayloadBuilderHandle, PayloadBuilderService}; use reth_primitives::{ChainSpec, PruneModes}; use reth_provider::{ - providers::BlockchainProvider, CanonStateSubscriptions, ProviderFactory, - StaticFileProviderFactory, + providers::{BlockchainProvider, StaticFileProvider}, + CanonStateSubscriptions, ProviderFactory, StaticFileProviderFactory, }; use reth_stages::Pipeline; use reth_static_file::StaticFileProducer; @@ -100,7 +100,9 @@ impl Command { .build(ProviderFactory::new( db, self.chain.clone(), - self.datadir.unwrap_or_chain_default(self.chain.chain).static_files(), + StaticFileProvider::read_only( + self.datadir.unwrap_or_chain_default(self.chain.chain).static_files(), + )?, )?) .start_network() .await?; @@ -120,8 +122,11 @@ impl Command { // Initialize the database let db = Arc::new(init_db(db_path, self.db.database_args())?); - let provider_factory = - ProviderFactory::new(db.clone(), self.chain.clone(), data_dir.static_files())?; + let provider_factory = ProviderFactory::new( + db.clone(), + self.chain.clone(), + StaticFileProvider::read_only(data_dir.static_files())?, + )?; let consensus: Arc = Arc::new(EthBeaconConsensus::new(Arc::clone(&self.chain))); diff --git a/bin/reth/src/commands/import.rs b/bin/reth/src/commands/import.rs index 70a2c339cad..efb5b780d2d 100644 --- a/bin/reth/src/commands/import.rs +++ b/bin/reth/src/commands/import.rs @@ -29,8 +29,9 @@ use reth_interfaces::p2p::{ use reth_node_events::node::NodeEvent; use reth_primitives::{stage::StageId, ChainSpec, PruneModes, B256}; use reth_provider::{ - BlockNumReader, ChainSpecProvider, HeaderProvider, HeaderSyncMode, ProviderError, - ProviderFactory, StageCheckpointReader, StaticFileProviderFactory, + providers::StaticFileProvider, BlockNumReader, ChainSpecProvider, HeaderProvider, + HeaderSyncMode, ProviderError, ProviderFactory, StageCheckpointReader, + StaticFileProviderFactory, }; use reth_stages::{prelude::*, Pipeline, StageSet}; use reth_static_file::StaticFileProducer; @@ -117,8 +118,11 @@ impl ImportCommand { info!(target: "reth::cli", path = ?db_path, "Opening database"); let db = Arc::new(init_db(db_path, self.db.database_args())?); info!(target: "reth::cli", "Database opened"); - let provider_factory = - ProviderFactory::new(db.clone(), self.chain.clone(), data_dir.static_files())?; + let provider_factory = ProviderFactory::new( + db.clone(), + self.chain.clone(), + StaticFileProvider::read_write(data_dir.static_files())?, + )?; debug!(target: "reth::cli", chain=%self.chain.chain, genesis=?self.chain.genesis_hash(), "Initializing genesis"); diff --git a/bin/reth/src/commands/import_op.rs b/bin/reth/src/commands/import_op.rs index a85fc4e3dcd..64ff23102cd 100644 --- a/bin/reth/src/commands/import_op.rs +++ b/bin/reth/src/commands/import_op.rs @@ -19,7 +19,10 @@ use reth_downloaders::file_client::{ ChunkedFileReader, FileClient, DEFAULT_BYTE_LEN_CHUNK_CHAIN_FILE, }; use reth_primitives::{op_mainnet::is_dup_tx, stage::StageId, PruneModes}; -use reth_provider::{ProviderFactory, StageCheckpointReader, StaticFileProviderFactory}; +use reth_provider::{ + providers::StaticFileProvider, ProviderFactory, StageCheckpointReader, + StaticFileProviderFactory, +}; use reth_static_file::StaticFileProducer; use std::{path::PathBuf, sync::Arc}; use tracing::{debug, error, info}; @@ -90,8 +93,11 @@ impl ImportOpCommand { let db = Arc::new(init_db(db_path, self.db.database_args())?); info!(target: "reth::cli", "Database opened"); - let provider_factory = - ProviderFactory::new(db.clone(), chain_spec.clone(), data_dir.static_files())?; + let provider_factory = ProviderFactory::new( + db.clone(), + chain_spec.clone(), + StaticFileProvider::read_write(data_dir.static_files())?, + )?; debug!(target: "reth::cli", chain=%chain_spec.chain, genesis=?chain_spec.genesis_hash(), "Initializing genesis"); diff --git a/bin/reth/src/commands/import_receipts.rs b/bin/reth/src/commands/import_receipts.rs index 018ff132b94..69513e82580 100644 --- a/bin/reth/src/commands/import_receipts.rs +++ b/bin/reth/src/commands/import_receipts.rs @@ -16,8 +16,8 @@ use reth_downloaders::{ use reth_node_core::version::SHORT_VERSION; use reth_primitives::{stage::StageId, ChainSpec, StaticFileSegment}; use reth_provider::{ - BundleStateWithReceipts, OriginalValuesKnown, ProviderFactory, StageCheckpointReader, - StateWriter, StaticFileProviderFactory, StaticFileWriter, + providers::StaticFileProvider, BundleStateWithReceipts, OriginalValuesKnown, ProviderFactory, + StageCheckpointReader, StateWriter, StaticFileProviderFactory, StaticFileWriter, }; use tracing::{debug, error, info, trace}; @@ -81,8 +81,11 @@ impl ImportReceiptsCommand { let db = Arc::new(init_db(db_path, self.db.database_args())?); info!(target: "reth::cli", "Database opened"); - let provider_factory = - ProviderFactory::new(db.clone(), self.chain.clone(), data_dir.static_files())?; + let provider_factory = ProviderFactory::new( + db.clone(), + self.chain.clone(), + StaticFileProvider::read_write(data_dir.static_files())?, + )?; let provider = provider_factory.provider_rw()?; let static_file_provider = provider_factory.static_file_provider(); diff --git a/bin/reth/src/commands/init_cmd.rs b/bin/reth/src/commands/init_cmd.rs index 3b900b3f01a..922ad5d325d 100644 --- a/bin/reth/src/commands/init_cmd.rs +++ b/bin/reth/src/commands/init_cmd.rs @@ -11,7 +11,7 @@ use clap::Parser; use reth_db::init_db; use reth_db_common::init::init_genesis; use reth_primitives::ChainSpec; -use reth_provider::ProviderFactory; +use reth_provider::{providers::StaticFileProvider, ProviderFactory}; use std::sync::Arc; use tracing::info; @@ -56,7 +56,11 @@ impl InitCommand { let db = Arc::new(init_db(&db_path, self.db.database_args())?); info!(target: "reth::cli", "Database opened"); - let provider_factory = ProviderFactory::new(db, self.chain, data_dir.static_files())?; + let provider_factory = ProviderFactory::new( + db, + self.chain, + StaticFileProvider::read_write(data_dir.static_files())?, + )?; info!(target: "reth::cli", "Writing genesis block"); diff --git a/bin/reth/src/commands/init_state.rs b/bin/reth/src/commands/init_state.rs index f5ee0c4b1c1..2e6c133d1df 100644 --- a/bin/reth/src/commands/init_state.rs +++ b/bin/reth/src/commands/init_state.rs @@ -12,7 +12,7 @@ use reth_config::config::EtlConfig; use reth_db::{database::Database, init_db}; use reth_db_common::init::init_from_state_dump; use reth_primitives::{ChainSpec, B256}; -use reth_provider::ProviderFactory; +use reth_provider::{providers::StaticFileProvider, ProviderFactory}; use std::{fs::File, io::BufReader, path::PathBuf, sync::Arc}; use tracing::info; @@ -78,7 +78,11 @@ impl InitStateCommand { let db = Arc::new(init_db(&db_path, self.db.database_args())?); info!(target: "reth::cli", "Database opened"); - let provider_factory = ProviderFactory::new(db, self.chain, data_dir.static_files())?; + let provider_factory = ProviderFactory::new( + db, + self.chain, + StaticFileProvider::read_write(data_dir.static_files())?, + )?; let etl_config = EtlConfig::new( Some(EtlConfig::from_datadir(data_dir.data_dir())), EtlConfig::default_file_size(), diff --git a/bin/reth/src/commands/p2p/mod.rs b/bin/reth/src/commands/p2p/mod.rs index b6710a363a9..574a7a8b3b4 100644 --- a/bin/reth/src/commands/p2p/mod.rs +++ b/bin/reth/src/commands/p2p/mod.rs @@ -17,7 +17,7 @@ use reth_db::create_db; use reth_interfaces::p2p::bodies::client::BodiesClient; use reth_network::NetworkConfigBuilder; use reth_primitives::{BlockHashOrNumber, ChainSpec}; -use reth_provider::ProviderFactory; +use reth_provider::{providers::StaticFileProvider, ProviderFactory}; use std::{ net::{IpAddr, SocketAddrV4, SocketAddrV6}, path::PathBuf, @@ -164,7 +164,7 @@ impl Command { let network_config = network_config_builder.build(Arc::new(ProviderFactory::new( noop_db, self.chain.clone(), - data_dir.static_files(), + StaticFileProvider::read_write(data_dir.static_files())?, )?)); let network = network_config.start_network().await?; diff --git a/bin/reth/src/commands/recover/storage_tries.rs b/bin/reth/src/commands/recover/storage_tries.rs index 583829bc39b..5d4d05bbb7a 100644 --- a/bin/reth/src/commands/recover/storage_tries.rs +++ b/bin/reth/src/commands/recover/storage_tries.rs @@ -12,7 +12,9 @@ use reth_db::{ use reth_db_common::init::init_genesis; use reth_node_core::args::DatabaseArgs; use reth_primitives::ChainSpec; -use reth_provider::{BlockNumReader, HeaderProvider, ProviderError, ProviderFactory}; +use reth_provider::{ + providers::StaticFileProvider, BlockNumReader, HeaderProvider, ProviderError, ProviderFactory, +}; use reth_trie::StateRoot; use std::{fs, sync::Arc}; use tracing::*; @@ -55,7 +57,11 @@ impl Command { fs::create_dir_all(&db_path)?; let db = Arc::new(init_db(db_path, self.db.database_args())?); - let factory = ProviderFactory::new(&db, self.chain.clone(), data_dir.static_files())?; + let factory = ProviderFactory::new( + &db, + self.chain.clone(), + StaticFileProvider::read_write(data_dir.static_files())?, + )?; debug!(target: "reth::cli", chain=%self.chain.chain, genesis=?self.chain.genesis_hash(), "Initializing genesis"); init_genesis(factory.clone())?; diff --git a/bin/reth/src/commands/stage/drop.rs b/bin/reth/src/commands/stage/drop.rs index fc3ef5768da..25ac44e7bb6 100644 --- a/bin/reth/src/commands/stage/drop.rs +++ b/bin/reth/src/commands/stage/drop.rs @@ -16,7 +16,10 @@ use reth_fs_util as fs; use reth_primitives::{ stage::StageId, static_file::find_fixed_range, ChainSpec, StaticFileSegment, }; -use reth_provider::{providers::StaticFileWriter, ProviderFactory, StaticFileProviderFactory}; +use reth_provider::{ + providers::{StaticFileProvider, StaticFileWriter}, + ProviderFactory, StaticFileProviderFactory, +}; use std::sync::Arc; /// `reth drop-stage` command @@ -59,8 +62,11 @@ impl Command { fs::create_dir_all(&db_path)?; let db = open_db(db_path.as_ref(), self.db.database_args())?; - let provider_factory = - ProviderFactory::new(db, self.chain.clone(), data_dir.static_files())?; + let provider_factory = ProviderFactory::new( + db, + self.chain.clone(), + StaticFileProvider::read_write(data_dir.static_files())?, + )?; let static_file_provider = provider_factory.static_file_provider(); let tool = DbTool::new(provider_factory, self.chain.clone())?; diff --git a/bin/reth/src/commands/stage/dump/execution.rs b/bin/reth/src/commands/stage/dump/execution.rs index d8f12b50af7..f5285987673 100644 --- a/bin/reth/src/commands/stage/dump/execution.rs +++ b/bin/reth/src/commands/stage/dump/execution.rs @@ -6,7 +6,7 @@ use reth_db::{ }; use reth_node_core::dirs::{ChainPath, DataDirPath}; use reth_primitives::stage::StageCheckpoint; -use reth_provider::{ChainSpecProvider, ProviderFactory}; +use reth_provider::{providers::StaticFileProvider, ChainSpecProvider, ProviderFactory}; use reth_stages::{stages::ExecutionStage, Stage, UnwindInput}; use tracing::info; @@ -25,7 +25,11 @@ pub(crate) async fn dump_execution_stage( if should_run { dry_run( - ProviderFactory::new(output_db, db_tool.chain.clone(), output_datadir.static_files())?, + ProviderFactory::new( + output_db, + db_tool.chain.clone(), + StaticFileProvider::read_only(output_datadir.static_files())?, + )?, to, from, ) diff --git a/bin/reth/src/commands/stage/dump/hashing_account.rs b/bin/reth/src/commands/stage/dump/hashing_account.rs index 2f28ba129a1..ec61476e139 100644 --- a/bin/reth/src/commands/stage/dump/hashing_account.rs +++ b/bin/reth/src/commands/stage/dump/hashing_account.rs @@ -4,7 +4,7 @@ use eyre::Result; use reth_db::{database::Database, table::TableImporter, tables, DatabaseEnv}; use reth_node_core::dirs::{ChainPath, DataDirPath}; use reth_primitives::{stage::StageCheckpoint, BlockNumber}; -use reth_provider::ProviderFactory; +use reth_provider::{providers::StaticFileProvider, ProviderFactory}; use reth_stages::{stages::AccountHashingStage, Stage, UnwindInput}; use tracing::info; @@ -30,7 +30,11 @@ pub(crate) async fn dump_hashing_account_stage( if should_run { dry_run( - ProviderFactory::new(output_db, db_tool.chain.clone(), output_datadir.static_files())?, + ProviderFactory::new( + output_db, + db_tool.chain.clone(), + StaticFileProvider::read_only(output_datadir.static_files())?, + )?, to, from, ) diff --git a/bin/reth/src/commands/stage/dump/hashing_storage.rs b/bin/reth/src/commands/stage/dump/hashing_storage.rs index 7d38892dc8b..4ede4e61ac0 100644 --- a/bin/reth/src/commands/stage/dump/hashing_storage.rs +++ b/bin/reth/src/commands/stage/dump/hashing_storage.rs @@ -4,7 +4,7 @@ use eyre::Result; use reth_db::{database::Database, table::TableImporter, tables, DatabaseEnv}; use reth_node_core::dirs::{ChainPath, DataDirPath}; use reth_primitives::stage::StageCheckpoint; -use reth_provider::ProviderFactory; +use reth_provider::{providers::StaticFileProvider, ProviderFactory}; use reth_stages::{stages::StorageHashingStage, Stage, UnwindInput}; use tracing::info; @@ -21,7 +21,11 @@ pub(crate) async fn dump_hashing_storage_stage( if should_run { dry_run( - ProviderFactory::new(output_db, db_tool.chain.clone(), output_datadir.static_files())?, + ProviderFactory::new( + output_db, + db_tool.chain.clone(), + StaticFileProvider::read_only(output_datadir.static_files())?, + )?, to, from, ) diff --git a/bin/reth/src/commands/stage/dump/merkle.rs b/bin/reth/src/commands/stage/dump/merkle.rs index 9b421be7ca3..bf7caec56a0 100644 --- a/bin/reth/src/commands/stage/dump/merkle.rs +++ b/bin/reth/src/commands/stage/dump/merkle.rs @@ -6,7 +6,7 @@ use reth_db::{database::Database, table::TableImporter, tables, DatabaseEnv}; use reth_exex::ExExManagerHandle; use reth_node_core::dirs::{ChainPath, DataDirPath}; use reth_primitives::{stage::StageCheckpoint, BlockNumber, PruneModes}; -use reth_provider::ProviderFactory; +use reth_provider::{providers::StaticFileProvider, ProviderFactory}; use reth_stages::{ stages::{ AccountHashingStage, ExecutionStage, ExecutionStageThresholds, MerkleStage, @@ -45,7 +45,11 @@ pub(crate) async fn dump_merkle_stage( if should_run { dry_run( - ProviderFactory::new(output_db, db_tool.chain.clone(), output_datadir.static_files())?, + ProviderFactory::new( + output_db, + db_tool.chain.clone(), + StaticFileProvider::read_only(output_datadir.static_files())?, + )?, to, from, ) diff --git a/bin/reth/src/commands/stage/dump/mod.rs b/bin/reth/src/commands/stage/dump/mod.rs index fa418435655..efe89cf27b1 100644 --- a/bin/reth/src/commands/stage/dump/mod.rs +++ b/bin/reth/src/commands/stage/dump/mod.rs @@ -17,7 +17,7 @@ use reth_db::{ }; use reth_node_core::dirs::PlatformPath; use reth_primitives::ChainSpec; -use reth_provider::ProviderFactory; +use reth_provider::{providers::StaticFileProvider, ProviderFactory}; use std::{path::PathBuf, sync::Arc}; use tracing::info; @@ -105,8 +105,11 @@ impl Command { let db_path = data_dir.db(); info!(target: "reth::cli", path = ?db_path, "Opening database"); let db = Arc::new(init_db(db_path, self.db.database_args())?); - let provider_factory = - ProviderFactory::new(db, self.chain.clone(), data_dir.static_files())?; + let provider_factory = ProviderFactory::new( + db, + self.chain.clone(), + StaticFileProvider::read_write(data_dir.static_files())?, + )?; info!(target: "reth::cli", "Database opened"); diff --git a/bin/reth/src/commands/stage/run.rs b/bin/reth/src/commands/stage/run.rs index d34b67db42a..ccbe6887b4a 100644 --- a/bin/reth/src/commands/stage/run.rs +++ b/bin/reth/src/commands/stage/run.rs @@ -25,7 +25,8 @@ use reth_downloaders::bodies::bodies::BodiesDownloaderBuilder; use reth_exex::ExExManagerHandle; use reth_primitives::ChainSpec; use reth_provider::{ - ProviderFactory, StageCheckpointReader, StageCheckpointWriter, StaticFileProviderFactory, + providers::StaticFileProvider, ProviderFactory, StageCheckpointReader, StageCheckpointWriter, + StaticFileProviderFactory, }; use reth_stages::{ stages::{ @@ -145,8 +146,11 @@ impl Command { let db = Arc::new(init_db(db_path, self.db.database_args())?); info!(target: "reth::cli", "Database opened"); - let factory = - ProviderFactory::new(Arc::clone(&db), self.chain.clone(), data_dir.static_files())?; + let factory = ProviderFactory::new( + Arc::clone(&db), + self.chain.clone(), + StaticFileProvider::read_write(data_dir.static_files())?, + )?; let mut provider_rw = factory.provider_rw()?; if let Some(listen_addr) = self.metrics { @@ -195,7 +199,7 @@ impl Command { let provider_factory = Arc::new(ProviderFactory::new( db.clone(), self.chain.clone(), - data_dir.static_files(), + StaticFileProvider::read_write(data_dir.static_files())?, )?); let network = self diff --git a/bin/reth/src/commands/stage/unwind.rs b/bin/reth/src/commands/stage/unwind.rs index d2ebe70db18..15ce7751844 100644 --- a/bin/reth/src/commands/stage/unwind.rs +++ b/bin/reth/src/commands/stage/unwind.rs @@ -10,8 +10,8 @@ use reth_exex::ExExManagerHandle; use reth_node_core::args::NetworkArgs; use reth_primitives::{BlockHashOrNumber, ChainSpec, PruneModes, B256}; use reth_provider::{ - BlockExecutionWriter, BlockNumReader, ChainSpecProvider, HeaderSyncMode, ProviderFactory, - StaticFileProviderFactory, + providers::StaticFileProvider, BlockExecutionWriter, BlockNumReader, ChainSpecProvider, + HeaderSyncMode, ProviderFactory, StaticFileProviderFactory, }; use reth_stages::{ sets::DefaultStages, @@ -81,8 +81,11 @@ impl Command { let config: Config = confy::load_path(config_path).unwrap_or_default(); let db = Arc::new(open_db(db_path.as_ref(), self.db.database_args())?); - let provider_factory = - ProviderFactory::new(db, self.chain.clone(), data_dir.static_files())?; + let provider_factory = ProviderFactory::new( + db, + self.chain.clone(), + StaticFileProvider::read_write(data_dir.static_files())?, + )?; let range = self.command.unwind_range(provider_factory.clone())?; if *range.start() == 0 { diff --git a/crates/consensus/beacon/src/engine/mod.rs b/crates/consensus/beacon/src/engine/mod.rs index c1ef6228746..5ad82da4724 100644 --- a/crates/consensus/beacon/src/engine/mod.rs +++ b/crates/consensus/beacon/src/engine/mod.rs @@ -2151,6 +2151,7 @@ mod tests { use reth_db::{tables, test_utils::create_test_static_files_dir, transaction::DbTxMut}; use reth_interfaces::test_utils::generators::random_block; use reth_primitives::U256; + use reth_provider::providers::StaticFileProvider; use reth_rpc_types::engine::ForkchoiceUpdateError; #[tokio::test] @@ -2206,8 +2207,12 @@ mod tests { let (_static_dir, static_dir_path) = create_test_static_files_dir(); insert_blocks( - ProviderFactory::new(env.db.as_ref(), chain_spec.clone(), static_dir_path) - .expect("create provider factory with static_files"), + ProviderFactory::new( + env.db.as_ref(), + chain_spec.clone(), + StaticFileProvider::read_write(static_dir_path).unwrap(), + ) + .expect("create provider factory with static_files"), [&genesis, &block1].into_iter(), ); env.db @@ -2263,8 +2268,12 @@ mod tests { let (_static_dir, static_dir_path) = create_test_static_files_dir(); insert_blocks( - ProviderFactory::new(env.db.as_ref(), chain_spec.clone(), static_dir_path) - .expect("create provider factory with static_files"), + ProviderFactory::new( + env.db.as_ref(), + chain_spec.clone(), + StaticFileProvider::read_write(static_dir_path).unwrap(), + ) + .expect("create provider factory with static_files"), [&genesis, &block1].into_iter(), ); @@ -2284,8 +2293,12 @@ mod tests { // Insert next head immediately after sending forkchoice update insert_blocks( - ProviderFactory::new(env.db.as_ref(), chain_spec.clone(), static_dir_path) - .expect("create provider factory with static_files"), + ProviderFactory::new( + env.db.as_ref(), + chain_spec.clone(), + StaticFileProvider::read_write(static_dir_path).unwrap(), + ) + .expect("create provider factory with static_files"), [&next_head].into_iter(), ); @@ -2325,8 +2338,12 @@ mod tests { let (_static_dir, static_dir_path) = create_test_static_files_dir(); insert_blocks( - ProviderFactory::new(env.db.as_ref(), chain_spec.clone(), static_dir_path) - .expect("create provider factory with static_files"), + ProviderFactory::new( + env.db.as_ref(), + chain_spec.clone(), + StaticFileProvider::read_write(static_dir_path).unwrap(), + ) + .expect("create provider factory with static_files"), [&genesis, &block1].into_iter(), ); @@ -2377,8 +2394,12 @@ mod tests { let (_static_dir, static_dir_path) = create_test_static_files_dir(); insert_blocks( - ProviderFactory::new(env.db.as_ref(), chain_spec.clone(), static_dir_path) - .expect("create provider factory with static_files"), + ProviderFactory::new( + env.db.as_ref(), + chain_spec.clone(), + StaticFileProvider::read_write(static_dir_path).unwrap(), + ) + .expect("create provider factory with static_files"), [&genesis, &block1, &block2, &block3].into_iter(), ); @@ -2423,8 +2444,12 @@ mod tests { let (_temp_dir, temp_dir_path) = create_test_static_files_dir(); insert_blocks( - ProviderFactory::new(env.db.as_ref(), chain_spec.clone(), temp_dir_path) - .expect("create provider factory with static_files"), + ProviderFactory::new( + env.db.as_ref(), + chain_spec.clone(), + StaticFileProvider::read_write(temp_dir_path).unwrap(), + ) + .expect("create provider factory with static_files"), [&genesis, &block1].into_iter(), ); @@ -2451,7 +2476,9 @@ mod tests { use reth_db::test_utils::create_test_static_files_dir; use reth_interfaces::test_utils::generators::random_block; use reth_primitives::{genesis::Genesis, Hardfork, U256}; - use reth_provider::test_utils::blocks::BlockchainTestData; + use reth_provider::{ + providers::StaticFileProvider, test_utils::blocks::BlockchainTestData, + }; use reth_testing_utils::GenesisAllocator; #[tokio::test] @@ -2523,8 +2550,12 @@ mod tests { let (_static_dir, static_dir_path) = create_test_static_files_dir(); insert_blocks( - ProviderFactory::new(env.db.as_ref(), chain_spec.clone(), static_dir_path) - .expect("create provider factory with static_files"), + ProviderFactory::new( + env.db.as_ref(), + chain_spec.clone(), + StaticFileProvider::read_write(static_dir_path).unwrap(), + ) + .expect("create provider factory with static_files"), [&genesis, &block1, &block2].into_iter(), ); @@ -2592,8 +2623,12 @@ mod tests { let (_static_dir, static_dir_path) = create_test_static_files_dir(); insert_blocks( - ProviderFactory::new(env.db.as_ref(), chain_spec.clone(), static_dir_path) - .expect("create provider factory with static_files"), + ProviderFactory::new( + env.db.as_ref(), + chain_spec.clone(), + StaticFileProvider::read_write(static_dir_path).unwrap(), + ) + .expect("create provider factory with static_files"), [&genesis, &block1].into_iter(), ); @@ -2636,8 +2671,12 @@ mod tests { let (_static_dir, static_dir_path) = create_test_static_files_dir(); insert_blocks( - ProviderFactory::new(env.db.as_ref(), chain_spec.clone(), static_dir_path) - .expect("create provider factory with static_files"), + ProviderFactory::new( + env.db.as_ref(), + chain_spec.clone(), + StaticFileProvider::read_write(static_dir_path).unwrap(), + ) + .expect("create provider factory with static_files"), [&genesis].into_iter(), ); @@ -2700,8 +2739,12 @@ mod tests { let (_static_dir, static_dir_path) = create_test_static_files_dir(); insert_blocks( - ProviderFactory::new(env.db.as_ref(), chain_spec.clone(), static_dir_path) - .expect("create provider factory with static_files"), + ProviderFactory::new( + env.db.as_ref(), + chain_spec.clone(), + StaticFileProvider::read_write(static_dir_path).unwrap(), + ) + .expect("create provider factory with static_files"), [&data.genesis, &block1].into_iter(), ); diff --git a/crates/net/downloaders/src/bodies/bodies.rs b/crates/net/downloaders/src/bodies/bodies.rs index a806f2fa62e..265fba29f38 100644 --- a/crates/net/downloaders/src/bodies/bodies.rs +++ b/crates/net/downloaders/src/bodies/bodies.rs @@ -608,7 +608,7 @@ mod tests { use reth_db::test_utils::{create_test_rw_db, create_test_static_files_dir}; use reth_interfaces::test_utils::{generators, generators::random_block_range}; use reth_primitives::{BlockBody, B256, MAINNET}; - use reth_provider::ProviderFactory; + use reth_provider::{providers::StaticFileProvider, ProviderFactory}; use std::collections::HashMap; // Check that the blocks are emitted in order of block number, not in order of @@ -629,7 +629,12 @@ mod tests { let mut downloader = BodiesDownloaderBuilder::default().build( client.clone(), Arc::new(TestConsensus::default()), - ProviderFactory::new(db, MAINNET.clone(), static_dir_path).unwrap(), + ProviderFactory::new( + db, + MAINNET.clone(), + StaticFileProvider::read_write(static_dir_path).unwrap(), + ) + .unwrap(), ); downloader.set_download_range(0..=19).expect("failed to set download range"); @@ -670,7 +675,12 @@ mod tests { BodiesDownloaderBuilder::default().with_request_limit(request_limit).build( client.clone(), Arc::new(TestConsensus::default()), - ProviderFactory::new(db, MAINNET.clone(), static_dir_path).unwrap(), + ProviderFactory::new( + db, + MAINNET.clone(), + StaticFileProvider::read_write(static_dir_path).unwrap(), + ) + .unwrap(), ); downloader.set_download_range(0..=199).expect("failed to set download range"); @@ -700,7 +710,12 @@ mod tests { .build( client.clone(), Arc::new(TestConsensus::default()), - ProviderFactory::new(db, MAINNET.clone(), static_dir_path).unwrap(), + ProviderFactory::new( + db, + MAINNET.clone(), + StaticFileProvider::read_write(static_dir_path).unwrap(), + ) + .unwrap(), ); let mut range_start = 0; @@ -732,7 +747,12 @@ mod tests { let mut downloader = BodiesDownloaderBuilder::default().with_stream_batch_size(100).build( client.clone(), Arc::new(TestConsensus::default()), - ProviderFactory::new(db, MAINNET.clone(), static_dir_path).unwrap(), + ProviderFactory::new( + db, + MAINNET.clone(), + StaticFileProvider::read_write(static_dir_path).unwrap(), + ) + .unwrap(), ); // Set and download the first range @@ -774,7 +794,12 @@ mod tests { .build( client.clone(), Arc::new(TestConsensus::default()), - ProviderFactory::new(db, MAINNET.clone(), static_dir_path).unwrap(), + ProviderFactory::new( + db, + MAINNET.clone(), + StaticFileProvider::read_write(static_dir_path).unwrap(), + ) + .unwrap(), ); // Set and download the entire range @@ -807,7 +832,12 @@ mod tests { .build( client.clone(), Arc::new(TestConsensus::default()), - ProviderFactory::new(db, MAINNET.clone(), static_dir_path).unwrap(), + ProviderFactory::new( + db, + MAINNET.clone(), + StaticFileProvider::read_write(static_dir_path).unwrap(), + ) + .unwrap(), ); // Download the requested range diff --git a/crates/node/builder/src/launch/common.rs b/crates/node/builder/src/launch/common.rs index 9c2d4a9e0ca..fcdd04094b2 100644 --- a/crates/node/builder/src/launch/common.rs +++ b/crates/node/builder/src/launch/common.rs @@ -329,7 +329,7 @@ where let factory = ProviderFactory::new( self.right().clone(), self.chain_spec(), - self.data_dir().static_files(), + StaticFileProvider::read_write(self.data_dir().static_files())?, )? .with_static_files_metrics(); @@ -340,11 +340,10 @@ where // Check for consistency between database and static files. If it fails, it unwinds to // the first block that's consistent between database and static files. - if let Some(unwind_target) = factory.static_file_provider().check_consistency( - &factory.provider()?, - has_receipt_pruning, - false, - )? { + if let Some(unwind_target) = factory + .static_file_provider() + .check_consistency(&factory.provider()?, has_receipt_pruning)? + { // Highly unlikely to happen, and given its destructive nature, it's better to panic // instead. if PipelineTarget::Unwind(0) == unwind_target { diff --git a/crates/prune/src/pruner.rs b/crates/prune/src/pruner.rs index f4111f131a5..c9020f2f7d4 100644 --- a/crates/prune/src/pruner.rs +++ b/crates/prune/src/pruner.rs @@ -335,14 +335,18 @@ mod tests { use crate::Pruner; use reth_db::test_utils::{create_test_rw_db, create_test_static_files_dir}; use reth_primitives::{FinishedExExHeight, MAINNET}; - use reth_provider::ProviderFactory; + use reth_provider::{providers::StaticFileProvider, ProviderFactory}; #[test] fn is_pruning_needed() { let db = create_test_rw_db(); let (_static_dir, static_dir_path) = create_test_static_files_dir(); - let provider_factory = ProviderFactory::new(db, MAINNET.clone(), static_dir_path) - .expect("create provide factory with static_files"); + let provider_factory = ProviderFactory::new( + db, + MAINNET.clone(), + StaticFileProvider::read_write(static_dir_path).unwrap(), + ) + .expect("create provide factory with static_files"); let (finished_exex_height_tx, finished_exex_height_rx) = tokio::sync::watch::channel(FinishedExExHeight::NoExExs); diff --git a/crates/stages/src/stages/mod.rs b/crates/stages/src/stages/mod.rs index e631fe49a70..e2cce034843 100644 --- a/crates/stages/src/stages/mod.rs +++ b/crates/stages/src/stages/mod.rs @@ -306,11 +306,8 @@ mod tests { } assert_eq!( - static_file_provider.check_consistency( - &db.factory.database_provider_ro().unwrap(), - is_full_node, - false - ), + static_file_provider + .check_consistency(&db.factory.database_provider_ro().unwrap(), is_full_node,), Ok(expected) ); } @@ -330,11 +327,9 @@ mod tests { provider_rw.commit().unwrap(); assert_eq!( - db.factory.static_file_provider().check_consistency( - &db.factory.database_provider_ro().unwrap(), - false, - false - ), + db.factory + .static_file_provider() + .check_consistency(&db.factory.database_provider_ro().unwrap(), false,), Ok(Some(expected)) ); } @@ -354,11 +349,9 @@ mod tests { provider_rw.commit().unwrap(); assert_eq!( - db.factory.static_file_provider().check_consistency( - &db.factory.database_provider_ro().unwrap(), - false, - false - ), + db.factory + .static_file_provider() + .check_consistency(&db.factory.database_provider_ro().unwrap(), false), Ok(expected) ); } @@ -369,7 +362,7 @@ mod tests { let db_provider = db.factory.database_provider_ro().unwrap(); assert_eq!( - db.factory.static_file_provider().check_consistency(&db_provider, false, false), + db.factory.static_file_provider().check_consistency(&db_provider, false), Ok(None) ); } diff --git a/crates/stages/src/test_utils/test_db.rs b/crates/stages/src/test_utils/test_db.rs index e8b57e03492..112de705fba 100644 --- a/crates/stages/src/test_utils/test_db.rs +++ b/crates/stages/src/test_utils/test_db.rs @@ -17,7 +17,7 @@ use reth_primitives::{ StaticFileSegment, StorageEntry, TxHash, TxNumber, B256, MAINNET, U256, }; use reth_provider::{ - providers::{StaticFileProviderRWRefMut, StaticFileWriter}, + providers::{StaticFileProvider, StaticFileProviderRWRefMut, StaticFileWriter}, HistoryWriter, ProviderError, ProviderFactory, StaticFileProviderFactory, }; use std::{collections::BTreeMap, path::Path, sync::Arc}; @@ -36,8 +36,12 @@ impl Default for TestStageDB { let (static_dir, static_dir_path) = create_test_static_files_dir(); Self { temp_static_files_dir: static_dir, - factory: ProviderFactory::new(create_test_rw_db(), MAINNET.clone(), static_dir_path) - .unwrap(), + factory: ProviderFactory::new( + create_test_rw_db(), + MAINNET.clone(), + StaticFileProvider::read_write(static_dir_path).unwrap(), + ) + .unwrap(), } } } @@ -51,7 +55,7 @@ impl TestStageDB { factory: ProviderFactory::new( create_test_rw_db_with_path(path), MAINNET.clone(), - static_dir_path, + StaticFileProvider::read_write(static_dir_path).unwrap(), ) .unwrap(), } diff --git a/crates/storage/db-common/src/init.rs b/crates/storage/db-common/src/init.rs index 05435ce37e9..a2034c7a22d 100644 --- a/crates/storage/db-common/src/init.rs +++ b/crates/storage/db-common/src/init.rs @@ -583,7 +583,7 @@ mod tests { ProviderFactory::new( factory.into_db(), MAINNET.clone(), - static_file_provider.path().into(), + StaticFileProvider::read_write(static_file_provider.path()).unwrap(), ) .unwrap(), ); diff --git a/crates/storage/errors/src/provider.rs b/crates/storage/errors/src/provider.rs index 6ae7aad8ed3..cebfe506b30 100644 --- a/crates/storage/errors/src/provider.rs +++ b/crates/storage/errors/src/provider.rs @@ -122,6 +122,9 @@ pub enum ProviderError { /// Trying to insert data from an unexpected block number. #[error("trying to append data to {0} as block #{1} but expected block #{2}")] UnexpectedStaticFileBlockNumber(StaticFileSegment, BlockNumber, BlockNumber), + /// Static File Provider was initialized as read-only. + #[error("cannot get a writer on a read-only environment.")] + ReadOnlyStaticFileAccess, /// Error encountered when the block number conversion from U256 to u64 causes an overflow. #[error("failed to convert block number U256 to u64: {0}")] BlockNumberOverflow(U256), diff --git a/crates/storage/nippy-jar/src/error.rs b/crates/storage/nippy-jar/src/error.rs index d59500842c7..58e27a76b4c 100644 --- a/crates/storage/nippy-jar/src/error.rs +++ b/crates/storage/nippy-jar/src/error.rs @@ -57,4 +57,6 @@ pub enum NippyJarError { InvalidPruning(u64, u64), #[error("jar has been frozen and cannot be modified.")] FrozenJar, + #[error("File is in an inconsistent state.")] + InconsistentState, } diff --git a/crates/storage/nippy-jar/src/lib.rs b/crates/storage/nippy-jar/src/lib.rs index 2eafe68c409..33ae80979fe 100644 --- a/crates/storage/nippy-jar/src/lib.rs +++ b/crates/storage/nippy-jar/src/lib.rs @@ -382,7 +382,7 @@ impl NippyJar { self.freeze_filters()?; // Creates the writer, data and offsets file - let mut writer = NippyJarWriter::new(self)?; + let mut writer = NippyJarWriter::new(self, true)?; // Append rows to file while holding offsets in memory writer.append_rows(columns, total_rows)?; @@ -1114,7 +1114,7 @@ mod tests { assert!(initial_offset_size > 0); // Appends a third row - let mut writer = NippyJarWriter::new(nippy).unwrap(); + let mut writer = NippyJarWriter::new(nippy, true).unwrap(); writer.append_column(Some(Ok(&col1[2]))).unwrap(); writer.append_column(Some(Ok(&col2[2]))).unwrap(); @@ -1145,7 +1145,7 @@ mod tests { // Writer will execute a consistency check and verify first that the offset list on disk // doesn't match the nippy.rows, and prune it. Then, it will prune the data file // accordingly as well. - let writer = NippyJarWriter::new(nippy).unwrap(); + let writer = NippyJarWriter::new(nippy, true).unwrap(); assert_eq!(initial_rows, writer.rows()); assert_eq!( initial_offset_size, @@ -1171,7 +1171,7 @@ mod tests { // Appends a third row, so we have an offset list in memory, which is not flushed to disk, // while the data has been. - let mut writer = NippyJarWriter::new(nippy).unwrap(); + let mut writer = NippyJarWriter::new(nippy, true).unwrap(); writer.append_column(Some(Ok(&col1[2]))).unwrap(); writer.append_column(Some(Ok(&col2[2]))).unwrap(); @@ -1194,7 +1194,7 @@ mod tests { // Writer will execute a consistency check and verify that the data file has more data than // it should, and resets it to the last offset of the list (on disk here) - let writer = NippyJarWriter::new(nippy).unwrap(); + let writer = NippyJarWriter::new(nippy, true).unwrap(); assert_eq!(initial_rows, writer.rows()); assert_eq!( initial_data_size, @@ -1210,7 +1210,7 @@ mod tests { assert_eq!(nippy.max_row_size, 0); assert_eq!(nippy.rows, 0); - let mut writer = NippyJarWriter::new(nippy).unwrap(); + let mut writer = NippyJarWriter::new(nippy, true).unwrap(); assert_eq!(writer.column(), 0); writer.append_column(Some(Ok(&col1[0]))).unwrap(); @@ -1245,7 +1245,7 @@ mod tests { assert_eq!(nippy.max_row_size, col1[0].len() + col2[0].len()); assert_eq!(nippy.rows, 1); - let mut writer = NippyJarWriter::new(nippy).unwrap(); + let mut writer = NippyJarWriter::new(nippy, true).unwrap(); assert_eq!(writer.column(), 0); writer.append_column(Some(Ok(&col1[1]))).unwrap(); @@ -1276,7 +1276,7 @@ mod tests { fn prune_rows(num_columns: usize, file_path: &Path, col1: &[Vec], col2: &[Vec]) { let nippy = NippyJar::load_without_header(file_path).unwrap(); - let mut writer = NippyJarWriter::new(nippy).unwrap(); + let mut writer = NippyJarWriter::new(nippy, true).unwrap(); // Appends a third row, so we have an offset list in memory, which is not flushed to disk writer.append_column(Some(Ok(&col1[2]))).unwrap(); @@ -1306,7 +1306,7 @@ mod tests { } // This should prune from the ondisk offset list and clear the jar. - let mut writer = NippyJarWriter::new(nippy).unwrap(); + let mut writer = NippyJarWriter::new(nippy, true).unwrap(); writer.prune_rows(1).unwrap(); assert_eq!(writer.rows(), 0); assert_eq!(writer.max_row_size(), 0); @@ -1343,6 +1343,6 @@ mod tests { data_file.set_len(data_len - 32 * missing_offsets).unwrap(); // runs the consistency check. - let _ = NippyJarWriter::new(nippy).unwrap(); + let _ = NippyJarWriter::new(nippy, true).unwrap(); } } diff --git a/crates/storage/nippy-jar/src/writer.rs b/crates/storage/nippy-jar/src/writer.rs index 908f80e80db..23c72c40210 100644 --- a/crates/storage/nippy-jar/src/writer.rs +++ b/crates/storage/nippy-jar/src/writer.rs @@ -43,7 +43,11 @@ pub struct NippyJarWriter { impl NippyJarWriter { /// Creates a [`NippyJarWriter`] from [`NippyJar`]. - pub fn new(jar: NippyJar) -> Result { + /// + /// If `with_consistency_heal` is set to true, it will self-heal on any inconsistent state. This + /// might be undesireable in concurrent situations. If it encounters an issue, it will return an + /// error instead. + pub fn new(jar: NippyJar, with_consistency_heal: bool) -> Result { let (data_file, offsets_file, is_created) = Self::create_or_open_files(jar.data_path(), &jar.offsets_path())?; @@ -63,8 +67,10 @@ impl NippyJarWriter { // If we are opening a previously created jar, we need to check its consistency, and make // changes if necessary. if !is_created { - writer.check_consistency_and_heal()?; - writer.commit()?; + writer.ensure_file_consistency(!with_consistency_heal)?; + if with_consistency_heal { + writer.commit()?; + } } Ok(writer) @@ -118,13 +124,17 @@ impl NippyJarWriter { Ok((data_file, offsets_file, is_created)) } - /// Performs consistency checks on the [`NippyJar`] file and acts upon any issues: + /// Performs consistency checks on the [`NippyJar`] file and heals upon any issues if + /// `read_only` is set to false: /// * Is the offsets file size expected? /// * Is the data file size expected? /// /// This is based on the assumption that [`NippyJar`] configuration is **always** the last one /// to be updated when something is written, as by the `commit()` function shows. - pub fn check_consistency_and_heal(&mut self) -> Result<(), NippyJarError> { + /// + /// If `read_only` is set to true and an issue is found it will return a + /// [NippyJarError::InconsistentState] error. + pub fn ensure_file_consistency(&mut self, read_only: bool) -> Result<(), NippyJarError> { let reader = self.jar.open_data_reader()?; // When an offset size is smaller than the initial (8), we are dealing with immutable @@ -138,6 +148,11 @@ impl NippyJarWriter { OFFSET_SIZE_BYTES as usize) as u64; // expected size of the data file let actual_offsets_file_size = self.offsets_file.get_ref().metadata()?.len(); + if read_only && expected_offsets_file_size.cmp(&actual_offsets_file_size) != Ordering::Equal + { + return Err(NippyJarError::InconsistentState) + } + // Offsets configuration wasn't properly committed match expected_offsets_file_size.cmp(&actual_offsets_file_size) { Ordering::Less => { @@ -165,6 +180,10 @@ impl NippyJarWriter { let last_offset = reader.reverse_offset(0)?; let data_file_len = self.data_file.get_ref().metadata()?.len(); + if read_only && last_offset.cmp(&data_file_len) != Ordering::Equal { + return Err(NippyJarError::InconsistentState) + } + // Offset list wasn't properly committed match last_offset.cmp(&data_file_len) { Ordering::Less => { @@ -191,7 +210,7 @@ impl NippyJarWriter { // Since we decrease the offset list, we need to check the consistency of // `self.jar.rows` again - self.check_consistency_and_heal()?; + self.ensure_file_consistency(false)?; break } } diff --git a/crates/storage/provider/src/lib.rs b/crates/storage/provider/src/lib.rs index 864a962414e..81b239ae0a4 100644 --- a/crates/storage/provider/src/lib.rs +++ b/crates/storage/provider/src/lib.rs @@ -21,7 +21,7 @@ pub mod providers; pub use providers::{ DatabaseProvider, DatabaseProviderRO, DatabaseProviderRW, HistoricalStateProvider, HistoricalStateProviderRef, LatestStateProvider, LatestStateProviderRef, ProviderFactory, - StaticFileWriter, + StaticFileAccess, StaticFileWriter, }; #[cfg(any(test, feature = "test-utils"))] diff --git a/crates/storage/provider/src/providers/database/mod.rs b/crates/storage/provider/src/providers/database/mod.rs index 60dc635eb3a..7ed717f7778 100644 --- a/crates/storage/provider/src/providers/database/mod.rs +++ b/crates/storage/provider/src/providers/database/mod.rs @@ -20,7 +20,7 @@ use reth_primitives::{ use revm::primitives::{BlockEnv, CfgEnvWithHandlerCfg}; use std::{ ops::{RangeBounds, RangeInclusive}, - path::{Path, PathBuf}, + path::Path, sync::Arc, }; use tracing::trace; @@ -50,13 +50,9 @@ impl ProviderFactory { pub fn new( db: DB, chain_spec: Arc, - static_files_path: PathBuf, + static_file_provider: StaticFileProvider, ) -> RethResult> { - Ok(Self { - db: Arc::new(db), - chain_spec, - static_file_provider: StaticFileProvider::new(static_files_path)?, - }) + Ok(Self { db: Arc::new(db), chain_spec, static_file_provider }) } /// Enables metrics on the static file provider. @@ -84,12 +80,12 @@ impl ProviderFactory { path: P, chain_spec: Arc, args: DatabaseArguments, - static_files_path: PathBuf, + static_file_provider: StaticFileProvider, ) -> RethResult { Ok(ProviderFactory:: { db: Arc::new(init_db(path, args).map_err(|e| RethError::Custom(e.to_string()))?), chain_spec, - static_file_provider: StaticFileProvider::new(static_files_path)?, + static_file_provider, }) } } @@ -572,8 +568,10 @@ impl Clone for ProviderFactory { mod tests { use super::ProviderFactory; use crate::{ - providers::StaticFileWriter, test_utils::create_test_provider_factory, BlockHashReader, - BlockNumReader, BlockWriter, HeaderSyncGapProvider, HeaderSyncMode, TransactionsProvider, + providers::{StaticFileProvider, StaticFileWriter}, + test_utils::create_test_provider_factory, + BlockHashReader, BlockNumReader, BlockWriter, HeaderSyncGapProvider, HeaderSyncMode, + TransactionsProvider, }; use alloy_rlp::Decodable; use assert_matches::assert_matches; @@ -632,7 +630,7 @@ mod tests { tempfile::TempDir::new().expect(ERROR_TEMPDIR).into_path(), Arc::new(chain_spec), DatabaseArguments::new(Default::default()), - static_dir_path, + StaticFileProvider::read_write(static_dir_path).unwrap(), ) .unwrap(); diff --git a/crates/storage/provider/src/providers/mod.rs b/crates/storage/provider/src/providers/mod.rs index d6a7d34c816..e5f0f61a93c 100644 --- a/crates/storage/provider/src/providers/mod.rs +++ b/crates/storage/provider/src/providers/mod.rs @@ -40,8 +40,8 @@ pub use database::*; mod static_file; pub use static_file::{ - StaticFileJarProvider, StaticFileProvider, StaticFileProviderRW, StaticFileProviderRWRefMut, - StaticFileWriter, + StaticFileAccess, StaticFileJarProvider, StaticFileProvider, StaticFileProviderRW, + StaticFileProviderRWRefMut, StaticFileWriter, }; mod state; diff --git a/crates/storage/provider/src/providers/static_file/manager.rs b/crates/storage/provider/src/providers/static_file/manager.rs index 54883a812f3..f41a8b6b313 100644 --- a/crates/storage/provider/src/providers/static_file/manager.rs +++ b/crates/storage/provider/src/providers/static_file/manager.rs @@ -43,17 +43,44 @@ use tracing::warn; /// range. type SegmentRanges = HashMap>; +/// Access mode on a static file provider. RO/RW. +#[derive(Debug, Default, PartialEq, Eq)] +pub enum StaticFileAccess { + /// Read-only access. + #[default] + RO, + /// Read-write access. + RW, +} + +impl StaticFileAccess { + /// Returns `true` if read-only access. + pub const fn is_read_only(&self) -> bool { + matches!(self, StaticFileAccess::RO) + } +} + /// [`StaticFileProvider`] manages all existing [`StaticFileJarProvider`]. #[derive(Debug, Default, Clone)] pub struct StaticFileProvider(pub(crate) Arc); impl StaticFileProvider { /// Creates a new [`StaticFileProvider`]. - pub fn new(path: impl AsRef) -> ProviderResult { - let provider = Self(Arc::new(StaticFileProviderInner::new(path)?)); + fn new(path: impl AsRef, env: StaticFileAccess) -> ProviderResult { + let provider = Self(Arc::new(StaticFileProviderInner::new(path, env)?)); provider.initialize_index()?; Ok(provider) } + + /// Creates a new [`StaticFileProvider`] with read-only access. + pub fn read_only(path: impl AsRef) -> ProviderResult { + Self::new(path, StaticFileAccess::RO) + } + + /// Creates a new [`StaticFileProvider`] with read-write access. + pub fn read_write(path: impl AsRef) -> ProviderResult { + Self::new(path, StaticFileAccess::RW) + } } impl Deref for StaticFileProvider { @@ -82,11 +109,13 @@ pub struct StaticFileProviderInner { /// Maintains a map of StaticFile writers for each [`StaticFileSegment`] writers: DashMap, metrics: Option>, + /// Whether the provider should operate on a read-only access. + env: StaticFileAccess, } impl StaticFileProviderInner { /// Creates a new [`StaticFileProviderInner`]. - fn new(path: impl AsRef) -> ProviderResult { + fn new(path: impl AsRef, env: StaticFileAccess) -> ProviderResult { let provider = Self { map: Default::default(), writers: Default::default(), @@ -95,10 +124,15 @@ impl StaticFileProviderInner { path: path.as_ref().to_path_buf(), load_filters: false, metrics: None, + env, }; Ok(provider) } + + pub fn is_read_only(&self) -> bool { + self.env.is_read_only() + } } impl StaticFileProvider { @@ -459,8 +493,10 @@ impl StaticFileProvider { /// 1) When a static file fails to commit but the underlying data was changed. /// 2) When a static file was committed, but the required database transaction was not. /// - /// For 1) it can self-heal if `read_only` is set to `false`. - /// For 2) the invariants below are checked, and if broken, require a pipeline unwind to heal. + /// For 1) it can self-heal if `read_only` is set to `false`. Otherwise, it will return an + /// error. + /// For 2) the invariants below are checked, and if broken, require a pipeline unwind + /// to heal. /// /// For each static file segment: /// * the corresponding database table should overlap or have continuity in their keys @@ -476,7 +512,6 @@ impl StaticFileProvider { &self, provider: &DatabaseProvider, has_receipt_pruning: bool, - read_only: bool, ) -> ProviderResult> { let mut unwind_target: Option = None; let mut update_unwind_target = |new_target: Option| { @@ -504,9 +539,7 @@ impl StaticFileProvider { // * pruning data was interrupted before a config commit, then we have deleted data that // we are expected to still have. We need to check the Database and unwind everything // accordingly. - if !read_only { - self.ensure_file_consistency(segment)?; - } + self.ensure_file_consistency(segment)?; // Only applies to block-based static files. (Headers) // @@ -880,7 +913,7 @@ pub trait StaticFileWriter { /// Commits all changes of all [`StaticFileProviderRW`] of all [`StaticFileSegment`]. fn commit(&self) -> ProviderResult<()>; - /// Checks consistency of the segment latest file and heals if necessary. + /// Checks consistency of the segment latest file and heals if possible. fn ensure_file_consistency(&self, segment: StaticFileSegment) -> ProviderResult<()>; } @@ -890,6 +923,10 @@ impl StaticFileWriter for StaticFileProvider { block: BlockNumber, segment: StaticFileSegment, ) -> ProviderResult> { + if self.env.is_read_only() { + return Err(ProviderError::ReadOnlyStaticFileAccess) + } + tracing::trace!(target: "providers::static_file", ?block, ?segment, "Getting static file writer."); Ok(match self.writers.entry(segment) { DashMapEntry::Occupied(entry) => entry.into_ref(), @@ -920,7 +957,25 @@ impl StaticFileWriter for StaticFileProvider { } fn ensure_file_consistency(&self, segment: StaticFileSegment) -> ProviderResult<()> { - self.latest_writer(segment)?.ensure_file_consistency() + match self.env { + StaticFileAccess::RO => { + let latest_block = self.get_highest_static_file_block(segment).unwrap_or_default(); + + let mut writer = StaticFileProviderRW::new( + segment, + latest_block, + Arc::downgrade(&self.0), + self.metrics.clone(), + )?; + + writer.ensure_file_consistency(self.env.is_read_only())?; + } + StaticFileAccess::RW => { + self.latest_writer(segment)?.ensure_file_consistency(self.env.is_read_only())?; + } + } + + Ok(()) } } diff --git a/crates/storage/provider/src/providers/static_file/mod.rs b/crates/storage/provider/src/providers/static_file/mod.rs index cb9f879dde6..e6f286637fb 100644 --- a/crates/storage/provider/src/providers/static_file/mod.rs +++ b/crates/storage/provider/src/providers/static_file/mod.rs @@ -1,5 +1,5 @@ mod manager; -pub use manager::{StaticFileProvider, StaticFileWriter}; +pub use manager::{StaticFileAccess, StaticFileProvider, StaticFileWriter}; mod jar; pub use jar::StaticFileJarProvider; @@ -150,7 +150,8 @@ mod tests { // Use providers to query Header data and compare if it matches { let db_provider = factory.provider().unwrap(); - let manager = StaticFileProvider::new(static_files_path.path()).unwrap().with_filters(); + let manager = + StaticFileProvider::read_write(static_files_path.path()).unwrap().with_filters(); let jar_provider = manager .get_segment_provider_from_block(StaticFileSegment::Headers, 0, Some(&static_file)) .unwrap(); diff --git a/crates/storage/provider/src/providers/static_file/writer.rs b/crates/storage/provider/src/providers/static_file/writer.rs index 49782e21f69..210390c56bb 100644 --- a/crates/storage/provider/src/providers/static_file/writer.rs +++ b/crates/storage/provider/src/providers/static_file/writer.rs @@ -90,7 +90,8 @@ impl StaticFileProviderRW { Err(err) => return Err(err), }; - let result = match NippyJarWriter::new(jar) { + let reader = Self::upgrade_provider_to_strong_reference(&reader); + let result = match NippyJarWriter::new(jar, !reader.is_read_only()) { Ok(writer) => Ok((writer, path)), Err(NippyJarError::FrozenJar) => { // This static file has been frozen, so we should @@ -110,14 +111,25 @@ impl StaticFileProviderRW { Ok(result) } - /// Checks the consistency of the file and heals it if necessary. + /// Checks the consistency of the file and heals it if necessary and `read_only` is set to + /// false. If the check fails, it will return an error. /// - /// Healing will update the end range on the [SegmentHeader]. However, for transaction based - /// segments, the block end range has to be found and healed externally. - pub fn ensure_file_consistency(&mut self) -> ProviderResult<()> { + /// If healing does happen, it will update the end range on the [SegmentHeader]. However, for + /// transaction based segments, the block end range has to be found and healed externally. + /// + /// Check [NippyJarWriter::ensure_file_consistency] for more on healing. + pub fn ensure_file_consistency(&mut self, read_only: bool) -> ProviderResult<()> { let err = |err: NippyJarError| ProviderError::NippyJar(err.to_string()); let initial_rows = self.writer.rows(); - self.writer.check_consistency_and_heal().map_err(err)?; + self.writer.ensure_file_consistency(read_only).map_err(|error| { + if matches!(error, NippyJarError::InconsistentState) { + return ProviderError::NippyJar( + "Inconsistent state found. Start the node to heal or run a manual unwind." + .to_string(), + ) + } + err(error) + })?; // If we have lost rows, we need to the [SegmentHeader] let pruned_rows = initial_rows - self.writer.rows(); diff --git a/crates/storage/provider/src/test_utils/mod.rs b/crates/storage/provider/src/test_utils/mod.rs index 2f546230944..7946263cd5b 100644 --- a/crates/storage/provider/src/test_utils/mod.rs +++ b/crates/storage/provider/src/test_utils/mod.rs @@ -1,4 +1,4 @@ -use crate::ProviderFactory; +use crate::{providers::StaticFileProvider, ProviderFactory}; use reth_db::{ test_utils::{create_test_rw_db, create_test_static_files_dir, TempDatabase}, DatabaseEnv, @@ -26,6 +26,10 @@ pub fn create_test_provider_factory_with_chain_spec( ) -> ProviderFactory>> { let (static_dir, _) = create_test_static_files_dir(); let db = create_test_rw_db(); - ProviderFactory::new(db, chain_spec, static_dir.into_path()) - .expect("create provider factory with static_files") + ProviderFactory::new( + db, + chain_spec, + StaticFileProvider::read_write(static_dir.into_path()).expect("static file provider"), + ) + .expect("create provider factory with static_files") } diff --git a/examples/db-access/src/main.rs b/examples/db-access/src/main.rs index c076b76dcf2..dd2d9d85c90 100644 --- a/examples/db-access/src/main.rs +++ b/examples/db-access/src/main.rs @@ -1,8 +1,8 @@ use reth_db::open_db_read_only; use reth_primitives::{Address, ChainSpecBuilder, B256}; use reth_provider::{ - AccountReader, BlockReader, BlockSource, HeaderProvider, ProviderFactory, ReceiptProvider, - StateProvider, TransactionsProvider, + providers::StaticFileProvider, AccountReader, BlockReader, BlockSource, HeaderProvider, + ProviderFactory, ReceiptProvider, StateProvider, TransactionsProvider, }; use reth_rpc_types::{Filter, FilteredParams}; use std::path::Path; @@ -24,7 +24,11 @@ fn main() -> eyre::Result<()> { // Instantiate a provider factory for Ethereum mainnet using the provided DB. // TODO: Should the DB version include the spec so that you do not need to specify it here? let spec = ChainSpecBuilder::mainnet().build(); - let factory = ProviderFactory::new(db, spec.into(), db_path.join("static_files"))?; + let factory = ProviderFactory::new( + db, + spec.into(), + StaticFileProvider::read_only(db_path.join("static_files"))?, + )?; // This call opens a RO transaction on the database. To write to the DB you'd need to call // the `provider_rw` function and look for the `Writer` variants of the traits. diff --git a/examples/rpc-db/src/main.rs b/examples/rpc-db/src/main.rs index 627da093c59..7797a37feed 100644 --- a/examples/rpc-db/src/main.rs +++ b/examples/rpc-db/src/main.rs @@ -14,7 +14,10 @@ use reth::{ primitives::ChainSpecBuilder, - providers::{providers::BlockchainProvider, ProviderFactory}, + providers::{ + providers::{BlockchainProvider, StaticFileProvider}, + ProviderFactory, + }, utils::db::open_db_read_only, }; use reth_db::{mdbx::DatabaseArguments, models::client_version::ClientVersion}; @@ -44,7 +47,11 @@ async fn main() -> eyre::Result<()> { DatabaseArguments::new(ClientVersion::default()), )?); let spec = Arc::new(ChainSpecBuilder::mainnet().build()); - let factory = ProviderFactory::new(db.clone(), spec.clone(), db_path.join("static_files"))?; + let factory = ProviderFactory::new( + db.clone(), + spec.clone(), + StaticFileProvider::read_only(db_path.join("static_files"))?, + )?; // 2. Setup the blockchain provider using only the database provider and a noop for the tree to // satisfy trait bounds. Tree is not used in this example since we are only operating on the diff --git a/testing/ef-tests/src/cases/blockchain_test.rs b/testing/ef-tests/src/cases/blockchain_test.rs index 27f62f88690..1ab918bdec5 100644 --- a/testing/ef-tests/src/cases/blockchain_test.rs +++ b/testing/ef-tests/src/cases/blockchain_test.rs @@ -8,7 +8,10 @@ use alloy_rlp::Decodable; use rayon::iter::{ParallelBridge, ParallelIterator}; use reth_db::test_utils::{create_test_rw_db, create_test_static_files_dir}; use reth_primitives::{BlockBody, SealedBlock, StaticFileSegment}; -use reth_provider::{providers::StaticFileWriter, HashingWriter, ProviderFactory}; +use reth_provider::{ + providers::{StaticFileProvider, StaticFileWriter}, + HashingWriter, ProviderFactory, +}; use reth_stages::{stages::ExecutionStage, ExecInput, Stage}; use std::{collections::BTreeMap, fs, path::Path, sync::Arc}; @@ -86,7 +89,7 @@ impl Case for BlockchainTestCase { let provider = ProviderFactory::new( db.as_ref(), Arc::new(case.network.clone().into()), - static_files_dir_path, + StaticFileProvider::read_write(static_files_dir_path).unwrap(), )? .provider_rw() .unwrap(); From a5be748bb9d843e6c7576bbf41d684c37a7037d8 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Wed, 29 May 2024 13:36:39 +0200 Subject: [PATCH 36/61] bump Starting unwind msg to info --- crates/stages-api/src/pipeline/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/stages-api/src/pipeline/mod.rs b/crates/stages-api/src/pipeline/mod.rs index 66a87a0f8a4..334c68fe818 100644 --- a/crates/stages-api/src/pipeline/mod.rs +++ b/crates/stages-api/src/pipeline/mod.rs @@ -316,7 +316,7 @@ where continue } - debug!( + info!( target: "sync::pipeline", from = %checkpoint.block_number, %to, From c07a75f996d62bdc11a62c049aa78dc261b0f3a2 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Wed, 29 May 2024 13:37:23 +0200 Subject: [PATCH 37/61] compare tx_len and block_len against number of rows instead --- .../provider/src/providers/static_file/writer.rs | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/crates/storage/provider/src/providers/static_file/writer.rs b/crates/storage/provider/src/providers/static_file/writer.rs index 210390c56bb..795f01c3a3f 100644 --- a/crates/storage/provider/src/providers/static_file/writer.rs +++ b/crates/storage/provider/src/providers/static_file/writer.rs @@ -120,7 +120,7 @@ impl StaticFileProviderRW { /// Check [NippyJarWriter::ensure_file_consistency] for more on healing. pub fn ensure_file_consistency(&mut self, read_only: bool) -> ProviderResult<()> { let err = |err: NippyJarError| ProviderError::NippyJar(err.to_string()); - let initial_rows = self.writer.rows(); + self.writer.ensure_file_consistency(read_only).map_err(|error| { if matches!(error, NippyJarError::InconsistentState) { return ProviderError::NippyJar( @@ -131,8 +131,13 @@ impl StaticFileProviderRW { err(error) })?; - // If we have lost rows, we need to the [SegmentHeader] - let pruned_rows = initial_rows - self.writer.rows(); + // If we have lost rows, we need to update the [SegmentHeader] + let expected_rows = if self.user_header().segment().is_headers() { + self.user_header().block_len().unwrap_or_default() + } else { + self.user_header().tx_len().unwrap_or_default() + }; + let pruned_rows = expected_rows - self.writer.rows() as u64; if pruned_rows > 0 { self.user_header_mut().prune(pruned_rows as u64); } From 76156e2abcdc08c6e55eb0714fd6705c865e4320 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Wed, 29 May 2024 13:38:22 +0200 Subject: [PATCH 38/61] point queries should return None if the static file cant be found for id --- .../src/providers/static_file/manager.rs | 66 +++++++++++++++---- 1 file changed, 54 insertions(+), 12 deletions(-) diff --git a/crates/storage/provider/src/providers/static_file/manager.rs b/crates/storage/provider/src/providers/static_file/manager.rs index 3e9a53444eb..3ebdd15042a 100644 --- a/crates/storage/provider/src/providers/static_file/manager.rs +++ b/crates/storage/provider/src/providers/static_file/manager.rs @@ -995,8 +995,15 @@ impl HeaderProvider for StaticFileProvider { } fn header_by_number(&self, num: BlockNumber) -> ProviderResult> { - self.get_segment_provider_from_block(StaticFileSegment::Headers, num, None)? - .header_by_number(num) + self.get_segment_provider_from_block(StaticFileSegment::Headers, num, None) + .and_then(|provider| provider.header_by_number(num)) + .or_else(|err| { + if let ProviderError::MissingStaticFileBlock(_, _) = err { + Ok(None) + } else { + Err(err) + } + }) } fn header_td(&self, block_hash: &BlockHash) -> ProviderResult> { @@ -1009,8 +1016,15 @@ impl HeaderProvider for StaticFileProvider { } fn header_td_by_number(&self, num: BlockNumber) -> ProviderResult> { - self.get_segment_provider_from_block(StaticFileSegment::Headers, num, None)? - .header_td_by_number(num) + self.get_segment_provider_from_block(StaticFileSegment::Headers, num, None) + .and_then(|provider| provider.header_td_by_number(num)) + .or_else(|err| { + if let ProviderError::MissingStaticFileBlock(_, _) = err { + Ok(None) + } else { + Err(err) + } + }) } fn headers_range(&self, range: impl RangeBounds) -> ProviderResult> { @@ -1023,8 +1037,15 @@ impl HeaderProvider for StaticFileProvider { } fn sealed_header(&self, num: BlockNumber) -> ProviderResult> { - self.get_segment_provider_from_block(StaticFileSegment::Headers, num, None)? - .sealed_header(num) + self.get_segment_provider_from_block(StaticFileSegment::Headers, num, None) + .and_then(|provider| provider.sealed_header(num)) + .or_else(|err| { + if let ProviderError::MissingStaticFileBlock(_, _) = err { + Ok(None) + } else { + Err(err) + } + }) } fn sealed_headers_while( @@ -1066,8 +1087,15 @@ impl BlockHashReader for StaticFileProvider { impl ReceiptProvider for StaticFileProvider { fn receipt(&self, num: TxNumber) -> ProviderResult> { - self.get_segment_provider_from_transaction(StaticFileSegment::Receipts, num, None)? - .receipt(num) + self.get_segment_provider_from_transaction(StaticFileSegment::Receipts, num, None) + .and_then(|provider| provider.receipt(num)) + .or_else(|err| { + if let ProviderError::MissingStaticFileTx(_, _) = err { + Ok(None) + } else { + Err(err) + } + }) } fn receipt_by_hash(&self, hash: TxHash) -> ProviderResult> { @@ -1171,16 +1199,30 @@ impl TransactionsProvider for StaticFileProvider { } fn transaction_by_id(&self, num: TxNumber) -> ProviderResult> { - self.get_segment_provider_from_transaction(StaticFileSegment::Transactions, num, None)? - .transaction_by_id(num) + self.get_segment_provider_from_transaction(StaticFileSegment::Transactions, num, None) + .and_then(|provider| provider.transaction_by_id(num)) + .or_else(|err| { + if let ProviderError::MissingStaticFileTx(_, _) = err { + Ok(None) + } else { + Err(err) + } + }) } fn transaction_by_id_no_hash( &self, num: TxNumber, ) -> ProviderResult> { - self.get_segment_provider_from_transaction(StaticFileSegment::Transactions, num, None)? - .transaction_by_id_no_hash(num) + self.get_segment_provider_from_transaction(StaticFileSegment::Transactions, num, None) + .and_then(|provider| provider.transaction_by_id_no_hash(num)) + .or_else(|err| { + if let ProviderError::MissingStaticFileTx(_, _) = err { + Ok(None) + } else { + Err(err) + } + }) } fn transaction_by_hash(&self, hash: TxHash) -> ProviderResult> { From 42a3764626c083aa16745427e86c8a9ce60a3698 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Wed, 29 May 2024 15:43:14 +0200 Subject: [PATCH 39/61] complete ensure_invariants --- .../src/providers/static_file/manager.rs | 75 +++++++++++++++---- 1 file changed, 60 insertions(+), 15 deletions(-) diff --git a/crates/storage/provider/src/providers/static_file/manager.rs b/crates/storage/provider/src/providers/static_file/manager.rs index 3ebdd15042a..802af1f891f 100644 --- a/crates/storage/provider/src/providers/static_file/manager.rs +++ b/crates/storage/provider/src/providers/static_file/manager.rs @@ -36,7 +36,7 @@ use std::{ sync::{mpsc, Arc}, }; use strum::IntoEnumIterator; -use tracing::warn; +use tracing::{info, warn}; /// Alias type for a map that can be queried for block ranges from a transaction /// segment respectively. It uses `TxNumber` to represent the transaction end of a static file @@ -563,6 +563,11 @@ impl StaticFileProvider { if indices.last_tx_num() <= highest_tx { break } + } else { + // If the block body indices can not be found, then it means that static + // files is ahead of database, and the `ensure_invariants` check will fix + // it by comparing with stage checkpoints. + break } if last_block == 0 { break @@ -575,22 +580,22 @@ impl StaticFileProvider { } if let Some(unwind) = match segment { - StaticFileSegment::Headers => self.check_invariants::<_, tables::Headers>( + StaticFileSegment::Headers => self.ensure_invariants::<_, tables::Headers>( provider, - StageId::Headers, + segment, highest_block, highest_block, )?, StaticFileSegment::Transactions => self - .check_invariants::<_, tables::Transactions>( + .ensure_invariants::<_, tables::Transactions>( provider, - StageId::Bodies, + segment, highest_tx, highest_block, )?, - StaticFileSegment::Receipts => self.check_invariants::<_, tables::Receipts>( + StaticFileSegment::Receipts => self.ensure_invariants::<_, tables::Receipts>( provider, - StageId::Execution, + segment, highest_tx, highest_block, )?, @@ -602,12 +607,19 @@ impl StaticFileProvider { Ok(unwind_target.map(PipelineTarget::Unwind)) } - /// Check invariants for each corresponding table and static file segment. See - /// [Self::check_consistency] for more. - fn check_invariants>( + /// Check invariants for each corresponding table and static file segment: + /// + /// * the corresponding database table should overlap or have continuity in their keys + /// ([TxNumber] or [BlockNumber]). + /// * its highest block should match the stage checkpoint block number if it's equal or higher + /// than the corresponding database table last entry. + /// * If the checkpoint block is higher, then request a pipeline unwind to the static file + /// block. + /// * If the checkpoint block is lower, then heal by removing rows from the static file. + fn ensure_invariants>( &self, provider: &DatabaseProvider, - stage_id: StageId, + segment: StaticFileSegment, highest_static_file_entry: Option, highest_static_file_block: Option, ) -> ProviderResult> { @@ -634,10 +646,43 @@ impl StaticFileProvider { // If static file entry is ahead of the database entries, then ensure the checkpoint block // number matches. - let block_number = - provider.get_stage_checkpoint(stage_id)?.unwrap_or_default().block_number; - if block_number != highest_static_file_block { - return Ok(Some(block_number.min(highest_static_file_block))); + let checkpoint_block_number = provider + .get_stage_checkpoint(match segment { + StaticFileSegment::Headers => StageId::Headers, + StaticFileSegment::Transactions => StageId::Bodies, + StaticFileSegment::Receipts => StageId::Execution, + })? + .unwrap_or_default() + .block_number; + + // If the checkpoint is ahead, then we lost static file data. May have been an interrupted + // unwind that committed the static file changes or data corruption. + if checkpoint_block_number > highest_static_file_block { + return Ok(Some(highest_static_file_block)); + } + + // If the checkpoint is behind, then we failed to do a database commit **but committed** to + // static files. All we need to do is to unwind those rows. + if checkpoint_block_number < highest_static_file_block { + info!( + target: "reth::providers", + ?segment, + from = highest_static_file_block, + to = checkpoint_block_number, + "Unwinding static file segment." + ); + let mut writer = self.latest_writer(segment)?; + if segment.is_headers() { + writer.prune_headers(highest_static_file_block - checkpoint_block_number)?; + } else if let Some(block) = provider.block_body_indices(checkpoint_block_number)? { + let number = highest_static_file_entry - block.last_tx_num(); + if segment.is_receipts() { + writer.prune_receipts(number, checkpoint_block_number)?; + } else { + writer.prune_transactions(number, checkpoint_block_number)?; + } + } + writer.commit()?; } Ok(None) From 1b8216fbe3c0e64bcd15ca49dc6913b33d164f0c Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Wed, 29 May 2024 16:00:38 +0200 Subject: [PATCH 40/61] ProviderFactor new no longer returns result --- bin/reth/src/commands/db/mod.rs | 6 ++-- bin/reth/src/commands/db/static_files/mod.rs | 2 +- .../src/commands/debug_cmd/build_block.rs | 4 +-- bin/reth/src/commands/debug_cmd/execution.rs | 4 +-- .../commands/debug_cmd/in_memory_merkle.rs | 4 +-- bin/reth/src/commands/debug_cmd/merkle.rs | 4 +-- .../src/commands/debug_cmd/replay_engine.rs | 4 +-- bin/reth/src/commands/import.rs | 2 +- bin/reth/src/commands/import_op.rs | 2 +- bin/reth/src/commands/import_receipts_op.rs | 2 +- bin/reth/src/commands/init_cmd.rs | 2 +- bin/reth/src/commands/init_state.rs | 2 +- bin/reth/src/commands/p2p/mod.rs | 2 +- .../src/commands/recover/storage_tries.rs | 2 +- bin/reth/src/commands/stage/drop.rs | 2 +- bin/reth/src/commands/stage/dump/execution.rs | 2 +- .../commands/stage/dump/hashing_account.rs | 2 +- .../commands/stage/dump/hashing_storage.rs | 2 +- bin/reth/src/commands/stage/dump/merkle.rs | 2 +- bin/reth/src/commands/stage/dump/mod.rs | 2 +- bin/reth/src/commands/stage/run.rs | 4 +-- bin/reth/src/commands/stage/unwind.rs | 2 +- crates/consensus/beacon/src/engine/mod.rs | 30 +++++++------------ crates/net/downloaders/src/bodies/bodies.rs | 18 ++++------- crates/node/builder/src/launch/common.rs | 2 +- crates/prune/src/pruner.rs | 3 +- crates/stages/src/test_utils/test_db.rs | 6 ++-- crates/storage/db-common/src/init.rs | 13 ++++---- .../provider/src/providers/database/mod.rs | 4 +-- .../src/providers/static_file/manager.rs | 1 + .../src/providers/static_file/writer.rs | 2 +- crates/storage/provider/src/test_utils/mod.rs | 1 - examples/db-access/src/main.rs | 2 +- examples/rpc-db/src/main.rs | 2 +- testing/ef-tests/src/cases/blockchain_test.rs | 1 - 35 files changed, 61 insertions(+), 84 deletions(-) diff --git a/bin/reth/src/commands/db/mod.rs b/bin/reth/src/commands/db/mod.rs index 5f69c963e51..1aecef717f1 100644 --- a/bin/reth/src/commands/db/mod.rs +++ b/bin/reth/src/commands/db/mod.rs @@ -97,7 +97,7 @@ macro_rules! db_ro_exec { ($chain:expr, $db_path:expr, $db_args:ident, $sfp:ident, $tool:ident, $command:block) => { let db = open_db_read_only($db_path, $db_args)?; let provider_factory = - ProviderFactory::new(db, $chain.clone(), StaticFileProvider::read_only($sfp)?)?; + ProviderFactory::new(db, $chain.clone(), StaticFileProvider::read_only($sfp)?); let $tool = DbTool::new(provider_factory, $chain.clone())?; $command; @@ -161,7 +161,7 @@ impl Command { db, self.chain.clone(), StaticFileProvider::read_write(&static_files_path)?, - )?; + ); let tool = DbTool::new(provider_factory, self.chain.clone())?; tool.drop(db_path, static_files_path)?; @@ -172,7 +172,7 @@ impl Command { db, self.chain.clone(), StaticFileProvider::read_write(static_files_path)?, - )?; + ); command.execute(provider_factory)?; } diff --git a/bin/reth/src/commands/db/static_files/mod.rs b/bin/reth/src/commands/db/static_files/mod.rs index bf657b00d03..f7532237fba 100644 --- a/bin/reth/src/commands/db/static_files/mod.rs +++ b/bin/reth/src/commands/db/static_files/mod.rs @@ -103,7 +103,7 @@ impl Command { db, chain, StaticFileProvider::read_only(data_dir.static_files())?, - )?); + )); { if !self.only_bench { diff --git a/bin/reth/src/commands/debug_cmd/build_block.rs b/bin/reth/src/commands/debug_cmd/build_block.rs index 8c22c12deeb..4b0c3775e67 100644 --- a/bin/reth/src/commands/debug_cmd/build_block.rs +++ b/bin/reth/src/commands/debug_cmd/build_block.rs @@ -119,7 +119,7 @@ impl Command { StaticFileProvider::read_only( self.datadir.unwrap_or_chain_default(self.chain.chain).static_files(), )?, - )?; + ); let provider = factory.provider()?; let best_number = @@ -161,7 +161,7 @@ impl Command { Arc::clone(&db), Arc::clone(&self.chain), StaticFileProvider::read_only(data_dir.static_files())?, - )?; + ); let consensus: Arc = Arc::new(EthBeaconConsensus::new(Arc::clone(&self.chain))); diff --git a/bin/reth/src/commands/debug_cmd/execution.rs b/bin/reth/src/commands/debug_cmd/execution.rs index d9e062fe099..1b5a36e4214 100644 --- a/bin/reth/src/commands/debug_cmd/execution.rs +++ b/bin/reth/src/commands/debug_cmd/execution.rs @@ -168,7 +168,7 @@ impl Command { self.network.discovery.addr, self.network.discovery.port, )) - .build(ProviderFactory::new(db, self.chain.clone(), static_files)?) + .build(ProviderFactory::new(db, self.chain.clone(), static_files)) .start_network() .await?; info!(target: "reth::cli", peer_id = %network.peer_id(), local_addr = %network.local_addr(), "Connected to P2P network"); @@ -213,7 +213,7 @@ impl Command { db.clone(), self.chain.clone(), StaticFileProvider::read_write(data_dir.static_files())?, - )?; + ); debug!(target: "reth::cli", chain=%self.chain.chain, genesis=?self.chain.genesis_hash(), "Initializing genesis"); init_genesis(provider_factory.clone())?; diff --git a/bin/reth/src/commands/debug_cmd/in_memory_merkle.rs b/bin/reth/src/commands/debug_cmd/in_memory_merkle.rs index c3ac033bd00..d09a688bbb9 100644 --- a/bin/reth/src/commands/debug_cmd/in_memory_merkle.rs +++ b/bin/reth/src/commands/debug_cmd/in_memory_merkle.rs @@ -100,7 +100,7 @@ impl Command { StaticFileProvider::read_only( self.datadir.unwrap_or_chain_default(self.chain.chain).static_files(), )?, - )?) + )) .start_network() .await?; info!(target: "reth::cli", peer_id = %network.peer_id(), local_addr = %network.local_addr(), "Connected to P2P network"); @@ -123,7 +123,7 @@ impl Command { &db, self.chain.clone(), StaticFileProvider::read_only(data_dir.static_files())?, - )?; + ); let provider = factory.provider()?; // Look up merkle checkpoint diff --git a/bin/reth/src/commands/debug_cmd/merkle.rs b/bin/reth/src/commands/debug_cmd/merkle.rs index 6ed481fd58a..dca65327317 100644 --- a/bin/reth/src/commands/debug_cmd/merkle.rs +++ b/bin/reth/src/commands/debug_cmd/merkle.rs @@ -106,7 +106,7 @@ impl Command { StaticFileProvider::read_only( self.datadir.unwrap_or_chain_default(self.chain.chain).static_files(), )?, - )?) + )) .start_network() .await?; info!(target: "reth::cli", peer_id = %network.peer_id(), local_addr = %network.local_addr(), "Connected to P2P network"); @@ -129,7 +129,7 @@ impl Command { &db, self.chain.clone(), StaticFileProvider::read_only(data_dir.static_files())?, - )?; + ); let provider_rw = factory.provider_rw()?; // Configure and build network diff --git a/bin/reth/src/commands/debug_cmd/replay_engine.rs b/bin/reth/src/commands/debug_cmd/replay_engine.rs index 49d3dc08add..98666888722 100644 --- a/bin/reth/src/commands/debug_cmd/replay_engine.rs +++ b/bin/reth/src/commands/debug_cmd/replay_engine.rs @@ -103,7 +103,7 @@ impl Command { StaticFileProvider::read_only( self.datadir.unwrap_or_chain_default(self.chain.chain).static_files(), )?, - )?) + )) .start_network() .await?; info!(target: "reth::cli", peer_id = %network.peer_id(), local_addr = %network.local_addr(), "Connected to P2P network"); @@ -126,7 +126,7 @@ impl Command { db.clone(), self.chain.clone(), StaticFileProvider::read_only(data_dir.static_files())?, - )?; + ); let consensus: Arc = Arc::new(EthBeaconConsensus::new(Arc::clone(&self.chain))); diff --git a/bin/reth/src/commands/import.rs b/bin/reth/src/commands/import.rs index 5e10f9f5654..67bc29b225b 100644 --- a/bin/reth/src/commands/import.rs +++ b/bin/reth/src/commands/import.rs @@ -122,7 +122,7 @@ impl ImportCommand { db.clone(), self.chain.clone(), StaticFileProvider::read_write(data_dir.static_files())?, - )?; + ); debug!(target: "reth::cli", chain=%self.chain.chain, genesis=?self.chain.genesis_hash(), "Initializing genesis"); diff --git a/bin/reth/src/commands/import_op.rs b/bin/reth/src/commands/import_op.rs index 01a0dd4f47d..a4ed5ad1b8c 100644 --- a/bin/reth/src/commands/import_op.rs +++ b/bin/reth/src/commands/import_op.rs @@ -98,7 +98,7 @@ impl ImportOpCommand { db.clone(), chain_spec.clone(), StaticFileProvider::read_write(data_dir.static_files())?, - )?; + ); debug!(target: "reth::cli", chain=%chain_spec.chain, genesis=?chain_spec.genesis_hash(), "Initializing genesis"); diff --git a/bin/reth/src/commands/import_receipts_op.rs b/bin/reth/src/commands/import_receipts_op.rs index d387832aac5..6fefc4ea2a3 100644 --- a/bin/reth/src/commands/import_receipts_op.rs +++ b/bin/reth/src/commands/import_receipts_op.rs @@ -81,7 +81,7 @@ impl ImportReceiptsOpCommand { db.clone(), chain_spec.clone(), StaticFileProvider::read_write(data_dir.static_files())?, - )?; + ); import_receipts_from_file( provider_factory, diff --git a/bin/reth/src/commands/init_cmd.rs b/bin/reth/src/commands/init_cmd.rs index 922ad5d325d..0da65aba503 100644 --- a/bin/reth/src/commands/init_cmd.rs +++ b/bin/reth/src/commands/init_cmd.rs @@ -60,7 +60,7 @@ impl InitCommand { db, self.chain, StaticFileProvider::read_write(data_dir.static_files())?, - )?; + ); info!(target: "reth::cli", "Writing genesis block"); diff --git a/bin/reth/src/commands/init_state.rs b/bin/reth/src/commands/init_state.rs index 2e6c133d1df..b672f11f206 100644 --- a/bin/reth/src/commands/init_state.rs +++ b/bin/reth/src/commands/init_state.rs @@ -82,7 +82,7 @@ impl InitStateCommand { db, self.chain, StaticFileProvider::read_write(data_dir.static_files())?, - )?; + ); let etl_config = EtlConfig::new( Some(EtlConfig::from_datadir(data_dir.data_dir())), EtlConfig::default_file_size(), diff --git a/bin/reth/src/commands/p2p/mod.rs b/bin/reth/src/commands/p2p/mod.rs index 574a7a8b3b4..dd7007654ed 100644 --- a/bin/reth/src/commands/p2p/mod.rs +++ b/bin/reth/src/commands/p2p/mod.rs @@ -165,7 +165,7 @@ impl Command { noop_db, self.chain.clone(), StaticFileProvider::read_write(data_dir.static_files())?, - )?)); + ))); let network = network_config.start_network().await?; let fetch_client = network.fetch_client().await?; diff --git a/bin/reth/src/commands/recover/storage_tries.rs b/bin/reth/src/commands/recover/storage_tries.rs index 5d4d05bbb7a..8aa56897492 100644 --- a/bin/reth/src/commands/recover/storage_tries.rs +++ b/bin/reth/src/commands/recover/storage_tries.rs @@ -61,7 +61,7 @@ impl Command { &db, self.chain.clone(), StaticFileProvider::read_write(data_dir.static_files())?, - )?; + ); debug!(target: "reth::cli", chain=%self.chain.chain, genesis=?self.chain.genesis_hash(), "Initializing genesis"); init_genesis(factory.clone())?; diff --git a/bin/reth/src/commands/stage/drop.rs b/bin/reth/src/commands/stage/drop.rs index e96be694a2a..042bafa3ada 100644 --- a/bin/reth/src/commands/stage/drop.rs +++ b/bin/reth/src/commands/stage/drop.rs @@ -66,7 +66,7 @@ impl Command { db, self.chain.clone(), StaticFileProvider::read_write(data_dir.static_files())?, - )?; + ); let static_file_provider = provider_factory.static_file_provider(); let tool = DbTool::new(provider_factory, self.chain.clone())?; diff --git a/bin/reth/src/commands/stage/dump/execution.rs b/bin/reth/src/commands/stage/dump/execution.rs index f5285987673..abe1fccd70f 100644 --- a/bin/reth/src/commands/stage/dump/execution.rs +++ b/bin/reth/src/commands/stage/dump/execution.rs @@ -29,7 +29,7 @@ pub(crate) async fn dump_execution_stage( output_db, db_tool.chain.clone(), StaticFileProvider::read_only(output_datadir.static_files())?, - )?, + ), to, from, ) diff --git a/bin/reth/src/commands/stage/dump/hashing_account.rs b/bin/reth/src/commands/stage/dump/hashing_account.rs index ec61476e139..ebcf1ad8c09 100644 --- a/bin/reth/src/commands/stage/dump/hashing_account.rs +++ b/bin/reth/src/commands/stage/dump/hashing_account.rs @@ -34,7 +34,7 @@ pub(crate) async fn dump_hashing_account_stage( output_db, db_tool.chain.clone(), StaticFileProvider::read_only(output_datadir.static_files())?, - )?, + ), to, from, ) diff --git a/bin/reth/src/commands/stage/dump/hashing_storage.rs b/bin/reth/src/commands/stage/dump/hashing_storage.rs index 4ede4e61ac0..f990357d189 100644 --- a/bin/reth/src/commands/stage/dump/hashing_storage.rs +++ b/bin/reth/src/commands/stage/dump/hashing_storage.rs @@ -25,7 +25,7 @@ pub(crate) async fn dump_hashing_storage_stage( output_db, db_tool.chain.clone(), StaticFileProvider::read_only(output_datadir.static_files())?, - )?, + ), to, from, ) diff --git a/bin/reth/src/commands/stage/dump/merkle.rs b/bin/reth/src/commands/stage/dump/merkle.rs index bf7caec56a0..bb3ab80bfb1 100644 --- a/bin/reth/src/commands/stage/dump/merkle.rs +++ b/bin/reth/src/commands/stage/dump/merkle.rs @@ -49,7 +49,7 @@ pub(crate) async fn dump_merkle_stage( output_db, db_tool.chain.clone(), StaticFileProvider::read_only(output_datadir.static_files())?, - )?, + ), to, from, ) diff --git a/bin/reth/src/commands/stage/dump/mod.rs b/bin/reth/src/commands/stage/dump/mod.rs index efe89cf27b1..03cff6055ce 100644 --- a/bin/reth/src/commands/stage/dump/mod.rs +++ b/bin/reth/src/commands/stage/dump/mod.rs @@ -109,7 +109,7 @@ impl Command { db, self.chain.clone(), StaticFileProvider::read_write(data_dir.static_files())?, - )?; + ); info!(target: "reth::cli", "Database opened"); diff --git a/bin/reth/src/commands/stage/run.rs b/bin/reth/src/commands/stage/run.rs index ccbe6887b4a..e6b96bdc6a4 100644 --- a/bin/reth/src/commands/stage/run.rs +++ b/bin/reth/src/commands/stage/run.rs @@ -150,7 +150,7 @@ impl Command { Arc::clone(&db), self.chain.clone(), StaticFileProvider::read_write(data_dir.static_files())?, - )?; + ); let mut provider_rw = factory.provider_rw()?; if let Some(listen_addr) = self.metrics { @@ -200,7 +200,7 @@ impl Command { db.clone(), self.chain.clone(), StaticFileProvider::read_write(data_dir.static_files())?, - )?); + )); let network = self .network diff --git a/bin/reth/src/commands/stage/unwind.rs b/bin/reth/src/commands/stage/unwind.rs index 15ce7751844..11efddd66c9 100644 --- a/bin/reth/src/commands/stage/unwind.rs +++ b/bin/reth/src/commands/stage/unwind.rs @@ -85,7 +85,7 @@ impl Command { db, self.chain.clone(), StaticFileProvider::read_write(data_dir.static_files())?, - )?; + ); let range = self.command.unwind_range(provider_factory.clone())?; if *range.start() == 0 { diff --git a/crates/consensus/beacon/src/engine/mod.rs b/crates/consensus/beacon/src/engine/mod.rs index 56700b97b0a..5712586e2c6 100644 --- a/crates/consensus/beacon/src/engine/mod.rs +++ b/crates/consensus/beacon/src/engine/mod.rs @@ -2214,8 +2214,7 @@ mod tests { env.db.as_ref(), chain_spec.clone(), StaticFileProvider::read_write(static_dir_path).unwrap(), - ) - .expect("create provider factory with static_files"), + ), [&genesis, &block1].into_iter(), ); env.db @@ -2275,8 +2274,7 @@ mod tests { env.db.as_ref(), chain_spec.clone(), StaticFileProvider::read_write(static_dir_path).unwrap(), - ) - .expect("create provider factory with static_files"), + ), [&genesis, &block1].into_iter(), ); @@ -2300,8 +2298,7 @@ mod tests { env.db.as_ref(), chain_spec.clone(), StaticFileProvider::read_write(static_dir_path).unwrap(), - ) - .expect("create provider factory with static_files"), + ), [&next_head].into_iter(), ); @@ -2345,8 +2342,7 @@ mod tests { env.db.as_ref(), chain_spec.clone(), StaticFileProvider::read_write(static_dir_path).unwrap(), - ) - .expect("create provider factory with static_files"), + ), [&genesis, &block1].into_iter(), ); @@ -2401,8 +2397,7 @@ mod tests { env.db.as_ref(), chain_spec.clone(), StaticFileProvider::read_write(static_dir_path).unwrap(), - ) - .expect("create provider factory with static_files"), + ), [&genesis, &block1, &block2, &block3].into_iter(), ); @@ -2451,8 +2446,7 @@ mod tests { env.db.as_ref(), chain_spec.clone(), StaticFileProvider::read_write(temp_dir_path).unwrap(), - ) - .expect("create provider factory with static_files"), + ), [&genesis, &block1].into_iter(), ); @@ -2557,8 +2551,7 @@ mod tests { env.db.as_ref(), chain_spec.clone(), StaticFileProvider::read_write(static_dir_path).unwrap(), - ) - .expect("create provider factory with static_files"), + ), [&genesis, &block1, &block2].into_iter(), ); @@ -2630,8 +2623,7 @@ mod tests { env.db.as_ref(), chain_spec.clone(), StaticFileProvider::read_write(static_dir_path).unwrap(), - ) - .expect("create provider factory with static_files"), + ), [&genesis, &block1].into_iter(), ); @@ -2678,8 +2670,7 @@ mod tests { env.db.as_ref(), chain_spec.clone(), StaticFileProvider::read_write(static_dir_path).unwrap(), - ) - .expect("create provider factory with static_files"), + ), [&genesis].into_iter(), ); @@ -2746,8 +2737,7 @@ mod tests { env.db.as_ref(), chain_spec.clone(), StaticFileProvider::read_write(static_dir_path).unwrap(), - ) - .expect("create provider factory with static_files"), + ), [&data.genesis, &block1].into_iter(), ); diff --git a/crates/net/downloaders/src/bodies/bodies.rs b/crates/net/downloaders/src/bodies/bodies.rs index e0f55d4076e..9d29788d4c8 100644 --- a/crates/net/downloaders/src/bodies/bodies.rs +++ b/crates/net/downloaders/src/bodies/bodies.rs @@ -633,8 +633,7 @@ mod tests { db, MAINNET.clone(), StaticFileProvider::read_write(static_dir_path).unwrap(), - ) - .unwrap(), + ), ); downloader.set_download_range(0..=19).expect("failed to set download range"); @@ -684,8 +683,7 @@ mod tests { db, MAINNET.clone(), StaticFileProvider::read_write(static_dir_path).unwrap(), - ) - .unwrap(), + ), ); downloader.set_download_range(0..=199).expect("failed to set download range"); @@ -719,8 +717,7 @@ mod tests { db, MAINNET.clone(), StaticFileProvider::read_write(static_dir_path).unwrap(), - ) - .unwrap(), + ), ); let mut range_start = 0; @@ -756,8 +753,7 @@ mod tests { db, MAINNET.clone(), StaticFileProvider::read_write(static_dir_path).unwrap(), - ) - .unwrap(), + ), ); // Set and download the first range @@ -803,8 +799,7 @@ mod tests { db, MAINNET.clone(), StaticFileProvider::read_write(static_dir_path).unwrap(), - ) - .unwrap(), + ), ); // Set and download the entire range @@ -841,8 +836,7 @@ mod tests { db, MAINNET.clone(), StaticFileProvider::read_write(static_dir_path).unwrap(), - ) - .unwrap(), + ), ); // Download the requested range diff --git a/crates/node/builder/src/launch/common.rs b/crates/node/builder/src/launch/common.rs index 4d33b39961f..373520d6fcd 100644 --- a/crates/node/builder/src/launch/common.rs +++ b/crates/node/builder/src/launch/common.rs @@ -328,7 +328,7 @@ where self.right().clone(), self.chain_spec(), StaticFileProvider::read_write(self.data_dir().static_files())?, - )? + ) .with_static_files_metrics(); let has_receipt_pruning = diff --git a/crates/prune/src/pruner.rs b/crates/prune/src/pruner.rs index c9020f2f7d4..c6e0fffae8e 100644 --- a/crates/prune/src/pruner.rs +++ b/crates/prune/src/pruner.rs @@ -345,8 +345,7 @@ mod tests { db, MAINNET.clone(), StaticFileProvider::read_write(static_dir_path).unwrap(), - ) - .expect("create provide factory with static_files"); + ); let (finished_exex_height_tx, finished_exex_height_rx) = tokio::sync::watch::channel(FinishedExExHeight::NoExExs); diff --git a/crates/stages/src/test_utils/test_db.rs b/crates/stages/src/test_utils/test_db.rs index 112de705fba..e7920208f77 100644 --- a/crates/stages/src/test_utils/test_db.rs +++ b/crates/stages/src/test_utils/test_db.rs @@ -40,8 +40,7 @@ impl Default for TestStageDB { create_test_rw_db(), MAINNET.clone(), StaticFileProvider::read_write(static_dir_path).unwrap(), - ) - .unwrap(), + ), } } } @@ -56,8 +55,7 @@ impl TestStageDB { create_test_rw_db_with_path(path), MAINNET.clone(), StaticFileProvider::read_write(static_dir_path).unwrap(), - ) - .unwrap(), + ), } } diff --git a/crates/storage/db-common/src/init.rs b/crates/storage/db-common/src/init.rs index a2034c7a22d..0d48b5c7ad3 100644 --- a/crates/storage/db-common/src/init.rs +++ b/crates/storage/db-common/src/init.rs @@ -579,14 +579,11 @@ mod tests { init_genesis(factory.clone()).unwrap(); // Try to init db with a different genesis block - let genesis_hash = init_genesis( - ProviderFactory::new( - factory.into_db(), - MAINNET.clone(), - StaticFileProvider::read_write(static_file_provider.path()).unwrap(), - ) - .unwrap(), - ); + let genesis_hash = init_genesis(ProviderFactory::new( + factory.into_db(), + MAINNET.clone(), + StaticFileProvider::read_write(static_file_provider.path()).unwrap(), + )); assert_eq!( genesis_hash.unwrap_err(), diff --git a/crates/storage/provider/src/providers/database/mod.rs b/crates/storage/provider/src/providers/database/mod.rs index eba882420d1..92d208674b2 100644 --- a/crates/storage/provider/src/providers/database/mod.rs +++ b/crates/storage/provider/src/providers/database/mod.rs @@ -54,8 +54,8 @@ impl ProviderFactory { db: DB, chain_spec: Arc, static_file_provider: StaticFileProvider, - ) -> ProviderResult> { - Ok(Self { db: Arc::new(db), chain_spec, static_file_provider }) + ) -> ProviderFactory { + Self { db: Arc::new(db), chain_spec, static_file_provider } } /// Enables metrics on the static file provider. diff --git a/crates/storage/provider/src/providers/static_file/manager.rs b/crates/storage/provider/src/providers/static_file/manager.rs index 802af1f891f..bf63c1c7424 100644 --- a/crates/storage/provider/src/providers/static_file/manager.rs +++ b/crates/storage/provider/src/providers/static_file/manager.rs @@ -508,6 +508,7 @@ impl StaticFileProvider { /// /// WARNING: No static file writer should be held before calling this function, otherwise it /// will deadlock. + #[allow(clippy::while_let_loop)] pub fn check_consistency( &self, provider: &DatabaseProvider, diff --git a/crates/storage/provider/src/providers/static_file/writer.rs b/crates/storage/provider/src/providers/static_file/writer.rs index 795f01c3a3f..c0c1f70659f 100644 --- a/crates/storage/provider/src/providers/static_file/writer.rs +++ b/crates/storage/provider/src/providers/static_file/writer.rs @@ -139,7 +139,7 @@ impl StaticFileProviderRW { }; let pruned_rows = expected_rows - self.writer.rows() as u64; if pruned_rows > 0 { - self.user_header_mut().prune(pruned_rows as u64); + self.user_header_mut().prune(pruned_rows); } self.writer.commit().map_err(err)?; diff --git a/crates/storage/provider/src/test_utils/mod.rs b/crates/storage/provider/src/test_utils/mod.rs index 7946263cd5b..6f5ecd52678 100644 --- a/crates/storage/provider/src/test_utils/mod.rs +++ b/crates/storage/provider/src/test_utils/mod.rs @@ -31,5 +31,4 @@ pub fn create_test_provider_factory_with_chain_spec( chain_spec, StaticFileProvider::read_write(static_dir.into_path()).expect("static file provider"), ) - .expect("create provider factory with static_files") } diff --git a/examples/db-access/src/main.rs b/examples/db-access/src/main.rs index dd2d9d85c90..c43aec47ce0 100644 --- a/examples/db-access/src/main.rs +++ b/examples/db-access/src/main.rs @@ -28,7 +28,7 @@ fn main() -> eyre::Result<()> { db, spec.into(), StaticFileProvider::read_only(db_path.join("static_files"))?, - )?; + ); // This call opens a RO transaction on the database. To write to the DB you'd need to call // the `provider_rw` function and look for the `Writer` variants of the traits. diff --git a/examples/rpc-db/src/main.rs b/examples/rpc-db/src/main.rs index 7797a37feed..e51f8fe1091 100644 --- a/examples/rpc-db/src/main.rs +++ b/examples/rpc-db/src/main.rs @@ -51,7 +51,7 @@ async fn main() -> eyre::Result<()> { db.clone(), spec.clone(), StaticFileProvider::read_only(db_path.join("static_files"))?, - )?; + ); // 2. Setup the blockchain provider using only the database provider and a noop for the tree to // satisfy trait bounds. Tree is not used in this example since we are only operating on the diff --git a/testing/ef-tests/src/cases/blockchain_test.rs b/testing/ef-tests/src/cases/blockchain_test.rs index 9022d1a0dfc..ff82f36bbeb 100644 --- a/testing/ef-tests/src/cases/blockchain_test.rs +++ b/testing/ef-tests/src/cases/blockchain_test.rs @@ -91,7 +91,6 @@ impl Case for BlockchainTestCase { Arc::new(case.network.clone().into()), StaticFileProvider::read_write(static_files_dir_path).unwrap(), ) - .map_err(|err| Error::RethError(err.into()))? .provider_rw() .unwrap(); From 25776f762492bd2f62601355058154314bce9944 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Wed, 29 May 2024 16:29:05 +0200 Subject: [PATCH 41/61] update_unwind_target takes BlockNumber instead --- .../provider/src/providers/static_file/manager.rs | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/crates/storage/provider/src/providers/static_file/manager.rs b/crates/storage/provider/src/providers/static_file/manager.rs index f8acda87302..3ee51520c39 100644 --- a/crates/storage/provider/src/providers/static_file/manager.rs +++ b/crates/storage/provider/src/providers/static_file/manager.rs @@ -515,8 +515,7 @@ impl StaticFileProvider { has_receipt_pruning: bool, ) -> ProviderResult> { let mut unwind_target: Option = None; - let mut update_unwind_target = |new_target: Option| { - let new_target = new_target.unwrap_or_default(); + let mut update_unwind_target = |new_target: BlockNumber| { if let Some(target) = unwind_target.as_mut() { *target = (*target).min(new_target); } else { @@ -548,7 +547,7 @@ impl StaticFileProvider { // interruption. let mut highest_block = self.get_highest_static_file_block(segment); if initial_highest_block != highest_block { - update_unwind_target(highest_block); + update_unwind_target(highest_block.unwrap_or_default()); } // Only applies to transaction-based static files. (Receipts & Transactions) @@ -576,7 +575,7 @@ impl StaticFileProvider { last_block -= 1; highest_block = Some(last_block); - update_unwind_target(highest_block); + update_unwind_target(last_block); } } @@ -601,7 +600,7 @@ impl StaticFileProvider { highest_block, )?, } { - update_unwind_target(Some(unwind)); + update_unwind_target(unwind); } } From 78b92c08bf20efe8d7658dbc300eae8d0a0a45e4 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Wed, 29 May 2024 16:45:17 +0200 Subject: [PATCH 42/61] fix test_consistency_checkpoints --- crates/stages/src/stages/mod.rs | 24 +++++++++++++++++++----- 1 file changed, 19 insertions(+), 5 deletions(-) diff --git a/crates/stages/src/stages/mod.rs b/crates/stages/src/stages/mod.rs index 472660bd877..82d3d567a17 100644 --- a/crates/stages/src/stages/mod.rs +++ b/crates/stages/src/stages/mod.rs @@ -316,7 +316,7 @@ mod tests { db: &TestStageDB, // replace DbType with your actual database type stage_id: StageId, checkpoint_block_number: BlockNumber, - expected: PipelineTarget, + expected: Option, ) { let provider_rw = db.factory.provider_rw().unwrap(); provider_rw @@ -328,7 +328,7 @@ mod tests { db.factory .static_file_provider() .check_consistency(&db.factory.database_provider_ro().unwrap(), false,), - Ok(Some(expected)) + Ok(expected) ); } @@ -398,11 +398,25 @@ mod tests { fn test_consistency_checkpoints() { let db = seed_data(90).unwrap(); - save_checkpoint_and_check(&db, StageId::Headers, 91, PipelineTarget::Unwind(89)); + // When a checkpoint is behind, we delete data from static files. + save_checkpoint_and_check(&db, StageId::Bodies, 87, None); + assert_eq!( + db.factory + .static_file_provider() + .get_highest_static_file_block(StaticFileSegment::Transactions), + Some(87) + ); - save_checkpoint_and_check(&db, StageId::Bodies, 87, PipelineTarget::Unwind(87)); + save_checkpoint_and_check(&db, StageId::Execution, 86, None); + assert_eq!( + db.factory + .static_file_provider() + .get_highest_static_file_block(StaticFileSegment::Receipts), + Some(86) + ); - save_checkpoint_and_check(&db, StageId::Execution, 50, PipelineTarget::Unwind(50)); + // When a checkpoint is ahead, we request a pipeline unwind. + save_checkpoint_and_check(&db, StageId::Headers, 91, Some(PipelineTarget::Unwind(89))); } #[test] From 3243cfb3031a487b2973fcdf2001a41d6847ed30 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Wed, 29 May 2024 16:57:11 +0200 Subject: [PATCH 43/61] remove cfg from user_header --- .../provider/src/providers/static_file/writer.rs | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/crates/storage/provider/src/providers/static_file/writer.rs b/crates/storage/provider/src/providers/static_file/writer.rs index c0c1f70659f..95935911b99 100644 --- a/crates/storage/provider/src/providers/static_file/writer.rs +++ b/crates/storage/provider/src/providers/static_file/writer.rs @@ -652,6 +652,11 @@ impl StaticFileProviderRW { provider.upgrade().map(StaticFileProvider).expect("StaticFileProvider is dropped") } + /// Helper function to access [`SegmentHeader`]. + pub fn user_header(&self) -> &SegmentHeader { + self.writer.user_header() + } + /// Helper function to access a mutable reference to [`SegmentHeader`]. pub fn user_header_mut(&mut self) -> &mut SegmentHeader { self.writer.user_header_mut() @@ -663,12 +668,6 @@ impl StaticFileProviderRW { self.writer.user_header_mut().set_block_range(*block_range.start(), *block_range.end()) } - /// Helper function to access [`SegmentHeader`]. - #[cfg(any(test, feature = "test-utils"))] - pub fn user_header(&self) -> &SegmentHeader { - self.writer.user_header() - } - /// Helper function to override block range for testing. #[cfg(any(test, feature = "test-utils"))] pub fn inner(&mut self) -> &mut NippyJarWriter { From 5d190dd9223b52ef7ad0c34778a585f9015802a8 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Wed, 29 May 2024 17:07:06 +0200 Subject: [PATCH 44/61] only update SegmentHeader if it's not read_only, otherwise error out --- .../src/providers/static_file/writer.rs | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/crates/storage/provider/src/providers/static_file/writer.rs b/crates/storage/provider/src/providers/static_file/writer.rs index 95935911b99..2743a3ec6f7 100644 --- a/crates/storage/provider/src/providers/static_file/writer.rs +++ b/crates/storage/provider/src/providers/static_file/writer.rs @@ -119,16 +119,17 @@ impl StaticFileProviderRW { /// /// Check [NippyJarWriter::ensure_file_consistency] for more on healing. pub fn ensure_file_consistency(&mut self, read_only: bool) -> ProviderResult<()> { - let err = |err: NippyJarError| ProviderError::NippyJar(err.to_string()); + let inconsistent_error = || { + ProviderError::NippyJar( + "Inconsistent state found. Restart the node to heal.".to_string(), + ) + }; self.writer.ensure_file_consistency(read_only).map_err(|error| { if matches!(error, NippyJarError::InconsistentState) { - return ProviderError::NippyJar( - "Inconsistent state found. Start the node to heal or run a manual unwind." - .to_string(), - ) + return inconsistent_error() } - err(error) + ProviderError::NippyJar(error.to_string()) })?; // If we have lost rows, we need to update the [SegmentHeader] @@ -139,10 +140,13 @@ impl StaticFileProviderRW { }; let pruned_rows = expected_rows - self.writer.rows() as u64; if pruned_rows > 0 { + if read_only { + return Err(inconsistent_error()) + } self.user_header_mut().prune(pruned_rows); } - self.writer.commit().map_err(err)?; + self.writer.commit().map_err(|error| ProviderError::NippyJar(error.to_string()))?; // Updates the [SnapshotProvider] manager self.update_index()?; From 4d3d9d09672f6b51dff4daa7873f4f2380edd639 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Wed, 29 May 2024 17:14:42 +0200 Subject: [PATCH 45/61] replace unavailable errors on BlockExecutionError --- crates/blockchain-tree-api/src/error.rs | 2 -- crates/evm/execution-errors/src/lib.rs | 8 -------- crates/evm/src/execute.rs | 2 +- crates/evm/src/noop.rs | 6 ++++-- 4 files changed, 5 insertions(+), 13 deletions(-) diff --git a/crates/blockchain-tree-api/src/error.rs b/crates/blockchain-tree-api/src/error.rs index 7e1f9f74808..1a564725044 100644 --- a/crates/blockchain-tree-api/src/error.rs +++ b/crates/blockchain-tree-api/src/error.rs @@ -296,8 +296,6 @@ impl InsertBlockErrorKind { BlockExecutionError::CanonicalRevert { .. } | BlockExecutionError::CanonicalCommit { .. } | BlockExecutionError::AppendChainDoesntConnect { .. } | - BlockExecutionError::UnavailableForTest | - BlockExecutionError::UnavailableForNoop => false, BlockExecutionError::Other(_) => false, } } diff --git a/crates/evm/execution-errors/src/lib.rs b/crates/evm/execution-errors/src/lib.rs index e572fb3c4d0..edd70923ab0 100644 --- a/crates/evm/execution-errors/src/lib.rs +++ b/crates/evm/execution-errors/src/lib.rs @@ -111,14 +111,6 @@ pub enum BlockExecutionError { /// The fork on the other chain other_chain_fork: Box, }, - /// Only used for TestExecutor - /// - /// Note: this is not feature gated for convenience. - #[error("execution unavailable for tests")] - UnavailableForTest, - /// Only used for NoopBlockExecutorProvider - #[error("execution unavailable for noop")] - UnavailableForNoop, /// Error when fetching latest block state. #[error(transparent)] LatestBlock(#[from] ProviderError), diff --git a/crates/evm/src/execute.rs b/crates/evm/src/execute.rs index f459eceb15e..ca9086eec47 100644 --- a/crates/evm/src/execute.rs +++ b/crates/evm/src/execute.rs @@ -245,7 +245,7 @@ mod tests { type Error = BlockExecutionError; fn execute(self, _input: Self::Input<'_>) -> Result { - Err(BlockExecutionError::UnavailableForTest) + Err(BlockExecutionError::msg("execution unavailable for tests")) } } diff --git a/crates/evm/src/noop.rs b/crates/evm/src/noop.rs index 5e419430fa1..712e6b29d35 100644 --- a/crates/evm/src/noop.rs +++ b/crates/evm/src/noop.rs @@ -10,6 +10,8 @@ use crate::execute::{ BlockExecutorProvider, Executor, }; +const UNAVAILABLE_FOR_NOOP: &'static str = "execution unavailable for noop"; + /// A [BlockExecutorProvider] implementation that does nothing. #[derive(Debug, Default, Clone)] #[non_exhaustive] @@ -41,7 +43,7 @@ impl Executor for NoopBlockExecutorProvider { type Error = BlockExecutionError; fn execute(self, _: Self::Input<'_>) -> Result { - Err(BlockExecutionError::UnavailableForNoop) + Err(BlockExecutionError::msg(UNAVAILABLE_FOR_NOOP)) } } @@ -51,7 +53,7 @@ impl BatchExecutor for NoopBlockExecutorProvider { type Error = BlockExecutionError; fn execute_and_verify_one(&mut self, _: Self::Input<'_>) -> Result<(), Self::Error> { - Err(BlockExecutionError::UnavailableForNoop) + Err(BlockExecutionError::msg(UNAVAILABLE_FOR_NOOP)) } fn finalize(self) -> Self::Output { From 5c6ef86add9a458d5876007d0c94bfbc68334a33 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Wed, 29 May 2024 17:28:58 +0200 Subject: [PATCH 46/61] clippy --- crates/evm/src/noop.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/evm/src/noop.rs b/crates/evm/src/noop.rs index 712e6b29d35..220bbb39a84 100644 --- a/crates/evm/src/noop.rs +++ b/crates/evm/src/noop.rs @@ -10,7 +10,7 @@ use crate::execute::{ BlockExecutorProvider, Executor, }; -const UNAVAILABLE_FOR_NOOP: &'static str = "execution unavailable for noop"; +const UNAVAILABLE_FOR_NOOP: &str = "execution unavailable for noop"; /// A [BlockExecutorProvider] implementation that does nothing. #[derive(Debug, Default, Clone)] From 65232a9bfdd6be4d0b53c471ec0763cf63c2cd86 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Wed, 29 May 2024 18:11:39 +0200 Subject: [PATCH 47/61] access not env for StaticFileAccess var --- .../src/providers/static_file/manager.rs | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/crates/storage/provider/src/providers/static_file/manager.rs b/crates/storage/provider/src/providers/static_file/manager.rs index 3ee51520c39..96225ad6dd3 100644 --- a/crates/storage/provider/src/providers/static_file/manager.rs +++ b/crates/storage/provider/src/providers/static_file/manager.rs @@ -66,8 +66,8 @@ pub struct StaticFileProvider(pub(crate) Arc); impl StaticFileProvider { /// Creates a new [`StaticFileProvider`]. - fn new(path: impl AsRef, env: StaticFileAccess) -> ProviderResult { - let provider = Self(Arc::new(StaticFileProviderInner::new(path, env)?)); + fn new(path: impl AsRef, access: StaticFileAccess) -> ProviderResult { + let provider = Self(Arc::new(StaticFileProviderInner::new(path, access)?)); provider.initialize_index()?; Ok(provider) } @@ -109,13 +109,13 @@ pub struct StaticFileProviderInner { /// Maintains a map of StaticFile writers for each [`StaticFileSegment`] writers: DashMap, metrics: Option>, - /// Whether the provider should operate on a read-only access. - env: StaticFileAccess, + /// Access rights of the provider. + access: StaticFileAccess, } impl StaticFileProviderInner { /// Creates a new [`StaticFileProviderInner`]. - fn new(path: impl AsRef, env: StaticFileAccess) -> ProviderResult { + fn new(path: impl AsRef, access: StaticFileAccess) -> ProviderResult { let provider = Self { map: Default::default(), writers: Default::default(), @@ -124,14 +124,14 @@ impl StaticFileProviderInner { path: path.as_ref().to_path_buf(), load_filters: false, metrics: None, - env, + access, }; Ok(provider) } pub fn is_read_only(&self) -> bool { - self.env.is_read_only() + self.access.is_read_only() } } @@ -968,7 +968,7 @@ impl StaticFileWriter for StaticFileProvider { block: BlockNumber, segment: StaticFileSegment, ) -> ProviderResult> { - if self.env.is_read_only() { + if self.access.is_read_only() { return Err(ProviderError::ReadOnlyStaticFileAccess) } @@ -1002,7 +1002,7 @@ impl StaticFileWriter for StaticFileProvider { } fn ensure_file_consistency(&self, segment: StaticFileSegment) -> ProviderResult<()> { - match self.env { + match self.access { StaticFileAccess::RO => { let latest_block = self.get_highest_static_file_block(segment).unwrap_or_default(); @@ -1013,10 +1013,10 @@ impl StaticFileWriter for StaticFileProvider { self.metrics.clone(), )?; - writer.ensure_file_consistency(self.env.is_read_only())?; + writer.ensure_file_consistency(self.access.is_read_only())?; } StaticFileAccess::RW => { - self.latest_writer(segment)?.ensure_file_consistency(self.env.is_read_only())?; + self.latest_writer(segment)?.ensure_file_consistency(self.access.is_read_only())?; } } From cc551cca897a083c3dd906d1a38398874dd7eff8 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Wed, 29 May 2024 18:21:44 +0200 Subject: [PATCH 48/61] fix test import --- crates/stages/src/stages/mod.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/stages/src/stages/mod.rs b/crates/stages/src/stages/mod.rs index 82d3d567a17..3885e9f12a6 100644 --- a/crates/stages/src/stages/mod.rs +++ b/crates/stages/src/stages/mod.rs @@ -53,7 +53,6 @@ mod tests { }; use reth_evm_ethereum::execute::EthExecutorProvider; use reth_exex::ExExManagerHandle; - use reth_interfaces::provider::ProviderResult; use reth_primitives::{ address, hex_literal::hex, @@ -64,7 +63,8 @@ mod tests { }; use reth_provider::{ providers::StaticFileWriter, AccountExtReader, DatabaseProviderFactory, ProviderFactory, - ReceiptProvider, StageCheckpointWriter, StaticFileProviderFactory, StorageReader, + ProviderResult, ReceiptProvider, StageCheckpointWriter, StaticFileProviderFactory, + StorageReader, }; use reth_stages_api::{ExecInput, Stage}; use reth_testing_utils::generators::{self, random_block, random_block_range, random_receipt}; From 57b4d389544204f0534c79d9fee4405abee8a13b Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Thu, 30 May 2024 14:45:13 +0200 Subject: [PATCH 49/61] change arg to read_only on nippy writer --- crates/storage/nippy-jar/src/lib.rs | 20 +++++++++---------- crates/storage/nippy-jar/src/writer.rs | 11 +++++----- .../src/providers/static_file/writer.rs | 2 +- 3 files changed, 16 insertions(+), 17 deletions(-) diff --git a/crates/storage/nippy-jar/src/lib.rs b/crates/storage/nippy-jar/src/lib.rs index b368b1858ee..18bbcdd0c79 100644 --- a/crates/storage/nippy-jar/src/lib.rs +++ b/crates/storage/nippy-jar/src/lib.rs @@ -382,7 +382,7 @@ impl NippyJar { self.freeze_filters()?; // Creates the writer, data and offsets file - let mut writer = NippyJarWriter::new(self, true)?; + let mut writer = NippyJarWriter::new(self, false)?; // Append rows to file while holding offsets in memory writer.append_rows(columns, total_rows)?; @@ -1114,7 +1114,7 @@ mod tests { assert!(initial_offset_size > 0); // Appends a third row - let mut writer = NippyJarWriter::new(nippy, true).unwrap(); + let mut writer = NippyJarWriter::new(nippy, false).unwrap(); writer.append_column(Some(Ok(&col1[2]))).unwrap(); writer.append_column(Some(Ok(&col2[2]))).unwrap(); @@ -1145,7 +1145,7 @@ mod tests { // Writer will execute a consistency check and verify first that the offset list on disk // doesn't match the nippy.rows, and prune it. Then, it will prune the data file // accordingly as well. - let writer = NippyJarWriter::new(nippy, true).unwrap(); + let writer = NippyJarWriter::new(nippy, false).unwrap(); assert_eq!(initial_rows, writer.rows()); assert_eq!( initial_offset_size, @@ -1171,7 +1171,7 @@ mod tests { // Appends a third row, so we have an offset list in memory, which is not flushed to disk, // while the data has been. - let mut writer = NippyJarWriter::new(nippy, true).unwrap(); + let mut writer = NippyJarWriter::new(nippy, false).unwrap(); writer.append_column(Some(Ok(&col1[2]))).unwrap(); writer.append_column(Some(Ok(&col2[2]))).unwrap(); @@ -1194,7 +1194,7 @@ mod tests { // Writer will execute a consistency check and verify that the data file has more data than // it should, and resets it to the last offset of the list (on disk here) - let writer = NippyJarWriter::new(nippy, true).unwrap(); + let writer = NippyJarWriter::new(nippy, false).unwrap(); assert_eq!(initial_rows, writer.rows()); assert_eq!( initial_data_size, @@ -1210,7 +1210,7 @@ mod tests { assert_eq!(nippy.max_row_size, 0); assert_eq!(nippy.rows, 0); - let mut writer = NippyJarWriter::new(nippy, true).unwrap(); + let mut writer = NippyJarWriter::new(nippy, false).unwrap(); assert_eq!(writer.column(), 0); writer.append_column(Some(Ok(&col1[0]))).unwrap(); @@ -1245,7 +1245,7 @@ mod tests { assert_eq!(nippy.max_row_size, col1[0].len() + col2[0].len()); assert_eq!(nippy.rows, 1); - let mut writer = NippyJarWriter::new(nippy, true).unwrap(); + let mut writer = NippyJarWriter::new(nippy, false).unwrap(); assert_eq!(writer.column(), 0); writer.append_column(Some(Ok(&col1[1]))).unwrap(); @@ -1276,7 +1276,7 @@ mod tests { fn prune_rows(num_columns: usize, file_path: &Path, col1: &[Vec], col2: &[Vec]) { let nippy = NippyJar::load_without_header(file_path).unwrap(); - let mut writer = NippyJarWriter::new(nippy, true).unwrap(); + let mut writer = NippyJarWriter::new(nippy, false).unwrap(); // Appends a third row, so we have an offset list in memory, which is not flushed to disk writer.append_column(Some(Ok(&col1[2]))).unwrap(); @@ -1306,7 +1306,7 @@ mod tests { } // This should prune from the ondisk offset list and clear the jar. - let mut writer = NippyJarWriter::new(nippy, true).unwrap(); + let mut writer = NippyJarWriter::new(nippy, false).unwrap(); writer.prune_rows(1).unwrap(); assert_eq!(writer.rows(), 0); assert_eq!(writer.max_row_size(), 0); @@ -1343,6 +1343,6 @@ mod tests { data_file.set_len(data_len - 32 * missing_offsets).unwrap(); // runs the consistency check. - let _ = NippyJarWriter::new(nippy, true).unwrap(); + let _ = NippyJarWriter::new(nippy, false).unwrap(); } } diff --git a/crates/storage/nippy-jar/src/writer.rs b/crates/storage/nippy-jar/src/writer.rs index 23c72c40210..1c53c36b643 100644 --- a/crates/storage/nippy-jar/src/writer.rs +++ b/crates/storage/nippy-jar/src/writer.rs @@ -44,10 +44,9 @@ pub struct NippyJarWriter { impl NippyJarWriter { /// Creates a [`NippyJarWriter`] from [`NippyJar`]. /// - /// If `with_consistency_heal` is set to true, it will self-heal on any inconsistent state. This - /// might be undesireable in concurrent situations. If it encounters an issue, it will return an - /// error instead. - pub fn new(jar: NippyJar, with_consistency_heal: bool) -> Result { + /// If `read_only` is set to `true`, any inconsistency issue won't be healed, and will return + /// [NippyJarError::InconsistentState] instead. + pub fn new(jar: NippyJar, read_only: bool) -> Result { let (data_file, offsets_file, is_created) = Self::create_or_open_files(jar.data_path(), &jar.offsets_path())?; @@ -67,8 +66,8 @@ impl NippyJarWriter { // If we are opening a previously created jar, we need to check its consistency, and make // changes if necessary. if !is_created { - writer.ensure_file_consistency(!with_consistency_heal)?; - if with_consistency_heal { + writer.ensure_file_consistency(read_only)?; + if !read_only { writer.commit()?; } } diff --git a/crates/storage/provider/src/providers/static_file/writer.rs b/crates/storage/provider/src/providers/static_file/writer.rs index 2743a3ec6f7..315683108a0 100644 --- a/crates/storage/provider/src/providers/static_file/writer.rs +++ b/crates/storage/provider/src/providers/static_file/writer.rs @@ -91,7 +91,7 @@ impl StaticFileProviderRW { }; let reader = Self::upgrade_provider_to_strong_reference(&reader); - let result = match NippyJarWriter::new(jar, !reader.is_read_only()) { + let result = match NippyJarWriter::new(jar, reader.is_read_only()) { Ok(writer) => Ok((writer, path)), Err(NippyJarError::FrozenJar) => { // This static file has been frozen, so we should From 463178151d206fcc451a9be207554e3839ddecf0 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Thu, 30 May 2024 15:22:09 +0200 Subject: [PATCH 50/61] add ConsistencyFailStrategy --- crates/storage/nippy-jar/src/lib.rs | 22 ++++----- crates/storage/nippy-jar/src/writer.rs | 49 ++++++++++++++----- .../src/providers/static_file/writer.rs | 20 ++++++-- 3 files changed, 64 insertions(+), 27 deletions(-) diff --git a/crates/storage/nippy-jar/src/lib.rs b/crates/storage/nippy-jar/src/lib.rs index 18bbcdd0c79..9915bbb472c 100644 --- a/crates/storage/nippy-jar/src/lib.rs +++ b/crates/storage/nippy-jar/src/lib.rs @@ -41,7 +41,7 @@ mod cursor; pub use cursor::NippyJarCursor; mod writer; -pub use writer::NippyJarWriter; +pub use writer::{ConsistencyFailStrategy, NippyJarWriter}; const NIPPY_JAR_VERSION: usize = 1; @@ -382,7 +382,7 @@ impl NippyJar { self.freeze_filters()?; // Creates the writer, data and offsets file - let mut writer = NippyJarWriter::new(self, false)?; + let mut writer = NippyJarWriter::new(self, ConsistencyFailStrategy::Heal)?; // Append rows to file while holding offsets in memory writer.append_rows(columns, total_rows)?; @@ -1114,7 +1114,7 @@ mod tests { assert!(initial_offset_size > 0); // Appends a third row - let mut writer = NippyJarWriter::new(nippy, false).unwrap(); + let mut writer = NippyJarWriter::new(nippy, ConsistencyFailStrategy::Heal).unwrap(); writer.append_column(Some(Ok(&col1[2]))).unwrap(); writer.append_column(Some(Ok(&col2[2]))).unwrap(); @@ -1145,7 +1145,7 @@ mod tests { // Writer will execute a consistency check and verify first that the offset list on disk // doesn't match the nippy.rows, and prune it. Then, it will prune the data file // accordingly as well. - let writer = NippyJarWriter::new(nippy, false).unwrap(); + let writer = NippyJarWriter::new(nippy, ConsistencyFailStrategy::Heal).unwrap(); assert_eq!(initial_rows, writer.rows()); assert_eq!( initial_offset_size, @@ -1171,7 +1171,7 @@ mod tests { // Appends a third row, so we have an offset list in memory, which is not flushed to disk, // while the data has been. - let mut writer = NippyJarWriter::new(nippy, false).unwrap(); + let mut writer = NippyJarWriter::new(nippy, ConsistencyFailStrategy::Heal).unwrap(); writer.append_column(Some(Ok(&col1[2]))).unwrap(); writer.append_column(Some(Ok(&col2[2]))).unwrap(); @@ -1194,7 +1194,7 @@ mod tests { // Writer will execute a consistency check and verify that the data file has more data than // it should, and resets it to the last offset of the list (on disk here) - let writer = NippyJarWriter::new(nippy, false).unwrap(); + let writer = NippyJarWriter::new(nippy, ConsistencyFailStrategy::Heal).unwrap(); assert_eq!(initial_rows, writer.rows()); assert_eq!( initial_data_size, @@ -1210,7 +1210,7 @@ mod tests { assert_eq!(nippy.max_row_size, 0); assert_eq!(nippy.rows, 0); - let mut writer = NippyJarWriter::new(nippy, false).unwrap(); + let mut writer = NippyJarWriter::new(nippy, ConsistencyFailStrategy::Heal).unwrap(); assert_eq!(writer.column(), 0); writer.append_column(Some(Ok(&col1[0]))).unwrap(); @@ -1245,7 +1245,7 @@ mod tests { assert_eq!(nippy.max_row_size, col1[0].len() + col2[0].len()); assert_eq!(nippy.rows, 1); - let mut writer = NippyJarWriter::new(nippy, false).unwrap(); + let mut writer = NippyJarWriter::new(nippy, ConsistencyFailStrategy::Heal).unwrap(); assert_eq!(writer.column(), 0); writer.append_column(Some(Ok(&col1[1]))).unwrap(); @@ -1276,7 +1276,7 @@ mod tests { fn prune_rows(num_columns: usize, file_path: &Path, col1: &[Vec], col2: &[Vec]) { let nippy = NippyJar::load_without_header(file_path).unwrap(); - let mut writer = NippyJarWriter::new(nippy, false).unwrap(); + let mut writer = NippyJarWriter::new(nippy, ConsistencyFailStrategy::Heal).unwrap(); // Appends a third row, so we have an offset list in memory, which is not flushed to disk writer.append_column(Some(Ok(&col1[2]))).unwrap(); @@ -1306,7 +1306,7 @@ mod tests { } // This should prune from the ondisk offset list and clear the jar. - let mut writer = NippyJarWriter::new(nippy, false).unwrap(); + let mut writer = NippyJarWriter::new(nippy, ConsistencyFailStrategy::Heal).unwrap(); writer.prune_rows(1).unwrap(); assert_eq!(writer.rows(), 0); assert_eq!(writer.max_row_size(), 0); @@ -1343,6 +1343,6 @@ mod tests { data_file.set_len(data_len - 32 * missing_offsets).unwrap(); // runs the consistency check. - let _ = NippyJarWriter::new(nippy, false).unwrap(); + let _ = NippyJarWriter::new(nippy, ConsistencyFailStrategy::Heal).unwrap(); } } diff --git a/crates/storage/nippy-jar/src/writer.rs b/crates/storage/nippy-jar/src/writer.rs index 1c53c36b643..8993f4c1916 100644 --- a/crates/storage/nippy-jar/src/writer.rs +++ b/crates/storage/nippy-jar/src/writer.rs @@ -46,7 +46,10 @@ impl NippyJarWriter { /// /// If `read_only` is set to `true`, any inconsistency issue won't be healed, and will return /// [NippyJarError::InconsistentState] instead. - pub fn new(jar: NippyJar, read_only: bool) -> Result { + pub fn new( + jar: NippyJar, + check_mode: ConsistencyFailStrategy, + ) -> Result { let (data_file, offsets_file, is_created) = Self::create_or_open_files(jar.data_path(), &jar.offsets_path())?; @@ -66,8 +69,8 @@ impl NippyJarWriter { // If we are opening a previously created jar, we need to check its consistency, and make // changes if necessary. if !is_created { - writer.ensure_file_consistency(read_only)?; - if !read_only { + writer.ensure_file_consistency(check_mode)?; + if check_mode.should_heal() { writer.commit()?; } } @@ -123,17 +126,17 @@ impl NippyJarWriter { Ok((data_file, offsets_file, is_created)) } - /// Performs consistency checks on the [`NippyJar`] file and heals upon any issues if - /// `read_only` is set to false: + /// Performs consistency checks on the [`NippyJar`] file and might self-heal or throw an error + /// according to [ConsistencyFailStrategy]. /// * Is the offsets file size expected? /// * Is the data file size expected? /// /// This is based on the assumption that [`NippyJar`] configuration is **always** the last one /// to be updated when something is written, as by the `commit()` function shows. - /// - /// If `read_only` is set to true and an issue is found it will return a - /// [NippyJarError::InconsistentState] error. - pub fn ensure_file_consistency(&mut self, read_only: bool) -> Result<(), NippyJarError> { + pub fn ensure_file_consistency( + &mut self, + check_mode: ConsistencyFailStrategy, + ) -> Result<(), NippyJarError> { let reader = self.jar.open_data_reader()?; // When an offset size is smaller than the initial (8), we are dealing with immutable @@ -147,7 +150,8 @@ impl NippyJarWriter { OFFSET_SIZE_BYTES as usize) as u64; // expected size of the data file let actual_offsets_file_size = self.offsets_file.get_ref().metadata()?.len(); - if read_only && expected_offsets_file_size.cmp(&actual_offsets_file_size) != Ordering::Equal + if check_mode.should_err() && + expected_offsets_file_size.cmp(&actual_offsets_file_size) != Ordering::Equal { return Err(NippyJarError::InconsistentState) } @@ -179,7 +183,7 @@ impl NippyJarWriter { let last_offset = reader.reverse_offset(0)?; let data_file_len = self.data_file.get_ref().metadata()?.len(); - if read_only && last_offset.cmp(&data_file_len) != Ordering::Equal { + if check_mode.should_err() && last_offset.cmp(&data_file_len) != Ordering::Equal { return Err(NippyJarError::InconsistentState) } @@ -209,7 +213,7 @@ impl NippyJarWriter { // Since we decrease the offset list, we need to check the consistency of // `self.jar.rows` again - self.ensure_file_consistency(false)?; + self.ensure_file_consistency(ConsistencyFailStrategy::Heal)?; break } } @@ -509,3 +513,24 @@ impl NippyJarWriter { &self.jar } } + +/// Strategy on encountering an inconsistent state when creating a [NippyJarWriter]. +#[derive(Debug, Copy, Clone)] +pub enum ConsistencyFailStrategy { + /// Writer should heal. + Heal, + /// Writer should throw an error. + ThrowError, +} + +impl ConsistencyFailStrategy { + /// Whether writer should heal. + fn should_heal(&self) -> bool { + matches!(self, Self::Heal) + } + + /// Whether writer should throw an error. + fn should_err(&self) -> bool { + matches!(self, Self::ThrowError) + } +} diff --git a/crates/storage/provider/src/providers/static_file/writer.rs b/crates/storage/provider/src/providers/static_file/writer.rs index 315683108a0..5a9db9dcdec 100644 --- a/crates/storage/provider/src/providers/static_file/writer.rs +++ b/crates/storage/provider/src/providers/static_file/writer.rs @@ -6,7 +6,7 @@ use super::{ use dashmap::mapref::one::RefMut; use reth_codecs::Compact; use reth_db::codecs::CompactU256; -use reth_nippy_jar::{NippyJar, NippyJarError, NippyJarWriter}; +use reth_nippy_jar::{ConsistencyFailStrategy, NippyJar, NippyJarError, NippyJarWriter}; use reth_primitives::{ static_file::{find_fixed_range, SegmentHeader, SegmentRangeInclusive}, BlockHash, BlockNumber, Header, Receipt, StaticFileSegment, TransactionSignedNoHash, TxNumber, @@ -91,7 +91,13 @@ impl StaticFileProviderRW { }; let reader = Self::upgrade_provider_to_strong_reference(&reader); - let result = match NippyJarWriter::new(jar, reader.is_read_only()) { + let access = if reader.is_read_only() { + ConsistencyFailStrategy::ThrowError + } else { + ConsistencyFailStrategy::Heal + }; + + let result = match NippyJarWriter::new(jar, access) { Ok(writer) => Ok((writer, path)), Err(NippyJarError::FrozenJar) => { // This static file has been frozen, so we should @@ -125,14 +131,20 @@ impl StaticFileProviderRW { ) }; - self.writer.ensure_file_consistency(read_only).map_err(|error| { + let check_mode = if read_only { + ConsistencyFailStrategy::ThrowError + } else { + ConsistencyFailStrategy::Heal + }; + + self.writer.ensure_file_consistency(check_mode).map_err(|error| { if matches!(error, NippyJarError::InconsistentState) { return inconsistent_error() } ProviderError::NippyJar(error.to_string()) })?; - // If we have lost rows, we need to update the [SegmentHeader] + // If we have lost rows (in this run or previous), we need to update the [SegmentHeader]. let expected_rows = if self.user_header().segment().is_headers() { self.user_header().block_len().unwrap_or_default() } else { From 8ff37365ede7b672f376f4982cb82b5b4ce27af5 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Thu, 30 May 2024 17:07:09 +0200 Subject: [PATCH 51/61] use upsert on headers and txlookup --- crates/stages/stages/src/stages/headers.rs | 6 +++++- crates/stages/stages/src/stages/tx_lookup.rs | 6 +++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/crates/stages/stages/src/stages/headers.rs b/crates/stages/stages/src/stages/headers.rs index 6ca02a4aa5d..2ec75d7e988 100644 --- a/crates/stages/stages/src/stages/headers.rs +++ b/crates/stages/stages/src/stages/headers.rs @@ -176,7 +176,11 @@ where RawValue::::from_vec(number), )?; } else { - cursor_header_numbers.insert( + // Upsert usage instead of insert: + // If an interrupted unwind deletes data from static files, but not from + // database, then we will have dangling entries, which would confflict when + // trying to advance again the chain. + cursor_header_numbers.upsert( RawKey::::from_vec(hash), RawValue::::from_vec(number), )?; diff --git a/crates/stages/stages/src/stages/tx_lookup.rs b/crates/stages/stages/src/stages/tx_lookup.rs index 27e8e68cfab..252044f5c4b 100644 --- a/crates/stages/stages/src/stages/tx_lookup.rs +++ b/crates/stages/stages/src/stages/tx_lookup.rs @@ -155,7 +155,11 @@ impl Stage for TransactionLookupStage { if append_only { txhash_cursor.append(key, RawValue::::from_vec(number))? } else { - txhash_cursor.insert(key, RawValue::::from_vec(number))? + // Upsert usage instead of insert: + // If an interrupted unwind deletes data from static files, but not from + // database, then we will have dangling entries, which would confflict when + // trying to advance again the chain. + txhash_cursor.upsert(key, RawValue::::from_vec(number))? } } From 6cd04b24b5724b8177de85058c0f46dcb5810438 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Thu, 30 May 2024 17:15:52 +0200 Subject: [PATCH 52/61] Revert "use upsert on headers and txlookup" This reverts commit 8ff37365ede7b672f376f4982cb82b5b4ce27af5. --- crates/stages/stages/src/stages/headers.rs | 6 +----- crates/stages/stages/src/stages/tx_lookup.rs | 6 +----- 2 files changed, 2 insertions(+), 10 deletions(-) diff --git a/crates/stages/stages/src/stages/headers.rs b/crates/stages/stages/src/stages/headers.rs index 2ec75d7e988..6ca02a4aa5d 100644 --- a/crates/stages/stages/src/stages/headers.rs +++ b/crates/stages/stages/src/stages/headers.rs @@ -176,11 +176,7 @@ where RawValue::::from_vec(number), )?; } else { - // Upsert usage instead of insert: - // If an interrupted unwind deletes data from static files, but not from - // database, then we will have dangling entries, which would confflict when - // trying to advance again the chain. - cursor_header_numbers.upsert( + cursor_header_numbers.insert( RawKey::::from_vec(hash), RawValue::::from_vec(number), )?; diff --git a/crates/stages/stages/src/stages/tx_lookup.rs b/crates/stages/stages/src/stages/tx_lookup.rs index 252044f5c4b..27e8e68cfab 100644 --- a/crates/stages/stages/src/stages/tx_lookup.rs +++ b/crates/stages/stages/src/stages/tx_lookup.rs @@ -155,11 +155,7 @@ impl Stage for TransactionLookupStage { if append_only { txhash_cursor.append(key, RawValue::::from_vec(number))? } else { - // Upsert usage instead of insert: - // If an interrupted unwind deletes data from static files, but not from - // database, then we will have dangling entries, which would confflict when - // trying to advance again the chain. - txhash_cursor.upsert(key, RawValue::::from_vec(number))? + txhash_cursor.insert(key, RawValue::::from_vec(number))? } } From aacbe7e6620225863a611322d06215224d313977 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Thu, 30 May 2024 17:19:36 +0200 Subject: [PATCH 53/61] on unwinds: commit to db first then static files --- crates/stages/api/src/pipeline/mod.rs | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/crates/stages/api/src/pipeline/mod.rs b/crates/stages/api/src/pipeline/mod.rs index 0fe52e29dc9..a7a8b043b58 100644 --- a/crates/stages/api/src/pipeline/mod.rs +++ b/crates/stages/api/src/pipeline/mod.rs @@ -353,8 +353,12 @@ where self.event_sender .notify(PipelineEvent::Unwound { stage_id, result: unwind_output }); - self.provider_factory.static_file_provider().commit()?; + // For unwinds it makes more sense to commit the database first, since if + // the unwind is interrupted before the static file commit, + // we can just truncate the static files according + // to the checkpoints on the next start-up. provider_rw.commit()?; + self.provider_factory.static_file_provider().commit()?; provider_rw = self.provider_factory.provider_rw()?; } From ea0627dc269a4953b50d404a365e799029a7ac8e Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Thu, 30 May 2024 17:25:38 +0200 Subject: [PATCH 54/61] fix doc --- crates/storage/provider/src/providers/static_file/manager.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/crates/storage/provider/src/providers/static_file/manager.rs b/crates/storage/provider/src/providers/static_file/manager.rs index 4fc59b206ef..ec0f16cc7cd 100644 --- a/crates/storage/provider/src/providers/static_file/manager.rs +++ b/crates/storage/provider/src/providers/static_file/manager.rs @@ -662,7 +662,8 @@ impl StaticFileProvider { } // If the checkpoint is behind, then we failed to do a database commit **but committed** to - // static files. All we need to do is to unwind those rows. + // static files on executing a stage, or the reverse on unwinding a stage. + // All we need to do is to prune the extra static file rows. if checkpoint_block_number < highest_static_file_block { info!( target: "reth::providers", From f69483ca50c6ac9972ba8753386fad3445f9fc9d Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Thu, 30 May 2024 17:41:34 +0200 Subject: [PATCH 55/61] fix docs --- .../provider/src/providers/static_file/manager.rs | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/crates/storage/provider/src/providers/static_file/manager.rs b/crates/storage/provider/src/providers/static_file/manager.rs index ec0f16cc7cd..0052868ad9c 100644 --- a/crates/storage/provider/src/providers/static_file/manager.rs +++ b/crates/storage/provider/src/providers/static_file/manager.rs @@ -486,16 +486,17 @@ impl StaticFileProvider { Ok(()) } - /// Ensures that any broken invariants return a pipeline target to unwind to. + /// Ensures that any broken invariants which cannot be healed on the spot return a pipeline + /// target to unwind to. /// /// Two types of consistency checks are done for: /// /// 1) When a static file fails to commit but the underlying data was changed. /// 2) When a static file was committed, but the required database transaction was not. /// - /// For 1) it can self-heal if `read_only` is set to `false`. Otherwise, it will return an - /// error. - /// For 2) the invariants below are checked, and if broken, require a pipeline unwind + /// For 1) it can self-heal if `self.access.is_read_only()` is set to `false`. Otherwise, it + /// will return an error. + /// For 2) the invariants below are checked, and if broken, might require a pipeline unwind /// to heal. /// /// For each static file segment: @@ -504,7 +505,7 @@ impl StaticFileProvider { /// * its highest block should match the stage checkpoint block number if it's equal or higher /// than the corresponding database table last entry. /// - /// Returns a [`Option`] of [`PipelineTarget::Unwind`] if any healing is required. + /// Returns a [`Option`] of [`PipelineTarget::Unwind`] if any healing is further required. /// /// WARNING: No static file writer should be held before calling this function, otherwise it /// will deadlock. @@ -655,8 +656,7 @@ impl StaticFileProvider { .unwrap_or_default() .block_number; - // If the checkpoint is ahead, then we lost static file data. May have been an interrupted - // unwind that committed the static file changes or data corruption. + // If the checkpoint is ahead, then we lost static file data. May be data corruption. if checkpoint_block_number > highest_static_file_block { return Ok(Some(highest_static_file_block)); } From 8dcb970a1e786b196edb2a7282b80cb6a24a5aef Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Fri, 31 May 2024 11:20:25 +0200 Subject: [PATCH 56/61] make tests more complete --- crates/stages/stages/src/stages/mod.rs | 44 +++++++++++++++++--------- 1 file changed, 29 insertions(+), 15 deletions(-) diff --git a/crates/stages/stages/src/stages/mod.rs b/crates/stages/stages/src/stages/mod.rs index 3885e9f12a6..6247aea11c0 100644 --- a/crates/stages/stages/src/stages/mod.rs +++ b/crates/stages/stages/src/stages/mod.rs @@ -62,9 +62,9 @@ mod tests { StaticFileSegment, B256, U256, }; use reth_provider::{ - providers::StaticFileWriter, AccountExtReader, DatabaseProviderFactory, ProviderFactory, - ProviderResult, ReceiptProvider, StageCheckpointWriter, StaticFileProviderFactory, - StorageReader, + providers::StaticFileWriter, AccountExtReader, BlockReader, DatabaseProviderFactory, + ProviderFactory, ProviderResult, ReceiptProvider, StageCheckpointWriter, + StaticFileProviderFactory, StorageReader, }; use reth_stages_api::{ExecInput, Stage}; use reth_testing_utils::generators::{self, random_block, random_block_range, random_receipt}; @@ -279,9 +279,9 @@ mod tests { Ok(db) } - /// Simulates a pruning job that was never committed and compare the check consistency result + /// Simulates losing data to corruption and compare the check consistency result /// against the expected one. - fn simulate_no_commit_prune_and_check( + fn simulate_behind_checkpoint_corruption( db: &TestStageDB, prune_count: usize, segment: StaticFileSegment, @@ -290,8 +290,8 @@ mod tests { ) { let static_file_provider = db.factory.static_file_provider(); - // Simulate pruning by removing `prune_count` rows from the data file without updating its - // offset list and configuration. + // Simulate corruption by removing `prune_count` rows from the data file without updating + // its offset list and configuration. { let mut headers_writer = static_file_provider.latest_writer(segment).unwrap(); let reader = headers_writer.inner().jar().open_data_reader().unwrap(); @@ -313,7 +313,7 @@ mod tests { /// Saves a checkpoint with `checkpoint_block_number` and compare the check consistency result /// against the expected one. fn save_checkpoint_and_check( - db: &TestStageDB, // replace DbType with your actual database type + db: &TestStageDB, stage_id: StageId, checkpoint_block_number: BlockNumber, expected: Option, @@ -373,11 +373,11 @@ mod tests { // Full node does not use receipts, therefore doesn't check for consistency on receipts // segment - simulate_no_commit_prune_and_check(&db, 1, StaticFileSegment::Receipts, full_node, None); + simulate_behind_checkpoint_corruption(&db, 1, StaticFileSegment::Receipts, full_node, None); // there are 2 to 3 transactions per block. however, if we lose one tx, we need to unwind to // the previous block. - simulate_no_commit_prune_and_check( + simulate_behind_checkpoint_corruption( &db, 1, StaticFileSegment::Receipts, @@ -385,7 +385,7 @@ mod tests { Some(PipelineTarget::Unwind(88)), ); - simulate_no_commit_prune_and_check( + simulate_behind_checkpoint_corruption( &db, 3, StaticFileSegment::Headers, @@ -399,20 +399,34 @@ mod tests { let db = seed_data(90).unwrap(); // When a checkpoint is behind, we delete data from static files. - save_checkpoint_and_check(&db, StageId::Bodies, 87, None); + let block = 87; + save_checkpoint_and_check(&db, StageId::Bodies, block, None); assert_eq!( db.factory .static_file_provider() .get_highest_static_file_block(StaticFileSegment::Transactions), - Some(87) + Some(block) + ); + assert_eq!( + db.factory + .static_file_provider() + .get_highest_static_file_tx(StaticFileSegment::Transactions), + db.factory.block_body_indices(block).unwrap().map(|b| b.last_tx_num()) ); - save_checkpoint_and_check(&db, StageId::Execution, 86, None); + let block = 86; + save_checkpoint_and_check(&db, StageId::Execution, block, None); assert_eq!( db.factory .static_file_provider() .get_highest_static_file_block(StaticFileSegment::Receipts), - Some(86) + Some(block) + ); + assert_eq!( + db.factory + .static_file_provider() + .get_highest_static_file_tx(StaticFileSegment::Receipts), + db.factory.block_body_indices(block).unwrap().map(|b| b.last_tx_num()) ); // When a checkpoint is ahead, we request a pipeline unwind. From b1651a586dab6c4a2e0ec770692bd7d3ac215fba Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Fri, 31 May 2024 11:25:34 +0200 Subject: [PATCH 57/61] add header as well to test --- crates/stages/stages/src/stages/mod.rs | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/crates/stages/stages/src/stages/mod.rs b/crates/stages/stages/src/stages/mod.rs index 6247aea11c0..1ecc8ac1e3f 100644 --- a/crates/stages/stages/src/stages/mod.rs +++ b/crates/stages/stages/src/stages/mod.rs @@ -429,8 +429,17 @@ mod tests { db.factory.block_body_indices(block).unwrap().map(|b| b.last_tx_num()) ); + let block = 80; + save_checkpoint_and_check(&db, StageId::Headers, block, None); + assert_eq!( + db.factory + .static_file_provider() + .get_highest_static_file_block(StaticFileSegment::Headers), + Some(block) + ); + // When a checkpoint is ahead, we request a pipeline unwind. - save_checkpoint_and_check(&db, StageId::Headers, 91, Some(PipelineTarget::Unwind(89))); + save_checkpoint_and_check(&db, StageId::Headers, 91, Some(PipelineTarget::Unwind(block))); } #[test] From fcdf452a7b360dcb46475931ed4e052442b85ddb Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Fri, 31 May 2024 11:28:50 +0200 Subject: [PATCH 58/61] add more docs --- crates/stages/api/src/pipeline/mod.rs | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/crates/stages/api/src/pipeline/mod.rs b/crates/stages/api/src/pipeline/mod.rs index a7a8b043b58..6b7b65b9b89 100644 --- a/crates/stages/api/src/pipeline/mod.rs +++ b/crates/stages/api/src/pipeline/mod.rs @@ -353,10 +353,10 @@ where self.event_sender .notify(PipelineEvent::Unwound { stage_id, result: unwind_output }); - // For unwinds it makes more sense to commit the database first, since if - // the unwind is interrupted before the static file commit, - // we can just truncate the static files according - // to the checkpoints on the next start-up. + // For unwinding it makes more sense to commit the database first, since if + // this function is interrupted before the static files commit, we can just + // truncate the static files according to the + // checkpoints on the next start-up. provider_rw.commit()?; self.provider_factory.static_file_provider().commit()?; @@ -463,6 +463,10 @@ where result: out.clone(), }); + // For execution it makes more sense to commit the static files first, since if + // this function is interrupted before the database commit, we can just truncate + // the static files according to the checkpoints on the next + // start-up. self.provider_factory.static_file_provider().commit()?; provider_rw.commit()?; From 938f2caef4a74afd6341e8c8895bffc163e53c48 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Fri, 31 May 2024 12:18:14 +0200 Subject: [PATCH 59/61] make sure to update last receipt block if unwinding empty blocks --- crates/stages/stages/src/stages/execution.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/crates/stages/stages/src/stages/execution.rs b/crates/stages/stages/src/stages/execution.rs index 2388d0d6ff9..e76b210925d 100644 --- a/crates/stages/stages/src/stages/execution.rs +++ b/crates/stages/stages/src/stages/execution.rs @@ -606,7 +606,9 @@ where // Check if we had any unexpected shutdown after committing to static files, but // NOT committing to database. match next_static_file_receipt_num.cmp(&next_receipt_num) { - Ordering::Greater => static_file_producer.prune_receipts( + // It can be equal when it's a chain of empty blocks, but we still need to update the last + // block in the range. + Ordering::Greater | Ordering::Equal => static_file_producer.prune_receipts( next_static_file_receipt_num - next_receipt_num, start_block.saturating_sub(1), )?, @@ -641,7 +643,6 @@ where segment: StaticFileSegment::Receipts, }) } - Ordering::Equal => {} } Ok(static_file_producer) From e721388e1ede236a3a5fac0021e0b4aec0857829 Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Fri, 31 May 2024 12:25:24 +0200 Subject: [PATCH 60/61] fix edge case on unwinding empty blocks across files --- .../src/providers/static_file/writer.rs | 63 ++++++++++++------- 1 file changed, 40 insertions(+), 23 deletions(-) diff --git a/crates/storage/provider/src/providers/static_file/writer.rs b/crates/storage/provider/src/providers/static_file/writer.rs index 4448a3b1f4f..953c8283433 100644 --- a/crates/storage/provider/src/providers/static_file/writer.rs +++ b/crates/storage/provider/src/providers/static_file/writer.rs @@ -340,10 +340,11 @@ impl StaticFileProviderRW { fn truncate( &mut self, segment: StaticFileSegment, - mut num_rows: u64, + truncated_rows: u64, last_block: Option, ) -> ProviderResult<()> { - while num_rows > 0 { + let mut remaining_rows = truncated_rows; + while remaining_rows > 0 { let len = match segment { StaticFileSegment::Headers => { self.writer.user_header().block_len().unwrap_or_default() @@ -353,26 +354,13 @@ impl StaticFileProviderRW { } }; - if num_rows >= len { + if remaining_rows >= len { // If there's more rows to delete than this static file contains, then just // delete the whole file and go to the next static file - let previous_snap = self.data_path.clone(); let block_start = self.writer.user_header().expected_block_start(); if block_start != 0 { - let (writer, data_path) = Self::open( - segment, - self.writer.user_header().expected_block_start() - 1, - self.reader.clone(), - self.metrics.clone(), - )?; - self.writer = writer; - self.data_path = data_path; - - NippyJar::::load(&previous_snap) - .map_err(|e| ProviderError::NippyJar(e.to_string()))? - .delete() - .map_err(|e| ProviderError::NippyJar(e.to_string()))?; + self.delete_current_and_open_previous()?; } else { // Update `SegmentHeader` self.writer.user_header_mut().prune(len); @@ -382,23 +370,33 @@ impl StaticFileProviderRW { break } - num_rows -= len; + remaining_rows -= len; } else { // Update `SegmentHeader` - self.writer.user_header_mut().prune(num_rows); + self.writer.user_header_mut().prune(remaining_rows); // Truncate data self.writer - .prune_rows(num_rows as usize) + .prune_rows(remaining_rows as usize) .map_err(|e| ProviderError::NippyJar(e.to_string()))?; - num_rows = 0; + remaining_rows = 0; } } // Only Transactions and Receipts if let Some(last_block) = last_block { - let header = self.writer.user_header_mut(); - header.set_block_range(header.expected_block_start(), last_block); + let mut expected_block_start = self.writer.user_header().expected_block_start(); + + if truncated_rows == 0 { + // Edge case for when we are unwinding a chain of empty blocks that goes across + // files, and therefore, the only reference point to know which file + // we are supposed to be at is `last_block`. + while last_block < expected_block_start { + self.delete_current_and_open_previous()?; + expected_block_start = self.writer.user_header().expected_block_start(); + } + } + self.writer.user_header_mut().set_block_range(expected_block_start, last_block); } // Commits new changes to disk. @@ -407,6 +405,25 @@ impl StaticFileProviderRW { Ok(()) } + /// Delete the current static file, and replace this provider writer with the previous static + /// file. + fn delete_current_and_open_previous(&mut self) -> Result<(), ProviderError> { + let current_path = self.data_path.clone(); + let (previous_writer, data_path) = Self::open( + self.user_header().segment(), + self.writer.user_header().expected_block_start() - 1, + self.reader.clone(), + self.metrics.clone(), + )?; + self.writer = previous_writer; + self.data_path = data_path; + NippyJar::::load(¤t_path) + .map_err(|e| ProviderError::NippyJar(e.to_string()))? + .delete() + .map_err(|e| ProviderError::NippyJar(e.to_string()))?; + Ok(()) + } + /// Appends column to static file. fn append_column(&mut self, column: T) -> ProviderResult<()> { self.buf.clear(); From 8f6bb248c60dfec0dc3243c2657660c37bd39f8e Mon Sep 17 00:00:00 2001 From: joshieDo <93316087+joshieDo@users.noreply.github.com> Date: Fri, 31 May 2024 14:42:16 +0200 Subject: [PATCH 61/61] rename back to num_rows --- crates/storage/provider/src/providers/static_file/writer.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/crates/storage/provider/src/providers/static_file/writer.rs b/crates/storage/provider/src/providers/static_file/writer.rs index 953c8283433..3b88812da7f 100644 --- a/crates/storage/provider/src/providers/static_file/writer.rs +++ b/crates/storage/provider/src/providers/static_file/writer.rs @@ -340,10 +340,10 @@ impl StaticFileProviderRW { fn truncate( &mut self, segment: StaticFileSegment, - truncated_rows: u64, + num_rows: u64, last_block: Option, ) -> ProviderResult<()> { - let mut remaining_rows = truncated_rows; + let mut remaining_rows = num_rows; while remaining_rows > 0 { let len = match segment { StaticFileSegment::Headers => { @@ -387,7 +387,7 @@ impl StaticFileProviderRW { if let Some(last_block) = last_block { let mut expected_block_start = self.writer.user_header().expected_block_start(); - if truncated_rows == 0 { + if num_rows == 0 { // Edge case for when we are unwinding a chain of empty blocks that goes across // files, and therefore, the only reference point to know which file // we are supposed to be at is `last_block`.