Revert "Cost model 1.7 (#20188)"

This reverts commit 1dd6dc3709.
This commit is contained in:
Trent Nelson
2021-10-06 16:25:24 -06:00
parent 1dd6dc3709
commit e4aecd9320
40 changed files with 266 additions and 3208 deletions

View File

@@ -1,56 +0,0 @@
//! defines block cost related limits
//!
use lazy_static::lazy_static;
use solana_sdk::{
feature, incinerator, native_loader, pubkey::Pubkey, secp256k1_program, system_program,
};
use std::collections::HashMap;
/// Static configurations:
///
/// Number of microseconds replaying a block should take, 400 millisecond block times
/// is curerntly publicly communicated on solana.com
pub const MAX_BLOCK_REPLAY_TIME_US: u64 = 400_000;
/// number of concurrent processes,
pub const MAX_CONCURRENCY: u64 = 10;
/// Cluster data, method of collecting at https://github.com/solana-labs/solana/issues/19627
///
/// cluster avergaed compute unit to microsec conversion rate
pub const COMPUTE_UNIT_TO_US_RATIO: u64 = 40;
/// Number of compute units for one signature verification.
pub const SIGNATURE_COST: u64 = COMPUTE_UNIT_TO_US_RATIO * 175;
/// Number of compute units for one write lock
pub const WRITE_LOCK_UNITS: u64 = COMPUTE_UNIT_TO_US_RATIO * 20;
/// Number of data bytes per compute units
pub const DATA_BYTES_UNITS: u64 = 220 /*bytes per us*/ / COMPUTE_UNIT_TO_US_RATIO;
// Number of compute units for each built-in programs
lazy_static! {
/// Number of compute units for each built-in programs
pub static ref BUILT_IN_INSTRUCTION_COSTS: HashMap<Pubkey, u64> = [
(feature::id(), COMPUTE_UNIT_TO_US_RATIO * 2),
(incinerator::id(), COMPUTE_UNIT_TO_US_RATIO * 2),
(native_loader::id(), COMPUTE_UNIT_TO_US_RATIO * 2),
(solana_sdk::stake::config::id(), COMPUTE_UNIT_TO_US_RATIO * 2),
(solana_sdk::stake::program::id(), COMPUTE_UNIT_TO_US_RATIO * 50),
(solana_vote_program::id(), COMPUTE_UNIT_TO_US_RATIO * 200),
(secp256k1_program::id(), COMPUTE_UNIT_TO_US_RATIO * 4),
(system_program::id(), COMPUTE_UNIT_TO_US_RATIO * 15),
]
.iter()
.cloned()
.collect();
}
/// Statically computed data:
///
/// Number of compute units that a block is allowed. A block's compute units are
/// accumualted by Transactions added to it; A transaction's compute units are
/// calculated by cost_model, based on transaction's signarures, write locks,
/// data size and built-in and BPF instructinos.
pub const MAX_BLOCK_UNITS: u64 =
MAX_BLOCK_REPLAY_TIME_US * COMPUTE_UNIT_TO_US_RATIO * MAX_CONCURRENCY;
/// Number of compute units that a writable account in a block is allowed. The
/// limit is to prevent too many transactions write to same account, threrefore
/// reduce block's paralellism.
pub const MAX_WRITABLE_ACCOUNT_UNITS: u64 = MAX_BLOCK_REPLAY_TIME_US * COMPUTE_UNIT_TO_US_RATIO;

View File

