refactor cost calculation (#21062)
* - cache calculated transaction cost to allow sharing; - atomic cost tracking op; - only lock accounts for transactions eligible for current block; - moved qos service and stats reporting to its own model; - add cost_weight default to neutral (as 1), vote has zero weight; Co-authored-by: Tyera Eulberg <teulberg@gmail.com> * Update core/src/qos_service.rs Co-authored-by: Tyera Eulberg <teulberg@gmail.com> * Update core/src/qos_service.rs Co-authored-by: Tyera Eulberg <teulberg@gmail.com> Co-authored-by: Tyera Eulberg <teulberg@gmail.com>
This commit is contained in:
@ -1,7 +1,10 @@
|
||||
//! The `banking_stage` processes Transaction messages. It is intended to be used
|
||||
//! to contruct a software pipeline. The stage uses all available CPU cores and
|
||||
//! can do its processing in parallel with signature verification on the GPU.
|
||||
use crate::packet_hasher::PacketHasher;
|
||||
use crate::{
|
||||
packet_hasher::PacketHasher,
|
||||
qos_service::{QosService, QosServiceStats},
|
||||
};
|
||||
use crossbeam_channel::{Receiver as CrossbeamReceiver, RecvTimeoutError};
|
||||
use itertools::Itertools;
|
||||
use lru::LruCache;
|
||||
@ -26,7 +29,6 @@ use solana_runtime::{
|
||||
},
|
||||
bank_utils,
|
||||
cost_model::CostModel,
|
||||
cost_tracker::CostTracker,
|
||||
transaction_batch::TransactionBatch,
|
||||
vote_sender_types::ReplayVoteSender,
|
||||
};
|
||||
@ -55,7 +57,7 @@ use std::{
|
||||
net::{SocketAddr, UdpSocket},
|
||||
ops::DerefMut,
|
||||
sync::atomic::{AtomicU64, AtomicUsize, Ordering},
|
||||
sync::{Arc, Mutex, RwLock, RwLockReadGuard},
|
||||
sync::{Arc, Mutex, RwLock},
|
||||
thread::{self, Builder, JoinHandle},
|
||||
time::Duration,
|
||||
time::Instant,
|
||||
@ -97,8 +99,6 @@ pub struct BankingStageStats {
|
||||
current_buffered_packet_batches_count: AtomicUsize,
|
||||
rebuffered_packets_count: AtomicUsize,
|
||||
consumed_buffered_packets_count: AtomicUsize,
|
||||
cost_tracker_check_count: AtomicUsize,
|
||||
cost_forced_retry_transactions_count: AtomicUsize,
|
||||
|
||||
// Timing
|
||||
consume_buffered_packets_elapsed: AtomicU64,
|
||||
@ -109,9 +109,6 @@ pub struct BankingStageStats {
|
||||
packet_conversion_elapsed: AtomicU64,
|
||||
unprocessed_packet_conversion_elapsed: AtomicU64,
|
||||
transaction_processing_elapsed: AtomicU64,
|
||||
cost_tracker_update_elapsed: AtomicU64,
|
||||
cost_tracker_clone_elapsed: AtomicU64,
|
||||
cost_tracker_check_elapsed: AtomicU64,
|
||||
}
|
||||
|
||||
impl BankingStageStats {
|
||||
@ -181,17 +178,6 @@ impl BankingStageStats {
|
||||
.swap(0, Ordering::Relaxed) as i64,
|
||||
i64
|
||||
),
|
||||
(
|
||||
"cost_tracker_check_count",
|
||||
self.cost_tracker_check_count.swap(0, Ordering::Relaxed) as i64,
|
||||
i64
|
||||
),
|
||||
(
|
||||
"cost_forced_retry_transactions_count",
|
||||
self.cost_forced_retry_transactions_count
|
||||
.swap(0, Ordering::Relaxed) as i64,
|
||||
i64
|
||||
),
|
||||
(
|
||||
"consume_buffered_packets_elapsed",
|
||||
self.consume_buffered_packets_elapsed
|
||||
@ -238,21 +224,6 @@ impl BankingStageStats {
|
||||
.swap(0, Ordering::Relaxed) as i64,
|
||||
i64
|
||||
),
|
||||
(
|
||||
"cost_tracker_update_elapsed",
|
||||
self.cost_tracker_update_elapsed.swap(0, Ordering::Relaxed) as i64,
|
||||
i64
|
||||
),
|
||||
(
|
||||
"cost_tracker_clone_elapsed",
|
||||
self.cost_tracker_clone_elapsed.swap(0, Ordering::Relaxed) as i64,
|
||||
i64
|
||||
),
|
||||
(
|
||||
"cost_tracker_check_elapsed",
|
||||
self.cost_tracker_check_elapsed.swap(0, Ordering::Relaxed) as i64,
|
||||
i64
|
||||
),
|
||||
);
|
||||
}
|
||||
}
|
||||
@ -457,7 +428,6 @@ impl BankingStage {
|
||||
my_pubkey,
|
||||
*next_leader,
|
||||
banking_stage_stats,
|
||||
cost_model,
|
||||
);
|
||||
Self::update_buffered_packets_with_new_unprocessed(
|
||||
original_unprocessed_indexes,
|
||||
@ -960,13 +930,32 @@ impl BankingStage {
|
||||
chunk_offset: usize,
|
||||
transaction_status_sender: Option<TransactionStatusSender>,
|
||||
gossip_vote_sender: &ReplayVoteSender,
|
||||
cost_model: &Arc<RwLock<CostModel>>,
|
||||
) -> (Result<usize, PohRecorderError>, Vec<usize>) {
|
||||
let mut lock_time = Measure::start("lock_time");
|
||||
let mut qos_service_stats = QosServiceStats::default();
|
||||
let qos_service = QosService::new(cost_model.clone());
|
||||
let tx_costs = qos_service.compute_transaction_costs(
|
||||
txs.iter(),
|
||||
bank.demote_program_write_locks(),
|
||||
&mut qos_service_stats,
|
||||
);
|
||||
|
||||
let transactions_qos_results = qos_service.select_transactions_per_cost(
|
||||
txs.iter(),
|
||||
tx_costs.iter(),
|
||||
bank,
|
||||
&mut qos_service_stats,
|
||||
);
|
||||
|
||||
// Only lock accounts for those transactions are selected for the block;
|
||||
// Once accounts are locked, other threads cannot encode transactions that will modify the
|
||||
// same account state
|
||||
let batch = bank.prepare_sanitized_batch(txs);
|
||||
let mut lock_time = Measure::start("lock_time");
|
||||
let batch = bank.prepare_sanitized_batch_with_results(txs, transactions_qos_results.iter());
|
||||
lock_time.stop();
|
||||
|
||||
// retryable_txs includes AccountInUse, WouldExceedMaxBlockCostLimit and
|
||||
// WouldExceedMaxAccountCostLimit
|
||||
let (result, mut retryable_txs) = Self::process_and_record_transactions_locked(
|
||||
bank,
|
||||
poh,
|
||||
@ -989,6 +978,8 @@ impl BankingStage {
|
||||
txs.len(),
|
||||
);
|
||||
|
||||
qos_service_stats.report();
|
||||
|
||||
(result, retryable_txs)
|
||||
}
|
||||
|
||||
@ -1003,6 +994,7 @@ impl BankingStage {
|
||||
poh: &TransactionRecorder,
|
||||
transaction_status_sender: Option<TransactionStatusSender>,
|
||||
gossip_vote_sender: &ReplayVoteSender,
|
||||
cost_model: &Arc<RwLock<CostModel>>,
|
||||
) -> (usize, Vec<usize>) {
|
||||
let mut chunk_start = 0;
|
||||
let mut unprocessed_txs = vec![];
|
||||
@ -1019,6 +1011,7 @@ impl BankingStage {
|
||||
chunk_start,
|
||||
transaction_status_sender.clone(),
|
||||
gossip_vote_sender,
|
||||
cost_model,
|
||||
);
|
||||
trace!("process_transactions result: {:?}", result);
|
||||
|
||||
@ -1087,24 +1080,17 @@ impl BankingStage {
|
||||
Some(&packet.data[msg_start..msg_end])
|
||||
}
|
||||
|
||||
// This function deserializes packets into transactions, computes the blake3 hash of transaction messages,
|
||||
// and verifies secp256k1 instructions. A list of valid transactions are returned with their message hashes
|
||||
// and packet indexes.
|
||||
// Also returned is packet indexes for transaction should be retried due to cost limits.
|
||||
// This function deserializes packets into transactions, computes the blake3 hash of transaction
|
||||
// messages, and verifies secp256k1 instructions. A list of sanitized transactions are returned
|
||||
// with their packet indexes.
|
||||
#[allow(clippy::needless_collect)]
|
||||
fn transactions_from_packets(
|
||||
msgs: &Packets,
|
||||
transaction_indexes: &[usize],
|
||||
feature_set: &Arc<feature_set::FeatureSet>,
|
||||
read_cost_tracker: &RwLockReadGuard<CostTracker>,
|
||||
banking_stage_stats: &BankingStageStats,
|
||||
demote_program_write_locks: bool,
|
||||
votes_only: bool,
|
||||
cost_model: &Arc<RwLock<CostModel>>,
|
||||
) -> (Vec<SanitizedTransaction>, Vec<usize>, Vec<usize>) {
|
||||
let mut retryable_transaction_packet_indexes: Vec<usize> = vec![];
|
||||
|
||||
let verified_transactions_with_packet_indexes: Vec<_> = transaction_indexes
|
||||
) -> (Vec<SanitizedTransaction>, Vec<usize>) {
|
||||
transaction_indexes
|
||||
.iter()
|
||||
.filter_map(|tx_index| {
|
||||
let p = &msgs.packets[*tx_index];
|
||||
@ -1125,51 +1111,7 @@ impl BankingStage {
|
||||
tx.verify_precompiles(feature_set).ok()?;
|
||||
Some((tx, *tx_index))
|
||||
})
|
||||
.collect();
|
||||
banking_stage_stats.cost_tracker_check_count.fetch_add(
|
||||
verified_transactions_with_packet_indexes.len(),
|
||||
Ordering::Relaxed,
|
||||
);
|
||||
|
||||
let mut cost_tracker_check_time = Measure::start("cost_tracker_check_time");
|
||||
let (filtered_transactions, filter_transaction_packet_indexes) = {
|
||||
verified_transactions_with_packet_indexes
|
||||
.into_iter()
|
||||
.filter_map(|(tx, tx_index)| {
|
||||
// excluding vote TX from cost_model, for now
|
||||
let is_vote = &msgs.packets[tx_index].meta.is_simple_vote_tx;
|
||||
if !is_vote
|
||||
&& read_cost_tracker
|
||||
.would_transaction_fit(
|
||||
&tx,
|
||||
&cost_model
|
||||
.read()
|
||||
.unwrap()
|
||||
.calculate_cost(&tx, demote_program_write_locks),
|
||||
)
|
||||
.is_err()
|
||||
{
|
||||
// put transaction into retry queue if it wouldn't fit
|
||||
// into current bank
|
||||
debug!("transaction {:?} would exceed limit", tx);
|
||||
retryable_transaction_packet_indexes.push(tx_index);
|
||||
return None;
|
||||
}
|
||||
Some((tx, tx_index))
|
||||
})
|
||||
.unzip()
|
||||
};
|
||||
cost_tracker_check_time.stop();
|
||||
|
||||
banking_stage_stats
|
||||
.cost_tracker_check_elapsed
|
||||
.fetch_add(cost_tracker_check_time.as_us(), Ordering::Relaxed);
|
||||
|
||||
(
|
||||
filtered_transactions,
|
||||
filter_transaction_packet_indexes,
|
||||
retryable_transaction_packet_indexes,
|
||||
)
|
||||
.unzip()
|
||||
}
|
||||
|
||||
/// This function filters pending packets that are still valid
|
||||
@ -1224,30 +1166,15 @@ impl BankingStage {
|
||||
cost_model: &Arc<RwLock<CostModel>>,
|
||||
) -> (usize, usize, Vec<usize>) {
|
||||
let mut packet_conversion_time = Measure::start("packet_conversion");
|
||||
let (transactions, transaction_to_packet_indexes, retryable_packet_indexes) =
|
||||
Self::transactions_from_packets(
|
||||
msgs,
|
||||
&packet_indexes,
|
||||
&bank.feature_set,
|
||||
&bank.read_cost_tracker().unwrap(),
|
||||
banking_stage_stats,
|
||||
bank.demote_program_write_locks(),
|
||||
bank.vote_only_bank(),
|
||||
cost_model,
|
||||
);
|
||||
let (transactions, transaction_to_packet_indexes) = Self::transactions_from_packets(
|
||||
msgs,
|
||||
&packet_indexes,
|
||||
&bank.feature_set,
|
||||
bank.vote_only_bank(),
|
||||
);
|
||||
packet_conversion_time.stop();
|
||||
inc_new_counter_info!("banking_stage-packet_conversion", 1);
|
||||
|
||||
banking_stage_stats
|
||||
.cost_forced_retry_transactions_count
|
||||
.fetch_add(retryable_packet_indexes.len(), Ordering::Relaxed);
|
||||
debug!(
|
||||
"bank: {} filtered transactions {} cost limited transactions {}",
|
||||
bank.slot(),
|
||||
transactions.len(),
|
||||
retryable_packet_indexes.len()
|
||||
);
|
||||
|
||||
let tx_len = transactions.len();
|
||||
|
||||
let mut process_tx_time = Measure::start("process_tx_time");
|
||||
@ -1258,6 +1185,7 @@ impl BankingStage {
|
||||
poh,
|
||||
transaction_status_sender,
|
||||
gossip_vote_sender,
|
||||
cost_model,
|
||||
);
|
||||
process_tx_time.stop();
|
||||
let unprocessed_tx_count = unprocessed_tx_indexes.len();
|
||||
@ -1266,23 +1194,8 @@ impl BankingStage {
|
||||
unprocessed_tx_count
|
||||
);
|
||||
|
||||
// applying cost of processed transactions to shared cost_tracker
|
||||
let mut cost_tracking_time = Measure::start("cost_tracking_time");
|
||||
transactions.iter().enumerate().for_each(|(index, tx)| {
|
||||
if unprocessed_tx_indexes.iter().all(|&i| i != index) {
|
||||
bank.write_cost_tracker().unwrap().add_transaction_cost(
|
||||
tx,
|
||||
&cost_model
|
||||
.read()
|
||||
.unwrap()
|
||||
.calculate_cost(tx, bank.demote_program_write_locks()),
|
||||
);
|
||||
}
|
||||
});
|
||||
cost_tracking_time.stop();
|
||||
|
||||
let mut filter_pending_packets_time = Measure::start("filter_pending_packets_time");
|
||||
let mut filtered_unprocessed_packet_indexes = Self::filter_pending_packets_from_pending_txs(
|
||||
let filtered_unprocessed_packet_indexes = Self::filter_pending_packets_from_pending_txs(
|
||||
bank,
|
||||
&transactions,
|
||||
&transaction_to_packet_indexes,
|
||||
@ -1295,19 +1208,12 @@ impl BankingStage {
|
||||
unprocessed_tx_count.saturating_sub(filtered_unprocessed_packet_indexes.len())
|
||||
);
|
||||
|
||||
// combine cost-related unprocessed transactions with bank determined unprocessed for
|
||||
// buffering
|
||||
filtered_unprocessed_packet_indexes.extend(retryable_packet_indexes);
|
||||
|
||||
banking_stage_stats
|
||||
.packet_conversion_elapsed
|
||||
.fetch_add(packet_conversion_time.as_us(), Ordering::Relaxed);
|
||||
banking_stage_stats
|
||||
.transaction_processing_elapsed
|
||||
.fetch_add(process_tx_time.as_us(), Ordering::Relaxed);
|
||||
banking_stage_stats
|
||||
.cost_tracker_update_elapsed
|
||||
.fetch_add(cost_tracking_time.as_us(), Ordering::Relaxed);
|
||||
banking_stage_stats
|
||||
.filter_pending_packets_elapsed
|
||||
.fetch_add(filter_pending_packets_time.as_us(), Ordering::Relaxed);
|
||||
@ -1322,7 +1228,6 @@ impl BankingStage {
|
||||
my_pubkey: &Pubkey,
|
||||
next_leader: Option<Pubkey>,
|
||||
banking_stage_stats: &BankingStageStats,
|
||||
cost_model: &Arc<RwLock<CostModel>>,
|
||||
) -> Vec<usize> {
|
||||
// Check if we are the next leader. If so, let's not filter the packets
|
||||
// as we'll filter it again while processing the packets.
|
||||
@ -1335,31 +1240,24 @@ impl BankingStage {
|
||||
|
||||
let mut unprocessed_packet_conversion_time =
|
||||
Measure::start("unprocessed_packet_conversion");
|
||||
let (transactions, transaction_to_packet_indexes, retry_packet_indexes) =
|
||||
Self::transactions_from_packets(
|
||||
msgs,
|
||||
transaction_indexes,
|
||||
&bank.feature_set,
|
||||
&bank.read_cost_tracker().unwrap(),
|
||||
banking_stage_stats,
|
||||
bank.demote_program_write_locks(),
|
||||
bank.vote_only_bank(),
|
||||
cost_model,
|
||||
);
|
||||
let (transactions, transaction_to_packet_indexes) = Self::transactions_from_packets(
|
||||
msgs,
|
||||
transaction_indexes,
|
||||
&bank.feature_set,
|
||||
bank.vote_only_bank(),
|
||||
);
|
||||
unprocessed_packet_conversion_time.stop();
|
||||
|
||||
let tx_count = transaction_to_packet_indexes.len();
|
||||
|
||||
let unprocessed_tx_indexes = (0..transactions.len()).collect_vec();
|
||||
let mut filtered_unprocessed_packet_indexes = Self::filter_pending_packets_from_pending_txs(
|
||||
let filtered_unprocessed_packet_indexes = Self::filter_pending_packets_from_pending_txs(
|
||||
bank,
|
||||
&transactions,
|
||||
&transaction_to_packet_indexes,
|
||||
&unprocessed_tx_indexes,
|
||||
);
|
||||
|
||||
filtered_unprocessed_packet_indexes.extend(retry_packet_indexes);
|
||||
|
||||
inc_new_counter_info!(
|
||||
"banking_stage-dropped_tx_before_forwarding",
|
||||
tx_count.saturating_sub(filtered_unprocessed_packet_indexes.len())
|
||||
@ -1496,7 +1394,6 @@ impl BankingStage {
|
||||
my_pubkey,
|
||||
next_leader,
|
||||
banking_stage_stats,
|
||||
cost_model,
|
||||
);
|
||||
Self::push_unprocessed(
|
||||
buffered_packets,
|
||||
@ -2342,6 +2239,7 @@ mod tests {
|
||||
0,
|
||||
None,
|
||||
&gossip_vote_sender,
|
||||
&Arc::new(RwLock::new(CostModel::default())),
|
||||
)
|
||||
.0
|
||||
.unwrap();
|
||||
@ -2383,6 +2281,7 @@ mod tests {
|
||||
0,
|
||||
None,
|
||||
&gossip_vote_sender,
|
||||
&Arc::new(RwLock::new(CostModel::default())),
|
||||
)
|
||||
.0,
|
||||
Err(PohRecorderError::MaxHeightReached)
|
||||
@ -2470,6 +2369,7 @@ mod tests {
|
||||
0,
|
||||
None,
|
||||
&gossip_vote_sender,
|
||||
&Arc::new(RwLock::new(CostModel::default())),
|
||||
);
|
||||
|
||||
poh_recorder
|
||||
@ -2578,6 +2478,7 @@ mod tests {
|
||||
&recorder,
|
||||
None,
|
||||
&gossip_vote_sender,
|
||||
&Arc::new(RwLock::new(CostModel::default())),
|
||||
);
|
||||
|
||||
assert_eq!(processed_transactions_count, 0,);
|
||||
@ -2670,6 +2571,7 @@ mod tests {
|
||||
enable_cpi_and_log_storage: false,
|
||||
}),
|
||||
&gossip_vote_sender,
|
||||
&Arc::new(RwLock::new(CostModel::default())),
|
||||
);
|
||||
|
||||
transaction_status_service.join().unwrap();
|
||||
@ -3130,32 +3032,22 @@ mod tests {
|
||||
make_test_packets(vec![transfer_tx.clone(), transfer_tx.clone()], vote_indexes);
|
||||
|
||||
let mut votes_only = false;
|
||||
let (txs, tx_packet_index, _retryable_packet_indexes) =
|
||||
BankingStage::transactions_from_packets(
|
||||
&packets,
|
||||
&packet_indexes,
|
||||
&Arc::new(FeatureSet::default()),
|
||||
&RwLock::new(CostTracker::default()).read().unwrap(),
|
||||
&BankingStageStats::default(),
|
||||
false,
|
||||
votes_only,
|
||||
&Arc::new(RwLock::new(CostModel::default())),
|
||||
);
|
||||
let (txs, tx_packet_index) = BankingStage::transactions_from_packets(
|
||||
&packets,
|
||||
&packet_indexes,
|
||||
&Arc::new(FeatureSet::default()),
|
||||
votes_only,
|
||||
);
|
||||
assert_eq!(2, txs.len());
|
||||
assert_eq!(vec![0, 1], tx_packet_index);
|
||||
|
||||
votes_only = true;
|
||||
let (txs, tx_packet_index, _retryable_packet_indexes) =
|
||||
BankingStage::transactions_from_packets(
|
||||
&packets,
|
||||
&packet_indexes,
|
||||
&Arc::new(FeatureSet::default()),
|
||||
&RwLock::new(CostTracker::default()).read().unwrap(),
|
||||
&BankingStageStats::default(),
|
||||
false,
|
||||
votes_only,
|
||||
&Arc::new(RwLock::new(CostModel::default())),
|
||||
);
|
||||
let (txs, tx_packet_index) = BankingStage::transactions_from_packets(
|
||||
&packets,
|
||||
&packet_indexes,
|
||||
&Arc::new(FeatureSet::default()),
|
||||
votes_only,
|
||||
);
|
||||
assert_eq!(0, txs.len());
|
||||
assert_eq!(0, tx_packet_index.len());
|
||||
}
|
||||
@ -3169,32 +3061,22 @@ mod tests {
|
||||
);
|
||||
|
||||
let mut votes_only = false;
|
||||
let (txs, tx_packet_index, _retryable_packet_indexes) =
|
||||
BankingStage::transactions_from_packets(
|
||||
&packets,
|
||||
&packet_indexes,
|
||||
&Arc::new(FeatureSet::default()),
|
||||
&RwLock::new(CostTracker::default()).read().unwrap(),
|
||||
&BankingStageStats::default(),
|
||||
false,
|
||||
votes_only,
|
||||
&Arc::new(RwLock::new(CostModel::default())),
|
||||
);
|
||||
let (txs, tx_packet_index) = BankingStage::transactions_from_packets(
|
||||
&packets,
|
||||
&packet_indexes,
|
||||
&Arc::new(FeatureSet::default()),
|
||||
votes_only,
|
||||
);
|
||||
assert_eq!(3, txs.len());
|
||||
assert_eq!(vec![0, 1, 2], tx_packet_index);
|
||||
|
||||
votes_only = true;
|
||||
let (txs, tx_packet_index, _retryable_packet_indexes) =
|
||||
BankingStage::transactions_from_packets(
|
||||
&packets,
|
||||
&packet_indexes,
|
||||
&Arc::new(FeatureSet::default()),
|
||||
&RwLock::new(CostTracker::default()).read().unwrap(),
|
||||
&BankingStageStats::default(),
|
||||
false,
|
||||
votes_only,
|
||||
&Arc::new(RwLock::new(CostModel::default())),
|
||||
);
|
||||
let (txs, tx_packet_index) = BankingStage::transactions_from_packets(
|
||||
&packets,
|
||||
&packet_indexes,
|
||||
&Arc::new(FeatureSet::default()),
|
||||
votes_only,
|
||||
);
|
||||
assert_eq!(2, txs.len());
|
||||
assert_eq!(vec![0, 2], tx_packet_index);
|
||||
}
|
||||
@ -3208,32 +3090,22 @@ mod tests {
|
||||
);
|
||||
|
||||
let mut votes_only = false;
|
||||
let (txs, tx_packet_index, _retryable_packet_indexes) =
|
||||
BankingStage::transactions_from_packets(
|
||||
&packets,
|
||||
&packet_indexes,
|
||||
&Arc::new(FeatureSet::default()),
|
||||
&RwLock::new(CostTracker::default()).read().unwrap(),
|
||||
&BankingStageStats::default(),
|
||||
false,
|
||||
votes_only,
|
||||
&Arc::new(RwLock::new(CostModel::default())),
|
||||
);
|
||||
let (txs, tx_packet_index) = BankingStage::transactions_from_packets(
|
||||
&packets,
|
||||
&packet_indexes,
|
||||
&Arc::new(FeatureSet::default()),
|
||||
votes_only,
|
||||
);
|
||||
assert_eq!(3, txs.len());
|
||||
assert_eq!(vec![0, 1, 2], tx_packet_index);
|
||||
|
||||
votes_only = true;
|
||||
let (txs, tx_packet_index, _retryable_packet_indexes) =
|
||||
BankingStage::transactions_from_packets(
|
||||
&packets,
|
||||
&packet_indexes,
|
||||
&Arc::new(FeatureSet::default()),
|
||||
&RwLock::new(CostTracker::default()).read().unwrap(),
|
||||
&BankingStageStats::default(),
|
||||
false,
|
||||
votes_only,
|
||||
&Arc::new(RwLock::new(CostModel::default())),
|
||||
);
|
||||
let (txs, tx_packet_index) = BankingStage::transactions_from_packets(
|
||||
&packets,
|
||||
&packet_indexes,
|
||||
&Arc::new(FeatureSet::default()),
|
||||
votes_only,
|
||||
);
|
||||
assert_eq!(3, txs.len());
|
||||
assert_eq!(vec![0, 1, 2], tx_packet_index);
|
||||
}
|
||||
|
@ -32,6 +32,7 @@ pub mod optimistic_confirmation_verifier;
|
||||
pub mod outstanding_requests;
|
||||
pub mod packet_hasher;
|
||||
pub mod progress_map;
|
||||
pub mod qos_service;
|
||||
pub mod repair_response;
|
||||
pub mod repair_service;
|
||||
pub mod repair_weight;
|
||||
|
242
core/src/qos_service.rs
Normal file
242
core/src/qos_service.rs
Normal file
@ -0,0 +1,242 @@
|
||||
//! Quality of service for block producer.
|
||||
//! Provides logic and functions to allow a Leader to prioritize
|
||||
//! how transactions are included in blocks, and optimize those blocks.
|
||||
//!
|
||||
use {
|
||||
solana_measure::measure::Measure,
|
||||
solana_runtime::{
|
||||
bank::Bank,
|
||||
cost_model::{CostModel, TransactionCost},
|
||||
cost_tracker::CostTrackerError,
|
||||
},
|
||||
solana_sdk::transaction::{self, SanitizedTransaction, TransactionError},
|
||||
std::sync::{Arc, RwLock},
|
||||
};
|
||||
|
||||
#[derive(Default)]
|
||||
pub struct QosServiceStats {
|
||||
compute_cost_time: u64,
|
||||
cost_tracking_time: u64,
|
||||
selected_txs_count: u64,
|
||||
retried_txs_per_block_limit_count: u64,
|
||||
retried_txs_per_account_limit_count: u64,
|
||||
}
|
||||
|
||||
impl QosServiceStats {
|
||||
pub fn report(&mut self) {
|
||||
datapoint_info!(
|
||||
"qos-service-stats",
|
||||
("compute_cost_time", self.compute_cost_time, i64),
|
||||
("cost_tracking_time", self.cost_tracking_time, i64),
|
||||
("selected_txs_count", self.selected_txs_count, i64),
|
||||
(
|
||||
"retried_txs_per_block_limit_count",
|
||||
self.retried_txs_per_block_limit_count,
|
||||
i64
|
||||
),
|
||||
(
|
||||
"retried_txs_per_account_limit_count",
|
||||
self.retried_txs_per_account_limit_count,
|
||||
i64
|
||||
),
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
pub struct QosService {
|
||||
cost_model: Arc<RwLock<CostModel>>,
|
||||
}
|
||||
|
||||
impl QosService {
|
||||
pub fn new(cost_model: Arc<RwLock<CostModel>>) -> Self {
|
||||
Self { cost_model }
|
||||
}
|
||||
|
||||
pub fn compute_transaction_costs<'a>(
|
||||
&self,
|
||||
transactions: impl Iterator<Item = &'a SanitizedTransaction>,
|
||||
demote_program_write_locks: bool,
|
||||
stats: &mut QosServiceStats,
|
||||
) -> Vec<TransactionCost> {
|
||||
let mut compute_cost_time = Measure::start("compute_cost_time");
|
||||
let cost_model = self.cost_model.read().unwrap();
|
||||
let txs_costs = transactions
|
||||
.map(|tx| {
|
||||
let cost = cost_model.calculate_cost(tx, demote_program_write_locks);
|
||||
debug!(
|
||||
"transaction {:?}, cost {:?}, cost sum {}",
|
||||
tx,
|
||||
cost,
|
||||
cost.sum()
|
||||
);
|
||||
cost
|
||||
})
|
||||
.collect();
|
||||
compute_cost_time.stop();
|
||||
stats.compute_cost_time += compute_cost_time.as_us();
|
||||
txs_costs
|
||||
}
|
||||
|
||||
// Given a list of transactions and their costs, this function returns a corresponding
|
||||
// list of Results that indicate if a transaction is selected to be included in the current block,
|
||||
pub fn select_transactions_per_cost<'a>(
|
||||
&self,
|
||||
transactions: impl Iterator<Item = &'a SanitizedTransaction>,
|
||||
transactions_costs: impl Iterator<Item = &'a TransactionCost>,
|
||||
bank: &Arc<Bank>,
|
||||
stats: &mut QosServiceStats,
|
||||
) -> Vec<transaction::Result<()>> {
|
||||
let mut cost_tracking_time = Measure::start("cost_tracking_time");
|
||||
let mut cost_tracker = bank.write_cost_tracker().unwrap();
|
||||
let select_results = transactions
|
||||
.zip(transactions_costs)
|
||||
.map(|(tx, cost)| match cost_tracker.try_add(tx, cost) {
|
||||
Ok(current_block_cost) => {
|
||||
debug!("slot {:?}, transaction {:?}, cost {:?}, fit into current block, current block cost {}", bank.slot(), tx, cost, current_block_cost);
|
||||
stats.selected_txs_count += 1;
|
||||
Ok(())
|
||||
},
|
||||
Err(e) => {
|
||||
debug!("slot {:?}, transaction {:?}, cost {:?}, not fit into current block, '{:?}'", bank.slot(), tx, cost, e);
|
||||
match e {
|
||||
CostTrackerError::WouldExceedBlockMaxLimit => {
|
||||
stats.retried_txs_per_block_limit_count += 1;
|
||||
Err(TransactionError::WouldExceedMaxBlockCostLimit)
|
||||
}
|
||||
CostTrackerError::WouldExceedAccountMaxLimit => {
|
||||
stats.retried_txs_per_account_limit_count += 1;
|
||||
Err(TransactionError::WouldExceedMaxAccountCostLimit)
|
||||
}
|
||||
}
|
||||
}
|
||||
})
|
||||
.collect();
|
||||
cost_tracking_time.stop();
|
||||
stats.cost_tracking_time += cost_tracking_time.as_us();
|
||||
select_results
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use {
|
||||
super::*,
|
||||
itertools::Itertools,
|
||||
solana_runtime::{
|
||||
bank::Bank,
|
||||
genesis_utils::{create_genesis_config, GenesisConfigInfo},
|
||||
},
|
||||
solana_sdk::{
|
||||
hash::Hash,
|
||||
signature::{Keypair, Signer},
|
||||
system_transaction,
|
||||
},
|
||||
solana_vote_program::vote_transaction,
|
||||
};
|
||||
|
||||
#[test]
|
||||
fn test_compute_transactions_costs() {
|
||||
solana_logger::setup();
|
||||
|
||||
// make a vec of txs
|
||||
let keypair = Keypair::new();
|
||||
let transfer_tx = SanitizedTransaction::from_transaction_for_tests(
|
||||
system_transaction::transfer(&keypair, &keypair.pubkey(), 1, Hash::default()),
|
||||
);
|
||||
let vote_tx = SanitizedTransaction::from_transaction_for_tests(
|
||||
vote_transaction::new_vote_transaction(
|
||||
vec![42],
|
||||
Hash::default(),
|
||||
Hash::default(),
|
||||
&keypair,
|
||||
&keypair,
|
||||
&keypair,
|
||||
None,
|
||||
),
|
||||
);
|
||||
let txs = vec![transfer_tx.clone(), vote_tx.clone(), vote_tx, transfer_tx];
|
||||
|
||||
let cost_model = Arc::new(RwLock::new(CostModel::default()));
|
||||
let qos_service = QosService::new(cost_model.clone());
|
||||
let txs_costs = qos_service.compute_transaction_costs(
|
||||
txs.iter(),
|
||||
false,
|
||||
&mut QosServiceStats::default(),
|
||||
);
|
||||
|
||||
// verify the size of txs_costs and its contents
|
||||
assert_eq!(txs_costs.len(), txs.len());
|
||||
txs_costs
|
||||
.iter()
|
||||
.enumerate()
|
||||
.map(|(index, cost)| {
|
||||
assert_eq!(
|
||||
cost.sum(),
|
||||
cost_model
|
||||
.read()
|
||||
.unwrap()
|
||||
.calculate_cost(&txs[index], false)
|
||||
.sum()
|
||||
);
|
||||
})
|
||||
.collect_vec();
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_select_transactions_per_cost() {
|
||||
solana_logger::setup();
|
||||
let GenesisConfigInfo { genesis_config, .. } = create_genesis_config(10);
|
||||
let bank = Arc::new(Bank::new_for_tests(&genesis_config));
|
||||
let cost_model = Arc::new(RwLock::new(CostModel::default()));
|
||||
|
||||
let keypair = Keypair::new();
|
||||
let transfer_tx = SanitizedTransaction::from_transaction_for_tests(
|
||||
system_transaction::transfer(&keypair, &keypair.pubkey(), 1, Hash::default()),
|
||||
);
|
||||
let vote_tx = SanitizedTransaction::from_transaction_for_tests(
|
||||
vote_transaction::new_vote_transaction(
|
||||
vec![42],
|
||||
Hash::default(),
|
||||
Hash::default(),
|
||||
&keypair,
|
||||
&keypair,
|
||||
&keypair,
|
||||
None,
|
||||
),
|
||||
);
|
||||
let transfer_tx_cost = cost_model
|
||||
.read()
|
||||
.unwrap()
|
||||
.calculate_cost(&transfer_tx, false)
|
||||
.sum();
|
||||
|
||||
// make a vec of txs
|
||||
let txs = vec![transfer_tx.clone(), vote_tx.clone(), transfer_tx, vote_tx];
|
||||
|
||||
let qos_service = QosService::new(cost_model);
|
||||
let txs_costs = qos_service.compute_transaction_costs(
|
||||
txs.iter(),
|
||||
false,
|
||||
&mut QosServiceStats::default(),
|
||||
);
|
||||
|
||||
// set cost tracker limit to fit 1 transfer tx, vote tx bypasses limit check
|
||||
let cost_limit = transfer_tx_cost;
|
||||
bank.write_cost_tracker()
|
||||
.unwrap()
|
||||
.set_limits(cost_limit, cost_limit);
|
||||
let results = qos_service.select_transactions_per_cost(
|
||||
txs.iter(),
|
||||
txs_costs.iter(),
|
||||
&bank,
|
||||
&mut QosServiceStats::default(),
|
||||
);
|
||||
|
||||
// verify that first transfer tx and all votes are allowed
|
||||
assert_eq!(results.len(), txs.len());
|
||||
assert!(results[0].is_ok());
|
||||
assert!(results[1].is_ok());
|
||||
assert!(results[2].is_err());
|
||||
assert!(results[3].is_ok());
|
||||
}
|
||||
}
|
Reference in New Issue
Block a user