solana_runtime/
serde_snapshot.rs

1#[cfg(target_os = "linux")]
2use std::ffi::{CStr, CString};
3use {
4    crate::{
5        bank::{Bank, BankFieldsToDeserialize, BankFieldsToSerialize, BankHashStats, BankRc},
6        epoch_stakes::{EpochStakes, VersionedEpochStakes},
7        runtime_config::RuntimeConfig,
8        snapshot_utils::{SnapshotError, StorageAndNextAccountsFileId},
9        stakes::{serde_stakes_to_delegation_format, Stakes, StakesEnum},
10    },
11    bincode::{self, config::Options, Error},
12    log::*,
13    serde::{de::DeserializeOwned, Deserialize, Serialize},
14    solana_accounts_db::{
15        accounts::Accounts,
16        accounts_db::{
17            AccountStorageEntry, AccountsDb, AccountsDbConfig, AccountsFileId,
18            AtomicAccountsFileId, DuplicatesLtHash, IndexGenerationInfo,
19        },
20        accounts_file::{AccountsFile, StorageAccess},
21        accounts_hash::{AccountsDeltaHash, AccountsHash},
22        accounts_update_notifier_interface::AccountsUpdateNotifier,
23        ancestors::AncestorsForSerialization,
24        blockhash_queue::BlockhashQueue,
25        epoch_accounts_hash::EpochAccountsHash,
26    },
27    solana_builtins::prototype::BuiltinPrototype,
28    solana_clock::{Epoch, Slot, UnixTimestamp},
29    solana_epoch_schedule::EpochSchedule,
30    solana_fee_calculator::{FeeCalculator, FeeRateGovernor},
31    solana_genesis_config::GenesisConfig,
32    solana_hard_forks::HardForks,
33    solana_hash::Hash,
34    solana_inflation::Inflation,
35    solana_measure::measure::Measure,
36    solana_pubkey::Pubkey,
37    solana_rent_collector::RentCollector,
38    solana_serde::default_on_eof,
39    solana_stake_interface::state::Delegation,
40    std::{
41        cell::RefCell,
42        collections::{HashMap, HashSet},
43        io::{self, BufReader, BufWriter, Read, Write},
44        path::{Path, PathBuf},
45        result::Result,
46        sync::{
47            atomic::{AtomicBool, AtomicUsize, Ordering},
48            Arc,
49        },
50        thread::Builder,
51    },
52    storage::SerializableStorage,
53    types::SerdeAccountsLtHash,
54};
55
56mod storage;
57mod tests;
58mod types;
59mod utils;
60
61pub(crate) use {
62    solana_accounts_db::accounts_hash::{
63        SerdeAccountsDeltaHash, SerdeAccountsHash, SerdeIncrementalAccountsHash,
64    },
65    storage::{SerializableAccountStorageEntry, SerializedAccountsFileId},
66};
67
68const MAX_STREAM_SIZE: u64 = 32 * 1024 * 1024 * 1024;
69
70#[cfg_attr(feature = "frozen-abi", derive(AbiExample))]
71#[derive(Clone, Debug, Deserialize, Serialize, PartialEq, Eq)]
72pub struct AccountsDbFields<T>(
73    HashMap<Slot, Vec<T>>,
74    u64, // obsolete, formerly write_version
75    Slot,
76    BankHashInfo,
77    /// all slots that were roots within the last epoch
78    #[serde(deserialize_with = "default_on_eof")]
79    Vec<Slot>,
80    /// slots that were roots within the last epoch for which we care about the hash value
81    #[serde(deserialize_with = "default_on_eof")]
82    Vec<(Slot, Hash)>,
83);
84
85/// Incremental snapshots only calculate their accounts hash based on the
86/// account changes WITHIN the incremental slot range. So, we need to keep track
87/// of the full snapshot expected accounts hash results. We also need to keep
88/// track of the hash and capitalization specific to the incremental snapshot
89/// slot range. The capitalization we calculate for the incremental slot will
90/// NOT be consistent with the bank's capitalization. It is not feasible to
91/// calculate a capitalization delta that is correct given just incremental
92/// slots account data and the full snapshot's capitalization.
93#[cfg_attr(feature = "frozen-abi", derive(AbiExample))]
94#[derive(Serialize, Deserialize, Clone, Debug, Default, PartialEq, Eq)]
95pub struct BankIncrementalSnapshotPersistence {
96    /// slot of full snapshot
97    pub full_slot: Slot,
98    /// accounts hash from the full snapshot
99    pub full_hash: SerdeAccountsHash,
100    /// capitalization from the full snapshot
101    pub full_capitalization: u64,
102    /// hash of the accounts in the incremental snapshot slot range, including zero-lamport accounts
103    pub incremental_hash: SerdeIncrementalAccountsHash,
104    /// capitalization of the accounts in the incremental snapshot slot range
105    pub incremental_capitalization: u64,
106}
107
108#[cfg_attr(feature = "frozen-abi", derive(AbiExample))]
109#[derive(Clone, Default, Debug, Serialize, Deserialize, PartialEq, Eq)]
110struct BankHashInfo {
111    accounts_delta_hash: SerdeAccountsDeltaHash,
112    accounts_hash: SerdeAccountsHash,
113    stats: BankHashStats,
114}
115
116#[cfg_attr(feature = "frozen-abi", derive(AbiExample))]
117#[derive(Default, Clone, PartialEq, Eq, Debug, Deserialize, Serialize)]
118struct UnusedAccounts {
119    unused1: HashSet<Pubkey>,
120    unused2: HashSet<Pubkey>,
121    unused3: HashMap<Pubkey, u64>,
122}
123
124// Deserializable version of Bank which need not be serializable,
125// because it's handled by SerializableVersionedBank.
126// So, sync fields with it!
127#[derive(Clone, Deserialize)]
128struct DeserializableVersionedBank {
129    blockhash_queue: BlockhashQueue,
130    ancestors: AncestorsForSerialization,
131    hash: Hash,
132    parent_hash: Hash,
133    parent_slot: Slot,
134    hard_forks: HardForks,
135    transaction_count: u64,
136    tick_height: u64,
137    signature_count: u64,
138    capitalization: u64,
139    max_tick_height: u64,
140    hashes_per_tick: Option<u64>,
141    ticks_per_slot: u64,
142    ns_per_slot: u128,
143    genesis_creation_time: UnixTimestamp,
144    slots_per_year: f64,
145    accounts_data_len: u64,
146    slot: Slot,
147    epoch: Epoch,
148    block_height: u64,
149    collector_id: Pubkey,
150    collector_fees: u64,
151    _fee_calculator: FeeCalculator,
152    fee_rate_governor: FeeRateGovernor,
153    collected_rent: u64,
154    rent_collector: RentCollector,
155    epoch_schedule: EpochSchedule,
156    inflation: Inflation,
157    stakes: Stakes<Delegation>,
158    #[allow(dead_code)]
159    unused_accounts: UnusedAccounts,
160    epoch_stakes: HashMap<Epoch, EpochStakes>,
161    is_delta: bool,
162}
163
164impl From<DeserializableVersionedBank> for BankFieldsToDeserialize {
165    fn from(dvb: DeserializableVersionedBank) -> Self {
166        BankFieldsToDeserialize {
167            blockhash_queue: dvb.blockhash_queue,
168            ancestors: dvb.ancestors,
169            hash: dvb.hash,
170            parent_hash: dvb.parent_hash,
171            parent_slot: dvb.parent_slot,
172            hard_forks: dvb.hard_forks,
173            transaction_count: dvb.transaction_count,
174            tick_height: dvb.tick_height,
175            signature_count: dvb.signature_count,
176            capitalization: dvb.capitalization,
177            max_tick_height: dvb.max_tick_height,
178            hashes_per_tick: dvb.hashes_per_tick,
179            ticks_per_slot: dvb.ticks_per_slot,
180            ns_per_slot: dvb.ns_per_slot,
181            genesis_creation_time: dvb.genesis_creation_time,
182            slots_per_year: dvb.slots_per_year,
183            accounts_data_len: dvb.accounts_data_len,
184            slot: dvb.slot,
185            epoch: dvb.epoch,
186            block_height: dvb.block_height,
187            collector_id: dvb.collector_id,
188            collector_fees: dvb.collector_fees,
189            fee_rate_governor: dvb.fee_rate_governor,
190            collected_rent: dvb.collected_rent,
191            rent_collector: dvb.rent_collector,
192            epoch_schedule: dvb.epoch_schedule,
193            inflation: dvb.inflation,
194            stakes: dvb.stakes,
195            epoch_stakes: dvb.epoch_stakes,
196            is_delta: dvb.is_delta,
197            incremental_snapshot_persistence: None,
198            epoch_accounts_hash: None,
199            accounts_lt_hash: None, // populated from ExtraFieldsToDeserialize
200            bank_hash_stats: BankHashStats::default(), // populated from AccountsDbFields
201        }
202    }
203}
204
205// Serializable version of Bank, not Deserializable to avoid cloning by using refs.
206// Sync fields with DeserializableVersionedBank!
207#[derive(Serialize)]
208struct SerializableVersionedBank {
209    blockhash_queue: BlockhashQueue,
210    ancestors: AncestorsForSerialization,
211    hash: Hash,
212    parent_hash: Hash,
213    parent_slot: Slot,
214    hard_forks: HardForks,
215    transaction_count: u64,
216    tick_height: u64,
217    signature_count: u64,
218    capitalization: u64,
219    max_tick_height: u64,
220    hashes_per_tick: Option<u64>,
221    ticks_per_slot: u64,
222    ns_per_slot: u128,
223    genesis_creation_time: UnixTimestamp,
224    slots_per_year: f64,
225    accounts_data_len: u64,
226    slot: Slot,
227    epoch: Epoch,
228    block_height: u64,
229    collector_id: Pubkey,
230    collector_fees: u64,
231    fee_calculator: FeeCalculator,
232    fee_rate_governor: FeeRateGovernor,
233    collected_rent: u64,
234    rent_collector: RentCollector,
235    epoch_schedule: EpochSchedule,
236    inflation: Inflation,
237    #[serde(serialize_with = "serde_stakes_to_delegation_format::serialize")]
238    stakes: StakesEnum,
239    unused_accounts: UnusedAccounts,
240    epoch_stakes: HashMap<Epoch, EpochStakes>,
241    is_delta: bool,
242}
243
244impl From<BankFieldsToSerialize> for SerializableVersionedBank {
245    fn from(rhs: BankFieldsToSerialize) -> Self {
246        Self {
247            blockhash_queue: rhs.blockhash_queue,
248            ancestors: rhs.ancestors,
249            hash: rhs.hash,
250            parent_hash: rhs.parent_hash,
251            parent_slot: rhs.parent_slot,
252            hard_forks: rhs.hard_forks,
253            transaction_count: rhs.transaction_count,
254            tick_height: rhs.tick_height,
255            signature_count: rhs.signature_count,
256            capitalization: rhs.capitalization,
257            max_tick_height: rhs.max_tick_height,
258            hashes_per_tick: rhs.hashes_per_tick,
259            ticks_per_slot: rhs.ticks_per_slot,
260            ns_per_slot: rhs.ns_per_slot,
261            genesis_creation_time: rhs.genesis_creation_time,
262            slots_per_year: rhs.slots_per_year,
263            accounts_data_len: rhs.accounts_data_len,
264            slot: rhs.slot,
265            epoch: rhs.epoch,
266            block_height: rhs.block_height,
267            collector_id: rhs.collector_id,
268            collector_fees: rhs.collector_fees,
269            fee_calculator: FeeCalculator::default(),
270            fee_rate_governor: rhs.fee_rate_governor,
271            collected_rent: rhs.collected_rent,
272            rent_collector: rhs.rent_collector,
273            epoch_schedule: rhs.epoch_schedule,
274            inflation: rhs.inflation,
275            stakes: rhs.stakes,
276            unused_accounts: UnusedAccounts::default(),
277            epoch_stakes: rhs.epoch_stakes,
278            is_delta: rhs.is_delta,
279        }
280    }
281}
282
283#[cfg(feature = "frozen-abi")]
284impl solana_frozen_abi::abi_example::TransparentAsHelper for SerializableVersionedBank {}
285
286/// Helper type to wrap BufReader streams when deserializing and reconstructing from either just a
287/// full snapshot, or both a full and incremental snapshot
288pub struct SnapshotStreams<'a, R> {
289    pub full_snapshot_stream: &'a mut BufReader<R>,
290    pub incremental_snapshot_stream: Option<&'a mut BufReader<R>>,
291}
292
293/// Helper type to wrap BankFields when reconstructing Bank from either just a full
294/// snapshot, or both a full and incremental snapshot
295#[derive(Debug)]
296pub struct SnapshotBankFields {
297    full: BankFieldsToDeserialize,
298    incremental: Option<BankFieldsToDeserialize>,
299}
300
301impl SnapshotBankFields {
302    pub fn new(
303        full: BankFieldsToDeserialize,
304        incremental: Option<BankFieldsToDeserialize>,
305    ) -> Self {
306        Self { full, incremental }
307    }
308
309    /// Collapse the SnapshotBankFields into a single (the latest) BankFieldsToDeserialize.
310    pub fn collapse_into(self) -> BankFieldsToDeserialize {
311        self.incremental.unwrap_or(self.full)
312    }
313}
314
315/// Helper type to wrap AccountsDbFields when reconstructing AccountsDb from either just a full
316/// snapshot, or both a full and incremental snapshot
317#[derive(Debug)]
318pub struct SnapshotAccountsDbFields<T> {
319    full_snapshot_accounts_db_fields: AccountsDbFields<T>,
320    incremental_snapshot_accounts_db_fields: Option<AccountsDbFields<T>>,
321}
322
323impl<T> SnapshotAccountsDbFields<T> {
324    pub fn new(
325        full_snapshot_accounts_db_fields: AccountsDbFields<T>,
326        incremental_snapshot_accounts_db_fields: Option<AccountsDbFields<T>>,
327    ) -> Self {
328        Self {
329            full_snapshot_accounts_db_fields,
330            incremental_snapshot_accounts_db_fields,
331        }
332    }
333
334    /// Collapse the SnapshotAccountsDbFields into a single AccountsDbFields.  If there is no
335    /// incremental snapshot, this returns the AccountsDbFields from the full snapshot.
336    /// Otherwise, use the AccountsDbFields from the incremental snapshot, and a combination
337    /// of the storages from both the full and incremental snapshots.
338    pub fn collapse_into(self) -> Result<AccountsDbFields<T>, Error> {
339        match self.incremental_snapshot_accounts_db_fields {
340            None => Ok(self.full_snapshot_accounts_db_fields),
341            Some(AccountsDbFields(
342                mut incremental_snapshot_storages,
343                incremental_snapshot_version,
344                incremental_snapshot_slot,
345                incremental_snapshot_bank_hash_info,
346                incremental_snapshot_historical_roots,
347                incremental_snapshot_historical_roots_with_hash,
348            )) => {
349                let full_snapshot_storages = self.full_snapshot_accounts_db_fields.0;
350                let full_snapshot_slot = self.full_snapshot_accounts_db_fields.2;
351
352                // filter out incremental snapshot storages with slot <= full snapshot slot
353                incremental_snapshot_storages.retain(|slot, _| *slot > full_snapshot_slot);
354
355                // There must not be any overlap in the slots of storages between the full snapshot and the incremental snapshot
356                incremental_snapshot_storages
357                    .iter()
358                    .all(|storage_entry| !full_snapshot_storages.contains_key(storage_entry.0)).then_some(()).ok_or_else(|| {
359                        io::Error::new(io::ErrorKind::InvalidData, "Snapshots are incompatible: There are storages for the same slot in both the full snapshot and the incremental snapshot!")
360                    })?;
361
362                let mut combined_storages = full_snapshot_storages;
363                combined_storages.extend(incremental_snapshot_storages);
364
365                Ok(AccountsDbFields(
366                    combined_storages,
367                    incremental_snapshot_version,
368                    incremental_snapshot_slot,
369                    incremental_snapshot_bank_hash_info,
370                    incremental_snapshot_historical_roots,
371                    incremental_snapshot_historical_roots_with_hash,
372                ))
373            }
374        }
375    }
376}
377
378fn deserialize_from<R, T>(reader: R) -> bincode::Result<T>
379where
380    R: Read,
381    T: DeserializeOwned,
382{
383    bincode::options()
384        .with_limit(MAX_STREAM_SIZE)
385        .with_fixint_encoding()
386        .allow_trailing_bytes()
387        .deserialize_from::<R, T>(reader)
388}
389
390fn deserialize_accounts_db_fields<R>(
391    stream: &mut BufReader<R>,
392) -> Result<AccountsDbFields<SerializableAccountStorageEntry>, Error>
393where
394    R: Read,
395{
396    deserialize_from::<_, _>(stream)
397}
398
399/// Extra fields that are deserialized from the end of snapshots.
400///
401/// Note that this struct's fields should stay synced with the fields in
402/// ExtraFieldsToSerialize with the exception that new "extra fields" should be
403/// added to this struct a minor release before they are added to the serialize
404/// struct.
405#[cfg_attr(feature = "frozen-abi", derive(AbiExample))]
406#[cfg_attr(feature = "dev-context-only-utils", derive(PartialEq))]
407#[derive(Clone, Debug, Deserialize)]
408struct ExtraFieldsToDeserialize {
409    #[serde(deserialize_with = "default_on_eof")]
410    lamports_per_signature: u64,
411    #[serde(deserialize_with = "default_on_eof")]
412    incremental_snapshot_persistence: Option<BankIncrementalSnapshotPersistence>,
413    #[serde(deserialize_with = "default_on_eof")]
414    epoch_accounts_hash: Option<Hash>,
415    #[serde(deserialize_with = "default_on_eof")]
416    versioned_epoch_stakes: HashMap<u64, VersionedEpochStakes>,
417    #[serde(deserialize_with = "default_on_eof")]
418    accounts_lt_hash: Option<SerdeAccountsLtHash>,
419}
420
421/// Extra fields that are serialized at the end of snapshots.
422///
423/// Note that this struct's fields should stay synced with the fields in
424/// ExtraFieldsToDeserialize with the exception that new "extra fields" should
425/// be added to the deserialize struct a minor release before they are added to
426/// this one.
427#[cfg_attr(feature = "frozen-abi", derive(AbiExample))]
428#[cfg_attr(feature = "dev-context-only-utils", derive(Default, PartialEq))]
429#[derive(Debug, Serialize)]
430pub struct ExtraFieldsToSerialize<'a> {
431    pub lamports_per_signature: u64,
432    pub incremental_snapshot_persistence: Option<&'a BankIncrementalSnapshotPersistence>,
433    pub epoch_accounts_hash: Option<EpochAccountsHash>,
434    pub versioned_epoch_stakes: HashMap<u64, VersionedEpochStakes>,
435    pub accounts_lt_hash: Option<SerdeAccountsLtHash>,
436}
437
438fn deserialize_bank_fields<R>(
439    mut stream: &mut BufReader<R>,
440) -> Result<
441    (
442        BankFieldsToDeserialize,
443        AccountsDbFields<SerializableAccountStorageEntry>,
444    ),
445    Error,
446>
447where
448    R: Read,
449{
450    let mut bank_fields: BankFieldsToDeserialize =
451        deserialize_from::<_, DeserializableVersionedBank>(&mut stream)?.into();
452    let accounts_db_fields = deserialize_accounts_db_fields(stream)?;
453    let extra_fields = deserialize_from(stream)?;
454
455    // Process extra fields
456    let ExtraFieldsToDeserialize {
457        lamports_per_signature,
458        incremental_snapshot_persistence,
459        epoch_accounts_hash,
460        versioned_epoch_stakes,
461        accounts_lt_hash,
462    } = extra_fields;
463
464    bank_fields.fee_rate_governor = bank_fields
465        .fee_rate_governor
466        .clone_with_lamports_per_signature(lamports_per_signature);
467    bank_fields.incremental_snapshot_persistence = incremental_snapshot_persistence;
468    bank_fields.epoch_accounts_hash = epoch_accounts_hash;
469
470    // If we deserialize the new epoch stakes, add all of the entries into the
471    // other deserialized map which could still have old epoch stakes entries
472    bank_fields.epoch_stakes.extend(
473        versioned_epoch_stakes
474            .into_iter()
475            .map(|(epoch, versioned_epoch_stakes)| (epoch, versioned_epoch_stakes.into())),
476    );
477
478    bank_fields.accounts_lt_hash = accounts_lt_hash.map(Into::into);
479
480    Ok((bank_fields, accounts_db_fields))
481}
482
483/// used by tests to compare contents of serialized bank fields
484/// serialized format is not deterministic - likely due to randomness in structs like hashmaps
485#[cfg(feature = "dev-context-only-utils")]
486pub(crate) fn compare_two_serialized_banks(
487    path1: impl AsRef<Path>,
488    path2: impl AsRef<Path>,
489) -> std::result::Result<bool, Error> {
490    use std::fs::File;
491    let file1 = File::open(path1)?;
492    let mut stream1 = BufReader::new(file1);
493    let file2 = File::open(path2)?;
494    let mut stream2 = BufReader::new(file2);
495
496    let fields1 = deserialize_bank_fields(&mut stream1)?;
497    let fields2 = deserialize_bank_fields(&mut stream2)?;
498    Ok(fields1 == fields2)
499}
500
501/// Get snapshot storage lengths from accounts_db_fields
502pub(crate) fn snapshot_storage_lengths_from_fields(
503    accounts_db_fields: &AccountsDbFields<SerializableAccountStorageEntry>,
504) -> HashMap<Slot, HashMap<SerializedAccountsFileId, usize>> {
505    let AccountsDbFields(snapshot_storage, ..) = &accounts_db_fields;
506    snapshot_storage
507        .iter()
508        .map(|(slot, slot_storage)| {
509            (
510                *slot,
511                slot_storage
512                    .iter()
513                    .map(|storage_entry| (storage_entry.id(), storage_entry.current_len()))
514                    .collect(),
515            )
516        })
517        .collect()
518}
519
520pub(crate) fn fields_from_stream<R: Read>(
521    snapshot_stream: &mut BufReader<R>,
522) -> std::result::Result<
523    (
524        BankFieldsToDeserialize,
525        AccountsDbFields<SerializableAccountStorageEntry>,
526    ),
527    Error,
528> {
529    deserialize_bank_fields(snapshot_stream)
530}
531
532#[cfg(feature = "dev-context-only-utils")]
533pub(crate) fn fields_from_streams(
534    snapshot_streams: &mut SnapshotStreams<impl Read>,
535) -> std::result::Result<
536    (
537        SnapshotBankFields,
538        SnapshotAccountsDbFields<SerializableAccountStorageEntry>,
539    ),
540    Error,
541> {
542    let (full_snapshot_bank_fields, full_snapshot_accounts_db_fields) =
543        fields_from_stream(snapshot_streams.full_snapshot_stream)?;
544    let (incremental_snapshot_bank_fields, incremental_snapshot_accounts_db_fields) =
545        snapshot_streams
546            .incremental_snapshot_stream
547            .as_mut()
548            .map(|stream| fields_from_stream(stream))
549            .transpose()?
550            .unzip();
551
552    let snapshot_bank_fields = SnapshotBankFields {
553        full: full_snapshot_bank_fields,
554        incremental: incremental_snapshot_bank_fields,
555    };
556    let snapshot_accounts_db_fields = SnapshotAccountsDbFields {
557        full_snapshot_accounts_db_fields,
558        incremental_snapshot_accounts_db_fields,
559    };
560    Ok((snapshot_bank_fields, snapshot_accounts_db_fields))
561}
562
563/// This struct contains side-info while reconstructing the bank from streams
564#[derive(Debug)]
565pub struct BankFromStreamsInfo {
566    pub duplicates_lt_hash: Option<Box<DuplicatesLtHash>>,
567}
568
569#[allow(clippy::too_many_arguments)]
570#[cfg(test)]
571pub(crate) fn bank_from_streams<R>(
572    snapshot_streams: &mut SnapshotStreams<R>,
573    account_paths: &[PathBuf],
574    storage_and_next_append_vec_id: StorageAndNextAccountsFileId,
575    genesis_config: &GenesisConfig,
576    runtime_config: &RuntimeConfig,
577    debug_keys: Option<Arc<HashSet<Pubkey>>>,
578    additional_builtins: Option<&[BuiltinPrototype]>,
579    limit_load_slot_count_from_snapshot: Option<usize>,
580    verify_index: bool,
581    accounts_db_config: Option<AccountsDbConfig>,
582    accounts_update_notifier: Option<AccountsUpdateNotifier>,
583    exit: Arc<AtomicBool>,
584) -> std::result::Result<(Bank, BankFromStreamsInfo), Error>
585where
586    R: Read,
587{
588    let (bank_fields, accounts_db_fields) = fields_from_streams(snapshot_streams)?;
589    let (bank, info) = reconstruct_bank_from_fields(
590        bank_fields,
591        accounts_db_fields,
592        genesis_config,
593        runtime_config,
594        account_paths,
595        storage_and_next_append_vec_id,
596        debug_keys,
597        additional_builtins,
598        limit_load_slot_count_from_snapshot,
599        verify_index,
600        accounts_db_config,
601        accounts_update_notifier,
602        exit,
603    )?;
604    Ok((
605        bank,
606        BankFromStreamsInfo {
607            duplicates_lt_hash: info.duplicates_lt_hash,
608        },
609    ))
610}
611
612#[cfg(test)]
613pub(crate) fn bank_to_stream<W>(
614    stream: &mut BufWriter<W>,
615    bank: &Bank,
616    snapshot_storages: &[Vec<Arc<AccountStorageEntry>>],
617) -> Result<(), Error>
618where
619    W: Write,
620{
621    bincode::serialize_into(
622        stream,
623        &SerializableBankAndStorage {
624            bank,
625            snapshot_storages,
626        },
627    )
628}
629
630#[cfg(test)]
631pub(crate) fn bank_to_stream_no_extra_fields<W>(
632    stream: &mut BufWriter<W>,
633    bank: &Bank,
634    snapshot_storages: &[Vec<Arc<AccountStorageEntry>>],
635) -> Result<(), Error>
636where
637    W: Write,
638{
639    bincode::serialize_into(
640        stream,
641        &SerializableBankAndStorageNoExtra {
642            bank,
643            snapshot_storages,
644        },
645    )
646}
647
648/// Serializes bank snapshot into `stream` with bincode
649pub fn serialize_bank_snapshot_into<W>(
650    stream: &mut BufWriter<W>,
651    bank_fields: BankFieldsToSerialize,
652    bank_hash_stats: BankHashStats,
653    accounts_delta_hash: AccountsDeltaHash,
654    accounts_hash: AccountsHash,
655    account_storage_entries: &[Vec<Arc<AccountStorageEntry>>],
656    extra_fields: ExtraFieldsToSerialize,
657    write_version: u64,
658) -> Result<(), Error>
659where
660    W: Write,
661{
662    let mut serializer = bincode::Serializer::new(
663        stream,
664        bincode::DefaultOptions::new().with_fixint_encoding(),
665    );
666    serialize_bank_snapshot_with(
667        &mut serializer,
668        bank_fields,
669        bank_hash_stats,
670        accounts_delta_hash,
671        accounts_hash,
672        account_storage_entries,
673        extra_fields,
674        write_version,
675    )
676}
677
678/// Serializes bank snapshot with `serializer`
679pub fn serialize_bank_snapshot_with<S>(
680    serializer: S,
681    bank_fields: BankFieldsToSerialize,
682    bank_hash_stats: BankHashStats,
683    accounts_delta_hash: AccountsDeltaHash,
684    accounts_hash: AccountsHash,
685    account_storage_entries: &[Vec<Arc<AccountStorageEntry>>],
686    extra_fields: ExtraFieldsToSerialize,
687    write_version: u64,
688) -> Result<S::Ok, S::Error>
689where
690    S: serde::Serializer,
691{
692    let slot = bank_fields.slot;
693    let serializable_bank = SerializableVersionedBank::from(bank_fields);
694    let serializable_accounts_db = SerializableAccountsDb::<'_> {
695        slot,
696        account_storage_entries,
697        bank_hash_stats,
698        accounts_delta_hash,
699        accounts_hash,
700        write_version,
701    };
702    (serializable_bank, serializable_accounts_db, extra_fields).serialize(serializer)
703}
704
705#[cfg(test)]
706struct SerializableBankAndStorage<'a> {
707    bank: &'a Bank,
708    snapshot_storages: &'a [Vec<Arc<AccountStorageEntry>>],
709}
710
711#[cfg(test)]
712impl Serialize for SerializableBankAndStorage<'_> {
713    fn serialize<S>(&self, serializer: S) -> std::result::Result<S::Ok, S::Error>
714    where
715        S: serde::ser::Serializer,
716    {
717        let slot = self.bank.slot();
718        let mut bank_fields = self.bank.get_fields_to_serialize();
719        let accounts_db = &self.bank.rc.accounts.accounts_db;
720        let bank_hash_stats = self.bank.get_bank_hash_stats();
721        let accounts_delta_hash = accounts_db.get_accounts_delta_hash(slot).unwrap();
722        let accounts_hash = accounts_db.get_accounts_hash(slot).unwrap().0;
723        let write_version = accounts_db.write_version.load(Ordering::Acquire);
724        let lamports_per_signature = bank_fields.fee_rate_governor.lamports_per_signature;
725        let versioned_epoch_stakes = std::mem::take(&mut bank_fields.versioned_epoch_stakes);
726        let accounts_lt_hash = bank_fields.accounts_lt_hash.clone().map(Into::into);
727        let bank_fields_to_serialize = (
728            SerializableVersionedBank::from(bank_fields),
729            SerializableAccountsDb::<'_> {
730                slot,
731                account_storage_entries: self.snapshot_storages,
732                bank_hash_stats,
733                accounts_delta_hash,
734                accounts_hash,
735                write_version,
736            },
737            ExtraFieldsToSerialize {
738                lamports_per_signature,
739                incremental_snapshot_persistence: None,
740                epoch_accounts_hash: self.bank.get_epoch_accounts_hash_to_serialize(),
741                versioned_epoch_stakes,
742                accounts_lt_hash,
743            },
744        );
745        bank_fields_to_serialize.serialize(serializer)
746    }
747}
748
749#[cfg(test)]
750struct SerializableBankAndStorageNoExtra<'a> {
751    bank: &'a Bank,
752    snapshot_storages: &'a [Vec<Arc<AccountStorageEntry>>],
753}
754
755#[cfg(test)]
756impl Serialize for SerializableBankAndStorageNoExtra<'_> {
757    fn serialize<S>(&self, serializer: S) -> std::result::Result<S::Ok, S::Error>
758    where
759        S: serde::ser::Serializer,
760    {
761        let slot = self.bank.slot();
762        let bank_fields = self.bank.get_fields_to_serialize();
763        let accounts_db = &self.bank.rc.accounts.accounts_db;
764        let bank_hash_stats = self.bank.get_bank_hash_stats();
765        let accounts_delta_hash = accounts_db.get_accounts_delta_hash(slot).unwrap();
766        let accounts_hash = accounts_db.get_accounts_hash(slot).unwrap().0;
767        let write_version = accounts_db.write_version.load(Ordering::Acquire);
768        (
769            SerializableVersionedBank::from(bank_fields),
770            SerializableAccountsDb::<'_> {
771                slot,
772                account_storage_entries: self.snapshot_storages,
773                bank_hash_stats,
774                accounts_delta_hash,
775                accounts_hash,
776                write_version,
777            },
778        )
779            .serialize(serializer)
780    }
781}
782
783#[cfg(test)]
784impl<'a> From<SerializableBankAndStorageNoExtra<'a>> for SerializableBankAndStorage<'a> {
785    fn from(s: SerializableBankAndStorageNoExtra<'a>) -> SerializableBankAndStorage<'a> {
786        let SerializableBankAndStorageNoExtra {
787            bank,
788            snapshot_storages,
789        } = s;
790        SerializableBankAndStorage {
791            bank,
792            snapshot_storages,
793        }
794    }
795}
796
797struct SerializableAccountsDb<'a> {
798    slot: Slot,
799    account_storage_entries: &'a [Vec<Arc<AccountStorageEntry>>],
800    bank_hash_stats: BankHashStats,
801    accounts_delta_hash: AccountsDeltaHash,
802    accounts_hash: AccountsHash,
803    write_version: u64,
804}
805
806impl Serialize for SerializableAccountsDb<'_> {
807    fn serialize<S>(&self, serializer: S) -> std::result::Result<S::Ok, S::Error>
808    where
809        S: serde::ser::Serializer,
810    {
811        // (1st of 3 elements) write the list of account storage entry lists out as a map
812        let entry_count = RefCell::<usize>::new(0);
813        let entries = utils::serialize_iter_as_map(self.account_storage_entries.iter().map(|x| {
814            *entry_count.borrow_mut() += x.len();
815            (
816                x.first().unwrap().slot(),
817                utils::serialize_iter_as_seq(
818                    x.iter()
819                        .map(|x| SerializableAccountStorageEntry::new(x.as_ref(), self.slot)),
820                ),
821            )
822        }));
823        let bank_hash_info = BankHashInfo {
824            accounts_delta_hash: self.accounts_delta_hash.into(),
825            accounts_hash: self.accounts_hash.into(),
826            stats: self.bank_hash_stats.clone(),
827        };
828
829        let historical_roots = Vec::<Slot>::default();
830        let historical_roots_with_hash = Vec::<(Slot, Hash)>::default();
831
832        let mut serialize_account_storage_timer = Measure::start("serialize_account_storage_ms");
833        let result = (
834            entries,
835            self.write_version,
836            self.slot,
837            bank_hash_info,
838            historical_roots,
839            historical_roots_with_hash,
840        )
841            .serialize(serializer);
842        serialize_account_storage_timer.stop();
843        datapoint_info!(
844            "serialize_account_storage_ms",
845            ("duration", serialize_account_storage_timer.as_ms(), i64),
846            ("num_entries", *entry_count.borrow(), i64),
847        );
848        result
849    }
850}
851
852#[cfg(feature = "frozen-abi")]
853impl solana_frozen_abi::abi_example::TransparentAsHelper for SerializableAccountsDb<'_> {}
854
855/// This struct contains side-info while reconstructing the bank from fields
856#[derive(Debug)]
857pub(crate) struct ReconstructedBankInfo {
858    pub(crate) duplicates_lt_hash: Option<Box<DuplicatesLtHash>>,
859}
860
861#[allow(clippy::too_many_arguments)]
862pub(crate) fn reconstruct_bank_from_fields<E>(
863    bank_fields: SnapshotBankFields,
864    snapshot_accounts_db_fields: SnapshotAccountsDbFields<E>,
865    genesis_config: &GenesisConfig,
866    runtime_config: &RuntimeConfig,
867    account_paths: &[PathBuf],
868    storage_and_next_append_vec_id: StorageAndNextAccountsFileId,
869    debug_keys: Option<Arc<HashSet<Pubkey>>>,
870    additional_builtins: Option<&[BuiltinPrototype]>,
871    limit_load_slot_count_from_snapshot: Option<usize>,
872    verify_index: bool,
873    accounts_db_config: Option<AccountsDbConfig>,
874    accounts_update_notifier: Option<AccountsUpdateNotifier>,
875    exit: Arc<AtomicBool>,
876) -> Result<(Bank, ReconstructedBankInfo), Error>
877where
878    E: SerializableStorage + std::marker::Sync,
879{
880    let capitalizations = (
881        bank_fields.full.capitalization,
882        bank_fields
883            .incremental
884            .as_ref()
885            .map(|bank_fields| bank_fields.capitalization),
886    );
887    let mut bank_fields = bank_fields.collapse_into();
888    let (accounts_db, reconstructed_accounts_db_info) = reconstruct_accountsdb_from_fields(
889        snapshot_accounts_db_fields,
890        account_paths,
891        storage_and_next_append_vec_id,
892        genesis_config,
893        limit_load_slot_count_from_snapshot,
894        verify_index,
895        accounts_db_config,
896        accounts_update_notifier,
897        exit,
898        bank_fields.epoch_accounts_hash,
899        capitalizations,
900        bank_fields.incremental_snapshot_persistence.as_ref(),
901        bank_fields.accounts_lt_hash.is_some(),
902    )?;
903    bank_fields.bank_hash_stats = reconstructed_accounts_db_info.bank_hash_stats;
904
905    let bank_rc = BankRc::new(Accounts::new(Arc::new(accounts_db)));
906    let runtime_config = Arc::new(runtime_config.clone());
907
908    // if limit_load_slot_count_from_snapshot is set, then we need to side-step some correctness checks beneath this call
909    let debug_do_not_add_builtins = limit_load_slot_count_from_snapshot.is_some();
910    let bank = Bank::new_from_fields(
911        bank_rc,
912        genesis_config,
913        runtime_config,
914        bank_fields,
915        debug_keys,
916        additional_builtins,
917        debug_do_not_add_builtins,
918        reconstructed_accounts_db_info.accounts_data_len,
919    );
920
921    info!("rent_collector: {:?}", bank.rent_collector());
922    Ok((
923        bank,
924        ReconstructedBankInfo {
925            duplicates_lt_hash: reconstructed_accounts_db_info.duplicates_lt_hash,
926        },
927    ))
928}
929
930pub(crate) fn reconstruct_single_storage(
931    slot: &Slot,
932    append_vec_path: &Path,
933    current_len: usize,
934    append_vec_id: AccountsFileId,
935    storage_access: StorageAccess,
936) -> Result<Arc<AccountStorageEntry>, SnapshotError> {
937    let accounts_file =
938        AccountsFile::new_for_startup(append_vec_path, current_len, storage_access)?;
939    Ok(Arc::new(AccountStorageEntry::new_existing(
940        *slot,
941        append_vec_id,
942        accounts_file,
943    )))
944}
945
946// Remap the AppendVec ID to handle any duplicate IDs that may previously existed
947// due to full snapshots and incremental snapshots generated from different
948// nodes
949pub(crate) fn remap_append_vec_file(
950    slot: Slot,
951    old_append_vec_id: SerializedAccountsFileId,
952    append_vec_path: &Path,
953    next_append_vec_id: &AtomicAccountsFileId,
954    num_collisions: &AtomicUsize,
955) -> io::Result<(AccountsFileId, PathBuf)> {
956    #[cfg(target_os = "linux")]
957    let append_vec_path_cstr = cstring_from_path(append_vec_path)?;
958
959    let mut remapped_append_vec_path = append_vec_path.to_path_buf();
960
961    // Break out of the loop in the following situations:
962    // 1. The new ID is the same as the original ID.  This means we do not need to
963    //    rename the file, since the ID is the "correct" one already.
964    // 2. There is not a file already at the new path.  This means it is safe to
965    //    rename the file to this new path.
966    let (remapped_append_vec_id, remapped_append_vec_path) = loop {
967        let remapped_append_vec_id = next_append_vec_id.fetch_add(1, Ordering::AcqRel);
968
969        // this can only happen in the first iteration of the loop
970        if old_append_vec_id == remapped_append_vec_id as SerializedAccountsFileId {
971            break (remapped_append_vec_id, remapped_append_vec_path);
972        }
973
974        let remapped_file_name = AccountsFile::file_name(slot, remapped_append_vec_id);
975        remapped_append_vec_path = append_vec_path.parent().unwrap().join(remapped_file_name);
976
977        #[cfg(all(target_os = "linux", target_env = "gnu"))]
978        {
979            let remapped_append_vec_path_cstr = cstring_from_path(&remapped_append_vec_path)?;
980
981            // On linux we use renameat2(NO_REPLACE) instead of IF metadata(path).is_err() THEN
982            // rename() in order to save a statx() syscall.
983            match rename_no_replace(&append_vec_path_cstr, &remapped_append_vec_path_cstr) {
984                // If the file was successfully renamed, break out of the loop
985                Ok(_) => break (remapped_append_vec_id, remapped_append_vec_path),
986                // If there's already a file at the new path, continue so we try
987                // the next ID
988                Err(e) if e.kind() == io::ErrorKind::AlreadyExists => {}
989                Err(e) => return Err(e),
990            }
991        }
992
993        #[cfg(any(
994            not(target_os = "linux"),
995            all(target_os = "linux", not(target_env = "gnu"))
996        ))]
997        if std::fs::metadata(&remapped_append_vec_path).is_err() {
998            break (remapped_append_vec_id, remapped_append_vec_path);
999        }
1000
1001        // If we made it this far, a file exists at the new path.  Record the collision
1002        // and try again.
1003        num_collisions.fetch_add(1, Ordering::Relaxed);
1004    };
1005
1006    // Only rename the file if the new ID is actually different from the original. In the target_os
1007    // = linux case, we have already renamed if necessary.
1008    #[cfg(any(
1009        not(target_os = "linux"),
1010        all(target_os = "linux", not(target_env = "gnu"))
1011    ))]
1012    if old_append_vec_id != remapped_append_vec_id as SerializedAccountsFileId {
1013        std::fs::rename(append_vec_path, &remapped_append_vec_path)?;
1014    }
1015
1016    Ok((remapped_append_vec_id, remapped_append_vec_path))
1017}
1018
1019pub(crate) fn remap_and_reconstruct_single_storage(
1020    slot: Slot,
1021    old_append_vec_id: SerializedAccountsFileId,
1022    current_len: usize,
1023    append_vec_path: &Path,
1024    next_append_vec_id: &AtomicAccountsFileId,
1025    num_collisions: &AtomicUsize,
1026    storage_access: StorageAccess,
1027) -> Result<Arc<AccountStorageEntry>, SnapshotError> {
1028    let (remapped_append_vec_id, remapped_append_vec_path) = remap_append_vec_file(
1029        slot,
1030        old_append_vec_id,
1031        append_vec_path,
1032        next_append_vec_id,
1033        num_collisions,
1034    )?;
1035    let storage = reconstruct_single_storage(
1036        &slot,
1037        &remapped_append_vec_path,
1038        current_len,
1039        remapped_append_vec_id,
1040        storage_access,
1041    )?;
1042    Ok(storage)
1043}
1044
1045/// This struct contains side-info while reconstructing the accounts DB from fields.
1046#[derive(Debug, Default, Clone)]
1047pub struct ReconstructedAccountsDbInfo {
1048    pub accounts_data_len: u64,
1049    pub duplicates_lt_hash: Option<Box<DuplicatesLtHash>>,
1050    pub bank_hash_stats: BankHashStats,
1051}
1052
1053#[allow(clippy::too_many_arguments)]
1054fn reconstruct_accountsdb_from_fields<E>(
1055    snapshot_accounts_db_fields: SnapshotAccountsDbFields<E>,
1056    account_paths: &[PathBuf],
1057    storage_and_next_append_vec_id: StorageAndNextAccountsFileId,
1058    genesis_config: &GenesisConfig,
1059    limit_load_slot_count_from_snapshot: Option<usize>,
1060    verify_index: bool,
1061    accounts_db_config: Option<AccountsDbConfig>,
1062    accounts_update_notifier: Option<AccountsUpdateNotifier>,
1063    exit: Arc<AtomicBool>,
1064    epoch_accounts_hash: Option<Hash>,
1065    capitalizations: (u64, Option<u64>),
1066    incremental_snapshot_persistence: Option<&BankIncrementalSnapshotPersistence>,
1067    has_accounts_lt_hash: bool,
1068) -> Result<(AccountsDb, ReconstructedAccountsDbInfo), Error>
1069where
1070    E: SerializableStorage + std::marker::Sync,
1071{
1072    let mut accounts_db = AccountsDb::new_with_config(
1073        account_paths.to_vec(),
1074        accounts_db_config,
1075        accounts_update_notifier,
1076        exit,
1077    );
1078
1079    if let Some(epoch_accounts_hash) = epoch_accounts_hash {
1080        accounts_db
1081            .epoch_accounts_hash_manager
1082            .set_valid(EpochAccountsHash::new(epoch_accounts_hash), 0);
1083    }
1084
1085    // Store the accounts hash & capitalization, from the full snapshot, in the new AccountsDb
1086    {
1087        let AccountsDbFields(_, _, slot, bank_hash_info, _, _) =
1088            &snapshot_accounts_db_fields.full_snapshot_accounts_db_fields;
1089
1090        if let Some(incremental_snapshot_persistence) = incremental_snapshot_persistence {
1091            // If we've booted from local state that was originally intended to be an incremental
1092            // snapshot, then we will use the incremental snapshot persistence field to set the
1093            // initial accounts hashes in accounts db.
1094            let old_accounts_hash = accounts_db.set_accounts_hash_from_snapshot(
1095                incremental_snapshot_persistence.full_slot,
1096                incremental_snapshot_persistence.full_hash.clone(),
1097                incremental_snapshot_persistence.full_capitalization,
1098            );
1099            assert!(
1100                old_accounts_hash.is_none(),
1101                "There should not already be an AccountsHash at slot {slot}: {old_accounts_hash:?}",
1102            );
1103            let old_incremental_accounts_hash = accounts_db
1104                .set_incremental_accounts_hash_from_snapshot(
1105                    *slot,
1106                    incremental_snapshot_persistence.incremental_hash.clone(),
1107                    incremental_snapshot_persistence.incremental_capitalization,
1108                );
1109            assert!(
1110                old_incremental_accounts_hash.is_none(),
1111                "There should not already be an IncrementalAccountsHash at slot {slot}: {old_incremental_accounts_hash:?}",
1112            );
1113        } else {
1114            // Otherwise, we've booted from a snapshot archive, or from local state that was *not*
1115            // intended to be an incremental snapshot.
1116            let old_accounts_hash = accounts_db.set_accounts_hash_from_snapshot(
1117                *slot,
1118                bank_hash_info.accounts_hash.clone(),
1119                capitalizations.0,
1120            );
1121            assert!(
1122                old_accounts_hash.is_none(),
1123                "There should not already be an AccountsHash at slot {slot}: {old_accounts_hash:?}",
1124            );
1125        }
1126    }
1127
1128    // Store the accounts hash & capitalization, from the incremental snapshot, in the new AccountsDb
1129    {
1130        if let Some(AccountsDbFields(_, _, slot, bank_hash_info, _, _)) =
1131            snapshot_accounts_db_fields
1132                .incremental_snapshot_accounts_db_fields
1133                .as_ref()
1134        {
1135            if let Some(incremental_snapshot_persistence) = incremental_snapshot_persistence {
1136                // Use the presence of a BankIncrementalSnapshotPersistence to indicate the
1137                // Incremental Accounts Hash feature is enabled, and use its accounts hashes
1138                // instead of `BankHashInfo`'s.
1139                let AccountsDbFields(_, _, full_slot, full_bank_hash_info, _, _) =
1140                    &snapshot_accounts_db_fields.full_snapshot_accounts_db_fields;
1141                let full_accounts_hash = &full_bank_hash_info.accounts_hash;
1142                assert_eq!(
1143                    incremental_snapshot_persistence.full_slot, *full_slot,
1144                    "The incremental snapshot's base slot ({}) must match the full snapshot's slot ({full_slot})!",
1145                    incremental_snapshot_persistence.full_slot,
1146                );
1147                assert_eq!(
1148                    &incremental_snapshot_persistence.full_hash, full_accounts_hash,
1149                    "The incremental snapshot's base accounts hash ({}) must match the full snapshot's accounts hash ({})!",
1150                    &incremental_snapshot_persistence.full_hash.0, full_accounts_hash.0,
1151                );
1152                assert_eq!(
1153                    incremental_snapshot_persistence.full_capitalization, capitalizations.0,
1154                    "The incremental snapshot's base capitalization ({}) must match the full snapshot's capitalization ({})!",
1155                    incremental_snapshot_persistence.full_capitalization, capitalizations.0,
1156                );
1157                let old_incremental_accounts_hash = accounts_db
1158                    .set_incremental_accounts_hash_from_snapshot(
1159                        *slot,
1160                        incremental_snapshot_persistence.incremental_hash.clone(),
1161                        incremental_snapshot_persistence.incremental_capitalization,
1162                    );
1163                assert!(
1164                    old_incremental_accounts_hash.is_none(),
1165                    "There should not already be an IncrementalAccountsHash at slot {slot}: {old_incremental_accounts_hash:?}",
1166                );
1167            } else {
1168                // ..and without a BankIncrementalSnapshotPersistence then the Incremental Accounts
1169                // Hash feature is disabled; the accounts hash in `BankHashInfo` is valid.
1170                let old_accounts_hash = accounts_db.set_accounts_hash_from_snapshot(
1171                    *slot,
1172                    bank_hash_info.accounts_hash.clone(),
1173                    capitalizations
1174                        .1
1175                        .expect("capitalization from incremental snapshot"),
1176                );
1177                assert!(
1178                    old_accounts_hash.is_none(),
1179                    "There should not already be an AccountsHash at slot {slot}: {old_accounts_hash:?}",
1180                );
1181            };
1182        }
1183    }
1184
1185    let AccountsDbFields(
1186        _snapshot_storages,
1187        snapshot_version,
1188        snapshot_slot,
1189        snapshot_bank_hash_info,
1190        _snapshot_historical_roots,
1191        _snapshot_historical_roots_with_hash,
1192    ) = snapshot_accounts_db_fields.collapse_into()?;
1193
1194    // Ensure all account paths exist
1195    for path in &accounts_db.paths {
1196        std::fs::create_dir_all(path)
1197            .unwrap_or_else(|err| panic!("Failed to create directory {}: {}", path.display(), err));
1198    }
1199
1200    let StorageAndNextAccountsFileId {
1201        storage,
1202        next_append_vec_id,
1203    } = storage_and_next_append_vec_id;
1204
1205    assert!(
1206        !storage.is_empty(),
1207        "At least one storage entry must exist from deserializing stream"
1208    );
1209
1210    let next_append_vec_id = next_append_vec_id.load(Ordering::Acquire);
1211    let max_append_vec_id = next_append_vec_id - 1;
1212    assert!(
1213        max_append_vec_id <= AccountsFileId::MAX / 2,
1214        "Storage id {max_append_vec_id} larger than allowed max"
1215    );
1216
1217    // Process deserialized data, set necessary fields in self
1218    let old_accounts_delta_hash = accounts_db.set_accounts_delta_hash_from_snapshot(
1219        snapshot_slot,
1220        snapshot_bank_hash_info.accounts_delta_hash,
1221    );
1222    assert!(
1223        old_accounts_delta_hash.is_none(),
1224        "There should not already be an AccountsDeltaHash at slot {snapshot_slot}: {old_accounts_delta_hash:?}",
1225        );
1226    accounts_db.storage.initialize(storage);
1227    accounts_db
1228        .next_id
1229        .store(next_append_vec_id, Ordering::Release);
1230    accounts_db
1231        .write_version
1232        .fetch_add(snapshot_version, Ordering::Release);
1233
1234    let mut measure_notify = Measure::start("accounts_notify");
1235
1236    let accounts_db = Arc::new(accounts_db);
1237    let accounts_db_clone = accounts_db.clone();
1238    let handle = Builder::new()
1239        .name("solNfyAccRestor".to_string())
1240        .spawn(move || {
1241            accounts_db_clone.notify_account_restore_from_snapshot();
1242        })
1243        .unwrap();
1244
1245    // When generating the index, we want to calculate the duplicates lt hash value (needed to do
1246    // the lattice-based verification of the accounts in the background) optimistically.
1247    // This means, either when the cli arg is set, or when the snapshot has an accounts lt hash.
1248    let is_accounts_lt_hash_enabled =
1249        accounts_db.is_experimental_accumulator_hash_enabled() || has_accounts_lt_hash;
1250    let IndexGenerationInfo {
1251        accounts_data_len,
1252        rent_paying_accounts_by_partition,
1253        duplicates_lt_hash,
1254    } = accounts_db.generate_index(
1255        limit_load_slot_count_from_snapshot,
1256        verify_index,
1257        genesis_config,
1258        is_accounts_lt_hash_enabled,
1259    );
1260    accounts_db
1261        .accounts_index
1262        .rent_paying_accounts_by_partition
1263        .set(rent_paying_accounts_by_partition)
1264        .unwrap();
1265
1266    handle.join().unwrap();
1267    measure_notify.stop();
1268
1269    datapoint_info!(
1270        "reconstruct_accountsdb_from_fields()",
1271        ("accountsdb-notify-at-start-us", measure_notify.as_us(), i64),
1272    );
1273
1274    Ok((
1275        Arc::try_unwrap(accounts_db).unwrap(),
1276        ReconstructedAccountsDbInfo {
1277            accounts_data_len,
1278            duplicates_lt_hash,
1279            bank_hash_stats: snapshot_bank_hash_info.stats,
1280        },
1281    ))
1282}
1283
1284// Rename `src` to `dest` only if `dest` doesn't already exist.
1285#[cfg(all(target_os = "linux", target_env = "gnu"))]
1286fn rename_no_replace(src: &CStr, dest: &CStr) -> io::Result<()> {
1287    let ret = unsafe {
1288        libc::renameat2(
1289            libc::AT_FDCWD,
1290            src.as_ptr() as *const _,
1291            libc::AT_FDCWD,
1292            dest.as_ptr() as *const _,
1293            libc::RENAME_NOREPLACE,
1294        )
1295    };
1296    if ret == -1 {
1297        return Err(io::Error::last_os_error());
1298    }
1299
1300    Ok(())
1301}
1302
1303#[cfg(target_os = "linux")]
1304fn cstring_from_path(path: &Path) -> io::Result<CString> {
1305    // It is better to allocate here than use the stack. Jemalloc is going to give us a chunk of a
1306    // preallocated small arena anyway. Instead if we used the stack since PATH_MAX=4096 it would
1307    // result in LLVM inserting a stack probe, see
1308    // https://docs.rs/compiler_builtins/latest/compiler_builtins/probestack/index.html.
1309    CString::new(path.as_os_str().as_encoded_bytes())
1310        .map_err(|e| io::Error::new(io::ErrorKind::InvalidInput, e))
1311}