Cost model 1.7 (#20188)
* Cost Model to limit transactions which are not parallelizeable (#16694) * * Add following to banking_stage: 1. CostModel as immutable ref shared between threads, to provide estimated cost for transactions. 2. CostTracker which is shared between threads, tracks transaction costs for each block. * replace hard coded program ID with id() calls * Add Account Access Cost as part of TransactionCost. Account Access cost are weighted differently between read and write, signed and non-signed. * Establish instruction_execution_cost_table, add function to update or insert instruction cost, unit tested. It is read-only for now; it allows Replay to insert realtime instruction execution costs to the table. * add test for cost_tracker atomically try_add operation, serves as safety guard for future changes * check cost against local copy of cost_tracker, return transactions that would exceed limit as unprocessed transaction to be buffered; only apply bank processed transactions cost to tracker; * bencher to new banking_stage with max cost limit to allow cost model being hit consistently during bench iterations * replay stage feed back program cost (#17731) * replay stage feeds back realtime per-program execution cost to cost model; * program cost execution table is initialized into empty table, no longer populated with hardcoded numbers; * changed cost unit to microsecond, using value collected from mainnet; * add ExecuteCostTable with fixed capacity for security concern, when its limit is reached, programs with old age AND less occurrence will be pushed out to make room for new programs. * investigate system performance test degradation (#17919) * Add stats and counter around cost model ops, mainly: - calculate transaction cost - check transaction can fit in a block - update block cost tracker after transactions are added to block - replay_stage to update/insert execution cost to table * Change mutex on cost_tracker to RwLock * removed cloning cost_tracker for local use, as the metrics show clone is very expensive. * acquire and hold locks for block of TXs, instead of acquire and release per transaction; * remove redundant would_fit check from cost_tracker update execution path * refactor cost checking with less frequent lock acquiring * avoid many Transaction_cost heap allocation when calculate cost, which is in the hot path - executed per transaction. * create hashmap with new_capacity to reduce runtime heap realloc. * code review changes: categorize stats, replace explicit drop calls, concisely initiate to default * address potential deadlock by acquiring locks one at time * Persist cost table to blockstore (#18123) * Add `ProgramCosts` Column Family to blockstore, implement LedgerColumn; add `delete_cf` to Rocks * Add ProgramCosts to compaction excluding list alone side with TransactionStatusIndex in one place: `excludes_from_compaction()` * Write cost table to blockstore after `replay_stage` replayed active banks; add stats to measure persist time * Deletes program from `ProgramCosts` in blockstore when they are removed from cost_table in memory * Only try to persist to blockstore when cost_table is changed. * Restore cost table during validator startup * Offload `cost_model` related operations from replay main thread to dedicated service thread, add channel to send execute_timings between these threads; * Move `cost_update_service` to its own module; replay_stage is now decoupled from cost_model. * log warning when channel send fails (#18391) * Aggregate cost_model into cost_tracker (#18374) * * aggregate cost_model into cost_tracker, decouple it from banking_stage to prevent accidental deadlock. * Simplified code, removed unused functions * review fixes * update ledger tool to restore cost table from blockstore (#18489) * update ledger tool to restore cost model from blockstore when compute-slot-cost * Move initialize_cost_table into cost_model, so the function can be tested and shared between validator and ledger-tool * refactor and simplify a test * manually fix merge conflicts * Per-program id timings (#17554) * more manual fixing * solve a merge conflict * featurize cost model * more merge fix * cost model uses compute_unit to replace microsecond as cost unit (#18934) * Reject blocks for costs above the max block cost (#18994) * Update block max cost limit to fix performance regession (#19276) * replace function with const var for better readability (#19285) * Add few more metrics data points (#19624) * periodically report sigverify_stage stats (#19674) * manual merge * cost model nits (#18528) * Accumulate consumed units (#18714) * tx wide compute budget (#18631) * more manual merge * ignore zerorize drop security * - update const cost values with data collected by #19627 - update cost calculation to closely proposed fee schedule #16984 * add transaction cost histogram metrics (#20350) * rebase to 1.7.15 * add tx count and thread id to stats (#20451) each stat reports and resets when slot changes * remove cost_model feature_set * ignore vote transactions from cost model Co-authored-by: sakridge <sakridge@gmail.com> Co-authored-by: Jeff Biseda <jbiseda@gmail.com> Co-authored-by: Jack May <jack@solana.com>
This commit is contained in:
56
ledger/src/block_cost_limits.rs
Normal file
56
ledger/src/block_cost_limits.rs
Normal file
@@ -0,0 +1,56 @@
|
||||
//! 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;
|
@@ -2701,6 +2701,26 @@ 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)
|
||||
@@ -8871,4 +8891,126 @@ 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");
|
||||
}
|
||||
}
|
||||
|
@@ -537,6 +537,11 @@ 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,
|
||||
@@ -1217,6 +1222,10 @@ 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>
|
||||
@@ -1364,7 +1373,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 must be excluded from LedgerCleanupService's rocksdb
|
||||
// TransactionStatusIndex and ProgramCosts 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> {
|
||||
|
@@ -1,4 +1,5 @@
|
||||
use crate::{
|
||||
block_cost_limits::*,
|
||||
block_error::BlockError,
|
||||
blockstore::Blockstore,
|
||||
blockstore_db::BlockstoreError,
|
||||
@@ -32,6 +33,7 @@ use solana_runtime::{
|
||||
};
|
||||
use solana_sdk::{
|
||||
clock::{Slot, MAX_PROCESSING_AGE},
|
||||
feature_set,
|
||||
genesis_config::GenesisConfig,
|
||||
hash::Hash,
|
||||
pubkey::Pubkey,
|
||||
@@ -48,11 +50,40 @@ use std::{
|
||||
collections::{HashMap, HashSet},
|
||||
path::PathBuf,
|
||||
result,
|
||||
sync::Arc,
|
||||
sync::{Arc, RwLock},
|
||||
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>;
|
||||
|
||||
@@ -100,12 +131,26 @@ 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();
|
||||
|
||||
@@ -117,6 +162,8 @@ 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,
|
||||
@@ -127,6 +174,29 @@ 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 {
|
||||
@@ -170,6 +240,7 @@ 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>) =
|
||||
@@ -185,6 +256,7 @@ 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);
|
||||
@@ -226,6 +298,7 @@ pub fn process_entries(
|
||||
transaction_status_sender,
|
||||
replay_vote_sender,
|
||||
&mut timings,
|
||||
Arc::new(RwLock::new(BlockCostCapacityMeter::default())),
|
||||
);
|
||||
|
||||
debug!("process_entries: {:?}", timings);
|
||||
@@ -241,6 +314,7 @@ 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![];
|
||||
@@ -262,6 +336,7 @@ fn process_entries_with_callback(
|
||||
transaction_status_sender,
|
||||
replay_vote_sender,
|
||||
timings,
|
||||
cost_capacity_meter.clone(),
|
||||
)?;
|
||||
batches.clear();
|
||||
for hash in &tick_hashes {
|
||||
@@ -313,6 +388,7 @@ fn process_entries_with_callback(
|
||||
transaction_status_sender,
|
||||
replay_vote_sender,
|
||||
timings,
|
||||
cost_capacity_meter.clone(),
|
||||
)?;
|
||||
batches.clear();
|
||||
}
|
||||
@@ -327,6 +403,7 @@ fn process_entries_with_callback(
|
||||
transaction_status_sender,
|
||||
replay_vote_sender,
|
||||
timings,
|
||||
cost_capacity_meter,
|
||||
)?;
|
||||
for hash in tick_hashes {
|
||||
bank.register_tick(hash);
|
||||
@@ -806,6 +883,7 @@ 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,
|
||||
@@ -815,6 +893,7 @@ pub fn confirm_slot(
|
||||
transaction_status_sender,
|
||||
replay_vote_sender,
|
||||
&mut execute_timings,
|
||||
cost_capacity_meter,
|
||||
)
|
||||
.map_err(BlockstoreProcessorError::from);
|
||||
replay_elapsed.stop();
|
||||
|
@@ -11,6 +11,7 @@ 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;
|
||||
|
Reference in New Issue
Block a user