solana_runtime/
bank_forks.rs

1//! The `bank_forks` module implements BankForks a DAG of checkpointed Banks
2
3use {
4    crate::{
5        accounts_background_service::{AbsRequestSender, SnapshotRequest, SnapshotRequestKind},
6        bank::{bank_hash_details, epoch_accounts_hash_utils, Bank, SquashTiming},
7        bank_hash_cache::DumpedSlotSubscription,
8        installed_scheduler_pool::{
9            BankWithScheduler, InstalledSchedulerPoolArc, SchedulingContext,
10        },
11        snapshot_config::SnapshotConfig,
12    },
13    crossbeam_channel::SendError,
14    log::*,
15    solana_measure::measure::Measure,
16    solana_program_runtime::loaded_programs::{BlockRelation, ForkGraph},
17    solana_sdk::{
18        clock::{BankId, Slot},
19        hash::Hash,
20    },
21    std::{
22        collections::{hash_map::Entry, HashMap, HashSet},
23        ops::Index,
24        sync::{
25            atomic::{AtomicBool, AtomicU64, Ordering},
26            Arc, RwLock,
27        },
28        time::Instant,
29    },
30    thiserror::Error,
31};
32
33pub const MAX_ROOT_DISTANCE_FOR_VOTE_ONLY: Slot = 400;
34pub type AtomicSlot = AtomicU64;
35pub struct ReadOnlyAtomicSlot {
36    slot: Arc<AtomicSlot>,
37}
38
39impl ReadOnlyAtomicSlot {
40    pub fn get(&self) -> Slot {
41        // The expectation is that an instance `ReadOnlyAtomicSlot` is on a different thread than
42        // BankForks *and* this instance is being accessed *without* locking BankForks first.
43        // Thus, to ensure atomic ordering correctness, we must use Acquire-Release semantics.
44        self.slot.load(Ordering::Acquire)
45    }
46}
47
48#[derive(Error, Debug)]
49pub enum SetRootError {
50    #[error("failed to send epoch accounts hash request for bank {0}: {1}")]
51    SendEpochAccountHashError(Slot, SendError<SnapshotRequest>),
52}
53
54#[derive(Debug, Default, Copy, Clone)]
55struct SetRootMetrics {
56    timings: SetRootTimings,
57    total_parent_banks: i64,
58    tx_count: i64,
59    dropped_banks_len: i64,
60    accounts_data_len: i64,
61}
62
63#[derive(Debug, Default, Copy, Clone)]
64struct SetRootTimings {
65    total_squash_time: SquashTiming,
66    total_snapshot_ms: i64,
67    prune_non_rooted_ms: i64,
68    drop_parent_banks_ms: i64,
69    prune_slots_ms: i64,
70    prune_remove_ms: i64,
71}
72
73pub struct BankForks {
74    banks: HashMap<Slot, BankWithScheduler>,
75    descendants: HashMap<Slot, HashSet<Slot>>,
76    root: Arc<AtomicSlot>,
77
78    pub snapshot_config: Option<SnapshotConfig>,
79
80    pub accounts_hash_interval_slots: Slot,
81    last_accounts_hash_slot: Slot,
82    in_vote_only_mode: Arc<AtomicBool>,
83    highest_slot_at_startup: Slot,
84    scheduler_pool: Option<InstalledSchedulerPoolArc>,
85
86    dumped_slot_subscribers: Vec<DumpedSlotSubscription>,
87}
88
89impl Index<u64> for BankForks {
90    type Output = Arc<Bank>;
91    fn index(&self, bank_slot: Slot) -> &Self::Output {
92        &self.banks[&bank_slot]
93    }
94}
95
96impl BankForks {
97    pub fn new_rw_arc(root_bank: Bank) -> Arc<RwLock<Self>> {
98        let root_bank = Arc::new(root_bank);
99        let root_slot = root_bank.slot();
100
101        let mut banks = HashMap::new();
102        banks.insert(
103            root_slot,
104            BankWithScheduler::new_without_scheduler(root_bank.clone()),
105        );
106
107        let parents = root_bank.parents();
108        for parent in parents {
109            if banks
110                .insert(
111                    parent.slot(),
112                    BankWithScheduler::new_without_scheduler(parent.clone()),
113                )
114                .is_some()
115            {
116                // All ancestors have already been inserted by another fork
117                break;
118            }
119        }
120
121        let mut descendants = HashMap::<_, HashSet<_>>::new();
122        descendants.entry(root_slot).or_default();
123        for parent in root_bank.proper_ancestors() {
124            descendants.entry(parent).or_default().insert(root_slot);
125        }
126
127        let bank_forks = Arc::new(RwLock::new(Self {
128            root: Arc::new(AtomicSlot::new(root_slot)),
129            banks,
130            descendants,
131            snapshot_config: None,
132            accounts_hash_interval_slots: u64::MAX,
133            last_accounts_hash_slot: root_slot,
134            in_vote_only_mode: Arc::new(AtomicBool::new(false)),
135            highest_slot_at_startup: 0,
136            scheduler_pool: None,
137            dumped_slot_subscribers: vec![],
138        }));
139
140        root_bank.set_fork_graph_in_program_cache(Arc::downgrade(&bank_forks));
141        bank_forks
142    }
143
144    pub fn banks(&self) -> &HashMap<Slot, BankWithScheduler> {
145        &self.banks
146    }
147
148    pub fn get_vote_only_mode_signal(&self) -> Arc<AtomicBool> {
149        self.in_vote_only_mode.clone()
150    }
151
152    pub fn len(&self) -> usize {
153        self.banks.len()
154    }
155
156    pub fn is_empty(&self) -> bool {
157        self.banks.is_empty()
158    }
159
160    /// Create a map of bank slot id to the set of ancestors for the bank slot.
161    pub fn ancestors(&self) -> HashMap<Slot, HashSet<Slot>> {
162        let root = self.root();
163        self.banks
164            .iter()
165            .map(|(slot, bank)| {
166                let ancestors = bank.proper_ancestors().filter(|k| *k >= root);
167                (*slot, ancestors.collect())
168            })
169            .collect()
170    }
171
172    /// Create a map of bank slot id to the set of all of its descendants
173    pub fn descendants(&self) -> HashMap<Slot, HashSet<Slot>> {
174        self.descendants.clone()
175    }
176
177    pub fn frozen_banks(&self) -> HashMap<Slot, Arc<Bank>> {
178        self.banks
179            .iter()
180            .filter(|(_, b)| b.is_frozen())
181            .map(|(&k, b)| (k, b.clone_without_scheduler()))
182            .collect()
183    }
184
185    pub fn active_bank_slots(&self) -> Vec<Slot> {
186        self.banks
187            .iter()
188            .filter(|(_, v)| !v.is_frozen())
189            .map(|(k, _v)| *k)
190            .collect()
191    }
192
193    pub fn get_with_scheduler(&self, bank_slot: Slot) -> Option<BankWithScheduler> {
194        self.banks.get(&bank_slot).map(|b| b.clone_with_scheduler())
195    }
196
197    pub fn get(&self, bank_slot: Slot) -> Option<Arc<Bank>> {
198        self.get_with_scheduler(bank_slot)
199            .map(|b| b.clone_without_scheduler())
200    }
201
202    pub fn get_with_checked_hash(
203        &self,
204        (bank_slot, expected_hash): (Slot, Hash),
205    ) -> Option<Arc<Bank>> {
206        let maybe_bank = self.get(bank_slot);
207        if let Some(bank) = &maybe_bank {
208            assert_eq!(bank.hash(), expected_hash);
209        }
210        maybe_bank
211    }
212
213    pub fn bank_hash(&self, slot: Slot) -> Option<Hash> {
214        self.get(slot).map(|bank| bank.hash())
215    }
216
217    pub fn root_bank(&self) -> Arc<Bank> {
218        self[self.root()].clone()
219    }
220
221    pub fn install_scheduler_pool(&mut self, pool: InstalledSchedulerPoolArc) {
222        info!("Installed new scheduler_pool into bank_forks: {:?}", pool);
223        assert!(
224            self.scheduler_pool.replace(pool).is_none(),
225            "Reinstalling scheduler pool isn't supported"
226        );
227    }
228
229    pub fn insert(&mut self, mut bank: Bank) -> BankWithScheduler {
230        if self.root.load(Ordering::Relaxed) < self.highest_slot_at_startup {
231            bank.set_check_program_modification_slot(true);
232        }
233
234        let bank = Arc::new(bank);
235        let bank = if let Some(scheduler_pool) = &self.scheduler_pool {
236            let context = SchedulingContext::new(bank.clone());
237            let scheduler = scheduler_pool.take_scheduler(context);
238            let bank_with_scheduler = BankWithScheduler::new(bank, Some(scheduler));
239            scheduler_pool.register_timeout_listener(bank_with_scheduler.create_timeout_listener());
240            bank_with_scheduler
241        } else {
242            BankWithScheduler::new_without_scheduler(bank)
243        };
244        let prev = self.banks.insert(bank.slot(), bank.clone_with_scheduler());
245        assert!(prev.is_none());
246        let slot = bank.slot();
247        self.descendants.entry(slot).or_default();
248        for parent in bank.proper_ancestors() {
249            self.descendants.entry(parent).or_default().insert(slot);
250        }
251        bank
252    }
253
254    pub fn insert_from_ledger(&mut self, bank: Bank) -> BankWithScheduler {
255        self.highest_slot_at_startup = std::cmp::max(self.highest_slot_at_startup, bank.slot());
256        self.insert(bank)
257    }
258
259    pub fn remove(&mut self, slot: Slot) -> Option<BankWithScheduler> {
260        let bank = self.banks.remove(&slot)?;
261        for parent in bank.proper_ancestors() {
262            let Entry::Occupied(mut entry) = self.descendants.entry(parent) else {
263                panic!("this should not happen!");
264            };
265            entry.get_mut().remove(&slot);
266            if entry.get().is_empty() && !self.banks.contains_key(&parent) {
267                entry.remove_entry();
268            }
269        }
270        let Entry::Occupied(entry) = self.descendants.entry(slot) else {
271            panic!("this should not happen!");
272        };
273        if entry.get().is_empty() {
274            entry.remove_entry();
275        }
276        Some(bank)
277    }
278
279    pub fn highest_slot(&self) -> Slot {
280        self.banks.values().map(|bank| bank.slot()).max().unwrap()
281    }
282
283    pub fn working_bank(&self) -> Arc<Bank> {
284        self[self.highest_slot()].clone()
285    }
286
287    pub fn working_bank_with_scheduler(&self) -> &BankWithScheduler {
288        &self.banks[&self.highest_slot()]
289    }
290
291    /// Register to be notified when a bank has been dumped (due to duplicate block handling)
292    /// from bank_forks.
293    pub fn register_dumped_slot_subscriber(&mut self, notifier: DumpedSlotSubscription) {
294        self.dumped_slot_subscribers.push(notifier);
295    }
296
297    /// Clears associated banks from BankForks and notifies subscribers that a dump has occured.
298    pub fn dump_slots<'a, I>(&mut self, slots: I) -> (Vec<(Slot, BankId)>, Vec<BankWithScheduler>)
299    where
300        I: Iterator<Item = &'a Slot>,
301    {
302        // Notify subscribers. It is fine that the lock is immediately released, since the bank_forks
303        // lock is held until the end of this function, so subscribers will not be able to interact
304        // with bank_forks anyway.
305        for subscriber in &self.dumped_slot_subscribers {
306            let mut lock = subscriber.lock().unwrap();
307            *lock = true;
308        }
309
310        slots
311            .map(|slot| {
312                // Clear the banks from BankForks
313                let bank = self
314                    .remove(*slot)
315                    .expect("BankForks should not have been purged yet");
316                bank_hash_details::write_bank_hash_details_file(&bank)
317                    .map_err(|err| {
318                        warn!("Unable to write bank hash details file: {err}");
319                    })
320                    .ok();
321                ((*slot, bank.bank_id()), bank)
322            })
323            .unzip()
324    }
325
326    /// Sends an EpochAccountsHash request if one of the `banks` crosses the EAH boundary.
327    /// Returns if the bank at slot `root` was squashed, and its timings.
328    ///
329    /// Panics if more than one bank in `banks` should send an EAH request.
330    pub fn send_eah_request_if_needed(
331        &mut self,
332        root: Slot,
333        banks: &[&Arc<Bank>],
334        accounts_background_request_sender: &AbsRequestSender,
335    ) -> Result<(bool, SquashTiming), SetRootError> {
336        let mut is_root_bank_squashed = false;
337        let mut squash_timing = SquashTiming::default();
338
339        // Go through all the banks and see if we should send an EAH request.
340        // Only one EAH bank is allowed to send an EAH request.
341        // NOTE: Instead of filter-collect-assert, `.find()` could be used instead.
342        // Once sufficient testing guarantees only one bank will ever request an EAH,
343        // change to `.find()`.
344        let eah_banks: Vec<_> = banks
345            .iter()
346            .filter(|bank| self.should_request_epoch_accounts_hash(bank))
347            .collect();
348        assert!(
349            eah_banks.len() <= 1,
350            "At most one bank should request an epoch accounts hash calculation! num banks: {}, bank slots: {:?}",
351            eah_banks.len(),
352            eah_banks.iter().map(|bank| bank.slot()).collect::<Vec<_>>(),
353        );
354        if let Some(&&eah_bank) = eah_banks.first() {
355            debug!(
356                "sending epoch accounts hash request, slot: {}",
357                eah_bank.slot(),
358            );
359
360            self.last_accounts_hash_slot = eah_bank.slot();
361            squash_timing += eah_bank.squash();
362            is_root_bank_squashed = eah_bank.slot() == root;
363
364            eah_bank
365                .rc
366                .accounts
367                .accounts_db
368                .epoch_accounts_hash_manager
369                .set_in_flight(eah_bank.slot());
370            if let Err(e) =
371                accounts_background_request_sender.send_snapshot_request(SnapshotRequest {
372                    snapshot_root_bank: Arc::clone(eah_bank),
373                    status_cache_slot_deltas: Vec::default(),
374                    request_kind: SnapshotRequestKind::EpochAccountsHash,
375                    enqueued: Instant::now(),
376                })
377            {
378                return Err(SetRootError::SendEpochAccountHashError(eah_bank.slot(), e));
379            };
380        }
381
382        Ok((is_root_bank_squashed, squash_timing))
383    }
384
385    fn do_set_root_return_metrics(
386        &mut self,
387        root: Slot,
388        accounts_background_request_sender: &AbsRequestSender,
389        highest_super_majority_root: Option<Slot>,
390    ) -> Result<(Vec<BankWithScheduler>, SetRootMetrics), SetRootError> {
391        let old_epoch = self.root_bank().epoch();
392        // To support `RootBankCache` (via `ReadOnlyAtomicSlot`) accessing `root` *without* locking
393        // BankForks first *and* from a different thread, this store *must* be at least Release to
394        // ensure atomic ordering correctness.
395        self.root.store(root, Ordering::Release);
396
397        let root_bank = &self
398            .get(root)
399            .expect("root bank didn't exist in bank_forks");
400        let new_epoch = root_bank.epoch();
401        if old_epoch != new_epoch {
402            info!(
403                "Root entering
404                    epoch: {},
405                    next_epoch_start_slot: {},
406                    epoch_stakes: {:#?}",
407                new_epoch,
408                root_bank
409                    .epoch_schedule()
410                    .get_first_slot_in_epoch(new_epoch + 1),
411                root_bank
412                    .epoch_stakes(new_epoch)
413                    .unwrap()
414                    .node_id_to_vote_accounts()
415            );
416        }
417        let root_tx_count = root_bank
418            .parents()
419            .last()
420            .map(|bank| bank.transaction_count())
421            .unwrap_or(0);
422        // Calculate the accounts hash at a fixed interval
423        let mut banks = vec![root_bank];
424        let parents = root_bank.parents();
425        banks.extend(parents.iter());
426        let total_parent_banks = banks.len();
427        let mut total_snapshot_ms = 0;
428
429        let (mut is_root_bank_squashed, mut squash_timing) =
430            self.send_eah_request_if_needed(root, &banks, accounts_background_request_sender)?;
431
432        // After checking for EAH requests, also check for regular snapshot requests.
433        //
434        // This is needed when a snapshot request occurs in a slot after an EAH request, and is
435        // part of the same set of `banks` in a single `set_root()` invocation.  While (very)
436        // unlikely for a validator with default snapshot intervals (and accounts hash verifier
437        // intervals), it *is* possible, and there are tests to exercise this possibility.
438        if let Some(bank) = banks.iter().find(|bank| {
439            bank.slot() > self.last_accounts_hash_slot
440                && bank.block_height() % self.accounts_hash_interval_slots == 0
441        }) {
442            let bank_slot = bank.slot();
443            self.last_accounts_hash_slot = bank_slot;
444            squash_timing += bank.squash();
445
446            is_root_bank_squashed = bank_slot == root;
447
448            let mut snapshot_time = Measure::start("squash::snapshot_time");
449            if self.snapshot_config.is_some()
450                && accounts_background_request_sender.is_snapshot_creation_enabled()
451            {
452                if bank.is_startup_verification_complete() {
453                    // Save off the status cache because these may get pruned if another
454                    // `set_root()` is called before the snapshots package can be generated
455                    let status_cache_slot_deltas =
456                        bank.status_cache.read().unwrap().root_slot_deltas();
457                    if let Err(e) =
458                        accounts_background_request_sender.send_snapshot_request(SnapshotRequest {
459                            snapshot_root_bank: Arc::clone(bank),
460                            status_cache_slot_deltas,
461                            request_kind: SnapshotRequestKind::Snapshot,
462                            enqueued: Instant::now(),
463                        })
464                    {
465                        warn!(
466                            "Error sending snapshot request for bank: {}, err: {:?}",
467                            bank_slot, e
468                        );
469                    }
470                } else {
471                    info!("Not sending snapshot request for bank: {}, startup verification is incomplete", bank_slot);
472                }
473            }
474            snapshot_time.stop();
475            total_snapshot_ms += snapshot_time.as_ms() as i64;
476        }
477
478        if !is_root_bank_squashed {
479            squash_timing += root_bank.squash();
480        }
481        let new_tx_count = root_bank.transaction_count();
482        let accounts_data_len = root_bank.load_accounts_data_size() as i64;
483        let mut prune_time = Measure::start("set_root::prune");
484        let (removed_banks, prune_slots_ms, prune_remove_ms) =
485            self.prune_non_rooted(root, highest_super_majority_root);
486        prune_time.stop();
487        let dropped_banks_len = removed_banks.len();
488
489        let mut drop_parent_banks_time = Measure::start("set_root::drop_banks");
490        drop(parents);
491        drop_parent_banks_time.stop();
492
493        Ok((
494            removed_banks,
495            SetRootMetrics {
496                timings: SetRootTimings {
497                    total_squash_time: squash_timing,
498                    total_snapshot_ms,
499                    prune_non_rooted_ms: prune_time.as_ms() as i64,
500                    drop_parent_banks_ms: drop_parent_banks_time.as_ms() as i64,
501                    prune_slots_ms: prune_slots_ms as i64,
502                    prune_remove_ms: prune_remove_ms as i64,
503                },
504                total_parent_banks: total_parent_banks as i64,
505                tx_count: (new_tx_count - root_tx_count) as i64,
506                dropped_banks_len: dropped_banks_len as i64,
507                accounts_data_len,
508            },
509        ))
510    }
511
512    pub fn prune_program_cache(&self, root: Slot) {
513        if let Some(root_bank) = self.banks.get(&root) {
514            root_bank.prune_program_cache(root, root_bank.epoch());
515        }
516    }
517
518    pub fn set_root(
519        &mut self,
520        root: Slot,
521        accounts_background_request_sender: &AbsRequestSender,
522        highest_super_majority_root: Option<Slot>,
523    ) -> Result<Vec<BankWithScheduler>, SetRootError> {
524        let program_cache_prune_start = Instant::now();
525        let set_root_start = Instant::now();
526        let (removed_banks, set_root_metrics) = self.do_set_root_return_metrics(
527            root,
528            accounts_background_request_sender,
529            highest_super_majority_root,
530        )?;
531        datapoint_info!(
532            "bank-forks_set_root",
533            (
534                "elapsed_ms",
535                set_root_start.elapsed().as_millis() as usize,
536                i64
537            ),
538            ("slot", root, i64),
539            (
540                "total_parent_banks",
541                set_root_metrics.total_parent_banks,
542                i64
543            ),
544            ("total_banks", self.banks.len(), i64),
545            (
546                "total_squash_cache_ms",
547                set_root_metrics.timings.total_squash_time.squash_cache_ms,
548                i64
549            ),
550            (
551                "total_squash_accounts_ms",
552                set_root_metrics
553                    .timings
554                    .total_squash_time
555                    .squash_accounts_ms,
556                i64
557            ),
558            (
559                "total_squash_accounts_index_ms",
560                set_root_metrics
561                    .timings
562                    .total_squash_time
563                    .squash_accounts_index_ms,
564                i64
565            ),
566            (
567                "total_squash_accounts_cache_ms",
568                set_root_metrics
569                    .timings
570                    .total_squash_time
571                    .squash_accounts_cache_ms,
572                i64
573            ),
574            (
575                "total_squash_accounts_store_ms",
576                set_root_metrics
577                    .timings
578                    .total_squash_time
579                    .squash_accounts_store_ms,
580                i64
581            ),
582            (
583                "total_snapshot_ms",
584                set_root_metrics.timings.total_snapshot_ms,
585                i64
586            ),
587            ("tx_count", set_root_metrics.tx_count, i64),
588            (
589                "prune_non_rooted_ms",
590                set_root_metrics.timings.prune_non_rooted_ms,
591                i64
592            ),
593            (
594                "drop_parent_banks_ms",
595                set_root_metrics.timings.drop_parent_banks_ms,
596                i64
597            ),
598            (
599                "prune_slots_ms",
600                set_root_metrics.timings.prune_slots_ms,
601                i64
602            ),
603            (
604                "prune_remove_ms",
605                set_root_metrics.timings.prune_remove_ms,
606                i64
607            ),
608            (
609                "program_cache_prune_ms",
610                program_cache_prune_start.elapsed().as_millis() as i64,
611                i64
612            ),
613            ("dropped_banks_len", set_root_metrics.dropped_banks_len, i64),
614            ("accounts_data_len", set_root_metrics.accounts_data_len, i64),
615        );
616        Ok(removed_banks)
617    }
618
619    pub fn root(&self) -> Slot {
620        self.root.load(Ordering::Relaxed)
621    }
622
623    /// Gets a read-only wrapper to an atomic slot holding the root slot.
624    pub fn get_atomic_root(&self) -> ReadOnlyAtomicSlot {
625        ReadOnlyAtomicSlot {
626            slot: self.root.clone(),
627        }
628    }
629
630    /// After setting a new root, prune the banks that are no longer on rooted paths
631    ///
632    /// Given the following banks and slots...
633    ///
634    /// ```text
635    /// slot 6                   * (G)
636    ///                         /
637    /// slot 5        (F)  *   /
638    ///                    |  /
639    /// slot 4    (E) *    | /
640    ///               |    |/
641    /// slot 3        |    * (D) <-- root, from set_root()
642    ///               |    |
643    /// slot 2    (C) *    |
644    ///                \   |
645    /// slot 1          \  * (B)
646    ///                  \ |
647    /// slot 0             * (A)  <-- highest confirmed root [1]
648    /// ```
649    ///
650    /// ...where (D) is set as root, clean up (C) and (E), since they are not rooted.
651    ///
652    /// (A) is kept because it is greater-than-or-equal-to the highest confirmed root, and (D) is
653    ///     one of its descendants
654    /// (B) is kept for the same reason as (A)
655    /// (C) is pruned since it is a lower slot than (D), but (D) is _not_ one of its descendants
656    /// (D) is kept since it is the root
657    /// (E) is pruned since it is not a descendant of (D)
658    /// (F) is kept since it is a descendant of (D)
659    /// (G) is kept for the same reason as (F)
660    ///
661    /// and in table form...
662    ///
663    /// ```text
664    ///       |          |  is root a  | is a descendant ||
665    ///  slot | is root? | descendant? |    of root?     || keep?
666    /// ------+----------+-------------+-----------------++-------
667    ///   (A) |     N    |      Y      |        N        ||   Y
668    ///   (B) |     N    |      Y      |        N        ||   Y
669    ///   (C) |     N    |      N      |        N        ||   N
670    ///   (D) |     Y    |      N      |        N        ||   Y
671    ///   (E) |     N    |      N      |        N        ||   N
672    ///   (F) |     N    |      N      |        Y        ||   Y
673    ///   (G) |     N    |      N      |        Y        ||   Y
674    /// ```
675    ///
676    /// [1] RPC has the concept of commitment level, which is based on the highest confirmed root,
677    /// i.e. the cluster-confirmed root.  This commitment is stronger than the local node's root.
678    /// So (A) and (B) are kept to facilitate RPC at different commitment levels.  Everything below
679    /// the highest confirmed root can be pruned.
680    fn prune_non_rooted(
681        &mut self,
682        root: Slot,
683        highest_super_majority_root: Option<Slot>,
684    ) -> (Vec<BankWithScheduler>, u64, u64) {
685        // We want to collect timing separately, and the 2nd collect requires
686        // a unique borrow to self which is already borrowed by self.banks
687        let mut prune_slots_time = Measure::start("prune_slots");
688        let highest_super_majority_root = highest_super_majority_root.unwrap_or(root);
689        let prune_slots: Vec<_> = self
690            .banks
691            .keys()
692            .copied()
693            .filter(|slot| {
694                let keep = *slot == root
695                    || self.descendants[&root].contains(slot)
696                    || (*slot < root
697                        && *slot >= highest_super_majority_root
698                        && self.descendants[slot].contains(&root));
699                !keep
700            })
701            .collect();
702        prune_slots_time.stop();
703
704        let mut prune_remove_time = Measure::start("prune_slots");
705        let removed_banks = prune_slots
706            .into_iter()
707            .filter_map(|slot| self.remove(slot))
708            .collect();
709        prune_remove_time.stop();
710
711        (
712            removed_banks,
713            prune_slots_time.as_ms(),
714            prune_remove_time.as_ms(),
715        )
716    }
717
718    pub fn set_snapshot_config(&mut self, snapshot_config: Option<SnapshotConfig>) {
719        self.snapshot_config = snapshot_config;
720    }
721
722    pub fn set_accounts_hash_interval_slots(&mut self, accounts_interval_slots: u64) {
723        self.accounts_hash_interval_slots = accounts_interval_slots;
724    }
725
726    /// Determine if this bank should request an epoch accounts hash
727    #[must_use]
728    fn should_request_epoch_accounts_hash(&self, bank: &Bank) -> bool {
729        if !epoch_accounts_hash_utils::is_enabled_this_epoch(bank) {
730            return false;
731        }
732
733        let start_slot = epoch_accounts_hash_utils::calculation_start(bank);
734        bank.slot() > self.last_accounts_hash_slot
735            && bank.parent_slot() < start_slot
736            && bank.slot() >= start_slot
737    }
738}
739
740impl ForkGraph for BankForks {
741    fn relationship(&self, a: Slot, b: Slot) -> BlockRelation {
742        let known_slot_range = self.root()..=self.highest_slot();
743        (known_slot_range.contains(&a) && known_slot_range.contains(&b))
744            .then(|| {
745                (a == b)
746                    .then_some(BlockRelation::Equal)
747                    .or_else(|| {
748                        self.banks.get(&b).and_then(|bank| {
749                            bank.ancestors
750                                .contains_key(&a)
751                                .then_some(BlockRelation::Ancestor)
752                        })
753                    })
754                    .or_else(|| {
755                        self.descendants.get(&b).and_then(|slots| {
756                            slots.contains(&a).then_some(BlockRelation::Descendant)
757                        })
758                    })
759                    .unwrap_or(BlockRelation::Unrelated)
760            })
761            .unwrap_or(BlockRelation::Unknown)
762    }
763}
764
765#[cfg(test)]
766mod tests {
767    use {
768        super::*,
769        crate::{
770            bank::test_utils::update_vote_account_timestamp,
771            genesis_utils::{
772                create_genesis_config, create_genesis_config_with_leader, GenesisConfigInfo,
773            },
774        },
775        assert_matches::assert_matches,
776        solana_accounts_db::epoch_accounts_hash::EpochAccountsHash,
777        solana_sdk::{
778            clock::UnixTimestamp,
779            epoch_schedule::EpochSchedule,
780            hash::Hash,
781            pubkey::Pubkey,
782            signature::{Keypair, Signer},
783        },
784        solana_vote_program::vote_state::BlockTimestamp,
785        std::{sync::atomic::Ordering::Relaxed, time::Duration},
786    };
787
788    #[test]
789    fn test_bank_forks_new_rw_arc_memory_leak() {
790        for _ in 0..1000 {
791            let GenesisConfigInfo { genesis_config, .. } = create_genesis_config(10_000);
792            BankForks::new_rw_arc(Bank::new_for_tests(&genesis_config));
793        }
794    }
795
796    #[test]
797    fn test_bank_forks_new() {
798        let GenesisConfigInfo { genesis_config, .. } = create_genesis_config(10_000);
799        let bank = Bank::new_for_tests(&genesis_config);
800        let bank_forks = BankForks::new_rw_arc(bank);
801        let mut bank_forks = bank_forks.write().unwrap();
802        let child_bank = Bank::new_from_parent(bank_forks[0].clone(), &Pubkey::default(), 1);
803        child_bank.register_default_tick_for_test();
804        bank_forks.insert(child_bank);
805        assert_eq!(bank_forks[1u64].tick_height(), 1);
806        assert_eq!(bank_forks.working_bank().tick_height(), 1);
807    }
808
809    #[test]
810    fn test_bank_forks_descendants() {
811        let GenesisConfigInfo { genesis_config, .. } = create_genesis_config(10_000);
812        let bank = Bank::new_for_tests(&genesis_config);
813        let bank_forks = BankForks::new_rw_arc(bank);
814        let mut bank_forks = bank_forks.write().unwrap();
815        let bank0 = bank_forks[0].clone();
816        let bank = Bank::new_from_parent(bank0.clone(), &Pubkey::default(), 1);
817        bank_forks.insert(bank);
818        let bank = Bank::new_from_parent(bank0, &Pubkey::default(), 2);
819        bank_forks.insert(bank);
820        let descendants = bank_forks.descendants();
821        let children: HashSet<u64> = [1u64, 2u64].iter().copied().collect();
822        assert_eq!(children, *descendants.get(&0).unwrap());
823        assert!(descendants[&1].is_empty());
824        assert!(descendants[&2].is_empty());
825    }
826
827    #[test]
828    fn test_bank_forks_ancestors() {
829        let GenesisConfigInfo { genesis_config, .. } = create_genesis_config(10_000);
830        let bank = Bank::new_for_tests(&genesis_config);
831        let bank_forks = BankForks::new_rw_arc(bank);
832        let mut bank_forks = bank_forks.write().unwrap();
833        let bank0 = bank_forks[0].clone();
834        let bank = Bank::new_from_parent(bank0.clone(), &Pubkey::default(), 1);
835        bank_forks.insert(bank);
836        let bank = Bank::new_from_parent(bank0, &Pubkey::default(), 2);
837        bank_forks.insert(bank);
838        let ancestors = bank_forks.ancestors();
839        assert!(ancestors[&0].is_empty());
840        let parents: Vec<u64> = ancestors[&1].iter().cloned().collect();
841        assert_eq!(parents, vec![0]);
842        let parents: Vec<u64> = ancestors[&2].iter().cloned().collect();
843        assert_eq!(parents, vec![0]);
844    }
845
846    #[test]
847    fn test_bank_forks_frozen_banks() {
848        let GenesisConfigInfo { genesis_config, .. } = create_genesis_config(10_000);
849        let bank = Bank::new_for_tests(&genesis_config);
850        let bank_forks = BankForks::new_rw_arc(bank);
851        let mut bank_forks = bank_forks.write().unwrap();
852        let bank0 = bank_forks[0].clone();
853        let child_bank = Bank::new_from_parent(bank0, &Pubkey::default(), 1);
854        bank_forks.insert(child_bank);
855        assert!(bank_forks.frozen_banks().contains_key(&0));
856        assert!(!bank_forks.frozen_banks().contains_key(&1));
857    }
858
859    #[test]
860    fn test_bank_forks_active_banks() {
861        let GenesisConfigInfo { genesis_config, .. } = create_genesis_config(10_000);
862        let bank = Bank::new_for_tests(&genesis_config);
863        let bank_forks = BankForks::new_rw_arc(bank);
864        let mut bank_forks = bank_forks.write().unwrap();
865        let bank0 = bank_forks[0].clone();
866        let child_bank = Bank::new_from_parent(bank0, &Pubkey::default(), 1);
867        bank_forks.insert(child_bank);
868        assert_eq!(bank_forks.active_bank_slots(), vec![1]);
869    }
870
871    #[test]
872    fn test_bank_forks_different_set_root() {
873        solana_logger::setup();
874        let leader_keypair = Keypair::new();
875        let GenesisConfigInfo {
876            mut genesis_config,
877            voting_keypair,
878            ..
879        } = create_genesis_config_with_leader(10_000, &leader_keypair.pubkey(), 1_000);
880        let slots_in_epoch = 32;
881        genesis_config.epoch_schedule = EpochSchedule::new(slots_in_epoch);
882
883        // Spin up a thread to be a fake Accounts Background Service.  Need to intercept and handle
884        // all EpochAccountsHash requests so future rooted banks do not hang in Bank::freeze()
885        // waiting for an in-flight EAH calculation to complete.
886        let (snapshot_request_sender, snapshot_request_receiver) = crossbeam_channel::unbounded();
887        let abs_request_sender = AbsRequestSender::new(snapshot_request_sender);
888        let bg_exit = Arc::new(AtomicBool::new(false));
889        let bg_thread = {
890            let exit = Arc::clone(&bg_exit);
891            std::thread::spawn(move || {
892                while !exit.load(Relaxed) {
893                    snapshot_request_receiver
894                        .try_iter()
895                        .filter(|snapshot_request| {
896                            snapshot_request.request_kind == SnapshotRequestKind::EpochAccountsHash
897                        })
898                        .for_each(|snapshot_request| {
899                            snapshot_request
900                                .snapshot_root_bank
901                                .rc
902                                .accounts
903                                .accounts_db
904                                .epoch_accounts_hash_manager
905                                .set_valid(
906                                    EpochAccountsHash::new(Hash::new_unique()),
907                                    snapshot_request.snapshot_root_bank.slot(),
908                                )
909                        });
910                    std::thread::sleep(Duration::from_millis(100));
911                }
912            })
913        };
914
915        let bank0 = Bank::new_for_tests(&genesis_config);
916        let bank_forks0 = BankForks::new_rw_arc(bank0);
917        let mut bank_forks0 = bank_forks0.write().unwrap();
918        bank_forks0.set_root(0, &abs_request_sender, None).unwrap();
919
920        let bank1 = Bank::new_for_tests(&genesis_config);
921        let bank_forks1 = BankForks::new_rw_arc(bank1);
922        let mut bank_forks1 = bank_forks1.write().unwrap();
923
924        let additional_timestamp_secs = 2;
925
926        let num_slots = slots_in_epoch + 1; // Advance past first epoch boundary
927        for slot in 1..num_slots {
928            // Just after the epoch boundary, timestamp a vote that will shift
929            // Clock::unix_timestamp from Bank::unix_timestamp_from_genesis()
930            let update_timestamp_case = slot == slots_in_epoch;
931
932            let child1 =
933                Bank::new_from_parent(bank_forks0[slot - 1].clone(), &Pubkey::default(), slot);
934            let child2 =
935                Bank::new_from_parent(bank_forks1[slot - 1].clone(), &Pubkey::default(), slot);
936
937            if update_timestamp_case {
938                for child in &[&child1, &child2] {
939                    let recent_timestamp: UnixTimestamp = child.unix_timestamp_from_genesis();
940                    update_vote_account_timestamp(
941                        BlockTimestamp {
942                            slot: child.slot(),
943                            timestamp: recent_timestamp + additional_timestamp_secs,
944                        },
945                        child,
946                        &voting_keypair.pubkey(),
947                    );
948                }
949            }
950
951            // Set root in bank_forks0 to truncate the ancestor history
952            bank_forks0.insert(child1);
953            bank_forks0
954                .set_root(slot, &abs_request_sender, None)
955                .unwrap();
956
957            // Don't set root in bank_forks1 to keep the ancestor history
958            bank_forks1.insert(child2);
959        }
960        let child1 = &bank_forks0.working_bank();
961        let child2 = &bank_forks1.working_bank();
962
963        child1.freeze();
964        child2.freeze();
965
966        info!("child0.ancestors: {:?}", child1.ancestors);
967        info!("child1.ancestors: {:?}", child2.ancestors);
968        assert_eq!(child1.hash(), child2.hash());
969
970        bg_exit.store(true, Relaxed);
971        bg_thread.join().unwrap();
972    }
973
974    fn make_hash_map(data: Vec<(Slot, Vec<Slot>)>) -> HashMap<Slot, HashSet<Slot>> {
975        data.into_iter()
976            .map(|(k, v)| (k, v.into_iter().collect()))
977            .collect()
978    }
979
980    fn extend_bank_forks(bank_forks: Arc<RwLock<BankForks>>, parent_child_pairs: &[(Slot, Slot)]) {
981        for (parent, child) in parent_child_pairs.iter() {
982            let parent: Arc<Bank> = bank_forks.read().unwrap().banks[parent].clone();
983            bank_forks.write().unwrap().insert(Bank::new_from_parent(
984                parent,
985                &Pubkey::default(),
986                *child,
987            ));
988        }
989    }
990
991    #[test]
992    fn test_bank_forks_with_set_root() {
993        let GenesisConfigInfo { genesis_config, .. } = create_genesis_config(10_000);
994        let bank = Bank::new_for_tests(&genesis_config);
995        let bank_forks = BankForks::new_rw_arc(bank);
996
997        let parent_child_pairs = vec![(0, 1), (1, 2), (0, 3), (3, 4)];
998        extend_bank_forks(bank_forks.clone(), &parent_child_pairs);
999
1000        assert_eq!(
1001            bank_forks.read().unwrap().ancestors(),
1002            make_hash_map(vec![
1003                (0, vec![]),
1004                (1, vec![0]),
1005                (2, vec![0, 1]),
1006                (3, vec![0]),
1007                (4, vec![0, 3]),
1008            ])
1009        );
1010        assert_eq!(
1011            bank_forks.read().unwrap().descendants(),
1012            make_hash_map(vec![
1013                (0, vec![1, 2, 3, 4]),
1014                (1, vec![2]),
1015                (2, vec![]),
1016                (3, vec![4]),
1017                (4, vec![]),
1018            ])
1019        );
1020        bank_forks
1021            .write()
1022            .unwrap()
1023            .set_root(
1024                2,
1025                &AbsRequestSender::default(),
1026                None, // highest confirmed root
1027            )
1028            .unwrap();
1029        bank_forks.read().unwrap().get(2).unwrap().squash();
1030        assert_eq!(
1031            bank_forks.read().unwrap().ancestors(),
1032            make_hash_map(vec![(2, vec![]),])
1033        );
1034        assert_eq!(
1035            bank_forks.read().unwrap().descendants(),
1036            make_hash_map(vec![(0, vec![2]), (1, vec![2]), (2, vec![]),])
1037        );
1038
1039        let parent_child_pairs = vec![(2, 5), (5, 6)];
1040        extend_bank_forks(bank_forks.clone(), &parent_child_pairs);
1041        assert_eq!(
1042            bank_forks.read().unwrap().ancestors(),
1043            make_hash_map(vec![(2, vec![]), (5, vec![2]), (6, vec![2, 5])])
1044        );
1045        assert_eq!(
1046            bank_forks.read().unwrap().descendants(),
1047            make_hash_map(vec![
1048                (0, vec![2]),
1049                (1, vec![2]),
1050                (2, vec![5, 6]),
1051                (5, vec![6]),
1052                (6, vec![])
1053            ])
1054        );
1055    }
1056
1057    #[test]
1058    fn test_bank_forks_with_highest_super_majority_root() {
1059        let GenesisConfigInfo { genesis_config, .. } = create_genesis_config(10_000);
1060        let bank = Bank::new_for_tests(&genesis_config);
1061        assert_eq!(bank.slot(), 0);
1062        let bank_forks = BankForks::new_rw_arc(bank);
1063
1064        let parent_child_pairs = vec![(0, 1), (1, 2), (0, 3), (3, 4)];
1065        extend_bank_forks(bank_forks.clone(), &parent_child_pairs);
1066
1067        assert_eq!(
1068            bank_forks.read().unwrap().ancestors(),
1069            make_hash_map(vec![
1070                (0, vec![]),
1071                (1, vec![0]),
1072                (2, vec![0, 1]),
1073                (3, vec![0]),
1074                (4, vec![0, 3]),
1075            ])
1076        );
1077        assert_eq!(
1078            bank_forks.read().unwrap().descendants(),
1079            make_hash_map(vec![
1080                (0, vec![1, 2, 3, 4]),
1081                (1, vec![2]),
1082                (2, vec![]),
1083                (3, vec![4]),
1084                (4, vec![]),
1085            ])
1086        );
1087        bank_forks
1088            .write()
1089            .unwrap()
1090            .set_root(
1091                2,
1092                &AbsRequestSender::default(),
1093                Some(1), // highest confirmed root
1094            )
1095            .unwrap();
1096        bank_forks.read().unwrap().get(2).unwrap().squash();
1097        assert_eq!(
1098            bank_forks.read().unwrap().ancestors(),
1099            make_hash_map(vec![(1, vec![]), (2, vec![]),])
1100        );
1101        assert_eq!(
1102            bank_forks.read().unwrap().descendants(),
1103            make_hash_map(vec![(0, vec![1, 2]), (1, vec![2]), (2, vec![]),])
1104        );
1105
1106        let parent_child_pairs = vec![(2, 5), (5, 6)];
1107        extend_bank_forks(bank_forks.clone(), &parent_child_pairs);
1108        assert_eq!(
1109            bank_forks.read().unwrap().ancestors(),
1110            make_hash_map(vec![
1111                (1, vec![]),
1112                (2, vec![]),
1113                (5, vec![2]),
1114                (6, vec![2, 5])
1115            ])
1116        );
1117        assert_eq!(
1118            bank_forks.read().unwrap().descendants(),
1119            make_hash_map(vec![
1120                (0, vec![1, 2]),
1121                (1, vec![2]),
1122                (2, vec![5, 6]),
1123                (5, vec![6]),
1124                (6, vec![])
1125            ])
1126        );
1127    }
1128
1129    #[test]
1130    fn test_fork_graph() {
1131        let GenesisConfigInfo { genesis_config, .. } = create_genesis_config(10_000);
1132        let bank = Bank::new_for_tests(&genesis_config);
1133        let bank_forks = BankForks::new_rw_arc(bank);
1134
1135        let parent_child_pairs = vec![
1136            (0, 1),
1137            (1, 3),
1138            (3, 8),
1139            (0, 2),
1140            (2, 4),
1141            (4, 5),
1142            (5, 10),
1143            (4, 6),
1144            (6, 12),
1145        ];
1146        extend_bank_forks(bank_forks.clone(), &parent_child_pairs);
1147
1148        // Fork graph created for the test
1149        //                   0
1150        //                 /   \
1151        //                1     2
1152        //                |     |
1153        //                3     4
1154        //                |     | \
1155        //                8     5  6
1156        //                      |   |
1157        //                      10  12
1158        let mut bank_forks = bank_forks.write().unwrap();
1159        assert_matches!(bank_forks.relationship(0, 3), BlockRelation::Ancestor);
1160        assert_matches!(bank_forks.relationship(0, 10), BlockRelation::Ancestor);
1161        assert_matches!(bank_forks.relationship(0, 12), BlockRelation::Ancestor);
1162        assert_matches!(bank_forks.relationship(1, 3), BlockRelation::Ancestor);
1163        assert_matches!(bank_forks.relationship(2, 10), BlockRelation::Ancestor);
1164        assert_matches!(bank_forks.relationship(2, 12), BlockRelation::Ancestor);
1165        assert_matches!(bank_forks.relationship(4, 10), BlockRelation::Ancestor);
1166        assert_matches!(bank_forks.relationship(4, 12), BlockRelation::Ancestor);
1167        assert_matches!(bank_forks.relationship(6, 10), BlockRelation::Unrelated);
1168        assert_matches!(bank_forks.relationship(5, 12), BlockRelation::Unrelated);
1169        assert_matches!(bank_forks.relationship(6, 12), BlockRelation::Ancestor);
1170
1171        assert_matches!(bank_forks.relationship(6, 2), BlockRelation::Descendant);
1172        assert_matches!(bank_forks.relationship(10, 2), BlockRelation::Descendant);
1173        assert_matches!(bank_forks.relationship(8, 3), BlockRelation::Descendant);
1174        assert_matches!(bank_forks.relationship(6, 3), BlockRelation::Unrelated);
1175        assert_matches!(bank_forks.relationship(12, 2), BlockRelation::Descendant);
1176        assert_matches!(bank_forks.relationship(12, 1), BlockRelation::Unrelated);
1177        assert_matches!(bank_forks.relationship(1, 2), BlockRelation::Unrelated);
1178
1179        assert_matches!(bank_forks.relationship(1, 13), BlockRelation::Unknown);
1180        assert_matches!(bank_forks.relationship(13, 2), BlockRelation::Unknown);
1181        bank_forks
1182            .set_root(
1183                2,
1184                &AbsRequestSender::default(),
1185                Some(1), // highest confirmed root
1186            )
1187            .unwrap();
1188        assert_matches!(bank_forks.relationship(1, 2), BlockRelation::Unknown);
1189        assert_matches!(bank_forks.relationship(2, 0), BlockRelation::Unknown);
1190    }
1191}