Move transaction sanitization earlier in the pipeline (#18655)

* Move transaction sanitization earlier in the pipeline

* Renamed HashedTransaction to SanitizedTransaction

* Implement deref for sanitized transaction

* bring back process_transactions test method

* Use sanitized transactions for cost model calculation
This commit is contained in:
Justin Starry
2021-07-15 22:51:27 -05:00
committed by GitHub
parent c03490b24a
commit d166b9856a
21 changed files with 448 additions and 369 deletions

View File

@ -41,14 +41,6 @@ fn deposit_many(bank: &Bank, pubkeys: &mut Vec<Pubkey>, num: usize) -> Result<()
Ok(())
}
#[bench]
fn bench_has_duplicates(bencher: &mut Bencher) {
bencher.iter(|| {
let data = test::black_box([1, 2, 3]);
assert!(!Accounts::has_duplicates(&data));
})
}
#[bench]
fn test_accounts_create(bencher: &mut Bencher) {
let (genesis_config, _) = create_genesis_config(10_000);

View File

@ -84,7 +84,7 @@ pub fn create_native_loader_transactions(
}
fn sync_bencher(bank: &Arc<Bank>, _bank_client: &BankClient, transactions: &[Transaction]) {
let results = bank.process_transactions(transactions);
let results = bank.process_transactions(transactions.iter());
assert!(results.iter().all(Result::is_ok));
}
@ -136,7 +136,7 @@ fn do_bench_transactions(
let transactions = create_transactions(&bank_client, &mint_keypair);
// Do once to fund accounts, load modules, etc...
let results = bank.process_transactions(&transactions);
let results = bank.process_transactions(transactions.iter());
assert!(results.iter().all(Result::is_ok));
bencher.iter(|| {

View File

@ -165,19 +165,6 @@ impl Accounts {
}
}
/// Return true if the slice has any duplicate elements
pub fn has_duplicates<T: PartialEq>(xs: &[T]) -> bool {
// Note: This is an O(n^2) algorithm, but requires no heap allocations. The benchmark
// `bench_has_duplicates` in benches/message_processor.rs shows that this implementation is
// ~50 times faster than using HashSet for very short slices.
for i in 1..xs.len() {
if xs[i..].contains(&xs[i - 1]) {
return true;
}
}
false
}
fn construct_instructions_account(message: &Message) -> AccountSharedData {
let mut data = message.serialize_instructions();
// add room for current instruction index.
@ -858,25 +845,13 @@ impl Accounts {
#[must_use]
#[allow(clippy::needless_collect)]
pub fn lock_accounts<'a>(&self, txs: impl Iterator<Item = &'a Transaction>) -> Vec<Result<()>> {
use solana_sdk::sanitize::Sanitize;
let keys: Vec<Result<_>> = txs
.map(|tx| {
tx.sanitize().map_err(TransactionError::from)?;
if Self::has_duplicates(&tx.message.account_keys) {
return Err(TransactionError::AccountLoadedTwice);
}
Ok(tx.message().get_account_keys_by_lock_type())
})
let keys: Vec<_> = txs
.map(|tx| tx.message().get_account_keys_by_lock_type())
.collect();
let mut account_locks = &mut self.account_locks.lock().unwrap();
keys.into_iter()
.map(|result| match result {
Ok((writable_keys, readonly_keys)) => {
self.lock_account(&mut account_locks, writable_keys, readonly_keys)
}
Err(e) => Err(e),
.map(|(writable_keys, readonly_keys)| {
self.lock_account(&mut account_locks, writable_keys, readonly_keys)
})
.collect()
}
@ -2035,12 +2010,6 @@ mod tests {
);
}
#[test]
fn test_has_duplicates() {
assert!(!Accounts::has_duplicates(&[1, 2]));
assert!(Accounts::has_duplicates(&[1, 2, 1]));
}
#[test]
fn huge_clean() {
solana_logger::setup();

View File

@ -84,7 +84,6 @@ use solana_sdk::{
genesis_config::{ClusterType, GenesisConfig},
hard_forks::HardForks,
hash::{extend_and_hash, hashv, Hash},
hashed_transaction::{HashedTransaction, HashedTransactionSlice},
incinerator,
inflation::Inflation,
instruction::CompiledInstruction,
@ -97,7 +96,7 @@ use solana_sdk::{
program_utils::limited_deserialize,
pubkey::Pubkey,
recent_blockhashes_account,
sanitize::Sanitize,
sanitized_transaction::{SanitizedTransaction, SanitizedTransactionSlice},
signature::{Keypair, Signature},
slot_hashes::SlotHashes,
slot_history::SlotHistory,
@ -2588,19 +2587,18 @@ impl Bank {
fn update_transaction_statuses(
&self,
hashed_txs: &[HashedTransaction],
sanitized_txs: &[SanitizedTransaction],
res: &[TransactionExecutionResult],
) {
let mut status_cache = self.src.status_cache.write().unwrap();
assert_eq!(hashed_txs.len(), res.len());
for (hashed_tx, (res, _nonce_rollback)) in hashed_txs.iter().zip(res) {
let tx = hashed_tx.transaction();
assert_eq!(sanitized_txs.len(), res.len());
for (tx, (res, _nonce_rollback)) in sanitized_txs.iter().zip(res) {
if Self::can_commit(res) && !tx.signatures.is_empty() {
// Add the message hash to the status cache to ensure that this message
// won't be processed again with a different signature.
status_cache.insert(
&tx.message().recent_blockhash,
&hashed_tx.message_hash,
&tx.message_hash,
self.slot(),
res.clone(),
);
@ -2654,44 +2652,37 @@ impl Bank {
pub fn prepare_batch<'a, 'b>(
&'a self,
txs: impl Iterator<Item = &'b Transaction>,
) -> TransactionBatch<'a, 'b> {
let hashed_txs: Vec<HashedTransaction> = txs.map(HashedTransaction::from).collect();
) -> Result<TransactionBatch<'a, 'b>> {
let sanitized_txs: Vec<SanitizedTransaction> = txs
.map(SanitizedTransaction::try_from)
.collect::<Result<_>>()?;
let lock_results = self
.rc
.accounts
.lock_accounts(hashed_txs.as_transactions_iter());
TransactionBatch::new(lock_results, self, Cow::Owned(hashed_txs))
.lock_accounts(sanitized_txs.as_transactions_iter());
Ok(TransactionBatch::new(
lock_results,
self,
Cow::Owned(sanitized_txs),
))
}
pub fn prepare_hashed_batch<'a, 'b>(
pub fn prepare_sanitized_batch<'a, 'b>(
&'a self,
hashed_txs: &'b [HashedTransaction],
sanitized_txs: &'b [SanitizedTransaction],
) -> TransactionBatch<'a, 'b> {
let lock_results = self
.rc
.accounts
.lock_accounts(hashed_txs.as_transactions_iter());
TransactionBatch::new(lock_results, self, Cow::Borrowed(hashed_txs))
.lock_accounts(sanitized_txs.as_transactions_iter());
TransactionBatch::new(lock_results, self, Cow::Borrowed(sanitized_txs))
}
pub(crate) fn prepare_simulation_batch<'a, 'b>(
&'a self,
tx: &'b Transaction,
tx: SanitizedTransaction<'b>,
) -> TransactionBatch<'a, 'b> {
let check_transaction = |tx: &Transaction| -> Result<()> {
tx.sanitize().map_err(TransactionError::from)?;
if Accounts::has_duplicates(&tx.message.account_keys) {
Err(TransactionError::AccountLoadedTwice)
} else {
Ok(())
}
};
let mut batch = TransactionBatch::new(
vec![check_transaction(tx)],
self,
Cow::Owned(vec![HashedTransaction::from(tx)]),
);
let mut batch = TransactionBatch::new(vec![Ok(())], self, Cow::Owned(vec![tx]));
batch.needs_unlock = false;
batch
}
@ -2707,7 +2698,10 @@ impl Bank {
) {
assert!(self.is_frozen(), "simulation bank must be frozen");
let batch = self.prepare_simulation_batch(transaction);
let batch = match SanitizedTransaction::try_from(transaction) {
Ok(sanitized_tx) => self.prepare_simulation_batch(sanitized_tx),
Err(err) => return (Err(err), vec![], vec![]),
};
let mut timings = ExecuteTimings::default();
@ -2795,11 +2789,11 @@ impl Bank {
fn is_tx_already_processed(
&self,
hashed_tx: &HashedTransaction,
sanitized_tx: &SanitizedTransaction,
status_cache: &StatusCache<Result<()>>,
) -> bool {
let key = &hashed_tx.message_hash;
let transaction_blockhash = &hashed_tx.transaction().message().recent_blockhash;
let key = &sanitized_tx.message_hash;
let transaction_blockhash = &sanitized_tx.message().recent_blockhash;
status_cache
.get_status(key, transaction_blockhash, &self.ancestors)
.is_some()
@ -2807,16 +2801,16 @@ impl Bank {
fn check_status_cache(
&self,
hashed_txs: &[HashedTransaction],
sanitized_txs: &[SanitizedTransaction],
lock_results: Vec<TransactionCheckResult>,
error_counters: &mut ErrorCounters,
) -> Vec<TransactionCheckResult> {
let rcache = self.src.status_cache.read().unwrap();
hashed_txs
sanitized_txs
.iter()
.zip(lock_results)
.map(|(hashed_tx, (lock_res, nonce_rollback))| {
if lock_res.is_ok() && self.is_tx_already_processed(hashed_tx, &rcache) {
.map(|(sanitized_tx, (lock_res, nonce_rollback))| {
if lock_res.is_ok() && self.is_tx_already_processed(sanitized_tx, &rcache) {
error_counters.already_processed += 1;
return (Err(TransactionError::AlreadyProcessed), None);
}
@ -2881,22 +2875,23 @@ impl Bank {
pub fn check_transactions(
&self,
hashed_txs: &[HashedTransaction],
sanitized_txs: &[SanitizedTransaction],
lock_results: &[Result<()>],
max_age: usize,
mut error_counters: &mut ErrorCounters,
) -> Vec<TransactionCheckResult> {
let age_results = self.check_age(
hashed_txs.as_transactions_iter(),
sanitized_txs.as_transactions_iter(),
lock_results.to_vec(),
max_age,
&mut error_counters,
);
let cache_results = self.check_status_cache(hashed_txs, age_results, &mut error_counters);
let cache_results =
self.check_status_cache(sanitized_txs, age_results, &mut error_counters);
if self.upgrade_epoch() {
// Reject all non-vote transactions
self.filter_by_vote_transactions(
hashed_txs.as_transactions_iter(),
sanitized_txs.as_transactions_iter(),
cache_results,
&mut error_counters,
)
@ -3131,9 +3126,9 @@ impl Bank {
u64,
u64,
) {
let hashed_txs = batch.hashed_transactions();
debug!("processing transactions: {}", hashed_txs.len());
inc_new_counter_info!("bank-process_transactions", hashed_txs.len());
let sanitized_txs = batch.sanitized_transactions();
debug!("processing transactions: {}", sanitized_txs.len());
inc_new_counter_info!("bank-process_transactions", sanitized_txs.len());
let mut error_counters = ErrorCounters::default();
let retryable_txs: Vec<_> = batch
@ -3152,7 +3147,7 @@ impl Bank {
let mut check_time = Measure::start("check_transactions");
let check_results = self.check_transactions(
hashed_txs,
sanitized_txs,
batch.lock_results(),
max_age,
&mut error_counters,
@ -3162,7 +3157,7 @@ impl Bank {
let mut load_time = Measure::start("accounts_load");
let mut loaded_txs = self.rc.accounts.load_accounts(
&self.ancestors,
hashed_txs.as_transactions_iter(),
sanitized_txs.as_transactions_iter(),
check_results,
&self.blockhash_queue.read().unwrap(),
&mut error_counters,
@ -3174,16 +3169,16 @@ impl Bank {
let mut execution_time = Measure::start("execution_time");
let mut signature_count: u64 = 0;
let mut inner_instructions: Vec<Option<InnerInstructionsList>> =
Vec::with_capacity(hashed_txs.len());
Vec::with_capacity(sanitized_txs.len());
let mut transaction_log_messages: Vec<Option<Vec<String>>> =
Vec::with_capacity(hashed_txs.len());
Vec::with_capacity(sanitized_txs.len());
let bpf_compute_budget = self
.bpf_compute_budget
.unwrap_or_else(BpfComputeBudget::new);
let executed: Vec<TransactionExecutionResult> = loaded_txs
.iter_mut()
.zip(hashed_txs.as_transactions_iter())
.zip(sanitized_txs.as_transactions_iter())
.map(|(accs, tx)| match accs {
(Err(e), _nonce_rollback) => {
inner_instructions.push(None);
@ -3270,7 +3265,7 @@ impl Bank {
check_time.as_us(),
load_time.as_us(),
execution_time.as_us(),
hashed_txs.len(),
sanitized_txs.len(),
);
timings.check_us += check_time.as_us();
timings.load_us += load_time.as_us();
@ -3281,8 +3276,7 @@ impl Bank {
let transaction_log_collector_config =
self.transaction_log_collector_config.read().unwrap();
for (i, ((r, _nonce_rollback), hashed_tx)) in executed.iter().zip(hashed_txs).enumerate() {
let tx = hashed_tx.transaction();
for (i, ((r, _nonce_rollback), tx)) in executed.iter().zip(sanitized_txs).enumerate() {
if let Some(debug_keys) = &self.transaction_debug_keys {
for key in &tx.message.account_keys {
if debug_keys.contains(key) {
@ -3423,7 +3417,7 @@ impl Bank {
pub fn commit_transactions(
&self,
hashed_txs: &[HashedTransaction],
sanitized_txs: &[SanitizedTransaction],
loaded_txs: &mut [TransactionLoadResult],
executed: &[TransactionExecutionResult],
tx_count: u64,
@ -3441,8 +3435,8 @@ impl Bank {
inc_new_counter_info!("bank-process_transactions-txs", tx_count as usize);
inc_new_counter_info!("bank-process_transactions-sigs", signature_count as usize);
if !hashed_txs.is_empty() {
let processed_tx_count = hashed_txs.len() as u64;
if !sanitized_txs.is_empty() {
let processed_tx_count = sanitized_txs.len() as u64;
let failed_tx_count = processed_tx_count.saturating_sub(tx_count);
self.transaction_error_count
.fetch_add(failed_tx_count, Relaxed);
@ -3461,7 +3455,7 @@ impl Bank {
let mut write_time = Measure::start("write_time");
self.rc.accounts.store_cached(
self.slot(),
hashed_txs.as_transactions_iter(),
sanitized_txs.as_transactions_iter(),
executed,
loaded_txs,
&self.rent_collector,
@ -3472,19 +3466,19 @@ impl Bank {
let rent_debits = self.collect_rent(executed, loaded_txs);
let overwritten_vote_accounts =
self.update_cached_accounts(hashed_txs.as_transactions_iter(), executed, loaded_txs);
self.update_cached_accounts(sanitized_txs.as_transactions_iter(), executed, loaded_txs);
// once committed there is no way to unroll
write_time.stop();
debug!(
"store: {}us txs_len={}",
write_time.as_us(),
hashed_txs.len()
sanitized_txs.len()
);
timings.store_us += write_time.as_us();
self.update_transaction_statuses(hashed_txs, executed);
let fee_collection_results =
self.filter_program_errors_and_collect_fee(hashed_txs.as_transactions_iter(), executed);
self.update_transaction_statuses(sanitized_txs, executed);
let fee_collection_results = self
.filter_program_errors_and_collect_fee(sanitized_txs.as_transactions_iter(), executed);
TransactionResults {
fee_collection_results,
@ -4115,7 +4109,7 @@ impl Bank {
);
let results = self.commit_transactions(
batch.hashed_transactions(),
batch.sanitized_transactions(),
&mut loaded_txs,
&executed,
tx_count,
@ -4136,19 +4130,35 @@ impl Bank {
}
/// Process a Transaction. This is used for unit tests and simply calls the vector
/// Bank::process_transactions method
/// Bank::process_transactions method.
pub fn process_transaction(&self, tx: &Transaction) -> Result<()> {
let batch = self.prepare_batch(std::iter::once(tx));
self.process_transaction_batch(&batch)[0].clone()?;
self.try_process_transactions(std::iter::once(tx))?[0].clone()?;
tx.signatures
.get(0)
.map_or(Ok(()), |sig| self.get_signature_status(sig).unwrap())
}
/// Process multiple transaction in a single batch. This is used for benches and unit tests.
///
/// # Panics
///
/// Panics if any of the transactions do not pass sanitization checks.
#[must_use]
pub fn process_transactions(&self, txs: &[Transaction]) -> Vec<Result<()>> {
let batch = self.prepare_batch(txs.iter());
self.process_transaction_batch(&batch)
pub fn process_transactions<'a>(
&self,
txs: impl Iterator<Item = &'a Transaction>,
) -> Vec<Result<()>> {
self.try_process_transactions(txs).unwrap()
}
/// Process multiple transaction in a single batch. This is used for benches and unit tests.
/// Short circuits if any of the transactions do not pass sanitization checks.
pub fn try_process_transactions<'a>(
&self,
txs: impl Iterator<Item = &'a Transaction>,
) -> Result<Vec<Result<()>>> {
let batch = self.prepare_batch(txs)?;
Ok(self.process_transaction_batch(&batch))
}
#[must_use]
@ -5813,7 +5823,8 @@ pub(crate) mod tests {
let t3 =
system_transaction::transfer(&keypair5, &keypair6.pubkey(), 1, genesis_config.hash());
let res = bank.process_transactions(&[t1.clone(), t2.clone(), t3]);
let txs = vec![t1.clone(), t2.clone(), t3];
let res = bank.process_transactions(txs.iter());
assert_eq!(res.len(), 3);
assert_eq!(res[0], Ok(()));
@ -5825,7 +5836,8 @@ pub(crate) mod tests {
let rwlockguard_bank_hash = bank.hash.read().unwrap();
let bank_hash = rwlockguard_bank_hash.as_ref();
let res = bank_with_success_txs.process_transactions(&[t2, t1]);
let txs = vec![t2, t1];
let res = bank_with_success_txs.process_transactions(txs.iter());
assert_eq!(res.len(), 2);
assert_eq!(res[0], Ok(()));
@ -6528,7 +6540,8 @@ pub(crate) mod tests {
genesis_config.hash(),
);
let res = bank.process_transactions(&[t6, t5, t1, t2, t3, t4]);
let txs = vec![t6, t5, t1, t2, t3, t4];
let res = bank.process_transactions(txs.iter());
assert_eq!(res.len(), 6);
assert_eq!(res[0], Ok(()));
@ -7717,7 +7730,8 @@ pub(crate) mod tests {
let t1 = system_transaction::transfer(&mint_keypair, &key1, 1, genesis_config.hash());
let t2 = system_transaction::transfer(&mint_keypair, &key2, 1, genesis_config.hash());
let res = bank.process_transactions(&[t1.clone(), t2.clone()]);
let txs = vec![t1.clone(), t2.clone()];
let res = bank.process_transactions(txs.iter());
assert_eq!(res.len(), 2);
assert_eq!(res[0], Ok(()));
@ -8164,7 +8178,7 @@ pub(crate) mod tests {
genesis_config.hash(),
);
let txs = vec![tx0, tx1];
let results = bank.process_transactions(&txs);
let results = bank.process_transactions(txs.iter());
assert!(results[1].is_err());
// Assert bad transactions aren't counted.
@ -8220,7 +8234,7 @@ pub(crate) mod tests {
bank.last_blockhash(),
);
let txs = vec![tx0, tx1];
let results = bank.process_transactions(&txs);
let results = bank.process_transactions(txs.iter());
// If multiple transactions attempt to read the same account, they should succeed.
// Vote authorized_voter and sysvar accounts are given read-only handling
@ -8241,7 +8255,7 @@ pub(crate) mod tests {
bank.last_blockhash(),
);
let txs = vec![tx0, tx1];
let results = bank.process_transactions(&txs);
let results = bank.process_transactions(txs.iter());
// However, an account may not be locked as read-only and writable at the same time.
assert_eq!(results[0], Ok(()));
assert_eq!(results[1], Err(TransactionError::AccountInUse));
@ -8258,7 +8272,7 @@ pub(crate) mod tests {
system_transaction::transfer(&mint_keypair, &alice.pubkey(), 1, genesis_config.hash());
let pay_alice = vec![tx1];
let lock_result = bank.prepare_batch(pay_alice.iter());
let lock_result = bank.prepare_batch(pay_alice.iter()).unwrap();
let results_alice = bank
.load_execute_and_commit_transactions(
&lock_result,
@ -8311,7 +8325,7 @@ pub(crate) mod tests {
let tx = Transaction::new(&[&key0], message, genesis_config.hash());
let txs = vec![tx];
let batch0 = bank.prepare_batch(txs.iter());
let batch0 = bank.prepare_batch(txs.iter()).unwrap();
assert!(batch0.lock_results()[0].is_ok());
// Try locking accounts, locking a previously read-only account as writable
@ -8329,7 +8343,7 @@ pub(crate) mod tests {
let tx = Transaction::new(&[&key1], message, genesis_config.hash());
let txs = vec![tx];
let batch1 = bank.prepare_batch(txs.iter());
let batch1 = bank.prepare_batch(txs.iter()).unwrap();
assert!(batch1.lock_results()[0].is_err());
// Try locking a previously read-only account a 2nd time; should succeed
@ -8346,7 +8360,7 @@ pub(crate) mod tests {
let tx = Transaction::new(&[&key2], message, genesis_config.hash());
let txs = vec![tx];
let batch2 = bank.prepare_batch(txs.iter());
let batch2 = bank.prepare_batch(txs.iter()).unwrap();
assert!(batch2.lock_results()[0].is_ok());
}
@ -10231,14 +10245,14 @@ pub(crate) mod tests {
instructions,
);
let txs = vec![tx0, tx1];
let batch = bank0.prepare_batch(txs.iter());
let batch = bank0.prepare_batch(txs.iter()).unwrap();
let balances = bank0.collect_balances(&batch);
assert_eq!(balances.len(), 2);
assert_eq!(balances[0], vec![8, 11, 1]);
assert_eq!(balances[1], vec![8, 0, 1]);
let txs: Vec<_> = txs.iter().rev().cloned().collect();
let batch = bank0.prepare_batch(txs.iter());
let batch = bank0.prepare_batch(txs.iter()).unwrap();
let balances = bank0.collect_balances(&batch);
assert_eq!(balances.len(), 2);
assert_eq!(balances[0], vec![8, 0, 1]);
@ -10272,7 +10286,7 @@ pub(crate) mod tests {
let tx2 = system_transaction::transfer(&keypair1, &pubkey2, 12, blockhash);
let txs = vec![tx0, tx1, tx2];
let lock_result = bank0.prepare_batch(txs.iter());
let lock_result = bank0.prepare_batch(txs.iter()).unwrap();
let (transaction_results, transaction_balances_set, inner_instructions, transaction_logs) =
bank0.load_execute_and_commit_transactions(
&lock_result,
@ -13335,10 +13349,9 @@ pub(crate) mod tests {
let success_sig = tx0.signatures[0];
let tx1 = system_transaction::transfer(&sender1, &recipient1, 110, blockhash); // Should produce insufficient funds log
let failure_sig = tx1.signatures[0];
let mut invalid_tx = system_transaction::transfer(&sender1, &recipient1, 10, blockhash);
invalid_tx.message.header.num_required_signatures = 0; // this tx won't be processed because it has no signers
let txs = vec![invalid_tx, tx1, tx0];
let batch = bank.prepare_batch(txs.iter());
let tx2 = system_transaction::transfer(&sender0, &recipient0, 1, blockhash);
let txs = vec![tx0, tx1, tx2];
let batch = bank.prepare_batch(txs.iter()).unwrap();
let log_results = bank
.load_execute_and_commit_transactions(
@ -13351,9 +13364,9 @@ pub(crate) mod tests {
)
.3;
assert_eq!(log_results.len(), 3);
assert!(log_results[0].as_ref().is_none());
assert!(log_results[0].as_ref().unwrap()[1].contains(&"success".to_string()));
assert!(log_results[1].as_ref().unwrap()[2].contains(&"failed".to_string()));
assert!(log_results[2].as_ref().unwrap()[1].contains(&"success".to_string()));
assert!(log_results[2].as_ref().is_none());
let stored_logs = &bank.transaction_log_collector.read().unwrap().logs;
let success_log_info = stored_logs

View File

@ -280,7 +280,7 @@ impl BankClient {
while let Ok(tx) = transaction_receiver.try_recv() {
transactions.push(tx);
}
let _ = bank.process_transactions(&transactions);
let _ = bank.try_process_transactions(transactions.iter());
}
}

View File

@ -3,7 +3,7 @@ use crate::{
genesis_utils::{self, GenesisConfigInfo, ValidatorVoteKeypairs},
vote_sender_types::ReplayVoteSender,
};
use solana_sdk::{hashed_transaction::HashedTransaction, pubkey::Pubkey, signature::Signer};
use solana_sdk::{pubkey::Pubkey, sanitized_transaction::SanitizedTransaction, signature::Signer};
use solana_vote_program::vote_transaction;
pub fn setup_bank_and_vote_pubkeys(num_vote_accounts: usize, stake: u64) -> (Bank, Vec<Pubkey>) {
@ -27,7 +27,7 @@ pub fn setup_bank_and_vote_pubkeys(num_vote_accounts: usize, stake: u64) -> (Ban
}
pub fn find_and_send_votes(
hashed_txs: &[HashedTransaction],
sanitized_txs: &[SanitizedTransaction],
tx_results: &TransactionResults,
vote_sender: Option<&ReplayVoteSender>,
) {
@ -41,7 +41,7 @@ pub fn find_and_send_votes(
assert!(execution_results[old_account.transaction_result_index]
.0
.is_ok());
let transaction = hashed_txs[old_account.transaction_index].transaction();
let transaction = &sanitized_txs[old_account.transaction_index];
if let Some(parsed_vote) = vote_transaction::parse_vote_transaction(transaction) {
if parsed_vote.1.slots.last().is_some() {
let _ = vote_sender.send(parsed_vote);

View File

@ -1,13 +1,16 @@
use crate::bank::Bank;
use solana_sdk::hashed_transaction::HashedTransaction;
use solana_sdk::transaction::{Result, Transaction};
use solana_sdk::{
sanitized_transaction::SanitizedTransaction,
transaction::{Result, Transaction},
};
use std::borrow::Cow;
use std::ops::Deref;
// Represents the results of trying to lock a set of accounts
pub struct TransactionBatch<'a, 'b> {
lock_results: Vec<Result<()>>,
bank: &'a Bank,
hashed_txs: Cow<'b, [HashedTransaction<'b>]>,
sanitized_txs: Cow<'b, [SanitizedTransaction<'b>]>,
pub(crate) needs_unlock: bool,
}
@ -15,13 +18,13 @@ impl<'a, 'b> TransactionBatch<'a, 'b> {
pub fn new(
lock_results: Vec<Result<()>>,
bank: &'a Bank,
hashed_txs: Cow<'b, [HashedTransaction<'b>]>,
sanitized_txs: Cow<'b, [SanitizedTransaction<'b>]>,
) -> Self {
assert_eq!(lock_results.len(), hashed_txs.len());
assert_eq!(lock_results.len(), sanitized_txs.len());
Self {
lock_results,
bank,
hashed_txs,
sanitized_txs,
needs_unlock: true,
}
}
@ -30,12 +33,12 @@ impl<'a, 'b> TransactionBatch<'a, 'b> {
&self.lock_results
}
pub fn hashed_transactions(&self) -> &[HashedTransaction] {
&self.hashed_txs
pub fn sanitized_transactions(&self) -> &[SanitizedTransaction] {
&self.sanitized_txs
}
pub fn transactions_iter(&self) -> impl Iterator<Item = &Transaction> {
self.hashed_txs.iter().map(|h| h.transaction())
self.sanitized_txs.iter().map(Deref::deref)
}
pub fn bank(&self) -> &Bank {
@ -55,43 +58,49 @@ mod tests {
use super::*;
use crate::genesis_utils::{create_genesis_config_with_leader, GenesisConfigInfo};
use solana_sdk::{signature::Keypair, system_transaction};
use std::convert::TryFrom;
#[test]
fn test_transaction_batch() {
let (bank, txs) = setup();
// Test getting locked accounts
let batch = bank.prepare_batch(txs.iter());
let batch = bank.prepare_batch(txs.iter()).unwrap();
// Grab locks
assert!(batch.lock_results().iter().all(|x| x.is_ok()));
// Trying to grab locks again should fail
let batch2 = bank.prepare_batch(txs.iter());
let batch2 = bank.prepare_batch(txs.iter()).unwrap();
assert!(batch2.lock_results().iter().all(|x| x.is_err()));
// Drop the first set of locks
drop(batch);
// Now grabbing locks should work again
let batch2 = bank.prepare_batch(txs.iter());
let batch2 = bank.prepare_batch(txs.iter()).unwrap();
assert!(batch2.lock_results().iter().all(|x| x.is_ok()));
}
#[test]
fn test_simulation_batch() {
let (bank, txs) = setup();
let txs = txs
.into_iter()
.map(SanitizedTransaction::try_from)
.collect::<Result<Vec<_>>>()
.unwrap();
// Prepare batch without locks
let batch = bank.prepare_simulation_batch(&txs[0]);
let batch = bank.prepare_simulation_batch(txs[0].clone());
assert!(batch.lock_results().iter().all(|x| x.is_ok()));
// Grab locks
let batch2 = bank.prepare_batch(txs.iter());
let batch2 = bank.prepare_sanitized_batch(&txs);
assert!(batch2.lock_results().iter().all(|x| x.is_ok()));
// Prepare another batch without locks
let batch3 = bank.prepare_simulation_batch(&txs[0]);
let batch3 = bank.prepare_simulation_batch(txs[0].clone());
assert!(batch3.lock_results().iter().all(|x| x.is_ok()));
}