Convert Measure::this to measure! and remove Measure::this (#25776)

* Remove the args param from Measure::this since we don't ever use it

* banking_stage.rs: convert to measure!

* poh_recorder.rs: convert to measure!

* cost_update_service.rs: convert to measure!

* poh_service.rs: convert to measure!

* bank.rs: convert to measure!

* measure.rs: Remove Measure::this now that all have been converted to measure!
This commit is contained in:
apfitzge 2022-06-06 20:21:05 -05:00 committed by GitHub
parent cbb0f07d54
commit e6c21a3036
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 214 additions and 428 deletions

View File

@ -24,7 +24,7 @@ use {
solana_entry::entry::hash_transactions, solana_entry::entry::hash_transactions,
solana_gossip::{cluster_info::ClusterInfo, contact_info::ContactInfo}, solana_gossip::{cluster_info::ClusterInfo, contact_info::ContactInfo},
solana_ledger::blockstore_processor::TransactionStatusSender, solana_ledger::blockstore_processor::TransactionStatusSender,
solana_measure::measure::Measure, solana_measure::{measure, measure::Measure},
solana_metrics::inc_new_counter_info, solana_metrics::inc_new_counter_info,
solana_perf::{ solana_perf::{
data_budget::DataBudget, data_budget::DataBudget,
@ -626,9 +626,8 @@ impl BankingStage {
let packets_to_process = packets_to_process.into_iter().collect_vec(); let packets_to_process = packets_to_process.into_iter().collect_vec();
// TODO: Right now we iterate through buffer and try the highest weighted transaction once // TODO: Right now we iterate through buffer and try the highest weighted transaction once
// but we should retry the highest weighted transactions more often. // but we should retry the highest weighted transactions more often.
let (bank_start, poh_recorder_lock_time) = Measure::this( let (bank_start, poh_recorder_lock_time) = measure!(
|_| poh_recorder.lock().unwrap().bank_start(), poh_recorder.lock().unwrap().bank_start(),
(),
"poh_recorder_lock", "poh_recorder_lock",
); );
slot_metrics_tracker.increment_consume_buffered_packets_poh_recorder_lock_us( slot_metrics_tracker.increment_consume_buffered_packets_poh_recorder_lock_us(
@ -642,8 +641,7 @@ impl BankingStage {
}) = bank_start }) = bank_start
{ {
let (process_transactions_summary, process_packets_transactions_time) = let (process_transactions_summary, process_packets_transactions_time) =
Measure::this( measure!(
|_| {
Self::process_packets_transactions( Self::process_packets_transactions(
&working_bank, &working_bank,
&bank_creation_time, &bank_creation_time,
@ -654,9 +652,7 @@ impl BankingStage {
banking_stage_stats, banking_stage_stats,
qos_service, qos_service,
slot_metrics_tracker, slot_metrics_tracker,
) ),
},
(),
"process_packets_transactions", "process_packets_transactions",
); );
slot_metrics_tracker.increment_process_packets_transactions_us( slot_metrics_tracker.increment_process_packets_transactions_us(
@ -676,11 +672,8 @@ impl BankingStage {
) )
{ {
let poh_recorder_lock_time = { let poh_recorder_lock_time = {
let (poh_recorder_locked, poh_recorder_lock_time) = Measure::this( let (poh_recorder_locked, poh_recorder_lock_time) =
|_| poh_recorder.lock().unwrap(), measure!(poh_recorder.lock().unwrap(), "poh_recorder_lock");
(),
"poh_recorder_lock",
);
reached_end_of_slot = Some(EndOfSlot { reached_end_of_slot = Some(EndOfSlot {
next_slot_leader: poh_recorder_locked.next_slot_leader(), next_slot_leader: poh_recorder_locked.next_slot_leader(),
@ -744,11 +737,8 @@ impl BankingStage {
// mark as end-of-slot to avoid aggressively lock poh for the remaining for // mark as end-of-slot to avoid aggressively lock poh for the remaining for
// packet batches in buffer // packet batches in buffer
let poh_recorder_lock_time = { let poh_recorder_lock_time = {
let (poh_recorder_locked, poh_recorder_lock_time) = Measure::this( let (poh_recorder_locked, poh_recorder_lock_time) =
|_| poh_recorder.lock().unwrap(), measure!(poh_recorder.lock().unwrap(), "poh_recorder_lock");
(),
"poh_recorder_lock",
);
reached_end_of_slot = Some(EndOfSlot { reached_end_of_slot = Some(EndOfSlot {
next_slot_leader: poh_recorder_locked.next_slot_leader(), next_slot_leader: poh_recorder_locked.next_slot_leader(),
@ -883,8 +873,8 @@ impl BankingStage {
qos_service: &QosService, qos_service: &QosService,
slot_metrics_tracker: &mut LeaderSlotMetricsTracker, slot_metrics_tracker: &mut LeaderSlotMetricsTracker,
) { ) {
let (decision, make_decision_time) = Measure::this( let (decision, make_decision_time) = measure!(
|_| { {
let bank_start; let bank_start;
let ( let (
leader_at_slot_offset, leader_at_slot_offset,
@ -913,15 +903,13 @@ impl BankingStage {
would_be_leader_shortly, would_be_leader_shortly,
) )
}, },
(),
"make_decision", "make_decision",
); );
slot_metrics_tracker.increment_make_decision_us(make_decision_time.as_us()); slot_metrics_tracker.increment_make_decision_us(make_decision_time.as_us());
match decision { match decision {
BufferedPacketsDecision::Consume(max_tx_ingestion_ns) => { BufferedPacketsDecision::Consume(max_tx_ingestion_ns) => {
let (_, consume_buffered_packets_time) = Measure::this( let (_, consume_buffered_packets_time) = measure!(
|_| {
Self::consume_buffered_packets( Self::consume_buffered_packets(
my_pubkey, my_pubkey,
max_tx_ingestion_ns, max_tx_ingestion_ns,
@ -935,17 +923,14 @@ impl BankingStage {
qos_service, qos_service,
slot_metrics_tracker, slot_metrics_tracker,
UNPROCESSED_BUFFER_STEP_SIZE, UNPROCESSED_BUFFER_STEP_SIZE,
) ),
},
(),
"consume_buffered_packets", "consume_buffered_packets",
); );
slot_metrics_tracker slot_metrics_tracker
.increment_consume_buffered_packets_us(consume_buffered_packets_time.as_us()); .increment_consume_buffered_packets_us(consume_buffered_packets_time.as_us());
} }
BufferedPacketsDecision::Forward => { BufferedPacketsDecision::Forward => {
let (_, forward_time) = Measure::this( let (_, forward_time) = measure!(
|_| {
Self::handle_forwarding( Self::handle_forwarding(
forward_option, forward_option,
cluster_info, cluster_info,
@ -955,16 +940,13 @@ impl BankingStage {
data_budget, data_budget,
slot_metrics_tracker, slot_metrics_tracker,
banking_stage_stats, banking_stage_stats,
) ),
},
(),
"forward", "forward",
); );
slot_metrics_tracker.increment_forward_us(forward_time.as_us()); slot_metrics_tracker.increment_forward_us(forward_time.as_us());
} }
BufferedPacketsDecision::ForwardAndHold => { BufferedPacketsDecision::ForwardAndHold => {
let (_, forward_and_hold_time) = Measure::this( let (_, forward_and_hold_time) = measure!(
|_| {
Self::handle_forwarding( Self::handle_forwarding(
forward_option, forward_option,
cluster_info, cluster_info,
@ -974,9 +956,7 @@ impl BankingStage {
data_budget, data_budget,
slot_metrics_tracker, slot_metrics_tracker,
banking_stage_stats, banking_stage_stats,
) ),
},
(),
"forward_and_hold", "forward_and_hold",
); );
slot_metrics_tracker.increment_forward_and_hold_us(forward_and_hold_time.as_us()); slot_metrics_tracker.increment_forward_and_hold_us(forward_and_hold_time.as_us());
@ -1064,8 +1044,7 @@ impl BankingStage {
loop { loop {
let my_pubkey = cluster_info.id(); let my_pubkey = cluster_info.id();
if !buffered_packet_batches.is_empty() { if !buffered_packet_batches.is_empty() {
let (_, process_buffered_packets_time) = Measure::this( let (_, process_buffered_packets_time) = measure!(
|_| {
Self::process_buffered_packets( Self::process_buffered_packets(
&my_pubkey, &my_pubkey,
poh_recorder, poh_recorder,
@ -1079,9 +1058,7 @@ impl BankingStage {
data_budget, data_budget,
&qos_service, &qos_service,
&mut slot_metrics_tracker, &mut slot_metrics_tracker,
) ),
},
(),
"process_buffered_packets", "process_buffered_packets",
); );
slot_metrics_tracker slot_metrics_tracker
@ -1089,15 +1066,14 @@ impl BankingStage {
} }
if last_metrics_update.elapsed() >= SLOT_BOUNDARY_CHECK_PERIOD { if last_metrics_update.elapsed() >= SLOT_BOUNDARY_CHECK_PERIOD {
let (_, slot_metrics_checker_check_slot_boundary_time) = Measure::this( let (_, slot_metrics_checker_check_slot_boundary_time) = measure!(
|_| { {
let current_poh_bank = { let current_poh_bank = {
let poh = poh_recorder.lock().unwrap(); let poh = poh_recorder.lock().unwrap();
poh.bank_start() poh.bank_start()
}; };
slot_metrics_tracker.update_on_leader_slot_boundary(&current_poh_bank); slot_metrics_tracker.update_on_leader_slot_boundary(&current_poh_bank);
}, },
(),
"slot_metrics_checker_check_slot_boundary", "slot_metrics_checker_check_slot_boundary",
); );
slot_metrics_tracker.increment_slot_metrics_check_slot_boundary_us( slot_metrics_tracker.increment_slot_metrics_check_slot_boundary_us(
@ -1118,8 +1094,7 @@ impl BankingStage {
Duration::from_millis(100) Duration::from_millis(100)
}; };
let (res, receive_and_buffer_packets_time) = Measure::this( let (res, receive_and_buffer_packets_time) = measure!(
|_| {
Self::receive_and_buffer_packets( Self::receive_and_buffer_packets(
verified_receiver, verified_receiver,
recv_start, recv_start,
@ -1128,9 +1103,7 @@ impl BankingStage {
&mut buffered_packet_batches, &mut buffered_packet_batches,
&mut banking_stage_stats, &mut banking_stage_stats,
&mut slot_metrics_tracker, &mut slot_metrics_tracker,
) ),
},
(),
"receive_and_buffer_packets", "receive_and_buffer_packets",
); );
slot_metrics_tracker slot_metrics_tracker
@ -1166,14 +1139,11 @@ impl BankingStage {
inc_new_counter_info!("banking_stage-record_count", 1); inc_new_counter_info!("banking_stage-record_count", 1);
inc_new_counter_info!("banking_stage-record_transactions", num_to_record); inc_new_counter_info!("banking_stage-record_transactions", num_to_record);
let (hash, hash_time) = Measure::this(|_| hash_transactions(&transactions), (), "hash"); let (hash, hash_time) = measure!(hash_transactions(&transactions), "hash");
record_transactions_timings.hash_us = hash_time.as_us(); record_transactions_timings.hash_us = hash_time.as_us();
let (res, poh_record_time) = Measure::this( let (res, poh_record_time) =
|_| recorder.record(bank_slot, hash, transactions), measure!(recorder.record(bank_slot, hash, transactions), "hash");
(),
"hash",
);
record_transactions_timings.poh_record_us = poh_record_time.as_us(); record_transactions_timings.poh_record_us = poh_record_time.as_us();
match res { match res {
@ -1209,8 +1179,8 @@ impl BankingStage {
let mut execute_and_commit_timings = LeaderExecuteAndCommitTimings::default(); let mut execute_and_commit_timings = LeaderExecuteAndCommitTimings::default();
let mut mint_decimals: HashMap<Pubkey, u8> = HashMap::new(); let mut mint_decimals: HashMap<Pubkey, u8> = HashMap::new();
let ((pre_balances, pre_token_balances), collect_balances_time) = Measure::this( let ((pre_balances, pre_token_balances), collect_balances_time) = measure!(
|_| { {
// Use a shorter maximum age when adding transactions into the pipeline. This will reduce // Use a shorter maximum age when adding transactions into the pipeline. This will reduce
// the likelihood of any single thread getting starved and processing old ids. // the likelihood of any single thread getting starved and processing old ids.
// TODO: Banking stage threads should be prioritized to complete faster then this queue // TODO: Banking stage threads should be prioritized to complete faster then this queue
@ -1229,13 +1199,11 @@ impl BankingStage {
(pre_balances, pre_token_balances) (pre_balances, pre_token_balances)
}, },
(),
"collect_balances", "collect_balances",
); );
execute_and_commit_timings.collect_balances_us = collect_balances_time.as_us(); execute_and_commit_timings.collect_balances_us = collect_balances_time.as_us();
let (load_and_execute_transactions_output, load_execute_time) = Measure::this( let (load_and_execute_transactions_output, load_execute_time) = measure!(
|_| {
bank.load_and_execute_transactions( bank.load_and_execute_transactions(
batch, batch,
MAX_PROCESSING_AGE, MAX_PROCESSING_AGE,
@ -1244,9 +1212,7 @@ impl BankingStage {
transaction_status_sender.is_some(), transaction_status_sender.is_some(),
&mut execute_and_commit_timings.execute_timings, &mut execute_and_commit_timings.execute_timings,
None, // account_overrides None, // account_overrides
) ),
},
(),
"load_execute", "load_execute",
); );
execute_and_commit_timings.load_execute_us = load_execute_time.as_us(); execute_and_commit_timings.load_execute_us = load_execute_time.as_us();
@ -1263,9 +1229,7 @@ impl BankingStage {
} = load_and_execute_transactions_output; } = load_and_execute_transactions_output;
let transactions_attempted_execution_count = execution_results.len(); let transactions_attempted_execution_count = execution_results.len();
let (executed_transactions, execution_results_to_transactions_time): (Vec<_>, Measure) = let (executed_transactions, execution_results_to_transactions_time): (Vec<_>, Measure) = measure!(
Measure::this(
|_| {
execution_results execution_results
.iter() .iter()
.zip(batch.sanitized_transactions()) .zip(batch.sanitized_transactions())
@ -1276,21 +1240,17 @@ impl BankingStage {
None None
} }
}) })
.collect() .collect(),
},
(),
"execution_results_to_transactions", "execution_results_to_transactions",
); );
let (last_blockhash, lamports_per_signature) = let (last_blockhash, lamports_per_signature) =
bank.last_blockhash_and_lamports_per_signature(); bank.last_blockhash_and_lamports_per_signature();
let (freeze_lock, freeze_lock_time) = let (freeze_lock, freeze_lock_time) = measure!(bank.freeze_lock(), "freeze_lock");
Measure::this(|_| bank.freeze_lock(), (), "freeze_lock");
execute_and_commit_timings.freeze_lock_us = freeze_lock_time.as_us(); execute_and_commit_timings.freeze_lock_us = freeze_lock_time.as_us();
let (record_transactions_summary, record_time) = Measure::this( let (record_transactions_summary, record_time) = measure!(
|_| Self::record_transactions(bank.slot(), executed_transactions, poh), Self::record_transactions(bank.slot(), executed_transactions, poh),
(),
"record_transactions", "record_transactions",
); );
execute_and_commit_timings.record_us = record_time.as_us(); execute_and_commit_timings.record_us = record_time.as_us();
@ -1332,8 +1292,7 @@ impl BankingStage {
executed_transactions_count executed_transactions_count
); );
let (tx_results, commit_time) = Measure::this( let (tx_results, commit_time) = measure!(
|_| {
bank.commit_transactions( bank.commit_transactions(
sanitized_txs, sanitized_txs,
&mut loaded_transactions, &mut loaded_transactions,
@ -1348,9 +1307,7 @@ impl BankingStage {
signature_count, signature_count,
}, },
&mut execute_and_commit_timings.execute_timings, &mut execute_and_commit_timings.execute_timings,
) ),
},
(),
"commit", "commit",
); );
let commit_time_us = commit_time.as_us(); let commit_time_us = commit_time.as_us();
@ -1367,8 +1324,8 @@ impl BankingStage {
}) })
.collect(); .collect();
let (_, find_and_send_votes_time) = Measure::this( let (_, find_and_send_votes_time) = measure!(
|_| { {
bank_utils::find_and_send_votes( bank_utils::find_and_send_votes(
sanitized_txs, sanitized_txs,
&tx_results, &tx_results,
@ -1392,7 +1349,6 @@ impl BankingStage {
); );
} }
}, },
(),
"find_and_send_votes", "find_and_send_votes",
); );
execute_and_commit_timings.find_and_send_votes_us = find_and_send_votes_time.as_us(); execute_and_commit_timings.find_and_send_votes_us = find_and_send_votes_time.as_us();
@ -1881,8 +1837,7 @@ impl BankingStage {
let ((transactions, transaction_to_packet_indexes), packet_conversion_time): ( let ((transactions, transaction_to_packet_indexes), packet_conversion_time): (
(Vec<SanitizedTransaction>, Vec<usize>), (Vec<SanitizedTransaction>, Vec<usize>),
_, _,
) = Measure::this( ) = measure!(
|_| {
deserialized_packets deserialized_packets
.enumerate() .enumerate()
.filter_map(|(i, deserialized_packet)| { .filter_map(|(i, deserialized_packet)| {
@ -1894,9 +1849,7 @@ impl BankingStage {
) )
.map(|transaction| (transaction, i)) .map(|transaction| (transaction, i))
}) })
.unzip() .unzip(),
},
(),
"packet_conversion", "packet_conversion",
); );
@ -1908,8 +1861,7 @@ impl BankingStage {
inc_new_counter_info!("banking_stage-packet_conversion", 1); inc_new_counter_info!("banking_stage-packet_conversion", 1);
// Process transactions // Process transactions
let (mut process_transactions_summary, process_transactions_time) = Measure::this( let (mut process_transactions_summary, process_transactions_time) = measure!(
|_| {
Self::process_transactions( Self::process_transactions(
bank, bank,
bank_creation_time, bank_creation_time,
@ -1918,9 +1870,7 @@ impl BankingStage {
transaction_status_sender, transaction_status_sender,
gossip_vote_sender, gossip_vote_sender,
qos_service, qos_service,
) ),
},
(),
"process_transaction_time", "process_transaction_time",
); );
let process_transactions_us = process_transactions_time.as_us(); let process_transactions_us = process_transactions_time.as_us();
@ -1942,16 +1892,13 @@ impl BankingStage {
inc_new_counter_info!("banking_stage-unprocessed_transactions", retryable_tx_count); inc_new_counter_info!("banking_stage-unprocessed_transactions", retryable_tx_count);
// Filter out the retryable transactions that are too old // Filter out the retryable transactions that are too old
let (filtered_retryable_transaction_indexes, filter_retryable_packets_time) = Measure::this( let (filtered_retryable_transaction_indexes, filter_retryable_packets_time) = measure!(
|_| {
Self::filter_pending_packets_from_pending_txs( Self::filter_pending_packets_from_pending_txs(
bank, bank,
&transactions, &transactions,
&transaction_to_packet_indexes, &transaction_to_packet_indexes,
retryable_transaction_indexes, retryable_transaction_indexes,
) ),
},
(),
"filter_pending_packets_time", "filter_pending_packets_time",
); );
let filter_retryable_packets_us = filter_retryable_packets_time.as_us(); let filter_retryable_packets_us = filter_retryable_packets_time.as_us();

View File

@ -6,7 +6,7 @@
use { use {
crossbeam_channel::Receiver, crossbeam_channel::Receiver,
solana_ledger::blockstore::Blockstore, solana_ledger::blockstore::Blockstore,
solana_measure::measure::Measure, solana_measure::measure,
solana_program_runtime::timings::ExecuteTimings, solana_program_runtime::timings::ExecuteTimings,
solana_runtime::{bank::Bank, cost_model::CostModel}, solana_runtime::{bank::Bank, cost_model::CostModel},
solana_sdk::timing::timestamp, solana_sdk::timing::timestamp,
@ -102,9 +102,8 @@ impl CostUpdateService {
CostUpdate::ExecuteTiming { CostUpdate::ExecuteTiming {
mut execute_timings, mut execute_timings,
} => { } => {
let (update_count, update_cost_model_time) = Measure::this( let (update_count, update_cost_model_time) = measure!(
|_| Self::update_cost_model(&cost_model, &mut execute_timings), Self::update_cost_model(&cost_model, &mut execute_timings),
(),
"update_cost_model_time", "update_cost_model_time",
); );
cost_update_service_timing.update(update_count, update_cost_model_time.as_us()); cost_update_service_timing.update(update_count, update_cost_model_time.as_us());

View File

@ -38,50 +38,6 @@ impl Measure {
pub fn as_s(&self) -> f32 { pub fn as_s(&self) -> f32 {
self.duration as f32 / (1000.0f32 * 1000.0f32 * 1000.0f32) self.duration as f32 / (1000.0f32 * 1000.0f32 * 1000.0f32)
} }
/// Measure this function
///
/// Use `Measure::this()` when you have a function that you want to measure. `this()` will
/// start a new `Measure`, call your function, stop the measure, then return the `Measure`
/// object along with your function's return value.
///
/// If your function takes more than one parameter, you will need to wrap your function in a
/// closure, and wrap the arguments in a tuple. The same thing applies to methods. See the
/// tests for more details.
///
/// # Examples
///
/// ```
/// // Call a function with a single argument
/// # use solana_measure::measure::Measure;
/// # fn my_function(fizz: i32) -> i32 { fizz }
/// let (result, measure) = Measure::this(my_function, 42, "my_func");
/// # assert_eq!(result, 42);
/// ```
///
/// ```
/// // Call a function with multiple arguments
/// # use solana_measure::measure::Measure;
/// let (result, measure) = Measure::this(|(arg1, arg2)| std::cmp::min(arg1, arg2), (42, 123), "minimum");
/// # assert_eq!(result, 42);
/// ```
///
/// ```
/// // Call a method
/// # use solana_measure::measure::Measure;
/// # struct Foo { x: i32 }
/// # impl Foo { fn bar(&self, arg: i32) -> i32 { self.x + arg } }
/// # let baz = 8;
/// let foo = Foo { x: 42 };
/// let (result, measure) = Measure::this(|(this, args)| Foo::bar(&this, args), (&foo, baz), "Foo::bar");
/// # assert_eq!(result, 50);
/// ```
pub fn this<T, R, F: FnOnce(T) -> R>(func: F, args: T, name: &'static str) -> (R, Self) {
let mut measure = Self::start(name);
let result = func(args);
measure.stop();
(result, measure)
}
} }
impl fmt::Display for Measure { impl fmt::Display for Measure {
@ -150,91 +106,4 @@ mod tests {
let measure = Measure::start("test_not_stopped"); let measure = Measure::start("test_not_stopped");
assert_eq!(format!("{}", measure), "test_not_stopped running"); assert_eq!(format!("{}", measure), "test_not_stopped running");
} }
fn my_multiply(x: i32, y: i32) -> i32 {
x * y
}
fn my_multiply_tuple(args: (i32, i32)) -> i32 {
let (x, y) = args;
my_multiply(x, y)
}
fn square(x: i32) -> i32 {
my_multiply(x, x)
}
struct SomeStruct {
x: i32,
}
impl SomeStruct {
fn add_to(&self, x: i32) -> i32 {
x + self.x
}
}
#[test]
fn test_measure_this() {
// Ensure that the measurement side actually works
{
let (_result, measure) = Measure::this(|s| sleep(Duration::from_secs(s)), 1, "test");
assert!(measure.as_s() >= 0.99f32 && measure.as_s() <= 1.01f32);
assert!(measure.as_ms() >= 990 && measure.as_ms() <= 1_010);
assert!(measure.as_us() >= 999_000 && measure.as_us() <= 1_010_000);
}
// Ensure that this() can be called with a simple closure
{
let expected = 1;
let (actual, _measure) = Measure::this(|x| x, expected, "test");
assert_eq!(actual, expected);
}
// Ensure that this() can be called with a tuple
{
let (result, _measure) = Measure::this(|(x, y)| x + y, (1, 2), "test");
assert_eq!(result, 1 + 2);
}
// Ensure that this() can be called with a normal function
{
let (result, _measure) = Measure::this(|(x, y)| my_multiply(x, y), (3, 4), "test");
assert_eq!(result, 3 * 4);
}
// Ensure that this() can be called with a normal function with one argument
{
let (result, _measure) = Measure::this(square, 5, "test");
assert_eq!(result, 5 * 5)
}
// Ensure that this() can be called with a normal function
{
let (result, _measure) = Measure::this(my_multiply_tuple, (3, 4), "test");
assert_eq!(result, 3 * 4);
}
// Ensure that this() can be called with a method (and self)
{
let some_struct = SomeStruct { x: 42 };
let (result, _measure) = Measure::this(
|(obj, x)| SomeStruct::add_to(&obj, x),
(some_struct, 4),
"test",
);
assert_eq!(result, 42 + 4);
}
// Ensure that this() can be called with a method (and &self)
{
let some_struct = SomeStruct { x: 42 };
let (result, _measure) = Measure::this(
|(obj, x)| SomeStruct::add_to(obj, x),
(&some_struct, 4),
"test",
);
assert_eq!(result, 42 + 4);
assert_eq!(some_struct.add_to(6), 42 + 6);
}
}
} }

View File

@ -21,7 +21,7 @@ use {
genesis_utils::{create_genesis_config, GenesisConfigInfo}, genesis_utils::{create_genesis_config, GenesisConfigInfo},
leader_schedule_cache::LeaderScheduleCache, leader_schedule_cache::LeaderScheduleCache,
}, },
solana_measure::measure::Measure, solana_measure::measure,
solana_metrics::poh_timing_point::{send_poh_timing_point, PohTimingSender, SlotPohTimingInfo}, solana_metrics::poh_timing_point::{send_poh_timing_point, PohTimingSender, SlotPohTimingInfo},
solana_runtime::bank::Bank, solana_runtime::bank::Bank,
solana_sdk::{ solana_sdk::{
@ -647,8 +647,8 @@ impl PohRecorder {
} }
pub fn tick(&mut self) { pub fn tick(&mut self) {
let ((poh_entry, target_time), tick_lock_contention_time) = Measure::this( let ((poh_entry, target_time), tick_lock_contention_time) = measure!(
|_| { {
let mut poh_l = self.poh.lock().unwrap(); let mut poh_l = self.poh.lock().unwrap();
let poh_entry = poh_l.tick(); let poh_entry = poh_l.tick();
let target_time = if poh_entry.is_some() { let target_time = if poh_entry.is_some() {
@ -658,7 +658,6 @@ impl PohRecorder {
}; };
(poh_entry, target_time) (poh_entry, target_time)
}, },
(),
"tick_lock_contention", "tick_lock_contention",
); );
self.tick_lock_contention_us += tick_lock_contention_time.as_us(); self.tick_lock_contention_us += tick_lock_contention_time.as_us();
@ -685,11 +684,11 @@ impl PohRecorder {
)); ));
let (_flush_res, flush_cache_and_tick_time) = let (_flush_res, flush_cache_and_tick_time) =
Measure::this(|_| self.flush_cache(true), (), "flush_cache_and_tick"); measure!(self.flush_cache(true), "flush_cache_and_tick");
self.flush_cache_tick_us += flush_cache_and_tick_time.as_us(); self.flush_cache_tick_us += flush_cache_and_tick_time.as_us();
let sleep_time = Measure::this( let sleep_time = measure!(
|_| { {
let target_time = target_time.unwrap(); let target_time = target_time.unwrap();
// sleep is not accurate enough to get a predictable time. // sleep is not accurate enough to get a predictable time.
// Kernel can not schedule the thread for a while. // Kernel can not schedule the thread for a while.
@ -698,7 +697,6 @@ impl PohRecorder {
std::hint::spin_loop(); std::hint::spin_loop();
} }
}, },
(),
"poh_sleep", "poh_sleep",
) )
.1; .1;
@ -749,13 +747,12 @@ impl PohRecorder {
// cannot be generated by `record()` // cannot be generated by `record()`
assert!(!transactions.is_empty(), "No transactions provided"); assert!(!transactions.is_empty(), "No transactions provided");
let ((), report_metrics_time) = let ((), report_metrics_time) = measure!(self.report_metrics(bank_slot), "report_metrics");
Measure::this(|_| self.report_metrics(bank_slot), (), "report_metrics");
self.report_metrics_us += report_metrics_time.as_us(); self.report_metrics_us += report_metrics_time.as_us();
loop { loop {
let (flush_cache_res, flush_cache_time) = let (flush_cache_res, flush_cache_time) =
Measure::this(|_| self.flush_cache(false), (), "flush_cache"); measure!(self.flush_cache(false), "flush_cache");
self.flush_cache_no_tick_us += flush_cache_time.as_us(); self.flush_cache_no_tick_us += flush_cache_time.as_us();
flush_cache_res?; flush_cache_res?;
@ -767,19 +764,18 @@ impl PohRecorder {
return Err(PohRecorderError::MaxHeightReached); return Err(PohRecorderError::MaxHeightReached);
} }
let (mut poh_lock, poh_lock_time) = let (mut poh_lock, poh_lock_time) = measure!(self.poh.lock().unwrap(), "poh_lock");
Measure::this(|_| self.poh.lock().unwrap(), (), "poh_lock");
self.record_lock_contention_us += poh_lock_time.as_us(); self.record_lock_contention_us += poh_lock_time.as_us();
let (record_mixin_res, record_mixin_time) = let (record_mixin_res, record_mixin_time) =
Measure::this(|_| poh_lock.record(mixin), (), "record_mixin"); measure!(poh_lock.record(mixin), "record_mixin");
self.record_us += record_mixin_time.as_us(); self.record_us += record_mixin_time.as_us();
drop(poh_lock); drop(poh_lock);
if let Some(poh_entry) = record_mixin_res { if let Some(poh_entry) = record_mixin_res {
let (send_entry_res, send_entry_time) = Measure::this( let (send_entry_res, send_entry_time) = measure!(
|_| { {
let entry = Entry { let entry = Entry {
num_hashes: poh_entry.num_hashes, num_hashes: poh_entry.num_hashes,
hash: poh_entry.hash, hash: poh_entry.hash,
@ -788,7 +784,6 @@ impl PohRecorder {
let bank_clone = working_bank.bank.clone(); let bank_clone = working_bank.bank.clone();
self.sender.send((bank_clone, (entry, self.tick_height))) self.sender.send((bank_clone, (entry, self.tick_height)))
}, },
(),
"send_poh_entry", "send_poh_entry",
); );
self.send_entry_us += send_entry_time.as_us(); self.send_entry_us += send_entry_time.as_us();

View File

@ -5,7 +5,7 @@ use {
crossbeam_channel::Receiver, crossbeam_channel::Receiver,
log::*, log::*,
solana_entry::poh::Poh, solana_entry::poh::Poh,
solana_measure::measure::Measure, solana_measure::{measure, measure::Measure},
solana_sdk::poh_config::PohConfig, solana_sdk::poh_config::PohConfig,
std::{ std::{
sync::{ sync::{
@ -264,7 +264,7 @@ impl PohService {
); );
// what do we do on failure here? Ignore for now. // what do we do on failure here? Ignore for now.
let (_send_res, send_record_result_time) = let (_send_res, send_record_result_time) =
Measure::this(|_| record.sender.send(res), (), "send_record_result"); measure!(record.sender.send(res), "send_record_result");
timing.total_send_record_result_us += send_record_result_time.as_us(); timing.total_send_record_result_us += send_record_result_time.as_us();
timing.num_hashes += 1; // note: may have also ticked inside record timing.num_hashes += 1; // note: may have also ticked inside record

View File

@ -1703,8 +1703,8 @@ impl Bank {
let epoch_schedule = parent.epoch_schedule; let epoch_schedule = parent.epoch_schedule;
let epoch = epoch_schedule.get_epoch(slot); let epoch = epoch_schedule.get_epoch(slot);
let (rc, bank_rc_time) = Measure::this( let (rc, bank_rc_time) = measure!(
|_| BankRc { BankRc {
accounts: Arc::new(Accounts::new_from_parent( accounts: Arc::new(Accounts::new_from_parent(
&parent.rc.accounts, &parent.rc.accounts,
slot, slot,
@ -1714,20 +1714,18 @@ impl Bank {
slot, slot,
bank_id_generator: parent.rc.bank_id_generator.clone(), bank_id_generator: parent.rc.bank_id_generator.clone(),
}, },
(),
"bank_rc_creation", "bank_rc_creation",
); );
let (src, status_cache_rc_time) = Measure::this( let (src, status_cache_rc_time) = measure!(
|_| StatusCacheRc { StatusCacheRc {
status_cache: parent.src.status_cache.clone(), status_cache: parent.src.status_cache.clone(),
}, },
(),
"status_cache_rc_creation", "status_cache_rc_creation",
); );
let ((fee_rate_governor, fee_calculator), fee_components_time) = Measure::this( let ((fee_rate_governor, fee_calculator), fee_components_time) = measure!(
|_| { {
let fee_rate_governor = FeeRateGovernor::new_derived( let fee_rate_governor = FeeRateGovernor::new_derived(
&parent.fee_rate_governor, &parent.fee_rate_governor,
parent.signature_count(), parent.signature_count(),
@ -1741,65 +1739,54 @@ impl Bank {
}; };
(fee_rate_governor, fee_calculator) (fee_rate_governor, fee_calculator)
}, },
(),
"fee_components_creation", "fee_components_creation",
); );
let bank_id = rc.bank_id_generator.fetch_add(1, Relaxed) + 1; let bank_id = rc.bank_id_generator.fetch_add(1, Relaxed) + 1;
let (blockhash_queue, blockhash_queue_time) = Measure::this( let (blockhash_queue, blockhash_queue_time) = measure!(
|_| RwLock::new(parent.blockhash_queue.read().unwrap().clone()), RwLock::new(parent.blockhash_queue.read().unwrap().clone()),
(),
"blockhash_queue_creation", "blockhash_queue_creation",
); );
let (stakes_cache, stakes_cache_time) = Measure::this( let (stakes_cache, stakes_cache_time) = measure!(
|_| StakesCache::new(parent.stakes_cache.stakes().clone()), StakesCache::new(parent.stakes_cache.stakes().clone()),
(),
"stakes_cache_creation", "stakes_cache_creation",
); );
let (epoch_stakes, epoch_stakes_time) = let (epoch_stakes, epoch_stakes_time) =
Measure::this(|_| parent.epoch_stakes.clone(), (), "epoch_stakes_creation"); measure!(parent.epoch_stakes.clone(), "epoch_stakes_creation");
let (builtin_programs, builtin_programs_time) = Measure::this( let (builtin_programs, builtin_programs_time) =
|_| parent.builtin_programs.clone(), measure!(parent.builtin_programs.clone(), "builtin_programs_creation");
(),
"builtin_programs_creation",
);
let (rewards_pool_pubkeys, rewards_pool_pubkeys_time) = Measure::this( let (rewards_pool_pubkeys, rewards_pool_pubkeys_time) = measure!(
|_| parent.rewards_pool_pubkeys.clone(), parent.rewards_pool_pubkeys.clone(),
(),
"rewards_pool_pubkeys_creation", "rewards_pool_pubkeys_creation",
); );
let (cached_executors, cached_executors_time) = Measure::this( let (cached_executors, cached_executors_time) = measure!(
|_| { {
let parent_bank_executors = parent.cached_executors.read().unwrap(); let parent_bank_executors = parent.cached_executors.read().unwrap();
RwLock::new(CachedExecutors::new_from_parent_bank_executors( RwLock::new(CachedExecutors::new_from_parent_bank_executors(
&parent_bank_executors, &parent_bank_executors,
epoch, epoch,
)) ))
}, },
(),
"cached_executors_creation", "cached_executors_creation",
); );
let (transaction_debug_keys, transaction_debug_keys_time) = Measure::this( let (transaction_debug_keys, transaction_debug_keys_time) = measure!(
|_| parent.transaction_debug_keys.clone(), parent.transaction_debug_keys.clone(),
(),
"transation_debug_keys_creation", "transation_debug_keys_creation",
); );
let (transaction_log_collector_config, transaction_log_collector_config_time) = let (transaction_log_collector_config, transaction_log_collector_config_time) = measure!(
Measure::this( parent.transaction_log_collector_config.clone(),
|_| parent.transaction_log_collector_config.clone(),
(),
"transaction_log_collector_config_creation", "transaction_log_collector_config_creation",
); );
let (feature_set, feature_set_time) = let (feature_set, feature_set_time) =
Measure::this(|_| parent.feature_set.clone(), (), "feature_set_creation"); measure!(parent.feature_set.clone(), "feature_set_creation");
let accounts_data_size_initial = parent.load_accounts_data_size(); let accounts_data_size_initial = parent.load_accounts_data_size();
let mut new = Bank { let mut new = Bank {
@ -1878,8 +1865,8 @@ impl Bank {
fee_structure: parent.fee_structure.clone(), fee_structure: parent.fee_structure.clone(),
}; };
let (_, ancestors_time) = Measure::this( let (_, ancestors_time) = measure!(
|_| { {
let mut ancestors = Vec::with_capacity(1 + new.parents().len()); let mut ancestors = Vec::with_capacity(1 + new.parents().len());
ancestors.push(new.slot()); ancestors.push(new.slot());
new.parents().iter().for_each(|p| { new.parents().iter().for_each(|p| {
@ -1887,48 +1874,43 @@ impl Bank {
}); });
new.ancestors = Ancestors::from(ancestors); new.ancestors = Ancestors::from(ancestors);
}, },
(),
"ancestors_creation", "ancestors_creation",
); );
// Following code may touch AccountsDb, requiring proper ancestors // Following code may touch AccountsDb, requiring proper ancestors
let parent_epoch = parent.epoch(); let parent_epoch = parent.epoch();
let (_, update_epoch_time) = Measure::this( let (_, update_epoch_time) = measure!(
|_| { {
if parent_epoch < new.epoch() { if parent_epoch < new.epoch() {
let (thread_pool, thread_pool_time) = Measure::this( let (thread_pool, thread_pool_time) = measure!(
|_| ThreadPoolBuilder::new().build().unwrap(), ThreadPoolBuilder::new().build().unwrap(),
(),
"thread_pool_creation", "thread_pool_creation",
); );
let (_, apply_feature_activations_time) = Measure::this( let (_, apply_feature_activations_time) = measure!(
|_| new.apply_feature_activations(false, false), new.apply_feature_activations(false, false),
(),
"apply_feature_activation", "apply_feature_activation",
); );
// Add new entry to stakes.stake_history, set appropriate epoch and // Add new entry to stakes.stake_history, set appropriate epoch and
// update vote accounts with warmed up stakes before saving a // update vote accounts with warmed up stakes before saving a
// snapshot of stakes in epoch stakes // snapshot of stakes in epoch stakes
let (_, activate_epoch_time) = Measure::this( let (_, activate_epoch_time) = measure!(
|_| new.stakes_cache.activate_epoch(epoch, &thread_pool), new.stakes_cache.activate_epoch(epoch, &thread_pool),
(),
"activate_epoch", "activate_epoch",
); );
// Save a snapshot of stakes for use in consensus and stake weighted networking // Save a snapshot of stakes for use in consensus and stake weighted networking
let leader_schedule_epoch = epoch_schedule.get_leader_schedule_epoch(slot); let leader_schedule_epoch = epoch_schedule.get_leader_schedule_epoch(slot);
let (_, update_epoch_stakes_time) = Measure::this( let (_, update_epoch_stakes_time) = measure!(
|_| new.update_epoch_stakes(leader_schedule_epoch), new.update_epoch_stakes(leader_schedule_epoch),
(),
"update_epoch_stakes", "update_epoch_stakes",
); );
let mut metrics = RewardsMetrics::default(); let mut metrics = RewardsMetrics::default();
// After saving a snapshot of stakes, apply stake rewards and commission // After saving a snapshot of stakes, apply stake rewards and commission
let (_, update_rewards_with_thread_pool_time) = Measure::this( let (_, update_rewards_with_thread_pool_time) = measure!(
|_| { {
new.update_rewards_with_thread_pool( new.update_rewards_with_thread_pool(
parent_epoch, parent_epoch,
reward_calc_tracer, reward_calc_tracer,
@ -1936,7 +1918,6 @@ impl Bank {
&mut metrics, &mut metrics,
) )
}, },
(),
"update_rewards_with_thread_pool", "update_rewards_with_thread_pool",
); );
@ -2004,27 +1985,22 @@ impl Bank {
new.update_epoch_stakes(leader_schedule_epoch); new.update_epoch_stakes(leader_schedule_epoch);
} }
}, },
(),
"update_epoch", "update_epoch",
); );
// Update sysvars before processing transactions // Update sysvars before processing transactions
let (_, update_sysvars_time) = Measure::this( let (_, update_sysvars_time) = measure!(
|_| { {
new.update_slot_hashes(); new.update_slot_hashes();
new.update_stake_history(Some(parent_epoch)); new.update_stake_history(Some(parent_epoch));
new.update_clock(Some(parent_epoch)); new.update_clock(Some(parent_epoch));
new.update_fees(); new.update_fees();
}, },
(),
"update_sysvars", "update_sysvars",
); );
let (_, fill_sysvar_cache_time) = Measure::this( let (_, fill_sysvar_cache_time) =
|_| new.fill_missing_sysvar_cache_entries(), measure!(new.fill_missing_sysvar_cache_entries(), "fill_sysvar_cache");
(),
"fill_sysvar_cache",
);
time.stop(); time.stop();