reth_stages/stages/
execution.rs

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