reth_db_common/
init.rs

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