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.
This commit is contained in:
Tao Zhu
2021-07-01 11:32:41 -05:00
committed by GitHub
parent 05924423c2
commit 5e424826ba
9 changed files with 575 additions and 149 deletions

View File

@ -12,11 +12,14 @@ use log::*;
use solana_sdk::{message::Message, pubkey::Pubkey, transaction::Transaction};
use std::collections::HashMap;
// Guestimated from mainnet-beta data, sigver averages 1us, read averages 7us and write avergae 25us
const SIGNED_WRITABLE_ACCOUNT_ACCESS_COST: u64 = 1 + 25;
const SIGNED_READONLY_ACCOUNT_ACCESS_COST: u64 = 1 + 7;
const NON_SIGNED_WRITABLE_ACCOUNT_ACCESS_COST: u64 = 25;
// Guestimated from mainnet-beta data, sigver averages 1us, average read 7us and average write 25us
const SIGVER_COST: u64 = 1;
const NON_SIGNED_READONLY_ACCOUNT_ACCESS_COST: u64 = 7;
const NON_SIGNED_WRITABLE_ACCOUNT_ACCESS_COST: u64 = 25;
const SIGNED_READONLY_ACCOUNT_ACCESS_COST: u64 =
SIGVER_COST + NON_SIGNED_READONLY_ACCOUNT_ACCESS_COST;
const SIGNED_WRITABLE_ACCOUNT_ACCESS_COST: u64 =
SIGVER_COST + NON_SIGNED_WRITABLE_ACCOUNT_ACCESS_COST;
// Sampled from mainnet-beta, the instruction execution timings stats are (in us):
// min=194, max=62164, avg=8214.49, med=2243

View File

