reth_cli_commands/
import_core.rs

1//! Core import functionality without CLI dependencies.
2
3use alloy_primitives::B256;
4use futures::StreamExt;
5use reth_config::Config;
6use reth_consensus::FullConsensus;
7use reth_db_api::{tables, transaction::DbTx};
8use reth_downloaders::{
9    bodies::bodies::BodiesDownloaderBuilder,
10    file_client::{ChunkedFileReader, FileClient, DEFAULT_BYTE_LEN_CHUNK_CHAIN_FILE},
11    headers::reverse_headers::ReverseHeadersDownloaderBuilder,
12};
13use reth_evm::ConfigureEvm;
14use reth_network_p2p::{
15    bodies::downloader::BodyDownloader,
16    headers::downloader::{HeaderDownloader, SyncTarget},
17};
18use reth_node_api::BlockTy;
19use reth_node_events::node::NodeEvent;
20use reth_provider::{
21    providers::ProviderNodeTypes, BlockNumReader, HeaderProvider, ProviderError, ProviderFactory,
22    StageCheckpointReader,
23};
24use reth_prune::PruneModes;
25use reth_stages::{prelude::*, Pipeline, StageId, StageSet};
26use reth_static_file::StaticFileProducer;
27use std::{path::Path, sync::Arc};
28use tokio::sync::watch;
29use tracing::{debug, error, info};
30
31/// Configuration for importing blocks from RLP files.
32#[derive(Debug, Clone, Default)]
33pub struct ImportConfig {
34    /// Disables stages that require state.
35    pub no_state: bool,
36    /// Chunk byte length to read from file.
37    pub chunk_len: Option<u64>,
38}
39
40/// Result of an import operation.
41#[derive(Debug)]
42pub struct ImportResult {
43    /// Total number of blocks decoded from the file.
44    pub total_decoded_blocks: usize,
45    /// Total number of transactions decoded from the file.
46    pub total_decoded_txns: usize,
47    /// Total number of blocks imported into the database.
48    pub total_imported_blocks: usize,
49    /// Total number of transactions imported into the database.
50    pub total_imported_txns: usize,
51}
52
53impl ImportResult {
54    /// Returns true if all blocks and transactions were imported successfully.
55    pub fn is_complete(&self) -> bool {
56        self.total_decoded_blocks == self.total_imported_blocks &&
57            self.total_decoded_txns == self.total_imported_txns
58    }
59}
60
61/// Imports blocks from an RLP-encoded file into the database.
62///
63/// This function reads RLP-encoded blocks from a file in chunks and imports them
64/// using the pipeline infrastructure. It's designed to be used both from the CLI
65/// and from test code.
66pub async fn import_blocks_from_file<N>(
67    path: &Path,
68    import_config: ImportConfig,
69    provider_factory: ProviderFactory<N>,
70    config: &Config,
71    executor: impl ConfigureEvm<Primitives = N::Primitives> + 'static,
72    consensus: Arc<
73        impl FullConsensus<N::Primitives, Error = reth_consensus::ConsensusError> + 'static,
74    >,
75) -> eyre::Result<ImportResult>
76where
77    N: ProviderNodeTypes,
78{
79    if import_config.no_state {
80        info!(target: "reth::import", "Disabled stages requiring state");
81    }
82
83    debug!(target: "reth::import",
84        chunk_byte_len=import_config.chunk_len.unwrap_or(DEFAULT_BYTE_LEN_CHUNK_CHAIN_FILE),
85        "Chunking chain import"
86    );
87
88    info!(target: "reth::import", "Consensus engine initialized");
89
90    // open file
91    let mut reader = ChunkedFileReader::new(path, import_config.chunk_len).await?;
92
93    let provider = provider_factory.provider()?;
94    let init_blocks = provider.tx_ref().entries::<tables::HeaderNumbers>()?;
95    let init_txns = provider.tx_ref().entries::<tables::TransactionHashNumbers>()?;
96    drop(provider);
97
98    let mut total_decoded_blocks = 0;
99    let mut total_decoded_txns = 0;
100
101    let mut sealed_header = provider_factory
102        .sealed_header(provider_factory.last_block_number()?)?
103        .expect("should have genesis");
104
105    while let Some(file_client) =
106        reader.next_chunk::<BlockTy<N>>(consensus.clone(), Some(sealed_header)).await?
107    {
108        // create a new FileClient from chunk read from file
109        info!(target: "reth::import",
110            "Importing chain file chunk"
111        );
112
113        let tip = file_client.tip().ok_or(eyre::eyre!("file client has no tip"))?;
114        info!(target: "reth::import", "Chain file chunk read");
115
116        total_decoded_blocks += file_client.headers_len();
117        total_decoded_txns += file_client.total_transactions();
118
119        let (mut pipeline, events) = build_import_pipeline_impl(
120            config,
121            provider_factory.clone(),
122            &consensus,
123            Arc::new(file_client),
124            StaticFileProducer::new(provider_factory.clone(), PruneModes::default()),
125            import_config.no_state,
126            executor.clone(),
127        )?;
128
129        // override the tip
130        pipeline.set_tip(tip);
131        debug!(target: "reth::import", ?tip, "Tip manually set");
132
133        let latest_block_number =
134            provider_factory.get_stage_checkpoint(StageId::Finish)?.map(|ch| ch.block_number);
135        tokio::spawn(reth_node_events::node::handle_events(None, latest_block_number, events));
136
137        // Run pipeline
138        info!(target: "reth::import", "Starting sync pipeline");
139        tokio::select! {
140            res = pipeline.run() => res?,
141            _ = tokio::signal::ctrl_c() => {
142                info!(target: "reth::import", "Import interrupted by user");
143                break;
144            },
145        }
146
147        sealed_header = provider_factory
148            .sealed_header(provider_factory.last_block_number()?)?
149            .expect("should have genesis");
150    }
151
152    let provider = provider_factory.provider()?;
153    let total_imported_blocks = provider.tx_ref().entries::<tables::HeaderNumbers>()? - init_blocks;
154    let total_imported_txns =
155        provider.tx_ref().entries::<tables::TransactionHashNumbers>()? - init_txns;
156
157    let result = ImportResult {
158        total_decoded_blocks,
159        total_decoded_txns,
160        total_imported_blocks,
161        total_imported_txns,
162    };
163
164    if !result.is_complete() {
165        error!(target: "reth::import",
166            total_decoded_blocks,
167            total_imported_blocks,
168            total_decoded_txns,
169            total_imported_txns,
170            "Chain was partially imported"
171        );
172    } else {
173        info!(target: "reth::import",
174            total_imported_blocks,
175            total_imported_txns,
176            "Chain was fully imported"
177        );
178    }
179
180    Ok(result)
181}
182
183/// Builds import pipeline.
184///
185/// If configured to execute, all stages will run. Otherwise, only stages that don't require state
186/// will run.
187pub fn build_import_pipeline_impl<N, C, E>(
188    config: &Config,
189    provider_factory: ProviderFactory<N>,
190    consensus: &Arc<C>,
191    file_client: Arc<FileClient<BlockTy<N>>>,
192    static_file_producer: StaticFileProducer<ProviderFactory<N>>,
193    disable_exec: bool,
194    evm_config: E,
195) -> eyre::Result<(Pipeline<N>, impl futures::Stream<Item = NodeEvent<N::Primitives>>)>
196where
197    N: ProviderNodeTypes,
198    C: FullConsensus<N::Primitives, Error = reth_consensus::ConsensusError> + 'static,
199    E: ConfigureEvm<Primitives = N::Primitives> + 'static,
200{
201    if !file_client.has_canonical_blocks() {
202        eyre::bail!("unable to import non canonical blocks");
203    }
204
205    // Retrieve latest header found in the database.
206    let last_block_number = provider_factory.last_block_number()?;
207    let local_head = provider_factory
208        .sealed_header(last_block_number)?
209        .ok_or_else(|| ProviderError::HeaderNotFound(last_block_number.into()))?;
210
211    let mut header_downloader = ReverseHeadersDownloaderBuilder::new(config.stages.headers)
212        .build(file_client.clone(), consensus.clone())
213        .into_task();
214    // TODO: The pipeline should correctly configure the downloader on its own.
215    // Find the possibility to remove unnecessary pre-configuration.
216    header_downloader.update_local_head(local_head);
217    header_downloader.update_sync_target(SyncTarget::Tip(file_client.tip().unwrap()));
218
219    let mut body_downloader = BodiesDownloaderBuilder::new(config.stages.bodies)
220        .build(file_client.clone(), consensus.clone(), provider_factory.clone())
221        .into_task();
222    // TODO: The pipeline should correctly configure the downloader on its own.
223    // Find the possibility to remove unnecessary pre-configuration.
224    body_downloader
225        .set_download_range(file_client.min_block().unwrap()..=file_client.max_block().unwrap())
226        .expect("failed to set download range");
227
228    let (tip_tx, tip_rx) = watch::channel(B256::ZERO);
229
230    let max_block = file_client.max_block().unwrap_or(0);
231
232    let pipeline = Pipeline::builder()
233        .with_tip_sender(tip_tx)
234        // we want to sync all blocks the file client provides or 0 if empty
235        .with_max_block(max_block)
236        .with_fail_on_unwind(true)
237        .add_stages(
238            DefaultStages::new(
239                provider_factory.clone(),
240                tip_rx,
241                consensus.clone(),
242                header_downloader,
243                body_downloader,
244                evm_config,
245                config.stages.clone(),
246                PruneModes::default(),
247                None,
248            )
249            .builder()
250            .disable_all_if(&StageId::STATE_REQUIRED, || disable_exec),
251        )
252        .build(provider_factory, static_file_producer);
253
254    let events = pipeline.events().map(Into::into);
255
256    Ok((pipeline, events))
257}