From b6ec634080882c1b00c2f5a4a3dbabdf1120856f Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Wed, 4 Dec 2024 14:58:52 +0000 Subject: [PATCH 01/25] Wire unified scheduler into banking experimentally --- Cargo.lock | 70 +- Cargo.toml | 3 + banking-bench/Cargo.toml | 2 + banking-bench/src/main.rs | 46 +- core/Cargo.toml | 3 +- core/benches/banking_stage.rs | 1 + core/src/banking_simulation.rs | 48 +- core/src/banking_stage.rs | 100 +- core/src/banking_stage/decision_maker.rs | 21 +- core/src/banking_stage/packet_deserializer.rs | 12 + core/src/banking_trace.rs | 16 + core/src/cluster_info_vote_listener.rs | 2 +- core/src/replay_stage.rs | 28 +- core/src/tpu.rs | 8 +- core/src/validator.rs | 67 +- core/tests/unified_scheduler.rs | 2 +- ledger-tool/Cargo.toml | 1 + ledger-tool/src/ledger_utils.rs | 93 +- ledger-tool/src/main.rs | 34 +- ledger/src/blockstore_processor.rs | 5 +- local-cluster/tests/local_cluster.rs | 39 +- perf/Cargo.toml | 1 + poh/src/poh_recorder.rs | 27 +- programs/sbf/Cargo.lock | 65 +- runtime/Cargo.toml | 2 +- runtime/src/bank_forks.rs | 53 +- runtime/src/installed_scheduler_pool.rs | 95 +- sdk/src/scheduling.rs | 8 + svm/examples/Cargo.lock | 65 +- unified-scheduler-logic/src/lib.rs | 91 +- unified-scheduler-pool/Cargo.toml | 8 +- unified-scheduler-pool/src/lib.rs | 1017 +++++++++++++---- validator/src/cli.rs | 9 + validator/src/main.rs | 11 + 34 files changed, 1697 insertions(+), 356 deletions(-) create mode 100644 sdk/src/scheduling.rs diff --git a/Cargo.lock b/Cargo.lock index 88da797360e920..5c33767345a65b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -214,6 +214,7 @@ dependencies = [ "solana-log-collector", "solana-logger", "solana-measure", + "solana-poh", "solana-program-runtime", "solana-rpc", "solana-runtime", @@ -1608,6 +1609,15 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6245d59a3e82a7fc217c5828a6692dbc6dfb63a0c8c90495621f7b9d79704a0e" +[[package]] +name = "convert_case" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ec182b0ca2f35d8fc196cf3404988fd8b8c739a4d270ff118a398feb0cbec1ca" +dependencies = [ + "unicode-segmentation", +] + [[package]] name = "core-foundation" version = "0.9.4" @@ -1967,13 +1977,35 @@ version = "0.99.16" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "40eebddd2156ce1bb37b20bbe5151340a31828b1f2d22ba4141f3531710e38df" dependencies = [ - "convert_case", + "convert_case 0.4.0", "proc-macro2", "quote", "rustc_version 0.3.3", "syn 1.0.109", ] +[[package]] +name = "derive_more" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4a9b99b9cbbe49445b21764dc0625032a89b145a2642e67603e1c936f5458d05" +dependencies = [ + "derive_more-impl", +] + +[[package]] +name = "derive_more-impl" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cb7330aeadfbe296029522e6c40f315320aba36fc43a5b3632f3795348f3bd22" +dependencies = [ + "convert_case 0.6.0", + "proc-macro2", + "quote", + "syn 2.0.95", + "unicode-xid", +] + [[package]] name = "dialoguer" version = "0.10.4" @@ -2097,6 +2129,12 @@ version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1435fa1053d8b2fbbe9be7e97eca7f33d37b28409959813daefc1446a14247f1" +[[package]] +name = "dyn-clone" +version = "1.0.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0d6ef0072f8a535281e4876be788938b528e9a1d43900b82c2569af7da799125" + [[package]] name = "eager" version = "0.1.0" @@ -3367,7 +3405,7 @@ version = "18.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d2b99d4207e2a04fb4581746903c2bb7eb376f88de9c699d0f3e10feeac0cd3a" dependencies = [ - "derive_more", + "derive_more 0.99.16", "futures 0.3.31", "jsonrpc-core", "jsonrpc-pubsub", @@ -6097,6 +6135,8 @@ dependencies = [ "solana-sdk", "solana-streamer", "solana-tpu-client", + "solana-unified-scheduler-logic", + "solana-unified-scheduler-pool", "solana-version", ] @@ -6870,6 +6910,7 @@ dependencies = [ "chrono", "crossbeam-channel", "dashmap", + "derive_more 1.0.0", "etcd-client", "fs_extra", "futures 0.3.31", @@ -8047,6 +8088,7 @@ dependencies = [ "bincode", "bv", "caps", + "crossbeam-channel", "curve25519-dalek 4.1.3", "dlopen2", "fnv", @@ -10116,13 +10158,15 @@ dependencies = [ name = "solana-unified-scheduler-pool" version = "2.2.0" dependencies = [ + "agave-banking-stage-ingress-types", "assert_matches", "crossbeam-channel", "dashmap", "derive-where", + "derive_more 1.0.0", + "dyn-clone", "lazy_static", "log", - "qualifier_attr", "scopeguard", "solana-clock", "solana-entry", @@ -10134,13 +10178,16 @@ dependencies = [ "solana-pubkey", "solana-runtime", "solana-runtime-transaction", + "solana-sdk", "solana-system-transaction", "solana-timings", "solana-transaction", "solana-transaction-error", "solana-unified-scheduler-logic", + "solana-unified-scheduler-pool", "static_assertions", "test-case", + "trait-set", "vec_extract_if_polyfill", ] @@ -11594,6 +11641,17 @@ dependencies = [ "tracing-core", ] +[[package]] +name = "trait-set" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b79e2e9c9ab44c6d7c20d5976961b47e8f49ac199154daa514b77cd1ab536625" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + [[package]] name = "trees" version = "0.4.2" @@ -11675,6 +11733,12 @@ dependencies = [ "tinyvec", ] +[[package]] +name = "unicode-segmentation" +version = "1.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f6ccf251212114b54433ec949fd6a7841275f9ada20dddd2f29e9ceea4501493" + [[package]] name = "unicode-width" version = "0.1.9" diff --git a/Cargo.toml b/Cargo.toml index b88d38c883be4f..de5c366522dd3e 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -312,11 +312,13 @@ curve25519-dalek = { version = "4.1.3", features = ["digest", "rand_core"] } dashmap = "5.5.3" derivation-path = { version = "0.2.0", default-features = false } derive-where = "1.2.7" +derive_more = { version = "1.0.0", features = ["full"] } dialoguer = "0.10.4" digest = "0.10.7" dir-diff = "0.3.3" dirs-next = "2.0.0" dlopen2 = "0.5.0" +dyn-clone = "1.0.17" eager = "0.1.0" ed25519-dalek = "=1.0.1" ed25519-dalek-bip32 = "0.2.0" @@ -660,6 +662,7 @@ tokio-util = "0.7" toml = "0.8.12" tonic = "0.9.2" tonic-build = "0.9.2" +trait-set = "0.3.0" trees = "0.4.2" tungstenite = "0.20.1" uriparse = "0.6.4" diff --git a/banking-bench/Cargo.toml b/banking-bench/Cargo.toml index a86629552ddf84..4deaa61dc985c7 100644 --- a/banking-bench/Cargo.toml +++ b/banking-bench/Cargo.toml @@ -28,6 +28,8 @@ solana-runtime = { workspace = true, features = ["dev-context-only-utils"] } solana-sdk = { workspace = true } solana-streamer = { workspace = true } solana-tpu-client = { workspace = true } +solana-unified-scheduler-logic = { workspace = true } +solana-unified-scheduler-pool = { workspace = true } solana-version = { workspace = true } [features] diff --git a/banking-bench/src/main.rs b/banking-bench/src/main.rs index 6196f713666e19..5a27ef3709ec4a 100644 --- a/banking-bench/src/main.rs +++ b/banking-bench/src/main.rs @@ -38,6 +38,8 @@ use { }, solana_streamer::socket::SocketAddrSpace, solana_tpu_client::tpu_client::DEFAULT_TPU_CONNECTION_POOL_SIZE, + solana_unified_scheduler_logic::SchedulingMode, + solana_unified_scheduler_pool::{DefaultSchedulerPool, SupportedSchedulingMode}, std::{ sync::{atomic::Ordering, Arc, RwLock}, thread::sleep, @@ -442,6 +444,28 @@ fn main() { BANKING_TRACE_DIR_DEFAULT_BYTE_LIMIT, ))) .unwrap(); + let prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); + let scheduler_pool = if matches!( + block_production_method, + BlockProductionMethod::UnifiedScheduler + ) { + let pool = DefaultSchedulerPool::new( + SupportedSchedulingMode::Either(SchedulingMode::BlockProduction), + None, + None, + None, + Some(replay_vote_sender.clone()), + prioritization_fee_cache.clone(), + poh_recorder.read().unwrap().new_recorder(), + ); + bank_forks + .write() + .unwrap() + .install_scheduler_pool(pool.clone()); + Some(pool) + } else { + None + }; let Channels { non_vote_sender, non_vote_receiver, @@ -449,7 +473,7 @@ fn main() { tpu_vote_receiver, gossip_vote_sender, gossip_vote_receiver, - } = banking_tracer.create_channels(false); + } = banking_tracer.create_channels_for_scheduler_pool(scheduler_pool.as_ref()); let cluster_info = { let keypair = Arc::new(Keypair::new()); let node = Node::new_localhost_with_pubkey(&keypair.pubkey()); @@ -469,7 +493,7 @@ fn main() { ) }; let banking_stage = BankingStage::new_num_threads( - block_production_method, + block_production_method.clone(), &cluster_info, &poh_recorder, non_vote_receiver, @@ -481,10 +505,23 @@ fn main() { None, Arc::new(connection_cache), bank_forks.clone(), - &Arc::new(PrioritizationFeeCache::new(0u64)), + &prioritization_fee_cache, false, + scheduler_pool, ); + // This bench processes transactions, starting from the very first bank, so special-casing is + // needed for unified scheduler. + if matches!( + block_production_method, + BlockProductionMethod::UnifiedScheduler + ) { + bank = bank_forks + .write() + .unwrap() + .reinstall_block_production_scheduler_into_working_genesis_bank(); + } + // This is so that the signal_receiver does not go out of scope after the closure. // If it is dropped before poh_service, then poh_service will error when // calling send() on the channel. @@ -545,10 +582,11 @@ fn main() { tx_total_us += now.elapsed().as_micros() as u64; let mut poh_time = Measure::start("poh_time"); - poh_recorder + let cleared_bank = poh_recorder .write() .unwrap() .reset(bank.clone(), Some((bank.slot(), bank.slot() + 1))); + assert_matches!(cleared_bank, None); poh_time.stop(); let mut new_bank_time = Measure::start("new_bank"); diff --git a/core/Cargo.toml b/core/Cargo.toml index 3425e63fd8fe1b..166dbd252c85da 100644 --- a/core/Cargo.toml +++ b/core/Cargo.toml @@ -26,6 +26,7 @@ bytes = { workspace = true } chrono = { workspace = true, features = ["default", "serde"] } crossbeam-channel = { workspace = true } dashmap = { workspace = true, features = ["rayon", "raw-api"] } +derive_more = { workspace = true } etcd-client = { workspace = true, features = ["tls"] } futures = { workspace = true } histogram = { workspace = true } @@ -92,6 +93,7 @@ solana-tls-utils = { workspace = true } solana-tpu-client = { workspace = true } solana-transaction-status = { workspace = true } solana-turbine = { workspace = true } +solana-unified-scheduler-logic = { workspace = true } solana-unified-scheduler-pool = { workspace = true } solana-version = { workspace = true } solana-vote = { workspace = true } @@ -106,7 +108,6 @@ tokio = { workspace = true, features = ["full"] } trees = { workspace = true } [dev-dependencies] -assert_matches = { workspace = true } fs_extra = { workspace = true } serde_json = { workspace = true } serial_test = { workspace = true } diff --git a/core/benches/banking_stage.rs b/core/benches/banking_stage.rs index dfad8bc8c227cf..37bea4b77f259c 100644 --- a/core/benches/banking_stage.rs +++ b/core/benches/banking_stage.rs @@ -310,6 +310,7 @@ fn bench_banking(bencher: &mut Bencher, tx_type: TransactionType) { bank_forks, &Arc::new(PrioritizationFeeCache::new(0u64)), false, + None, ); let chunk_len = verified.len() / CHUNKS; diff --git a/core/src/banking_simulation.rs b/core/src/banking_simulation.rs index e811c9c6df9bd8..3832963652716c 100644 --- a/core/src/banking_simulation.rs +++ b/core/src/banking_simulation.rs @@ -27,7 +27,7 @@ use { }, solana_net_utils::bind_to_localhost, solana_poh::{ - poh_recorder::{PohRecorder, GRACE_TICKS_FACTOR, MAX_GRACE_SLOTS}, + poh_recorder::{NewPohRecorder, PohRecorder, GRACE_TICKS_FACTOR, MAX_GRACE_SLOTS}, poh_service::{PohService, DEFAULT_HASHES_PER_BATCH, DEFAULT_PINNED_CPU_CORE}, }, solana_runtime::{ @@ -46,6 +46,7 @@ use { }, solana_streamer::socket::SocketAddrSpace, solana_turbine::broadcast_stage::{BroadcastStage, BroadcastStageType}, + solana_unified_scheduler_pool::DefaultSchedulerPool, std::{ collections::BTreeMap, fmt::Display, @@ -679,6 +680,8 @@ impl BankingSimulator { bank_forks: Arc>, blockstore: Arc, block_production_method: BlockProductionMethod, + unified_scheduler_pool: Option>, + new_poh_recorder: Option, ) -> (SenderLoop, SimulatorLoop, SimulatorThreads) { let parent_slot = self.parent_slot().unwrap(); let mut packet_batches_by_time = self.banking_trace_events.packet_batches_by_time; @@ -696,7 +699,10 @@ impl BankingSimulator { simulated_leader, self.first_simulated_slot, ); - let exit = Arc::new(AtomicBool::default()); + let exit = new_poh_recorder + .as_ref() + .map(|(poh_recorder, ..)| poh_recorder.is_exited.clone()) + .unwrap_or_else(|| Arc::new(AtomicBool::default())); if let Some(end_slot) = blockstore .slot_meta_iterator(self.first_simulated_slot) @@ -714,20 +720,23 @@ impl BankingSimulator { info!("Poh is starting!"); - let (poh_recorder, entry_receiver, record_receiver) = PohRecorder::new_with_clear_signal( - bank.tick_height(), - bank.last_blockhash(), - bank.clone(), - None, - bank.ticks_per_slot(), - false, - blockstore.clone(), - blockstore.get_new_shred_signal(0), - &leader_schedule_cache, - &genesis_config.poh_config, - None, - exit.clone(), - ); + let (poh_recorder, entry_receiver, record_receiver) = + new_poh_recorder.unwrap_or_else(|| { + PohRecorder::new_with_clear_signal( + bank.tick_height(), + bank.last_blockhash(), + bank.clone(), + None, + bank.ticks_per_slot(), + false, + blockstore.clone(), + blockstore.get_new_shred_signal(0), + &leader_schedule_cache, + &genesis_config.poh_config, + None, + exit.clone(), + ) + }); let poh_recorder = Arc::new(RwLock::new(poh_recorder)); let poh_service = PohService::new( poh_recorder.clone(), @@ -768,7 +777,7 @@ impl BankingSimulator { tpu_vote_receiver, gossip_vote_sender, gossip_vote_receiver, - } = retracer.create_channels(false); + } = retracer.create_channels_for_scheduler_pool(unified_scheduler_pool.as_ref()); let connection_cache = Arc::new(ConnectionCache::new("connection_cache_sim")); let (replay_vote_sender, _replay_vote_receiver) = unbounded(); @@ -823,6 +832,7 @@ impl BankingSimulator { bank_forks.clone(), prioritization_fee_cache, false, + unified_scheduler_pool, ); let (&_slot, &raw_base_event_time) = freeze_time_by_slot @@ -896,12 +906,16 @@ impl BankingSimulator { bank_forks: Arc>, blockstore: Arc, block_production_method: BlockProductionMethod, + unified_scheduler_pool: Option>, + new_poh_recorder: Option, ) -> Result<(), SimulateError> { let (sender_loop, simulator_loop, simulator_threads) = self.prepare_simulation( genesis_config, bank_forks, blockstore, block_production_method, + unified_scheduler_pool, + new_poh_recorder, ); sender_loop.log_starting(); diff --git a/core/src/banking_stage.rs b/core/src/banking_stage.rs index 845e2981d2fe6d..e617b31d45ba41 100644 --- a/core/src/banking_stage.rs +++ b/core/src/banking_stage.rs @@ -41,6 +41,8 @@ use { vote_sender_types::ReplayVoteSender, }, solana_sdk::{pubkey::Pubkey, timing::AtomicInterval}, + solana_unified_scheduler_logic::SchedulingMode, + solana_unified_scheduler_pool::{BankingStageAdapter, DefaultSchedulerPool}, std::{ cmp, env, ops::Deref, @@ -363,6 +365,7 @@ impl BankingStage { bank_forks: Arc>, prioritization_fee_cache: &Arc, enable_forwarding: bool, + unified_scheduler_pool: Option>, ) -> Self { Self::new_num_threads( block_production_method, @@ -379,6 +382,7 @@ impl BankingStage { bank_forks, prioritization_fee_cache, enable_forwarding, + unified_scheduler_pool, ) } @@ -398,9 +402,12 @@ impl BankingStage { bank_forks: Arc>, prioritization_fee_cache: &Arc, enable_forwarding: bool, + unified_scheduler_pool: Option>, ) -> Self { + use BlockProductionMethod::*; + match block_production_method { - BlockProductionMethod::CentralScheduler => Self::new_central_scheduler( + CentralScheduler => Self::new_central_scheduler( cluster_info, poh_recorder, non_vote_receiver, @@ -415,6 +422,16 @@ impl BankingStage { prioritization_fee_cache, enable_forwarding, ), + UnifiedScheduler => Self::new_unified_scheduler( + cluster_info, + poh_recorder, + non_vote_receiver, + tpu_vote_receiver, + gossip_vote_receiver, + num_threads, + bank_forks, + unified_scheduler_pool.unwrap(), + ), } } @@ -562,6 +579,69 @@ impl BankingStage { Self { bank_thread_hdls } } + pub fn new_unified_scheduler( + cluster_info: &impl LikeClusterInfo, + poh_recorder: &Arc>, + non_vote_receiver: BankingPacketReceiver, + tpu_vote_receiver: BankingPacketReceiver, + gossip_vote_receiver: BankingPacketReceiver, + _num_threads: u32, + bank_forks: Arc>, + unified_scheduler_pool: Arc, + ) -> Self { + assert!(non_vote_receiver.same_channel(&tpu_vote_receiver)); + assert!(non_vote_receiver.same_channel(&gossip_vote_receiver)); + drop((tpu_vote_receiver, gossip_vote_receiver)); + + let unified_receiver = non_vote_receiver; + let decision_maker = DecisionMaker::new(cluster_info.id(), poh_recorder.clone()); + let banking_stage_monitor = Box::new(decision_maker.clone()); + + unified_scheduler_pool.register_banking_stage( + unified_receiver, + 1, /* todo */ + banking_stage_monitor, + Box::new(move |adapter: Arc| { + let decision_maker = decision_maker.clone(); + let bank_forks = bank_forks.clone(); + + Box::new(move |batches, task_submitter| { + let decision = decision_maker.make_consume_or_forward_decision(); + if matches!(decision, BufferedPacketsDecision::Forward) { + return; + } + let bank = bank_forks.read().unwrap().root_bank(); + for batch in batches.iter() { + // over-provision nevertheless some of packets could be invalid. + let task_id_base = adapter.generate_task_ids(batch.len()); + let packets = PacketDeserializer::deserialize_packets_with_indexes(batch); + + for (packet, packet_index) in packets { + let Some((transaction, _deactivation_slot)) = packet + .build_sanitized_transaction( + bank.vote_only_bank(), + &bank, + bank.get_reserved_account_keys(), + ) + else { + continue; + }; + + let index = task_id_base + packet_index; + + let task = adapter.create_new_task(transaction, index); + task_submitter(task); + } + } + }) + }), + ); + + Self { + bank_thread_hdls: vec![], + } + } + fn spawn_thread_local_multi_iterator_thread( id: u32, packet_receiver: BankingPacketReceiver, @@ -725,11 +805,18 @@ pub(crate) fn update_bank_forks_and_poh_recorder_for_new_tpu_bank( tpu_bank: Bank, track_transaction_indexes: bool, ) { - let tpu_bank = bank_forks.write().unwrap().insert(tpu_bank); - poh_recorder + // A write lock for the poh recorder must be grabbed for the entire duration of inserting new + // tpu bank into the bank forks. That's because any buffered transactions could immediately be + // executed after the bank forks update, when unified scheduler is enabled for block + // production. And then, the unified scheduler would be hit with false errors due to having no + // bank in the poh recorder otherwise. + let mut poh_recorder = poh_recorder.write().unwrap(); + + let tpu_bank = bank_forks .write() .unwrap() - .set_bank(tpu_bank, track_transaction_indexes); + .insert_with_scheduling_mode(SchedulingMode::BlockProduction, tpu_bank); + poh_recorder.set_bank(tpu_bank, track_transaction_indexes); } #[cfg(test)] @@ -832,6 +919,7 @@ mod tests { bank_forks, &Arc::new(PrioritizationFeeCache::new(0u64)), false, + None, ); drop(non_vote_sender); drop(tpu_vote_sender); @@ -892,6 +980,7 @@ mod tests { bank_forks, &Arc::new(PrioritizationFeeCache::new(0u64)), false, + None, ); trace!("sending bank"); drop(non_vote_sender); @@ -976,6 +1065,7 @@ mod tests { bank_forks.clone(), // keep a local-copy of bank-forks so worker threads do not lose weak access to bank-forks &Arc::new(PrioritizationFeeCache::new(0u64)), false, + None, ); // fund another account so we can send 2 good transactions in a single batch. @@ -1146,6 +1236,7 @@ mod tests { bank_forks, &Arc::new(PrioritizationFeeCache::new(0u64)), false, + None, ); // wait for banking_stage to eat the packets @@ -1342,6 +1433,7 @@ mod tests { bank_forks, &Arc::new(PrioritizationFeeCache::new(0u64)), false, + None, ); let keypairs = (0..100).map(|_| Keypair::new()).collect_vec(); diff --git a/core/src/banking_stage/decision_maker.rs b/core/src/banking_stage/decision_maker.rs index 17f2a6cc982a26..95a8721e93c596 100644 --- a/core/src/banking_stage/decision_maker.rs +++ b/core/src/banking_stage/decision_maker.rs @@ -7,7 +7,8 @@ use { }, pubkey::Pubkey, }, - std::sync::{Arc, RwLock}, + solana_unified_scheduler_pool::{BankingStageMonitor, BankingStageStatus}, + std::sync::{atomic::Ordering::Relaxed, Arc, RwLock}, }; #[derive(Debug, Clone)] @@ -28,9 +29,10 @@ impl BufferedPacketsDecision { } } -#[derive(Clone)] +#[derive(Clone, derive_more::Debug)] pub struct DecisionMaker { my_pubkey: Pubkey, + #[debug("{poh_recorder:p}")] poh_recorder: Arc>, } @@ -112,6 +114,21 @@ impl DecisionMaker { } } +impl BankingStageMonitor for DecisionMaker { + fn status(&self) -> BankingStageStatus { + if self.poh_recorder.read().unwrap().is_exited.load(Relaxed) { + BankingStageStatus::Exited + } else if matches!( + self.make_consume_or_forward_decision(), + BufferedPacketsDecision::Forward, + ) { + BankingStageStatus::Inactive + } else { + BankingStageStatus::Active + } + } +} + #[cfg(test)] mod tests { use { diff --git a/core/src/banking_stage/packet_deserializer.rs b/core/src/banking_stage/packet_deserializer.rs index 3c1a56b43b01de..3e5688c0513e60 100644 --- a/core/src/banking_stage/packet_deserializer.rs +++ b/core/src/banking_stage/packet_deserializer.rs @@ -198,6 +198,18 @@ impl PacketDeserializer { } }) } + + pub(crate) fn deserialize_packets_with_indexes( + packet_batch: &PacketBatch, + ) -> impl Iterator + '_ { + let packet_indexes = PacketDeserializer::generate_packet_indexes(packet_batch); + packet_indexes.into_iter().filter_map(move |packet_index| { + let packet = packet_batch[packet_index].clone(); + ImmutableDeserializedPacket::new(packet) + .ok() + .map(|packet| (packet, packet_index)) + }) + } } #[cfg(test)] diff --git a/core/src/banking_trace.rs b/core/src/banking_trace.rs index 0c9fb856aecb87..ee47d5827cc972 100644 --- a/core/src/banking_trace.rs +++ b/core/src/banking_trace.rs @@ -5,6 +5,7 @@ use { crossbeam_channel::{unbounded, Receiver, SendError, Sender, TryRecvError}, rolling_file::{RollingCondition, RollingConditionBasic, RollingFileAppender}, solana_sdk::{hash::Hash, slot_history::Slot}, + solana_unified_scheduler_pool::DefaultSchedulerPool, std::{ fs::{create_dir_all, remove_dir_all}, io::{self, Write}, @@ -59,6 +60,11 @@ pub struct BankingTracer { active_tracer: Option, } +#[cfg_attr( + feature = "frozen-abi", + derive(AbiExample), + frozen_abi(digest = "DAdZnX6ijBWaxKAyksq4nJa6PAZqT4RShZqLWTtNvyAM") +)] #[derive(Serialize, Deserialize, Debug)] pub struct TimedTracedEvent(pub std::time::SystemTime, pub TracedEvent); @@ -258,6 +264,16 @@ impl BankingTracer { } } + pub fn create_channels_for_scheduler_pool( + &self, + pool: Option<&Arc>, + ) -> Channels { + self.create_channels( + pool.map(|pool| pool.block_production_supported()) + .unwrap_or_default(), + ) + } + fn create_channel(&self, label: ChannelLabel) -> (BankingPacketSender, BankingPacketReceiver) { Self::channel(label, self.active_tracer.as_ref().cloned()) } diff --git a/core/src/cluster_info_vote_listener.rs b/core/src/cluster_info_vote_listener.rs index 1f82c765741cb7..fa5c813086f294 100644 --- a/core/src/cluster_info_vote_listener.rs +++ b/core/src/cluster_info_vote_listener.rs @@ -195,6 +195,7 @@ impl ClusterInfoVoteListener { verified_packets_sender: BankingPacketSender, vote_tracker: Arc, bank_forks: Arc>, + mut root_bank_cache: RootBankCache, subscriptions: Arc, verified_vote_sender: VerifiedVoteSender, gossip_verified_vote_hash_sender: GossipVerifiedVoteHashSender, @@ -206,7 +207,6 @@ impl ClusterInfoVoteListener { let (verified_vote_transactions_sender, verified_vote_transactions_receiver) = unbounded(); let listen_thread = { let exit = exit.clone(); - let mut root_bank_cache = RootBankCache::new(bank_forks.clone()); Builder::new() .name("solCiVoteLstnr".to_string()) .spawn(move || { diff --git a/core/src/replay_stage.rs b/core/src/replay_stage.rs index 2b40bafb4bc3ec..cbe19d19c7c537 100644 --- a/core/src/replay_stage.rs +++ b/core/src/replay_stage.rs @@ -76,6 +76,7 @@ use { transaction::Transaction, }, solana_timings::ExecuteTimings, + solana_unified_scheduler_logic::SchedulingMode, solana_vote_program::vote_state::{VoteState, VoteTransaction}, std::{ collections::{HashMap, HashSet}, @@ -2803,6 +2804,28 @@ impl ReplayStage { } } + fn wait_for_cleared_bank(bank: BankWithScheduler) { + if matches!( + bank.scheduling_mode(), + Some(SchedulingMode::BlockProduction) + ) { + info!("Reaping cleared tpu_bank: {}...", bank.slot()); + if let Some((result, _completed_execute_timings)) = bank.wait_for_completed_scheduler() + { + info!( + "Reaped aborted tpu_bank with unified scheduler: {} {:?}", + bank.slot(), + result + ); + } else { + info!( + "Skipped to reap a tpu_bank (seems unified scheduler is disabled): {}", + bank.slot() + ); + } + } + } + fn reset_poh_recorder( my_pubkey: &Pubkey, blockstore: &Blockstore, @@ -2821,7 +2844,10 @@ impl ReplayStage { GRACE_TICKS_FACTOR * MAX_GRACE_SLOTS, ); - poh_recorder.write().unwrap().reset(bank, next_leader_slot); + let cleared_bank = poh_recorder.write().unwrap().reset(bank, next_leader_slot); + if let Some(cleared_bank) = cleared_bank { + Self::wait_for_cleared_bank(cleared_bank); + } let next_leader_msg = if let Some(next_leader_slot) = next_leader_slot { format!("My next leader slot is {}", next_leader_slot.0) diff --git a/core/src/tpu.rs b/core/src/tpu.rs index d715bb5c7b0534..d2b1a042a3b600 100644 --- a/core/src/tpu.rs +++ b/core/src/tpu.rs @@ -33,6 +33,7 @@ use { solana_runtime::{ bank_forks::BankForks, prioritization_fee_cache::PrioritizationFeeCache, + root_bank_cache::RootBankCache, vote_sender_types::{ReplayVoteReceiver, ReplayVoteSender}, }, solana_sdk::{clock::Slot, pubkey::Pubkey, quic::NotifyKeyUpdate, signature::Keypair}, @@ -44,6 +45,7 @@ use { streamer::StakedNodes, }, solana_turbine::broadcast_stage::{BroadcastStage, BroadcastStageType}, + solana_unified_scheduler_pool::DefaultSchedulerPool, std::{ collections::HashMap, net::{SocketAddr, UdpSocket}, @@ -99,6 +101,7 @@ impl Tpu { shred_version: u16, vote_tracker: Arc, bank_forks: Arc>, + root_bank_cache: RootBankCache, verified_vote_sender: VerifiedVoteSender, gossip_verified_vote_hash_sender: GossipVerifiedVoteHashSender, replay_vote_receiver: ReplayVoteReceiver, @@ -120,6 +123,7 @@ impl Tpu { block_production_method: BlockProductionMethod, enable_block_production_forwarding: bool, _generator_config: Option, /* vestigial code for replay invalidator */ + unified_scheduler_pool: Option>, ) -> (Self, Vec>) { let TpuSockets { transactions: transactions_sockets, @@ -163,7 +167,7 @@ impl Tpu { tpu_vote_receiver, gossip_vote_sender, gossip_vote_receiver, - } = banking_tracer.create_channels(false); + } = banking_tracer.create_channels_for_scheduler_pool(unified_scheduler_pool.as_ref()); // Streamer for Votes: let SpawnServerResult { @@ -258,6 +262,7 @@ impl Tpu { gossip_vote_sender, vote_tracker, bank_forks.clone(), + root_bank_cache, subscriptions.clone(), verified_vote_sender, gossip_verified_vote_hash_sender, @@ -281,6 +286,7 @@ impl Tpu { bank_forks.clone(), prioritization_fee_cache, enable_block_production_forwarding, + unified_scheduler_pool, ); let (entry_receiver, tpu_entry_notifier) = diff --git a/core/src/validator.rs b/core/src/validator.rs index 802dcc5fefdc2b..2bc08a29a23d48 100644 --- a/core/src/validator.rs +++ b/core/src/validator.rs @@ -104,6 +104,7 @@ use { bank_forks::BankForks, commitment::BlockCommitmentCache, prioritization_fee_cache::PrioritizationFeeCache, + root_bank_cache::RootBankCache, runtime_config::RuntimeConfig, snapshot_archive_info::SnapshotArchiveInfoGetter, snapshot_bank_utils::{self, DISABLED_SNAPSHOT_ARCHIVE_INTERVAL}, @@ -126,7 +127,8 @@ use { solana_send_transaction_service::send_transaction_service, solana_streamer::{socket::SocketAddrSpace, streamer::StakedNodes}, solana_turbine::{self, broadcast_stage::BroadcastStageType}, - solana_unified_scheduler_pool::DefaultSchedulerPool, + solana_unified_scheduler_logic::SchedulingMode, + solana_unified_scheduler_pool::{DefaultSchedulerPool, SupportedSchedulingMode}, solana_vote_program::vote_state, solana_wen_restart::wen_restart::{wait_for_wen_restart, WenRestartConfig}, std::{ @@ -183,11 +185,14 @@ impl BlockVerificationMethod { } } -#[derive(Clone, EnumString, EnumVariantNames, Default, IntoStaticStr, Display)] +#[derive( + Clone, EnumCount, EnumIter, EnumString, EnumVariantNames, Default, IntoStaticStr, Display, +)] #[strum(serialize_all = "kebab-case")] pub enum BlockProductionMethod { - #[default] CentralScheduler, + #[default] + UnifiedScheduler, } impl BlockProductionMethod { @@ -207,6 +212,23 @@ impl BlockProductionMethod { } } +pub fn supported_scheduling_mode( + (verification, production): (&BlockVerificationMethod, &BlockProductionMethod), +) -> SupportedSchedulingMode { + match (verification, production) { + (BlockVerificationMethod::UnifiedScheduler, BlockProductionMethod::UnifiedScheduler) => { + SupportedSchedulingMode::Both + } + (BlockVerificationMethod::UnifiedScheduler, _) => { + SupportedSchedulingMode::Either(SchedulingMode::BlockVerification) + } + (_, BlockProductionMethod::UnifiedScheduler) => { + SupportedSchedulingMode::Either(SchedulingMode::BlockProduction) + } + _ => unreachable!("seems unified scheduler is disabled"), + } +} + /// Configuration for the block generator invalidator for replay. #[derive(Clone, Debug)] pub struct GeneratorConfig { @@ -901,30 +923,38 @@ impl Validator { }; let poh_recorder = Arc::new(RwLock::new(poh_recorder)); - match &config.block_verification_method { - BlockVerificationMethod::BlockstoreProcessor => { - info!("no scheduler pool is installed for block verification..."); - if let Some(count) = config.unified_scheduler_handler_threads { - warn!( - "--unified-scheduler-handler-threads={count} is ignored because unified \ - scheduler isn't enabled" - ); - } - } - BlockVerificationMethod::UnifiedScheduler => { - let scheduler_pool = DefaultSchedulerPool::new_dyn( + let unified_scheduler_pool = match ( + &config.block_verification_method, + &config.block_production_method, + ) { + methods @ (BlockVerificationMethod::UnifiedScheduler, _) + | methods @ (_, BlockProductionMethod::UnifiedScheduler) => { + let pool = DefaultSchedulerPool::new( + supported_scheduling_mode(methods), config.unified_scheduler_handler_threads, config.runtime_config.log_messages_bytes_limit, transaction_status_sender.clone(), Some(replay_vote_sender.clone()), prioritization_fee_cache.clone(), + poh_recorder.read().unwrap().new_recorder(), ); bank_forks .write() .unwrap() - .install_scheduler_pool(scheduler_pool); + .install_scheduler_pool(pool.clone()); + Some(pool) } - } + _ => { + info!("no scheduler pool is installed for block verification/production..."); + if let Some(count) = config.unified_scheduler_handler_threads { + warn!( + "--unified-scheduler-handler-threads={count} is ignored because unified \ + scheduler isn't enabled" + ); + } + None + } + }; let entry_notification_sender = entry_notifier_service .as_ref() @@ -1398,6 +1428,7 @@ impl Validator { let cluster_slots = Arc::new(crate::cluster_slots_service::cluster_slots::ClusterSlots::default()); + let root_bank_cache = RootBankCache::new(bank_forks.clone()); let tvu = Tvu::new( vote_account, authorized_voter_keypairs, @@ -1505,6 +1536,7 @@ impl Validator { node.info.shred_version(), vote_tracker, bank_forks.clone(), + root_bank_cache, verified_vote_sender, gossip_verified_vote_hash_sender, replay_vote_receiver, @@ -1526,6 +1558,7 @@ impl Validator { config.block_production_method.clone(), config.enable_block_production_forwarding, config.generator_config.clone(), + unified_scheduler_pool, ); datapoint_info!( diff --git a/core/tests/unified_scheduler.rs b/core/tests/unified_scheduler.rs index c2f24923aba23e..ecd41dde68608a 100644 --- a/core/tests/unified_scheduler.rs +++ b/core/tests/unified_scheduler.rs @@ -69,7 +69,7 @@ fn test_scheduler_waited_by_drop_bank_service() { let genesis_bank = Bank::new_for_tests(&genesis_config); let bank_forks = BankForks::new_rw_arc(genesis_bank); let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); - let pool_raw = SchedulerPool::, _>::new( + let pool_raw = SchedulerPool::, _>::new_for_verification( None, None, None, diff --git a/ledger-tool/Cargo.toml b/ledger-tool/Cargo.toml index 5a857c1ecb55f1..32a79c20627588 100644 --- a/ledger-tool/Cargo.toml +++ b/ledger-tool/Cargo.toml @@ -43,6 +43,7 @@ solana-ledger = { workspace = true, features = ["dev-context-only-utils"] } solana-log-collector = { workspace = true } solana-logger = { workspace = true } solana-measure = { workspace = true } +solana-poh = { workspace = true } solana-program-runtime = { workspace = true, features = ["metrics"] } solana-rpc = { workspace = true } solana-runtime = { workspace = true, features = ["dev-context-only-utils"] } diff --git a/ledger-tool/src/ledger_utils.rs b/ledger-tool/src/ledger_utils.rs index 6382b35bfc88c0..43e63e690b2005 100644 --- a/ledger-tool/src/ledger_utils.rs +++ b/ledger-tool/src/ledger_utils.rs @@ -9,7 +9,8 @@ use { }, solana_core::{ accounts_hash_verifier::AccountsHashVerifier, - snapshot_packager_service::PendingSnapshotPackages, validator::BlockVerificationMethod, + snapshot_packager_service::PendingSnapshotPackages, + validator::{supported_scheduling_mode, BlockProductionMethod, BlockVerificationMethod}, }, solana_geyser_plugin_manager::geyser_plugin_service::{ GeyserPluginService, GeyserPluginServiceError, @@ -24,6 +25,7 @@ use { use_snapshot_archives_at_startup::UseSnapshotArchivesAtStartup, }, solana_measure::measure_time, + solana_poh::poh_recorder::{NewPohRecorder, PohRecorder}, solana_rpc::{ cache_block_meta_service::CacheBlockMetaService, transaction_status_service::TransactionStatusService, @@ -66,6 +68,8 @@ pub struct LoadAndProcessLedgerOutput { // not. It is safe to let ABS continue in the background, and ABS will stop // if/when it finally checks the exit flag pub accounts_background_service: AccountsBackgroundService, + pub unified_scheduler_pool: Option>, + pub new_poh_recorder: Option, } const PROCESS_SLOTS_HELP_STRING: &str = @@ -352,43 +356,86 @@ pub fn load_and_process_ledger( exit.clone(), ) .map_err(LoadAndProcessLedgerError::LoadBankForks)?; + let leader_schedule_cache = Arc::new(leader_schedule_cache); let block_verification_method = value_t!( arg_matches, "block_verification_method", BlockVerificationMethod ) .unwrap_or_default(); + let block_production_method = value_t!( + arg_matches, + "block_production_method", + BlockProductionMethod + ) + .inspect(|method| { + if matches!(method, BlockProductionMethod::UnifiedScheduler) + && !arg_matches.is_present("enable_experimental_block_production_method") + { + error!( + "Currently, the unified-scheduler method is experimental for block-production. \ + Explicitly pass --enable-experimental-block-production-method to supress this error" + ); + } + }) + .unwrap_or_default(); info!( - "Using: block-verification-method: {}", - block_verification_method, + "Using: block-verification-method: {}, block-production-method: {}", + block_verification_method, block_production_method ); let unified_scheduler_handler_threads = value_t!(arg_matches, "unified_scheduler_handler_threads", usize).ok(); - match block_verification_method { - BlockVerificationMethod::BlockstoreProcessor => { - info!("no scheduler pool is installed for block verification..."); - if let Some(count) = unified_scheduler_handler_threads { - warn!( - "--unified-scheduler-handler-threads={count} is ignored because unified \ - scheduler isn't enabled" + let (unified_scheduler_pool, new_poh_recorder) = + match (&block_verification_method, &block_production_method) { + methods @ (BlockVerificationMethod::UnifiedScheduler, _) + | methods @ (_, BlockProductionMethod::UnifiedScheduler) => { + let no_replay_vote_sender = None; + let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); + + let exit = Arc::new(AtomicBool::new(false)); + let poh_bank = bank_forks.read().unwrap().working_bank(); + let new_poh_recorder = PohRecorder::new_with_clear_signal( + poh_bank.tick_height(), + poh_bank.last_blockhash(), + poh_bank.clone(), + None, + poh_bank.ticks_per_slot(), + false, + blockstore.clone(), + blockstore.get_new_shred_signal(0), + &leader_schedule_cache, + &genesis_config.poh_config, + None, + exit.clone(), ); - } - } - BlockVerificationMethod::UnifiedScheduler => { - let no_replay_vote_sender = None; - let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); - bank_forks - .write() - .unwrap() - .install_scheduler_pool(DefaultSchedulerPool::new_dyn( + drop(poh_bank); + + let pool = DefaultSchedulerPool::new( + supported_scheduling_mode(methods), unified_scheduler_handler_threads, process_options.runtime_config.log_messages_bytes_limit, transaction_status_sender.clone(), no_replay_vote_sender, ignored_prioritization_fee_cache, - )); - } - } + new_poh_recorder.0.new_recorder(), + ); + bank_forks + .write() + .unwrap() + .install_scheduler_pool(pool.clone()); + (Some(pool), Some(new_poh_recorder)) + } + _ => { + info!("no scheduler pool is installed for block verification/production..."); + if let Some(count) = unified_scheduler_handler_threads { + warn!( + "--unified-scheduler-handler-threads={count} is ignored because unified \ + scheduler isn't enabled" + ); + } + (None, None) + } + }; let pending_snapshot_packages = Arc::new(Mutex::new(PendingSnapshotPackages::default())); let (accounts_package_sender, accounts_package_receiver) = crossbeam_channel::unbounded(); @@ -437,6 +484,8 @@ pub fn load_and_process_ledger( bank_forks, starting_snapshot_hashes, accounts_background_service, + unified_scheduler_pool, + new_poh_recorder, }) .map_err(LoadAndProcessLedgerError::ProcessBlockstoreFromRoot); diff --git a/ledger-tool/src/main.rs b/ledger-tool/src/main.rs index 8e0cdc5a019895..ace1c6fc76cfdd 100644 --- a/ledger-tool/src/main.rs +++ b/ledger-tool/src/main.rs @@ -991,6 +991,15 @@ fn main() { .global(true) .help(DefaultSchedulerPool::cli_message()), ) + .arg( + Arg::with_name("enable_experimental_block_production_method") + .long("enable-experimental-block-production-method") + .takes_value(false) + .help( + "Accept unified-scheduler to be used as an experimental block \ + production method", + ), + ) .arg( Arg::with_name("output_format") .long("output") @@ -2045,6 +2054,7 @@ fn main() { bank_forks, starting_snapshot_hashes, accounts_background_service, + .. } = load_and_process_ledger_or_exit( arg_matches, &genesis_config, @@ -2477,14 +2487,18 @@ fn main() { AccessType::Primary, // needed for purging already existing simulated block shreds... )); let genesis_config = open_genesis_config_by(&ledger_path, arg_matches); - let LoadAndProcessLedgerOutput { bank_forks, .. } = - load_and_process_ledger_or_exit( - arg_matches, - &genesis_config, - blockstore.clone(), - process_options, - None, // transaction status sender - ); + let LoadAndProcessLedgerOutput { + bank_forks, + unified_scheduler_pool, + new_poh_recorder, + .. + } = load_and_process_ledger_or_exit( + arg_matches, + &genesis_config, + blockstore.clone(), + process_options, + None, // transaction status sender + ); let block_production_method = value_t!( arg_matches, @@ -2493,13 +2507,13 @@ fn main() { ) .unwrap_or_default(); - info!("Using: block-production-method: {block_production_method}"); - match simulator.start( genesis_config, bank_forks, blockstore, block_production_method, + unified_scheduler_pool, + new_poh_recorder, ) { Ok(()) => println!("Ok"), Err(error) => { diff --git a/ledger/src/blockstore_processor.rs b/ledger/src/blockstore_processor.rs index 6f0f35d8f4bb7a..0dd0222d7e15bc 100644 --- a/ledger/src/blockstore_processor.rs +++ b/ledger/src/blockstore_processor.rs @@ -5034,16 +5034,17 @@ pub mod tests { .. } = create_genesis_config_with_leader(500, &dummy_leader_pubkey, 100); let bank = Arc::new(Bank::new_for_tests(&genesis_config)); - let context = SchedulingContext::new(bank.clone()); + let context = SchedulingContext::for_verification(bank.clone()); let txs = create_test_transactions(&mint_keypair, &genesis_config.hash()); let mut mocked_scheduler = MockInstalledScheduler::new(); let seq = Arc::new(Mutex::new(mockall::Sequence::new())); let seq_cloned = seq.clone(); + // Used for assertions in BankWithScheduler::{new, schedule_transaction_executions} mocked_scheduler .expect_context() - .times(1) + .times(2) .in_sequence(&mut seq.lock().unwrap()) .return_const(context); if should_succeed { diff --git a/local-cluster/tests/local_cluster.rs b/local-cluster/tests/local_cluster.rs index e3d2099e8ffc2c..8258b7cb980b65 100644 --- a/local-cluster/tests/local_cluster.rs +++ b/local-cluster/tests/local_cluster.rs @@ -16,7 +16,7 @@ use { }, optimistic_confirmation_verifier::OptimisticConfirmationVerifier, replay_stage::DUPLICATE_THRESHOLD, - validator::{BlockVerificationMethod, ValidatorConfig}, + validator::{BlockProductionMethod, BlockVerificationMethod, ValidatorConfig}, }, solana_download_utils::download_snapshot_archive, solana_entry::entry::create_ticks, @@ -5866,6 +5866,43 @@ fn test_randomly_mixed_block_verification_methods_between_bootstrap_and_not() { ); } +#[test] +#[serial] +fn test_randomly_mixed_block_production_methods_between_bootstrap_and_not() { + // tailored logging just to see two block production methods are working correctly + solana_logger::setup_with_default( + "solana_metrics::metrics=warn,\ + solana_core=warn,\ + solana_runtime::installed_scheduler_pool=trace,\ + solana_ledger::blockstore_processor=debug,\ + info", + ); + + let num_nodes = BlockVerificationMethod::COUNT; + let mut config = ClusterConfig::new_with_equal_stakes( + num_nodes, + DEFAULT_CLUSTER_LAMPORTS, + DEFAULT_NODE_STAKE, + ); + + // Overwrite block_production_method with shuffled variants + let mut methods = BlockProductionMethod::iter().collect::>(); + methods.shuffle(&mut rand::thread_rng()); + for (validator_config, method) in config.validator_configs.iter_mut().zip_eq(methods) { + validator_config.block_production_method = method; + } + + let local = LocalCluster::new(&mut config, SocketAddrSpace::Unspecified); + cluster_tests::spend_and_verify_all_nodes( + &local.entry_point_info, + &local.funding_keypair, + num_nodes, + HashSet::new(), + SocketAddrSpace::Unspecified, + &local.connection_cache, + ); +} + /// Forks previous marked invalid should be marked as such in fork choice on restart #[test] #[ignore] diff --git a/perf/Cargo.toml b/perf/Cargo.toml index bf41c12bb99507..cce3287cf01e80 100644 --- a/perf/Cargo.toml +++ b/perf/Cargo.toml @@ -13,6 +13,7 @@ edition = { workspace = true } ahash = { workspace = true } bincode = { workspace = true } bv = { workspace = true, features = ["serde"] } +crossbeam-channel = { workspace = true } curve25519-dalek = { workspace = true } dlopen2 = { workspace = true } fnv = { workspace = true } diff --git a/poh/src/poh_recorder.rs b/poh/src/poh_recorder.rs index 5a16cc791b0d42..45739d7a216648 100644 --- a/poh/src/poh_recorder.rs +++ b/poh/src/poh_recorder.rs @@ -150,6 +150,13 @@ impl TransactionRecorder { } } + pub fn new_dummy() -> Self { + Self { + record_sender: crossbeam_channel::unbounded().0, + is_exited: Arc::new(AtomicBool::default()), + } + } + /// Hashes `transactions` and sends to PoH service for recording. Waits for response up to 1s. /// Panics on unexpected (non-`MaxHeightReached`) errors. pub fn record_transactions( @@ -308,8 +315,11 @@ pub struct PohRecorder { pub is_exited: Arc, } +pub type NewPohRecorder = (PohRecorder, Receiver, Receiver); + impl PohRecorder { - fn clear_bank(&mut self) { + fn clear_bank(&mut self) -> Option { + let mut cleared_bank = None; if let Some(WorkingBank { bank, start, .. }) = self.working_bank.take() { self.leader_bank_notifier.set_completed(bank.slot()); let next_leader_slot = self.leader_schedule_cache.next_leader_slot( @@ -335,6 +345,7 @@ impl PohRecorder { ("slot", bank.slot(), i64), ("elapsed", start.elapsed().as_millis(), i64), ); + cleared_bank = Some(bank); } if let Some(ref signal) = self.clear_bank_signal { @@ -348,6 +359,7 @@ impl PohRecorder { } } } + cleared_bank } pub fn would_be_leader(&self, within_next_n_ticks: u64) -> bool { @@ -656,8 +668,12 @@ impl PohRecorder { } // synchronize PoH with a bank - pub fn reset(&mut self, reset_bank: Arc, next_leader_slot: Option<(Slot, Slot)>) { - self.clear_bank(); + pub fn reset( + &mut self, + reset_bank: Arc, + next_leader_slot: Option<(Slot, Slot)>, + ) -> Option { + let cleared_bank = self.clear_bank(); self.reset_poh(reset_bank, true); if let Some(ref sender) = self.poh_timing_point_sender { @@ -678,6 +694,7 @@ impl PohRecorder { self.leader_first_tick_height_including_grace_ticks = leader_first_tick_height_including_grace_ticks; self.leader_last_tick_height = leader_last_tick_height; + cleared_bank } pub fn set_bank(&mut self, bank: BankWithScheduler, track_transaction_indexes: bool) { @@ -1023,7 +1040,7 @@ impl PohRecorder { poh_config: &PohConfig, poh_timing_point_sender: Option, is_exited: Arc, - ) -> (Self, Receiver, Receiver) { + ) -> NewPohRecorder { let tick_number = 0; let poh = Arc::new(Mutex::new(Poh::new_with_slot_info( last_entry_hash, @@ -1093,7 +1110,7 @@ impl PohRecorder { leader_schedule_cache: &Arc, poh_config: &PohConfig, is_exited: Arc, - ) -> (Self, Receiver, Receiver) { + ) -> NewPohRecorder { let delay_leader_block_for_pending_fork = false; Self::new_with_clear_signal( tick_height, diff --git a/programs/sbf/Cargo.lock b/programs/sbf/Cargo.lock index fd4cd2c2049ed5..96768f54395ea7 100644 --- a/programs/sbf/Cargo.lock +++ b/programs/sbf/Cargo.lock @@ -1124,6 +1124,15 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6245d59a3e82a7fc217c5828a6692dbc6dfb63a0c8c90495621f7b9d79704a0e" +[[package]] +name = "convert_case" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ec182b0ca2f35d8fc196cf3404988fd8b8c739a4d270ff118a398feb0cbec1ca" +dependencies = [ + "unicode-segmentation", +] + [[package]] name = "core-foundation" version = "0.9.3" @@ -1402,13 +1411,35 @@ version = "0.99.17" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4fb810d30a7c1953f91334de7244731fc3f3c10d7fe163338a35b9f640960321" dependencies = [ - "convert_case", + "convert_case 0.4.0", "proc-macro2", "quote", "rustc_version", "syn 1.0.109", ] +[[package]] +name = "derive_more" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4a9b99b9cbbe49445b21764dc0625032a89b145a2642e67603e1c936f5458d05" +dependencies = [ + "derive_more-impl", +] + +[[package]] +name = "derive_more-impl" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cb7330aeadfbe296029522e6c40f315320aba36fc43a5b3632f3795348f3bd22" +dependencies = [ + "convert_case 0.6.0", + "proc-macro2", + "quote", + "syn 2.0.87", + "unicode-xid", +] + [[package]] name = "dialoguer" version = "0.10.4" @@ -1517,6 +1548,12 @@ version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1435fa1053d8b2fbbe9be7e97eca7f33d37b28409959813daefc1446a14247f1" +[[package]] +name = "dyn-clone" +version = "1.0.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0d6ef0072f8a535281e4876be788938b528e9a1d43900b82c2569af7da799125" + [[package]] name = "eager" version = "0.1.0" @@ -2680,7 +2717,7 @@ version = "18.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d2b99d4207e2a04fb4581746903c2bb7eb376f88de9c699d0f3e10feeac0cd3a" dependencies = [ - "derive_more", + "derive_more 0.99.17", "futures 0.3.31", "jsonrpc-core", "jsonrpc-pubsub", @@ -5513,6 +5550,7 @@ dependencies = [ "chrono", "crossbeam-channel", "dashmap", + "derive_more 1.0.0", "etcd-client", "futures 0.3.31", "histogram", @@ -6306,6 +6344,7 @@ dependencies = [ "bincode", "bv", "caps", + "crossbeam-channel", "curve25519-dalek 4.1.3", "dlopen2", "fnv", @@ -6886,6 +6925,7 @@ dependencies = [ "ahash 0.8.11", "aquamarine", "arrayref", + "assert_matches", "base64 0.22.1", "bincode", "blake3", @@ -8376,8 +8416,9 @@ dependencies = [ "crossbeam-channel", "dashmap", "derive-where", + "derive_more 1.0.0", + "dyn-clone", "log", - "qualifier_attr", "scopeguard", "solana-ledger", "solana-poh", @@ -8389,6 +8430,7 @@ dependencies = [ "solana-transaction-error", "solana-unified-scheduler-logic", "static_assertions", + "trait-set", "vec_extract_if_polyfill", ] @@ -9621,6 +9663,17 @@ dependencies = [ "tracing-core", ] +[[package]] +name = "trait-set" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b79e2e9c9ab44c6d7c20d5976961b47e8f49ac199154daa514b77cd1ab536625" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + [[package]] name = "trees" version = "0.4.2" @@ -9696,6 +9749,12 @@ dependencies = [ "tinyvec", ] +[[package]] +name = "unicode-segmentation" +version = "1.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f6ccf251212114b54433ec949fd6a7841275f9ada20dddd2f29e9ceea4501493" + [[package]] name = "unicode-width" version = "0.1.8" diff --git a/runtime/Cargo.toml b/runtime/Cargo.toml index 0687e594a9dead..eda10e43de1082 100644 --- a/runtime/Cargo.toml +++ b/runtime/Cargo.toml @@ -13,6 +13,7 @@ edition = { workspace = true } ahash = { workspace = true } aquamarine = { workspace = true } arrayref = { workspace = true } +assert_matches = { workspace = true } base64 = { workspace = true } bincode = { workspace = true } blake3 = { workspace = true } @@ -101,7 +102,6 @@ name = "solana_runtime" [dev-dependencies] agave-transaction-view = { workspace = true } -assert_matches = { workspace = true } ed25519-dalek = { workspace = true } libsecp256k1 = { workspace = true } memoffset = { workspace = true } diff --git a/runtime/src/bank_forks.rs b/runtime/src/bank_forks.rs index 1627f8113021db..2e52945f2595e5 100644 --- a/runtime/src/bank_forks.rs +++ b/runtime/src/bank_forks.rs @@ -18,6 +18,7 @@ use { clock::{BankId, Slot}, hash::Hash, }, + solana_unified_scheduler_logic::SchedulingMode, std::{ collections::{hash_map::Entry, HashMap, HashSet}, ops::Index, @@ -226,18 +227,22 @@ impl BankForks { ); } - pub fn insert(&mut self, mut bank: Bank) -> BankWithScheduler { + pub fn insert(&mut self, bank: Bank) -> BankWithScheduler { + self.insert_with_scheduling_mode(SchedulingMode::BlockVerification, bank) + } + + pub fn insert_with_scheduling_mode( + &mut self, + mode: SchedulingMode, + mut bank: Bank, + ) -> BankWithScheduler { if self.root.load(Ordering::Relaxed) < self.highest_slot_at_startup { bank.set_check_program_modification_slot(true); } let bank = Arc::new(bank); let bank = if let Some(scheduler_pool) = &self.scheduler_pool { - let context = SchedulingContext::new(bank.clone()); - let scheduler = scheduler_pool.take_scheduler(context); - let bank_with_scheduler = BankWithScheduler::new(bank, Some(scheduler)); - scheduler_pool.register_timeout_listener(bank_with_scheduler.create_timeout_listener()); - bank_with_scheduler + Self::install_scheduler_into_bank(scheduler_pool, mode, bank, false) } else { BankWithScheduler::new_without_scheduler(bank) }; @@ -251,6 +256,42 @@ impl BankForks { bank } + fn install_scheduler_into_bank( + scheduler_pool: &InstalledSchedulerPoolArc, + mode: SchedulingMode, + bank: Arc, + is_reinstall: bool, + ) -> BankWithScheduler { + trace!( + "Inserting bank (slot: {}) with scheduler (mode: {:?}, reinstall: {:?})", + bank.slot(), + mode, + is_reinstall, + ); + let context = SchedulingContext::new(mode, Some(bank.clone())); + let Some(scheduler) = scheduler_pool.take_scheduler(context) else { + return BankWithScheduler::new_without_scheduler(bank); + }; + let bank_with_scheduler = BankWithScheduler::new(bank, Some(scheduler)); + scheduler_pool.register_timeout_listener(bank_with_scheduler.create_timeout_listener()); + bank_with_scheduler + } + + #[cfg(feature = "dev-context-only-utils")] + pub fn reinstall_block_production_scheduler_into_working_genesis_bank( + &mut self, + ) -> BankWithScheduler { + let bank = self.working_bank(); + assert!(self.banks.len() == 1 && bank.slot() == 0 && !bank.is_frozen()); + let pool = self.scheduler_pool.as_ref().unwrap(); + let mode = SchedulingMode::BlockProduction; + let bank = Self::install_scheduler_into_bank(pool, mode, bank, true); + self.banks + .insert(bank.slot(), bank.clone_with_scheduler()) + .expect("some removed bank"); + bank + } + pub fn insert_from_ledger(&mut self, bank: Bank) -> BankWithScheduler { self.highest_slot_at_startup = std::cmp::max(self.highest_slot_at_startup, bank.slot()); self.insert(bank) diff --git a/runtime/src/installed_scheduler_pool.rs b/runtime/src/installed_scheduler_pool.rs index 69e806c5f48ce5..9807278f62f4b5 100644 --- a/runtime/src/installed_scheduler_pool.rs +++ b/runtime/src/installed_scheduler_pool.rs @@ -22,6 +22,7 @@ use { crate::bank::Bank, + assert_matches::assert_matches, log::*, solana_runtime_transaction::runtime_transaction::RuntimeTransaction, solana_sdk::{ @@ -47,7 +48,7 @@ pub fn initialized_result_with_timings() -> ResultWithTimings { } pub trait InstalledSchedulerPool: Send + Sync + Debug { - fn take_scheduler(&self, context: SchedulingContext) -> InstalledSchedulerBox { + fn take_scheduler(&self, context: SchedulingContext) -> Option { self.take_resumed_scheduler(context, initialized_result_with_timings()) } @@ -55,7 +56,7 @@ pub trait InstalledSchedulerPool: Send + Sync + Debug { &self, context: SchedulingContext, result_with_timings: ResultWithTimings, - ) -> InstalledSchedulerBox; + ) -> Option; fn register_timeout_listener(&self, timeout_listener: TimeoutListener); } @@ -229,24 +230,22 @@ pub type SchedulerId = u64; #[derive(Clone, Debug)] pub struct SchedulingContext { mode: SchedulingMode, - bank: Arc, + bank: Option>, } impl SchedulingContext { - pub fn new(bank: Arc) -> Self { - // mode will be configurable later - Self { - mode: SchedulingMode::BlockVerification, - bank, - } + pub fn new(mode: SchedulingMode, bank: Option>) -> Self { + Self { mode, bank } + } + + #[cfg_attr(feature = "dev-context-only-utils", qualifiers(pub))] + fn for_verification(bank: Arc) -> Self { + Self::new(SchedulingMode::BlockVerification, Some(bank)) } #[cfg(feature = "dev-context-only-utils")] pub fn for_production(bank: Arc) -> Self { - Self { - mode: SchedulingMode::BlockProduction, - bank, - } + Self::new(SchedulingMode::BlockProduction, Some(bank)) } pub fn mode(&self) -> SchedulingMode { @@ -254,11 +253,11 @@ impl SchedulingContext { } pub fn bank(&self) -> &Arc { - &self.bank + self.bank.as_ref().unwrap() } - pub fn slot(&self) -> Slot { - self.bank().slot() + pub fn slot(&self) -> Option { + self.bank.as_ref().map(|bank| bank.slot()) } } @@ -318,7 +317,7 @@ pub enum SchedulerStatus { /// Scheduler is idling for long time, returning scheduler back to the pool. /// This will be immediately (i.e. transaparently) transitioned to Active as soon as there's /// new transaction to be executed. - Stale(InstalledSchedulerPoolArc, ResultWithTimings), + Stale(InstalledSchedulerPoolArc, SchedulingMode, ResultWithTimings), } impl SchedulerStatus { @@ -329,13 +328,23 @@ impl SchedulerStatus { } } + fn scheduling_mode(&self) -> Option { + match self { + SchedulerStatus::Unavailable => None, + SchedulerStatus::Active(scheduler) => Some(scheduler.context().mode()), + SchedulerStatus::Stale(_, mode, _) => Some(*mode), + } + } + fn transition_from_stale_to_active( &mut self, f: impl FnOnce(InstalledSchedulerPoolArc, ResultWithTimings) -> InstalledSchedulerBox, ) { - let Self::Stale(pool, result_with_timings) = mem::replace(self, Self::Unavailable) else { + let Self::Stale(pool, mode, result_with_timings) = mem::replace(self, Self::Unavailable) + else { panic!("transition to Active failed: {self:?}"); }; + assert_matches!(mode, SchedulingMode::BlockVerification); *self = Self::Active(f(pool, result_with_timings)); } @@ -349,8 +358,9 @@ impl SchedulerStatus { let Self::Active(scheduler) = mem::replace(self, Self::Unavailable) else { unreachable!("not active: {self:?}"); }; + let mode = scheduler.context().mode; let (pool, result_with_timings) = f(scheduler); - *self = Self::Stale(pool, result_with_timings); + *self = Self::Stale(pool, mode, result_with_timings); } fn transition_from_active_to_unavailable(&mut self) -> InstalledSchedulerBox { @@ -361,7 +371,8 @@ impl SchedulerStatus { } fn transition_from_stale_to_unavailable(&mut self) -> ResultWithTimings { - let Self::Stale(_pool, result_with_timings) = mem::replace(self, Self::Unavailable) else { + let Self::Stale(_pool, _mode, result_with_timings) = mem::replace(self, Self::Unavailable) + else { panic!("transition to Unavailable failed: {self:?}"); }; result_with_timings @@ -471,6 +482,10 @@ impl BankWithScheduler { ); let schedule_result: ScheduleResult = self.inner.with_active_scheduler(|scheduler| { + assert_matches!( + scheduler.context().mode(), + SchedulingMode::BlockVerification + ); for (sanitized_transaction, index) in transactions_with_indexes { scheduler.schedule_execution(sanitized_transaction, index)?; } @@ -524,6 +539,10 @@ impl BankWithScheduler { ) } + pub fn scheduling_mode(&self) -> Option { + self.inner.scheduler.read().unwrap().scheduling_mode() + } + pub const fn no_scheduler_available() -> InstalledSchedulerRwLock { RwLock::new(SchedulerStatus::Unavailable) } @@ -540,22 +559,26 @@ impl BankWithSchedulerInner { // This is the fast path, needing single read-lock most of time. f(scheduler) } - SchedulerStatus::Stale(_pool, (result, _timings)) if result.is_err() => { + SchedulerStatus::Stale(_pool, mode, (result, _timings)) if result.is_err() => { + assert_matches!(mode, SchedulingMode::BlockVerification); trace!( "with_active_scheduler: bank (slot: {}) has a stale aborted scheduler...", self.bank.slot(), ); Err(SchedulerAborted) } - SchedulerStatus::Stale(pool, _result_with_timings) => { + SchedulerStatus::Stale(pool, mode, _result_with_timings) => { + assert_matches!(mode, SchedulingMode::BlockVerification); let pool = pool.clone(); drop(scheduler); - let context = SchedulingContext::new(self.bank.clone()); + let context = SchedulingContext::for_verification(self.bank.clone()); let mut scheduler = self.scheduler.write().unwrap(); trace!("with_active_scheduler: {:?}", scheduler); scheduler.transition_from_stale_to_active(|pool, result_with_timings| { - let scheduler = pool.take_resumed_scheduler(context, result_with_timings); + let scheduler = pool + .take_resumed_scheduler(context, result_with_timings) + .expect("successful retaking"); info!( "with_active_scheduler: bank (slot: {}) got active, taking scheduler (id: {})", self.bank.slot(), @@ -588,23 +611,24 @@ impl BankWithSchedulerInner { return; }; + let mut id = None; scheduler.maybe_transition_from_active_to_stale(|scheduler| { // The scheduler hasn't still been wait_for_termination()-ed after awhile... // Return the installed scheduler back to the scheduler pool as soon as the // scheduler gets idle after executing all currently-scheduled transactions. - let id = scheduler.id(); + id = Some(scheduler.id()); let (result_with_timings, uninstalled_scheduler) = scheduler.wait_for_termination(false); uninstalled_scheduler.return_to_pool(); info!( - "timeout_listener: bank (slot: {}) got stale, returning scheduler (id: {})", + "timeout_listener: bank (slot: {}) got stale, returned scheduler (id: {:?})", bank.bank.slot(), id, ); (pool, result_with_timings) }); - trace!("timeout_listener: {:?}", scheduler); + trace!("timeout_listener: {:?}", id); }) } @@ -614,7 +638,8 @@ impl BankWithSchedulerInner { let mut scheduler = self.scheduler.write().unwrap(); match &mut *scheduler { SchedulerStatus::Active(scheduler) => scheduler.recover_error_after_abort(), - SchedulerStatus::Stale(_pool, (result, _timings)) if result.is_err() => { + SchedulerStatus::Stale(_pool, mode, (result, _timings)) if result.is_err() => { + assert_matches!(mode, SchedulingMode::BlockVerification); result.clone().unwrap_err() } _ => unreachable!("no error in {:?}", self.scheduler), @@ -656,12 +681,12 @@ impl BankWithSchedulerInner { uninstalled_scheduler.return_to_pool(); (false, Some(result_with_timings)) } - SchedulerStatus::Stale(_pool, _result_with_timings) if reason.is_paused() => { + SchedulerStatus::Stale(_pool, _mode, _result_with_timings) if reason.is_paused() => { // Do nothing for pauses because the scheduler termination is guaranteed to be // called later. (true, None) } - SchedulerStatus::Stale(_pool, _result_with_timings) => { + SchedulerStatus::Stale(_pool, _mode, _result_with_timings) => { let result_with_timings = scheduler.transition_from_stale_to_unavailable(); (true, Some(result_with_timings)) } @@ -728,7 +753,6 @@ mod tests { bank::test_utils::goto_end_of_slot_with_scheduler, genesis_utils::{create_genesis_config, GenesisConfigInfo}, }, - assert_matches::assert_matches, mockall::Sequence, solana_sdk::system_transaction, std::sync::Mutex, @@ -738,14 +762,16 @@ mod tests { bank: Arc, is_dropped_flags: impl Iterator, f: Option, + extra_context_use: usize, ) -> InstalledSchedulerBox { let mut mock = MockInstalledScheduler::new(); let seq = Arc::new(Mutex::new(Sequence::new())); + // Could be used for assertions in BankWithScheduler::{new, schedule_transaction_executions} mock.expect_context() - .times(1) + .times(1 + extra_context_use) .in_sequence(&mut seq.lock().unwrap()) - .return_const(SchedulingContext::new(bank)); + .return_const(SchedulingContext::for_verification(bank)); for wait_reason in is_dropped_flags { let seq_cloned = seq.clone(); @@ -782,6 +808,7 @@ mod tests { bank, is_dropped_flags, None:: ()>, + 0, ) } @@ -843,6 +870,7 @@ mod tests { .times(1) .returning(|| ()); }), + 0, )), ); goto_end_of_slot_with_scheduler(&bank); @@ -884,6 +912,7 @@ mod tests { .returning(|| TransactionError::InsufficientFundsForFee); } }), + 1, ); let bank = BankWithScheduler::new(bank, Some(mocked_scheduler)); diff --git a/sdk/src/scheduling.rs b/sdk/src/scheduling.rs new file mode 100644 index 00000000000000..ebab21a912f292 --- /dev/null +++ b/sdk/src/scheduling.rs @@ -0,0 +1,8 @@ +//! Primitive types relevant to transaction scheduling +#![cfg(feature = "full")] + +#[derive(Debug, Clone, Copy, PartialEq)] +pub enum SchedulingMode { + BlockVerification, + BlockProduction, +} diff --git a/svm/examples/Cargo.lock b/svm/examples/Cargo.lock index f38ef7446fa02c..2a4548a254100b 100644 --- a/svm/examples/Cargo.lock +++ b/svm/examples/Cargo.lock @@ -1038,6 +1038,15 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6245d59a3e82a7fc217c5828a6692dbc6dfb63a0c8c90495621f7b9d79704a0e" +[[package]] +name = "convert_case" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ec182b0ca2f35d8fc196cf3404988fd8b8c739a4d270ff118a398feb0cbec1ca" +dependencies = [ + "unicode-segmentation", +] + [[package]] name = "core-foundation" version = "0.9.4" @@ -1312,13 +1321,35 @@ version = "0.99.18" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5f33878137e4dafd7fa914ad4e259e18a4e8e532b9617a2d0150262bf53abfce" dependencies = [ - "convert_case", + "convert_case 0.4.0", "proc-macro2", "quote", "rustc_version", "syn 2.0.87", ] +[[package]] +name = "derive_more" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4a9b99b9cbbe49445b21764dc0625032a89b145a2642e67603e1c936f5458d05" +dependencies = [ + "derive_more-impl", +] + +[[package]] +name = "derive_more-impl" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cb7330aeadfbe296029522e6c40f315320aba36fc43a5b3632f3795348f3bd22" +dependencies = [ + "convert_case 0.6.0", + "proc-macro2", + "quote", + "syn 2.0.87", + "unicode-xid", +] + [[package]] name = "dialoguer" version = "0.10.4" @@ -1427,6 +1458,12 @@ version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1435fa1053d8b2fbbe9be7e97eca7f33d37b28409959813daefc1446a14247f1" +[[package]] +name = "dyn-clone" +version = "1.0.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0d6ef0072f8a535281e4876be788938b528e9a1d43900b82c2569af7da799125" + [[package]] name = "eager" version = "0.1.0" @@ -2636,7 +2673,7 @@ version = "18.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d2b99d4207e2a04fb4581746903c2bb7eb376f88de9c699d0f3e10feeac0cd3a" dependencies = [ - "derive_more", + "derive_more 0.99.18", "futures 0.3.31", "jsonrpc-core", "jsonrpc-pubsub", @@ -5364,6 +5401,7 @@ dependencies = [ "chrono", "crossbeam-channel", "dashmap", + "derive_more 1.0.0", "etcd-client", "futures 0.3.31", "histogram", @@ -6126,6 +6164,7 @@ dependencies = [ "bincode", "bv", "caps", + "crossbeam-channel", "curve25519-dalek 4.1.3", "dlopen2", "fnv", @@ -6706,6 +6745,7 @@ dependencies = [ "ahash 0.8.11", "aquamarine", "arrayref", + "assert_matches", "base64 0.22.1", "bincode", "blake3", @@ -7712,8 +7752,9 @@ dependencies = [ "crossbeam-channel", "dashmap", "derive-where", + "derive_more 1.0.0", + "dyn-clone", "log", - "qualifier_attr", "scopeguard", "solana-ledger", "solana-poh", @@ -7725,6 +7766,7 @@ dependencies = [ "solana-transaction-error", "solana-unified-scheduler-logic", "static_assertions", + "trait-set", "vec_extract_if_polyfill", ] @@ -8927,6 +8969,17 @@ dependencies = [ "tracing-core", ] +[[package]] +name = "trait-set" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b79e2e9c9ab44c6d7c20d5976961b47e8f49ac199154daa514b77cd1ab536625" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + [[package]] name = "trees" version = "0.4.2" @@ -8999,6 +9052,12 @@ dependencies = [ "tinyvec", ] +[[package]] +name = "unicode-segmentation" +version = "1.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f6ccf251212114b54433ec949fd6a7841275f9ada20dddd2f29e9ceea4501493" + [[package]] name = "unicode-width" version = "0.1.14" diff --git a/unified-scheduler-logic/src/lib.rs b/unified-scheduler-logic/src/lib.rs index c7b5e7a1e8ca76..682fd037259e73 100644 --- a/unified-scheduler-logic/src/lib.rs +++ b/unified-scheduler-logic/src/lib.rs @@ -105,7 +105,7 @@ use { std::{collections::VecDeque, mem, sync::Arc}, }; -#[derive(Clone, Copy, Debug)] +#[derive(Clone, Copy, Debug, PartialEq)] pub enum SchedulingMode { BlockVerification, BlockProduction, @@ -457,6 +457,10 @@ impl TaskInner { .with_borrow_mut(token, |usage_count| usage_count.decrement_self().is_zero()); did_unblock.then_some(self) } + + pub fn into_transaction(self: Task) -> RuntimeTransaction { + Task::into_inner(self).unwrap().transaction + } } /// [`Task`]'s per-address context to lock a [usage_queue](UsageQueue) with [certain kind of @@ -625,23 +629,37 @@ const_assert_eq!(mem::size_of::(), 8); pub struct SchedulingStateMachine { unblocked_task_queue: VecDeque, active_task_count: ShortCounter, + executing_task_count: ShortCounter, + max_executing_task_count: u32, handled_task_count: ShortCounter, unblocked_task_count: ShortCounter, total_task_count: ShortCounter, count_token: BlockedUsageCountToken, usage_queue_token: UsageQueueToken, } -const_assert_eq!(mem::size_of::(), 48); +const_assert_eq!(mem::size_of::(), 56); impl SchedulingStateMachine { pub fn has_no_active_task(&self) -> bool { self.active_task_count.is_zero() } + pub fn has_no_executing_task(&self) -> bool { + self.executing_task_count.current() == 0 + } + pub fn has_unblocked_task(&self) -> bool { !self.unblocked_task_queue.is_empty() } + pub fn has_runnable_task(&mut self) -> bool { + self.is_task_runnable() && self.has_unblocked_task() + } + + pub fn is_task_runnable(&self) -> bool { + self.executing_task_count.current() < self.max_executing_task_count + } + pub fn unblocked_task_queue_count(&self) -> usize { self.unblocked_task_queue.len() } @@ -670,14 +688,28 @@ impl SchedulingStateMachine { /// Note that this function takes ownership of the task to allow for future optimizations. #[must_use] pub fn schedule_task(&mut self, task: Task) -> Option { + self.do_schedule_task(task, false) + } + + pub fn do_schedule_task(&mut self, task: Task, force_buffer_mode: bool) -> Option { self.total_task_count.increment_self(); self.active_task_count.increment_self(); - self.try_lock_usage_queues(task) + self.try_lock_usage_queues(task).and_then(|task| { + if self.is_task_runnable() && !force_buffer_mode { + self.executing_task_count.increment_self(); + Some(task) + } else { + self.unblocked_task_count.increment_self(); + self.unblocked_task_queue.push_back(task); + None + } + }) } #[must_use] pub fn schedule_next_unblocked_task(&mut self) -> Option { self.unblocked_task_queue.pop_front().inspect(|_| { + self.executing_task_count.increment_self(); self.unblocked_task_count.increment_self(); }) } @@ -693,6 +725,7 @@ impl SchedulingStateMachine { /// tasks inside `SchedulingStateMachine` to provide an offloading-based optimization /// opportunity for callers. pub fn deschedule_task(&mut self, task: &Task) { + self.executing_task_count.decrement_self(); self.active_task_count.decrement_self(); self.handled_task_count.increment_self(); self.unlock_usage_queues(task); @@ -779,6 +812,14 @@ impl SchedulingStateMachine { transaction: RuntimeTransaction, index: usize, usage_queue_loader: &mut impl FnMut(Pubkey) -> UsageQueue, + ) -> Task { + Self::do_create_task(transaction, index, usage_queue_loader) + } + + pub fn do_create_task( + transaction: RuntimeTransaction, + index: usize, + usage_queue_loader: &mut impl FnMut(Pubkey) -> UsageQueue, ) -> Task { // It's crucial for tasks to be validated with // `account_locks::validate_account_locks()` prior to the creation. @@ -848,11 +889,14 @@ impl SchedulingStateMachine { /// other slots. pub fn reinitialize(&mut self) { assert!(self.has_no_active_task()); + assert_eq!(self.executing_task_count.current(), 0); assert_eq!(self.unblocked_task_queue.len(), 0); // nice trick to ensure all fields are handled here if new one is added. let Self { unblocked_task_queue: _, active_task_count, + executing_task_count: _, + max_executing_task_count: _, handled_task_count, unblocked_task_count, total_task_count, @@ -872,12 +916,16 @@ impl SchedulingStateMachine { /// # Safety /// Call this exactly once for each thread. See [`TokenCell`] for details. #[must_use] - pub unsafe fn exclusively_initialize_current_thread_for_scheduling() -> Self { + pub unsafe fn exclusively_initialize_current_thread_for_scheduling( + max_executing_task_count: u32, + ) -> Self { Self { // It's very unlikely this is desired to be configurable, like // `UsageQueueInner::blocked_usages_from_tasks`'s cap. unblocked_task_queue: VecDeque::with_capacity(1024), active_task_count: ShortCounter::zero(), + executing_task_count: ShortCounter::zero(), + max_executing_task_count, handled_task_count: ShortCounter::zero(), unblocked_task_count: ShortCounter::zero(), total_task_count: ShortCounter::zero(), @@ -885,6 +933,11 @@ impl SchedulingStateMachine { usage_queue_token: unsafe { UsageQueueToken::assume_exclusive_mutating_thread() }, } } + + #[cfg(test)] + unsafe fn exclusively_initialize_current_thread_for_scheduling_for_test() -> Self { + Self::exclusively_initialize_current_thread_for_scheduling(200) + } } #[cfg(test)] @@ -946,7 +999,7 @@ mod tests { #[test] fn test_scheduling_state_machine_creation() { let state_machine = unsafe { - SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling() + SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling_for_test() }; assert_eq!(state_machine.active_task_count(), 0); assert_eq!(state_machine.total_task_count(), 0); @@ -956,7 +1009,7 @@ mod tests { #[test] fn test_scheduling_state_machine_good_reinitialization() { let mut state_machine = unsafe { - SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling() + SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling_for_test() }; state_machine.total_task_count.increment_self(); assert_eq!(state_machine.total_task_count(), 1); @@ -968,7 +1021,7 @@ mod tests { #[should_panic(expected = "assertion failed: self.has_no_active_task()")] fn test_scheduling_state_machine_bad_reinitialization() { let mut state_machine = unsafe { - SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling() + SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling_for_test() }; let address_loader = &mut create_address_loader(None); let task = SchedulingStateMachine::create_task(simplest_transaction(), 3, address_loader); @@ -993,7 +1046,7 @@ mod tests { let task = SchedulingStateMachine::create_task(sanitized, 3, address_loader); let mut state_machine = unsafe { - SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling() + SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling_for_test() }; let task = state_machine.schedule_task(task).unwrap(); assert_eq!(state_machine.active_task_count(), 1); @@ -1013,7 +1066,7 @@ mod tests { let task3 = SchedulingStateMachine::create_task(sanitized.clone(), 103, address_loader); let mut state_machine = unsafe { - SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling() + SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling_for_test() }; assert_matches!( state_machine @@ -1065,7 +1118,7 @@ mod tests { let task3 = SchedulingStateMachine::create_task(sanitized.clone(), 103, address_loader); let mut state_machine = unsafe { - SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling() + SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling_for_test() }; assert_matches!( state_machine @@ -1115,7 +1168,7 @@ mod tests { let task2 = SchedulingStateMachine::create_task(sanitized2, 102, address_loader); let mut state_machine = unsafe { - SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling() + SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling_for_test() }; // both of read-only tasks should be immediately runnable assert_matches!( @@ -1156,7 +1209,7 @@ mod tests { let task3 = SchedulingStateMachine::create_task(sanitized3, 103, address_loader); let mut state_machine = unsafe { - SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling() + SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling_for_test() }; assert_matches!( state_machine @@ -1207,7 +1260,7 @@ mod tests { let task3 = SchedulingStateMachine::create_task(sanitized3, 103, address_loader); let mut state_machine = unsafe { - SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling() + SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling_for_test() }; assert_matches!( state_machine @@ -1249,7 +1302,7 @@ mod tests { let task2 = SchedulingStateMachine::create_task(sanitized2, 102, address_loader); let mut state_machine = unsafe { - SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling() + SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling_for_test() }; assert_matches!( state_machine @@ -1285,7 +1338,7 @@ mod tests { let task4 = SchedulingStateMachine::create_task(sanitized4, 104, address_loader); let mut state_machine = unsafe { - SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling() + SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling_for_test() }; assert_matches!( state_machine @@ -1341,7 +1394,7 @@ mod tests { let task2 = SchedulingStateMachine::create_task(sanitized2, 102, address_loader); let mut state_machine = unsafe { - SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling() + SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling_for_test() }; assert_matches!( state_machine @@ -1381,7 +1434,7 @@ mod tests { #[should_panic(expected = "internal error: entered unreachable code")] fn test_unreachable_unlock_conditions1() { let mut state_machine = unsafe { - SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling() + SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling_for_test() }; let usage_queue = UsageQueue::default(); usage_queue @@ -1395,7 +1448,7 @@ mod tests { #[should_panic(expected = "internal error: entered unreachable code")] fn test_unreachable_unlock_conditions2() { let mut state_machine = unsafe { - SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling() + SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling_for_test() }; let usage_queue = UsageQueue::default(); usage_queue @@ -1410,7 +1463,7 @@ mod tests { #[should_panic(expected = "internal error: entered unreachable code")] fn test_unreachable_unlock_conditions3() { let mut state_machine = unsafe { - SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling() + SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling_for_test() }; let usage_queue = UsageQueue::default(); usage_queue diff --git a/unified-scheduler-pool/Cargo.toml b/unified-scheduler-pool/Cargo.toml index b720642a91906c..6134a144e9a9c7 100644 --- a/unified-scheduler-pool/Cargo.toml +++ b/unified-scheduler-pool/Cargo.toml @@ -10,23 +10,27 @@ license = { workspace = true } edition = { workspace = true } [dependencies] +agave-banking-stage-ingress-types = { workspace = true } assert_matches = { workspace = true } crossbeam-channel = { workspace = true } dashmap = { workspace = true } derive-where = { workspace = true } +derive_more = { workspace = true } +dyn-clone = { workspace = true } log = { workspace = true } -qualifier_attr = { workspace = true } scopeguard = { workspace = true } solana-ledger = { workspace = true } solana-poh = { workspace = true } solana-pubkey = { workspace = true } solana-runtime = { workspace = true } solana-runtime-transaction = { workspace = true } +solana-sdk = { workspace = true } solana-timings = { workspace = true } solana-transaction = { workspace = true } solana-transaction-error = { workspace = true } solana-unified-scheduler-logic = { workspace = true } static_assertions = { workspace = true } +trait-set = { workspace = true } vec_extract_if_polyfill = { workspace = true } [dev-dependencies] @@ -39,6 +43,8 @@ solana-keypair = { workspace = true } solana-logger = { workspace = true } solana-runtime = { workspace = true, features = ["dev-context-only-utils"] } solana-system-transaction = { workspace = true } +# See order-crates-for-publishing.py for using this unusual `path = "."` +solana-unified-scheduler-pool = { path = ".", features = ["dev-context-only-utils"] } test-case = { workspace = true } [features] diff --git a/unified-scheduler-pool/src/lib.rs b/unified-scheduler-pool/src/lib.rs index f926945c769089..68be2547b4fd91 100644 --- a/unified-scheduler-pool/src/lib.rs +++ b/unified-scheduler-pool/src/lib.rs @@ -8,13 +8,13 @@ //! and commits any side-effects (i.e. on-chain state changes) into the associated `Bank` via //! `solana-ledger`'s helper function called `execute_batch()`. -#[cfg(feature = "dev-context-only-utils")] -use qualifier_attr::qualifiers; use { + agave_banking_stage_ingress_types::{BankingPacketBatch, BankingPacketReceiver}, assert_matches::assert_matches, crossbeam_channel::{self, never, select_biased, Receiver, RecvError, SendError, Sender}, dashmap::DashMap, derive_where::derive_where, + dyn_clone::{clone_trait_object, DynClone}, log::*, scopeguard::defer, solana_ledger::blockstore_processor::{ @@ -25,9 +25,9 @@ use { solana_runtime::{ installed_scheduler_pool::{ initialized_result_with_timings, InstalledScheduler, InstalledSchedulerBox, - InstalledSchedulerPool, InstalledSchedulerPoolArc, ResultWithTimings, ScheduleResult, - SchedulerAborted, SchedulerId, SchedulingContext, TimeoutListener, - UninstalledScheduler, UninstalledSchedulerBox, + InstalledSchedulerPool, ResultWithTimings, ScheduleResult, SchedulerAborted, + SchedulerId, SchedulingContext, TimeoutListener, UninstalledScheduler, + UninstalledSchedulerBox, }, prioritization_fee_cache::PrioritizationFeeCache, vote_sender_types::ReplayVoteSender, @@ -37,7 +37,7 @@ use { solana_transaction::sanitized::SanitizedTransaction, solana_transaction_error::{TransactionError, TransactionResult as Result}, solana_unified_scheduler_logic::{ - SchedulingMode::{BlockProduction, BlockVerification}, + SchedulingMode::{self, BlockProduction, BlockVerification}, SchedulingStateMachine, Task, UsageQueue, }, static_assertions::const_assert_eq, @@ -46,15 +46,23 @@ use { marker::PhantomData, mem, sync::{ - atomic::{AtomicU64, Ordering::Relaxed}, - Arc, Mutex, OnceLock, Weak, + atomic::{AtomicU64, AtomicUsize, Ordering::Relaxed}, + Arc, Condvar, Mutex, MutexGuard, OnceLock, Weak, }, thread::{self, sleep, JoinHandle}, time::{Duration, Instant}, }, + trait_set::trait_set, vec_extract_if_polyfill::MakeExtractIf, }; +#[derive(Clone)] +pub struct BankingStageContext { + adapter: Arc, + banking_packet_receiver: BankingPacketReceiver, + on_banking_packet_receive: Box, +} + mod sleepless_testing; use crate::sleepless_testing::BuilderTracked; @@ -72,15 +80,40 @@ enum CheckPoint { type AtomicSchedulerId = AtomicU64; +#[derive(Debug)] +pub enum SupportedSchedulingMode { + Either(SchedulingMode), + Both, +} + +impl SupportedSchedulingMode { + fn is_supported(&self, requested_mode: SchedulingMode) -> bool { + match (self, requested_mode) { + (Self::Both, _) => true, + (Self::Either(ref supported), ref requested) if supported == requested => true, + _ => false, + } + } + + #[cfg(feature = "dev-context-only-utils")] + fn block_verification_only() -> Self { + Self::Either(BlockVerification) + } +} + // SchedulerPool must be accessed as a dyn trait from solana-runtime, because SchedulerPool // contains some internal fields, whose types aren't available in solana-runtime (currently // TransactionStatusSender; also, PohRecorder in the future)... #[derive(Debug)] pub struct SchedulerPool, TH: TaskHandler> { + supported_scheduling_mode: SupportedSchedulingMode, scheduler_inners: Mutex>, + block_production_scheduler_inner: Mutex<(Option, Option)>, + block_production_scheduler_condvar: Condvar, + block_production_scheduler_respawner: Mutex>, trashed_scheduler_inners: Mutex>, timeout_listeners: Mutex>, - handler_count: usize, + block_verification_handler_count: usize, handler_context: HandlerContext, // weak_self could be elided by changing InstalledScheduler::take_scheduler()'s receiver to // Arc from &Self, because SchedulerPool is used as in the form of Arc @@ -104,15 +137,15 @@ pub struct HandlerContext { transaction_status_sender: Option, replay_vote_sender: Option, prioritization_fee_cache: Arc, - transaction_recorder: Option, + transaction_recorder: TransactionRecorder, } pub type DefaultSchedulerPool = SchedulerPool, DefaultTaskHandler>; -const DEFAULT_POOL_CLEANER_INTERVAL: Duration = Duration::from_secs(10); +const DEFAULT_POOL_CLEANER_INTERVAL: Duration = Duration::from_secs(5); const DEFAULT_MAX_POOLING_DURATION: Duration = Duration::from_secs(180); -const DEFAULT_TIMEOUT_DURATION: Duration = Duration::from_secs(12); +const DEFAULT_TIMEOUT_DURATION: Duration = Duration::from_secs(5); // Rough estimate of max UsageQueueLoader size in bytes: // UsageFromTask * UsageQueue's capacity * DEFAULT_MAX_USAGE_QUEUE_COUNT // 16 bytes * 128 items * 262_144 entries == 512 MiB @@ -129,27 +162,47 @@ const DEFAULT_TIMEOUT_DURATION: Duration = Duration::from_secs(12); // because UsageQueueLoader won't grow that much to begin with. const DEFAULT_MAX_USAGE_QUEUE_COUNT: usize = 262_144; +trait_set! { + pub trait BatchConverter = + DynClone + (for<'a> Fn(BankingPacketBatch, &'a dyn Fn(Task))) + Send + 'static; +} + +clone_trait_object!(BatchConverter); + +type BatchConverterCreator = + Box) -> Box) + Send>; + +#[derive(derive_more::Debug)] +struct BlockProductionSchedulerRespawner { + handler_count: usize, + #[debug("{on_spawn_block_production_scheduler:p}")] + on_spawn_block_production_scheduler: BatchConverterCreator, + banking_packet_receiver: BankingPacketReceiver, + banking_stage_monitor: Box, +} + impl SchedulerPool where S: SpawnableScheduler, TH: TaskHandler, { - // Some internal impl and test code want an actual concrete type, NOT the - // `dyn InstalledSchedulerPool`. So don't merge this into `Self::new_dyn()`. - #[cfg_attr(feature = "dev-context-only-utils", qualifiers(pub))] - fn new( + pub fn new( + supported_scheduling_mode: SupportedSchedulingMode, handler_count: Option, log_messages_bytes_limit: Option, transaction_status_sender: Option, replay_vote_sender: Option, prioritization_fee_cache: Arc, + transaction_recorder: TransactionRecorder, ) -> Arc { Self::do_new( + supported_scheduling_mode, handler_count, log_messages_bytes_limit, transaction_status_sender, replay_vote_sender, prioritization_fee_cache, + transaction_recorder, DEFAULT_POOL_CLEANER_INTERVAL, DEFAULT_MAX_POOLING_DURATION, DEFAULT_MAX_USAGE_QUEUE_COUNT, @@ -157,32 +210,61 @@ where ) } + #[cfg(feature = "dev-context-only-utils")] + pub fn new_for_verification( + handler_count: Option, + log_messages_bytes_limit: Option, + transaction_status_sender: Option, + replay_vote_sender: Option, + prioritization_fee_cache: Arc, + ) -> Arc { + Self::new( + SupportedSchedulingMode::block_verification_only(), + handler_count, + log_messages_bytes_limit, + transaction_status_sender, + replay_vote_sender, + prioritization_fee_cache, + TransactionRecorder::new_dummy(), + ) + } + + #[allow(clippy::too_many_arguments)] fn do_new( + supported_scheduling_mode: SupportedSchedulingMode, handler_count: Option, log_messages_bytes_limit: Option, transaction_status_sender: Option, replay_vote_sender: Option, prioritization_fee_cache: Arc, + mut transaction_recorder: TransactionRecorder, pool_cleaner_interval: Duration, max_pooling_duration: Duration, max_usage_queue_count: usize, timeout_duration: Duration, ) -> Arc { let handler_count = handler_count.unwrap_or(Self::default_handler_count()); - assert!(handler_count >= 1); + let bp_is_supported = supported_scheduling_mode.is_supported(BlockProduction); + + if !bp_is_supported { + transaction_recorder = TransactionRecorder::new_dummy(); + } let scheduler_pool = Arc::new_cyclic(|weak_self| Self { + supported_scheduling_mode, scheduler_inners: Mutex::default(), + block_production_scheduler_inner: Mutex::default(), + block_production_scheduler_condvar: Condvar::default(), + block_production_scheduler_respawner: Mutex::default(), trashed_scheduler_inners: Mutex::default(), timeout_listeners: Mutex::default(), - handler_count, + block_verification_handler_count: handler_count, handler_context: HandlerContext { log_messages_bytes_limit, transaction_status_sender, replay_vote_sender, prioritization_fee_cache, - // will be configurable later - transaction_recorder: None, + transaction_recorder, }, weak_self: weak_self.clone(), next_scheduler_id: AtomicSchedulerId::default(), @@ -193,8 +275,10 @@ where let cleaner_main_loop = { let weak_scheduler_pool = Arc::downgrade(&scheduler_pool); + let mut exiting = false; move || loop { sleep(pool_cleaner_interval); + trace!("Scheduler pool cleaner: start!!!",); let Some(scheduler_pool) = weak_scheduler_pool.upgrade() else { break; @@ -227,6 +311,12 @@ where idle_inner_count }; + let banking_stage_status = scheduler_pool.banking_stage_status(); + if !exiting && matches!(banking_stage_status, Some(BankingStageStatus::Exited)) { + exiting = true; + scheduler_pool.unregister_banking_stage(); + } + let trashed_inner_count = { let Ok(mut trashed_scheduler_inners) = scheduler_pool.trashed_scheduler_inners.lock() @@ -241,7 +331,7 @@ where trashed_inner_count }; - let triggered_timeout_listener_count = { + let (triggered_timeout_listener_count, active_timeout_listener_count) = { // Pre-allocate rather large capacity to avoid reallocation inside the lock. let mut expired_listeners = Vec::with_capacity(128); let Ok(mut timeout_listeners) = scheduler_pool.timeout_listeners.lock() else { @@ -253,24 +343,59 @@ where now.duration_since(*registered_at) > timeout_duration }, )); + let not_expired_count = timeout_listeners.len(); drop(timeout_listeners); - let count = expired_listeners.len(); + let expired_count = expired_listeners.len(); for (timeout_listener, _registered_at) in expired_listeners { timeout_listener.trigger(scheduler_pool.clone()); } - count + (expired_count, not_expired_count) }; + if matches!(banking_stage_status, Some(BankingStageStatus::Inactive)) { + let mut id_and_inner = scheduler_pool + .block_production_scheduler_inner + .lock() + .unwrap(); + if let Some(pooled) = &id_and_inner.1 { + if pooled.is_overgrown() { + let pooled = id_and_inner.1.take().unwrap(); + assert_eq!(Some(pooled.id()), id_and_inner.0.take()); + scheduler_pool.spawn_block_production_scheduler(&mut id_and_inner); + drop(id_and_inner); + drop(pooled); + } else { + pooled.reset(); + } + } + } + info!( - "Scheduler pool cleaner: dropped {} idle inners, {} trashed inners, triggered {} timeout listeners", - idle_inner_count, trashed_inner_count, triggered_timeout_listener_count, + "Scheduler pool cleaner: dropped {} idle inners, {} trashed inners, triggered {} timeout listeners, (exit: {:?})", + idle_inner_count, trashed_inner_count, triggered_timeout_listener_count, exiting, ); sleepless_testing::at(CheckPoint::IdleSchedulerCleaned(idle_inner_count)); sleepless_testing::at(CheckPoint::TrashedSchedulerCleaned(trashed_inner_count)); sleepless_testing::at(CheckPoint::TimeoutListenerTriggered( triggered_timeout_listener_count, )); + + if exiting && active_timeout_listener_count == 0 { + // Wait a bit to ensure the replay stage has gone. + sleep(Duration::from_secs(1)); + + let mut id_and_inner = scheduler_pool + .block_production_scheduler_inner + .lock() + .unwrap(); + if let Some(pooled) = id_and_inner.1.take() { + assert_eq!(Some(pooled.id()), id_and_inner.0.take()); + drop(id_and_inner); + drop(pooled); + } + break; + } } }; @@ -283,24 +408,6 @@ where scheduler_pool } - // This apparently-meaningless wrapper is handy, because some callers explicitly want - // `dyn InstalledSchedulerPool` to be returned for type inference convenience. - pub fn new_dyn( - handler_count: Option, - log_messages_bytes_limit: Option, - transaction_status_sender: Option, - replay_vote_sender: Option, - prioritization_fee_cache: Arc, - ) -> InstalledSchedulerPoolArc { - Self::new( - handler_count, - log_messages_bytes_limit, - transaction_status_sender, - replay_vote_sender, - prioritization_fee_cache, - ) - } - // See a comment at the weak_self field for justification of this method's existence. fn self_arc(&self) -> Arc { self.weak_self @@ -314,24 +421,52 @@ where // This fn needs to return immediately due to being part of the blocking // `::wait_for_termination()` call. - fn return_scheduler(&self, scheduler: S::Inner) { + fn return_scheduler(&self, mut scheduler: S::Inner) { // Refer to the comment in is_aborted() as to the exact definition of the concept of // _trashed_ and the interaction among different parts of unified scheduler. let should_trash = scheduler.is_trashed(); + let id = scheduler.id(); + debug!("return_scheduler(): id: {id} should_trash: {should_trash}"); + let mut id_and_inner = self.block_production_scheduler_inner.lock().unwrap(); + let is_block_production_scheduler_returned = Some(id) == id_and_inner.0.as_ref().copied(); + if should_trash { + if is_block_production_scheduler_returned { + // Abort this trashed scheduler to stop receiving BankingPacketBatch anymore... + scheduler.ensure_abort(); + } // Delay drop()-ing this trashed returned scheduler inner by stashing it in // self.trashed_scheduler_inners, which is periodically drained by the `solScCleaner` // thread. Dropping it could take long time (in fact, - // PooledSchedulerInner::usage_queue_loader can contain many entries to drop). + // TaskCreator::usage_queue_loader() can contain many entries to drop). self.trashed_scheduler_inners .lock() .expect("not poisoned") .push(scheduler); + + if is_block_production_scheduler_returned && self.should_respawn() { + info!("respawning scheduler after being trashed..."); + assert_eq!(id_and_inner.0.take(), Some(id)); + self.spawn_block_production_scheduler(&mut id_and_inner); + info!("respawned scheduler after being trashed."); + } + drop(id_and_inner); } else { - self.scheduler_inners - .lock() - .expect("not poisoned") - .push((scheduler, Instant::now())); + drop(id_and_inner); + if !is_block_production_scheduler_returned { + self.scheduler_inners + .lock() + .expect("not poisoned") + .push((scheduler, Instant::now())); + } else { + assert!(self + .block_production_scheduler_inner + .lock() + .unwrap() + .1 + .replace(scheduler) + .is_none()); + } } } @@ -347,13 +482,35 @@ where ) -> S { assert_matches!(result_with_timings, (Ok(_), _)); - // pop is intentional for filo, expecting relatively warmed-up scheduler due to having been - // returned recently - if let Some((inner, _pooled_at)) = self.scheduler_inners.lock().expect("not poisoned").pop() - { - S::from_inner(inner, context, result_with_timings) + if matches!(context.mode(), BlockVerification) { + // pop is intentional for filo, expecting relatively warmed-up scheduler due to having been + // returned recently + if let Some((inner, _pooled_at)) = + self.scheduler_inners.lock().expect("not poisoned").pop() + { + S::from_inner(inner, context, result_with_timings) + } else { + S::spawn( + self.block_verification_handler_count, + self.self_arc(), + context, + result_with_timings, + None, + ) + } } else { - S::spawn(self.self_arc(), context, result_with_timings) + let mut id_and_inner = self + .block_production_scheduler_inner + .lock() + .expect("not poisoned"); + id_and_inner = self + .block_production_scheduler_condvar + .wait_while(id_and_inner, |id_and_inner| id_and_inner.0.is_none()) + .unwrap(); + let Some(inner) = id_and_inner.1.take() else { + panic!("double take: {:?}, {:?}", context.slot(), context.mode()); + }; + S::from_inner(inner, context, result_with_timings) } } @@ -362,6 +519,95 @@ where self.scheduler_inners.lock().expect("not poisoned").len() } + pub fn block_production_supported(&self) -> bool { + self.supported_scheduling_mode.is_supported(BlockProduction) + } + + pub fn register_banking_stage( + &self, + banking_packet_receiver: BankingPacketReceiver, + handler_count: usize, + banking_stage_monitor: Box, + on_spawn_block_production_scheduler: BatchConverterCreator, + ) { + *self.block_production_scheduler_respawner.lock().unwrap() = + Some(BlockProductionSchedulerRespawner { + handler_count, + banking_packet_receiver, + on_spawn_block_production_scheduler, + banking_stage_monitor, + }); + self.spawn_block_production_scheduler( + &mut self.block_production_scheduler_inner.lock().unwrap(), + ); + } + + fn unregister_banking_stage(&self) { + assert!(self + .block_production_scheduler_respawner + .lock() + .unwrap() + .take() + .is_some()); + } + + fn banking_stage_status(&self) -> Option { + self.block_production_scheduler_respawner + .lock() + .unwrap() + .as_ref() + .map(|respawner| respawner.banking_stage_monitor.status()) + } + + fn should_respawn(&self) -> bool { + !matches!( + self.banking_stage_status(), + None | Some(BankingStageStatus::Exited) + ) + } + + fn spawn_block_production_scheduler( + &self, + id_and_inner: &mut MutexGuard<'_, (Option, Option)>, + ) { + trace!("spawn block production scheduler: start!"); + let (handler_count, banking_stage_context) = { + let mut respawner_write = self.block_production_scheduler_respawner.lock().unwrap(); + let BlockProductionSchedulerRespawner { + handler_count, + banking_packet_receiver, + on_spawn_block_production_scheduler, + banking_stage_monitor: _, + } = &mut *respawner_write.as_mut().unwrap(); + + let adapter = Arc::new(BankingStageAdapter::default()); + + ( + *handler_count, + BankingStageContext { + banking_packet_receiver: banking_packet_receiver.clone(), + on_banking_packet_receive: on_spawn_block_production_scheduler(adapter.clone()), + adapter, + }, + ) + }; + + let scheduler = S::spawn( + handler_count, + self.self_arc(), + SchedulingContext::new(BlockProduction, None), + initialized_result_with_timings(), + Some(banking_stage_context), + ); + let ((Ok(_result), _timings), inner) = scheduler.into_inner() else { + panic!() + }; + assert!(id_and_inner.0.replace(inner.id()).is_none()); + assert!(id_and_inner.1.replace(inner).is_none()); + self.block_production_scheduler_condvar.notify_all(); + trace!("spawn block production scheduler: end!"); + } + pub fn default_handler_count() -> usize { Self::calculate_default_handler_count( thread::available_parallelism() @@ -403,8 +649,14 @@ where &self, context: SchedulingContext, result_with_timings: ResultWithTimings, - ) -> InstalledSchedulerBox { - Box::new(self.do_take_resumed_scheduler(context, result_with_timings)) + ) -> Option { + if !self.supported_scheduling_mode.is_supported(context.mode()) { + return None; + } + + Some(Box::new( + self.do_take_resumed_scheduler(context, result_with_timings), + )) } fn register_timeout_listener(&self, timeout_listener: TimeoutListener) { @@ -476,8 +728,6 @@ impl TaskHandler for DefaultTaskHandler { .. } = handler_context .transaction_recorder - .as_ref() - .unwrap() .record_transactions(bank.slot(), vec![transaction.to_versioned_transaction()]); match result { Ok(()) => Ok(starting_transaction_index), @@ -512,6 +762,10 @@ impl ExecutedTask { result_with_timings: initialized_result_with_timings(), }) } + + fn into_inner(self) -> Task { + self.task + } } // A very tiny generic message type to signal about opening and closing of subchannels, which are @@ -524,6 +778,8 @@ enum SubchanneledPayload { Payload(P1), OpenSubchannel(P2), CloseSubchannel, + Disconnect, + Reset, } type NewTaskPayload = SubchanneledPayload>; @@ -715,10 +971,37 @@ pub struct PooledScheduler { context: SchedulingContext, } +#[derive(Debug)] +enum TaskCreator { + ForBlockVerification { + usage_queue_loader: UsageQueueLoader, + }, + ForBlockProduction { + banking_stage_adapter: Arc, + }, +} + +impl TaskCreator { + fn usage_queue_loader(&self) -> &UsageQueueLoader { + use TaskCreator::*; + + match self { + ForBlockVerification { usage_queue_loader } => usage_queue_loader, + ForBlockProduction { + banking_stage_adapter, + } => &banking_stage_adapter.usage_queue_loader, + } + } + + fn is_overgrown(&self, max_usage_queue_count: usize) -> bool { + self.usage_queue_loader().count() > max_usage_queue_count + } +} + #[derive(Debug)] pub struct PooledSchedulerInner, TH: TaskHandler> { thread_manager: ThreadManager, - usage_queue_loader: UsageQueueLoader, + task_creator: TaskCreator, } impl Drop for ThreadManager @@ -749,7 +1032,7 @@ where // Ensure to initiate thread shutdown via disconnected new_task_receiver by replacing the // current new_task_sender with a random one... - self.new_task_sender = crossbeam_channel::unbounded().0; + self.disconnect_new_task_sender(); self.ensure_join_threads(true); assert_matches!(self.session_result_with_timings, Some((Ok(_), _))); @@ -758,7 +1041,7 @@ where impl PooledSchedulerInner where - S: SpawnableScheduler, + S: SpawnableScheduler, TH: TaskHandler, { fn is_aborted(&self) -> bool { @@ -781,10 +1064,6 @@ where // before that. self.thread_manager.are_threads_joined() } - - fn is_overgrown(&self) -> bool { - self.usage_queue_loader.count() > self.thread_manager.pool.max_usage_queue_count - } } // This type manages the OS threads for scheduling and executing transactions. The term @@ -796,7 +1075,7 @@ where struct ThreadManager, TH: TaskHandler> { scheduler_id: SchedulerId, pool: Arc>, - new_task_sender: Sender, + new_task_sender: Arc>, new_task_receiver: Option>, session_result_sender: Sender, session_result_receiver: Receiver, @@ -816,7 +1095,7 @@ impl, TH: TaskHandler> ThreadManager { Self { scheduler_id: pool.new_scheduler_id(), pool, - new_task_sender, + new_task_sender: Arc::new(new_task_sender), new_task_receiver: Some(new_task_receiver), session_result_sender, session_result_receiver, @@ -843,20 +1122,37 @@ impl, TH: TaskHandler> ThreadManager { #[must_use] fn accumulate_result_with_timings( + context: &SchedulingContext, (result, timings): &mut ResultWithTimings, executed_task: HandlerResult, - ) -> Option> { + ) -> Option<(Box, bool)> { let Ok(executed_task) = executed_task else { return None; }; timings.accumulate(&executed_task.result_with_timings.1); - match executed_task.result_with_timings.0 { - Ok(()) => Some(executed_task), - Err(error) => { - error!("error is detected while accumulating....: {error:?}"); - *result = Err(error); - None - } + match context.mode() { + BlockVerification => match executed_task.result_with_timings.0 { + Ok(()) => Some((executed_task, false)), + Err(error) => { + error!("error is detected while accumulating....: {error:?}"); + *result = Err(error); + None + } + }, + BlockProduction => match executed_task.result_with_timings.0 { + Ok(()) => Some((executed_task, false)), + Err(TransactionError::CommitCancelled) + | Err(TransactionError::WouldExceedMaxBlockCostLimit) + | Err(TransactionError::WouldExceedMaxVoteCostLimit) + | Err(TransactionError::WouldExceedMaxAccountCostLimit) + | Err(TransactionError::WouldExceedAccountDataBlockLimit) => { + Some((executed_task, true)) + } + Err(ref error) => { + debug!("error is detected while accumulating....: {error:?}"); + Some((executed_task, false)) + } + }, } } @@ -878,9 +1174,18 @@ impl, TH: TaskHandler> ThreadManager { // for type safety. fn start_threads( &mut self, - context: SchedulingContext, + handler_count: usize, + mut context: SchedulingContext, mut result_with_timings: ResultWithTimings, + banking_stage_context: Option, ) { + assert!(handler_count >= 1); + + let postfix = match context.mode() { + BlockVerification => "V", + BlockProduction => "P", + }; + // Firstly, setup bi-directional messaging between the scheduler and handlers to pass // around tasks, by creating 2 channels (one for to-be-handled tasks from the scheduler to // the handlers and the other for finished tasks from the handlers to the scheduler). @@ -958,7 +1263,7 @@ impl, TH: TaskHandler> ThreadManager { // prioritization further. Consequently, this also contributes to alleviate the known // heuristic's caveat for the first task of linearized runs, which is described above. let (mut runnable_task_sender, runnable_task_receiver) = - chained_channel::unbounded::(context); + chained_channel::unbounded::(context.clone()); // Create two handler-to-scheduler channels to prioritize the finishing of blocked tasks, // because it is more likely that a blocked task will have more blocked tasks behind it, // which should be scheduled while minimizing the delay to clear buffered linearized runs @@ -978,7 +1283,6 @@ impl, TH: TaskHandler> ThreadManager { // 5. the handler thread reply back to the scheduler thread as an executed task. // 6. the scheduler thread post-processes the executed task. let scheduler_main_loop = { - let handler_count = self.pool.handler_count; let session_result_sender = self.session_result_sender.clone(); // Taking new_task_receiver here is important to ensure there's a single receiver. In // this way, the replay stage will get .send() failures reliably, after this scheduler @@ -988,7 +1292,14 @@ impl, TH: TaskHandler> ThreadManager { .take() .expect("no 2nd start_threads()"); + let banking_stage_context = banking_stage_context.clone(); + let mut session_ending = false; + let (mut session_pausing, mut is_finished) = match context.mode() { + BlockVerification => (false, false), + BlockProduction => (true, true), + }; + let mut session_resetting = false; // Now, this is the main loop for the scheduler thread, which is a special beast. // @@ -1039,7 +1350,9 @@ impl, TH: TaskHandler> ThreadManager { }; let mut state_machine = unsafe { - SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling() + SchedulingStateMachine::exclusively_initialize_current_thread_for_scheduling( + handler_count.checked_mul(2).unwrap().try_into().unwrap(), + ) }; // The following loop maintains and updates ResultWithTimings as its @@ -1049,14 +1362,13 @@ impl, TH: TaskHandler> ThreadManager { // 2. Subsequent result_with_timings are propagated explicitly from // the new_task_receiver.recv() invocation located at the end of loop. 'nonaborted_main_loop: loop { - let mut is_finished = false; while !is_finished { // ALL recv selectors are eager-evaluated ALWAYS by current crossbeam impl, // which isn't great and is inconsistent with `if`s in the Rust's match // arm. So, eagerly binding the result to a variable unconditionally here // makes no perf. difference... let dummy_unblocked_task_receiver = - dummy_receiver(state_machine.has_unblocked_task()); + dummy_receiver(state_machine.has_runnable_task() && !session_pausing); // There's something special called dummy_unblocked_task_receiver here. // This odd pattern was needed to react to newly unblocked tasks from @@ -1070,13 +1382,26 @@ impl, TH: TaskHandler> ThreadManager { // to measure _actual_ cpu usage easily with the select approach. select_biased! { recv(finished_blocked_task_receiver) -> executed_task => { - let Some(executed_task) = Self::accumulate_result_with_timings( + let Ok(executed_task) = executed_task else { + assert_matches!(context.mode(), BlockProduction); + break 'nonaborted_main_loop; + }; + + let Some((executed_task, should_pause)) = Self::accumulate_result_with_timings( + &context, &mut result_with_timings, - executed_task.expect("alive handler"), + executed_task, ) else { break 'nonaborted_main_loop; }; state_machine.deschedule_task(&executed_task.task); + if should_pause && !session_ending { + let task = banking_stage_context.as_ref().unwrap().adapter.recreate_task( + executed_task.into_inner(), + ); + state_machine.do_schedule_task(task, true); + session_pausing = true; + } }, recv(dummy_unblocked_task_receiver) -> dummy => { assert_matches!(dummy, Err(RecvError)); @@ -1087,7 +1412,7 @@ impl, TH: TaskHandler> ThreadManager { runnable_task_sender.send_payload(task).unwrap(); }, recv(new_task_receiver) -> message => { - assert!(!session_ending); + assert!(context.mode() == BlockProduction || !session_ending); match message { Ok(NewTaskPayload::Payload(task)) => { @@ -1097,11 +1422,22 @@ impl, TH: TaskHandler> ThreadManager { } } Ok(NewTaskPayload::CloseSubchannel) => { - session_ending = true; + match context.mode() { + BlockVerification => { + session_ending = true; + }, + BlockProduction => { + session_pausing = true; + }, + } + } + Ok(NewTaskPayload::Reset) => { + session_pausing = true; + session_resetting = true; } Ok(NewTaskPayload::OpenSubchannel(_context_and_result_with_timings)) => unreachable!(), - Err(RecvError) => { + Ok(NewTaskPayload::Disconnect) | Err(RecvError) => { // Mostly likely is that this scheduler is dropped for pruned blocks of // abandoned forks... // This short-circuiting is tested with test_scheduler_drop_short_circuiting. @@ -1110,49 +1446,87 @@ impl, TH: TaskHandler> ThreadManager { } }, recv(finished_idle_task_receiver) -> executed_task => { - let Some(executed_task) = Self::accumulate_result_with_timings( + let Some((executed_task, should_pause)) = Self::accumulate_result_with_timings( + &context, &mut result_with_timings, executed_task.expect("alive handler"), ) else { break 'nonaborted_main_loop; }; state_machine.deschedule_task(&executed_task.task); + if should_pause && !session_ending { + let task = banking_stage_context.as_ref().unwrap().adapter.recreate_task( + executed_task.into_inner(), + ); + state_machine.do_schedule_task(task, true); + session_pausing = true; + } }, }; - is_finished = session_ending && state_machine.has_no_active_task(); + is_finished = session_ending && state_machine.has_no_active_task() + || session_pausing && state_machine.has_no_executing_task(); } + assert!(mem::replace(&mut is_finished, false)); // Finalize the current session after asserting it's explicitly requested so. - assert!(session_ending); + assert!(session_ending ^ session_pausing); // Send result first because this is blocking the replay code-path. session_result_sender .send(result_with_timings) .expect("always outlived receiver"); - state_machine.reinitialize(); - session_ending = false; - { + loop { + if session_resetting { + while let Some(task) = state_machine.schedule_next_unblocked_task() { + state_machine.deschedule_task(&task); + drop(task); + } + session_resetting = false; + } // Prepare for the new session. - match new_task_receiver.recv() { - Ok(NewTaskPayload::OpenSubchannel(context_and_result_with_timings)) => { + match (new_task_receiver.recv(), context.mode()) { + ( + Ok(NewTaskPayload::OpenSubchannel(context_and_result_with_timings)), + _, + ) => { let (new_context, new_result_with_timings) = *context_and_result_with_timings; // We just received subsequent (= not initial) session and about to // enter into the preceding `while(!is_finished) {...}` loop again. // Before that, propagate new SchedulingContext to handler threads + assert_eq!(context.mode(), new_context.mode()); + + if session_ending { + state_machine.reinitialize(); + session_ending = false; + } else { + session_pausing = false; + } + runnable_task_sender .send_chained_channel(&new_context, handler_count) .unwrap(); + context = new_context; result_with_timings = new_result_with_timings; + break; } - Err(_) => { + (Ok(NewTaskPayload::CloseSubchannel), BlockProduction) => { + info!("ignoring duplicate CloseSubchannel..."); + } + (Ok(NewTaskPayload::Reset), BlockProduction) => { + session_resetting = true; + } + (Ok(NewTaskPayload::Payload(task)), BlockProduction) => { + assert!(state_machine.do_schedule_task(task, true).is_none()); + } + (Ok(NewTaskPayload::Disconnect), BlockProduction) | (Err(_), _) => { // This unusual condition must be triggered by ThreadManager::drop(). // Initialize result_with_timings with a harmless value... result_with_timings = initialized_result_with_timings(); break 'nonaborted_main_loop; } - Ok(_) => unreachable!(), + (Ok(_), _) => unreachable!(), } } } @@ -1181,6 +1555,9 @@ impl, TH: TaskHandler> ThreadManager { }; let handler_main_loop = || { + let banking_stage_context = banking_stage_context.clone(); + let new_task_sender = Arc::downgrade(&self.new_task_sender); + let pool = self.pool.clone(); let mut runnable_task_receiver = runnable_task_receiver.clone(); let finished_blocked_task_sender = finished_blocked_task_sender.clone(); @@ -1195,6 +1572,12 @@ impl, TH: TaskHandler> ThreadManager { // `select_biased!`, which are sent from `.send_chained_channel()` in the scheduler // thread for all-but-initial sessions. move || { + let banking_packet_receiver = if let Some(b) = banking_stage_context.as_ref() { + &b.banking_packet_receiver + } else { + &never() + }; + loop { let (task, sender) = select_biased! { recv(runnable_task_receiver.for_select()) -> message => { @@ -1215,6 +1598,30 @@ impl, TH: TaskHandler> ThreadManager { continue; } }, + recv(banking_packet_receiver) -> banking_packet => { + let Some(new_task_sender) = new_task_sender.upgrade() else { + info!("dead new_task_sender"); + break; + }; + + let Ok(banking_packet) = banking_packet else { + info!("disconnected banking_packet_receiver"); + let current_thread = thread::current(); + if new_task_sender.send(NewTaskPayload::Disconnect).is_ok() { + info!("notified a disconnect from {:?}", current_thread); + } else { + // It seems that the scheduler thread has been aborted already... + warn!("failed to notify a disconnect from {:?}", current_thread); + } + break; + }; + (banking_stage_context.as_ref().unwrap().on_banking_packet_receive)(banking_packet, &move |task| { + new_task_sender + .send(NewTaskPayload::Payload(task)) + .unwrap(); + }); + continue; + }, }; defer! { if !thread::panicking() { @@ -1249,16 +1656,16 @@ impl, TH: TaskHandler> ThreadManager { self.scheduler_thread = Some( thread::Builder::new() - .name("solScheduler".to_owned()) + .name(format!("solSchedule{postfix}")) .spawn_tracked(scheduler_main_loop) .unwrap(), ); - self.handler_threads = (0..self.pool.handler_count) + self.handler_threads = (0..handler_count) .map({ |thx| { thread::Builder::new() - .name(format!("solScHandler{:02}", thx)) + .name(format!("solScHandle{postfix}{:02}", thx)) .spawn_tracked(handler_main_loop()) .unwrap() } @@ -1310,17 +1717,8 @@ impl, TH: TaskHandler> ThreadManager { }; } - fn ensure_join_threads_after_abort( - &mut self, - should_receive_aborted_session_result: bool, - ) -> TransactionError { + fn ensure_join_threads_after_abort(&mut self, should_receive_aborted_session_result: bool) { self.ensure_join_threads(should_receive_aborted_session_result); - self.session_result_with_timings - .as_mut() - .unwrap() - .0 - .clone() - .unwrap_err() } fn are_threads_joined(&self) -> bool { @@ -1334,7 +1732,7 @@ impl, TH: TaskHandler> ThreadManager { } } - fn end_session(&mut self) { + fn do_end_session(&mut self, nonblocking: bool) { if self.are_threads_joined() { assert!(self.session_result_with_timings.is_some()); debug!("end_session(): skipping; already joined the aborted threads.."); @@ -1358,6 +1756,10 @@ impl, TH: TaskHandler> ThreadManager { return; } + if nonblocking { + return; + } + // Even if abort is detected, it's guaranteed that the scheduler thread puts the last // message into the session_result_sender before terminating. let result_with_timings = self.session_result_receiver.recv().unwrap(); @@ -1369,6 +1771,10 @@ impl, TH: TaskHandler> ThreadManager { debug!("end_session(): ended session at {:?}...", thread::current()); } + fn end_session(&mut self) { + self.do_end_session(false) + } + fn start_session( &mut self, context: SchedulingContext, @@ -1383,11 +1789,18 @@ impl, TH: TaskHandler> ThreadManager { )))) .expect("no new session after aborted"); } + + fn disconnect_new_task_sender(&mut self) { + self.new_task_sender = Arc::new(crossbeam_channel::unbounded().0); + } } pub trait SchedulerInner { fn id(&self) -> SchedulerId; fn is_trashed(&self) -> bool; + fn is_overgrown(&self) -> bool; + fn reset(&self); + fn ensure_abort(&mut self); } pub trait SpawnableScheduler: InstalledScheduler { @@ -1402,9 +1815,11 @@ pub trait SpawnableScheduler: InstalledScheduler { ) -> Self; fn spawn( + handler_count: usize, pool: Arc>, context: SchedulingContext, result_with_timings: ResultWithTimings, + banking_stage_context: Option, ) -> Self where Self: Sized; @@ -1434,21 +1849,82 @@ impl SpawnableScheduler for PooledScheduler { } fn spawn( + handler_count: usize, pool: Arc>, context: SchedulingContext, result_with_timings: ResultWithTimings, + banking_stage_context: Option, ) -> Self { + info!("spawning new scheduler for slot: {:?}", context.slot()); + let task_creator = match context.mode() { + BlockVerification => TaskCreator::ForBlockVerification { + usage_queue_loader: UsageQueueLoader::default(), + }, + BlockProduction => TaskCreator::ForBlockProduction { + banking_stage_adapter: banking_stage_context.as_ref().unwrap().adapter.clone(), + }, + }; let mut inner = Self::Inner { thread_manager: ThreadManager::new(pool), - usage_queue_loader: UsageQueueLoader::default(), + task_creator, }; - inner - .thread_manager - .start_threads(context.clone(), result_with_timings); + inner.thread_manager.start_threads( + handler_count, + context.clone(), + result_with_timings, + banking_stage_context, + ); Self { inner, context } } } +#[derive(Debug)] +pub enum BankingStageStatus { + Active, + Inactive, + Exited, +} + +pub trait BankingStageMonitor: Send + Debug { + fn status(&self) -> BankingStageStatus; +} + +#[derive(Debug, Default)] +pub struct BankingStageAdapter { + usage_queue_loader: UsageQueueLoader, + next_task_id: AtomicUsize, +} + +impl BankingStageAdapter { + pub fn generate_task_ids(&self, count: usize) -> usize { + self.next_task_id.fetch_add(count, Relaxed) + } + + fn do_create_task( + &self, + transaction: RuntimeTransaction, + index: usize, + ) -> Task { + SchedulingStateMachine::do_create_task(transaction, index, &mut |pubkey| { + self.usage_queue_loader.load(pubkey) + }) + } + + pub fn create_new_task( + &self, + transaction: RuntimeTransaction, + index: usize, + ) -> Task { + self.do_create_task(transaction, index) + } + + fn recreate_task(&self, task: Task) -> Task { + let new_index = self.generate_task_ids(1); + let transaction = task.into_transaction(); + self.do_create_task(transaction, new_index) + } +} + impl InstalledScheduler for PooledScheduler { fn id(&self) -> SchedulerId { self.inner.id() @@ -1463,8 +1939,9 @@ impl InstalledScheduler for PooledScheduler { transaction: RuntimeTransaction, index: usize, ) -> ScheduleResult { + assert_matches!(self.context().mode(), BlockVerification); let task = SchedulingStateMachine::create_task(transaction, index, &mut |pubkey| { - self.inner.usage_queue_loader.load(pubkey) + self.inner.task_creator.usage_queue_loader().load(pubkey) }); self.inner.thread_manager.send_task(task) } @@ -1472,7 +1949,15 @@ impl InstalledScheduler for PooledScheduler { fn recover_error_after_abort(&mut self) -> TransactionError { self.inner .thread_manager - .ensure_join_threads_after_abort(true) + .ensure_join_threads_after_abort(true); + self.inner + .thread_manager + .session_result_with_timings + .as_mut() + .unwrap() + .0 + .clone() + .unwrap_err() } fn wait_for_termination( @@ -1484,13 +1969,16 @@ impl InstalledScheduler for PooledScheduler { } fn pause_for_recent_blockhash(&mut self) { - self.inner.thread_manager.end_session(); + // this fn is called from poh thread, while it's being locked. so, we can't wait scheduler + // termination here to avoid deadlock. just async signaling is enough + let nonblocking = matches!(self.context().mode(), BlockProduction); + self.inner.thread_manager.do_end_session(nonblocking); } } impl SchedulerInner for PooledSchedulerInner where - S: SpawnableScheduler, + S: SpawnableScheduler, TH: TaskHandler, { fn id(&self) -> SchedulerId { @@ -1500,6 +1988,28 @@ where fn is_trashed(&self) -> bool { self.is_aborted() || self.is_overgrown() } + + fn is_overgrown(&self) -> bool { + self.task_creator + .is_overgrown(self.thread_manager.pool.max_usage_queue_count) + } + + fn reset(&self) { + if let Err(a) = self + .thread_manager + .new_task_sender + .send(NewTaskPayload::Reset) + { + warn!("failed to send a reset due to error: {a:?}"); + } + } + + fn ensure_abort(&mut self) { + if self.thread_manager.are_threads_joined() { + return; + } + self.thread_manager.disconnect_new_task_sender() + } } impl UninstalledScheduler for PooledSchedulerInner @@ -1533,7 +2043,9 @@ mod tests { bank::Bank, bank_forks::BankForks, genesis_utils::{create_genesis_config, GenesisConfigInfo}, - installed_scheduler_pool::{BankWithScheduler, SchedulingContext}, + installed_scheduler_pool::{ + BankWithScheduler, InstalledSchedulerPoolArc, SchedulingContext, + }, prioritization_fee_cache::PrioritizationFeeCache, }, solana_system_transaction as system_transaction, @@ -1547,6 +2059,58 @@ mod tests { test_case::test_matrix, }; + impl SchedulerPool + where + S: SpawnableScheduler, + TH: TaskHandler, + { + fn do_new_for_verification( + handler_count: Option, + log_messages_bytes_limit: Option, + transaction_status_sender: Option, + replay_vote_sender: Option, + prioritization_fee_cache: Arc, + pool_cleaner_interval: Duration, + max_pooling_duration: Duration, + max_usage_queue_count: usize, + timeout_duration: Duration, + ) -> Arc { + Self::do_new( + SupportedSchedulingMode::block_verification_only(), + handler_count, + log_messages_bytes_limit, + transaction_status_sender, + replay_vote_sender, + prioritization_fee_cache, + TransactionRecorder::new_dummy(), + pool_cleaner_interval, + max_pooling_duration, + max_usage_queue_count, + timeout_duration, + ) + } + + // This apparently-meaningless wrapper is handy, because some callers explicitly want + // `dyn InstalledSchedulerPool` to be returned for type inference convenience. + fn new_dyn_for_verification( + handler_count: Option, + log_messages_bytes_limit: Option, + transaction_status_sender: Option, + replay_vote_sender: Option, + prioritization_fee_cache: Arc, + ) -> InstalledSchedulerPoolArc { + Self::new( + SupportedSchedulingMode::block_verification_only(), + handler_count, + log_messages_bytes_limit, + transaction_status_sender, + replay_vote_sender, + prioritization_fee_cache, + TransactionRecorder::new_dummy(), + ) + } + } + #[derive(Debug)] enum TestCheckPoint { BeforeNewTask, @@ -1567,8 +2131,13 @@ mod tests { solana_logger::setup(); let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); - let pool = - DefaultSchedulerPool::new_dyn(None, None, None, None, ignored_prioritization_fee_cache); + let pool = DefaultSchedulerPool::new_dyn_for_verification( + None, + None, + None, + None, + ignored_prioritization_fee_cache, + ); // this indirectly proves that there should be circular link because there's only one Arc // at this moment now @@ -1583,11 +2152,16 @@ mod tests { solana_logger::setup(); let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); - let pool = - DefaultSchedulerPool::new_dyn(None, None, None, None, ignored_prioritization_fee_cache); + let pool = DefaultSchedulerPool::new_dyn_for_verification( + None, + None, + None, + None, + ignored_prioritization_fee_cache, + ); let bank = Arc::new(Bank::default_for_tests()); - let context = SchedulingContext::new(bank); - let scheduler = pool.take_scheduler(context); + let context = SchedulingContext::for_verification(bank); + let scheduler = pool.take_scheduler(context).unwrap(); let debug = format!("{scheduler:#?}"); assert!(!debug.is_empty()); @@ -1608,7 +2182,7 @@ mod tests { ]); let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); - let pool_raw = DefaultSchedulerPool::do_new( + let pool_raw = DefaultSchedulerPool::do_new_for_verification( None, None, None, @@ -1621,7 +2195,7 @@ mod tests { ); let pool = pool_raw.clone(); let bank = Arc::new(Bank::default_for_tests()); - let context1 = SchedulingContext::new(bank); + let context1 = SchedulingContext::for_verification(bank); let context2 = context1.clone(); let old_scheduler = pool.do_take_scheduler(context1); @@ -1673,7 +2247,7 @@ mod tests { let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); const REDUCED_MAX_USAGE_QUEUE_COUNT: usize = 1; - let pool_raw = DefaultSchedulerPool::do_new( + let pool_raw = DefaultSchedulerPool::do_new_for_verification( None, None, None, @@ -1686,7 +2260,7 @@ mod tests { ); let pool = pool_raw.clone(); let bank = Arc::new(Bank::default_for_tests()); - let context1 = SchedulingContext::new(bank); + let context1 = SchedulingContext::for_verification(bank); let context2 = context1.clone(); let small_scheduler = pool.do_take_scheduler(context1); @@ -1694,14 +2268,16 @@ mod tests { for _ in 0..REDUCED_MAX_USAGE_QUEUE_COUNT { small_scheduler .inner - .usage_queue_loader + .task_creator + .usage_queue_loader() .load(Pubkey::new_unique()); } let big_scheduler = pool.do_take_scheduler(context2); for _ in 0..REDUCED_MAX_USAGE_QUEUE_COUNT + 1 { big_scheduler .inner - .usage_queue_loader + .task_creator + .usage_queue_loader() .load(Pubkey::new_unique()); } @@ -1748,7 +2324,7 @@ mod tests { ]); let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); - let pool_raw = DefaultSchedulerPool::do_new( + let pool_raw = DefaultSchedulerPool::do_new_for_verification( None, None, None, @@ -1761,8 +2337,8 @@ mod tests { ); let pool = pool_raw.clone(); let bank = Arc::new(Bank::default_for_tests()); - let context = SchedulingContext::new(bank.clone()); - let scheduler = pool.take_scheduler(context); + let context = SchedulingContext::for_verification(bank.clone()); + let scheduler = pool.take_scheduler(context).unwrap(); let bank = BankWithScheduler::new(bank, Some(scheduler)); pool.register_timeout_listener(bank.create_timeout_listener()); assert_eq!(pool_raw.scheduler_inners.lock().unwrap().len(), 0); @@ -1796,17 +2372,18 @@ mod tests { ]); let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); - let pool_raw = SchedulerPool::, _>::do_new( - None, - None, - None, - None, - ignored_prioritization_fee_cache, - SHORTENED_POOL_CLEANER_INTERVAL, - DEFAULT_MAX_POOLING_DURATION, - DEFAULT_MAX_USAGE_QUEUE_COUNT, - SHORTENED_TIMEOUT_DURATION, - ); + let pool_raw = + SchedulerPool::, _>::do_new_for_verification( + None, + None, + None, + None, + ignored_prioritization_fee_cache, + SHORTENED_POOL_CLEANER_INTERVAL, + DEFAULT_MAX_POOLING_DURATION, + DEFAULT_MAX_USAGE_QUEUE_COUNT, + SHORTENED_TIMEOUT_DURATION, + ); #[derive(Debug)] struct ExecuteTimingCounter; @@ -1831,9 +2408,9 @@ mod tests { let bank = Bank::new_for_tests(&genesis_config); let (bank, _bank_forks) = setup_dummy_fork_graph(bank); - let context = SchedulingContext::new(bank.clone()); + let context = SchedulingContext::for_verification(bank.clone()); - let scheduler = pool.take_scheduler(context); + let scheduler = pool.take_scheduler(context).unwrap(); let bank = BankWithScheduler::new(bank, Some(scheduler)); pool.register_timeout_listener(bank.create_timeout_listener()); @@ -1882,7 +2459,7 @@ mod tests { ]); let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); - let pool_raw = DefaultSchedulerPool::do_new( + let pool_raw = DefaultSchedulerPool::do_new_for_verification( None, None, None, @@ -1899,9 +2476,9 @@ mod tests { let bank = Bank::new_for_tests(&genesis_config); let (bank, _bank_forks) = setup_dummy_fork_graph(bank); - let context = SchedulingContext::new(bank.clone()); + let context = SchedulingContext::for_verification(bank.clone()); - let scheduler = pool.take_scheduler(context); + let scheduler = pool.take_scheduler(context).unwrap(); let bank = BankWithScheduler::new(bank, Some(scheduler)); pool.register_timeout_listener(bank.create_timeout_listener()); @@ -1929,7 +2506,7 @@ mod tests { ]); let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); - let pool_raw = SchedulerPool::, _>::do_new( + let pool_raw = SchedulerPool::, _>::do_new_for_verification( None, None, None, @@ -1951,9 +2528,9 @@ mod tests { let bank = Bank::new_for_tests(&genesis_config); let (bank, _bank_forks) = setup_dummy_fork_graph(bank); - let context = SchedulingContext::new(bank.clone()); + let context = SchedulingContext::for_verification(bank.clone()); - let scheduler = pool.take_scheduler(context); + let scheduler = pool.take_scheduler(context).unwrap(); let bank = BankWithScheduler::new(bank, Some(scheduler)); pool.register_timeout_listener(bank.create_timeout_listener()); @@ -2031,14 +2608,14 @@ mod tests { let bank = Bank::new_for_tests(&genesis_config); let (bank, _bank_forks) = setup_dummy_fork_graph(bank); let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); - let pool = SchedulerPool::, _>::new( + let pool = SchedulerPool::, _>::new_for_verification( None, None, None, None, ignored_prioritization_fee_cache, ); - let context = SchedulingContext::new(bank.clone()); + let context = SchedulingContext::for_verification(bank.clone()); let scheduler = pool.do_take_scheduler(context); scheduler.schedule_execution(tx, 0).unwrap(); @@ -2123,14 +2700,14 @@ mod tests { let bank = Bank::new_for_tests(&genesis_config); let (bank, _bank_forks) = setup_dummy_fork_graph(bank); let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); - let pool = SchedulerPool::, _>::new( + let pool = SchedulerPool::, _>::new_for_verification( None, None, None, None, ignored_prioritization_fee_cache, ); - let context = SchedulingContext::new(bank.clone()); + let context = SchedulingContext::for_verification(bank.clone()); let scheduler = pool.do_take_scheduler(context); // This test is racy. @@ -2163,10 +2740,15 @@ mod tests { solana_logger::setup(); let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); - let pool = - DefaultSchedulerPool::new(None, None, None, None, ignored_prioritization_fee_cache); + let pool = DefaultSchedulerPool::new_for_verification( + None, + None, + None, + None, + ignored_prioritization_fee_cache, + ); let bank = Arc::new(Bank::default_for_tests()); - let context = &SchedulingContext::new(bank); + let context = &SchedulingContext::for_verification(bank); let scheduler1 = pool.do_take_scheduler(context.clone()); let scheduler_id1 = scheduler1.id(); @@ -2192,10 +2774,15 @@ mod tests { solana_logger::setup(); let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); - let pool = - DefaultSchedulerPool::new(None, None, None, None, ignored_prioritization_fee_cache); + let pool = DefaultSchedulerPool::new_for_verification( + None, + None, + None, + None, + ignored_prioritization_fee_cache, + ); let bank = Arc::new(Bank::default_for_tests()); - let context = &SchedulingContext::new(bank); + let context = &SchedulingContext::for_verification(bank); let mut scheduler = pool.do_take_scheduler(context.clone()); // should never panic. @@ -2211,20 +2798,25 @@ mod tests { solana_logger::setup(); let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); - let pool = - DefaultSchedulerPool::new(None, None, None, None, ignored_prioritization_fee_cache); + let pool = DefaultSchedulerPool::new_for_verification( + None, + None, + None, + None, + ignored_prioritization_fee_cache, + ); let old_bank = &Arc::new(Bank::default_for_tests()); let new_bank = &Arc::new(Bank::default_for_tests()); assert!(!Arc::ptr_eq(old_bank, new_bank)); - let old_context = &SchedulingContext::new(old_bank.clone()); - let new_context = &SchedulingContext::new(new_bank.clone()); + let old_context = &SchedulingContext::for_verification(old_bank.clone()); + let new_context = &SchedulingContext::for_verification(new_bank.clone()); let scheduler = pool.do_take_scheduler(old_context.clone()); let scheduler_id = scheduler.id(); pool.return_scheduler(scheduler.into_inner().1); - let scheduler = pool.take_scheduler(new_context.clone()); + let scheduler = pool.take_scheduler(new_context.clone()).unwrap(); assert_eq!(scheduler_id, scheduler.id()); assert!(Arc::ptr_eq(scheduler.context().bank(), new_bank)); } @@ -2237,8 +2829,13 @@ mod tests { let bank_forks = BankForks::new_rw_arc(bank); let mut bank_forks = bank_forks.write().unwrap(); let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); - let pool = - DefaultSchedulerPool::new_dyn(None, None, None, None, ignored_prioritization_fee_cache); + let pool = DefaultSchedulerPool::new_dyn_for_verification( + None, + None, + None, + None, + ignored_prioritization_fee_cache, + ); bank_forks.install_scheduler_pool(pool); } @@ -2251,8 +2848,13 @@ mod tests { let child_bank = Bank::new_from_parent(bank, &Pubkey::default(), 1); let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); - let pool = - DefaultSchedulerPool::new_dyn(None, None, None, None, ignored_prioritization_fee_cache); + let pool = DefaultSchedulerPool::new_dyn_for_verification( + None, + None, + None, + None, + ignored_prioritization_fee_cache, + ); let bank = Bank::default_for_tests(); let bank_forks = BankForks::new_rw_arc(bank); @@ -2301,12 +2903,17 @@ mod tests { let bank = Bank::new_for_tests(&genesis_config); let (bank, _bank_forks) = setup_dummy_fork_graph(bank); let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); - let pool = - DefaultSchedulerPool::new_dyn(None, None, None, None, ignored_prioritization_fee_cache); - let context = SchedulingContext::new(bank.clone()); + let pool = DefaultSchedulerPool::new_dyn_for_verification( + None, + None, + None, + None, + ignored_prioritization_fee_cache, + ); + let context = SchedulingContext::for_verification(bank.clone()); assert_eq!(bank.transaction_count(), 0); - let scheduler = pool.take_scheduler(context); + let scheduler = pool.take_scheduler(context).unwrap(); scheduler.schedule_execution(tx0, 0).unwrap(); let bank = BankWithScheduler::new(bank, Some(scheduler)); assert_matches!(bank.wait_for_completed_scheduler(), Some((Ok(()), _))); @@ -2336,7 +2943,7 @@ mod tests { let (bank, _bank_forks) = setup_dummy_fork_graph(bank); let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); - let pool_raw = DefaultSchedulerPool::do_new( + let pool_raw = DefaultSchedulerPool::do_new_for_verification( None, None, None, @@ -2348,8 +2955,8 @@ mod tests { DEFAULT_TIMEOUT_DURATION, ); let pool = pool_raw.clone(); - let context = SchedulingContext::new(bank.clone()); - let scheduler = pool.take_scheduler(context); + let context = SchedulingContext::for_verification(bank.clone()); + let scheduler = pool.take_scheduler(context).unwrap(); let unfunded_keypair = Keypair::new(); let bad_tx = RuntimeTransaction::from_transaction_for_tests(system_transaction::transfer( @@ -2469,16 +3076,16 @@ mod tests { const TX_COUNT: usize = 2; let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); - let pool = SchedulerPool::, _>::new_dyn( + let pool = SchedulerPool::, _>::new_dyn_for_verification( Some(TX_COUNT), // fix to use exactly 2 handlers None, None, None, ignored_prioritization_fee_cache, ); - let context = SchedulingContext::new(bank.clone()); + let context = SchedulingContext::for_verification(bank.clone()); - let scheduler = pool.take_scheduler(context); + let scheduler = pool.take_scheduler(context).unwrap(); for index in 0..TX_COUNT { // Use 2 non-conflicting txes to exercise the channel disconnected case as well. @@ -2544,14 +3151,14 @@ mod tests { let bank = Bank::new_for_tests(&genesis_config); let (bank, _bank_forks) = setup_dummy_fork_graph(bank); let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); - let pool = SchedulerPool::, _>::new( + let pool = SchedulerPool::, _>::new_for_verification( None, None, None, None, ignored_prioritization_fee_cache, ); - let context = SchedulingContext::new(bank.clone()); + let context = SchedulingContext::for_verification(bank.clone()); let scheduler = pool.do_take_scheduler(context); for i in 0..10 { @@ -2628,17 +3235,17 @@ mod tests { let bank = Bank::new_for_tests(&genesis_config); let (bank, _bank_forks) = setup_dummy_fork_graph(bank); let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); - let pool = SchedulerPool::, _>::new_dyn( + let pool = SchedulerPool::, _>::new_dyn_for_verification( None, None, None, None, ignored_prioritization_fee_cache, ); - let context = SchedulingContext::new(bank.clone()); + let context = SchedulingContext::for_verification(bank.clone()); assert_eq!(bank.transaction_count(), 0); - let scheduler = pool.take_scheduler(context); + let scheduler = pool.take_scheduler(context).unwrap(); // Stall handling tx0 and tx1 let lock_to_stall = LOCK_TO_STALL.lock().unwrap(); @@ -2694,7 +3301,7 @@ mod tests { )); let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); - let pool = SchedulerPool::, _>::new( + let pool = SchedulerPool::, _>::new_for_verification( Some(4), // spawn 4 threads None, None, @@ -2710,8 +3317,8 @@ mod tests { 2, genesis_config.hash(), )); - let context0 = &SchedulingContext::new(bank0.clone()); - let context1 = &SchedulingContext::new(bank1.clone()); + let context0 = &SchedulingContext::for_verification(bank0.clone()); + let context1 = &SchedulingContext::for_verification(bank1.clone()); // Exercise the scheduler by busy-looping to expose the race condition for (context, index) in [(context0, 0), (context1, 1)] @@ -2719,7 +3326,7 @@ mod tests { .cycle() .take(10000) { - let scheduler = pool.take_scheduler(context.clone()); + let scheduler = pool.take_scheduler(context.clone()).unwrap(); scheduler .schedule_execution(dummy_tx.clone(), index) .unwrap(); @@ -2829,6 +3436,18 @@ mod tests { fn is_trashed(&self) -> bool { false } + + fn is_overgrown(&self) -> bool { + todo!() + } + + fn reset(&self) { + todo!() + } + + fn ensure_abort(&mut self) { + todo!() + } } impl UninstalledScheduler @@ -2858,9 +3477,11 @@ mod tests { } fn spawn( + _handler_count: usize, pool: Arc>, context: SchedulingContext, _result_with_timings: ResultWithTimings, + _banking_stage_context: Option, ) -> Self { AsyncScheduler::( Mutex::new(initialized_result_with_timings()), @@ -2900,18 +3521,18 @@ mod tests { ); } let (bank, _bank_forks) = setup_dummy_fork_graph(bank); - let context = SchedulingContext::new(bank.clone()); + let context = SchedulingContext::for_verification(bank.clone()); let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); let pool = - SchedulerPool::, DefaultTaskHandler>::new_dyn( + SchedulerPool::, DefaultTaskHandler>::new_dyn_for_verification( None, None, None, None, ignored_prioritization_fee_cache, ); - let scheduler = pool.take_scheduler(context); + let scheduler = pool.take_scheduler(context).unwrap(); let bank = BankWithScheduler::new(bank, Some(scheduler)); assert_eq!(bank.transaction_count(), 0); @@ -2986,13 +3607,13 @@ mod tests { let result = &mut Ok(()); let timings = &mut ExecuteTimings::default(); let prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); - let scheduling_context = &SchedulingContext::new(bank.clone()); + let scheduling_context = &SchedulingContext::for_verification(bank.clone()); let handler_context = &HandlerContext { log_messages_bytes_limit: None, transaction_status_sender: None, replay_vote_sender: None, prioritization_fee_cache, - transaction_recorder: None, + transaction_recorder: TransactionRecorder::new_dummy(), }; let task = SchedulingStateMachine::create_task(tx, 0, &mut |_| UsageQueue::default()); @@ -3073,7 +3694,7 @@ mod tests { transaction_status_sender: Some(TransactionStatusSender { sender }), replay_vote_sender: None, prioritization_fee_cache, - transaction_recorder: Some(poh_recorder.read().unwrap().new_recorder()), + transaction_recorder: poh_recorder.read().unwrap().new_recorder(), }; let task = diff --git a/validator/src/cli.rs b/validator/src/cli.rs index 2efa6317bf21f8..f68da7574fc123 100644 --- a/validator/src/cli.rs +++ b/validator/src/cli.rs @@ -1607,6 +1607,15 @@ pub fn app<'a>(version: &'a str, default_args: &'a DefaultArgs) -> App<'a, 'a> { .validator(|s| is_within_range(s, 1..)) .help(DefaultSchedulerPool::cli_message()), ) + .arg( + Arg::with_name("enable_experimental_block_production_method") + .long("enable-experimental-block-production-method") + .takes_value(false) + .help( + "Accept unified-scheduler to be used as an experimental block \ + production method", + ), + ) .arg( Arg::with_name("wen_restart") .long("wen-restart") diff --git a/validator/src/main.rs b/validator/src/main.rs index 252f68689d4680..f2809757ae1e87 100644 --- a/validator/src/main.rs +++ b/validator/src/main.rs @@ -1859,6 +1859,17 @@ pub fn main() { "block_production_method", BlockProductionMethod ) + .inspect(|method| { + if matches!(method, BlockProductionMethod::UnifiedScheduler) + && !matches.is_present("enable_experimental_block_production_method") + { + eprintln!( + "Currently, the unified-scheduler method is experimental for block-production. \ + Explicitly pass --enable-experimental-block-production-method to use it." + ); + exit(1); + } + }) .unwrap_or_default(); validator_config.enable_block_production_forwarding = staked_nodes_overrides_path.is_some(); validator_config.unified_scheduler_handler_threads = From 038559e8403ba4293f9ff98fee8ef1c7514fc0c7 Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Wed, 11 Dec 2024 07:15:10 +0000 Subject: [PATCH 02/25] Move SchedulingMode out of solana-sdk --- core/Cargo.toml | 1 - programs/sbf/Cargo.lock | 1 + sdk/src/scheduling.rs | 8 -------- svm/examples/Cargo.lock | 1 + unified-scheduler-logic/src/lib.rs | 2 +- 5 files changed, 3 insertions(+), 10 deletions(-) delete mode 100644 sdk/src/scheduling.rs diff --git a/core/Cargo.toml b/core/Cargo.toml index 166dbd252c85da..6310529603f221 100644 --- a/core/Cargo.toml +++ b/core/Cargo.toml @@ -124,7 +124,6 @@ solana-program-runtime = { workspace = true, features = ["metrics"] } solana-sdk = { workspace = true, features = ["dev-context-only-utils"] } solana-stake-program = { workspace = true } solana-system-program = { workspace = true } -solana-unified-scheduler-logic = { workspace = true } solana-unified-scheduler-pool = { workspace = true, features = [ "dev-context-only-utils", ] } diff --git a/programs/sbf/Cargo.lock b/programs/sbf/Cargo.lock index 96768f54395ea7..d0ecad6425ac9b 100644 --- a/programs/sbf/Cargo.lock +++ b/programs/sbf/Cargo.lock @@ -5611,6 +5611,7 @@ dependencies = [ "solana-tpu-client", "solana-transaction-status", "solana-turbine", + "solana-unified-scheduler-logic", "solana-unified-scheduler-pool", "solana-version", "solana-vote", diff --git a/sdk/src/scheduling.rs b/sdk/src/scheduling.rs deleted file mode 100644 index ebab21a912f292..00000000000000 --- a/sdk/src/scheduling.rs +++ /dev/null @@ -1,8 +0,0 @@ -//! Primitive types relevant to transaction scheduling -#![cfg(feature = "full")] - -#[derive(Debug, Clone, Copy, PartialEq)] -pub enum SchedulingMode { - BlockVerification, - BlockProduction, -} diff --git a/svm/examples/Cargo.lock b/svm/examples/Cargo.lock index 2a4548a254100b..148b231a7c36b3 100644 --- a/svm/examples/Cargo.lock +++ b/svm/examples/Cargo.lock @@ -5462,6 +5462,7 @@ dependencies = [ "solana-tpu-client", "solana-transaction-status", "solana-turbine", + "solana-unified-scheduler-logic", "solana-unified-scheduler-pool", "solana-version", "solana-vote", diff --git a/unified-scheduler-logic/src/lib.rs b/unified-scheduler-logic/src/lib.rs index 682fd037259e73..6487d716ad8ecb 100644 --- a/unified-scheduler-logic/src/lib.rs +++ b/unified-scheduler-logic/src/lib.rs @@ -105,7 +105,7 @@ use { std::{collections::VecDeque, mem, sync::Arc}, }; -#[derive(Clone, Copy, Debug, PartialEq)] +#[derive(Debug, Clone, Copy, PartialEq)] pub enum SchedulingMode { BlockVerification, BlockProduction, From 52275461daf2e7b1f0c1703fb87659b5b557cc8f Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Sat, 14 Dec 2024 00:07:22 +0900 Subject: [PATCH 03/25] Remove needless type equality binds --- unified-scheduler-pool/src/lib.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/unified-scheduler-pool/src/lib.rs b/unified-scheduler-pool/src/lib.rs index 68be2547b4fd91..696ac4af8eb819 100644 --- a/unified-scheduler-pool/src/lib.rs +++ b/unified-scheduler-pool/src/lib.rs @@ -1041,7 +1041,7 @@ where impl PooledSchedulerInner where - S: SpawnableScheduler, + S: SpawnableScheduler, TH: TaskHandler, { fn is_aborted(&self) -> bool { @@ -1978,7 +1978,7 @@ impl InstalledScheduler for PooledScheduler { impl SchedulerInner for PooledSchedulerInner where - S: SpawnableScheduler, + S: SpawnableScheduler, TH: TaskHandler, { fn id(&self) -> SchedulerId { From 7e8feaa55011a57e066d737344cb51025ef17f84 Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Fri, 13 Dec 2024 23:37:40 +0900 Subject: [PATCH 04/25] Move is_trashed() to SchedulerInner --- unified-scheduler-pool/src/lib.rs | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/unified-scheduler-pool/src/lib.rs b/unified-scheduler-pool/src/lib.rs index 696ac4af8eb819..579ac099204ca8 100644 --- a/unified-scheduler-pool/src/lib.rs +++ b/unified-scheduler-pool/src/lib.rs @@ -426,6 +426,9 @@ where // _trashed_ and the interaction among different parts of unified scheduler. let should_trash = scheduler.is_trashed(); let id = scheduler.id(); + if should_trash { + info!("trashing scheduler (id: {})...", scheduler.id()); + } debug!("return_scheduler(): id: {id} should_trash: {should_trash}"); let mut id_and_inner = self.block_production_scheduler_inner.lock().unwrap(); let is_block_production_scheduler_returned = Some(id) == id_and_inner.0.as_ref().copied(); @@ -1994,6 +1997,10 @@ where .is_overgrown(self.thread_manager.pool.max_usage_queue_count) } + fn is_trashed(&self) -> bool { + self.is_aborted() || self.is_overgrown() + } + fn reset(&self) { if let Err(a) = self .thread_manager @@ -3441,6 +3448,10 @@ mod tests { todo!() } + fn is_trashed(&self) -> bool { + todo!() + } + fn reset(&self) { todo!() } From 256c21be7f2a78a17458e3dbb7118143658e9e37 Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Sat, 14 Dec 2024 11:50:46 +0000 Subject: [PATCH 05/25] Fix CI --- unified-scheduler-pool/src/lib.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/unified-scheduler-pool/src/lib.rs b/unified-scheduler-pool/src/lib.rs index 579ac099204ca8..b773997de53ee9 100644 --- a/unified-scheduler-pool/src/lib.rs +++ b/unified-scheduler-pool/src/lib.rs @@ -3445,19 +3445,19 @@ mod tests { } fn is_overgrown(&self) -> bool { - todo!() + unimplemented!() } fn is_trashed(&self) -> bool { - todo!() + false } fn reset(&self) { - todo!() + unimplemented!() } fn ensure_abort(&mut self) { - todo!() + unimplemented!() } } From 5633cd600b38af3f75dbbc924ed6686f833fdd96 Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Sat, 14 Dec 2024 22:18:29 +0900 Subject: [PATCH 06/25] Improve bank_forks/unified scheduler docs in core --- core/src/banking_stage.rs | 4 ++-- core/src/validator.rs | 12 ++++++++++++ 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/core/src/banking_stage.rs b/core/src/banking_stage.rs index e617b31d45ba41..53b9ee8251a9cb 100644 --- a/core/src/banking_stage.rs +++ b/core/src/banking_stage.rs @@ -808,8 +808,8 @@ pub(crate) fn update_bank_forks_and_poh_recorder_for_new_tpu_bank( // A write lock for the poh recorder must be grabbed for the entire duration of inserting new // tpu bank into the bank forks. That's because any buffered transactions could immediately be // executed after the bank forks update, when unified scheduler is enabled for block - // production. And then, the unified scheduler would be hit with false errors due to having no - // bank in the poh recorder otherwise. + // production. If transactions were executed prematurely, the unified scheduler would be hit + // with false errors due to having no bank in the poh recorder otherwise. let mut poh_recorder = poh_recorder.write().unwrap(); let tpu_bank = bank_forks diff --git a/core/src/validator.rs b/core/src/validator.rs index 2bc08a29a23d48..54405170b33f28 100644 --- a/core/src/validator.rs +++ b/core/src/validator.rs @@ -1428,6 +1428,18 @@ impl Validator { let cluster_slots = Arc::new(crate::cluster_slots_service::cluster_slots::ClusterSlots::default()); + // bank_forks could be write-locked temporarily here, if unified scheduler is enabled for + // block production. That's because the started ReplayStage inside Tvu could immediately + // try to insert a tpu bank into bank_forks, like with local development clusters + // consisting of a single staked node. Such insertion could be blocked with the lock held + // by unified scheduler until it's fully setup by the banking stage. This is needed, + // because completion of insertion needs to strictly correspond with the initiation of + // producing blocks in unified scheduler. + // As a consequence of this, there's a corner case where the banking stage hasn't called + // register_banking_stage() yet for unified scheduler because Tpu setup follows Tvu setup. + // This means any setup which accesses bank_forks must be done before or after the short + // duration of Tvu setup and Tpu setup to avoid deadlocks. So, RootBankCache must be setup + // here in advance as being one of such setups. let root_bank_cache = RootBankCache::new(bank_forks.clone()); let tvu = Tvu::new( vote_account, From 7e0926ec8548d2feb9645f3ad4b131e99712ed09 Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Sat, 14 Dec 2024 22:21:21 +0900 Subject: [PATCH 07/25] Clean do_take_resumed_scheduler() a bit --- unified-scheduler-pool/src/lib.rs | 61 +++++++++++++++++-------------- 1 file changed, 34 insertions(+), 27 deletions(-) diff --git a/unified-scheduler-pool/src/lib.rs b/unified-scheduler-pool/src/lib.rs index b773997de53ee9..ea0486771628d3 100644 --- a/unified-scheduler-pool/src/lib.rs +++ b/unified-scheduler-pool/src/lib.rs @@ -485,35 +485,42 @@ where ) -> S { assert_matches!(result_with_timings, (Ok(_), _)); - if matches!(context.mode(), BlockVerification) { - // pop is intentional for filo, expecting relatively warmed-up scheduler due to having been - // returned recently - if let Some((inner, _pooled_at)) = - self.scheduler_inners.lock().expect("not poisoned").pop() - { + match context.mode() { + BlockVerification => { + // pop is intentional for filo, expecting relatively warmed-up scheduler due to having been + // returned recently + if let Some((inner, _pooled_at)) = + self.scheduler_inners.lock().expect("not poisoned").pop() + { + S::from_inner(inner, context, result_with_timings) + } else { + S::spawn( + self.block_verification_handler_count, + self.self_arc(), + context, + result_with_timings, + None, + ) + } + } + BlockProduction => { + let id_and_inner = self + .block_production_scheduler_inner + .lock() + .expect("not poisoned"); + // Wait with condvar until scheduler is spawned elsewhere, which is guaranteed to + // be done eventually. + let (id, inner) = &mut *self + .block_production_scheduler_condvar + .wait_while(id_and_inner, |id_and_inner| id_and_inner.0.is_none()) + .unwrap(); + + let Some(inner) = inner.take() else { + panic!("already taken: id: {id:?} slot: {:?}", context.slot()); + }; + S::from_inner(inner, context, result_with_timings) - } else { - S::spawn( - self.block_verification_handler_count, - self.self_arc(), - context, - result_with_timings, - None, - ) } - } else { - let mut id_and_inner = self - .block_production_scheduler_inner - .lock() - .expect("not poisoned"); - id_and_inner = self - .block_production_scheduler_condvar - .wait_while(id_and_inner, |id_and_inner| id_and_inner.0.is_none()) - .unwrap(); - let Some(inner) = id_and_inner.1.take() else { - panic!("double take: {:?}, {:?}", context.slot(), context.mode()); - }; - S::from_inner(inner, context, result_with_timings) } } From ff28823eb3ddf14695b024288a2a1a0aaf75a138 Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Sun, 15 Dec 2024 15:10:21 +0900 Subject: [PATCH 08/25] Reword a bit --- core/src/validator.rs | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/core/src/validator.rs b/core/src/validator.rs index 54405170b33f28..fa2e1404d47420 100644 --- a/core/src/validator.rs +++ b/core/src/validator.rs @@ -1429,17 +1429,17 @@ impl Validator { Arc::new(crate::cluster_slots_service::cluster_slots::ClusterSlots::default()); // bank_forks could be write-locked temporarily here, if unified scheduler is enabled for - // block production. That's because the started ReplayStage inside Tvu could immediately - // try to insert a tpu bank into bank_forks, like with local development clusters - // consisting of a single staked node. Such insertion could be blocked with the lock held - // by unified scheduler until it's fully setup by the banking stage. This is needed, - // because completion of insertion needs to strictly correspond with the initiation of - // producing blocks in unified scheduler. - // As a consequence of this, there's a corner case where the banking stage hasn't called - // register_banking_stage() yet for unified scheduler because Tpu setup follows Tvu setup. - // This means any setup which accesses bank_forks must be done before or after the short - // duration of Tvu setup and Tpu setup to avoid deadlocks. So, RootBankCache must be setup - // here in advance as being one of such setups. + // block production. That's because ReplayStage started inside Tvu could immediately try to + // insert a tpu bank into bank_forks, like with local development clusters consisting of a + // single staked node. Such insertion could be blocked with the lock held by unified + // scheduler until it's fully setup by the banking stage. This is intentional because + // completion of insertion needs to strictly correspond with the initiation of producing + // blocks in unified scheduler. Because Tpu setup follows Tvu setup, there's a corner case + // where the banking stage hasn't yet called register_banking_stage() to finish the unified + // scheduler setup. + // This means any setup which accesses bank_forks must be done before + // or after the short duration of Tvu and Tpu setups to avoid deadlocks. So, + // RootBankCache needs to be created here in advance as being one of such setups. let root_bank_cache = RootBankCache::new(bank_forks.clone()); let tvu = Tvu::new( vote_account, From cf8d658c3b1485d4c7354eeec7b78b0515a2d990 Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Tue, 17 Dec 2024 23:31:30 +0900 Subject: [PATCH 09/25] Revert "Create RootBankCache earlier for unified scheduler" This reverts commit 736b2d052398244a75bf9baed14ca0dbc32f8222. --- core/src/cluster_info_vote_listener.rs | 2 +- core/src/tpu.rs | 3 --- core/src/validator.rs | 15 --------------- 3 files changed, 1 insertion(+), 19 deletions(-) diff --git a/core/src/cluster_info_vote_listener.rs b/core/src/cluster_info_vote_listener.rs index fa5c813086f294..1f82c765741cb7 100644 --- a/core/src/cluster_info_vote_listener.rs +++ b/core/src/cluster_info_vote_listener.rs @@ -195,7 +195,6 @@ impl ClusterInfoVoteListener { verified_packets_sender: BankingPacketSender, vote_tracker: Arc, bank_forks: Arc>, - mut root_bank_cache: RootBankCache, subscriptions: Arc, verified_vote_sender: VerifiedVoteSender, gossip_verified_vote_hash_sender: GossipVerifiedVoteHashSender, @@ -207,6 +206,7 @@ impl ClusterInfoVoteListener { let (verified_vote_transactions_sender, verified_vote_transactions_receiver) = unbounded(); let listen_thread = { let exit = exit.clone(); + let mut root_bank_cache = RootBankCache::new(bank_forks.clone()); Builder::new() .name("solCiVoteLstnr".to_string()) .spawn(move || { diff --git a/core/src/tpu.rs b/core/src/tpu.rs index d2b1a042a3b600..c2f5c968eb7d8e 100644 --- a/core/src/tpu.rs +++ b/core/src/tpu.rs @@ -33,7 +33,6 @@ use { solana_runtime::{ bank_forks::BankForks, prioritization_fee_cache::PrioritizationFeeCache, - root_bank_cache::RootBankCache, vote_sender_types::{ReplayVoteReceiver, ReplayVoteSender}, }, solana_sdk::{clock::Slot, pubkey::Pubkey, quic::NotifyKeyUpdate, signature::Keypair}, @@ -101,7 +100,6 @@ impl Tpu { shred_version: u16, vote_tracker: Arc, bank_forks: Arc>, - root_bank_cache: RootBankCache, verified_vote_sender: VerifiedVoteSender, gossip_verified_vote_hash_sender: GossipVerifiedVoteHashSender, replay_vote_receiver: ReplayVoteReceiver, @@ -262,7 +260,6 @@ impl Tpu { gossip_vote_sender, vote_tracker, bank_forks.clone(), - root_bank_cache, subscriptions.clone(), verified_vote_sender, gossip_verified_vote_hash_sender, diff --git a/core/src/validator.rs b/core/src/validator.rs index fa2e1404d47420..86cfa176d1031d 100644 --- a/core/src/validator.rs +++ b/core/src/validator.rs @@ -104,7 +104,6 @@ use { bank_forks::BankForks, commitment::BlockCommitmentCache, prioritization_fee_cache::PrioritizationFeeCache, - root_bank_cache::RootBankCache, runtime_config::RuntimeConfig, snapshot_archive_info::SnapshotArchiveInfoGetter, snapshot_bank_utils::{self, DISABLED_SNAPSHOT_ARCHIVE_INTERVAL}, @@ -1428,19 +1427,6 @@ impl Validator { let cluster_slots = Arc::new(crate::cluster_slots_service::cluster_slots::ClusterSlots::default()); - // bank_forks could be write-locked temporarily here, if unified scheduler is enabled for - // block production. That's because ReplayStage started inside Tvu could immediately try to - // insert a tpu bank into bank_forks, like with local development clusters consisting of a - // single staked node. Such insertion could be blocked with the lock held by unified - // scheduler until it's fully setup by the banking stage. This is intentional because - // completion of insertion needs to strictly correspond with the initiation of producing - // blocks in unified scheduler. Because Tpu setup follows Tvu setup, there's a corner case - // where the banking stage hasn't yet called register_banking_stage() to finish the unified - // scheduler setup. - // This means any setup which accesses bank_forks must be done before - // or after the short duration of Tvu and Tpu setups to avoid deadlocks. So, - // RootBankCache needs to be created here in advance as being one of such setups. - let root_bank_cache = RootBankCache::new(bank_forks.clone()); let tvu = Tvu::new( vote_account, authorized_voter_keypairs, @@ -1548,7 +1534,6 @@ impl Validator { node.info.shred_version(), vote_tracker, bank_forks.clone(), - root_bank_cache, verified_vote_sender, gossip_verified_vote_hash_sender, replay_vote_receiver, From 6921b38cf4bda241c8c46405fe2b3d6718f64f9a Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Tue, 17 Dec 2024 22:26:26 +0900 Subject: [PATCH 10/25] Initialize unified scheduler at once earliest --- banking-bench/src/main.rs | 1 - core/benches/banking_stage.rs | 1 - core/src/banking_simulation.rs | 1 - core/src/banking_stage.rs | 91 ++------------------- core/src/banking_stage/unified_scheduler.rs | 60 ++++++++++++++ core/src/tpu.rs | 9 +- core/src/validator.rs | 75 ++++++++++++----- unified-scheduler-pool/src/lib.rs | 16 +--- 8 files changed, 127 insertions(+), 127 deletions(-) create mode 100644 core/src/banking_stage/unified_scheduler.rs diff --git a/banking-bench/src/main.rs b/banking-bench/src/main.rs index 5a27ef3709ec4a..946dc118c6ba85 100644 --- a/banking-bench/src/main.rs +++ b/banking-bench/src/main.rs @@ -507,7 +507,6 @@ fn main() { bank_forks.clone(), &prioritization_fee_cache, false, - scheduler_pool, ); // This bench processes transactions, starting from the very first bank, so special-casing is diff --git a/core/benches/banking_stage.rs b/core/benches/banking_stage.rs index 37bea4b77f259c..dfad8bc8c227cf 100644 --- a/core/benches/banking_stage.rs +++ b/core/benches/banking_stage.rs @@ -310,7 +310,6 @@ fn bench_banking(bencher: &mut Bencher, tx_type: TransactionType) { bank_forks, &Arc::new(PrioritizationFeeCache::new(0u64)), false, - None, ); let chunk_len = verified.len() / CHUNKS; diff --git a/core/src/banking_simulation.rs b/core/src/banking_simulation.rs index 3832963652716c..abffb4c303c563 100644 --- a/core/src/banking_simulation.rs +++ b/core/src/banking_simulation.rs @@ -832,7 +832,6 @@ impl BankingSimulator { bank_forks.clone(), prioritization_fee_cache, false, - unified_scheduler_pool, ); let (&_slot, &raw_base_event_time) = freeze_time_by_slot diff --git a/core/src/banking_stage.rs b/core/src/banking_stage.rs index 53b9ee8251a9cb..cbe78a4097d420 100644 --- a/core/src/banking_stage.rs +++ b/core/src/banking_stage.rs @@ -42,7 +42,6 @@ use { }, solana_sdk::{pubkey::Pubkey, timing::AtomicInterval}, solana_unified_scheduler_logic::SchedulingMode, - solana_unified_scheduler_pool::{BankingStageAdapter, DefaultSchedulerPool}, std::{ cmp, env, ops::Deref, @@ -70,7 +69,7 @@ pub mod unprocessed_packet_batches; pub mod unprocessed_transaction_storage; mod consume_worker; -mod decision_maker; +pub(crate) mod decision_maker; mod forward_packet_batches_by_accounts; mod immutable_deserialized_packet; mod latest_unprocessed_votes; @@ -82,6 +81,7 @@ mod packet_receiver; mod read_write_account_set; mod scheduler_messages; mod transaction_scheduler; +pub mod unified_scheduler; // Fixed thread size seems to be fastest on GCP setup pub const NUM_THREADS: u32 = 6; @@ -365,7 +365,6 @@ impl BankingStage { bank_forks: Arc>, prioritization_fee_cache: &Arc, enable_forwarding: bool, - unified_scheduler_pool: Option>, ) -> Self { Self::new_num_threads( block_production_method, @@ -382,7 +381,6 @@ impl BankingStage { bank_forks, prioritization_fee_cache, enable_forwarding, - unified_scheduler_pool, ) } @@ -402,12 +400,9 @@ impl BankingStage { bank_forks: Arc>, prioritization_fee_cache: &Arc, enable_forwarding: bool, - unified_scheduler_pool: Option>, ) -> Self { - use BlockProductionMethod::*; - match block_production_method { - CentralScheduler => Self::new_central_scheduler( + BlockProductionMethod::CentralScheduler => Self::new_central_scheduler( cluster_info, poh_recorder, non_vote_receiver, @@ -422,16 +417,9 @@ impl BankingStage { prioritization_fee_cache, enable_forwarding, ), - UnifiedScheduler => Self::new_unified_scheduler( - cluster_info, - poh_recorder, - non_vote_receiver, - tpu_vote_receiver, - gossip_vote_receiver, - num_threads, - bank_forks, - unified_scheduler_pool.unwrap(), - ), + BlockProductionMethod::UnifiedScheduler => Self { + bank_thread_hdls: vec![], + }, } } @@ -579,69 +567,6 @@ impl BankingStage { Self { bank_thread_hdls } } - pub fn new_unified_scheduler( - cluster_info: &impl LikeClusterInfo, - poh_recorder: &Arc>, - non_vote_receiver: BankingPacketReceiver, - tpu_vote_receiver: BankingPacketReceiver, - gossip_vote_receiver: BankingPacketReceiver, - _num_threads: u32, - bank_forks: Arc>, - unified_scheduler_pool: Arc, - ) -> Self { - assert!(non_vote_receiver.same_channel(&tpu_vote_receiver)); - assert!(non_vote_receiver.same_channel(&gossip_vote_receiver)); - drop((tpu_vote_receiver, gossip_vote_receiver)); - - let unified_receiver = non_vote_receiver; - let decision_maker = DecisionMaker::new(cluster_info.id(), poh_recorder.clone()); - let banking_stage_monitor = Box::new(decision_maker.clone()); - - unified_scheduler_pool.register_banking_stage( - unified_receiver, - 1, /* todo */ - banking_stage_monitor, - Box::new(move |adapter: Arc| { - let decision_maker = decision_maker.clone(); - let bank_forks = bank_forks.clone(); - - Box::new(move |batches, task_submitter| { - let decision = decision_maker.make_consume_or_forward_decision(); - if matches!(decision, BufferedPacketsDecision::Forward) { - return; - } - let bank = bank_forks.read().unwrap().root_bank(); - for batch in batches.iter() { - // over-provision nevertheless some of packets could be invalid. - let task_id_base = adapter.generate_task_ids(batch.len()); - let packets = PacketDeserializer::deserialize_packets_with_indexes(batch); - - for (packet, packet_index) in packets { - let Some((transaction, _deactivation_slot)) = packet - .build_sanitized_transaction( - bank.vote_only_bank(), - &bank, - bank.get_reserved_account_keys(), - ) - else { - continue; - }; - - let index = task_id_base + packet_index; - - let task = adapter.create_new_task(transaction, index); - task_submitter(task); - } - } - }) - }), - ); - - Self { - bank_thread_hdls: vec![], - } - } - fn spawn_thread_local_multi_iterator_thread( id: u32, packet_receiver: BankingPacketReceiver, @@ -919,7 +844,6 @@ mod tests { bank_forks, &Arc::new(PrioritizationFeeCache::new(0u64)), false, - None, ); drop(non_vote_sender); drop(tpu_vote_sender); @@ -980,7 +904,6 @@ mod tests { bank_forks, &Arc::new(PrioritizationFeeCache::new(0u64)), false, - None, ); trace!("sending bank"); drop(non_vote_sender); @@ -1065,7 +988,6 @@ mod tests { bank_forks.clone(), // keep a local-copy of bank-forks so worker threads do not lose weak access to bank-forks &Arc::new(PrioritizationFeeCache::new(0u64)), false, - None, ); // fund another account so we can send 2 good transactions in a single batch. @@ -1433,7 +1355,6 @@ mod tests { bank_forks, &Arc::new(PrioritizationFeeCache::new(0u64)), false, - None, ); let keypairs = (0..100).map(|_| Keypair::new()).collect_vec(); diff --git a/core/src/banking_stage/unified_scheduler.rs b/core/src/banking_stage/unified_scheduler.rs new file mode 100644 index 00000000000000..0042806cab36d2 --- /dev/null +++ b/core/src/banking_stage/unified_scheduler.rs @@ -0,0 +1,60 @@ +use { + super::decision_maker::{BufferedPacketsDecision, DecisionMaker}, + crate::banking_stage::packet_deserializer::PacketDeserializer, + solana_perf::packet::BankingPacketReceiver, + solana_runtime::bank_forks::BankForks, + solana_unified_scheduler_pool::{BankingStageAdapter, BatchConverterCreator}, + std::sync::{Arc, RwLock}, +}; + +pub(crate) fn unified_receiver( + non_vote_receiver: BankingPacketReceiver, + tpu_vote_receiver: BankingPacketReceiver, + gossip_vote_receiver: BankingPacketReceiver, +) -> BankingPacketReceiver { + assert!(non_vote_receiver.same_channel(&tpu_vote_receiver)); + assert!(non_vote_receiver.same_channel(&gossip_vote_receiver)); + drop((tpu_vote_receiver, gossip_vote_receiver)); + + non_vote_receiver +} + +pub(crate) fn batch_converter_creator( + decision_maker: DecisionMaker, + bank_forks: Arc>, +) -> BatchConverterCreator { + Box::new(move |adapter: Arc| { + let decision_maker = decision_maker.clone(); + let bank_forks = bank_forks.clone(); + + Box::new(move |batches, task_submitter| { + let decision = decision_maker.make_consume_or_forward_decision(); + if matches!(decision, BufferedPacketsDecision::Forward) { + return; + } + let bank = bank_forks.read().unwrap().root_bank(); + for batch in batches.iter() { + // over-provision nevertheless some of packets could be invalid. + let task_id_base = adapter.generate_task_ids(batch.len()); + let packets = PacketDeserializer::deserialize_packets_with_indexes(batch); + + for (packet, packet_index) in packets { + let Some((transaction, _deactivation_slot)) = packet + .build_sanitized_transaction( + bank.vote_only_bank(), + &bank, + bank.get_reserved_account_keys(), + ) + else { + continue; + }; + + let index = task_id_base + packet_index; + + let task = adapter.create_new_task(transaction, index); + task_submitter(task); + } + } + }) + }) +} diff --git a/core/src/tpu.rs b/core/src/tpu.rs index c2f5c968eb7d8e..3a7d03dff20495 100644 --- a/core/src/tpu.rs +++ b/core/src/tpu.rs @@ -5,7 +5,7 @@ pub use solana_sdk::net::DEFAULT_TPU_COALESCE; use { crate::{ banking_stage::BankingStage, - banking_trace::{BankingTracer, Channels, TracerThread}, + banking_trace::{Channels, TracerThread}, cluster_info_vote_listener::{ ClusterInfoVoteListener, DuplicateConfirmedSlotsSender, GossipVerifiedVoteHashSender, VerifiedVoteSender, VoteTracker, @@ -44,7 +44,6 @@ use { streamer::StakedNodes, }, solana_turbine::broadcast_stage::{BroadcastStage, BroadcastStageType}, - solana_unified_scheduler_pool::DefaultSchedulerPool, std::{ collections::HashMap, net::{SocketAddr, UdpSocket}, @@ -113,7 +112,7 @@ impl Tpu { log_messages_bytes_limit: Option, staked_nodes: &Arc>, shared_staked_nodes_overrides: Arc>>, - banking_tracer: Arc, + banking_tracer_channels: Channels, tracer_thread_hdl: TracerThread, tpu_enable_udp: bool, tpu_max_connections_per_ipaddr_per_minute: u64, @@ -121,7 +120,6 @@ impl Tpu { block_production_method: BlockProductionMethod, enable_block_production_forwarding: bool, _generator_config: Option, /* vestigial code for replay invalidator */ - unified_scheduler_pool: Option>, ) -> (Self, Vec>) { let TpuSockets { transactions: transactions_sockets, @@ -165,7 +163,7 @@ impl Tpu { tpu_vote_receiver, gossip_vote_sender, gossip_vote_receiver, - } = banking_tracer.create_channels_for_scheduler_pool(unified_scheduler_pool.as_ref()); + } = banking_tracer_channels; // Streamer for Votes: let SpawnServerResult { @@ -283,7 +281,6 @@ impl Tpu { bank_forks.clone(), prioritization_fee_cache, enable_block_production_forwarding, - unified_scheduler_pool, ); let (entry_receiver, tpu_entry_notifier) = diff --git a/core/src/validator.rs b/core/src/validator.rs index 86cfa176d1031d..b4e012fce85e32 100644 --- a/core/src/validator.rs +++ b/core/src/validator.rs @@ -922,12 +922,51 @@ impl Validator { }; let poh_recorder = Arc::new(RwLock::new(poh_recorder)); - let unified_scheduler_pool = match ( + let (banking_tracer, tracer_thread) = + BankingTracer::new((config.banking_trace_dir_byte_limit > 0).then_some(( + &blockstore.banking_trace_path(), + exit.clone(), + config.banking_trace_dir_byte_limit, + )))?; + if banking_tracer.is_enabled() { + info!( + "Enabled banking trace (dir_byte_limit: {})", + config.banking_trace_dir_byte_limit + ); + } else { + info!("Disabled banking trace"); + } + + let banking_tracer_channels = match ( &config.block_verification_method, &config.block_production_method, ) { methods @ (BlockVerificationMethod::UnifiedScheduler, _) | methods @ (_, BlockProductionMethod::UnifiedScheduler) => { + let banking_tracer_channels = banking_tracer.create_channels(true); + use crate::{ + banking_stage::{ + decision_maker::DecisionMaker, unified_scheduler, BankingStage, + }, + banking_trace::Channels, + }; + + let Channels { + non_vote_sender: _, + non_vote_receiver, + tpu_vote_sender: _, + tpu_vote_receiver, + gossip_vote_sender: _, + gossip_vote_receiver, + } = &banking_tracer_channels; + let unified_receiver = unified_scheduler::unified_receiver( + non_vote_receiver.clone(), + tpu_vote_receiver.clone(), + gossip_vote_receiver.clone(), + ); + let block_producing_scheduler_handler_threads = + BankingStage::num_threads() as usize; + let pool = DefaultSchedulerPool::new( supported_scheduling_mode(methods), config.unified_scheduler_handler_threads, @@ -937,11 +976,23 @@ impl Validator { prioritization_fee_cache.clone(), poh_recorder.read().unwrap().new_recorder(), ); + let decision_maker = DecisionMaker::new(cluster_info.id(), poh_recorder.clone()); + let banking_stage_monitor = Box::new(decision_maker.clone()); + let converter = + unified_scheduler::batch_converter_creator(decision_maker, bank_forks.clone()); + pool.register_banking_stage( + unified_receiver, + block_producing_scheduler_handler_threads, + banking_stage_monitor, + converter, + ); + bank_forks .write() .unwrap() .install_scheduler_pool(pool.clone()); - Some(pool) + // this actually won't be used and but return this for type safety + banking_tracer_channels } _ => { info!("no scheduler pool is installed for block verification/production..."); @@ -951,7 +1002,7 @@ impl Validator { scheduler isn't enabled" ); } - None + banking_tracer.create_channels(false) } }; @@ -1291,21 +1342,6 @@ impl Validator { let (gossip_verified_vote_hash_sender, gossip_verified_vote_hash_receiver) = unbounded(); let (duplicate_confirmed_slot_sender, duplicate_confirmed_slots_receiver) = unbounded(); - let (banking_tracer, tracer_thread) = - BankingTracer::new((config.banking_trace_dir_byte_limit > 0).then_some(( - &blockstore.banking_trace_path(), - exit.clone(), - config.banking_trace_dir_byte_limit, - )))?; - if banking_tracer.is_enabled() { - info!( - "Enabled banking trace (dir_byte_limit: {})", - config.banking_trace_dir_byte_limit - ); - } else { - info!("Disabled banking trace"); - } - let entry_notification_sender = entry_notifier_service .as_ref() .map(|service| service.sender_cloned()); @@ -1547,7 +1583,7 @@ impl Validator { config.runtime_config.log_messages_bytes_limit, &staked_nodes, config.staked_nodes_overrides.clone(), - banking_tracer, + banking_tracer_channels, tracer_thread, tpu_enable_udp, tpu_max_connections_per_ipaddr_per_minute, @@ -1555,7 +1591,6 @@ impl Validator { config.block_production_method.clone(), config.enable_block_production_forwarding, config.generator_config.clone(), - unified_scheduler_pool, ); datapoint_info!( diff --git a/unified-scheduler-pool/src/lib.rs b/unified-scheduler-pool/src/lib.rs index ea0486771628d3..a7a01d06e442f0 100644 --- a/unified-scheduler-pool/src/lib.rs +++ b/unified-scheduler-pool/src/lib.rs @@ -47,7 +47,7 @@ use { mem, sync::{ atomic::{AtomicU64, AtomicUsize, Ordering::Relaxed}, - Arc, Condvar, Mutex, MutexGuard, OnceLock, Weak, + Arc, Mutex, MutexGuard, OnceLock, Weak, }, thread::{self, sleep, JoinHandle}, time::{Duration, Instant}, @@ -109,7 +109,6 @@ pub struct SchedulerPool, TH: TaskHandler> { supported_scheduling_mode: SupportedSchedulingMode, scheduler_inners: Mutex>, block_production_scheduler_inner: Mutex<(Option, Option)>, - block_production_scheduler_condvar: Condvar, block_production_scheduler_respawner: Mutex>, trashed_scheduler_inners: Mutex>, timeout_listeners: Mutex>, @@ -169,7 +168,7 @@ trait_set! { clone_trait_object!(BatchConverter); -type BatchConverterCreator = +pub type BatchConverterCreator = Box) -> Box) + Send>; #[derive(derive_more::Debug)] @@ -254,7 +253,6 @@ where supported_scheduling_mode, scheduler_inners: Mutex::default(), block_production_scheduler_inner: Mutex::default(), - block_production_scheduler_condvar: Condvar::default(), block_production_scheduler_respawner: Mutex::default(), trashed_scheduler_inners: Mutex::default(), timeout_listeners: Mutex::default(), @@ -504,17 +502,10 @@ where } } BlockProduction => { - let id_and_inner = self + let (id, inner) = &mut *self .block_production_scheduler_inner .lock() .expect("not poisoned"); - // Wait with condvar until scheduler is spawned elsewhere, which is guaranteed to - // be done eventually. - let (id, inner) = &mut *self - .block_production_scheduler_condvar - .wait_while(id_and_inner, |id_and_inner| id_and_inner.0.is_none()) - .unwrap(); - let Some(inner) = inner.take() else { panic!("already taken: id: {id:?} slot: {:?}", context.slot()); }; @@ -614,7 +605,6 @@ where }; assert!(id_and_inner.0.replace(inner.id()).is_none()); assert!(id_and_inner.1.replace(inner).is_none()); - self.block_production_scheduler_condvar.notify_all(); trace!("spawn block production scheduler: end!"); } From da3f787fa84749cdadcb551ba76612779864c796 Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Thu, 19 Dec 2024 14:14:11 +0900 Subject: [PATCH 11/25] Convert session_ending condition to assert!() --- unified-scheduler-pool/src/lib.rs | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/unified-scheduler-pool/src/lib.rs b/unified-scheduler-pool/src/lib.rs index a7a01d06e442f0..a75c068c034c97 100644 --- a/unified-scheduler-pool/src/lib.rs +++ b/unified-scheduler-pool/src/lib.rs @@ -1395,7 +1395,8 @@ impl, TH: TaskHandler> ThreadManager { break 'nonaborted_main_loop; }; state_machine.deschedule_task(&executed_task.task); - if should_pause && !session_ending { + if should_pause { + assert!(!session_ending); let task = banking_stage_context.as_ref().unwrap().adapter.recreate_task( executed_task.into_inner(), ); @@ -1454,7 +1455,8 @@ impl, TH: TaskHandler> ThreadManager { break 'nonaborted_main_loop; }; state_machine.deschedule_task(&executed_task.task); - if should_pause && !session_ending { + if should_pause { + assert!(!session_ending); let task = banking_stage_context.as_ref().unwrap().adapter.recreate_task( executed_task.into_inner(), ); From 21fcc1ff8efff2b9b3a18576490f49de848f46af Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Thu, 19 Dec 2024 16:20:29 +0900 Subject: [PATCH 12/25] Revert update_bank_forks_and_poh_recorder_for_new_tpu_bank() --- core/src/banking_stage.rs | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/core/src/banking_stage.rs b/core/src/banking_stage.rs index cbe78a4097d420..b988d588b2873c 100644 --- a/core/src/banking_stage.rs +++ b/core/src/banking_stage.rs @@ -730,18 +730,14 @@ pub(crate) fn update_bank_forks_and_poh_recorder_for_new_tpu_bank( tpu_bank: Bank, track_transaction_indexes: bool, ) { - // A write lock for the poh recorder must be grabbed for the entire duration of inserting new - // tpu bank into the bank forks. That's because any buffered transactions could immediately be - // executed after the bank forks update, when unified scheduler is enabled for block - // production. If transactions were executed prematurely, the unified scheduler would be hit - // with false errors due to having no bank in the poh recorder otherwise. - let mut poh_recorder = poh_recorder.write().unwrap(); - let tpu_bank = bank_forks .write() .unwrap() .insert_with_scheduling_mode(SchedulingMode::BlockProduction, tpu_bank); - poh_recorder.set_bank(tpu_bank, track_transaction_indexes); + poh_recorder + .write() + .unwrap() + .set_bank(tpu_bank, track_transaction_indexes); } #[cfg(test)] From f3199027d3325683866ad04280e01cb8537ba85d Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Thu, 19 Dec 2024 15:54:32 +0900 Subject: [PATCH 13/25] Use unblocking mechanism instead of nested locks --- core/src/banking_stage.rs | 3 ++- runtime/src/installed_scheduler_pool.rs | 12 ++++++++++++ unified-scheduler-pool/src/lib.rs | 24 +++++++++++++++++++++++- 3 files changed, 37 insertions(+), 2 deletions(-) diff --git a/core/src/banking_stage.rs b/core/src/banking_stage.rs index b988d588b2873c..47c5fa048b3b5e 100644 --- a/core/src/banking_stage.rs +++ b/core/src/banking_stage.rs @@ -737,7 +737,8 @@ pub(crate) fn update_bank_forks_and_poh_recorder_for_new_tpu_bank( poh_recorder .write() .unwrap() - .set_bank(tpu_bank, track_transaction_indexes); + .set_bank(tpu_bank.clone_with_scheduler(), track_transaction_indexes); + tpu_bank.unblock_block_production(); } #[cfg(test)] diff --git a/runtime/src/installed_scheduler_pool.rs b/runtime/src/installed_scheduler_pool.rs index 9807278f62f4b5..cc6bbbade16680 100644 --- a/runtime/src/installed_scheduler_pool.rs +++ b/runtime/src/installed_scheduler_pool.rs @@ -170,6 +170,8 @@ pub trait InstalledScheduler: Send + Sync + Debug + 'static { index: usize, ) -> ScheduleResult; + fn unblock_scheduling(&self); + /// Return the error which caused the scheduler to abort. /// /// Note that this must not be called until it's observed that `schedule_execution()` has @@ -506,6 +508,16 @@ impl BankWithScheduler { Ok(()) } + pub fn unblock_block_production(&self) { + self.inner + .with_active_scheduler(|scheduler| { + assert_matches!(scheduler.context().mode(), SchedulingMode::BlockProduction); + scheduler.unblock_scheduling(); + Ok(()) + }) + .unwrap(); + } + #[cfg_attr(feature = "dev-context-only-utils", qualifiers(pub))] pub(crate) fn create_timeout_listener(&self) -> TimeoutListener { self.inner.do_create_timeout_listener() diff --git a/unified-scheduler-pool/src/lib.rs b/unified-scheduler-pool/src/lib.rs index a75c068c034c97..5bd0daa1cc9c18 100644 --- a/unified-scheduler-pool/src/lib.rs +++ b/unified-scheduler-pool/src/lib.rs @@ -774,10 +774,12 @@ impl ExecutedTask { // Note that the above properties can be upheld only when this is used inside MPSC or SPSC channels // (i.e. the consumer side needs to be single threaded). For the multiple consumer cases, // ChainedChannel can be used instead. +#[derive(Debug)] enum SubchanneledPayload { Payload(P1), OpenSubchannel(P2), CloseSubchannel, + Unblock, Disconnect, Reset, } @@ -1436,7 +1438,7 @@ impl, TH: TaskHandler> ThreadManager { session_pausing = true; session_resetting = true; } - Ok(NewTaskPayload::OpenSubchannel(_context_and_result_with_timings)) => + Ok(NewTaskPayload::OpenSubchannel(_) | NewTaskPayload::Unblock) => unreachable!(), Ok(NewTaskPayload::Disconnect) | Err(RecvError) => { // Mostly likely is that this scheduler is dropped for pruned blocks of @@ -1504,6 +1506,12 @@ impl, TH: TaskHandler> ThreadManager { session_ending = false; } else { session_pausing = false; + // Prevent processing transactions and thus touching poh until + // unblocked, which signals successful poh `set_bank()`-ing. + assert_matches!( + new_task_receiver.recv().unwrap(), + NewTaskPayload::Unblock + ); } runnable_task_sender @@ -1792,6 +1800,12 @@ impl, TH: TaskHandler> ThreadManager { .expect("no new session after aborted"); } + fn unblock_session(&self) { + self.new_task_sender + .send(NewTaskPayload::Unblock) + .expect("no new session after aborted"); + } + fn disconnect_new_task_sender(&mut self) { self.new_task_sender = Arc::new(crossbeam_channel::unbounded().0); } @@ -1948,6 +1962,10 @@ impl InstalledScheduler for PooledScheduler { self.inner.thread_manager.send_task(task) } + fn unblock_scheduling(&self) { + self.inner.thread_manager.unblock_session(); + } + fn recover_error_after_abort(&mut self) -> TransactionError { self.inner .thread_manager @@ -3408,6 +3426,10 @@ mod tests { Ok(()) } + fn unblock_scheduling(&self) { + unimplemented!(); + } + fn recover_error_after_abort(&mut self) -> TransactionError { unimplemented!(); } From 656444cbe05381a64dbd18ff9ae88ddc69868419 Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Fri, 20 Dec 2024 09:38:29 +0900 Subject: [PATCH 14/25] fix ci? --- unified-scheduler-pool/src/lib.rs | 20 +++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/unified-scheduler-pool/src/lib.rs b/unified-scheduler-pool/src/lib.rs index 5bd0daa1cc9c18..652fa57358df48 100644 --- a/unified-scheduler-pool/src/lib.rs +++ b/unified-scheduler-pool/src/lib.rs @@ -1480,6 +1480,7 @@ impl, TH: TaskHandler> ThreadManager { .send(result_with_timings) .expect("always outlived receiver"); + let mut new_result_with_timings = initialized_result_with_timings(); loop { if session_resetting { while let Some(task) = state_machine.schedule_next_unblocked_task() { @@ -1494,7 +1495,8 @@ impl, TH: TaskHandler> ThreadManager { Ok(NewTaskPayload::OpenSubchannel(context_and_result_with_timings)), _, ) => { - let (new_context, new_result_with_timings) = + let new_context; + (new_context, new_result_with_timings) = *context_and_result_with_timings; // We just received subsequent (= not initial) session and about to // enter into the preceding `while(!is_finished) {...}` loop again. @@ -1506,20 +1508,16 @@ impl, TH: TaskHandler> ThreadManager { session_ending = false; } else { session_pausing = false; - // Prevent processing transactions and thus touching poh until - // unblocked, which signals successful poh `set_bank()`-ing. - assert_matches!( - new_task_receiver.recv().unwrap(), - NewTaskPayload::Unblock - ); } runnable_task_sender .send_chained_channel(&new_context, handler_count) .unwrap(); context = new_context; - result_with_timings = new_result_with_timings; - break; + if context.mode() == BlockVerification { + result_with_timings = new_result_with_timings; + break; + } } (Ok(NewTaskPayload::CloseSubchannel), BlockProduction) => { info!("ignoring duplicate CloseSubchannel..."); @@ -1530,6 +1528,10 @@ impl, TH: TaskHandler> ThreadManager { (Ok(NewTaskPayload::Payload(task)), BlockProduction) => { assert!(state_machine.do_schedule_task(task, true).is_none()); } + (Ok(NewTaskPayload::Unblock), BlockProduction) => { + result_with_timings = new_result_with_timings; + break; + } (Ok(NewTaskPayload::Disconnect), BlockProduction) | (Err(_), _) => { // This unusual condition must be triggered by ThreadManager::drop(). // Initialize result_with_timings with a harmless value... From 1ea4bbfd33d0d36aad8cbb432198d2c6dcae29eb Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Fri, 20 Dec 2024 16:04:35 +0900 Subject: [PATCH 15/25] fix ci?? --- runtime/src/installed_scheduler_pool.rs | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/runtime/src/installed_scheduler_pool.rs b/runtime/src/installed_scheduler_pool.rs index cc6bbbade16680..01d074165a3dfb 100644 --- a/runtime/src/installed_scheduler_pool.rs +++ b/runtime/src/installed_scheduler_pool.rs @@ -509,13 +509,10 @@ impl BankWithScheduler { } pub fn unblock_block_production(&self) { - self.inner - .with_active_scheduler(|scheduler| { - assert_matches!(scheduler.context().mode(), SchedulingMode::BlockProduction); - scheduler.unblock_scheduling(); - Ok(()) - }) - .unwrap(); + if let SchedulerStatus::Active(scheduler) = &*self.inner.scheduler.read().unwrap() { + assert_matches!(scheduler.context().mode(), SchedulingMode::BlockProduction); + scheduler.unblock_scheduling(); + } } #[cfg_attr(feature = "dev-context-only-utils", qualifiers(pub))] From 030b979958a08bd1424b3d2e0af9c79eaed3ad41 Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Sun, 12 Jan 2025 00:14:31 +0900 Subject: [PATCH 16/25] rebase fixups --- core/src/banking_stage.rs | 1 - unified-scheduler-pool/src/lib.rs | 8 -------- 2 files changed, 9 deletions(-) diff --git a/core/src/banking_stage.rs b/core/src/banking_stage.rs index 47c5fa048b3b5e..48f60db869955f 100644 --- a/core/src/banking_stage.rs +++ b/core/src/banking_stage.rs @@ -1155,7 +1155,6 @@ mod tests { bank_forks, &Arc::new(PrioritizationFeeCache::new(0u64)), false, - None, ); // wait for banking_stage to eat the packets diff --git a/unified-scheduler-pool/src/lib.rs b/unified-scheduler-pool/src/lib.rs index 652fa57358df48..e11426e6a63cd4 100644 --- a/unified-scheduler-pool/src/lib.rs +++ b/unified-scheduler-pool/src/lib.rs @@ -2016,10 +2016,6 @@ where .is_overgrown(self.thread_manager.pool.max_usage_queue_count) } - fn is_trashed(&self) -> bool { - self.is_aborted() || self.is_overgrown() - } - fn reset(&self) { if let Err(a) = self .thread_manager @@ -3471,10 +3467,6 @@ mod tests { unimplemented!() } - fn is_trashed(&self) -> bool { - false - } - fn reset(&self) { unimplemented!() } From 3df47ba4b91000b8e91112816ea43fe29377d0fa Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Sun, 12 Jan 2025 00:36:49 +0900 Subject: [PATCH 17/25] rebase fixups --- core/src/banking_stage/unified_scheduler.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/banking_stage/unified_scheduler.rs b/core/src/banking_stage/unified_scheduler.rs index 0042806cab36d2..b4f639a24e39dc 100644 --- a/core/src/banking_stage/unified_scheduler.rs +++ b/core/src/banking_stage/unified_scheduler.rs @@ -1,7 +1,7 @@ use { super::decision_maker::{BufferedPacketsDecision, DecisionMaker}, crate::banking_stage::packet_deserializer::PacketDeserializer, - solana_perf::packet::BankingPacketReceiver, + agave_banking_stage_ingress_types::BankingPacketReceiver, solana_runtime::bank_forks::BankForks, solana_unified_scheduler_pool::{BankingStageAdapter, BatchConverterCreator}, std::sync::{Arc, RwLock}, From 6a34d0c60d4c71cdcd59614c1614e538a168a394 Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Sun, 12 Jan 2025 13:51:57 +0900 Subject: [PATCH 18/25] rebase fixups --- programs/sbf/Cargo.lock | 2 ++ svm/examples/Cargo.lock | 2 ++ 2 files changed, 4 insertions(+) diff --git a/programs/sbf/Cargo.lock b/programs/sbf/Cargo.lock index d0ecad6425ac9b..727227c15e8585 100644 --- a/programs/sbf/Cargo.lock +++ b/programs/sbf/Cargo.lock @@ -8413,6 +8413,7 @@ dependencies = [ name = "solana-unified-scheduler-pool" version = "2.2.0" dependencies = [ + "agave-banking-stage-ingress-types", "assert_matches", "crossbeam-channel", "dashmap", @@ -8426,6 +8427,7 @@ dependencies = [ "solana-pubkey", "solana-runtime", "solana-runtime-transaction", + "solana-sdk", "solana-timings", "solana-transaction", "solana-transaction-error", diff --git a/svm/examples/Cargo.lock b/svm/examples/Cargo.lock index 148b231a7c36b3..5f3a1415b54a29 100644 --- a/svm/examples/Cargo.lock +++ b/svm/examples/Cargo.lock @@ -7749,6 +7749,7 @@ dependencies = [ name = "solana-unified-scheduler-pool" version = "2.2.0" dependencies = [ + "agave-banking-stage-ingress-types", "assert_matches", "crossbeam-channel", "dashmap", @@ -7762,6 +7763,7 @@ dependencies = [ "solana-pubkey", "solana-runtime", "solana-runtime-transaction", + "solana-sdk", "solana-timings", "solana-transaction", "solana-transaction-error", From a0e63ee86b8ce965b700282ee7b0d279db35f4b6 Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Mon, 13 Jan 2025 22:37:11 +0900 Subject: [PATCH 19/25] Remove solana-sdk dep from unified-scheduler-pool --- Cargo.lock | 1 - programs/sbf/Cargo.lock | 1 - svm/examples/Cargo.lock | 1 - unified-scheduler-pool/Cargo.toml | 1 - 4 files changed, 4 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 5c33767345a65b..aa5614643f8c4e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10178,7 +10178,6 @@ dependencies = [ "solana-pubkey", "solana-runtime", "solana-runtime-transaction", - "solana-sdk", "solana-system-transaction", "solana-timings", "solana-transaction", diff --git a/programs/sbf/Cargo.lock b/programs/sbf/Cargo.lock index 727227c15e8585..4689ce1f24a2a2 100644 --- a/programs/sbf/Cargo.lock +++ b/programs/sbf/Cargo.lock @@ -8427,7 +8427,6 @@ dependencies = [ "solana-pubkey", "solana-runtime", "solana-runtime-transaction", - "solana-sdk", "solana-timings", "solana-transaction", "solana-transaction-error", diff --git a/svm/examples/Cargo.lock b/svm/examples/Cargo.lock index 5f3a1415b54a29..000c3462662aa8 100644 --- a/svm/examples/Cargo.lock +++ b/svm/examples/Cargo.lock @@ -7763,7 +7763,6 @@ dependencies = [ "solana-pubkey", "solana-runtime", "solana-runtime-transaction", - "solana-sdk", "solana-timings", "solana-transaction", "solana-transaction-error", diff --git a/unified-scheduler-pool/Cargo.toml b/unified-scheduler-pool/Cargo.toml index 6134a144e9a9c7..169ccb2085b122 100644 --- a/unified-scheduler-pool/Cargo.toml +++ b/unified-scheduler-pool/Cargo.toml @@ -24,7 +24,6 @@ solana-poh = { workspace = true } solana-pubkey = { workspace = true } solana-runtime = { workspace = true } solana-runtime-transaction = { workspace = true } -solana-sdk = { workspace = true } solana-timings = { workspace = true } solana-transaction = { workspace = true } solana-transaction-error = { workspace = true } From a0535cd15617827a14e52f0828bcbf864d0b9dce Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Mon, 13 Jan 2025 22:40:38 +0900 Subject: [PATCH 20/25] Reorder Derives on SchedulingMode --- unified-scheduler-logic/src/lib.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/unified-scheduler-logic/src/lib.rs b/unified-scheduler-logic/src/lib.rs index 6487d716ad8ecb..682fd037259e73 100644 --- a/unified-scheduler-logic/src/lib.rs +++ b/unified-scheduler-logic/src/lib.rs @@ -105,7 +105,7 @@ use { std::{collections::VecDeque, mem, sync::Arc}, }; -#[derive(Debug, Clone, Copy, PartialEq)] +#[derive(Clone, Copy, Debug, PartialEq)] pub enum SchedulingMode { BlockVerification, BlockProduction, From d4bd053e56b30648c7e8ccee4a0da7ca20c5c981 Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Mon, 13 Jan 2025 22:43:06 +0900 Subject: [PATCH 21/25] Clean up block producing unified scheduler setup --- banking-bench/src/main.rs | 32 +++++++------ core/src/banking_simulation.rs | 18 ++++++-- core/src/banking_stage.rs | 6 +++ core/src/banking_stage/unified_scheduler.rs | 51 +++++++++++++++------ core/src/banking_trace.rs | 21 +++++---- core/src/validator.rs | 50 +++++--------------- 6 files changed, 97 insertions(+), 81 deletions(-) diff --git a/banking-bench/src/main.rs b/banking-bench/src/main.rs index 946dc118c6ba85..33ae65bdfdf26a 100644 --- a/banking-bench/src/main.rs +++ b/banking-bench/src/main.rs @@ -9,7 +9,10 @@ use { rayon::prelude::*, solana_client::connection_cache::ConnectionCache, solana_core::{ - banking_stage::{update_bank_forks_and_poh_recorder_for_new_tpu_bank, BankingStage}, + banking_stage::{ + unified_scheduler::ensure_banking_stage_setup, + update_bank_forks_and_poh_recorder_for_new_tpu_bank, BankingStage, + }, banking_trace::{BankingTracer, Channels, BANKING_TRACE_DIR_DEFAULT_BYTE_LIMIT}, validator::BlockProductionMethod, }, @@ -445,7 +448,13 @@ fn main() { ))) .unwrap(); let prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); - let scheduler_pool = if matches!( + let cluster_info = { + let keypair = Arc::new(Keypair::new()); + let node = Node::new_localhost_with_pubkey(&keypair.pubkey()); + ClusterInfo::new(node.info, keypair, SocketAddrSpace::Unspecified) + }; + let cluster_info = Arc::new(cluster_info); + let banking_tracer_channels = if matches!( block_production_method, BlockProductionMethod::UnifiedScheduler ) { @@ -458,13 +467,12 @@ fn main() { prioritization_fee_cache.clone(), poh_recorder.read().unwrap().new_recorder(), ); - bank_forks - .write() - .unwrap() - .install_scheduler_pool(pool.clone()); - Some(pool) + let channels = banking_tracer.create_channels_for_scheduler_pool(&pool); + ensure_banking_stage_setup(&pool, &bank_forks, &channels, &cluster_info, &poh_recorder); + bank_forks.write().unwrap().install_scheduler_pool(pool); + channels } else { - None + banking_tracer.create_channels(false) }; let Channels { non_vote_sender, @@ -473,13 +481,7 @@ fn main() { tpu_vote_receiver, gossip_vote_sender, gossip_vote_receiver, - } = banking_tracer.create_channels_for_scheduler_pool(scheduler_pool.as_ref()); - let cluster_info = { - let keypair = Arc::new(Keypair::new()); - let node = Node::new_localhost_with_pubkey(&keypair.pubkey()); - ClusterInfo::new(node.info, keypair, SocketAddrSpace::Unspecified) - }; - let cluster_info = Arc::new(cluster_info); + } = banking_tracer_channels; let tpu_disable_quic = matches.is_present("tpu_disable_quic"); let connection_cache = if tpu_disable_quic { ConnectionCache::with_udp( diff --git a/core/src/banking_simulation.rs b/core/src/banking_simulation.rs index abffb4c303c563..fbe584be00ec5b 100644 --- a/core/src/banking_simulation.rs +++ b/core/src/banking_simulation.rs @@ -2,6 +2,7 @@ use { crate::{ banking_stage::{ + unified_scheduler::ensure_banking_stage_setup, update_bank_forks_and_poh_recorder_for_new_tpu_bank, BankingStage, LikeClusterInfo, }, banking_trace::{ @@ -770,6 +771,17 @@ impl BankingSimulator { BANKING_TRACE_DIR_DEFAULT_BYTE_LIMIT, ); + // Create a partially-dummy ClusterInfo for the banking stage. + let cluster_info = Arc::new(DummyClusterInfo { + id: simulated_leader.into(), + }); + let banking_tracer_channels = if let Some(pool) = unified_scheduler_pool { + let channels = retracer.create_channels_for_scheduler_pool(&pool); + ensure_banking_stage_setup(&pool, &bank_forks, &channels, &cluster_info, &poh_recorder); + channels + } else { + retracer.create_channels(false) + }; let Channels { non_vote_sender, non_vote_receiver, @@ -777,7 +789,7 @@ impl BankingSimulator { tpu_vote_receiver, gossip_vote_sender, gossip_vote_receiver, - } = retracer.create_channels_for_scheduler_pool(unified_scheduler_pool.as_ref()); + } = banking_tracer_channels; let connection_cache = Arc::new(ConnectionCache::new("connection_cache_sim")); let (replay_vote_sender, _replay_vote_receiver) = unbounded(); @@ -812,10 +824,6 @@ impl BankingSimulator { ); info!("Start banking stage!..."); - // Create a partially-dummy ClusterInfo for the banking stage. - let cluster_info = Arc::new(DummyClusterInfo { - id: simulated_leader.into(), - }); let prioritization_fee_cache = &Arc::new(PrioritizationFeeCache::new(0u64)); let banking_stage = BankingStage::new_num_threads( block_production_method.clone(), diff --git a/core/src/banking_stage.rs b/core/src/banking_stage.rs index 48f60db869955f..baff7fc834db73 100644 --- a/core/src/banking_stage.rs +++ b/core/src/banking_stage.rs @@ -81,6 +81,12 @@ mod packet_receiver; mod read_write_account_set; mod scheduler_messages; mod transaction_scheduler; + +// proc_macro_hygiene needs to be stabilzied to use qualifier_attr... +// error[E0658]: non-inline modules in proc macro input are unstable +#[cfg(not(feature = "dev-context-only-utils"))] +pub(crate) mod unified_scheduler; +#[cfg(feature = "dev-context-only-utils")] pub mod unified_scheduler; // Fixed thread size seems to be fastest on GCP setup diff --git a/core/src/banking_stage/unified_scheduler.rs b/core/src/banking_stage/unified_scheduler.rs index b4f639a24e39dc..f4ecdcdd38ff21 100644 --- a/core/src/banking_stage/unified_scheduler.rs +++ b/core/src/banking_stage/unified_scheduler.rs @@ -1,24 +1,19 @@ +#[cfg(feature = "dev-context-only-utils")] +use qualifier_attr::qualifiers; use { super::decision_maker::{BufferedPacketsDecision, DecisionMaker}, - crate::banking_stage::packet_deserializer::PacketDeserializer, - agave_banking_stage_ingress_types::BankingPacketReceiver, + crate::{ + banking_stage::{packet_deserializer::PacketDeserializer, BankingStage, LikeClusterInfo}, + banking_trace::Channels, + }, + solana_poh::poh_recorder::PohRecorder, solana_runtime::bank_forks::BankForks, - solana_unified_scheduler_pool::{BankingStageAdapter, BatchConverterCreator}, + solana_unified_scheduler_pool::{ + BankingStageAdapter, BatchConverterCreator, DefaultSchedulerPool, + }, std::sync::{Arc, RwLock}, }; -pub(crate) fn unified_receiver( - non_vote_receiver: BankingPacketReceiver, - tpu_vote_receiver: BankingPacketReceiver, - gossip_vote_receiver: BankingPacketReceiver, -) -> BankingPacketReceiver { - assert!(non_vote_receiver.same_channel(&tpu_vote_receiver)); - assert!(non_vote_receiver.same_channel(&gossip_vote_receiver)); - drop((tpu_vote_receiver, gossip_vote_receiver)); - - non_vote_receiver -} - pub(crate) fn batch_converter_creator( decision_maker: DecisionMaker, bank_forks: Arc>, @@ -58,3 +53,29 @@ pub(crate) fn batch_converter_creator( }) }) } + +#[cfg_attr(feature = "dev-context-only-utils", qualifiers(pub))] +pub(crate) fn ensure_banking_stage_setup( + pool: &DefaultSchedulerPool, + bank_forks: &Arc>, + channels: &Channels, + cluster_info: &impl LikeClusterInfo, + poh_recorder: &Arc>, +) { + if !pool.block_production_supported() { + return; + } + + let unified_receiver = channels.unified_receiver().clone(); + let block_producing_scheduler_handler_threads = BankingStage::num_threads() as usize; + let decision_maker = DecisionMaker::new(cluster_info.id(), poh_recorder.clone()); + let banking_stage_monitor = Box::new(decision_maker.clone()); + let converter = batch_converter_creator(decision_maker, bank_forks.clone()); + + pool.register_banking_stage( + unified_receiver, + block_producing_scheduler_handler_threads, + banking_stage_monitor, + converter, + ); +} diff --git a/core/src/banking_trace.rs b/core/src/banking_trace.rs index ee47d5827cc972..27602b57732206 100644 --- a/core/src/banking_trace.rs +++ b/core/src/banking_trace.rs @@ -185,6 +185,16 @@ pub struct Channels { pub gossip_vote_receiver: BankingPacketReceiver, } +impl Channels { + pub(crate) fn unified_receiver(&self) -> &BankingPacketReceiver { + assert!(self.non_vote_receiver.same_channel(&self.tpu_vote_receiver)); + assert!(self + .non_vote_receiver + .same_channel(&self.gossip_vote_receiver)); + &self.non_vote_receiver + } +} + impl BankingTracer { pub fn new( maybe_config: Option<(&PathBuf, Arc, DirByteLimit)>, @@ -264,14 +274,9 @@ impl BankingTracer { } } - pub fn create_channels_for_scheduler_pool( - &self, - pool: Option<&Arc>, - ) -> Channels { - self.create_channels( - pool.map(|pool| pool.block_production_supported()) - .unwrap_or_default(), - ) + pub fn create_channels_for_scheduler_pool(&self, pool: &DefaultSchedulerPool) -> Channels { + let should_unify = pool.block_production_supported(); + self.create_channels(should_unify) } fn create_channel(&self, label: ChannelLabel) -> (BankingPacketSender, BankingPacketReceiver) { diff --git a/core/src/validator.rs b/core/src/validator.rs index b4e012fce85e32..f57e5254a32927 100644 --- a/core/src/validator.rs +++ b/core/src/validator.rs @@ -5,6 +5,7 @@ use { crate::{ accounts_hash_verifier::AccountsHashVerifier, admin_rpc_post_init::AdminRpcRequestMetadataPostInit, + banking_stage::unified_scheduler::ensure_banking_stage_setup, banking_trace::{self, BankingTracer, TraceError}, cluster_info_vote_listener::VoteTracker, completed_data_sets_service::CompletedDataSetsService, @@ -943,31 +944,7 @@ impl Validator { ) { methods @ (BlockVerificationMethod::UnifiedScheduler, _) | methods @ (_, BlockProductionMethod::UnifiedScheduler) => { - let banking_tracer_channels = banking_tracer.create_channels(true); - use crate::{ - banking_stage::{ - decision_maker::DecisionMaker, unified_scheduler, BankingStage, - }, - banking_trace::Channels, - }; - - let Channels { - non_vote_sender: _, - non_vote_receiver, - tpu_vote_sender: _, - tpu_vote_receiver, - gossip_vote_sender: _, - gossip_vote_receiver, - } = &banking_tracer_channels; - let unified_receiver = unified_scheduler::unified_receiver( - non_vote_receiver.clone(), - tpu_vote_receiver.clone(), - gossip_vote_receiver.clone(), - ); - let block_producing_scheduler_handler_threads = - BankingStage::num_threads() as usize; - - let pool = DefaultSchedulerPool::new( + let scheduler_pool = DefaultSchedulerPool::new( supported_scheduling_mode(methods), config.unified_scheduler_handler_threads, config.runtime_config.log_messages_bytes_limit, @@ -976,23 +953,20 @@ impl Validator { prioritization_fee_cache.clone(), poh_recorder.read().unwrap().new_recorder(), ); - let decision_maker = DecisionMaker::new(cluster_info.id(), poh_recorder.clone()); - let banking_stage_monitor = Box::new(decision_maker.clone()); - let converter = - unified_scheduler::batch_converter_creator(decision_maker, bank_forks.clone()); - pool.register_banking_stage( - unified_receiver, - block_producing_scheduler_handler_threads, - banking_stage_monitor, - converter, - ); + let channels = banking_tracer.create_channels_for_scheduler_pool(&scheduler_pool); + ensure_banking_stage_setup( + &scheduler_pool, + &bank_forks, + &channels, + &cluster_info, + &poh_recorder, + ); bank_forks .write() .unwrap() - .install_scheduler_pool(pool.clone()); - // this actually won't be used and but return this for type safety - banking_tracer_channels + .install_scheduler_pool(scheduler_pool); + channels } _ => { info!("no scheduler pool is installed for block verification/production..."); From b4bfe88236776cd5d6657e460eaa8133b08129f9 Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Thu, 16 Jan 2025 14:52:42 +0900 Subject: [PATCH 22/25] Revert pub(crate) on decision_maker --- core/src/banking_stage.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/banking_stage.rs b/core/src/banking_stage.rs index baff7fc834db73..de1acf8b6df499 100644 --- a/core/src/banking_stage.rs +++ b/core/src/banking_stage.rs @@ -69,7 +69,7 @@ pub mod unprocessed_packet_batches; pub mod unprocessed_transaction_storage; mod consume_worker; -pub(crate) mod decision_maker; +mod decision_maker; mod forward_packet_batches_by_accounts; mod immutable_deserialized_packet; mod latest_unprocessed_votes; From af401de33edbc204c15dadeb25f2d84ec20f6440 Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Thu, 16 Jan 2025 14:56:09 +0900 Subject: [PATCH 23/25] Simplify unified_receiver() --- core/src/banking_trace.rs | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/core/src/banking_trace.rs b/core/src/banking_trace.rs index 27602b57732206..b991999446d6f1 100644 --- a/core/src/banking_trace.rs +++ b/core/src/banking_trace.rs @@ -187,11 +187,10 @@ pub struct Channels { impl Channels { pub(crate) fn unified_receiver(&self) -> &BankingPacketReceiver { - assert!(self.non_vote_receiver.same_channel(&self.tpu_vote_receiver)); - assert!(self - .non_vote_receiver - .same_channel(&self.gossip_vote_receiver)); - &self.non_vote_receiver + let unified_receiver = &self.non_vote_receiver; + assert!(unified_receiver.same_channel(&self.tpu_vote_receiver)); + assert!(unified_receiver.same_channel(&self.gossip_vote_receiver)); + unified_receiver } } From 89353533c47d8873a02570fbe93053c8bc4726c4 Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Fri, 17 Jan 2025 12:37:30 +0900 Subject: [PATCH 24/25] cargo fmt --- core/src/banking_simulation.rs | 8 +++++++- local-cluster/tests/local_cluster.rs | 7 ++----- 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/core/src/banking_simulation.rs b/core/src/banking_simulation.rs index 407278c0c27259..d41b97a426f80c 100644 --- a/core/src/banking_simulation.rs +++ b/core/src/banking_simulation.rs @@ -775,7 +775,13 @@ impl BankingSimulator { }); let banking_tracer_channels = if let Some(pool) = unified_scheduler_pool { let channels = retracer.create_channels_for_scheduler_pool(&pool); - ensure_banking_stage_setup(&pool, &bank_forks, &channels, &cluster_info_for_banking, &poh_recorder); + ensure_banking_stage_setup( + &pool, + &bank_forks, + &channels, + &cluster_info_for_banking, + &poh_recorder, + ); channels } else { retracer.create_channels(false) diff --git a/local-cluster/tests/local_cluster.rs b/local-cluster/tests/local_cluster.rs index 13c4d251a17cb3..d2969736271365 100644 --- a/local-cluster/tests/local_cluster.rs +++ b/local-cluster/tests/local_cluster.rs @@ -5861,11 +5861,8 @@ fn test_randomly_mixed_block_production_methods_between_bootstrap_and_not() { ); let num_nodes = BlockVerificationMethod::COUNT; - let mut config = ClusterConfig::new_with_equal_stakes( - num_nodes, - DEFAULT_MINT_LAMPORTS, - DEFAULT_NODE_STAKE, - ); + let mut config = + ClusterConfig::new_with_equal_stakes(num_nodes, DEFAULT_MINT_LAMPORTS, DEFAULT_NODE_STAKE); // Overwrite block_production_method with shuffled variants let mut methods = BlockProductionMethod::iter().collect::>(); From 0d3de56f3d89214a6b02ff68e39380feb5c8d288 Mon Sep 17 00:00:00 2001 From: Ryo Onodera Date: Fri, 17 Jan 2025 13:38:08 +0900 Subject: [PATCH 25/25] Add test: test_scheduler_producing_blocks() --- core/src/banking_trace.rs | 12 +++ core/tests/unified_scheduler.rs | 131 ++++++++++++++++++++++++++++-- unified-scheduler-pool/src/lib.rs | 1 + 3 files changed, 139 insertions(+), 5 deletions(-) diff --git a/core/src/banking_trace.rs b/core/src/banking_trace.rs index b991999446d6f1..9a2c6fe3c0c688 100644 --- a/core/src/banking_trace.rs +++ b/core/src/banking_trace.rs @@ -186,6 +186,18 @@ pub struct Channels { } impl Channels { + #[cfg(feature = "dev-context-only-utils")] + pub fn unified_sender(&self) -> &BankingPacketSender { + let unified_sender = &self.non_vote_sender; + assert!(unified_sender + .sender + .same_channel(&self.tpu_vote_sender.sender)); + assert!(unified_sender + .sender + .same_channel(&self.gossip_vote_sender.sender)); + unified_sender + } + pub(crate) fn unified_receiver(&self) -> &BankingPacketReceiver { let unified_receiver = &self.non_vote_receiver; assert!(unified_receiver.same_channel(&self.tpu_vote_receiver)); diff --git a/core/tests/unified_scheduler.rs b/core/tests/unified_scheduler.rs index ecd41dde68608a..45fb352cfe885b 100644 --- a/core/tests/unified_scheduler.rs +++ b/core/tests/unified_scheduler.rs @@ -1,8 +1,12 @@ use { + agave_banking_stage_ingress_types::BankingPacketBatch, + assert_matches::assert_matches, crossbeam_channel::unbounded, itertools::Itertools, log::*, solana_core::{ + banking_stage::unified_scheduler::ensure_banking_stage_setup, + banking_trace::BankingTracer, consensus::{ heaviest_subtree_fork_choice::HeaviestSubtreeForkChoice, progress_map::{ForkProgress, ProgressMap}, @@ -14,22 +18,36 @@ use { replay_stage::ReplayStage, unfrozen_gossip_verified_vote_hashes::UnfrozenGossipVerifiedVoteHashes, }, - solana_ledger::genesis_utils::create_genesis_config, + solana_entry::entry::Entry, + solana_gossip::cluster_info::{ClusterInfo, Node}, + solana_ledger::{ + blockstore::Blockstore, create_new_tmp_ledger_auto_delete, + genesis_utils::create_genesis_config, leader_schedule_cache::LeaderScheduleCache, + }, + solana_perf::packet::to_packet_batches, + solana_poh::poh_recorder::create_test_recorder, solana_runtime::{ accounts_background_service::AbsRequestSender, bank::Bank, bank_forks::BankForks, genesis_utils::GenesisConfigInfo, installed_scheduler_pool::SchedulingContext, prioritization_fee_cache::PrioritizationFeeCache, }, solana_runtime_transaction::runtime_transaction::RuntimeTransaction, - solana_sdk::{hash::Hash, pubkey::Pubkey, system_transaction, transaction::Result}, + solana_sdk::{ + hash::Hash, pubkey::Pubkey, signature::Signer, signer::keypair::Keypair, + system_transaction, transaction::Result, + }, + solana_streamer::socket::SocketAddrSpace, solana_timings::ExecuteTimings, - solana_unified_scheduler_logic::Task, + solana_unified_scheduler_logic::{SchedulingMode, Task}, solana_unified_scheduler_pool::{ - DefaultTaskHandler, HandlerContext, PooledScheduler, SchedulerPool, TaskHandler, + DefaultSchedulerPool, DefaultTaskHandler, HandlerContext, PooledScheduler, SchedulerPool, + SupportedSchedulingMode, TaskHandler, }, std::{ collections::HashMap, - sync::{Arc, Mutex}, + sync::{atomic::Ordering, Arc, Mutex}, + thread::sleep, + time::Duration, }, }; @@ -185,3 +203,106 @@ fn test_scheduler_waited_by_drop_bank_service() { // the scheduler used by the pruned_bank have been returned now. assert_eq!(pool_raw.pooled_scheduler_count(), 1); } + +#[test] +fn test_scheduler_producing_blocks() { + solana_logger::setup(); + + let GenesisConfigInfo { + genesis_config, + mint_keypair, + .. + } = create_genesis_config(10_000); + let (ledger_path, _blockhash) = create_new_tmp_ledger_auto_delete!(&genesis_config); + let blockstore = Arc::new(Blockstore::open(ledger_path.path()).unwrap()); + + // Setup bank_forks with block-producing unified scheduler enabled + let genesis_bank = Bank::new_for_tests(&genesis_config); + let bank_forks = BankForks::new_rw_arc(genesis_bank); + let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64)); + let genesis_bank = bank_forks.read().unwrap().working_bank_with_scheduler(); + genesis_bank.set_fork_graph_in_program_cache(Arc::downgrade(&bank_forks)); + let leader_schedule_cache = Arc::new(LeaderScheduleCache::new_from_bank(&genesis_bank)); + let (exit, poh_recorder, poh_service, signal_receiver) = create_test_recorder( + genesis_bank.clone(), + blockstore.clone(), + None, + Some(leader_schedule_cache), + ); + let pool = DefaultSchedulerPool::new( + SupportedSchedulingMode::Either(SchedulingMode::BlockProduction), + None, + None, + None, + None, + ignored_prioritization_fee_cache, + poh_recorder.read().unwrap().new_recorder(), + ); + let channels = { + let banking_tracer = BankingTracer::new_disabled(); + banking_tracer.create_channels_for_scheduler_pool(&pool) + }; + let cluster_info = { + let keypair = Arc::new(Keypair::new()); + let node = Node::new_localhost_with_pubkey(&keypair.pubkey()); + Arc::new(ClusterInfo::new( + node.info, + keypair, + SocketAddrSpace::Unspecified, + )) + }; + ensure_banking_stage_setup(&pool, &bank_forks, &channels, &cluster_info, &poh_recorder); + bank_forks.write().unwrap().install_scheduler_pool(pool); + + // Wait until genesis_bank reaches its tick height... + while poh_recorder.read().unwrap().bank().is_some() { + sleep(Duration::from_millis(100)); + } + + // Create test tx + let tx = system_transaction::transfer( + &mint_keypair, + &solana_pubkey::new_rand(), + 1, + genesis_config.hash(), + ); + let banking_packet_batch = BankingPacketBatch::new(to_packet_batches(&vec![tx.clone(); 1], 1)); + let tx = RuntimeTransaction::from_transaction_for_tests(tx); + + // Crate tpu_bank + let tpu_bank = Bank::new_from_parent(genesis_bank.clone(), &Pubkey::default(), 2); + let tpu_bank = bank_forks + .write() + .unwrap() + .insert_with_scheduling_mode(SchedulingMode::BlockProduction, tpu_bank); + poh_recorder + .write() + .unwrap() + .set_bank(tpu_bank.clone_with_scheduler(), false); + tpu_bank.unblock_block_production(); + let tpu_bank = bank_forks.read().unwrap().working_bank_with_scheduler(); + assert_eq!(tpu_bank.transaction_count(), 0); + + // Now, send transaction + channels + .unified_sender() + .send(banking_packet_batch) + .unwrap(); + + // Wait until tpu_bank reaches its tick height... + while poh_recorder.read().unwrap().bank().is_some() { + sleep(Duration::from_millis(100)); + } + assert_matches!(tpu_bank.wait_for_completed_scheduler(), Some((Ok(()), _))); + + // Verify transactions are committed and poh-recorded + assert_eq!(tpu_bank.transaction_count(), 1); + assert_matches!( + signal_receiver.into_iter().find(|(_, (entry, _))| !entry.is_tick()), + Some((_, (Entry {transactions, ..}, _))) if transactions == [tx.to_versioned_transaction()] + ); + + // Stop things. + exit.store(true, Ordering::Relaxed); + poh_service.join().unwrap(); +} diff --git a/unified-scheduler-pool/src/lib.rs b/unified-scheduler-pool/src/lib.rs index e11426e6a63cd4..b99d0214c84817 100644 --- a/unified-scheduler-pool/src/lib.rs +++ b/unified-scheduler-pool/src/lib.rs @@ -729,6 +729,7 @@ impl TaskHandler for DefaultTaskHandler { } = handler_context .transaction_recorder .record_transactions(bank.slot(), vec![transaction.to_versioned_transaction()]); + trace!("pre_commit_callback: poh: {result:?}"); match result { Ok(()) => Ok(starting_transaction_index), Err(_) => Err(TransactionError::CommitCancelled),