@ -0,0 +1,275 @@
//! this service receives instruction ExecuteTimings from replay_stage,
//! update cost_model which is shared with banking_stage to optimize
//! 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_sdk::timing::timestamp;
use std::{
sync::{
atomic::{AtomicBool, Ordering},
mpsc::Receiver,
Arc, RwLock,
},
thread::{self, Builder, JoinHandle},
time::Duration,
};
#[derive(Default)]
pub struct CostUpdateServiceTiming {
last_print: u64,
update_cost_model_count: u64,
update_cost_model_elapsed: u64,
persist_cost_table_elapsed: u64,
}
impl CostUpdateServiceTiming {
fn update(
&mut self,
update_cost_model_count: u64,
update_cost_model_elapsed: u64,
persist_cost_table_elapsed: u64,
) {
self.update_cost_model_count += update_cost_model_count;
self.update_cost_model_elapsed += update_cost_model_elapsed;
self.persist_cost_table_elapsed += persist_cost_table_elapsed;
let now = timestamp();
let elapsed_ms = now - self.last_print;
if elapsed_ms > 1000 {
datapoint_info!(
"replay-service-timing-stats",
("total_elapsed_us", elapsed_ms * 1000, i64),
(
"update_cost_model_count",
self.update_cost_model_count as i64,
i64
),
(
"update_cost_model_elapsed",
self.update_cost_model_elapsed as i64,
i64
),
(
"persist_cost_table_elapsed",
self.persist_cost_table_elapsed as i64,
i64
),
);
*self = CostUpdateServiceTiming::default();
self.last_print = now;
}
}
}
pub type CostUpdateReceiver = Receiver<ExecuteTimings>;
pub struct CostUpdateService {
thread_hdl: JoinHandle<()>,
}
impl CostUpdateService {
#[allow(clippy::new_ret_no_self)]
pub fn new(
exit: Arc<AtomicBool>,
blockstore: Arc<Blockstore>,
cost_model: Arc<RwLock<CostModel>>,
cost_update_receiver: CostUpdateReceiver,
) -> Self {
let thread_hdl = Builder::new()
.name("solana-cost-update-service".to_string())
.spawn(move || {
Self::service_loop(exit, blockstore, cost_model, cost_update_receiver);
})
.unwrap();
Self { thread_hdl }
}
pub fn join(self) -> thread::Result<()> {
self.thread_hdl.join()
}
fn service_loop(
exit: Arc<AtomicBool>,
blockstore: Arc<Blockstore>,
cost_model: Arc<RwLock<CostModel>>,
cost_update_receiver: CostUpdateReceiver,
) {
let mut cost_update_service_timing = CostUpdateServiceTiming::default();
let mut dirty = false;
let wait_timer = Duration::from_millis(100);
loop {
if exit.load(Ordering::Relaxed) {
break;
}
let mut update_count = 0_u64;
let mut update_cost_model_time = Measure::start("update_cost_model_time");
for cost_update in cost_update_receiver.try_iter() {
dirty |= Self::update_cost_model(&cost_model, &cost_update);
update_count += 1;
}
update_cost_model_time.stop();
let mut persist_cost_table_time = Measure::start("persist_cost_table_time");
if dirty {
Self::persist_cost_table(&blockstore, &cost_model);
}
persist_cost_table_time.stop();
cost_update_service_timing.update(
update_count,
update_cost_model_time.as_us(),
persist_cost_table_time.as_us(),
);
thread::sleep(wait_timer);
}
}
fn update_cost_model(cost_model: &RwLock<CostModel>, execute_timings: &ExecuteTimings) -> bool {
let mut dirty = false;
let mut cost_model_mutable = cost_model.write().unwrap();
for (program_id, stats) in &execute_timings.details.per_program_timings {
let cost = stats.0 / stats.1 as u64;
match cost_model_mutable.upsert_instruction_cost(program_id, &cost) {
Ok(c) => {
debug!(
"after replayed into bank, instruction {:?} has averaged cost {}",
program_id, c
);
dirty = true;
}
Err(err) => {
debug!(
"after replayed into bank, instruction {:?} failed to update cost, err: {}",
program_id, err
);
}
}
}
drop(cost_model_mutable);
debug!(
"after replayed into bank, updated cost model instruction cost table, current values: {:?}",
cost_model.read().unwrap().get_instruction_cost_table()
);
dirty
}
fn persist_cost_table(blockstore: &Blockstore, cost_model: &RwLock<CostModel>) {
let cost_model_read = cost_model.read().unwrap();
let cost_table = cost_model_read.get_instruction_cost_table();
let db_records = blockstore.read_program_costs().expect("read programs");
// delete records from blockstore if they are no longer in cost_table
db_records.iter().for_each(|(pubkey, _)| {
if cost_table.get(pubkey).is_none() {
blockstore
.delete_program_cost(pubkey)
.expect("delete old program");
}
});
for (key, cost) in cost_table.iter() {
blockstore
.write_program_cost(key, cost)
.expect("persist program costs to blockstore");
}
}
}
#[cfg(test)]
mod tests {
use super::*;
use solana_sdk::pubkey::Pubkey;
#[test]
fn test_update_cost_model_with_empty_execute_timings() {
let cost_model = Arc::new(RwLock::new(CostModel::default()));
let empty_execute_timings = ExecuteTimings::default();
CostUpdateService::update_cost_model(&cost_model, &empty_execute_timings);
assert_eq!(
0,
cost_model
.read()
.unwrap()
.get_instruction_cost_table()
.len()
);
}
#[test]
fn test_update_cost_model_with_execute_timings() {
let cost_model = Arc::new(RwLock::new(CostModel::default()));
let mut execute_timings = ExecuteTimings::default();
let program_key_1 = Pubkey::new_unique();
let mut expected_cost: u64;
// add new program
{
let accumulated_us: u64 = 1000;
let count: u32 = 10;
expected_cost = accumulated_us / count as u64;
execute_timings
.details
.per_program_timings
.insert(program_key_1, (accumulated_us, count));
CostUpdateService::update_cost_model(&cost_model, &execute_timings);
assert_eq!(
1,
cost_model
.read()
.unwrap()
.get_instruction_cost_table()
.len()
);
assert_eq!(
Some(&expected_cost),
cost_model
.read()
.unwrap()
.get_instruction_cost_table()
.get(&program_key_1)
);
}
// update program
{
let accumulated_us: u64 = 2000;
let count: u32 = 10;
// to expect new cost is Average(new_value, existing_value)
expected_cost = ((accumulated_us / count as u64) + expected_cost) / 2;
execute_timings
.details
.per_program_timings
.insert(program_key_1, (accumulated_us, count));
CostUpdateService::update_cost_model(&cost_model, &execute_timings);
assert_eq!(
1,
cost_model
.read()
.unwrap()
.get_instruction_cost_table()
.len()
);
assert_eq!(
Some(&expected_cost),
cost_model
.read()
.unwrap()
.get_instruction_cost_table()
.get(&program_key_1)
);
}
}
}