@@ -2701,26 +2701,6 @@ impl Blockstore {
self.perf_samples_cf.put(index, perf_sample)
}
pub fn read_program_costs(&self) -> Result<Vec<(Pubkey, u64)>> {
Ok(self
.db
.iter::<cf::ProgramCosts>(IteratorMode::End)?
.map(|(pubkey, data)| {
let program_cost: ProgramCost = deserialize(&data).unwrap();
(pubkey, program_cost.cost)
})
.collect())
}
pub fn write_program_cost(&self, key: &Pubkey, value: &u64) -> Result<()> {
self.program_costs_cf
.put(*key, &ProgramCost { cost: *value })
}
pub fn delete_program_cost(&self, key: &Pubkey) -> Result<()> {
self.program_costs_cf.delete(*key)
}
/// Returns the entry vector for the slot starting with `shred_start_index`
pub fn get_slot_entries(&self, slot: Slot, shred_start_index: u64) -> Result<Vec<Entry>> {
self.get_slot_entries_with_shred_info(slot, shred_start_index, false)
@@ -8891,126 +8871,4 @@ pub mod tests {
Blockstore::destroy(&blockstore_path).expect("Expected successful database destruction");
}
#[test]
fn test_read_write_cost_table() {
let blockstore_path = get_tmp_ledger_path!();
{
let blockstore = Blockstore::open(&blockstore_path).unwrap();
let num_entries: usize = 10;
let mut cost_table: HashMap<Pubkey, u64> = HashMap::new();
for x in 1..num_entries + 1 {
cost_table.insert(Pubkey::new_unique(), (x + 100) as u64);
}
// write to db
for (key, cost) in cost_table.iter() {
blockstore
.write_program_cost(key, cost)
.expect("write a program");
}
// read back from db
let read_back = blockstore.read_program_costs().expect("read programs");
// verify
assert_eq!(read_back.len(), cost_table.len());
for (read_key, read_cost) in read_back {
assert_eq!(read_cost, *cost_table.get(&read_key).unwrap());
}
// update value, write to db
for val in cost_table.values_mut() {
*val += 100;
}
for (key, cost) in cost_table.iter() {
blockstore
.write_program_cost(key, cost)
.expect("write a program");
}
// add a new record
let new_program_key = Pubkey::new_unique();
let new_program_cost = 999;
blockstore
.write_program_cost(&new_program_key, &new_program_cost)
.unwrap();
// confirm value updated
let read_back = blockstore.read_program_costs().expect("read programs");
// verify
assert_eq!(read_back.len(), cost_table.len() + 1);
for (key, cost) in cost_table.iter() {
assert_eq!(*cost, read_back.iter().find(|(k, _v)| k == key).unwrap().1);
}
assert_eq!(
new_program_cost,
read_back
.iter()
.find(|(k, _v)| *k == new_program_key)
.unwrap()
.1
);
// test delete
blockstore
.delete_program_cost(&new_program_key)
.expect("delete a progrma");
let read_back = blockstore.read_program_costs().expect("read programs");
// verify
assert_eq!(read_back.len(), cost_table.len());
for (read_key, read_cost) in read_back {
assert_eq!(read_cost, *cost_table.get(&read_key).unwrap());
}
}
Blockstore::destroy(&blockstore_path).expect("Expected successful database destruction");
}
#[test]
fn test_delete_old_records_from_cost_table() {
let blockstore_path = get_tmp_ledger_path!();
{
let blockstore = Blockstore::open(&blockstore_path).unwrap();
let num_entries: usize = 10;
let mut cost_table: HashMap<Pubkey, u64> = HashMap::new();
for x in 1..num_entries + 1 {
cost_table.insert(Pubkey::new_unique(), (x + 100) as u64);
}
// write to db
for (key, cost) in cost_table.iter() {
blockstore
.write_program_cost(key, cost)
.expect("write a program");
}
// remove a record
let mut removed_key = Pubkey::new_unique();
for (key, cost) in cost_table.iter() {
if *cost == 101_u64 {
removed_key = *key;
break;
}
}
cost_table.remove(&removed_key);
// delete records from blockstore if they are no longer in cost_table
let db_records = blockstore.read_program_costs().expect("read programs");
db_records.iter().for_each(|(pubkey, _)| {
if !cost_table.iter().any(|(key, _)| key == pubkey) {
assert_eq!(*pubkey, removed_key);
blockstore
.delete_program_cost(pubkey)
.expect("delete old program");
}
});
// read back from db
let read_back = blockstore.read_program_costs().expect("read programs");
// verify
assert_eq!(read_back.len(), cost_table.len());
for (read_key, read_cost) in read_back {
assert_eq!(read_cost, *cost_table.get(&read_key).unwrap());
}
}
Blockstore::destroy(&blockstore_path).expect("Expected successful database destruction");
}
}

