initializes thread-pools with lazy_static instead of thread_local (#24853)

In addition to thread_local -> lazy_static change, a number of thread-pools are
initialized with get_max_thread_count to achieve parity with the older code in
terms of number of validator threads.
This commit is contained in:
behzad nouri 2022-05-05 20:00:50 +00:00 committed by GitHub
parent 7100f1c94b
commit a01291069a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 237 additions and 279 deletions

2
Cargo.lock generated
View File

@ -4618,6 +4618,7 @@ dependencies = [
"fs_extra", "fs_extra",
"histogram", "histogram",
"itertools", "itertools",
"lazy_static",
"log", "log",
"lru", "lru",
"matches", "matches",
@ -4721,6 +4722,7 @@ dependencies = [
"crossbeam-channel", "crossbeam-channel",
"dlopen", "dlopen",
"dlopen_derive", "dlopen_derive",
"lazy_static",
"log", "log",
"matches", "matches",
"rand 0.7.3", "rand 0.7.3",

View File

@ -25,6 +25,7 @@ etcd-client = { version = "0.9.1", features = ["tls"] }
fs_extra = "1.2.0" fs_extra = "1.2.0"
histogram = "0.6.9" histogram = "0.6.9"
itertools = "0.10.3" itertools = "0.10.3"
lazy_static = "1.4.0"
log = "0.4.17" log = "0.4.17"
lru = "0.7.5" lru = "0.7.5"
min-max-heap = "1.3.0" min-max-heap = "1.3.0"

View File

@ -1,5 +1,6 @@
use { use {
crossbeam_channel::{Receiver, RecvTimeoutError, Sender}, crossbeam_channel::{Receiver, RecvTimeoutError, Sender},
lazy_static::lazy_static,
rayon::{prelude::*, ThreadPool}, rayon::{prelude::*, ThreadPool},
solana_gossip::cluster_info::ClusterInfo, solana_gossip::cluster_info::ClusterInfo,
solana_measure::measure::Measure, solana_measure::measure::Measure,
@ -9,7 +10,6 @@ use {
solana_sdk::timing::timestamp, solana_sdk::timing::timestamp,
solana_streamer::streamer::{self, StreamerError}, solana_streamer::streamer::{self, StreamerError},
std::{ std::{
cell::RefCell,
collections::HashMap, collections::HashMap,
net::IpAddr, net::IpAddr,
sync::{Arc, RwLock}, sync::{Arc, RwLock},
@ -20,11 +20,13 @@ use {
const IP_TO_STAKE_REFRESH_DURATION: Duration = Duration::from_secs(5); const IP_TO_STAKE_REFRESH_DURATION: Duration = Duration::from_secs(5);
thread_local!(static PAR_THREAD_POOL: RefCell<ThreadPool> = RefCell::new(rayon::ThreadPoolBuilder::new() lazy_static! {
.num_threads(get_thread_count()) static ref PAR_THREAD_POOL: ThreadPool = rayon::ThreadPoolBuilder::new()
.thread_name(|ix| format!("transaction_sender_stake_stage_{}", ix)) .num_threads(get_thread_count())
.build() .thread_name(|ix| format!("transaction_sender_stake_stage_{}", ix))
.unwrap())); .build()
.unwrap();
}
pub type FindPacketSenderStakeSender = Sender<Vec<PacketBatch>>; pub type FindPacketSenderStakeSender = Sender<Vec<PacketBatch>>;
pub type FindPacketSenderStakeReceiver = Receiver<Vec<PacketBatch>>; pub type FindPacketSenderStakeReceiver = Receiver<Vec<PacketBatch>>;
@ -166,16 +168,14 @@ impl FindPacketSenderStakeStage {
} }
fn apply_sender_stakes(batches: &mut [PacketBatch], ip_to_stake: &HashMap<IpAddr, u64>) { fn apply_sender_stakes(batches: &mut [PacketBatch], ip_to_stake: &HashMap<IpAddr, u64>) {
PAR_THREAD_POOL.with(|thread_pool| { PAR_THREAD_POOL.install(|| {
thread_pool.borrow().install(|| { batches
batches .into_par_iter()
.into_par_iter() .flat_map(|batch| batch.packets.par_iter_mut())
.flat_map(|batch| batch.packets.par_iter_mut()) .for_each(|packet| {
.for_each(|packet| { packet.meta.sender_stake =
packet.meta.sender_stake = *ip_to_stake.get(&packet.meta.addr().ip()).unwrap_or(&0);
*ip_to_stake.get(&packet.meta.addr().ip()).unwrap_or(&0); });
});
})
}); });
} }

View File

@ -14,6 +14,7 @@ bincode = "1.3.3"
crossbeam-channel = "0.5" crossbeam-channel = "0.5"
dlopen = "0.1.8" dlopen = "0.1.8"
dlopen_derive = "0.1.4" dlopen_derive = "0.1.4"
lazy_static = "1.4.0"
log = "0.4.17" log = "0.4.17"
rand = "0.7.0" rand = "0.7.0"
rayon = "1.5.2" rayon = "1.5.2"

View File

@ -7,6 +7,7 @@ use {
crossbeam_channel::{Receiver, Sender}, crossbeam_channel::{Receiver, Sender},
dlopen::symbor::{Container, SymBorApi, Symbol}, dlopen::symbor::{Container, SymBorApi, Symbol},
dlopen_derive::SymBorApi, dlopen_derive::SymBorApi,
lazy_static::lazy_static,
log::*, log::*,
rand::{thread_rng, Rng}, rand::{thread_rng, Rng},
rayon::{prelude::*, ThreadPool}, rayon::{prelude::*, ThreadPool},
@ -21,7 +22,7 @@ use {
recycler::Recycler, recycler::Recycler,
sigverify, sigverify,
}, },
solana_rayon_threadlimit::get_thread_count, solana_rayon_threadlimit::get_max_thread_count,
solana_sdk::{ solana_sdk::{
hash::Hash, hash::Hash,
packet::Meta, packet::Meta,
@ -32,7 +33,6 @@ use {
}, },
}, },
std::{ std::{
cell::RefCell,
cmp, cmp,
ffi::OsStr, ffi::OsStr,
sync::{ sync::{
@ -44,11 +44,15 @@ use {
}, },
}; };
thread_local!(static PAR_THREAD_POOL: RefCell<ThreadPool> = RefCell::new(rayon::ThreadPoolBuilder::new() // get_max_thread_count to match number of threads in the old code.
.num_threads(get_thread_count()) // see: https://github.com/solana-labs/solana/pull/24853
.thread_name(|ix| format!("entry_{}", ix)) lazy_static! {
.build() static ref PAR_THREAD_POOL: ThreadPool = rayon::ThreadPoolBuilder::new()
.unwrap())); .num_threads(get_max_thread_count())
.thread_name(|ix| format!("entry_{}", ix))
.build()
.unwrap();
}
pub type EntrySender = Sender<Vec<Entry>>; pub type EntrySender = Sender<Vec<Entry>>;
pub type EntryReceiver = Receiver<Vec<Entry>>; pub type EntryReceiver = Receiver<Vec<Entry>>;
@ -341,25 +345,22 @@ impl EntryVerificationState {
.expect("unwrap Arc") .expect("unwrap Arc")
.into_inner() .into_inner()
.expect("into_inner"); .expect("into_inner");
let res = PAR_THREAD_POOL.with(|thread_pool| { let res = PAR_THREAD_POOL.install(|| {
thread_pool.borrow().install(|| { hashes
hashes .into_par_iter()
.into_par_iter() .cloned()
.cloned() .zip(verification_state.verifications.take().unwrap())
.zip(verification_state.verifications.take().unwrap()) .all(|(hash, (action, expected))| {
.all(|(hash, (action, expected))| { let actual = match action {
let actual = match action { VerifyAction::Mixin(mixin) => {
VerifyAction::Mixin(mixin) => { Poh::new(hash, None).record(mixin).unwrap().hash
Poh::new(hash, None).record(mixin).unwrap().hash }
} VerifyAction::Tick => Poh::new(hash, None).tick().unwrap().hash,
VerifyAction::Tick => Poh::new(hash, None).tick().unwrap().hash, VerifyAction::None => hash,
VerifyAction::None => hash, };
}; actual == expected
actual == expected })
})
})
}); });
verify_check_time.stop(); verify_check_time.stop();
self.poh_duration_us += gpu_time_us + verify_check_time.as_us(); self.poh_duration_us += gpu_time_us + verify_check_time.as_us();
@ -381,25 +382,23 @@ pub fn verify_transactions(
entries: Vec<Entry>, entries: Vec<Entry>,
verify: Arc<dyn Fn(VersionedTransaction) -> Result<SanitizedTransaction> + Send + Sync>, verify: Arc<dyn Fn(VersionedTransaction) -> Result<SanitizedTransaction> + Send + Sync>,
) -> Result<Vec<EntryType>> { ) -> Result<Vec<EntryType>> {
PAR_THREAD_POOL.with(|thread_pool| { PAR_THREAD_POOL.install(|| {
thread_pool.borrow().install(|| { entries
entries .into_par_iter()
.into_par_iter() .map(|entry| {
.map(|entry| { if entry.transactions.is_empty() {
if entry.transactions.is_empty() { Ok(EntryType::Tick(entry.hash))
Ok(EntryType::Tick(entry.hash)) } else {
} else { Ok(EntryType::Transactions(
Ok(EntryType::Transactions( entry
entry .transactions
.transactions .into_par_iter()
.into_par_iter() .map(verify.as_ref())
.map(verify.as_ref()) .collect::<Result<Vec<_>>>()?,
.collect::<Result<Vec<_>>>()?, ))
)) }
} })
}) .collect()
.collect()
})
}) })
} }
@ -615,23 +614,20 @@ impl EntrySlice for [Entry] {
transactions: vec![], transactions: vec![],
}]; }];
let entry_pairs = genesis.par_iter().chain(self).zip(self); let entry_pairs = genesis.par_iter().chain(self).zip(self);
let res = PAR_THREAD_POOL.with(|thread_pool| { let res = PAR_THREAD_POOL.install(|| {
thread_pool.borrow().install(|| { entry_pairs.all(|(x0, x1)| {
entry_pairs.all(|(x0, x1)| { let r = x1.verify(&x0.hash);
let r = x1.verify(&x0.hash); if !r {
if !r { warn!(
warn!( "entry invalid!: x0: {:?}, x1: {:?} num txs: {}",
"entry invalid!: x0: {:?}, x1: {:?} num txs: {}", x0.hash,
x0.hash, x1.hash,
x1.hash, x1.transactions.len()
x1.transactions.len() );
); }
} r
r
})
}) })
}); });
let poh_duration_us = timing::duration_as_us(&now.elapsed()); let poh_duration_us = timing::duration_as_us(&now.elapsed());
EntryVerificationState { EntryVerificationState {
verification_status: if res { verification_status: if res {
@ -675,45 +671,43 @@ impl EntrySlice for [Entry] {
num_hashes.resize(aligned_len, 0); num_hashes.resize(aligned_len, 0);
let num_hashes: Vec<_> = num_hashes.chunks(simd_len).collect(); let num_hashes: Vec<_> = num_hashes.chunks(simd_len).collect();
let res = PAR_THREAD_POOL.with(|thread_pool| { let res = PAR_THREAD_POOL.install(|| {
thread_pool.borrow().install(|| { hashes_chunked
hashes_chunked .par_iter_mut()
.par_iter_mut() .zip(num_hashes)
.zip(num_hashes) .enumerate()
.enumerate() .all(|(i, (chunk, num_hashes))| {
.all(|(i, (chunk, num_hashes))| { match simd_len {
match simd_len { 8 => unsafe {
8 => unsafe { (api().unwrap().poh_verify_many_simd_avx2)(
(api().unwrap().poh_verify_many_simd_avx2)( chunk.as_mut_ptr(),
chunk.as_mut_ptr(), num_hashes.as_ptr(),
num_hashes.as_ptr(), );
); },
}, 16 => unsafe {
16 => unsafe { (api().unwrap().poh_verify_many_simd_avx512skx)(
(api().unwrap().poh_verify_many_simd_avx512skx)( chunk.as_mut_ptr(),
chunk.as_mut_ptr(), num_hashes.as_ptr(),
num_hashes.as_ptr(), );
); },
}, _ => {
_ => { panic!("unsupported simd len: {}", simd_len);
panic!("unsupported simd len: {}", simd_len);
}
} }
let entry_start = i * simd_len; }
// The last chunk may produce indexes larger than what we have in the reference entries let entry_start = i * simd_len;
// because it is aligned to simd_len. // The last chunk may produce indexes larger than what we have in the reference entries
let entry_end = std::cmp::min(entry_start + simd_len, self.len()); // because it is aligned to simd_len.
self[entry_start..entry_end] let entry_end = std::cmp::min(entry_start + simd_len, self.len());
.iter() self[entry_start..entry_end]
.enumerate() .iter()
.all(|(j, ref_entry)| { .enumerate()
let start = j * HASH_BYTES; .all(|(j, ref_entry)| {
let end = start + HASH_BYTES; let start = j * HASH_BYTES;
let hash = Hash::new(&chunk[start..end]); let end = start + HASH_BYTES;
compare_hashes(hash, ref_entry) let hash = Hash::new(&chunk[start..end]);
}) compare_hashes(hash, ref_entry)
}) })
}) })
}); });
let poh_duration_us = timing::duration_as_us(&now.elapsed()); let poh_duration_us = timing::duration_as_us(&now.elapsed());
EntryVerificationState { EntryVerificationState {
@ -812,26 +806,23 @@ impl EntrySlice for [Entry] {
timing::duration_as_us(&gpu_wait.elapsed()) timing::duration_as_us(&gpu_wait.elapsed())
}); });
let verifications = PAR_THREAD_POOL.with(|thread_pool| { let verifications = PAR_THREAD_POOL.install(|| {
thread_pool.borrow().install(|| { self.into_par_iter()
self.into_par_iter() .map(|entry| {
.map(|entry| { let answer = entry.hash;
let answer = entry.hash; let action = if entry.transactions.is_empty() {
let action = if entry.transactions.is_empty() { if entry.num_hashes == 0 {
if entry.num_hashes == 0 { VerifyAction::None
VerifyAction::None
} else {
VerifyAction::Tick
}
} else { } else {
VerifyAction::Mixin(hash_transactions(&entry.transactions)) VerifyAction::Tick
}; }
(action, answer) } else {
}) VerifyAction::Mixin(hash_transactions(&entry.transactions))
.collect() };
}) (action, answer)
})
.collect()
}); });
let device_verification_data = DeviceVerificationData::Gpu(GpuVerificationData { let device_verification_data = DeviceVerificationData::Gpu(GpuVerificationData {
thread_h: Some(gpu_verify_thread), thread_h: Some(gpu_verify_thread),
verifications: Some(verifications), verifications: Some(verifications),

View File

@ -28,7 +28,7 @@ use {
datapoint_debug, datapoint_error, datapoint_debug, datapoint_error,
poh_timing_point::{send_poh_timing_point, PohTimingSender, SlotPohTimingInfo}, poh_timing_point::{send_poh_timing_point, PohTimingSender, SlotPohTimingInfo},
}, },
solana_rayon_threadlimit::get_thread_count, solana_rayon_threadlimit::get_max_thread_count,
solana_runtime::hardened_unpack::{unpack_genesis_archive, MAX_GENESIS_ARCHIVE_UNPACKED_SIZE}, solana_runtime::hardened_unpack::{unpack_genesis_archive, MAX_GENESIS_ARCHIVE_UNPACKED_SIZE},
solana_sdk::{ solana_sdk::{
clock::{Slot, UnixTimestamp, DEFAULT_TICKS_PER_SECOND, MS_PER_TICK}, clock::{Slot, UnixTimestamp, DEFAULT_TICKS_PER_SECOND, MS_PER_TICK},
@ -75,17 +75,20 @@ pub use {
pub const BLOCKSTORE_DIRECTORY_ROCKS_LEVEL: &str = "rocksdb"; pub const BLOCKSTORE_DIRECTORY_ROCKS_LEVEL: &str = "rocksdb";
pub const BLOCKSTORE_DIRECTORY_ROCKS_FIFO: &str = "rocksdb_fifo"; pub const BLOCKSTORE_DIRECTORY_ROCKS_FIFO: &str = "rocksdb_fifo";
thread_local!(static PAR_THREAD_POOL: RefCell<ThreadPool> = RefCell::new(rayon::ThreadPoolBuilder::new() // get_max_thread_count to match number of threads in the old code.
.num_threads(get_thread_count()) // see: https://github.com/solana-labs/solana/pull/24853
.thread_name(|ix| format!("blockstore_{}", ix)) lazy_static! {
.build() static ref PAR_THREAD_POOL: ThreadPool = rayon::ThreadPoolBuilder::new()
.unwrap())); .num_threads(get_max_thread_count())
.thread_name(|ix| format!("blockstore_{}", ix))
thread_local!(static PAR_THREAD_POOL_ALL_CPUS: RefCell<ThreadPool> = RefCell::new(rayon::ThreadPoolBuilder::new() .build()
.num_threads(num_cpus::get()) .unwrap();
.thread_name(|ix| format!("blockstore_{}", ix)) static ref PAR_THREAD_POOL_ALL_CPUS: ThreadPool = rayon::ThreadPoolBuilder::new()
.build() .num_threads(num_cpus::get())
.unwrap())); .thread_name(|ix| format!("blockstore_{}", ix))
.build()
.unwrap();
}
pub const MAX_REPLAY_WAKE_UP_SIGNALS: usize = 1; pub const MAX_REPLAY_WAKE_UP_SIGNALS: usize = 1;
pub const MAX_COMPLETED_SLOTS_IN_CHANNEL: usize = 100_000; pub const MAX_COMPLETED_SLOTS_IN_CHANNEL: usize = 100_000;
@ -2790,22 +2793,14 @@ impl Blockstore {
.map(|(_, end_index)| u64::from(*end_index) - start_index + 1) .map(|(_, end_index)| u64::from(*end_index) - start_index + 1)
.unwrap_or(0); .unwrap_or(0);
let entries: Result<Vec<Vec<Entry>>> = PAR_THREAD_POOL.with(|thread_pool| { let entries: Result<Vec<Vec<Entry>>> = PAR_THREAD_POOL.install(|| {
thread_pool.borrow().install(|| { completed_ranges
completed_ranges .par_iter()
.par_iter() .map(|(start_index, end_index)| {
.map(|(start_index, end_index)| { self.get_entries_in_data_block(slot, *start_index, *end_index, Some(&slot_meta))
self.get_entries_in_data_block( })
slot, .collect()
*start_index,
*end_index,
Some(&slot_meta),
)
})
.collect()
})
}); });
let entries: Vec<Entry> = entries?.into_iter().flatten().collect(); let entries: Vec<Entry> = entries?.into_iter().flatten().collect();
Ok((entries, num_shreds, slot_meta.is_full())) Ok((entries, num_shreds, slot_meta.is_full()))
} }
@ -2935,23 +2930,15 @@ impl Blockstore {
} }
let slot_meta = slot_meta.unwrap(); let slot_meta = slot_meta.unwrap();
let entries: Vec<Vec<Entry>> = PAR_THREAD_POOL_ALL_CPUS.with(|thread_pool| { let entries: Vec<Vec<Entry>> = PAR_THREAD_POOL_ALL_CPUS.install(|| {
thread_pool.borrow().install(|| { completed_ranges
completed_ranges .par_iter()
.par_iter() .map(|(start_index, end_index)| {
.map(|(start_index, end_index)| { self.get_entries_in_data_block(slot, *start_index, *end_index, Some(&slot_meta))
self.get_entries_in_data_block(
slot,
*start_index,
*end_index,
Some(&slot_meta),
)
.unwrap_or_default() .unwrap_or_default()
}) })
.collect() .collect()
})
}); });
entries.into_iter().flatten().collect() entries.into_iter().flatten().collect()
} }

View File

@ -15,7 +15,7 @@ use {
solana_measure::measure::Measure, solana_measure::measure::Measure,
solana_metrics::{datapoint_error, inc_new_counter_debug}, solana_metrics::{datapoint_error, inc_new_counter_debug},
solana_program_runtime::timings::{ExecuteTimingType, ExecuteTimings}, solana_program_runtime::timings::{ExecuteTimingType, ExecuteTimings},
solana_rayon_threadlimit::get_thread_count, solana_rayon_threadlimit::{get_max_thread_count, get_thread_count},
solana_runtime::{ solana_runtime::{
accounts_background_service::AbsRequestSender, accounts_background_service::AbsRequestSender,
accounts_db::{AccountShrinkThreshold, AccountsDbConfig}, accounts_db::{AccountShrinkThreshold, AccountsDbConfig},
@ -55,7 +55,6 @@ use {
}, },
std::{ std::{
borrow::Cow, borrow::Cow,
cell::RefCell,
collections::{HashMap, HashSet}, collections::{HashMap, HashSet},
path::PathBuf, path::PathBuf,
result, result,
@ -94,12 +93,15 @@ impl BlockCostCapacityMeter {
} }
} }
thread_local!(static PAR_THREAD_POOL: RefCell<ThreadPool> = RefCell::new(rayon::ThreadPoolBuilder::new() // get_max_thread_count to match number of threads in the old code.
.num_threads(get_thread_count()) // see: https://github.com/solana-labs/solana/pull/24853
.thread_name(|ix| format!("blockstore_processor_{}", ix)) lazy_static! {
.build() static ref PAR_THREAD_POOL: ThreadPool = rayon::ThreadPoolBuilder::new()
.unwrap()) .num_threads(get_max_thread_count())
); .thread_name(|ix| format!("blockstore_processor_{}", ix))
.build()
.unwrap();
}
fn first_err(results: &[Result<()>]) -> Result<()> { fn first_err(results: &[Result<()>]) -> Result<()> {
for r in results { for r in results {
@ -263,29 +265,26 @@ fn execute_batches_internal(
) -> Result<()> { ) -> Result<()> {
inc_new_counter_debug!("bank-par_execute_entries-count", batches.len()); inc_new_counter_debug!("bank-par_execute_entries-count", batches.len());
let (results, new_timings): (Vec<Result<()>>, Vec<ExecuteTimings>) = let (results, new_timings): (Vec<Result<()>>, Vec<ExecuteTimings>) =
PAR_THREAD_POOL.with(|thread_pool| { PAR_THREAD_POOL.install(|| {
thread_pool.borrow().install(|| { batches
batches .into_par_iter()
.into_par_iter() .map(|batch| {
.map(|batch| { let mut timings = ExecuteTimings::default();
let mut timings = ExecuteTimings::default(); let result = execute_batch(
let result = execute_batch( batch,
batch, bank,
bank, transaction_status_sender,
transaction_status_sender, replay_vote_sender,
replay_vote_sender, &mut timings,
&mut timings, cost_capacity_meter.clone(),
cost_capacity_meter.clone(), );
); if let Some(entry_callback) = entry_callback {
if let Some(entry_callback) = entry_callback { entry_callback(bank);
entry_callback(bank); }
} (result, timings)
(result, timings) })
}) .unzip()
.unzip()
})
}); });
timings.saturating_add_in_place(ExecuteTimingType::TotalBatchesLen, batches.len() as u64); timings.saturating_add_in_place(ExecuteTimingType::TotalBatchesLen, batches.len() as u64);
timings.saturating_add_in_place(ExecuteTimingType::NumExecuteBatches, 1); timings.saturating_add_in_place(ExecuteTimingType::NumExecuteBatches, 1);
for timing in new_timings { for timing in new_timings {
@ -546,7 +545,6 @@ pub struct ProcessOptions {
pub full_leader_cache: bool, pub full_leader_cache: bool,
pub halt_at_slot: Option<Slot>, pub halt_at_slot: Option<Slot>,
pub entry_callback: Option<ProcessCallback>, pub entry_callback: Option<ProcessCallback>,
pub override_num_threads: Option<usize>,
pub new_hard_forks: Option<Vec<Slot>>, pub new_hard_forks: Option<Vec<Slot>>,
pub debug_keys: Option<Arc<HashSet<Pubkey>>>, pub debug_keys: Option<Arc<HashSet<Pubkey>>>,
pub account_indexes: AccountSecondaryIndexes, pub account_indexes: AccountSecondaryIndexes,
@ -635,15 +633,6 @@ pub fn process_blockstore_from_root(
cache_block_meta_sender: Option<&CacheBlockMetaSender>, cache_block_meta_sender: Option<&CacheBlockMetaSender>,
accounts_background_request_sender: &AbsRequestSender, accounts_background_request_sender: &AbsRequestSender,
) -> result::Result<(), BlockstoreProcessorError> { ) -> result::Result<(), BlockstoreProcessorError> {
if let Some(num_threads) = opts.override_num_threads {
PAR_THREAD_POOL.with(|pool| {
*pool.borrow_mut() = rayon::ThreadPoolBuilder::new()
.num_threads(num_threads)
.build()
.unwrap()
});
}
// Starting slot must be a root, and thus has no parents // Starting slot must be a root, and thus has no parents
assert_eq!(bank_forks.read().unwrap().banks().len(), 1); assert_eq!(bank_forks.read().unwrap().banks().len(), 1);
let bank = bank_forks.read().unwrap().root_bank(); let bank = bank_forks.read().unwrap().root_bank();
@ -2409,23 +2398,6 @@ pub mod tests {
assert_eq!(bank.tick_height(), 1); assert_eq!(bank.tick_height(), 1);
} }
#[test]
fn test_process_ledger_options_override_threads() {
let GenesisConfigInfo { genesis_config, .. } = create_genesis_config(123);
let (ledger_path, _blockhash) = create_new_tmp_ledger_auto_delete!(&genesis_config);
let blockstore = Blockstore::open(ledger_path.path()).unwrap();
let opts = ProcessOptions {
override_num_threads: Some(1),
accounts_db_test_hash_calculation: true,
..ProcessOptions::default()
};
test_process_blockstore(&genesis_config, &blockstore, &opts);
PAR_THREAD_POOL.with(|pool| {
assert_eq!(pool.borrow().current_num_threads(), 1);
});
}
#[test] #[test]
fn test_process_ledger_options_full_leader_cache() { fn test_process_ledger_options_full_leader_cache() {
let GenesisConfigInfo { genesis_config, .. } = create_genesis_config(123); let GenesisConfigInfo { genesis_config, .. } = create_genesis_config(123);
@ -2493,7 +2465,6 @@ pub mod tests {
}; };
let opts = ProcessOptions { let opts = ProcessOptions {
override_num_threads: Some(1),
entry_callback: Some(entry_callback), entry_callback: Some(entry_callback),
accounts_db_test_hash_calculation: true, accounts_db_test_hash_calculation: true,
..ProcessOptions::default() ..ProcessOptions::default()

View File

@ -5,6 +5,7 @@ use {
}, },
shred_stats::ProcessShredsStats, shred_stats::ProcessShredsStats,
}, },
lazy_static::lazy_static,
rayon::{prelude::*, ThreadPool}, rayon::{prelude::*, ThreadPool},
reed_solomon_erasure::{ reed_solomon_erasure::{
galois_8::Field, galois_8::Field,
@ -14,14 +15,16 @@ use {
solana_measure::measure::Measure, solana_measure::measure::Measure,
solana_rayon_threadlimit::get_thread_count, solana_rayon_threadlimit::get_thread_count,
solana_sdk::{clock::Slot, signature::Keypair}, solana_sdk::{clock::Slot, signature::Keypair},
std::{cell::RefCell, fmt::Debug}, std::fmt::Debug,
}; };
thread_local!(static PAR_THREAD_POOL: RefCell<ThreadPool> = RefCell::new(rayon::ThreadPoolBuilder::new() lazy_static! {
.num_threads(get_thread_count()) static ref PAR_THREAD_POOL: ThreadPool = rayon::ThreadPoolBuilder::new()
.thread_name(|ix| format!("shredder_{}", ix)) .num_threads(get_thread_count())
.build() .thread_name(|ix| format!("shredder_{}", ix))
.unwrap())); .build()
.unwrap();
}
type ReedSolomon = reed_solomon_erasure::ReedSolomon<Field>; type ReedSolomon = reed_solomon_erasure::ReedSolomon<Field>;
@ -138,17 +141,15 @@ impl Shredder {
shred.sign(keypair); shred.sign(keypair);
shred shred
}; };
let data_shreds: Vec<Shred> = PAR_THREAD_POOL.with(|thread_pool| { let data_shreds: Vec<Shred> = PAR_THREAD_POOL.install(|| {
thread_pool.borrow().install(|| { serialized_shreds
serialized_shreds .par_chunks(payload_capacity)
.par_chunks(payload_capacity) .enumerate()
.enumerate() .map(|(i, shred_data)| {
.map(|(i, shred_data)| { let shred_index = next_shred_index + i as u32;
let shred_index = next_shred_index + i as u32; make_data_shred(shred_index, shred_data)
make_data_shred(shred_index, shred_data) })
}) .collect()
.collect()
})
}); });
gen_data_time.stop(); gen_data_time.stop();
@ -170,43 +171,39 @@ impl Shredder {
} }
let mut gen_coding_time = Measure::start("gen_coding_shreds"); let mut gen_coding_time = Measure::start("gen_coding_shreds");
// 1) Generate coding shreds // 1) Generate coding shreds
let mut coding_shreds: Vec<_> = PAR_THREAD_POOL.with(|thread_pool| { let mut coding_shreds: Vec<_> = PAR_THREAD_POOL.install(|| {
thread_pool.borrow().install(|| { data_shreds
data_shreds .par_chunks(MAX_DATA_SHREDS_PER_FEC_BLOCK as usize)
.par_chunks(MAX_DATA_SHREDS_PER_FEC_BLOCK as usize) .enumerate()
.enumerate() .flat_map(|(i, shred_data_batch)| {
.flat_map(|(i, shred_data_batch)| { // Assumption here is that, for now, each fec block has
// Assumption here is that, for now, each fec block has // as many coding shreds as data shreds (except for the
// as many coding shreds as data shreds (except for the // last one in the slot).
// last one in the slot). // TODO: tie this more closely with
// TODO: tie this more closely with // generate_coding_shreds.
// generate_coding_shreds. let next_code_index = next_code_index
let next_code_index = next_code_index .checked_add(
.checked_add( u32::try_from(i)
u32::try_from(i) .unwrap()
.unwrap() .checked_mul(MAX_DATA_SHREDS_PER_FEC_BLOCK)
.checked_mul(MAX_DATA_SHREDS_PER_FEC_BLOCK) .unwrap(),
.unwrap(),
)
.unwrap();
Shredder::generate_coding_shreds(
shred_data_batch,
is_last_in_slot,
next_code_index,
) )
}) .unwrap();
.collect() Shredder::generate_coding_shreds(
}) shred_data_batch,
is_last_in_slot,
next_code_index,
)
})
.collect()
}); });
gen_coding_time.stop(); gen_coding_time.stop();
let mut sign_coding_time = Measure::start("sign_coding_shreds"); let mut sign_coding_time = Measure::start("sign_coding_shreds");
// 2) Sign coding shreds // 2) Sign coding shreds
PAR_THREAD_POOL.with(|thread_pool| { PAR_THREAD_POOL.install(|| {
thread_pool.borrow().install(|| { coding_shreds.par_iter_mut().for_each(|coding_shred| {
coding_shreds.par_iter_mut().for_each(|coding_shred| { coding_shred.sign(keypair);
coding_shred.sign(keypair);
})
}) })
}); });
sign_coding_time.stop(); sign_coding_time.stop();

View File

@ -4292,6 +4292,7 @@ dependencies = [
"fs_extra", "fs_extra",
"histogram", "histogram",
"itertools", "itertools",
"lazy_static",
"log", "log",
"lru", "lru",
"min-max-heap", "min-max-heap",
@ -4353,6 +4354,7 @@ dependencies = [
"crossbeam-channel", "crossbeam-channel",
"dlopen", "dlopen",
"dlopen_derive", "dlopen_derive",
"lazy_static",
"log", "log",
"rand 0.7.3", "rand 0.7.3",
"rayon", "rayon",

View File

@ -16,3 +16,9 @@ lazy_static! {
pub fn get_thread_count() -> usize { pub fn get_thread_count() -> usize {
*MAX_RAYON_THREADS *MAX_RAYON_THREADS
} }
// Only used in legacy code.
// Use get_thread_count instead in all new code.
pub fn get_max_thread_count() -> usize {
get_thread_count().saturating_mul(2)
}