- move cost tracker into bank, so each bank has its own cost tracker; (#20527)
- move related modules to runtime
This commit is contained in:
@ -8,9 +8,6 @@ use log::*;
|
||||
use rand::{thread_rng, Rng};
|
||||
use rayon::prelude::*;
|
||||
use solana_core::banking_stage::{BankingStage, BankingStageStats};
|
||||
use solana_core::cost_model::CostModel;
|
||||
use solana_core::cost_tracker::CostTracker;
|
||||
use solana_core::cost_tracker_stats::CostTrackerStats;
|
||||
use solana_entry::entry::{next_hash, Entry};
|
||||
use solana_gossip::cluster_info::ClusterInfo;
|
||||
use solana_gossip::cluster_info::Node;
|
||||
@ -21,6 +18,9 @@ use solana_perf::packet::to_packets_chunked;
|
||||
use solana_perf::test_tx::test_tx;
|
||||
use solana_poh::poh_recorder::{create_test_recorder, WorkingBankEntry};
|
||||
use solana_runtime::bank::Bank;
|
||||
use solana_runtime::cost_model::CostModel;
|
||||
use solana_runtime::cost_tracker::CostTracker;
|
||||
use solana_runtime::cost_tracker_stats::CostTrackerStats;
|
||||
use solana_sdk::genesis_config::GenesisConfig;
|
||||
use solana_sdk::hash::Hash;
|
||||
use solana_sdk::message::Message;
|
||||
|
@ -1,9 +1,7 @@
|
||||
//! 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::{
|
||||
cost_tracker::CostTracker, cost_tracker_stats::CostTrackerStats, packet_hasher::PacketHasher,
|
||||
};
|
||||
use crate::packet_hasher::PacketHasher;
|
||||
use crossbeam_channel::{Receiver as CrossbeamReceiver, RecvTimeoutError};
|
||||
use itertools::Itertools;
|
||||
use lru::LruCache;
|
||||
@ -27,6 +25,8 @@ use solana_runtime::{
|
||||
TransactionExecutionResult,
|
||||
},
|
||||
bank_utils,
|
||||
cost_tracker::CostTracker,
|
||||
cost_tracker_stats::CostTrackerStats,
|
||||
transaction_batch::TransactionBatch,
|
||||
vote_sender_types::ReplayVoteSender,
|
||||
};
|
||||
@ -1697,7 +1697,6 @@ where
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
use crate::cost_model::CostModel;
|
||||
use crossbeam_channel::unbounded;
|
||||
use itertools::Itertools;
|
||||
use solana_entry::entry::{next_entry, Entry, EntrySlice};
|
||||
@ -1714,6 +1713,7 @@ mod tests {
|
||||
poh_service::PohService,
|
||||
};
|
||||
use solana_rpc::transaction_status_service::TransactionStatusService;
|
||||
use solana_runtime::cost_model::CostModel;
|
||||
use solana_sdk::{
|
||||
hash::Hash,
|
||||
instruction::InstructionError,
|
||||
|
@ -1,539 +0,0 @@
|
||||
//! 'cost_model` provides service to estimate a transaction's cost
|
||||
//! following proposed fee schedule #16984; Relevant cluster cost
|
||||
//! measuring is described by #19627
|
||||
//!
|
||||
//! The main function is `calculate_cost` which returns &TransactionCost.
|
||||
//!
|
||||
use crate::execute_cost_table::ExecuteCostTable;
|
||||
use log::*;
|
||||
use solana_ledger::block_cost_limits::*;
|
||||
use solana_sdk::{pubkey::Pubkey, transaction::SanitizedTransaction};
|
||||
use std::collections::HashMap;
|
||||
|
||||
const MAX_WRITABLE_ACCOUNTS: usize = 256;
|
||||
|
||||
#[derive(Debug, Clone)]
|
||||
pub enum CostModelError {
|
||||
/// transaction that would fail sanitize, cost model is not able to process
|
||||
/// such transaction.
|
||||
InvalidTransaction,
|
||||
|
||||
/// would exceed block max limit
|
||||
WouldExceedBlockMaxLimit,
|
||||
|
||||
/// would exceed account max limit
|
||||
WouldExceedAccountMaxLimit,
|
||||
}
|
||||
|
||||
#[derive(Default, Debug)]
|
||||
pub struct TransactionCost {
|
||||
pub writable_accounts: Vec<Pubkey>,
|
||||
pub signature_cost: u64,
|
||||
pub write_lock_cost: u64,
|
||||
pub data_bytes_cost: u64,
|
||||
pub execution_cost: u64,
|
||||
}
|
||||
|
||||
impl TransactionCost {
|
||||
pub fn new_with_capacity(capacity: usize) -> Self {
|
||||
Self {
|
||||
writable_accounts: Vec::with_capacity(capacity),
|
||||
..Self::default()
|
||||
}
|
||||
}
|
||||
|
||||
pub fn reset(&mut self) {
|
||||
self.writable_accounts.clear();
|
||||
self.signature_cost = 0;
|
||||
self.write_lock_cost = 0;
|
||||
self.data_bytes_cost = 0;
|
||||
self.execution_cost = 0;
|
||||
}
|
||||
|
||||
pub fn sum(&self) -> u64 {
|
||||
self.signature_cost + self.write_lock_cost + self.data_bytes_cost + self.execution_cost
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
pub struct CostModel {
|
||||
account_cost_limit: u64,
|
||||
block_cost_limit: u64,
|
||||
instruction_execution_cost_table: ExecuteCostTable,
|
||||
|
||||
// reusable variables
|
||||
transaction_cost: TransactionCost,
|
||||
}
|
||||
|
||||
impl Default for CostModel {
|
||||
fn default() -> Self {
|
||||
CostModel::new(MAX_WRITABLE_ACCOUNT_UNITS, MAX_BLOCK_UNITS)
|
||||
}
|
||||
}
|
||||
|
||||
impl CostModel {
|
||||
pub fn new(chain_max: u64, block_max: u64) -> Self {
|
||||
Self {
|
||||
account_cost_limit: chain_max,
|
||||
block_cost_limit: block_max,
|
||||
instruction_execution_cost_table: ExecuteCostTable::default(),
|
||||
transaction_cost: TransactionCost::new_with_capacity(MAX_WRITABLE_ACCOUNTS),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn get_account_cost_limit(&self) -> u64 {
|
||||
self.account_cost_limit
|
||||
}
|
||||
|
||||
pub fn get_block_cost_limit(&self) -> u64 {
|
||||
self.block_cost_limit
|
||||
}
|
||||
|
||||
pub fn initialize_cost_table(&mut self, cost_table: &[(Pubkey, u64)]) {
|
||||
cost_table
|
||||
.iter()
|
||||
.map(|(key, cost)| (key, cost))
|
||||
.chain(BUILT_IN_INSTRUCTION_COSTS.iter())
|
||||
.for_each(|(program_id, cost)| {
|
||||
match self
|
||||
.instruction_execution_cost_table
|
||||
.upsert(program_id, *cost)
|
||||
{
|
||||
Some(c) => {
|
||||
debug!(
|
||||
"initiating cost table, instruction {:?} has cost {}",
|
||||
program_id, c
|
||||
);
|
||||
}
|
||||
None => {
|
||||
debug!(
|
||||
"initiating cost table, failed for instruction {:?}",
|
||||
program_id
|
||||
);
|
||||
}
|
||||
}
|
||||
});
|
||||
debug!(
|
||||
"restored cost model instruction cost table from blockstore, current values: {:?}",
|
||||
self.get_instruction_cost_table()
|
||||
);
|
||||
}
|
||||
|
||||
pub fn calculate_cost(
|
||||
&mut self,
|
||||
transaction: &SanitizedTransaction,
|
||||
demote_program_write_locks: bool,
|
||||
) -> &TransactionCost {
|
||||
self.transaction_cost.reset();
|
||||
|
||||
self.transaction_cost.signature_cost = self.get_signature_cost(transaction);
|
||||
self.get_write_lock_cost(transaction, demote_program_write_locks);
|
||||
self.transaction_cost.data_bytes_cost = self.get_data_bytes_cost(transaction);
|
||||
self.transaction_cost.execution_cost = self.get_transaction_cost(transaction);
|
||||
|
||||
debug!(
|
||||
"transaction {:?} has cost {:?}",
|
||||
transaction, self.transaction_cost
|
||||
);
|
||||
&self.transaction_cost
|
||||
}
|
||||
|
||||
pub fn upsert_instruction_cost(
|
||||
&mut self,
|
||||
program_key: &Pubkey,
|
||||
cost: u64,
|
||||
) -> Result<u64, &'static str> {
|
||||
self.instruction_execution_cost_table
|
||||
.upsert(program_key, cost);
|
||||
match self.instruction_execution_cost_table.get_cost(program_key) {
|
||||
Some(cost) => Ok(*cost),
|
||||
None => Err("failed to upsert to ExecuteCostTable"),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn get_instruction_cost_table(&self) -> &HashMap<Pubkey, u64> {
|
||||
self.instruction_execution_cost_table.get_cost_table()
|
||||
}
|
||||
|
||||
fn get_signature_cost(&self, transaction: &SanitizedTransaction) -> u64 {
|
||||
transaction.signatures().len() as u64 * SIGNATURE_COST
|
||||
}
|
||||
|
||||
fn get_write_lock_cost(
|
||||
&mut self,
|
||||
transaction: &SanitizedTransaction,
|
||||
demote_program_write_locks: bool,
|
||||
) {
|
||||
let message = transaction.message();
|
||||
message.account_keys_iter().enumerate().for_each(|(i, k)| {
|
||||
let is_writable = message.is_writable(i, demote_program_write_locks);
|
||||
|
||||
if is_writable {
|
||||
self.transaction_cost.writable_accounts.push(*k);
|
||||
self.transaction_cost.write_lock_cost += WRITE_LOCK_UNITS;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
fn get_data_bytes_cost(&self, transaction: &SanitizedTransaction) -> u64 {
|
||||
let mut data_bytes_cost: u64 = 0;
|
||||
transaction
|
||||
.message()
|
||||
.program_instructions_iter()
|
||||
.for_each(|(_, ix)| {
|
||||
data_bytes_cost += ix.data.len() as u64 / DATA_BYTES_UNITS;
|
||||
});
|
||||
data_bytes_cost
|
||||
}
|
||||
|
||||
fn get_transaction_cost(&self, transaction: &SanitizedTransaction) -> u64 {
|
||||
let mut cost: u64 = 0;
|
||||
|
||||
for (program_id, instruction) in transaction.message().program_instructions_iter() {
|
||||
let instruction_cost = self.find_instruction_cost(program_id);
|
||||
trace!(
|
||||
"instruction {:?} has cost of {}",
|
||||
instruction,
|
||||
instruction_cost
|
||||
);
|
||||
cost = cost.saturating_add(instruction_cost);
|
||||
}
|
||||
cost
|
||||
}
|
||||
|
||||
fn find_instruction_cost(&self, program_key: &Pubkey) -> u64 {
|
||||
match self.instruction_execution_cost_table.get_cost(program_key) {
|
||||
Some(cost) => *cost,
|
||||
None => {
|
||||
let default_value = self.instruction_execution_cost_table.get_mode();
|
||||
debug!(
|
||||
"Program key {:?} does not have assigned cost, using mode {}",
|
||||
program_key, default_value
|
||||
);
|
||||
default_value
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
use solana_runtime::{
|
||||
bank::Bank,
|
||||
genesis_utils::{create_genesis_config, GenesisConfigInfo},
|
||||
};
|
||||
use solana_sdk::{
|
||||
bpf_loader,
|
||||
hash::Hash,
|
||||
instruction::CompiledInstruction,
|
||||
message::Message,
|
||||
signature::{Keypair, Signer},
|
||||
system_instruction::{self},
|
||||
system_program, system_transaction,
|
||||
transaction::Transaction,
|
||||
};
|
||||
use std::{
|
||||
convert::{TryFrom, TryInto},
|
||||
str::FromStr,
|
||||
sync::{Arc, RwLock},
|
||||
thread::{self, JoinHandle},
|
||||
};
|
||||
|
||||
fn test_setup() -> (Keypair, Hash) {
|
||||
solana_logger::setup();
|
||||
let GenesisConfigInfo {
|
||||
genesis_config,
|
||||
mint_keypair,
|
||||
..
|
||||
} = create_genesis_config(10);
|
||||
let bank = Arc::new(Bank::new_no_wallclock_throttle_for_tests(&genesis_config));
|
||||
let start_hash = bank.last_blockhash();
|
||||
(mint_keypair, start_hash)
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_cost_model_instruction_cost() {
|
||||
let mut testee = CostModel::default();
|
||||
|
||||
let known_key = Pubkey::from_str("known11111111111111111111111111111111111111").unwrap();
|
||||
testee.upsert_instruction_cost(&known_key, 100).unwrap();
|
||||
// find cost for known programs
|
||||
assert_eq!(100, testee.find_instruction_cost(&known_key));
|
||||
|
||||
testee
|
||||
.upsert_instruction_cost(&bpf_loader::id(), 1999)
|
||||
.unwrap();
|
||||
assert_eq!(1999, testee.find_instruction_cost(&bpf_loader::id()));
|
||||
|
||||
// unknown program is assigned with default cost
|
||||
assert_eq!(
|
||||
testee.instruction_execution_cost_table.get_mode(),
|
||||
testee.find_instruction_cost(
|
||||
&Pubkey::from_str("unknown111111111111111111111111111111111111").unwrap()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_cost_model_simple_transaction() {
|
||||
let (mint_keypair, start_hash) = test_setup();
|
||||
|
||||
let keypair = Keypair::new();
|
||||
let simple_transaction: SanitizedTransaction =
|
||||
system_transaction::transfer(&mint_keypair, &keypair.pubkey(), 2, start_hash)
|
||||
.try_into()
|
||||
.unwrap();
|
||||
debug!(
|
||||
"system_transaction simple_transaction {:?}",
|
||||
simple_transaction
|
||||
);
|
||||
|
||||
// expected cost for one system transfer instructions
|
||||
let expected_cost = 8;
|
||||
|
||||
let mut testee = CostModel::default();
|
||||
testee
|
||||
.upsert_instruction_cost(&system_program::id(), expected_cost)
|
||||
.unwrap();
|
||||
assert_eq!(
|
||||
expected_cost,
|
||||
testee.get_transaction_cost(&simple_transaction)
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_cost_model_transaction_many_transfer_instructions() {
|
||||
let (mint_keypair, start_hash) = test_setup();
|
||||
|
||||
let key1 = solana_sdk::pubkey::new_rand();
|
||||
let key2 = solana_sdk::pubkey::new_rand();
|
||||
let instructions =
|
||||
system_instruction::transfer_many(&mint_keypair.pubkey(), &[(key1, 1), (key2, 1)]);
|
||||
let message = Message::new(&instructions, Some(&mint_keypair.pubkey()));
|
||||
let tx: SanitizedTransaction = Transaction::new(&[&mint_keypair], message, start_hash)
|
||||
.try_into()
|
||||
.unwrap();
|
||||
debug!("many transfer transaction {:?}", tx);
|
||||
|
||||
// expected cost for two system transfer instructions
|
||||
let program_cost = 8;
|
||||
let expected_cost = program_cost * 2;
|
||||
|
||||
let mut testee = CostModel::default();
|
||||
testee
|
||||
.upsert_instruction_cost(&system_program::id(), program_cost)
|
||||
.unwrap();
|
||||
assert_eq!(expected_cost, testee.get_transaction_cost(&tx));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_cost_model_message_many_different_instructions() {
|
||||
let (mint_keypair, start_hash) = test_setup();
|
||||
|
||||
// construct a transaction with multiple random instructions
|
||||
let key1 = solana_sdk::pubkey::new_rand();
|
||||
let key2 = solana_sdk::pubkey::new_rand();
|
||||
let prog1 = solana_sdk::pubkey::new_rand();
|
||||
let prog2 = solana_sdk::pubkey::new_rand();
|
||||
let instructions = vec![
|
||||
CompiledInstruction::new(3, &(), vec![0, 1]),
|
||||
CompiledInstruction::new(4, &(), vec![0, 2]),
|
||||
];
|
||||
let tx: SanitizedTransaction = Transaction::new_with_compiled_instructions(
|
||||
&[&mint_keypair],
|
||||
&[key1, key2],
|
||||
start_hash,
|
||||
vec![prog1, prog2],
|
||||
instructions,
|
||||
)
|
||||
.try_into()
|
||||
.unwrap();
|
||||
debug!("many random transaction {:?}", tx);
|
||||
|
||||
let testee = CostModel::default();
|
||||
let result = testee.get_transaction_cost(&tx);
|
||||
|
||||
// expected cost for two random/unknown program is
|
||||
let expected_cost = testee.instruction_execution_cost_table.get_mode() * 2;
|
||||
assert_eq!(expected_cost, result);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_cost_model_sort_message_accounts_by_type() {
|
||||
// construct a transaction with two random instructions with same signer
|
||||
let signer1 = Keypair::new();
|
||||
let signer2 = Keypair::new();
|
||||
let key1 = Pubkey::new_unique();
|
||||
let key2 = Pubkey::new_unique();
|
||||
let prog1 = Pubkey::new_unique();
|
||||
let prog2 = Pubkey::new_unique();
|
||||
let instructions = vec![
|
||||
CompiledInstruction::new(4, &(), vec![0, 2]),
|
||||
CompiledInstruction::new(5, &(), vec![1, 3]),
|
||||
];
|
||||
let tx: SanitizedTransaction = Transaction::new_with_compiled_instructions(
|
||||
&[&signer1, &signer2],
|
||||
&[key1, key2],
|
||||
Hash::new_unique(),
|
||||
vec![prog1, prog2],
|
||||
instructions,
|
||||
)
|
||||
.try_into()
|
||||
.unwrap();
|
||||
|
||||
let mut cost_model = CostModel::default();
|
||||
let tx_cost = cost_model.calculate_cost(&tx, /*demote_program_write_locks=*/ true);
|
||||
assert_eq!(2 + 2, tx_cost.writable_accounts.len());
|
||||
assert_eq!(signer1.pubkey(), tx_cost.writable_accounts[0]);
|
||||
assert_eq!(signer2.pubkey(), tx_cost.writable_accounts[1]);
|
||||
assert_eq!(key1, tx_cost.writable_accounts[2]);
|
||||
assert_eq!(key2, tx_cost.writable_accounts[3]);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_cost_model_insert_instruction_cost() {
|
||||
let key1 = Pubkey::new_unique();
|
||||
let cost1 = 100;
|
||||
|
||||
let mut cost_model = CostModel::default();
|
||||
// Using default cost for unknown instruction
|
||||
assert_eq!(
|
||||
cost_model.instruction_execution_cost_table.get_mode(),
|
||||
cost_model.find_instruction_cost(&key1)
|
||||
);
|
||||
|
||||
// insert instruction cost to table
|
||||
assert!(cost_model.upsert_instruction_cost(&key1, cost1).is_ok());
|
||||
|
||||
// now it is known insturction with known cost
|
||||
assert_eq!(cost1, cost_model.find_instruction_cost(&key1));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_cost_model_calculate_cost() {
|
||||
let (mint_keypair, start_hash) = test_setup();
|
||||
let tx: SanitizedTransaction =
|
||||
system_transaction::transfer(&mint_keypair, &Keypair::new().pubkey(), 2, start_hash)
|
||||
.try_into()
|
||||
.unwrap();
|
||||
|
||||
let expected_account_cost = WRITE_LOCK_UNITS * 2;
|
||||
let expected_execution_cost = 8;
|
||||
|
||||
let mut cost_model = CostModel::default();
|
||||
cost_model
|
||||
.upsert_instruction_cost(&system_program::id(), expected_execution_cost)
|
||||
.unwrap();
|
||||
let tx_cost = cost_model.calculate_cost(&tx, /*demote_program_write_locks=*/ true);
|
||||
assert_eq!(expected_account_cost, tx_cost.write_lock_cost);
|
||||
assert_eq!(expected_execution_cost, tx_cost.execution_cost);
|
||||
assert_eq!(2, tx_cost.writable_accounts.len());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_cost_model_update_instruction_cost() {
|
||||
let key1 = Pubkey::new_unique();
|
||||
let cost1 = 100;
|
||||
let cost2 = 200;
|
||||
let updated_cost = (cost1 + cost2) / 2;
|
||||
|
||||
let mut cost_model = CostModel::default();
|
||||
|
||||
// insert instruction cost to table
|
||||
assert!(cost_model.upsert_instruction_cost(&key1, cost1).is_ok());
|
||||
assert_eq!(cost1, cost_model.find_instruction_cost(&key1));
|
||||
|
||||
// update instruction cost
|
||||
assert!(cost_model.upsert_instruction_cost(&key1, cost2).is_ok());
|
||||
assert_eq!(updated_cost, cost_model.find_instruction_cost(&key1));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_cost_model_can_be_shared_concurrently_with_rwlock() {
|
||||
let (mint_keypair, start_hash) = test_setup();
|
||||
// construct a transaction with multiple random instructions
|
||||
let key1 = solana_sdk::pubkey::new_rand();
|
||||
let key2 = solana_sdk::pubkey::new_rand();
|
||||
let prog1 = solana_sdk::pubkey::new_rand();
|
||||
let prog2 = solana_sdk::pubkey::new_rand();
|
||||
let instructions = vec![
|
||||
CompiledInstruction::new(3, &(), vec![0, 1]),
|
||||
CompiledInstruction::new(4, &(), vec![0, 2]),
|
||||
];
|
||||
let tx = Arc::new(
|
||||
SanitizedTransaction::try_from(Transaction::new_with_compiled_instructions(
|
||||
&[&mint_keypair],
|
||||
&[key1, key2],
|
||||
start_hash,
|
||||
vec![prog1, prog2],
|
||||
instructions,
|
||||
))
|
||||
.unwrap(),
|
||||
);
|
||||
|
||||
let number_threads = 10;
|
||||
let expected_account_cost = WRITE_LOCK_UNITS * 3;
|
||||
let cost1 = 100;
|
||||
let cost2 = 200;
|
||||
// execution cost can be either 2 * Default (before write) or cost1+cost2 (after write)
|
||||
|
||||
let cost_model: Arc<RwLock<CostModel>> = Arc::new(RwLock::new(CostModel::default()));
|
||||
|
||||
let thread_handlers: Vec<JoinHandle<()>> = (0..number_threads)
|
||||
.map(|i| {
|
||||
let cost_model = cost_model.clone();
|
||||
let tx = tx.clone();
|
||||
|
||||
if i == 5 {
|
||||
thread::spawn(move || {
|
||||
let mut cost_model = cost_model.write().unwrap();
|
||||
assert!(cost_model.upsert_instruction_cost(&prog1, cost1).is_ok());
|
||||
assert!(cost_model.upsert_instruction_cost(&prog2, cost2).is_ok());
|
||||
})
|
||||
} else {
|
||||
thread::spawn(move || {
|
||||
let mut cost_model = cost_model.write().unwrap();
|
||||
let tx_cost = cost_model
|
||||
.calculate_cost(&tx, /*demote_program_write_locks=*/ true);
|
||||
assert_eq!(3, tx_cost.writable_accounts.len());
|
||||
assert_eq!(expected_account_cost, tx_cost.write_lock_cost);
|
||||
})
|
||||
}
|
||||
})
|
||||
.collect();
|
||||
|
||||
for th in thread_handlers {
|
||||
th.join().unwrap();
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_initialize_cost_table() {
|
||||
// build cost table
|
||||
let cost_table = vec![
|
||||
(Pubkey::new_unique(), 10),
|
||||
(Pubkey::new_unique(), 20),
|
||||
(Pubkey::new_unique(), 30),
|
||||
];
|
||||
|
||||
// init cost model
|
||||
let mut cost_model = CostModel::default();
|
||||
cost_model.initialize_cost_table(&cost_table);
|
||||
|
||||
// verify
|
||||
for (id, cost) in cost_table.iter() {
|
||||
assert_eq!(*cost, cost_model.find_instruction_cost(id));
|
||||
}
|
||||
|
||||
// verify built-in programs
|
||||
assert!(cost_model
|
||||
.instruction_execution_cost_table
|
||||
.get_cost(&system_program::id())
|
||||
.is_some());
|
||||
assert!(cost_model
|
||||
.instruction_execution_cost_table
|
||||
.get_cost(&solana_vote_program::id())
|
||||
.is_some());
|
||||
}
|
||||
}
|
@ -1,482 +0,0 @@
|
||||
//! `cost_tracker` keeps tracking transaction cost per chained accounts as well as for entire block
|
||||
//! It aggregates `cost_model`, which provides service of calculating transaction cost.
|
||||
//! The main functions are:
|
||||
//! - would_transaction_fit(&tx), immutable function to test if `tx` would fit into current block
|
||||
//! - add_transaction_cost(&tx), mutable function to accumulate `tx` cost to tracker.
|
||||
//!
|
||||
use crate::cost_model::{CostModel, CostModelError, TransactionCost};
|
||||
use crate::cost_tracker_stats::CostTrackerStats;
|
||||
use solana_sdk::{clock::Slot, pubkey::Pubkey, transaction::SanitizedTransaction};
|
||||
use std::{
|
||||
collections::HashMap,
|
||||
sync::{Arc, RwLock},
|
||||
};
|
||||
|
||||
const WRITABLE_ACCOUNTS_PER_BLOCK: usize = 512;
|
||||
|
||||
#[derive(Debug)]
|
||||
pub struct CostTracker {
|
||||
cost_model: Arc<RwLock<CostModel>>,
|
||||
account_cost_limit: u64,
|
||||
block_cost_limit: u64,
|
||||
current_bank_slot: Slot,
|
||||
cost_by_writable_accounts: HashMap<Pubkey, u64>,
|
||||
block_cost: u64,
|
||||
}
|
||||
|
||||
impl CostTracker {
|
||||
pub fn new(cost_model: Arc<RwLock<CostModel>>) -> Self {
|
||||
let (account_cost_limit, block_cost_limit) = {
|
||||
let cost_model = cost_model.read().unwrap();
|
||||
(
|
||||
cost_model.get_account_cost_limit(),
|
||||
cost_model.get_block_cost_limit(),
|
||||
)
|
||||
};
|
||||
assert!(account_cost_limit <= block_cost_limit);
|
||||
Self {
|
||||
cost_model,
|
||||
account_cost_limit,
|
||||
block_cost_limit,
|
||||
current_bank_slot: 0,
|
||||
cost_by_writable_accounts: HashMap::with_capacity(WRITABLE_ACCOUNTS_PER_BLOCK),
|
||||
block_cost: 0,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn would_transaction_fit(
|
||||
&self,
|
||||
transaction: &SanitizedTransaction,
|
||||
demote_program_write_locks: bool,
|
||||
stats: &mut CostTrackerStats,
|
||||
) -> Result<(), CostModelError> {
|
||||
let mut cost_model = self.cost_model.write().unwrap();
|
||||
let tx_cost = cost_model.calculate_cost(transaction, demote_program_write_locks);
|
||||
self.would_fit(&tx_cost.writable_accounts, &tx_cost.sum(), stats)
|
||||
}
|
||||
|
||||
pub fn add_transaction_cost(
|
||||
&mut self,
|
||||
transaction: &SanitizedTransaction,
|
||||
demote_program_write_locks: bool,
|
||||
stats: &mut CostTrackerStats,
|
||||
) {
|
||||
let mut cost_model = self.cost_model.write().unwrap();
|
||||
let tx_cost = cost_model.calculate_cost(transaction, demote_program_write_locks);
|
||||
let cost = tx_cost.sum();
|
||||
for account_key in tx_cost.writable_accounts.iter() {
|
||||
*self
|
||||
.cost_by_writable_accounts
|
||||
.entry(*account_key)
|
||||
.or_insert(0) += cost;
|
||||
}
|
||||
self.block_cost += cost;
|
||||
|
||||
stats.transaction_count += 1;
|
||||
stats.block_cost += cost;
|
||||
}
|
||||
|
||||
pub fn reset_if_new_bank(&mut self, slot: Slot, stats: &mut CostTrackerStats) -> bool {
|
||||
// report stats when slot changes
|
||||
if slot != stats.bank_slot {
|
||||
stats.report();
|
||||
*stats = CostTrackerStats::new(stats.id, slot);
|
||||
}
|
||||
|
||||
if slot != self.current_bank_slot {
|
||||
self.current_bank_slot = slot;
|
||||
self.cost_by_writable_accounts.clear();
|
||||
self.block_cost = 0;
|
||||
|
||||
true
|
||||
} else {
|
||||
false
|
||||
}
|
||||
}
|
||||
|
||||
pub fn try_add(
|
||||
&mut self,
|
||||
transaction_cost: &TransactionCost,
|
||||
stats: &mut CostTrackerStats,
|
||||
) -> Result<u64, CostModelError> {
|
||||
let cost = transaction_cost.sum();
|
||||
self.would_fit(&transaction_cost.writable_accounts, &cost, stats)?;
|
||||
|
||||
self.add_transaction(&transaction_cost.writable_accounts, &cost);
|
||||
Ok(self.block_cost)
|
||||
}
|
||||
|
||||
fn would_fit(
|
||||
&self,
|
||||
keys: &[Pubkey],
|
||||
cost: &u64,
|
||||
stats: &mut CostTrackerStats,
|
||||
) -> Result<(), CostModelError> {
|
||||
stats.transaction_cost_histogram.increment(*cost).unwrap();
|
||||
|
||||
// check against the total package cost
|
||||
if self.block_cost + cost > self.block_cost_limit {
|
||||
return Err(CostModelError::WouldExceedBlockMaxLimit);
|
||||
}
|
||||
|
||||
// check if the transaction itself is more costly than the account_cost_limit
|
||||
if *cost > self.account_cost_limit {
|
||||
return Err(CostModelError::WouldExceedAccountMaxLimit);
|
||||
}
|
||||
|
||||
// check each account against account_cost_limit,
|
||||
for account_key in keys.iter() {
|
||||
match self.cost_by_writable_accounts.get(account_key) {
|
||||
Some(chained_cost) => {
|
||||
stats
|
||||
.writable_accounts_cost_histogram
|
||||
.increment(*chained_cost)
|
||||
.unwrap();
|
||||
|
||||
if chained_cost + cost > self.account_cost_limit {
|
||||
return Err(CostModelError::WouldExceedAccountMaxLimit);
|
||||
} else {
|
||||
continue;
|
||||
}
|
||||
}
|
||||
None => continue,
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn add_transaction(&mut self, keys: &[Pubkey], cost: &u64) {
|
||||
for account_key in keys.iter() {
|
||||
*self
|
||||
.cost_by_writable_accounts
|
||||
.entry(*account_key)
|
||||
.or_insert(0) += cost;
|
||||
}
|
||||
self.block_cost += cost;
|
||||
}
|
||||
}
|
||||
|
||||
// CostStats can be collected by util, such as ledger_tool
|
||||
#[derive(Default, Debug)]
|
||||
pub struct CostStats {
|
||||
pub bank_slot: Slot,
|
||||
pub total_cost: u64,
|
||||
pub number_of_accounts: usize,
|
||||
pub costliest_account: Pubkey,
|
||||
pub costliest_account_cost: u64,
|
||||
}
|
||||
|
||||
impl CostTracker {
|
||||
pub fn get_stats(&self) -> CostStats {
|
||||
let mut stats = CostStats {
|
||||
bank_slot: self.current_bank_slot,
|
||||
total_cost: self.block_cost,
|
||||
number_of_accounts: self.cost_by_writable_accounts.len(),
|
||||
costliest_account: Pubkey::default(),
|
||||
costliest_account_cost: 0,
|
||||
};
|
||||
|
||||
for (key, cost) in self.cost_by_writable_accounts.iter() {
|
||||
if cost > &stats.costliest_account_cost {
|
||||
stats.costliest_account = *key;
|
||||
stats.costliest_account_cost = *cost;
|
||||
}
|
||||
}
|
||||
|
||||
stats
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
use solana_runtime::{
|
||||
bank::Bank,
|
||||
genesis_utils::{create_genesis_config, GenesisConfigInfo},
|
||||
};
|
||||
use solana_sdk::{
|
||||
hash::Hash,
|
||||
signature::{Keypair, Signer},
|
||||
system_transaction,
|
||||
transaction::Transaction,
|
||||
};
|
||||
use std::{cmp, sync::Arc};
|
||||
|
||||
fn test_setup() -> (Keypair, Hash) {
|
||||
solana_logger::setup();
|
||||
let GenesisConfigInfo {
|
||||
genesis_config,
|
||||
mint_keypair,
|
||||
..
|
||||
} = create_genesis_config(10);
|
||||
let bank = Arc::new(Bank::new_no_wallclock_throttle_for_tests(&genesis_config));
|
||||
let start_hash = bank.last_blockhash();
|
||||
(mint_keypair, start_hash)
|
||||
}
|
||||
|
||||
fn build_simple_transaction(
|
||||
mint_keypair: &Keypair,
|
||||
start_hash: &Hash,
|
||||
) -> (Transaction, Vec<Pubkey>, u64) {
|
||||
let keypair = Keypair::new();
|
||||
let simple_transaction =
|
||||
system_transaction::transfer(mint_keypair, &keypair.pubkey(), 2, *start_hash);
|
||||
|
||||
(simple_transaction, vec![mint_keypair.pubkey()], 5)
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_cost_tracker_initialization() {
|
||||
let testee = CostTracker::new(Arc::new(RwLock::new(CostModel::new(10, 11))));
|
||||
assert_eq!(10, testee.account_cost_limit);
|
||||
assert_eq!(11, testee.block_cost_limit);
|
||||
assert_eq!(0, testee.cost_by_writable_accounts.len());
|
||||
assert_eq!(0, testee.block_cost);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_cost_tracker_ok_add_one() {
|
||||
let (mint_keypair, start_hash) = test_setup();
|
||||
let (_tx, keys, cost) = build_simple_transaction(&mint_keypair, &start_hash);
|
||||
|
||||
// build testee to have capacity for one simple transaction
|
||||
let mut testee = CostTracker::new(Arc::new(RwLock::new(CostModel::new(cost, cost))));
|
||||
assert!(testee
|
||||
.would_fit(&keys, &cost, &mut CostTrackerStats::default())
|
||||
.is_ok());
|
||||
testee.add_transaction(&keys, &cost);
|
||||
assert_eq!(cost, testee.block_cost);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_cost_tracker_ok_add_two_same_accounts() {
|
||||
let (mint_keypair, start_hash) = test_setup();
|
||||
// build two transactions with same signed account
|
||||
let (_tx1, keys1, cost1) = build_simple_transaction(&mint_keypair, &start_hash);
|
||||
let (_tx2, keys2, cost2) = build_simple_transaction(&mint_keypair, &start_hash);
|
||||
|
||||
// build testee to have capacity for two simple transactions, with same accounts
|
||||
let mut testee = CostTracker::new(Arc::new(RwLock::new(CostModel::new(
|
||||
cost1 + cost2,
|
||||
cost1 + cost2,
|
||||
))));
|
||||
{
|
||||
assert!(testee
|
||||
.would_fit(&keys1, &cost1, &mut CostTrackerStats::default())
|
||||
.is_ok());
|
||||
testee.add_transaction(&keys1, &cost1);
|
||||
}
|
||||
{
|
||||
assert!(testee
|
||||
.would_fit(&keys2, &cost2, &mut CostTrackerStats::default())
|
||||
.is_ok());
|
||||
testee.add_transaction(&keys2, &cost2);
|
||||
}
|
||||
assert_eq!(cost1 + cost2, testee.block_cost);
|
||||
assert_eq!(1, testee.cost_by_writable_accounts.len());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_cost_tracker_ok_add_two_diff_accounts() {
|
||||
let (mint_keypair, start_hash) = test_setup();
|
||||
// build two transactions with diff accounts
|
||||
let (_tx1, keys1, cost1) = build_simple_transaction(&mint_keypair, &start_hash);
|
||||
let second_account = Keypair::new();
|
||||
let (_tx2, keys2, cost2) = build_simple_transaction(&second_account, &start_hash);
|
||||
|
||||
// build testee to have capacity for two simple transactions, with same accounts
|
||||
let mut testee = CostTracker::new(Arc::new(RwLock::new(CostModel::new(
|
||||
cmp::max(cost1, cost2),
|
||||
cost1 + cost2,
|
||||
))));
|
||||
{
|
||||
assert!(testee
|
||||
.would_fit(&keys1, &cost1, &mut CostTrackerStats::default())
|
||||
.is_ok());
|
||||
testee.add_transaction(&keys1, &cost1);
|
||||
}
|
||||
{
|
||||
assert!(testee
|
||||
.would_fit(&keys2, &cost2, &mut CostTrackerStats::default())
|
||||
.is_ok());
|
||||
testee.add_transaction(&keys2, &cost2);
|
||||
}
|
||||
assert_eq!(cost1 + cost2, testee.block_cost);
|
||||
assert_eq!(2, testee.cost_by_writable_accounts.len());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_cost_tracker_chain_reach_limit() {
|
||||
let (mint_keypair, start_hash) = test_setup();
|
||||
// build two transactions with same signed account
|
||||
let (_tx1, keys1, cost1) = build_simple_transaction(&mint_keypair, &start_hash);
|
||||
let (_tx2, keys2, cost2) = build_simple_transaction(&mint_keypair, &start_hash);
|
||||
|
||||
// build testee to have capacity for two simple transactions, but not for same accounts
|
||||
let mut testee = CostTracker::new(Arc::new(RwLock::new(CostModel::new(
|
||||
cmp::min(cost1, cost2),
|
||||
cost1 + cost2,
|
||||
))));
|
||||
// should have room for first transaction
|
||||
{
|
||||
assert!(testee
|
||||
.would_fit(&keys1, &cost1, &mut CostTrackerStats::default())
|
||||
.is_ok());
|
||||
testee.add_transaction(&keys1, &cost1);
|
||||
}
|
||||
// but no more sapce on the same chain (same signer account)
|
||||
{
|
||||
assert!(testee
|
||||
.would_fit(&keys2, &cost2, &mut CostTrackerStats::default())
|
||||
.is_err());
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_cost_tracker_reach_limit() {
|
||||
let (mint_keypair, start_hash) = test_setup();
|
||||
// build two transactions with diff accounts
|
||||
let (_tx1, keys1, cost1) = build_simple_transaction(&mint_keypair, &start_hash);
|
||||
let second_account = Keypair::new();
|
||||
let (_tx2, keys2, cost2) = build_simple_transaction(&second_account, &start_hash);
|
||||
|
||||
// build testee to have capacity for each chain, but not enough room for both transactions
|
||||
let mut testee = CostTracker::new(Arc::new(RwLock::new(CostModel::new(
|
||||
cmp::max(cost1, cost2),
|
||||
cost1 + cost2 - 1,
|
||||
))));
|
||||
// should have room for first transaction
|
||||
{
|
||||
assert!(testee
|
||||
.would_fit(&keys1, &cost1, &mut CostTrackerStats::default())
|
||||
.is_ok());
|
||||
testee.add_transaction(&keys1, &cost1);
|
||||
}
|
||||
// but no more room for package as whole
|
||||
{
|
||||
assert!(testee
|
||||
.would_fit(&keys2, &cost2, &mut CostTrackerStats::default())
|
||||
.is_err());
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_cost_tracker_reset() {
|
||||
let (mint_keypair, start_hash) = test_setup();
|
||||
// build two transactions with same signed account
|
||||
let (_tx1, keys1, cost1) = build_simple_transaction(&mint_keypair, &start_hash);
|
||||
let (_tx2, keys2, cost2) = build_simple_transaction(&mint_keypair, &start_hash);
|
||||
|
||||
// build testee to have capacity for two simple transactions, but not for same accounts
|
||||
let mut testee = CostTracker::new(Arc::new(RwLock::new(CostModel::new(
|
||||
cmp::min(cost1, cost2),
|
||||
cost1 + cost2,
|
||||
))));
|
||||
// should have room for first transaction
|
||||
{
|
||||
assert!(testee
|
||||
.would_fit(&keys1, &cost1, &mut CostTrackerStats::default())
|
||||
.is_ok());
|
||||
testee.add_transaction(&keys1, &cost1);
|
||||
assert_eq!(1, testee.cost_by_writable_accounts.len());
|
||||
assert_eq!(cost1, testee.block_cost);
|
||||
}
|
||||
// but no more sapce on the same chain (same signer account)
|
||||
{
|
||||
assert!(testee
|
||||
.would_fit(&keys2, &cost2, &mut CostTrackerStats::default())
|
||||
.is_err());
|
||||
}
|
||||
// reset the tracker
|
||||
{
|
||||
testee.reset_if_new_bank(100, &mut CostTrackerStats::default());
|
||||
assert_eq!(0, testee.cost_by_writable_accounts.len());
|
||||
assert_eq!(0, testee.block_cost);
|
||||
}
|
||||
//now the second transaction can be added
|
||||
{
|
||||
assert!(testee
|
||||
.would_fit(&keys2, &cost2, &mut CostTrackerStats::default())
|
||||
.is_ok());
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_cost_tracker_try_add_is_atomic() {
|
||||
let acct1 = Pubkey::new_unique();
|
||||
let acct2 = Pubkey::new_unique();
|
||||
let acct3 = Pubkey::new_unique();
|
||||
let cost = 100;
|
||||
let account_max = cost * 2;
|
||||
let block_max = account_max * 3; // for three accts
|
||||
|
||||
let mut testee = CostTracker::new(Arc::new(RwLock::new(CostModel::new(
|
||||
account_max,
|
||||
block_max,
|
||||
))));
|
||||
|
||||
// case 1: a tx writes to 3 accounts, should success, we will have:
|
||||
// | acct1 | $cost |
|
||||
// | acct2 | $cost |
|
||||
// | acct2 | $cost |
|
||||
// and block_cost = $cost
|
||||
{
|
||||
let tx_cost = TransactionCost {
|
||||
writable_accounts: vec![acct1, acct2, acct3],
|
||||
execution_cost: cost,
|
||||
..TransactionCost::default()
|
||||
};
|
||||
assert!(testee
|
||||
.try_add(&tx_cost, &mut CostTrackerStats::default())
|
||||
.is_ok());
|
||||
let stat = testee.get_stats();
|
||||
assert_eq!(cost, stat.total_cost);
|
||||
assert_eq!(3, stat.number_of_accounts);
|
||||
assert_eq!(cost, stat.costliest_account_cost);
|
||||
}
|
||||
|
||||
// case 2: add tx writes to acct2 with $cost, should succeed, result to
|
||||
// | acct1 | $cost |
|
||||
// | acct2 | $cost * 2 |
|
||||
// | acct2 | $cost |
|
||||
// and block_cost = $cost * 2
|
||||
{
|
||||
let tx_cost = TransactionCost {
|
||||
writable_accounts: vec![acct2],
|
||||
execution_cost: cost,
|
||||
..TransactionCost::default()
|
||||
};
|
||||
assert!(testee
|
||||
.try_add(&tx_cost, &mut CostTrackerStats::default())
|
||||
.is_ok());
|
||||
let stat = testee.get_stats();
|
||||
assert_eq!(cost * 2, stat.total_cost);
|
||||
assert_eq!(3, stat.number_of_accounts);
|
||||
assert_eq!(cost * 2, stat.costliest_account_cost);
|
||||
assert_eq!(acct2, stat.costliest_account);
|
||||
}
|
||||
|
||||
// case 3: add tx writes to [acct1, acct2], acct2 exceeds limit, should failed atomically,
|
||||
// we shoudl still have:
|
||||
// | acct1 | $cost |
|
||||
// | acct2 | $cost |
|
||||
// | acct2 | $cost |
|
||||
// and block_cost = $cost
|
||||
{
|
||||
let tx_cost = TransactionCost {
|
||||
writable_accounts: vec![acct1, acct2],
|
||||
execution_cost: cost,
|
||||
..TransactionCost::default()
|
||||
};
|
||||
assert!(testee
|
||||
.try_add(&tx_cost, &mut CostTrackerStats::default())
|
||||
.is_err());
|
||||
let stat = testee.get_stats();
|
||||
assert_eq!(cost * 2, stat.total_cost);
|
||||
assert_eq!(3, stat.number_of_accounts);
|
||||
assert_eq!(cost * 2, stat.costliest_account_cost);
|
||||
assert_eq!(acct2, stat.costliest_account);
|
||||
}
|
||||
}
|
||||
}
|
@ -1,75 +0,0 @@
|
||||
//! The Stats is not thread safe, each thread should have its own
|
||||
//! instance of stat with `id`; Stat reports and reset for each slot.
|
||||
#[derive(Debug, Default)]
|
||||
pub struct CostTrackerStats {
|
||||
pub id: u32,
|
||||
pub transaction_cost_histogram: histogram::Histogram,
|
||||
pub writable_accounts_cost_histogram: histogram::Histogram,
|
||||
pub transaction_count: u64,
|
||||
pub block_cost: u64,
|
||||
pub bank_slot: u64,
|
||||
}
|
||||
|
||||
impl CostTrackerStats {
|
||||
pub fn new(id: u32, bank_slot: u64) -> Self {
|
||||
CostTrackerStats {
|
||||
id,
|
||||
bank_slot,
|
||||
..CostTrackerStats::default()
|
||||
}
|
||||
}
|
||||
|
||||
pub fn report(&self) {
|
||||
datapoint_info!(
|
||||
"cost_tracker_stats",
|
||||
("id", self.id as i64, i64),
|
||||
(
|
||||
"transaction_cost_unit_min",
|
||||
self.transaction_cost_histogram.minimum().unwrap_or(0),
|
||||
i64
|
||||
),
|
||||
(
|
||||
"transaction_cost_unit_max",
|
||||
self.transaction_cost_histogram.maximum().unwrap_or(0),
|
||||
i64
|
||||
),
|
||||
(
|
||||
"transaction_cost_unit_mean",
|
||||
self.transaction_cost_histogram.mean().unwrap_or(0),
|
||||
i64
|
||||
),
|
||||
(
|
||||
"transaction_cost_unit_2nd_std",
|
||||
self.transaction_cost_histogram
|
||||
.percentile(95.0)
|
||||
.unwrap_or(0),
|
||||
i64
|
||||
),
|
||||
(
|
||||
"writable_accounts_cost_min",
|
||||
self.writable_accounts_cost_histogram.minimum().unwrap_or(0),
|
||||
i64
|
||||
),
|
||||
(
|
||||
"writable_accounts_cost_max",
|
||||
self.writable_accounts_cost_histogram.maximum().unwrap_or(0),
|
||||
i64
|
||||
),
|
||||
(
|
||||
"writable_accounts_cost_mean",
|
||||
self.writable_accounts_cost_histogram.mean().unwrap_or(0),
|
||||
i64
|
||||
),
|
||||
(
|
||||
"writable_accounts_cost_2nd_std",
|
||||
self.writable_accounts_cost_histogram
|
||||
.percentile(95.0)
|
||||
.unwrap_or(0),
|
||||
i64
|
||||
),
|
||||
("transaction_count", self.transaction_count as i64, i64),
|
||||
("block_cost", self.block_cost as i64, i64),
|
||||
("bank_slot", self.bank_slot as i64, i64),
|
||||
);
|
||||
}
|
||||
}
|
@ -3,10 +3,9 @@
|
||||
//! packing transactions into block; it also triggers persisting cost
|
||||
//! table to blockstore.
|
||||
|
||||
use crate::cost_model::CostModel;
|
||||
use solana_ledger::blockstore::Blockstore;
|
||||
use solana_measure::measure::Measure;
|
||||
use solana_runtime::bank::ExecuteTimings;
|
||||
use solana_runtime::{bank::ExecuteTimings, cost_model::CostModel};
|
||||
use solana_sdk::timing::timestamp;
|
||||
use std::{
|
||||
sync::{
|
||||
|
@ -1,279 +0,0 @@
|
||||
/// ExecuteCostTable is aggregated by Cost Model, it keeps each program's
|
||||
/// average cost in its HashMap, with fixed capacity to avoid from growing
|
||||
/// unchecked.
|
||||
/// When its capacity limit is reached, it prunes old and less-used programs
|
||||
/// to make room for new ones.
|
||||
use log::*;
|
||||
use solana_sdk::pubkey::Pubkey;
|
||||
use std::{collections::HashMap, time::SystemTime};
|
||||
|
||||
// prune is rather expensive op, free up bulk space in each operation
|
||||
// would be more efficient. PRUNE_RATIO defines the after prune table
|
||||
// size will be original_size * PRUNE_RATIO.
|
||||
const PRUNE_RATIO: f64 = 0.75;
|
||||
// with 50_000 TPS as norm, weights occurrences '100' per microsec
|
||||
const OCCURRENCES_WEIGHT: i64 = 100;
|
||||
|
||||
const DEFAULT_CAPACITY: usize = 1024;
|
||||
|
||||
#[derive(Debug)]
|
||||
pub struct ExecuteCostTable {
|
||||
capacity: usize,
|
||||
table: HashMap<Pubkey, u64>,
|
||||
occurrences: HashMap<Pubkey, (usize, SystemTime)>,
|
||||
}
|
||||
|
||||
impl Default for ExecuteCostTable {
|
||||
fn default() -> Self {
|
||||
ExecuteCostTable::new(DEFAULT_CAPACITY)
|
||||
}
|
||||
}
|
||||
|
||||
impl ExecuteCostTable {
|
||||
pub fn new(cap: usize) -> Self {
|
||||
Self {
|
||||
capacity: cap,
|
||||
table: HashMap::with_capacity(cap),
|
||||
occurrences: HashMap::with_capacity(cap),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn get_cost_table(&self) -> &HashMap<Pubkey, u64> {
|
||||
&self.table
|
||||
}
|
||||
|
||||
pub fn get_count(&self) -> usize {
|
||||
self.table.len()
|
||||
}
|
||||
|
||||
// instead of assigning unknown program with a configured/hard-coded cost
|
||||
// use average or mode function to make a educated guess.
|
||||
pub fn get_average(&self) -> u64 {
|
||||
if self.table.is_empty() {
|
||||
0
|
||||
} else {
|
||||
self.table.iter().map(|(_, value)| value).sum::<u64>() / self.get_count() as u64
|
||||
}
|
||||
}
|
||||
|
||||
pub fn get_mode(&self) -> u64 {
|
||||
if self.occurrences.is_empty() {
|
||||
0
|
||||
} else {
|
||||
let key = self
|
||||
.occurrences
|
||||
.iter()
|
||||
.max_by_key(|&(_, count)| count)
|
||||
.map(|(key, _)| key)
|
||||
.expect("cannot find mode from cost table");
|
||||
|
||||
*self.table.get(key).unwrap()
|
||||
}
|
||||
}
|
||||
|
||||
// returns None if program doesn't exist in table. In this case,
|
||||
// client is advised to call `get_average()` or `get_mode()` to
|
||||
// assign a 'default' value for new program.
|
||||
pub fn get_cost(&self, key: &Pubkey) -> Option<&u64> {
|
||||
self.table.get(key)
|
||||
}
|
||||
|
||||
pub fn upsert(&mut self, key: &Pubkey, value: u64) -> Option<u64> {
|
||||
let need_to_add = self.table.get(key).is_none();
|
||||
let current_size = self.get_count();
|
||||
if current_size == self.capacity && need_to_add {
|
||||
self.prune_to(&((current_size as f64 * PRUNE_RATIO) as usize));
|
||||
}
|
||||
|
||||
let program_cost = self.table.entry(*key).or_insert(value);
|
||||
*program_cost = (*program_cost + value) / 2;
|
||||
|
||||
let (count, timestamp) = self
|
||||
.occurrences
|
||||
.entry(*key)
|
||||
.or_insert((0, SystemTime::now()));
|
||||
*count += 1;
|
||||
*timestamp = SystemTime::now();
|
||||
|
||||
Some(*program_cost)
|
||||
}
|
||||
|
||||
// prune the old programs so the table contains `new_size` of records,
|
||||
// where `old` is defined as weighted age, which is negatively correlated
|
||||
// with program's age and
|
||||
// positively correlated with how frequently the program
|
||||
// is executed (eg. occurrence),
|
||||
fn prune_to(&mut self, new_size: &usize) {
|
||||
debug!(
|
||||
"prune cost table, current size {}, new size {}",
|
||||
self.get_count(),
|
||||
new_size
|
||||
);
|
||||
|
||||
if *new_size == self.get_count() {
|
||||
return;
|
||||
}
|
||||
|
||||
if *new_size == 0 {
|
||||
self.table.clear();
|
||||
self.occurrences.clear();
|
||||
return;
|
||||
}
|
||||
|
||||
let now = SystemTime::now();
|
||||
let mut sorted_by_weighted_age: Vec<_> = self
|
||||
.occurrences
|
||||
.iter()
|
||||
.map(|(key, (count, timestamp))| {
|
||||
let age = now.duration_since(*timestamp).unwrap().as_micros();
|
||||
let weighted_age = *count as i64 * OCCURRENCES_WEIGHT + -(age as i64);
|
||||
(weighted_age, *key)
|
||||
})
|
||||
.collect();
|
||||
sorted_by_weighted_age.sort_by(|x, y| x.0.partial_cmp(&y.0).unwrap());
|
||||
|
||||
for i in sorted_by_weighted_age.iter() {
|
||||
self.table.remove(&i.1);
|
||||
self.occurrences.remove(&i.1);
|
||||
if *new_size == self.get_count() {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
|
||||
#[test]
|
||||
fn test_execute_cost_table_prune_simple_table() {
|
||||
solana_logger::setup();
|
||||
let capacity: usize = 3;
|
||||
let mut testee = ExecuteCostTable::new(capacity);
|
||||
|
||||
let key1 = Pubkey::new_unique();
|
||||
let key2 = Pubkey::new_unique();
|
||||
let key3 = Pubkey::new_unique();
|
||||
|
||||
testee.upsert(&key1, 1);
|
||||
testee.upsert(&key2, 2);
|
||||
testee.upsert(&key3, 3);
|
||||
|
||||
testee.prune_to(&(capacity - 1));
|
||||
|
||||
// the oldest, key1, should be pruned
|
||||
assert!(testee.get_cost(&key1).is_none());
|
||||
assert!(testee.get_cost(&key2).is_some());
|
||||
assert!(testee.get_cost(&key2).is_some());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_execute_cost_table_prune_weighted_table() {
|
||||
solana_logger::setup();
|
||||
let capacity: usize = 3;
|
||||
let mut testee = ExecuteCostTable::new(capacity);
|
||||
|
||||
let key1 = Pubkey::new_unique();
|
||||
let key2 = Pubkey::new_unique();
|
||||
let key3 = Pubkey::new_unique();
|
||||
|
||||
testee.upsert(&key1, 1);
|
||||
testee.upsert(&key1, 1);
|
||||
testee.upsert(&key2, 2);
|
||||
testee.upsert(&key3, 3);
|
||||
|
||||
testee.prune_to(&(capacity - 1));
|
||||
|
||||
// the oldest, key1, has 2 counts; 2nd oldest Key2 has 1 count;
|
||||
// expect key2 to be pruned.
|
||||
assert!(testee.get_cost(&key1).is_some());
|
||||
assert!(testee.get_cost(&key2).is_none());
|
||||
assert!(testee.get_cost(&key3).is_some());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_execute_cost_table_upsert_within_capacity() {
|
||||
solana_logger::setup();
|
||||
let mut testee = ExecuteCostTable::default();
|
||||
|
||||
let key1 = Pubkey::new_unique();
|
||||
let key2 = Pubkey::new_unique();
|
||||
let cost1: u64 = 100;
|
||||
let cost2: u64 = 110;
|
||||
|
||||
// query empty table
|
||||
assert!(testee.get_cost(&key1).is_none());
|
||||
|
||||
// insert one record
|
||||
testee.upsert(&key1, cost1);
|
||||
assert_eq!(1, testee.get_count());
|
||||
assert_eq!(cost1, testee.get_average());
|
||||
assert_eq!(cost1, testee.get_mode());
|
||||
assert_eq!(&cost1, testee.get_cost(&key1).unwrap());
|
||||
|
||||
// insert 2nd record
|
||||
testee.upsert(&key2, cost2);
|
||||
assert_eq!(2, testee.get_count());
|
||||
assert_eq!((cost1 + cost2) / 2_u64, testee.get_average());
|
||||
assert_eq!(cost2, testee.get_mode());
|
||||
assert_eq!(&cost1, testee.get_cost(&key1).unwrap());
|
||||
assert_eq!(&cost2, testee.get_cost(&key2).unwrap());
|
||||
|
||||
// update 1st record
|
||||
testee.upsert(&key1, cost2);
|
||||
assert_eq!(2, testee.get_count());
|
||||
assert_eq!(((cost1 + cost2) / 2 + cost2) / 2, testee.get_average());
|
||||
assert_eq!((cost1 + cost2) / 2, testee.get_mode());
|
||||
assert_eq!(&((cost1 + cost2) / 2), testee.get_cost(&key1).unwrap());
|
||||
assert_eq!(&cost2, testee.get_cost(&key2).unwrap());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_execute_cost_table_upsert_exceeds_capacity() {
|
||||
solana_logger::setup();
|
||||
let capacity: usize = 2;
|
||||
let mut testee = ExecuteCostTable::new(capacity);
|
||||
|
||||
let key1 = Pubkey::new_unique();
|
||||
let key2 = Pubkey::new_unique();
|
||||
let key3 = Pubkey::new_unique();
|
||||
let key4 = Pubkey::new_unique();
|
||||
let cost1: u64 = 100;
|
||||
let cost2: u64 = 110;
|
||||
let cost3: u64 = 120;
|
||||
let cost4: u64 = 130;
|
||||
|
||||
// insert one record
|
||||
testee.upsert(&key1, cost1);
|
||||
assert_eq!(1, testee.get_count());
|
||||
assert_eq!(&cost1, testee.get_cost(&key1).unwrap());
|
||||
|
||||
// insert 2nd record
|
||||
testee.upsert(&key2, cost2);
|
||||
assert_eq!(2, testee.get_count());
|
||||
assert_eq!(&cost1, testee.get_cost(&key1).unwrap());
|
||||
assert_eq!(&cost2, testee.get_cost(&key2).unwrap());
|
||||
|
||||
// insert 3rd record, pushes out the oldest (eg 1st) record
|
||||
testee.upsert(&key3, cost3);
|
||||
assert_eq!(2, testee.get_count());
|
||||
assert_eq!((cost2 + cost3) / 2_u64, testee.get_average());
|
||||
assert_eq!(cost3, testee.get_mode());
|
||||
assert!(testee.get_cost(&key1).is_none());
|
||||
assert_eq!(&cost2, testee.get_cost(&key2).unwrap());
|
||||
assert_eq!(&cost3, testee.get_cost(&key3).unwrap());
|
||||
|
||||
// update 2nd record, so the 3rd becomes the oldest
|
||||
// add 4th record, pushes out 3rd key
|
||||
testee.upsert(&key2, cost1);
|
||||
testee.upsert(&key4, cost4);
|
||||
assert_eq!(((cost1 + cost2) / 2 + cost4) / 2_u64, testee.get_average());
|
||||
assert_eq!((cost1 + cost2) / 2, testee.get_mode());
|
||||
assert_eq!(2, testee.get_count());
|
||||
assert!(testee.get_cost(&key1).is_none());
|
||||
assert_eq!(&((cost1 + cost2) / 2), testee.get_cost(&key2).unwrap());
|
||||
assert!(testee.get_cost(&key3).is_none());
|
||||
assert_eq!(&cost4, testee.get_cost(&key4).unwrap());
|
||||
}
|
||||
}
|
@ -20,12 +20,8 @@ pub mod cluster_slots_service;
|
||||
pub mod commitment_service;
|
||||
pub mod completed_data_sets_service;
|
||||
pub mod consensus;
|
||||
pub mod cost_model;
|
||||
pub mod cost_tracker;
|
||||
pub mod cost_tracker_stats;
|
||||
pub mod cost_update_service;
|
||||
pub mod duplicate_repair_status;
|
||||
pub mod execute_cost_table;
|
||||
pub mod fetch_stage;
|
||||
pub mod fork_choice;
|
||||
pub mod gen_keys;
|
||||
|
@ -8,8 +8,6 @@ use crate::{
|
||||
ClusterInfoVoteListener, GossipDuplicateConfirmedSlotsSender, GossipVerifiedVoteHashSender,
|
||||
VerifiedVoteSender, VoteTracker,
|
||||
},
|
||||
cost_model::CostModel,
|
||||
cost_tracker::CostTracker,
|
||||
fetch_stage::FetchStage,
|
||||
sigverify::TransactionSigVerifier,
|
||||
sigverify_stage::SigVerifyStage,
|
||||
@ -24,6 +22,8 @@ use solana_rpc::{
|
||||
};
|
||||
use solana_runtime::{
|
||||
bank_forks::BankForks,
|
||||
cost_model::CostModel,
|
||||
cost_tracker::CostTracker,
|
||||
vote_sender_types::{ReplayVoteReceiver, ReplayVoteSender},
|
||||
};
|
||||
use std::{
|
||||
|
@ -12,7 +12,6 @@ use crate::{
|
||||
cluster_slots::ClusterSlots,
|
||||
completed_data_sets_service::CompletedDataSetsSender,
|
||||
consensus::Tower,
|
||||
cost_model::CostModel,
|
||||
cost_update_service::CostUpdateService,
|
||||
ledger_cleanup_service::LedgerCleanupService,
|
||||
replay_stage::{ReplayStage, ReplayStageConfig},
|
||||
@ -43,6 +42,7 @@ use solana_runtime::{
|
||||
bank::ExecuteTimings,
|
||||
bank_forks::BankForks,
|
||||
commitment::BlockCommitmentCache,
|
||||
cost_model::CostModel,
|
||||
snapshot_config::SnapshotConfig,
|
||||
snapshot_package::{AccountsPackageReceiver, AccountsPackageSender, PendingSnapshotPackage},
|
||||
vote_sender_types::ReplayVoteSender,
|
||||
|
@ -8,7 +8,6 @@ use {
|
||||
cluster_info_vote_listener::VoteTracker,
|
||||
completed_data_sets_service::CompletedDataSetsService,
|
||||
consensus::{reconcile_blockstore_roots_with_tower, Tower},
|
||||
cost_model::CostModel,
|
||||
rewards_recorder_service::{RewardsRecorderSender, RewardsRecorderService},
|
||||
sample_performance_service::SamplePerformanceService,
|
||||
serve_repair::ServeRepair,
|
||||
@ -69,6 +68,7 @@ use {
|
||||
bank::Bank,
|
||||
bank_forks::BankForks,
|
||||
commitment::BlockCommitmentCache,
|
||||
cost_model::CostModel,
|
||||
hardened_unpack::{open_genesis_config, MAX_GENESIS_ARCHIVE_UNPACKED_SIZE},
|
||||
snapshot_archive_info::SnapshotArchiveInfoGetter,
|
||||
snapshot_config::SnapshotConfig,
|
||||
|
Reference in New Issue
Block a user