View File

@@ -537,11 +537,6 @@ impl Rocks {
Ok(())
}
fn delete_cf(&self, cf: &ColumnFamily, key: &[u8]) -> Result<()> {
self.0.delete_cf(cf, key)?;
Ok(())
}
fn iterator_cf<C>(&self, cf: &ColumnFamily, iterator_mode: IteratorMode<C::Index>) -> DBIterator
where
C: Column,
@@ -1222,10 +1217,6 @@ where
self.backend
.put_cf(self.handle(), &C::key(key), &serialized_value)
}
pub fn delete(&self, key: C::Index) -> Result<()> {
self.backend.delete_cf(self.handle(), &C::key(key))
}
}
impl<C> LedgerColumn<C>
@@ -1373,7 +1364,7 @@ fn get_cf_options<C: 'static + Column + ColumnName>(
options.set_max_bytes_for_level_base(total_size_base);
options.set_target_file_size_base(file_size_base);
// TransactionStatusIndex and ProgramCosts must be excluded from LedgerCleanupService's rocksdb
// TransactionStatusIndex must be excluded from LedgerCleanupService's rocksdb
// compactions....
if matches!(access_type, AccessType::PrimaryOnly) && !excludes_from_compaction(C::NAME) {
options.set_compaction_filter_factory(PurgedSlotFilterFactory::<C> {

View File

@@ -1,5 +1,4 @@
use crate::{
block_cost_limits::*,
block_error::BlockError,
blockstore::Blockstore,
blockstore_db::BlockstoreError,
@@ -33,7 +32,6 @@ use solana_runtime::{
};
use solana_sdk::{
clock::{Slot, MAX_PROCESSING_AGE},
feature_set,
genesis_config::GenesisConfig,
hash::Hash,
pubkey::Pubkey,
@@ -50,40 +48,11 @@ use std::{
collections::{HashMap, HashSet},
path::PathBuf,
result,
sync::{Arc, RwLock},
sync::Arc,
time::{Duration, Instant},
};
use thiserror::Error;
// it tracks the block cost available capacity - number of compute-units allowed
// by max blockl cost limit
#[derive(Debug)]
pub struct BlockCostCapacityMeter {
pub capacity: u64,
pub accumulated_cost: u64,
}
impl Default for BlockCostCapacityMeter {
fn default() -> Self {
BlockCostCapacityMeter::new(MAX_BLOCK_UNITS)
}
}
impl BlockCostCapacityMeter {
pub fn new(capacity_limit: u64) -> Self {
Self {
capacity: capacity_limit,
accumulated_cost: 0_u64,
}
}
// return the remaining capacity
pub fn accumulate(&mut self, cost: u64) -> u64 {
self.accumulated_cost += cost;
self.capacity.saturating_sub(self.accumulated_cost)
}
}
pub type BlockstoreProcessorResult =
result::Result<(BankForks, LeaderScheduleCache), BlockstoreProcessorError>;
@@ -131,26 +100,12 @@ fn get_first_error(
first_err
}
fn aggregate_total_execution_units(execute_timings: &ExecuteTimings) -> u64 {
let mut execute_cost_units: u64 = 0;
for (program_id, timing) in &execute_timings.details.per_program_timings {
if timing.count < 1 {
continue;
}
execute_cost_units =
execute_cost_units.saturating_add(timing.accumulated_units / timing.count as u64);
trace!("aggregated execution cost of {:?} {:?}", program_id, timing);
}
execute_cost_units
}
fn execute_batch(
batch: &TransactionBatch,
bank: &Arc<Bank>,
transaction_status_sender: Option<&TransactionStatusSender>,
replay_vote_sender: Option<&ReplayVoteSender>,
timings: &mut ExecuteTimings,
cost_capacity_meter: Arc<RwLock<BlockCostCapacityMeter>>,
) -> Result<()> {
let record_token_balances = transaction_status_sender.is_some();
@@ -162,8 +117,6 @@ fn execute_batch(
vec![]
};
let pre_process_units: u64 = aggregate_total_execution_units(timings);
let (tx_results, balances, inner_instructions, transaction_logs) =
batch.bank().load_execute_and_commit_transactions(
batch,
@@ -174,29 +127,6 @@ fn execute_batch(
timings,
);
if bank
.feature_set
.is_active(&feature_set::gate_large_block::id())
{
let execution_cost_units = aggregate_total_execution_units(timings) - pre_process_units;
let remaining_block_cost_cap = cost_capacity_meter
.write()
.unwrap()
.accumulate(execution_cost_units);
debug!(
"bank {} executed a batch, number of transactions {}, total execute cu {}, remaining block cost cap {}",
bank.slot(),
batch.hashed_transactions().len(),
execution_cost_units,
remaining_block_cost_cap,
);
if remaining_block_cost_cap == 0_u64 {
return Err(TransactionError::WouldExceedMaxBlockCostLimit);
}
}
bank_utils::find_and_send_votes(batch.hashed_transactions(), &tx_results, replay_vote_sender);
let TransactionResults {
@@ -240,7 +170,6 @@ fn execute_batches(
transaction_status_sender: Option<&TransactionStatusSender>,
replay_vote_sender: Option<&ReplayVoteSender>,
timings: &mut ExecuteTimings,
cost_capacity_meter: Arc<RwLock<BlockCostCapacityMeter>>,
) -> Result<()> {
inc_new_counter_debug!("bank-par_execute_entries-count", batches.len());
let (results, new_timings): (Vec<Result<()>>, Vec<ExecuteTimings>) =
@@ -256,7 +185,6 @@ fn execute_batches(
transaction_status_sender,
replay_vote_sender,
&mut timings,
cost_capacity_meter.clone(),
);
if let Some(entry_callback) = entry_callback {
entry_callback(bank);
@@ -298,7 +226,6 @@ pub fn process_entries(
transaction_status_sender,
replay_vote_sender,
&mut timings,
Arc::new(RwLock::new(BlockCostCapacityMeter::default())),
);
debug!("process_entries: {:?}", timings);
@@ -314,7 +241,6 @@ fn process_entries_with_callback(
transaction_status_sender: Option<&TransactionStatusSender>,
replay_vote_sender: Option<&ReplayVoteSender>,
timings: &mut ExecuteTimings,
cost_capacity_meter: Arc<RwLock<BlockCostCapacityMeter>>,
) -> Result<()> {
// accumulator for entries that can be processed in parallel
let mut batches = vec![];
@@ -336,7 +262,6 @@ fn process_entries_with_callback(
transaction_status_sender,
replay_vote_sender,
timings,
cost_capacity_meter.clone(),
)?;
batches.clear();
for hash in &tick_hashes {
@@ -388,7 +313,6 @@ fn process_entries_with_callback(
transaction_status_sender,
replay_vote_sender,
timings,
cost_capacity_meter.clone(),
)?;
batches.clear();
}
@@ -403,7 +327,6 @@ fn process_entries_with_callback(
transaction_status_sender,
replay_vote_sender,
timings,
cost_capacity_meter,
)?;
for hash in tick_hashes {
bank.register_tick(hash);
@@ -883,7 +806,6 @@ pub fn confirm_slot(
let mut entries = check_result.unwrap();
let mut replay_elapsed = Measure::start("replay_elapsed");
let mut execute_timings = ExecuteTimings::default();
let cost_capacity_meter = Arc::new(RwLock::new(BlockCostCapacityMeter::default()));
// Note: This will shuffle entries' transactions in-place.
let process_result = process_entries_with_callback(
bank,
@@ -893,7 +815,6 @@ pub fn confirm_slot(
transaction_status_sender,
replay_vote_sender,
&mut execute_timings,
cost_capacity_meter,
)
.map_err(BlockstoreProcessorError::from);
replay_elapsed.stop();

View File

@@ -11,7 +11,6 @@ pub mod block_error;
#[macro_use]
pub mod blockstore;
pub mod ancestor_iterator;
pub mod block_cost_limits;
pub mod blockstore_db;
pub mod blockstore_meta;
pub mod blockstore_processor;