solana_core/
validator.rs

1//! The `validator` module hosts all the validator microservices.
2
3pub use solana_perf::report_target_features;
4use {
5    crate::{
6        admin_rpc_post_init::{AdminRpcRequestMetadataPostInit, KeyUpdaterType, KeyUpdaters},
7        banking_stage::{
8            transaction_scheduler::scheduler_controller::SchedulerConfig, BankingStage,
9        },
10        banking_trace::{self, BankingTracer, TraceError},
11        cluster_info_vote_listener::VoteTracker,
12        completed_data_sets_service::CompletedDataSetsService,
13        consensus::{
14            reconcile_blockstore_roots_with_external_source,
15            tower_storage::{NullTowerStorage, TowerStorage},
16            ExternalRootSource, Tower,
17        },
18        repair::{
19            self,
20            quic_endpoint::{RepairQuicAsyncSenders, RepairQuicSenders, RepairQuicSockets},
21            repair_handler::RepairHandlerType,
22            serve_repair_service::ServeRepairService,
23        },
24        resource_limits::{
25            adjust_nofile_limit, validate_memlock_limit_for_disk_io, ResourceLimitError,
26        },
27        sample_performance_service::SamplePerformanceService,
28        sigverify,
29        snapshot_packager_service::SnapshotPackagerService,
30        stats_reporter_service::StatsReporterService,
31        system_monitor_service::{
32            verify_net_stats_access, SystemMonitorService, SystemMonitorStatsReportConfig,
33        },
34        tpu::{ForwardingClientOption, Tpu, TpuSockets},
35        tvu::{Tvu, TvuConfig, TvuSockets},
36    },
37    agave_snapshots::{
38        snapshot_archive_info::SnapshotArchiveInfoGetter as _, snapshot_config::SnapshotConfig,
39        snapshot_hash::StartingSnapshotHashes, SnapshotInterval,
40    },
41    anyhow::{anyhow, Context, Result},
42    crossbeam_channel::{bounded, unbounded, Receiver},
43    quinn::Endpoint,
44    serde::{Deserialize, Serialize},
45    solana_account::ReadableAccount,
46    solana_accounts_db::{
47        accounts_db::{AccountsDbConfig, ACCOUNTS_DB_CONFIG_FOR_TESTING},
48        accounts_update_notifier_interface::AccountsUpdateNotifier,
49        utils::move_and_async_delete_path_contents,
50    },
51    solana_client::{
52        client_option::ClientOption,
53        connection_cache::{ConnectionCache, Protocol},
54    },
55    solana_clock::Slot,
56    solana_cluster_type::ClusterType,
57    solana_entry::poh::compute_hash_time,
58    solana_epoch_schedule::MAX_LEADER_SCHEDULE_EPOCH_OFFSET,
59    solana_genesis_config::GenesisConfig,
60    solana_genesis_utils::{
61        open_genesis_config, OpenGenesisConfigError, MAX_GENESIS_ARCHIVE_UNPACKED_SIZE,
62    },
63    solana_geyser_plugin_manager::{
64        geyser_plugin_service::GeyserPluginService, GeyserPluginManagerRequest,
65    },
66    solana_gossip::{
67        cluster_info::{
68            ClusterInfo, DEFAULT_CONTACT_DEBUG_INTERVAL_MILLIS,
69            DEFAULT_CONTACT_SAVE_INTERVAL_MILLIS,
70        },
71        contact_info::ContactInfo,
72        crds_gossip_pull::CRDS_GOSSIP_PULL_CRDS_TIMEOUT_MS,
73        gossip_service::GossipService,
74        node::{Node, NodeMultihoming},
75    },
76    solana_hard_forks::HardForks,
77    solana_hash::Hash,
78    solana_keypair::Keypair,
79    solana_ledger::{
80        bank_forks_utils,
81        blockstore::{
82            Blockstore, BlockstoreError, PurgeType, MAX_COMPLETED_SLOTS_IN_CHANNEL,
83            MAX_REPLAY_WAKE_UP_SIGNALS,
84        },
85        blockstore_metric_report_service::BlockstoreMetricReportService,
86        blockstore_options::{BlockstoreOptions, BLOCKSTORE_DIRECTORY_ROCKS_LEVEL},
87        blockstore_processor::{self, TransactionStatusSender},
88        entry_notifier_interface::EntryNotifierArc,
89        entry_notifier_service::{EntryNotifierSender, EntryNotifierService},
90        leader_schedule::FixedSchedule,
91        leader_schedule_cache::LeaderScheduleCache,
92        use_snapshot_archives_at_startup::UseSnapshotArchivesAtStartup,
93    },
94    solana_measure::measure::Measure,
95    solana_metrics::{datapoint_info, metrics::metrics_config_sanity_check},
96    solana_poh::{
97        poh_controller::PohController,
98        poh_recorder::PohRecorder,
99        poh_service::{self, PohService},
100        record_channels::record_channels,
101        transaction_recorder::TransactionRecorder,
102    },
103    solana_pubkey::Pubkey,
104    solana_rayon_threadlimit::get_thread_count,
105    solana_rpc::{
106        max_slots::MaxSlots,
107        optimistically_confirmed_bank_tracker::{
108            BankNotificationSenderConfig, OptimisticallyConfirmedBank,
109            OptimisticallyConfirmedBankTracker,
110        },
111        rpc::JsonRpcConfig,
112        rpc_completed_slots_service::RpcCompletedSlotsService,
113        rpc_pubsub_service::{PubSubConfig, PubSubService},
114        rpc_service::{JsonRpcService, JsonRpcServiceConfig},
115        rpc_subscriptions::RpcSubscriptions,
116        transaction_notifier_interface::TransactionNotifierArc,
117        transaction_status_service::TransactionStatusService,
118    },
119    solana_runtime::{
120        accounts_background_service::{
121            AbsRequestHandlers, AccountsBackgroundService, DroppedSlotsReceiver,
122            PendingSnapshotPackages, PrunedBanksRequestHandler, SnapshotRequestHandler,
123        },
124        bank::Bank,
125        bank_forks::BankForks,
126        commitment::BlockCommitmentCache,
127        dependency_tracker::DependencyTracker,
128        prioritization_fee_cache::PrioritizationFeeCache,
129        runtime_config::RuntimeConfig,
130        snapshot_bank_utils,
131        snapshot_controller::SnapshotController,
132        snapshot_utils::{self, clean_orphaned_account_snapshot_dirs},
133    },
134    solana_send_transaction_service::send_transaction_service::Config as SendTransactionServiceConfig,
135    solana_shred_version::compute_shred_version,
136    solana_signer::Signer,
137    solana_streamer::{
138        nonblocking::{simple_qos::SimpleQosConfig, swqos::SwQosConfig},
139        quic::{QuicStreamerConfig, SimpleQosQuicStreamerConfig, SwQosQuicStreamerConfig},
140        socket::SocketAddrSpace,
141        streamer::StakedNodes,
142    },
143    solana_time_utils::timestamp,
144    solana_tpu_client::tpu_client::{
145        DEFAULT_TPU_CONNECTION_POOL_SIZE, DEFAULT_TPU_USE_QUIC, DEFAULT_VOTE_USE_QUIC,
146    },
147    solana_turbine::{
148        self,
149        broadcast_stage::BroadcastStageType,
150        xdp::{XdpConfig, XdpRetransmitter},
151    },
152    solana_unified_scheduler_pool::DefaultSchedulerPool,
153    solana_validator_exit::Exit,
154    solana_vote_program::vote_state::VoteStateV4,
155    solana_wen_restart::wen_restart::{wait_for_wen_restart, WenRestartConfig},
156    std::{
157        borrow::Cow,
158        collections::{HashMap, HashSet},
159        net::SocketAddr,
160        num::{NonZeroU64, NonZeroUsize},
161        path::{Path, PathBuf},
162        str::FromStr,
163        sync::{
164            atomic::{AtomicBool, AtomicU64, Ordering},
165            Arc, Mutex, RwLock,
166        },
167        thread::{sleep, Builder, JoinHandle},
168        time::{Duration, Instant},
169    },
170    strum::VariantNames,
171    strum_macros::{Display, EnumCount, EnumIter, EnumString, EnumVariantNames, IntoStaticStr},
172    thiserror::Error,
173    tokio::runtime::Runtime as TokioRuntime,
174    tokio_util::sync::CancellationToken,
175};
176
177const MAX_COMPLETED_DATA_SETS_IN_CHANNEL: usize = 100_000;
178const WAIT_FOR_SUPERMAJORITY_THRESHOLD_PERCENT: u64 = 80;
179// Right now since we reuse the wait for supermajority code, the
180// following threshold should always greater than or equal to
181// WAIT_FOR_SUPERMAJORITY_THRESHOLD_PERCENT.
182const WAIT_FOR_WEN_RESTART_SUPERMAJORITY_THRESHOLD_PERCENT: u64 =
183    WAIT_FOR_SUPERMAJORITY_THRESHOLD_PERCENT;
184
185#[derive(
186    Clone, EnumCount, EnumIter, EnumString, EnumVariantNames, Default, IntoStaticStr, Display,
187)]
188#[strum(serialize_all = "kebab-case")]
189pub enum BlockVerificationMethod {
190    BlockstoreProcessor,
191    #[default]
192    UnifiedScheduler,
193}
194
195impl BlockVerificationMethod {
196    pub const fn cli_names() -> &'static [&'static str] {
197        Self::VARIANTS
198    }
199
200    pub fn cli_message() -> &'static str {
201        "Switch transaction scheduling method for verifying ledger entries"
202    }
203}
204
205#[derive(
206    Clone,
207    Debug,
208    EnumString,
209    EnumVariantNames,
210    Default,
211    IntoStaticStr,
212    Display,
213    Serialize,
214    Deserialize,
215    PartialEq,
216    Eq,
217)]
218#[strum(serialize_all = "kebab-case")]
219#[serde(rename_all = "kebab-case")]
220pub enum BlockProductionMethod {
221    CentralScheduler,
222    #[default]
223    CentralSchedulerGreedy,
224}
225
226impl BlockProductionMethod {
227    pub const fn cli_names() -> &'static [&'static str] {
228        Self::VARIANTS
229    }
230
231    pub fn cli_message() -> &'static str {
232        "Switch transaction scheduling method for producing ledger entries"
233    }
234}
235
236#[derive(
237    Clone,
238    Debug,
239    EnumString,
240    EnumVariantNames,
241    Default,
242    IntoStaticStr,
243    Display,
244    Serialize,
245    Deserialize,
246    PartialEq,
247    Eq,
248)]
249#[strum(serialize_all = "kebab-case")]
250#[serde(rename_all = "kebab-case")]
251pub enum TransactionStructure {
252    Sdk,
253    #[default]
254    View,
255}
256
257impl TransactionStructure {
258    pub const fn cli_names() -> &'static [&'static str] {
259        Self::VARIANTS
260    }
261
262    pub fn cli_message() -> &'static str {
263        "DEPRECATED: has no impact on banking stage; will be removed in a future version"
264    }
265}
266
267#[derive(
268    Clone, Debug, EnumVariantNames, IntoStaticStr, Display, Serialize, Deserialize, PartialEq, Eq,
269)]
270#[strum(serialize_all = "kebab-case")]
271#[serde(rename_all = "kebab-case")]
272pub enum SchedulerPacing {
273    Disabled,
274    FillTimeMillis(NonZeroU64),
275}
276
277impl FromStr for SchedulerPacing {
278    type Err = String;
279
280    fn from_str(s: &str) -> std::result::Result<Self, Self::Err> {
281        if s.eq_ignore_ascii_case("disabled") {
282            Ok(SchedulerPacing::Disabled)
283        } else {
284            match s.parse::<u64>() {
285                Ok(v) if v > 0 => Ok(SchedulerPacing::FillTimeMillis(
286                    NonZeroU64::new(v).ok_or_else(|| "value must be non-zero".to_string())?,
287                )),
288                _ => Err("value must be a positive integer or 'disabled'".to_string()),
289            }
290        }
291    }
292}
293
294impl SchedulerPacing {
295    pub fn fill_time(&self) -> Option<Duration> {
296        match self {
297            SchedulerPacing::Disabled => None,
298            SchedulerPacing::FillTimeMillis(millis) => Some(Duration::from_millis(millis.get())),
299        }
300    }
301}
302
303/// Configuration for the block generator invalidator for replay.
304#[derive(Clone, Debug)]
305pub struct GeneratorConfig {
306    pub accounts_path: String,
307    pub starting_keypairs: Arc<Vec<Keypair>>,
308}
309
310pub struct ValidatorConfig {
311    pub halt_at_slot: Option<Slot>,
312    pub expected_genesis_hash: Option<Hash>,
313    pub expected_bank_hash: Option<Hash>,
314    pub expected_shred_version: Option<u16>,
315    pub voting_disabled: bool,
316    pub account_paths: Vec<PathBuf>,
317    pub account_snapshot_paths: Vec<PathBuf>,
318    pub rpc_config: JsonRpcConfig,
319    /// Specifies which plugins to start up with
320    pub on_start_geyser_plugin_config_files: Option<Vec<PathBuf>>,
321    pub geyser_plugin_always_enabled: bool,
322    pub rpc_addrs: Option<(SocketAddr, SocketAddr)>, // (JsonRpc, JsonRpcPubSub)
323    pub pubsub_config: PubSubConfig,
324    pub snapshot_config: SnapshotConfig,
325    pub max_ledger_shreds: Option<u64>,
326    pub blockstore_options: BlockstoreOptions,
327    pub broadcast_stage_type: BroadcastStageType,
328    pub turbine_disabled: Arc<AtomicBool>,
329    pub fixed_leader_schedule: Option<FixedSchedule>,
330    pub wait_for_supermajority: Option<Slot>,
331    pub new_hard_forks: Option<Vec<Slot>>,
332    pub known_validators: Option<HashSet<Pubkey>>, // None = trust all
333    pub repair_validators: Option<HashSet<Pubkey>>, // None = repair from all
334    pub repair_whitelist: Arc<RwLock<HashSet<Pubkey>>>, // Empty = repair with all
335    pub gossip_validators: Option<HashSet<Pubkey>>, // None = gossip with all
336    pub max_genesis_archive_unpacked_size: u64,
337    /// Run PoH, transaction signature and other transaction verifications during blockstore
338    /// processing.
339    pub run_verification: bool,
340    pub require_tower: bool,
341    pub tower_storage: Arc<dyn TowerStorage>,
342    pub debug_keys: Option<Arc<HashSet<Pubkey>>>,
343    pub contact_debug_interval: u64,
344    pub contact_save_interval: u64,
345    pub send_transaction_service_config: SendTransactionServiceConfig,
346    pub no_poh_speed_test: bool,
347    pub no_os_memory_stats_reporting: bool,
348    pub no_os_network_stats_reporting: bool,
349    pub no_os_cpu_stats_reporting: bool,
350    pub no_os_disk_stats_reporting: bool,
351    pub enforce_ulimit_nofile: bool,
352    pub poh_pinned_cpu_core: usize,
353    pub poh_hashes_per_batch: u64,
354    pub process_ledger_before_services: bool,
355    pub accounts_db_config: AccountsDbConfig,
356    pub warp_slot: Option<Slot>,
357    pub accounts_db_skip_shrink: bool,
358    pub accounts_db_force_initial_clean: bool,
359    pub staked_nodes_overrides: Arc<RwLock<HashMap<Pubkey, u64>>>,
360    pub validator_exit: Arc<RwLock<Exit>>,
361    pub validator_exit_backpressure: HashMap<String, Arc<AtomicBool>>,
362    pub no_wait_for_vote_to_start_leader: bool,
363    pub wait_to_vote_slot: Option<Slot>,
364    pub runtime_config: RuntimeConfig,
365    pub banking_trace_dir_byte_limit: banking_trace::DirByteLimit,
366    pub block_verification_method: BlockVerificationMethod,
367    pub block_production_method: BlockProductionMethod,
368    pub block_production_num_workers: NonZeroUsize,
369    pub block_production_scheduler_config: SchedulerConfig,
370    pub enable_block_production_forwarding: bool,
371    pub generator_config: Option<GeneratorConfig>,
372    pub use_snapshot_archives_at_startup: UseSnapshotArchivesAtStartup,
373    pub wen_restart_proto_path: Option<PathBuf>,
374    pub wen_restart_coordinator: Option<Pubkey>,
375    pub unified_scheduler_handler_threads: Option<usize>,
376    pub ip_echo_server_threads: NonZeroUsize,
377    pub rayon_global_threads: NonZeroUsize,
378    pub replay_forks_threads: NonZeroUsize,
379    pub replay_transactions_threads: NonZeroUsize,
380    pub tvu_shred_sigverify_threads: NonZeroUsize,
381    pub delay_leader_block_for_pending_fork: bool,
382    pub use_tpu_client_next: bool,
383    pub retransmit_xdp: Option<XdpConfig>,
384    pub repair_handler_type: RepairHandlerType,
385}
386
387impl ValidatorConfig {
388    pub fn default_for_test() -> Self {
389        let max_thread_count =
390            NonZeroUsize::new(num_cpus::get()).expect("thread count is non-zero");
391
392        Self {
393            halt_at_slot: None,
394            expected_genesis_hash: None,
395            expected_bank_hash: None,
396            expected_shred_version: None,
397            voting_disabled: false,
398            max_ledger_shreds: None,
399            blockstore_options: BlockstoreOptions::default_for_tests(),
400            account_paths: Vec::new(),
401            account_snapshot_paths: Vec::new(),
402            rpc_config: JsonRpcConfig::default_for_test(),
403            on_start_geyser_plugin_config_files: None,
404            geyser_plugin_always_enabled: false,
405            rpc_addrs: None,
406            pubsub_config: PubSubConfig::default(),
407            snapshot_config: SnapshotConfig::new_load_only(),
408            broadcast_stage_type: BroadcastStageType::Standard,
409            turbine_disabled: Arc::<AtomicBool>::default(),
410            fixed_leader_schedule: None,
411            wait_for_supermajority: None,
412            new_hard_forks: None,
413            known_validators: None,
414            repair_validators: None,
415            repair_whitelist: Arc::new(RwLock::new(HashSet::default())),
416            gossip_validators: None,
417            max_genesis_archive_unpacked_size: MAX_GENESIS_ARCHIVE_UNPACKED_SIZE,
418            run_verification: true,
419            require_tower: false,
420            tower_storage: Arc::new(NullTowerStorage::default()),
421            debug_keys: None,
422            contact_debug_interval: DEFAULT_CONTACT_DEBUG_INTERVAL_MILLIS,
423            contact_save_interval: DEFAULT_CONTACT_SAVE_INTERVAL_MILLIS,
424            send_transaction_service_config: SendTransactionServiceConfig::default(),
425            no_poh_speed_test: true,
426            no_os_memory_stats_reporting: true,
427            no_os_network_stats_reporting: true,
428            no_os_cpu_stats_reporting: true,
429            no_os_disk_stats_reporting: true,
430            // No need to enforce nofile limit in tests
431            enforce_ulimit_nofile: false,
432            poh_pinned_cpu_core: poh_service::DEFAULT_PINNED_CPU_CORE,
433            poh_hashes_per_batch: poh_service::DEFAULT_HASHES_PER_BATCH,
434            process_ledger_before_services: false,
435            warp_slot: None,
436            accounts_db_skip_shrink: false,
437            accounts_db_force_initial_clean: false,
438            staked_nodes_overrides: Arc::new(RwLock::new(HashMap::new())),
439            validator_exit: Arc::new(RwLock::new(Exit::default())),
440            validator_exit_backpressure: HashMap::default(),
441            no_wait_for_vote_to_start_leader: true,
442            accounts_db_config: ACCOUNTS_DB_CONFIG_FOR_TESTING,
443            wait_to_vote_slot: None,
444            runtime_config: RuntimeConfig::default(),
445            banking_trace_dir_byte_limit: 0,
446            block_verification_method: BlockVerificationMethod::default(),
447            block_production_method: BlockProductionMethod::default(),
448            block_production_num_workers: BankingStage::default_num_workers(),
449            block_production_scheduler_config: SchedulerConfig::default(),
450            // enable forwarding by default for tests
451            enable_block_production_forwarding: true,
452            generator_config: None,
453            use_snapshot_archives_at_startup: UseSnapshotArchivesAtStartup::default(),
454            wen_restart_proto_path: None,
455            wen_restart_coordinator: None,
456            unified_scheduler_handler_threads: None,
457            ip_echo_server_threads: NonZeroUsize::new(1).expect("1 is non-zero"),
458            rayon_global_threads: max_thread_count,
459            replay_forks_threads: NonZeroUsize::new(1).expect("1 is non-zero"),
460            replay_transactions_threads: max_thread_count,
461            tvu_shred_sigverify_threads: NonZeroUsize::new(get_thread_count())
462                .expect("thread count is non-zero"),
463            delay_leader_block_for_pending_fork: false,
464            use_tpu_client_next: true,
465            retransmit_xdp: None,
466            repair_handler_type: RepairHandlerType::default(),
467        }
468    }
469
470    pub fn enable_default_rpc_block_subscribe(&mut self) {
471        let pubsub_config = PubSubConfig {
472            enable_block_subscription: true,
473            ..PubSubConfig::default()
474        };
475        let rpc_config = JsonRpcConfig {
476            enable_rpc_transaction_history: true,
477            ..JsonRpcConfig::default_for_test()
478        };
479
480        self.pubsub_config = pubsub_config;
481        self.rpc_config = rpc_config;
482    }
483}
484
485// `ValidatorStartProgress` contains status information that is surfaced to the node operator over
486// the admin RPC channel to help them to follow the general progress of node startup without
487// having to watch log messages.
488#[derive(Debug, Clone, Copy, Serialize, Deserialize, PartialEq, Eq, Default)]
489pub enum ValidatorStartProgress {
490    #[default]
491    Initializing, // Catch all, default state
492    SearchingForRpcService,
493    DownloadingSnapshot {
494        slot: Slot,
495        rpc_addr: SocketAddr,
496    },
497    CleaningBlockStore,
498    CleaningAccounts,
499    LoadingLedger,
500    ProcessingLedger {
501        slot: Slot,
502        max_slot: Slot,
503    },
504    StartingServices,
505    Halted, // Validator halted due to `--dev-halt-at-slot` argument
506    WaitingForSupermajority {
507        slot: Slot,
508        gossip_stake_percent: u64,
509    },
510
511    // `Running` is the terminal state once the validator fully starts and all services are
512    // operational
513    Running,
514}
515
516struct BlockstoreRootScan {
517    thread: Option<JoinHandle<Result<usize, BlockstoreError>>>,
518}
519
520impl BlockstoreRootScan {
521    fn new(config: &ValidatorConfig, blockstore: Arc<Blockstore>, exit: Arc<AtomicBool>) -> Self {
522        let thread = if config.rpc_addrs.is_some()
523            && config.rpc_config.enable_rpc_transaction_history
524            && config.rpc_config.rpc_scan_and_fix_roots
525        {
526            Some(
527                Builder::new()
528                    .name("solBStoreRtScan".to_string())
529                    .spawn(move || blockstore.scan_and_fix_roots(None, None, &exit))
530                    .unwrap(),
531            )
532        } else {
533            None
534        };
535        Self { thread }
536    }
537
538    fn join(self) {
539        if let Some(blockstore_root_scan) = self.thread {
540            if let Err(err) = blockstore_root_scan.join() {
541                warn!("blockstore_root_scan failed to join {err:?}");
542            }
543        }
544    }
545}
546
547#[derive(Default)]
548struct TransactionHistoryServices {
549    transaction_status_sender: Option<TransactionStatusSender>,
550    transaction_status_service: Option<TransactionStatusService>,
551    max_complete_transaction_status_slot: Arc<AtomicU64>,
552}
553
554/// A struct easing passing Validator TPU Configurations
555pub struct ValidatorTpuConfig {
556    /// Controls if to use QUIC for sending regular TPU transaction
557    pub use_quic: bool,
558    /// Controls if to use QUIC for sending TPU votes
559    pub vote_use_quic: bool,
560    /// Controls the connection cache pool size
561    pub tpu_connection_pool_size: usize,
562    /// Controls if to enable UDP for TPU tansactions.
563    pub tpu_enable_udp: bool,
564    /// QUIC server config for regular TPU
565    pub tpu_quic_server_config: SwQosQuicStreamerConfig,
566    /// QUIC server config for TPU forward
567    pub tpu_fwd_quic_server_config: SwQosQuicStreamerConfig,
568    /// QUIC server config for Vote
569    pub vote_quic_server_config: SimpleQosQuicStreamerConfig,
570}
571
572impl ValidatorTpuConfig {
573    /// A convenient function to build a ValidatorTpuConfig for testing with good
574    /// default.
575    pub fn new_for_tests(tpu_enable_udp: bool) -> Self {
576        let tpu_quic_server_config = SwQosQuicStreamerConfig {
577            quic_streamer_config: QuicStreamerConfig {
578                max_connections_per_ipaddr_per_min: 32,
579                accumulator_channel_size: 100_000, // smaller channel size for faster test
580                ..Default::default()
581            },
582            qos_config: SwQosConfig::default(),
583        };
584
585        let tpu_fwd_quic_server_config = SwQosQuicStreamerConfig {
586            quic_streamer_config: QuicStreamerConfig {
587                max_connections_per_ipaddr_per_min: 32,
588                max_unstaked_connections: 0,
589                accumulator_channel_size: 100_000, // smaller channel size for faster test
590                ..Default::default()
591            },
592            qos_config: SwQosConfig::default(),
593        };
594
595        // vote and tpu_fwd share the same characteristics -- disallow non-staked connections:
596        let vote_quic_server_config = SimpleQosQuicStreamerConfig {
597            quic_streamer_config: QuicStreamerConfig {
598                max_connections_per_ipaddr_per_min: 32,
599                max_unstaked_connections: 0,
600                accumulator_channel_size: 100_000, // smaller channel size for faster test
601                ..Default::default()
602            },
603            qos_config: SimpleQosConfig::default(),
604        };
605
606        ValidatorTpuConfig {
607            use_quic: DEFAULT_TPU_USE_QUIC,
608            vote_use_quic: DEFAULT_VOTE_USE_QUIC,
609            tpu_connection_pool_size: DEFAULT_TPU_CONNECTION_POOL_SIZE,
610            tpu_enable_udp,
611            tpu_quic_server_config,
612            tpu_fwd_quic_server_config,
613            vote_quic_server_config,
614        }
615    }
616}
617
618pub struct Validator {
619    validator_exit: Arc<RwLock<Exit>>,
620    json_rpc_service: Option<JsonRpcService>,
621    pubsub_service: Option<PubSubService>,
622    rpc_completed_slots_service: Option<JoinHandle<()>>,
623    optimistically_confirmed_bank_tracker: Option<OptimisticallyConfirmedBankTracker>,
624    transaction_status_service: Option<TransactionStatusService>,
625    entry_notifier_service: Option<EntryNotifierService>,
626    system_monitor_service: Option<SystemMonitorService>,
627    sample_performance_service: Option<SamplePerformanceService>,
628    stats_reporter_service: StatsReporterService,
629    gossip_service: GossipService,
630    serve_repair_service: ServeRepairService,
631    completed_data_sets_service: Option<CompletedDataSetsService>,
632    snapshot_packager_service: Option<SnapshotPackagerService>,
633    poh_recorder: Arc<RwLock<PohRecorder>>,
634    poh_service: PohService,
635    tpu: Tpu,
636    tvu: Tvu,
637    ip_echo_server: Option<solana_net_utils::IpEchoServer>,
638    pub cluster_info: Arc<ClusterInfo>,
639    pub bank_forks: Arc<RwLock<BankForks>>,
640    pub blockstore: Arc<Blockstore>,
641    geyser_plugin_service: Option<GeyserPluginService>,
642    blockstore_metric_report_service: BlockstoreMetricReportService,
643    accounts_background_service: AccountsBackgroundService,
644    turbine_quic_endpoint: Option<Endpoint>,
645    turbine_quic_endpoint_runtime: Option<TokioRuntime>,
646    turbine_quic_endpoint_join_handle: Option<solana_turbine::quic_endpoint::AsyncTryJoinHandle>,
647    repair_quic_endpoints: Option<[Endpoint; 3]>,
648    repair_quic_endpoints_runtime: Option<TokioRuntime>,
649    repair_quic_endpoints_join_handle: Option<repair::quic_endpoint::AsyncTryJoinHandle>,
650    xdp_retransmitter: Option<XdpRetransmitter>,
651    // This runtime is used to run the client owned by SendTransactionService.
652    // We don't wait for its JoinHandle here because ownership and shutdown
653    // are managed elsewhere. This variable is intentionally unused.
654    _tpu_client_next_runtime: Option<TokioRuntime>,
655}
656
657impl Validator {
658    #[allow(clippy::too_many_arguments)]
659    pub fn new(
660        mut node: Node,
661        identity_keypair: Arc<Keypair>,
662        ledger_path: &Path,
663        vote_account: &Pubkey,
664        authorized_voter_keypairs: Arc<RwLock<Vec<Arc<Keypair>>>>,
665        cluster_entrypoints: Vec<ContactInfo>,
666        config: &ValidatorConfig,
667        should_check_duplicate_instance: bool,
668        rpc_to_plugin_manager_receiver: Option<Receiver<GeyserPluginManagerRequest>>,
669        start_progress: Arc<RwLock<ValidatorStartProgress>>,
670        socket_addr_space: SocketAddrSpace,
671        tpu_config: ValidatorTpuConfig,
672        admin_rpc_service_post_init: Arc<RwLock<Option<AdminRpcRequestMetadataPostInit>>>,
673    ) -> Result<Self> {
674        #[cfg(debug_assertions)]
675        const DEBUG_ASSERTION_STATUS: &str = "enabled";
676        #[cfg(not(debug_assertions))]
677        const DEBUG_ASSERTION_STATUS: &str = "disabled";
678        info!("debug-assertion status: {DEBUG_ASSERTION_STATUS}");
679
680        let ValidatorTpuConfig {
681            use_quic,
682            vote_use_quic,
683            tpu_connection_pool_size,
684            tpu_enable_udp,
685            tpu_quic_server_config,
686            tpu_fwd_quic_server_config,
687            vote_quic_server_config,
688        } = tpu_config;
689
690        let start_time = Instant::now();
691
692        adjust_nofile_limit(config.enforce_ulimit_nofile)?;
693
694        // Initialize the global rayon pool first to ensure the value in config
695        // is honored. Otherwise, some code accessing the global pool could
696        // cause it to get initialized with Rayon's default (not ours)
697        if rayon::ThreadPoolBuilder::new()
698            .thread_name(|i| format!("solRayonGlob{i:02}"))
699            .num_threads(config.rayon_global_threads.get())
700            .build_global()
701            .is_err()
702        {
703            warn!("Rayon global thread pool already initialized");
704        }
705
706        let id = identity_keypair.pubkey();
707        assert_eq!(&id, node.info.pubkey());
708
709        info!("identity pubkey: {id}");
710        info!("vote account pubkey: {vote_account}");
711
712        if !config.no_os_network_stats_reporting {
713            verify_net_stats_access().map_err(|e| {
714                ValidatorError::Other(format!("Failed to access network stats: {e:?}"))
715            })?;
716        }
717
718        let mut bank_notification_senders = Vec::new();
719
720        let exit = Arc::new(AtomicBool::new(false));
721
722        let geyser_plugin_config_files = config
723            .on_start_geyser_plugin_config_files
724            .as_ref()
725            .map(Cow::Borrowed)
726            .or_else(|| {
727                config
728                    .geyser_plugin_always_enabled
729                    .then_some(Cow::Owned(vec![]))
730            });
731        let geyser_plugin_service =
732            if let Some(geyser_plugin_config_files) = geyser_plugin_config_files {
733                let (confirmed_bank_sender, confirmed_bank_receiver) = unbounded();
734                bank_notification_senders.push(confirmed_bank_sender);
735                let rpc_to_plugin_manager_receiver_and_exit =
736                    rpc_to_plugin_manager_receiver.map(|receiver| (receiver, exit.clone()));
737                Some(
738                    GeyserPluginService::new_with_receiver(
739                        confirmed_bank_receiver,
740                        config.geyser_plugin_always_enabled,
741                        geyser_plugin_config_files.as_ref(),
742                        rpc_to_plugin_manager_receiver_and_exit,
743                    )
744                    .map_err(|err| {
745                        ValidatorError::Other(format!("Failed to load the Geyser plugin: {err:?}"))
746                    })?,
747                )
748            } else {
749                None
750            };
751
752        if config.voting_disabled {
753            warn!("voting disabled");
754            authorized_voter_keypairs.write().unwrap().clear();
755        } else {
756            for authorized_voter_keypair in authorized_voter_keypairs.read().unwrap().iter() {
757                warn!("authorized voter: {}", authorized_voter_keypair.pubkey());
758            }
759        }
760
761        for cluster_entrypoint in &cluster_entrypoints {
762            info!("entrypoint: {cluster_entrypoint:?}");
763        }
764
765        if solana_perf::perf_libs::api().is_some() {
766            info!("Initializing sigverify, this could take a while...");
767        } else {
768            info!("Initializing sigverify...");
769        }
770        sigverify::init();
771        info!("Initializing sigverify done.");
772
773        validate_memlock_limit_for_disk_io(config.accounts_db_config.memlock_budget_size)?;
774
775        if !ledger_path.is_dir() {
776            return Err(anyhow!(
777                "ledger directory does not exist or is not accessible: {ledger_path:?}"
778            ));
779        }
780        let genesis_config = load_genesis(config, ledger_path)?;
781        metrics_config_sanity_check(genesis_config.cluster_type)?;
782
783        info!("Cleaning accounts paths..");
784        *start_progress.write().unwrap() = ValidatorStartProgress::CleaningAccounts;
785        let mut timer = Measure::start("clean_accounts_paths");
786        cleanup_accounts_paths(config);
787        timer.stop();
788        info!("Cleaning accounts paths done. {timer}");
789
790        snapshot_utils::purge_incomplete_bank_snapshots(&config.snapshot_config.bank_snapshots_dir);
791        snapshot_utils::purge_old_bank_snapshots_at_startup(
792            &config.snapshot_config.bank_snapshots_dir,
793        );
794
795        info!("Cleaning orphaned account snapshot directories..");
796        let mut timer = Measure::start("clean_orphaned_account_snapshot_dirs");
797        clean_orphaned_account_snapshot_dirs(
798            &config.snapshot_config.bank_snapshots_dir,
799            &config.account_snapshot_paths,
800        )
801        .context("failed to clean orphaned account snapshot directories")?;
802        timer.stop();
803        info!("Cleaning orphaned account snapshot directories done. {timer}");
804
805        // token used to cancel tpu-client-next and streamer.
806        let cancel = CancellationToken::new();
807        {
808            let exit = exit.clone();
809            config
810                .validator_exit
811                .write()
812                .unwrap()
813                .register_exit(Box::new(move || exit.store(true, Ordering::Relaxed)));
814            let cancel = cancel.clone();
815            config
816                .validator_exit
817                .write()
818                .unwrap()
819                .register_exit(Box::new(move || cancel.cancel()));
820        }
821
822        let (
823            accounts_update_notifier,
824            transaction_notifier,
825            entry_notifier,
826            block_metadata_notifier,
827            slot_status_notifier,
828        ) = if let Some(service) = &geyser_plugin_service {
829            (
830                service.get_accounts_update_notifier(),
831                service.get_transaction_notifier(),
832                service.get_entry_notifier(),
833                service.get_block_metadata_notifier(),
834                service.get_slot_status_notifier(),
835            )
836        } else {
837            (None, None, None, None, None)
838        };
839
840        info!(
841            "Geyser plugin: accounts_update_notifier: {}, transaction_notifier: {}, \
842             entry_notifier: {}",
843            accounts_update_notifier.is_some(),
844            transaction_notifier.is_some(),
845            entry_notifier.is_some()
846        );
847
848        let system_monitor_service = Some(SystemMonitorService::new(
849            exit.clone(),
850            SystemMonitorStatsReportConfig {
851                report_os_memory_stats: !config.no_os_memory_stats_reporting,
852                report_os_network_stats: !config.no_os_network_stats_reporting,
853                report_os_cpu_stats: !config.no_os_cpu_stats_reporting,
854                report_os_disk_stats: !config.no_os_disk_stats_reporting,
855            },
856        ));
857
858        let dependency_tracker = Arc::new(DependencyTracker::default());
859
860        let (
861            bank_forks,
862            blockstore,
863            original_blockstore_root,
864            ledger_signal_receiver,
865            leader_schedule_cache,
866            starting_snapshot_hashes,
867            TransactionHistoryServices {
868                transaction_status_sender,
869                transaction_status_service,
870                max_complete_transaction_status_slot,
871            },
872            blockstore_process_options,
873            blockstore_root_scan,
874            pruned_banks_receiver,
875            entry_notifier_service,
876        ) = load_blockstore(
877            config,
878            ledger_path,
879            &genesis_config,
880            exit.clone(),
881            &start_progress,
882            accounts_update_notifier,
883            transaction_notifier,
884            entry_notifier,
885            config
886                .rpc_addrs
887                .is_some()
888                .then(|| dependency_tracker.clone()),
889        )
890        .map_err(ValidatorError::Other)?;
891
892        if !config.no_poh_speed_test {
893            check_poh_speed(&bank_forks.read().unwrap().root_bank(), None)?;
894        }
895
896        let (root_slot, hard_forks) = {
897            let root_bank = bank_forks.read().unwrap().root_bank();
898            (root_bank.slot(), root_bank.hard_forks())
899        };
900        let shred_version = compute_shred_version(&genesis_config.hash(), Some(&hard_forks));
901        info!("shred version: {shred_version}, hard forks: {hard_forks:?}");
902
903        if let Some(expected_shred_version) = config.expected_shred_version {
904            if expected_shred_version != shred_version {
905                return Err(ValidatorError::ShredVersionMismatch {
906                    actual: shred_version,
907                    expected: expected_shred_version,
908                }
909                .into());
910            }
911        }
912
913        if let Some(start_slot) = should_cleanup_blockstore_incorrect_shred_versions(
914            config,
915            &blockstore,
916            root_slot,
917            &hard_forks,
918        )? {
919            *start_progress.write().unwrap() = ValidatorStartProgress::CleaningBlockStore;
920            cleanup_blockstore_incorrect_shred_versions(
921                &blockstore,
922                config,
923                start_slot,
924                shred_version,
925            )?;
926        } else {
927            info!("Skipping the blockstore check for shreds with incorrect version");
928        }
929
930        node.info.set_shred_version(shred_version);
931        node.info.set_wallclock(timestamp());
932        Self::print_node_info(&node);
933
934        let mut cluster_info = ClusterInfo::new(
935            node.info.clone(),
936            identity_keypair.clone(),
937            socket_addr_space,
938        );
939        cluster_info.set_contact_debug_interval(config.contact_debug_interval);
940        cluster_info.set_entrypoints(cluster_entrypoints);
941        cluster_info.restore_contact_info(ledger_path, config.contact_save_interval);
942        cluster_info.set_bind_ip_addrs(node.bind_ip_addrs.clone());
943        let cluster_info = Arc::new(cluster_info);
944        let node_multihoming = Arc::new(NodeMultihoming::from(&node));
945
946        assert!(is_snapshot_config_valid(&config.snapshot_config));
947
948        let (snapshot_request_sender, snapshot_request_receiver) = unbounded();
949        let snapshot_controller = Arc::new(SnapshotController::new(
950            snapshot_request_sender.clone(),
951            config.snapshot_config.clone(),
952            bank_forks.read().unwrap().root(),
953        ));
954
955        let pending_snapshot_packages = Arc::new(Mutex::new(PendingSnapshotPackages::default()));
956        let snapshot_packager_service = if snapshot_controller
957            .snapshot_config()
958            .should_generate_snapshots()
959        {
960            let exit_backpressure = config
961                .validator_exit_backpressure
962                .get(SnapshotPackagerService::NAME)
963                .cloned();
964            let enable_gossip_push = true;
965            let snapshot_packager_service = SnapshotPackagerService::new(
966                pending_snapshot_packages.clone(),
967                starting_snapshot_hashes,
968                exit.clone(),
969                exit_backpressure,
970                cluster_info.clone(),
971                snapshot_controller.clone(),
972                enable_gossip_push,
973            );
974            Some(snapshot_packager_service)
975        } else {
976            None
977        };
978
979        let snapshot_request_handler = SnapshotRequestHandler {
980            snapshot_controller: snapshot_controller.clone(),
981            snapshot_request_receiver,
982            pending_snapshot_packages,
983        };
984        let pruned_banks_request_handler = PrunedBanksRequestHandler {
985            pruned_banks_receiver,
986        };
987        let accounts_background_service = AccountsBackgroundService::new(
988            bank_forks.clone(),
989            exit.clone(),
990            AbsRequestHandlers {
991                snapshot_request_handler,
992                pruned_banks_request_handler,
993            },
994        );
995        info!(
996            "Using: block-verification-method: {}, block-production-method: {}",
997            config.block_verification_method, config.block_production_method,
998        );
999
1000        let (replay_vote_sender, replay_vote_receiver) = unbounded();
1001
1002        // block min prioritization fee cache should be readable by RPC, and writable by validator
1003        // (by both replay stage and banking stage)
1004        let prioritization_fee_cache = Arc::new(PrioritizationFeeCache::default());
1005
1006        let leader_schedule_cache = Arc::new(leader_schedule_cache);
1007        let (poh_recorder, entry_receiver) = {
1008            let bank = &bank_forks.read().unwrap().working_bank();
1009            PohRecorder::new_with_clear_signal(
1010                bank.tick_height(),
1011                bank.last_blockhash(),
1012                bank.clone(),
1013                None,
1014                bank.ticks_per_slot(),
1015                config.delay_leader_block_for_pending_fork,
1016                blockstore.clone(),
1017                blockstore.get_new_shred_signal(0),
1018                &leader_schedule_cache,
1019                &genesis_config.poh_config,
1020                exit.clone(),
1021            )
1022        };
1023        let (record_sender, record_receiver) = record_channels(transaction_status_sender.is_some());
1024        let transaction_recorder = TransactionRecorder::new(record_sender);
1025        let poh_recorder = Arc::new(RwLock::new(poh_recorder));
1026        let (poh_controller, poh_service_message_receiver) = PohController::new();
1027
1028        let (banking_tracer, tracer_thread) =
1029            BankingTracer::new((config.banking_trace_dir_byte_limit > 0).then_some((
1030                &blockstore.banking_trace_path(),
1031                exit.clone(),
1032                config.banking_trace_dir_byte_limit,
1033            )))?;
1034        if banking_tracer.is_enabled() {
1035            info!(
1036                "Enabled banking trace (dir_byte_limit: {})",
1037                config.banking_trace_dir_byte_limit
1038            );
1039        } else {
1040            info!("Disabled banking trace");
1041        }
1042        let banking_tracer_channels = banking_tracer.create_channels(false);
1043
1044        match &config.block_verification_method {
1045            BlockVerificationMethod::BlockstoreProcessor => {
1046                info!("no scheduler pool is installed for block verification...");
1047                if let Some(count) = config.unified_scheduler_handler_threads {
1048                    warn!(
1049                        "--unified-scheduler-handler-threads={count} is ignored because unified \
1050                         scheduler isn't enabled"
1051                    );
1052                }
1053            }
1054            BlockVerificationMethod::UnifiedScheduler => {
1055                let scheduler_pool = DefaultSchedulerPool::new_dyn(
1056                    config.unified_scheduler_handler_threads,
1057                    config.runtime_config.log_messages_bytes_limit,
1058                    transaction_status_sender.clone(),
1059                    Some(replay_vote_sender.clone()),
1060                    prioritization_fee_cache.clone(),
1061                );
1062                bank_forks
1063                    .write()
1064                    .unwrap()
1065                    .install_scheduler_pool(scheduler_pool);
1066            }
1067        }
1068
1069        let entry_notification_sender = entry_notifier_service
1070            .as_ref()
1071            .map(|service| service.sender());
1072        let mut process_blockstore = ProcessBlockStore::new(
1073            &id,
1074            vote_account,
1075            &start_progress,
1076            &blockstore,
1077            original_blockstore_root,
1078            &bank_forks,
1079            &leader_schedule_cache,
1080            &blockstore_process_options,
1081            transaction_status_sender.as_ref(),
1082            entry_notification_sender,
1083            blockstore_root_scan,
1084            &snapshot_controller,
1085            config,
1086        );
1087
1088        maybe_warp_slot(
1089            config,
1090            &mut process_blockstore,
1091            ledger_path,
1092            &bank_forks,
1093            &leader_schedule_cache,
1094            &snapshot_controller,
1095        )
1096        .map_err(ValidatorError::Other)?;
1097
1098        if config.process_ledger_before_services {
1099            process_blockstore
1100                .process()
1101                .map_err(ValidatorError::Other)?;
1102        }
1103        *start_progress.write().unwrap() = ValidatorStartProgress::StartingServices;
1104
1105        let sample_performance_service =
1106            if config.rpc_addrs.is_some() && config.rpc_config.enable_rpc_transaction_history {
1107                Some(SamplePerformanceService::new(
1108                    &bank_forks,
1109                    blockstore.clone(),
1110                    exit.clone(),
1111                ))
1112            } else {
1113                None
1114            };
1115
1116        let mut block_commitment_cache = BlockCommitmentCache::default();
1117        let bank_forks_guard = bank_forks.read().unwrap();
1118        block_commitment_cache.initialize_slots(
1119            bank_forks_guard.working_bank().slot(),
1120            bank_forks_guard.root(),
1121        );
1122        drop(bank_forks_guard);
1123        let block_commitment_cache = Arc::new(RwLock::new(block_commitment_cache));
1124
1125        let optimistically_confirmed_bank =
1126            OptimisticallyConfirmedBank::locked_from_bank_forks_root(&bank_forks);
1127
1128        let max_slots = Arc::new(MaxSlots::default());
1129
1130        let staked_nodes = Arc::new(RwLock::new(StakedNodes::default()));
1131
1132        let mut tpu_transactions_forwards_client_sockets =
1133            Some(node.sockets.tpu_transaction_forwarding_clients);
1134        let connection_cache = match (config.use_tpu_client_next, use_quic) {
1135            (false, true) => Some(Arc::new(ConnectionCache::new_with_client_options(
1136                "connection_cache_tpu_quic",
1137                tpu_connection_pool_size,
1138                Some({
1139                    // this conversion is not beautiful but rust does not allow popping single
1140                    // elements from a boxed slice
1141                    let socketbox: Box<[_; 1]> = tpu_transactions_forwards_client_sockets
1142                        .take()
1143                        .unwrap()
1144                        .try_into()
1145                        .expect("Multihoming support for connection cache is not available");
1146                    let [sock] = *socketbox;
1147                    sock
1148                }),
1149                Some((
1150                    &identity_keypair,
1151                    node.info
1152                        .tpu(Protocol::UDP)
1153                        .ok_or_else(|| {
1154                            ValidatorError::Other(String::from("Invalid UDP address for TPU"))
1155                        })?
1156                        .ip(),
1157                )),
1158                Some((&staked_nodes, &identity_keypair.pubkey())),
1159            ))),
1160            (false, false) => Some(Arc::new(ConnectionCache::with_udp(
1161                "connection_cache_tpu_udp",
1162                tpu_connection_pool_size,
1163            ))),
1164            (true, _) => None,
1165        };
1166
1167        let vote_connection_cache = if vote_use_quic {
1168            let vote_connection_cache = ConnectionCache::new_with_client_options(
1169                "connection_cache_vote_quic",
1170                tpu_connection_pool_size,
1171                Some(node.sockets.quic_vote_client),
1172                Some((
1173                    &identity_keypair,
1174                    node.info
1175                        .tpu_vote(Protocol::QUIC)
1176                        .ok_or_else(|| {
1177                            ValidatorError::Other(String::from("Invalid QUIC address for TPU Vote"))
1178                        })?
1179                        .ip(),
1180                )),
1181                Some((&staked_nodes, &identity_keypair.pubkey())),
1182            );
1183            Arc::new(vote_connection_cache)
1184        } else {
1185            Arc::new(ConnectionCache::with_udp(
1186                "connection_cache_vote_udp",
1187                tpu_connection_pool_size,
1188            ))
1189        };
1190
1191        // test-validator crate may start the validator in a tokio runtime
1192        // context which forces us to use the same runtime because a nested
1193        // runtime will cause panic at drop. Outside test-validator crate, we
1194        // always need a tokio runtime (and the respective handle) to initialize
1195        // the turbine QUIC endpoint.
1196        let current_runtime_handle = tokio::runtime::Handle::try_current();
1197        let tpu_client_next_runtime =
1198            (current_runtime_handle.is_err() && config.use_tpu_client_next).then(|| {
1199                tokio::runtime::Builder::new_multi_thread()
1200                    .enable_all()
1201                    .worker_threads(2)
1202                    .thread_name("solTpuClientRt")
1203                    .build()
1204                    .unwrap()
1205            });
1206
1207        let rpc_override_health_check =
1208            Arc::new(AtomicBool::new(config.rpc_config.disable_health_check));
1209        let (
1210            json_rpc_service,
1211            rpc_subscriptions,
1212            pubsub_service,
1213            completed_data_sets_sender,
1214            completed_data_sets_service,
1215            rpc_completed_slots_service,
1216            optimistically_confirmed_bank_tracker,
1217            bank_notification_sender,
1218        ) = if let Some((rpc_addr, rpc_pubsub_addr)) = config.rpc_addrs {
1219            assert_eq!(
1220                node.info.rpc().map(|addr| socket_addr_space.check(&addr)),
1221                node.info
1222                    .rpc_pubsub()
1223                    .map(|addr| socket_addr_space.check(&addr))
1224            );
1225            let (bank_notification_sender, bank_notification_receiver) = unbounded();
1226            let confirmed_bank_subscribers = if !bank_notification_senders.is_empty() {
1227                Some(Arc::new(RwLock::new(bank_notification_senders)))
1228            } else {
1229                None
1230            };
1231
1232            let client_option = if config.use_tpu_client_next {
1233                let runtime_handle = tpu_client_next_runtime
1234                    .as_ref()
1235                    .map(TokioRuntime::handle)
1236                    .unwrap_or_else(|| current_runtime_handle.as_ref().unwrap());
1237                ClientOption::TpuClientNext(
1238                    Arc::as_ref(&identity_keypair),
1239                    node.sockets.rpc_sts_client,
1240                    runtime_handle.clone(),
1241                    cancel.clone(),
1242                )
1243            } else {
1244                let Some(connection_cache) = &connection_cache else {
1245                    panic!("ConnectionCache should exist by construction.");
1246                };
1247                ClientOption::ConnectionCache(connection_cache.clone())
1248            };
1249            let rpc_svc_config = JsonRpcServiceConfig {
1250                rpc_addr,
1251                rpc_config: config.rpc_config.clone(),
1252                snapshot_config: Some(snapshot_controller.snapshot_config().clone()),
1253                bank_forks: bank_forks.clone(),
1254                block_commitment_cache: block_commitment_cache.clone(),
1255                blockstore: blockstore.clone(),
1256                cluster_info: cluster_info.clone(),
1257                poh_recorder: Some(poh_recorder.clone()),
1258                genesis_hash: genesis_config.hash(),
1259                ledger_path: ledger_path.to_path_buf(),
1260                validator_exit: config.validator_exit.clone(),
1261                exit: exit.clone(),
1262                override_health_check: rpc_override_health_check.clone(),
1263                optimistically_confirmed_bank: optimistically_confirmed_bank.clone(),
1264                send_transaction_service_config: config.send_transaction_service_config.clone(),
1265                max_slots: max_slots.clone(),
1266                leader_schedule_cache: leader_schedule_cache.clone(),
1267                max_complete_transaction_status_slot: max_complete_transaction_status_slot.clone(),
1268                prioritization_fee_cache: prioritization_fee_cache.clone(),
1269                client_option,
1270            };
1271            let json_rpc_service =
1272                JsonRpcService::new_with_config(rpc_svc_config).map_err(ValidatorError::Other)?;
1273            let rpc_subscriptions = Arc::new(RpcSubscriptions::new_with_config(
1274                exit.clone(),
1275                max_complete_transaction_status_slot,
1276                blockstore.clone(),
1277                bank_forks.clone(),
1278                block_commitment_cache.clone(),
1279                optimistically_confirmed_bank.clone(),
1280                &config.pubsub_config,
1281                None,
1282            ));
1283            let pubsub_service = if !config.rpc_config.full_api {
1284                None
1285            } else {
1286                let (trigger, pubsub_service) = PubSubService::new(
1287                    config.pubsub_config.clone(),
1288                    &rpc_subscriptions,
1289                    rpc_pubsub_addr,
1290                );
1291                config
1292                    .validator_exit
1293                    .write()
1294                    .unwrap()
1295                    .register_exit(Box::new(move || trigger.cancel()));
1296
1297                Some(pubsub_service)
1298            };
1299
1300            let (completed_data_sets_sender, completed_data_sets_service) =
1301                if !config.rpc_config.full_api {
1302                    (None, None)
1303                } else {
1304                    let (completed_data_sets_sender, completed_data_sets_receiver) =
1305                        bounded(MAX_COMPLETED_DATA_SETS_IN_CHANNEL);
1306                    let completed_data_sets_service = CompletedDataSetsService::new(
1307                        completed_data_sets_receiver,
1308                        blockstore.clone(),
1309                        rpc_subscriptions.clone(),
1310                        exit.clone(),
1311                        max_slots.clone(),
1312                    );
1313                    (
1314                        Some(completed_data_sets_sender),
1315                        Some(completed_data_sets_service),
1316                    )
1317                };
1318
1319            let rpc_completed_slots_service =
1320                if config.rpc_config.full_api || geyser_plugin_service.is_some() {
1321                    let (completed_slots_sender, completed_slots_receiver) =
1322                        bounded(MAX_COMPLETED_SLOTS_IN_CHANNEL);
1323                    blockstore.add_completed_slots_signal(completed_slots_sender);
1324
1325                    Some(RpcCompletedSlotsService::spawn(
1326                        completed_slots_receiver,
1327                        rpc_subscriptions.clone(),
1328                        slot_status_notifier.clone(),
1329                        exit.clone(),
1330                    ))
1331                } else {
1332                    None
1333                };
1334
1335            let dependency_tracker = transaction_status_sender
1336                .is_some()
1337                .then_some(dependency_tracker);
1338            let optimistically_confirmed_bank_tracker =
1339                Some(OptimisticallyConfirmedBankTracker::new(
1340                    bank_notification_receiver,
1341                    exit.clone(),
1342                    bank_forks.clone(),
1343                    optimistically_confirmed_bank,
1344                    rpc_subscriptions.clone(),
1345                    confirmed_bank_subscribers,
1346                    prioritization_fee_cache.clone(),
1347                    dependency_tracker.clone(),
1348                ));
1349            let bank_notification_sender_config = Some(BankNotificationSenderConfig {
1350                sender: bank_notification_sender,
1351                should_send_parents: geyser_plugin_service.is_some(),
1352                dependency_tracker,
1353            });
1354            (
1355                Some(json_rpc_service),
1356                Some(rpc_subscriptions),
1357                pubsub_service,
1358                completed_data_sets_sender,
1359                completed_data_sets_service,
1360                rpc_completed_slots_service,
1361                optimistically_confirmed_bank_tracker,
1362                bank_notification_sender_config,
1363            )
1364        } else {
1365            (None, None, None, None, None, None, None, None)
1366        };
1367
1368        if config.halt_at_slot.is_some() {
1369            // Simulate a confirmed root to avoid RPC errors with CommitmentConfig::finalized() and
1370            // to ensure RPC endpoints like getConfirmedBlock, which require a confirmed root, work
1371            block_commitment_cache
1372                .write()
1373                .unwrap()
1374                .set_highest_super_majority_root(bank_forks.read().unwrap().root());
1375
1376            // Park with the RPC service running, ready for inspection!
1377            warn!("Validator halted");
1378            *start_progress.write().unwrap() = ValidatorStartProgress::Halted;
1379            std::thread::park();
1380        }
1381        let ip_echo_server = match node.sockets.ip_echo {
1382            None => None,
1383            Some(tcp_listener) => Some(solana_net_utils::ip_echo_server(
1384                tcp_listener,
1385                config.ip_echo_server_threads,
1386                Some(node.info.shred_version()),
1387            )),
1388        };
1389
1390        let (stats_reporter_sender, stats_reporter_receiver) = unbounded();
1391
1392        let stats_reporter_service =
1393            StatsReporterService::new(stats_reporter_receiver, exit.clone());
1394
1395        let gossip_service = GossipService::new(
1396            &cluster_info,
1397            Some(bank_forks.clone()),
1398            node.sockets.gossip.clone(),
1399            config.gossip_validators.clone(),
1400            should_check_duplicate_instance,
1401            Some(stats_reporter_sender.clone()),
1402            exit.clone(),
1403        );
1404        let serve_repair = config.repair_handler_type.create_serve_repair(
1405            blockstore.clone(),
1406            cluster_info.clone(),
1407            bank_forks.read().unwrap().sharable_banks(),
1408            config.repair_whitelist.clone(),
1409        );
1410        let (repair_request_quic_sender, repair_request_quic_receiver) = unbounded();
1411        let (repair_response_quic_sender, repair_response_quic_receiver) = unbounded();
1412        let (ancestor_hashes_response_quic_sender, ancestor_hashes_response_quic_receiver) =
1413            unbounded();
1414
1415        let waited_for_supermajority = wait_for_supermajority(
1416            config,
1417            Some(&mut process_blockstore),
1418            &bank_forks,
1419            &cluster_info,
1420            rpc_override_health_check,
1421            &start_progress,
1422        )?;
1423
1424        let blockstore_metric_report_service =
1425            BlockstoreMetricReportService::new(blockstore.clone(), exit.clone());
1426
1427        let wait_for_vote_to_start_leader =
1428            !waited_for_supermajority && !config.no_wait_for_vote_to_start_leader;
1429
1430        let poh_service = PohService::new(
1431            poh_recorder.clone(),
1432            &genesis_config.poh_config,
1433            exit.clone(),
1434            bank_forks.read().unwrap().root_bank().ticks_per_slot(),
1435            config.poh_pinned_cpu_core,
1436            config.poh_hashes_per_batch,
1437            record_receiver,
1438            poh_service_message_receiver,
1439        );
1440        assert_eq!(
1441            blockstore.get_new_shred_signals_len(),
1442            1,
1443            "New shred signal for the TVU should be the same as the clear bank signal."
1444        );
1445
1446        let vote_tracker = Arc::<VoteTracker>::default();
1447
1448        let (retransmit_slots_sender, retransmit_slots_receiver) = unbounded();
1449        let (verified_vote_sender, verified_vote_receiver) = unbounded();
1450        let (gossip_verified_vote_hash_sender, gossip_verified_vote_hash_receiver) = unbounded();
1451        let (duplicate_confirmed_slot_sender, duplicate_confirmed_slots_receiver) = unbounded();
1452
1453        let entry_notification_sender = entry_notifier_service
1454            .as_ref()
1455            .map(|service| service.sender_cloned());
1456
1457        let turbine_quic_endpoint_runtime = (current_runtime_handle.is_err()
1458            && genesis_config.cluster_type != ClusterType::MainnetBeta)
1459            .then(|| {
1460                tokio::runtime::Builder::new_multi_thread()
1461                    .enable_all()
1462                    .thread_name("solTurbineQuic")
1463                    .build()
1464                    .unwrap()
1465            });
1466        let (turbine_quic_endpoint_sender, turbine_quic_endpoint_receiver) = unbounded();
1467        let (
1468            turbine_quic_endpoint,
1469            turbine_quic_endpoint_sender,
1470            turbine_quic_endpoint_join_handle,
1471        ) = if genesis_config.cluster_type == ClusterType::MainnetBeta {
1472            let (sender, _receiver) = tokio::sync::mpsc::channel(1);
1473            (None, sender, None)
1474        } else {
1475            solana_turbine::quic_endpoint::new_quic_endpoint(
1476                turbine_quic_endpoint_runtime
1477                    .as_ref()
1478                    .map(TokioRuntime::handle)
1479                    .unwrap_or_else(|| current_runtime_handle.as_ref().unwrap()),
1480                &identity_keypair,
1481                node.sockets.tvu_quic,
1482                turbine_quic_endpoint_sender,
1483                bank_forks.clone(),
1484            )
1485            .map(|(endpoint, sender, join_handle)| (Some(endpoint), sender, Some(join_handle)))
1486            .unwrap()
1487        };
1488
1489        // Repair quic endpoint.
1490        let repair_quic_endpoints_runtime = (current_runtime_handle.is_err()
1491            && genesis_config.cluster_type != ClusterType::MainnetBeta)
1492            .then(|| {
1493                tokio::runtime::Builder::new_multi_thread()
1494                    .enable_all()
1495                    .thread_name("solRepairQuic")
1496                    .build()
1497                    .unwrap()
1498            });
1499        let (repair_quic_endpoints, repair_quic_async_senders, repair_quic_endpoints_join_handle) =
1500            if genesis_config.cluster_type == ClusterType::MainnetBeta {
1501                (None, RepairQuicAsyncSenders::new_dummy(), None)
1502            } else {
1503                let repair_quic_sockets = RepairQuicSockets {
1504                    repair_server_quic_socket: node.sockets.serve_repair_quic,
1505                    repair_client_quic_socket: node.sockets.repair_quic,
1506                    ancestor_hashes_quic_socket: node.sockets.ancestor_hashes_requests_quic,
1507                };
1508                let repair_quic_senders = RepairQuicSenders {
1509                    repair_request_quic_sender: repair_request_quic_sender.clone(),
1510                    repair_response_quic_sender,
1511                    ancestor_hashes_response_quic_sender,
1512                };
1513                repair::quic_endpoint::new_quic_endpoints(
1514                    repair_quic_endpoints_runtime
1515                        .as_ref()
1516                        .map(TokioRuntime::handle)
1517                        .unwrap_or_else(|| current_runtime_handle.as_ref().unwrap()),
1518                    &identity_keypair,
1519                    repair_quic_sockets,
1520                    repair_quic_senders,
1521                    bank_forks.clone(),
1522                )
1523                .map(|(endpoints, senders, join_handle)| {
1524                    (Some(endpoints), senders, Some(join_handle))
1525                })
1526                .unwrap()
1527            };
1528        let serve_repair_service = ServeRepairService::new(
1529            serve_repair,
1530            // Incoming UDP repair requests are adapted into RemoteRequest
1531            // and also sent through the same channel.
1532            repair_request_quic_sender,
1533            repair_request_quic_receiver,
1534            repair_quic_async_senders.repair_response_quic_sender,
1535            node.sockets.serve_repair,
1536            socket_addr_space,
1537            stats_reporter_sender,
1538            exit.clone(),
1539        );
1540
1541        let in_wen_restart = config.wen_restart_proto_path.is_some() && !waited_for_supermajority;
1542        let wen_restart_repair_slots = if in_wen_restart {
1543            Some(Arc::new(RwLock::new(Vec::new())))
1544        } else {
1545            None
1546        };
1547        let tower = match process_blockstore.process_to_create_tower() {
1548            Ok(tower) => {
1549                info!("Tower state: {tower:?}");
1550                tower
1551            }
1552            Err(e) => {
1553                warn!("Unable to retrieve tower: {e:?} creating default tower....");
1554                Tower::default()
1555            }
1556        };
1557        let last_vote = tower.last_vote();
1558
1559        let outstanding_repair_requests =
1560            Arc::<RwLock<repair::repair_service::OutstandingShredRepairs>>::default();
1561        let root_bank = bank_forks.read().unwrap().root_bank();
1562        let cluster_slots = Arc::new({
1563            crate::cluster_slots_service::cluster_slots::ClusterSlots::new(
1564                &root_bank,
1565                &cluster_info,
1566            )
1567        });
1568
1569        // If RPC is supported and ConnectionCache is used, pass ConnectionCache for being warmup inside Tvu.
1570        let connection_cache_for_warmup =
1571            if json_rpc_service.is_some() && connection_cache.is_some() {
1572                connection_cache.as_ref()
1573            } else {
1574                None
1575            };
1576        let (xdp_retransmitter, xdp_sender) =
1577            if let Some(xdp_config) = config.retransmit_xdp.clone() {
1578                let src_port = node.sockets.retransmit_sockets[0]
1579                    .local_addr()
1580                    .expect("failed to get local address")
1581                    .port();
1582                let (rtx, sender) = XdpRetransmitter::new(xdp_config, src_port)
1583                    .expect("failed to create xdp retransmitter");
1584                (Some(rtx), Some(sender))
1585            } else {
1586                (None, None)
1587            };
1588
1589        // disable all2all tests if not allowed for a given cluster type
1590        let alpenglow_socket = if genesis_config.cluster_type == ClusterType::Testnet
1591            || genesis_config.cluster_type == ClusterType::Development
1592        {
1593            node.sockets.alpenglow
1594        } else {
1595            None
1596        };
1597
1598        let tvu = Tvu::new(
1599            vote_account,
1600            authorized_voter_keypairs,
1601            &bank_forks,
1602            &cluster_info,
1603            TvuSockets {
1604                repair: node.sockets.repair.try_clone().unwrap(),
1605                retransmit: node.sockets.retransmit_sockets,
1606                fetch: node.sockets.tvu,
1607                ancestor_hashes_requests: node.sockets.ancestor_hashes_requests,
1608                alpenglow: alpenglow_socket,
1609            },
1610            blockstore.clone(),
1611            ledger_signal_receiver,
1612            rpc_subscriptions.clone(),
1613            &poh_recorder,
1614            poh_controller,
1615            tower,
1616            config.tower_storage.clone(),
1617            &leader_schedule_cache,
1618            exit.clone(),
1619            block_commitment_cache,
1620            config.turbine_disabled.clone(),
1621            transaction_status_sender.clone(),
1622            entry_notification_sender.clone(),
1623            vote_tracker.clone(),
1624            retransmit_slots_sender,
1625            gossip_verified_vote_hash_receiver,
1626            verified_vote_receiver,
1627            replay_vote_sender.clone(),
1628            completed_data_sets_sender,
1629            bank_notification_sender.clone(),
1630            duplicate_confirmed_slots_receiver,
1631            TvuConfig {
1632                max_ledger_shreds: config.max_ledger_shreds,
1633                shred_version: node.info.shred_version(),
1634                repair_validators: config.repair_validators.clone(),
1635                repair_whitelist: config.repair_whitelist.clone(),
1636                wait_for_vote_to_start_leader,
1637                replay_forks_threads: config.replay_forks_threads,
1638                replay_transactions_threads: config.replay_transactions_threads,
1639                shred_sigverify_threads: config.tvu_shred_sigverify_threads,
1640                xdp_sender: xdp_sender.clone(),
1641            },
1642            &max_slots,
1643            block_metadata_notifier,
1644            config.wait_to_vote_slot,
1645            Some(snapshot_controller.clone()),
1646            config.runtime_config.log_messages_bytes_limit,
1647            connection_cache_for_warmup,
1648            &prioritization_fee_cache,
1649            banking_tracer.clone(),
1650            turbine_quic_endpoint_sender.clone(),
1651            turbine_quic_endpoint_receiver,
1652            repair_response_quic_receiver,
1653            repair_quic_async_senders.repair_request_quic_sender,
1654            repair_quic_async_senders.ancestor_hashes_request_quic_sender,
1655            ancestor_hashes_response_quic_receiver,
1656            outstanding_repair_requests.clone(),
1657            cluster_slots.clone(),
1658            wen_restart_repair_slots.clone(),
1659            slot_status_notifier,
1660            vote_connection_cache,
1661        )
1662        .map_err(ValidatorError::Other)?;
1663
1664        if in_wen_restart {
1665            info!("Waiting for wen_restart to finish");
1666            wait_for_wen_restart(WenRestartConfig {
1667                wen_restart_path: config.wen_restart_proto_path.clone().unwrap(),
1668                wen_restart_coordinator: config.wen_restart_coordinator.unwrap(),
1669                last_vote,
1670                blockstore: blockstore.clone(),
1671                cluster_info: cluster_info.clone(),
1672                bank_forks: bank_forks.clone(),
1673                wen_restart_repair_slots: wen_restart_repair_slots.clone(),
1674                wait_for_supermajority_threshold_percent:
1675                    WAIT_FOR_WEN_RESTART_SUPERMAJORITY_THRESHOLD_PERCENT,
1676                snapshot_controller: Some(snapshot_controller.clone()),
1677                abs_status: accounts_background_service.status().clone(),
1678                genesis_config_hash: genesis_config.hash(),
1679                exit: exit.clone(),
1680            })?;
1681            return Err(ValidatorError::WenRestartFinished.into());
1682        }
1683
1684        let key_notifiers = Arc::new(RwLock::new(KeyUpdaters::default()));
1685        let forwarding_tpu_client = if let Some(connection_cache) = &connection_cache {
1686            ForwardingClientOption::ConnectionCache(connection_cache.clone())
1687        } else {
1688            let runtime_handle = tpu_client_next_runtime
1689                .as_ref()
1690                .map(TokioRuntime::handle)
1691                .unwrap_or_else(|| current_runtime_handle.as_ref().unwrap());
1692            ForwardingClientOption::TpuClientNext((
1693                Arc::as_ref(&identity_keypair),
1694                tpu_transactions_forwards_client_sockets.take().unwrap(),
1695                runtime_handle.clone(),
1696                cancel.clone(),
1697                node_multihoming.clone(),
1698            ))
1699        };
1700        let tpu = Tpu::new_with_client(
1701            &cluster_info,
1702            &poh_recorder,
1703            transaction_recorder,
1704            entry_receiver,
1705            retransmit_slots_receiver,
1706            TpuSockets {
1707                transactions: node.sockets.tpu,
1708                transaction_forwards: node.sockets.tpu_forwards,
1709                vote: node.sockets.tpu_vote,
1710                broadcast: node.sockets.broadcast,
1711                transactions_quic: node.sockets.tpu_quic,
1712                transactions_forwards_quic: node.sockets.tpu_forwards_quic,
1713                vote_quic: node.sockets.tpu_vote_quic,
1714                vote_forwarding_client: node.sockets.tpu_vote_forwarding_client,
1715                vortexor_receivers: node.sockets.vortexor_receivers,
1716            },
1717            rpc_subscriptions.clone(),
1718            transaction_status_sender,
1719            entry_notification_sender,
1720            blockstore.clone(),
1721            &config.broadcast_stage_type,
1722            xdp_sender,
1723            exit,
1724            node.info.shred_version(),
1725            vote_tracker,
1726            bank_forks.clone(),
1727            verified_vote_sender,
1728            gossip_verified_vote_hash_sender,
1729            replay_vote_receiver,
1730            replay_vote_sender,
1731            bank_notification_sender,
1732            duplicate_confirmed_slot_sender,
1733            forwarding_tpu_client,
1734            turbine_quic_endpoint_sender,
1735            &identity_keypair,
1736            config.runtime_config.log_messages_bytes_limit,
1737            &staked_nodes,
1738            config.staked_nodes_overrides.clone(),
1739            banking_tracer_channels,
1740            tracer_thread,
1741            tpu_enable_udp,
1742            tpu_quic_server_config,
1743            tpu_fwd_quic_server_config,
1744            vote_quic_server_config,
1745            &prioritization_fee_cache,
1746            config.block_production_method.clone(),
1747            config.block_production_num_workers,
1748            config.block_production_scheduler_config.clone(),
1749            config.enable_block_production_forwarding,
1750            config.generator_config.clone(),
1751            key_notifiers.clone(),
1752            cancel,
1753        );
1754
1755        datapoint_info!(
1756            "validator-new",
1757            ("id", id.to_string(), String),
1758            ("version", solana_version::version!(), String),
1759            ("cluster_type", genesis_config.cluster_type as u32, i64),
1760            ("elapsed_ms", start_time.elapsed().as_millis() as i64, i64),
1761            ("waited_for_supermajority", waited_for_supermajority, bool),
1762            ("shred_version", shred_version as i64, i64),
1763        );
1764
1765        *start_progress.write().unwrap() = ValidatorStartProgress::Running;
1766        if config.use_tpu_client_next {
1767            if let Some(json_rpc_service) = &json_rpc_service {
1768                key_notifiers.write().unwrap().add(
1769                    KeyUpdaterType::RpcService,
1770                    json_rpc_service.get_client_key_updater(),
1771                );
1772            }
1773            // note, that we don't need to add ConnectionClient to key_notifiers
1774            // because it is added inside Tpu.
1775        }
1776
1777        *admin_rpc_service_post_init.write().unwrap() = Some(AdminRpcRequestMetadataPostInit {
1778            bank_forks: bank_forks.clone(),
1779            cluster_info: cluster_info.clone(),
1780            vote_account: *vote_account,
1781            repair_whitelist: config.repair_whitelist.clone(),
1782            notifies: key_notifiers,
1783            repair_socket: Arc::new(node.sockets.repair),
1784            outstanding_repair_requests,
1785            cluster_slots,
1786            node: Some(node_multihoming),
1787            banking_stage: tpu.banking_stage(),
1788        });
1789
1790        Ok(Self {
1791            stats_reporter_service,
1792            gossip_service,
1793            serve_repair_service,
1794            json_rpc_service,
1795            pubsub_service,
1796            rpc_completed_slots_service,
1797            optimistically_confirmed_bank_tracker,
1798            transaction_status_service,
1799            entry_notifier_service,
1800            system_monitor_service,
1801            sample_performance_service,
1802            snapshot_packager_service,
1803            completed_data_sets_service,
1804            tpu,
1805            tvu,
1806            poh_service,
1807            poh_recorder,
1808            ip_echo_server,
1809            validator_exit: config.validator_exit.clone(),
1810            cluster_info,
1811            bank_forks,
1812            blockstore,
1813            geyser_plugin_service,
1814            blockstore_metric_report_service,
1815            accounts_background_service,
1816            turbine_quic_endpoint,
1817            turbine_quic_endpoint_runtime,
1818            turbine_quic_endpoint_join_handle,
1819            repair_quic_endpoints,
1820            repair_quic_endpoints_runtime,
1821            repair_quic_endpoints_join_handle,
1822            xdp_retransmitter,
1823            _tpu_client_next_runtime: tpu_client_next_runtime,
1824        })
1825    }
1826
1827    // Used for notifying many nodes in parallel to exit
1828    pub fn exit(&mut self) {
1829        self.validator_exit.write().unwrap().exit();
1830
1831        // drop all signals in blockstore
1832        self.blockstore.drop_signal();
1833    }
1834
1835    pub fn close(mut self) {
1836        self.exit();
1837        self.join();
1838    }
1839
1840    fn print_node_info(node: &Node) {
1841        info!("{:?}", node.info);
1842        info!(
1843            "local gossip address: {}",
1844            node.sockets.gossip[0].local_addr().unwrap()
1845        );
1846        info!(
1847            "local broadcast address: {}",
1848            node.sockets
1849                .broadcast
1850                .first()
1851                .unwrap()
1852                .local_addr()
1853                .unwrap()
1854        );
1855        info!(
1856            "local repair address: {}",
1857            node.sockets.repair.local_addr().unwrap()
1858        );
1859        info!(
1860            "local retransmit address: {}",
1861            node.sockets.retransmit_sockets[0].local_addr().unwrap()
1862        );
1863    }
1864
1865    pub fn join(self) {
1866        drop(self.bank_forks);
1867        drop(self.cluster_info);
1868
1869        self.poh_service.join().expect("poh_service");
1870        drop(self.poh_recorder);
1871
1872        if let Some(json_rpc_service) = self.json_rpc_service {
1873            json_rpc_service.join().expect("rpc_service");
1874        }
1875
1876        if let Some(pubsub_service) = self.pubsub_service {
1877            pubsub_service.join().expect("pubsub_service");
1878        }
1879
1880        if let Some(rpc_completed_slots_service) = self.rpc_completed_slots_service {
1881            rpc_completed_slots_service
1882                .join()
1883                .expect("rpc_completed_slots_service");
1884        }
1885
1886        if let Some(optimistically_confirmed_bank_tracker) =
1887            self.optimistically_confirmed_bank_tracker
1888        {
1889            optimistically_confirmed_bank_tracker
1890                .join()
1891                .expect("optimistically_confirmed_bank_tracker");
1892        }
1893
1894        if let Some(transaction_status_service) = self.transaction_status_service {
1895            transaction_status_service
1896                .join()
1897                .expect("transaction_status_service");
1898        }
1899
1900        if let Some(system_monitor_service) = self.system_monitor_service {
1901            system_monitor_service
1902                .join()
1903                .expect("system_monitor_service");
1904        }
1905
1906        if let Some(sample_performance_service) = self.sample_performance_service {
1907            sample_performance_service
1908                .join()
1909                .expect("sample_performance_service");
1910        }
1911
1912        if let Some(entry_notifier_service) = self.entry_notifier_service {
1913            entry_notifier_service
1914                .join()
1915                .expect("entry_notifier_service");
1916        }
1917
1918        if let Some(s) = self.snapshot_packager_service {
1919            s.join().expect("snapshot_packager_service");
1920        }
1921
1922        self.gossip_service.join().expect("gossip_service");
1923        self.repair_quic_endpoints
1924            .iter()
1925            .flatten()
1926            .for_each(repair::quic_endpoint::close_quic_endpoint);
1927        self.serve_repair_service
1928            .join()
1929            .expect("serve_repair_service");
1930        if let Some(repair_quic_endpoints_join_handle) = self.repair_quic_endpoints_join_handle {
1931            self.repair_quic_endpoints_runtime
1932                .map(|runtime| runtime.block_on(repair_quic_endpoints_join_handle))
1933                .transpose()
1934                .unwrap();
1935        }
1936        self.stats_reporter_service
1937            .join()
1938            .expect("stats_reporter_service");
1939        self.blockstore_metric_report_service
1940            .join()
1941            .expect("ledger_metric_report_service");
1942        self.accounts_background_service
1943            .join()
1944            .expect("accounts_background_service");
1945        if let Some(turbine_quic_endpoint) = &self.turbine_quic_endpoint {
1946            solana_turbine::quic_endpoint::close_quic_endpoint(turbine_quic_endpoint);
1947        }
1948        if let Some(xdp_retransmitter) = self.xdp_retransmitter {
1949            xdp_retransmitter.join().expect("xdp_retransmitter");
1950        }
1951        self.tpu.join().expect("tpu");
1952        self.tvu.join().expect("tvu");
1953        if let Some(turbine_quic_endpoint_join_handle) = self.turbine_quic_endpoint_join_handle {
1954            self.turbine_quic_endpoint_runtime
1955                .map(|runtime| runtime.block_on(turbine_quic_endpoint_join_handle))
1956                .transpose()
1957                .unwrap();
1958        }
1959        if let Some(completed_data_sets_service) = self.completed_data_sets_service {
1960            completed_data_sets_service
1961                .join()
1962                .expect("completed_data_sets_service");
1963        }
1964        if let Some(ip_echo_server) = self.ip_echo_server {
1965            ip_echo_server.shutdown_background();
1966        }
1967
1968        if let Some(geyser_plugin_service) = self.geyser_plugin_service {
1969            geyser_plugin_service.join().expect("geyser_plugin_service");
1970        }
1971    }
1972}
1973
1974fn active_vote_account_exists_in_bank(bank: &Bank, vote_account: &Pubkey) -> bool {
1975    if let Some(account) = &bank.get_account(vote_account) {
1976        if let Ok(vote_state) = VoteStateV4::deserialize(account.data(), vote_account) {
1977            return !vote_state.votes.is_empty();
1978        }
1979    }
1980    false
1981}
1982
1983fn check_poh_speed(bank: &Bank, maybe_hash_samples: Option<u64>) -> Result<(), ValidatorError> {
1984    let Some(hashes_per_tick) = bank.hashes_per_tick() else {
1985        warn!("Unable to read hashes per tick from Bank, skipping PoH speed check");
1986        return Ok(());
1987    };
1988
1989    let ticks_per_slot = bank.ticks_per_slot();
1990    let hashes_per_slot = hashes_per_tick * ticks_per_slot;
1991    let hash_samples = maybe_hash_samples.unwrap_or(hashes_per_slot);
1992
1993    let hash_time = compute_hash_time(hash_samples);
1994    let my_hashes_per_second = (hash_samples as f64 / hash_time.as_secs_f64()) as u64;
1995
1996    let target_slot_duration = Duration::from_nanos(bank.ns_per_slot as u64);
1997    let target_hashes_per_second =
1998        (hashes_per_slot as f64 / target_slot_duration.as_secs_f64()) as u64;
1999
2000    info!(
2001        "PoH speed check: computed hashes per second {my_hashes_per_second}, target hashes per \
2002         second {target_hashes_per_second}"
2003    );
2004    if my_hashes_per_second < target_hashes_per_second {
2005        return Err(ValidatorError::PohTooSlow {
2006            mine: my_hashes_per_second,
2007            target: target_hashes_per_second,
2008        });
2009    }
2010
2011    Ok(())
2012}
2013
2014fn maybe_cluster_restart_with_hard_fork(config: &ValidatorConfig, root_slot: Slot) -> Option<Slot> {
2015    // detect cluster restart (hard fork) indirectly via wait_for_supermajority...
2016    if let Some(wait_slot_for_supermajority) = config.wait_for_supermajority {
2017        if wait_slot_for_supermajority == root_slot {
2018            return Some(wait_slot_for_supermajority);
2019        }
2020    }
2021
2022    None
2023}
2024
2025fn post_process_restored_tower(
2026    restored_tower: crate::consensus::Result<Tower>,
2027    validator_identity: &Pubkey,
2028    vote_account: &Pubkey,
2029    config: &ValidatorConfig,
2030    bank_forks: &BankForks,
2031) -> Result<Tower, String> {
2032    let mut should_require_tower = config.require_tower;
2033
2034    let restored_tower = restored_tower.and_then(|tower| {
2035        let root_bank = bank_forks.root_bank();
2036        let slot_history = root_bank.get_slot_history();
2037        // make sure tower isn't corrupted first before the following hard fork check
2038        let tower = tower.adjust_lockouts_after_replay(root_bank.slot(), &slot_history);
2039
2040        if let Some(hard_fork_restart_slot) =
2041            maybe_cluster_restart_with_hard_fork(config, root_bank.slot())
2042        {
2043            // intentionally fail to restore tower; we're supposedly in a new hard fork; past
2044            // out-of-chain vote state doesn't make sense at all
2045            // what if --wait-for-supermajority again if the validator restarted?
2046            let message =
2047                format!("Hard fork is detected; discarding tower restoration result: {tower:?}");
2048            datapoint_error!("tower_error", ("error", message, String),);
2049            error!("{message}");
2050
2051            // unconditionally relax tower requirement so that we can always restore tower
2052            // from root bank.
2053            should_require_tower = false;
2054            return Err(crate::consensus::TowerError::HardFork(
2055                hard_fork_restart_slot,
2056            ));
2057        }
2058
2059        if let Some(warp_slot) = config.warp_slot {
2060            // unconditionally relax tower requirement so that we can always restore tower
2061            // from root bank after the warp
2062            should_require_tower = false;
2063            return Err(crate::consensus::TowerError::HardFork(warp_slot));
2064        }
2065
2066        tower
2067    });
2068
2069    let restored_tower = match restored_tower {
2070        Ok(tower) => tower,
2071        Err(err) => {
2072            let voting_has_been_active =
2073                active_vote_account_exists_in_bank(&bank_forks.working_bank(), vote_account);
2074            if !err.is_file_missing() {
2075                datapoint_error!(
2076                    "tower_error",
2077                    ("error", format!("Unable to restore tower: {err}"), String),
2078                );
2079            }
2080            if should_require_tower && voting_has_been_active {
2081                return Err(format!(
2082                    "Requested mandatory tower restore failed: {err}. And there is an existing \
2083                     vote_account containing actual votes. Aborting due to possible conflicting \
2084                     duplicate votes"
2085                ));
2086            }
2087            if err.is_file_missing() && !voting_has_been_active {
2088                // Currently, don't protect against spoofed snapshots with no tower at all
2089                info!(
2090                    "Ignoring expected failed tower restore because this is the initial validator \
2091                     start with the vote account..."
2092                );
2093            } else {
2094                error!(
2095                    "Rebuilding a new tower from the latest vote account due to failed tower \
2096                     restore: {err}"
2097                );
2098            }
2099
2100            Tower::new_from_bankforks(bank_forks, validator_identity, vote_account)
2101        }
2102    };
2103
2104    Ok(restored_tower)
2105}
2106
2107fn load_genesis(
2108    config: &ValidatorConfig,
2109    ledger_path: &Path,
2110) -> Result<GenesisConfig, ValidatorError> {
2111    let genesis_config = open_genesis_config(ledger_path, config.max_genesis_archive_unpacked_size)
2112        .map_err(ValidatorError::OpenGenesisConfig)?;
2113
2114    // This needs to be limited otherwise the state in the VoteAccount data
2115    // grows too large
2116    let leader_schedule_slot_offset = genesis_config.epoch_schedule.leader_schedule_slot_offset;
2117    let slots_per_epoch = genesis_config.epoch_schedule.slots_per_epoch;
2118    let leader_epoch_offset = leader_schedule_slot_offset.div_ceil(slots_per_epoch);
2119    assert!(leader_epoch_offset <= MAX_LEADER_SCHEDULE_EPOCH_OFFSET);
2120
2121    let genesis_hash = genesis_config.hash();
2122    info!("genesis hash: {genesis_hash}");
2123
2124    if let Some(expected_genesis_hash) = config.expected_genesis_hash {
2125        if genesis_hash != expected_genesis_hash {
2126            return Err(ValidatorError::GenesisHashMismatch(
2127                genesis_hash,
2128                expected_genesis_hash,
2129            ));
2130        }
2131    }
2132
2133    Ok(genesis_config)
2134}
2135
2136#[allow(clippy::type_complexity)]
2137fn load_blockstore(
2138    config: &ValidatorConfig,
2139    ledger_path: &Path,
2140    genesis_config: &GenesisConfig,
2141    exit: Arc<AtomicBool>,
2142    start_progress: &Arc<RwLock<ValidatorStartProgress>>,
2143    accounts_update_notifier: Option<AccountsUpdateNotifier>,
2144    transaction_notifier: Option<TransactionNotifierArc>,
2145    entry_notifier: Option<EntryNotifierArc>,
2146    dependency_tracker: Option<Arc<DependencyTracker>>,
2147) -> Result<
2148    (
2149        Arc<RwLock<BankForks>>,
2150        Arc<Blockstore>,
2151        Slot,
2152        Receiver<bool>,
2153        LeaderScheduleCache,
2154        Option<StartingSnapshotHashes>,
2155        TransactionHistoryServices,
2156        blockstore_processor::ProcessOptions,
2157        BlockstoreRootScan,
2158        DroppedSlotsReceiver,
2159        Option<EntryNotifierService>,
2160    ),
2161    String,
2162> {
2163    info!("loading ledger from {ledger_path:?}...");
2164    *start_progress.write().unwrap() = ValidatorStartProgress::LoadingLedger;
2165
2166    let blockstore = Blockstore::open_with_options(ledger_path, config.blockstore_options.clone())
2167        .map_err(|err| format!("Failed to open Blockstore: {err:?}"))?;
2168
2169    let (ledger_signal_sender, ledger_signal_receiver) = bounded(MAX_REPLAY_WAKE_UP_SIGNALS);
2170    blockstore.add_new_shred_signal(ledger_signal_sender);
2171
2172    // following boot sequence (esp BankForks) could set root. so stash the original value
2173    // of blockstore root away here as soon as possible.
2174    let original_blockstore_root = blockstore.max_root();
2175
2176    let blockstore = Arc::new(blockstore);
2177    let blockstore_root_scan = BlockstoreRootScan::new(config, blockstore.clone(), exit.clone());
2178    let halt_at_slot = config
2179        .halt_at_slot
2180        .or_else(|| blockstore.highest_slot().unwrap_or(None));
2181
2182    let process_options = blockstore_processor::ProcessOptions {
2183        run_verification: config.run_verification,
2184        halt_at_slot,
2185        new_hard_forks: config.new_hard_forks.clone(),
2186        debug_keys: config.debug_keys.clone(),
2187        accounts_db_config: config.accounts_db_config.clone(),
2188        accounts_db_skip_shrink: config.accounts_db_skip_shrink,
2189        accounts_db_force_initial_clean: config.accounts_db_force_initial_clean,
2190        runtime_config: config.runtime_config.clone(),
2191        use_snapshot_archives_at_startup: config.use_snapshot_archives_at_startup,
2192        ..blockstore_processor::ProcessOptions::default()
2193    };
2194
2195    let enable_rpc_transaction_history =
2196        config.rpc_addrs.is_some() && config.rpc_config.enable_rpc_transaction_history;
2197    let is_plugin_transaction_history_required = transaction_notifier.as_ref().is_some();
2198    let transaction_history_services =
2199        if enable_rpc_transaction_history || is_plugin_transaction_history_required {
2200            initialize_rpc_transaction_history_services(
2201                blockstore.clone(),
2202                exit.clone(),
2203                enable_rpc_transaction_history,
2204                config.rpc_config.enable_extended_tx_metadata_storage,
2205                transaction_notifier,
2206                dependency_tracker,
2207            )
2208        } else {
2209            TransactionHistoryServices::default()
2210        };
2211
2212    let entry_notifier_service = entry_notifier
2213        .map(|entry_notifier| EntryNotifierService::new(entry_notifier, exit.clone()));
2214
2215    let (bank_forks, mut leader_schedule_cache, starting_snapshot_hashes) =
2216        bank_forks_utils::load_bank_forks(
2217            genesis_config,
2218            &blockstore,
2219            config.account_paths.clone(),
2220            &config.snapshot_config,
2221            &process_options,
2222            transaction_history_services
2223                .transaction_status_sender
2224                .as_ref(),
2225            entry_notifier_service
2226                .as_ref()
2227                .map(|service| service.sender()),
2228            accounts_update_notifier,
2229            exit,
2230        )
2231        .map_err(|err| err.to_string())?;
2232
2233    // Before replay starts, set the callbacks in each of the banks in BankForks so that
2234    // all dropped banks come through the `pruned_banks_receiver` channel. This way all bank
2235    // drop behavior can be safely synchronized with any other ongoing accounts activity like
2236    // cache flush, clean, shrink, as long as the same thread performing those activities also
2237    // is processing the dropped banks from the `pruned_banks_receiver` channel.
2238    let pruned_banks_receiver =
2239        AccountsBackgroundService::setup_bank_drop_callback(bank_forks.clone());
2240
2241    leader_schedule_cache.set_fixed_leader_schedule(config.fixed_leader_schedule.clone());
2242
2243    Ok((
2244        bank_forks,
2245        blockstore,
2246        original_blockstore_root,
2247        ledger_signal_receiver,
2248        leader_schedule_cache,
2249        starting_snapshot_hashes,
2250        transaction_history_services,
2251        process_options,
2252        blockstore_root_scan,
2253        pruned_banks_receiver,
2254        entry_notifier_service,
2255    ))
2256}
2257
2258pub struct ProcessBlockStore<'a> {
2259    id: &'a Pubkey,
2260    vote_account: &'a Pubkey,
2261    start_progress: &'a Arc<RwLock<ValidatorStartProgress>>,
2262    blockstore: &'a Blockstore,
2263    original_blockstore_root: Slot,
2264    bank_forks: &'a Arc<RwLock<BankForks>>,
2265    leader_schedule_cache: &'a LeaderScheduleCache,
2266    process_options: &'a blockstore_processor::ProcessOptions,
2267    transaction_status_sender: Option<&'a TransactionStatusSender>,
2268    entry_notification_sender: Option<&'a EntryNotifierSender>,
2269    blockstore_root_scan: Option<BlockstoreRootScan>,
2270    snapshot_controller: &'a SnapshotController,
2271    config: &'a ValidatorConfig,
2272    tower: Option<Tower>,
2273}
2274
2275impl<'a> ProcessBlockStore<'a> {
2276    #[allow(clippy::too_many_arguments)]
2277    fn new(
2278        id: &'a Pubkey,
2279        vote_account: &'a Pubkey,
2280        start_progress: &'a Arc<RwLock<ValidatorStartProgress>>,
2281        blockstore: &'a Blockstore,
2282        original_blockstore_root: Slot,
2283        bank_forks: &'a Arc<RwLock<BankForks>>,
2284        leader_schedule_cache: &'a LeaderScheduleCache,
2285        process_options: &'a blockstore_processor::ProcessOptions,
2286        transaction_status_sender: Option<&'a TransactionStatusSender>,
2287        entry_notification_sender: Option<&'a EntryNotifierSender>,
2288        blockstore_root_scan: BlockstoreRootScan,
2289        snapshot_controller: &'a SnapshotController,
2290        config: &'a ValidatorConfig,
2291    ) -> Self {
2292        Self {
2293            id,
2294            vote_account,
2295            start_progress,
2296            blockstore,
2297            original_blockstore_root,
2298            bank_forks,
2299            leader_schedule_cache,
2300            process_options,
2301            transaction_status_sender,
2302            entry_notification_sender,
2303            blockstore_root_scan: Some(blockstore_root_scan),
2304            snapshot_controller,
2305            config,
2306            tower: None,
2307        }
2308    }
2309
2310    pub(crate) fn process(&mut self) -> Result<(), String> {
2311        if self.tower.is_none() {
2312            let previous_start_process = *self.start_progress.read().unwrap();
2313            *self.start_progress.write().unwrap() = ValidatorStartProgress::LoadingLedger;
2314
2315            let exit = Arc::new(AtomicBool::new(false));
2316            if let Ok(Some(max_slot)) = self.blockstore.highest_slot() {
2317                let bank_forks = self.bank_forks.clone();
2318                let exit = exit.clone();
2319                let start_progress = self.start_progress.clone();
2320
2321                let _ = Builder::new()
2322                    .name("solRptLdgrStat".to_string())
2323                    .spawn(move || {
2324                        while !exit.load(Ordering::Relaxed) {
2325                            let slot = bank_forks.read().unwrap().working_bank().slot();
2326                            *start_progress.write().unwrap() =
2327                                ValidatorStartProgress::ProcessingLedger { slot, max_slot };
2328                            sleep(Duration::from_secs(2));
2329                        }
2330                    })
2331                    .unwrap();
2332            }
2333            blockstore_processor::process_blockstore_from_root(
2334                self.blockstore,
2335                self.bank_forks,
2336                self.leader_schedule_cache,
2337                self.process_options,
2338                self.transaction_status_sender,
2339                self.entry_notification_sender,
2340                Some(self.snapshot_controller),
2341            )
2342            .map_err(|err| {
2343                exit.store(true, Ordering::Relaxed);
2344                format!("Failed to load ledger: {err:?}")
2345            })?;
2346            exit.store(true, Ordering::Relaxed);
2347
2348            if let Some(blockstore_root_scan) = self.blockstore_root_scan.take() {
2349                blockstore_root_scan.join();
2350            }
2351
2352            self.tower = Some({
2353                let restored_tower = Tower::restore(self.config.tower_storage.as_ref(), self.id);
2354                if let Ok(tower) = &restored_tower {
2355                    // reconciliation attempt 1 of 2 with tower
2356                    reconcile_blockstore_roots_with_external_source(
2357                        ExternalRootSource::Tower(tower.root()),
2358                        self.blockstore,
2359                        &mut self.original_blockstore_root,
2360                    )
2361                    .map_err(|err| format!("Failed to reconcile blockstore with tower: {err:?}"))?;
2362                }
2363
2364                post_process_restored_tower(
2365                    restored_tower,
2366                    self.id,
2367                    self.vote_account,
2368                    self.config,
2369                    &self.bank_forks.read().unwrap(),
2370                )?
2371            });
2372
2373            if let Some(hard_fork_restart_slot) = maybe_cluster_restart_with_hard_fork(
2374                self.config,
2375                self.bank_forks.read().unwrap().root(),
2376            ) {
2377                // reconciliation attempt 2 of 2 with hard fork
2378                // this should be #2 because hard fork root > tower root in almost all cases
2379                reconcile_blockstore_roots_with_external_source(
2380                    ExternalRootSource::HardFork(hard_fork_restart_slot),
2381                    self.blockstore,
2382                    &mut self.original_blockstore_root,
2383                )
2384                .map_err(|err| format!("Failed to reconcile blockstore with hard fork: {err:?}"))?;
2385            }
2386
2387            *self.start_progress.write().unwrap() = previous_start_process;
2388        }
2389        Ok(())
2390    }
2391
2392    pub(crate) fn process_to_create_tower(mut self) -> Result<Tower, String> {
2393        self.process()?;
2394        Ok(self.tower.unwrap())
2395    }
2396}
2397
2398fn maybe_warp_slot(
2399    config: &ValidatorConfig,
2400    process_blockstore: &mut ProcessBlockStore,
2401    ledger_path: &Path,
2402    bank_forks: &RwLock<BankForks>,
2403    leader_schedule_cache: &LeaderScheduleCache,
2404    snapshot_controller: &SnapshotController,
2405) -> Result<(), String> {
2406    if let Some(warp_slot) = config.warp_slot {
2407        let mut bank_forks = bank_forks.write().unwrap();
2408
2409        let working_bank = bank_forks.working_bank();
2410
2411        if warp_slot <= working_bank.slot() {
2412            return Err(format!(
2413                "warp slot ({}) cannot be less than the working bank slot ({})",
2414                warp_slot,
2415                working_bank.slot()
2416            ));
2417        }
2418        info!("warping to slot {warp_slot}");
2419
2420        let root_bank = bank_forks.root_bank();
2421
2422        // An accounts hash calculation from storages will occur in warp_from_parent() below.  This
2423        // requires that the accounts cache has been flushed, which requires the parent slot to be
2424        // rooted.
2425        root_bank.squash();
2426        root_bank.force_flush_accounts_cache();
2427
2428        bank_forks.insert(Bank::warp_from_parent(
2429            root_bank,
2430            &Pubkey::default(),
2431            warp_slot,
2432        ));
2433        bank_forks.set_root(warp_slot, Some(snapshot_controller), Some(warp_slot));
2434        leader_schedule_cache.set_root(&bank_forks.root_bank());
2435
2436        let full_snapshot_archive_info = match snapshot_bank_utils::bank_to_full_snapshot_archive(
2437            ledger_path,
2438            &bank_forks.root_bank(),
2439            None,
2440            &config.snapshot_config.full_snapshot_archives_dir,
2441            &config.snapshot_config.incremental_snapshot_archives_dir,
2442            config.snapshot_config.archive_format,
2443        ) {
2444            Ok(archive_info) => archive_info,
2445            Err(e) => return Err(format!("Unable to create snapshot: {e}")),
2446        };
2447        info!(
2448            "created snapshot: {}",
2449            full_snapshot_archive_info.path().display()
2450        );
2451
2452        drop(bank_forks);
2453        // Process blockstore after warping bank forks to make sure tower and
2454        // bank forks are in sync.
2455        process_blockstore.process()?;
2456    }
2457    Ok(())
2458}
2459
2460/// Returns the starting slot at which the blockstore should be scanned for
2461/// shreds with an incorrect shred version, or None if the check is unnecessary
2462fn should_cleanup_blockstore_incorrect_shred_versions(
2463    config: &ValidatorConfig,
2464    blockstore: &Blockstore,
2465    root_slot: Slot,
2466    hard_forks: &HardForks,
2467) -> Result<Option<Slot>, BlockstoreError> {
2468    // Perform the check if we are booting as part of a cluster restart at slot root_slot
2469    let maybe_cluster_restart_slot = maybe_cluster_restart_with_hard_fork(config, root_slot);
2470    if maybe_cluster_restart_slot.is_some() {
2471        return Ok(Some(root_slot + 1));
2472    }
2473
2474    // If there are no hard forks, the shred version cannot have changed
2475    let Some(latest_hard_fork) = hard_forks.iter().last().map(|(slot, _)| *slot) else {
2476        return Ok(None);
2477    };
2478
2479    // If the blockstore is empty, there are certainly no shreds with an incorrect version
2480    let Some(blockstore_max_slot) = blockstore.highest_slot()? else {
2481        return Ok(None);
2482    };
2483    let blockstore_min_slot = blockstore.lowest_slot();
2484    info!(
2485        "Blockstore contains data from slot {blockstore_min_slot} to {blockstore_max_slot}, the \
2486         latest hard fork is {latest_hard_fork}"
2487    );
2488
2489    if latest_hard_fork < blockstore_min_slot {
2490        // latest_hard_fork < blockstore_min_slot <= blockstore_max_slot
2491        //
2492        // All slots in the blockstore are newer than the latest hard fork, and only shreds with
2493        // the correct shred version should have been inserted since the latest hard fork
2494        //
2495        // This is the normal case where the last cluster restart & hard fork was a while ago; we
2496        // can skip the check for this case
2497        Ok(None)
2498    } else if latest_hard_fork < blockstore_max_slot {
2499        // blockstore_min_slot < latest_hard_fork < blockstore_max_slot
2500        //
2501        // This could be a case where there was a cluster restart, but this node was not part of
2502        // the supermajority that actually restarted the cluster. Rather, this node likely
2503        // downloaded a new snapshot while retaining the blockstore, including slots beyond the
2504        // chosen restart slot. We need to perform the blockstore check for this case
2505        //
2506        // Note that the downloaded snapshot slot (root_slot) could be greater than the latest hard
2507        // fork slot. Even though this node will only replay slots after root_slot, start the check
2508        // at latest_hard_fork + 1 to check (and possibly purge) any invalid state.
2509        Ok(Some(latest_hard_fork + 1))
2510    } else {
2511        // blockstore_min_slot <= blockstore_max_slot <= latest_hard_fork
2512        //
2513        // All slots in the blockstore are older than the latest hard fork. The blockstore check
2514        // would start from latest_hard_fork + 1; skip the check as there are no slots to check
2515        //
2516        // This is kind of an unusual case to hit, maybe a node has been offline for a long time
2517        // and just restarted with a new downloaded snapshot but the old blockstore
2518        Ok(None)
2519    }
2520}
2521
2522/// Searches the blockstore for data shreds with a shred version that differs
2523/// from the passed `expected_shred_version`
2524fn scan_blockstore_for_incorrect_shred_version(
2525    blockstore: &Blockstore,
2526    start_slot: Slot,
2527    expected_shred_version: u16,
2528) -> Result<Option<u16>, BlockstoreError> {
2529    const TIMEOUT: Duration = Duration::from_secs(60);
2530    let timer = Instant::now();
2531    // Search for shreds with incompatible version in blockstore
2532    let slot_meta_iterator = blockstore.slot_meta_iterator(start_slot)?;
2533
2534    info!("Searching blockstore for shred with incorrect version from slot {start_slot}");
2535    for (slot, _meta) in slot_meta_iterator {
2536        let shreds = blockstore.get_data_shreds_for_slot(slot, 0)?;
2537        for shred in &shreds {
2538            if shred.version() != expected_shred_version {
2539                return Ok(Some(shred.version()));
2540            }
2541        }
2542        if timer.elapsed() > TIMEOUT {
2543            info!("Didn't find incorrect shreds after 60 seconds, aborting");
2544            break;
2545        }
2546    }
2547    Ok(None)
2548}
2549
2550/// If the blockstore contains any shreds with the incorrect shred version,
2551/// copy them to a backup blockstore and purge them from the actual blockstore.
2552fn cleanup_blockstore_incorrect_shred_versions(
2553    blockstore: &Blockstore,
2554    config: &ValidatorConfig,
2555    start_slot: Slot,
2556    expected_shred_version: u16,
2557) -> Result<(), BlockstoreError> {
2558    let incorrect_shred_version = scan_blockstore_for_incorrect_shred_version(
2559        blockstore,
2560        start_slot,
2561        expected_shred_version,
2562    )?;
2563    let Some(incorrect_shred_version) = incorrect_shred_version else {
2564        info!("Only shreds with the correct version were found in the blockstore");
2565        return Ok(());
2566    };
2567
2568    // .unwrap() safe because getting to this point implies blockstore has slots/shreds
2569    let end_slot = blockstore.highest_slot()?.unwrap();
2570
2571    // Backing up the shreds that will be deleted from primary blockstore is
2572    // not critical, so swallow errors from backup blockstore operations.
2573    let backup_folder = format!(
2574        "{BLOCKSTORE_DIRECTORY_ROCKS_LEVEL}_backup_{incorrect_shred_version}_{start_slot}_{end_slot}"
2575    );
2576    match Blockstore::open_with_options(
2577        &blockstore.ledger_path().join(backup_folder),
2578        config.blockstore_options.clone(),
2579    ) {
2580        Ok(backup_blockstore) => {
2581            info!("Backing up slots from {start_slot} to {end_slot}");
2582            let mut timer = Measure::start("blockstore backup");
2583
2584            const PRINT_INTERVAL: Duration = Duration::from_secs(5);
2585            let mut print_timer = Instant::now();
2586            let mut num_slots_copied = 0;
2587            let slot_meta_iterator = blockstore.slot_meta_iterator(start_slot)?;
2588            for (slot, _meta) in slot_meta_iterator {
2589                let shreds = blockstore.get_data_shreds_for_slot(slot, 0)?;
2590                let shreds = shreds.into_iter().map(Cow::Owned);
2591                let _ = backup_blockstore.insert_cow_shreds(shreds, None, true);
2592                num_slots_copied += 1;
2593
2594                if print_timer.elapsed() > PRINT_INTERVAL {
2595                    info!("Backed up {num_slots_copied} slots thus far");
2596                    print_timer = Instant::now();
2597                }
2598            }
2599
2600            timer.stop();
2601            info!("Backing up slots done. {timer}");
2602        }
2603        Err(err) => {
2604            warn!("Unable to backup shreds with incorrect shred version: {err}");
2605        }
2606    }
2607
2608    info!("Purging slots {start_slot} to {end_slot} from blockstore");
2609    let mut timer = Measure::start("blockstore purge");
2610    blockstore.purge_from_next_slots(start_slot, end_slot);
2611    blockstore.purge_slots(start_slot, end_slot, PurgeType::Exact);
2612    timer.stop();
2613    info!("Purging slots done. {timer}");
2614
2615    Ok(())
2616}
2617
2618fn initialize_rpc_transaction_history_services(
2619    blockstore: Arc<Blockstore>,
2620    exit: Arc<AtomicBool>,
2621    enable_rpc_transaction_history: bool,
2622    enable_extended_tx_metadata_storage: bool,
2623    transaction_notifier: Option<TransactionNotifierArc>,
2624    dependency_tracker: Option<Arc<DependencyTracker>>,
2625) -> TransactionHistoryServices {
2626    let max_complete_transaction_status_slot = Arc::new(AtomicU64::new(blockstore.max_root()));
2627    let (transaction_status_sender, transaction_status_receiver) = unbounded();
2628    let transaction_status_sender = Some(TransactionStatusSender {
2629        sender: transaction_status_sender,
2630        dependency_tracker: dependency_tracker.clone(),
2631    });
2632    let transaction_status_service = Some(TransactionStatusService::new(
2633        transaction_status_receiver,
2634        max_complete_transaction_status_slot.clone(),
2635        enable_rpc_transaction_history,
2636        transaction_notifier,
2637        blockstore.clone(),
2638        enable_extended_tx_metadata_storage,
2639        dependency_tracker,
2640        exit.clone(),
2641    ));
2642
2643    TransactionHistoryServices {
2644        transaction_status_sender,
2645        transaction_status_service,
2646        max_complete_transaction_status_slot,
2647    }
2648}
2649
2650#[derive(Error, Debug)]
2651pub enum ValidatorError {
2652    #[error("bank hash mismatch: actual={0}, expected={1}")]
2653    BankHashMismatch(Hash, Hash),
2654
2655    #[error("blockstore error: {0}")]
2656    Blockstore(#[source] BlockstoreError),
2657
2658    #[error("genesis hash mismatch: actual={0}, expected={1}")]
2659    GenesisHashMismatch(Hash, Hash),
2660
2661    #[error(
2662        "ledger does not have enough data to wait for supermajority: current slot={0}, needed \
2663         slot={1}"
2664    )]
2665    NotEnoughLedgerData(Slot, Slot),
2666
2667    #[error("failed to open genesis: {0}")]
2668    OpenGenesisConfig(#[source] OpenGenesisConfigError),
2669
2670    #[error("{0}")]
2671    Other(String),
2672
2673    #[error(
2674        "PoH hashes/second rate is slower than the cluster target: mine {mine}, cluster {target}"
2675    )]
2676    PohTooSlow { mine: u64, target: u64 },
2677
2678    #[error(transparent)]
2679    ResourceLimitError(#[from] ResourceLimitError),
2680
2681    #[error("shred version mismatch: actual {actual}, expected {expected}")]
2682    ShredVersionMismatch { actual: u16, expected: u16 },
2683
2684    #[error(transparent)]
2685    TraceError(#[from] TraceError),
2686
2687    #[error("Wen Restart finished, please continue with --wait-for-supermajority")]
2688    WenRestartFinished,
2689}
2690
2691// Return if the validator waited on other nodes to start. In this case
2692// it should not wait for one of it's votes to land to produce blocks
2693// because if the whole network is waiting, then it will stall.
2694//
2695// Error indicates that a bad hash was encountered or another condition
2696// that is unrecoverable and the validator should exit.
2697fn wait_for_supermajority(
2698    config: &ValidatorConfig,
2699    process_blockstore: Option<&mut ProcessBlockStore>,
2700    bank_forks: &RwLock<BankForks>,
2701    cluster_info: &ClusterInfo,
2702    rpc_override_health_check: Arc<AtomicBool>,
2703    start_progress: &Arc<RwLock<ValidatorStartProgress>>,
2704) -> Result<bool, ValidatorError> {
2705    match config.wait_for_supermajority {
2706        None => Ok(false),
2707        Some(wait_for_supermajority_slot) => {
2708            if let Some(process_blockstore) = process_blockstore {
2709                process_blockstore
2710                    .process()
2711                    .map_err(ValidatorError::Other)?;
2712            }
2713
2714            let bank = bank_forks.read().unwrap().working_bank();
2715            match wait_for_supermajority_slot.cmp(&bank.slot()) {
2716                std::cmp::Ordering::Less => return Ok(false),
2717                std::cmp::Ordering::Greater => {
2718                    return Err(ValidatorError::NotEnoughLedgerData(
2719                        bank.slot(),
2720                        wait_for_supermajority_slot,
2721                    ));
2722                }
2723                _ => {}
2724            }
2725
2726            if let Some(expected_bank_hash) = config.expected_bank_hash {
2727                if bank.hash() != expected_bank_hash {
2728                    return Err(ValidatorError::BankHashMismatch(
2729                        bank.hash(),
2730                        expected_bank_hash,
2731                    ));
2732                }
2733            }
2734
2735            for i in 1.. {
2736                let logging = i % 10 == 1;
2737                if logging {
2738                    info!(
2739                        "Waiting for {}% of activated stake at slot {} to be in gossip...",
2740                        WAIT_FOR_SUPERMAJORITY_THRESHOLD_PERCENT,
2741                        bank.slot()
2742                    );
2743                }
2744
2745                let gossip_stake_percent =
2746                    get_stake_percent_in_gossip(&bank, cluster_info, logging);
2747
2748                *start_progress.write().unwrap() =
2749                    ValidatorStartProgress::WaitingForSupermajority {
2750                        slot: wait_for_supermajority_slot,
2751                        gossip_stake_percent,
2752                    };
2753
2754                if gossip_stake_percent >= WAIT_FOR_SUPERMAJORITY_THRESHOLD_PERCENT {
2755                    info!(
2756                        "Supermajority reached, {gossip_stake_percent}% active stake detected, \
2757                         starting up now.",
2758                    );
2759                    break;
2760                }
2761                // The normal RPC health checks don't apply as the node is waiting, so feign health to
2762                // prevent load balancers from removing the node from their list of candidates during a
2763                // manual restart.
2764                rpc_override_health_check.store(true, Ordering::Relaxed);
2765                sleep(Duration::new(1, 0));
2766            }
2767            rpc_override_health_check.store(false, Ordering::Relaxed);
2768            Ok(true)
2769        }
2770    }
2771}
2772
2773// Get the activated stake percentage (based on the provided bank) that is visible in gossip
2774fn get_stake_percent_in_gossip(bank: &Bank, cluster_info: &ClusterInfo, log: bool) -> u64 {
2775    let mut online_stake = 0;
2776    let mut wrong_shred_stake = 0;
2777    let mut wrong_shred_nodes = vec![];
2778    let mut offline_stake = 0;
2779    let mut offline_nodes = vec![];
2780
2781    let mut total_activated_stake = 0;
2782    let now = timestamp();
2783    // Nodes contact infos are saved to disk and restored on validator startup.
2784    // Staked nodes entries will not expire until an epoch after. So it
2785    // is necessary here to filter for recent entries to establish liveness.
2786    let peers: HashMap<_, _> = cluster_info
2787        .tvu_peers(ContactInfo::clone)
2788        .into_iter()
2789        .filter(|node| {
2790            let age = now.saturating_sub(node.wallclock());
2791            // Contact infos are refreshed twice during this period.
2792            age < CRDS_GOSSIP_PULL_CRDS_TIMEOUT_MS
2793        })
2794        .map(|node| (*node.pubkey(), node))
2795        .collect();
2796    let my_shred_version = cluster_info.my_shred_version();
2797    let my_id = cluster_info.id();
2798
2799    for (activated_stake, vote_account) in bank.vote_accounts().values() {
2800        let activated_stake = *activated_stake;
2801        total_activated_stake += activated_stake;
2802
2803        if activated_stake == 0 {
2804            continue;
2805        }
2806        let vote_state_node_pubkey = *vote_account.node_pubkey();
2807
2808        if let Some(peer) = peers.get(&vote_state_node_pubkey) {
2809            if peer.shred_version() == my_shred_version {
2810                trace!(
2811                    "observed {vote_state_node_pubkey} in gossip, \
2812                     (activated_stake={activated_stake})"
2813                );
2814                online_stake += activated_stake;
2815            } else {
2816                wrong_shred_stake += activated_stake;
2817                wrong_shred_nodes.push((activated_stake, vote_state_node_pubkey));
2818            }
2819        } else if vote_state_node_pubkey == my_id {
2820            online_stake += activated_stake; // This node is online
2821        } else {
2822            offline_stake += activated_stake;
2823            offline_nodes.push((activated_stake, vote_state_node_pubkey));
2824        }
2825    }
2826
2827    let online_stake_percentage = (online_stake as f64 / total_activated_stake as f64) * 100.;
2828    if log {
2829        info!("{online_stake_percentage:.3}% of active stake visible in gossip");
2830
2831        if !wrong_shred_nodes.is_empty() {
2832            info!(
2833                "{:.3}% of active stake has the wrong shred version in gossip",
2834                (wrong_shred_stake as f64 / total_activated_stake as f64) * 100.,
2835            );
2836            wrong_shred_nodes.sort_by(|b, a| a.0.cmp(&b.0)); // sort by reverse stake weight
2837            for (stake, identity) in wrong_shred_nodes {
2838                info!(
2839                    "    {:.3}% - {}",
2840                    (stake as f64 / total_activated_stake as f64) * 100.,
2841                    identity
2842                );
2843            }
2844        }
2845
2846        if !offline_nodes.is_empty() {
2847            info!(
2848                "{:.3}% of active stake is not visible in gossip",
2849                (offline_stake as f64 / total_activated_stake as f64) * 100.
2850            );
2851            offline_nodes.sort_by(|b, a| a.0.cmp(&b.0)); // sort by reverse stake weight
2852            for (stake, identity) in offline_nodes {
2853                info!(
2854                    "    {:.3}% - {}",
2855                    (stake as f64 / total_activated_stake as f64) * 100.,
2856                    identity
2857                );
2858            }
2859        }
2860    }
2861
2862    online_stake_percentage as u64
2863}
2864
2865fn cleanup_accounts_paths(config: &ValidatorConfig) {
2866    for account_path in &config.account_paths {
2867        move_and_async_delete_path_contents(account_path);
2868    }
2869    if let Some(shrink_paths) = &config.accounts_db_config.shrink_paths {
2870        for shrink_path in shrink_paths {
2871            move_and_async_delete_path_contents(shrink_path);
2872        }
2873    }
2874}
2875
2876pub fn is_snapshot_config_valid(snapshot_config: &SnapshotConfig) -> bool {
2877    // if the snapshot config is configured to *not* take snapshots, then it is valid
2878    if !snapshot_config.should_generate_snapshots() {
2879        return true;
2880    }
2881
2882    let SnapshotInterval::Slots(full_snapshot_interval_slots) =
2883        snapshot_config.full_snapshot_archive_interval
2884    else {
2885        // if we *are* generating snapshots, then the full snapshot interval cannot be disabled
2886        return false;
2887    };
2888
2889    match snapshot_config.incremental_snapshot_archive_interval {
2890        SnapshotInterval::Disabled => true,
2891        SnapshotInterval::Slots(incremental_snapshot_interval_slots) => {
2892            full_snapshot_interval_slots > incremental_snapshot_interval_slots
2893        }
2894    }
2895}
2896
2897#[cfg(test)]
2898mod tests {
2899    use {
2900        super::*,
2901        crossbeam_channel::{bounded, RecvTimeoutError},
2902        solana_entry::entry,
2903        solana_genesis_config::create_genesis_config,
2904        solana_gossip::contact_info::ContactInfo,
2905        solana_ledger::{
2906            blockstore, create_new_tmp_ledger, genesis_utils::create_genesis_config_with_leader,
2907            get_tmp_ledger_path_auto_delete,
2908        },
2909        solana_poh_config::PohConfig,
2910        solana_sha256_hasher::hash,
2911        solana_tpu_client::tpu_client::DEFAULT_TPU_ENABLE_UDP,
2912        std::{fs::remove_dir_all, num::NonZeroU64, thread, time::Duration},
2913    };
2914
2915    #[test]
2916    fn validator_exit() {
2917        agave_logger::setup();
2918        let leader_keypair = Keypair::new();
2919        let leader_node = Node::new_localhost_with_pubkey(&leader_keypair.pubkey());
2920
2921        let validator_keypair = Keypair::new();
2922        let validator_node = Node::new_localhost_with_pubkey(&validator_keypair.pubkey());
2923        let genesis_config =
2924            create_genesis_config_with_leader(10_000, &leader_keypair.pubkey(), 1000)
2925                .genesis_config;
2926        let (validator_ledger_path, _blockhash) = create_new_tmp_ledger!(&genesis_config);
2927
2928        let voting_keypair = Arc::new(Keypair::new());
2929        let config = ValidatorConfig {
2930            rpc_addrs: Some((
2931                validator_node.info.rpc().unwrap(),
2932                validator_node.info.rpc_pubsub().unwrap(),
2933            )),
2934            ..ValidatorConfig::default_for_test()
2935        };
2936        let start_progress = Arc::new(RwLock::new(ValidatorStartProgress::default()));
2937        let validator = Validator::new(
2938            validator_node,
2939            Arc::new(validator_keypair),
2940            &validator_ledger_path,
2941            &voting_keypair.pubkey(),
2942            Arc::new(RwLock::new(vec![voting_keypair])),
2943            vec![leader_node.info],
2944            &config,
2945            true, // should_check_duplicate_instance
2946            None, // rpc_to_plugin_manager_receiver
2947            start_progress.clone(),
2948            SocketAddrSpace::Unspecified,
2949            ValidatorTpuConfig::new_for_tests(DEFAULT_TPU_ENABLE_UDP),
2950            Arc::new(RwLock::new(None)),
2951        )
2952        .expect("assume successful validator start");
2953        assert_eq!(
2954            *start_progress.read().unwrap(),
2955            ValidatorStartProgress::Running
2956        );
2957        validator.close();
2958        remove_dir_all(validator_ledger_path).unwrap();
2959    }
2960
2961    #[test]
2962    fn test_should_cleanup_blockstore_incorrect_shred_versions() {
2963        agave_logger::setup();
2964
2965        let ledger_path = get_tmp_ledger_path_auto_delete!();
2966        let blockstore = Blockstore::open(ledger_path.path()).unwrap();
2967
2968        let mut validator_config = ValidatorConfig::default_for_test();
2969        let mut hard_forks = HardForks::default();
2970        let mut root_slot;
2971
2972        // Do check from root_slot + 1 if wait_for_supermajority (10) == root_slot (10)
2973        root_slot = 10;
2974        validator_config.wait_for_supermajority = Some(root_slot);
2975        assert_eq!(
2976            should_cleanup_blockstore_incorrect_shred_versions(
2977                &validator_config,
2978                &blockstore,
2979                root_slot,
2980                &hard_forks
2981            )
2982            .unwrap(),
2983            Some(root_slot + 1)
2984        );
2985
2986        // No check if wait_for_supermajority (10) < root_slot (15) (no hard forks)
2987        // Arguably operator error to pass a value for wait_for_supermajority in this case
2988        root_slot = 15;
2989        assert_eq!(
2990            should_cleanup_blockstore_incorrect_shred_versions(
2991                &validator_config,
2992                &blockstore,
2993                root_slot,
2994                &hard_forks
2995            )
2996            .unwrap(),
2997            None,
2998        );
2999
3000        // Emulate cluster restart at slot 10
3001        // No check if wait_for_supermajority (10) < root_slot (15) (empty blockstore)
3002        hard_forks.register(10);
3003        assert_eq!(
3004            should_cleanup_blockstore_incorrect_shred_versions(
3005                &validator_config,
3006                &blockstore,
3007                root_slot,
3008                &hard_forks
3009            )
3010            .unwrap(),
3011            None,
3012        );
3013
3014        // Insert some shreds at newer slots than hard fork
3015        let entries = entry::create_ticks(1, 0, Hash::default());
3016        for i in 20..35 {
3017            let shreds = blockstore::entries_to_test_shreds(
3018                &entries,
3019                i,     // slot
3020                i - 1, // parent_slot
3021                true,  // is_full_slot
3022                1,     // version
3023            );
3024            blockstore.insert_shreds(shreds, None, true).unwrap();
3025        }
3026
3027        // No check as all blockstore data is newer than latest hard fork
3028        assert_eq!(
3029            should_cleanup_blockstore_incorrect_shred_versions(
3030                &validator_config,
3031                &blockstore,
3032                root_slot,
3033                &hard_forks
3034            )
3035            .unwrap(),
3036            None,
3037        );
3038
3039        // Emulate cluster restart at slot 25
3040        // Do check from root_slot + 1 regardless of whether wait_for_supermajority set correctly
3041        root_slot = 25;
3042        hard_forks.register(root_slot);
3043        validator_config.wait_for_supermajority = Some(root_slot);
3044        assert_eq!(
3045            should_cleanup_blockstore_incorrect_shred_versions(
3046                &validator_config,
3047                &blockstore,
3048                root_slot,
3049                &hard_forks
3050            )
3051            .unwrap(),
3052            Some(root_slot + 1),
3053        );
3054        validator_config.wait_for_supermajority = None;
3055        assert_eq!(
3056            should_cleanup_blockstore_incorrect_shred_versions(
3057                &validator_config,
3058                &blockstore,
3059                root_slot,
3060                &hard_forks
3061            )
3062            .unwrap(),
3063            Some(root_slot + 1),
3064        );
3065
3066        // Do check with advanced root slot, even without wait_for_supermajority set correctly
3067        // Check starts from latest hard fork + 1
3068        root_slot = 30;
3069        let latest_hard_fork = hard_forks.iter().last().unwrap().0;
3070        assert_eq!(
3071            should_cleanup_blockstore_incorrect_shred_versions(
3072                &validator_config,
3073                &blockstore,
3074                root_slot,
3075                &hard_forks
3076            )
3077            .unwrap(),
3078            Some(latest_hard_fork + 1),
3079        );
3080
3081        // Purge blockstore up to latest hard fork
3082        // No check since all blockstore data newer than latest hard fork
3083        blockstore.purge_slots(0, latest_hard_fork, PurgeType::Exact);
3084        assert_eq!(
3085            should_cleanup_blockstore_incorrect_shred_versions(
3086                &validator_config,
3087                &blockstore,
3088                root_slot,
3089                &hard_forks
3090            )
3091            .unwrap(),
3092            None,
3093        );
3094    }
3095
3096    #[test]
3097    fn test_cleanup_blockstore_incorrect_shred_versions() {
3098        agave_logger::setup();
3099
3100        let validator_config = ValidatorConfig::default_for_test();
3101        let ledger_path = get_tmp_ledger_path_auto_delete!();
3102        let blockstore = Blockstore::open(ledger_path.path()).unwrap();
3103
3104        let entries = entry::create_ticks(1, 0, Hash::default());
3105        for i in 1..10 {
3106            let shreds = blockstore::entries_to_test_shreds(
3107                &entries,
3108                i,     // slot
3109                i - 1, // parent_slot
3110                true,  // is_full_slot
3111                1,     // version
3112            );
3113            blockstore.insert_shreds(shreds, None, true).unwrap();
3114        }
3115
3116        // this purges and compacts all slots greater than or equal to 5
3117        cleanup_blockstore_incorrect_shred_versions(&blockstore, &validator_config, 5, 2).unwrap();
3118        // assert that slots less than 5 aren't affected
3119        assert!(blockstore.meta(4).unwrap().unwrap().next_slots.is_empty());
3120        for i in 5..10 {
3121            assert!(blockstore
3122                .get_data_shreds_for_slot(i, 0)
3123                .unwrap()
3124                .is_empty());
3125        }
3126    }
3127
3128    #[test]
3129    fn validator_parallel_exit() {
3130        let leader_keypair = Keypair::new();
3131        let leader_node = Node::new_localhost_with_pubkey(&leader_keypair.pubkey());
3132        let genesis_config =
3133            create_genesis_config_with_leader(10_000, &leader_keypair.pubkey(), 1000)
3134                .genesis_config;
3135
3136        let mut ledger_paths = vec![];
3137        let mut validators: Vec<Validator> = (0..2)
3138            .map(|_| {
3139                let validator_keypair = Keypair::new();
3140                let validator_node = Node::new_localhost_with_pubkey(&validator_keypair.pubkey());
3141                let (validator_ledger_path, _blockhash) = create_new_tmp_ledger!(&genesis_config);
3142                ledger_paths.push(validator_ledger_path.clone());
3143                let vote_account_keypair = Keypair::new();
3144                let config = ValidatorConfig {
3145                    rpc_addrs: Some((
3146                        validator_node.info.rpc().unwrap(),
3147                        validator_node.info.rpc_pubsub().unwrap(),
3148                    )),
3149                    ..ValidatorConfig::default_for_test()
3150                };
3151                Validator::new(
3152                    validator_node,
3153                    Arc::new(validator_keypair),
3154                    &validator_ledger_path,
3155                    &vote_account_keypair.pubkey(),
3156                    Arc::new(RwLock::new(vec![Arc::new(vote_account_keypair)])),
3157                    vec![leader_node.info.clone()],
3158                    &config,
3159                    true, // should_check_duplicate_instance.
3160                    None, // rpc_to_plugin_manager_receiver
3161                    Arc::new(RwLock::new(ValidatorStartProgress::default())),
3162                    SocketAddrSpace::Unspecified,
3163                    ValidatorTpuConfig::new_for_tests(DEFAULT_TPU_ENABLE_UDP),
3164                    Arc::new(RwLock::new(None)),
3165                )
3166                .expect("assume successful validator start")
3167            })
3168            .collect();
3169
3170        // Each validator can exit in parallel to speed many sequential calls to join`
3171        validators.iter_mut().for_each(|v| v.exit());
3172
3173        // spawn a new thread to wait for the join of the validator
3174        let (sender, receiver) = bounded(0);
3175        let _ = thread::spawn(move || {
3176            validators.into_iter().for_each(|validator| {
3177                validator.join();
3178            });
3179            sender.send(()).unwrap();
3180        });
3181
3182        let timeout = Duration::from_secs(60);
3183        if let Err(RecvTimeoutError::Timeout) = receiver.recv_timeout(timeout) {
3184            panic!("timeout for shutting down validators",);
3185        }
3186
3187        for path in ledger_paths {
3188            remove_dir_all(path).unwrap();
3189        }
3190    }
3191
3192    #[test]
3193    fn test_wait_for_supermajority() {
3194        agave_logger::setup();
3195        let node_keypair = Arc::new(Keypair::new());
3196        let cluster_info = ClusterInfo::new(
3197            ContactInfo::new_localhost(&node_keypair.pubkey(), timestamp()),
3198            node_keypair,
3199            SocketAddrSpace::Unspecified,
3200        );
3201
3202        let (genesis_config, _mint_keypair) = create_genesis_config(1);
3203        let bank_forks = BankForks::new_rw_arc(Bank::new_for_tests(&genesis_config));
3204        let mut config = ValidatorConfig::default_for_test();
3205        let rpc_override_health_check = Arc::new(AtomicBool::new(false));
3206        let start_progress = Arc::new(RwLock::new(ValidatorStartProgress::default()));
3207
3208        assert!(!wait_for_supermajority(
3209            &config,
3210            None,
3211            &bank_forks,
3212            &cluster_info,
3213            rpc_override_health_check.clone(),
3214            &start_progress,
3215        )
3216        .unwrap());
3217
3218        // bank=0, wait=1, should fail
3219        config.wait_for_supermajority = Some(1);
3220        assert!(matches!(
3221            wait_for_supermajority(
3222                &config,
3223                None,
3224                &bank_forks,
3225                &cluster_info,
3226                rpc_override_health_check.clone(),
3227                &start_progress,
3228            ),
3229            Err(ValidatorError::NotEnoughLedgerData(_, _)),
3230        ));
3231
3232        // bank=1, wait=0, should pass, bank is past the wait slot
3233        let bank_forks = BankForks::new_rw_arc(Bank::new_from_parent(
3234            bank_forks.read().unwrap().root_bank(),
3235            &Pubkey::default(),
3236            1,
3237        ));
3238        config.wait_for_supermajority = Some(0);
3239        assert!(!wait_for_supermajority(
3240            &config,
3241            None,
3242            &bank_forks,
3243            &cluster_info,
3244            rpc_override_health_check.clone(),
3245            &start_progress,
3246        )
3247        .unwrap());
3248
3249        // bank=1, wait=1, equal, but bad hash provided
3250        config.wait_for_supermajority = Some(1);
3251        config.expected_bank_hash = Some(hash(&[1]));
3252        assert!(matches!(
3253            wait_for_supermajority(
3254                &config,
3255                None,
3256                &bank_forks,
3257                &cluster_info,
3258                rpc_override_health_check,
3259                &start_progress,
3260            ),
3261            Err(ValidatorError::BankHashMismatch(_, _)),
3262        ));
3263    }
3264
3265    #[test]
3266    fn test_is_snapshot_config_valid() {
3267        fn new_snapshot_config(
3268            full_snapshot_archive_interval_slots: Slot,
3269            incremental_snapshot_archive_interval_slots: Slot,
3270        ) -> SnapshotConfig {
3271            SnapshotConfig {
3272                full_snapshot_archive_interval: SnapshotInterval::Slots(
3273                    NonZeroU64::new(full_snapshot_archive_interval_slots).unwrap(),
3274                ),
3275                incremental_snapshot_archive_interval: SnapshotInterval::Slots(
3276                    NonZeroU64::new(incremental_snapshot_archive_interval_slots).unwrap(),
3277                ),
3278                ..SnapshotConfig::default()
3279            }
3280        }
3281
3282        // default config must be valid
3283        assert!(is_snapshot_config_valid(&SnapshotConfig::default()));
3284
3285        // disabled incremental snapshot must be valid
3286        assert!(is_snapshot_config_valid(&SnapshotConfig {
3287            incremental_snapshot_archive_interval: SnapshotInterval::Disabled,
3288            ..SnapshotConfig::default()
3289        }));
3290
3291        // disabled full snapshot must be invalid though (if generating snapshots)
3292        assert!(!is_snapshot_config_valid(&SnapshotConfig {
3293            full_snapshot_archive_interval: SnapshotInterval::Disabled,
3294            ..SnapshotConfig::default()
3295        }));
3296
3297        // simple config must be valid
3298        assert!(is_snapshot_config_valid(&new_snapshot_config(400, 200)));
3299        assert!(is_snapshot_config_valid(&new_snapshot_config(100, 42)));
3300        assert!(is_snapshot_config_valid(&new_snapshot_config(444, 200)));
3301        assert!(is_snapshot_config_valid(&new_snapshot_config(400, 222)));
3302
3303        // config where full interval is not larger than incremental interval must be invalid
3304        assert!(!is_snapshot_config_valid(&new_snapshot_config(42, 100)));
3305        assert!(!is_snapshot_config_valid(&new_snapshot_config(100, 100)));
3306        assert!(!is_snapshot_config_valid(&new_snapshot_config(100, 200)));
3307
3308        // config with snapshots disabled (or load-only) must be valid
3309        assert!(is_snapshot_config_valid(&SnapshotConfig::new_disabled()));
3310        assert!(is_snapshot_config_valid(&SnapshotConfig::new_load_only()));
3311        assert!(is_snapshot_config_valid(&SnapshotConfig {
3312            full_snapshot_archive_interval: SnapshotInterval::Slots(NonZeroU64::new(37).unwrap()),
3313            incremental_snapshot_archive_interval: SnapshotInterval::Slots(
3314                NonZeroU64::new(41).unwrap()
3315            ),
3316            ..SnapshotConfig::new_load_only()
3317        }));
3318        assert!(is_snapshot_config_valid(&SnapshotConfig {
3319            full_snapshot_archive_interval: SnapshotInterval::Disabled,
3320            incremental_snapshot_archive_interval: SnapshotInterval::Disabled,
3321            ..SnapshotConfig::new_load_only()
3322        }));
3323    }
3324
3325    fn target_tick_duration() -> Duration {
3326        // DEFAULT_MS_PER_SLOT = 400
3327        // DEFAULT_TICKS_PER_SLOT = 64
3328        // MS_PER_TICK = 6
3329        //
3330        // But, DEFAULT_MS_PER_SLOT / DEFAULT_TICKS_PER_SLOT = 6.25
3331        //
3332        // So, convert to microseconds first to avoid the integer rounding error
3333        let target_tick_duration_us =
3334            solana_clock::DEFAULT_MS_PER_SLOT * 1000 / solana_clock::DEFAULT_TICKS_PER_SLOT;
3335        assert_eq!(target_tick_duration_us, 6250);
3336        Duration::from_micros(target_tick_duration_us)
3337    }
3338
3339    #[test]
3340    fn test_poh_speed() {
3341        agave_logger::setup();
3342        let poh_config = PohConfig {
3343            target_tick_duration: target_tick_duration(),
3344            // make PoH rate really fast to cause the panic condition
3345            hashes_per_tick: Some(100 * solana_clock::DEFAULT_HASHES_PER_TICK),
3346            ..PohConfig::default()
3347        };
3348        let genesis_config = GenesisConfig {
3349            poh_config,
3350            ..GenesisConfig::default()
3351        };
3352        let bank = Bank::new_for_tests(&genesis_config);
3353        assert!(check_poh_speed(&bank, Some(10_000)).is_err());
3354    }
3355
3356    #[test]
3357    fn test_poh_speed_no_hashes_per_tick() {
3358        agave_logger::setup();
3359        let poh_config = PohConfig {
3360            target_tick_duration: target_tick_duration(),
3361            hashes_per_tick: None,
3362            ..PohConfig::default()
3363        };
3364        let genesis_config = GenesisConfig {
3365            poh_config,
3366            ..GenesisConfig::default()
3367        };
3368        let bank = Bank::new_for_tests(&genesis_config);
3369        check_poh_speed(&bank, Some(10_000)).unwrap();
3370    }
3371}