View File

@ -20,6 +20,7 @@ pub mod completed_data_sets_service;
pub mod consensus;
pub mod cost_model;
pub mod cost_tracker;
pub mod cost_update_service;
pub mod execute_cost_table;
pub mod fetch_stage;
pub mod fork_choice;

View File

@ -13,13 +13,11 @@ use crate::{
consensus::{
ComputedBankState, Stake, SwitchForkDecision, Tower, VotedStakes, SWITCH_FORK_THRESHOLD,
},
cost_model::CostModel,
fork_choice::{ForkChoice, SelectVoteAndResetForkResult},
heaviest_subtree_fork_choice::HeaviestSubtreeForkChoice,
latest_validator_votes_for_frozen_banks::LatestValidatorVotesForFrozenBanks,
progress_map::{ForkProgress, ProgressMap, PropagatedStats},
repair_service::DuplicateSlotsResetReceiver,
result::Result,
rewards_recorder_service::RewardsRecorderSender,
unfrozen_gossip_verified_vote_hashes::UnfrozenGossipVerifiedVoteHashes,
window_service::DuplicateSlotReceiver,
@ -276,7 +274,7 @@ impl ReplayTiming {
"process_duplicate_slots_elapsed",
self.process_duplicate_slots_elapsed as i64,
i64
)
),
);
*self = ReplayTiming::default();
@ -286,7 +284,7 @@ impl ReplayTiming {
}
pub struct ReplayStage {
t_replay: JoinHandle<Result<()>>,
t_replay: JoinHandle<()>,
commitment_service: AggregateCommitmentService,
}
@ -309,7 +307,7 @@ impl ReplayStage {
gossip_duplicate_confirmed_slots_receiver: GossipDuplicateConfirmedSlotsReceiver,
gossip_verified_vote_hash_receiver: GossipVerifiedVoteHashReceiver,
cluster_slots_update_sender: ClusterSlotsUpdateSender,
cost_model: Arc<RwLock<CostModel>>,
cost_update_sender: Sender<ExecuteTimings>,
) -> Self {
let ReplayStageConfig {
vote_account,
@ -406,7 +404,7 @@ impl ReplayStage {
&mut unfrozen_gossip_verified_vote_hashes,
&mut latest_validator_votes_for_frozen_banks,
&cluster_slots_update_sender,
&cost_model,
&cost_update_sender,
);
replay_active_banks_time.stop();
@ -734,7 +732,6 @@ impl ReplayStage {
process_duplicate_slots_time.as_us(),
);
}
Ok(())
})
.unwrap();
@ -1668,10 +1665,11 @@ impl ReplayStage {
unfrozen_gossip_verified_vote_hashes: &mut UnfrozenGossipVerifiedVoteHashes,
latest_validator_votes_for_frozen_banks: &mut LatestValidatorVotesForFrozenBanks,
cluster_slots_update_sender: &ClusterSlotsUpdateSender,
cost_model: &RwLock<CostModel>,
cost_update_sender: &Sender<ExecuteTimings>,
) -> bool {
let mut did_complete_bank = false;
let mut tx_count = 0;
let mut execute_timings = ExecuteTimings::default();
let active_banks = bank_forks.read().unwrap().active_banks();
trace!("active banks {:?}", active_banks);
@ -1719,7 +1717,7 @@ impl ReplayStage {
replay_vote_sender,
verify_recyclers,
);
Self::update_cost_model(cost_model, &bank_progress.replay_stats.execute_timings);
execute_timings.accumulate(&bank_progress.replay_stats.execute_timings);
match replay_result {
Ok(replay_tx_count) => tx_count += replay_tx_count,
Err(err) => {
@ -1804,6 +1802,12 @@ impl ReplayStage {
);
}
}
// send accumulated excute-timings to cost_update_service
cost_update_sender
.send(execute_timings)
.expect("send execution cost update to cost_model");
inc_new_counter_info!("replay_stage-replay_transactions", tx_count);
did_complete_bank
}
@ -1910,44 +1914,6 @@ impl ReplayStage {
new_stats
}
fn update_cost_model(cost_model: &RwLock<CostModel>, execute_timings: &ExecuteTimings) {
let mut update_cost_model_time = Measure::start("update_cost_model_time");
let mut cost_model_mutable = cost_model.write().unwrap();
for (program_id, stats) in &execute_timings.details.per_program_timings {
let cost = stats.0 / stats.1 as u64;
match cost_model_mutable.upsert_instruction_cost(program_id, &cost) {
Ok(c) => {
debug!(
"after replayed into bank, instruction {:?} has averaged cost {}",
program_id, c
);
}
Err(err) => {
debug!(
"after replayed into bank, instruction {:?} failed to update cost, err: {}",
program_id, err
);
}
}
}
drop(cost_model_mutable);
debug!(
"after replayed into bank, updated cost model instruction cost table, current values: {:?}",
cost_model.read().unwrap().get_instruction_cost_table()
);
update_cost_model_time.stop();
inc_new_counter_info!("replay_stage-update_cost_model", 1);
datapoint_info!(
"replay-loop-timing-stats",
(
"update_cost_model_elapsed",
update_cost_model_time.as_us() as i64,
i64
)
);
}
fn update_propagation_status(
progress: &mut ProgressMap,
slot: Slot,
@ -4911,91 +4877,6 @@ mod tests {
);
assert_eq!(tower.last_voted_slot().unwrap(), 1);
}
#[test]
fn test_update_cost_model_with_empty_execute_timings() {
let cost_model = Arc::new(RwLock::new(CostModel::default()));
let empty_execute_timings = ExecuteTimings::default();
ReplayStage::update_cost_model(&cost_model, &empty_execute_timings);
assert_eq!(
0,
cost_model
.read()
.unwrap()
.get_instruction_cost_table()
.len()
);
}
#[test]
fn test_update_cost_model_with_execute_timings() {
let cost_model = Arc::new(RwLock::new(CostModel::default()));
let mut execute_timings = ExecuteTimings::default();
let program_key_1 = Pubkey::new_unique();
let mut expected_cost: u64;
// add new program
{
let accumulated_us: u64 = 1000;
let count: u32 = 10;
expected_cost = accumulated_us / count as u64;
execute_timings
.details
.per_program_timings
.insert(program_key_1, (accumulated_us, count));
ReplayStage::update_cost_model(&cost_model, &execute_timings);
assert_eq!(
1,
cost_model
.read()
.unwrap()
.get_instruction_cost_table()
.len()
);
assert_eq!(
Some(&expected_cost),
cost_model
.read()
.unwrap()
.get_instruction_cost_table()
.get(&program_key_1)
);
}
// update program
{
let accumulated_us: u64 = 2000;
let count: u32 = 10;
// to expect new cost is Average(new_value, existing_value)
expected_cost = ((accumulated_us / count as u64) + expected_cost) / 2;
execute_timings
.details
.per_program_timings
.insert(program_key_1, (accumulated_us, count));
ReplayStage::update_cost_model(&cost_model, &execute_timings);
assert_eq!(
1,
cost_model
.read()
.unwrap()
.get_instruction_cost_table()
.len()
);
assert_eq!(
Some(&expected_cost),
cost_model
.read()
.unwrap()
.get_instruction_cost_table()
.get(&program_key_1)
);
}
}
fn run_compute_and_select_forks(
bank_forks: &RwLock<BankForks>,
progress: &mut ProgressMap,

View File

@ -13,6 +13,7 @@ use crate::{
completed_data_sets_service::CompletedDataSetsSender,
consensus::Tower,
cost_model::CostModel,
cost_update_service::CostUpdateService,
ledger_cleanup_service::LedgerCleanupService,
replay_stage::{ReplayStage, ReplayStageConfig},
retransmit_stage::RetransmitStage,
@ -38,6 +39,7 @@ use solana_runtime::{
AbsRequestHandler, AbsRequestSender, AccountsBackgroundService, SnapshotRequestHandler,
},
accounts_db::AccountShrinkThreshold,
bank::ExecuteTimings,
bank_forks::BankForks,
commitment::BlockCommitmentCache,
snapshot_config::SnapshotConfig,
@ -50,7 +52,7 @@ use std::{
net::UdpSocket,
sync::{
atomic::AtomicBool,
mpsc::{channel, Receiver},
mpsc::{channel, Receiver, Sender},
Arc, Mutex, RwLock,
},
thread,
@ -64,6 +66,7 @@ pub struct Tvu {
ledger_cleanup_service: Option<LedgerCleanupService>,
accounts_background_service: AccountsBackgroundService,
accounts_hash_verifier: AccountsHashVerifier,
cost_update_service: CostUpdateService,
}
pub struct Sockets {
@ -270,6 +273,17 @@ impl Tvu {
wait_for_vote_to_start_leader: tvu_config.wait_for_vote_to_start_leader,
};
let (cost_update_sender, cost_update_receiver): (
Sender<ExecuteTimings>,
Receiver<ExecuteTimings>,
) = channel();
let cost_update_service = CostUpdateService::new(
exit.clone(),
blockstore.clone(),
cost_model.clone(),
cost_update_receiver,
);
let replay_stage = ReplayStage::new(
replay_stage_config,
blockstore.clone(),
@ -287,7 +301,7 @@ impl Tvu {
gossip_confirmed_slots_receiver,
gossip_verified_vote_hash_receiver,
cluster_slots_update_sender,
cost_model.clone(),
cost_update_sender,
);
let ledger_cleanup_service = tvu_config.max_ledger_shreds.map(|max_ledger_shreds| {
@ -318,6 +332,7 @@ impl Tvu {
ledger_cleanup_service,
accounts_background_service,
accounts_hash_verifier,
cost_update_service,
}
}
@ -331,6 +346,7 @@ impl Tvu {
self.accounts_background_service.join()?;
self.replay_stage.join()?;
self.accounts_hash_verifier.join()?;
self.cost_update_service.join()?;
Ok(())
}
}

View File

@ -662,6 +662,7 @@ impl Validator {
ACCOUNT_MAX_COST,
BLOCK_MAX_COST,
)));
Self::initiate_cost_model(&cost_model, &blockstore.read_program_costs().unwrap());
let (retransmit_slots_sender, retransmit_slots_receiver) = unbounded();
let (verified_vote_sender, verified_vote_receiver) = unbounded();
@ -892,6 +893,31 @@ impl Validator {
ip_echo_server.shutdown_background();
}
}
fn initiate_cost_model(cost_model: &RwLock<CostModel>, cost_table: &[(Pubkey, u64)]) {
let mut cost_model_mutable = cost_model.write().unwrap();
for (program_id, cost) in cost_table {
match cost_model_mutable.upsert_instruction_cost(program_id, cost) {
Ok(c) => {
debug!(
"initiating cost table, instruction {:?} has cost {}",
program_id, c
);
}
Err(err) => {
debug!(
"initiating cost table, failed for instruction {:?}, err: {}",
program_id, err
);
}
}
}
drop(cost_model_mutable);
debug!(
"restored cost model instruction cost table from blockstore, current values: {:?}",
cost_model.read().unwrap().get_instruction_cost_table()
);
}
}
fn active_vote_account_exists_in_bank(bank: &Arc<Bank>, vote_account: &Pubkey) -> bool {