Rename LoadedPrograms to ProgramCache for readability (#339)

This commit is contained in:
Dmitri Makarov 2024-03-20 15:26:45 -04:00 committed by GHA: Update Upstream From Fork
parent 2273098c55
commit 1d89ea01cc
5 changed files with 110 additions and 123 deletions

View File

@ -145,7 +145,7 @@ pub struct LoadedProgram {
pub latest_access_slot: AtomicU64,
}
/// Global cache statistics for [LoadedPrograms].
/// Global cache statistics for [ProgramCache].
#[derive(Debug, Default)]
pub struct Stats {
/// a program was already in the cache
@ -568,7 +568,7 @@ struct SecondLevel {
/// - allows for cooperative loading of TX batches which hit the same missing programs simultaneously.
/// - enforces that all programs used in a batch are eagerly loaded ahead of execution.
/// - is not persisted to disk or a snapshot, so it needs to cold start and warm up first.
pub struct LoadedPrograms<FG: ForkGraph> {
pub struct ProgramCache<FG: ForkGraph> {
/// A two level index:
///
/// The first level is for the address at which programs are deployed and the second level for the slot (and thus also fork).
@ -595,9 +595,9 @@ pub struct LoadedPrograms<FG: ForkGraph> {
pub loading_task_waiter: Arc<LoadingTaskWaiter>,
}
impl<FG: ForkGraph> Debug for LoadedPrograms<FG> {
impl<FG: ForkGraph> Debug for ProgramCache<FG> {
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
f.debug_struct("LoadedPrograms")
f.debug_struct("ProgramCache")
.field("root slot", &self.latest_root_slot)
.field("root epoch", &self.latest_root_epoch)
.field("stats", &self.stats)
@ -606,11 +606,11 @@ impl<FG: ForkGraph> Debug for LoadedPrograms<FG> {
}
}
/// Local view into [LoadedPrograms] which was extracted for a specific TX batch.
/// Local view into [ProgramCache] which was extracted for a specific TX batch.
///
/// This isolation enables the global [LoadedPrograms] to continue to evolve (e.g. evictions),
/// This isolation enables the global [ProgramCache] to continue to evolve (e.g. evictions),
/// while the TX batch is guaranteed it will continue to find all the programs it requires.
/// For program management instructions this also buffers them before they are merged back into the global [LoadedPrograms].
/// For program management instructions this also buffers them before they are merged back into the global [ProgramCache].
#[derive(Clone, Debug, Default)]
pub struct LoadedProgramsForTxBatch {
/// Pubkey is the address of a program.
@ -681,7 +681,7 @@ pub enum LoadedProgramMatchCriteria {
NoCriteria,
}
impl<FG: ForkGraph> LoadedPrograms<FG> {
impl<FG: ForkGraph> ProgramCache<FG> {
pub fn new(root_slot: Slot, root_epoch: Epoch) -> Self {
Self {
entries: HashMap::new(),
@ -734,7 +734,7 @@ impl<FG: ForkGraph> LoadedPrograms<FG> {
(LoadedProgramType::Unloaded(_), LoadedProgramType::TestLoaded(_)) => {}
_ => {
// Something is wrong, I can feel it ...
error!("LoadedPrograms::assign_program() failed key={:?} existing={:?} entry={:?}", key, slot_versions, entry);
error!("ProgramCache::assign_program() failed key={:?} existing={:?} entry={:?}", key, slot_versions, entry);
debug_assert!(false, "Unexpected replacement of an entry");
self.stats.replacements.fetch_add(1, Ordering::Relaxed);
return true;
@ -1146,9 +1146,9 @@ impl solana_frozen_abi::abi_example::AbiExample for LoadedProgram {
}
#[cfg(RUSTC_WITH_SPECIALIZATION)]
impl<FG: ForkGraph> solana_frozen_abi::abi_example::AbiExample for LoadedPrograms<FG> {
impl<FG: ForkGraph> solana_frozen_abi::abi_example::AbiExample for ProgramCache<FG> {
fn example() -> Self {
// LoadedPrograms isn't serializable by definition.
// ProgramCache isn't serializable by definition.
Self::new(Slot::default(), Epoch::default())
}
}
@ -1158,7 +1158,7 @@ mod tests {
use {
crate::loaded_programs::{
BlockRelation, ForkGraph, LoadedProgram, LoadedProgramMatchCriteria, LoadedProgramType,
LoadedPrograms, LoadedProgramsForTxBatch, ProgramRuntimeEnvironment,
LoadedProgramsForTxBatch, ProgramCache, ProgramRuntimeEnvironment,
ProgramRuntimeEnvironments, DELAY_VISIBILITY_SLOT_OFFSET,
},
assert_matches::assert_matches,
@ -1178,8 +1178,8 @@ mod tests {
static MOCK_ENVIRONMENT: std::sync::OnceLock<ProgramRuntimeEnvironment> =
std::sync::OnceLock::<ProgramRuntimeEnvironment>::new();
fn new_mock_cache<FG: ForkGraph>() -> LoadedPrograms<FG> {
let mut cache = LoadedPrograms::new(0, 0);
fn new_mock_cache<FG: ForkGraph>() -> ProgramCache<FG> {
let mut cache = ProgramCache::new(0, 0);
cache.environments.program_runtime_v1 = MOCK_ENVIRONMENT
.get_or_init(|| Arc::new(BuiltinProgram::new_mock()))
@ -1220,7 +1220,7 @@ mod tests {
}
fn set_tombstone<FG: ForkGraph>(
cache: &mut LoadedPrograms<FG>,
cache: &mut ProgramCache<FG>,
key: Pubkey,
slot: Slot,
reason: LoadedProgramType,
@ -1231,7 +1231,7 @@ mod tests {
}
fn insert_unloaded_program<FG: ForkGraph>(
cache: &mut LoadedPrograms<FG>,
cache: &mut ProgramCache<FG>,
key: Pubkey,
slot: Slot,
) -> Arc<LoadedProgram> {
@ -1254,7 +1254,7 @@ mod tests {
unloaded
}
fn num_matching_entries<P, FG>(cache: &LoadedPrograms<FG>, predicate: P) -> usize
fn num_matching_entries<P, FG>(cache: &ProgramCache<FG>, predicate: P) -> usize
where
P: Fn(&LoadedProgramType) -> bool,
FG: ForkGraph,
@ -1302,7 +1302,7 @@ mod tests {
}
fn program_deploy_test_helper(
cache: &mut LoadedPrograms<TestForkGraph>,
cache: &mut ProgramCache<TestForkGraph>,
program: Pubkey,
deployment_slots: Vec<Slot>,
usage_counters: Vec<u64>,
@ -2574,28 +2574,28 @@ mod tests {
let tombstone = Arc::new(LoadedProgram::new_tombstone(0, LoadedProgramType::Closed));
assert!(
LoadedPrograms::<TestForkGraph>::matches_loaded_program_criteria(
ProgramCache::<TestForkGraph>::matches_loaded_program_criteria(
&tombstone,
&LoadedProgramMatchCriteria::NoCriteria
)
);
assert!(
LoadedPrograms::<TestForkGraph>::matches_loaded_program_criteria(
ProgramCache::<TestForkGraph>::matches_loaded_program_criteria(
&tombstone,
&LoadedProgramMatchCriteria::Tombstone
)
);
assert!(
LoadedPrograms::<TestForkGraph>::matches_loaded_program_criteria(
ProgramCache::<TestForkGraph>::matches_loaded_program_criteria(
&tombstone,
&LoadedProgramMatchCriteria::DeployedOnOrAfterSlot(0)
)
);
assert!(
!LoadedPrograms::<TestForkGraph>::matches_loaded_program_criteria(
!ProgramCache::<TestForkGraph>::matches_loaded_program_criteria(
&tombstone,
&LoadedProgramMatchCriteria::DeployedOnOrAfterSlot(1)
)
@ -2604,28 +2604,28 @@ mod tests {
let program = new_test_loaded_program(0, 1);
assert!(
LoadedPrograms::<TestForkGraph>::matches_loaded_program_criteria(
ProgramCache::<TestForkGraph>::matches_loaded_program_criteria(
&program,
&LoadedProgramMatchCriteria::NoCriteria
)
);
assert!(
!LoadedPrograms::<TestForkGraph>::matches_loaded_program_criteria(
!ProgramCache::<TestForkGraph>::matches_loaded_program_criteria(
&program,
&LoadedProgramMatchCriteria::Tombstone
)
);
assert!(
LoadedPrograms::<TestForkGraph>::matches_loaded_program_criteria(
ProgramCache::<TestForkGraph>::matches_loaded_program_criteria(
&program,
&LoadedProgramMatchCriteria::DeployedOnOrAfterSlot(0)
)
);
assert!(
!LoadedPrograms::<TestForkGraph>::matches_loaded_program_criteria(
!ProgramCache::<TestForkGraph>::matches_loaded_program_criteria(
&program,
&LoadedProgramMatchCriteria::DeployedOnOrAfterSlot(1)
)
@ -2638,28 +2638,28 @@ mod tests {
));
assert!(
LoadedPrograms::<TestForkGraph>::matches_loaded_program_criteria(
ProgramCache::<TestForkGraph>::matches_loaded_program_criteria(
&program,
&LoadedProgramMatchCriteria::NoCriteria
)
);
assert!(
!LoadedPrograms::<TestForkGraph>::matches_loaded_program_criteria(
!ProgramCache::<TestForkGraph>::matches_loaded_program_criteria(
&program,
&LoadedProgramMatchCriteria::Tombstone
)
);
assert!(
LoadedPrograms::<TestForkGraph>::matches_loaded_program_criteria(
ProgramCache::<TestForkGraph>::matches_loaded_program_criteria(
&program,
&LoadedProgramMatchCriteria::DeployedOnOrAfterSlot(0)
)
);
assert!(
!LoadedPrograms::<TestForkGraph>::matches_loaded_program_criteria(
!ProgramCache::<TestForkGraph>::matches_loaded_program_criteria(
&program,
&LoadedProgramMatchCriteria::DeployedOnOrAfterSlot(1)
)

View File

@ -99,7 +99,7 @@ use {
compute_budget_processor::process_compute_budget_instructions,
invoke_context::BuiltinFunctionWithContext,
loaded_programs::{
LoadedProgram, LoadedProgramMatchCriteria, LoadedProgramType, LoadedPrograms,
LoadedProgram, LoadedProgramMatchCriteria, LoadedProgramType, ProgramCache,
ProgramRuntimeEnvironments,
},
runtime_config::RuntimeConfig,
@ -547,7 +547,7 @@ impl PartialEq for Bank {
accounts_data_size_delta_off_chain: _,
fee_structure: _,
incremental_snapshot_persistence: _,
loaded_programs_cache: _,
program_cache: _,
epoch_reward_status: _,
transaction_processor: _,
check_program_modification_slot: _,
@ -806,7 +806,7 @@ pub struct Bank {
pub incremental_snapshot_persistence: Option<BankIncrementalSnapshotPersistence>,
loaded_programs_cache: Arc<RwLock<LoadedPrograms<BankForks>>>,
program_cache: Arc<RwLock<ProgramCache<BankForks>>>,
epoch_reward_status: EpochRewardStatus,
@ -993,7 +993,7 @@ impl Bank {
accounts_data_size_delta_on_chain: AtomicI64::new(0),
accounts_data_size_delta_off_chain: AtomicI64::new(0),
fee_structure: FeeStructure::default(),
loaded_programs_cache: Arc::new(RwLock::new(LoadedPrograms::new(
program_cache: Arc::new(RwLock::new(ProgramCache::new(
Slot::default(),
Epoch::default(),
))),
@ -1008,7 +1008,7 @@ impl Bank {
bank.epoch_schedule.clone(),
bank.fee_structure.clone(),
bank.runtime_config.clone(),
bank.loaded_programs_cache.clone(),
bank.program_cache.clone(),
);
let accounts_data_size_initial = bank.get_total_accounts_stats().unwrap().data_len as u64;
@ -1315,7 +1315,7 @@ impl Bank {
accounts_data_size_delta_on_chain: AtomicI64::new(0),
accounts_data_size_delta_off_chain: AtomicI64::new(0),
fee_structure: parent.fee_structure.clone(),
loaded_programs_cache: parent.loaded_programs_cache.clone(),
program_cache: parent.program_cache.clone(),
epoch_reward_status: parent.epoch_reward_status.clone(),
transaction_processor: TransactionBatchProcessor::default(),
check_program_modification_slot: false,
@ -1327,7 +1327,7 @@ impl Bank {
new.epoch_schedule.clone(),
new.fee_structure.clone(),
new.runtime_config.clone(),
new.loaded_programs_cache.clone(),
new.program_cache.clone(),
);
let (_, ancestors_time_us) = measure_us!({
@ -1367,13 +1367,12 @@ impl Bank {
.min(slots_in_epoch)
.checked_div(2)
.unwrap();
let mut loaded_programs_cache = new.loaded_programs_cache.write().unwrap();
if loaded_programs_cache.upcoming_environments.is_some() {
if let Some((key, program_to_recompile)) =
loaded_programs_cache.programs_to_recompile.pop()
let mut program_cache = new.program_cache.write().unwrap();
if program_cache.upcoming_environments.is_some() {
if let Some((key, program_to_recompile)) = program_cache.programs_to_recompile.pop()
{
let effective_epoch = loaded_programs_cache.latest_root_epoch.saturating_add(1);
drop(loaded_programs_cache);
let effective_epoch = program_cache.latest_root_epoch.saturating_add(1);
drop(program_cache);
let recompiled = new.load_program(&key, false, effective_epoch);
recompiled
.tx_usage_counter
@ -1381,17 +1380,17 @@ impl Bank {
recompiled
.ix_usage_counter
.fetch_add(program_to_recompile.ix_usage_counter.load(Relaxed), Relaxed);
let mut loaded_programs_cache = new.loaded_programs_cache.write().unwrap();
loaded_programs_cache.assign_program(key, recompiled);
let mut program_cache = new.program_cache.write().unwrap();
program_cache.assign_program(key, recompiled);
}
} else if new.epoch() != loaded_programs_cache.latest_root_epoch
} else if new.epoch() != program_cache.latest_root_epoch
|| slot_index.saturating_add(slots_in_recompilation_phase) >= slots_in_epoch
{
// Anticipate the upcoming program runtime environment for the next epoch,
// so we can try to recompile loaded programs before the feature transition hits.
drop(loaded_programs_cache);
drop(program_cache);
let (feature_set, _new_feature_activations) = new.compute_active_feature_set(true);
let mut loaded_programs_cache = new.loaded_programs_cache.write().unwrap();
let mut program_cache = new.program_cache.write().unwrap();
let program_runtime_environment_v1 = create_program_runtime_environment_v1(
&feature_set,
&new.runtime_config.compute_budget.unwrap_or_default(),
@ -1403,7 +1402,7 @@ impl Bank {
&new.runtime_config.compute_budget.unwrap_or_default(),
false, /* debugging_features */
);
let mut upcoming_environments = loaded_programs_cache.environments.clone();
let mut upcoming_environments = program_cache.environments.clone();
let changed_program_runtime_v1 =
*upcoming_environments.program_runtime_v1 != program_runtime_environment_v1;
let changed_program_runtime_v2 =
@ -1416,10 +1415,10 @@ impl Bank {
upcoming_environments.program_runtime_v2 =
Arc::new(program_runtime_environment_v2);
}
loaded_programs_cache.upcoming_environments = Some(upcoming_environments);
loaded_programs_cache.programs_to_recompile = loaded_programs_cache
program_cache.upcoming_environments = Some(upcoming_environments);
program_cache.programs_to_recompile = program_cache
.get_flattened_entries(changed_program_runtime_v1, changed_program_runtime_v2);
loaded_programs_cache
program_cache
.programs_to_recompile
.sort_by_cached_key(|(_id, program)| program.decayed_usage_counter(slot));
}
@ -1464,32 +1463,32 @@ impl Bank {
);
parent
.loaded_programs_cache
.program_cache
.read()
.unwrap()
.stats
.submit(parent.slot());
new.loaded_programs_cache.write().unwrap().stats.reset();
new.program_cache.write().unwrap().stats.reset();
new
}
pub fn set_fork_graph_in_program_cache(&self, fork_graph: Arc<RwLock<BankForks>>) {
self.loaded_programs_cache
self.program_cache
.write()
.unwrap()
.set_fork_graph(fork_graph);
}
pub fn prune_program_cache(&self, new_root_slot: Slot, new_root_epoch: Epoch) {
self.loaded_programs_cache
self.program_cache
.write()
.unwrap()
.prune(new_root_slot, new_root_epoch);
}
pub fn prune_program_cache_by_deployment_slot(&self, deployment_slot: Slot) {
self.loaded_programs_cache
self.program_cache
.write()
.unwrap()
.prune_by_deployment_slot(deployment_slot);
@ -1497,7 +1496,7 @@ impl Bank {
pub fn get_runtime_environments_for_slot(&self, slot: Slot) -> ProgramRuntimeEnvironments {
let epoch = self.epoch_schedule.get_epoch(slot);
self.loaded_programs_cache
self.program_cache
.read()
.unwrap()
.get_environments_for_epoch(epoch)
@ -1863,10 +1862,7 @@ impl Bank {
accounts_data_size_delta_on_chain: AtomicI64::new(0),
accounts_data_size_delta_off_chain: AtomicI64::new(0),
fee_structure: FeeStructure::default(),
loaded_programs_cache: Arc::new(RwLock::new(LoadedPrograms::new(
fields.slot,
fields.epoch,
))),
program_cache: Arc::new(RwLock::new(ProgramCache::new(fields.slot, fields.epoch))),
epoch_reward_status: fields.epoch_reward_status,
transaction_processor: TransactionBatchProcessor::default(),
check_program_modification_slot: false,
@ -1878,7 +1874,7 @@ impl Bank {
bank.epoch_schedule.clone(),
bank.fee_structure.clone(),
bank.runtime_config.clone(),
bank.loaded_programs_cache.clone(),
bank.program_cache.clone(),
);
bank.finish_init(
@ -4987,7 +4983,7 @@ impl Bank {
} = execution_result
{
if details.status.is_ok() {
let mut cache = self.loaded_programs_cache.write().unwrap();
let mut cache = self.program_cache.write().unwrap();
cache.merge(programs_modified_by_tx);
}
}
@ -6013,10 +6009,10 @@ impl Bank {
}
}
let mut loaded_programs_cache = self.loaded_programs_cache.write().unwrap();
loaded_programs_cache.latest_root_slot = self.slot();
loaded_programs_cache.latest_root_epoch = self.epoch();
loaded_programs_cache.environments.program_runtime_v1 = Arc::new(
let mut program_cache = self.program_cache.write().unwrap();
program_cache.latest_root_slot = self.slot();
program_cache.latest_root_epoch = self.epoch();
program_cache.environments.program_runtime_v1 = Arc::new(
create_program_runtime_environment_v1(
&self.feature_set,
&self.runtime_config.compute_budget.unwrap_or_default(),
@ -6025,7 +6021,7 @@ impl Bank {
)
.unwrap(),
);
loaded_programs_cache.environments.program_runtime_v2 =
program_cache.environments.program_runtime_v2 =
Arc::new(create_program_runtime_environment_v2(
&self.runtime_config.compute_budget.unwrap_or_default(),
false, /* debugging_features */
@ -7094,7 +7090,7 @@ impl Bank {
debug!("Adding program {} under {:?}", name, program_id);
self.add_builtin_account(name.as_str(), &program_id, false);
self.builtin_programs.insert(program_id);
self.loaded_programs_cache
self.program_cache
.write()
.unwrap()
.assign_program(program_id, Arc::new(builtin));
@ -7399,7 +7395,7 @@ impl Bank {
self.store_account(new_address, &AccountSharedData::default());
// Unload a program from the bank's cache
self.loaded_programs_cache
self.program_cache
.write()
.unwrap()
.remove_programs([*old_address].into_iter());

View File

@ -11910,14 +11910,14 @@ fn test_feature_activation_loaded_programs_recompilation_phase() {
goto_end_of_slot(bank.clone());
let bank = new_bank_from_parent_with_bank_forks(&bank_forks, bank, &Pubkey::default(), 16);
let current_env = bank
.loaded_programs_cache
.program_cache
.read()
.unwrap()
.get_environments_for_epoch(0)
.program_runtime_v1
.clone();
let upcoming_env = bank
.loaded_programs_cache
.program_cache
.read()
.unwrap()
.get_environments_for_epoch(1)
@ -11926,9 +11926,8 @@ fn test_feature_activation_loaded_programs_recompilation_phase() {
// Advance the bank to recompile the program.
{
let loaded_programs_cache = bank.loaded_programs_cache.read().unwrap();
let slot_versions =
loaded_programs_cache.get_slot_versions_for_tests(&program_keypair.pubkey());
let program_cache = bank.program_cache.read().unwrap();
let slot_versions = program_cache.get_slot_versions_for_tests(&program_keypair.pubkey());
assert_eq!(slot_versions.len(), 1);
assert!(Arc::ptr_eq(
slot_versions[0].program.get_environment().unwrap(),
@ -11938,9 +11937,8 @@ fn test_feature_activation_loaded_programs_recompilation_phase() {
goto_end_of_slot(bank.clone());
let bank = new_from_parent_with_fork_next_slot(bank, bank_forks.as_ref());
{
let loaded_programs_cache = bank.loaded_programs_cache.read().unwrap();
let slot_versions =
loaded_programs_cache.get_slot_versions_for_tests(&program_keypair.pubkey());
let program_cache = bank.program_cache.read().unwrap();
let slot_versions = program_cache.get_slot_versions_for_tests(&program_keypair.pubkey());
assert_eq!(slot_versions.len(), 2);
assert!(Arc::ptr_eq(
slot_versions[0].program.get_environment().unwrap(),

View File

@ -17,7 +17,7 @@ use {
compute_budget::ComputeBudget,
loaded_programs::{
ForkGraph, LoadProgramMetrics, LoadedProgram, LoadedProgramMatchCriteria,
LoadedProgramType, LoadedPrograms, LoadedProgramsForTxBatch, ProgramRuntimeEnvironment,
LoadedProgramType, LoadedProgramsForTxBatch, ProgramCache, ProgramRuntimeEnvironment,
ProgramRuntimeEnvironments, DELAY_VISIBILITY_SLOT_OFFSET,
},
log_collector::LogCollector,
@ -137,7 +137,8 @@ pub struct TransactionBatchProcessor<FG: ForkGraph> {
pub sysvar_cache: RwLock<SysvarCache>,
pub loaded_programs_cache: Arc<RwLock<LoadedPrograms<FG>>>,
/// Programs required for transaction batch processing
pub program_cache: Arc<RwLock<ProgramCache<FG>>>,
}
impl<FG: ForkGraph> Debug for TransactionBatchProcessor<FG> {
@ -149,7 +150,7 @@ impl<FG: ForkGraph> Debug for TransactionBatchProcessor<FG> {
.field("fee_structure", &self.fee_structure)
.field("runtime_config", &self.runtime_config)
.field("sysvar_cache", &self.sysvar_cache)
.field("loaded_programs_cache", &self.loaded_programs_cache)
.field("program_cache", &self.program_cache)
.finish()
}
}
@ -163,7 +164,7 @@ impl<FG: ForkGraph> Default for TransactionBatchProcessor<FG> {
fee_structure: FeeStructure::default(),
runtime_config: Arc::<RuntimeConfig>::default(),
sysvar_cache: RwLock::<SysvarCache>::default(),
loaded_programs_cache: Arc::new(RwLock::new(LoadedPrograms::new(
program_cache: Arc::new(RwLock::new(ProgramCache::new(
Slot::default(),
Epoch::default(),
))),
@ -178,7 +179,7 @@ impl<FG: ForkGraph> TransactionBatchProcessor<FG> {
epoch_schedule: EpochSchedule,
fee_structure: FeeStructure,
runtime_config: Arc<RuntimeConfig>,
loaded_programs_cache: Arc<RwLock<LoadedPrograms<FG>>>,
program_cache: Arc<RwLock<ProgramCache<FG>>>,
) -> Self {
Self {
slot,
@ -187,7 +188,7 @@ impl<FG: ForkGraph> TransactionBatchProcessor<FG> {
fee_structure,
runtime_config,
sysvar_cache: RwLock::<SysvarCache>::default(),
loaded_programs_cache,
program_cache,
}
}
@ -308,7 +309,7 @@ impl<FG: ForkGraph> TransactionBatchProcessor<FG> {
execution_time.stop();
const SHRINK_LOADED_PROGRAMS_TO_PERCENTAGE: u8 = 90;
self.loaded_programs_cache
self.program_cache
.write()
.unwrap()
.evict_using_2s_random_selection(
@ -374,8 +375,8 @@ impl<FG: ForkGraph> TransactionBatchProcessor<FG> {
result
}
/// Load program with a specific pubkey from loaded programs
/// cache, and update the program's access slot as a side-effect.
/// Load program with a specific pubkey from program cache, and
/// update the program's access slot as a side-effect.
pub fn load_program_with_pubkey<CB: TransactionProcessingCallback>(
&self,
callbacks: &CB,
@ -383,8 +384,8 @@ impl<FG: ForkGraph> TransactionBatchProcessor<FG> {
reload: bool,
effective_epoch: Epoch,
) -> Arc<LoadedProgram> {
let loaded_programs_cache = self.loaded_programs_cache.read().unwrap();
let environments = loaded_programs_cache.get_environments_for_epoch(effective_epoch);
let program_cache = self.program_cache.read().unwrap();
let environments = program_cache.get_environments_for_epoch(effective_epoch);
let mut load_program_metrics = LoadProgramMetrics {
program_id: pubkey.to_string(),
..LoadProgramMetrics::default()
@ -463,10 +464,10 @@ impl<FG: ForkGraph> TransactionBatchProcessor<FG> {
load_program_metrics.submit_datapoint(&mut timings);
if !Arc::ptr_eq(
&environments.program_runtime_v1,
&loaded_programs_cache.environments.program_runtime_v1,
&program_cache.environments.program_runtime_v1,
) || !Arc::ptr_eq(
&environments.program_runtime_v2,
&loaded_programs_cache.environments.program_runtime_v2,
&program_cache.environments.program_runtime_v2,
) {
// There can be two entries per program when the environment changes.
// One for the old environment before the epoch boundary and one for the new environment after the epoch boundary.
@ -502,21 +503,18 @@ impl<FG: ForkGraph> TransactionBatchProcessor<FG> {
loop {
let (program_to_load, task_cookie, task_waiter) = {
// Lock the global cache.
let mut loaded_programs_cache = self.loaded_programs_cache.write().unwrap();
let mut program_cache = self.program_cache.write().unwrap();
// Initialize our local cache.
let is_first_round = loaded_programs_for_txs.is_none();
if is_first_round {
loaded_programs_for_txs = Some(LoadedProgramsForTxBatch::new(
self.slot,
loaded_programs_cache
.get_environments_for_epoch(self.epoch)
.clone(),
program_cache.get_environments_for_epoch(self.epoch).clone(),
));
}
// Submit our last completed loading task.
if let Some((key, program)) = program_to_store.take() {
if loaded_programs_cache
.finish_cooperative_loading_task(self.slot, key, program)
if program_cache.finish_cooperative_loading_task(self.slot, key, program)
&& limit_to_load_programs
{
// This branch is taken when there is an error in assigning a program to a
@ -524,21 +522,19 @@ impl<FG: ForkGraph> TransactionBatchProcessor<FG> {
// tests purposes.
let mut ret = LoadedProgramsForTxBatch::new(
self.slot,
loaded_programs_cache
.get_environments_for_epoch(self.epoch)
.clone(),
program_cache.get_environments_for_epoch(self.epoch).clone(),
);
ret.hit_max_limit = true;
return ret;
}
}
// Figure out which program needs to be loaded next.
let program_to_load = loaded_programs_cache.extract(
let program_to_load = program_cache.extract(
&mut missing_programs,
loaded_programs_for_txs.as_mut().unwrap(),
is_first_round,
);
let task_waiter = Arc::clone(&loaded_programs_cache.loading_task_waiter);
let task_waiter = Arc::clone(&program_cache.loading_task_waiter);
(program_to_load, task_waiter.cookie(), task_waiter)
// Unlock the global cache again.
};
@ -1266,7 +1262,7 @@ mod tests {
0,
LoadedProgramType::FailedVerification(
batch_processor
.loaded_programs_cache
.program_cache
.read()
.unwrap()
.get_environments_for_epoch(20)
@ -1294,7 +1290,7 @@ mod tests {
0,
LoadedProgramType::FailedVerification(
batch_processor
.loaded_programs_cache
.program_cache
.read()
.unwrap()
.get_environments_for_epoch(20)
@ -1367,7 +1363,7 @@ mod tests {
0,
LoadedProgramType::FailedVerification(
batch_processor
.loaded_programs_cache
.program_cache
.read()
.unwrap()
.get_environments_for_epoch(0)
@ -1447,7 +1443,7 @@ mod tests {
0,
LoadedProgramType::FailedVerification(
batch_processor
.loaded_programs_cache
.program_cache
.read()
.unwrap()
.get_environments_for_epoch(0)
@ -1506,7 +1502,7 @@ mod tests {
let batch_processor = TransactionBatchProcessor::<TestForkGraph>::default();
batch_processor
.loaded_programs_cache
.program_cache
.write()
.unwrap()
.upcoming_environments = Some(ProgramRuntimeEnvironments::default());
@ -1801,11 +1797,8 @@ mod tests {
// Case 1
let mut mock_bank = MockBankCallback::default();
let batch_processor = TransactionBatchProcessor::<TestForkGraph>::default();
batch_processor
.loaded_programs_cache
.write()
.unwrap()
.fork_graph = Some(Arc::new(RwLock::new(TestForkGraph {})));
batch_processor.program_cache.write().unwrap().fork_graph =
Some(Arc::new(RwLock::new(TestForkGraph {})));
let key1 = Pubkey::new_unique();
let key2 = Pubkey::new_unique();
let owner = Pubkey::new_unique();

View File

@ -7,7 +7,7 @@ use {
compute_budget::ComputeBudget,
invoke_context::InvokeContext,
loaded_programs::{
BlockRelation, ForkGraph, LoadedProgram, LoadedPrograms, ProgramRuntimeEnvironments,
BlockRelation, ForkGraph, LoadedProgram, ProgramCache, ProgramRuntimeEnvironments,
},
runtime_config::RuntimeConfig,
solana_rbpf::{
@ -113,8 +113,8 @@ fn create_custom_environment<'a>() -> BuiltinProgram<InvokeContext<'a>> {
fn create_executable_environment(
mock_bank: &mut MockBankCallback,
) -> (LoadedPrograms<MockForkGraph>, Vec<Pubkey>) {
let mut programs_cache = LoadedPrograms::<MockForkGraph>::new(0, 20);
) -> (ProgramCache<MockForkGraph>, Vec<Pubkey>) {
let mut program_cache = ProgramCache::<MockForkGraph>::new(0, 20);
// We must register the bpf loader account as a loadable account, otherwise programs
// won't execute.
@ -127,7 +127,7 @@ fn create_executable_environment(
.insert(bpf_loader::id(), account_data);
// The bpf loader needs an executable as well
programs_cache.assign_program(
program_cache.assign_program(
bpf_loader::id(),
Arc::new(LoadedProgram::new_builtin(
DEPLOYMENT_SLOT,
@ -136,7 +136,7 @@ fn create_executable_environment(
)),
);
programs_cache.environments = ProgramRuntimeEnvironments {
program_cache.environments = ProgramRuntimeEnvironments {
program_runtime_v1: Arc::new(create_custom_environment()),
// We are not using program runtime v2
program_runtime_v2: Arc::new(BuiltinProgram::new_loader(
@ -145,11 +145,11 @@ fn create_executable_environment(
)),
};
programs_cache.fork_graph = Some(Arc::new(RwLock::new(MockForkGraph {})));
program_cache.fork_graph = Some(Arc::new(RwLock::new(MockForkGraph {})));
// Inform SVM of the registered builins
let registered_built_ins = vec![bpf_loader::id()];
(programs_cache, registered_built_ins)
(program_cache, registered_built_ins)
}
fn prepare_transactions(
@ -224,15 +224,15 @@ fn prepare_transactions(
fn svm_integration() {
let mut mock_bank = MockBankCallback::default();
let (transactions, mut check_results) = prepare_transactions(&mut mock_bank);
let (programs_cache, builtins) = create_executable_environment(&mut mock_bank);
let programs_cache = Arc::new(RwLock::new(programs_cache));
let (program_cache, builtins) = create_executable_environment(&mut mock_bank);
let program_cache = Arc::new(RwLock::new(program_cache));
let batch_processor = TransactionBatchProcessor::<MockForkGraph>::new(
EXECUTION_SLOT,
EXECUTION_EPOCH,
EpochSchedule::default(),
FeeStructure::default(),
Arc::new(RuntimeConfig::default()),
programs_cache.clone(),
program_cache.clone(),
);
let mut error_counter = TransactionErrorMetrics::default();