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:
@ -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
|
||||
|
275
core/src/cost_update_service.rs
Normal file
275
core/src/cost_update_service.rs
Normal 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)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
@ -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;
|
||||
|
@ -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,
|
||||
|
@ -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(())
|
||||
}
|
||||
}
|
||||
|
@ -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 {
|
||||
|
Reference in New Issue
Block a user