From 4d9ab5a4aa64be9097e7b758fd57f9b6a6f939a1 Mon Sep 17 00:00:00 2001 From: Federico Borello <156438142+fborello-lambda@users.noreply.github.com> Date: Thu, 20 Feb 2025 10:32:31 -0300 Subject: [PATCH] refactor(l1): add enum that specifies EVM (#1949) Alternative to #1897 **Motivation** When doing the `EVM` trait in #1897, i've ended up needing an enum to wrap the structs that implement such trait. This approach ended up being quite complex, and the raw use of an enum was discussed. **Description** - Implement an `enum` as unique entrypoint for the crate `ethrex-vm`. Closes https://github.com/lambdaclass/ethrex/issues/1661 --- .github/workflows/common_hive_reports.yaml | 15 +- Cargo.lock | 1 - cmd/ef_tests/state/runner/levm_runner.rs | 118 +-- cmd/ef_tests/state/runner/mod.rs | 2 + cmd/ef_tests/state/runner/revm_runner.rs | 25 +- crates/blockchain/Cargo.toml | 3 +- crates/blockchain/blockchain.rs | 23 +- crates/blockchain/payload.rs | 290 ++----- crates/l2/proposer/l1_committer.rs | 9 +- .../prover/zkvm/interface/risc0/src/main.rs | 12 +- .../l2/prover/zkvm/interface/sp1/src/main.rs | 11 +- crates/vm/backends/constants.rs | 19 +- crates/vm/backends/levm.rs | 755 ++++++++++-------- crates/vm/backends/mod.rs | 345 +++++++- crates/vm/backends/{revm.rs => revm_b.rs} | 342 ++++---- crates/vm/execution_db.rs | 9 +- crates/vm/levm/src/vm.rs | 12 +- crates/vm/vm.rs | 143 +--- 18 files changed, 1100 insertions(+), 1034 deletions(-) rename crates/vm/backends/{revm.rs => revm_b.rs} (69%) diff --git a/.github/workflows/common_hive_reports.yaml b/.github/workflows/common_hive_reports.yaml index 6f6a523efc..d51fb7eeac 100644 --- a/.github/workflows/common_hive_reports.yaml +++ b/.github/workflows/common_hive_reports.yaml @@ -37,11 +37,6 @@ jobs: - { name: "Sync tests", file_name: sync, simulation: ethereum/sync } steps: - - name: Pull image - run: | - docker pull ghcr.io/lambdaclass/ethrex:latest - docker tag ghcr.io/lambdaclass/ethrex:latest ethrex:latest - - name: Checkout sources if: ${{ inputs.job_type != 'main' }} uses: actions/checkout@v4 @@ -52,6 +47,16 @@ jobs: with: ref: main + - name: Build image + if: ${{ inputs.job_type != 'main' }} + run: make build-image + + - name: Pull image + if: ${{ inputs.job_type == 'main' }} + run: | + docker pull ghcr.io/lambdaclass/ethrex:latest + docker tag ghcr.io/lambdaclass/ethrex:latest ethrex:latest + - name: Setup Go uses: actions/setup-go@v5 diff --git a/Cargo.lock b/Cargo.lock index 9bb0775192..fc6b495a59 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2575,7 +2575,6 @@ dependencies = [ "bytes", "cfg-if", "ethrex-common", - "ethrex-levm", "ethrex-metrics", "ethrex-rlp", "ethrex-storage", diff --git a/cmd/ef_tests/state/runner/levm_runner.rs b/cmd/ef_tests/state/runner/levm_runner.rs index 57583319c9..c4447d67fb 100644 --- a/cmd/ef_tests/state/runner/levm_runner.rs +++ b/cmd/ef_tests/state/runner/levm_runner.rs @@ -4,9 +4,8 @@ use crate::{ types::{EFTest, TransactionExpectedException}, utils::{self, effective_gas_price}, }; -use bytes::Bytes; use ethrex_common::{ - types::{code_hash, tx_fields::*, AccountInfo, Fork}, + types::{tx_fields::*, Fork}, H256, U256, }; use ethrex_levm::{ @@ -16,7 +15,10 @@ use ethrex_levm::{ Environment, }; use ethrex_storage::AccountUpdate; -use ethrex_vm::db::{EvmState, StoreWrapper}; +use ethrex_vm::{ + backends::{self}, + db::StoreWrapper, +}; use keccak_hash::keccak; use std::{collections::HashMap, sync::Arc}; @@ -313,8 +315,18 @@ pub fn ensure_post_state( // Execution result was successful and no exception was expected. None => { let (initial_state, block_hash) = utils::load_initial_state(test); - let levm_account_updates = - get_state_transitions(&initial_state, block_hash, execution_report, fork); + let levm_account_updates = backends::levm::LEVM::get_state_transitions( + Some(*fork), + &initial_state, + block_hash, + &execution_report.new_state, + ) + .map_err(|_| { + InternalError::Custom( + "Error at LEVM::get_state_transitions in ensure_post_state()" + .to_owned(), + ) + })?; let pos_state_root = post_state_root(&levm_account_updates, test); let expected_post_state_root_hash = test.post.vector_post_value(vector, *fork).hash; @@ -369,102 +381,6 @@ pub fn ensure_post_state( Ok(()) } -pub fn get_state_transitions( - initial_state: &EvmState, - block_hash: H256, - execution_report: &ExecutionReport, - fork: &Fork, -) -> Vec { - let current_db = match initial_state { - EvmState::Store(state) => state.database.store.clone(), - EvmState::Execution(_cache_db) => unreachable!("Execution state should not be passed here"), - }; - let mut account_updates: Vec = vec![]; - for (new_state_account_address, new_state_account) in &execution_report.new_state { - let initial_account_state = current_db - .get_account_info_by_hash(block_hash, *new_state_account_address) - .expect("Error getting account info by address") - .unwrap_or_default(); - let mut updates = 0; - if initial_account_state.balance != new_state_account.info.balance { - updates += 1; - } - if initial_account_state.nonce != new_state_account.info.nonce { - updates += 1; - } - let code = if new_state_account.info.bytecode.is_empty() { - // The new state account has no code - None - } else { - // Get the code hash of the new state account bytecode - let potential_new_bytecode_hash = code_hash(&new_state_account.info.bytecode); - // Look into the current database to see if the bytecode hash is already present - let current_bytecode = current_db - .get_account_code(potential_new_bytecode_hash) - .expect("Error getting account code by hash"); - let code = new_state_account.info.bytecode.clone(); - // The code is present in the current database - if let Some(current_bytecode) = current_bytecode { - if current_bytecode != code { - // The code has changed - Some(code) - } else { - // The code has not changed - None - } - } else { - // The new state account code is not present in the current - // database, then it must be new - Some(code) - } - }; - if code.is_some() { - updates += 1; - } - let mut added_storage = HashMap::new(); - for (key, value) in &new_state_account.storage { - added_storage.insert(*key, value.current_value); - updates += 1; - } - - if updates == 0 && !new_state_account.is_empty() { - continue; - } - - let account_update = AccountUpdate { - address: *new_state_account_address, - removed: new_state_account.is_empty(), - info: Some(AccountInfo { - code_hash: code_hash(&new_state_account.info.bytecode), - balance: new_state_account.info.balance, - nonce: new_state_account.info.nonce, - }), - code, - added_storage, - }; - - if let Some(old_info) = current_db - .get_account_info_by_hash(block_hash, account_update.address) - .unwrap() - { - // https://eips.ethereum.org/EIPS/eip-161 - // if an account was empty and is now empty, after spurious dragon, it should be removed - if account_update.removed - && old_info.balance.is_zero() - && old_info.nonce == 0 - && old_info.code_hash == code_hash(&Bytes::new()) - && *fork < Fork::SpuriousDragon - { - continue; - } - } - - account_updates.push(account_update); - } - - account_updates -} - pub fn post_state_root(account_updates: &[AccountUpdate], test: &EFTest) -> H256 { let (initial_state, block_hash) = utils::load_initial_state(test); initial_state diff --git a/cmd/ef_tests/state/runner/mod.rs b/cmd/ef_tests/state/runner/mod.rs index 0daa427b61..c9c33b7a31 100644 --- a/cmd/ef_tests/state/runner/mod.rs +++ b/cmd/ef_tests/state/runner/mod.rs @@ -39,6 +39,8 @@ pub enum InternalError { ReRunInternal(String, TestReRunReport), #[error("Main runner failed unexpectedly: {0}")] MainRunnerInternal(String), + #[error("{0}")] + Custom(String), } #[derive(Parser)] diff --git a/cmd/ef_tests/state/runner/revm_runner.rs b/cmd/ef_tests/state/runner/revm_runner.rs index ad13fd5ba1..79906c4d7d 100644 --- a/cmd/ef_tests/state/runner/revm_runner.rs +++ b/cmd/ef_tests/state/runner/revm_runner.rs @@ -1,9 +1,6 @@ use crate::{ report::{ComparisonReport, EFTestReport, EFTestReportForkResult, TestReRunReport, TestVector}, - runner::{ - levm_runner::{self, post_state_root}, - EFTestRunnerError, InternalError, - }, + runner::{levm_runner::post_state_root, EFTestRunnerError, InternalError}, types::EFTest, utils::{effective_gas_price, load_initial_state}, }; @@ -18,6 +15,7 @@ use ethrex_levm::{ }; use ethrex_storage::{error::StoreError, AccountUpdate}; use ethrex_vm::{ + backends::{self}, db::{EvmState, StoreWrapper}, fork_to_spec_id, RevmAddress, RevmU256, }; @@ -330,13 +328,19 @@ pub fn ensure_post_state( // We only want to compare account updates when no exception is expected. None => { let (initial_state, block_hash) = load_initial_state(test); - let levm_account_updates = levm_runner::get_state_transitions( + let levm_account_updates = backends::levm::LEVM::get_state_transitions( + Some(*fork), &initial_state, block_hash, - levm_execution_report, - fork, - ); - let revm_account_updates = ethrex_vm::get_state_transitions(revm_state); + &levm_execution_report.new_state, + ) + .map_err(|_| { + InternalError::Custom("Error at LEVM::get_state_transitions()".to_owned()) + })?; + let revm_account_updates = backends::revm_b::REVM::get_state_transitions(revm_state) + .map_err(|_| { + InternalError::Custom("Error at REVM::get_state_transitions()".to_owned()) + })?; let account_updates_report = compare_levm_revm_account_updates( vector, test, @@ -517,7 +521,8 @@ pub fn _ensure_post_state_revm( } // Execution result was successful and no exception was expected. None => { - let revm_account_updates = ethrex_vm::get_state_transitions(revm_state); + let revm_account_updates = + backends::revm_b::REVM::get_state_transitions(revm_state).unwrap(); let pos_state_root = post_state_root(&revm_account_updates, test); let expected_post_state_root_hash = test.post.vector_post_value(vector, *fork).hash; diff --git a/crates/blockchain/Cargo.toml b/crates/blockchain/Cargo.toml index c69496c798..6b000ed0f6 100644 --- a/crates/blockchain/Cargo.toml +++ b/crates/blockchain/Cargo.toml @@ -10,7 +10,6 @@ ethrex-rlp.workspace = true ethrex-common.workspace = true ethrex-storage.workspace = true ethrex-vm.workspace = true -ethrex-levm.workspace = true thiserror.workspace = true sha3.workspace = true @@ -31,5 +30,5 @@ path = "./blockchain.rs" [features] default = [] -c-kzg = ["ethrex-common/c-kzg", "ethrex-vm/c-kzg", "ethrex-levm/c-kzg"] +c-kzg = ["ethrex-common/c-kzg", "ethrex-vm/c-kzg"] metrics = ["ethrex-metrics/transactions"] diff --git a/crates/blockchain/blockchain.rs b/crates/blockchain/blockchain.rs index a7924dea3f..ceb26499e6 100644 --- a/crates/blockchain/blockchain.rs +++ b/crates/blockchain/blockchain.rs @@ -19,9 +19,7 @@ use ethrex_common::H256; use ethrex_storage::error::StoreError; use ethrex_storage::Store; use ethrex_vm::db::evm_state; - -use ethrex_vm::EVM_BACKEND; -use ethrex_vm::{backends, backends::EVM}; +use ethrex_vm::{backends::BlockExecutionResult, get_evm_backend_or_default}; //TODO: Implement a struct Chain or BlockChain to encapsulate //functionality and canonical chain state and config @@ -45,20 +43,11 @@ pub fn add_block(block: &Block, storage: &Store) -> Result<(), ChainError> { // Validate the block pre-execution validate_block(block, &parent_header, &chain_config)?; - let (receipts, requests, account_updates) = { - match EVM_BACKEND.get() { - Some(EVM::LEVM) => { - let r = backends::levm::execute_block(block, &mut state)?; - (r.receipts, r.requests, r.account_updates) - } - // This means we are using REVM as default for tests - Some(EVM::REVM) | None => { - let (receipts, requests) = backends::revm::execute_block(block, &mut state)?; - let account_updates = ethrex_vm::get_state_transitions(&mut state); - (receipts, requests, account_updates) - } - } - }; + let BlockExecutionResult { + receipts, + requests, + account_updates, + } = get_evm_backend_or_default().execute_block(block, &mut state)?; validate_gas_used(&receipts, &block.header)?; diff --git a/crates/blockchain/payload.rs b/crates/blockchain/payload.rs index a851cf1715..46c9fb4561 100644 --- a/crates/blockchain/payload.rs +++ b/crates/blockchain/payload.rs @@ -9,20 +9,17 @@ use ethrex_common::{ calculate_base_fee_per_blob_gas, calculate_base_fee_per_gas, compute_receipts_root, compute_requests_hash, compute_transactions_root, compute_withdrawals_root, requests::Requests, BlobsBundle, Block, BlockBody, BlockHash, BlockHeader, BlockNumber, - ChainConfig, Fork, MempoolTransaction, Receipt, Transaction, Withdrawal, - DEFAULT_OMMERS_HASH, DEFAULT_REQUESTS_HASH, + ChainConfig, MempoolTransaction, Receipt, Transaction, Withdrawal, DEFAULT_OMMERS_HASH, + DEFAULT_REQUESTS_HASH, }, Address, Bloom, Bytes, H256, U256, }; -use ethrex_common::types::GWEI_TO_WEI; -use ethrex_levm::{db::CacheDB, vm::EVMConfig, Account, AccountInfo}; use ethrex_vm::{ - backends::{self, levm::extract_all_requests_levm, revm::extract_all_requests, EVM}, - db::{evm_state, EvmState, StoreWrapper}, - get_state_transitions, spec_id, EvmError, SpecId, EVM_BACKEND, + backends::levm::CacheDB, + db::{evm_state, EvmState}, + get_evm_backend_or_default, EvmError, }; -use std::sync::Arc; use ethrex_rlp::encode::RLPEncode; use ethrex_storage::{error::StoreError, Store}; @@ -250,138 +247,34 @@ pub fn build_payload( } pub fn apply_withdrawals(context: &mut PayloadBuildContext) -> Result<(), EvmError> { - match EVM_BACKEND.get() { - Some(EVM::LEVM) => { - if let Some(withdrawals) = &context.payload.body.withdrawals { - // For every withdrawal we increment the target account's balance - for (address, increment) in withdrawals - .iter() - .filter(|withdrawal| withdrawal.amount > 0) - .map(|w| (w.address, u128::from(w.amount) * u128::from(GWEI_TO_WEI))) - { - // We check if it was in block_cache, if not, we get it from DB. - let mut account = context.block_cache.get(&address).cloned().unwrap_or({ - let acc_info = context - .store() - .ok_or(StoreError::MissingStore)? - .get_account_info_by_hash(context.parent_hash(), address)? - .unwrap_or_default(); - let acc_code = context - .store() - .ok_or(StoreError::MissingStore)? - .get_account_code(acc_info.code_hash)? - .unwrap_or_default(); - - Account { - info: AccountInfo { - balance: acc_info.balance, - bytecode: acc_code, - nonce: acc_info.nonce, - }, - // This is the added_storage for the withdrawal. - // If not involved in the TX, there won't be any updates in the storage - storage: HashMap::new(), - } - }); - - account.info.balance += increment.into(); - context.block_cache.insert(address, account); - } - } - } - Some(EVM::REVM) | None => { - backends::revm::process_withdrawals( - context.evm_state, - context - .payload - .body - .withdrawals - .as_ref() - .unwrap_or(&Vec::new()), - )?; - } - } - Ok(()) + let binding = Vec::new(); + let withdrawals = context + .payload + .body + .withdrawals + .as_ref() + .unwrap_or(&binding); + get_evm_backend_or_default() + .process_withdrawals( + withdrawals, + context.evm_state, + &context.payload.header, + &mut context.block_cache, + ) + .map_err(EvmError::from) } // This function applies system level operations: // - Call beacon root contract, and obtain the new state root // - Call block hash process contract, and store parent block hash pub fn apply_system_operations(context: &mut PayloadBuildContext) -> Result<(), EvmError> { - match EVM_BACKEND.get() { - Some(EVM::LEVM) => { - let fork = context - .chain_config()? - .fork(context.payload.header.timestamp); - let blob_schedule = context - .chain_config()? - .get_fork_blob_schedule(context.payload.header.timestamp) - .unwrap_or(EVMConfig::canonical_values(fork)); - let config = EVMConfig::new(fork, blob_schedule); - let mut new_state = HashMap::new(); - - if context.payload.header.parent_beacon_block_root.is_some() && fork >= Fork::Cancun { - let store_wrapper = Arc::new(StoreWrapper { - store: context.evm_state.database().unwrap().clone(), - block_hash: context.payload.header.parent_hash, - }); - let report = backends::levm::beacon_root_contract_call_levm( - store_wrapper, - &context.payload.header, - config, - )?; - - new_state.extend(report.new_state); - } - - if fork >= Fork::Prague { - let store_wrapper = Arc::new(StoreWrapper { - store: context.evm_state.database().unwrap().clone(), - block_hash: context.payload.header.parent_hash, - }); - let report = backends::levm::process_block_hash_history( - store_wrapper, - &context.payload.header, - config, - )?; - - new_state.extend(report.new_state); - } - - // Now original_value is going to be the same as the current_value, for the next transaction. - // It should have only one value but it is convenient to keep on using our CacheDB structure - for account in new_state.values_mut() { - for storage_slot in account.storage.values_mut() { - storage_slot.original_value = storage_slot.current_value; - } - } - - context.block_cache.extend(new_state); - } - // This means we are using REVM as default for tests - Some(EVM::REVM) | None => { - // Apply withdrawals & call beacon root contract, and obtain the new state root - let spec_id = spec_id(&context.chain_config()?, context.payload.header.timestamp); - if context.payload.header.parent_beacon_block_root.is_some() - && spec_id >= SpecId::CANCUN - { - backends::revm::beacon_root_contract_call( - context.evm_state, - &context.payload.header, - spec_id, - )?; - } - - if spec_id >= SpecId::PRAGUE { - backends::revm::process_block_hash_history( - context.evm_state, - &context.payload.header, - spec_id, - )?; - } - } - } - Ok(()) + let chain_config = context.chain_config()?; + get_evm_backend_or_default().apply_system_calls( + context.evm_state, + &context.payload.header, + &mut context.block_cache, + &chain_config, + ) } /// Fetches suitable transactions from the mempool @@ -584,114 +477,37 @@ fn apply_plain_transaction( head: &HeadTransaction, context: &mut PayloadBuildContext, ) -> Result { - match EVM_BACKEND.get() { - Some(EVM::LEVM) => { - let store_wrapper = Arc::new(StoreWrapper { - store: context.evm_state.database().unwrap().clone(), - block_hash: context.payload.header.parent_hash, - }); - - let fork = context - .chain_config()? - .fork(context.payload.header.timestamp); - let blob_schedule = context - .chain_config()? - .get_fork_blob_schedule(context.payload.header.timestamp) - .unwrap_or(EVMConfig::canonical_values(fork)); - let config = EVMConfig::new(fork, blob_schedule); - - let report = backends::levm::execute_tx_levm( - &head.tx, - &context.payload.header, - store_wrapper.clone(), - context.block_cache.clone(), - config, - ) - .map_err(|e| EvmError::Transaction(format!("Invalid Transaction: {e:?}")))?; - context.remaining_gas = context.remaining_gas.saturating_sub(report.gas_used); - context.block_value += U256::from(report.gas_used) * head.tip; - - let mut new_state = report.new_state.clone(); - - // Now original_value is going to be the same as the current_value, for the next transaction. - // It should have only one value but it is convenient to keep on using our CacheDB structure - for account in new_state.values_mut() { - for storage_slot in account.storage.values_mut() { - storage_slot.original_value = storage_slot.current_value; - } - } - - context.block_cache.extend(new_state); - - let receipt = Receipt::new( - head.tx.tx_type(), - report.is_success(), - context.payload.header.gas_limit - context.remaining_gas, - report.logs.clone(), - ); - Ok(receipt) - } - // This means we are using REVM as default for tests - Some(EVM::REVM) | None => { - let report = backends::revm::execute_tx( - &head.tx, - &context.payload.header, - context.evm_state, - spec_id( - &context.chain_config().map_err(ChainError::from)?, - context.payload.header.timestamp, - ), - )?; - context.remaining_gas = context.remaining_gas.saturating_sub(report.gas_used()); - context.block_value += U256::from(report.gas_used()) * head.tip; - let receipt = Receipt::new( - head.tx.tx_type(), - report.is_success(), - context.payload.header.gas_limit - context.remaining_gas, - report.logs(), - ); - Ok(receipt) - } - } + let chain_config = context.chain_config()?; + let (report, gas_used) = get_evm_backend_or_default().execute_tx( + context.evm_state, + &head.tx, + &context.payload.header, + &mut context.block_cache, + &chain_config, + &mut context.remaining_gas, + )?; + context.block_value += U256::from(gas_used) * head.tip; + Ok(report) } pub fn extract_requests(context: &mut PayloadBuildContext) -> Result<(), EvmError> { - match EVM_BACKEND.get() { - Some(EVM::LEVM) => { - let requests = extract_all_requests_levm( - &context.receipts, - context.evm_state, - &context.payload.header, - &mut context.block_cache, - )?; - context.requests = requests; - } - // This means we are using REVM as default for tests - Some(EVM::REVM) | None => { - let requests = extract_all_requests( - &context.receipts, - context.evm_state, - &context.payload.header, - )?; - context.requests = requests; - } - } + let requests = get_evm_backend_or_default().extract_requests( + &context.receipts, + context.evm_state, + &context.payload.header, + &mut context.block_cache, + ); + context.requests = requests?; Ok(()) } fn finalize_payload(context: &mut PayloadBuildContext) -> Result<(), ChainError> { - let config = context.chain_config()?; - let is_prague_activated = config.is_prague_activated(context.payload.header.timestamp); - let account_updates = match EVM_BACKEND.get() { - Some(EVM::LEVM) => backends::levm::get_state_transitions_levm( - context.evm_state, - context.parent_hash(), - &context.block_cache.clone(), - ), - // This means we are using REVM as default for tests - Some(EVM::REVM) | None => get_state_transitions(context.evm_state), - }; + let account_updates = get_evm_backend_or_default().get_state_transitions( + context.evm_state, + context.parent_hash(), + &context.block_cache, + )?; context.payload.header.state_root = context .store() @@ -701,8 +517,10 @@ fn finalize_payload(context: &mut PayloadBuildContext) -> Result<(), ChainError> context.payload.header.transactions_root = compute_transactions_root(&context.payload.body.transactions); context.payload.header.receipts_root = compute_receipts_root(&context.receipts); - context.payload.header.requests_hash = - is_prague_activated.then_some(compute_requests_hash(&context.requests)); + context.payload.header.requests_hash = context + .chain_config()? + .is_prague_activated(context.payload.header.timestamp) + .then_some(compute_requests_hash(&context.requests)); context.payload.header.gas_used = context.payload.header.gas_limit - context.remaining_gas; Ok(()) } diff --git a/crates/l2/proposer/l1_committer.rs b/crates/l2/proposer/l1_committer.rs index f3ef9e4881..ba515a394a 100644 --- a/crates/l2/proposer/l1_committer.rs +++ b/crates/l2/proposer/l1_committer.rs @@ -20,7 +20,7 @@ use ethrex_rpc::clients::eth::{ eth_sender::Overrides, BlockByNumber, EthClient, WrappedTransaction, }; use ethrex_storage::{error::StoreError, Store}; -use ethrex_vm::{backends::revm::execute_block, db::evm_state, get_state_transitions}; +use ethrex_vm::{backends::EVM, db::evm_state}; use keccak_hash::keccak; use secp256k1::SecretKey; use std::{collections::HashMap, str::FromStr, time::Duration}; @@ -255,8 +255,11 @@ impl Committer { info!("Preparing state diff for block {}", block.header.number); let mut state = evm_state(store.clone(), block.header.parent_hash); - execute_block(block, &mut state).map_err(CommitterError::from)?; - let account_updates = get_state_transitions(&mut state); + + let result = EVM::default() + .execute_block(block, &mut state) + .map_err(CommitterError::from)?; + let account_updates = result.account_updates; let mut modified_accounts = HashMap::new(); for account_update in &account_updates { diff --git a/crates/l2/prover/zkvm/interface/risc0/src/main.rs b/crates/l2/prover/zkvm/interface/risc0/src/main.rs index b7d4cfcd40..e698e9b494 100644 --- a/crates/l2/prover/zkvm/interface/risc0/src/main.rs +++ b/crates/l2/prover/zkvm/interface/risc0/src/main.rs @@ -1,7 +1,8 @@ use risc0_zkvm::guest::env; -use ethrex_blockchain::{error::ChainError, validate_block, validate_gas_used}; -use ethrex_vm::{backends::revm::execute_block, db::EvmState, get_state_transitions}; +use ethrex_blockchain::{validate_block, validate_gas_used}; +use ethrex_vm::{backends::revm_b::REVM, db::EvmState}; + use zkvm_interface::{ io::{ProgramInput, ProgramOutput}, trie::{update_tries, verify_db}, @@ -16,7 +17,6 @@ fn main() { let mut state = EvmState::from(db.clone()); let chain_config = state .chain_config() - .map_err(ChainError::from) .expect("Failed to get chain config from state"); // Validate the block @@ -36,7 +36,9 @@ fn main() { panic!("invalid database") }; - let (receipts, _) = execute_block(&block, &mut state).expect("failed to execute block"); + let result = REVM::execute_block(&block, &mut state).expect("failed to execute block"); + let receipts = result.receipts; + let account_updates = result.account_updates; validate_gas_used(&receipts, &block.header).expect("invalid gas used"); // Output gas for measurement purposes @@ -46,8 +48,6 @@ fn main() { .unwrap_or_default(); env::write(&cumulative_gas_used); - let account_updates = get_state_transitions(&mut state); - // Update tries and calculate final state root hash update_tries(&mut state_trie, &mut storage_tries, &account_updates) .expect("failed to update state and storage tries"); diff --git a/crates/l2/prover/zkvm/interface/sp1/src/main.rs b/crates/l2/prover/zkvm/interface/sp1/src/main.rs index 7e2fb0df82..3ac91af82a 100644 --- a/crates/l2/prover/zkvm/interface/sp1/src/main.rs +++ b/crates/l2/prover/zkvm/interface/sp1/src/main.rs @@ -1,7 +1,7 @@ #![no_main] -use ethrex_blockchain::{error::ChainError, validate_block, validate_gas_used}; -use ethrex_vm::{backends::revm::execute_block, db::EvmState, get_state_transitions}; +use ethrex_blockchain::{validate_block, validate_gas_used}; +use ethrex_vm::{backends::revm_b::REVM, db::EvmState}; use zkvm_interface::{ io::{ProgramInput, ProgramOutput}, trie::{update_tries, verify_db}, @@ -18,7 +18,6 @@ pub fn main() { let mut state = EvmState::from(db.clone()); let chain_config = state .chain_config() - .map_err(ChainError::from) .expect("Failed to get chain config from state"); // Validate the block @@ -38,7 +37,9 @@ pub fn main() { panic!("invalid database") }; - let (receipts, _) = execute_block(&block, &mut state).expect("failed to execute block"); + let result = REVM::execute_block(&block, &mut state).expect("failed to execute block"); + let receipts = result.receipts; + let account_updates = result.account_updates; validate_gas_used(&receipts, &block.header).expect("invalid gas used"); // Output gas for measurement purposes @@ -48,8 +49,6 @@ pub fn main() { .unwrap_or_default(); sp1_zkvm::io::commit(&cumulative_gas_used); - let account_updates = get_state_transitions(&mut state); - // Update tries and calculate final state root hash update_tries(&mut state_trie, &mut storage_tries, &account_updates) .expect("failed to update state and storage tries"); diff --git a/crates/vm/backends/constants.rs b/crates/vm/backends/constants.rs index 4760aba876..558e700024 100644 --- a/crates/vm/backends/constants.rs +++ b/crates/vm/backends/constants.rs @@ -1,6 +1,13 @@ -pub const SYSTEM_ADDRESS_STR: &str = "fffffffffffffffffffffffffffffffffffffffe"; -pub const BEACON_ROOTS_ADDRESS_STR: &str = "000F3df6D732807Ef1319fB7B8bB8522d0Beac02"; -pub const HISTORY_STORAGE_ADDRESS_STR: &str = "0000F90827F1C53a10cb7A02335B175320002935"; -pub const WITHDRAWAL_REQUEST_PREDEPLOY_ADDRESS: &str = "00000961Ef480Eb55e80D19ad83579A64c007002"; -pub const CONSOLIDATION_REQUEST_PREDEPLOY_ADDRESS: &str = - "0000BBdDc7CE488642fb579F8B00f3a590007251"; +use ethrex_common::Address; +use std::{str::FromStr, sync::LazyLock}; + +pub static SYSTEM_ADDRESS: LazyLock
= + LazyLock::new(|| Address::from_str("fffffffffffffffffffffffffffffffffffffffe").unwrap()); +pub static BEACON_ROOTS_ADDRESS: LazyLock
= + LazyLock::new(|| Address::from_str("000F3df6D732807Ef1319fB7B8bB8522d0Beac02").unwrap()); +pub static HISTORY_STORAGE_ADDRESS: LazyLock
= + LazyLock::new(|| Address::from_str("0000F90827F1C53a10cb7A02335B175320002935").unwrap()); +pub static WITHDRAWAL_REQUEST_PREDEPLOY_ADDRESS: LazyLock
= + LazyLock::new(|| Address::from_str("00000961Ef480Eb55e80D19ad83579A64c007002").unwrap()); +pub static CONSOLIDATION_REQUEST_PREDEPLOY_ADDRESS: LazyLock
= + LazyLock::new(|| Address::from_str("0000BBdDc7CE488642fb579F8B00f3a590007251").unwrap()); diff --git a/crates/vm/backends/levm.rs b/crates/vm/backends/levm.rs index a481372893..9a3e4d17fe 100644 --- a/crates/vm/backends/levm.rs +++ b/crates/vm/backends/levm.rs @@ -1,7 +1,10 @@ use super::constants::{ - BEACON_ROOTS_ADDRESS_STR, CONSOLIDATION_REQUEST_PREDEPLOY_ADDRESS, HISTORY_STORAGE_ADDRESS_STR, - SYSTEM_ADDRESS_STR, WITHDRAWAL_REQUEST_PREDEPLOY_ADDRESS, + BEACON_ROOTS_ADDRESS, CONSOLIDATION_REQUEST_PREDEPLOY_ADDRESS, HISTORY_STORAGE_ADDRESS, + SYSTEM_ADDRESS, WITHDRAWAL_REQUEST_PREDEPLOY_ADDRESS, }; +use super::BlockExecutionResult; +use crate::backends::get_state_transitions; + use crate::db::StoreWrapper; use crate::EvmError; use crate::EvmState; @@ -9,206 +12,257 @@ use ethrex_common::types::requests::Requests; use ethrex_common::types::Fork; use ethrex_common::{ types::{ - code_hash, AccountInfo, Block, BlockHeader, Receipt, Transaction, TxKind, GWEI_TO_WEI, + code_hash, AccountInfo, Block, BlockHeader, ChainConfig, Receipt, Transaction, TxKind, + Withdrawal, GWEI_TO_WEI, }, - Address, Bytes as CoreBytes, H256, U256, + Address, H256, U256, }; - use ethrex_levm::{ - db::{CacheDB, Database as LevmDatabase}, + db::Database as LevmDatabase, errors::{ExecutionReport, TxResult, VMError}, vm::{EVMConfig, VM}, - Account, Environment, + Account, AccountInfo as LevmAccountInfo, Environment, }; -use ethrex_storage::AccountUpdate; -use lazy_static::lazy_static; +use ethrex_storage::{error::StoreError, AccountUpdate, Store}; use revm_primitives::Bytes; use std::{collections::HashMap, sync::Arc}; -pub struct BlockExecutionResult { - pub receipts: Vec, - pub requests: Vec, - pub account_updates: Vec, -} - -/// Executes all transactions in a block and returns their receipts. -pub fn execute_block( - block: &Block, - state: &mut EvmState, -) -> Result { - let store_wrapper = Arc::new(StoreWrapper { - store: state.database().unwrap().clone(), - block_hash: block.header.parent_hash, - }); - - let mut block_cache: CacheDB = HashMap::new(); - let block_header = &block.header; - let fork = state.chain_config()?.fork(block_header.timestamp); - // If there's no blob schedule in chain_config use the - // default/canonical values - let blob_schedule = state - .chain_config()? - .get_fork_blob_schedule(block_header.timestamp) - .unwrap_or(EVMConfig::canonical_values(fork)); - let config = EVMConfig::new(fork, blob_schedule); - cfg_if::cfg_if! { - if #[cfg(not(feature = "l2"))] { - if block_header.parent_beacon_block_root.is_some() && fork >= Fork::Cancun { - let report = beacon_root_contract_call_levm(store_wrapper.clone(), block_header, config)?; - block_cache.extend(report.new_state); - } - - if fork >= Fork::Prague { - //eip 2935: stores parent block hash in system contract - let report = process_block_hash_history(store_wrapper.clone(), block_header, config)?; - block_cache.extend(report.new_state); +// Export needed types +pub use ethrex_levm::db::CacheDB; +/// The struct implements the following functions: +/// [LEVM::execute_block] +/// [LEVM::execute_tx] +/// [LEVM::get_state_transitions] +/// [LEVM::process_withdrawals] +#[derive(Debug)] +pub struct LEVM; + +impl LEVM { + pub fn execute_block( + block: &Block, + state: &mut EvmState, + ) -> Result { + let store_wrapper = Arc::new(StoreWrapper { + store: state.database().unwrap().clone(), + block_hash: block.header.parent_hash, + }); + + let mut block_cache: CacheDB = HashMap::new(); + let block_header = &block.header; + let config = state.chain_config()?; + cfg_if::cfg_if! { + if #[cfg(not(feature = "l2"))] { + let fork = config.fork(block_header.timestamp); + if block_header.parent_beacon_block_root.is_some() && fork >= Fork::Cancun { + Self::beacon_root_contract_call(block_header, state, &mut block_cache)?; + } + + if fork >= Fork::Prague { + //eip 2935: stores parent block hash in system contract + Self::process_block_hash_history(block_header, state, &mut block_cache)?; + } } } - } - // Account updates are initialized like this because of the beacon_root_contract_call, it is going to be empty if it wasn't called. - let mut account_updates = crate::get_state_transitions(state); + // Account updates are initialized like this because of the beacon_root_contract_call, it is going to be empty if it wasn't called. + // Here we get the state_transitions from the db and then we get the state_transitions from the cache_db. + let mut account_updates = get_state_transitions(state); + let mut receipts = Vec::new(); + let mut cumulative_gas_used = 0; + + for tx in block.body.transactions.iter() { + let report = Self::execute_tx( + tx, + block_header, + store_wrapper.clone(), + block_cache.clone(), + &config, + ) + .map_err(EvmError::from)?; + + let mut new_state = report.new_state.clone(); + // Now original_value is going to be the same as the current_value, for the next transaction. + // It should have only one value but it is convenient to keep on using our CacheDB structure + for account in new_state.values_mut() { + for storage_slot in account.storage.values_mut() { + storage_slot.original_value = storage_slot.current_value; + } + } - let mut receipts = Vec::new(); - let mut cumulative_gas_used = 0; + block_cache.extend(new_state); - for tx in block.body.transactions.iter() { - let report = execute_tx_levm( - tx, - block_header, - store_wrapper.clone(), - block_cache.clone(), - config, - ) - .map_err(EvmError::from)?; + // Currently, in LEVM, we don't substract refunded gas to used gas, but that can change in the future. + let gas_used = report.gas_used - report.gas_refunded; + cumulative_gas_used += gas_used; + let receipt = Receipt::new( + tx.tx_type(), + matches!(report.result.clone(), TxResult::Success), + cumulative_gas_used, + report.logs.clone(), + ); - let mut new_state = report.new_state.clone(); + receipts.push(receipt); + } - // Now original_value is going to be the same as the current_value, for the next transaction. - // It should have only one value but it is convenient to keep on using our CacheDB structure - for account in new_state.values_mut() { - for storage_slot in account.storage.values_mut() { - storage_slot.original_value = storage_slot.current_value; + // Here we update block_cache with balance increments caused by withdrawals. + if let Some(withdrawals) = &block.body.withdrawals { + // For every withdrawal we increment the target account's balance + for (address, increment) in withdrawals + .iter() + .filter(|withdrawal| withdrawal.amount > 0) + .map(|w| (w.address, u128::from(w.amount) * u128::from(GWEI_TO_WEI))) + { + // We check if it was in block_cache, if not, we get it from DB. + let mut account = block_cache.get(&address).cloned().unwrap_or({ + let acc_info = store_wrapper.get_account_info(address); + Account::from(acc_info) + }); + + account.info.balance += increment.into(); + + block_cache.insert(address, account); } } - block_cache.extend(new_state); - - // Currently, in LEVM, we don't substract refunded gas to used gas, but that can change in the future. - let gas_used = report.gas_used - report.gas_refunded; - cumulative_gas_used += gas_used; - let receipt = Receipt::new( - tx.tx_type(), - matches!(report.result.clone(), TxResult::Success), - cumulative_gas_used, - report.logs.clone(), - ); - - receipts.push(receipt); + let requests = + extract_all_requests_levm(&receipts, state, &block.header, &mut block_cache)?; + + account_updates.extend(Self::get_state_transitions( + None, + state, + block.header.parent_hash, + &block_cache, + )?); + + Ok(BlockExecutionResult { + receipts, + requests, + account_updates, + }) } - // Here we update block_cache with balance increments caused by withdrawals. - if let Some(withdrawals) = &block.body.withdrawals { - // For every withdrawal we increment the target account's balance - for (address, increment) in withdrawals - .iter() - .filter(|withdrawal| withdrawal.amount > 0) - .map(|w| (w.address, u128::from(w.amount) * u128::from(GWEI_TO_WEI))) - { - // We check if it was in block_cache, if not, we get it from DB. - let mut account = block_cache.get(&address).cloned().unwrap_or({ - let acc_info = store_wrapper.get_account_info(address); - Account::from(acc_info) - }); + pub fn execute_tx( + // The transaction to execute. + tx: &Transaction, + // The block header for the current block. + block_header: &BlockHeader, + // The database to use for EVM state access. This is wrapped in an `Arc` for shared ownership. + db: Arc, + // A cache database for intermediate state changes during execution. + block_cache: CacheDB, + // The EVM configuration to use. + chain_config: &ChainConfig, + ) -> Result { + let gas_price: U256 = tx + .effective_gas_price(block_header.base_fee_per_gas) + .ok_or(VMError::InvalidTransaction)? + .into(); + + let config = EVMConfig::new_from_chain_config(chain_config, block_header); + let env = Environment { + origin: tx.sender(), + refunded_gas: 0, + gas_limit: tx.gas_limit(), + config, + block_number: block_header.number.into(), + coinbase: block_header.coinbase, + timestamp: block_header.timestamp.into(), + prev_randao: Some(block_header.prev_randao), + chain_id: tx.chain_id().unwrap_or_default().into(), + base_fee_per_gas: block_header.base_fee_per_gas.unwrap_or_default().into(), + gas_price, + block_excess_blob_gas: block_header.excess_blob_gas.map(U256::from), + block_blob_gas_used: block_header.blob_gas_used.map(U256::from), + tx_blob_hashes: tx.blob_versioned_hashes(), + tx_max_priority_fee_per_gas: tx.max_priority_fee().map(U256::from), + tx_max_fee_per_gas: tx.max_fee_per_gas().map(U256::from), + tx_max_fee_per_blob_gas: tx.max_fee_per_blob_gas().map(U256::from), + tx_nonce: tx.nonce(), + block_gas_limit: block_header.gas_limit, + transient_storage: HashMap::new(), + }; - account.info.balance += increment.into(); + let mut vm = VM::new( + tx.to(), + env, + tx.value(), + tx.data().clone(), + db, + block_cache.clone(), + tx.access_list(), + tx.authorization_list(), + )?; - block_cache.insert(address, account); - } + vm.execute().map_err(VMError::into) } - let requests = extract_all_requests_levm(&receipts, state, &block.header, &mut block_cache)?; - - account_updates.extend(get_state_transitions_levm( - state, - block.header.parent_hash, - &block_cache, - )); - - Ok(BlockExecutionResult { - receipts, - requests, - account_updates, - }) -} - -pub fn execute_tx_levm( - tx: &Transaction, - block_header: &BlockHeader, - db: Arc, - block_cache: CacheDB, - config: EVMConfig, -) -> Result { - let gas_price: U256 = tx - .effective_gas_price(block_header.base_fee_per_gas) - .ok_or(VMError::InvalidTransaction)? - .into(); - - let env = Environment { - origin: tx.sender(), - refunded_gas: 0, - gas_limit: tx.gas_limit(), - config, - block_number: block_header.number.into(), - coinbase: block_header.coinbase, - timestamp: block_header.timestamp.into(), - prev_randao: Some(block_header.prev_randao), - chain_id: tx.chain_id().unwrap_or_default().into(), - base_fee_per_gas: block_header.base_fee_per_gas.unwrap_or_default().into(), - gas_price, - block_excess_blob_gas: block_header.excess_blob_gas.map(U256::from), - block_blob_gas_used: block_header.blob_gas_used.map(U256::from), - tx_blob_hashes: tx.blob_versioned_hashes(), - tx_max_priority_fee_per_gas: tx.max_priority_fee().map(U256::from), - tx_max_fee_per_gas: tx.max_fee_per_gas().map(U256::from), - tx_max_fee_per_blob_gas: tx.max_fee_per_blob_gas().map(U256::from), - tx_nonce: tx.nonce(), - block_gas_limit: block_header.gas_limit, - transient_storage: HashMap::new(), - }; + pub fn get_state_transitions( + // Warning only pass the fork if running the ef-tests. + // ISSUE #2021: https://github.com/lambdaclass/ethrex/issues/2021 + ef_tests: Option, + initial_state: &EvmState, + block_hash: H256, + new_state: &CacheDB, + ) -> Result, EvmError> { + let current_db = match initial_state { + EvmState::Store(state) => state.database.store.clone(), + EvmState::Execution(_cache_db) => { + unreachable!("Execution state should not be passed here") + } + }; + let mut account_updates: Vec = vec![]; + for (new_state_account_address, new_state_account) in new_state { + let initial_account_state = current_db + .get_account_info_by_hash(block_hash, *new_state_account_address) + .expect("Error getting account info by address") + .unwrap_or_default(); + let mut updates = 0; + if initial_account_state.balance != new_state_account.info.balance { + updates += 1; + } + if initial_account_state.nonce != new_state_account.info.nonce { + updates += 1; + } + let code = if new_state_account.info.bytecode.is_empty() { + // The new state account has no code + None + } else { + // Get the code hash of the new state account bytecode + let potential_new_bytecode_hash = code_hash(&new_state_account.info.bytecode); + // Look into the current database to see if the bytecode hash is already present + let current_bytecode = current_db + .get_account_code(potential_new_bytecode_hash) + .expect("Error getting account code by hash"); + let code = new_state_account.info.bytecode.clone(); + // The code is present in the current database + if let Some(current_bytecode) = current_bytecode { + if current_bytecode != code { + // The code has changed + Some(code) + } else { + // The code has not changed + None + } + } else { + // The new state account code is not present in the current + // database, then it must be new + Some(code) + } + }; + if code.is_some() { + updates += 1; + } + let mut added_storage = HashMap::new(); + for (key, value) in &new_state_account.storage { + added_storage.insert(*key, value.current_value); + updates += 1; + } - let mut vm = VM::new( - tx.to(), - env, - tx.value(), - tx.data().clone(), - db, - block_cache, - tx.access_list(), - tx.authorization_list(), - )?; - - vm.execute() -} + if updates == 0 && !new_state_account.is_empty() { + continue; + } -pub fn get_state_transitions_levm( - initial_state: &EvmState, - block_hash: H256, - new_state: &CacheDB, -) -> Vec { - let current_db = match initial_state { - EvmState::Store(state) => state.database.store.clone(), - EvmState::Execution(_cache_db) => unreachable!("Execution state should not be passed here"), - }; - let mut account_updates: Vec = vec![]; - for (new_state_account_address, new_state_account) in new_state { - let initial_account_state = current_db - .get_account_info_by_hash(block_hash, *new_state_account_address) - .expect("Error getting account info by address"); - - if initial_account_state.is_none() { - // New account, update everything - let new_account = AccountUpdate { + let account_update = AccountUpdate { address: *new_state_account_address, removed: new_state_account.is_empty(), info: Some(AccountInfo { @@ -216,171 +270,179 @@ pub fn get_state_transitions_levm( balance: new_state_account.info.balance, nonce: new_state_account.info.nonce, }), - code: Some(new_state_account.info.bytecode.clone()), - added_storage: new_state_account - .storage - .iter() - .map(|(key, storage_slot)| (*key, storage_slot.current_value)) - .collect(), + code, + added_storage, }; - account_updates.push(new_account); - continue; - } - - // This unwrap is safe, just checked upside - let initial_account_state = initial_account_state.unwrap(); - let mut account_update = AccountUpdate::new(*new_state_account_address); - - // Account state after block execution. - let new_state_acc_info = AccountInfo { - code_hash: code_hash(&new_state_account.info.bytecode), - balance: new_state_account.info.balance, - nonce: new_state_account.info.nonce, - }; - - // Compare Account Info - if initial_account_state != new_state_acc_info { - account_update.info = Some(new_state_acc_info.clone()); - } - - // If code hash is different it means the code is different too. - if initial_account_state.code_hash != new_state_acc_info.code_hash { - account_update.code = Some(new_state_account.info.bytecode.clone()); - } - - let mut updated_storage = HashMap::new(); - for (key, storage_slot) in &new_state_account.storage { - // original_value in storage_slot is not the original_value on the DB, be careful. - let original_value = current_db - .get_storage_at_hash(block_hash, *new_state_account_address, *key) - .unwrap() - .unwrap_or_default(); // Option inside result, I guess I have to assume it is zero. - - if original_value != storage_slot.current_value { - updated_storage.insert(*key, storage_slot.current_value); + let block_header = current_db + .get_block_header_by_hash(block_hash)? + .ok_or(StoreError::MissingStore)?; + let fork_from_config = initial_state.chain_config()?.fork(block_header.timestamp); + // Here we take the passed fork through the ef_tests variable, or we set it to the fork based on the timestamp. + let fork = ef_tests.unwrap_or(fork_from_config); + if let Some(old_info) = + current_db.get_account_info_by_hash(block_hash, account_update.address)? + { + // https://eips.ethereum.org/EIPS/eip-161 + // if an account was empty and is now empty, after spurious dragon, it should be removed + if account_update.removed + && old_info.balance.is_zero() + && old_info.nonce == 0 + && old_info.code_hash == code_hash(&Bytes::new()) + && fork < Fork::SpuriousDragon + { + continue; + } } - } - account_update.added_storage = updated_storage; - - account_update.removed = new_state_account.is_empty(); - if account_update != AccountUpdate::new(*new_state_account_address) { account_updates.push(account_update); } + Ok(account_updates) } - account_updates -} - -/// Calls the eip4788 beacon block root system call contract -/// More info on https://eips.ethereum.org/EIPS/eip-4788 -pub fn beacon_root_contract_call_levm( - store_wrapper: Arc, - header: &BlockHeader, - config: EVMConfig, -) -> Result { - lazy_static! { - static ref CONTRACT_ADDRESS: Address = - Address::from_slice(&hex::decode(BEACON_ROOTS_ADDRESS_STR).unwrap(),); - }; - - let beacon_root = header.parent_beacon_block_root.ok_or(EvmError::Header( - "parent_beacon_block_root field is missing".to_string(), - ))?; - let calldata = Bytes::copy_from_slice(beacon_root.as_bytes()).into(); - - generic_system_call(*CONTRACT_ADDRESS, calldata, store_wrapper, header, config) -} - -/// Calls the EIP-2935 process block hashes history system call contract -/// NOTE: This was implemented by making use of an EVM system contract, but can be changed to a -/// direct state trie update after the verkle fork, as explained in https://eips.ethereum.org/EIPS/eip-2935 -pub fn process_block_hash_history( - store_wrapper: Arc, - block_header: &BlockHeader, - config: EVMConfig, -) -> Result { - lazy_static! { - static ref CONTRACT_ADDRESS: Address = - Address::from_slice(&hex::decode(HISTORY_STORAGE_ADDRESS_STR).unwrap(),); - }; - - let calldata = Bytes::copy_from_slice(block_header.parent_hash.as_bytes()).into(); - - generic_system_call( - *CONTRACT_ADDRESS, - calldata, - store_wrapper, - block_header, - config, - ) -} + pub fn process_withdrawals( + block_cache: &mut CacheDB, + withdrawals: &[Withdrawal], + store: Option<&Store>, + parent_hash: H256, + ) -> Result<(), ethrex_storage::error::StoreError> { + // For every withdrawal we increment the target account's balance + for (address, increment) in withdrawals + .iter() + .filter(|withdrawal| withdrawal.amount > 0) + .map(|w| (w.address, u128::from(w.amount) * u128::from(GWEI_TO_WEI))) + { + // We check if it was in block_cache, if not, we get it from DB. + let mut account = block_cache.get(&address).cloned().unwrap_or({ + let acc_info = store + .ok_or(StoreError::MissingStore)? + .get_account_info_by_hash(parent_hash, address)? + .unwrap_or_default(); + let acc_code = store + .ok_or(StoreError::MissingStore)? + .get_account_code(acc_info.code_hash)? + .unwrap_or_default(); + + Account { + info: LevmAccountInfo { + balance: acc_info.balance, + bytecode: acc_code, + nonce: acc_info.nonce, + }, + // This is the added_storage for the withdrawal. + // If not involved in the TX, there won't be any updates in the storage + storage: HashMap::new(), + } + }); -fn read_withdrawal_requests_levm( - store_wrapper: Arc, - block_header: &BlockHeader, - config: EVMConfig, -) -> Option { - lazy_static! { - static ref CONTRACT_ADDRESS: Address = - Address::from_slice(&hex::decode(WITHDRAWAL_REQUEST_PREDEPLOY_ADDRESS).unwrap()); - }; + account.info.balance += increment.into(); + block_cache.insert(address, account); + } + Ok(()) + } - let report = generic_system_call( - *CONTRACT_ADDRESS, - CoreBytes::new(), - store_wrapper, - block_header, - config, - ) - .ok()?; + // SYSTEM CONTRACTS + /// `new_state` is being modified inside [generic_system_contract_levm]. + pub fn beacon_root_contract_call( + block_header: &BlockHeader, + state: &mut EvmState, + new_state: &mut CacheDB, + ) -> Result<(), EvmError> { + let beacon_root = match block_header.parent_beacon_block_root { + None => { + return Err(EvmError::Header( + "parent_beacon_block_root field is missing".to_string(), + )) + } + Some(beacon_root) => beacon_root, + }; - match report.result { - TxResult::Success => Some(report), - _ => None, + generic_system_contract_levm( + block_header, + Bytes::copy_from_slice(beacon_root.as_bytes()), + state, + new_state, + *BEACON_ROOTS_ADDRESS, + *SYSTEM_ADDRESS, + )?; + Ok(()) } -} - -fn dequeue_consolidation_requests( - store_wrapper: Arc, - block_header: &BlockHeader, - config: EVMConfig, -) -> Option { - lazy_static! { - static ref CONTRACT_ADDRESS: Address = - Address::from_slice(&hex::decode(CONSOLIDATION_REQUEST_PREDEPLOY_ADDRESS).unwrap()); - }; + /// `new_state` is being modified inside [generic_system_contract_levm]. + pub fn process_block_hash_history( + block_header: &BlockHeader, + state: &mut EvmState, + new_state: &mut CacheDB, + ) -> Result<(), EvmError> { + generic_system_contract_levm( + block_header, + Bytes::copy_from_slice(block_header.parent_hash.as_bytes()), + state, + new_state, + *HISTORY_STORAGE_ADDRESS, + *SYSTEM_ADDRESS, + )?; + Ok(()) + } + pub(crate) fn read_withdrawal_requests( + block_header: &BlockHeader, + state: &mut EvmState, + new_state: &mut CacheDB, + ) -> Option { + let report = generic_system_contract_levm( + block_header, + Bytes::new(), + state, + new_state, + *WITHDRAWAL_REQUEST_PREDEPLOY_ADDRESS, + *SYSTEM_ADDRESS, + ) + .ok()?; - let report = generic_system_call( - *CONTRACT_ADDRESS, - CoreBytes::new(), - store_wrapper, - block_header, - config, - ) - .ok()?; + match report.result { + TxResult::Success => Some(report), + _ => None, + } + } + pub(crate) fn dequeue_consolidation_requests( + block_header: &BlockHeader, + state: &mut EvmState, + new_state: &mut CacheDB, + ) -> Option { + let report = generic_system_contract_levm( + block_header, + Bytes::new(), + state, + new_state, + *CONSOLIDATION_REQUEST_PREDEPLOY_ADDRESS, + *SYSTEM_ADDRESS, + ) + .ok()?; - match report.result { - TxResult::Success => Some(report), - _ => None, + match report.result { + TxResult::Success => Some(report), + _ => None, + } } } -pub fn generic_system_call( - contract_address: Address, - calldata: CoreBytes, - store_wrapper: Arc, +/// `new_state` is being modified at the end. +pub fn generic_system_contract_levm( block_header: &BlockHeader, - config: EVMConfig, + calldata: Bytes, + state: &mut EvmState, + new_state: &mut CacheDB, + contract_address: Address, + system_address: Address, ) -> Result { - lazy_static! { - static ref SYSTEM_ADDRESS: Address = - Address::from_slice(&hex::decode(SYSTEM_ADDRESS_STR).unwrap()); - }; + let store_wrapper = Arc::new(StoreWrapper { + store: state.database().unwrap().clone(), + block_hash: block_header.parent_hash, + }); + let chain_config = state.chain_config()?; + let config = EVMConfig::new_from_chain_config(&chain_config, block_header); let env = Environment { - origin: *SYSTEM_ADDRESS, + origin: system_address, gas_limit: 30_000_000, block_number: block_header.number.into(), coinbase: block_header.coinbase, @@ -396,13 +458,11 @@ pub fn generic_system_call( ..Default::default() }; - // Here execute with LEVM but just return transaction report. And I will handle it in the calling place. - let mut vm = VM::new( TxKind::Call(contract_address), env, U256::zero(), - calldata, + calldata.into(), store_wrapper, CacheDB::new(), vec![], @@ -412,7 +472,21 @@ pub fn generic_system_call( let mut report = vm.execute().map_err(EvmError::from)?; - report.new_state.remove(&*SYSTEM_ADDRESS); + report.new_state.remove(&system_address); + + match report.result { + TxResult::Success => { + new_state.extend(report.new_state.clone()); + } + _ => { + return Err(EvmError::Custom( + "ERROR in generic_system_contract_levm(). TX didn't succeed.".to_owned(), + )) + } + } + + // new_state is a CacheDB coming from outside the function + new_state.extend(report.new_state.clone()); Ok(report) } @@ -442,35 +516,18 @@ pub fn extract_all_requests_levm( "deposit_contract_address config is missing".to_string(), ))?; - let blob_schedule = config - .get_fork_blob_schedule(header.timestamp) - .unwrap_or(EVMConfig::canonical_values(fork)); - - let evm_config = EVMConfig::new(fork, blob_schedule); - - let store_wrapper = Arc::new(StoreWrapper { - store: state.database().unwrap().clone(), - block_hash: header.parent_hash, - }); - - let withdrawals_data: Vec = - match read_withdrawal_requests_levm(store_wrapper, header, evm_config) { - Some(report) => { - cache.extend(report.new_state.clone()); - report.output.into() - } - None => Default::default(), - }; - - let store_wrapper = Arc::new(StoreWrapper { - store: state.database().unwrap().clone(), - block_hash: header.parent_hash, - }); + let withdrawals_data: Vec = match LEVM::read_withdrawal_requests(header, state, cache) { + Some(report) => { + // the cache is updated inside the generic_system_call + report.output.into() + } + None => Default::default(), + }; let consolidation_data: Vec = - match dequeue_consolidation_requests(store_wrapper, header, evm_config) { + match LEVM::dequeue_consolidation_requests(header, state, cache) { Some(report) => { - cache.extend(report.new_state.clone()); + // the cache is updated inside the generic_system_call report.output.into() } None => Default::default(), diff --git a/crates/vm/backends/mod.rs b/crates/vm/backends/mod.rs index a5955c779b..dff1598a4b 100644 --- a/crates/vm/backends/mod.rs +++ b/crates/vm/backends/mod.rs @@ -1,11 +1,25 @@ mod constants; pub mod levm; -pub mod revm; +pub mod revm_b; -use crate::errors::EvmError; +use crate::{db::StoreWrapper, errors::EvmError, spec_id, EvmState, SpecId}; +use ethrex_common::types::requests::Requests; +use ethrex_common::types::{ + Block, BlockHeader, ChainConfig, Fork, Receipt, Transaction, Withdrawal, +}; +use ethrex_common::{types::AccountInfo, Address, BigEndianHash, H256, U256}; +use ethrex_levm::db::CacheDB; +use ethrex_storage::{error::StoreError, AccountUpdate}; +use levm::LEVM; +use revm_b::REVM; use std::str::FromStr; +use std::sync::Arc; -#[derive(Debug, Clone, Default)] +use revm::db::states::bundle_state::BundleRetention; +use revm::db::{AccountState, AccountStatus}; +use revm::primitives::B256; + +#[derive(Debug, Default, Clone)] pub enum EVM { #[default] REVM, @@ -22,3 +36,328 @@ impl FromStr for EVM { } } } + +pub struct BlockExecutionResult { + pub receipts: Vec, + pub requests: Vec, + pub account_updates: Vec, +} + +impl EVM { + /// Wraps [REVM::execute_block] and [LEVM::execute_block]. + /// The output is [BlockExecutionResult]. + pub fn execute_block( + &self, + block: &Block, + state: &mut EvmState, + ) -> Result { + match self { + EVM::REVM => REVM::execute_block(block, state), + EVM::LEVM => LEVM::execute_block(block, state), + } + } + + /// Wraps [REVM::execute_tx] and [LEVM::execute_tx]. + /// The output is `(Receipt, u64)` == (transaction_receipt, gas_used). + pub fn execute_tx( + &self, + state: &mut EvmState, + tx: &Transaction, + block_header: &BlockHeader, + block_cache: &mut CacheDB, + chain_config: &ChainConfig, + remaining_gas: &mut u64, + ) -> Result<(Receipt, u64), EvmError> { + match self { + EVM::REVM => { + let execution_result = REVM::execute_tx( + tx, + block_header, + state, + spec_id(chain_config, block_header.timestamp), + )?; + + *remaining_gas = remaining_gas.saturating_sub(execution_result.gas_used()); + + let receipt = Receipt::new( + tx.tx_type(), + execution_result.is_success(), + block_header.gas_limit - *remaining_gas, + execution_result.logs(), + ); + + Ok((receipt, execution_result.gas_used())) + } + EVM::LEVM => { + let store_wrapper = Arc::new(StoreWrapper { + store: state.database().unwrap().clone(), + block_hash: block_header.parent_hash, + }); + + let execution_report = LEVM::execute_tx( + tx, + block_header, + store_wrapper.clone(), + block_cache.clone(), + chain_config, + )?; + + *remaining_gas = remaining_gas.saturating_sub(execution_report.gas_used); + + let mut new_state = execution_report.new_state.clone(); + + // Now original_value is going to be the same as the current_value, for the next transaction. + // It should have only one value but it is convenient to keep on using our CacheDB structure + for account in new_state.values_mut() { + for storage_slot in account.storage.values_mut() { + storage_slot.original_value = storage_slot.current_value; + } + } + block_cache.extend(new_state); + + let receipt = Receipt::new( + tx.tx_type(), + execution_report.is_success(), + block_header.gas_limit - *remaining_gas, + execution_report.logs.clone(), + ); + Ok((receipt, execution_report.gas_used)) + } + } + } + + /// Wraps [REVM::beacon_root_contract_call], [REVM::process_block_hash_history] + /// and [LEVM::beacon_root_contract_call], [LEVM::process_block_hash_history]. + /// This function is used to run/apply all the system contracts to the state. + pub fn apply_system_calls( + &self, + state: &mut EvmState, + block_header: &BlockHeader, + block_cache: &mut CacheDB, + chain_config: &ChainConfig, + ) -> Result<(), EvmError> { + match self { + EVM::REVM => { + let spec_id = spec_id(chain_config, block_header.timestamp); + if block_header.parent_beacon_block_root.is_some() && spec_id >= SpecId::CANCUN { + REVM::beacon_root_contract_call(block_header, state)?; + } + + if spec_id >= SpecId::PRAGUE { + REVM::process_block_hash_history(block_header, state)?; + } + Ok(()) + } + EVM::LEVM => { + let fork = chain_config.fork(block_header.timestamp); + let mut new_state = CacheDB::new(); + + if block_header.parent_beacon_block_root.is_some() && fork >= Fork::Cancun { + LEVM::beacon_root_contract_call(block_header, state, &mut new_state)?; + } + + if fork >= Fork::Prague { + LEVM::process_block_hash_history(block_header, state, &mut new_state)?; + } + + // Now original_value is going to be the same as the current_value, for the next transaction. + // It should have only one value but it is convenient to keep on using our CacheDB structure + for account in new_state.values_mut() { + for storage_slot in account.storage.values_mut() { + storage_slot.original_value = storage_slot.current_value; + } + } + + block_cache.extend(new_state); + Ok(()) + } + } + } + + /// Wraps the [REVM::get_state_transitions] and [LEVM::get_state_transitions]. + /// The output is `Vec`. + /// WARNING: + /// [REVM::get_state_transitions] gathers the information from the DB, the functionality of this function + /// is used in [LEVM::execute_block]. + /// [LEVM::get_state_transitions] gathers the information from a [CacheDB]. + /// + /// They may have the same name, but they serve for different purposes. + pub fn get_state_transitions( + &self, + state: &mut EvmState, + parent_hash: H256, + block_cache: &CacheDB, + ) -> Result, EvmError> { + match self { + EVM::REVM => REVM::get_state_transitions(state), + EVM::LEVM => LEVM::get_state_transitions(None, state, parent_hash, block_cache), + } + } + + /// Wraps the [REVM::process_withdrawals] and [LEVM::process_withdrawals]. + /// Applies the withdrawals to the state or the block_chache if using [LEVM]. + pub fn process_withdrawals( + &self, + withdrawals: &[Withdrawal], + state: &mut EvmState, + block_header: &BlockHeader, + block_cache: &mut CacheDB, + ) -> Result<(), StoreError> { + match self { + EVM::REVM => REVM::process_withdrawals(state, withdrawals), + EVM::LEVM => { + let parent_hash = block_header.parent_hash; + let mut new_state = CacheDB::new(); + LEVM::process_withdrawals( + &mut new_state, + withdrawals, + state.database(), + parent_hash, + )?; + block_cache.extend(new_state); + Ok(()) + } + } + } + + pub fn extract_requests( + &self, + receipts: &[Receipt], + state: &mut EvmState, + header: &BlockHeader, + cache: &mut CacheDB, + ) -> Result, EvmError> { + match self { + EVM::LEVM => levm::extract_all_requests_levm(receipts, state, header, cache), + EVM::REVM => revm_b::extract_all_requests(receipts, state, header), + } + } +} + +/// Gets the state_transitions == [AccountUpdate] from the [EvmState]. +/// This function is primarily used in [LEVM::execute_block] and [REVM::execute_block]. +pub fn get_state_transitions(initial_state: &mut EvmState) -> Vec { + match initial_state { + EvmState::Store(db) => { + db.merge_transitions(BundleRetention::PlainState); + let bundle = db.take_bundle(); + + // Update accounts + let mut account_updates = Vec::new(); + for (address, account) in bundle.state() { + if account.status.is_not_modified() { + continue; + } + let address = Address::from_slice(address.0.as_slice()); + // Remove account from DB if destroyed (Process DestroyedChanged as changed account) + if matches!( + account.status, + AccountStatus::Destroyed | AccountStatus::DestroyedAgain + ) { + account_updates.push(AccountUpdate::removed(address)); + continue; + } + + // If account is empty, do not add to the database + if account + .account_info() + .is_some_and(|acc_info| acc_info.is_empty()) + { + continue; + } + + // Apply account changes to DB + let mut account_update = AccountUpdate::new(address); + // If the account was changed then both original and current info will be present in the bundle account + if account.is_info_changed() { + // Update account info in DB + if let Some(new_acc_info) = account.account_info() { + let code_hash = H256::from_slice(new_acc_info.code_hash.as_slice()); + let account_info = AccountInfo { + code_hash, + balance: U256::from_little_endian(new_acc_info.balance.as_le_slice()), + nonce: new_acc_info.nonce, + }; + account_update.info = Some(account_info); + if account.is_contract_changed() { + // Update code in db + if let Some(code) = new_acc_info.code { + account_update.code = Some(code.original_bytes().clone().0); + } + } + } + } + // Update account storage in DB + for (key, slot) in account.storage.iter() { + if slot.is_changed() { + // TODO check if we need to remove the value from our db when value is zero + // if slot.present_value().is_zero() { + // account_update.removed_keys.push(H256::from_uint(&U256::from_little_endian(key.as_le_slice()))) + // } + account_update.added_storage.insert( + H256::from_uint(&U256::from_little_endian(key.as_le_slice())), + U256::from_little_endian(slot.present_value().as_le_slice()), + ); + } + } + account_updates.push(account_update) + } + account_updates + } + EvmState::Execution(db) => { + // Update accounts + let mut account_updates = Vec::new(); + for (revm_address, account) in &db.accounts { + if account.account_state == AccountState::None { + // EVM didn't interact with this account + continue; + } + + let address = Address::from_slice(revm_address.0.as_slice()); + // Remove account from DB if destroyed + if account.account_state == AccountState::NotExisting { + account_updates.push(AccountUpdate::removed(address)); + continue; + } + + // If account is empty, do not add to the database + if account.info().is_some_and(|acc_info| acc_info.is_empty()) { + continue; + } + + // Apply account changes to DB + let mut account_update = AccountUpdate::new(address); + // Update account info in DB + if let Some(new_acc_info) = account.info() { + // If code changed, update + if matches!(db.db.accounts.get(&address), Some(account) if B256::from(account.code_hash.0) != new_acc_info.code_hash) + { + account_update.code = new_acc_info + .code + .map(|code| bytes::Bytes::copy_from_slice(code.bytes_slice())); + } + + let account_info = AccountInfo { + code_hash: H256::from_slice(new_acc_info.code_hash.as_slice()), + balance: U256::from_little_endian(new_acc_info.balance.as_le_slice()), + nonce: new_acc_info.nonce, + }; + account_update.info = Some(account_info); + } + // Update account storage in DB + for (key, slot) in account.storage.iter() { + // TODO check if we need to remove the value from our db when value is zero + // if slot.present_value().is_zero() { + // account_update.removed_keys.push(H256::from_uint(&U256::from_little_endian(key.as_le_slice()))) + // } + account_update.added_storage.insert( + H256::from_uint(&U256::from_little_endian(key.as_le_slice())), + U256::from_little_endian(slot.as_le_slice()), + ); + } + account_updates.push(account_update) + } + account_updates + } + } +} diff --git a/crates/vm/backends/revm.rs b/crates/vm/backends/revm_b.rs similarity index 69% rename from crates/vm/backends/revm.rs rename to crates/vm/backends/revm_b.rs index 5e9b504362..74be0b39d9 100644 --- a/crates/vm/backends/revm.rs +++ b/crates/vm/backends/revm_b.rs @@ -1,14 +1,16 @@ use super::constants::{ - BEACON_ROOTS_ADDRESS_STR, CONSOLIDATION_REQUEST_PREDEPLOY_ADDRESS, HISTORY_STORAGE_ADDRESS_STR, - SYSTEM_ADDRESS_STR, WITHDRAWAL_REQUEST_PREDEPLOY_ADDRESS, + BEACON_ROOTS_ADDRESS, CONSOLIDATION_REQUEST_PREDEPLOY_ADDRESS, HISTORY_STORAGE_ADDRESS, + SYSTEM_ADDRESS, WITHDRAWAL_REQUEST_PREDEPLOY_ADDRESS, }; +use super::BlockExecutionResult; +use crate::backends::get_state_transitions; + use crate::spec_id; use crate::EvmError; use crate::EvmState; use crate::ExecutionResult; -use ethrex_common::types::requests::Requests; -use ethrex_storage::error::StoreError; -use lazy_static::lazy_static; +use ethrex_storage::{error::StoreError, AccountUpdate}; + use revm::{ inspectors::TracerEip3155, primitives::{BlobExcessGasAndPrice, BlockEnv, TxEnv, B256}, @@ -16,9 +18,12 @@ use revm::{ }; use revm_inspectors::access_list::AccessListInspector; // Rename imported types for clarity -use ethrex_common::types::{ - Block, BlockHeader, GenericTransaction, Receipt, Transaction, TxKind, Withdrawal, GWEI_TO_WEI, - INITIAL_BASE_FEE, +use ethrex_common::{ + types::{ + requests::Requests, Block, BlockHeader, GenericTransaction, Receipt, Transaction, TxKind, + Withdrawal, GWEI_TO_WEI, INITIAL_BASE_FEE, + }, + Address, }; use revm_primitives::Bytes; use revm_primitives::{ @@ -28,70 +33,192 @@ use revm_primitives::{ }; use std::cmp::min; +#[derive(Debug)] +pub struct REVM; + #[cfg(feature = "l2")] use crate::mods; -/// Executes all transactions in a block and returns their receipts. -pub fn execute_block( - block: &Block, - state: &mut EvmState, -) -> Result<(Vec, Vec), EvmError> { - let block_header = &block.header; - let spec_id = spec_id(&state.chain_config()?, block_header.timestamp); - //eip 4788: execute beacon_root_contract_call before block transactions - cfg_if::cfg_if! { - if #[cfg(not(feature = "l2"))] { - //eip 4788: execute beacon_root_contract_call before block transactions - if block_header.parent_beacon_block_root.is_some() && spec_id >= SpecId::CANCUN { - beacon_root_contract_call(state, block_header, spec_id)?; +/// The struct implements the following functions: +/// [REVM::execute_block] +/// [REVM::execute_tx] +/// [REVM::get_state_transitions] +/// [REVM::process_withdrawals] +impl REVM { + pub fn execute_block( + block: &Block, + state: &mut EvmState, + ) -> Result { + let block_header = &block.header; + let spec_id: SpecId = spec_id(&state.chain_config()?, block_header.timestamp); + cfg_if::cfg_if! { + if #[cfg(not(feature = "l2"))] { + if block_header.parent_beacon_block_root.is_some() && spec_id >= SpecId::CANCUN { + Self::beacon_root_contract_call(block_header, state)?; + } + + //eip 2935: stores parent block hash in system contract + if spec_id >= SpecId::PRAGUE { + Self::process_block_hash_history(block_header, state)?; + } } + } + let mut receipts = Vec::new(); + let mut cumulative_gas_used = 0; + + for tx in block.body.transactions.iter() { + let result = Self::execute_tx(tx, block_header, state, spec_id)?; + cumulative_gas_used += result.gas_used(); + let receipt = Receipt::new( + tx.tx_type(), + result.is_success(), + cumulative_gas_used, + result.logs(), + ); + receipts.push(receipt); + } + + if let Some(withdrawals) = &block.body.withdrawals { + Self::process_withdrawals(state, withdrawals)?; + } - //eip 2935: stores parent block hash in system contract - if spec_id >= SpecId::PRAGUE { - process_block_hash_history(state, block_header, spec_id)?; + cfg_if::cfg_if! { + if #[cfg(not(feature = "l2"))] { + let requests = extract_all_requests(&receipts, state, block_header)?; + } else { + let requests = Default::default(); } } + + let account_updates = get_state_transitions(state); + + Ok(BlockExecutionResult { + receipts, + requests, + account_updates, + }) } - let mut receipts = Vec::new(); - let mut cumulative_gas_used = 0; - - for transaction in block.body.transactions.iter() { - let result = execute_tx(transaction, block_header, state, spec_id)?; - cumulative_gas_used += result.gas_used(); - let receipt = Receipt::new( - transaction.tx_type(), - result.is_success(), - cumulative_gas_used, - result.logs(), - ); - receipts.push(receipt); + pub fn execute_tx( + tx: &Transaction, + header: &BlockHeader, + state: &mut EvmState, + spec_id: SpecId, + ) -> Result { + let block_env = block_env(header, spec_id); + let tx_env = tx_env(tx); + run_evm(tx_env, block_env, state, spec_id) } - if let Some(withdrawals) = &block.body.withdrawals { - process_withdrawals(state, withdrawals)?; + pub fn get_state_transitions( + initial_state: &mut EvmState, + ) -> Result, EvmError> { + Ok(get_state_transitions(initial_state)) } - cfg_if::cfg_if! { - if #[cfg(not(feature = "l2"))] { - let requests = extract_all_requests(&receipts, state, block_header)?; - } else { - let requests = Default::default(); + pub fn process_withdrawals( + initial_state: &mut EvmState, + withdrawals: &[Withdrawal], + ) -> Result<(), StoreError> { + match initial_state { + EvmState::Store(db) => { + //balance_increments is a vector of tuples (Address, increment as u128) + let balance_increments = withdrawals + .iter() + .filter(|withdrawal| withdrawal.amount > 0) + .map(|withdrawal| { + ( + RevmAddress::from_slice(withdrawal.address.as_bytes()), + (withdrawal.amount as u128 * GWEI_TO_WEI as u128), + ) + }) + .collect::>(); + + db.increment_balances(balance_increments)?; + } + EvmState::Execution(_) => { + // TODO: We should check withdrawals are valid + // (by checking that accounts exist if this is the only error) but there's no state to + // change. + } } + Ok(()) } - Ok((receipts, requests)) -} -// Executes a single tx, doesn't perform state transitions -pub fn execute_tx( - tx: &Transaction, - header: &BlockHeader, - state: &mut EvmState, - spec_id: SpecId, -) -> Result { - let block_env = block_env(header, spec_id); - let tx_env = tx_env(tx); - run_evm(tx_env, block_env, state, spec_id) + // SYSTEM CONTRACTS + pub fn beacon_root_contract_call( + block_header: &BlockHeader, + state: &mut EvmState, + ) -> Result<(), EvmError> { + let beacon_root = match block_header.parent_beacon_block_root { + None => { + return Err(EvmError::Header( + "parent_beacon_block_root field is missing".to_string(), + )) + } + Some(beacon_root) => beacon_root, + }; + + generic_system_contract_revm( + block_header, + Bytes::copy_from_slice(beacon_root.as_bytes()), + state, + *BEACON_ROOTS_ADDRESS, + *SYSTEM_ADDRESS, + )?; + Ok(()) + } + pub fn process_block_hash_history( + block_header: &BlockHeader, + state: &mut EvmState, + ) -> Result<(), EvmError> { + generic_system_contract_revm( + block_header, + Bytes::copy_from_slice(block_header.parent_hash.as_bytes()), + state, + *HISTORY_STORAGE_ADDRESS, + *SYSTEM_ADDRESS, + )?; + Ok(()) + } + pub(crate) fn read_withdrawal_requests( + block_header: &BlockHeader, + state: &mut EvmState, + ) -> Option> { + let tx_result = generic_system_contract_revm( + block_header, + Bytes::new(), + state, + *WITHDRAWAL_REQUEST_PREDEPLOY_ADDRESS, + *SYSTEM_ADDRESS, + ) + .ok()?; + + if tx_result.is_success() { + Some(tx_result.output().into()) + } else { + None + } + } + pub(crate) fn dequeue_consolidation_requests( + block_header: &BlockHeader, + state: &mut EvmState, + ) -> Option> { + let tx_result = generic_system_contract_revm( + block_header, + Bytes::new(), + state, + *CONSOLIDATION_REQUEST_PREDEPLOY_ADDRESS, + *SYSTEM_ADDRESS, + ) + .ok()?; + + if tx_result.is_success() { + Some(tx_result.output().into()) + } else { + None + } + } } /// Runs the transaction and returns the result, but does not commit it. @@ -401,103 +528,22 @@ fn adjust_disabled_base_fee( } } -/// Calls the eip4788 beacon block root system call contract -/// As of the Cancun hard-fork, parent_beacon_block_root needs to be present in the block header. -pub fn beacon_root_contract_call( - state: &mut EvmState, - header: &BlockHeader, - spec_id: SpecId, -) -> Result { - lazy_static! { - static ref CONTRACT_ADDRESS: RevmAddress = - RevmAddress::from_slice(&hex::decode(BEACON_ROOTS_ADDRESS_STR).unwrap()); - }; - - let beacon_root = header.parent_beacon_block_root.ok_or(EvmError::Header( - "parent_beacon_block_root field is missing".to_string(), - ))?; - - let calldata = revm::primitives::Bytes::copy_from_slice(beacon_root.as_bytes()); - generic_system_call(*CONTRACT_ADDRESS, calldata, state, header, spec_id) -} - -/// Calls the EIP-2935 process block hashes history system call contract -/// NOTE: This was implemented by making use of an EVM system contract, but can be changed to a -/// direct state trie update after the verkle fork, as explained in https://eips.ethereum.org/EIPS/eip-2935 -pub fn process_block_hash_history( - state: &mut EvmState, - header: &BlockHeader, - spec_id: SpecId, -) -> Result { - lazy_static! { - static ref CONTRACT_ADDRESS: RevmAddress = - RevmAddress::from_slice(&hex::decode(HISTORY_STORAGE_ADDRESS_STR).unwrap(),); - }; - - let calldata = revm::primitives::Bytes::copy_from_slice(header.parent_hash.as_bytes()); - generic_system_call(*CONTRACT_ADDRESS, calldata, state, header, spec_id) -} - -fn read_withdrawal_requests( - state: &mut EvmState, - header: &BlockHeader, - spec_id: SpecId, -) -> Option> { - lazy_static! { - static ref CONTRACT_ADDRESS: RevmAddress = - RevmAddress::from_slice(&hex::decode(WITHDRAWAL_REQUEST_PREDEPLOY_ADDRESS).unwrap(),); - }; - - let tx_result = - generic_system_call(*CONTRACT_ADDRESS, Bytes::new(), state, header, spec_id).ok()?; - - if tx_result.is_success() { - Some(tx_result.output().into()) - } else { - None - } -} - -fn dequeue_consolidation_requests( - state: &mut EvmState, - header: &BlockHeader, - spec_id: SpecId, -) -> Option> { - lazy_static! { - static ref CONTRACT_ADDRESS: RevmAddress = RevmAddress::from_slice( - &hex::decode(CONSOLIDATION_REQUEST_PREDEPLOY_ADDRESS).unwrap(), - ); - }; - - let tx_result = - generic_system_call(*CONTRACT_ADDRESS, Bytes::new(), state, header, spec_id).ok()?; - - if tx_result.is_success() { - Some(tx_result.output().into()) - } else { - None - } -} - -pub fn generic_system_call( - contract_address: RevmAddress, +pub(crate) fn generic_system_contract_revm( + block_header: &BlockHeader, calldata: Bytes, state: &mut EvmState, - header: &BlockHeader, - spec_id: SpecId, + contract_address: Address, + system_address: Address, ) -> Result { - lazy_static! { - static ref SYSTEM_ADDRESS: RevmAddress = - RevmAddress::from_slice(&hex::decode(SYSTEM_ADDRESS_STR).unwrap()); - }; + let spec_id = spec_id(&state.chain_config()?, block_header.timestamp); let tx_env = TxEnv { - caller: *SYSTEM_ADDRESS, - transact_to: RevmTxKind::Call(contract_address), + caller: RevmAddress::from_slice(system_address.as_bytes()), + transact_to: RevmTxKind::Call(RevmAddress::from_slice(contract_address.as_bytes())), gas_limit: 30_000_000, data: calldata, ..Default::default() }; - let mut block_env = block_env(header, spec_id); + let mut block_env = block_env(block_header, spec_id); block_env.basefee = RevmU256::ZERO; block_env.gas_limit = RevmU256::from(30_000_000); @@ -512,7 +558,7 @@ pub fn generic_system_call( let transaction_result = evm.transact()?; let mut result_state = transaction_result.state; - result_state.remove(&*SYSTEM_ADDRESS); + result_state.remove(SYSTEM_ADDRESS.as_ref()); result_state.remove(&evm.block().coinbase); evm.context.evm.db.commit(result_state); @@ -559,8 +605,8 @@ pub fn extract_all_requests( ))?; let deposits = Requests::from_deposit_receipts(deposit_contract_address, receipts); - let withdrawals_data = read_withdrawal_requests(state, header, spec_id); - let consolidation_data = dequeue_consolidation_requests(state, header, spec_id); + let withdrawals_data = REVM::read_withdrawal_requests(header, state); + let consolidation_data = REVM::dequeue_consolidation_requests(header, state); let withdrawals = Requests::from_withdrawals_data(withdrawals_data.unwrap_or_default()); let consolidation = Requests::from_consolidation_data(consolidation_data.unwrap_or_default()); diff --git a/crates/vm/execution_db.rs b/crates/vm/execution_db.rs index f7605e415d..e1feb932e9 100644 --- a/crates/vm/execution_db.rs +++ b/crates/vm/execution_db.rs @@ -21,10 +21,11 @@ use revm_primitives::SpecId; use serde::{Deserialize, Serialize}; use crate::{ + backends::{self}, block_env, db::{evm_state, StoreWrapper}, errors::ExecutionDBError, - execute_block, get_state_transitions, spec_id, tx_env, EvmError, + spec_id, tx_env, EvmError, }; /// In-memory EVM database for single execution data. @@ -184,10 +185,8 @@ impl ExecutionDB { let mut state = evm_state(store.clone(), block.header.parent_hash); - execute_block(block, &mut state).map_err(Box::new)?; - - let account_updates = get_state_transitions(&mut state); - Ok(account_updates) + let result = backends::revm_b::REVM::execute_block(block, &mut state).map_err(Box::new)?; + Ok(result.account_updates) } pub fn get_chain_config(&self) -> ChainConfig { diff --git a/crates/vm/levm/src/vm.rs b/crates/vm/levm/src/vm.rs index fc92f58bba..d52a0ef60f 100644 --- a/crates/vm/levm/src/vm.rs +++ b/crates/vm/levm/src/vm.rs @@ -21,7 +21,7 @@ use bytes::Bytes; use ethrex_common::{ types::{ tx_fields::{AccessList, AuthorizationList}, - Fork, ForkBlobSchedule, TxKind, + BlockHeader, ChainConfig, Fork, ForkBlobSchedule, TxKind, }, Address, H256, U256, }; @@ -92,6 +92,16 @@ impl EVMConfig { } } + pub fn new_from_chain_config(chain_config: &ChainConfig, block_header: &BlockHeader) -> Self { + let fork = chain_config.fork(block_header.timestamp); + + let blob_schedule = chain_config + .get_fork_blob_schedule(block_header.timestamp) + .unwrap_or_else(|| EVMConfig::canonical_values(fork)); + + EVMConfig::new(fork, blob_schedule) + } + /// This function is used for running the EF tests. If you don't /// have acces to a EVMConfig (mainly in the form of a /// genesis.json file) you can use this function to get the diff --git a/crates/vm/vm.rs b/crates/vm/vm.rs index cfc465db03..93d8b3056f 100644 --- a/crates/vm/vm.rs +++ b/crates/vm/vm.rs @@ -9,19 +9,16 @@ mod mods; use backends::EVM; use db::EvmState; -use crate::backends::revm::*; +use crate::backends::revm_b::*; use ethrex_common::{ types::{ - tx_fields::AccessList, AccountInfo, BlockHeader, ChainConfig, Fork, GenericTransaction, - INITIAL_BASE_FEE, + tx_fields::AccessList, BlockHeader, ChainConfig, Fork, GenericTransaction, INITIAL_BASE_FEE, }, - Address, BigEndianHash, H256, U256, + Address, H256, }; -use ethrex_storage::AccountUpdate; use revm::{ - db::{states::bundle_state::BundleRetention, AccountState, AccountStatus}, inspector_handle_register, - primitives::{BlockEnv, TxEnv, B256}, + primitives::{BlockEnv, TxEnv}, Evm, }; // Rename imported types for clarity @@ -38,6 +35,10 @@ use std::sync::OnceLock; // Then, we can retrieve the evm with: // EVM_BACKEND.get(); -> returns Option pub static EVM_BACKEND: OnceLock = OnceLock::new(); +/// Function used to access the global variable holding the chosen backend. +pub fn get_evm_backend_or_default() -> EVM { + EVM_BACKEND.get().unwrap_or(&EVM::default()).clone() +} // ================== Commonly used functions ====================== @@ -134,134 +135,6 @@ fn create_access_list_inner( Ok((tx_result.result.into(), access_list)) } -/// Merges transitions stored when executing transactions and returns the resulting account updates -/// Doesn't update the DB -pub fn get_state_transitions(state: &mut EvmState) -> Vec { - match state { - EvmState::Store(db) => { - db.merge_transitions(BundleRetention::PlainState); - let bundle = db.take_bundle(); - - // Update accounts - let mut account_updates = Vec::new(); - for (address, account) in bundle.state() { - if account.status.is_not_modified() { - continue; - } - let address = Address::from_slice(address.0.as_slice()); - // Remove account from DB if destroyed (Process DestroyedChanged as changed account) - if matches!( - account.status, - AccountStatus::Destroyed | AccountStatus::DestroyedAgain - ) { - account_updates.push(AccountUpdate::removed(address)); - continue; - } - - // If account is empty, do not add to the database - if account - .account_info() - .is_some_and(|acc_info| acc_info.is_empty()) - { - continue; - } - - // Apply account changes to DB - let mut account_update = AccountUpdate::new(address); - // If the account was changed then both original and current info will be present in the bundle account - if account.is_info_changed() { - // Update account info in DB - if let Some(new_acc_info) = account.account_info() { - let code_hash = H256::from_slice(new_acc_info.code_hash.as_slice()); - let account_info = AccountInfo { - code_hash, - balance: U256::from_little_endian(new_acc_info.balance.as_le_slice()), - nonce: new_acc_info.nonce, - }; - account_update.info = Some(account_info); - if account.is_contract_changed() { - // Update code in db - if let Some(code) = new_acc_info.code { - account_update.code = Some(code.original_bytes().clone().0); - } - } - } - } - // Update account storage in DB - for (key, slot) in account.storage.iter() { - if slot.is_changed() { - // TODO check if we need to remove the value from our db when value is zero - // if slot.present_value().is_zero() { - // account_update.removed_keys.push(H256::from_uint(&U256::from_little_endian(key.as_le_slice()))) - // } - account_update.added_storage.insert( - H256::from_uint(&U256::from_little_endian(key.as_le_slice())), - U256::from_little_endian(slot.present_value().as_le_slice()), - ); - } - } - account_updates.push(account_update) - } - account_updates - } - EvmState::Execution(db) => { - // Update accounts - let mut account_updates = Vec::new(); - for (revm_address, account) in &db.accounts { - if account.account_state == AccountState::None { - // EVM didn't interact with this account - continue; - } - - let address = Address::from_slice(revm_address.0.as_slice()); - // Remove account from DB if destroyed - if account.account_state == AccountState::NotExisting { - account_updates.push(AccountUpdate::removed(address)); - continue; - } - - // If account is empty, do not add to the database - if account.info().is_some_and(|acc_info| acc_info.is_empty()) { - continue; - } - - // Apply account changes to DB - let mut account_update = AccountUpdate::new(address); - // Update account info in DB - if let Some(new_acc_info) = account.info() { - // If code changed, update - if matches!(db.db.accounts.get(&address), Some(account) if B256::from(account.code_hash.0) != new_acc_info.code_hash) - { - account_update.code = new_acc_info - .code - .map(|code| bytes::Bytes::copy_from_slice(code.bytes_slice())); - } - - let account_info = AccountInfo { - code_hash: H256::from_slice(new_acc_info.code_hash.as_slice()), - balance: U256::from_little_endian(new_acc_info.balance.as_le_slice()), - nonce: new_acc_info.nonce, - }; - account_update.info = Some(account_info); - } - // Update account storage in DB - for (key, slot) in account.storage.iter() { - // TODO check if we need to remove the value from our db when value is zero - // if slot.present_value().is_zero() { - // account_update.removed_keys.push(H256::from_uint(&U256::from_little_endian(key.as_le_slice()))) - // } - account_update.added_storage.insert( - H256::from_uint(&U256::from_little_endian(key.as_le_slice())), - U256::from_little_endian(slot.as_le_slice()), - ); - } - account_updates.push(account_update) - } - account_updates - } - } -} - /// Returns the spec id according to the block timestamp and the stored chain config /// WARNING: Assumes at least Merge fork is active pub fn spec_id(chain_config: &ChainConfig, block_timestamp: u64) -> SpecId {