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