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