reth_stages/stages/
execution.rs

1use crate::stages::MERKLE_STAGE_DEFAULT_CLEAN_THRESHOLD;
2use alloy_consensus::{BlockHeader, Header, Sealable};
3use alloy_eips::{eip1898::BlockWithParent, NumHash};
4use alloy_primitives::BlockNumber;
5use num_traits::Zero;
6use reth_config::config::ExecutionConfig;
7use reth_db::{static_file::HeaderMask, tables};
8use reth_db_api::{cursor::DbCursorRO, transaction::DbTx};
9use reth_evm::{
10    execute::{BatchExecutor, BlockExecutorProvider},
11    metrics::ExecutorMetrics,
12};
13use reth_execution_types::Chain;
14use reth_exex::{ExExManagerHandle, ExExNotification, ExExNotificationSource};
15use reth_primitives::StaticFileSegment;
16use reth_primitives_traits::{format_gas_throughput, Block, BlockBody, NodePrimitives};
17use reth_provider::{
18    providers::{StaticFileProvider, StaticFileWriter},
19    BlockHashReader, BlockReader, DBProvider, HeaderProvider, LatestStateProviderRef,
20    OriginalValuesKnown, ProviderError, StateCommitmentProvider, StateWriter,
21    StaticFileProviderFactory, StatsReader, StorageLocation, TransactionVariant,
22};
23use reth_prune_types::PruneModes;
24use reth_revm::database::StateProviderDatabase;
25use reth_stages_api::{
26    BlockErrorKind, CheckpointBlockRange, EntitiesCheckpoint, ExecInput, ExecOutput,
27    ExecutionCheckpoint, ExecutionStageThresholds, Stage, StageCheckpoint, StageError, StageId,
28    UnwindInput, UnwindOutput,
29};
30use std::{
31    cmp::Ordering,
32    ops::RangeInclusive,
33    sync::Arc,
34    task::{ready, Context, Poll},
35    time::{Duration, Instant},
36};
37use tracing::*;
38
39use super::missing_static_data_error;
40
41/// The execution stage executes all transactions and
42/// update history indexes.
43///
44/// Input tables:
45/// - [`tables::CanonicalHeaders`] get next block to execute.
46/// - [`tables::Headers`] get for revm environment variables.
47/// - [`tables::HeaderTerminalDifficulties`]
48/// - [`tables::BlockBodyIndices`] to get tx number
49/// - [`tables::Transactions`] to execute
50///
51/// For state access [`LatestStateProviderRef`] provides us latest state and history state
52/// For latest most recent state [`LatestStateProviderRef`] would need (Used for execution Stage):
53/// - [`tables::PlainAccountState`]
54/// - [`tables::Bytecodes`]
55/// - [`tables::PlainStorageState`]
56///
57/// Tables updated after state finishes execution:
58/// - [`tables::PlainAccountState`]
59/// - [`tables::PlainStorageState`]
60/// - [`tables::Bytecodes`]
61/// - [`tables::AccountChangeSets`]
62/// - [`tables::StorageChangeSets`]
63///
64/// For unwinds we are accessing:
65/// - [`tables::BlockBodyIndices`] get tx index to know what needs to be unwinded
66/// - [`tables::AccountsHistory`] to remove change set and apply old values to
67/// - [`tables::PlainAccountState`] [`tables::StoragesHistory`] to remove change set and apply old
68///   values to [`tables::PlainStorageState`]
69// false positive, we cannot derive it if !DB: Debug.
70#[allow(missing_debug_implementations)]
71pub struct ExecutionStage<E>
72where
73    E: BlockExecutorProvider,
74{
75    /// The stage's internal block executor
76    executor_provider: E,
77    /// The commit thresholds of the execution stage.
78    thresholds: ExecutionStageThresholds,
79    /// The highest threshold (in number of blocks) for switching between incremental
80    /// and full calculations across [`super::MerkleStage`], [`super::AccountHashingStage`] and
81    /// [`super::StorageHashingStage`]. This is required to figure out if can prune or not
82    /// changesets on subsequent pipeline runs.
83    external_clean_threshold: u64,
84    /// Pruning configuration.
85    prune_modes: PruneModes,
86    /// Input for the post execute commit hook.
87    /// Set after every [`ExecutionStage::execute`] and cleared after
88    /// [`ExecutionStage::post_execute_commit`].
89    post_execute_commit_input: Option<Chain<E::Primitives>>,
90    /// Input for the post unwind commit hook.
91    /// Set after every [`ExecutionStage::unwind`] and cleared after
92    /// [`ExecutionStage::post_unwind_commit`].
93    post_unwind_commit_input: Option<Chain<E::Primitives>>,
94    /// Handle to communicate with `ExEx` manager.
95    exex_manager_handle: ExExManagerHandle<E::Primitives>,
96    /// Executor metrics.
97    metrics: ExecutorMetrics,
98}
99
100impl<E> ExecutionStage<E>
101where
102    E: BlockExecutorProvider,
103{
104    /// Create new execution stage with specified config.
105    pub fn new(
106        executor_provider: E,
107        thresholds: ExecutionStageThresholds,
108        external_clean_threshold: u64,
109        prune_modes: PruneModes,
110        exex_manager_handle: ExExManagerHandle<E::Primitives>,
111    ) -> Self {
112        Self {
113            external_clean_threshold,
114            executor_provider,
115            thresholds,
116            prune_modes,
117            post_execute_commit_input: None,
118            post_unwind_commit_input: None,
119            exex_manager_handle,
120            metrics: ExecutorMetrics::default(),
121        }
122    }
123
124    /// Create an execution stage with the provided executor.
125    ///
126    /// The commit threshold will be set to [`MERKLE_STAGE_DEFAULT_CLEAN_THRESHOLD`].
127    pub fn new_with_executor(executor_provider: E) -> Self {
128        Self::new(
129            executor_provider,
130            ExecutionStageThresholds::default(),
131            MERKLE_STAGE_DEFAULT_CLEAN_THRESHOLD,
132            PruneModes::none(),
133            ExExManagerHandle::empty(),
134        )
135    }
136
137    /// Create new instance of [`ExecutionStage`] from configuration.
138    pub fn from_config(
139        executor_provider: E,
140        config: ExecutionConfig,
141        external_clean_threshold: u64,
142        prune_modes: PruneModes,
143    ) -> Self {
144        Self::new(
145            executor_provider,
146            config.into(),
147            external_clean_threshold,
148            prune_modes,
149            ExExManagerHandle::empty(),
150        )
151    }
152
153    /// Adjusts the prune modes related to changesets.
154    ///
155    /// This function verifies whether the [`super::MerkleStage`] or Hashing stages will run from
156    /// scratch. If at least one stage isn't starting anew, it implies that pruning of
157    /// changesets cannot occur. This is determined by checking the highest clean threshold
158    /// (`self.external_clean_threshold`) across the stages.
159    ///
160    /// Given that `start_block` changes with each checkpoint, it's necessary to inspect
161    /// [`tables::AccountsTrie`] to ensure that [`super::MerkleStage`] hasn't
162    /// been previously executed.
163    fn adjust_prune_modes(
164        &self,
165        provider: impl StatsReader,
166        start_block: u64,
167        max_block: u64,
168    ) -> Result<PruneModes, StageError> {
169        let mut prune_modes = self.prune_modes.clone();
170
171        // If we're not executing MerkleStage from scratch (by threshold or first-sync), then erase
172        // changeset related pruning configurations
173        if !(max_block - start_block > self.external_clean_threshold ||
174            provider.count_entries::<tables::AccountsTrie>()?.is_zero())
175        {
176            prune_modes.account_history = None;
177            prune_modes.storage_history = None;
178        }
179        Ok(prune_modes)
180    }
181
182    /// Performs consistency check on static files.
183    ///
184    /// This function compares the highest receipt number recorded in the database with that in the
185    /// static file to detect any discrepancies due to unexpected shutdowns or database rollbacks.
186    /// **If the height in the static file is higher**, it rolls back (unwinds) the static file.
187    /// **Conversely, if the height in the database is lower**, it triggers a rollback in the
188    /// database (by returning [`StageError`]) until the heights in both the database and static
189    /// file match.
190    fn ensure_consistency<Provider>(
191        &self,
192        provider: &Provider,
193        checkpoint: u64,
194        unwind_to: Option<u64>,
195    ) -> Result<(), StageError>
196    where
197        Provider: StaticFileProviderFactory + DBProvider + BlockReader + HeaderProvider,
198    {
199        // If thre's any receipts pruning configured, receipts are written directly to database and
200        // inconsistencies are expected.
201        if self.prune_modes.has_receipts_pruning() {
202            return Ok(())
203        }
204
205        // Get next expected receipt number
206        let tx = provider.tx_ref();
207        let next_receipt_num = tx
208            .cursor_read::<tables::BlockBodyIndices>()?
209            .seek_exact(checkpoint)?
210            .map(|(_, value)| value.next_tx_num())
211            .unwrap_or(0);
212
213        let static_file_provider = provider.static_file_provider();
214
215        // Get next expected receipt number in static files
216        let next_static_file_receipt_num = static_file_provider
217            .get_highest_static_file_tx(StaticFileSegment::Receipts)
218            .map(|num| num + 1)
219            .unwrap_or(0);
220
221        // Check if we had any unexpected shutdown after committing to static files, but
222        // NOT committing to database.
223        match next_static_file_receipt_num.cmp(&next_receipt_num) {
224            // It can be equal when it's a chain of empty blocks, but we still need to update the
225            // last block in the range.
226            Ordering::Greater | Ordering::Equal => {
227                let mut static_file_producer =
228                    static_file_provider.latest_writer(StaticFileSegment::Receipts)?;
229                static_file_producer
230                    .prune_receipts(next_static_file_receipt_num - next_receipt_num, checkpoint)?;
231                // Since this is a database <-> static file inconsistency, we commit the change
232                // straight away.
233                static_file_producer.commit()?;
234            }
235            Ordering::Less => {
236                // If we are already in the process of unwind, this might be fine because we will
237                // fix the inconsistency right away.
238                if let Some(unwind_to) = unwind_to {
239                    let next_receipt_num_after_unwind = provider
240                        .tx_ref()
241                        .get::<tables::BlockBodyIndices>(unwind_to)?
242                        .map(|b| b.next_tx_num())
243                        .ok_or(ProviderError::BlockBodyIndicesNotFound(unwind_to))?;
244
245                    if next_receipt_num_after_unwind > next_static_file_receipt_num {
246                        // This means we need a deeper unwind.
247                    } else {
248                        return Ok(())
249                    }
250                }
251
252                return Err(missing_static_data_error(
253                    next_static_file_receipt_num.saturating_sub(1),
254                    &static_file_provider,
255                    provider,
256                    StaticFileSegment::Receipts,
257                )?)
258            }
259        }
260
261        Ok(())
262    }
263}
264
265impl<E, Provider> Stage<Provider> for ExecutionStage<E>
266where
267    E: BlockExecutorProvider,
268    Provider: DBProvider
269        + BlockReader<
270            Block = <E::Primitives as NodePrimitives>::Block,
271            Header = <E::Primitives as NodePrimitives>::BlockHeader,
272        > + StaticFileProviderFactory
273        + StatsReader
274        + BlockHashReader
275        + StateWriter<Receipt = <E::Primitives as NodePrimitives>::Receipt>
276        + StateCommitmentProvider,
277{
278    /// Return the id of the stage
279    fn id(&self) -> StageId {
280        StageId::Execution
281    }
282
283    fn poll_execute_ready(
284        &mut self,
285        cx: &mut Context<'_>,
286        _: ExecInput,
287    ) -> Poll<Result<(), StageError>> {
288        ready!(self.exex_manager_handle.poll_ready(cx));
289
290        Poll::Ready(Ok(()))
291    }
292
293    /// Execute the stage
294    fn execute(&mut self, provider: &Provider, input: ExecInput) -> Result<ExecOutput, StageError> {
295        if input.target_reached() {
296            return Ok(ExecOutput::done(input.checkpoint()))
297        }
298
299        let start_block = input.next_block();
300        let max_block = input.target();
301        let prune_modes = self.adjust_prune_modes(provider, start_block, max_block)?;
302        let static_file_provider = provider.static_file_provider();
303
304        self.ensure_consistency(provider, input.checkpoint().block_number, None)?;
305
306        let db = StateProviderDatabase(LatestStateProviderRef::new(provider));
307        let mut executor = self.executor_provider.batch_executor(db);
308        executor.set_tip(max_block);
309        executor.set_prune_modes(prune_modes);
310
311        // Progress tracking
312        let mut stage_progress = start_block;
313        let mut stage_checkpoint = execution_checkpoint(
314            &static_file_provider,
315            start_block,
316            max_block,
317            input.checkpoint(),
318        )?;
319
320        let mut fetch_block_duration = Duration::default();
321        let mut execution_duration = Duration::default();
322
323        let mut last_block = start_block;
324        let mut last_execution_duration = Duration::default();
325        let mut last_cumulative_gas = 0;
326        let mut last_log_instant = Instant::now();
327        let log_duration = Duration::from_secs(10);
328
329        debug!(target: "sync::stages::execution", start = start_block, end = max_block, "Executing range");
330
331        // Execute block range
332        let mut cumulative_gas = 0;
333        let batch_start = Instant::now();
334
335        let mut blocks = Vec::new();
336        for block_number in start_block..=max_block {
337            // Fetch the block
338            let fetch_block_start = Instant::now();
339
340            let td = provider
341                .header_td_by_number(block_number)?
342                .ok_or_else(|| ProviderError::HeaderNotFound(block_number.into()))?;
343
344            // we need the block's transactions but we don't need the transaction hashes
345            let block = provider
346                .block_with_senders(block_number.into(), TransactionVariant::NoHash)?
347                .ok_or_else(|| ProviderError::HeaderNotFound(block_number.into()))?;
348
349            fetch_block_duration += fetch_block_start.elapsed();
350
351            cumulative_gas += block.header().gas_used();
352
353            // Configure the executor to use the current state.
354            trace!(target: "sync::stages::execution", number = block_number, txs = block.body().transactions().len(), "Executing block");
355
356            // Execute the block
357            let execute_start = Instant::now();
358
359            self.metrics.metered_one((&block, td).into(), |input| {
360                executor.execute_and_verify_one(input).map_err(|error| {
361                    let header = block.header();
362                    StageError::Block {
363                        block: Box::new(BlockWithParent::new(
364                            header.parent_hash(),
365                            NumHash::new(header.number(), header.hash_slow()),
366                        )),
367                        error: BlockErrorKind::Execution(error),
368                    }
369                })
370            })?;
371
372            execution_duration += execute_start.elapsed();
373
374            // Log execution throughput
375            if last_log_instant.elapsed() >= log_duration {
376                info!(
377                    target: "sync::stages::execution",
378                    start = last_block,
379                    end = block_number,
380                    throughput = format_gas_throughput(cumulative_gas - last_cumulative_gas, execution_duration - last_execution_duration),
381                    "Executed block range"
382                );
383
384                last_block = block_number + 1;
385                last_execution_duration = execution_duration;
386                last_cumulative_gas = cumulative_gas;
387                last_log_instant = Instant::now();
388            }
389
390            stage_progress = block_number;
391            stage_checkpoint.progress.processed += block.header().gas_used();
392
393            // If we have ExExes we need to save the block in memory for later
394            if self.exex_manager_handle.has_exexs() {
395                blocks.push(block);
396            }
397
398            // Check if we should commit now
399            let bundle_size_hint = executor.size_hint().unwrap_or_default() as u64;
400            if self.thresholds.is_end_of_batch(
401                block_number - start_block,
402                bundle_size_hint,
403                cumulative_gas,
404                batch_start.elapsed(),
405            ) {
406                break
407            }
408        }
409
410        // prepare execution output for writing
411        let time = Instant::now();
412        let state = executor.finalize();
413        let write_preparation_duration = time.elapsed();
414
415        // log the gas per second for the range we just executed
416        debug!(
417            target: "sync::stages::execution",
418            start = start_block,
419            end = stage_progress,
420            throughput = format_gas_throughput(cumulative_gas, execution_duration),
421            "Finished executing block range"
422        );
423
424        // Prepare the input for post execute commit hook, where an `ExExNotification` will be sent.
425        //
426        // Note: Since we only write to `blocks` if there are any ExExes, we don't need to perform
427        // the `has_exexs` check here as well
428        if !blocks.is_empty() {
429            let blocks = blocks.into_iter().map(|block| {
430                let hash = block.header().hash_slow();
431                block.seal(hash)
432            });
433
434            let previous_input =
435                self.post_execute_commit_input.replace(Chain::new(blocks, state.clone(), None));
436
437            if previous_input.is_some() {
438                // Not processing the previous post execute commit input is a critical error, as it
439                // means that we didn't send the notification to ExExes
440                return Err(StageError::PostExecuteCommit(
441                    "Previous post execute commit input wasn't processed",
442                ))
443            }
444        }
445
446        let time = Instant::now();
447
448        // write output
449        provider.write_state(state, OriginalValuesKnown::Yes, StorageLocation::StaticFiles)?;
450
451        let db_write_duration = time.elapsed();
452        debug!(
453            target: "sync::stages::execution",
454            block_fetch = ?fetch_block_duration,
455            execution = ?execution_duration,
456            write_preparation = ?write_preparation_duration,
457            write = ?db_write_duration,
458            "Execution time"
459        );
460
461        let done = stage_progress == max_block;
462        Ok(ExecOutput {
463            checkpoint: StageCheckpoint::new(stage_progress)
464                .with_execution_stage_checkpoint(stage_checkpoint),
465            done,
466        })
467    }
468
469    fn post_execute_commit(&mut self) -> Result<(), StageError> {
470        let Some(chain) = self.post_execute_commit_input.take() else { return Ok(()) };
471
472        // NOTE: We can ignore the error here, since an error means that the channel is closed,
473        // which means the manager has died, which then in turn means the node is shutting down.
474        let _ = self.exex_manager_handle.send(
475            ExExNotificationSource::Pipeline,
476            ExExNotification::ChainCommitted { new: Arc::new(chain) },
477        );
478
479        Ok(())
480    }
481
482    /// Unwind the stage.
483    fn unwind(
484        &mut self,
485        provider: &Provider,
486        input: UnwindInput,
487    ) -> Result<UnwindOutput, StageError> {
488        let (range, unwind_to, _) =
489            input.unwind_block_range_with_threshold(self.thresholds.max_blocks.unwrap_or(u64::MAX));
490        if range.is_empty() {
491            return Ok(UnwindOutput {
492                checkpoint: input.checkpoint.with_block_number(input.unwind_to),
493            })
494        }
495
496        self.ensure_consistency(provider, input.checkpoint.block_number, Some(unwind_to))?;
497
498        // Unwind account and storage changesets, as well as receipts.
499        //
500        // This also updates `PlainStorageState` and `PlainAccountState`.
501        let bundle_state_with_receipts =
502            provider.take_state_above(unwind_to, StorageLocation::Both)?;
503
504        // Prepare the input for post unwind commit hook, where an `ExExNotification` will be sent.
505        if self.exex_manager_handle.has_exexs() {
506            // Get the blocks for the unwound range.
507            let blocks = provider.sealed_block_with_senders_range(range.clone())?;
508            let previous_input = self.post_unwind_commit_input.replace(Chain::new(
509                blocks,
510                bundle_state_with_receipts,
511                None,
512            ));
513
514            debug_assert!(
515                previous_input.is_none(),
516                "Previous post unwind commit input wasn't processed"
517            );
518            if let Some(previous_input) = previous_input {
519                tracing::debug!(target: "sync::stages::execution", ?previous_input, "Previous post unwind commit input wasn't processed");
520            }
521        }
522
523        // Update the checkpoint.
524        let mut stage_checkpoint = input.checkpoint.execution_stage_checkpoint();
525        if let Some(stage_checkpoint) = stage_checkpoint.as_mut() {
526            for block_number in range {
527                stage_checkpoint.progress.processed -= provider
528                    .block_by_number(block_number)?
529                    .ok_or_else(|| ProviderError::HeaderNotFound(block_number.into()))?
530                    .header()
531                    .gas_used();
532            }
533        }
534        let checkpoint = if let Some(stage_checkpoint) = stage_checkpoint {
535            StageCheckpoint::new(unwind_to).with_execution_stage_checkpoint(stage_checkpoint)
536        } else {
537            StageCheckpoint::new(unwind_to)
538        };
539
540        Ok(UnwindOutput { checkpoint })
541    }
542
543    fn post_unwind_commit(&mut self) -> Result<(), StageError> {
544        let Some(chain) = self.post_unwind_commit_input.take() else { return Ok(()) };
545
546        // NOTE: We can ignore the error here, since an error means that the channel is closed,
547        // which means the manager has died, which then in turn means the node is shutting down.
548        let _ = self.exex_manager_handle.send(
549            ExExNotificationSource::Pipeline,
550            ExExNotification::ChainReverted { old: Arc::new(chain) },
551        );
552
553        Ok(())
554    }
555}
556
557fn execution_checkpoint<N: NodePrimitives>(
558    provider: &StaticFileProvider<N>,
559    start_block: BlockNumber,
560    max_block: BlockNumber,
561    checkpoint: StageCheckpoint,
562) -> Result<ExecutionCheckpoint, ProviderError> {
563    Ok(match checkpoint.execution_stage_checkpoint() {
564        // If checkpoint block range fully matches our range,
565        // we take the previously used stage checkpoint as-is.
566        Some(stage_checkpoint @ ExecutionCheckpoint { block_range, .. })
567            if block_range == CheckpointBlockRange::from(start_block..=max_block) =>
568        {
569            stage_checkpoint
570        }
571        // If checkpoint block range precedes our range seamlessly, we take the previously used
572        // stage checkpoint and add the amount of gas from our range to the checkpoint total.
573        Some(ExecutionCheckpoint {
574            block_range: CheckpointBlockRange { to, .. },
575            progress: EntitiesCheckpoint { processed, total },
576        }) if to == start_block - 1 => ExecutionCheckpoint {
577            block_range: CheckpointBlockRange { from: start_block, to: max_block },
578            progress: EntitiesCheckpoint {
579                processed,
580                total: total + calculate_gas_used_from_headers(provider, start_block..=max_block)?,
581            },
582        },
583        // If checkpoint block range ends on the same block as our range, we take the previously
584        // used stage checkpoint.
585        Some(ExecutionCheckpoint { block_range: CheckpointBlockRange { to, .. }, progress })
586            if to == max_block =>
587        {
588            ExecutionCheckpoint {
589                block_range: CheckpointBlockRange { from: start_block, to: max_block },
590                progress,
591            }
592        }
593        // If there's any other non-empty checkpoint, we calculate the remaining amount of total gas
594        // to be processed not including the checkpoint range.
595        Some(ExecutionCheckpoint { progress: EntitiesCheckpoint { processed, .. }, .. }) => {
596            let after_checkpoint_block_number =
597                calculate_gas_used_from_headers(provider, checkpoint.block_number + 1..=max_block)?;
598
599            ExecutionCheckpoint {
600                block_range: CheckpointBlockRange { from: start_block, to: max_block },
601                progress: EntitiesCheckpoint {
602                    processed,
603                    total: processed + after_checkpoint_block_number,
604                },
605            }
606        }
607        // Otherwise, we recalculate the whole stage checkpoint including the amount of gas
608        // already processed, if there's any.
609        _ => {
610            let processed = calculate_gas_used_from_headers(provider, 0..=start_block - 1)?;
611
612            ExecutionCheckpoint {
613                block_range: CheckpointBlockRange { from: start_block, to: max_block },
614                progress: EntitiesCheckpoint {
615                    processed,
616                    total: processed +
617                        calculate_gas_used_from_headers(provider, start_block..=max_block)?,
618                },
619            }
620        }
621    })
622}
623
624fn calculate_gas_used_from_headers<N: NodePrimitives>(
625    provider: &StaticFileProvider<N>,
626    range: RangeInclusive<BlockNumber>,
627) -> Result<u64, ProviderError> {
628    debug!(target: "sync::stages::execution", ?range, "Calculating gas used from headers");
629
630    let mut gas_total = 0;
631
632    let start = Instant::now();
633
634    for entry in provider.fetch_range_iter(
635        StaticFileSegment::Headers,
636        *range.start()..*range.end() + 1,
637        |cursor, number| cursor.get_one::<HeaderMask<Header>>(number.into()),
638    )? {
639        let Header { gas_used, .. } = entry?;
640        gas_total += gas_used;
641    }
642
643    let duration = start.elapsed();
644    debug!(target: "sync::stages::execution", ?range, ?duration, "Finished calculating gas used from headers");
645
646    Ok(gas_total)
647}
648
649#[cfg(test)]
650mod tests {
651    use super::*;
652    use crate::test_utils::TestStageDB;
653    use alloy_primitives::{address, hex_literal::hex, keccak256, Address, B256, U256};
654    use alloy_rlp::Decodable;
655    use assert_matches::assert_matches;
656    use reth_chainspec::ChainSpecBuilder;
657    use reth_db_api::{models::AccountBeforeTx, transaction::DbTxMut};
658    use reth_evm::execute::BasicBlockExecutorProvider;
659    use reth_evm_ethereum::execute::EthExecutionStrategyFactory;
660    use reth_execution_errors::BlockValidationError;
661    use reth_primitives::{Account, Bytecode, SealedBlock, StorageEntry};
662    use reth_provider::{
663        test_utils::create_test_provider_factory, AccountReader, DatabaseProviderFactory,
664        ReceiptProvider, StaticFileProviderFactory,
665    };
666    use reth_prune_types::{PruneMode, ReceiptsLogPruneConfig};
667    use reth_stages_api::StageUnitCheckpoint;
668    use std::collections::BTreeMap;
669
670    fn stage() -> ExecutionStage<BasicBlockExecutorProvider<EthExecutionStrategyFactory>> {
671        let strategy_factory = EthExecutionStrategyFactory::ethereum(Arc::new(
672            ChainSpecBuilder::mainnet().berlin_activated().build(),
673        ));
674        let executor_provider = BasicBlockExecutorProvider::new(strategy_factory);
675        ExecutionStage::new(
676            executor_provider,
677            ExecutionStageThresholds {
678                max_blocks: Some(100),
679                max_changes: None,
680                max_cumulative_gas: None,
681                max_duration: None,
682            },
683            MERKLE_STAGE_DEFAULT_CLEAN_THRESHOLD,
684            PruneModes::none(),
685            ExExManagerHandle::empty(),
686        )
687    }
688
689    #[test]
690    fn execution_checkpoint_matches() {
691        let factory = create_test_provider_factory();
692
693        let previous_stage_checkpoint = ExecutionCheckpoint {
694            block_range: CheckpointBlockRange { from: 0, to: 0 },
695            progress: EntitiesCheckpoint { processed: 1, total: 2 },
696        };
697        let previous_checkpoint = StageCheckpoint {
698            block_number: 0,
699            stage_checkpoint: Some(StageUnitCheckpoint::Execution(previous_stage_checkpoint)),
700        };
701
702        let stage_checkpoint = execution_checkpoint(
703            &factory.static_file_provider(),
704            previous_stage_checkpoint.block_range.from,
705            previous_stage_checkpoint.block_range.to,
706            previous_checkpoint,
707        );
708
709        assert_eq!(stage_checkpoint, Ok(previous_stage_checkpoint));
710    }
711
712    #[test]
713    fn execution_checkpoint_precedes() {
714        let factory = create_test_provider_factory();
715        let provider = factory.provider_rw().unwrap();
716
717        let mut genesis_rlp = hex!("f901faf901f5a00000000000000000000000000000000000000000000000000000000000000000a01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347942adc25665018aa1fe0e6bc666dac8fc2697ff9baa045571b40ae66ca7480791bbb2887286e4e4c4b1b298b191c889d6959023a32eda056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421b901000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000083020000808502540be400808000a00000000000000000000000000000000000000000000000000000000000000000880000000000000000c0c0").as_slice();
718        let genesis = SealedBlock::decode(&mut genesis_rlp).unwrap();
719        let mut block_rlp = hex!("f90262f901f9a075c371ba45999d87f4542326910a11af515897aebce5265d3f6acd1f1161f82fa01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347942adc25665018aa1fe0e6bc666dac8fc2697ff9baa098f2dcd87c8ae4083e7017a05456c14eea4b1db2032126e27b3b1563d57d7cc0a08151d548273f6683169524b66ca9fe338b9ce42bc3540046c828fd939ae23bcba03f4e5c2ec5b2170b711d97ee755c160457bb58d8daa338e835ec02ae6860bbabb901000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000083020000018502540be40082a8798203e800a00000000000000000000000000000000000000000000000000000000000000000880000000000000000f863f861800a8405f5e10094100000000000000000000000000000000000000080801ba07e09e26678ed4fac08a249ebe8ed680bf9051a5e14ad223e4b2b9d26e0208f37a05f6e3f188e3e6eab7d7d3b6568f5eac7d687b08d307d3154ccd8c87b4630509bc0").as_slice();
720        let block = SealedBlock::decode(&mut block_rlp).unwrap();
721        provider
722            .insert_historical_block(
723                genesis
724                    .try_seal_with_senders()
725                    .map_err(|_| BlockValidationError::SenderRecoveryError)
726                    .unwrap(),
727            )
728            .unwrap();
729        provider.insert_historical_block(block.clone().try_seal_with_senders().unwrap()).unwrap();
730        provider
731            .static_file_provider()
732            .latest_writer(StaticFileSegment::Headers)
733            .unwrap()
734            .commit()
735            .unwrap();
736        provider.commit().unwrap();
737
738        let previous_stage_checkpoint = ExecutionCheckpoint {
739            block_range: CheckpointBlockRange { from: 0, to: 0 },
740            progress: EntitiesCheckpoint { processed: 1, total: 1 },
741        };
742        let previous_checkpoint = StageCheckpoint {
743            block_number: 1,
744            stage_checkpoint: Some(StageUnitCheckpoint::Execution(previous_stage_checkpoint)),
745        };
746
747        let stage_checkpoint =
748            execution_checkpoint(&factory.static_file_provider(), 1, 1, previous_checkpoint);
749
750        assert_matches!(stage_checkpoint, Ok(ExecutionCheckpoint {
751            block_range: CheckpointBlockRange { from: 1, to: 1 },
752            progress: EntitiesCheckpoint {
753                processed,
754                total
755            }
756        }) if processed == previous_stage_checkpoint.progress.processed &&
757            total == previous_stage_checkpoint.progress.total + block.gas_used);
758    }
759
760    #[test]
761    fn execution_checkpoint_recalculate_full_previous_some() {
762        let factory = create_test_provider_factory();
763        let provider = factory.provider_rw().unwrap();
764
765        let mut genesis_rlp = hex!("f901faf901f5a00000000000000000000000000000000000000000000000000000000000000000a01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347942adc25665018aa1fe0e6bc666dac8fc2697ff9baa045571b40ae66ca7480791bbb2887286e4e4c4b1b298b191c889d6959023a32eda056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421b901000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000083020000808502540be400808000a00000000000000000000000000000000000000000000000000000000000000000880000000000000000c0c0").as_slice();
766        let genesis = SealedBlock::decode(&mut genesis_rlp).unwrap();
767        let mut block_rlp = hex!("f90262f901f9a075c371ba45999d87f4542326910a11af515897aebce5265d3f6acd1f1161f82fa01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347942adc25665018aa1fe0e6bc666dac8fc2697ff9baa098f2dcd87c8ae4083e7017a05456c14eea4b1db2032126e27b3b1563d57d7cc0a08151d548273f6683169524b66ca9fe338b9ce42bc3540046c828fd939ae23bcba03f4e5c2ec5b2170b711d97ee755c160457bb58d8daa338e835ec02ae6860bbabb901000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000083020000018502540be40082a8798203e800a00000000000000000000000000000000000000000000000000000000000000000880000000000000000f863f861800a8405f5e10094100000000000000000000000000000000000000080801ba07e09e26678ed4fac08a249ebe8ed680bf9051a5e14ad223e4b2b9d26e0208f37a05f6e3f188e3e6eab7d7d3b6568f5eac7d687b08d307d3154ccd8c87b4630509bc0").as_slice();
768        let block = SealedBlock::decode(&mut block_rlp).unwrap();
769        provider.insert_historical_block(genesis.try_seal_with_senders().unwrap()).unwrap();
770        provider.insert_historical_block(block.clone().try_seal_with_senders().unwrap()).unwrap();
771        provider
772            .static_file_provider()
773            .latest_writer(StaticFileSegment::Headers)
774            .unwrap()
775            .commit()
776            .unwrap();
777        provider.commit().unwrap();
778
779        let previous_stage_checkpoint = ExecutionCheckpoint {
780            block_range: CheckpointBlockRange { from: 0, to: 0 },
781            progress: EntitiesCheckpoint { processed: 1, total: 1 },
782        };
783        let previous_checkpoint = StageCheckpoint {
784            block_number: 1,
785            stage_checkpoint: Some(StageUnitCheckpoint::Execution(previous_stage_checkpoint)),
786        };
787
788        let stage_checkpoint =
789            execution_checkpoint(&factory.static_file_provider(), 1, 1, previous_checkpoint);
790
791        assert_matches!(stage_checkpoint, Ok(ExecutionCheckpoint {
792            block_range: CheckpointBlockRange { from: 1, to: 1 },
793            progress: EntitiesCheckpoint {
794                processed,
795                total
796            }
797        }) if processed == previous_stage_checkpoint.progress.processed &&
798            total == previous_stage_checkpoint.progress.total + block.gas_used);
799    }
800
801    #[test]
802    fn execution_checkpoint_recalculate_full_previous_none() {
803        let factory = create_test_provider_factory();
804        let provider = factory.provider_rw().unwrap();
805
806        let mut genesis_rlp = hex!("f901faf901f5a00000000000000000000000000000000000000000000000000000000000000000a01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347942adc25665018aa1fe0e6bc666dac8fc2697ff9baa045571b40ae66ca7480791bbb2887286e4e4c4b1b298b191c889d6959023a32eda056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421b901000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000083020000808502540be400808000a00000000000000000000000000000000000000000000000000000000000000000880000000000000000c0c0").as_slice();
807        let genesis = SealedBlock::decode(&mut genesis_rlp).unwrap();
808        let mut block_rlp = hex!("f90262f901f9a075c371ba45999d87f4542326910a11af515897aebce5265d3f6acd1f1161f82fa01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347942adc25665018aa1fe0e6bc666dac8fc2697ff9baa098f2dcd87c8ae4083e7017a05456c14eea4b1db2032126e27b3b1563d57d7cc0a08151d548273f6683169524b66ca9fe338b9ce42bc3540046c828fd939ae23bcba03f4e5c2ec5b2170b711d97ee755c160457bb58d8daa338e835ec02ae6860bbabb901000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000083020000018502540be40082a8798203e800a00000000000000000000000000000000000000000000000000000000000000000880000000000000000f863f861800a8405f5e10094100000000000000000000000000000000000000080801ba07e09e26678ed4fac08a249ebe8ed680bf9051a5e14ad223e4b2b9d26e0208f37a05f6e3f188e3e6eab7d7d3b6568f5eac7d687b08d307d3154ccd8c87b4630509bc0").as_slice();
809        let block = SealedBlock::decode(&mut block_rlp).unwrap();
810        provider.insert_historical_block(genesis.try_seal_with_senders().unwrap()).unwrap();
811        provider.insert_historical_block(block.clone().try_seal_with_senders().unwrap()).unwrap();
812        provider
813            .static_file_provider()
814            .latest_writer(StaticFileSegment::Headers)
815            .unwrap()
816            .commit()
817            .unwrap();
818        provider.commit().unwrap();
819
820        let previous_checkpoint = StageCheckpoint { block_number: 1, stage_checkpoint: None };
821
822        let stage_checkpoint =
823            execution_checkpoint(&factory.static_file_provider(), 1, 1, previous_checkpoint);
824
825        assert_matches!(stage_checkpoint, Ok(ExecutionCheckpoint {
826            block_range: CheckpointBlockRange { from: 1, to: 1 },
827            progress: EntitiesCheckpoint {
828                processed: 0,
829                total
830            }
831        }) if total == block.gas_used);
832    }
833
834    #[tokio::test(flavor = "multi_thread")]
835    async fn sanity_execution_of_block() {
836        let factory = create_test_provider_factory();
837        let provider = factory.provider_rw().unwrap();
838        let input = ExecInput { target: Some(1), checkpoint: None };
839        let mut genesis_rlp = hex!("f901faf901f5a00000000000000000000000000000000000000000000000000000000000000000a01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347942adc25665018aa1fe0e6bc666dac8fc2697ff9baa045571b40ae66ca7480791bbb2887286e4e4c4b1b298b191c889d6959023a32eda056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421b901000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000083020000808502540be400808000a00000000000000000000000000000000000000000000000000000000000000000880000000000000000c0c0").as_slice();
840        let genesis = SealedBlock::decode(&mut genesis_rlp).unwrap();
841        let mut block_rlp = hex!("f90262f901f9a075c371ba45999d87f4542326910a11af515897aebce5265d3f6acd1f1161f82fa01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347942adc25665018aa1fe0e6bc666dac8fc2697ff9baa098f2dcd87c8ae4083e7017a05456c14eea4b1db2032126e27b3b1563d57d7cc0a08151d548273f6683169524b66ca9fe338b9ce42bc3540046c828fd939ae23bcba03f4e5c2ec5b2170b711d97ee755c160457bb58d8daa338e835ec02ae6860bbabb901000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000083020000018502540be40082a8798203e800a00000000000000000000000000000000000000000000000000000000000000000880000000000000000f863f861800a8405f5e10094100000000000000000000000000000000000000080801ba07e09e26678ed4fac08a249ebe8ed680bf9051a5e14ad223e4b2b9d26e0208f37a05f6e3f188e3e6eab7d7d3b6568f5eac7d687b08d307d3154ccd8c87b4630509bc0").as_slice();
842        let block = SealedBlock::decode(&mut block_rlp).unwrap();
843        provider.insert_historical_block(genesis.try_seal_with_senders().unwrap()).unwrap();
844        provider.insert_historical_block(block.clone().try_seal_with_senders().unwrap()).unwrap();
845        provider
846            .static_file_provider()
847            .latest_writer(StaticFileSegment::Headers)
848            .unwrap()
849            .commit()
850            .unwrap();
851        {
852            let static_file_provider = provider.static_file_provider();
853            let mut receipts_writer =
854                static_file_provider.latest_writer(StaticFileSegment::Receipts).unwrap();
855            receipts_writer.increment_block(0).unwrap();
856            receipts_writer.commit().unwrap();
857        }
858        provider.commit().unwrap();
859
860        // insert pre state
861        let provider = factory.provider_rw().unwrap();
862
863        let db_tx = provider.tx_ref();
864        let acc1 = address!("1000000000000000000000000000000000000000");
865        let acc2 = address!("a94f5374fce5edbc8e2a8697c15331677e6ebf0b");
866        let code = hex!("5a465a905090036002900360015500");
867        let balance = U256::from(0x3635c9adc5dea00000u128);
868        let code_hash = keccak256(code);
869        db_tx
870            .put::<tables::PlainAccountState>(
871                acc1,
872                Account { nonce: 0, balance: U256::ZERO, bytecode_hash: Some(code_hash) },
873            )
874            .unwrap();
875        db_tx
876            .put::<tables::PlainAccountState>(
877                acc2,
878                Account { nonce: 0, balance, bytecode_hash: None },
879            )
880            .unwrap();
881        db_tx.put::<tables::Bytecodes>(code_hash, Bytecode::new_raw(code.to_vec().into())).unwrap();
882        provider.commit().unwrap();
883
884        // execute
885
886        // If there is a pruning configuration, then it's forced to use the database.
887        // This way we test both cases.
888        let modes = [None, Some(PruneModes::none())];
889        let random_filter =
890            ReceiptsLogPruneConfig(BTreeMap::from([(Address::random(), PruneMode::Full)]));
891
892        // Tests node with database and node with static files
893        for mut mode in modes {
894            let mut provider = factory.database_provider_rw().unwrap();
895
896            if let Some(mode) = &mut mode {
897                // Simulating a full node where we write receipts to database
898                mode.receipts_log_filter = random_filter.clone();
899            }
900
901            let mut execution_stage = stage();
902            execution_stage.prune_modes = mode.clone().unwrap_or_default();
903            provider.set_prune_modes(mode.clone().unwrap_or_default());
904
905            let output = execution_stage.execute(&provider, input).unwrap();
906            provider.commit().unwrap();
907
908            assert_matches!(output, ExecOutput {
909                checkpoint: StageCheckpoint {
910                    block_number: 1,
911                    stage_checkpoint: Some(StageUnitCheckpoint::Execution(ExecutionCheckpoint {
912                        block_range: CheckpointBlockRange {
913                            from: 1,
914                            to: 1,
915                        },
916                        progress: EntitiesCheckpoint {
917                            processed,
918                            total
919                        }
920                    }))
921                },
922                done: true
923            } if processed == total && total == block.gas_used);
924
925            let provider = factory.provider().unwrap();
926
927            // check post state
928            let account1 = address!("1000000000000000000000000000000000000000");
929            let account1_info =
930                Account { balance: U256::ZERO, nonce: 0x00, bytecode_hash: Some(code_hash) };
931            let account2 = address!("2adc25665018aa1fe0e6bc666dac8fc2697ff9ba");
932            let account2_info = Account {
933                balance: U256::from(0x1bc16d674ece94bau128),
934                nonce: 0x00,
935                bytecode_hash: None,
936            };
937            let account3 = address!("a94f5374fce5edbc8e2a8697c15331677e6ebf0b");
938            let account3_info = Account {
939                balance: U256::from(0x3635c9adc5de996b46u128),
940                nonce: 0x01,
941                bytecode_hash: None,
942            };
943
944            // assert accounts
945            assert_eq!(
946                provider.basic_account(account1),
947                Ok(Some(account1_info)),
948                "Post changed of a account"
949            );
950            assert_eq!(
951                provider.basic_account(account2),
952                Ok(Some(account2_info)),
953                "Post changed of a account"
954            );
955            assert_eq!(
956                provider.basic_account(account3),
957                Ok(Some(account3_info)),
958                "Post changed of a account"
959            );
960            // assert storage
961            // Get on dupsort would return only first value. This is good enough for this test.
962            assert_eq!(
963                provider.tx_ref().get::<tables::PlainStorageState>(account1),
964                Ok(Some(StorageEntry {
965                    key: B256::with_last_byte(1),
966                    value: U256::from(2),
967                    ..Default::default()
968                })),
969                "Post changed of a account"
970            );
971
972            let mut provider = factory.database_provider_rw().unwrap();
973            let mut stage = stage();
974            stage.prune_modes = mode.clone().unwrap_or_default();
975            provider.set_prune_modes(mode.unwrap_or_default());
976
977            let _result = stage
978                .unwind(
979                    &provider,
980                    UnwindInput { checkpoint: output.checkpoint, unwind_to: 0, bad_block: None },
981                )
982                .unwrap();
983            provider.commit().unwrap();
984        }
985    }
986
987    #[tokio::test(flavor = "multi_thread")]
988    async fn sanity_execute_unwind() {
989        let factory = create_test_provider_factory();
990        let provider = factory.provider_rw().unwrap();
991        let input = ExecInput { target: Some(1), checkpoint: None };
992        let mut genesis_rlp = hex!("f901faf901f5a00000000000000000000000000000000000000000000000000000000000000000a01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347942adc25665018aa1fe0e6bc666dac8fc2697ff9baa045571b40ae66ca7480791bbb2887286e4e4c4b1b298b191c889d6959023a32eda056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421b901000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000083020000808502540be400808000a00000000000000000000000000000000000000000000000000000000000000000880000000000000000c0c0").as_slice();
993        let genesis = SealedBlock::decode(&mut genesis_rlp).unwrap();
994        let mut block_rlp = hex!("f90262f901f9a075c371ba45999d87f4542326910a11af515897aebce5265d3f6acd1f1161f82fa01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347942adc25665018aa1fe0e6bc666dac8fc2697ff9baa098f2dcd87c8ae4083e7017a05456c14eea4b1db2032126e27b3b1563d57d7cc0a08151d548273f6683169524b66ca9fe338b9ce42bc3540046c828fd939ae23bcba03f4e5c2ec5b2170b711d97ee755c160457bb58d8daa338e835ec02ae6860bbabb901000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000083020000018502540be40082a8798203e800a00000000000000000000000000000000000000000000000000000000000000000880000000000000000f863f861800a8405f5e10094100000000000000000000000000000000000000080801ba07e09e26678ed4fac08a249ebe8ed680bf9051a5e14ad223e4b2b9d26e0208f37a05f6e3f188e3e6eab7d7d3b6568f5eac7d687b08d307d3154ccd8c87b4630509bc0").as_slice();
995        let block = SealedBlock::decode(&mut block_rlp).unwrap();
996        provider.insert_historical_block(genesis.try_seal_with_senders().unwrap()).unwrap();
997        provider.insert_historical_block(block.clone().try_seal_with_senders().unwrap()).unwrap();
998        provider
999            .static_file_provider()
1000            .latest_writer(StaticFileSegment::Headers)
1001            .unwrap()
1002            .commit()
1003            .unwrap();
1004        {
1005            let static_file_provider = provider.static_file_provider();
1006            let mut receipts_writer =
1007                static_file_provider.latest_writer(StaticFileSegment::Receipts).unwrap();
1008            receipts_writer.increment_block(0).unwrap();
1009            receipts_writer.commit().unwrap();
1010        }
1011        provider.commit().unwrap();
1012
1013        // variables
1014        let code = hex!("5a465a905090036002900360015500");
1015        let balance = U256::from(0x3635c9adc5dea00000u128);
1016        let code_hash = keccak256(code);
1017        // pre state
1018        let provider = factory.provider_rw().unwrap();
1019
1020        let db_tx = provider.tx_ref();
1021        let acc1 = address!("1000000000000000000000000000000000000000");
1022        let acc1_info = Account { nonce: 0, balance: U256::ZERO, bytecode_hash: Some(code_hash) };
1023        let acc2 = address!("a94f5374fce5edbc8e2a8697c15331677e6ebf0b");
1024        let acc2_info = Account { nonce: 0, balance, bytecode_hash: None };
1025
1026        db_tx.put::<tables::PlainAccountState>(acc1, acc1_info).unwrap();
1027        db_tx.put::<tables::PlainAccountState>(acc2, acc2_info).unwrap();
1028        db_tx.put::<tables::Bytecodes>(code_hash, Bytecode::new_raw(code.to_vec().into())).unwrap();
1029        provider.commit().unwrap();
1030
1031        // execute
1032        let mut provider = factory.database_provider_rw().unwrap();
1033
1034        // If there is a pruning configuration, then it's forced to use the database.
1035        // This way we test both cases.
1036        let modes = [None, Some(PruneModes::none())];
1037        let random_filter =
1038            ReceiptsLogPruneConfig(BTreeMap::from([(Address::random(), PruneMode::Full)]));
1039
1040        // Tests node with database and node with static files
1041        for mut mode in modes {
1042            if let Some(mode) = &mut mode {
1043                // Simulating a full node where we write receipts to database
1044                mode.receipts_log_filter = random_filter.clone();
1045            }
1046
1047            // Test Execution
1048            let mut execution_stage = stage();
1049            execution_stage.prune_modes = mode.clone().unwrap_or_default();
1050            provider.set_prune_modes(mode.clone().unwrap_or_default());
1051
1052            let result = execution_stage.execute(&provider, input).unwrap();
1053            provider.commit().unwrap();
1054
1055            // Test Unwind
1056            provider = factory.database_provider_rw().unwrap();
1057            let mut stage = stage();
1058            stage.prune_modes = mode.clone().unwrap_or_default();
1059            provider.set_prune_modes(mode.clone().unwrap_or_default());
1060
1061            let result = stage
1062                .unwind(
1063                    &provider,
1064                    UnwindInput { checkpoint: result.checkpoint, unwind_to: 0, bad_block: None },
1065                )
1066                .unwrap();
1067
1068            assert_matches!(result, UnwindOutput {
1069                checkpoint: StageCheckpoint {
1070                    block_number: 0,
1071                    stage_checkpoint: Some(StageUnitCheckpoint::Execution(ExecutionCheckpoint {
1072                        block_range: CheckpointBlockRange {
1073                            from: 1,
1074                            to: 1,
1075                        },
1076                        progress: EntitiesCheckpoint {
1077                            processed: 0,
1078                            total
1079                        }
1080                    }))
1081                }
1082            } if total == block.gas_used);
1083
1084            // assert unwind stage
1085            assert_eq!(
1086                provider.basic_account(acc1),
1087                Ok(Some(acc1_info)),
1088                "Pre changed of a account"
1089            );
1090            assert_eq!(
1091                provider.basic_account(acc2),
1092                Ok(Some(acc2_info)),
1093                "Post changed of a account"
1094            );
1095
1096            let miner_acc = address!("2adc25665018aa1fe0e6bc666dac8fc2697ff9ba");
1097            assert_eq!(
1098                provider.basic_account(miner_acc),
1099                Ok(None),
1100                "Third account should be unwound"
1101            );
1102
1103            assert_eq!(provider.receipt(0), Ok(None), "First receipt should be unwound");
1104        }
1105    }
1106
1107    #[tokio::test(flavor = "multi_thread")]
1108    async fn test_selfdestruct() {
1109        let test_db = TestStageDB::default();
1110        let provider = test_db.factory.database_provider_rw().unwrap();
1111        let input = ExecInput { target: Some(1), checkpoint: None };
1112        let mut genesis_rlp = hex!("f901f8f901f3a00000000000000000000000000000000000000000000000000000000000000000a01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347942adc25665018aa1fe0e6bc666dac8fc2697ff9baa0c9ceb8372c88cb461724d8d3d87e8b933f6fc5f679d4841800e662f4428ffd0da056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421b90100000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000008302000080830f4240808000a00000000000000000000000000000000000000000000000000000000000000000880000000000000000c0c0").as_slice();
1113        let genesis = SealedBlock::decode(&mut genesis_rlp).unwrap();
1114        let mut block_rlp = hex!("f9025ff901f7a0c86e8cc0310ae7c531c758678ddbfd16fc51c8cef8cec650b032de9869e8b94fa01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347942adc25665018aa1fe0e6bc666dac8fc2697ff9baa050554882fbbda2c2fd93fdc466db9946ea262a67f7a76cc169e714f105ab583da00967f09ef1dfed20c0eacfaa94d5cd4002eda3242ac47eae68972d07b106d192a0e3c8b47fbfc94667ef4cceb17e5cc21e3b1eebd442cebb27f07562b33836290db90100000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000008302000001830f42408238108203e800a00000000000000000000000000000000000000000000000000000000000000000880000000000000000f862f860800a83061a8094095e7baea6a6c7c4c2dfeb977efac326af552d8780801ba072ed817487b84ba367d15d2f039b5fc5f087d0a8882fbdf73e8cb49357e1ce30a0403d800545b8fc544f92ce8124e2255f8c3c6af93f28243a120585d4c4c6a2a3c0").as_slice();
1115        let block = SealedBlock::decode(&mut block_rlp).unwrap();
1116        provider.insert_historical_block(genesis.try_seal_with_senders().unwrap()).unwrap();
1117        provider.insert_historical_block(block.clone().try_seal_with_senders().unwrap()).unwrap();
1118        provider
1119            .static_file_provider()
1120            .latest_writer(StaticFileSegment::Headers)
1121            .unwrap()
1122            .commit()
1123            .unwrap();
1124        {
1125            let static_file_provider = provider.static_file_provider();
1126            let mut receipts_writer =
1127                static_file_provider.latest_writer(StaticFileSegment::Receipts).unwrap();
1128            receipts_writer.increment_block(0).unwrap();
1129            receipts_writer.commit().unwrap();
1130        }
1131        provider.commit().unwrap();
1132
1133        // variables
1134        let caller_address = address!("a94f5374fce5edbc8e2a8697c15331677e6ebf0b");
1135        let destroyed_address = address!("095e7baea6a6c7c4c2dfeb977efac326af552d87");
1136        let beneficiary_address = address!("2adc25665018aa1fe0e6bc666dac8fc2697ff9ba");
1137
1138        let code = hex!("73095e7baea6a6c7c4c2dfeb977efac326af552d8731ff00");
1139        let balance = U256::from(0x0de0b6b3a7640000u64);
1140        let code_hash = keccak256(code);
1141
1142        // pre state
1143        let caller_info = Account { nonce: 0, balance, bytecode_hash: None };
1144        let destroyed_info =
1145            Account { nonce: 0, balance: U256::ZERO, bytecode_hash: Some(code_hash) };
1146
1147        // set account
1148        let provider = test_db.factory.provider_rw().unwrap();
1149        provider.tx_ref().put::<tables::PlainAccountState>(caller_address, caller_info).unwrap();
1150        provider
1151            .tx_ref()
1152            .put::<tables::PlainAccountState>(destroyed_address, destroyed_info)
1153            .unwrap();
1154        provider
1155            .tx_ref()
1156            .put::<tables::Bytecodes>(code_hash, Bytecode::new_raw(code.to_vec().into()))
1157            .unwrap();
1158        // set storage to check when account gets destroyed.
1159        provider
1160            .tx_ref()
1161            .put::<tables::PlainStorageState>(
1162                destroyed_address,
1163                StorageEntry { key: B256::ZERO, value: U256::ZERO, ..Default::default() },
1164            )
1165            .unwrap();
1166        provider
1167            .tx_ref()
1168            .put::<tables::PlainStorageState>(
1169                destroyed_address,
1170                StorageEntry {
1171                    key: B256::with_last_byte(1),
1172                    value: U256::from(1u64),
1173                    ..Default::default()
1174                },
1175            )
1176            .unwrap();
1177
1178        provider.commit().unwrap();
1179
1180        // execute
1181        let provider = test_db.factory.database_provider_rw().unwrap();
1182        let mut execution_stage = stage();
1183        let _ = execution_stage.execute(&provider, input).unwrap();
1184        provider.commit().unwrap();
1185
1186        // assert unwind stage
1187        let provider = test_db.factory.database_provider_rw().unwrap();
1188        assert_eq!(provider.basic_account(destroyed_address), Ok(None), "Account was destroyed");
1189
1190        assert_eq!(
1191            provider.tx_ref().get::<tables::PlainStorageState>(destroyed_address),
1192            Ok(None),
1193            "There is storage for destroyed account"
1194        );
1195        // drops tx so that it returns write privilege to test_tx
1196        drop(provider);
1197        let plain_accounts = test_db.table::<tables::PlainAccountState>().unwrap();
1198        let plain_storage = test_db.table::<tables::PlainStorageState>().unwrap();
1199
1200        assert_eq!(
1201            plain_accounts,
1202            vec![
1203                (
1204                    beneficiary_address,
1205                    Account {
1206                        nonce: 0,
1207                        balance: U256::from(0x1bc16d674eca30a0u64),
1208                        bytecode_hash: None
1209                    }
1210                ),
1211                (
1212                    caller_address,
1213                    Account {
1214                        nonce: 1,
1215                        balance: U256::from(0xde0b6b3a761cf60u64),
1216                        bytecode_hash: None
1217                    }
1218                )
1219            ]
1220        );
1221        assert!(plain_storage.is_empty());
1222
1223        let account_changesets = test_db.table::<tables::AccountChangeSets>().unwrap();
1224        let storage_changesets = test_db.table::<tables::StorageChangeSets>().unwrap();
1225
1226        assert_eq!(
1227            account_changesets,
1228            vec![
1229                (
1230                    block.number,
1231                    AccountBeforeTx { address: destroyed_address, info: Some(destroyed_info) },
1232                ),
1233                (block.number, AccountBeforeTx { address: beneficiary_address, info: None }),
1234                (
1235                    block.number,
1236                    AccountBeforeTx { address: caller_address, info: Some(caller_info) }
1237                ),
1238            ]
1239        );
1240
1241        assert_eq!(
1242            storage_changesets,
1243            vec![
1244                (
1245                    (block.number, destroyed_address).into(),
1246                    StorageEntry { key: B256::ZERO, value: U256::ZERO, ..Default::default() }
1247                ),
1248                (
1249                    (block.number, destroyed_address).into(),
1250                    StorageEntry {
1251                        key: B256::with_last_byte(1),
1252                        value: U256::from(1u64),
1253                        ..Default::default()
1254                    }
1255                )
1256            ]
1257        );
1258    }
1259}