Refactor - executor_cache (#28332)

* Renames CachedExecutors => BankExecutorCache.

* Renames TransactionExecutor => TransactionExecutorCacheEntry.

* Renames TransactionExecutorStatus => TxBankExecutorCacheDiff.

* Adds TransactionExecutorCache.

* Renames the items of TxBankExecutorCacheDiff.
This commit is contained in:
Alexander Meißner 2022-10-12 18:09:03 +02:00 committed by GitHub
parent 9fe46bb038
commit 664339e239
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 252 additions and 250 deletions

View File

@ -12,7 +12,7 @@ use {
ops::Div,
sync::{
atomic::{AtomicU64, Ordering::Relaxed},
Arc,
Arc, RwLock,
},
},
};
@ -27,81 +27,97 @@ pub trait Executor: Debug + Send + Sync {
) -> Result<(), InstructionError>;
}
pub type Executors = HashMap<Pubkey, TransactionExecutor>;
/// Relation between a TransactionExecutorCacheEntry and its matching BankExecutorCacheEntry
#[repr(u8)]
#[derive(PartialEq, Debug)]
enum TransactionExecutorStatus {
/// Executor was already in the cache, no update needed
Cached,
/// Executor was missing from the cache, but not updated
Missing,
/// Executor is for an updated program
#[derive(Clone, Copy, PartialEq, Debug)]
pub enum TxBankExecutorCacheDiff {
/// The TransactionExecutorCacheEntry did not change and is the same as the BankExecutorCacheEntry.
None,
/// The TransactionExecutorCacheEntry was inserted, no matching BankExecutorCacheEntry exists, so it needs to be inserted.
Inserted,
/// The TransactionExecutorCacheEntry was replaced, the matching BankExecutorCacheEntry needs to be updated.
Updated,
}
/// Tracks whether a given executor is "dirty" and needs to updated in the
/// executors cache
/// An entry of the TransactionExecutorCache
#[derive(Debug)]
pub struct TransactionExecutor {
pub(crate) executor: Arc<dyn Executor>,
status: TransactionExecutorStatus,
pub struct TransactionExecutorCacheEntry {
executor: Arc<dyn Executor>,
difference: TxBankExecutorCacheDiff,
}
impl TransactionExecutor {
/// Wraps an executor and tracks that it doesn't need to be updated in the
/// executors cache.
pub fn new_cached(executor: Arc<dyn Executor>) -> Self {
/// A subset of the BankExecutorCache containing only the executors relevant to one transaction
///
/// The BankExecutorCache can not be updated directly as transaction batches are
/// processed in parallel, which would cause a race condition.
#[derive(Default, Debug)]
pub struct TransactionExecutorCache {
pub executors: HashMap<Pubkey, TransactionExecutorCacheEntry>,
}
impl TransactionExecutorCache {
pub fn new(executable_keys: impl Iterator<Item = (Pubkey, Arc<dyn Executor>)>) -> Self {
Self {
executor,
status: TransactionExecutorStatus::Cached,
executors: executable_keys
.map(|(key, executor)| {
let entry = TransactionExecutorCacheEntry {
executor,
difference: TxBankExecutorCacheDiff::None,
};
(key, entry)
})
.collect(),
}
}
/// Wraps an executor and tracks that it needs to be updated in the
/// executors cache.
pub fn new_miss(executor: Arc<dyn Executor>) -> Self {
Self {
pub fn get(&self, key: &Pubkey) -> Option<Arc<dyn Executor>> {
self.executors.get(key).map(|entry| entry.executor.clone())
}
pub fn set(&mut self, key: Pubkey, executor: Arc<dyn Executor>, replacement: bool) {
let difference = if replacement {
TxBankExecutorCacheDiff::Updated
} else {
TxBankExecutorCacheDiff::Inserted
};
let entry = TransactionExecutorCacheEntry {
executor,
status: TransactionExecutorStatus::Missing,
difference,
};
let _was_replaced = self.executors.insert(key, entry).is_some();
}
pub fn update_global_cache(
&self,
global_cache: &RwLock<BankExecutorCache>,
selector: impl Fn(TxBankExecutorCacheDiff) -> bool,
) {
let executors_delta: Vec<_> = self
.executors
.iter()
.filter_map(|(key, entry)| {
selector(entry.difference).then(|| (key, entry.executor.clone()))
})
.collect();
if !executors_delta.is_empty() {
global_cache.write().unwrap().put(&executors_delta);
}
}
/// Wraps an executor and tracks that it needs to be updated in the
/// executors cache only if the transaction succeeded.
pub fn new_updated(executor: Arc<dyn Executor>) -> Self {
Self {
executor,
status: TransactionExecutorStatus::Updated,
}
}
pub fn is_missing(&self) -> bool {
self.status == TransactionExecutorStatus::Missing
}
pub fn is_updated(&self) -> bool {
self.status == TransactionExecutorStatus::Updated
}
pub fn get(&self) -> Arc<dyn Executor> {
self.executor.clone()
}
}
/// Capacity of `CachedExecutors`
/// Capacity of `BankExecutorCache`
pub const MAX_CACHED_EXECUTORS: usize = 256;
/// An `Executor` and its statistics tracked in `CachedExecutors`
/// An entry of the BankExecutorCache
#[derive(Debug)]
pub struct CachedExecutorsEntry {
pub struct BankExecutorCacheEntry {
prev_epoch_count: u64,
epoch_count: AtomicU64,
executor: Arc<dyn Executor>,
pub hit_count: AtomicU64,
}
impl Clone for CachedExecutorsEntry {
impl Clone for BankExecutorCacheEntry {
fn clone(&self) -> Self {
Self {
prev_epoch_count: self.prev_epoch_count,
@ -112,16 +128,16 @@ impl Clone for CachedExecutorsEntry {
}
}
/// LFU Cache of executors with single-epoch memory of usage counts
/// LFU Cache of executors which exists once per bank
#[derive(Debug)]
pub struct CachedExecutors {
pub struct BankExecutorCache {
capacity: usize,
current_epoch: Epoch,
pub executors: HashMap<Pubkey, CachedExecutorsEntry>,
pub executors: HashMap<Pubkey, BankExecutorCacheEntry>,
pub stats: Stats,
}
impl Default for CachedExecutors {
impl Default for BankExecutorCache {
fn default() -> Self {
Self {
capacity: MAX_CACHED_EXECUTORS,
@ -133,16 +149,16 @@ impl Default for CachedExecutors {
}
#[cfg(RUSTC_WITH_SPECIALIZATION)]
impl solana_frozen_abi::abi_example::AbiExample for CachedExecutors {
impl solana_frozen_abi::abi_example::AbiExample for BankExecutorCache {
fn example() -> Self {
// Delegate AbiExample impl to Default before going deep and stuck with
// not easily impl-able Arc<dyn Executor> due to rust's coherence issue
// This is safe because CachedExecutors isn't serializable by definition.
// This is safe because BankExecutorCache isn't serializable by definition.
Self::default()
}
}
impl CachedExecutors {
impl BankExecutorCache {
pub fn new(max_capacity: usize, current_epoch: Epoch) -> Self {
Self {
capacity: max_capacity,
@ -153,7 +169,7 @@ impl CachedExecutors {
}
pub fn new_from_parent_bank_executors(
parent_bank_executors: &CachedExecutors,
parent_bank_executors: &BankExecutorCache,
current_epoch: Epoch,
) -> Self {
let executors = if parent_bank_executors.current_epoch == current_epoch {
@ -163,7 +179,7 @@ impl CachedExecutors {
.executors
.iter()
.map(|(&key, entry)| {
let entry = CachedExecutorsEntry {
let entry = BankExecutorCacheEntry {
prev_epoch_count: entry.epoch_count.load(Relaxed),
epoch_count: AtomicU64::default(),
executor: entry.executor.clone(),
@ -194,7 +210,7 @@ impl CachedExecutors {
}
}
pub fn put(&mut self, executors: &[(&Pubkey, Arc<dyn Executor>)]) {
fn put(&mut self, executors: &[(&Pubkey, Arc<dyn Executor>)]) {
let mut new_executors: Vec<_> = executors
.iter()
.filter_map(|(key, executor)| {
@ -242,7 +258,7 @@ impl CachedExecutors {
}
for ((key, executor), primer_count) in new_executors.drain(..).zip(primer_counts) {
let entry = CachedExecutorsEntry {
let entry = BankExecutorCacheEntry {
prev_epoch_count: 0,
epoch_count: AtomicU64::new(primer_count),
executor: executor.clone(),
@ -253,7 +269,7 @@ impl CachedExecutors {
}
}
pub fn remove(&mut self, pubkey: &Pubkey) -> Option<CachedExecutorsEntry> {
pub fn remove(&mut self, pubkey: &Pubkey) -> Option<BankExecutorCacheEntry> {
let maybe_entry = self.executors.remove(pubkey);
if let Some(entry) = maybe_entry.as_ref() {
if entry.hit_count.load(Relaxed) == 1 {
@ -264,7 +280,7 @@ impl CachedExecutors {
}
pub fn clear(&mut self) {
*self = CachedExecutors::default();
*self = BankExecutorCache::default();
}
pub fn get_primer_count_upper_bound_inclusive(counts: &[(&Pubkey, u64)]) -> u64 {
@ -303,7 +319,7 @@ impl CachedExecutors {
}
}
/// Statistics of the entrie `CachedExecutors`
/// Statistics of the entire `BankExecutorCache`
#[derive(Debug, Default)]
pub struct Stats {
pub hits: AtomicU64,
@ -378,13 +394,13 @@ mod tests {
}
#[test]
fn test_cached_executors() {
fn test_executor_cache() {
let key1 = solana_sdk::pubkey::new_rand();
let key2 = solana_sdk::pubkey::new_rand();
let key3 = solana_sdk::pubkey::new_rand();
let key4 = solana_sdk::pubkey::new_rand();
let executor: Arc<dyn Executor> = Arc::new(TestExecutor {});
let mut cache = CachedExecutors::new(3, 0);
let mut cache = BankExecutorCache::new(3, 0);
cache.put(&[(&key1, executor.clone())]);
cache.put(&[(&key2, executor.clone())]);
@ -423,7 +439,7 @@ mod tests {
let key3 = solana_sdk::pubkey::new_rand();
let key4 = solana_sdk::pubkey::new_rand();
let executor: Arc<dyn Executor> = Arc::new(TestExecutor {});
let mut cache = CachedExecutors::new(3, 0);
let mut cache = BankExecutorCache::new(3, 0);
assert!(cache.current_epoch == 0);
cache.put(&[(&key1, executor.clone())]);
@ -433,7 +449,7 @@ mod tests {
assert!(cache.get(&key1).is_some());
assert!(cache.get(&key1).is_some());
let mut cache = CachedExecutors::new_from_parent_bank_executors(&cache, 1);
let mut cache = BankExecutorCache::new_from_parent_bank_executors(&cache, 1);
assert!(cache.current_epoch == 1);
assert!(cache.get(&key2).is_some());
@ -458,7 +474,7 @@ mod tests {
.count();
assert_eq!(num_retained, 1);
cache = CachedExecutors::new_from_parent_bank_executors(&cache, 2);
cache = BankExecutorCache::new_from_parent_bank_executors(&cache, 2);
assert!(cache.current_epoch == 2);
cache.put(&[(&key3, executor.clone())]);
@ -466,12 +482,12 @@ mod tests {
}
#[test]
fn test_cached_executors_evicts_smallest() {
fn test_executor_cache_evicts_smallest() {
let key1 = solana_sdk::pubkey::new_rand();
let key2 = solana_sdk::pubkey::new_rand();
let key3 = solana_sdk::pubkey::new_rand();
let executor: Arc<dyn Executor> = Arc::new(TestExecutor {});
let mut cache = CachedExecutors::new(2, 0);
let mut cache = BankExecutorCache::new(2, 0);
cache.put(&[(&key1, executor.clone())]);
for _ in 0..5 {
@ -495,8 +511,8 @@ mod tests {
}
#[test]
fn test_cached_executors_one_hit_wonder_counter() {
let mut cache = CachedExecutors::new(1, 0);
fn test_executor_cache_one_hit_wonder_counter() {
let mut cache = BankExecutorCache::new(1, 0);
let one_hit_wonder = Pubkey::new_unique();
let popular = Pubkey::new_unique();
@ -532,23 +548,23 @@ mod tests {
let pubkey = Pubkey::default();
let v = [];
assert_eq!(
CachedExecutors::get_primer_count_upper_bound_inclusive(&v),
BankExecutorCache::get_primer_count_upper_bound_inclusive(&v),
0
);
let v = [(&pubkey, 1)];
assert_eq!(
CachedExecutors::get_primer_count_upper_bound_inclusive(&v),
BankExecutorCache::get_primer_count_upper_bound_inclusive(&v),
1
);
let v = (0u64..10).map(|i| (&pubkey, i)).collect::<Vec<_>>();
assert_eq!(
CachedExecutors::get_primer_count_upper_bound_inclusive(v.as_slice()),
BankExecutorCache::get_primer_count_upper_bound_inclusive(v.as_slice()),
7
);
}
#[test]
fn test_cached_executors_stats() {
fn test_executor_cache_stats() {
#[derive(Debug, Default, PartialEq)]
struct ComparableStats {
hits: u64,
@ -580,7 +596,7 @@ mod tests {
}
const CURRENT_EPOCH: Epoch = 0;
let mut cache = CachedExecutors::new(2, CURRENT_EPOCH);
let mut cache = BankExecutorCache::new(2, CURRENT_EPOCH);
let mut expected_stats = ComparableStats::default();
let program_id1 = Pubkey::new_unique();
@ -623,13 +639,13 @@ mod tests {
// make sure stats are cleared in new_from_parent
assert_eq!(
ComparableStats::from(
&CachedExecutors::new_from_parent_bank_executors(&cache, CURRENT_EPOCH).stats
&BankExecutorCache::new_from_parent_bank_executors(&cache, CURRENT_EPOCH).stats
),
ComparableStats::default()
);
assert_eq!(
ComparableStats::from(
&CachedExecutors::new_from_parent_bank_executors(&cache, CURRENT_EPOCH + 1).stats
&BankExecutorCache::new_from_parent_bank_executors(&cache, CURRENT_EPOCH + 1).stats
),
ComparableStats::default()
);

View File

@ -2,7 +2,7 @@ use {
crate::{
accounts_data_meter::AccountsDataMeter,
compute_budget::ComputeBudget,
executor_cache::{Executor, Executors, TransactionExecutor},
executor_cache::TransactionExecutorCache,
ic_logger_msg, ic_msg,
log_collector::LogCollector,
pre_account::PreAccount,
@ -121,7 +121,7 @@ pub struct InvokeContext<'a> {
current_compute_budget: ComputeBudget,
compute_meter: Rc<RefCell<ComputeMeter>>,
accounts_data_meter: AccountsDataMeter,
executors: Rc<RefCell<Executors>>,
pub tx_executor_cache: Rc<RefCell<TransactionExecutorCache>>,
pub feature_set: Arc<FeatureSet>,
pub timings: ExecuteDetailsTimings,
pub blockhash: Hash,
@ -138,7 +138,7 @@ impl<'a> InvokeContext<'a> {
sysvar_cache: Cow<'a, SysvarCache>,
log_collector: Option<Rc<RefCell<LogCollector>>>,
compute_budget: ComputeBudget,
executors: Rc<RefCell<Executors>>,
tx_executor_cache: Rc<RefCell<TransactionExecutorCache>>,
feature_set: Arc<FeatureSet>,
blockhash: Hash,
lamports_per_signature: u64,
@ -155,7 +155,7 @@ impl<'a> InvokeContext<'a> {
compute_budget,
compute_meter: ComputeMeter::new_ref(compute_budget.compute_unit_limit),
accounts_data_meter: AccountsDataMeter::new(prev_accounts_data_len),
executors,
tx_executor_cache,
feature_set,
timings: ExecuteDetailsTimings::default(),
blockhash,
@ -193,7 +193,7 @@ impl<'a> InvokeContext<'a> {
Cow::Owned(sysvar_cache),
Some(LogCollector::new_ref()),
ComputeBudget::default(),
Rc::new(RefCell::new(Executors::default())),
Rc::new(RefCell::new(TransactionExecutorCache::default())),
Arc::new(FeatureSet::all_enabled()),
Hash::default(),
0,
@ -794,28 +794,6 @@ impl<'a> InvokeContext<'a> {
&self.accounts_data_meter
}
/// Cache an executor that wasn't found in the cache
pub fn add_executor(&self, pubkey: &Pubkey, executor: Arc<dyn Executor>) {
self.executors
.borrow_mut()
.insert(*pubkey, TransactionExecutor::new_miss(executor));
}
/// Cache an executor that has changed
pub fn update_executor(&self, pubkey: &Pubkey, executor: Arc<dyn Executor>) {
self.executors
.borrow_mut()
.insert(*pubkey, TransactionExecutor::new_updated(executor));
}
/// Get the completed loader work that can be re-used across execution
pub fn get_executor(&self, pubkey: &Pubkey) -> Option<Arc<dyn Executor>> {
self.executors
.borrow()
.get(pubkey)
.map(|tx_executor| tx_executor.executor.clone())
}
/// Get this invocation's compute budget
pub fn get_compute_budget(&self) -> &ComputeBudget {
&self.current_compute_budget

View File

@ -440,24 +440,27 @@ fn process_instruction_common(
drop(program);
let mut get_or_create_executor_time = Measure::start("get_or_create_executor_time");
let executor = match invoke_context.get_executor(program_id) {
Some(executor) => executor,
None => {
let executor = create_executor(
first_instruction_account,
program_data_offset,
invoke_context,
use_jit,
false, /* reject_deployment_of_broken_elfs */
// allow _sol_alloc_free syscall for execution
false, /* disable_sol_alloc_free_syscall */
)?;
let transaction_context = &invoke_context.transaction_context;
let instruction_context = transaction_context.get_current_instruction_context()?;
let program_id = instruction_context.get_last_program_key(transaction_context)?;
invoke_context.add_executor(program_id, executor.clone());
executor
}
let cached_executor = invoke_context.tx_executor_cache.borrow().get(program_id);
let executor = if let Some(executor) = cached_executor {
executor
} else {
let executor = create_executor(
first_instruction_account,
program_data_offset,
invoke_context,
use_jit,
false, /* reject_deployment_of_broken_elfs */
// allow _sol_alloc_free syscall for execution
false, /* disable_sol_alloc_free_syscall */
)?;
let transaction_context = &invoke_context.transaction_context;
let instruction_context = transaction_context.get_current_instruction_context()?;
let program_id = instruction_context.get_last_program_key(transaction_context)?;
invoke_context
.tx_executor_cache
.borrow_mut()
.set(*program_id, executor.clone(), false);
executor
};
get_or_create_executor_time.stop();
saturating_add_assign!(
@ -684,7 +687,10 @@ fn process_loader_upgradeable_instruction(
.feature_set
.is_active(&disable_deploy_of_alloc_free_syscall::id()),
)?;
invoke_context.update_executor(&new_program_id, executor);
invoke_context
.tx_executor_cache
.borrow_mut()
.set(new_program_id, executor, true);
let transaction_context = &invoke_context.transaction_context;
let instruction_context = transaction_context.get_current_instruction_context()?;
@ -852,7 +858,10 @@ fn process_loader_upgradeable_instruction(
.feature_set
.is_active(&disable_deploy_of_alloc_free_syscall::id()),
)?;
invoke_context.update_executor(&new_program_id, executor);
invoke_context
.tx_executor_cache
.borrow_mut()
.set(new_program_id, executor, true);
let transaction_context = &invoke_context.transaction_context;
let instruction_context = transaction_context.get_current_instruction_context()?;
@ -1271,7 +1280,10 @@ fn process_loader_instruction(
let instruction_context = transaction_context.get_current_instruction_context()?;
let mut program =
instruction_context.try_borrow_instruction_account(transaction_context, 0)?;
invoke_context.update_executor(program.get_key(), executor);
invoke_context
.tx_executor_cache
.borrow_mut()
.set(*program.get_key(), executor, true);
program.set_executable(true)?;
ic_msg!(invoke_context, "Finalized account {:?}", program.get_key());
}

View File

@ -1416,7 +1416,7 @@ mod tests {
},
assert_matches::assert_matches,
solana_address_lookup_table_program::state::LookupTableMeta,
solana_program_runtime::executor_cache::Executors,
solana_program_runtime::executor_cache::TransactionExecutorCache,
solana_sdk::{
account::{AccountSharedData, WritableAccount},
epoch_schedule::EpochSchedule,
@ -1466,7 +1466,7 @@ mod tests {
executed_units: 0,
accounts_data_len_delta: 0,
},
executors: Rc::new(RefCell::new(Executors::default())),
tx_executor_cache: Rc::new(RefCell::new(TransactionExecutorCache::default())),
}
}

View File

@ -87,7 +87,10 @@ use {
solana_program_runtime::{
accounts_data_meter::MAX_ACCOUNTS_DATA_LEN,
compute_budget::{self, ComputeBudget},
executor_cache::{CachedExecutors, Executors, TransactionExecutor, MAX_CACHED_EXECUTORS},
executor_cache::{
BankExecutorCache, TransactionExecutorCache, TxBankExecutorCacheDiff,
MAX_CACHED_EXECUTORS,
},
invoke_context::{BuiltinProgram, ProcessInstructionWithContext},
log_collector::LogCollector,
sysvar_cache::SysvarCache,
@ -388,7 +391,7 @@ pub struct TransactionExecutionDetails {
pub enum TransactionExecutionResult {
Executed {
details: TransactionExecutionDetails,
executors: Rc<RefCell<Executors>>,
tx_executor_cache: Rc<RefCell<TransactionExecutorCache>>,
},
NotExecuted(TransactionError),
}
@ -830,7 +833,7 @@ impl PartialEq for Bank {
cluster_type: _,
lazy_rent_collection: _,
rewards_pool_pubkeys: _,
cached_executors: _,
executor_cache: _,
transaction_debug_keys: _,
transaction_log_collector_config: _,
transaction_log_collector: _,
@ -1063,7 +1066,7 @@ pub struct Bank {
pub rewards_pool_pubkeys: Arc<HashSet<Pubkey>>,
/// Cached executors
cached_executors: RwLock<CachedExecutors>,
executor_cache: RwLock<BankExecutorCache>,
transaction_debug_keys: Option<Arc<HashSet<Pubkey>>>,
@ -1276,7 +1279,7 @@ impl Bank {
cluster_type: Option::<ClusterType>::default(),
lazy_rent_collection: AtomicBool::default(),
rewards_pool_pubkeys: Arc::<HashSet<Pubkey>>::default(),
cached_executors: RwLock::<CachedExecutors>::default(),
executor_cache: RwLock::<BankExecutorCache>::default(),
transaction_debug_keys: Option::<Arc<HashSet<Pubkey>>>::default(),
transaction_log_collector_config: Arc::<RwLock<TransactionLogCollectorConfig>>::default(
),
@ -1509,15 +1512,15 @@ impl Bank {
"rewards_pool_pubkeys_creation",
);
let (cached_executors, cached_executors_time) = measure!(
let (executor_cache, executor_cache_time) = measure!(
{
let parent_bank_executors = parent.cached_executors.read().unwrap();
RwLock::new(CachedExecutors::new_from_parent_bank_executors(
let parent_bank_executors = parent.executor_cache.read().unwrap();
RwLock::new(BankExecutorCache::new_from_parent_bank_executors(
&parent_bank_executors,
epoch,
))
},
"cached_executors_creation",
"executor_cache_creation",
);
let (transaction_debug_keys, transaction_debug_keys_time) = measure!(
@ -1584,7 +1587,7 @@ impl Bank {
cluster_type: parent.cluster_type,
lazy_rent_collection: AtomicBool::new(parent.lazy_rent_collection.load(Relaxed)),
rewards_pool_pubkeys,
cached_executors,
executor_cache,
transaction_debug_keys,
transaction_log_collector_config,
transaction_log_collector: Arc::new(RwLock::new(TransactionLogCollector::default())),
@ -1781,7 +1784,7 @@ impl Bank {
rewards_pool_pubkeys_time.as_us(),
i64
),
("cached_executors_us", cached_executors_time.as_us(), i64),
("executor_cache_us", executor_cache_time.as_us(), i64),
(
"transaction_debug_keys_us",
transaction_debug_keys_time.as_us(),
@ -1800,7 +1803,7 @@ impl Bank {
);
parent
.cached_executors
.executor_cache
.read()
.unwrap()
.stats
@ -1943,7 +1946,7 @@ impl Bank {
cluster_type: Some(genesis_config.cluster_type),
lazy_rent_collection: new(),
rewards_pool_pubkeys: new(),
cached_executors: RwLock::new(CachedExecutors::new(MAX_CACHED_EXECUTORS, fields.epoch)),
executor_cache: RwLock::new(BankExecutorCache::new(MAX_CACHED_EXECUTORS, fields.epoch)),
transaction_debug_keys: debug_keys,
transaction_log_collector_config: new(),
transaction_log_collector: new(),
@ -3990,7 +3993,10 @@ impl Bank {
}
/// Get any cached executors needed by the transaction
fn get_executors(&self, accounts: &[TransactionAccount]) -> Rc<RefCell<Executors>> {
fn get_tx_executor_cache(
&self,
accounts: &[TransactionAccount],
) -> Rc<RefCell<TransactionExecutorCache>> {
let executable_keys: Vec<_> = accounts
.iter()
.filter_map(|(key, account)| {
@ -4003,58 +4009,46 @@ impl Bank {
.collect();
if executable_keys.is_empty() {
return Rc::new(RefCell::new(Executors::default()));
return Rc::new(RefCell::new(TransactionExecutorCache::default()));
}
let executors = {
let cache = self.cached_executors.read().unwrap();
executable_keys
.into_iter()
.filter_map(|key| {
cache
.get(key)
.map(|executor| (*key, TransactionExecutor::new_cached(executor)))
})
.collect()
let tx_executor_cache = {
let cache = self.executor_cache.read().unwrap();
TransactionExecutorCache::new(
executable_keys
.into_iter()
.filter_map(|key| cache.get(key).map(|executor| (*key, executor))),
)
};
Rc::new(RefCell::new(executors))
Rc::new(RefCell::new(tx_executor_cache))
}
/// Add executors back to the bank's cache if they were missing and not updated
fn store_missing_executors(&self, executors: &RefCell<Executors>) {
self.store_executors_internal(executors, |e| e.is_missing())
fn store_missing_executors(&self, tx_executor_cache: &RefCell<TransactionExecutorCache>) {
tx_executor_cache
.borrow()
.update_global_cache(&self.executor_cache, |difference| {
difference == TxBankExecutorCacheDiff::Inserted
});
}
/// Add updated executors back to the bank's cache
fn store_updated_executors(&self, executors: &RefCell<Executors>) {
self.store_executors_internal(executors, |e| e.is_updated())
}
/// Helper to write a selection of executors to the bank's cache
fn store_executors_internal(
&self,
executors: &RefCell<Executors>,
selector: impl Fn(&TransactionExecutor) -> bool,
) {
let executors = executors.borrow();
let dirty_executors: Vec<_> = executors
.iter()
.filter_map(|(key, executor)| selector(executor).then(|| (key, executor.get())))
.collect();
if !dirty_executors.is_empty() {
self.cached_executors.write().unwrap().put(&dirty_executors);
}
fn store_updated_executors(&self, tx_executor_cache: &RefCell<TransactionExecutorCache>) {
tx_executor_cache
.borrow()
.update_global_cache(&self.executor_cache, |difference| {
difference == TxBankExecutorCacheDiff::Updated
});
}
/// Remove an executor from the bank's cache
fn remove_executor(&self, pubkey: &Pubkey) {
let _ = self.cached_executors.write().unwrap().remove(pubkey);
let _ = self.executor_cache.write().unwrap().remove(pubkey);
}
pub fn clear_executors(&self) {
self.cached_executors.write().unwrap().clear();
self.executor_cache.write().unwrap().clear();
}
/// Execute a transaction using the provided loaded accounts and update
@ -4073,12 +4067,12 @@ impl Bank {
error_counters: &mut TransactionErrorMetrics,
log_messages_bytes_limit: Option<usize>,
) -> TransactionExecutionResult {
let mut get_executors_time = Measure::start("get_executors_time");
let executors = self.get_executors(&loaded_transaction.accounts);
get_executors_time.stop();
let mut get_tx_executor_cache_time = Measure::start("get_tx_executor_cache_time");
let tx_executor_cache = self.get_tx_executor_cache(&loaded_transaction.accounts);
get_tx_executor_cache_time.stop();
saturating_add_assign!(
timings.execute_accessories.get_executors_us,
get_executors_time.as_us()
get_tx_executor_cache_time.as_us()
);
let prev_accounts_data_len = self.load_accounts_data_size();
@ -4136,7 +4130,7 @@ impl Bank {
&mut transaction_context,
self.rent_collector.rent,
log_collector.clone(),
executors.clone(),
tx_executor_cache.clone(),
self.feature_set.clone(),
compute_budget,
timings,
@ -4154,7 +4148,7 @@ impl Bank {
);
let mut store_missing_executors_time = Measure::start("store_missing_executors_time");
self.store_missing_executors(&executors);
self.store_missing_executors(&tx_executor_cache);
store_missing_executors_time.stop();
saturating_add_assign!(
timings.execute_accessories.update_executors_us,
@ -4248,7 +4242,7 @@ impl Bank {
executed_units,
accounts_data_len_delta,
},
executors,
tx_executor_cache,
}
}
@ -4827,9 +4821,13 @@ impl Bank {
let mut store_updated_executors_time = Measure::start("store_updated_executors_time");
for execution_result in &execution_results {
if let TransactionExecutionResult::Executed { details, executors } = execution_result {
if let TransactionExecutionResult::Executed {
details,
tx_executor_cache,
} = execution_result
{
if details.status.is_ok() {
self.store_updated_executors(executors);
self.store_updated_executors(tx_executor_cache);
}
}
}
@ -7939,7 +7937,7 @@ pub(crate) mod tests {
executed_units: 0,
accounts_data_len_delta: 0,
},
executors: Rc::new(RefCell::new(Executors::default())),
tx_executor_cache: Rc::new(RefCell::new(TransactionExecutorCache::default())),
}
}
@ -15102,55 +15100,53 @@ pub(crate) mod tests {
];
// don't do any work if not dirty
let mut executors = Executors::default();
executors.insert(key1, TransactionExecutor::new_cached(executor.clone()));
executors.insert(key2, TransactionExecutor::new_cached(executor.clone()));
executors.insert(key3, TransactionExecutor::new_cached(executor.clone()));
executors.insert(key4, TransactionExecutor::new_cached(executor.clone()));
let executors =
TransactionExecutorCache::new((0..4).map(|i| (accounts[i].0, executor.clone())));
let executors = Rc::new(RefCell::new(executors));
bank.store_missing_executors(&executors);
bank.store_updated_executors(&executors);
let executors = bank.get_executors(accounts);
assert_eq!(executors.borrow().len(), 0);
let stored_executors = bank.get_tx_executor_cache(accounts);
assert_eq!(stored_executors.borrow().executors.len(), 0);
// do work
let mut executors = Executors::default();
executors.insert(key1, TransactionExecutor::new_miss(executor.clone()));
executors.insert(key2, TransactionExecutor::new_miss(executor.clone()));
executors.insert(key3, TransactionExecutor::new_updated(executor.clone()));
executors.insert(key4, TransactionExecutor::new_miss(executor.clone()));
let mut executors =
TransactionExecutorCache::new((2..3).map(|i| (accounts[i].0, executor.clone())));
executors.set(key1, executor.clone(), false);
executors.set(key2, executor.clone(), false);
executors.set(key3, executor.clone(), true);
executors.set(key4, executor.clone(), false);
let executors = Rc::new(RefCell::new(executors));
// store the new_miss
// store Missing
bank.store_missing_executors(&executors);
let stored_executors = bank.get_executors(accounts);
assert_eq!(stored_executors.borrow().len(), 2);
assert!(stored_executors.borrow().contains_key(&key1));
assert!(stored_executors.borrow().contains_key(&key2));
let stored_executors = bank.get_tx_executor_cache(accounts);
assert_eq!(stored_executors.borrow().executors.len(), 2);
assert!(stored_executors.borrow().executors.contains_key(&key1));
assert!(stored_executors.borrow().executors.contains_key(&key2));
// store the new_updated
// store Updated
bank.store_updated_executors(&executors);
let stored_executors = bank.get_executors(accounts);
assert_eq!(stored_executors.borrow().len(), 3);
assert!(stored_executors.borrow().contains_key(&key1));
assert!(stored_executors.borrow().contains_key(&key2));
assert!(stored_executors.borrow().contains_key(&key3));
let stored_executors = bank.get_tx_executor_cache(accounts);
assert_eq!(stored_executors.borrow().executors.len(), 3);
assert!(stored_executors.borrow().executors.contains_key(&key1));
assert!(stored_executors.borrow().executors.contains_key(&key2));
assert!(stored_executors.borrow().executors.contains_key(&key3));
// Check inheritance
let bank = Bank::new_from_parent(&Arc::new(bank), &solana_sdk::pubkey::new_rand(), 1);
let executors = bank.get_executors(accounts);
assert_eq!(executors.borrow().len(), 3);
assert!(executors.borrow().contains_key(&key1));
assert!(executors.borrow().contains_key(&key2));
assert!(executors.borrow().contains_key(&key3));
let stored_executors = bank.get_tx_executor_cache(accounts);
assert_eq!(stored_executors.borrow().executors.len(), 3);
assert!(stored_executors.borrow().executors.contains_key(&key1));
assert!(stored_executors.borrow().executors.contains_key(&key2));
assert!(stored_executors.borrow().executors.contains_key(&key3));
// Remove all
bank.remove_executor(&key1);
bank.remove_executor(&key2);
bank.remove_executor(&key3);
bank.remove_executor(&key4);
let executors = bank.get_executors(accounts);
assert_eq!(executors.borrow().len(), 0);
let stored_executors = bank.get_tx_executor_cache(accounts);
assert_eq!(stored_executors.borrow().executors.len(), 0);
}
#[test]
@ -15175,37 +15171,37 @@ pub(crate) mod tests {
];
// add one to root bank
let mut executors = Executors::default();
executors.insert(key1, TransactionExecutor::new_miss(executor.clone()));
let mut executors = TransactionExecutorCache::default();
executors.set(key1, executor.clone(), false);
let executors = Rc::new(RefCell::new(executors));
root.store_missing_executors(&executors);
let executors = root.get_executors(accounts);
assert_eq!(executors.borrow().len(), 1);
let executors = root.get_tx_executor_cache(accounts);
assert_eq!(executors.borrow().executors.len(), 1);
let fork1 = Bank::new_from_parent(&root, &Pubkey::default(), 1);
let fork2 = Bank::new_from_parent(&root, &Pubkey::default(), 2);
let executors = fork1.get_executors(accounts);
assert_eq!(executors.borrow().len(), 1);
let executors = fork2.get_executors(accounts);
assert_eq!(executors.borrow().len(), 1);
let executors = fork1.get_tx_executor_cache(accounts);
assert_eq!(executors.borrow().executors.len(), 1);
let executors = fork2.get_tx_executor_cache(accounts);
assert_eq!(executors.borrow().executors.len(), 1);
let mut executors = Executors::default();
executors.insert(key2, TransactionExecutor::new_miss(executor.clone()));
let mut executors = TransactionExecutorCache::default();
executors.set(key2, executor.clone(), false);
let executors = Rc::new(RefCell::new(executors));
fork1.store_missing_executors(&executors);
let executors = fork1.get_executors(accounts);
assert_eq!(executors.borrow().len(), 2);
let executors = fork2.get_executors(accounts);
assert_eq!(executors.borrow().len(), 1);
let executors = fork1.get_tx_executor_cache(accounts);
assert_eq!(executors.borrow().executors.len(), 2);
let executors = fork2.get_tx_executor_cache(accounts);
assert_eq!(executors.borrow().executors.len(), 1);
fork1.remove_executor(&key1);
let executors = fork1.get_executors(accounts);
assert_eq!(executors.borrow().len(), 1);
let executors = fork2.get_executors(accounts);
assert_eq!(executors.borrow().len(), 1);
let executors = fork1.get_tx_executor_cache(accounts);
assert_eq!(executors.borrow().executors.len(), 1);
let executors = fork2.get_tx_executor_cache(accounts);
assert_eq!(executors.borrow().executors.len(), 1);
}
#[test]

View File

@ -3,7 +3,7 @@ use {
solana_measure::measure::Measure,
solana_program_runtime::{
compute_budget::ComputeBudget,
executor_cache::Executors,
executor_cache::TransactionExecutorCache,
invoke_context::{BuiltinProgram, InvokeContext},
log_collector::LogCollector,
sysvar_cache::SysvarCache,
@ -57,7 +57,7 @@ impl MessageProcessor {
transaction_context: &mut TransactionContext,
rent: Rent,
log_collector: Option<Rc<RefCell<LogCollector>>>,
executors: Rc<RefCell<Executors>>,
tx_executor_cache: Rc<RefCell<TransactionExecutorCache>>,
feature_set: Arc<FeatureSet>,
compute_budget: ComputeBudget,
timings: &mut ExecuteTimings,
@ -74,7 +74,7 @@ impl MessageProcessor {
Cow::Borrowed(sysvar_cache),
log_collector,
compute_budget,
executors,
tx_executor_cache,
feature_set,
blockhash,
lamports_per_signature,
@ -275,7 +275,7 @@ mod tests {
let mut transaction_context =
TransactionContext::new(accounts, Some(Rent::default()), 1, 3);
let program_indices = vec![vec![2]];
let executors = Rc::new(RefCell::new(Executors::default()));
let tx_executor_cache = Rc::new(RefCell::new(TransactionExecutorCache::default()));
let account_keys = (0..transaction_context.get_number_of_accounts())
.map(|index| {
*transaction_context
@ -311,7 +311,7 @@ mod tests {
&mut transaction_context,
rent_collector.rent,
None,
executors.clone(),
tx_executor_cache.clone(),
Arc::new(FeatureSet::all_enabled()),
ComputeBudget::default(),
&mut ExecuteTimings::default(),
@ -361,7 +361,7 @@ mod tests {
&mut transaction_context,
rent_collector.rent,
None,
executors.clone(),
tx_executor_cache.clone(),
Arc::new(FeatureSet::all_enabled()),
ComputeBudget::default(),
&mut ExecuteTimings::default(),
@ -401,7 +401,7 @@ mod tests {
&mut transaction_context,
rent_collector.rent,
None,
executors,
tx_executor_cache,
Arc::new(FeatureSet::all_enabled()),
ComputeBudget::default(),
&mut ExecuteTimings::default(),
@ -505,7 +505,7 @@ mod tests {
let mut transaction_context =
TransactionContext::new(accounts, Some(Rent::default()), 1, 3);
let program_indices = vec![vec![2]];
let executors = Rc::new(RefCell::new(Executors::default()));
let tx_executor_cache = Rc::new(RefCell::new(TransactionExecutorCache::default()));
let account_metas = vec![
AccountMeta::new(
*transaction_context.get_key_of_account_at_index(0).unwrap(),
@ -538,7 +538,7 @@ mod tests {
&mut transaction_context,
rent_collector.rent,
None,
executors.clone(),
tx_executor_cache.clone(),
Arc::new(FeatureSet::all_enabled()),
ComputeBudget::default(),
&mut ExecuteTimings::default(),
@ -572,7 +572,7 @@ mod tests {
&mut transaction_context,
rent_collector.rent,
None,
executors.clone(),
tx_executor_cache.clone(),
Arc::new(FeatureSet::all_enabled()),
ComputeBudget::default(),
&mut ExecuteTimings::default(),
@ -603,7 +603,7 @@ mod tests {
&mut transaction_context,
rent_collector.rent,
None,
executors,
tx_executor_cache,
Arc::new(FeatureSet::all_enabled()),
ComputeBudget::default(),
&mut ExecuteTimings::default(),
@ -683,7 +683,7 @@ mod tests {
&mut transaction_context,
RentCollector::default().rent,
None,
Rc::new(RefCell::new(Executors::default())),
Rc::new(RefCell::new(TransactionExecutorCache::default())),
Arc::new(FeatureSet::all_enabled()),
ComputeBudget::default(),
&mut ExecuteTimings::default(),