anvil/eth/backend/mem/
storage.rs

1//! In-memory blockchain storage
2use crate::eth::{
3    backend::{
4        db::{
5            MaybeFullDatabase, SerializableBlock, SerializableHistoricalStates,
6            SerializableTransaction, StateDb,
7        },
8        env::Env,
9        mem::cache::DiskStateCache,
10    },
11    pool::transactions::PoolTransaction,
12};
13use alloy_consensus::{Header, constants::EMPTY_WITHDRAWALS};
14use alloy_eips::eip7685::EMPTY_REQUESTS_HASH;
15use alloy_primitives::{
16    B256, Bytes, U256,
17    map::{B256HashMap, HashMap},
18};
19use alloy_rpc_types::{
20    BlockId, BlockNumberOrTag, TransactionInfo as RethTransactionInfo,
21    trace::{
22        otterscan::{InternalOperation, OperationType},
23        parity::LocalizedTransactionTrace,
24    },
25};
26use anvil_core::eth::{
27    block::{Block, create_block},
28    transaction::{MaybeImpersonatedTransaction, ReceiptResponse, TransactionInfo},
29};
30use foundry_evm::{
31    backend::MemDb,
32    traces::{CallKind, ParityTraceBuilder, TracingInspectorConfig},
33};
34use foundry_primitives::FoundryReceiptEnvelope;
35use parking_lot::RwLock;
36use revm::{context::Block as RevmBlock, primitives::hardfork::SpecId};
37use std::{collections::VecDeque, fmt, path::PathBuf, sync::Arc, time::Duration};
38// use yansi::Paint;
39
40// === various limits in number of blocks ===
41
42pub const DEFAULT_HISTORY_LIMIT: usize = 500;
43const MIN_HISTORY_LIMIT: usize = 10;
44// 1hr of up-time at lowest 1s interval
45const MAX_ON_DISK_HISTORY_LIMIT: usize = 3_600;
46
47/// Represents the complete state of single block
48pub struct InMemoryBlockStates {
49    /// The states at a certain block
50    states: B256HashMap<StateDb>,
51    /// states which data is moved to disk
52    on_disk_states: B256HashMap<StateDb>,
53    /// How many states to store at most
54    in_memory_limit: usize,
55    /// minimum amount of states we keep in memory
56    min_in_memory_limit: usize,
57    /// maximum amount of states we keep on disk
58    ///
59    /// Limiting the states will prevent disk blow up, especially in interval mining mode
60    max_on_disk_limit: usize,
61    /// the oldest states written to disk
62    oldest_on_disk: VecDeque<B256>,
63    /// all states present, used to enforce `in_memory_limit`
64    present: VecDeque<B256>,
65    /// Stores old states on disk
66    disk_cache: DiskStateCache,
67}
68
69impl InMemoryBlockStates {
70    /// Creates a new instance with limited slots
71    pub fn new(in_memory_limit: usize, on_disk_limit: usize) -> Self {
72        Self {
73            states: Default::default(),
74            on_disk_states: Default::default(),
75            in_memory_limit,
76            min_in_memory_limit: in_memory_limit.min(MIN_HISTORY_LIMIT),
77            max_on_disk_limit: on_disk_limit,
78            oldest_on_disk: Default::default(),
79            present: Default::default(),
80            disk_cache: Default::default(),
81        }
82    }
83
84    /// Configures no disk caching
85    pub fn memory_only(mut self) -> Self {
86        self.max_on_disk_limit = 0;
87        self
88    }
89
90    /// Configures the path on disk where the states will cached.
91    pub fn disk_path(mut self, path: PathBuf) -> Self {
92        self.disk_cache = self.disk_cache.with_path(path);
93        self
94    }
95
96    /// This modifies the `limit` what to keep stored in memory.
97    ///
98    /// This will ensure the new limit adjusts based on the block time.
99    /// The lowest blocktime is 1s which should increase the limit slightly
100    pub fn update_interval_mine_block_time(&mut self, block_time: Duration) {
101        let block_time = block_time.as_secs();
102        // for block times lower than 2s we increase the mem limit since we're mining _small_ blocks
103        // very fast
104        // this will gradually be decreased once the max limit was reached
105        if block_time <= 2 {
106            self.in_memory_limit = DEFAULT_HISTORY_LIMIT * 3;
107            self.enforce_limits();
108        }
109    }
110
111    /// Returns true if only memory caching is supported.
112    fn is_memory_only(&self) -> bool {
113        self.max_on_disk_limit == 0
114    }
115
116    /// Inserts a new (hash -> state) pair
117    ///
118    /// When the configured limit for the number of states that can be stored in memory is reached,
119    /// the oldest state is removed.
120    ///
121    /// Since we keep a snapshot of the entire state as history, the size of the state will increase
122    /// with the transactions processed. To counter this, we gradually decrease the cache limit with
123    /// the number of states/blocks until we reached the `min_limit`.
124    ///
125    /// When a state that was previously written to disk is requested, it is simply read from disk.
126    pub fn insert(&mut self, hash: B256, state: StateDb) {
127        if !self.is_memory_only() && self.present.len() >= self.in_memory_limit {
128            // once we hit the max limit we gradually decrease it
129            self.in_memory_limit =
130                self.in_memory_limit.saturating_sub(1).max(self.min_in_memory_limit);
131        }
132
133        self.enforce_limits();
134
135        self.states.insert(hash, state);
136        self.present.push_back(hash);
137    }
138
139    /// Enforces configured limits
140    fn enforce_limits(&mut self) {
141        // enforce memory limits
142        while self.present.len() >= self.in_memory_limit {
143            // evict the oldest block
144            if let Some((hash, mut state)) = self
145                .present
146                .pop_front()
147                .and_then(|hash| self.states.remove(&hash).map(|state| (hash, state)))
148            {
149                // only write to disk if supported
150                if !self.is_memory_only() {
151                    let state_snapshot = state.0.clear_into_state_snapshot();
152                    self.disk_cache.write(hash, state_snapshot);
153                    self.on_disk_states.insert(hash, state);
154                    self.oldest_on_disk.push_back(hash);
155                }
156            }
157        }
158
159        // enforce on disk limit and purge the oldest state cached on disk
160        while !self.is_memory_only() && self.oldest_on_disk.len() >= self.max_on_disk_limit {
161            // evict the oldest block
162            if let Some(hash) = self.oldest_on_disk.pop_front() {
163                self.on_disk_states.remove(&hash);
164                self.disk_cache.remove(hash);
165            }
166        }
167    }
168
169    /// Returns the in-memory state for the given `hash` if present
170    pub fn get_state(&self, hash: &B256) -> Option<&StateDb> {
171        self.states.get(hash)
172    }
173
174    /// Returns on-disk state for the given `hash` if present
175    pub fn get_on_disk_state(&mut self, hash: &B256) -> Option<&StateDb> {
176        if let Some(state) = self.on_disk_states.get_mut(hash)
177            && let Some(cached) = self.disk_cache.read(*hash)
178        {
179            state.init_from_state_snapshot(cached);
180            return Some(state);
181        }
182
183        None
184    }
185
186    /// Sets the maximum number of stats we keep in memory
187    pub fn set_cache_limit(&mut self, limit: usize) {
188        self.in_memory_limit = limit;
189    }
190
191    /// Clears all entries
192    pub fn clear(&mut self) {
193        self.states.clear();
194        self.on_disk_states.clear();
195        self.present.clear();
196        for on_disk in std::mem::take(&mut self.oldest_on_disk) {
197            self.disk_cache.remove(on_disk)
198        }
199    }
200
201    /// Serialize all states to a list of serializable historical states
202    pub fn serialized_states(&mut self) -> SerializableHistoricalStates {
203        // Get in-memory states
204        let mut states = self
205            .states
206            .iter_mut()
207            .map(|(hash, state)| (*hash, state.serialize_state()))
208            .collect::<Vec<_>>();
209
210        // Get on-disk state snapshots
211        self.on_disk_states.iter().for_each(|(hash, _)| {
212            if let Some(state_snapshot) = self.disk_cache.read(*hash) {
213                states.push((*hash, state_snapshot));
214            }
215        });
216
217        SerializableHistoricalStates::new(states)
218    }
219
220    /// Load states from serialized data
221    pub fn load_states(&mut self, states: SerializableHistoricalStates) {
222        for (hash, state_snapshot) in states {
223            let mut state_db = StateDb::new(MemDb::default());
224            state_db.init_from_state_snapshot(state_snapshot);
225            self.insert(hash, state_db);
226        }
227    }
228}
229
230impl fmt::Debug for InMemoryBlockStates {
231    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
232        f.debug_struct("InMemoryBlockStates")
233            .field("in_memory_limit", &self.in_memory_limit)
234            .field("min_in_memory_limit", &self.min_in_memory_limit)
235            .field("max_on_disk_limit", &self.max_on_disk_limit)
236            .field("oldest_on_disk", &self.oldest_on_disk)
237            .field("present", &self.present)
238            .finish_non_exhaustive()
239    }
240}
241
242impl Default for InMemoryBlockStates {
243    fn default() -> Self {
244        // enough in memory to store `DEFAULT_HISTORY_LIMIT` blocks in memory
245        Self::new(DEFAULT_HISTORY_LIMIT, MAX_ON_DISK_HISTORY_LIMIT)
246    }
247}
248
249/// Stores the blockchain data (blocks, transactions)
250#[derive(Clone, Debug)]
251pub struct BlockchainStorage {
252    /// all stored blocks (block hash -> block)
253    pub blocks: B256HashMap<Block>,
254    /// mapping from block number -> block hash
255    pub hashes: HashMap<u64, B256>,
256    /// The current best hash
257    pub best_hash: B256,
258    /// The current best block number
259    pub best_number: u64,
260    /// genesis hash of the chain
261    pub genesis_hash: B256,
262    /// Mapping from the transaction hash to a tuple containing the transaction as well as the
263    /// transaction receipt
264    pub transactions: B256HashMap<MinedTransaction>,
265    /// The total difficulty of the chain until this block
266    pub total_difficulty: U256,
267}
268
269impl BlockchainStorage {
270    /// Creates a new storage with a genesis block
271    pub fn new(
272        env: &Env,
273        spec_id: SpecId,
274        base_fee: Option<u64>,
275        timestamp: u64,
276        genesis_number: u64,
277    ) -> Self {
278        let is_shanghai = spec_id >= SpecId::SHANGHAI;
279        let is_cancun = spec_id >= SpecId::CANCUN;
280        let is_prague = spec_id >= SpecId::PRAGUE;
281
282        // create a dummy genesis block
283        let header = Header {
284            timestamp,
285            base_fee_per_gas: base_fee,
286            gas_limit: env.evm_env.block_env.gas_limit,
287            beneficiary: env.evm_env.block_env.beneficiary,
288            difficulty: env.evm_env.block_env.difficulty,
289            blob_gas_used: env.evm_env.block_env.blob_excess_gas_and_price.as_ref().map(|_| 0),
290            excess_blob_gas: env.evm_env.block_env.blob_excess_gas(),
291            number: genesis_number,
292            parent_beacon_block_root: is_cancun.then_some(Default::default()),
293            withdrawals_root: is_shanghai.then_some(EMPTY_WITHDRAWALS),
294            requests_hash: is_prague.then_some(EMPTY_REQUESTS_HASH),
295            ..Default::default()
296        };
297        let block = create_block(header, Vec::<MaybeImpersonatedTransaction>::new());
298        let genesis_hash = block.header.hash_slow();
299        let best_hash = genesis_hash;
300        let best_number = genesis_number;
301
302        let mut blocks = B256HashMap::default();
303        blocks.insert(genesis_hash, block);
304
305        let mut hashes = HashMap::default();
306        hashes.insert(best_number, genesis_hash);
307        Self {
308            blocks,
309            hashes,
310            best_hash,
311            best_number,
312            genesis_hash,
313            transactions: Default::default(),
314            total_difficulty: Default::default(),
315        }
316    }
317
318    pub fn forked(block_number: u64, block_hash: B256, total_difficulty: U256) -> Self {
319        let mut hashes = HashMap::default();
320        hashes.insert(block_number, block_hash);
321
322        Self {
323            blocks: B256HashMap::default(),
324            hashes,
325            best_hash: block_hash,
326            best_number: block_number,
327            genesis_hash: Default::default(),
328            transactions: Default::default(),
329            total_difficulty,
330        }
331    }
332
333    /// Unwind the chain state back to the given block in storage.
334    ///
335    /// The block identified by `block_number` and `block_hash` is __non-inclusive__, i.e. it will
336    /// remain in the state.
337    pub fn unwind_to(&mut self, block_number: u64, block_hash: B256) -> Vec<Block> {
338        let mut removed = vec![];
339        let best_num: u64 = self.best_number;
340        for i in (block_number + 1)..=best_num {
341            if let Some(hash) = self.hashes.get(&i).copied() {
342                // First remove the block's transactions while the mappings still exist
343                self.remove_block_transactions_by_number(i);
344
345                // Now remove the block from storage (may already be empty of txs) and drop mapping
346                if let Some(block) = self.blocks.remove(&hash) {
347                    removed.push(block);
348                }
349                self.hashes.remove(&i);
350            }
351        }
352        self.best_hash = block_hash;
353        self.best_number = block_number;
354        removed
355    }
356
357    pub fn empty() -> Self {
358        Self {
359            blocks: Default::default(),
360            hashes: Default::default(),
361            best_hash: Default::default(),
362            best_number: Default::default(),
363            genesis_hash: Default::default(),
364            transactions: Default::default(),
365            total_difficulty: Default::default(),
366        }
367    }
368
369    /// Removes all stored transactions for the given block number
370    pub fn remove_block_transactions_by_number(&mut self, num: u64) {
371        if let Some(hash) = self.hashes.get(&num).copied() {
372            self.remove_block_transactions(hash);
373        }
374    }
375
376    /// Removes all stored transactions for the given block hash
377    pub fn remove_block_transactions(&mut self, block_hash: B256) {
378        if let Some(block) = self.blocks.get_mut(&block_hash) {
379            for tx in &block.body.transactions {
380                self.transactions.remove(&tx.hash());
381            }
382            block.body.transactions.clear();
383        }
384    }
385}
386
387impl BlockchainStorage {
388    /// Returns the hash for [BlockNumberOrTag]
389    pub fn hash(&self, number: BlockNumberOrTag) -> Option<B256> {
390        let slots_in_an_epoch = 32;
391        match number {
392            BlockNumberOrTag::Latest => Some(self.best_hash),
393            BlockNumberOrTag::Earliest => Some(self.genesis_hash),
394            BlockNumberOrTag::Pending => None,
395            BlockNumberOrTag::Number(num) => self.hashes.get(&num).copied(),
396            BlockNumberOrTag::Safe => {
397                if self.best_number > (slots_in_an_epoch) {
398                    self.hashes.get(&(self.best_number - (slots_in_an_epoch))).copied()
399                } else {
400                    Some(self.genesis_hash) // treat the genesis block as safe "by definition"
401                }
402            }
403            BlockNumberOrTag::Finalized => {
404                if self.best_number > (slots_in_an_epoch * 2) {
405                    self.hashes.get(&(self.best_number - (slots_in_an_epoch * 2))).copied()
406                } else {
407                    Some(self.genesis_hash)
408                }
409            }
410        }
411    }
412
413    pub fn serialized_blocks(&self) -> Vec<SerializableBlock> {
414        self.blocks.values().map(|block| block.clone().into()).collect()
415    }
416
417    pub fn serialized_transactions(&self) -> Vec<SerializableTransaction> {
418        self.transactions.values().map(|tx: &MinedTransaction| tx.clone().into()).collect()
419    }
420
421    /// Deserialize and add all blocks data to the backend storage
422    pub fn load_blocks(&mut self, serializable_blocks: Vec<SerializableBlock>) {
423        for serializable_block in &serializable_blocks {
424            let block: Block = serializable_block.clone().into();
425            let block_hash = block.header.hash_slow();
426            let block_number = block.header.number;
427            self.blocks.insert(block_hash, block);
428            self.hashes.insert(block_number, block_hash);
429        }
430    }
431
432    /// Deserialize and add all blocks data to the backend storage
433    pub fn load_transactions(&mut self, serializable_transactions: Vec<SerializableTransaction>) {
434        for serializable_transaction in &serializable_transactions {
435            let transaction: MinedTransaction = serializable_transaction.clone().into();
436            self.transactions.insert(transaction.info.transaction_hash, transaction);
437        }
438    }
439}
440
441/// A simple in-memory blockchain
442#[derive(Clone, Debug)]
443pub struct Blockchain {
444    /// underlying storage that supports concurrent reads
445    pub storage: Arc<RwLock<BlockchainStorage>>,
446}
447
448impl Blockchain {
449    /// Creates a new storage with a genesis block
450    pub fn new(
451        env: &Env,
452        spec_id: SpecId,
453        base_fee: Option<u64>,
454        timestamp: u64,
455        genesis_number: u64,
456    ) -> Self {
457        Self {
458            storage: Arc::new(RwLock::new(BlockchainStorage::new(
459                env,
460                spec_id,
461                base_fee,
462                timestamp,
463                genesis_number,
464            ))),
465        }
466    }
467
468    pub fn forked(block_number: u64, block_hash: B256, total_difficulty: U256) -> Self {
469        Self {
470            storage: Arc::new(RwLock::new(BlockchainStorage::forked(
471                block_number,
472                block_hash,
473                total_difficulty,
474            ))),
475        }
476    }
477
478    /// returns the header hash of given block
479    pub fn hash(&self, id: BlockId) -> Option<B256> {
480        match id {
481            BlockId::Hash(h) => Some(h.block_hash),
482            BlockId::Number(num) => self.storage.read().hash(num),
483        }
484    }
485
486    pub fn get_block_by_hash(&self, hash: &B256) -> Option<Block> {
487        self.storage.read().blocks.get(hash).cloned()
488    }
489
490    pub fn get_transaction_by_hash(&self, hash: &B256) -> Option<MinedTransaction> {
491        self.storage.read().transactions.get(hash).cloned()
492    }
493
494    /// Returns the total number of blocks
495    pub fn blocks_count(&self) -> usize {
496        self.storage.read().blocks.len()
497    }
498}
499
500/// Represents the outcome of mining a new block
501#[derive(Clone, Debug)]
502pub struct MinedBlockOutcome {
503    /// The block that was mined
504    pub block_number: u64,
505    /// All transactions included in the block
506    pub included: Vec<Arc<PoolTransaction>>,
507    /// All transactions that were attempted to be included but were invalid at the time of
508    /// execution
509    pub invalid: Vec<Arc<PoolTransaction>>,
510}
511
512/// Container type for a mined transaction
513#[derive(Clone, Debug)]
514pub struct MinedTransaction {
515    pub info: TransactionInfo,
516    pub receipt: FoundryReceiptEnvelope,
517    pub block_hash: B256,
518    pub block_number: u64,
519}
520
521impl MinedTransaction {
522    /// Returns the traces of the transaction for `trace_transaction`
523    pub fn parity_traces(&self) -> Vec<LocalizedTransactionTrace> {
524        ParityTraceBuilder::new(
525            self.info.traces.clone(),
526            None,
527            TracingInspectorConfig::default_parity(),
528        )
529        .into_localized_transaction_traces(RethTransactionInfo {
530            hash: Some(self.info.transaction_hash),
531            index: Some(self.info.transaction_index),
532            block_hash: Some(self.block_hash),
533            block_number: Some(self.block_number),
534            base_fee: None,
535        })
536    }
537
538    pub fn ots_internal_operations(&self) -> Vec<InternalOperation> {
539        self.info
540            .traces
541            .iter()
542            .filter_map(|node| {
543                let r#type = match node.trace.kind {
544                    _ if node.is_selfdestruct() => OperationType::OpSelfDestruct,
545                    CallKind::Call if !node.trace.value.is_zero() => OperationType::OpTransfer,
546                    CallKind::Create => OperationType::OpCreate,
547                    CallKind::Create2 => OperationType::OpCreate2,
548                    _ => return None,
549                };
550                let mut from = node.trace.caller;
551                let mut to = node.trace.address;
552                let mut value = node.trace.value;
553                if node.is_selfdestruct() {
554                    from = node.trace.address;
555                    to = node.trace.selfdestruct_refund_target.unwrap_or_default();
556                    value = node.trace.selfdestruct_transferred_value.unwrap_or_default();
557                }
558                Some(InternalOperation { r#type, from, to, value })
559            })
560            .collect()
561    }
562}
563
564/// Intermediary Anvil representation of a receipt
565#[derive(Clone, Debug)]
566pub struct MinedTransactionReceipt {
567    /// The actual json rpc receipt object
568    pub inner: ReceiptResponse,
569    /// Output data for the transaction
570    pub out: Option<Bytes>,
571}
572
573#[cfg(test)]
574mod tests {
575    use super::*;
576    use crate::eth::backend::db::Db;
577    use alloy_primitives::{Address, hex};
578    use alloy_rlp::Decodable;
579    use foundry_primitives::FoundryTxEnvelope;
580    use revm::{database::DatabaseRef, state::AccountInfo};
581
582    #[test]
583    fn test_interval_update() {
584        let mut storage = InMemoryBlockStates::default();
585        storage.update_interval_mine_block_time(Duration::from_secs(1));
586        assert_eq!(storage.in_memory_limit, DEFAULT_HISTORY_LIMIT * 3);
587    }
588
589    #[test]
590    fn test_init_state_limits() {
591        let mut storage = InMemoryBlockStates::default();
592        assert_eq!(storage.in_memory_limit, DEFAULT_HISTORY_LIMIT);
593        assert_eq!(storage.min_in_memory_limit, MIN_HISTORY_LIMIT);
594        assert_eq!(storage.max_on_disk_limit, MAX_ON_DISK_HISTORY_LIMIT);
595
596        storage = storage.memory_only();
597        assert!(storage.is_memory_only());
598
599        storage = InMemoryBlockStates::new(1, 0);
600        assert!(storage.is_memory_only());
601        assert_eq!(storage.in_memory_limit, 1);
602        assert_eq!(storage.min_in_memory_limit, 1);
603        assert_eq!(storage.max_on_disk_limit, 0);
604
605        storage = InMemoryBlockStates::new(1, 2);
606        assert!(!storage.is_memory_only());
607        assert_eq!(storage.in_memory_limit, 1);
608        assert_eq!(storage.min_in_memory_limit, 1);
609        assert_eq!(storage.max_on_disk_limit, 2);
610    }
611
612    #[tokio::test(flavor = "multi_thread")]
613    async fn can_read_write_cached_state() {
614        let mut storage = InMemoryBlockStates::new(1, MAX_ON_DISK_HISTORY_LIMIT);
615        let one = B256::from(U256::from(1));
616        let two = B256::from(U256::from(2));
617
618        let mut state = MemDb::default();
619        let addr = Address::random();
620        let info = AccountInfo::from_balance(U256::from(1337));
621        state.insert_account(addr, info);
622        storage.insert(one, StateDb::new(state));
623        storage.insert(two, StateDb::new(MemDb::default()));
624
625        // wait for files to be flushed
626        tokio::time::sleep(std::time::Duration::from_secs(1)).await;
627
628        assert_eq!(storage.on_disk_states.len(), 1);
629        assert!(storage.on_disk_states.contains_key(&one));
630
631        let loaded = storage.get_on_disk_state(&one).unwrap();
632
633        let acc = loaded.basic_ref(addr).unwrap().unwrap();
634        assert_eq!(acc.balance, U256::from(1337u64));
635    }
636
637    #[tokio::test(flavor = "multi_thread")]
638    async fn can_decrease_state_cache_size() {
639        let limit = 15;
640        let mut storage = InMemoryBlockStates::new(limit, MAX_ON_DISK_HISTORY_LIMIT);
641
642        let num_states = 30;
643        for idx in 0..num_states {
644            let mut state = MemDb::default();
645            let hash = B256::from(U256::from(idx));
646            let addr = Address::from_word(hash);
647            let balance = (idx * 2) as u64;
648            let info = AccountInfo::from_balance(U256::from(balance));
649            state.insert_account(addr, info);
650            storage.insert(hash, StateDb::new(state));
651        }
652
653        // wait for files to be flushed
654        tokio::time::sleep(std::time::Duration::from_secs(1)).await;
655
656        let on_disk_states_len = num_states - storage.min_in_memory_limit;
657
658        assert_eq!(storage.on_disk_states.len(), on_disk_states_len);
659        assert_eq!(storage.present.len(), storage.min_in_memory_limit);
660
661        for idx in 0..num_states {
662            let hash = B256::from(U256::from(idx));
663            let addr = Address::from_word(hash);
664
665            let loaded = if idx < on_disk_states_len {
666                storage.get_on_disk_state(&hash).unwrap()
667            } else {
668                storage.get_state(&hash).unwrap()
669            };
670
671            let acc = loaded.basic_ref(addr).unwrap().unwrap();
672            let balance = (idx * 2) as u64;
673            assert_eq!(acc.balance, U256::from(balance));
674        }
675    }
676
677    // verifies that blocks and transactions in BlockchainStorage remain the same when dumped and
678    // reloaded
679    #[test]
680    fn test_storage_dump_reload_cycle() {
681        let mut dump_storage = BlockchainStorage::empty();
682
683        let header = Header { gas_limit: 123456, ..Default::default() };
684        let bytes_first = &mut &hex::decode("f86b02843b9aca00830186a094d3e8763675e4c425df46cc3b5c0f6cbdac39604687038d7ea4c68000802ba00eb96ca19e8a77102767a41fc85a36afd5c61ccb09911cec5d3e86e193d9c5aea03a456401896b1b6055311536bf00a718568c744d8c1f9df59879e8350220ca18").unwrap()[..];
685        let tx: MaybeImpersonatedTransaction =
686            FoundryTxEnvelope::decode(&mut &bytes_first[..]).unwrap().into();
687        let block = create_block(header.clone(), vec![tx.clone()]);
688        let block_hash = block.header.hash_slow();
689        dump_storage.blocks.insert(block_hash, block);
690
691        let serialized_blocks = dump_storage.serialized_blocks();
692        let serialized_transactions = dump_storage.serialized_transactions();
693
694        let mut load_storage = BlockchainStorage::empty();
695
696        load_storage.load_blocks(serialized_blocks);
697        load_storage.load_transactions(serialized_transactions);
698
699        let loaded_block = load_storage.blocks.get(&block_hash).unwrap();
700        assert_eq!(loaded_block.header.gas_limit, { header.gas_limit });
701        let loaded_tx = loaded_block.body.transactions.first().unwrap();
702        assert_eq!(loaded_tx, &tx);
703    }
704}