From bdd08c043d038e7dfafa511570d6dfb31a7b361c Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Thu, 22 Dec 2022 15:17:02 +0900 Subject: [PATCH] Squash them all --- banking-bench/src/main.rs | 37 +- bench-tps/src/bench.rs | 33 +- bench-tps/src/cli.rs | 16 + core/src/banking_stage.rs | 1932 +++-------------- core/src/banking_stage/commit_executor.rs | 165 ++ core/src/banking_stage/consume_executor.rs | 515 +++++ core/src/banking_stage/decision_maker.rs | 91 + core/src/banking_stage/external_scheduler.rs | 189 ++ core/src/banking_stage/forward_executor.rs | 135 ++ .../banking_stage/multi_iterator_scheduler.rs | 914 ++++++++ core/src/banking_stage/packet_receiver.rs | 540 +++++ core/src/banking_stage/record_executor.rs | 73 + core/src/banking_stage/scheduler_error.rs | 21 + core/src/banking_stage/scheduler_handle.rs | 84 + .../thread_aware_account_locks.rs | 267 +++ .../banking_stage/thread_local_scheduler.rs | 376 ++++ core/src/leader_slot_banking_stage_metrics.rs | 234 +- ...eader_slot_banking_stage_timing_metrics.rs | 19 - core/src/lib.rs | 2 + core/src/multi_iterator_scanner.rs | 6 +- core/src/next_leader.rs | 48 + core/src/read_write_account_set.rs | 4 +- core/src/scheduler_stage.rs | 205 ++ core/src/tpu.rs | 56 +- core/src/tracer_packet_stats.rs | 73 - core/src/unprocessed_transaction_storage.rs | 6 +- core/src/validator.rs | 3 + core/src/voting_service.rs | 4 +- poh/src/poh_recorder.rs | 4 + runtime/src/bank.rs | 62 +- validator/src/cli.rs | 5 + validator/src/main.rs | 2 + 32 files changed, 4148 insertions(+), 1973 deletions(-) create mode 100644 core/src/banking_stage/commit_executor.rs create mode 100644 core/src/banking_stage/consume_executor.rs create mode 100644 core/src/banking_stage/decision_maker.rs create mode 100644 core/src/banking_stage/external_scheduler.rs create mode 100644 core/src/banking_stage/forward_executor.rs create mode 100644 core/src/banking_stage/multi_iterator_scheduler.rs create mode 100644 core/src/banking_stage/packet_receiver.rs create mode 100644 core/src/banking_stage/record_executor.rs create mode 100644 core/src/banking_stage/scheduler_error.rs create mode 100644 core/src/banking_stage/scheduler_handle.rs create mode 100644 core/src/banking_stage/thread_aware_account_locks.rs create mode 100644 core/src/banking_stage/thread_local_scheduler.rs create mode 100644 core/src/next_leader.rs create mode 100644 core/src/scheduler_stage.rs diff --git a/banking-bench/src/main.rs b/banking-bench/src/main.rs index f1077140e796a2..21370fc13c4260 100644 --- a/banking-bench/src/main.rs +++ b/banking-bench/src/main.rs @@ -1,4 +1,5 @@ #![allow(clippy::integer_arithmetic)] + use { clap::{crate_description, crate_name, Arg, ArgEnum, Command}, crossbeam_channel::{unbounded, Receiver}, @@ -6,7 +7,10 @@ use { rand::{thread_rng, Rng}, rayon::prelude::*, solana_client::connection_cache::ConnectionCache, - solana_core::banking_stage::BankingStage, + solana_core::{ + banking_stage::BankingStage, + scheduler_stage::{SchedulerKind, SchedulerStage}, + }, solana_gossip::cluster_info::{ClusterInfo, Node}, solana_ledger::{ blockstore::Blockstore, @@ -421,19 +425,43 @@ fn main() { true => ConnectionCache::new(DEFAULT_TPU_CONNECTION_POOL_SIZE), false => ConnectionCache::with_udp(DEFAULT_TPU_CONNECTION_POOL_SIZE), }; - let banking_stage = BankingStage::new_num_threads( + + let (scheduler_stage, transactions_receivers, processed_transactions_sender) = + SchedulerStage::new_num_threads( + SchedulerKind::MultiIteratorScheduler, + num_banking_threads as usize - 2, + verified_receiver, + bank_forks.clone(), + poh_recorder.clone(), + &cluster_info, + ); + let banking_stage = BankingStage::new_external_scheduler( &cluster_info, &poh_recorder, - verified_receiver, + transactions_receivers.unwrap(), + processed_transactions_sender.unwrap(), tpu_vote_receiver, vote_receiver, - num_banking_threads, None, replay_vote_sender, None, Arc::new(connection_cache), bank_forks.clone(), ); + + // let banking_stage = BankingStage::new_num_threads( + // &cluster_info, + // &poh_recorder, + // verified_receiver, + // tpu_vote_receiver, + // vote_receiver, + // num_banking_threads, + // None, + // replay_vote_sender, + // None, + // Arc::new(connection_cache), + // bank_forks.clone(), + // ); poh_recorder.write().unwrap().set_bank(&bank, false); // This is so that the signal_receiver does not go out of scope after the closure. @@ -578,6 +606,7 @@ fn main() { drop(tpu_vote_sender); drop(vote_sender); exit.store(true, Ordering::Relaxed); + scheduler_stage.join().unwrap(); banking_stage.join().unwrap(); debug!("waited for banking_stage"); poh_service.join().unwrap(); diff --git a/bench-tps/src/bench.rs b/bench-tps/src/bench.rs index d899ace9c86b2e..651b45c4ab15c3 100644 --- a/bench-tps/src/bench.rs +++ b/bench-tps/src/bench.rs @@ -87,6 +87,30 @@ impl<'a> KeypairChunks<'a> { dest: dest_keypair_chunks, } } + + /// Split input vector of keypairs into conflict sets with chunks of given size + fn new_with_conflict_groups( + keypairs: &'a [Keypair], + chunk_size: usize, + num_conflict_groups: usize, + ) -> Self { + let mut source: Vec> = Vec::new(); + let mut dest: Vec> = Vec::new(); + + // Transfer from accounts to the first account in each conflict group + for chunk in keypairs.chunks_exact(2 * chunk_size) { + source.push(chunk[..chunk_size].iter().collect()); + dest.push( + chunk[..chunk_size] + .iter() + .enumerate() + .map(|(idx, _)| &chunk[chunk_size + idx % num_conflict_groups]) + .collect(), + ); + } + + Self { source, dest } + } } struct TransactionChunkGenerator<'a, 'b, T: ?Sized> { @@ -110,8 +134,13 @@ where chunk_size: usize, use_randomized_compute_unit_price: bool, instruction_padding_config: Option, + num_conflict_groups: Option, ) -> Self { - let account_chunks = KeypairChunks::new(gen_keypairs, chunk_size); + let account_chunks = if let Some(num_conflict_groups) = num_conflict_groups { + KeypairChunks::new_with_conflict_groups(gen_keypairs, chunk_size, num_conflict_groups) + } else { + KeypairChunks::new(gen_keypairs, chunk_size) + }; let nonce_chunks = nonce_keypairs.map(|nonce_keypairs| KeypairChunks::new(nonce_keypairs, chunk_size)); @@ -353,6 +382,7 @@ where use_randomized_compute_unit_price, use_durable_nonce, instruction_padding_config, + num_conflict_groups, .. } = config; @@ -364,6 +394,7 @@ where tx_count, use_randomized_compute_unit_price, instruction_padding_config, + num_conflict_groups, ); let first_tx_count = loop { diff --git a/bench-tps/src/cli.rs b/bench-tps/src/cli.rs index ed4730fe1283c4..d7dc92b9458233 100644 --- a/bench-tps/src/cli.rs +++ b/bench-tps/src/cli.rs @@ -65,6 +65,7 @@ pub struct Config { pub use_randomized_compute_unit_price: bool, pub use_durable_nonce: bool, pub instruction_padding_config: Option, + pub num_conflict_groups: Option, } impl Default for Config { @@ -95,6 +96,7 @@ impl Default for Config { use_randomized_compute_unit_price: false, use_durable_nonce: false, instruction_padding_config: None, + num_conflict_groups: None, } } } @@ -342,6 +344,12 @@ pub fn build_args<'a, 'b>(version: &'b str) -> App<'a, 'b> { .takes_value(true) .help("If set, wraps all instructions in the instruction padding program, with the given amount of padding bytes in instruction data."), ) + .arg( + Arg::with_name("num_conflict_groups") + .long("num-conflict-groups") + .takes_value(true) + .help("If set, creates different sets of conflicting transactions") + ) } /// Parses a clap `ArgMatches` structure into a `Config` @@ -494,5 +502,13 @@ pub fn extract_args(matches: &ArgMatches) -> Config { }); } + if let Some(num_conflict_groups) = matches.value_of("num_conflict_groups") { + args.num_conflict_groups = Some( + num_conflict_groups + .parse() + .expect("Can't parse conflict groups"), + ); + } + args } diff --git a/core/src/banking_stage.rs b/core/src/banking_stage.rs index 3a13e6734e25e6..c297821b5c23f0 100644 --- a/core/src/banking_stage.rs +++ b/core/src/banking_stage.rs @@ -3,88 +3,66 @@ //! can do its processing in parallel with signature verification on the GPU. use { + self::{ + commit_executor::CommitExecutor, consume_executor::ConsumeExecutor, + decision_maker::DecisionMaker, forward_executor::ForwardExecutor, + packet_receiver::PacketReceiver, record_executor::RecordExecutor, + scheduler_handle::SchedulerHandle, + }, crate::{ - forward_packet_batches_by_accounts::ForwardPacketBatchesByAccounts, - immutable_deserialized_packet::ImmutableDeserializedPacket, latest_unprocessed_votes::{LatestUnprocessedVotes, VoteSource}, - leader_slot_banking_stage_metrics::{ - LeaderSlotMetricsTracker, MetricsTrackerAction, ProcessTransactionsSummary, - }, - leader_slot_banking_stage_timing_metrics::{ - LeaderExecuteAndCommitTimings, RecordTransactionsTimings, - }, - packet_deserializer::{PacketDeserializer, ReceivePacketResults}, + leader_slot_banking_stage_metrics::LeaderSlotMetricsTracker, + leader_slot_banking_stage_timing_metrics::LeaderExecuteAndCommitTimings, + packet_deserializer::PacketDeserializer, qos_service::QosService, + scheduler_stage::{ProcessedTransactionsSender, ScheduledTransactionsReceiver}, sigverify::SigverifyTracerPacketStats, tracer_packet_stats::TracerPacketStats, unprocessed_packet_batches::*, - unprocessed_transaction_storage::{ - ConsumeScannerPayload, ThreadType, UnprocessedTransactionStorage, - }, - }, - core::iter::repeat, - crossbeam_channel::{ - Receiver as CrossbeamReceiver, RecvTimeoutError, Sender as CrossbeamSender, - }, - histogram::Histogram, - itertools::Itertools, - solana_client::{connection_cache::ConnectionCache, tpu_connection::TpuConnection}, - solana_entry::entry::hash_transactions, - solana_gossip::{cluster_info::ClusterInfo, contact_info::ContactInfo}, - solana_ledger::{ - blockstore_processor::TransactionStatusSender, token_balances::collect_token_balances, - }, - solana_measure::{measure, measure::Measure}, - solana_metrics::inc_new_counter_info, - solana_perf::{ - data_budget::DataBudget, - packet::{Packet, PacketBatch, PACKETS_PER_BATCH}, + unprocessed_transaction_storage::{ThreadType, UnprocessedTransactionStorage}, }, - solana_poh::poh_recorder::{BankStart, PohRecorder, PohRecorderError, TransactionRecorder}, - solana_program_runtime::timings::ExecuteTimings, + crossbeam_channel::{Receiver as CrossbeamReceiver, Sender as CrossbeamSender}, + solana_client::connection_cache::ConnectionCache, + solana_gossip::cluster_info::ClusterInfo, + solana_ledger::blockstore_processor::TransactionStatusSender, + solana_perf::{data_budget::DataBudget, packet::PacketBatch}, + solana_poh::poh_recorder::{PohRecorder, PohRecorderError}, solana_runtime::{ - accounts::TransactionLoadResult, - bank::{ - Bank, CommitTransactionCounts, LoadAndExecuteTransactionsOutput, - TransactionBalancesSet, TransactionCheckResult, TransactionExecutionResult, - TransactionResults, - }, - bank_forks::BankForks, - bank_utils, - transaction_batch::TransactionBatch, - transaction_error_metrics::TransactionErrorMetrics, + self, bank_forks::BankForks, transaction_error_metrics::TransactionErrorMetrics, vote_sender_types::ReplayVoteSender, }, solana_sdk::{ - clock::{ - Slot, DEFAULT_TICKS_PER_SLOT, FORWARD_TRANSACTIONS_TO_LEADER_AT_SLOT_OFFSET, - HOLD_TRANSACTIONS_SLOT_OFFSET, MAX_PROCESSING_AGE, - }, - feature_set::allow_votes_to_directly_update_vote_state, - pubkey::Pubkey, - saturating_add_assign, - timing::{duration_as_ms, timestamp, AtomicInterval}, - transaction::{self, SanitizedTransaction, TransactionError, VersionedTransaction}, - transport::TransportError, - }, - solana_streamer::sendmmsg::batch_send, - solana_transaction_status::{ - token_balances::TransactionTokenBalancesSet, TransactionTokenBalance, + feature_set::allow_votes_to_directly_update_vote_state, pubkey::Pubkey, + timing::AtomicInterval, }, + solana_transaction_status::TransactionTokenBalance, std::{ cmp, collections::HashMap, env, - net::{SocketAddr, UdpSocket}, + net::UdpSocket, sync::{ atomic::{AtomicU64, AtomicUsize, Ordering}, Arc, RwLock, }, thread::{self, Builder, JoinHandle}, - time::{Duration, Instant}, + time::Duration, }, }; +pub mod commit_executor; +pub mod consume_executor; +pub mod decision_maker; +pub mod external_scheduler; +pub mod forward_executor; +pub mod multi_iterator_scheduler; +pub mod packet_receiver; +pub mod record_executor; +pub mod scheduler_error; +pub mod scheduler_handle; +pub mod thread_aware_account_locks; +pub mod thread_local_scheduler; + // Fixed thread size seems to be fastest on GCP setup pub const NUM_THREADS: u32 = 6; @@ -109,15 +87,6 @@ pub struct ProcessTransactionBatchOutput { execute_and_commit_transactions_output: ExecuteAndCommitTransactionsOutput, } -struct RecordTransactionsSummary { - // Metrics describing how time was spent recording transactions - record_transactions_timings: RecordTransactionsTimings, - // Result of trying to record the transactions into the PoH stream - result: Result<(), PohRecorderError>, - // Index in the slot of the first transaction recorded - starting_transaction_index: Option, -} - #[derive(Clone, Debug, PartialEq, Eq)] pub enum CommitTransactionDetails { Committed { compute_units: u64 }, @@ -147,20 +116,13 @@ pub struct ExecuteAndCommitTransactionsOutput { pub struct BankingStageStats { last_report: AtomicInterval, id: u32, - receive_and_buffer_packets_count: AtomicUsize, - dropped_packets_count: AtomicUsize, - pub(crate) dropped_duplicated_packets_count: AtomicUsize, - newly_buffered_packets_count: AtomicUsize, - current_buffered_packets_count: AtomicUsize, rebuffered_packets_count: AtomicUsize, consumed_buffered_packets_count: AtomicUsize, forwarded_transaction_count: AtomicUsize, forwarded_vote_count: AtomicUsize, - batch_packet_indexes_len: Histogram, // Timing consume_buffered_packets_elapsed: AtomicU64, - receive_and_buffer_packets_elapsed: AtomicU64, filter_pending_packets_elapsed: AtomicU64, pub(crate) packet_conversion_elapsed: AtomicU64, transaction_processing_elapsed: AtomicU64, @@ -170,38 +132,21 @@ impl BankingStageStats { pub fn new(id: u32) -> Self { BankingStageStats { id, - batch_packet_indexes_len: Histogram::configure() - .max_value(PACKETS_PER_BATCH as u64) - .build() - .unwrap(), ..BankingStageStats::default() } } fn is_empty(&self) -> bool { - 0 == self - .receive_and_buffer_packets_count - .load(Ordering::Relaxed) as u64 - + self.dropped_packets_count.load(Ordering::Relaxed) as u64 - + self - .dropped_duplicated_packets_count - .load(Ordering::Relaxed) as u64 - + self.newly_buffered_packets_count.load(Ordering::Relaxed) as u64 - + self.current_buffered_packets_count.load(Ordering::Relaxed) as u64 - + self.rebuffered_packets_count.load(Ordering::Relaxed) as u64 + 0 == self.rebuffered_packets_count.load(Ordering::Relaxed) as u64 + self.consumed_buffered_packets_count.load(Ordering::Relaxed) as u64 + self .consume_buffered_packets_elapsed .load(Ordering::Relaxed) - + self - .receive_and_buffer_packets_elapsed - .load(Ordering::Relaxed) + self.filter_pending_packets_elapsed.load(Ordering::Relaxed) + self.packet_conversion_elapsed.load(Ordering::Relaxed) + self.transaction_processing_elapsed.load(Ordering::Relaxed) + self.forwarded_transaction_count.load(Ordering::Relaxed) as u64 + self.forwarded_vote_count.load(Ordering::Relaxed) as u64 - + self.batch_packet_indexes_len.entries() } fn report(&mut self, report_interval_ms: u64) { @@ -213,34 +158,6 @@ impl BankingStageStats { datapoint_info!( "banking_stage-loop-stats", ("id", self.id as i64, i64), - ( - "receive_and_buffer_packets_count", - self.receive_and_buffer_packets_count - .swap(0, Ordering::Relaxed) as i64, - i64 - ), - ( - "dropped_packets_count", - self.dropped_packets_count.swap(0, Ordering::Relaxed) as i64, - i64 - ), - ( - "dropped_duplicated_packets_count", - self.dropped_duplicated_packets_count - .swap(0, Ordering::Relaxed) as i64, - i64 - ), - ( - "newly_buffered_packets_count", - self.newly_buffered_packets_count.swap(0, Ordering::Relaxed) as i64, - i64 - ), - ( - "current_buffered_packets_count", - self.current_buffered_packets_count - .swap(0, Ordering::Relaxed) as i64, - i64 - ), ( "rebuffered_packets_count", self.rebuffered_packets_count.swap(0, Ordering::Relaxed) as i64, @@ -268,12 +185,6 @@ impl BankingStageStats { .swap(0, Ordering::Relaxed) as i64, i64 ), - ( - "receive_and_buffer_packets_elapsed", - self.receive_and_buffer_packets_elapsed - .swap(0, Ordering::Relaxed) as i64, - i64 - ), ( "filter_pending_packets_elapsed", self.filter_pending_packets_elapsed @@ -290,35 +201,14 @@ impl BankingStageStats { self.transaction_processing_elapsed .swap(0, Ordering::Relaxed) as i64, i64 - ), - ( - "packet_batch_indices_len_min", - self.batch_packet_indexes_len.minimum().unwrap_or(0) as i64, - i64 - ), - ( - "packet_batch_indices_len_max", - self.batch_packet_indexes_len.maximum().unwrap_or(0) as i64, - i64 - ), - ( - "packet_batch_indices_len_mean", - self.batch_packet_indexes_len.mean().unwrap_or(0) as i64, - i64 - ), - ( - "packet_batch_indices_len_90pct", - self.batch_packet_indexes_len.percentile(90.0).unwrap_or(0) as i64, - i64 ) ); - self.batch_packet_indexes_len.clear(); } } } #[derive(Default)] -struct PreBalanceInfo { +pub(self) struct PreBalanceInfo { native: Vec>, token: Vec>, mint_decimals: HashMap, @@ -353,14 +243,6 @@ pub struct BankingStage { bank_thread_hdls: Vec>, } -#[derive(Debug, Clone)] -pub enum BufferedPacketsDecision { - Consume(BankStart), - Forward, - ForwardAndHold, - Hold, -} - #[derive(Debug, Clone)] pub enum ForwardOption { NotForward, @@ -407,6 +289,66 @@ impl BankingStage { ) } + #[allow(clippy::too_many_arguments)] + pub fn new_external_scheduler( + cluster_info: &Arc, + poh_recorder: &Arc>, + transactions_receivers: Vec, + processed_transactions_sender: ProcessedTransactionsSender, + tpu_verified_vote_receiver: BankingPacketReceiver, + verified_vote_receiver: BankingPacketReceiver, + transaction_status_sender: Option, + gossip_vote_sender: ReplayVoteSender, + log_messages_bytes_limit: Option, + connection_cache: Arc, + bank_forks: Arc>, + ) -> Self { + let num_threads = transactions_receivers.len(); + // Single thread to generate entries from many banks. + // This thread talks to poh_service and broadcasts the entries once they have been recorded. + // Once an entry has been recorded, its blockhash is registered with the bank. + let data_budget = Arc::new(DataBudget::default()); + let batch_limit = TOTAL_BUFFERED_PACKETS / num_threads; + // Many banks that process transactions in parallel. + let mut bank_thread_hdls = Self::spawn_voting_threads( + cluster_info.clone(), + poh_recorder.clone(), + tpu_verified_vote_receiver, + verified_vote_receiver, + transaction_status_sender.clone(), + gossip_vote_sender.clone(), + log_messages_bytes_limit, + connection_cache.clone(), + bank_forks, + data_budget.clone(), + batch_limit, + ); + + // Add non-vote transaction threads + let index_to_id_offset = bank_thread_hdls.len() as u32; + for (index, transactions_receiver) in transactions_receivers.into_iter().enumerate() { + let id = index as u32 + index_to_id_offset; + let scheduler_handle = SchedulerHandle::new_external_scheduler( + id, + transactions_receiver, + processed_transactions_sender.clone(), + ); + bank_thread_hdls.push(Self::spawn_banking_thread_with_scheduler( + id, + scheduler_handle, + poh_recorder.clone(), + cluster_info.clone(), + connection_cache.clone(), + data_budget.clone(), + log_messages_bytes_limit, + transaction_status_sender.clone(), + gossip_vote_sender.clone(), + )); + } + + Self { bank_thread_hdls } + } + #[allow(clippy::too_many_arguments)] pub fn new_num_threads( cluster_info: &Arc, @@ -428,6 +370,68 @@ impl BankingStage { let data_budget = Arc::new(DataBudget::default()); let batch_limit = TOTAL_BUFFERED_PACKETS / ((num_threads - NUM_VOTE_PROCESSING_THREADS) as usize); + + // Many banks that process transactions in parallel. + let mut bank_thread_hdls = Self::spawn_voting_threads( + cluster_info.clone(), + poh_recorder.clone(), + tpu_verified_vote_receiver, + verified_vote_receiver, + transaction_status_sender.clone(), + gossip_vote_sender.clone(), + log_messages_bytes_limit, + connection_cache.clone(), + bank_forks.clone(), + data_budget.clone(), + batch_limit, + ); + // Add non-vote transaction threads + for id in 2..num_threads { + let unprocessed_transaction_storage = + UnprocessedTransactionStorage::new_transaction_storage( + UnprocessedPacketBatches::with_capacity(batch_limit), + ThreadType::Transactions, + ); + let packet_deserializer = PacketDeserializer::new(verified_receiver.clone()); + let scheduler_handle = Self::build_thread_local_scheduler_handle( + packet_deserializer, + poh_recorder.clone(), + bank_forks.clone(), + cluster_info.clone(), + id, + unprocessed_transaction_storage, + ); + + bank_thread_hdls.push(Self::spawn_banking_thread_with_scheduler( + id, + scheduler_handle, + poh_recorder.clone(), + cluster_info.clone(), + connection_cache.clone(), + data_budget.clone(), + log_messages_bytes_limit, + transaction_status_sender.clone(), + gossip_vote_sender.clone(), + )); + } + + Self { bank_thread_hdls } + } + + #[allow(clippy::too_many_arguments)] + fn spawn_voting_threads( + cluster_info: Arc, + poh_recorder: Arc>, + tpu_verified_vote_receiver: BankingPacketReceiver, + verified_vote_receiver: BankingPacketReceiver, + transaction_status_sender: Option, + gossip_vote_sender: ReplayVoteSender, + log_messages_bytes_limit: Option, + connection_cache: Arc, + bank_forks: Arc>, + data_budget: Arc, + batch_limit: usize, + ) -> Vec> { // Keeps track of extraneous vote transactions for the vote threads let latest_unprocessed_votes = Arc::new(LatestUnprocessedVotes::new()); let should_split_voting_threads = bank_forks @@ -438,671 +442,199 @@ impl BankingStage { .is_active(&allow_votes_to_directly_update_vote_state::id()) }) .unwrap_or(false); - // Many banks that process transactions in parallel. - let bank_thread_hdls: Vec> = (0..num_threads) - .map(|i| { - let (verified_receiver, unprocessed_transaction_storage) = - match (i, should_split_voting_threads) { - (0, false) => ( - verified_vote_receiver.clone(), - UnprocessedTransactionStorage::new_transaction_storage( - UnprocessedPacketBatches::with_capacity(batch_limit), - ThreadType::Voting(VoteSource::Gossip), - ), - ), - (0, true) => ( - verified_vote_receiver.clone(), - UnprocessedTransactionStorage::new_vote_storage( - latest_unprocessed_votes.clone(), - VoteSource::Gossip, - ), - ), - (1, false) => ( - tpu_verified_vote_receiver.clone(), - UnprocessedTransactionStorage::new_transaction_storage( - UnprocessedPacketBatches::with_capacity(batch_limit), - ThreadType::Voting(VoteSource::Tpu), - ), - ), - (1, true) => ( - tpu_verified_vote_receiver.clone(), - UnprocessedTransactionStorage::new_vote_storage( - latest_unprocessed_votes.clone(), - VoteSource::Tpu, - ), - ), - _ => ( - verified_receiver.clone(), - UnprocessedTransactionStorage::new_transaction_storage( - UnprocessedPacketBatches::with_capacity(batch_limit), - ThreadType::Transactions, - ), - ), - }; - let mut packet_deserializer = PacketDeserializer::new(verified_receiver); - let poh_recorder = poh_recorder.clone(); - let cluster_info = cluster_info.clone(); - let mut recv_start = Instant::now(); - let transaction_status_sender = transaction_status_sender.clone(); - let gossip_vote_sender = gossip_vote_sender.clone(); - let data_budget = data_budget.clone(); - let connection_cache = connection_cache.clone(); - let bank_forks = bank_forks.clone(); - Builder::new() - .name(format!("solBanknStgTx{:02}", i)) - .spawn(move || { - Self::process_loop( - &mut packet_deserializer, - &poh_recorder, - &cluster_info, - &mut recv_start, - i, - transaction_status_sender, - gossip_vote_sender, - &data_budget, - log_messages_bytes_limit, - connection_cache, - &bank_forks, - unprocessed_transaction_storage, - ); - }) - .unwrap() - }) - .collect(); - Self { bank_thread_hdls } - } - - /// Forwards all valid, unprocessed packets in the buffer, up to a rate limit. Returns - /// the number of successfully forwarded packets in second part of tuple - fn forward_buffered_packets<'a>( - connection_cache: &ConnectionCache, - forward_option: &ForwardOption, - cluster_info: &ClusterInfo, - poh_recorder: &Arc>, - socket: &UdpSocket, - forwardable_packets: impl Iterator, - data_budget: &DataBudget, - banking_stage_stats: &BankingStageStats, - ) -> ( - std::result::Result<(), TransportError>, - usize, - Option, - ) { - let leader_and_addr = match forward_option { - ForwardOption::NotForward => return (Ok(()), 0, None), - ForwardOption::ForwardTransaction => { - next_leader_tpu_forwards(cluster_info, poh_recorder) - } - - ForwardOption::ForwardTpuVote => next_leader_tpu_vote(cluster_info, poh_recorder), - }; - let (leader_pubkey, addr) = match leader_and_addr { - Some(leader_and_addr) => leader_and_addr, - None => return (Ok(()), 0, None), - }; - - const INTERVAL_MS: u64 = 100; - // 12 MB outbound limit per second - const MAX_BYTES_PER_SECOND: usize = 12_000_000; - const MAX_BYTES_PER_INTERVAL: usize = MAX_BYTES_PER_SECOND * INTERVAL_MS as usize / 1000; - const MAX_BYTES_BUDGET: usize = MAX_BYTES_PER_INTERVAL * 5; - data_budget.update(INTERVAL_MS, |bytes| { - std::cmp::min( - bytes.saturating_add(MAX_BYTES_PER_INTERVAL), - MAX_BYTES_BUDGET, - ) - }); - - let packet_vec: Vec<_> = forwardable_packets - .filter_map(|p| { - if !p.meta.forwarded() && data_budget.take(p.meta.size) { - Some(p.data(..)?.to_vec()) - } else { - None - } - }) - .collect(); - - let packet_vec_len = packet_vec.len(); - // TODO: see https://github.com/solana-labs/solana/issues/23819 - // fix this so returns the correct number of succeeded packets - // when there's an error sending the batch. This was left as-is for now - // in favor of shipping Quic support, which was considered higher-priority - if !packet_vec.is_empty() { - inc_new_counter_info!("banking_stage-forwarded_packets", packet_vec_len); - - let mut measure = Measure::start("banking_stage-forward-us"); - - let res = if let ForwardOption::ForwardTpuVote = forward_option { - // The vote must be forwarded using only UDP. - banking_stage_stats - .forwarded_vote_count - .fetch_add(packet_vec_len, Ordering::Relaxed); - let pkts: Vec<_> = packet_vec.into_iter().zip(repeat(addr)).collect(); - batch_send(socket, &pkts).map_err(|err| err.into()) - } else { - // All other transactions can be forwarded using QUIC, get_connection() will use - // system wide setting to pick the correct connection object. - banking_stage_stats - .forwarded_transaction_count - .fetch_add(packet_vec_len, Ordering::Relaxed); - let conn = connection_cache.get_connection(&addr); - conn.send_wire_transaction_batch_async(packet_vec) - }; - - measure.stop(); - inc_new_counter_info!( - "banking_stage-forward-us", - measure.as_us() as usize, - 1000, - 1000 - ); - - if let Err(err) = res { - inc_new_counter_info!("banking_stage-forward_packets-failed-batches", 1); - return (Err(err), 0, Some(leader_pubkey)); - } - } - - (Ok(()), packet_vec_len, Some(leader_pubkey)) - } - - #[allow(clippy::too_many_arguments)] - fn do_process_packets( - bank_start: &BankStart, - payload: &mut ConsumeScannerPayload, - recorder: &TransactionRecorder, - transaction_status_sender: &Option, - gossip_vote_sender: &ReplayVoteSender, - banking_stage_stats: &BankingStageStats, - qos_service: &QosService, - log_messages_bytes_limit: Option, - consumed_buffered_packets_count: &mut usize, - rebuffered_packet_count: &mut usize, - test_fn: &Option, - packets_to_process: &Vec>, - ) -> Option> { - if payload.reached_end_of_slot { - return None; - } - - let packets_to_process_len = packets_to_process.len(); - let (process_transactions_summary, process_packets_transactions_time) = measure!( - Self::process_packets_transactions( - &bank_start.working_bank, - &bank_start.bank_creation_time, - recorder, - &payload.sanitized_transactions, + vec![ + // Gossip voting thread + Self::spawn_voting_thread( + 0, + VoteSource::Gossip, + should_split_voting_threads, + batch_limit, + latest_unprocessed_votes.clone(), + PacketDeserializer::new(verified_vote_receiver), + poh_recorder.clone(), + bank_forks.clone(), + cluster_info.clone(), + connection_cache.clone(), + data_budget.clone(), + log_messages_bytes_limit, + transaction_status_sender.clone(), + gossip_vote_sender.clone(), + ), + // TPU voting thread + Self::spawn_voting_thread( + 1, + VoteSource::Tpu, + should_split_voting_threads, + batch_limit, + latest_unprocessed_votes, + PacketDeserializer::new(tpu_verified_vote_receiver), + poh_recorder, + bank_forks, + cluster_info, + connection_cache, + data_budget, + log_messages_bytes_limit, transaction_status_sender, gossip_vote_sender, - banking_stage_stats, - qos_service, - payload.slot_metrics_tracker, - log_messages_bytes_limit ), - "process_packets_transactions", - ); - payload - .slot_metrics_tracker - .increment_process_packets_transactions_us(process_packets_transactions_time.as_us()); - - // Clear payload for next iteration - payload.sanitized_transactions.clear(); - payload.account_locks.clear(); - - let ProcessTransactionsSummary { - reached_max_poh_height, - retryable_transaction_indexes, - .. - } = process_transactions_summary; - - if reached_max_poh_height || !bank_start.should_working_bank_still_be_processing_txs() { - payload.reached_end_of_slot = true; - } - - // The difference between all transactions passed to execution and the ones that - // are retryable were the ones that were either: - // 1) Committed into the block - // 2) Dropped without being committed because they had some fatal error (too old, - // duplicate signature, etc.) - // - // Note: This assumes that every packet deserializes into one transaction! - *consumed_buffered_packets_count += - packets_to_process_len.saturating_sub(retryable_transaction_indexes.len()); - - // Out of the buffered packets just retried, collect any still unprocessed - // transactions in this batch for forwarding - *rebuffered_packet_count += retryable_transaction_indexes.len(); - if let Some(test_fn) = test_fn { - test_fn(); - } - - payload - .slot_metrics_tracker - .increment_retryable_packets_count(retryable_transaction_indexes.len() as u64); - - Some(retryable_transaction_indexes) + ] } #[allow(clippy::too_many_arguments)] - pub fn consume_buffered_packets( - bank_start: &BankStart, - unprocessed_transaction_storage: &mut UnprocessedTransactionStorage, - transaction_status_sender: &Option, - gossip_vote_sender: &ReplayVoteSender, - test_fn: Option, - banking_stage_stats: &BankingStageStats, - recorder: &TransactionRecorder, - qos_service: &QosService, - slot_metrics_tracker: &mut LeaderSlotMetricsTracker, + fn spawn_voting_thread( + id: u32, + voting_source: VoteSource, + should_split_voting_threads: bool, + buffer_capacity: usize, + latest_unprocessed_votes: Arc, + packet_deserializer: PacketDeserializer, + poh_recorder: Arc>, + bank_forks: Arc>, + cluster_info: Arc, + connection_cache: Arc, + data_budget: Arc, log_messages_bytes_limit: Option, - ) { - let mut rebuffered_packet_count = 0; - let mut consumed_buffered_packets_count = 0; - let mut proc_start = Measure::start("consume_buffered_process"); - let num_packets_to_process = unprocessed_transaction_storage.len(); - - let reached_end_of_slot = unprocessed_transaction_storage.process_packets( - bank_start.working_bank.clone(), - banking_stage_stats, - slot_metrics_tracker, - |packets_to_process, payload| { - Self::do_process_packets( - bank_start, - payload, - recorder, - transaction_status_sender, - gossip_vote_sender, - banking_stage_stats, - qos_service, - log_messages_bytes_limit, - &mut consumed_buffered_packets_count, - &mut rebuffered_packet_count, - &test_fn, - packets_to_process, - ) - }, - ); - - if reached_end_of_slot { - slot_metrics_tracker.set_end_of_slot_unprocessed_buffer_len( - unprocessed_transaction_storage.len() as u64, - ); - } - - proc_start.stop(); - debug!( - "@{:?} done processing buffered batches: {} time: {:?}ms tx count: {} tx/s: {}", - timestamp(), - num_packets_to_process, - proc_start.as_ms(), - consumed_buffered_packets_count, - (consumed_buffered_packets_count as f32) / (proc_start.as_s()) - ); - - banking_stage_stats - .consume_buffered_packets_elapsed - .fetch_add(proc_start.as_us(), Ordering::Relaxed); - banking_stage_stats - .rebuffered_packets_count - .fetch_add(rebuffered_packet_count, Ordering::Relaxed); - banking_stage_stats - .consumed_buffered_packets_count - .fetch_add(consumed_buffered_packets_count, Ordering::Relaxed); - } - - fn consume_or_forward_packets( - my_pubkey: &Pubkey, - leader_pubkey: Option, - bank_start: Option, - would_be_leader: bool, - would_be_leader_shortly: bool, - ) -> BufferedPacketsDecision { - // If has active bank, then immediately process buffered packets - // otherwise, based on leader schedule to either forward or hold packets - if let Some(bank_start) = bank_start { - // If the bank is available, this node is the leader - BufferedPacketsDecision::Consume(bank_start) - } else if would_be_leader_shortly { - // If the node will be the leader soon, hold the packets for now - BufferedPacketsDecision::Hold - } else if would_be_leader { - // Node will be leader within ~20 slots, hold the transactions in - // case it is the only node which produces an accepted slot. - BufferedPacketsDecision::ForwardAndHold - } else if let Some(x) = leader_pubkey { - if x != *my_pubkey { - // If the current node is not the leader, forward the buffered packets - BufferedPacketsDecision::Forward - } else { - // If the current node is the leader, return the buffered packets as is - BufferedPacketsDecision::Hold - } + transaction_status_sender: Option, + gossip_vote_sender: ReplayVoteSender, + ) -> JoinHandle<()> { + let buffer = if should_split_voting_threads { + UnprocessedTransactionStorage::new_vote_storage(latest_unprocessed_votes, voting_source) } else { - // We don't know the leader. Hold the packets for now - BufferedPacketsDecision::Hold - } - } - - fn make_consume_or_forward_decision( - my_pubkey: &Pubkey, - poh_recorder: &RwLock, - slot_metrics_tracker: &mut LeaderSlotMetricsTracker, - ) -> (MetricsTrackerAction, BufferedPacketsDecision) { - let (leader_at_slot_offset, bank_start, would_be_leader, would_be_leader_shortly) = { - let poh = poh_recorder.read().unwrap(); - let bank_start = poh - .bank_start() - .filter(|bank_start| bank_start.should_working_bank_still_be_processing_txs()); - ( - poh.leader_after_n_slots(FORWARD_TRANSACTIONS_TO_LEADER_AT_SLOT_OFFSET), - bank_start, - poh.would_be_leader(HOLD_TRANSACTIONS_SLOT_OFFSET * DEFAULT_TICKS_PER_SLOT), - poh.would_be_leader( - (FORWARD_TRANSACTIONS_TO_LEADER_AT_SLOT_OFFSET - 1) * DEFAULT_TICKS_PER_SLOT, - ), + UnprocessedTransactionStorage::new_transaction_storage( + UnprocessedPacketBatches::with_capacity(buffer_capacity), + ThreadType::Voting(voting_source), ) }; - ( - slot_metrics_tracker.check_leader_slot_boundary(&bank_start), - Self::consume_or_forward_packets( - my_pubkey, - leader_at_slot_offset, - bank_start, - would_be_leader, - would_be_leader_shortly, - ), + let scheduler_handle = Self::build_thread_local_scheduler_handle( + packet_deserializer, + poh_recorder.clone(), + bank_forks, + cluster_info.clone(), + id, + buffer, + ); + + Self::spawn_banking_thread_with_scheduler( + id, + scheduler_handle, + poh_recorder, + cluster_info, + connection_cache, + data_budget, + log_messages_bytes_limit, + transaction_status_sender, + gossip_vote_sender, ) } - #[allow(clippy::too_many_arguments)] - fn process_buffered_packets( - my_pubkey: &Pubkey, - socket: &UdpSocket, - poh_recorder: &Arc>, - cluster_info: &ClusterInfo, - unprocessed_transaction_storage: &mut UnprocessedTransactionStorage, - transaction_status_sender: &Option, - gossip_vote_sender: &ReplayVoteSender, - banking_stage_stats: &BankingStageStats, - recorder: &TransactionRecorder, - data_budget: &DataBudget, - qos_service: &QosService, - slot_metrics_tracker: &mut LeaderSlotMetricsTracker, + fn spawn_banking_thread_with_scheduler( + id: u32, + scheduler_handle: SchedulerHandle, + poh_recorder: Arc>, + cluster_info: Arc, + connection_cache: Arc, + data_budget: Arc, log_messages_bytes_limit: Option, - connection_cache: &ConnectionCache, - tracer_packet_stats: &mut TracerPacketStats, - bank_forks: &Arc>, - ) { - if unprocessed_transaction_storage.should_not_process() { - return; - } - let ((metrics_action, decision), make_decision_time) = measure!( - Self::make_consume_or_forward_decision(my_pubkey, poh_recorder, slot_metrics_tracker) + transaction_status_sender: Option, + gossip_vote_sender: ReplayVoteSender, + ) -> JoinHandle<()> { + let (forward_executor, consume_executor) = Self::build_executors( + id, + poh_recorder, + cluster_info, + connection_cache, + data_budget, + log_messages_bytes_limit, + transaction_status_sender, + gossip_vote_sender, ); - slot_metrics_tracker.increment_make_decision_us(make_decision_time.as_us()); - - match decision { - BufferedPacketsDecision::Consume(bank_start) => { - // Take metrics action before consume packets (potentially resetting the - // slot metrics tracker to the next slot) so that we don't count the - // packet processing metrics from the next slot towards the metrics - // of the previous slot - slot_metrics_tracker.apply_action(metrics_action); - let (_, consume_buffered_packets_time) = measure!( - Self::consume_buffered_packets( - &bank_start, - unprocessed_transaction_storage, - transaction_status_sender, - gossip_vote_sender, - None::>, - banking_stage_stats, - recorder, - qos_service, - slot_metrics_tracker, - log_messages_bytes_limit - ), - "consume_buffered_packets", - ); - slot_metrics_tracker - .increment_consume_buffered_packets_us(consume_buffered_packets_time.as_us()); - } - BufferedPacketsDecision::Forward => { - let (_, forward_time) = measure!( - Self::handle_forwarding( - cluster_info, - unprocessed_transaction_storage, - poh_recorder, - socket, - false, - data_budget, - slot_metrics_tracker, - banking_stage_stats, - connection_cache, - tracer_packet_stats, - bank_forks, - ), - "forward", - ); - slot_metrics_tracker.increment_forward_us(forward_time.as_us()); - // Take metrics action after forwarding packets to include forwarded - // metrics into current slot - slot_metrics_tracker.apply_action(metrics_action); - } - BufferedPacketsDecision::ForwardAndHold => { - let (_, forward_and_hold_time) = measure!( - Self::handle_forwarding( - cluster_info, - unprocessed_transaction_storage, - poh_recorder, - socket, - true, - data_budget, - slot_metrics_tracker, - banking_stage_stats, - connection_cache, - tracer_packet_stats, - bank_forks, - ), - "forward_and_hold", - ); - slot_metrics_tracker.increment_forward_and_hold_us(forward_and_hold_time.as_us()); - // Take metrics action after forwarding packets - slot_metrics_tracker.apply_action(metrics_action); - } - _ => (), - } + Builder::new() + .name(format!("solBanknStgTx{:02}", id)) + .spawn(move || { + Self::process_loop(id, scheduler_handle, forward_executor, consume_executor); + }) + .unwrap() } - #[allow(clippy::too_many_arguments)] - fn handle_forwarding( - cluster_info: &ClusterInfo, - unprocessed_transaction_storage: &mut UnprocessedTransactionStorage, - poh_recorder: &Arc>, - socket: &UdpSocket, - hold: bool, - data_budget: &DataBudget, - slot_metrics_tracker: &mut LeaderSlotMetricsTracker, - banking_stage_stats: &BankingStageStats, - connection_cache: &ConnectionCache, - tracer_packet_stats: &mut TracerPacketStats, - bank_forks: &Arc>, - ) { - let forward_option = unprocessed_transaction_storage.forward_option(); - - // get current root bank from bank_forks, use it to sanitize transaction and - // load all accounts from address loader; - let current_bank = bank_forks.read().unwrap().root_bank(); - - let mut forward_packet_batches_by_accounts = - ForwardPacketBatchesByAccounts::new_with_default_batch_limits(); - - // sanitize and filter packets that are no longer valid (could be too old, a duplicate of something - // already processed), then add to forwarding buffer. - let filter_forwarding_result = unprocessed_transaction_storage - .filter_forwardable_packets_and_add_batches( - current_bank, - &mut forward_packet_batches_by_accounts, - ); - slot_metrics_tracker.increment_transactions_from_packets_us( - filter_forwarding_result.total_packet_conversion_us, + fn build_executors( + id: u32, + poh_recorder: Arc>, + cluster_info: Arc, + connection_cache: Arc, + data_budget: Arc, + log_messages_bytes_limit: Option, + transaction_status_sender: Option, + gossip_vote_sender: ReplayVoteSender, + ) -> (ForwardExecutor, ConsumeExecutor) { + let transaction_recorder = poh_recorder.read().unwrap().recorder(); + let forward_executor = ForwardExecutor::new( + poh_recorder, + UdpSocket::bind("0.0.0.0:0").unwrap(), + cluster_info, + connection_cache, + data_budget, ); - banking_stage_stats.packet_conversion_elapsed.fetch_add( - filter_forwarding_result.total_packet_conversion_us, - Ordering::Relaxed, + let record_executor = RecordExecutor::new(transaction_recorder); + let commit_executor = CommitExecutor::new(transaction_status_sender, gossip_vote_sender); + let consume_executor = ConsumeExecutor::new( + record_executor, + commit_executor, + QosService::new(id), + log_messages_bytes_limit, ); - banking_stage_stats - .filter_pending_packets_elapsed - .fetch_add( - filter_forwarding_result.total_filter_packets_us, - Ordering::Relaxed, - ); - - forward_packet_batches_by_accounts - .iter_batches() - .filter(|&batch| !batch.is_empty()) - .for_each(|forward_batch| { - slot_metrics_tracker.increment_forwardable_batches_count(1); - - let batched_forwardable_packets_count = forward_batch.len(); - let (_forward_result, sucessful_forwarded_packets_count, leader_pubkey) = - Self::forward_buffered_packets( - connection_cache, - &forward_option, - cluster_info, - poh_recorder, - socket, - forward_batch.get_forwardable_packets(), - data_budget, - banking_stage_stats, - ); - - if let Some(leader_pubkey) = leader_pubkey { - tracer_packet_stats.increment_total_forwardable_tracer_packets( - filter_forwarding_result.total_forwardable_tracer_packets, - leader_pubkey, - ); - } - let failed_forwarded_packets_count = batched_forwardable_packets_count - .saturating_sub(sucessful_forwarded_packets_count); - - if failed_forwarded_packets_count > 0 { - slot_metrics_tracker.increment_failed_forwarded_packets_count( - failed_forwarded_packets_count as u64, - ); - slot_metrics_tracker.increment_packet_batch_forward_failure_count(1); - } - if sucessful_forwarded_packets_count > 0 { - slot_metrics_tracker.increment_successful_forwarded_packets_count( - sucessful_forwarded_packets_count as u64, - ); - } - }); + (forward_executor, consume_executor) + } - if !hold { - slot_metrics_tracker.increment_cleared_from_buffer_after_forward_count( - filter_forwarding_result.total_forwardable_packets as u64, - ); - tracer_packet_stats.increment_total_cleared_from_buffer_after_forward( - filter_forwarding_result.total_tracer_packets_in_buffer, - ); - unprocessed_transaction_storage.clear_forwarded_packets(); - } + fn build_thread_local_scheduler_handle( + packet_deserializer: PacketDeserializer, + poh_recorder: Arc>, + bank_forks: Arc>, + cluster_info: Arc, + id: u32, + unprocessed_transaction_storage: UnprocessedTransactionStorage, + ) -> SchedulerHandle { + let packet_receiver = PacketReceiver::new(id, packet_deserializer); + let decision_maker = DecisionMaker::new(cluster_info.id(), poh_recorder); + SchedulerHandle::new_thread_local_scheduler( + id, + decision_maker, + unprocessed_transaction_storage, + bank_forks, + packet_receiver, + ) } - #[allow(clippy::too_many_arguments)] fn process_loop( - packet_deserializer: &mut PacketDeserializer, - poh_recorder: &Arc>, - cluster_info: &ClusterInfo, - recv_start: &mut Instant, id: u32, - transaction_status_sender: Option, - gossip_vote_sender: ReplayVoteSender, - data_budget: &DataBudget, - log_messages_bytes_limit: Option, - connection_cache: Arc, - bank_forks: &Arc>, - mut unprocessed_transaction_storage: UnprocessedTransactionStorage, + mut scheduler_handle: SchedulerHandle, + forward_executor: ForwardExecutor, + consume_executor: ConsumeExecutor, ) { - let recorder = poh_recorder.read().unwrap().recorder(); - let socket = UdpSocket::bind("0.0.0.0:0").unwrap(); - let mut banking_stage_stats = BankingStageStats::new(id); let mut tracer_packet_stats = TracerPacketStats::new(id); - let qos_service = QosService::new(id); - let mut slot_metrics_tracker = LeaderSlotMetricsTracker::new(id); - let mut last_metrics_update = Instant::now(); loop { - let my_pubkey = cluster_info.id(); - if !unprocessed_transaction_storage.is_empty() - || last_metrics_update.elapsed() >= SLOT_BOUNDARY_CHECK_PERIOD - { - let (_, process_buffered_packets_time) = measure!( - Self::process_buffered_packets( - &my_pubkey, - &socket, - poh_recorder, - cluster_info, - &mut unprocessed_transaction_storage, - &transaction_status_sender, - &gossip_vote_sender, - &banking_stage_stats, - &recorder, - data_budget, - &qos_service, - &mut slot_metrics_tracker, - log_messages_bytes_limit, - &connection_cache, - &mut tracer_packet_stats, - bank_forks, - ), - "process_buffered_packets", - ); - slot_metrics_tracker - .increment_process_buffered_packets_us(process_buffered_packets_time.as_us()); - last_metrics_update = Instant::now(); + // Do scheduled work (processing packets) + if let Err(err) = scheduler_handle.do_scheduled_work( + &consume_executor, + &forward_executor, + &mut tracer_packet_stats, + &mut slot_metrics_tracker, + ) { + warn!("Banking stage scheduler error: {:?}", err); + break; } - tracer_packet_stats.report(1000); - - // Gossip thread will almost always not wait because the transaction storage will most likely not be empty - let recv_timeout = if !unprocessed_transaction_storage.is_empty() { - // If there are buffered packets, run the equivalent of try_recv to try reading more - // packets. This prevents starving BankingStage::consume_buffered_packets due to - // buffered_packet_batches containing transactions that exceed the cost model for - // the current bank. - Duration::from_millis(0) - } else { - // Default wait time - Duration::from_millis(100) - }; - - let (res, receive_and_buffer_packets_time) = measure!( - Self::receive_and_buffer_packets( - packet_deserializer, - recv_start, - recv_timeout, - id, - &mut unprocessed_transaction_storage, - &mut banking_stage_stats, - &mut tracer_packet_stats, - &mut slot_metrics_tracker, - ), - "receive_and_buffer_packets", - ); - slot_metrics_tracker - .increment_receive_and_buffer_packets_us(receive_and_buffer_packets_time.as_us()); - - match res { - Ok(()) | Err(RecvTimeoutError::Timeout) => (), - Err(RecvTimeoutError::Disconnected) => break, + // Do any necessary updates - check if scheduler is still valid + if let Err(err) = scheduler_handle.tick() { + warn!("Banking stage scheduler error: {:?}", err); + break; } - banking_stage_stats.report(1000); + + tracer_packet_stats.report(1000); } } @@ -1115,790 +647,8 @@ impl BankingStage { ) } - fn record_transactions( - bank_slot: Slot, - transactions: Vec, - recorder: &TransactionRecorder, - ) -> RecordTransactionsSummary { - let mut record_transactions_timings = RecordTransactionsTimings::default(); - let mut starting_transaction_index = None; - - if !transactions.is_empty() { - let num_to_record = transactions.len(); - inc_new_counter_info!("banking_stage-record_count", 1); - inc_new_counter_info!("banking_stage-record_transactions", num_to_record); - - let (hash, hash_time) = measure!(hash_transactions(&transactions), "hash"); - record_transactions_timings.hash_us = hash_time.as_us(); - - let (res, poh_record_time) = - measure!(recorder.record(bank_slot, hash, transactions), "hash"); - record_transactions_timings.poh_record_us = poh_record_time.as_us(); - - match res { - Ok(starting_index) => { - starting_transaction_index = starting_index; - } - Err(PohRecorderError::MaxHeightReached) => { - inc_new_counter_info!("banking_stage-max_height_reached", 1); - inc_new_counter_info!( - "banking_stage-max_height_reached_num_to_commit", - num_to_record - ); - return RecordTransactionsSummary { - record_transactions_timings, - result: Err(PohRecorderError::MaxHeightReached), - starting_transaction_index: None, - }; - } - Err(e) => panic!("Poh recorder returned unexpected error: {:?}", e), - } - } - - RecordTransactionsSummary { - record_transactions_timings, - result: Ok(()), - starting_transaction_index, - } - } - - #[allow(clippy::too_many_arguments)] - fn commit_transactions( - batch: &TransactionBatch, - loaded_transactions: &mut [TransactionLoadResult], - execution_results: Vec, - sanitized_txs: &[SanitizedTransaction], - starting_transaction_index: Option, - bank: &Arc, - pre_balance_info: &mut PreBalanceInfo, - execute_and_commit_timings: &mut LeaderExecuteAndCommitTimings, - transaction_status_sender: &Option, - gossip_vote_sender: &ReplayVoteSender, - signature_count: u64, - executed_transactions_count: usize, - executed_with_successful_result_count: usize, - ) -> (u64, Vec) { - inc_new_counter_info!( - "banking_stage-record_transactions_num_to_commit", - executed_transactions_count - ); - - let (last_blockhash, lamports_per_signature) = - bank.last_blockhash_and_lamports_per_signature(); - - let (tx_results, commit_time) = measure!( - bank.commit_transactions( - sanitized_txs, - loaded_transactions, - execution_results, - last_blockhash, - lamports_per_signature, - CommitTransactionCounts { - committed_transactions_count: executed_transactions_count as u64, - committed_with_failure_result_count: executed_transactions_count - .saturating_sub(executed_with_successful_result_count) - as u64, - signature_count, - }, - &mut execute_and_commit_timings.execute_timings, - ), - "commit", - ); - let commit_time_us = commit_time.as_us(); - execute_and_commit_timings.commit_us = commit_time_us; - - let commit_transaction_statuses = tx_results - .execution_results - .iter() - .map(|execution_result| match execution_result.details() { - Some(details) => CommitTransactionDetails::Committed { - compute_units: details.executed_units, - }, - None => CommitTransactionDetails::NotCommitted, - }) - .collect(); - - let (_, find_and_send_votes_time) = measure!( - { - bank_utils::find_and_send_votes( - sanitized_txs, - &tx_results, - Some(gossip_vote_sender), - ); - Self::collect_balances_and_send_status_batch( - transaction_status_sender, - tx_results, - bank, - batch, - pre_balance_info, - starting_transaction_index, - ); - }, - "find_and_send_votes", - ); - execute_and_commit_timings.find_and_send_votes_us = find_and_send_votes_time.as_us(); - (commit_time_us, commit_transaction_statuses) - } - - fn collect_balances_and_send_status_batch( - transaction_status_sender: &Option, - tx_results: TransactionResults, - bank: &Arc, - batch: &TransactionBatch, - pre_balance_info: &mut PreBalanceInfo, - starting_transaction_index: Option, - ) { - if let Some(transaction_status_sender) = transaction_status_sender { - let txs = batch.sanitized_transactions().to_vec(); - let post_balances = bank.collect_balances(batch); - let post_token_balances = - collect_token_balances(bank, batch, &mut pre_balance_info.mint_decimals); - let mut transaction_index = starting_transaction_index.unwrap_or_default(); - let batch_transaction_indexes: Vec<_> = tx_results - .execution_results - .iter() - .map(|result| { - if result.was_executed() { - let this_transaction_index = transaction_index; - saturating_add_assign!(transaction_index, 1); - this_transaction_index - } else { - 0 - } - }) - .collect(); - transaction_status_sender.send_transaction_status_batch( - bank.clone(), - txs, - tx_results.execution_results, - TransactionBalancesSet::new( - std::mem::take(&mut pre_balance_info.native), - post_balances, - ), - TransactionTokenBalancesSet::new( - std::mem::take(&mut pre_balance_info.token), - post_token_balances, - ), - tx_results.rent_debits, - batch_transaction_indexes, - ); - } - } - - fn execute_and_commit_transactions_locked( - bank: &Arc, - poh: &TransactionRecorder, - batch: &TransactionBatch, - transaction_status_sender: &Option, - gossip_vote_sender: &ReplayVoteSender, - log_messages_bytes_limit: Option, - ) -> ExecuteAndCommitTransactionsOutput { - let mut execute_and_commit_timings = LeaderExecuteAndCommitTimings::default(); - - let mut pre_balance_info = PreBalanceInfo::default(); - let (_, collect_balances_time) = measure!( - { - // If the extra meta-data services are enabled for RPC, collect the - // pre-balances for native and token programs. - if transaction_status_sender.is_some() { - pre_balance_info.native = bank.collect_balances(batch); - pre_balance_info.token = - collect_token_balances(bank, batch, &mut pre_balance_info.mint_decimals) - } - }, - "collect_balances", - ); - execute_and_commit_timings.collect_balances_us = collect_balances_time.as_us(); - - let (load_and_execute_transactions_output, load_execute_time) = measure!( - bank.load_and_execute_transactions( - batch, - MAX_PROCESSING_AGE, - transaction_status_sender.is_some(), - transaction_status_sender.is_some(), - transaction_status_sender.is_some(), - &mut execute_and_commit_timings.execute_timings, - None, // account_overrides - log_messages_bytes_limit - ), - "load_execute", - ); - execute_and_commit_timings.load_execute_us = load_execute_time.as_us(); - - let LoadAndExecuteTransactionsOutput { - mut loaded_transactions, - execution_results, - mut retryable_transaction_indexes, - executed_transactions_count, - executed_with_successful_result_count, - signature_count, - error_counters, - .. - } = load_and_execute_transactions_output; - - let transactions_attempted_execution_count = execution_results.len(); - let (executed_transactions, execution_results_to_transactions_time): (Vec<_>, Measure) = measure!( - execution_results - .iter() - .zip(batch.sanitized_transactions()) - .filter_map(|(execution_result, tx)| { - if execution_result.was_executed() { - Some(tx.to_versioned_transaction()) - } else { - None - } - }) - .collect(), - "execution_results_to_transactions", - ); - - let (freeze_lock, freeze_lock_time) = measure!(bank.freeze_lock(), "freeze_lock"); - execute_and_commit_timings.freeze_lock_us = freeze_lock_time.as_us(); - - let (record_transactions_summary, record_time) = measure!( - Self::record_transactions(bank.slot(), executed_transactions, poh), - "record_transactions", - ); - execute_and_commit_timings.record_us = record_time.as_us(); - - let RecordTransactionsSummary { - result: record_transactions_result, - record_transactions_timings, - starting_transaction_index, - } = record_transactions_summary; - execute_and_commit_timings.record_transactions_timings = RecordTransactionsTimings { - execution_results_to_transactions_us: execution_results_to_transactions_time.as_us(), - ..record_transactions_timings - }; - - if let Err(recorder_err) = record_transactions_result { - inc_new_counter_info!( - "banking_stage-record_transactions_retryable_record_txs", - executed_transactions_count - ); - - retryable_transaction_indexes.extend(execution_results.iter().enumerate().filter_map( - |(index, execution_result)| execution_result.was_executed().then_some(index), - )); - - return ExecuteAndCommitTransactionsOutput { - transactions_attempted_execution_count, - executed_transactions_count, - executed_with_successful_result_count, - retryable_transaction_indexes, - commit_transactions_result: Err(recorder_err), - execute_and_commit_timings, - error_counters, - }; - } - - let sanitized_txs = batch.sanitized_transactions(); - let (commit_time_us, commit_transaction_statuses) = if executed_transactions_count != 0 { - Self::commit_transactions( - batch, - &mut loaded_transactions, - execution_results, - sanitized_txs, - starting_transaction_index, - bank, - &mut pre_balance_info, - &mut execute_and_commit_timings, - transaction_status_sender, - gossip_vote_sender, - signature_count, - executed_transactions_count, - executed_with_successful_result_count, - ) - } else { - ( - 0, - vec![CommitTransactionDetails::NotCommitted; execution_results.len()], - ) - }; - - drop(freeze_lock); - - debug!( - "bank: {} process_and_record_locked: {}us record: {}us commit: {}us txs_len: {}", - bank.slot(), - load_execute_time.as_us(), - record_time.as_us(), - commit_time_us, - sanitized_txs.len(), - ); - - debug!( - "execute_and_commit_transactions_locked: {:?}", - execute_and_commit_timings.execute_timings, - ); - - debug_assert_eq!( - commit_transaction_statuses.len(), - transactions_attempted_execution_count - ); - - ExecuteAndCommitTransactionsOutput { - transactions_attempted_execution_count, - executed_transactions_count, - executed_with_successful_result_count, - retryable_transaction_indexes, - commit_transactions_result: Ok(commit_transaction_statuses), - execute_and_commit_timings, - error_counters, - } - } - - pub fn process_and_record_transactions( - bank: &Arc, - txs: &[SanitizedTransaction], - poh: &TransactionRecorder, - chunk_offset: usize, - transaction_status_sender: &Option, - gossip_vote_sender: &ReplayVoteSender, - qos_service: &QosService, - log_messages_bytes_limit: Option, - ) -> ProcessTransactionBatchOutput { - let ( - (transaction_costs, transactions_qos_results, cost_model_throttled_transactions_count), - cost_model_time, - ) = measure!(qos_service.select_and_accumulate_transaction_costs(bank, txs)); - - // Only lock accounts for those transactions are selected for the block; - // Once accounts are locked, other threads cannot encode transactions that will modify the - // same account state - let (batch, lock_time) = measure!( - bank.prepare_sanitized_batch_with_results(txs, transactions_qos_results.iter()) - ); - - // retryable_txs includes AccountInUse, WouldExceedMaxBlockCostLimit - // WouldExceedMaxAccountCostLimit, WouldExceedMaxVoteCostLimit - // and WouldExceedMaxAccountDataCostLimit - let mut execute_and_commit_transactions_output = - Self::execute_and_commit_transactions_locked( - bank, - poh, - &batch, - transaction_status_sender, - gossip_vote_sender, - log_messages_bytes_limit, - ); - - // Once the accounts are new transactions can enter the pipeline to process them - let (_, unlock_time) = measure!(drop(batch)); - - let ExecuteAndCommitTransactionsOutput { - ref mut retryable_transaction_indexes, - ref execute_and_commit_timings, - ref commit_transactions_result, - .. - } = execute_and_commit_transactions_output; - - QosService::update_or_remove_transaction_costs( - transaction_costs.iter(), - transactions_qos_results.iter(), - commit_transactions_result.as_ref().ok(), - bank, - ); - - retryable_transaction_indexes - .iter_mut() - .for_each(|x| *x += chunk_offset); - - let (cu, us) = - Self::accumulate_execute_units_and_time(&execute_and_commit_timings.execute_timings); - qos_service.accumulate_actual_execute_cu(cu); - qos_service.accumulate_actual_execute_time(us); - - // reports qos service stats for this batch - qos_service.report_metrics(bank.clone()); - - debug!( - "bank: {} lock: {}us unlock: {}us txs_len: {}", - bank.slot(), - lock_time.as_us(), - unlock_time.as_us(), - txs.len(), - ); - - ProcessTransactionBatchOutput { - cost_model_throttled_transactions_count, - cost_model_us: cost_model_time.as_us(), - execute_and_commit_transactions_output, - } - } - - fn accumulate_execute_units_and_time(execute_timings: &ExecuteTimings) -> (u64, u64) { - let (units, times): (Vec<_>, Vec<_>) = execute_timings - .details - .per_program_timings - .values() - .map(|program_timings| { - ( - program_timings.accumulated_units, - program_timings.accumulated_us, - ) - }) - .unzip(); - (units.iter().sum(), times.iter().sum()) - } - - /// Sends transactions to the bank. - /// - /// Returns the number of transactions successfully processed by the bank, which may be less - /// than the total number if max PoH height was reached and the bank halted - fn process_transactions( - bank: &Arc, - bank_creation_time: &Instant, - transactions: &[SanitizedTransaction], - poh: &TransactionRecorder, - transaction_status_sender: &Option, - gossip_vote_sender: &ReplayVoteSender, - qos_service: &QosService, - log_messages_bytes_limit: Option, - ) -> ProcessTransactionsSummary { - let mut chunk_start = 0; - let mut all_retryable_tx_indexes = vec![]; - // All the transactions that attempted execution. See description of - // struct ProcessTransactionsSummary above for possible outcomes. - let mut total_transactions_attempted_execution_count: usize = 0; - // All transactions that were executed and committed - let mut total_committed_transactions_count: usize = 0; - // All transactions that were executed and committed with a successful result - let mut total_committed_transactions_with_successful_result_count: usize = 0; - // All transactions that were executed but then failed record because the - // slot ended - let mut total_failed_commit_count: usize = 0; - let mut total_cost_model_throttled_transactions_count: usize = 0; - let mut total_cost_model_us: u64 = 0; - let mut total_execute_and_commit_timings = LeaderExecuteAndCommitTimings::default(); - let mut total_error_counters = TransactionErrorMetrics::default(); - let mut reached_max_poh_height = false; - while chunk_start != transactions.len() { - let chunk_end = std::cmp::min( - transactions.len(), - chunk_start + MAX_NUM_TRANSACTIONS_PER_BATCH, - ); - let process_transaction_batch_output = Self::process_and_record_transactions( - bank, - &transactions[chunk_start..chunk_end], - poh, - chunk_start, - transaction_status_sender, - gossip_vote_sender, - qos_service, - log_messages_bytes_limit, - ); - - let ProcessTransactionBatchOutput { - cost_model_throttled_transactions_count: new_cost_model_throttled_transactions_count, - cost_model_us: new_cost_model_us, - execute_and_commit_transactions_output, - } = process_transaction_batch_output; - total_cost_model_throttled_transactions_count = - total_cost_model_throttled_transactions_count - .saturating_add(new_cost_model_throttled_transactions_count); - total_cost_model_us = total_cost_model_us.saturating_add(new_cost_model_us); - - let ExecuteAndCommitTransactionsOutput { - transactions_attempted_execution_count: new_transactions_attempted_execution_count, - executed_transactions_count: new_executed_transactions_count, - executed_with_successful_result_count: new_executed_with_successful_result_count, - retryable_transaction_indexes: new_retryable_transaction_indexes, - commit_transactions_result: new_commit_transactions_result, - execute_and_commit_timings: new_execute_and_commit_timings, - error_counters: new_error_counters, - .. - } = execute_and_commit_transactions_output; - - total_execute_and_commit_timings.accumulate(&new_execute_and_commit_timings); - total_error_counters.accumulate(&new_error_counters); - total_transactions_attempted_execution_count = - total_transactions_attempted_execution_count - .saturating_add(new_transactions_attempted_execution_count); - - trace!( - "process_transactions result: {:?}", - new_commit_transactions_result - ); - - if new_commit_transactions_result.is_ok() { - total_committed_transactions_count = total_committed_transactions_count - .saturating_add(new_executed_transactions_count); - total_committed_transactions_with_successful_result_count = - total_committed_transactions_with_successful_result_count - .saturating_add(new_executed_with_successful_result_count); - } else { - total_failed_commit_count = - total_failed_commit_count.saturating_add(new_executed_transactions_count); - } - - // Add the retryable txs (transactions that errored in a way that warrants a retry) - // to the list of unprocessed txs. - all_retryable_tx_indexes.extend_from_slice(&new_retryable_transaction_indexes); - - // If `bank_creation_time` is None, it's a test so ignore the option so - // allow processing - let should_bank_still_be_processing_txs = - Bank::should_bank_still_be_processing_txs(bank_creation_time, bank.ns_per_slot); - match ( - new_commit_transactions_result, - should_bank_still_be_processing_txs, - ) { - (Err(PohRecorderError::MaxHeightReached), _) | (_, false) => { - info!( - "process transactions: max height reached slot: {} height: {}", - bank.slot(), - bank.tick_height() - ); - // process_and_record_transactions has returned all retryable errors in - // transactions[chunk_start..chunk_end], so we just need to push the remaining - // transactions into the unprocessed queue. - all_retryable_tx_indexes.extend(chunk_end..transactions.len()); - reached_max_poh_height = true; - break; - } - _ => (), - } - // Don't exit early on any other type of error, continue processing... - chunk_start = chunk_end; - } - - ProcessTransactionsSummary { - reached_max_poh_height, - transactions_attempted_execution_count: total_transactions_attempted_execution_count, - committed_transactions_count: total_committed_transactions_count, - committed_transactions_with_successful_result_count: - total_committed_transactions_with_successful_result_count, - failed_commit_count: total_failed_commit_count, - retryable_transaction_indexes: all_retryable_tx_indexes, - cost_model_throttled_transactions_count: total_cost_model_throttled_transactions_count, - cost_model_us: total_cost_model_us, - execute_and_commit_timings: total_execute_and_commit_timings, - error_counters: total_error_counters, - } - } - - /// This function creates a filter of transaction results with Ok() for every pending - /// transaction. The non-pending transactions are marked with TransactionError - fn prepare_filter_for_pending_transactions( - transactions_len: usize, - pending_tx_indexes: &[usize], - ) -> Vec> { - let mut mask = vec![Err(TransactionError::BlockhashNotFound); transactions_len]; - pending_tx_indexes.iter().for_each(|x| mask[*x] = Ok(())); - mask - } - - /// This function returns a vector containing index of all valid transactions. A valid - /// transaction has result Ok() as the value - fn filter_valid_transaction_indexes(valid_txs: &[TransactionCheckResult]) -> Vec { - valid_txs - .iter() - .enumerate() - .filter_map(|(index, (x, _h))| if x.is_ok() { Some(index) } else { None }) - .collect_vec() - } - - /// This function filters pending packets that are still valid - /// # Arguments - /// * `transactions` - a batch of transactions deserialized from packets - /// * `pending_indexes` - identifies which indexes in the `transactions` list are still pending - fn filter_pending_packets_from_pending_txs( - bank: &Arc, - transactions: &[SanitizedTransaction], - pending_indexes: &[usize], - ) -> Vec { - let filter = - Self::prepare_filter_for_pending_transactions(transactions.len(), pending_indexes); - - let results = bank.check_transactions_with_forwarding_delay( - transactions, - &filter, - FORWARD_TRANSACTIONS_TO_LEADER_AT_SLOT_OFFSET, - ); - - Self::filter_valid_transaction_indexes(&results) - } - - #[allow(clippy::too_many_arguments)] - fn process_packets_transactions<'a>( - bank: &'a Arc, - bank_creation_time: &Instant, - poh: &'a TransactionRecorder, - sanitized_transactions: &[SanitizedTransaction], - transaction_status_sender: &Option, - gossip_vote_sender: &'a ReplayVoteSender, - banking_stage_stats: &'a BankingStageStats, - qos_service: &'a QosService, - slot_metrics_tracker: &'a mut LeaderSlotMetricsTracker, - log_messages_bytes_limit: Option, - ) -> ProcessTransactionsSummary { - // Process transactions - let (mut process_transactions_summary, process_transactions_time) = measure!( - Self::process_transactions( - bank, - bank_creation_time, - sanitized_transactions, - poh, - transaction_status_sender, - gossip_vote_sender, - qos_service, - log_messages_bytes_limit, - ), - "process_transaction_time", - ); - let process_transactions_us = process_transactions_time.as_us(); - slot_metrics_tracker.increment_process_transactions_us(process_transactions_us); - banking_stage_stats - .transaction_processing_elapsed - .fetch_add(process_transactions_us, Ordering::Relaxed); - - let ProcessTransactionsSummary { - ref retryable_transaction_indexes, - ref error_counters, - .. - } = process_transactions_summary; - - slot_metrics_tracker.accumulate_process_transactions_summary(&process_transactions_summary); - slot_metrics_tracker.accumulate_transaction_errors(error_counters); - - let retryable_tx_count = retryable_transaction_indexes.len(); - inc_new_counter_info!("banking_stage-unprocessed_transactions", retryable_tx_count); - - // Filter out the retryable transactions that are too old - let (filtered_retryable_transaction_indexes, filter_retryable_packets_time) = measure!( - Self::filter_pending_packets_from_pending_txs( - bank, - sanitized_transactions, - retryable_transaction_indexes, - ), - "filter_pending_packets_time", - ); - let filter_retryable_packets_us = filter_retryable_packets_time.as_us(); - slot_metrics_tracker.increment_filter_retryable_packets_us(filter_retryable_packets_us); - banking_stage_stats - .filter_pending_packets_elapsed - .fetch_add(filter_retryable_packets_us, Ordering::Relaxed); - - let retryable_packets_filtered_count = retryable_transaction_indexes - .len() - .saturating_sub(filtered_retryable_transaction_indexes.len()); - slot_metrics_tracker - .increment_retryable_packets_filtered_count(retryable_packets_filtered_count as u64); - - inc_new_counter_info!( - "banking_stage-dropped_tx_before_forwarding", - retryable_transaction_indexes - .len() - .saturating_sub(filtered_retryable_transaction_indexes.len()) - ); - - process_transactions_summary.retryable_transaction_indexes = - filtered_retryable_transaction_indexes; - process_transactions_summary - } - - #[allow(clippy::too_many_arguments)] - /// Receive incoming packets, push into unprocessed buffer with packet indexes - fn receive_and_buffer_packets( - packet_deserializer: &mut PacketDeserializer, - recv_start: &mut Instant, - recv_timeout: Duration, - id: u32, - unprocessed_transaction_storage: &mut UnprocessedTransactionStorage, - banking_stage_stats: &mut BankingStageStats, - tracer_packet_stats: &mut TracerPacketStats, - slot_metrics_tracker: &mut LeaderSlotMetricsTracker, - ) -> Result<(), RecvTimeoutError> { - let mut recv_time = Measure::start("receive_and_buffer_packets_recv"); - let ReceivePacketResults { - deserialized_packets, - new_tracer_stats_option, - passed_sigverify_count, - failed_sigverify_count, - } = packet_deserializer.handle_received_packets( - recv_timeout, - unprocessed_transaction_storage.max_receive_size(), - )?; - let packet_count = deserialized_packets.len(); - debug!( - "@{:?} process start stalled for: {:?}ms txs: {} id: {}", - timestamp(), - duration_as_ms(&recv_start.elapsed()), - packet_count, - id, - ); - - if let Some(new_sigverify_stats) = &new_tracer_stats_option { - tracer_packet_stats.aggregate_sigverify_tracer_packet_stats(new_sigverify_stats); - } - - // Track all the packets incoming from sigverify, both valid and invalid - slot_metrics_tracker.increment_total_new_valid_packets(passed_sigverify_count); - slot_metrics_tracker.increment_newly_failed_sigverify_count(failed_sigverify_count); - - let mut dropped_packets_count = 0; - let mut newly_buffered_packets_count = 0; - Self::push_unprocessed( - unprocessed_transaction_storage, - deserialized_packets, - &mut dropped_packets_count, - &mut newly_buffered_packets_count, - banking_stage_stats, - slot_metrics_tracker, - tracer_packet_stats, - ); - recv_time.stop(); - - banking_stage_stats - .receive_and_buffer_packets_elapsed - .fetch_add(recv_time.as_us(), Ordering::Relaxed); - banking_stage_stats - .receive_and_buffer_packets_count - .fetch_add(packet_count, Ordering::Relaxed); - banking_stage_stats - .dropped_packets_count - .fetch_add(dropped_packets_count, Ordering::Relaxed); - banking_stage_stats - .newly_buffered_packets_count - .fetch_add(newly_buffered_packets_count, Ordering::Relaxed); - banking_stage_stats - .current_buffered_packets_count - .swap(unprocessed_transaction_storage.len(), Ordering::Relaxed); - *recv_start = Instant::now(); - Ok(()) - } - - fn push_unprocessed( - unprocessed_transaction_storage: &mut UnprocessedTransactionStorage, - deserialized_packets: Vec, - dropped_packets_count: &mut usize, - newly_buffered_packets_count: &mut usize, - banking_stage_stats: &mut BankingStageStats, - slot_metrics_tracker: &mut LeaderSlotMetricsTracker, - tracer_packet_stats: &mut TracerPacketStats, - ) { - if !deserialized_packets.is_empty() { - let _ = banking_stage_stats - .batch_packet_indexes_len - .increment(deserialized_packets.len() as u64); - - *newly_buffered_packets_count += deserialized_packets.len(); - slot_metrics_tracker - .increment_newly_buffered_packets_count(deserialized_packets.len() as u64); - - let insert_packet_batches_summary = - unprocessed_transaction_storage.insert_batch(deserialized_packets); - slot_metrics_tracker - .accumulate_insert_packet_batches_summary(&insert_packet_batches_summary); - saturating_add_assign!( - *dropped_packets_count, - insert_packet_batches_summary.total_dropped_packets() - ); - tracer_packet_stats.increment_total_exceeded_banking_stage_buffer( - insert_packet_batches_summary.dropped_tracer_packets(), - ); - } + pub fn num_non_vote_threads() -> u32 { + Self::num_threads() - NUM_VOTE_PROCESSING_THREADS } pub fn join(self) -> thread::Result<()> { @@ -1908,45 +658,3 @@ impl BankingStage { Ok(()) } } - -pub(crate) fn next_leader_tpu( - cluster_info: &ClusterInfo, - poh_recorder: &RwLock, -) -> Option<(Pubkey, std::net::SocketAddr)> { - next_leader_x(cluster_info, poh_recorder, |leader| leader.tpu) -} - -fn next_leader_tpu_forwards( - cluster_info: &ClusterInfo, - poh_recorder: &RwLock, -) -> Option<(Pubkey, std::net::SocketAddr)> { - next_leader_x(cluster_info, poh_recorder, |leader| leader.tpu_forwards) -} - -pub(crate) fn next_leader_tpu_vote( - cluster_info: &ClusterInfo, - poh_recorder: &RwLock, -) -> Option<(Pubkey, std::net::SocketAddr)> { - next_leader_x(cluster_info, poh_recorder, |leader| leader.tpu_vote) -} - -fn next_leader_x( - cluster_info: &ClusterInfo, - poh_recorder: &RwLock, - port_selector: F, -) -> Option<(Pubkey, std::net::SocketAddr)> -where - F: FnOnce(&ContactInfo) -> SocketAddr, -{ - let leader_pubkey = poh_recorder - .read() - .unwrap() - .leader_after_n_slots(FORWARD_TRANSACTIONS_TO_LEADER_AT_SLOT_OFFSET); - if let Some(leader_pubkey) = leader_pubkey { - cluster_info - .lookup_contact_info(&leader_pubkey, port_selector) - .map(|addr| (leader_pubkey, addr)) - } else { - None - } -} diff --git a/core/src/banking_stage/commit_executor.rs b/core/src/banking_stage/commit_executor.rs new file mode 100644 index 00000000000000..fb37c223f584cb --- /dev/null +++ b/core/src/banking_stage/commit_executor.rs @@ -0,0 +1,165 @@ +use { + super::{CommitTransactionDetails, PreBalanceInfo}, + crate::leader_slot_banking_stage_timing_metrics::LeaderExecuteAndCommitTimings, + solana_ledger::{ + blockstore_processor::TransactionStatusSender, token_balances::collect_token_balances, + }, + solana_measure::measure, + solana_runtime::{ + accounts::TransactionLoadResult, + bank::{ + Bank, CommitTransactionCounts, TransactionBalancesSet, TransactionExecutionResult, + TransactionResults, + }, + bank_utils, + transaction_batch::TransactionBatch, + vote_sender_types::ReplayVoteSender, + }, + solana_sdk::{saturating_add_assign, transaction::SanitizedTransaction}, + solana_transaction_status::token_balances::TransactionTokenBalancesSet, + std::sync::Arc, +}; + +pub struct CommitExecutor { + transaction_status_sender: Option, + gossip_vote_sender: ReplayVoteSender, +} + +impl CommitExecutor { + pub fn new( + transaction_status_sender: Option, + gossip_vote_sender: ReplayVoteSender, + ) -> Self { + Self { + transaction_status_sender, + gossip_vote_sender, + } + } + + pub fn has_status_sender(&self) -> bool { + self.transaction_status_sender.is_some() + } + + #[allow(clippy::too_many_arguments)] + pub(super) fn commit_transactions( + &self, + batch: &TransactionBatch, + loaded_transactions: &mut [TransactionLoadResult], + execution_results: Vec, + sanitized_txs: &[SanitizedTransaction], + starting_transaction_index: Option, + bank: &Arc, + pre_balance_info: &mut PreBalanceInfo, + execute_and_commit_timings: &mut LeaderExecuteAndCommitTimings, + signature_count: u64, + executed_transactions_count: usize, + executed_with_successful_result_count: usize, + ) -> (u64, Vec) { + inc_new_counter_info!( + "banking_stage-record_transactions_num_to_commit", + executed_transactions_count + ); + + let (last_blockhash, lamports_per_signature) = + bank.last_blockhash_and_lamports_per_signature(); + + let (tx_results, commit_time) = measure!( + bank.commit_transactions( + sanitized_txs, + loaded_transactions, + execution_results, + last_blockhash, + lamports_per_signature, + CommitTransactionCounts { + committed_transactions_count: executed_transactions_count as u64, + committed_with_failure_result_count: executed_transactions_count + .saturating_sub(executed_with_successful_result_count) + as u64, + signature_count, + }, + &mut execute_and_commit_timings.execute_timings, + ), + "commit", + ); + let commit_time_us = commit_time.as_us(); + execute_and_commit_timings.commit_us = commit_time_us; + + let commit_transaction_statuses = tx_results + .execution_results + .iter() + .map(|execution_result| match execution_result.details() { + Some(details) => CommitTransactionDetails::Committed { + compute_units: details.executed_units, + }, + None => CommitTransactionDetails::NotCommitted, + }) + .collect(); + + let (_, find_and_send_votes_time) = measure!( + { + bank_utils::find_and_send_votes( + sanitized_txs, + &tx_results, + Some(&self.gossip_vote_sender), + ); + self.collect_balances_and_send_status_batch( + tx_results, + bank, + batch, + pre_balance_info, + starting_transaction_index, + ); + }, + "find_and_send_votes", + ); + execute_and_commit_timings.find_and_send_votes_us = find_and_send_votes_time.as_us(); + (commit_time_us, commit_transaction_statuses) + } + + fn collect_balances_and_send_status_batch( + &self, + tx_results: TransactionResults, + bank: &Arc, + batch: &TransactionBatch, + pre_balance_info: &mut PreBalanceInfo, + starting_transaction_index: Option, + ) { + if let Some(transaction_status_sender) = &self.transaction_status_sender { + error!("doing send status batch..."); + let txs = batch.sanitized_transactions().to_vec(); + let post_balances = bank.collect_balances(batch); + let post_token_balances = + collect_token_balances(bank, batch, &mut pre_balance_info.mint_decimals); + let mut transaction_index = starting_transaction_index.unwrap_or_default(); + let batch_transaction_indexes: Vec<_> = tx_results + .execution_results + .iter() + .map(|result| { + if result.was_executed() { + let this_transaction_index = transaction_index; + saturating_add_assign!(transaction_index, 1); + this_transaction_index + } else { + 0 + } + }) + .collect(); + transaction_status_sender.send_transaction_status_batch( + bank.clone(), + txs, + tx_results.execution_results, + TransactionBalancesSet::new( + std::mem::take(&mut pre_balance_info.native), + post_balances, + ), + TransactionTokenBalancesSet::new( + std::mem::take(&mut pre_balance_info.token), + post_token_balances, + ), + tx_results.rent_debits, + batch_transaction_indexes, + ); + error!("finished send status batch..."); + } + } +} diff --git a/core/src/banking_stage/consume_executor.rs b/core/src/banking_stage/consume_executor.rs new file mode 100644 index 00000000000000..f1dee4303da0d2 --- /dev/null +++ b/core/src/banking_stage/consume_executor.rs @@ -0,0 +1,515 @@ +use { + super::BankingStageStats, + crate::{ + banking_stage::{ + commit_executor::CommitExecutor, + record_executor::{RecordExecutor, RecordTransactionsSummary}, + CommitTransactionDetails, ExecuteAndCommitTransactionsOutput, PreBalanceInfo, + ProcessTransactionBatchOutput, MAX_NUM_TRANSACTIONS_PER_BATCH, + }, + leader_slot_banking_stage_metrics::{LeaderSlotMetricsTracker, ProcessTransactionsSummary}, + leader_slot_banking_stage_timing_metrics::{ + LeaderExecuteAndCommitTimings, RecordTransactionsTimings, + }, + qos_service::QosService, + }, + itertools::Itertools, + solana_ledger::token_balances::collect_token_balances, + solana_measure::{measure, measure_us}, + solana_poh::poh_recorder::{BankStart, PohRecorderError}, + solana_program_runtime::timings::ExecuteTimings, + solana_runtime::{ + bank::{Bank, LoadAndExecuteTransactionsOutput, TransactionCheckResult}, + transaction_batch::TransactionBatch, + transaction_error_metrics::TransactionErrorMetrics, + }, + solana_sdk::{ + clock::{FORWARD_TRANSACTIONS_TO_LEADER_AT_SLOT_OFFSET, MAX_PROCESSING_AGE}, + transaction::{self, SanitizedTransaction, TransactionError}, + }, + std::sync::{atomic::Ordering, Arc}, +}; + +pub struct ConsumeExecutor { + record_executor: RecordExecutor, + commit_executor: CommitExecutor, + qos_service: QosService, + log_messages_bytes_limit: Option, +} + +impl ConsumeExecutor { + pub fn new( + record_executor: RecordExecutor, + commit_executor: CommitExecutor, + qos_service: QosService, + log_messages_bytes_limit: Option, + ) -> Self { + Self { + record_executor, + commit_executor, + qos_service, + log_messages_bytes_limit, + } + } + + pub(crate) fn process_packets_transactions<'a>( + &self, + bank_start: &BankStart, + sanitized_transactions: &[SanitizedTransaction], + banking_stage_stats: &'a BankingStageStats, + slot_metrics_tracker: &'a mut LeaderSlotMetricsTracker, + ) -> ProcessTransactionsSummary { + // Process transactions + let (mut process_transactions_summary, process_transactions_us) = + measure_us!(self.process_transactions(bank_start, sanitized_transactions,)); + slot_metrics_tracker.increment_process_transactions_us(process_transactions_us); + banking_stage_stats + .transaction_processing_elapsed + .fetch_add(process_transactions_us, Ordering::Relaxed); + + let ProcessTransactionsSummary { + ref retryable_transaction_indexes, + ref error_counters, + .. + } = process_transactions_summary; + + slot_metrics_tracker.accumulate_process_transactions_summary(&process_transactions_summary); + slot_metrics_tracker.accumulate_transaction_errors(error_counters); + + let retryable_tx_count = retryable_transaction_indexes.len(); + inc_new_counter_info!("banking_stage-unprocessed_transactions", retryable_tx_count); + + // Filter out the retryable transactions that are too old + let (filtered_retryable_transaction_indexes, filter_retryable_packets_us) = + measure_us!(Self::filter_pending_packets_from_pending_txs( + &bank_start.working_bank, + sanitized_transactions, + retryable_transaction_indexes, + )); + slot_metrics_tracker.increment_filter_retryable_packets_us(filter_retryable_packets_us); + banking_stage_stats + .filter_pending_packets_elapsed + .fetch_add(filter_retryable_packets_us, Ordering::Relaxed); + + let retryable_packets_filtered_count = retryable_transaction_indexes + .len() + .saturating_sub(filtered_retryable_transaction_indexes.len()); + slot_metrics_tracker + .increment_retryable_packets_filtered_count(retryable_packets_filtered_count as u64); + + inc_new_counter_info!( + "banking_stage-dropped_tx_before_forwarding", + retryable_transaction_indexes + .len() + .saturating_sub(filtered_retryable_transaction_indexes.len()) + ); + + process_transactions_summary.retryable_transaction_indexes = + filtered_retryable_transaction_indexes; + process_transactions_summary + } + + /// Sends transactions to the bank. + /// + /// Returns the number of transactions successfully processed by the bank, which may be less + /// than the total number if max PoH height was reached and the bank halted + fn process_transactions( + &self, + bank_start: &BankStart, + transactions: &[SanitizedTransaction], + ) -> ProcessTransactionsSummary { + let mut chunk_start = 0; + let mut all_retryable_tx_indexes = vec![]; + // All the transactions that attempted execution. See description of + // struct ProcessTransactionsSummary above for possible outcomes. + let mut total_transactions_attempted_execution_count: usize = 0; + // All transactions that were executed and committed + let mut total_committed_transactions_count: usize = 0; + // All transactions that were executed and committed with a successful result + let mut total_committed_transactions_with_successful_result_count: usize = 0; + // All transactions that were executed but then failed record because the + // slot ended + let mut total_failed_commit_count: usize = 0; + let mut total_cost_model_throttled_transactions_count: usize = 0; + let mut total_cost_model_us: u64 = 0; + let mut total_execute_and_commit_timings = LeaderExecuteAndCommitTimings::default(); + let mut total_error_counters = TransactionErrorMetrics::default(); + let mut reached_max_poh_height = false; + while chunk_start != transactions.len() { + let chunk_end = std::cmp::min( + transactions.len(), + chunk_start + MAX_NUM_TRANSACTIONS_PER_BATCH, + ); + let process_transaction_batch_output = self.process_and_record_transactions( + &bank_start.working_bank, + &transactions[chunk_start..chunk_end], + chunk_start, + ); + + let ProcessTransactionBatchOutput { + cost_model_throttled_transactions_count: new_cost_model_throttled_transactions_count, + cost_model_us: new_cost_model_us, + execute_and_commit_transactions_output, + } = process_transaction_batch_output; + total_cost_model_throttled_transactions_count = + total_cost_model_throttled_transactions_count + .saturating_add(new_cost_model_throttled_transactions_count); + total_cost_model_us = total_cost_model_us.saturating_add(new_cost_model_us); + + let ExecuteAndCommitTransactionsOutput { + transactions_attempted_execution_count: new_transactions_attempted_execution_count, + executed_transactions_count: new_executed_transactions_count, + executed_with_successful_result_count: new_executed_with_successful_result_count, + retryable_transaction_indexes: new_retryable_transaction_indexes, + commit_transactions_result: new_commit_transactions_result, + execute_and_commit_timings: new_execute_and_commit_timings, + error_counters: new_error_counters, + .. + } = execute_and_commit_transactions_output; + + total_execute_and_commit_timings.accumulate(&new_execute_and_commit_timings); + total_error_counters.accumulate(&new_error_counters); + total_transactions_attempted_execution_count = + total_transactions_attempted_execution_count + .saturating_add(new_transactions_attempted_execution_count); + + trace!( + "process_transactions result: {:?}", + new_commit_transactions_result + ); + + if new_commit_transactions_result.is_ok() { + total_committed_transactions_count = total_committed_transactions_count + .saturating_add(new_executed_transactions_count); + total_committed_transactions_with_successful_result_count = + total_committed_transactions_with_successful_result_count + .saturating_add(new_executed_with_successful_result_count); + } else { + total_failed_commit_count = + total_failed_commit_count.saturating_add(new_executed_transactions_count); + } + + // Add the retryable txs (transactions that errored in a way that warrants a retry) + // to the list of unprocessed txs. + all_retryable_tx_indexes.extend_from_slice(&new_retryable_transaction_indexes); + + // If `bank_creation_time` is None, it's a test so ignore the option so + // allow processing + let should_bank_still_be_processing_txs = + bank_start.should_working_bank_still_be_processing_txs(); + match ( + new_commit_transactions_result, + should_bank_still_be_processing_txs, + ) { + (Err(PohRecorderError::MaxHeightReached), _) | (_, false) => { + info!( + "process transactions: max height reached slot: {} height: {}", + bank_start.working_bank.slot(), + bank_start.working_bank.tick_height() + ); + // process_and_record_transactions has returned all retryable errors in + // transactions[chunk_start..chunk_end], so we just need to push the remaining + // transactions into the unprocessed queue. + all_retryable_tx_indexes.extend(chunk_end..transactions.len()); + reached_max_poh_height = true; + break; + } + _ => (), + } + // Don't exit early on any other type of error, continue processing... + chunk_start = chunk_end; + } + + ProcessTransactionsSummary { + reached_max_poh_height, + transactions_attempted_execution_count: total_transactions_attempted_execution_count, + committed_transactions_count: total_committed_transactions_count, + committed_transactions_with_successful_result_count: + total_committed_transactions_with_successful_result_count, + failed_commit_count: total_failed_commit_count, + retryable_transaction_indexes: all_retryable_tx_indexes, + cost_model_throttled_transactions_count: total_cost_model_throttled_transactions_count, + cost_model_us: total_cost_model_us, + execute_and_commit_timings: total_execute_and_commit_timings, + error_counters: total_error_counters, + } + } + + pub fn process_and_record_transactions( + &self, + bank: &Arc, + txs: &[SanitizedTransaction], + chunk_offset: usize, + ) -> ProcessTransactionBatchOutput { + let ( + (transaction_costs, transactions_qos_results, cost_model_throttled_transactions_count), + cost_model_us, + ) = measure_us!(self + .qos_service + .select_and_accumulate_transaction_costs(bank, txs)); + + // Only lock accounts for those transactions are selected for the block; + // Once accounts are locked, other threads cannot encode transactions that will modify the + // same account state + let (batch, lock_us) = measure_us!( + bank.prepare_sanitized_batch_with_results(txs, transactions_qos_results.iter()) + ); + + // retryable_txs includes AccountInUse, WouldExceedMaxBlockCostLimit + // WouldExceedMaxAccountCostLimit, WouldExceedMaxVoteCostLimit + // and WouldExceedMaxAccountDataCostLimit + let mut execute_and_commit_transactions_output = + self.execute_and_commit_transactions_locked(bank, &batch); + + // Once the accounts are new transactions can enter the pipeline to process them + let (_, unlock_us) = measure_us!(drop(batch)); + + let ExecuteAndCommitTransactionsOutput { + ref mut retryable_transaction_indexes, + ref execute_and_commit_timings, + ref commit_transactions_result, + .. + } = execute_and_commit_transactions_output; + + QosService::update_or_remove_transaction_costs( + transaction_costs.iter(), + transactions_qos_results.iter(), + commit_transactions_result.as_ref().ok(), + bank, + ); + + retryable_transaction_indexes + .iter_mut() + .for_each(|x| *x += chunk_offset); + + let (cu, us) = + Self::accumulate_execute_units_and_time(&execute_and_commit_timings.execute_timings); + self.qos_service.accumulate_actual_execute_cu(cu); + self.qos_service.accumulate_actual_execute_time(us); + + // reports qos service stats for this batch + self.qos_service.report_metrics(bank.clone()); + + debug!( + "bank: {} lock: {}us unlock: {}us txs_len: {}", + bank.slot(), + lock_us, + unlock_us, + txs.len(), + ); + + ProcessTransactionBatchOutput { + cost_model_throttled_transactions_count, + cost_model_us, + execute_and_commit_transactions_output, + } + } + + fn execute_and_commit_transactions_locked( + &self, + bank: &Arc, + batch: &TransactionBatch, + ) -> ExecuteAndCommitTransactionsOutput { + let has_status_sender = self.commit_executor.has_status_sender(); + let mut execute_and_commit_timings = LeaderExecuteAndCommitTimings::default(); + + let mut pre_balance_info = PreBalanceInfo::default(); + let (_, collect_balances_us) = measure_us!({ + // If the extra meta-data services are enabled for RPC, collect the + // pre-balances for native and token programs. + if has_status_sender { + pre_balance_info.native = bank.collect_balances(batch); + pre_balance_info.token = + collect_token_balances(bank, batch, &mut pre_balance_info.mint_decimals) + } + }); + execute_and_commit_timings.collect_balances_us = collect_balances_us; + + let (load_and_execute_transactions_output, load_execute_us) = measure_us!(bank + .load_and_execute_transactions( + batch, + MAX_PROCESSING_AGE, + has_status_sender, + has_status_sender, + has_status_sender, + &mut execute_and_commit_timings.execute_timings, + None, // account_overrides + self.log_messages_bytes_limit + )); + execute_and_commit_timings.load_execute_us = load_execute_us; + + let LoadAndExecuteTransactionsOutput { + mut loaded_transactions, + execution_results, + mut retryable_transaction_indexes, + executed_transactions_count, + executed_with_successful_result_count, + signature_count, + error_counters, + .. + } = load_and_execute_transactions_output; + + let transactions_attempted_execution_count = execution_results.len(); + let (executed_transactions, execution_results_to_transactions_us): (Vec<_>, u64) = + measure_us!(execution_results + .iter() + .zip(batch.sanitized_transactions()) + .filter_map(|(execution_result, tx)| { + if execution_result.was_executed() { + Some(tx.to_versioned_transaction()) + } else { + None + } + }) + .collect()); + + let (freeze_lock, freeze_lock_us) = measure_us!(bank.freeze_lock()); + execute_and_commit_timings.freeze_lock_us = freeze_lock_us; + + let (record_transactions_summary, record_us) = measure_us!(self + .record_executor + .record_transactions(bank.slot(), executed_transactions)); + execute_and_commit_timings.record_us = record_us; + + let RecordTransactionsSummary { + result: record_transactions_result, + record_transactions_timings, + starting_transaction_index, + } = record_transactions_summary; + execute_and_commit_timings.record_transactions_timings = RecordTransactionsTimings { + execution_results_to_transactions_us, + ..record_transactions_timings + }; + + if let Err(recorder_err) = record_transactions_result { + inc_new_counter_info!( + "banking_stage-record_transactions_retryable_record_txs", + executed_transactions_count + ); + + retryable_transaction_indexes.extend(execution_results.iter().enumerate().filter_map( + |(index, execution_result)| execution_result.was_executed().then_some(index), + )); + + return ExecuteAndCommitTransactionsOutput { + transactions_attempted_execution_count, + executed_transactions_count, + executed_with_successful_result_count, + retryable_transaction_indexes, + commit_transactions_result: Err(recorder_err), + execute_and_commit_timings, + error_counters, + }; + } + + let sanitized_txs = batch.sanitized_transactions(); + let (commit_time_us, commit_transaction_statuses) = if executed_transactions_count != 0 { + self.commit_executor.commit_transactions( + batch, + &mut loaded_transactions, + execution_results, + sanitized_txs, + starting_transaction_index, + bank, + &mut pre_balance_info, + &mut execute_and_commit_timings, + signature_count, + executed_transactions_count, + executed_with_successful_result_count, + ) + } else { + ( + 0, + vec![CommitTransactionDetails::NotCommitted; execution_results.len()], + ) + }; + + drop(freeze_lock); + + debug!( + "bank: {} process_and_record_locked: {}us record: {}us commit: {}us txs_len: {}", + bank.slot(), + load_execute_us, + record_us, + commit_time_us, + sanitized_txs.len(), + ); + + debug!( + "execute_and_commit_transactions_locked: {:?}", + execute_and_commit_timings.execute_timings, + ); + + debug_assert_eq!( + commit_transaction_statuses.len(), + transactions_attempted_execution_count + ); + + ExecuteAndCommitTransactionsOutput { + transactions_attempted_execution_count, + executed_transactions_count, + executed_with_successful_result_count, + retryable_transaction_indexes, + commit_transactions_result: Ok(commit_transaction_statuses), + execute_and_commit_timings, + error_counters, + } + } + + fn accumulate_execute_units_and_time(execute_timings: &ExecuteTimings) -> (u64, u64) { + let (units, times): (Vec<_>, Vec<_>) = execute_timings + .details + .per_program_timings + .values() + .map(|program_timings| { + ( + program_timings.accumulated_units, + program_timings.accumulated_us, + ) + }) + .unzip(); + (units.iter().sum(), times.iter().sum()) + } + + /// This function filters pending packets that are still valid + /// # Arguments + /// * `transactions` - a batch of transactions deserialized from packets + /// * `pending_indexes` - identifies which indexes in the `transactions` list are still pending + fn filter_pending_packets_from_pending_txs( + bank: &Arc, + transactions: &[SanitizedTransaction], + pending_indexes: &[usize], + ) -> Vec { + let filter = + Self::prepare_filter_for_pending_transactions(transactions.len(), pending_indexes); + + let results = bank.check_transactions_with_forwarding_delay( + transactions.iter(), + &filter, + FORWARD_TRANSACTIONS_TO_LEADER_AT_SLOT_OFFSET, + ); + + Self::filter_valid_transaction_indexes(&results) + } + + /// This function creates a filter of transaction results with Ok() for every pending + /// transaction. The non-pending transactions are marked with TransactionError + fn prepare_filter_for_pending_transactions( + transactions_len: usize, + pending_tx_indexes: &[usize], + ) -> Vec> { + let mut mask = vec![Err(TransactionError::BlockhashNotFound); transactions_len]; + pending_tx_indexes.iter().for_each(|x| mask[*x] = Ok(())); + mask + } + + /// This function returns a vector containing index of all valid transactions. A valid + /// transaction has result Ok() as the value + fn filter_valid_transaction_indexes(valid_txs: &[TransactionCheckResult]) -> Vec { + valid_txs + .iter() + .enumerate() + .filter_map(|(index, (x, _h))| if x.is_ok() { Some(index) } else { None }) + .collect_vec() + } +} diff --git a/core/src/banking_stage/decision_maker.rs b/core/src/banking_stage/decision_maker.rs new file mode 100644 index 00000000000000..a4126b2fe2ec03 --- /dev/null +++ b/core/src/banking_stage/decision_maker.rs @@ -0,0 +1,91 @@ +use { + solana_poh::poh_recorder::{BankStart, PohRecorder}, + solana_sdk::{ + clock::{ + DEFAULT_TICKS_PER_SLOT, FORWARD_TRANSACTIONS_TO_LEADER_AT_SLOT_OFFSET, + HOLD_TRANSACTIONS_SLOT_OFFSET, + }, + pubkey::Pubkey, + }, + std::sync::{Arc, RwLock}, +}; + +#[derive(Debug, Clone)] +pub enum BufferedPacketsDecision { + Consume(BankStart), + Forward, + ForwardAndHold, + Hold, +} + +pub struct DecisionMaker { + my_pubkey: Pubkey, + poh_recorder: Arc>, +} + +impl DecisionMaker { + pub fn new(my_pubkey: Pubkey, poh_recorder: Arc>) -> Self { + Self { + my_pubkey, + poh_recorder, + } + } + + pub fn make_consume_or_forward_decision(&self) -> BufferedPacketsDecision { + let (leader_at_slot_offset, bank_start, would_be_leader, would_be_leader_shortly) = { + let poh = self.poh_recorder.read().unwrap(); + let bank_start = poh + .bank_start() + .filter(|bank_start| bank_start.should_working_bank_still_be_processing_txs()); + ( + poh.leader_after_n_slots(FORWARD_TRANSACTIONS_TO_LEADER_AT_SLOT_OFFSET), + bank_start, + poh.would_be_leader(HOLD_TRANSACTIONS_SLOT_OFFSET * DEFAULT_TICKS_PER_SLOT), + poh.would_be_leader( + (FORWARD_TRANSACTIONS_TO_LEADER_AT_SLOT_OFFSET - 1) * DEFAULT_TICKS_PER_SLOT, + ), + ) + }; + + Self::consume_or_forward_packets( + &self.my_pubkey, + leader_at_slot_offset, + bank_start, + would_be_leader, + would_be_leader_shortly, + ) + } + + fn consume_or_forward_packets( + my_pubkey: &Pubkey, + leader_pubkey: Option, + bank_start: Option, + would_be_leader: bool, + would_be_leader_shortly: bool, + ) -> BufferedPacketsDecision { + // If has active bank, then immediately process buffered packets + // otherwise, based on leader schedule to either forward or hold packets + if let Some(bank_start) = bank_start { + // If the bank is available, this node is the leader + BufferedPacketsDecision::Consume(bank_start) + } else if would_be_leader_shortly { + // If the node will be the leader soon, hold the packets for now + BufferedPacketsDecision::Hold + } else if would_be_leader { + // Node will be leader within ~20 slots, hold the transactions in + // case it is the only node which produces an accepted slot. + BufferedPacketsDecision::ForwardAndHold + } else if let Some(x) = leader_pubkey { + if x != *my_pubkey { + // If the current node is not the leader, forward the buffered packets + BufferedPacketsDecision::Forward + } else { + // If the current node is the leader, return the buffered packets as is + BufferedPacketsDecision::Hold + } + } else { + // We don't know the leader. Hold the packets for now + BufferedPacketsDecision::Hold + } + } +} diff --git a/core/src/banking_stage/external_scheduler.rs b/core/src/banking_stage/external_scheduler.rs new file mode 100644 index 00000000000000..829d8462fce2a3 --- /dev/null +++ b/core/src/banking_stage/external_scheduler.rs @@ -0,0 +1,189 @@ +use { + super::{ + consume_executor::ConsumeExecutor, decision_maker::BufferedPacketsDecision, + forward_executor::ForwardExecutor, scheduler_error::SchedulerError, BankingStageStats, + ForwardOption, + }, + crate::{ + leader_slot_banking_stage_metrics::LeaderSlotMetricsTracker, + scheduler_stage::{ + ProcessedTransactions, ProcessedTransactionsSender, ScheduledTransactions, + ScheduledTransactionsReceiver, + }, + }, + crossbeam_channel::TryRecvError, + solana_measure::{measure, measure_us}, +}; + +/// Handle interface for interacting with an external scheduler +pub struct ExternalSchedulerHandle { + scheduled_transactions_receiver: ScheduledTransactionsReceiver, + processed_transactions_sender: ProcessedTransactionsSender, + banking_stage_stats: BankingStageStats, +} + +impl ExternalSchedulerHandle { + pub fn new( + id: u32, + scheduled_transactions_receiver: ScheduledTransactionsReceiver, + processed_transactions_sender: ProcessedTransactionsSender, + ) -> Self { + Self { + scheduled_transactions_receiver, + processed_transactions_sender, + banking_stage_stats: BankingStageStats::new(id), + } + } + + pub fn tick(&mut self) -> Result<(), SchedulerError> { + Ok(()) + } + + pub fn do_scheduled_work( + &mut self, + consume_executor: &ConsumeExecutor, + forward_executor: &ForwardExecutor, + slot_metrics_tracker: &mut LeaderSlotMetricsTracker, + ) -> Result<(), SchedulerError> { + loop { + match self.scheduled_transactions_receiver.try_recv() { + Ok(scheduled_transactions) => { + let procssed_transactions = self.process_scheduled_transactions( + scheduled_transactions, + consume_executor, + forward_executor, + slot_metrics_tracker, + ); + + if self + .processed_transactions_sender + .send(procssed_transactions) + .is_err() + { + return Err(SchedulerError::ProcessedTransactionsSenderDisconnected); + } + } + Err(TryRecvError::Disconnected) => { + return Err(SchedulerError::ScheduledWorkReceiverDisconnected) + } + _ => break, // No more work to do + } + } + + Ok(()) + } + + // TODO: fix forwarding leader metrics + fn process_scheduled_transactions( + &self, + scheduled_transactions: ScheduledTransactions, + consume_executor: &ConsumeExecutor, + forward_executor: &ForwardExecutor, + slot_metrics_tracker: &mut LeaderSlotMetricsTracker, + ) -> ProcessedTransactions { + // Check if the scheduled transactions are still valid + // TODO: metrics + if !scheduled_transactions.is_valid() { + let num_packets = scheduled_transactions.transactions.len(); + return ProcessedTransactions { + thread_id: scheduled_transactions.thread_id, + packets: scheduled_transactions.packets, + transactions: scheduled_transactions.transactions, + retryable: vec![true; num_packets], + invalidated: true, + }; + } + + match scheduled_transactions.decision { + BufferedPacketsDecision::Consume(ref bank_start) => { + slot_metrics_tracker.apply_working_bank(Some(bank_start)); + let (process_transactions_summary, process_packets_transactions_us) = + measure_us!(consume_executor.process_packets_transactions( + bank_start, + &scheduled_transactions.transactions, + &self.banking_stage_stats, + slot_metrics_tracker, + )); + if process_transactions_summary.reached_max_poh_height { + scheduled_transactions.mark_as_invalid(); + } + + slot_metrics_tracker + .increment_process_packets_transactions_us(process_packets_transactions_us); + + let retryable_transaction_indexes = + process_transactions_summary.retryable_transaction_indexes; + slot_metrics_tracker + .increment_retryable_packets_count(retryable_transaction_indexes.len() as u64); + + let mut retryable = vec![false; scheduled_transactions.transactions.len()]; + for retryable_transaction_index in retryable_transaction_indexes { + retryable[retryable_transaction_index] = true; + } + ProcessedTransactions { + thread_id: scheduled_transactions.thread_id, + packets: scheduled_transactions.packets, + transactions: scheduled_transactions.transactions, + retryable, + invalidated: false, + } + } + BufferedPacketsDecision::Forward => { + slot_metrics_tracker.apply_working_bank(None); + let (_forward_result, sucessful_forwarded_packets_count, _leader_pubkey) = + forward_executor.forward_buffered_packets( + &ForwardOption::ForwardTransaction, // Only support transactions for now + scheduled_transactions + .packets + .iter() + .map(|p| p.immutable_section().original_packet()), + &self.banking_stage_stats, + ); + slot_metrics_tracker.increment_successful_forwarded_packets_count( + sucessful_forwarded_packets_count as u64, + ); + + ProcessedTransactions::default() // No retryable packets + } + + BufferedPacketsDecision::ForwardAndHold => { + slot_metrics_tracker.apply_working_bank(None); + let (_forward_result, sucessful_forwarded_packets_count, _leader_pubkey) = + forward_executor.forward_buffered_packets( + &ForwardOption::ForwardTransaction, // Only support transactions for now + scheduled_transactions + .packets + .iter() + .map(|p| p.immutable_section().original_packet()), + &self.banking_stage_stats, + ); + slot_metrics_tracker.increment_successful_forwarded_packets_count( + sucessful_forwarded_packets_count as u64, + ); + + let ScheduledTransactions { + thread_id, + mut packets, + transactions, + .. + } = scheduled_transactions; + + // Mark all packets as forwarded + for packet in &mut packets { + packet.forwarded = true; + } + let num_transactions = transactions.len(); + + ProcessedTransactions { + thread_id, + packets, + transactions, + retryable: vec![true; num_transactions], + invalidated: false, + } + } + + _ => panic!("Unexpected decision"), + } + } +} diff --git a/core/src/banking_stage/forward_executor.rs b/core/src/banking_stage/forward_executor.rs new file mode 100644 index 00000000000000..8a401523f44ba5 --- /dev/null +++ b/core/src/banking_stage/forward_executor.rs @@ -0,0 +1,135 @@ +use { + super::{BankingStageStats, ForwardOption}, + crate::next_leader, + solana_client::{connection_cache::ConnectionCache, tpu_connection::TpuConnection}, + solana_gossip::cluster_info::ClusterInfo, + solana_measure::measure::Measure, + solana_perf::{data_budget::DataBudget, packet::Packet}, + solana_poh::poh_recorder::PohRecorder, + solana_sdk::{pubkey::Pubkey, transport::TransportError}, + solana_streamer::sendmmsg::batch_send, + std::{ + iter::repeat, + net::UdpSocket, + sync::{atomic::Ordering, Arc, RwLock}, + }, +}; + +pub struct ForwardExecutor { + poh_recorder: Arc>, + socket: UdpSocket, + cluster_info: Arc, + connection_cache: Arc, + data_budget: Arc, +} + +impl ForwardExecutor { + pub fn new( + poh_recorder: Arc>, + socket: UdpSocket, + cluster_info: Arc, + connection_cache: Arc, + data_budget: Arc, + ) -> Self { + Self { + poh_recorder, + socket, + cluster_info, + connection_cache, + data_budget, + } + } + + /// Forwards all valid, unprocessed packets in the buffer, up to a rate limit. Returns + /// the number of successfully forwarded packets in second part of tuple + pub fn forward_buffered_packets<'a>( + &self, + forward_option: &ForwardOption, + forwardable_packets: impl Iterator, + banking_stage_stats: &BankingStageStats, + ) -> ( + std::result::Result<(), TransportError>, + usize, + Option, + ) { + let leader_and_addr = match forward_option { + ForwardOption::NotForward => return (Ok(()), 0, None), + ForwardOption::ForwardTransaction => { + next_leader::next_leader_tpu_forwards(&self.cluster_info, &self.poh_recorder) + } + + ForwardOption::ForwardTpuVote => { + next_leader::next_leader_tpu_vote(&self.cluster_info, &self.poh_recorder) + } + }; + let (leader_pubkey, addr) = match leader_and_addr { + Some(leader_and_addr) => leader_and_addr, + None => return (Ok(()), 0, None), + }; + + const INTERVAL_MS: u64 = 100; + // 12 MB outbound limit per second + const MAX_BYTES_PER_SECOND: usize = 12_000_000; + const MAX_BYTES_PER_INTERVAL: usize = MAX_BYTES_PER_SECOND * INTERVAL_MS as usize / 1000; + const MAX_BYTES_BUDGET: usize = MAX_BYTES_PER_INTERVAL * 5; + self.data_budget.update(INTERVAL_MS, |bytes| { + std::cmp::min( + bytes.saturating_add(MAX_BYTES_PER_INTERVAL), + MAX_BYTES_BUDGET, + ) + }); + + let packet_vec: Vec<_> = forwardable_packets + .filter_map(|p| { + if !p.meta.forwarded() && self.data_budget.take(p.meta.size) { + Some(p.data(..)?.to_vec()) + } else { + None + } + }) + .collect(); + + let packet_vec_len = packet_vec.len(); + // TODO: see https://github.com/solana-labs/solana/issues/23819 + // fix this so returns the correct number of succeeded packets + // when there's an error sending the batch. This was left as-is for now + // in favor of shipping Quic support, which was considered higher-priority + if !packet_vec.is_empty() { + inc_new_counter_info!("banking_stage-forwarded_packets", packet_vec_len); + + let mut measure = Measure::start("banking_stage-forward-us"); + + let res = if let ForwardOption::ForwardTpuVote = forward_option { + // The vote must be forwarded using only UDP. + banking_stage_stats + .forwarded_vote_count + .fetch_add(packet_vec_len, Ordering::Relaxed); + let pkts: Vec<_> = packet_vec.into_iter().zip(repeat(addr)).collect(); + batch_send(&self.socket, &pkts).map_err(|err| err.into()) + } else { + // All other transactions can be forwarded using QUIC, get_connection() will use + // system wide setting to pick the correct connection object. + banking_stage_stats + .forwarded_transaction_count + .fetch_add(packet_vec_len, Ordering::Relaxed); + let conn = self.connection_cache.get_connection(&addr); + conn.send_wire_transaction_batch_async(packet_vec) + }; + + measure.stop(); + inc_new_counter_info!( + "banking_stage-forward-us", + measure.as_us() as usize, + 1000, + 1000 + ); + + if let Err(err) = res { + inc_new_counter_info!("banking_stage-forward_packets-failed-batches", 1); + return (Err(err), 0, Some(leader_pubkey)); + } + } + + (Ok(()), packet_vec_len, Some(leader_pubkey)) + } +} diff --git a/core/src/banking_stage/multi_iterator_scheduler.rs b/core/src/banking_stage/multi_iterator_scheduler.rs new file mode 100644 index 00000000000000..6ea6a868778495 --- /dev/null +++ b/core/src/banking_stage/multi_iterator_scheduler.rs @@ -0,0 +1,914 @@ +//! Central scheduler using a multi-iterator to schedule transactions for consuming. +//! + +use { + super::{ + decision_maker::{BufferedPacketsDecision, DecisionMaker}, + scheduler_error::SchedulerError, + thread_aware_account_locks::{ThreadAwareAccountLocks, ThreadId, ThreadSet}, + }, + crate::{ + multi_iterator_scanner::{MultiIteratorScanner, ProcessingDecision}, + packet_deserializer::{PacketDeserializer, ReceivePacketResults}, + read_write_account_set::ReadWriteAccountSet, + scheduler_stage::{ + ProcessedTransactions, ProcessedTransactionsReceiver, ScheduledTransactions, + ScheduledTransactionsSender, + }, + unprocessed_packet_batches::DeserializedPacket, + }, + crossbeam_channel::{RecvTimeoutError, TryRecvError}, + itertools::Itertools, + min_max_heap::MinMaxHeap, + solana_measure::{measure, measure_us}, + solana_perf::perf_libs, + solana_poh::poh_recorder::BankStart, + solana_runtime::{ + bank::{Bank, BankStatusCache}, + blockhash_queue::BlockhashQueue, + root_bank_cache::RootBankCache, + transaction_error_metrics::TransactionErrorMetrics, + }, + solana_sdk::{ + clock::{ + FORWARD_TRANSACTIONS_TO_LEADER_AT_SLOT_OFFSET, MAX_PROCESSING_AGE, + MAX_TRANSACTION_FORWARDING_DELAY, MAX_TRANSACTION_FORWARDING_DELAY_GPU, + }, + nonce::state::DurableNonce, + timing::AtomicInterval, + transaction::SanitizedTransaction, + }, + std::{ + sync::{ + atomic::{AtomicBool, Ordering}, + Arc, + }, + time::{Duration, Instant}, + }, +}; + +const BATCH_SIZE: u32 = 64; + +#[derive(Debug, PartialEq, Eq)] +struct TransactionPacket { + packet: DeserializedPacket, + transaction: SanitizedTransaction, +} + +impl PartialOrd for TransactionPacket { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +impl Ord for TransactionPacket { + fn cmp(&self, other: &Self) -> std::cmp::Ordering { + self.packet.cmp(&other.packet) + } +} + +pub struct MultiIteratorScheduler { + /// Number of executing threads + num_threads: usize, + /// Makes decisions about whether to consume or forward packets + decision_maker: DecisionMaker, + /// Priority queue of buffered packets and sanitized transactions + priority_queue: MinMaxHeap, + /// Receiver for packets from sigverify + packet_deserializer: PacketDeserializer, + /// Cached root bank for sanitizing transactions - updates only as new root banks are set + root_bank_cache: RootBankCache, + /// Sends scheduled transactions to the execution threads for consuming or forwarding + transaction_senders: Vec, + /// Scheduled account locks for pending transactions + account_locks: ThreadAwareAccountLocks, + /// Receives processed transactions from the execution threads + processed_transaction_receiver: ProcessedTransactionsReceiver, + /// Scheduling metrics + metrics: MultiIteratorSchedulerMetrics, +} + +impl MultiIteratorScheduler { + pub fn new( + num_threads: usize, + decision_maker: DecisionMaker, + packet_deserializer: PacketDeserializer, + root_bank_cache: RootBankCache, + transaction_senders: Vec, + processed_transaction_receiver: ProcessedTransactionsReceiver, + buffer_capacity: usize, + ) -> Self { + assert_eq!(num_threads, transaction_senders.len()); + Self { + num_threads, + decision_maker, + priority_queue: MinMaxHeap::with_capacity(buffer_capacity), + packet_deserializer, + root_bank_cache, + transaction_senders, + account_locks: ThreadAwareAccountLocks::new(num_threads as u8), + processed_transaction_receiver, + metrics: MultiIteratorSchedulerMetrics::default(), + } + } + + pub fn run(mut self) { + loop { + let (decision, make_decision_time_us) = + measure_us!(self.decision_maker.make_consume_or_forward_decision()); + self.metrics.make_decision_time_us += make_decision_time_us; + match decision { + BufferedPacketsDecision::Consume(bank_start) => { + self.metrics.make_decision_consume_count += 1; + let (_, schedule_consume_time_us) = + measure_us!(self.schedule_consume(bank_start)); + self.metrics.schedule_consume_time_us += schedule_consume_time_us; + } + BufferedPacketsDecision::Forward => { + self.metrics.make_decision_forward_count += 1; + let (_, schedule_forward_time_us) = measure_us!(self.schedule_forward(false)); + self.metrics.schedule_forward_time_us += schedule_forward_time_us; + } + BufferedPacketsDecision::ForwardAndHold => { + self.metrics.make_decision_forward_count += 1; + let (_, schedule_forward_time_us) = measure_us!(self.schedule_forward(true)); + self.metrics.schedule_forward_time_us += schedule_forward_time_us; + } + BufferedPacketsDecision::Hold => { + self.metrics.make_decision_hold_count += 1; + } + } + + let (receive_result, receive_and_buffer_time_us) = + measure_us!(self.receive_and_buffer_packets()); + self.metrics.receive_and_buffer_time_us += receive_and_buffer_time_us; + if receive_result.is_err() { + break; + } + + let (receive_processed_transactions_result, receive_processed_transactions_time_us) = + measure_us!(self.receive_processed_transactions()); + self.metrics.receive_processed_transactions_time_us += + receive_processed_transactions_time_us; + if receive_processed_transactions_result.is_err() { + break; + } + + self.metrics.report(1000); + } + } + + fn schedule_consume(&mut self, bank_start: BankStart) { + let decision = BufferedPacketsDecision::Consume(bank_start.clone()); + + // Drain priority queue into a vector of transactions + let (transaction_packets, queue_drain_time_us) = + measure_us!(self.priority_queue.drain_desc().collect_vec()); + self.metrics.max_consumed_buffer_size = self + .metrics + .max_consumed_buffer_size + .max(transaction_packets.len()); + self.metrics.consume_drain_queue_time_us += queue_drain_time_us; + + // Create a multi-iterator scanner over the transactions + let mut scanner = MultiIteratorScanner::new( + &transaction_packets, + self.num_threads * BATCH_SIZE as usize, + MultiIteratorSchedulerConsumePayload::new(self), + #[inline(always)] + |transaction_packet: &TransactionPacket, + payload: &mut MultiIteratorSchedulerConsumePayload| { + Self::should_consume(transaction_packet, payload) + }, + ); + + // Validity check for sent transactions + let validity_check = Arc::new(AtomicBool::new(true)); + + // Loop over batches of transactions + let mut iterate_time = Instant::now(); + while let Some((transactions, payload)) = scanner.iterate() { + // Grab reference to scheduler from payload + let scheduler = &mut *payload.scheduler; + + assert_eq!(transactions.len(), payload.thread_indices.len()); // TOOD: Remove after testing + + let iterate_time_us = iterate_time.elapsed().as_micros() as u64; + scheduler.metrics.total_consume_iterator_time_us += iterate_time_us; + scheduler.metrics.max_consume_iterator_time_us = scheduler + .metrics + .max_consume_iterator_time_us + .max(iterate_time_us); + + // TODO: Consider receiving and unlocking processed transactions here + // NOTE: If we do this, we need to update the priority queue extend below + + // Batches of transactions to send + let mut batches = (0..scheduler.num_threads) + .map(|idx| { + ScheduledTransactions::with_capacity( + idx as ThreadId, + decision.clone(), + BATCH_SIZE as usize, + validity_check.clone(), + ) + }) + .collect_vec(); + + // Move all transactions into their respective batches + // TODO: Optimize cloning - sanitized transaction clone. + for (transaction, thread_index) in + transactions.iter().zip(payload.thread_indices.iter()) + { + batches[*thread_index as usize] + .packets + .push(transaction.packet.clone()); + batches[*thread_index as usize] + .transactions + .push(transaction.transaction.clone()); + } + + // Send batches to the execution threads + for (thread_index, batch) in batches.into_iter().enumerate() { + if batch.packets.is_empty() { + continue; + } + + let packet_count = batch.packets.len(); + scheduler.metrics.consumed_batch_count += 1; + scheduler.metrics.consumed_packet_count += packet_count; + scheduler.metrics.consumed_min_batch_size = + scheduler.metrics.consumed_min_batch_size.min(packet_count); + scheduler.metrics.consumed_max_batch_size = + scheduler.metrics.consumed_min_batch_size.max(packet_count); + + scheduler.transaction_senders[thread_index] + .send(batch) + .expect("transaction sender should be connected"); + } + + // Reset the iterator payload for next iteration + payload.reset(); + + // Check if we've reached the end of the slot + if bank_start.reached_max_tick_height() + || !bank_start.should_working_bank_still_be_processing_txs() + { + validity_check.store(false, Ordering::Relaxed); + break; + } + + // Stop scheduling if a scheduled transaction batch was invalidated + if !validity_check.load(Ordering::Relaxed) { + break; + } + + // Reset time for next iteration + iterate_time = Instant::now(); + } + + // Get the final payload from the scanner and whether or not each packet was handled + let (_payload, already_handled) = scanner.finalize(); + + // Push unprocessed packets back into the priority queue + let (_, push_queue_time_us) = measure_us!({ + self.priority_queue.extend( + transaction_packets + .into_iter() + .zip(already_handled.into_iter()) + .filter(|(_, handled)| !handled) + .map(|(transaction_packet, _)| transaction_packet), + ) + }); + self.metrics.consume_push_queue_time_us += push_queue_time_us; + } + + fn schedule_forward(&mut self, hold: bool) { + let decision = if hold { + BufferedPacketsDecision::ForwardAndHold + } else { + BufferedPacketsDecision::Forward + }; + + // Drain priority queue into a vector of transactions + let transaction_packets = self.priority_queue.drain_desc().collect_vec(); + + // Grab read locks for checking age and status cache + let root_bank = self.root_bank_cache.root_bank(); + let blockhash_queue = root_bank.blockhash_queue.read().unwrap(); + let status_cache = root_bank.status_cache.read().unwrap(); + let next_durable_nonce = DurableNonce::from_blockhash(&blockhash_queue.last_hash()); + + // Calculate max forwarding age + let max_age = (MAX_PROCESSING_AGE) + .saturating_sub(if perf_libs::api().is_some() { + MAX_TRANSACTION_FORWARDING_DELAY + } else { + MAX_TRANSACTION_FORWARDING_DELAY_GPU + }) + .saturating_sub(FORWARD_TRANSACTIONS_TO_LEADER_AT_SLOT_OFFSET as usize); + let mut error_counters = TransactionErrorMetrics::default(); + + // Create a multi-iterator scanner over the transactions + let mut scanner = MultiIteratorScanner::new( + &transaction_packets, + self.num_threads * BATCH_SIZE as usize, + MultiIteratorSchedulerForwardPayload::default(), + #[inline(always)] + |transaction_packet: &TransactionPacket, + payload: &mut MultiIteratorSchedulerForwardPayload| { + Self::should_forward( + transaction_packet, + &root_bank, + &blockhash_queue, + &status_cache, + &next_durable_nonce, + max_age, + &mut error_counters, + payload, + ) + }, + ); + + let validity_check = Arc::new(AtomicBool::new(true)); + // Loop over batches of transactions + while let Some((transactions, payload)) = scanner.iterate() { + // Create batches to fill with transactions + let mut batches = (0..self.num_threads) + .map(|idx| { + ScheduledTransactions::with_capacity( + idx as ThreadId, + decision.clone(), + BATCH_SIZE as usize, + validity_check.clone(), + ) + }) + .collect_vec(); + + // Fill batches - striped access + for (index, transaction) in transactions.iter().copied().enumerate() { + let batch = &mut batches[index % self.num_threads]; + batch.packets.push(transaction.packet.clone()); + batch.transactions.push(transaction.transaction.clone()); + } + + // Send batches to the execution threads + for (thread_index, batch) in batches.into_iter().enumerate() { + if batch.packets.is_empty() { + continue; + } + + let packet_count = batch.packets.len(); + self.metrics.forward_batch_count += 1; + self.metrics.forward_packet_count += packet_count; + self.metrics.forward_min_batch_size = + self.metrics.forward_min_batch_size.min(packet_count); + self.metrics.forward_max_batch_size = + self.metrics.forward_max_batch_size.max(packet_count); + + self.transaction_senders[thread_index] + .send(batch) + .expect("transaction sender should be connected") + } + + // Reset the iterator payload for next iteration + payload.reset(); + } + + // Get the final payload from the scanner and whether or not each packet was handled + let (_payload, _already_handled) = scanner.finalize(); + } + + fn receive_and_buffer_packets(&mut self) -> Result<(), SchedulerError> { + const EMPTY_RECEIVE_TIMEOUT: Duration = Duration::from_millis(100); + const NON_EMPTY_RECEIVE_TIMEOUT: Duration = Duration::from_millis(0); + let timeout = if self.priority_queue.is_empty() { + EMPTY_RECEIVE_TIMEOUT + } else { + NON_EMPTY_RECEIVE_TIMEOUT + }; + let remaining_capacity = self.remaining_capacity(); + + let (receive_packet_results, receive_packets_time_us) = measure_us!(self + .packet_deserializer + .handle_received_packets(timeout, remaining_capacity)); + self.metrics.receive_packets_time_us += receive_packets_time_us; + + match receive_packet_results { + Ok(receive_packet_results) => { + let (_, buffer_new_packets_time_us) = + measure_us!(self.insert_received_packets(receive_packet_results)); + self.metrics.buffer_new_packets_time_us += buffer_new_packets_time_us; + } + Err(RecvTimeoutError::Disconnected) => { + return Err(SchedulerError::PacketReceiverDisconnected) + } + Err(RecvTimeoutError::Timeout) => {} + } + + Ok(()) + } + + fn receive_processed_transactions(&mut self) -> Result<(), SchedulerError> { + loop { + let (processed_transactions, receive_time_us) = + measure_us!(self.processed_transaction_receiver.try_recv()); + self.metrics.processed_transactions_recv_time_us += receive_time_us; + match processed_transactions { + Ok(processed_transactions) => { + self.handle_processed_transactions(processed_transactions)?; + } + Err(TryRecvError::Disconnected) => { + return Err(SchedulerError::ProcessedTransactionsReceiverDisconnected); + } + Err(TryRecvError::Empty) => break, + } + } + Ok(()) + } + + fn handle_processed_transactions( + &mut self, + processed_transactions: ProcessedTransactions, + ) -> Result<(), SchedulerError> { + for ((packet, transaction), retryable) in processed_transactions + .packets + .into_iter() + .zip(processed_transactions.transactions.into_iter()) + .zip(processed_transactions.retryable.into_iter()) + { + // Unlock accounts + let (_, unlock_time_us) = measure_us!({ + let transaction_account_locks = transaction.get_account_locks_unchecked(); + self.account_locks.unlock_accounts( + transaction_account_locks.writable.into_iter(), + transaction_account_locks.readonly.into_iter(), + processed_transactions.thread_id, + ); + }); + self.metrics.processed_transactions_unlock_time_us += unlock_time_us; + + // Push retryable packets back into the buffer + let (_, buffer_time_us) = measure_us!(if retryable { + if processed_transactions.invalidated { + self.metrics.invalidated_packet_count += 1; + } else { + self.metrics.retryable_packet_count += 1; + } + self.push_priority_queue(TransactionPacket { + packet, + transaction, + }); + }); + self.metrics.processed_transactions_buffer_time_us += buffer_time_us; + } + + Ok(()) + } + + /// Inserts received packets, after sanitizing the transactions, into the priority queue. + // TODO: Collect stats + fn insert_received_packets(&mut self, receive_packet_results: ReceivePacketResults) { + let root_bank = self.root_bank_cache.root_bank(); + let tx_account_lock_limit = root_bank.get_transaction_account_lock_limit(); + + self.metrics.received_packet_count += receive_packet_results.deserialized_packets.len(); + + for (packet, transaction) in receive_packet_results + .deserialized_packets + .into_iter() + .filter_map(|packet| { + packet + .build_sanitized_transaction( + &root_bank.feature_set, + root_bank.vote_only_bank(), + root_bank.as_ref(), + ) + .map(|tx| (packet, tx)) + }) + .filter(|(_, transaction)| { + SanitizedTransaction::validate_account_locks( + transaction.message(), + tx_account_lock_limit, + ) + .is_ok() + }) + { + let transaction_packet = TransactionPacket { + packet: DeserializedPacket::from_immutable_section(packet), + transaction, + }; + + self.push_priority_queue(transaction_packet); + } + } + + fn remaining_capacity(&self) -> usize { + self.priority_queue.capacity() - self.priority_queue.len() + } + + fn should_consume( + transaction_packet: &TransactionPacket, + payload: &mut MultiIteratorSchedulerConsumePayload, + ) -> ProcessingDecision { + // If locks clash with the current batch of transactions, then we should process + // the transaction later. + let (has_batch_conflict, conflict_batch_time_us) = measure_us!(!payload + .batch_account_locks + .check_sanitized_message_account_locks(transaction_packet.transaction.message())); + payload.scheduler.metrics.conflict_batch_time_us += conflict_batch_time_us; + + if has_batch_conflict { + payload.scheduler.metrics.conflict_batch_count += 1; + return ProcessingDecision::Later; + } + + // Check if we can schedule to any thread. + let transaction_account_locks = + transaction_packet.transaction.get_account_locks_unchecked(); + + let (schedulable_threads, schedulable_threads_time_us) = measure_us!(payload + .scheduler + .account_locks + .accounts_schedulable_threads( + transaction_account_locks.writable.iter().copied(), + transaction_account_locks.readonly.iter().copied(), + )); + + // Combine with non-full threads + let schedulable_threads = schedulable_threads & payload.schedulable_threads; + payload.scheduler.metrics.conflict_locks_time_us += schedulable_threads_time_us; + + if schedulable_threads.is_empty() { + payload.scheduler.metrics.conflict_locks_count += 1; + return ProcessingDecision::Later; + } + + // Iterate over schedulable threads and find the thread with the least number of transactions in current batch. + // TODO: Might want to also consider the number of transactions in the queue. + let thread_id = schedulable_threads + .threads_iter() + .map(|thread_id| (thread_id, payload.batch_counts[thread_id as usize])) + .min_by_key(|(_, count)| *count) + .unwrap() + .0; + + // Take locks + payload + .batch_account_locks + .add_sanitized_message_account_locks(transaction_packet.transaction.message()); + payload.scheduler.account_locks.lock_accounts( + transaction_account_locks.writable.into_iter(), + transaction_account_locks.readonly.into_iter(), + thread_id, + ); + + // Update payload + payload.thread_indices.push(thread_id); + payload.batch_counts[thread_id as usize] += 1; + if payload.batch_counts[thread_id as usize] == BATCH_SIZE { + payload.schedulable_threads.remove(thread_id); + } + + ProcessingDecision::Now + } + + fn should_forward( + transaction_packet: &TransactionPacket, + bank: &Bank, + blockhash_queue: &BlockhashQueue, + status_cache: &BankStatusCache, + next_durable_nonce: &DurableNonce, + max_age: usize, + error_counters: &mut TransactionErrorMetrics, + payload: &mut MultiIteratorSchedulerForwardPayload, + ) -> ProcessingDecision { + // Throw out transactions that are too old. + if bank + .check_transaction_age( + &transaction_packet.transaction, + max_age, + next_durable_nonce, + blockhash_queue, + error_counters, + ) + .0 + .is_err() + { + return ProcessingDecision::Never; + } + + // If the transaction is already in the bank, then we don't need to forward it. + if bank.is_transaction_already_processed(&transaction_packet.transaction, status_cache) { + error_counters.already_processed += 1; + return ProcessingDecision::Never; + } + + // If locks clash with the current batch of transactions, then we should forward + // the transaction later. + if payload + .account_locks + .try_locking(transaction_packet.transaction.message()) + { + ProcessingDecision::Now + } else { + ProcessingDecision::Later + } + } + + fn push_priority_queue(&mut self, transaction_packet: TransactionPacket) { + if self.priority_queue.len() == self.priority_queue.capacity() { + self.priority_queue.push_pop_min(transaction_packet); + self.metrics.dropped_packet_count += 1; + } else { + self.priority_queue.push(transaction_packet); + } + } +} + +struct MultiIteratorSchedulerConsumePayload<'a> { + /// Mutable reference to the scheduler struct + scheduler: &'a mut MultiIteratorScheduler, + /// Read and write accounts that are used by the current batch of transactions. + batch_account_locks: ReadWriteAccountSet, + /// Thread index for each transaction in the batch. + thread_indices: Vec, + /// Batch counts + batch_counts: Vec, + /// Schedulable threads (based on batch_counts) + schedulable_threads: ThreadSet, +} + +impl<'a> MultiIteratorSchedulerConsumePayload<'a> { + fn new(scheduler: &'a mut MultiIteratorScheduler) -> Self { + let num_threads = scheduler.num_threads; + Self { + scheduler, + batch_account_locks: ReadWriteAccountSet::default(), + thread_indices: Vec::with_capacity(num_threads * BATCH_SIZE as usize), + batch_counts: vec![0; num_threads], + schedulable_threads: ThreadSet::any(num_threads as u8), + } + } + + fn reset(&mut self) { + self.batch_account_locks.clear(); + self.thread_indices.clear(); + self.batch_counts.fill(0); + self.schedulable_threads = ThreadSet::any(self.scheduler.num_threads as u8); + } +} + +#[derive(Default)] +struct MultiIteratorSchedulerForwardPayload { + /// Account locks used to prevent us from spam forwarding hot accounts + account_locks: ReadWriteAccountSet, +} + +impl MultiIteratorSchedulerForwardPayload { + fn reset(&mut self) { + self.account_locks.clear(); + } +} + +struct MultiIteratorSchedulerMetrics { + last_reported: AtomicInterval, + + // Receive and buffer metrics + received_packet_count: usize, + receive_and_buffer_time_us: u64, + receive_packets_time_us: u64, + buffer_new_packets_time_us: u64, + + // Receive processed transactions time + receive_processed_transactions_time_us: u64, + processed_transactions_recv_time_us: u64, + processed_transactions_unlock_time_us: u64, + processed_transactions_buffer_time_us: u64, + + // Decision making metrics + make_decision_consume_count: usize, + make_decision_forward_count: usize, + make_decision_hold_count: usize, + make_decision_time_us: u64, + + // Consume metrics + conflict_batch_count: usize, + conflict_locks_count: usize, + conflict_batch_time_us: u64, + conflict_locks_time_us: u64, + consumed_batch_count: usize, + consumed_packet_count: usize, + consumed_min_batch_size: usize, + consumed_max_batch_size: usize, + max_consumed_buffer_size: usize, + schedule_consume_time_us: u64, + max_consume_iterator_time_us: u64, + total_consume_iterator_time_us: u64, + consume_drain_queue_time_us: u64, + consume_push_queue_time_us: u64, + + // Forward metrics + forward_batch_count: usize, + forward_packet_count: usize, + forward_min_batch_size: usize, + forward_max_batch_size: usize, + schedule_forward_time_us: u64, + + // Misc metrics + retryable_packet_count: usize, + invalidated_packet_count: usize, + dropped_packet_count: usize, +} + +impl Default for MultiIteratorSchedulerMetrics { + fn default() -> Self { + Self { + last_reported: AtomicInterval::default(), + received_packet_count: 0, + receive_and_buffer_time_us: 0, + receive_packets_time_us: 0, + buffer_new_packets_time_us: 0, + receive_processed_transactions_time_us: 0, + processed_transactions_recv_time_us: 0, + processed_transactions_unlock_time_us: 0, + processed_transactions_buffer_time_us: 0, + make_decision_consume_count: 0, + make_decision_forward_count: 0, + make_decision_hold_count: 0, + make_decision_time_us: 0, + conflict_batch_count: 0, + conflict_locks_count: 0, + conflict_batch_time_us: 0, + conflict_locks_time_us: 0, + consumed_batch_count: 0, + consumed_packet_count: 0, + consumed_min_batch_size: usize::MAX, + consumed_max_batch_size: 0, + max_consumed_buffer_size: 0, + schedule_consume_time_us: 0, + max_consume_iterator_time_us: 0, + consume_drain_queue_time_us: 0, + consume_push_queue_time_us: 0, + total_consume_iterator_time_us: 0, + forward_batch_count: 0, + forward_packet_count: 0, + forward_min_batch_size: usize::MAX, + forward_max_batch_size: 0, + schedule_forward_time_us: 0, + retryable_packet_count: 0, + invalidated_packet_count: 0, + dropped_packet_count: 0, + } + } +} + +impl MultiIteratorSchedulerMetrics { + fn report(&mut self, report_interval_ms: u64) { + if self.last_reported.should_update(report_interval_ms) { + datapoint_info!( + "multi_iterator_scheduler", + ("received_packet_count", self.received_packet_count, i64), + ( + "receive_and_buffer_time_us", + self.receive_and_buffer_time_us, + i64 + ), + ("receive_packets_time_us", self.receive_packets_time_us, i64), + ( + "buffer_new_packets_time_us", + self.buffer_new_packets_time_us, + i64 + ), + ( + "receive_processed_transactions_time_us", + self.receive_processed_transactions_time_us, + i64 + ), + ( + "processed_transactions_recv_time_us", + self.processed_transactions_recv_time_us, + i64 + ), + ( + "processed_transactions_unlock_time_us", + self.processed_transactions_unlock_time_us, + i64 + ), + ( + "processed_transactions_buffer_time_us", + self.processed_transactions_buffer_time_us, + i64 + ), + ( + "make_decision_consume_count", + self.make_decision_consume_count, + i64 + ), + ( + "make_decision_forward_count", + self.make_decision_forward_count, + i64 + ), + ( + "make_decision_hold_count", + self.make_decision_hold_count, + i64 + ), + ("make_decision_time_us", self.make_decision_time_us, i64), + ("conflict_batch_count", self.conflict_batch_count, i64), + ("conflict_locks_count", self.conflict_locks_count, i64), + ("conflict_batch_time_us", self.conflict_batch_time_us, i64), + ("conflict_locks_time_us", self.conflict_locks_time_us, i64), + ("consumed_batch_count", self.consumed_batch_count, i64), + ("consumed_packet_count", self.consumed_packet_count, i64), + ("consumed_min_batch_size", self.consumed_min_batch_size, i64), + ("consumed_max_batch_size", self.consumed_max_batch_size, i64), + ( + "max_consumed_buffer_size", + self.max_consumed_buffer_size, + i64 + ), + ( + "schedule_consume_time_us", + self.schedule_consume_time_us, + i64 + ), + ( + "total_consume_iterator_time_us", + self.total_consume_iterator_time_us, + i64 + ), + ( + "max_consume_iterator_time_us", + self.max_consume_iterator_time_us, + i64 + ), + ( + "consume_drain_queue_time_us", + self.consume_drain_queue_time_us, + i64 + ), + ( + "consume_push_queue_time_us", + self.consume_push_queue_time_us, + i64 + ), + ("forward_batch_count", self.forward_batch_count, i64), + ("forward_packet_count", self.forward_packet_count, i64), + ("forward_min_batch_size", self.forward_min_batch_size, i64), + ("forward_max_batch_size", self.forward_max_batch_size, i64), + ( + "schedule_forward_time_us", + self.schedule_forward_time_us, + i64 + ), + ("retryable_packet_count", self.retryable_packet_count, i64), + ( + "invalidated_packet_count", + self.invalidated_packet_count, + i64 + ), + ("dropped_packet_count", self.dropped_packet_count, i64), + ); + self.reset(); + } + } + + fn reset(&mut self) { + self.received_packet_count = 0; + self.receive_and_buffer_time_us = 0; + self.receive_packets_time_us = 0; + self.buffer_new_packets_time_us = 0; + self.receive_processed_transactions_time_us = 0; + self.processed_transactions_recv_time_us = 0; + self.processed_transactions_unlock_time_us = 0; + self.processed_transactions_buffer_time_us = 0; + self.make_decision_consume_count = 0; + self.make_decision_forward_count = 0; + self.make_decision_hold_count = 0; + self.make_decision_time_us = 0; + self.conflict_batch_count = 0; + self.conflict_locks_count = 0; + self.conflict_batch_time_us = 0; + self.conflict_locks_time_us = 0; + self.consumed_batch_count = 0; + self.consumed_packet_count = 0; + self.consumed_min_batch_size = usize::MAX; + self.consumed_max_batch_size = 0; + self.max_consumed_buffer_size = 0; + self.schedule_consume_time_us = 0; + self.max_consume_iterator_time_us = 0; + self.consume_drain_queue_time_us = 0; + self.consume_push_queue_time_us = 0; + self.total_consume_iterator_time_us = 0; + self.forward_batch_count = 0; + self.forward_packet_count = 0; + self.forward_min_batch_size = usize::MAX; + self.forward_max_batch_size = 0; + self.schedule_forward_time_us = 0; + self.retryable_packet_count = 0; + self.invalidated_packet_count = 0; + self.dropped_packet_count = 0; + } +} diff --git a/core/src/banking_stage/packet_receiver.rs b/core/src/banking_stage/packet_receiver.rs new file mode 100644 index 00000000000000..7a6e11bda40b8a --- /dev/null +++ b/core/src/banking_stage/packet_receiver.rs @@ -0,0 +1,540 @@ +use { + crate::{ + immutable_deserialized_packet::ImmutableDeserializedPacket, + packet_deserializer::{PacketDeserializer, ReceivePacketResults}, + sigverify::SigverifyTracerPacketStats, + unprocessed_transaction_storage::{ + InsertPacketBatchSummary, UnprocessedTransactionStorage, + }, + }, + crossbeam_channel::RecvTimeoutError, + histogram::Histogram, + solana_measure::{measure, measure::Measure, measure_us}, + solana_perf::packet::PACKETS_PER_BATCH, + solana_poh::poh_recorder::Slot, + solana_sdk::{ + saturating_add_assign, + timing::{duration_as_ms, timestamp, AtomicInterval}, + }, + std::{ + sync::atomic::{AtomicU64, AtomicUsize, Ordering}, + time::{Duration, Instant}, + }, +}; + +pub struct PacketReceiver { + id: u32, + packet_deserializer: PacketDeserializer, + last_receive_time: Instant, + stats: PacketReceiverStats, + tracer_stats: TracerPacketReceiverStats, + leader_stats: LeaderSlotReceiveMetricsTracker, +} + +impl PacketReceiver { + pub fn new(id: u32, packet_deserializer: PacketDeserializer) -> Self { + Self { + id, + packet_deserializer, + last_receive_time: Instant::now(), + stats: PacketReceiverStats::new(id), + tracer_stats: TracerPacketReceiverStats::new(id), + leader_stats: LeaderSlotReceiveMetricsTracker::new(id), + } + } + + pub fn check_leader_slot_boundary(&mut self, leader_slot: Option) { + self.leader_stats.check_leader_slot_boundary(leader_slot); + } + + pub fn do_packet_receiving_and_buffering( + &mut self, + unprocessed_transaction_storage: &mut UnprocessedTransactionStorage, + ) -> Result<(), RecvTimeoutError> { + // Gossip thread will almost always not wait because the transaction storage will most likely not be empty + let recv_timeout = if !unprocessed_transaction_storage.is_empty() { + // If there are buffered packets, run the equivalent of try_recv to try reading more + // packets. This prevents starving BankingStage::consume_buffered_packets due to + // buffered_packet_batches containing transactions that exceed the cost model for + // the current bank. + Duration::from_millis(0) + } else { + // Default wait time + Duration::from_millis(100) + }; + + let (res, receive_and_buffer_packets_us) = measure_us!( + self.receive_and_buffer_packets(recv_timeout, unprocessed_transaction_storage,) + ); + self.leader_stats + .increment_receive_and_buffer_packets_us(receive_and_buffer_packets_us); + + // Report receiving stats on interval + self.stats.report(1000); + self.tracer_stats.report(1000); + + res + } + + /// Receive incoming packets, push into unprocessed buffer with packet indexes + fn receive_and_buffer_packets( + &mut self, + recv_timeout: Duration, + unprocessed_transaction_storage: &mut UnprocessedTransactionStorage, + ) -> Result<(), RecvTimeoutError> { + let mut recv_time = Measure::start("receive_and_buffer_packets_recv"); + let ReceivePacketResults { + deserialized_packets, + new_tracer_stats_option, + passed_sigverify_count, + failed_sigverify_count, + } = self.packet_deserializer.handle_received_packets( + recv_timeout, + unprocessed_transaction_storage.max_receive_size(), + )?; + let packet_count = deserialized_packets.len(); + debug!( + "@{:?} process start stalled for: {:?}ms txs: {} id: {}", + timestamp(), + duration_as_ms(&self.last_receive_time.elapsed()), + packet_count, + self.id, + ); + + if let Some(new_sigverify_stats) = &new_tracer_stats_option { + self.tracer_stats + .aggregate_sigverify_tracer_packet_stats(new_sigverify_stats); + } + + // Track all the packets incoming from sigverify, both valid and invalid + self.leader_stats + .increment_total_new_valid_packets(passed_sigverify_count); + self.leader_stats + .increment_newly_failed_sigverify_count(failed_sigverify_count); + + let mut dropped_packets_count = 0; + let mut newly_buffered_packets_count = 0; + self.push_unprocessed( + unprocessed_transaction_storage, + deserialized_packets, + &mut dropped_packets_count, + &mut newly_buffered_packets_count, + ); + recv_time.stop(); + + self.stats + .receive_and_buffer_packets_elapsed + .fetch_add(recv_time.as_us(), Ordering::Relaxed); + self.stats + .receive_and_buffer_packets_count + .fetch_add(packet_count, Ordering::Relaxed); + self.stats + .dropped_packets_count + .fetch_add(dropped_packets_count, Ordering::Relaxed); + self.stats + .newly_buffered_packets_count + .fetch_add(newly_buffered_packets_count, Ordering::Relaxed); + self.stats + .current_buffered_packets_count + .swap(unprocessed_transaction_storage.len(), Ordering::Relaxed); + self.last_receive_time = Instant::now(); + Ok(()) + } + + fn push_unprocessed( + &mut self, + unprocessed_transaction_storage: &mut UnprocessedTransactionStorage, + deserialized_packets: Vec, + dropped_packets_count: &mut usize, + newly_buffered_packets_count: &mut usize, + ) { + if !deserialized_packets.is_empty() { + let _ = self + .stats + .batch_packet_indexes_len + .increment(deserialized_packets.len() as u64); + + *newly_buffered_packets_count += deserialized_packets.len(); + self.leader_stats + .increment_newly_buffered_packets_count(deserialized_packets.len() as u64); + + let insert_packet_batches_summary = + unprocessed_transaction_storage.insert_batch(deserialized_packets); + self.leader_stats + .accumulate_insert_packet_batches_summary(&insert_packet_batches_summary); + saturating_add_assign!( + *dropped_packets_count, + insert_packet_batches_summary.total_dropped_packets() + ); + self.tracer_stats + .increment_total_exceeded_banking_stage_buffer( + insert_packet_batches_summary.dropped_tracer_packets(), + ); + } + } +} + +#[derive(Default, Debug)] +pub struct PacketReceiverStats { + last_report: AtomicInterval, + id: u32, + receive_and_buffer_packets_count: AtomicUsize, + dropped_packets_count: AtomicUsize, + dropped_duplicated_packets_count: AtomicUsize, + newly_buffered_packets_count: AtomicUsize, + current_buffered_packets_count: AtomicUsize, + batch_packet_indexes_len: Histogram, + receive_and_buffer_packets_elapsed: AtomicU64, +} + +impl PacketReceiverStats { + pub fn new(id: u32) -> Self { + Self { + id, + batch_packet_indexes_len: Histogram::configure() + .max_value(PACKETS_PER_BATCH as u64) + .build() + .unwrap(), + ..PacketReceiverStats::default() + } + } + + fn is_empty(&self) -> bool { + 0 == self + .receive_and_buffer_packets_count + .load(Ordering::Relaxed) + + self.dropped_packets_count.load(Ordering::Relaxed) + + self + .dropped_duplicated_packets_count + .load(Ordering::Relaxed) + + self.newly_buffered_packets_count.load(Ordering::Relaxed) + + self.current_buffered_packets_count.load(Ordering::Relaxed) + + self.batch_packet_indexes_len.entries() as usize + } + + fn report(&mut self, report_interval_ms: u64) { + if self.is_empty() { + return; + } + + if self.last_report.should_update(report_interval_ms) { + datapoint_info!( + "banking_stage-packet_receiver_stats", + ("id", self.id, i64), + ( + "receive_and_buffer_packet_counts", + self.receive_and_buffer_packets_count + .swap(0, Ordering::Relaxed), + i64 + ), + ( + "dropped_packets_count", + self.dropped_packets_count.swap(0, Ordering::Relaxed), + i64 + ), + ( + "dropped_duplicated_packets_count", + self.dropped_duplicated_packets_count + .swap(0, Ordering::Relaxed), + i64 + ), + ( + "newly_buffered_packets_count", + self.newly_buffered_packets_count.swap(0, Ordering::Relaxed), + i64 + ), + ( + "current_buffered_packets_count", + self.current_buffered_packets_count.load(Ordering::Relaxed), + i64 + ), + ( + "packet_batch_indices_len_min", + self.batch_packet_indexes_len.minimum().unwrap_or(0), + i64 + ), + ( + "packet_batch_indices_len_max", + self.batch_packet_indexes_len.maximum().unwrap_or(0), + i64 + ), + ( + "packet_batch_indices_len_mean", + self.batch_packet_indexes_len.mean().unwrap_or(0), + i64 + ), + ( + "packet_batch_indices_len_90pct", + self.batch_packet_indexes_len.percentile(90.0).unwrap_or(0), + i64 + ), + ( + "receive_and_buffer_packets_elapsed", + self.receive_and_buffer_packets_elapsed + .swap(0, Ordering::Relaxed), + i64 + ), + ); + self.batch_packet_indexes_len.clear(); + } + } +} + +#[derive(Default, Debug)] +pub struct TracerPacketReceiverStats { + id: u32, + last_report: AtomicInterval, + sigverify_tracer_packet_stats: SigverifyTracerPacketStats, + total_exceeded_banking_stage_buffer: AtomicUsize, +} + +impl TracerPacketReceiverStats { + pub fn new(id: u32) -> Self { + Self { + id, + ..TracerPacketReceiverStats::default() + } + } + + fn is_empty(&self) -> bool { + // If sigverify didn't see any, then nothing to report + 0 == self + .sigverify_tracer_packet_stats + .total_tracer_packets_received_in_sigverify_stage + } + + fn report(&mut self, report_interval_ms: u64) { + if self.is_empty() { + return; + } + + if self.last_report.should_update(report_interval_ms) { + datapoint_info!( + "tracer_packet_receiver_stats", + ("id", self.id, i64), + ( + "total_removed_before_sigverify", + self.sigverify_tracer_packet_stats + .total_removed_before_sigverify_stage, + i64 + ), + ( + "total_tracer_packets_received_in_sigverify", + self.sigverify_tracer_packet_stats + .total_tracer_packets_received_in_sigverify_stage, + i64 + ), + ( + "total_tracer_packets_deduped_in_sigverify", + self.sigverify_tracer_packet_stats + .total_tracer_packets_deduped, + i64 + ), + ( + "total_excess_tracer_packets_discarded_in_sigverify", + self.sigverify_tracer_packet_stats + .total_excess_tracer_packets, + i64 + ), + ( + "total_tracker_packets_passed_sigverify", + self.sigverify_tracer_packet_stats + .total_tracker_packets_passed_sigverify, + i64 + ), + ( + "total_exceeded_banking_stage_buffer", + self.total_exceeded_banking_stage_buffer + .swap(0, Ordering::Relaxed), + i64 + ) + ); + + *self = Self::new(self.id); + } + } + + fn aggregate_sigverify_tracer_packet_stats(&mut self, new_stats: &SigverifyTracerPacketStats) { + self.sigverify_tracer_packet_stats.aggregate(new_stats); + } + + fn increment_total_exceeded_banking_stage_buffer(&mut self, count: usize) { + self.total_exceeded_banking_stage_buffer + .fetch_add(count, Ordering::Relaxed); + } +} + +struct LeaderSlotReceiveMetricsTracker { + id: u32, + leader_slot_metrics: Option, +} + +struct LeaderSlotReceiveMetrics { + /// Aggregate metrics per slot + slot: Slot, + + // Counts: + /// Total number of live packets TPU received from verified receiver for processing. + total_new_valid_packets: u64, + /// Total number of packets TPU received from sigverify that failed signature verification. + newly_failed_sigverify_count: u64, + /// Total number of dropped packet due to the thread's buffered packets capacity being reached. + exceeded_buffer_limit_dropped_packets_count: u64, + /// Total number of packets that got added to the pending buffer after arriving to BankingStage + newly_buffered_packets_count: u64, + /// How many votes ingested from gossip were dropped + dropped_gossip_votes: u64, + /// How many votes ingested from tpu were dropped + dropped_tpu_votes: u64, + + // Timings: + /// Time spent processing new incoming packets to the banking thread + receive_and_buffer_packets_us: u64, + /// The number of times the function to receive and buffer new packets + /// was called + receive_and_buffer_packets_invoked_count: u64, +} + +impl LeaderSlotReceiveMetricsTracker { + fn new(id: u32) -> Self { + Self { + id, + leader_slot_metrics: None, + } + } + + fn check_leader_slot_boundary(&mut self, leader_slot: Option) { + match (&mut self.leader_slot_metrics, leader_slot) { + (Some(_), None) => { + self.report(); + self.leader_slot_metrics = None; + } + (Some(metrics), Some(leader_slot)) => { + if metrics.slot != leader_slot { + self.report(); + self.leader_slot_metrics = Some(LeaderSlotReceiveMetrics::new(leader_slot)); + } + } + (None, Some(leader_slot)) => { + self.leader_slot_metrics = Some(LeaderSlotReceiveMetrics::new(leader_slot)); + } + _ => (), + } + } + + fn report(&mut self) { + if let Some(metrics) = &mut self.leader_slot_metrics { + datapoint_info!( + "leader_slot_receive_metrics", + ("id", self.id, i64), + ("slot", metrics.slot, i64), + ( + "total_new_valid_packets", + metrics.total_new_valid_packets, + i64 + ), + ( + "newly_failed_sigverify_count", + metrics.newly_failed_sigverify_count, + i64 + ), + ( + "exceeded_buffer_limit_dropped_packets_count", + metrics.exceeded_buffer_limit_dropped_packets_count, + i64 + ), + ( + "newly_buffered_packets_count", + metrics.newly_buffered_packets_count, + i64 + ), + ("dropped_gossip_votes", metrics.dropped_gossip_votes, i64), + ("dropped_tpu_votes", metrics.dropped_tpu_votes, i64), + ( + "receive_and_buffer_packets_us", + metrics.receive_and_buffer_packets_us, + i64 + ), + ( + "receive_and_buffer_packets_invoked_count", + metrics.receive_and_buffer_packets_invoked_count, + i64 + ), + ); + } + } + + fn increment_receive_and_buffer_packets_us(&mut self, us: u64) { + if let Some(metrics) = &mut self.leader_slot_metrics { + saturating_add_assign!(metrics.receive_and_buffer_packets_us, us); + saturating_add_assign!(metrics.receive_and_buffer_packets_invoked_count, 1); + } + } + + fn increment_total_new_valid_packets(&mut self, count: u64) { + if let Some(metrics) = &mut self.leader_slot_metrics { + saturating_add_assign!(metrics.total_new_valid_packets, count); + } + } + + fn increment_newly_failed_sigverify_count(&mut self, count: u64) { + if let Some(metrics) = &mut self.leader_slot_metrics { + saturating_add_assign!(metrics.newly_failed_sigverify_count, count); + } + } + + fn increment_exceeded_buffer_limit_dropped_packets_count(&mut self, count: u64) { + if let Some(metrics) = &mut self.leader_slot_metrics { + saturating_add_assign!(metrics.exceeded_buffer_limit_dropped_packets_count, count); + } + } + + fn increment_newly_buffered_packets_count(&mut self, count: u64) { + if let Some(metrics) = &mut self.leader_slot_metrics { + saturating_add_assign!(metrics.newly_buffered_packets_count, count); + } + } + + fn accumulate_insert_packet_batches_summary( + &mut self, + insert_packet_batches_summary: &InsertPacketBatchSummary, + ) { + self.increment_exceeded_buffer_limit_dropped_packets_count( + insert_packet_batches_summary.total_dropped_packets() as u64, + ); + self.increment_dropped_gossip_vote_count( + insert_packet_batches_summary.dropped_gossip_packets() as u64, + ); + self.increment_dropped_tpu_vote_count( + insert_packet_batches_summary.dropped_tpu_packets() as u64 + ); + } + + fn increment_dropped_gossip_vote_count(&mut self, count: u64) { + if let Some(metrics) = &mut self.leader_slot_metrics { + saturating_add_assign!(metrics.dropped_gossip_votes, count); + } + } + + fn increment_dropped_tpu_vote_count(&mut self, count: u64) { + if let Some(metrics) = &mut self.leader_slot_metrics { + saturating_add_assign!(metrics.dropped_tpu_votes, count); + } + } +} + +impl LeaderSlotReceiveMetrics { + pub fn new(slot: Slot) -> Self { + Self { + slot, + total_new_valid_packets: 0, + newly_failed_sigverify_count: 0, + exceeded_buffer_limit_dropped_packets_count: 0, + newly_buffered_packets_count: 0, + dropped_gossip_votes: 0, + dropped_tpu_votes: 0, + receive_and_buffer_packets_us: 0, + receive_and_buffer_packets_invoked_count: 0, + } + } +} diff --git a/core/src/banking_stage/record_executor.rs b/core/src/banking_stage/record_executor.rs new file mode 100644 index 00000000000000..47866a5b6cd952 --- /dev/null +++ b/core/src/banking_stage/record_executor.rs @@ -0,0 +1,73 @@ +use { + crate::leader_slot_banking_stage_timing_metrics::RecordTransactionsTimings, + solana_entry::entry::hash_transactions, + solana_measure::measure, + solana_poh::poh_recorder::{PohRecorderError, Slot, TransactionRecorder}, + solana_sdk::transaction::VersionedTransaction, +}; + +pub struct RecordTransactionsSummary { + // Metrics describing how time was spent recording transactions + pub record_transactions_timings: RecordTransactionsTimings, + // Result of trying to record the transactions into the PoH stream + pub result: Result<(), PohRecorderError>, + // Index in the slot of the first transaction recorded + pub starting_transaction_index: Option, +} + +pub struct RecordExecutor { + recorder: TransactionRecorder, +} + +impl RecordExecutor { + pub fn new(recorder: TransactionRecorder) -> Self { + Self { recorder } + } + + pub fn record_transactions( + &self, + bank_slot: Slot, + transactions: Vec, + ) -> RecordTransactionsSummary { + let mut record_transactions_timings = RecordTransactionsTimings::default(); + let mut starting_transaction_index = None; + + if !transactions.is_empty() { + let num_to_record = transactions.len(); + inc_new_counter_info!("banking_stage-record_count", 1); + inc_new_counter_info!("banking_stage-record_transactions", num_to_record); + + let (hash, hash_time) = measure!(hash_transactions(&transactions), "hash"); + record_transactions_timings.hash_us = hash_time.as_us(); + + let (res, poh_record_time) = + measure!(self.recorder.record(bank_slot, hash, transactions), "hash"); + record_transactions_timings.poh_record_us = poh_record_time.as_us(); + + match res { + Ok(starting_index) => { + starting_transaction_index = starting_index; + } + Err(PohRecorderError::MaxHeightReached) => { + inc_new_counter_info!("banking_stage-max_height_reached", 1); + inc_new_counter_info!( + "banking_stage-max_height_reached_num_to_commit", + num_to_record + ); + return RecordTransactionsSummary { + record_transactions_timings, + result: Err(PohRecorderError::MaxHeightReached), + starting_transaction_index: None, + }; + } + Err(e) => panic!("Poh recorder returned unexpected error: {:?}", e), + } + } + + RecordTransactionsSummary { + record_transactions_timings, + result: Ok(()), + starting_transaction_index, + } + } +} diff --git a/core/src/banking_stage/scheduler_error.rs b/core/src/banking_stage/scheduler_error.rs new file mode 100644 index 00000000000000..de3c78a7691015 --- /dev/null +++ b/core/src/banking_stage/scheduler_error.rs @@ -0,0 +1,21 @@ +use thiserror::Error; + +/// Reasons a scheduler might fail. +#[derive(Error, Debug)] +pub enum SchedulerError { + /// Packet receiver was disconnected. + #[error("Packet receiver was disconnected")] + PacketReceiverDisconnected, + + /// Scheduled work receiver was disconnected + #[error("Scheduled work receiver was disconnected")] + ScheduledWorkReceiverDisconnected, + + /// Processed transactions sender was disconnected. + #[error("Processed transactions sender was disconnected")] + ProcessedTransactionsSenderDisconnected, + + /// Processed transactions receiver was disconnected. + #[error("Processed transactions receiver was disconnected")] + ProcessedTransactionsReceiverDisconnected, +} diff --git a/core/src/banking_stage/scheduler_handle.rs b/core/src/banking_stage/scheduler_handle.rs new file mode 100644 index 00000000000000..efed9778849fed --- /dev/null +++ b/core/src/banking_stage/scheduler_handle.rs @@ -0,0 +1,84 @@ +use { + super::{ + consume_executor::ConsumeExecutor, decision_maker::DecisionMaker, + external_scheduler::ExternalSchedulerHandle, forward_executor::ForwardExecutor, + packet_receiver::PacketReceiver, scheduler_error::SchedulerError, + thread_local_scheduler::ThreadLocalScheduler, + }, + crate::{ + leader_slot_banking_stage_metrics::LeaderSlotMetricsTracker, + scheduler_stage::{ProcessedTransactionsSender, ScheduledTransactionsReceiver}, + tracer_packet_stats::TracerPacketStats, + unprocessed_transaction_storage::UnprocessedTransactionStorage, + }, + solana_runtime::bank_forks::BankForks, + std::sync::{Arc, RwLock}, +}; + +#[allow(dead_code)] +pub(crate) enum SchedulerHandle { + ThreadLocalScheduler(ThreadLocalScheduler), + ExternalScheduler(ExternalSchedulerHandle), +} + +impl SchedulerHandle { + pub fn new_thread_local_scheduler( + id: u32, + decision_maker: DecisionMaker, + unprocessed_transaction_storage: UnprocessedTransactionStorage, + bank_forks: Arc>, + packet_receiver: PacketReceiver, + ) -> Self { + Self::ThreadLocalScheduler(ThreadLocalScheduler::new( + id, + decision_maker, + unprocessed_transaction_storage, + bank_forks, + packet_receiver, + )) + } + + pub fn new_external_scheduler( + id: u32, + scheduled_transactions_receiver: ScheduledTransactionsReceiver, + processed_transactions_sender: ProcessedTransactionsSender, + ) -> Self { + Self::ExternalScheduler(ExternalSchedulerHandle::new( + id, + scheduled_transactions_receiver, + processed_transactions_sender, + )) + } + + /// Do necessary updates to the scheduler interface + pub fn tick(&mut self) -> Result<(), SchedulerError> { + match self { + Self::ThreadLocalScheduler(thread_local_scheduler) => thread_local_scheduler.tick(), + Self::ExternalScheduler(external_scheduler) => external_scheduler.tick(), + } + } + + /// Do work that is scheduled + pub fn do_scheduled_work( + &mut self, + consume_executor: &ConsumeExecutor, + forward_executor: &ForwardExecutor, + tracer_packet_stats: &mut TracerPacketStats, + slot_metrics_tracker: &mut LeaderSlotMetricsTracker, + ) -> Result<(), SchedulerError> { + match self { + Self::ThreadLocalScheduler(thread_local_scheduler) => thread_local_scheduler + .do_scheduled_work( + consume_executor, + forward_executor, + tracer_packet_stats, + slot_metrics_tracker, + ), + Self::ExternalScheduler(external_scheduler) => external_scheduler.do_scheduled_work( + consume_executor, + forward_executor, + slot_metrics_tracker, + ), + } + } +} diff --git a/core/src/banking_stage/thread_aware_account_locks.rs b/core/src/banking_stage/thread_aware_account_locks.rs new file mode 100644 index 00000000000000..1d6143827d036a --- /dev/null +++ b/core/src/banking_stage/thread_aware_account_locks.rs @@ -0,0 +1,267 @@ +//! Account locks that keep track of which threads hold them. +//! + +use { + solana_sdk::pubkey::Pubkey, + std::{ + collections::{hash_map::Entry, HashMap}, + ops::{BitAnd, BitAndAssign}, + }, +}; + +pub const MAX_THREADS: u8 = 8; + +/// Identifier for a thread. +pub type ThreadId = u8; // Only supports up to 8 threads currently. + +/// Set of threads an account is scheduled or can be scheduled for. +#[derive(Copy, Clone, Debug, PartialEq, Eq)] +#[repr(transparent)] +pub struct ThreadSet { + /// Bitset for threads - `u8` is sufficient for up to 8 threads. + set: u8, +} + +/// Thread-aware account locks which allows for scheduling on threads +/// that already hold locks. This is useful for allowing queued +/// transactions to be scheduled on a thread while the transaction is +/// still executing on that thread. +pub struct ThreadAwareAccountLocks { + /// Number of threads. + num_threads: u8, + /// Write locks - only one thread can hold a write lock at a time. + /// Contains how many write locks are held by the thread. + write_locks: HashMap, + /// Read locks - multiple threads can hold a read lock at the same time. + /// Contains thread-set for easily checking which threads are scheudled. + /// Contains how many read locks are held by each thread. + read_locks: HashMap, +} + +impl ThreadAwareAccountLocks { + /// Creates a new `ThreadAwareAccountLocks` with the given number of threads. + pub fn new(num_threads: u8) -> Self { + assert!(num_threads <= MAX_THREADS); + Self { + num_threads, + write_locks: HashMap::new(), + read_locks: HashMap::new(), + } + } + + /// Returns `ThreadSet` that the given accounts can be scheduled on. + pub fn accounts_schedulable_threads<'a>( + &self, + write_account_locks: impl Iterator, + read_account_locks: impl Iterator, + ) -> ThreadSet { + let mut schedulable_threads = ThreadSet::any(self.num_threads); + + // Get schedulable threads for write-locked accounts. + write_account_locks.for_each(|pubkey| { + schedulable_threads &= self.account_write_lockable_threads(pubkey); + }); + + // Get schedulable threads for read-locked accounts. + read_account_locks.for_each(|pubkey| { + schedulable_threads &= self.account_read_lockable_threads(pubkey); + }); + + schedulable_threads + } + + /// Adds locks for all writable and readable accounts. + pub fn lock_accounts<'a>( + &mut self, + write_account_locks: impl Iterator, + read_account_locks: impl Iterator, + thread_id: ThreadId, + ) { + // Lock write-locked accounts. + write_account_locks.for_each(|pubkey| { + self.lock_account_write(pubkey, thread_id); + }); + + // Lock read-locked accounts. + read_account_locks.for_each(|pubkey| { + self.lock_account_read(pubkey, thread_id); + }); + } + + /// Removes locks for all writable and readable accounts. + pub fn unlock_accounts<'a>( + &mut self, + write_account_locks: impl Iterator, + read_account_locks: impl Iterator, + thread_id: ThreadId, + ) { + // Unlock write-locked accounts. + write_account_locks.for_each(|pubkey| { + self.unlock_account_write(pubkey, thread_id); + }); + + // Unlock read-locked accounts. + read_account_locks.for_each(|pubkey| { + self.unlock_account_read(pubkey, thread_id); + }); + } + + /// Returns `ThreadSet` that the given `pubkey` can be scheduled on for writing. + fn account_write_lockable_threads(&self, pubkey: &Pubkey) -> ThreadSet { + // If the account is write-locked, only the thread that holds the lock can schedule it. + // Otherwise, we need to check against read-locks. + if let Some((thread_id, _)) = self.write_locks.get(pubkey) { + ThreadSet::only(*thread_id) + } else if let Some((read_thread_set, _)) = self.read_locks.get(pubkey) { + // If the account is read-locked, then it can only be write-locked iff there is only one + // thread that holds the read-lock currently. + (read_thread_set.num_threads() == 1) + .then_some(*read_thread_set) + .unwrap_or_else(ThreadSet::none) + } else { + ThreadSet::any(self.num_threads) + } + } + + /// Returns `ThreadSet` that the given `pubkey` can be scheduled on for reading. + fn account_read_lockable_threads(&self, pubkey: &Pubkey) -> ThreadSet { + self.write_locks + .get(pubkey) + .map(|(thread_id, _)| ThreadSet::only(*thread_id)) + .unwrap_or_else(|| ThreadSet::any(self.num_threads)) + } + + /// Locks the given `pubkey` for writing by the given `thread_id`. + fn lock_account_write(&mut self, pubkey: &Pubkey, thread_id: ThreadId) { + match self.write_locks.entry(*pubkey) { + Entry::Occupied(mut entry) => { + let (lock_thread_id, lock_count) = entry.get_mut(); + assert_eq!(*lock_thread_id, thread_id); + *lock_count += 1; + } + Entry::Vacant(entry) => { + entry.insert((thread_id, 1)); + } + } + } + + /// Unlocks the given `pubkey` for writing by the given `thread_id`. + fn unlock_account_write(&mut self, pubkey: &Pubkey, thread_id: ThreadId) { + match self.write_locks.entry(*pubkey) { + Entry::Occupied(mut entry) => { + let (lock_thread_id, lock_count) = entry.get_mut(); + assert_eq!(*lock_thread_id, thread_id); + *lock_count -= 1; + if *lock_count == 0 { + entry.remove(); + } + } + Entry::Vacant(_) => panic!("write lock not held for pubkey: {pubkey:?}"), + } + } + + /// Locks the given `pubkey` for reading by the given `thread_id`. + fn lock_account_read(&mut self, pubkey: &Pubkey, thread_id: ThreadId) { + match self.read_locks.entry(*pubkey) { + Entry::Occupied(mut entry) => { + let (read_thread_set, read_lock_counts) = entry.get_mut(); + read_thread_set.insert(thread_id); + read_lock_counts[thread_id as usize] += 1; + } + Entry::Vacant(entry) => { + let mut read_lock_counts = [0; MAX_THREADS as usize]; + read_lock_counts[thread_id as usize] = 1; + entry.insert((ThreadSet::only(thread_id), read_lock_counts)); + } + } + } + + /// Unlocks the given `pubkey` for reading by the given `thread_id`. + fn unlock_account_read(&mut self, pubkey: &Pubkey, thread_id: ThreadId) { + match self.read_locks.entry(*pubkey) { + Entry::Occupied(mut entry) => { + let (read_thread_set, read_lock_counts) = entry.get_mut(); + read_lock_counts[thread_id as usize] -= 1; + if read_lock_counts[thread_id as usize] == 0 { + read_thread_set.remove(thread_id); + } + if read_thread_set.is_empty() { + entry.remove(); + } + } + Entry::Vacant(_) => panic!("read lock not held for pubkey: {pubkey:?}"), + } + } +} + +impl ThreadSet { + #[inline(always)] + pub fn none() -> Self { + Self { set: 0 } + } + + #[inline(always)] + pub fn any(num_threads: u8) -> Self { + Self { + set: (1 << num_threads) - 1, + } + } + + #[inline(always)] + pub fn only(thread_id: ThreadId) -> Self { + Self { + set: 1 << thread_id, + } + } + + #[inline(always)] + pub fn num_threads(&self) -> u8 { + self.set.count_ones() as u8 + } + + #[inline(always)] + pub fn only_one_scheduled(&self) -> Option { + (self.num_threads() == 1).then_some(self.set.trailing_zeros() as ThreadId) + } + + #[inline(always)] + pub fn is_empty(&self) -> bool { + self.set == 0 + } + + #[inline(always)] + pub fn contains(&self, thread_id: ThreadId) -> bool { + self.set & (1 << thread_id) != 0 + } + + #[inline(always)] + pub fn insert(&mut self, thread_id: ThreadId) { + self.set |= 1 << thread_id; + } + + #[inline(always)] + pub fn remove(&mut self, thread_id: ThreadId) { + self.set &= !(1 << thread_id); + } + + #[inline(always)] + pub fn threads_iter(self) -> impl Iterator { + (0..MAX_THREADS as ThreadId).filter(move |thread_id| self.contains(*thread_id)) + } +} + +impl BitAndAssign for ThreadSet { + fn bitand_assign(&mut self, rhs: Self) { + self.set &= rhs.set; + } +} + +impl BitAnd for ThreadSet { + type Output = Self; + + fn bitand(self, rhs: Self) -> Self::Output { + Self { + set: self.set & rhs.set, + } + } +} diff --git a/core/src/banking_stage/thread_local_scheduler.rs b/core/src/banking_stage/thread_local_scheduler.rs new file mode 100644 index 00000000000000..9a6bbbf5495606 --- /dev/null +++ b/core/src/banking_stage/thread_local_scheduler.rs @@ -0,0 +1,376 @@ +use { + super::{ + consume_executor::ConsumeExecutor, + decision_maker::{BufferedPacketsDecision, DecisionMaker}, + forward_executor::ForwardExecutor, + packet_receiver::PacketReceiver, + scheduler_error::SchedulerError, + BankingStageStats, SLOT_BOUNDARY_CHECK_PERIOD, + }, + crate::{ + forward_packet_batches_by_accounts::ForwardPacketBatchesByAccounts, + immutable_deserialized_packet::ImmutableDeserializedPacket, + leader_slot_banking_stage_metrics::{LeaderSlotMetricsTracker, ProcessTransactionsSummary}, + tracer_packet_stats::TracerPacketStats, + unprocessed_transaction_storage::{ConsumeScannerPayload, UnprocessedTransactionStorage}, + }, + crossbeam_channel::RecvTimeoutError, + solana_measure::{measure, measure::Measure, measure_us}, + solana_poh::poh_recorder::BankStart, + solana_runtime::bank_forks::BankForks, + solana_sdk::timing::timestamp, + std::{ + sync::{atomic::Ordering, Arc, RwLock}, + time::Instant, + }, +}; + +/// Scheduler that lives in the same thread as executors. Handle is equivalent +/// to the scheduler itself. +pub struct ThreadLocalScheduler { + decision_maker: DecisionMaker, + unprocessed_transaction_storage: UnprocessedTransactionStorage, + bank_forks: Arc>, + packet_receiver: PacketReceiver, + last_metrics_update: Instant, + banking_stage_stats: BankingStageStats, +} + +impl ThreadLocalScheduler { + pub fn new( + id: u32, + decision_maker: DecisionMaker, + unprocessed_transaction_storage: UnprocessedTransactionStorage, + bank_forks: Arc>, + packet_receiver: PacketReceiver, + ) -> Self { + Self { + decision_maker, + unprocessed_transaction_storage, + bank_forks, + packet_receiver, + last_metrics_update: Instant::now(), + banking_stage_stats: BankingStageStats::new(id), + } + } + + pub fn tick(&mut self) -> Result<(), SchedulerError> { + let result = if matches!( + self.packet_receiver + .do_packet_receiving_and_buffering(&mut self.unprocessed_transaction_storage,), + Err(RecvTimeoutError::Disconnected) + ) { + Err(SchedulerError::PacketReceiverDisconnected) + } else { + Ok(()) + }; + + self.banking_stage_stats.report(1000); + + result + } + + pub fn do_scheduled_work( + &mut self, + consume_executor: &ConsumeExecutor, + forward_executor: &ForwardExecutor, + tracer_packet_stats: &mut TracerPacketStats, + slot_metrics_tracker: &mut LeaderSlotMetricsTracker, + ) -> Result<(), SchedulerError> { + if !self.unprocessed_transaction_storage.is_empty() + || self.last_metrics_update.elapsed() >= SLOT_BOUNDARY_CHECK_PERIOD + { + let (_, process_buffered_packets_us) = measure_us!(self.process_buffered_packets( + consume_executor, + forward_executor, + tracer_packet_stats, + slot_metrics_tracker, + )); + slot_metrics_tracker.increment_process_buffered_packets_us(process_buffered_packets_us); + self.last_metrics_update = Instant::now(); + } + + Ok(()) + } + + fn process_buffered_packets( + &mut self, + consume_executor: &ConsumeExecutor, + forward_executor: &ForwardExecutor, + tracer_packet_stats: &mut TracerPacketStats, + slot_metrics_tracker: &mut LeaderSlotMetricsTracker, + ) { + if self.unprocessed_transaction_storage.should_not_process() { + return; + } + + let (decision, make_decision_us) = + measure_us!(self.decision_maker.make_consume_or_forward_decision()); + slot_metrics_tracker.increment_make_decision_us(make_decision_us); + + let leader_slot = match &decision { + BufferedPacketsDecision::Consume(bank_start) => Some(bank_start.working_bank.slot()), + _ => None, + }; + self.packet_receiver.check_leader_slot_boundary(leader_slot); + + match decision { + BufferedPacketsDecision::Consume(bank_start) => { + // Take metrics action before consume packets (potentially resetting the + // slot metrics tracker to the next slot) so that we don't count the + // packet processing metrics from the next slot towards the metrics + // of the previous slot + slot_metrics_tracker.apply_working_bank(Some(&bank_start)); + let (_, consume_buffered_packets_us) = measure_us!(Self::consume_buffered_packets( + consume_executor, + &bank_start, + &mut self.unprocessed_transaction_storage, + None::>, + &self.banking_stage_stats, + slot_metrics_tracker + )); + slot_metrics_tracker + .increment_consume_buffered_packets_us(consume_buffered_packets_us); + } + BufferedPacketsDecision::Forward => { + let (_, forward_us) = measure_us!(Self::handle_forwarding( + forward_executor, + &mut self.unprocessed_transaction_storage, + &self.bank_forks, + false, + slot_metrics_tracker, + &self.banking_stage_stats, + tracer_packet_stats, + )); + slot_metrics_tracker.increment_forward_us(forward_us); + // Take metrics action after forwarding packets to include forwarded + // metrics into current slot + slot_metrics_tracker.apply_working_bank(None); + } + BufferedPacketsDecision::ForwardAndHold => { + let (_, forward_and_hold_us) = measure_us!(Self::handle_forwarding( + forward_executor, + &mut self.unprocessed_transaction_storage, + &self.bank_forks, + true, + slot_metrics_tracker, + &self.banking_stage_stats, + tracer_packet_stats, + )); + slot_metrics_tracker.increment_forward_and_hold_us(forward_and_hold_us); + // Take metrics action after forwarding packets + slot_metrics_tracker.apply_working_bank(None); + } + _ => (), + } + } + + pub fn consume_buffered_packets( + consume_executor: &ConsumeExecutor, + bank_start: &BankStart, + unprocessed_transaction_storage: &mut UnprocessedTransactionStorage, + test_fn: Option, + banking_stage_stats: &BankingStageStats, + slot_metrics_tracker: &mut LeaderSlotMetricsTracker, + ) { + let mut rebuffered_packet_count = 0; + let mut consumed_buffered_packets_count = 0; + let mut proc_start = Measure::start("consume_buffered_process"); + let num_packets_to_process = unprocessed_transaction_storage.len(); + + let reached_end_of_slot = unprocessed_transaction_storage.process_packets( + bank_start.working_bank.clone(), + banking_stage_stats, + slot_metrics_tracker, + |packets_to_process, payload| { + Self::do_process_packets( + consume_executor, + bank_start, + payload, + banking_stage_stats, + &mut consumed_buffered_packets_count, + &mut rebuffered_packet_count, + &test_fn, + packets_to_process, + ) + }, + ); + + if reached_end_of_slot { + slot_metrics_tracker.set_end_of_slot_unprocessed_buffer_len( + unprocessed_transaction_storage.len() as u64, + ); + } + + proc_start.stop(); + debug!( + "@{:?} done processing buffered batches: {} time: {:?}ms tx count: {} tx/s: {}", + timestamp(), + num_packets_to_process, + proc_start.as_ms(), + consumed_buffered_packets_count, + (consumed_buffered_packets_count as f32) / (proc_start.as_s()) + ); + + banking_stage_stats + .consume_buffered_packets_elapsed + .fetch_add(proc_start.as_us(), Ordering::Relaxed); + banking_stage_stats + .rebuffered_packets_count + .fetch_add(rebuffered_packet_count, Ordering::Relaxed); + banking_stage_stats + .consumed_buffered_packets_count + .fetch_add(consumed_buffered_packets_count, Ordering::Relaxed); + } + + fn do_process_packets( + consume_executor: &ConsumeExecutor, + bank_start: &BankStart, + payload: &mut ConsumeScannerPayload, + banking_stage_stats: &BankingStageStats, + consumed_buffered_packets_count: &mut usize, + rebuffered_packet_count: &mut usize, + test_fn: &Option, + packets_to_process: &Vec>, + ) -> Option> { + if payload.reached_end_of_slot { + return None; + } + + let packets_to_process_len = packets_to_process.len(); + let (process_transactions_summary, process_packets_transactions_us) = + measure_us!(consume_executor.process_packets_transactions( + bank_start, + &payload.sanitized_transactions, + banking_stage_stats, + payload.slot_metrics_tracker, + )); + payload + .slot_metrics_tracker + .increment_process_packets_transactions_us(process_packets_transactions_us); + + // Clear payload for next iteration + payload.sanitized_transactions.clear(); + payload.account_locks.clear(); + + let ProcessTransactionsSummary { + reached_max_poh_height, + retryable_transaction_indexes, + .. + } = process_transactions_summary; + + if reached_max_poh_height || !bank_start.should_working_bank_still_be_processing_txs() { + payload.reached_end_of_slot = true; + } + + // The difference between all transactions passed to execution and the ones that + // are retryable were the ones that were either: + // 1) Committed into the block + // 2) Dropped without being committed because they had some fatal error (too old, + // duplicate signature, etc.) + // + // Note: This assumes that every packet deserializes into one transaction! + *consumed_buffered_packets_count += + packets_to_process_len.saturating_sub(retryable_transaction_indexes.len()); + + // Out of the buffered packets just retried, collect any still unprocessed + // transactions in this batch for forwarding + *rebuffered_packet_count += retryable_transaction_indexes.len(); + if let Some(test_fn) = test_fn { + test_fn(); + } + + payload + .slot_metrics_tracker + .increment_retryable_packets_count(retryable_transaction_indexes.len() as u64); + + Some(retryable_transaction_indexes) + } + + pub fn handle_forwarding( + forward_executor: &ForwardExecutor, + unprocessed_transaction_storage: &mut UnprocessedTransactionStorage, + bank_forks: &RwLock, + hold: bool, + slot_metrics_tracker: &mut LeaderSlotMetricsTracker, + banking_stage_stats: &BankingStageStats, + tracer_packet_stats: &mut TracerPacketStats, + ) { + let forward_option = unprocessed_transaction_storage.forward_option(); + + // get current root bank from bank_forks, use it to sanitize transaction and + // load all accounts from address loader; + let current_bank = bank_forks.read().unwrap().root_bank(); + + let mut forward_packet_batches_by_accounts = + ForwardPacketBatchesByAccounts::new_with_default_batch_limits(); + + // sanitize and filter packets that are no longer valid (could be too old, a duplicate of something + // already processed), then add to forwarding buffer. + let filter_forwarding_result = unprocessed_transaction_storage + .filter_forwardable_packets_and_add_batches( + current_bank, + &mut forward_packet_batches_by_accounts, + ); + slot_metrics_tracker.increment_transactions_from_packets_us( + filter_forwarding_result.total_packet_conversion_us, + ); + banking_stage_stats.packet_conversion_elapsed.fetch_add( + filter_forwarding_result.total_packet_conversion_us, + Ordering::Relaxed, + ); + banking_stage_stats + .filter_pending_packets_elapsed + .fetch_add( + filter_forwarding_result.total_filter_packets_us, + Ordering::Relaxed, + ); + + forward_packet_batches_by_accounts + .iter_batches() + .filter(|&batch| !batch.is_empty()) + .for_each(|forward_batch| { + slot_metrics_tracker.increment_forwardable_batches_count(1); + + let batched_forwardable_packets_count = forward_batch.len(); + let (_forward_result, sucessful_forwarded_packets_count, leader_pubkey) = + forward_executor.forward_buffered_packets( + &forward_option, + forward_batch.get_forwardable_packets(), + banking_stage_stats, + ); + + if let Some(leader_pubkey) = leader_pubkey { + tracer_packet_stats.increment_total_forwardable_tracer_packets( + filter_forwarding_result.total_forwardable_tracer_packets, + leader_pubkey, + ); + } + let failed_forwarded_packets_count = batched_forwardable_packets_count + .saturating_sub(sucessful_forwarded_packets_count); + + if failed_forwarded_packets_count > 0 { + slot_metrics_tracker.increment_failed_forwarded_packets_count( + failed_forwarded_packets_count as u64, + ); + slot_metrics_tracker.increment_packet_batch_forward_failure_count(1); + } + + if sucessful_forwarded_packets_count > 0 { + slot_metrics_tracker.increment_successful_forwarded_packets_count( + sucessful_forwarded_packets_count as u64, + ); + } + }); + + if !hold { + slot_metrics_tracker.increment_cleared_from_buffer_after_forward_count( + filter_forwarding_result.total_forwardable_packets as u64, + ); + tracer_packet_stats.increment_total_cleared_from_buffer_after_forward( + filter_forwarding_result.total_tracer_packets_in_buffer, + ); + unprocessed_transaction_storage.clear_forwarded_packets(); + } + } +} diff --git a/core/src/leader_slot_banking_stage_metrics.rs b/core/src/leader_slot_banking_stage_metrics.rs index cfd14443fa906b..f8cdf63bad5448 100644 --- a/core/src/leader_slot_banking_stage_metrics.rs +++ b/core/src/leader_slot_banking_stage_metrics.rs @@ -1,8 +1,5 @@ use { - crate::{ - leader_slot_banking_stage_timing_metrics::*, - unprocessed_transaction_storage::InsertPacketBatchSummary, - }, + crate::leader_slot_banking_stage_timing_metrics::*, solana_poh::poh_recorder::BankStart, solana_runtime::transaction_error_metrics::*, solana_sdk::{clock::Slot, saturating_add_assign}, @@ -58,18 +55,6 @@ pub(crate) struct ProcessTransactionsSummary { // validator's leader slot #[derive(Debug, Default)] struct LeaderSlotPacketCountMetrics { - // total number of live packets TPU received from verified receiver for processing. - total_new_valid_packets: u64, - - // total number of packets TPU received from sigverify that failed signature verification. - newly_failed_sigverify_count: u64, - - // total number of dropped packet due to the thread's buffered packets capacity being reached. - exceeded_buffer_limit_dropped_packets_count: u64, - - // total number of packets that got added to the pending buffer after arriving to BankingStage - newly_buffered_packets_count: u64, - // total number of transactions in the buffer that were filtered out due to things like age and // duplicate signature checks retryable_packets_filtered_count: u64, @@ -150,26 +135,6 @@ impl LeaderSlotPacketCountMetrics { "banking_stage-leader_slot_packet_counts", ("id", id as i64, i64), ("slot", slot as i64, i64), - ( - "total_new_valid_packets", - self.total_new_valid_packets as i64, - i64 - ), - ( - "newly_failed_sigverify_count", - self.newly_failed_sigverify_count as i64, - i64 - ), - ( - "exceeded_buffer_limit_dropped_packets_count", - self.exceeded_buffer_limit_dropped_packets_count as i64, - i64 - ), - ( - "newly_buffered_packets_count", - self.newly_buffered_packets_count as i64, - i64 - ), ( "retryable_packets_filtered_count", self.retryable_packets_filtered_count as i64, @@ -273,8 +238,6 @@ pub(crate) struct LeaderSlotMetrics { transaction_error_metrics: TransactionErrorMetrics, - vote_packet_count_metrics: VotePacketCountMetrics, - timing_metrics: LeaderSlotTimingMetrics, // Used by tests to check if the `self.report()` method was called @@ -288,7 +251,6 @@ impl LeaderSlotMetrics { slot, packet_count_metrics: LeaderSlotPacketCountMetrics::new(), transaction_error_metrics: TransactionErrorMetrics::new(), - vote_packet_count_metrics: VotePacketCountMetrics::new(), timing_metrics: LeaderSlotTimingMetrics::new(bank_creation_time), is_reported: false, } @@ -300,7 +262,6 @@ impl LeaderSlotMetrics { self.timing_metrics.report(self.id, self.slot); self.transaction_error_metrics.report(self.id, self.slot); self.packet_count_metrics.report(self.id, self.slot); - self.vote_packet_count_metrics.report(self.id, self.slot); } /// Returns `Some(self.slot)` if the metrics have been reported, otherwise returns None @@ -317,41 +278,6 @@ impl LeaderSlotMetrics { } } -// Metrics describing vote tx packets that were processed in the tpu vote thread as well as -// extraneous votes that were filtered out -#[derive(Debug, Default)] -pub(crate) struct VotePacketCountMetrics { - // How many votes ingested from gossip were dropped - dropped_gossip_votes: u64, - - // How many votes ingested from tpu were dropped - dropped_tpu_votes: u64, -} - -impl VotePacketCountMetrics { - fn new() -> Self { - Self { ..Self::default() } - } - - fn report(&self, id: u32, slot: Slot) { - datapoint_info!( - "banking_stage-vote_packet_counts", - ("id", id, i64), - ("slot", slot, i64), - ("dropped_gossip_votes", self.dropped_gossip_votes, i64), - ("dropped_tpu_votes", self.dropped_tpu_votes, i64) - ); - } -} - -#[derive(Debug)] -pub(crate) enum MetricsTrackerAction { - Noop, - ReportAndResetTracker, - NewTracker(Option), - ReportAndNewTracker(Option), -} - #[derive(Debug)] pub struct LeaderSlotMetricsTracker { // Only `Some` if BankingStage detects it's time to construct our leader slot, @@ -368,68 +294,41 @@ impl LeaderSlotMetricsTracker { } } - // Check leader slot, return MetricsTrackerAction to be applied by apply_action() - pub(crate) fn check_leader_slot_boundary( - &mut self, - bank_start: &Option, - ) -> MetricsTrackerAction { + pub(crate) fn apply_working_bank(&mut self, bank_start: Option<&BankStart>) -> Option { match (self.leader_slot_metrics.as_mut(), bank_start) { - (None, None) => MetricsTrackerAction::Noop, + (None, None) => None, (Some(leader_slot_metrics), None) => { leader_slot_metrics.mark_slot_end_detected(); - MetricsTrackerAction::ReportAndResetTracker + leader_slot_metrics.report(); + let reported_slot = leader_slot_metrics.reported_slot(); + self.leader_slot_metrics = None; + reported_slot } - // Our leader slot has begain, time to create a new slot tracker (None, Some(bank_start)) => { - MetricsTrackerAction::NewTracker(Some(LeaderSlotMetrics::new( + self.leader_slot_metrics = Some(LeaderSlotMetrics::new( self.id, bank_start.working_bank.slot(), &bank_start.bank_creation_time, - ))) + )); + self.leader_slot_metrics.as_ref().unwrap().reported_slot() } (Some(leader_slot_metrics), Some(bank_start)) => { if leader_slot_metrics.slot != bank_start.working_bank.slot() { - // Last slot has ended, new slot has began leader_slot_metrics.mark_slot_end_detected(); - MetricsTrackerAction::ReportAndNewTracker(Some(LeaderSlotMetrics::new( + leader_slot_metrics.report(); + let reported_slot = leader_slot_metrics.reported_slot(); + self.leader_slot_metrics = Some(LeaderSlotMetrics::new( self.id, bank_start.working_bank.slot(), &bank_start.bank_creation_time, - ))) + )); + reported_slot } else { - MetricsTrackerAction::Noop - } - } - } - } - - pub(crate) fn apply_action(&mut self, action: MetricsTrackerAction) -> Option { - match action { - MetricsTrackerAction::Noop => None, - MetricsTrackerAction::ReportAndResetTracker => { - let mut reported_slot = None; - if let Some(leader_slot_metrics) = self.leader_slot_metrics.as_mut() { - leader_slot_metrics.report(); - reported_slot = leader_slot_metrics.reported_slot(); - } - self.leader_slot_metrics = None; - reported_slot - } - MetricsTrackerAction::NewTracker(new_slot_metrics) => { - self.leader_slot_metrics = new_slot_metrics; - self.leader_slot_metrics.as_ref().unwrap().reported_slot() - } - MetricsTrackerAction::ReportAndNewTracker(new_slot_metrics) => { - let mut reported_slot = None; - if let Some(leader_slot_metrics) = self.leader_slot_metrics.as_mut() { - leader_slot_metrics.report(); - reported_slot = leader_slot_metrics.reported_slot(); + None } - self.leader_slot_metrics = new_slot_metrics; - reported_slot } } } @@ -532,21 +431,6 @@ impl LeaderSlotMetricsTracker { } } - pub(crate) fn accumulate_insert_packet_batches_summary( - &mut self, - insert_packet_batches_summary: &InsertPacketBatchSummary, - ) { - self.increment_exceeded_buffer_limit_dropped_packets_count( - insert_packet_batches_summary.total_dropped_packets() as u64, - ); - self.increment_dropped_gossip_vote_count( - insert_packet_batches_summary.dropped_gossip_packets() as u64, - ); - self.increment_dropped_tpu_vote_count( - insert_packet_batches_summary.dropped_tpu_packets() as u64 - ); - } - pub(crate) fn accumulate_transaction_errors( &mut self, error_metrics: &TransactionErrorMetrics, @@ -558,51 +442,6 @@ impl LeaderSlotMetricsTracker { } } - // Packet inflow/outflow/processing metrics - pub(crate) fn increment_total_new_valid_packets(&mut self, count: u64) { - if let Some(leader_slot_metrics) = &mut self.leader_slot_metrics { - saturating_add_assign!( - leader_slot_metrics - .packet_count_metrics - .total_new_valid_packets, - count - ); - } - } - - pub(crate) fn increment_newly_failed_sigverify_count(&mut self, count: u64) { - if let Some(leader_slot_metrics) = &mut self.leader_slot_metrics { - saturating_add_assign!( - leader_slot_metrics - .packet_count_metrics - .newly_failed_sigverify_count, - count - ); - } - } - - pub(crate) fn increment_exceeded_buffer_limit_dropped_packets_count(&mut self, count: u64) { - if let Some(leader_slot_metrics) = &mut self.leader_slot_metrics { - saturating_add_assign!( - leader_slot_metrics - .packet_count_metrics - .exceeded_buffer_limit_dropped_packets_count, - count - ); - } - } - - pub(crate) fn increment_newly_buffered_packets_count(&mut self, count: u64) { - if let Some(leader_slot_metrics) = &mut self.leader_slot_metrics { - saturating_add_assign!( - leader_slot_metrics - .packet_count_metrics - .newly_buffered_packets_count, - count - ); - } - } - pub(crate) fn increment_retryable_packets_filtered_count(&mut self, count: u64) { if let Some(leader_slot_metrics) = &mut self.leader_slot_metrics { saturating_add_assign!( @@ -701,25 +540,6 @@ impl LeaderSlotMetricsTracker { } } - pub(crate) fn increment_receive_and_buffer_packets_us(&mut self, us: u64) { - if let Some(leader_slot_metrics) = &mut self.leader_slot_metrics { - saturating_add_assign!( - leader_slot_metrics - .timing_metrics - .outer_loop_timings - .receive_and_buffer_packets_us, - us - ); - saturating_add_assign!( - leader_slot_metrics - .timing_metrics - .outer_loop_timings - .receive_and_buffer_packets_invoked_count, - 1 - ); - } - } - // Processing buffer timing metrics pub(crate) fn increment_make_decision_us(&mut self, us: u64) { if let Some(leader_slot_metrics) = &mut self.leader_slot_metrics { @@ -817,26 +637,4 @@ impl LeaderSlotMetricsTracker { ); } } - - pub(crate) fn increment_dropped_gossip_vote_count(&mut self, count: u64) { - if let Some(leader_slot_metrics) = &mut self.leader_slot_metrics { - saturating_add_assign!( - leader_slot_metrics - .vote_packet_count_metrics - .dropped_gossip_votes, - count - ); - } - } - - pub(crate) fn increment_dropped_tpu_vote_count(&mut self, count: u64) { - if let Some(leader_slot_metrics) = &mut self.leader_slot_metrics { - saturating_add_assign!( - leader_slot_metrics - .vote_packet_count_metrics - .dropped_tpu_votes, - count - ); - } - } } diff --git a/core/src/leader_slot_banking_stage_timing_metrics.rs b/core/src/leader_slot_banking_stage_timing_metrics.rs index a0977b4eab4196..9ad26b4819e7fd 100644 --- a/core/src/leader_slot_banking_stage_timing_metrics.rs +++ b/core/src/leader_slot_banking_stage_timing_metrics.rs @@ -133,13 +133,6 @@ pub(crate) struct OuterLoopTimings { // Time spent processing buffered packets pub process_buffered_packets_us: u64, - // Time spent processing new incoming packets to the banking thread - pub receive_and_buffer_packets_us: u64, - - // The number of times the function to receive and buffer new packets - // was called - pub receive_and_buffer_packets_invoked_count: u64, - // Elapsed time between bank was detected and slot end was detected pub bank_detected_to_slot_end_detected_us: u64, } @@ -150,8 +143,6 @@ impl OuterLoopTimings { bank_detected_time: Instant::now(), bank_detected_delay_us: bank_creation_time.elapsed().as_micros() as u64, process_buffered_packets_us: 0, - receive_and_buffer_packets_us: 0, - receive_and_buffer_packets_invoked_count: 0, bank_detected_to_slot_end_detected_us: 0, } } @@ -183,16 +174,6 @@ impl OuterLoopTimings { self.process_buffered_packets_us, i64 ), - ( - "receive_and_buffer_packets_us", - self.receive_and_buffer_packets_us, - i64 - ), - ( - "receive_and_buffer_packets_invoked_count", - self.receive_and_buffer_packets_invoked_count, - i64 - ) ); } } diff --git a/core/src/lib.rs b/core/src/lib.rs index 8c816dee699fda..c692520a11587d 100644 --- a/core/src/lib.rs +++ b/core/src/lib.rs @@ -38,6 +38,7 @@ pub mod leader_slot_banking_stage_timing_metrics; pub mod ledger_cleanup_service; pub mod ledger_metric_report_service; pub mod multi_iterator_scanner; +pub mod next_leader; pub mod optimistic_confirmation_verifier; pub mod outstanding_requests; pub mod packet_deserializer; @@ -59,6 +60,7 @@ mod result; pub mod retransmit_stage; pub mod rewards_recorder_service; pub mod sample_performance_service; +pub mod scheduler_stage; pub mod serve_repair; pub mod serve_repair_service; mod shred_fetch_stage; diff --git a/core/src/multi_iterator_scanner.rs b/core/src/multi_iterator_scanner.rs index 56f5e3d8f25f01..2c201b76d5120d 100644 --- a/core/src/multi_iterator_scanner.rs +++ b/core/src/multi_iterator_scanner.rs @@ -106,9 +106,9 @@ where self.get_current_items() } - /// Consume the iterator and return the payload. - pub fn finalize(self) -> U { - self.payload + /// Consume the iterator and return the payload and whether each element was processed. + pub fn finalize(self) -> (U, Vec) { + (self.payload, self.already_handled) } /// Initialize the `current_positions` vector for the first batch. diff --git a/core/src/next_leader.rs b/core/src/next_leader.rs new file mode 100644 index 00000000000000..19b006bb3017af --- /dev/null +++ b/core/src/next_leader.rs @@ -0,0 +1,48 @@ +use { + solana_gossip::{cluster_info::ClusterInfo, contact_info::ContactInfo}, + solana_poh::poh_recorder::PohRecorder, + solana_sdk::{clock::FORWARD_TRANSACTIONS_TO_LEADER_AT_SLOT_OFFSET, pubkey::Pubkey}, + std::{net::SocketAddr, sync::RwLock}, +}; + +pub(crate) fn next_leader_tpu( + cluster_info: &ClusterInfo, + poh_recorder: &RwLock, +) -> Option<(Pubkey, SocketAddr)> { + next_leader_x(cluster_info, poh_recorder, |leader| leader.tpu) +} + +pub(crate) fn next_leader_tpu_forwards( + cluster_info: &ClusterInfo, + poh_recorder: &RwLock, +) -> Option<(Pubkey, SocketAddr)> { + next_leader_x(cluster_info, poh_recorder, |leader| leader.tpu_forwards) +} + +pub(crate) fn next_leader_tpu_vote( + cluster_info: &ClusterInfo, + poh_recorder: &RwLock, +) -> Option<(Pubkey, SocketAddr)> { + next_leader_x(cluster_info, poh_recorder, |leader| leader.tpu_vote) +} + +fn next_leader_x( + cluster_info: &ClusterInfo, + poh_recorder: &RwLock, + port_selector: F, +) -> Option<(Pubkey, SocketAddr)> +where + F: FnOnce(&ContactInfo) -> SocketAddr, +{ + let leader_pubkey = poh_recorder + .read() + .unwrap() + .leader_after_n_slots(FORWARD_TRANSACTIONS_TO_LEADER_AT_SLOT_OFFSET); + if let Some(leader_pubkey) = leader_pubkey { + cluster_info + .lookup_contact_info(&leader_pubkey, port_selector) + .map(|addr| (leader_pubkey, addr)) + } else { + None + } +} diff --git a/core/src/read_write_account_set.rs b/core/src/read_write_account_set.rs index e2f296e3c6e829..e27c9b424b40ad 100644 --- a/core/src/read_write_account_set.rs +++ b/core/src/read_write_account_set.rs @@ -49,7 +49,7 @@ impl ReadWriteAccountSet { } /// Check if a sanitized message's account locks are available. - fn check_sanitized_message_account_locks(&self, message: &SanitizedMessage) -> bool { + pub fn check_sanitized_message_account_locks(&self, message: &SanitizedMessage) -> bool { !message .account_keys() .iter() @@ -64,7 +64,7 @@ impl ReadWriteAccountSet { } /// Insert the read and write locks for a sanitized message. - fn add_sanitized_message_account_locks(&mut self, message: &SanitizedMessage) { + pub fn add_sanitized_message_account_locks(&mut self, message: &SanitizedMessage) { message .account_keys() .iter() diff --git a/core/src/scheduler_stage.rs b/core/src/scheduler_stage.rs new file mode 100644 index 00000000000000..3860dfc6ebf883 --- /dev/null +++ b/core/src/scheduler_stage.rs @@ -0,0 +1,205 @@ +//! Stage for central transaction scheduler(s). +//! + +use { + crate::{ + banking_stage::{ + decision_maker::{BufferedPacketsDecision, DecisionMaker}, + multi_iterator_scheduler::MultiIteratorScheduler, + thread_aware_account_locks::ThreadId, + BankingStage, + }, + packet_deserializer::{BankingPacketReceiver, PacketDeserializer}, + unprocessed_packet_batches::DeserializedPacket, + }, + crossbeam_channel::{Receiver, Sender}, + solana_gossip::cluster_info::ClusterInfo, + solana_poh::poh_recorder::PohRecorder, + solana_runtime::{bank_forks::BankForks, root_bank_cache::RootBankCache}, + solana_sdk::transaction::SanitizedTransaction, + std::{ + sync::{ + atomic::{AtomicBool, Ordering}, + Arc, RwLock, + }, + thread::JoinHandle, + }, +}; + +pub enum SchedulerKind { + /// Run scheduler's inside of banking stage threads. + ThreadLocalSchedulers, + /// Run central multi-iterator scheduler + MultiIteratorScheduler, +} + +/// Message: Scheduler -> Executor +pub struct ScheduledTransactions { + pub thread_id: ThreadId, + pub decision: BufferedPacketsDecision, + pub packets: Vec, + pub transactions: Vec, + /// Alows scheduler to mark transactions as invalid after they've been sent to the executor + validity_check: Arc, +} + +impl ScheduledTransactions { + pub fn with_capacity( + thread_id: ThreadId, + decision: BufferedPacketsDecision, + capacity: usize, + validity_check: Arc, + ) -> Self { + Self { + thread_id, + decision, + packets: Vec::with_capacity(capacity), + transactions: Vec::with_capacity(capacity), + validity_check, + } + } + + pub fn mark_as_invalid(&self) { + self.validity_check.store(false, Ordering::Relaxed); + } + + pub fn is_valid(&self) -> bool { + self.validity_check.load(Ordering::Relaxed) + } +} + +/// Message: Executor -> Scheduler +#[derive(Default)] +pub struct ProcessedTransactions { + pub thread_id: ThreadId, + pub packets: Vec, + pub transactions: Vec, + pub retryable: Vec, + pub invalidated: bool, +} + +pub type ScheduledTransactionsSender = Sender; +pub type ScheduledTransactionsReceiver = Receiver; +pub type ProcessedTransactionsSender = Sender; +pub type ProcessedTransactionsReceiver = Receiver; + +#[derive(Default)] +pub struct SchedulerStage { + /// Optional scheduler thread handles + /// Empty if scheduler is running in banking stage threads + scheduler_thread_handles: Vec>, +} + +impl SchedulerStage { + pub fn new( + option: SchedulerKind, + packet_receiver: BankingPacketReceiver, + bank_forks: Arc>, + poh_recorder: Arc>, + cluster_info: &ClusterInfo, + ) -> ( + Self, + Option>, + Option, + ) { + Self::new_num_threads( + option, + BankingStage::num_non_vote_threads() as usize, + packet_receiver, + bank_forks, + poh_recorder, + cluster_info, + ) + } + + pub fn new_num_threads( + option: SchedulerKind, + num_non_vote_threads: usize, + packet_receiver: BankingPacketReceiver, + bank_forks: Arc>, + poh_recorder: Arc>, + cluster_info: &ClusterInfo, + ) -> ( + Self, + Option>, + Option, + ) { + match option { + SchedulerKind::ThreadLocalSchedulers => ( + Self { + scheduler_thread_handles: vec![], + }, + None, + None, + ), + SchedulerKind::MultiIteratorScheduler => { + let (transaction_senders, transaction_receivers) = + Self::create_channel_pairs(num_non_vote_threads); + let (processed_transactions_sender, processed_transactions_receiver) = + crossbeam_channel::unbounded(); + + ( + Self { + scheduler_thread_handles: Self::start_multi_iterator_scheduler_thread( + num_non_vote_threads, + packet_receiver, + bank_forks, + poh_recorder, + cluster_info, + transaction_senders, + processed_transactions_receiver, + ), + }, + Some(transaction_receivers), + Some(processed_transactions_sender), + ) + } + } + } + + pub fn join(self) -> std::thread::Result<()> { + for handle in self.scheduler_thread_handles { + handle.join()?; + } + + Ok(()) + } + + fn start_multi_iterator_scheduler_thread( + num_executor_threads: usize, + packet_receiver: BankingPacketReceiver, + bank_forks: Arc>, + poh_recorder: Arc>, + cluster_info: &ClusterInfo, + transaction_senders: Vec, + processed_transactions_receiver: ProcessedTransactionsReceiver, + ) -> Vec> { + let scheduler = MultiIteratorScheduler::new( + num_executor_threads, + DecisionMaker::new(cluster_info.my_contact_info().id, poh_recorder), + PacketDeserializer::new(packet_receiver), + RootBankCache::new(bank_forks), + transaction_senders, + processed_transactions_receiver, + 100_000, + ); + + vec![std::thread::Builder::new() + .name("solCMISched".to_owned()) + .spawn(move || { + scheduler.run(); + }) + .unwrap()] + } + + fn create_channel_pairs(num_executor_threads: usize) -> (Vec>, Vec>) { + let mut transaction_senders = Vec::with_capacity(num_executor_threads); + let mut transaction_receivers = Vec::with_capacity(num_executor_threads); + for _ in 0..num_executor_threads { + let (transaction_sender, transaction_receiver) = crossbeam_channel::unbounded(); + transaction_senders.push(transaction_sender); + transaction_receivers.push(transaction_receiver); + } + (transaction_senders, transaction_receivers) + } +} diff --git a/core/src/tpu.rs b/core/src/tpu.rs index 4c816d263b7744..cdf7e44e0a5168 100644 --- a/core/src/tpu.rs +++ b/core/src/tpu.rs @@ -11,6 +11,7 @@ use { }, fetch_stage::FetchStage, find_packet_sender_stake_stage::FindPacketSenderStakeStage, + scheduler_stage::{SchedulerKind, SchedulerStage}, sigverify::TransactionSigVerifier, sigverify_stage::SigVerifyStage, staked_nodes_updater_service::StakedNodesUpdaterService, @@ -60,6 +61,7 @@ pub struct Tpu { fetch_stage: FetchStage, sigverify_stage: SigVerifyStage, vote_sigverify_stage: SigVerifyStage, + scheduler_stage: SchedulerStage, banking_stage: BankingStage, cluster_info_vote_listener: ClusterInfoVoteListener, broadcast_stage: BroadcastStage, @@ -99,6 +101,7 @@ impl Tpu { staked_nodes: &Arc>, shared_staked_nodes_overrides: Arc>>, tpu_enable_udp: bool, + use_central_scheduler: bool, ) -> Self { let TpuSockets { transactions: transactions_sockets, @@ -222,18 +225,45 @@ impl Tpu { cluster_confirmed_slot_sender, ); - let banking_stage = BankingStage::new( - cluster_info, - poh_recorder, - verified_receiver, - verified_tpu_vote_packets_receiver, - verified_gossip_vote_packets_receiver, - transaction_status_sender, - replay_vote_sender, - log_messages_bytes_limit, - connection_cache.clone(), - bank_forks.clone(), - ); + let (scheduler_stage, banking_stage) = if use_central_scheduler { + let (scheduler_stage, transactions_receivers, processed_transactions_sender) = + SchedulerStage::new( + SchedulerKind::MultiIteratorScheduler, + verified_receiver, + bank_forks.clone(), + poh_recorder.clone(), + cluster_info, + ); + let banking_stage = BankingStage::new_external_scheduler( + cluster_info, + poh_recorder, + transactions_receivers.unwrap(), + processed_transactions_sender.unwrap(), + verified_tpu_vote_packets_receiver, + verified_gossip_vote_packets_receiver, + transaction_status_sender, + replay_vote_sender, + log_messages_bytes_limit, + connection_cache.clone(), + bank_forks.clone(), + ); + + (scheduler_stage, banking_stage) + } else { + let banking_stage = BankingStage::new( + cluster_info, + poh_recorder, + verified_receiver, + verified_tpu_vote_packets_receiver, + verified_gossip_vote_packets_receiver, + transaction_status_sender, + replay_vote_sender, + log_messages_bytes_limit, + connection_cache.clone(), + bank_forks.clone(), + ); + (SchedulerStage::default(), banking_stage) + }; let broadcast_stage = broadcast_type.new_broadcast_stage( broadcast_sockets, @@ -250,6 +280,7 @@ impl Tpu { fetch_stage, sigverify_stage, vote_sigverify_stage, + scheduler_stage, banking_stage, cluster_info_vote_listener, broadcast_stage, @@ -267,6 +298,7 @@ impl Tpu { self.sigverify_stage.join(), self.vote_sigverify_stage.join(), self.cluster_info_vote_listener.join(), + self.scheduler_stage.join(), self.banking_stage.join(), self.find_packet_sender_stake_stage.join(), self.vote_find_packet_sender_stake_stage.join(), diff --git a/core/src/tracer_packet_stats.rs b/core/src/tracer_packet_stats.rs index 03068e4ee663d6..b001621223b652 100644 --- a/core/src/tracer_packet_stats.rs +++ b/core/src/tracer_packet_stats.rs @@ -1,12 +1,10 @@ use { - crate::sigverify::SigverifyTracerPacketStats, solana_sdk::{pubkey::Pubkey, saturating_add_assign, timing::timestamp}, std::collections::HashSet, }; #[derive(Debug, Default)] pub struct BankingStageTracerPacketStats { - total_exceeded_banking_stage_buffer: usize, // This is the total number of tracer packets removed from the buffer // after a leader's set of slots. Of these, only a subset that were in // the buffer were actually forwardable (didn't arrive on forward port and haven't been @@ -18,7 +16,6 @@ pub struct BankingStageTracerPacketStats { #[derive(Debug, Default)] pub struct ModifiableTracerPacketStats { - sigverify_tracer_packet_stats: SigverifyTracerPacketStats, banking_stage_tracer_packet_stats: BankingStageTracerPacketStats, } @@ -44,33 +41,6 @@ impl TracerPacketStats { self.modifiable_tracer_packet_stats.as_mut().unwrap() } - pub fn aggregate_sigverify_tracer_packet_stats( - &mut self, - new_sigverify_stats: &SigverifyTracerPacketStats, - ) { - if !new_sigverify_stats.is_default() { - let stats = self.get_mutable_stats(); - stats - .sigverify_tracer_packet_stats - .aggregate(new_sigverify_stats); - } - } - - pub fn increment_total_exceeded_banking_stage_buffer( - &mut self, - total_exceeded_banking_stage_buffer: usize, - ) { - if total_exceeded_banking_stage_buffer != 0 { - let stats = self.get_mutable_stats(); - saturating_add_assign!( - stats - .banking_stage_tracer_packet_stats - .total_exceeded_banking_stage_buffer, - total_exceeded_banking_stage_buffer - ); - } - } - pub fn increment_total_cleared_from_buffer_after_forward( &mut self, total_cleared_from_buffer_after_forward: usize, @@ -117,49 +87,6 @@ impl TracerPacketStats { datapoint_info!( "tracer-packet-stats", ("id", self.id, i64), - ( - "total_removed_before_sigverify", - modifiable_tracer_packet_stats - .sigverify_tracer_packet_stats - .total_removed_before_sigverify_stage as i64, - i64 - ), - ( - "total_tracer_packets_received_in_sigverify", - modifiable_tracer_packet_stats - .sigverify_tracer_packet_stats - .total_tracer_packets_received_in_sigverify_stage - as i64, - i64 - ), - ( - "total_tracer_packets_deduped_in_sigverify", - modifiable_tracer_packet_stats - .sigverify_tracer_packet_stats - .total_tracer_packets_deduped as i64, - i64 - ), - ( - "total_excess_tracer_packets_discarded_in_sigverify", - modifiable_tracer_packet_stats - .sigverify_tracer_packet_stats - .total_excess_tracer_packets as i64, - i64 - ), - ( - "total_tracker_packets_passed_sigverify", - modifiable_tracer_packet_stats - .sigverify_tracer_packet_stats - .total_tracker_packets_passed_sigverify as i64, - i64 - ), - ( - "total_exceeded_banking_stage_buffer", - modifiable_tracer_packet_stats - .banking_stage_tracer_packet_stats - .total_exceeded_banking_stage_buffer as i64, - i64 - ), ( "total_cleared_from_buffer_after_forward", modifiable_tracer_packet_stats diff --git a/core/src/unprocessed_transaction_storage.rs b/core/src/unprocessed_transaction_storage.rs index 9c3019442f05e6..d8bd534a56c278 100644 --- a/core/src/unprocessed_transaction_storage.rs +++ b/core/src/unprocessed_transaction_storage.rs @@ -481,7 +481,7 @@ impl VoteStorage { } } - scanner.finalize().reached_end_of_slot + scanner.finalize().0.reached_end_of_slot } } @@ -747,7 +747,7 @@ impl ThreadLocalUnprocessedPackets { ) -> Vec { let filter = vec![Ok(()); transactions.len()]; let results = bank.check_transactions_with_forwarding_delay( - transactions, + transactions.iter(), &filter, FORWARD_TRANSACTIONS_TO_LEADER_AT_SLOT_OFFSET, ); @@ -901,7 +901,7 @@ impl ThreadLocalUnprocessedPackets { new_retryable_packets.extend(retryable_packets); } - let reached_end_of_slot = scanner.finalize().reached_end_of_slot; + let reached_end_of_slot = scanner.finalize().0.reached_end_of_slot; self.unprocessed_packet_batches.packet_priority_queue = new_retryable_packets; self.verify_priority_queue(original_capacity); diff --git a/core/src/validator.rs b/core/src/validator.rs index 270ccacfb1489f..29273af0250e32 100644 --- a/core/src/validator.rs +++ b/core/src/validator.rs @@ -174,6 +174,7 @@ pub struct ValidatorConfig { pub ledger_column_options: LedgerColumnOptions, pub runtime_config: RuntimeConfig, pub replay_slots_concurrently: bool, + pub use_central_scheduler: bool, } impl Default for ValidatorConfig { @@ -236,6 +237,7 @@ impl Default for ValidatorConfig { ledger_column_options: LedgerColumnOptions::default(), runtime_config: RuntimeConfig::default(), replay_slots_concurrently: false, + use_central_scheduler: false, } } } @@ -1017,6 +1019,7 @@ impl Validator { &staked_nodes, config.staked_nodes_overrides.clone(), tpu_enable_udp, + config.use_central_scheduler, ); datapoint_info!( diff --git a/core/src/voting_service.rs b/core/src/voting_service.rs index cbd53a1c3bc23b..1ce1950c023583 100644 --- a/core/src/voting_service.rs +++ b/core/src/voting_service.rs @@ -82,9 +82,9 @@ impl VotingService { } let pubkey_and_target_address = if send_to_tpu_vote_port { - crate::banking_stage::next_leader_tpu_vote(cluster_info, poh_recorder) + crate::next_leader::next_leader_tpu_vote(cluster_info, poh_recorder) } else { - crate::banking_stage::next_leader_tpu(cluster_info, poh_recorder) + crate::next_leader::next_leader_tpu(cluster_info, poh_recorder) }; let _ = cluster_info.send_transaction( vote_op.tx(), diff --git a/poh/src/poh_recorder.rs b/poh/src/poh_recorder.rs index fc79623cd341c8..e24f3494a4ac27 100644 --- a/poh/src/poh_recorder.rs +++ b/poh/src/poh_recorder.rs @@ -82,6 +82,10 @@ impl BankStart { self.working_bank.ns_per_slot, ) } + + pub fn reached_max_tick_height(&self) -> bool { + self.working_bank.tick_height() >= self.working_bank.max_tick_height() + } } // Sends the Result of the record operation, including the index in the slot of the first diff --git a/runtime/src/bank.rs b/runtime/src/bank.rs index 85faed9904bbf8..6c7d630ef108e2 100644 --- a/runtime/src/bank.rs +++ b/runtime/src/bank.rs @@ -962,7 +962,7 @@ pub struct Bank { pub status_cache: Arc>, /// FIFO queue of `recent_blockhash` items - blockhash_queue: RwLock, + pub blockhash_queue: RwLock, /// The set of parents including this bank pub ancestors: Ancestors, @@ -3947,25 +3947,40 @@ impl Bank { txs.zip(lock_results) .map(|(tx, lock_res)| match lock_res { - Ok(()) => { - let recent_blockhash = tx.message().recent_blockhash(); - if hash_queue.is_hash_valid_for_age(recent_blockhash, max_age) { - (Ok(()), None) - } else if let Some((address, account)) = - self.check_transaction_for_nonce(tx, &next_durable_nonce) - { - (Ok(()), Some(NoncePartial::new(address, account))) - } else { - error_counters.blockhash_not_found += 1; - (Err(TransactionError::BlockhashNotFound), None) - } - } + Ok(()) => self.check_transaction_age( + tx, + max_age, + &next_durable_nonce, + &hash_queue, + error_counters, + ), Err(e) => (Err(e.clone()), None), }) .collect() } - fn is_transaction_already_processed( + pub fn check_transaction_age( + &self, + tx: &SanitizedTransaction, + max_age: usize, + next_durable_nonce: &DurableNonce, + hash_queue: &BlockhashQueue, + error_counters: &mut TransactionErrorMetrics, + ) -> TransactionCheckResult { + let recent_blockhash = tx.message().recent_blockhash(); + if hash_queue.is_hash_valid_for_age(recent_blockhash, max_age) { + (Ok(()), None) + } else if let Some((address, account)) = + self.check_transaction_for_nonce(tx, next_durable_nonce) + { + (Ok(()), Some(NoncePartial::new(address, account))) + } else { + error_counters.blockhash_not_found += 1; + (Err(TransactionError::BlockhashNotFound), None) + } + } + + pub fn is_transaction_already_processed( &self, sanitized_tx: &SanitizedTransaction, status_cache: &BankStatusCache, @@ -3977,15 +3992,14 @@ impl Bank { .is_some() } - fn check_status_cache( + fn check_status_cache<'a>( &self, - sanitized_txs: &[SanitizedTransaction], + sanitized_txs: impl Iterator, lock_results: Vec, error_counters: &mut TransactionErrorMetrics, ) -> Vec { let rcache = self.status_cache.read().unwrap(); sanitized_txs - .iter() .zip(lock_results) .map(|(sanitized_tx, (lock_result, nonce))| { if lock_result.is_ok() @@ -4040,15 +4054,15 @@ impl Bank { } } - pub fn check_transactions( + pub fn check_transactions<'a>( &self, - sanitized_txs: &[SanitizedTransaction], + sanitized_txs: impl Iterator + Clone, lock_results: &[Result<()>], max_age: usize, error_counters: &mut TransactionErrorMetrics, ) -> Vec { let age_results = - self.check_age(sanitized_txs.iter(), lock_results, max_age, error_counters); + self.check_age(sanitized_txs.clone(), lock_results, max_age, error_counters); self.check_status_cache(sanitized_txs, age_results, error_counters) } @@ -4440,7 +4454,7 @@ impl Bank { let mut check_time = Measure::start("check_transactions"); let check_results = self.check_transactions( - sanitized_txs, + sanitized_txs.iter(), batch.lock_results(), max_age, &mut error_counters, @@ -7742,9 +7756,9 @@ impl Bank { } /// Checks a batch of sanitized transactions again bank for age and status - pub fn check_transactions_with_forwarding_delay( + pub fn check_transactions_with_forwarding_delay<'a>( &self, - transactions: &[SanitizedTransaction], + transactions: impl Iterator + Clone, filter: &[transaction::Result<()>], forward_transactions_to_leader_at_slot_offset: u64, ) -> Vec { diff --git a/validator/src/cli.rs b/validator/src/cli.rs index af919e53745b3a..fa4b56ace71b9b 100644 --- a/validator/src/cli.rs +++ b/validator/src/cli.rs @@ -1304,6 +1304,11 @@ pub fn app<'a>(version: &'a str, default_args: &'a DefaultArgs) -> App<'a, 'a> { .long("replay-slots-concurrently") .help("Allow concurrent replay of slots on different forks") ) + .arg( + Arg::with_name("use_central_scheduler") + .long("use-central-scheduler") + .help("Use centralized scheduler for non-vote banking threads") + ) .args(&get_deprecated_arguments()) .after_help("The default subcommand is run") .subcommand( diff --git a/validator/src/main.rs b/validator/src/main.rs index b1f07fdb81e66b..0ed8b07d5b07a9 100644 --- a/validator/src/main.rs +++ b/validator/src/main.rs @@ -1424,6 +1424,8 @@ pub fn main() { } } + validator_config.use_central_scheduler = matches.is_present("use_central_scheduler"); + let mut ledger_lock = ledger_lockfile(&ledger_path); let _ledger_write_guard = lock_ledger(&ledger_path, &mut ledger_lock);