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:
parent
9fe46bb038
commit
664339e239
|
@ -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()
|
||||
);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
|
|
@ -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())),
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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]
|
||||
|
|
|
@ -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(),
|
||||
|
|
Loading…
Reference in New Issue