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