Revert "Count compute units even when transaction errors (backport #22059) (#22154)" (#22175)

This reverts commit 401c542d2a.
This commit is contained in:
carllin
2021-12-30 02:39:25 -05:00
committed by GitHub
parent 6a556c5adb
commit 7efd0391e9
13 changed files with 93 additions and 305 deletions

View File

@ -26,7 +26,7 @@ use {
TransactionExecutionResult, TransactionExecutionResult,
}, },
bank_utils, bank_utils,
cost_model::{CostModel, ExecutionCost}, cost_model::CostModel,
transaction_batch::TransactionBatch, transaction_batch::TransactionBatch,
vote_sender_types::ReplayVoteSender, vote_sender_types::ReplayVoteSender,
}, },
@ -984,20 +984,14 @@ impl BankingStage {
let tx_costs = qos_service.compute_transaction_costs(txs.iter()); let tx_costs = qos_service.compute_transaction_costs(txs.iter());
let transactions_qos_results = let transactions_qos_results =
qos_service.select_transactions_per_cost(txs.iter(), tx_costs.into_iter(), bank); qos_service.select_transactions_per_cost(txs.iter(), tx_costs.iter(), bank);
// Only lock accounts for those transactions are selected for the block; // Only lock accounts for those transactions are selected for the block;
// Once accounts are locked, other threads cannot encode transactions that will modify the // Once accounts are locked, other threads cannot encode transactions that will modify the
// same account state // same account state
let mut lock_time = Measure::start("lock_time"); let mut lock_time = Measure::start("lock_time");
let batch = bank.prepare_sanitized_batch_with_results( let batch =
txs, bank.prepare_sanitized_batch_with_results(txs, transactions_qos_results.into_iter());
transactions_qos_results
.into_iter()
.map(|transaction_cost_result| {
transaction_cost_result.map(|transaction_cost| transaction_cost.execution_cost)
}),
);
lock_time.stop(); lock_time.stop();
// retryable_txs includes AccountInUse, WouldExceedMaxBlockCostLimit and // retryable_txs includes AccountInUse, WouldExceedMaxBlockCostLimit and
@ -1094,9 +1088,9 @@ impl BankingStage {
fn prepare_filter_for_pending_transactions( fn prepare_filter_for_pending_transactions(
transactions_len: usize, transactions_len: usize,
pending_tx_indexes: &[usize], pending_tx_indexes: &[usize],
) -> Vec<transaction::Result<ExecutionCost>> { ) -> Vec<transaction::Result<()>> {
let mut mask = vec![Err(TransactionError::BlockhashNotFound); transactions_len]; let mut mask = vec![Err(TransactionError::BlockhashNotFound); transactions_len];
pending_tx_indexes.iter().for_each(|x| mask[*x] = Ok(0)); pending_tx_indexes.iter().for_each(|x| mask[*x] = Ok(()));
mask mask
} }
@ -1187,7 +1181,7 @@ impl BankingStage {
let results = bank.check_transactions( let results = bank.check_transactions(
transactions, transactions,
filter.into_iter(), &filter,
(MAX_PROCESSING_AGE) (MAX_PROCESSING_AGE)
.saturating_sub(max_tx_fwd_delay) .saturating_sub(max_tx_fwd_delay)
.saturating_sub(FORWARD_TRANSACTIONS_TO_LEADER_AT_SLOT_OFFSET as usize), .saturating_sub(FORWARD_TRANSACTIONS_TO_LEADER_AT_SLOT_OFFSET as usize),
@ -2099,20 +2093,20 @@ mod tests {
vec![ vec![
Err(TransactionError::BlockhashNotFound), Err(TransactionError::BlockhashNotFound),
Err(TransactionError::BlockhashNotFound), Err(TransactionError::BlockhashNotFound),
Ok(0), Ok(()),
Err(TransactionError::BlockhashNotFound), Err(TransactionError::BlockhashNotFound),
Ok(0), Ok(()),
Ok(0) Ok(())
] ]
); );
assert_eq!( assert_eq!(
BankingStage::prepare_filter_for_pending_transactions(6, &[0, 2, 3]), BankingStage::prepare_filter_for_pending_transactions(6, &[0, 2, 3]),
vec![ vec![
Ok(0), Ok(()),
Err(TransactionError::BlockhashNotFound), Err(TransactionError::BlockhashNotFound),
Ok(0), Ok(()),
Ok(0), Ok(()),
Err(TransactionError::BlockhashNotFound), Err(TransactionError::BlockhashNotFound),
Err(TransactionError::BlockhashNotFound), Err(TransactionError::BlockhashNotFound),
] ]
@ -2126,10 +2120,10 @@ mod tests {
&[ &[
(Err(TransactionError::BlockhashNotFound), None), (Err(TransactionError::BlockhashNotFound), None),
(Err(TransactionError::BlockhashNotFound), None), (Err(TransactionError::BlockhashNotFound), None),
(Ok(0), None), (Ok(()), None),
(Err(TransactionError::BlockhashNotFound), None), (Err(TransactionError::BlockhashNotFound), None),
(Ok(0), None), (Ok(()), None),
(Ok(0), None), (Ok(()), None),
], ],
&[2, 4, 5, 9, 11, 13] &[2, 4, 5, 9, 11, 13]
), ),
@ -2139,12 +2133,12 @@ mod tests {
assert_eq!( assert_eq!(
BankingStage::filter_valid_transaction_indexes( BankingStage::filter_valid_transaction_indexes(
&[ &[
(Ok(0), None), (Ok(()), None),
(Err(TransactionError::BlockhashNotFound), None), (Err(TransactionError::BlockhashNotFound), None),
(Err(TransactionError::BlockhashNotFound), None), (Err(TransactionError::BlockhashNotFound), None),
(Ok(0), None), (Ok(()), None),
(Ok(0), None), (Ok(()), None),
(Ok(0), None), (Ok(()), None),
], ],
&[1, 6, 7, 9, 31, 43] &[1, 6, 7, 9, 31, 43]
), ),

View File

@ -246,7 +246,6 @@ mod tests {
ProgramTiming { ProgramTiming {
accumulated_us, accumulated_us,
accumulated_units, accumulated_units,
current_cost_model_estimated_units: 0,
count, count,
}, },
); );
@ -282,7 +281,6 @@ mod tests {
ProgramTiming { ProgramTiming {
accumulated_us, accumulated_us,
accumulated_units, accumulated_units,
current_cost_model_estimated_units: 0,
count, count,
}, },
); );

View File

@ -108,18 +108,18 @@ impl QosService {
pub fn select_transactions_per_cost<'a>( pub fn select_transactions_per_cost<'a>(
&self, &self,
transactions: impl Iterator<Item = &'a SanitizedTransaction>, transactions: impl Iterator<Item = &'a SanitizedTransaction>,
transactions_costs: impl Iterator<Item = TransactionCost>, transactions_costs: impl Iterator<Item = &'a TransactionCost>,
bank: &Arc<Bank>, bank: &Arc<Bank>,
) -> Vec<transaction::Result<TransactionCost>> { ) -> Vec<transaction::Result<()>> {
let mut cost_tracking_time = Measure::start("cost_tracking_time"); let mut cost_tracking_time = Measure::start("cost_tracking_time");
let mut cost_tracker = bank.write_cost_tracker().unwrap(); let mut cost_tracker = bank.write_cost_tracker().unwrap();
let select_results = transactions let select_results = transactions
.zip(transactions_costs) .zip(transactions_costs)
.map(|(tx, cost)| match cost_tracker.try_add(tx, &cost) { .map(|(tx, cost)| match cost_tracker.try_add(tx, cost) {
Ok(current_block_cost) => { Ok(current_block_cost) => {
debug!("slot {:?}, transaction {:?}, cost {:?}, fit into current block, current block cost {}", bank.slot(), tx, cost, current_block_cost); debug!("slot {:?}, transaction {:?}, cost {:?}, fit into current block, current block cost {}", bank.slot(), tx, cost, current_block_cost);
self.metrics.selected_txs_count.fetch_add(1, Ordering::Relaxed); self.metrics.selected_txs_count.fetch_add(1, Ordering::Relaxed);
Ok(cost) Ok(())
}, },
Err(e) => { Err(e) => {
debug!("slot {:?}, transaction {:?}, cost {:?}, not fit into current block, '{:?}'", bank.slot(), tx, cost, e); debug!("slot {:?}, transaction {:?}, cost {:?}, not fit into current block, '{:?}'", bank.slot(), tx, cost, e);
@ -304,8 +304,7 @@ mod tests {
bank.write_cost_tracker() bank.write_cost_tracker()
.unwrap() .unwrap()
.set_limits(cost_limit, cost_limit); .set_limits(cost_limit, cost_limit);
let results = let results = qos_service.select_transactions_per_cost(txs.iter(), txs_costs.iter(), &bank);
qos_service.select_transactions_per_cost(txs.iter(), txs_costs.into_iter(), &bank);
// verify that first transfer tx and all votes are allowed // verify that first transfer tx and all votes are allowed
assert_eq!(results.len(), txs.len()); assert_eq!(results.len(), txs.len());

View File

@ -102,10 +102,10 @@ thread_local!(static PAR_THREAD_POOL: RefCell<ThreadPool> = RefCell::new(rayon::
.unwrap()) .unwrap())
); );
fn first_err<T>(results: &[Result<T>]) -> Result<()> { fn first_err(results: &[Result<()>]) -> Result<()> {
for r in results { for r in results {
if let Err(e) = r { if r.is_err() {
return Err(e.clone()); return r.clone();
} }
} }
Ok(()) Ok(())

View File

@ -30,12 +30,6 @@ pub type TransactionAccountRefCells = Vec<TransactionAccountRefCell>;
pub type ProcessInstructionWithContext = pub type ProcessInstructionWithContext =
fn(usize, &[u8], &mut InvokeContext) -> Result<(), InstructionError>; fn(usize, &[u8], &mut InvokeContext) -> Result<(), InstructionError>;
#[derive(Debug, PartialEq)]
pub struct ProcessInstructionResult {
pub compute_units_consumed: u64,
pub result: Result<(), InstructionError>,
}
#[derive(Clone)] #[derive(Clone)]
pub struct BuiltinProgram { pub struct BuiltinProgram {
pub program_id: Pubkey, pub program_id: Pubkey,
@ -505,8 +499,7 @@ impl<'a> InvokeContext<'a> {
&program_indices, &program_indices,
&account_indices, &account_indices,
&caller_write_privileges, &caller_write_privileges,
) )?;
.result?;
// Verify the called program has not misbehaved // Verify the called program has not misbehaved
let do_support_realloc = self.feature_set.is_active(&do_support_realloc::id()); let do_support_realloc = self.feature_set.is_active(&do_support_realloc::id());
@ -647,48 +640,37 @@ impl<'a> InvokeContext<'a> {
program_indices: &[usize], program_indices: &[usize],
account_indices: &[usize], account_indices: &[usize],
caller_write_privileges: &[bool], caller_write_privileges: &[bool],
) -> ProcessInstructionResult { ) -> Result<u64, InstructionError> {
let is_lowest_invocation_level = self.invoke_stack.is_empty(); let is_lowest_invocation_level = self.invoke_stack.is_empty();
if !is_lowest_invocation_level { if !is_lowest_invocation_level {
// Verify the calling program hasn't misbehaved // Verify the calling program hasn't misbehaved
let result = self.verify_and_update(instruction, account_indices, caller_write_privileges)?;
self.verify_and_update(instruction, account_indices, caller_write_privileges);
if result.is_err() {
return ProcessInstructionResult {
compute_units_consumed: 0,
result,
};
}
} }
let mut compute_units_consumed = 0;
let result = self let result = self
.push(message, instruction, program_indices, account_indices) .push(message, instruction, program_indices, account_indices)
.and_then(|_| { .and_then(|_| {
self.return_data = (*instruction.program_id(&message.account_keys), Vec::new()); self.return_data = (*instruction.program_id(&message.account_keys), Vec::new());
let pre_remaining_units = self.compute_meter.borrow().get_remaining(); let pre_remaining_units = self.compute_meter.borrow().get_remaining();
let execution_result = self.process_executable_chain(&instruction.data); self.process_executable_chain(&instruction.data)?;
let post_remaining_units = self.compute_meter.borrow().get_remaining(); let post_remaining_units = self.compute_meter.borrow().get_remaining();
compute_units_consumed = pre_remaining_units.saturating_sub(post_remaining_units);
execution_result?;
// Verify the called program has not misbehaved // Verify the called program has not misbehaved
if is_lowest_invocation_level { if is_lowest_invocation_level {
self.verify(message, instruction, program_indices) self.verify(message, instruction, program_indices)?;
} else { } else {
let write_privileges: Vec<bool> = (0..message.account_keys.len()) let write_privileges: Vec<bool> = (0..message.account_keys.len())
.map(|i| message.is_writable(i)) .map(|i| message.is_writable(i))
.collect(); .collect();
self.verify_and_update(instruction, account_indices, &write_privileges) self.verify_and_update(instruction, account_indices, &write_privileges)?;
} }
Ok(pre_remaining_units.saturating_sub(post_remaining_units))
}); });
// Pop the invoke_stack to restore previous state // Pop the invoke_stack to restore previous state
self.pop(); self.pop();
ProcessInstructionResult { result
compute_units_consumed,
result,
}
} }
/// Calls the instruction's program entrypoint method /// Calls the instruction's program entrypoint method
@ -1001,10 +983,6 @@ mod tests {
ModifyOwned, ModifyOwned,
ModifyNotOwned, ModifyNotOwned,
ModifyReadonly, ModifyReadonly,
ConsumeComputeUnits {
compute_units_consumed: u64,
desired_result: Result<(), InstructionError>,
},
} }
#[test] #[test]
@ -1074,17 +1052,6 @@ mod tests {
.try_account_ref_mut()? .try_account_ref_mut()?
.data_as_mut_slice()[0] = 1 .data_as_mut_slice()[0] = 1
} }
MockInstruction::ConsumeComputeUnits {
compute_units_consumed,
desired_result,
} => {
invoke_context
.get_compute_meter()
.borrow_mut()
.consume(compute_units_consumed)
.unwrap();
return desired_result;
}
} }
} else { } else {
return Err(InstructionError::InvalidInstructionData); return Err(InstructionError::InvalidInstructionData);
@ -1293,15 +1260,13 @@ mod tests {
.collect::<Vec<bool>>(); .collect::<Vec<bool>>();
accounts[0].1.borrow_mut().data_as_mut_slice()[0] = 1; accounts[0].1.borrow_mut().data_as_mut_slice()[0] = 1;
assert_eq!( assert_eq!(
invoke_context invoke_context.process_instruction(
.process_instruction( &message,
&message, &message.instructions[0],
&message.instructions[0], &program_indices[1..],
&program_indices[1..], &account_indices,
&account_indices, &caller_write_privileges,
&caller_write_privileges, ),
)
.result,
Err(InstructionError::ExternalAccountDataModified) Err(InstructionError::ExternalAccountDataModified)
); );
accounts[0].1.borrow_mut().data_as_mut_slice()[0] = 0; accounts[0].1.borrow_mut().data_as_mut_slice()[0] = 0;
@ -1309,15 +1274,13 @@ mod tests {
// readonly account modified by the invoker // readonly account modified by the invoker
accounts[2].1.borrow_mut().data_as_mut_slice()[0] = 1; accounts[2].1.borrow_mut().data_as_mut_slice()[0] = 1;
assert_eq!( assert_eq!(
invoke_context invoke_context.process_instruction(
.process_instruction( &message,
&message, &message.instructions[0],
&message.instructions[0], &program_indices[1..],
&program_indices[1..], &account_indices,
&account_indices, &caller_write_privileges,
&caller_write_privileges, ),
)
.result,
Err(InstructionError::ReadonlyDataModified) Err(InstructionError::ReadonlyDataModified)
); );
accounts[2].1.borrow_mut().data_as_mut_slice()[0] = 0; accounts[2].1.borrow_mut().data_as_mut_slice()[0] = 0;
@ -1325,33 +1288,15 @@ mod tests {
invoke_context.pop(); invoke_context.pop();
let cases = vec![ let cases = vec![
( (MockInstruction::NoopSuccess, Ok(0)),
MockInstruction::NoopSuccess,
ProcessInstructionResult {
result: Ok(()),
compute_units_consumed: 0,
},
),
( (
MockInstruction::NoopFail, MockInstruction::NoopFail,
ProcessInstructionResult { Err(InstructionError::GenericError),
result: Err(InstructionError::GenericError),
compute_units_consumed: 0,
},
),
(
MockInstruction::ModifyOwned,
ProcessInstructionResult {
result: Ok(()),
compute_units_consumed: 0,
},
), ),
(MockInstruction::ModifyOwned, Ok(0)),
( (
MockInstruction::ModifyNotOwned, MockInstruction::ModifyNotOwned,
ProcessInstructionResult { Err(InstructionError::ExternalAccountDataModified),
result: Err(InstructionError::ExternalAccountDataModified),
compute_units_consumed: 0,
},
), ),
]; ];
for case in cases { for case in cases {
@ -1552,92 +1497,4 @@ mod tests {
); );
invoke_context.pop(); invoke_context.pop();
} }
#[test]
fn test_process_instruction_compute_budget() {
let caller_program_id = solana_sdk::pubkey::new_rand();
let callee_program_id = solana_sdk::pubkey::new_rand();
let owned_account = AccountSharedData::new(42, 1, &callee_program_id);
let not_owned_account = AccountSharedData::new(84, 1, &solana_sdk::pubkey::new_rand());
let readonly_account = AccountSharedData::new(168, 1, &solana_sdk::pubkey::new_rand());
let loader_account = AccountSharedData::new(0, 0, &native_loader::id());
let mut program_account = AccountSharedData::new(1, 0, &native_loader::id());
program_account.set_executable(true);
let accounts = vec![
(
solana_sdk::pubkey::new_rand(),
Rc::new(RefCell::new(owned_account)),
),
(
solana_sdk::pubkey::new_rand(),
Rc::new(RefCell::new(not_owned_account)),
),
(
solana_sdk::pubkey::new_rand(),
Rc::new(RefCell::new(readonly_account)),
),
(caller_program_id, Rc::new(RefCell::new(loader_account))),
(callee_program_id, Rc::new(RefCell::new(program_account))),
];
let account_indices = [0, 1, 2];
let program_indices = [3, 4];
let metas = vec![
AccountMeta::new(accounts[0].0, false),
AccountMeta::new(accounts[1].0, false),
AccountMeta::new_readonly(accounts[2].0, false),
];
let builtin_programs = &[BuiltinProgram {
program_id: callee_program_id,
process_instruction: mock_process_instruction,
}];
let mut invoke_context = InvokeContext::new_mock(&accounts, builtin_programs);
let compute_units_consumed = 10;
let desired_results = vec![Ok(()), Err(InstructionError::GenericError)];
for desired_result in desired_results {
let caller_instruction =
CompiledInstruction::new(program_indices[0] as u8, &(), vec![0, 1, 2, 3, 4]);
let callee_instruction = Instruction::new_with_bincode(
callee_program_id,
&MockInstruction::ConsumeComputeUnits {
compute_units_consumed,
desired_result: desired_result.clone(),
},
metas.clone(),
);
let message = Message::new(&[callee_instruction.clone()], None);
invoke_context
.push(&message, &caller_instruction, &program_indices[..1], &[])
.unwrap();
let caller_write_privileges = message
.account_keys
.iter()
.enumerate()
.map(|(i, _)| message.is_writable(i))
.collect::<Vec<bool>>();
let result = invoke_context.process_instruction(
&message,
&message.instructions[0],
&program_indices[1..],
&account_indices,
&caller_write_privileges,
);
// Because the instruction had compute cost > 0, then regardless of the execution result,
// the number of compute units consumed should be a non-default which is something greater
// than zero.
assert!(result.compute_units_consumed > 0);
assert_eq!(
result,
ProcessInstructionResult {
compute_units_consumed,
result: desired_result,
}
);
}
}
} }

View File

@ -4,7 +4,6 @@ use {solana_sdk::pubkey::Pubkey, std::collections::HashMap};
pub struct ProgramTiming { pub struct ProgramTiming {
pub accumulated_us: u64, pub accumulated_us: u64,
pub accumulated_units: u64, pub accumulated_units: u64,
pub current_cost_model_estimated_units: u64,
pub count: u32, pub count: u32,
} }
@ -47,24 +46,10 @@ impl ExecuteDetailsTimings {
program_timing.count = program_timing.count.saturating_add(other.count); program_timing.count = program_timing.count.saturating_add(other.count);
} }
} }
pub fn accumulate_program( pub fn accumulate_program(&mut self, program_id: &Pubkey, us: u64, units: u64) {
&mut self,
program_id: &Pubkey,
us: u64,
actual_compute_units_consumed: u64,
estimated_execution_cost: u64,
is_error: bool,
) {
let program_timing = self.per_program_timings.entry(*program_id).or_default(); let program_timing = self.per_program_timings.entry(*program_id).or_default();
program_timing.accumulated_us = program_timing.accumulated_us.saturating_add(us); program_timing.accumulated_us = program_timing.accumulated_us.saturating_add(us);
let compute_units_update = if is_error { program_timing.accumulated_units = program_timing.accumulated_units.saturating_add(units);
std::cmp::max(actual_compute_units_consumed, estimated_execution_cost)
} else {
actual_compute_units_consumed
};
program_timing.accumulated_units = program_timing
.accumulated_units
.saturating_add(compute_units_update);
program_timing.count = program_timing.count.saturating_add(1); program_timing.count = program_timing.count.saturating_add(1);
} }
} }

View File

@ -322,7 +322,6 @@ impl solana_sdk::program_stubs::SyscallStubs for SyscallStubs {
&account_indices, &account_indices,
&caller_privileges, &caller_privileges,
) )
.result
.map_err(|err| ProgramError::try_from(err).unwrap_or_else(|err| panic!("{}", err)))?; .map_err(|err| ProgramError::try_from(err).unwrap_or_else(|err| panic!("{}", err)))?;
// Copy writeable account modifications back into the caller's AccountInfos // Copy writeable account modifications back into the caller's AccountInfos

View File

@ -2372,7 +2372,6 @@ fn call<'a, 'b: 'a>(
&account_indices, &account_indices,
&caller_write_privileges, &caller_write_privileges,
) )
.result
.map_err(SyscallError::InstructionError)?; .map_err(SyscallError::InstructionError)?;
// Copy results back to caller // Copy results back to caller

View File

@ -13,7 +13,6 @@ use {
TransactionExecutionResult, TransactionExecutionResult,
}, },
blockhash_queue::BlockhashQueue, blockhash_queue::BlockhashQueue,
cost_model::ExecutionCost,
rent_collector::RentCollector, rent_collector::RentCollector,
system_instruction_processor::{get_system_account_kind, SystemAccountKind}, system_instruction_processor::{get_system_account_kind, SystemAccountKind},
}, },
@ -115,7 +114,6 @@ pub struct LoadedTransaction {
pub program_indices: TransactionProgramIndices, pub program_indices: TransactionProgramIndices,
pub rent: TransactionRent, pub rent: TransactionRent,
pub rent_debits: RentDebits, pub rent_debits: RentDebits,
pub estimated_execution_cost: ExecutionCost,
} }
pub type TransactionLoadResult = (Result<LoadedTransaction>, Option<NonceFull>); pub type TransactionLoadResult = (Result<LoadedTransaction>, Option<NonceFull>);
@ -233,7 +231,6 @@ impl Accounts {
error_counters: &mut ErrorCounters, error_counters: &mut ErrorCounters,
rent_collector: &RentCollector, rent_collector: &RentCollector,
feature_set: &FeatureSet, feature_set: &FeatureSet,
estimated_execution_cost: ExecutionCost,
) -> Result<LoadedTransaction> { ) -> Result<LoadedTransaction> {
// Copy all the accounts // Copy all the accounts
let message = tx.message(); let message = tx.message();
@ -377,7 +374,6 @@ impl Accounts {
program_indices, program_indices,
rent: tx_rent, rent: tx_rent,
rent_debits, rent_debits,
estimated_execution_cost,
}) })
} else { } else {
error_counters.account_not_found += 1; error_counters.account_not_found += 1;
@ -471,7 +467,7 @@ impl Accounts {
txs.iter() txs.iter()
.zip(lock_results) .zip(lock_results)
.map(|etx| match etx { .map(|etx| match etx {
(tx, (Ok(execution_cost), nonce)) => { (tx, (Ok(()), nonce)) => {
let lamports_per_signature = nonce let lamports_per_signature = nonce
.as_ref() .as_ref()
.map(|nonce| nonce.lamports_per_signature()) .map(|nonce| nonce.lamports_per_signature())
@ -491,7 +487,6 @@ impl Accounts {
error_counters, error_counters,
rent_collector, rent_collector,
feature_set, feature_set,
execution_cost,
) { ) {
Ok(loaded_transaction) => loaded_transaction, Ok(loaded_transaction) => loaded_transaction,
Err(e) => return (Err(e), None), Err(e) => return (Err(e), None),
@ -957,14 +952,11 @@ impl Accounts {
pub fn lock_accounts<'a>( pub fn lock_accounts<'a>(
&self, &self,
txs: impl Iterator<Item = &'a SanitizedTransaction>, txs: impl Iterator<Item = &'a SanitizedTransaction>,
) -> Vec<Result<ExecutionCost>> { ) -> Vec<Result<()>> {
let keys: Vec<_> = txs.map(|tx| tx.get_account_locks()).collect(); let keys: Vec<_> = txs.map(|tx| tx.get_account_locks()).collect();
let account_locks = &mut self.account_locks.lock().unwrap(); let account_locks = &mut self.account_locks.lock().unwrap();
keys.into_iter() keys.into_iter()
.map(|keys| { .map(|keys| self.lock_account(account_locks, keys.writable, keys.readonly))
self.lock_account(account_locks, keys.writable, keys.readonly)
.map(|_| 0)
})
.collect() .collect()
} }
@ -973,12 +965,12 @@ impl Accounts {
pub fn lock_accounts_with_results<'a>( pub fn lock_accounts_with_results<'a>(
&self, &self,
txs: impl Iterator<Item = &'a SanitizedTransaction>, txs: impl Iterator<Item = &'a SanitizedTransaction>,
results: impl Iterator<Item = Result<ExecutionCost>>, results: impl Iterator<Item = Result<()>>,
) -> Vec<Result<ExecutionCost>> { ) -> Vec<Result<()>> {
let key_results: Vec<_> = txs let key_results: Vec<_> = txs
.zip(results) .zip(results)
.map(|(tx, result)| match result { .map(|(tx, result)| match result {
Ok(execution_cost) => Ok((tx.get_account_locks(), execution_cost)), Ok(()) => Ok(tx.get_account_locks()),
Err(e) => Err(e), Err(e) => Err(e),
}) })
.collect(); .collect();
@ -986,9 +978,7 @@ impl Accounts {
key_results key_results
.into_iter() .into_iter()
.map(|key_result| match key_result { .map(|key_result| match key_result {
Ok((keys, execution_cost)) => self Ok(keys) => self.lock_account(account_locks, keys.writable, keys.readonly),
.lock_account(account_locks, keys.writable, keys.readonly)
.map(|_| execution_cost),
Err(e) => Err(e), Err(e) => Err(e),
}) })
.collect() .collect()
@ -999,7 +989,7 @@ impl Accounts {
pub fn unlock_accounts<'a>( pub fn unlock_accounts<'a>(
&self, &self,
txs: impl Iterator<Item = &'a SanitizedTransaction>, txs: impl Iterator<Item = &'a SanitizedTransaction>,
results: &[Result<ExecutionCost>], results: &[Result<()>],
) { ) {
let keys: Vec<_> = txs let keys: Vec<_> = txs
.zip(results) .zip(results)
@ -1283,7 +1273,7 @@ mod tests {
accounts.load_accounts( accounts.load_accounts(
&ancestors, &ancestors,
&[sanitized_tx], &[sanitized_tx],
vec![(Ok(0), None)], vec![(Ok(()), None)],
&hash_queue, &hash_queue,
error_counters, error_counters,
rent_collector, rent_collector,
@ -2427,9 +2417,9 @@ mod tests {
let txs = vec![tx0, tx1, tx2]; let txs = vec![tx0, tx1, tx2];
let qos_results = vec![ let qos_results = vec![
Ok(0), Ok(()),
Err(TransactionError::WouldExceedMaxBlockCostLimit), Err(TransactionError::WouldExceedMaxBlockCostLimit),
Ok(0), Ok(()),
]; ];
let results = accounts.lock_accounts_with_results(txs.iter(), qos_results.into_iter()); let results = accounts.lock_accounts_with_results(txs.iter(), qos_results.into_iter());
@ -2522,7 +2512,6 @@ mod tests {
program_indices: vec![], program_indices: vec![],
rent: 0, rent: 0,
rent_debits: RentDebits::default(), rent_debits: RentDebits::default(),
estimated_execution_cost: 0,
}), }),
None, None,
); );
@ -2533,7 +2522,6 @@ mod tests {
program_indices: vec![], program_indices: vec![],
rent: 0, rent: 0,
rent_debits: RentDebits::default(), rent_debits: RentDebits::default(),
estimated_execution_cost: 0,
}), }),
None, None,
); );
@ -2627,7 +2615,7 @@ mod tests {
accounts.load_accounts( accounts.load_accounts(
&ancestors, &ancestors,
&[tx], &[tx],
vec![(Ok(0), None)], vec![(Ok(()), None)],
&hash_queue, &hash_queue,
&mut error_counters, &mut error_counters,
&rent_collector, &rent_collector,
@ -2963,7 +2951,6 @@ mod tests {
program_indices: vec![], program_indices: vec![],
rent: 0, rent: 0,
rent_debits: RentDebits::default(), rent_debits: RentDebits::default(),
estimated_execution_cost: 0,
}), }),
nonce.clone(), nonce.clone(),
); );
@ -3074,7 +3061,6 @@ mod tests {
program_indices: vec![], program_indices: vec![],
rent: 0, rent: 0,
rent_debits: RentDebits::default(), rent_debits: RentDebits::default(),
estimated_execution_cost: 0,
}), }),
nonce.clone(), nonce.clone(),
); );

View File

@ -47,7 +47,6 @@ use {
ancestors::{Ancestors, AncestorsForSerialization}, ancestors::{Ancestors, AncestorsForSerialization},
blockhash_queue::BlockhashQueue, blockhash_queue::BlockhashQueue,
builtins::{self, ActivationType, Builtin, Builtins}, builtins::{self, ActivationType, Builtin, Builtins},
cost_model::ExecutionCost,
cost_tracker::CostTracker, cost_tracker::CostTracker,
epoch_stakes::{EpochStakes, NodeVoteAccounts}, epoch_stakes::{EpochStakes, NodeVoteAccounts},
inline_spl_token, inline_spl_token,
@ -503,7 +502,7 @@ impl StatusCacheRc {
} }
} }
pub type TransactionCheckResult<'a> = (Result<ExecutionCost>, Option<NoncePartial>); pub type TransactionCheckResult = (Result<()>, Option<NoncePartial>);
pub type TransactionExecutionResult = (Result<()>, Option<NonceFull>); pub type TransactionExecutionResult = (Result<()>, Option<NonceFull>);
pub struct TransactionResults { pub struct TransactionResults {
pub fee_collection_results: Vec<Result<()>>, pub fee_collection_results: Vec<Result<()>>,
@ -3093,7 +3092,7 @@ impl Bank {
pub fn prepare_sanitized_batch_with_results<'a, 'b>( pub fn prepare_sanitized_batch_with_results<'a, 'b>(
&'a self, &'a self,
transactions: &'b [SanitizedTransaction], transactions: &'b [SanitizedTransaction],
transaction_results: impl Iterator<Item = Result<ExecutionCost>>, transaction_results: impl Iterator<Item = Result<()>>,
) -> TransactionBatch<'a, 'b> { ) -> TransactionBatch<'a, 'b> {
// this lock_results could be: Ok, AccountInUse, WouldExceedBlockMaxLimit or WouldExceedAccountMaxLimit // this lock_results could be: Ok, AccountInUse, WouldExceedBlockMaxLimit or WouldExceedAccountMaxLimit
let lock_results = self let lock_results = self
@ -3108,7 +3107,7 @@ impl Bank {
&'a self, &'a self,
transaction: SanitizedTransaction, transaction: SanitizedTransaction,
) -> TransactionBatch<'a, '_> { ) -> TransactionBatch<'a, '_> {
let mut batch = TransactionBatch::new(vec![Ok(0)], self, Cow::Owned(vec![transaction])); let mut batch = TransactionBatch::new(vec![Ok(())], self, Cow::Owned(vec![transaction]));
batch.needs_unlock = false; batch.needs_unlock = false;
batch batch
} }
@ -3204,29 +3203,23 @@ impl Bank {
self.rc.accounts.accounts_db.set_shrink_paths(paths); self.rc.accounts.accounts_db.set_shrink_paths(paths);
} }
fn check_age<'a, T>( fn check_age<'a>(
&self, &self,
txs: impl Iterator<Item = &'a SanitizedTransaction>, txs: impl Iterator<Item = &'a SanitizedTransaction>,
lock_results: impl Iterator<Item = T>, lock_results: &[Result<()>],
max_age: usize, max_age: usize,
error_counters: &mut ErrorCounters, error_counters: &mut ErrorCounters,
) -> Vec<TransactionCheckResult> ) -> Vec<TransactionCheckResult> {
where
T: std::borrow::Borrow<Result<ExecutionCost>>,
{
let hash_queue = self.blockhash_queue.read().unwrap(); let hash_queue = self.blockhash_queue.read().unwrap();
txs.zip(lock_results) txs.zip(lock_results)
.map(|(tx, lock_res)| match lock_res.borrow() { .map(|(tx, lock_res)| match lock_res {
Ok(execution_cost) => { Ok(()) => {
let recent_blockhash = tx.message().recent_blockhash(); let recent_blockhash = tx.message().recent_blockhash();
let hash_age = hash_queue.check_hash_age(recent_blockhash, max_age); let hash_age = hash_queue.check_hash_age(recent_blockhash, max_age);
if hash_age == Some(true) { if hash_age == Some(true) {
(Ok(*execution_cost), None) (Ok(()), None)
} else if let Some((address, account)) = self.check_transaction_for_nonce(tx) { } else if let Some((address, account)) = self.check_transaction_for_nonce(tx) {
( (Ok(()), Some(NoncePartial::new(address, account)))
Ok(*execution_cost),
Some(NoncePartial::new(address, account)),
)
} else if hash_age == Some(false) { } else if hash_age == Some(false) {
error_counters.blockhash_too_old += 1; error_counters.blockhash_too_old += 1;
(Err(TransactionError::BlockhashNotFound), None) (Err(TransactionError::BlockhashNotFound), None)
@ -3296,16 +3289,13 @@ impl Bank {
}) })
} }
pub fn check_transactions<T>( pub fn check_transactions(
&self, &self,
sanitized_txs: &[SanitizedTransaction], sanitized_txs: &[SanitizedTransaction],
lock_results: impl Iterator<Item = T>, lock_results: &[Result<()>],
max_age: usize, max_age: usize,
error_counters: &mut ErrorCounters, error_counters: &mut ErrorCounters,
) -> Vec<TransactionCheckResult> ) -> Vec<TransactionCheckResult> {
where
T: std::borrow::Borrow<Result<ExecutionCost>>,
{
let age_results = let age_results =
self.check_age(sanitized_txs.iter(), lock_results, max_age, error_counters); self.check_age(sanitized_txs.iter(), lock_results, max_age, error_counters);
self.check_status_cache(sanitized_txs, age_results, error_counters) self.check_status_cache(sanitized_txs, age_results, error_counters)
@ -3528,7 +3518,7 @@ impl Bank {
let mut check_time = Measure::start("check_transactions"); let mut check_time = Measure::start("check_transactions");
let check_results = self.check_transactions( let check_results = self.check_transactions(
sanitized_txs, sanitized_txs,
batch.lock_results().iter(), batch.lock_results(),
max_age, max_age,
&mut error_counters, &mut error_counters,
); );
@ -3607,7 +3597,6 @@ impl Bank {
&self.builtin_programs.vec, &self.builtin_programs.vec,
legacy_message, legacy_message,
&loaded_transaction.program_indices, &loaded_transaction.program_indices,
loaded_transaction.estimated_execution_cost,
&account_refcells, &account_refcells,
self.rent_collector.rent, self.rent_collector.rent,
log_collector.clone(), log_collector.clone(),

View File

@ -15,7 +15,6 @@ use {
}; };
const MAX_WRITABLE_ACCOUNTS: usize = 256; const MAX_WRITABLE_ACCOUNTS: usize = 256;
pub type ExecutionCost = u64;
// costs are stored in number of 'compute unit's // costs are stored in number of 'compute unit's
#[derive(Debug)] #[derive(Debug)]
@ -24,7 +23,7 @@ pub struct TransactionCost {
pub signature_cost: u64, pub signature_cost: u64,
pub write_lock_cost: u64, pub write_lock_cost: u64,
pub data_bytes_cost: u64, pub data_bytes_cost: u64,
pub execution_cost: ExecutionCost, pub execution_cost: u64,
// `cost_weight` is a multiplier could be applied to transaction cost, // `cost_weight` is a multiplier could be applied to transaction cost,
// if set to zero allows the transaction to bypass cost limit check. // if set to zero allows the transaction to bypass cost limit check.
pub cost_weight: u32, pub cost_weight: u32,

View File

@ -1,13 +1,9 @@
use { use {
crate::cost_model::ExecutionCost,
serde::{Deserialize, Serialize}, serde::{Deserialize, Serialize},
solana_measure::measure::Measure, solana_measure::measure::Measure,
solana_program_runtime::{ solana_program_runtime::{
instruction_recorder::InstructionRecorder, instruction_recorder::InstructionRecorder,
invoke_context::{ invoke_context::{BuiltinProgram, Executors, InvokeContext, TransactionAccountRefCell},
BuiltinProgram, Executors, InvokeContext, ProcessInstructionResult,
TransactionAccountRefCell,
},
log_collector::LogCollector, log_collector::LogCollector,
timings::ExecuteDetailsTimings, timings::ExecuteDetailsTimings,
}, },
@ -49,7 +45,6 @@ impl MessageProcessor {
builtin_programs: &[BuiltinProgram], builtin_programs: &[BuiltinProgram],
message: &Message, message: &Message,
program_indices: &[Vec<usize>], program_indices: &[Vec<usize>],
estimated_execution_cost: ExecutionCost,
accounts: &[TransactionAccountRefCell], accounts: &[TransactionAccountRefCell],
rent: Rent, rent: Rent,
log_collector: Option<Rc<RefCell<LogCollector>>>, log_collector: Option<Rc<RefCell<LogCollector>>>,
@ -110,21 +105,16 @@ impl MessageProcessor {
Some(&instruction_recorders[instruction_index]); Some(&instruction_recorders[instruction_index]);
} }
let mut time = Measure::start("execute_instruction"); let mut time = Measure::start("execute_instruction");
let ProcessInstructionResult { let compute_meter_consumption = invoke_context
compute_units_consumed, .process_instruction(message, instruction, program_indices, &[], &[])
result, .map_err(|err| TransactionError::InstructionError(instruction_index as u8, err))?;
} = invoke_context.process_instruction(message, instruction, program_indices, &[], &[]);
time.stop(); time.stop();
timings.accumulate_program( timings.accumulate_program(
instruction.program_id(&message.account_keys), instruction.program_id(&message.account_keys),
time.as_us(), time.as_us(),
compute_units_consumed, compute_meter_consumption,
estimated_execution_cost,
result.is_err(),
); );
timings.accumulate(&invoke_context.timings); timings.accumulate(&invoke_context.timings);
result
.map_err(|err| TransactionError::InstructionError(instruction_index as u8, err))?;
} }
Ok(()) Ok(())
} }
@ -240,7 +230,6 @@ mod tests {
builtin_programs, builtin_programs,
&message, &message,
&program_indices, &program_indices,
0,
&accounts, &accounts,
rent_collector.rent, rent_collector.rent,
None, None,
@ -270,7 +259,6 @@ mod tests {
builtin_programs, builtin_programs,
&message, &message,
&program_indices, &program_indices,
0,
&accounts, &accounts,
rent_collector.rent, rent_collector.rent,
None, None,
@ -304,7 +292,6 @@ mod tests {
builtin_programs, builtin_programs,
&message, &message,
&program_indices, &program_indices,
0,
&accounts, &accounts,
rent_collector.rent, rent_collector.rent,
None, None,
@ -449,7 +436,6 @@ mod tests {
builtin_programs, builtin_programs,
&message, &message,
&program_indices, &program_indices,
0,
&accounts, &accounts,
rent_collector.rent, rent_collector.rent,
None, None,
@ -483,7 +469,6 @@ mod tests {
builtin_programs, builtin_programs,
&message, &message,
&program_indices, &program_indices,
0,
&accounts, &accounts,
rent_collector.rent, rent_collector.rent,
None, None,
@ -514,7 +499,6 @@ mod tests {
builtin_programs, builtin_programs,
&message, &message,
&program_indices, &program_indices,
0,
&accounts, &accounts,
rent_collector.rent, rent_collector.rent,
None, None,
@ -572,7 +556,6 @@ mod tests {
builtin_programs, builtin_programs,
&message, &message,
&[vec![0], vec![1]], &[vec![0], vec![1]],
0,
&accounts, &accounts,
RentCollector::default().rent, RentCollector::default().rent,
None, None,

View File

@ -1,12 +1,12 @@
use { use {
crate::{bank::Bank, cost_model::ExecutionCost}, crate::bank::Bank,
solana_sdk::transaction::{Result, SanitizedTransaction}, solana_sdk::transaction::{Result, SanitizedTransaction},
std::borrow::Cow, std::borrow::Cow,
}; };
// Represents the results of trying to lock a set of accounts // Represents the results of trying to lock a set of accounts
pub struct TransactionBatch<'a, 'b> { pub struct TransactionBatch<'a, 'b> {
lock_results: Vec<Result<ExecutionCost>>, lock_results: Vec<Result<()>>,
bank: &'a Bank, bank: &'a Bank,
sanitized_txs: Cow<'b, [SanitizedTransaction]>, sanitized_txs: Cow<'b, [SanitizedTransaction]>,
pub(crate) needs_unlock: bool, pub(crate) needs_unlock: bool,
@ -14,7 +14,7 @@ pub struct TransactionBatch<'a, 'b> {
impl<'a, 'b> TransactionBatch<'a, 'b> { impl<'a, 'b> TransactionBatch<'a, 'b> {
pub fn new( pub fn new(
lock_results: Vec<Result<ExecutionCost>>, lock_results: Vec<Result<()>>,
bank: &'a Bank, bank: &'a Bank,
sanitized_txs: Cow<'b, [SanitizedTransaction]>, sanitized_txs: Cow<'b, [SanitizedTransaction]>,
) -> Self { ) -> Self {
@ -27,7 +27,7 @@ impl<'a, 'b> TransactionBatch<'a, 'b> {
} }
} }
pub fn lock_results(&self) -> &Vec<Result<ExecutionCost>> { pub fn lock_results(&self) -> &Vec<Result<()>> {
&self.lock_results &self.lock_results
} }