feat(executor): Execute block of transactions and return tx patches (#238)

* wip executor commit

* finish commit, make execution as functions

* rlp block encoding, fixing proof

* Block execution and test example
This commit is contained in:
rakita
2022-11-22 22:57:22 +01:00
committed by GitHub
parent a523cb7024
commit 7e693046c6
11 changed files with 672 additions and 172 deletions

5
Cargo.lock generated
View File

@ -3216,6 +3216,7 @@ dependencies = [
"auto_impl",
"eyre",
"hash-db",
"hashbrown 0.12.3",
"plain_hasher",
"reth-consensus",
"reth-interfaces",
@ -3502,9 +3503,9 @@ dependencies = [
[[package]]
name = "revm"
version = "2.1.0"
version = "2.3.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "87344ffd3eec06b568e1fc69c225e4cbd8d68d8d9051b6d2652d596947efa1ce"
checksum = "d3d296f0199135e573ba817ed0024d0d4eafa7add1c9a8dd08488c3515598156"
dependencies = [
"arrayref",
"auto_impl",

View File

@ -337,6 +337,21 @@ where
}
}
pub fn encode_iter<'a, K>(i: impl Iterator<Item = &'a K> + Clone, out: &mut dyn BufMut)
where
K: Encodable + 'a,
{
let mut h = Header { list: true, payload_length: 0 };
for x in i.clone() {
h.payload_length += x.length();
}
h.encode(out);
for x in i {
x.encode(out);
}
}
pub fn encode_fixed_size<E: MaxEncodedLen<LEN>, const LEN: usize>(v: &E) -> ArrayVec<u8, LEN> {
let mut out = ArrayVec::from([0_u8; LEN]);
@ -379,6 +394,12 @@ mod tests {
out1
}
fn encoded_iter<'a, T: Encodable + 'a>(iter: impl Iterator<Item = &'a T> + Clone) -> BytesMut {
let mut out = BytesMut::new();
encode_iter(iter, &mut out);
out
}
#[test]
fn rlp_str() {
assert_eq!(encoded("")[..], hex!("80")[..]);
@ -534,6 +555,15 @@ mod tests {
assert_eq!(encoded_list(&[0xFFCCB5_u64, 0xFFC0B5_u64]), &hex!("c883ffccb583ffc0b5")[..]);
}
#[test]
fn rlp_iter() {
assert_eq!(encoded_iter::<u64>([].iter()), &hex!("c0")[..]);
assert_eq!(
encoded_iter([0xFFCCB5_u64, 0xFFC0B5_u64].iter()),
&hex!("c883ffccb583ffc0b5")[..]
);
}
#[cfg(feature = "smol_str")]
#[test]
fn rlp_smol_str() {

View File

@ -11,8 +11,8 @@ pub use bytes::BufMut;
pub use decode::{Decodable, DecodeError, Rlp};
pub use encode::{
const_add, encode_fixed_size, encode_list, length_of_length, list_length, Encodable,
MaxEncodedLen, MaxEncodedLenAssoc,
const_add, encode_fixed_size, encode_iter, encode_list, length_of_length, list_length,
Encodable, MaxEncodedLen, MaxEncodedLenAssoc,
};
pub use types::*;

View File

@ -13,7 +13,9 @@ reth-interfaces = { path = "../interfaces" }
reth-rlp = { path = "../common/rlp" }
reth-consensus = { path = "../consensus" }
revm = "2.1"
revm = "2.2"
# remove from reth and reexport from revm
hashbrown = "0.12"
# common
async-trait = "0.1.57"

View File

@ -1,5 +1,85 @@
//! Reth block execution/validation configuration and constants
use reth_primitives::{BlockNumber, U256};
/// Configuration for executor
#[derive(Debug, Clone)]
pub struct Config {}
pub struct Config {
/// Chain id.
pub chain_id: U256,
/// Spec upgrades.
pub spec_upgrades: SpecUpgrades,
}
impl Config {
/// Create new config for ethereum.
pub fn new_ethereum() -> Self {
Self { chain_id: 1.into(), spec_upgrades: SpecUpgrades::new_ethereum() }
}
}
/// Spec with there ethereum codenames.
#[derive(Debug, Clone)]
#[allow(missing_docs)]
pub struct SpecUpgrades {
pub frontier: BlockNumber,
//pub frontier_thawing: BlockNumber,
pub homestead: BlockNumber,
//pub dao_fork: BlockNumber,
pub tangerine_whistle: BlockNumber,
pub spurious_dragon: BlockNumber,
pub byzantium: BlockNumber,
//pub constantinople: BlockNumber,
pub petersburg: BlockNumber, //Overrider Constantinople
pub istanbul: BlockNumber,
//pub muir_glacier: BlockNumber,
pub berlin: BlockNumber,
pub london: BlockNumber,
//pub arrow_glacier: BlockNumber,
//pub gray_glacier: BlockNumber,
pub paris: BlockNumber, // Aka the merge
pub shanghai: BlockNumber,
}
impl SpecUpgrades {
/// Ethereum mainnet spec
pub fn new_ethereum() -> Self {
Self {
frontier: 0,
//frontier_thawing: 200000,
homestead: 1150000,
//dao_fork: 1920000,
tangerine_whistle: 2463000,
spurious_dragon: 2675000,
byzantium: 4370000,
//constantinople: 7280000,
petersburg: 7280000, //Overrider Constantinople
istanbul: 9069000,
//muir_glacier: 9200000,
berlin: 12244000,
london: 12965000,
//arrow_glacier: 13773000,
//gray_glacier: 15050000,
paris: 15537394, // TheMerge,
shanghai: u64::MAX,
}
}
/// return revm_spec from spec configuration.
pub fn revm_spec(&self, for_block: BlockNumber) -> revm::SpecId {
match for_block {
b if self.shanghai >= b => revm::MERGE_EOF,
b if self.paris >= b => revm::MERGE,
b if self.london >= b => revm::LONDON,
b if self.berlin >= b => revm::BERLIN,
b if self.istanbul >= b => revm::ISTANBUL,
b if self.petersburg >= b => revm::PETERSBURG,
b if self.byzantium >= b => revm::BYZANTIUM,
b if self.spurious_dragon >= b => revm::SPURIOUS_DRAGON,
b if self.tangerine_whistle >= b => revm::TANGERINE,
b if self.homestead >= b => revm::HOMESTEAD,
b if self.frontier >= b => revm::FRONTIER,
_ => panic!("wrong configuration"),
}
}
}

View File

@ -1,13 +1,18 @@
use crate::{
revm_wrap::{self, State, SubState},
revm_wrap::{self, SubState},
Config,
};
use reth_interfaces::{
executor::{BlockExecutor, Error},
provider::StateProvider,
use hashbrown::hash_map::Entry;
use reth_interfaces::{executor::Error, provider::StateProvider};
use reth_primitives::{
bloom::logs_bloom, Account, Address, Bloom, Log, Receipt, SealedHeader,
TransactionSignedEcRecovered, H256, U256,
};
use reth_primitives::{bloom::logs_bloom, BlockLocked, Bloom, Log, Receipt};
use revm::{AnalysisKind, ExecutionResult, SpecId, EVM};
use revm::{
db::AccountState, Account as RevmAccount, AccountInfo, AnalysisKind, Bytecode, ExecutionResult,
EVM,
};
use std::collections::BTreeMap;
/// Main block executor
pub struct Executor {
@ -15,101 +20,474 @@ pub struct Executor {
pub config: Config,
}
impl Executor {
/// Create new Executor
pub fn new(config: Config) -> Self {
Self { config }
/// Diff change set that is neede for creating history index and updating current world state.
#[derive(Debug, Clone)]
pub struct AccountChangeSet {
/// Old and New account.
/// If account is deleted (selfdestructed) we would have (Some,None).
/// If account is newly created we would have (None,Some).
/// If it is only a storage change we would have (None,None)
/// If account is changed (balance,nonce) we would have (Some,Some)
pub account: (Option<Account>, Option<Account>),
/// Storage containing key -> (OldValue,NewValue). in case that old value is not existing
/// we can expect to have U256::zero(), same with new value.
pub storage: BTreeMap<U256, (U256, U256)>,
/// Just to make sure that we are taking selfdestruct cleaning we have this field that wipes
/// storage. There are instances where storage is changed but account is not touched, so we
/// can't take into account that if new account is None that it is selfdestruct.
pub wipe_storage: bool,
}
/// Commit chgange to database and return change diff that is used to update state and create
/// history index
///
/// ChangeDiff consists of:
/// address->AccountChangeSet (It contains old and new account info,storage wipe flag, and
/// old/new storage) bytecode_hash->bytecodes mapping
///
/// BTreeMap is used to have sorted values
pub fn commit_changes<DB: StateProvider>(
db: &mut SubState<DB>,
changes: hashbrown::HashMap<Address, RevmAccount>,
) -> (BTreeMap<Address, AccountChangeSet>, BTreeMap<H256, Bytecode>) {
let mut change = BTreeMap::new();
let mut new_bytecodes = BTreeMap::new();
// iterate over all changed accounts
for (address, account) in changes {
if account.is_destroyed {
// get old account that we are destroying.
let db_account = match db.accounts.entry(address) {
Entry::Occupied(entry) => entry.into_mut(),
Entry::Vacant(_entry) => {
panic!("Left panic to critically jumpout if happens as this should not hapen.");
}
};
// Insert into `change` a old account and None for new account
// and mark storage to be mapped
change.insert(
address,
AccountChangeSet {
account: (
Some(Account {
nonce: db_account.info.nonce,
balance: db_account.info.balance,
bytecode_hash: Some(db_account.info.code_hash), /* TODO none if
* KECCAK_EMPTY */
}),
None,
),
storage: BTreeMap::new(),
wipe_storage: true,
},
);
// clear cached DB and mark account as not existing
db_account.storage.clear();
db_account.account_state = AccountState::NotExisting;
db_account.info = AccountInfo::default();
continue
} else {
// check if account code is new or old.
// does it exist inside cached contracts if it doesn't it is new bytecode that
// we are inserting inside `change`
if let Some(ref code) = account.info.code {
if !code.is_empty() {
match db.contracts.entry(account.info.code_hash) {
Entry::Vacant(entry) => {
entry.insert(code.clone());
new_bytecodes.insert(account.info.code_hash, code.clone());
}
Entry::Occupied(mut entry) => {
entry.insert(code.clone());
}
}
}
}
// get old account that is going to be overwritten or none if it does not exist
// and get new account that was just inserted. new account mut ref is used for
// inserting storage
let (old_account, has_change, new_account) = match db.accounts.entry(address) {
Entry::Vacant(entry) => {
let entry = entry.insert(Default::default());
entry.info = account.info.clone();
(None, true, entry)
}
Entry::Occupied(entry) => {
let entry = entry.into_mut();
let (old_account, has_change) =
if matches!(entry.account_state, AccountState::NotExisting) {
(None, true)
} else {
(Some(entry.info.clone()), entry.info != account.info)
};
entry.info = account.info.clone();
(old_account, has_change, entry)
}
};
// cast from revm type to reth type
let old_account = old_account.map(|acc| Account {
balance: acc.balance,
nonce: acc.nonce,
bytecode_hash: Some(acc.code_hash), // TODO if KECCAK_EMPTY return None
});
let mut wipe_storage = false;
new_account.account_state = if account.storage_cleared {
new_account.storage.clear();
wipe_storage = true;
AccountState::StorageCleared
} else {
AccountState::Touched
};
// Insert storage.
let mut storage = BTreeMap::new();
// insert storage into new db account.
new_account.storage.extend(account.storage.into_iter().map(|(key, value)| {
storage.insert(key, (value.original_value(), value.present_value()));
(key, value.present_value())
}));
// Insert into change.
change.insert(
address,
AccountChangeSet {
account: if has_change {
(
old_account,
Some(new_account).map(|acc| {
Account {
balance: acc.info.balance,
nonce: acc.info.nonce,
bytecode_hash: Some(acc.info.code_hash), // TODO if KECCAK_EMPTY return None
}
}),
)
} else {
(None, None)
},
storage,
wipe_storage,
},
);
}
}
(change, new_bytecodes)
}
/// After transaction is executed this structure contain
/// every change to state that this transaction made and its old values
/// so that history account table can be updated. Receipts and new bytecodes
/// from created bytecodes.
#[derive(Debug, Clone)]
pub struct TransactionStatePatch {
/// Transaction receipt
pub receipt: Receipt,
/// State change that this transaction made on state.
pub state_diff: BTreeMap<Address, AccountChangeSet>,
/// new bytecode created as result of transaction execution.
pub new_bytecodes: BTreeMap<H256, Bytecode>,
}
/// Execute and verify block
pub fn execute_and_verify_receipt<DB: StateProvider>(
header: &SealedHeader,
transactions: &[TransactionSignedEcRecovered],
config: &Config,
db: &mut SubState<DB>,
) -> Result<Vec<TransactionStatePatch>, Error> {
let transaction_patches = execute(header, transactions, config, db)?;
let receipts_iter = transaction_patches.iter().map(|patch| &patch.receipt);
verify_receipt(header.receipts_root, header.logs_bloom, receipts_iter)?;
Ok(transaction_patches)
}
/// Verify receipts
pub fn verify_receipt<'a>(
expected_receipts_root: H256,
expected_logs_bloom: Bloom,
receipts: impl Iterator<Item = &'a Receipt> + Clone,
) -> Result<(), Error> {
// Check receipts root.
let receipts_root = reth_primitives::proofs::calculate_receipt_root(receipts.clone());
if receipts_root != expected_receipts_root {
return Err(Error::ReceiptRootDiff { got: receipts_root, expected: expected_receipts_root })
}
/// Verify block. Execute all transaction and compare results.
pub fn verify<DB: StateProvider>(&self, block: &BlockLocked, db: DB) -> Result<(), Error> {
let db = SubState::new(State::new(db));
let mut evm = EVM::new();
evm.database(db);
// Create header log bloom.
let logs_bloom = receipts.fold(Bloom::zero(), |bloom, r| bloom | r.bloom);
if logs_bloom != expected_logs_bloom {
return Err(Error::BloomLogDiff {
expected: Box::new(expected_logs_bloom),
got: Box::new(logs_bloom),
})
}
Ok(())
}
evm.env.cfg.chain_id = 1.into();
evm.env.cfg.spec_id = SpecId::LATEST;
evm.env.cfg.perf_all_precompiles_have_balance = true;
evm.env.cfg.perf_analyse_created_bytecodes = AnalysisKind::Raw;
/// Verify block. Execute all transaction and compare results.
/// Return diff is on transaction granularity. We are returning vector of
pub fn execute<DB: StateProvider>(
header: &SealedHeader,
transactions: &[TransactionSignedEcRecovered],
config: &Config,
db: &mut SubState<DB>,
) -> Result<Vec<TransactionStatePatch>, Error> {
let mut evm = EVM::new();
evm.database(db);
revm_wrap::fill_block_env(&mut evm.env.block, block);
let mut cumulative_gas_used = 0;
let mut receipts = Vec::with_capacity(block.body.len());
evm.env.cfg.chain_id = config.chain_id;
evm.env.cfg.spec_id = config.spec_upgrades.revm_spec(header.number);
evm.env.cfg.perf_all_precompiles_have_balance = true;
evm.env.cfg.perf_analyse_created_bytecodes = AnalysisKind::Raw;
for transaction in block.body.iter() {
// The sum of the transactions gas limit, Tg, and the gas utilised in this block prior,
// must be no greater than the blocks gasLimit.
let block_available_gas = block.gas_limit - cumulative_gas_used;
if transaction.gas_limit() > block_available_gas {
return Err(Error::TransactionGasLimitMoreThenAvailableBlockGas {
transaction_gas_limit: transaction.gas_limit(),
block_available_gas,
})
}
revm_wrap::fill_block_env(&mut evm.env.block, header);
let mut cumulative_gas_used = 0;
// output of verification
let mut transaction_patch = Vec::with_capacity(transactions.len());
// Fill revm structure.
revm_wrap::fill_tx_env(&mut evm.env.tx, transaction.as_ref());
// Execute transaction.
let ExecutionResult { exit_reason, gas_used, logs, .. } = evm.transact_commit();
// Fatal internal error.
if exit_reason == revm::Return::FatalExternalError {
return Err(Error::ExecutionFatalError)
}
// Success flag was added in `EIP-658: Embedding transaction status code in receipts`.
let is_success = matches!(
exit_reason,
revm::Return::Continue |
revm::Return::Stop |
revm::Return::Return |
revm::Return::SelfDestruct
);
// Add spend gas.
cumulative_gas_used += gas_used;
// Transform logs to reth format.
let logs: Vec<Log> = logs
.into_iter()
.map(|l| Log { address: l.address, topics: l.topics, data: l.data })
.collect();
// Push receipts for calculating header bloom filter.
receipts.push(Receipt {
tx_type: transaction.tx_type(),
success: is_success,
cumulative_gas_used,
bloom: logs_bloom(logs.iter()), // TODO
logs,
});
}
// TODO do state root.
// Check if gas used matches the value set in header.
if block.gas_used != cumulative_gas_used {
return Err(Error::BlockGasUsed { got: cumulative_gas_used, expected: block.gas_used })
}
// Check receipts root.
let receipts_root = reth_primitives::proofs::calculate_receipt_root(receipts.iter());
if block.receipts_root != receipts_root {
return Err(Error::ReceiptRootDiff { got: receipts_root, expected: block.receipts_root })
}
// Create header log bloom.
let expected_logs_bloom = receipts.iter().fold(Bloom::zero(), |bloom, r| bloom | r.bloom);
if expected_logs_bloom != block.logs_bloom {
return Err(Error::BloomLogDiff {
expected: Box::new(block.logs_bloom),
got: Box::new(expected_logs_bloom),
for transaction in transactions.iter() {
// The sum of the transactions gas limit, Tg, and the gas utilised in this block prior,
// must be no greater than the blocks gasLimit.
let block_available_gas = header.gas_limit - cumulative_gas_used;
if transaction.gas_limit() > block_available_gas {
return Err(Error::TransactionGasLimitMoreThenAvailableBlockGas {
transaction_gas_limit: transaction.gas_limit(),
block_available_gas,
})
}
Ok(())
// Fill revm structure.
revm_wrap::fill_tx_env(&mut evm.env.tx, transaction);
// Execute transaction.
let (ExecutionResult { exit_reason, gas_used, logs, .. }, state) = evm.transact();
// Fatal internal error.
if exit_reason == revm::Return::FatalExternalError {
return Err(Error::ExecutionFatalError)
}
// Success flag was added in `EIP-658: Embedding transaction status code in receipts`.
let is_success = matches!(
exit_reason,
revm::Return::Continue |
revm::Return::Stop |
revm::Return::Return |
revm::Return::SelfDestruct
);
// Add spend gas.
cumulative_gas_used += gas_used;
// Transform logs to reth format.
let logs: Vec<Log> = logs
.into_iter()
.map(|l| Log { address: l.address, topics: l.topics, data: l.data })
.collect();
// commit state
let (state_diff, new_bytecodes) = commit_changes(evm.db().unwrap(), state);
// Push transaction patch and calculte header bloom filter for receipt.
transaction_patch.push(TransactionStatePatch {
receipt: Receipt {
tx_type: transaction.tx_type(),
success: is_success,
cumulative_gas_used,
bloom: logs_bloom(logs.iter()),
logs,
},
state_diff,
new_bytecodes,
})
}
// Check if gas used matches the value set in header.
if header.gas_used != cumulative_gas_used {
return Err(Error::BlockGasUsed { got: cumulative_gas_used, expected: header.gas_used })
}
// TODO add validator block reward. Currently not added.
// https://github.com/foundry-rs/reth/issues/237
Ok(transaction_patch)
}
impl BlockExecutor for Executor {}
#[cfg(test)]
mod tests {
use std::collections::HashMap;
use crate::revm_wrap::State;
use reth_interfaces::provider::{AccountProvider, StateProvider};
use reth_primitives::{
hex_literal::hex, keccak256, Account, Address, BlockLocked, Bytes, StorageKey, H160, H256,
U256,
};
use reth_rlp::Decodable;
use super::*;
#[derive(Debug, Default, Clone, Eq, PartialEq)]
struct StateProviderTest {
accounts: HashMap<Address, (HashMap<StorageKey, U256>, Account)>,
contracts: HashMap<H256, Bytes>,
block_hash: HashMap<U256, H256>,
}
impl StateProviderTest {
/// Insert account.
fn insert_account(
&mut self,
address: Address,
mut account: Account,
bytecode: Option<Bytes>,
storage: HashMap<StorageKey, U256>,
) {
if let Some(bytecode) = bytecode {
let hash = keccak256(&bytecode);
account.bytecode_hash = Some(hash);
self.contracts.insert(hash, bytecode);
}
self.accounts.insert(address, (storage, account));
}
}
impl AccountProvider for StateProviderTest {
fn basic_account(&self, address: Address) -> reth_interfaces::Result<Option<Account>> {
let ret = Ok(self.accounts.get(&address).map(|(_, acc)| acc.clone()));
ret
}
}
impl StateProvider for StateProviderTest {
fn storage(
&self,
account: Address,
storage_key: reth_primitives::StorageKey,
) -> reth_interfaces::Result<Option<reth_primitives::StorageValue>> {
Ok(self
.accounts
.get(&account)
.map(|(storage, _)| storage.get(&storage_key).cloned())
.flatten())
}
fn bytecode_by_hash(&self, code_hash: H256) -> reth_interfaces::Result<Option<Bytes>> {
Ok(self.contracts.get(&code_hash).cloned())
}
fn block_hash(&self, number: U256) -> reth_interfaces::Result<Option<H256>> {
Ok(self.block_hash.get(&number).cloned())
}
}
#[test]
fn sanity_execution() {
// Got rlp block from: src/GeneralStateTestsFiller/stChainId/chainIdGasCostFiller.json
let mut block_rlp = hex!("f90262f901f9a075c371ba45999d87f4542326910a11af515897aebce5265d3f6acd1f1161f82fa01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347942adc25665018aa1fe0e6bc666dac8fc2697ff9baa098f2dcd87c8ae4083e7017a05456c14eea4b1db2032126e27b3b1563d57d7cc0a08151d548273f6683169524b66ca9fe338b9ce42bc3540046c828fd939ae23bcba03f4e5c2ec5b2170b711d97ee755c160457bb58d8daa338e835ec02ae6860bbabb901000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000083020000018502540be40082a8798203e800a00000000000000000000000000000000000000000000000000000000000000000880000000000000000f863f861800a8405f5e10094100000000000000000000000000000000000000080801ba07e09e26678ed4fac08a249ebe8ed680bf9051a5e14ad223e4b2b9d26e0208f37a05f6e3f188e3e6eab7d7d3b6568f5eac7d687b08d307d3154ccd8c87b4630509bc0").as_slice();
let block = BlockLocked::decode(&mut block_rlp).unwrap();
let mut db = StateProviderTest::default();
// pre staet
db.insert_account(
H160(hex!("1000000000000000000000000000000000000000")),
Account { balance: 0x00.into(), nonce: 0x00, bytecode_hash: None },
Some(hex!("5a465a905090036002900360015500").into()),
HashMap::new(),
);
let account3_old_info = Account {
balance: 0x3635c9adc5dea00000u128.into(),
nonce: 0x00,
bytecode_hash: Some(H256(hex!(
"c5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470"
))),
};
db.insert_account(
H160(hex!("a94f5374fce5edbc8e2a8697c15331677e6ebf0b")),
Account { balance: 0x3635c9adc5dea00000u128.into(), nonce: 0x00, bytecode_hash: None },
None,
HashMap::new(),
);
let mut config = Config::new_ethereum();
// make it berlin fork
config.spec_upgrades.berlin = 0;
let mut db = SubState::new(State::new(db));
let transactions: Vec<TransactionSignedEcRecovered> =
block.body.iter().map(|tx| tx.try_ecrecovered().unwrap()).collect();
// execute chain and verify receipts
let out =
execute_and_verify_receipt(&block.header, &transactions, &config, &mut db).unwrap();
assert_eq!(out.len(), 1, "Should executed one transaction");
let patch = out[0].clone();
assert_eq!(patch.new_bytecodes.len(), 0, "Should have zero new bytecodes");
let account1 = H160(hex!("1000000000000000000000000000000000000000"));
let _account1_info = Account {
balance: 0x00.into(),
nonce: 0x00,
bytecode_hash: Some(H256(hex!(
"c5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470"
))),
};
let account2 = H160(hex!("2adc25665018aa1fe0e6bc666dac8fc2697ff9ba"));
let account2_info = Account {
balance: (0x1bc16d674ece94bau128 - 0x1bc16d674ec80000u128).into(), /* TODO remove
* 2eth block
* reward */
nonce: 0x00,
bytecode_hash: Some(H256(hex!(
"c5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470"
))),
};
let account3 = H160(hex!("a94f5374fce5edbc8e2a8697c15331677e6ebf0b"));
let account3_info = Account {
balance: 0x3635c9adc5de996b46u128.into(),
nonce: 0x01,
bytecode_hash: Some(H256(hex!(
"c5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470"
))),
};
assert_eq!(
patch.state_diff.get(&account1).unwrap().account,
(None, None),
"No change to account"
);
assert_eq!(
patch.state_diff.get(&account2).unwrap().account,
(None, Some(account2_info)),
"New acccount"
);
assert_eq!(
patch.state_diff.get(&account3).unwrap().account,
(Some(account3_old_info), Some(account3_info)),
"Change to account state"
);
assert_eq!(patch.new_bytecodes.len(), 0, "No new bytecodes");
// check torage
let storage = &patch.state_diff.get(&account1).unwrap().storage;
assert_eq!(storage.len(), 1, "Only one storage change");
assert_eq!(
storage.get(&1.into()),
Some(&(0.into(), 2.into())),
"Storage change from 0 to 2 on slot 1"
);
}
}

View File

@ -1,5 +1,8 @@
use reth_interfaces::{provider::StateProvider, Error};
use reth_primitives::{BlockLocked, Transaction, TransactionKind, H160, H256, KECCAK_EMPTY, U256};
use reth_primitives::{
Header, Transaction, TransactionKind, TransactionSignedEcRecovered, H160, H256, KECCAK_EMPTY,
U256,
};
use revm::{
db::{CacheDB, DatabaseRef},
BlockEnv, TransactTo, TxEnv,
@ -9,7 +12,7 @@ use revm::{
pub type SubState<DB> = CacheDB<State<DB>>;
/// Wrapper around ExeuctorDb that implements revm database trait
pub struct State<DB: StateProvider>(DB);
pub struct State<DB: StateProvider>(pub DB);
impl<DB: StateProvider> State<DB> {
/// Create new State with generic ExecutorDb.
@ -63,18 +66,19 @@ impl<DB: StateProvider> DatabaseRef for State<DB> {
}
/// Fill block environment from Block.
pub fn fill_block_env(block_env: &mut BlockEnv, block: &BlockLocked) {
block_env.number = block.header.number.into();
block_env.coinbase = block.header.beneficiary;
block_env.timestamp = block.header.timestamp.into();
block_env.difficulty = block.header.difficulty;
block_env.basefee = block.header.base_fee_per_gas.unwrap_or_default().into();
block_env.gas_limit = block.header.gas_limit.into();
pub fn fill_block_env(block_env: &mut BlockEnv, header: &Header) {
block_env.number = header.number.into();
block_env.coinbase = header.beneficiary;
block_env.timestamp = header.timestamp.into();
block_env.difficulty = header.difficulty;
block_env.basefee = header.base_fee_per_gas.unwrap_or_default().into();
block_env.gas_limit = header.gas_limit.into();
}
/// Fill transaction environment from Transaction.
pub fn fill_tx_env(tx_env: &mut TxEnv, transaction: &Transaction) {
match transaction {
pub fn fill_tx_env(tx_env: &mut TxEnv, transaction: &TransactionSignedEcRecovered) {
tx_env.caller = transaction.signer();
match transaction.as_ref().as_ref() {
Transaction::Legacy { nonce, chain_id, gas_price, gas_limit, to, value, input } => {
tx_env.gas_limit = *gas_limit;
tx_env.gas_price = (*gas_price).into();

View File

@ -1,14 +1,14 @@
use crate::{Header, SealedHeader, Transaction, TransactionSigned, H256};
use reth_rlp::{Decodable, DecodeError, Encodable};
use crate::{Header, SealedHeader, TransactionSigned, H256};
use reth_rlp::{Decodable, DecodeError, Encodable, RlpDecodable, RlpEncodable};
use std::ops::Deref;
/// Ethereum full block.
#[derive(Debug, Clone, PartialEq, Eq, Default)]
#[derive(Debug, Clone, PartialEq, Eq, Default, RlpEncodable, RlpDecodable)]
pub struct Block {
/// Block header.
pub header: Header,
/// Transactions in this block.
pub body: Vec<Transaction>,
pub body: Vec<TransactionSigned>,
/// Ommers/uncles header
pub ommers: Vec<SealedHeader>,
}
@ -21,7 +21,7 @@ impl Deref for Block {
}
/// Sealed Ethereum full block.
#[derive(Debug, Clone, PartialEq, Eq, Default)]
#[derive(Debug, Clone, PartialEq, Eq, Default, RlpEncodable, RlpDecodable)]
pub struct BlockLocked {
/// Locked block header.
pub header: SealedHeader,

View File

@ -210,6 +210,23 @@ pub struct SealedHeader {
hash: BlockHash,
}
impl Encodable for SealedHeader {
fn encode(&self, out: &mut dyn BufMut) {
self.header.encode(out);
}
}
impl Decodable for SealedHeader {
fn decode(buf: &mut &[u8]) -> Result<Self, reth_rlp::DecodeError> {
let header = Header::decode(buf)?;
// TODO make this more performant, we are not encoding again for a hash.
// But i dont know how much of buf is the header or if takeing rlp::Header will
// going to consume those buf bytes.
let hash = header.hash_slow();
Ok(SealedHeader { header, hash })
}
}
impl AsRef<Header> for SealedHeader {
fn as_ref(&self) -> &Header {
&self.header

View File

@ -1,10 +1,9 @@
use crate::{keccak256, Bytes, Header, Log, Receipt, TransactionSigned, H256};
use ethers_core::utils::rlp::RlpStream;
use crate::{keccak256, Header, Log, Receipt, TransactionSigned, H256};
use hash_db::Hasher;
use hex_literal::hex;
use plain_hasher::PlainHasher;
use reth_rlp::Encodable;
use triehash::sec_trie_root;
use triehash::ordered_trie_root;
/// Keccak-256 hash of the RLP of an empty list, KEC("\xc0").
pub const EMPTY_LIST_HASH: H256 =
@ -36,71 +35,50 @@ impl Hasher for KeccakHasher {
pub fn calculate_transaction_root<'a>(
transactions: impl IntoIterator<Item = &'a TransactionSigned>,
) -> H256 {
sec_trie_root::<KeccakHasher, _, _, _>(
transactions
.into_iter()
.enumerate()
.map(|(index, tx)| {
// TODO replace with reth-rlp
let mut stream = RlpStream::new();
stream.append(&index);
let mut bytes = Vec::new();
tx.encode(&mut bytes);
(stream.out().freeze().into(), bytes)
})
.collect::<Vec<(Bytes, Vec<u8>)>>(),
)
ordered_trie_root::<KeccakHasher, _>(transactions.into_iter().map(|tx| {
let mut tx_rlp = Vec::new();
tx.encode_inner(&mut tx_rlp, false);
tx_rlp
}))
}
/// Calculates the receipt root for a header.
pub fn calculate_receipt_root<'a>(receipts: impl IntoIterator<Item = &'a Receipt>) -> H256 {
sec_trie_root::<KeccakHasher, _, _, _>(
receipts
.into_iter()
.enumerate()
.map(|(index, receipt)| {
let mut stream = RlpStream::new();
stream.append(&index);
let mut bytes = Vec::new();
receipt.encode(&mut bytes);
(stream.out().freeze().into(), bytes)
})
.collect::<Vec<(Bytes, Vec<u8>)>>(),
)
pub fn calculate_receipt_root<'a>(receipts: impl Iterator<Item = &'a Receipt>) -> H256 {
ordered_trie_root::<KeccakHasher, _>(receipts.into_iter().map(|receipt| {
let mut receipt_rlp = Vec::new();
receipt.encode(&mut receipt_rlp);
receipt_rlp
}))
}
/// Calculates the log root for a header.
pub fn calculate_log_root<'a>(logs: impl IntoIterator<Item = &'a Log>) -> H256 {
pub fn calculate_log_root<'a>(logs: impl Iterator<Item = &'a Log> + Clone) -> H256 {
//https://github.com/ethereum/go-ethereum/blob/356bbe343a30789e77bb38f25983c8f2f2bfbb47/cmd/evm/internal/t8ntool/execution.go#L255
let mut stream = RlpStream::new();
stream.begin_unbounded_list();
for log in logs {
stream.begin_list(3);
stream.append(&log.address);
stream.append_list(&log.topics);
stream.append(&log.data);
}
stream.finalize_unbounded_list();
let out = stream.out().freeze();
keccak256(out)
let mut logs_rlp = Vec::new();
reth_rlp::encode_iter(logs, &mut logs_rlp);
keccak256(logs_rlp)
}
/// Calculates the root hash for ommer/uncle headers.
pub fn calculate_ommers_root<'a>(_ommers: impl IntoIterator<Item = &'a Header>) -> H256 {
pub fn calculate_ommers_root<'a>(ommers: impl Iterator<Item = &'a Header> + Clone) -> H256 {
// RLP Encode
let mut stream = RlpStream::new();
stream.begin_unbounded_list();
/* TODO
for ommer in ommers {
stream.append(ommer)
}
*/
stream.finalize_unbounded_list();
let bytes = stream.out().freeze();
keccak256(bytes)
let mut ommers_rlp = Vec::new();
reth_rlp::encode_iter(ommers, &mut ommers_rlp);
keccak256(ommers_rlp)
}
// TODO state root
#[cfg(test)]
mod tests {
use crate::{hex_literal::hex, proofs::calculate_transaction_root, Block};
use reth_rlp::Decodable;
// TODO bloom
#[test]
fn check_transaction_root() {
let data = &hex!("f90262f901f9a092230ce5476ae868e98c7979cfc165a93f8b6ad1922acf2df62e340916efd49da01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347942adc25665018aa1fe0e6bc666dac8fc2697ff9baa02307107a867056ca33b5087e77c4174f47625e48fb49f1c70ced34890ddd88f3a08151d548273f6683169524b66ca9fe338b9ce42bc3540046c828fd939ae23bcba0c598f69a5674cae9337261b669970e24abc0b46e6d284372a239ec8ccbf20b0ab901000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000083020000018502540be40082a8618203e800a00000000000000000000000000000000000000000000000000000000000000000880000000000000000f863f861800a8405f5e10094100000000000000000000000000000000000000080801ba07e09e26678ed4fac08a249ebe8ed680bf9051a5e14ad223e4b2b9d26e0208f37a05f6e3f188e3e6eab7d7d3b6568f5eac7d687b08d307d3154ccd8c87b4630509bc0");
let block_rlp = &mut data.as_slice();
let block: Block = Block::decode(block_rlp).unwrap();
let tx_root = calculate_transaction_root(block.body.iter());
assert_eq!(block.transactions_root, tx_root, "Should be same");
}
}

View File

@ -629,7 +629,7 @@ impl TransactionSigned {
/// Inner encoding function that is used for both rlp [`Encodable`] trait and for calculating
/// hash that for eip2728 does not require rlp header
fn encode_inner(&self, out: &mut dyn bytes::BufMut, with_header: bool) {
pub(crate) fn encode_inner(&self, out: &mut dyn bytes::BufMut, with_header: bool) {
if let Transaction::Legacy { chain_id, .. } = self.transaction {
let header = Header { list: true, payload_length: self.payload_len() };
header.encode(out);
@ -729,6 +729,16 @@ pub struct TransactionSignedEcRecovered {
signer: Address,
}
impl Encodable for TransactionSignedEcRecovered {
fn length(&self) -> usize {
self.signed_transaction.length()
}
fn encode(&self, out: &mut dyn bytes::BufMut) {
self.signed_transaction.encode(out)
}
}
impl TransactionSignedEcRecovered {
/// Signer of transaction recovered from signature
pub fn signer(&self) -> Address {