reth_db_common/
init.rs

1//! Reth genesis initialization utility functions.
2
3use alloy_consensus::BlockHeader;
4use alloy_genesis::GenesisAccount;
5use alloy_primitives::{Address, B256, U256};
6use reth_chainspec::EthChainSpec;
7use reth_codecs::Compact;
8use reth_config::config::EtlConfig;
9use reth_db::tables;
10use reth_db_api::{transaction::DbTxMut, DatabaseError};
11use reth_etl::Collector;
12use reth_primitives::{
13    Account, Bytecode, GotExpected, NodePrimitives, Receipts, StaticFileSegment, StorageEntry,
14};
15use reth_provider::{
16    errors::provider::ProviderResult, providers::StaticFileWriter, writer::UnifiedStorageWriter,
17    BlockHashReader, BlockNumReader, BundleStateInit, ChainSpecProvider, DBProvider,
18    DatabaseProviderFactory, ExecutionOutcome, HashingWriter, HeaderProvider, HistoryWriter,
19    OriginalValuesKnown, ProviderError, RevertsInit, StageCheckpointReader, StageCheckpointWriter,
20    StateWriter, StaticFileProviderFactory, StorageLocation, TrieWriter,
21};
22use reth_stages_types::{StageCheckpoint, StageId};
23use reth_trie::{IntermediateStateRootState, StateRoot as StateRootComputer, StateRootProgress};
24use reth_trie_db::DatabaseStateRoot;
25use serde::{Deserialize, Serialize};
26use std::{collections::HashMap, io::BufRead};
27use tracing::{debug, error, info, trace};
28
29/// Default soft limit for number of bytes to read from state dump file, before inserting into
30/// database.
31///
32/// Default is 1 GB.
33pub const DEFAULT_SOFT_LIMIT_BYTE_LEN_ACCOUNTS_CHUNK: usize = 1_000_000_000;
34
35/// Approximate number of accounts per 1 GB of state dump file. One account is approximately 3.5 KB
36///
37/// Approximate is 285 228 accounts.
38//
39// (14.05 GB OP mainnet state dump at Bedrock block / 4 007 565 accounts in file > 3.5 KB per
40// account)
41pub const AVERAGE_COUNT_ACCOUNTS_PER_GB_STATE_DUMP: usize = 285_228;
42
43/// Soft limit for the number of flushed updates after which to log progress summary.
44const SOFT_LIMIT_COUNT_FLUSHED_UPDATES: usize = 1_000_000;
45
46/// Storage initialization error type.
47#[derive(Debug, thiserror::Error, PartialEq, Eq, Clone)]
48pub enum InitStorageError {
49    /// Genesis header found on static files but the database is empty.
50    #[error("static files found, but the database is uninitialized. If attempting to re-syncing, delete both.")]
51    UninitializedDatabase,
52    /// An existing genesis block was found in the database, and its hash did not match the hash of
53    /// the chainspec.
54    #[error("genesis hash in the storage does not match the specified chainspec: chainspec is {chainspec_hash}, database is {storage_hash}")]
55    GenesisHashMismatch {
56        /// Expected genesis hash.
57        chainspec_hash: B256,
58        /// Actual genesis hash.
59        storage_hash: B256,
60    },
61    /// Provider error.
62    #[error(transparent)]
63    Provider(#[from] ProviderError),
64    /// State root doesn't match the expected one.
65    #[error("state root mismatch: {_0}")]
66    StateRootMismatch(GotExpected<B256>),
67}
68
69impl From<DatabaseError> for InitStorageError {
70    fn from(error: DatabaseError) -> Self {
71        Self::Provider(ProviderError::Database(error))
72    }
73}
74
75/// Write the genesis block if it has not already been written
76pub fn init_genesis<PF>(factory: &PF) -> Result<B256, InitStorageError>
77where
78    PF: DatabaseProviderFactory
79        + StaticFileProviderFactory<Primitives: NodePrimitives<BlockHeader: Compact>>
80        + ChainSpecProvider
81        + StageCheckpointReader
82        + BlockHashReader,
83    PF::ProviderRW: StaticFileProviderFactory<Primitives = PF::Primitives>
84        + StageCheckpointWriter
85        + HistoryWriter
86        + HeaderProvider
87        + HashingWriter
88        + StateWriter
89        + StateWriter
90        + AsRef<PF::ProviderRW>,
91    PF::ChainSpec: EthChainSpec<Header = <PF::Primitives as NodePrimitives>::BlockHeader>,
92{
93    let chain = factory.chain_spec();
94
95    let genesis = chain.genesis();
96    let hash = chain.genesis_hash();
97
98    // Check if we already have the genesis header or if we have the wrong one.
99    match factory.block_hash(0) {
100        Ok(None) | Err(ProviderError::MissingStaticFileBlock(StaticFileSegment::Headers, 0)) => {}
101        Ok(Some(block_hash)) => {
102            if block_hash == hash {
103                // Some users will at times attempt to re-sync from scratch by just deleting the
104                // database. Since `factory.block_hash` will only query the static files, we need to
105                // make sure that our database has been written to, and throw error if it's empty.
106                if factory.get_stage_checkpoint(StageId::Headers)?.is_none() {
107                    error!(target: "reth::storage", "Genesis header found on static files, but database is uninitialized.");
108                    return Err(InitStorageError::UninitializedDatabase)
109                }
110
111                debug!("Genesis already written, skipping.");
112                return Ok(hash)
113            }
114
115            return Err(InitStorageError::GenesisHashMismatch {
116                chainspec_hash: hash,
117                storage_hash: block_hash,
118            })
119        }
120        Err(e) => {
121            debug!(?e);
122            return Err(e.into());
123        }
124    }
125
126    debug!("Writing genesis block.");
127
128    let alloc = &genesis.alloc;
129
130    // use transaction to insert genesis header
131    let provider_rw = factory.database_provider_rw()?;
132    insert_genesis_hashes(&provider_rw, alloc.iter())?;
133    insert_genesis_history(&provider_rw, alloc.iter())?;
134
135    // Insert header
136    insert_genesis_header(&provider_rw, &chain)?;
137
138    insert_genesis_state(&provider_rw, alloc.iter())?;
139
140    // insert sync stage
141    for stage in StageId::ALL {
142        provider_rw.save_stage_checkpoint(stage, Default::default())?;
143    }
144
145    let static_file_provider = provider_rw.static_file_provider();
146    // Static file segments start empty, so we need to initialize the genesis block.
147    let segment = StaticFileSegment::Receipts;
148    static_file_provider.latest_writer(segment)?.increment_block(0)?;
149
150    let segment = StaticFileSegment::Transactions;
151    static_file_provider.latest_writer(segment)?.increment_block(0)?;
152
153    // `commit_unwind`` will first commit the DB and then the static file provider, which is
154    // necessary on `init_genesis`.
155    UnifiedStorageWriter::commit_unwind(provider_rw)?;
156
157    Ok(hash)
158}
159
160/// Inserts the genesis state into the database.
161pub fn insert_genesis_state<'a, 'b, Provider>(
162    provider: &Provider,
163    alloc: impl Iterator<Item = (&'a Address, &'b GenesisAccount)>,
164) -> ProviderResult<()>
165where
166    Provider: StaticFileProviderFactory
167        + DBProvider<Tx: DbTxMut>
168        + HeaderProvider
169        + StateWriter
170        + AsRef<Provider>,
171{
172    insert_state(provider, alloc, 0)
173}
174
175/// Inserts state at given block into database.
176pub fn insert_state<'a, 'b, Provider>(
177    provider: &Provider,
178    alloc: impl Iterator<Item = (&'a Address, &'b GenesisAccount)>,
179    block: u64,
180) -> ProviderResult<()>
181where
182    Provider: StaticFileProviderFactory
183        + DBProvider<Tx: DbTxMut>
184        + HeaderProvider
185        + StateWriter
186        + AsRef<Provider>,
187{
188    let capacity = alloc.size_hint().1.unwrap_or(0);
189    let mut state_init: BundleStateInit = HashMap::with_capacity(capacity);
190    let mut reverts_init = HashMap::with_capacity(capacity);
191    let mut contracts: HashMap<B256, Bytecode> = HashMap::with_capacity(capacity);
192
193    for (address, account) in alloc {
194        let bytecode_hash = if let Some(code) = &account.code {
195            match Bytecode::new_raw_checked(code.clone()) {
196                Ok(bytecode) => {
197                    let hash = bytecode.hash_slow();
198                    contracts.insert(hash, bytecode);
199                    Some(hash)
200                }
201                Err(err) => {
202                    error!(%address, %err, "Failed to decode genesis bytecode.");
203                    return Err(DatabaseError::Other(err.to_string()).into());
204                }
205            }
206        } else {
207            None
208        };
209
210        // get state
211        let storage = account
212            .storage
213            .as_ref()
214            .map(|m| {
215                m.iter()
216                    .map(|(key, value)| {
217                        let value = U256::from_be_bytes(value.0);
218                        let is_private = false;
219                        (*key, ((U256::ZERO, false), (value, is_private)))
220                    })
221                    .collect::<HashMap<_, _>>()
222            })
223            .unwrap_or_default();
224
225        reverts_init.insert(
226            *address,
227            (
228                Some(None),
229                storage.keys().map(|k| StorageEntry { key: *k, ..Default::default() }).collect(),
230            ),
231        );
232
233        state_init.insert(
234            *address,
235            (
236                None,
237                Some(Account {
238                    nonce: account.nonce.unwrap_or_default(),
239                    balance: account.balance,
240                    bytecode_hash,
241                }),
242                storage,
243            ),
244        );
245    }
246    let all_reverts_init: RevertsInit = HashMap::from([(block, reverts_init)]);
247
248    let execution_outcome = ExecutionOutcome::new_init(
249        state_init,
250        all_reverts_init,
251        contracts,
252        Receipts::default(),
253        block,
254        Vec::new(),
255    );
256
257    provider.write_state(execution_outcome, OriginalValuesKnown::Yes, StorageLocation::Database)?;
258
259    trace!(target: "reth::cli", "Inserted state");
260
261    Ok(())
262}
263
264/// Inserts hashes for the genesis state.
265pub fn insert_genesis_hashes<'a, 'b, Provider>(
266    provider: &Provider,
267    alloc: impl Iterator<Item = (&'a Address, &'b GenesisAccount)> + Clone,
268) -> ProviderResult<()>
269where
270    Provider: DBProvider<Tx: DbTxMut> + HashingWriter,
271{
272    // insert and hash accounts to hashing table
273    let alloc_accounts = alloc.clone().map(|(addr, account)| (*addr, Some(Account::from(account))));
274    provider.insert_account_for_hashing(alloc_accounts)?;
275
276    trace!(target: "reth::cli", "Inserted account hashes");
277
278    let alloc_storage = alloc.filter_map(|(addr, account)| {
279        // only return Some if there is storage
280        account.storage.as_ref().map(|storage| {
281            (
282                *addr,
283                storage.clone().into_iter().map(|(key, value)| StorageEntry {
284                    key,
285                    value: value.into(),
286                    is_private: false,
287                }),
288            )
289        })
290    });
291    provider.insert_storage_for_hashing(alloc_storage)?;
292
293    trace!(target: "reth::cli", "Inserted storage hashes");
294
295    Ok(())
296}
297
298/// Inserts history indices for genesis accounts and storage.
299pub fn insert_genesis_history<'a, 'b, Provider>(
300    provider: &Provider,
301    alloc: impl Iterator<Item = (&'a Address, &'b GenesisAccount)> + Clone,
302) -> ProviderResult<()>
303where
304    Provider: DBProvider<Tx: DbTxMut> + HistoryWriter,
305{
306    insert_history(provider, alloc, 0)
307}
308
309/// Inserts history indices for genesis accounts and storage.
310pub fn insert_history<'a, 'b, Provider>(
311    provider: &Provider,
312    alloc: impl Iterator<Item = (&'a Address, &'b GenesisAccount)> + Clone,
313    block: u64,
314) -> ProviderResult<()>
315where
316    Provider: DBProvider<Tx: DbTxMut> + HistoryWriter,
317{
318    let account_transitions = alloc.clone().map(|(addr, _)| (*addr, [block]));
319    provider.insert_account_history_index(account_transitions)?;
320
321    trace!(target: "reth::cli", "Inserted account history");
322
323    let storage_transitions = alloc
324        .filter_map(|(addr, account)| account.storage.as_ref().map(|storage| (addr, storage)))
325        .flat_map(|(addr, storage)| storage.iter().map(|(key, _)| ((*addr, *key), [block])));
326    provider.insert_storage_history_index(storage_transitions)?;
327
328    trace!(target: "reth::cli", "Inserted storage history");
329
330    Ok(())
331}
332
333/// Inserts header for the genesis state.
334pub fn insert_genesis_header<Provider, Spec>(
335    provider: &Provider,
336    chain: &Spec,
337) -> ProviderResult<()>
338where
339    Provider: StaticFileProviderFactory<Primitives: NodePrimitives<BlockHeader: Compact>>
340        + DBProvider<Tx: DbTxMut>,
341    Spec: EthChainSpec<Header = <Provider::Primitives as NodePrimitives>::BlockHeader>,
342{
343    let (header, block_hash) = (chain.genesis_header(), chain.genesis_hash());
344    let static_file_provider = provider.static_file_provider();
345
346    match static_file_provider.block_hash(0) {
347        Ok(None) | Err(ProviderError::MissingStaticFileBlock(StaticFileSegment::Headers, 0)) => {
348            let (difficulty, hash) = (header.difficulty(), block_hash);
349            let mut writer = static_file_provider.latest_writer(StaticFileSegment::Headers)?;
350            writer.append_header(header, difficulty, &hash)?;
351        }
352        Ok(Some(_)) => {}
353        Err(e) => return Err(e),
354    }
355
356    provider.tx_ref().put::<tables::HeaderNumbers>(block_hash, 0)?;
357    provider.tx_ref().put::<tables::BlockBodyIndices>(0, Default::default())?;
358
359    Ok(())
360}
361
362/// Reads account state from a [`BufRead`] reader and initializes it at the highest block that can
363/// be found on database.
364///
365/// It's similar to [`init_genesis`] but supports importing state too big to fit in memory, and can
366/// be set to the highest block present. One practical usecase is to import OP mainnet state at
367/// bedrock transition block.
368pub fn init_from_state_dump<Provider>(
369    mut reader: impl BufRead,
370    provider_rw: &Provider,
371    etl_config: EtlConfig,
372) -> eyre::Result<B256>
373where
374    Provider: StaticFileProviderFactory
375        + DBProvider<Tx: DbTxMut>
376        + BlockNumReader
377        + BlockHashReader
378        + ChainSpecProvider
379        + StageCheckpointWriter
380        + HistoryWriter
381        + HeaderProvider
382        + HashingWriter
383        + TrieWriter
384        + StateWriter
385        + AsRef<Provider>,
386{
387    let block = provider_rw.last_block_number()?;
388    let hash = provider_rw.block_hash(block)?.unwrap();
389    let expected_state_root = provider_rw
390        .header_by_number(block)?
391        .ok_or_else(|| ProviderError::HeaderNotFound(block.into()))?
392        .state_root();
393
394    // first line can be state root
395    let dump_state_root = parse_state_root(&mut reader)?;
396    if expected_state_root != dump_state_root {
397        error!(target: "reth::cli",
398            ?dump_state_root,
399            ?expected_state_root,
400            "State root from state dump does not match state root in current header."
401        );
402        return Err(InitStorageError::StateRootMismatch(GotExpected {
403            got: dump_state_root,
404            expected: expected_state_root,
405        })
406        .into())
407    }
408
409    debug!(target: "reth::cli",
410        block,
411        chain=%provider_rw.chain_spec().chain(),
412        "Initializing state at block"
413    );
414
415    // remaining lines are accounts
416    let collector = parse_accounts(&mut reader, etl_config)?;
417
418    // write state to db
419    dump_state(collector, provider_rw, block)?;
420
421    // compute and compare state root. this advances the stage checkpoints.
422    let computed_state_root = compute_state_root(provider_rw)?;
423    if computed_state_root == expected_state_root {
424        info!(target: "reth::cli",
425            ?computed_state_root,
426            "Computed state root matches state root in state dump"
427        );
428    } else {
429        error!(target: "reth::cli",
430            ?computed_state_root,
431            ?expected_state_root,
432            "Computed state root does not match state root in state dump"
433        );
434
435        return Err(InitStorageError::StateRootMismatch(GotExpected {
436            got: computed_state_root,
437            expected: expected_state_root,
438        })
439        .into())
440    }
441
442    // insert sync stages for stages that require state
443    for stage in StageId::STATE_REQUIRED {
444        provider_rw.save_stage_checkpoint(stage, StageCheckpoint::new(block))?;
445    }
446
447    Ok(hash)
448}
449
450/// Parses and returns expected state root.
451fn parse_state_root(reader: &mut impl BufRead) -> eyre::Result<B256> {
452    let mut line = String::new();
453    reader.read_line(&mut line)?;
454
455    let expected_state_root = serde_json::from_str::<StateRoot>(&line)?.root;
456    trace!(target: "reth::cli",
457        root=%expected_state_root,
458        "Read state root from file"
459    );
460    Ok(expected_state_root)
461}
462
463/// Parses accounts and pushes them to a [`Collector`].
464fn parse_accounts(
465    mut reader: impl BufRead,
466    etl_config: EtlConfig,
467) -> Result<Collector<Address, GenesisAccount>, eyre::Error> {
468    let mut line = String::new();
469    let mut collector = Collector::new(etl_config.file_size, etl_config.dir);
470
471    while let Ok(n) = reader.read_line(&mut line) {
472        if n == 0 {
473            break
474        }
475
476        let GenesisAccountWithAddress { genesis_account, address } = serde_json::from_str(&line)?;
477        collector.insert(address, genesis_account)?;
478
479        if !collector.is_empty() && collector.len() % AVERAGE_COUNT_ACCOUNTS_PER_GB_STATE_DUMP == 0
480        {
481            info!(target: "reth::cli",
482                parsed_new_accounts=collector.len(),
483            );
484        }
485
486        line.clear();
487    }
488
489    Ok(collector)
490}
491
492/// Takes a [`Collector`] and processes all accounts.
493fn dump_state<Provider>(
494    mut collector: Collector<Address, GenesisAccount>,
495    provider_rw: &Provider,
496    block: u64,
497) -> Result<(), eyre::Error>
498where
499    Provider: StaticFileProviderFactory
500        + DBProvider<Tx: DbTxMut>
501        + HeaderProvider
502        + HashingWriter
503        + HistoryWriter
504        + StateWriter
505        + AsRef<Provider>,
506{
507    let accounts_len = collector.len();
508    let mut accounts = Vec::with_capacity(AVERAGE_COUNT_ACCOUNTS_PER_GB_STATE_DUMP);
509    let mut total_inserted_accounts = 0;
510
511    for (index, entry) in collector.iter()?.enumerate() {
512        let (address, account) = entry?;
513        let (address, _) = Address::from_compact(address.as_slice(), address.len());
514        let (account, _) = GenesisAccount::from_compact(account.as_slice(), account.len());
515
516        accounts.push((address, account));
517
518        if (index > 0 && index % AVERAGE_COUNT_ACCOUNTS_PER_GB_STATE_DUMP == 0) ||
519            index == accounts_len - 1
520        {
521            total_inserted_accounts += accounts.len();
522
523            info!(target: "reth::cli",
524                total_inserted_accounts,
525                "Writing accounts to db"
526            );
527
528            // use transaction to insert genesis header
529            insert_genesis_hashes(
530                provider_rw,
531                accounts.iter().map(|(address, account)| (address, account)),
532            )?;
533
534            insert_history(
535                provider_rw,
536                accounts.iter().map(|(address, account)| (address, account)),
537                block,
538            )?;
539
540            // block is already written to static files
541            insert_state(
542                provider_rw,
543                accounts.iter().map(|(address, account)| (address, account)),
544                block,
545            )?;
546
547            accounts.clear();
548        }
549    }
550    Ok(())
551}
552
553/// Computes the state root (from scratch) based on the accounts and storages present in the
554/// database.
555fn compute_state_root<Provider>(provider: &Provider) -> eyre::Result<B256>
556where
557    Provider: DBProvider<Tx: DbTxMut> + TrieWriter,
558{
559    trace!(target: "reth::cli", "Computing state root");
560
561    let tx = provider.tx_ref();
562    let mut intermediate_state: Option<IntermediateStateRootState> = None;
563    let mut total_flushed_updates = 0;
564
565    loop {
566        match StateRootComputer::from_tx(tx)
567            .with_intermediate_state(intermediate_state)
568            .root_with_progress()?
569        {
570            StateRootProgress::Progress(state, _, updates) => {
571                let updated_len = provider.write_trie_updates(&updates)?;
572                total_flushed_updates += updated_len;
573
574                trace!(target: "reth::cli",
575                    last_account_key = %state.last_account_key,
576                    updated_len,
577                    total_flushed_updates,
578                    "Flushing trie updates"
579                );
580
581                intermediate_state = Some(*state);
582
583                if total_flushed_updates % SOFT_LIMIT_COUNT_FLUSHED_UPDATES == 0 {
584                    info!(target: "reth::cli",
585                        total_flushed_updates,
586                        "Flushing trie updates"
587                    );
588                }
589            }
590            StateRootProgress::Complete(root, _, updates) => {
591                let updated_len = provider.write_trie_updates(&updates)?;
592                total_flushed_updates += updated_len;
593
594                trace!(target: "reth::cli",
595                    %root,
596                    updated_len,
597                    total_flushed_updates,
598                    "State root has been computed"
599                );
600
601                return Ok(root)
602            }
603        }
604    }
605}
606
607/// Type to deserialize state root from state dump file.
608#[derive(Debug, Serialize, Deserialize, PartialEq, Eq)]
609struct StateRoot {
610    root: B256,
611}
612
613/// An account as in the state dump file. This contains a [`GenesisAccount`] and the account's
614/// address.
615#[derive(Debug, Serialize, Deserialize)]
616struct GenesisAccountWithAddress {
617    /// The account's balance, nonce, code, and storage.
618    #[serde(flatten)]
619    genesis_account: GenesisAccount,
620    /// The account's address.
621    address: Address,
622}
623
624#[cfg(test)]
625mod tests {
626    use super::*;
627    use alloy_consensus::constants::{
628        HOLESKY_GENESIS_HASH, MAINNET_GENESIS_HASH, SEPOLIA_GENESIS_HASH,
629    };
630    use alloy_genesis::Genesis;
631    use reth_chainspec::{Chain, ChainSpec, HOLESKY, MAINNET, SEPOLIA};
632    use reth_db::DatabaseEnv;
633    use reth_db_api::{
634        cursor::DbCursorRO,
635        models::{storage_sharded_key::StorageShardedKey, IntegerList, ShardedKey},
636        table::{Table, TableRow},
637        transaction::DbTx,
638        Database,
639    };
640    use reth_provider::{
641        test_utils::{create_test_provider_factory_with_chain_spec, MockNodeTypesWithDB},
642        ProviderFactory,
643    };
644    use std::{collections::BTreeMap, sync::Arc};
645
646    fn collect_table_entries<DB, T>(
647        tx: &<DB as Database>::TX,
648    ) -> Result<Vec<TableRow<T>>, InitStorageError>
649    where
650        DB: Database,
651        T: Table,
652    {
653        Ok(tx.cursor_read::<T>()?.walk_range(..)?.collect::<Result<Vec<_>, _>>()?)
654    }
655
656    #[test]
657    fn success_init_genesis_mainnet() {
658        let genesis_hash =
659            init_genesis(&create_test_provider_factory_with_chain_spec(MAINNET.clone())).unwrap();
660
661        // actual, expected
662        assert_eq!(genesis_hash, MAINNET_GENESIS_HASH);
663    }
664
665    #[test]
666    fn success_init_genesis_sepolia() {
667        let genesis_hash =
668            init_genesis(&create_test_provider_factory_with_chain_spec(SEPOLIA.clone())).unwrap();
669
670        // actual, expected
671        assert_eq!(genesis_hash, SEPOLIA_GENESIS_HASH);
672    }
673
674    #[test]
675    fn success_init_genesis_holesky() {
676        let genesis_hash =
677            init_genesis(&create_test_provider_factory_with_chain_spec(HOLESKY.clone())).unwrap();
678
679        // actual, expected
680        assert_eq!(genesis_hash, HOLESKY_GENESIS_HASH);
681    }
682
683    #[test]
684    fn fail_init_inconsistent_db() {
685        let factory = create_test_provider_factory_with_chain_spec(SEPOLIA.clone());
686        let static_file_provider = factory.static_file_provider();
687        init_genesis(&factory).unwrap();
688
689        // Try to init db with a different genesis block
690        let genesis_hash = init_genesis(&ProviderFactory::<MockNodeTypesWithDB>::new(
691            factory.into_db(),
692            MAINNET.clone(),
693            static_file_provider,
694        ));
695
696        assert_eq!(
697            genesis_hash.unwrap_err(),
698            InitStorageError::GenesisHashMismatch {
699                chainspec_hash: MAINNET_GENESIS_HASH,
700                storage_hash: SEPOLIA_GENESIS_HASH
701            }
702        )
703    }
704
705    #[test]
706    fn init_genesis_history() {
707        let address_with_balance = Address::with_last_byte(1);
708        let address_with_storage = Address::with_last_byte(2);
709        let storage_key = B256::with_last_byte(1);
710        let chain_spec = Arc::new(ChainSpec {
711            chain: Chain::from_id(1),
712            genesis: Genesis {
713                alloc: BTreeMap::from([
714                    (
715                        address_with_balance,
716                        GenesisAccount { balance: U256::from(1), ..Default::default() },
717                    ),
718                    (
719                        address_with_storage,
720                        GenesisAccount {
721                            storage: Some(BTreeMap::from([(storage_key, B256::random())])),
722                            ..Default::default()
723                        },
724                    ),
725                ]),
726                ..Default::default()
727            },
728            hardforks: Default::default(),
729            genesis_hash: Default::default(),
730            paris_block_and_final_difficulty: None,
731            deposit_contract: None,
732            ..Default::default()
733        });
734
735        let factory = create_test_provider_factory_with_chain_spec(chain_spec);
736        init_genesis(&factory).unwrap();
737
738        let provider = factory.provider().unwrap();
739
740        let tx = provider.tx_ref();
741
742        assert_eq!(
743            collect_table_entries::<Arc<DatabaseEnv>, tables::AccountsHistory>(tx)
744                .expect("failed to collect"),
745            vec![
746                (ShardedKey::new(address_with_balance, u64::MAX), IntegerList::new([0]).unwrap()),
747                (ShardedKey::new(address_with_storage, u64::MAX), IntegerList::new([0]).unwrap())
748            ],
749        );
750
751        assert_eq!(
752            collect_table_entries::<Arc<DatabaseEnv>, tables::StoragesHistory>(tx)
753                .expect("failed to collect"),
754            vec![(
755                StorageShardedKey::new(address_with_storage, storage_key, u64::MAX),
756                IntegerList::new([0]).unwrap()
757            )],
758        );
759    }
760}