Store versioned transactions in the ledger, disabled by default (#19139)
* Add support for versioned transactions, but disable by default * merge conflicts * trent's feedback * bump Cargo.lock * Fix transaction error encoding * Rename legacy_transaction method * cargo clippy * Clean up casts, int arithmetic, and unused methods * Check for duplicates in sanitized message conversion * fix clippy * fix new test * Fix bpf conditional compilation for message module
This commit is contained in:
parent
098e2b2de3
commit
c50b01cb60
5
Cargo.lock
generated
5
Cargo.lock
generated
@ -307,9 +307,9 @@ checksum = "349f9b6a179ed607305526ca489b34ad0a41aed5f7980fa90eb03160b69598fb"
|
||||
|
||||
[[package]]
|
||||
name = "bitflags"
|
||||
version = "1.2.1"
|
||||
version = "1.3.2"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "cf1de2fe8c75bc145a2f577add951f8134889b4795d47466a54a5c846d691693"
|
||||
checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a"
|
||||
|
||||
[[package]]
|
||||
name = "bitvec"
|
||||
@ -5411,6 +5411,7 @@ version = "1.8.0"
|
||||
dependencies = [
|
||||
"assert_matches",
|
||||
"bincode",
|
||||
"bitflags",
|
||||
"blake3 1.0.0",
|
||||
"borsh",
|
||||
"borsh-derive",
|
||||
|
@ -30,7 +30,7 @@ use solana_sdk::signature::Signer;
|
||||
use solana_sdk::system_instruction;
|
||||
use solana_sdk::system_transaction;
|
||||
use solana_sdk::timing::{duration_as_us, timestamp};
|
||||
use solana_sdk::transaction::Transaction;
|
||||
use solana_sdk::transaction::{Transaction, VersionedTransaction};
|
||||
use solana_streamer::socket::SocketAddrSpace;
|
||||
use std::collections::VecDeque;
|
||||
use std::sync::atomic::Ordering;
|
||||
@ -287,7 +287,7 @@ fn bench_banking_stage_multi_programs(bencher: &mut Bencher) {
|
||||
fn simulate_process_entries(
|
||||
randomize_txs: bool,
|
||||
mint_keypair: &Keypair,
|
||||
mut tx_vector: Vec<Transaction>,
|
||||
mut tx_vector: Vec<VersionedTransaction>,
|
||||
genesis_config: &GenesisConfig,
|
||||
keypairs: &[Keypair],
|
||||
initial_lamports: u64,
|
||||
@ -301,12 +301,15 @@ fn simulate_process_entries(
|
||||
}
|
||||
|
||||
for i in (0..num_accounts).step_by(2) {
|
||||
tx_vector.push(system_transaction::transfer(
|
||||
&keypairs[i],
|
||||
&keypairs[i + 1].pubkey(),
|
||||
initial_lamports,
|
||||
bank.last_blockhash(),
|
||||
));
|
||||
tx_vector.push(
|
||||
system_transaction::transfer(
|
||||
&keypairs[i],
|
||||
&keypairs[i + 1].pubkey(),
|
||||
initial_lamports,
|
||||
bank.last_blockhash(),
|
||||
)
|
||||
.into(),
|
||||
);
|
||||
}
|
||||
|
||||
// Transfer lamports to each other
|
||||
@ -315,7 +318,7 @@ fn simulate_process_entries(
|
||||
hash: next_hash(&bank.last_blockhash(), 1, &tx_vector),
|
||||
transactions: tx_vector,
|
||||
};
|
||||
process_entries(&bank, &mut [entry], randomize_txs, None, None).unwrap();
|
||||
process_entries(&bank, vec![entry], randomize_txs, None, None).unwrap();
|
||||
}
|
||||
|
||||
#[allow(clippy::same_item_push)]
|
||||
@ -335,7 +338,7 @@ fn bench_process_entries(randomize_txs: bool, bencher: &mut Bencher) {
|
||||
} = create_genesis_config((num_accounts + 1) as u64 * initial_lamports);
|
||||
|
||||
let mut keypairs: Vec<Keypair> = vec![];
|
||||
let tx_vector: Vec<Transaction> = Vec::with_capacity(num_accounts / 2);
|
||||
let tx_vector: Vec<VersionedTransaction> = Vec::with_capacity(num_accounts / 2);
|
||||
|
||||
for _ in 0..num_accounts {
|
||||
let keypair = Keypair::new();
|
||||
|
@ -20,7 +20,7 @@ fn make_test_entry(txs_per_entry: u64) -> Entry {
|
||||
Entry {
|
||||
num_hashes: 100_000,
|
||||
hash: Hash::default(),
|
||||
transactions: vec![test_tx::test_tx(); txs_per_entry as usize],
|
||||
transactions: vec![test_tx::test_tx().into(); txs_per_entry as usize],
|
||||
}
|
||||
}
|
||||
fn make_large_unchained_entries(txs_per_entry: u64, num_entries: u64) -> Vec<Entry> {
|
||||
|
@ -34,17 +34,15 @@ use solana_sdk::{
|
||||
},
|
||||
message::Message,
|
||||
pubkey::Pubkey,
|
||||
sanitized_transaction::SanitizedTransaction,
|
||||
short_vec::decode_shortu16_len,
|
||||
signature::Signature,
|
||||
timing::{duration_as_ms, timestamp, AtomicInterval},
|
||||
transaction::{self, Transaction, TransactionError},
|
||||
transaction::{self, SanitizedTransaction, TransactionError, VersionedTransaction},
|
||||
};
|
||||
use solana_transaction_status::token_balances::{
|
||||
collect_token_balances, TransactionTokenBalancesSet,
|
||||
};
|
||||
use std::{
|
||||
borrow::Cow,
|
||||
cmp,
|
||||
collections::{HashMap, VecDeque},
|
||||
env,
|
||||
@ -729,9 +727,9 @@ impl BankingStage {
|
||||
}
|
||||
|
||||
#[allow(clippy::match_wild_err_arm)]
|
||||
fn record_transactions<'a>(
|
||||
fn record_transactions(
|
||||
bank_slot: Slot,
|
||||
txs: impl Iterator<Item = &'a Transaction>,
|
||||
txs: &[SanitizedTransaction],
|
||||
results: &[TransactionExecutionResult],
|
||||
recorder: &TransactionRecorder,
|
||||
) -> (Result<usize, PohRecorderError>, Vec<usize>) {
|
||||
@ -740,9 +738,9 @@ impl BankingStage {
|
||||
.iter()
|
||||
.zip(txs)
|
||||
.enumerate()
|
||||
.filter_map(|(i, ((r, _n), x))| {
|
||||
.filter_map(|(i, ((r, _n), tx))| {
|
||||
if Bank::can_commit(r) {
|
||||
Some((x.clone(), i))
|
||||
Some((tx.to_versioned_transaction(), i))
|
||||
} else {
|
||||
None
|
||||
}
|
||||
@ -835,7 +833,7 @@ impl BankingStage {
|
||||
|
||||
let mut record_time = Measure::start("record_time");
|
||||
let (num_to_commit, retryable_record_txs) =
|
||||
Self::record_transactions(bank.slot(), batch.transactions_iter(), &results, poh);
|
||||
Self::record_transactions(bank.slot(), batch.sanitized_transactions(), &results, poh);
|
||||
inc_new_counter_info!(
|
||||
"banking_stage-record_transactions_num_to_commit",
|
||||
*num_to_commit.as_ref().unwrap_or(&0)
|
||||
@ -865,7 +863,7 @@ impl BankingStage {
|
||||
|
||||
bank_utils::find_and_send_votes(sanitized_txs, &tx_results, Some(gossip_vote_sender));
|
||||
if let Some(transaction_status_sender) = transaction_status_sender {
|
||||
let txs = batch.transactions_iter().cloned().collect();
|
||||
let txs = batch.sanitized_transactions().to_vec();
|
||||
let post_balances = bank.collect_balances(batch);
|
||||
let post_token_balances = collect_token_balances(bank, batch, &mut mint_decimals);
|
||||
transaction_status_sender.send_transaction_status_batch(
|
||||
@ -1050,19 +1048,22 @@ impl BankingStage {
|
||||
libsecp256k1_0_5_upgrade_enabled: bool,
|
||||
cost_tracker: &Arc<RwLock<CostTracker>>,
|
||||
banking_stage_stats: &BankingStageStats,
|
||||
) -> (Vec<SanitizedTransaction<'static>>, Vec<usize>, Vec<usize>) {
|
||||
) -> (Vec<SanitizedTransaction>, Vec<usize>, Vec<usize>) {
|
||||
let mut retryable_transaction_packet_indexes: Vec<usize> = vec![];
|
||||
|
||||
let verified_transactions_with_packet_indexes: Vec<_> = transaction_indexes
|
||||
.iter()
|
||||
.filter_map(|tx_index| {
|
||||
let p = &msgs.packets[*tx_index];
|
||||
let tx: Transaction = limited_deserialize(&p.data[0..p.meta.size]).ok()?;
|
||||
tx.verify_precompiles(libsecp256k1_0_5_upgrade_enabled)
|
||||
.ok()?;
|
||||
let tx: VersionedTransaction = limited_deserialize(&p.data[0..p.meta.size]).ok()?;
|
||||
let message_bytes = Self::packet_message(p)?;
|
||||
let message_hash = Message::hash_raw_message(message_bytes);
|
||||
let tx = SanitizedTransaction::try_create(Cow::Owned(tx), message_hash).ok()?;
|
||||
let tx = SanitizedTransaction::try_create(tx, message_hash, |_| {
|
||||
Err(TransactionError::UnsupportedVersion)
|
||||
})
|
||||
.ok()?;
|
||||
tx.verify_precompiles(libsecp256k1_0_5_upgrade_enabled)
|
||||
.ok()?;
|
||||
Some((tx, *tx_index))
|
||||
})
|
||||
.collect();
|
||||
@ -1565,12 +1566,12 @@ mod tests {
|
||||
signature::{Keypair, Signer},
|
||||
system_instruction::SystemError,
|
||||
system_transaction,
|
||||
transaction::TransactionError,
|
||||
transaction::{Transaction, TransactionError},
|
||||
};
|
||||
use solana_streamer::socket::SocketAddrSpace;
|
||||
use solana_transaction_status::TransactionWithStatusMeta;
|
||||
use std::{
|
||||
convert::TryInto,
|
||||
convert::{TryFrom, TryInto},
|
||||
net::SocketAddr,
|
||||
path::Path,
|
||||
sync::{
|
||||
@ -1794,7 +1795,7 @@ mod tests {
|
||||
if !entries.is_empty() {
|
||||
blockhash = entries.last().unwrap().hash;
|
||||
for entry in entries {
|
||||
bank.process_transactions(entry.transactions.iter())
|
||||
bank.process_entry_transactions(entry.transactions)
|
||||
.iter()
|
||||
.for_each(|x| assert_eq!(*x, Ok(())));
|
||||
}
|
||||
@ -1906,8 +1907,8 @@ mod tests {
|
||||
.collect();
|
||||
|
||||
let bank = Bank::new_no_wallclock_throttle_for_tests(&genesis_config);
|
||||
for entry in &entries {
|
||||
bank.process_transactions(entry.transactions.iter())
|
||||
for entry in entries {
|
||||
bank.process_entry_transactions(entry.transactions)
|
||||
.iter()
|
||||
.for_each(|x| assert_eq!(*x, Ok(())));
|
||||
}
|
||||
@ -1920,6 +1921,12 @@ mod tests {
|
||||
Blockstore::destroy(&ledger_path).unwrap();
|
||||
}
|
||||
|
||||
fn sanitize_transactions(txs: Vec<Transaction>) -> Vec<SanitizedTransaction> {
|
||||
txs.into_iter()
|
||||
.map(|tx| SanitizedTransaction::try_from(tx).unwrap())
|
||||
.collect()
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_bank_record_transactions() {
|
||||
solana_logger::setup();
|
||||
@ -1964,20 +1971,15 @@ mod tests {
|
||||
let keypair2 = Keypair::new();
|
||||
let pubkey2 = solana_sdk::pubkey::new_rand();
|
||||
|
||||
let transactions = vec![
|
||||
let txs = sanitize_transactions(vec![
|
||||
system_transaction::transfer(&mint_keypair, &pubkey, 1, genesis_config.hash()),
|
||||
system_transaction::transfer(&keypair2, &pubkey2, 1, genesis_config.hash()),
|
||||
];
|
||||
]);
|
||||
|
||||
let mut results = vec![(Ok(()), None), (Ok(()), None)];
|
||||
let _ = BankingStage::record_transactions(
|
||||
bank.slot(),
|
||||
transactions.iter(),
|
||||
&results,
|
||||
&recorder,
|
||||
);
|
||||
let _ = BankingStage::record_transactions(bank.slot(), &txs, &results, &recorder);
|
||||
let (_bank, (entry, _tick_height)) = entry_receiver.recv().unwrap();
|
||||
assert_eq!(entry.transactions.len(), transactions.len());
|
||||
assert_eq!(entry.transactions.len(), txs.len());
|
||||
|
||||
// InstructionErrors should still be recorded
|
||||
results[0] = (
|
||||
@ -1987,39 +1989,27 @@ mod tests {
|
||||
)),
|
||||
None,
|
||||
);
|
||||
let (res, retryable) = BankingStage::record_transactions(
|
||||
bank.slot(),
|
||||
transactions.iter(),
|
||||
&results,
|
||||
&recorder,
|
||||
);
|
||||
let (res, retryable) =
|
||||
BankingStage::record_transactions(bank.slot(), &txs, &results, &recorder);
|
||||
res.unwrap();
|
||||
assert!(retryable.is_empty());
|
||||
let (_bank, (entry, _tick_height)) = entry_receiver.recv().unwrap();
|
||||
assert_eq!(entry.transactions.len(), transactions.len());
|
||||
assert_eq!(entry.transactions.len(), txs.len());
|
||||
|
||||
// Other TransactionErrors should not be recorded
|
||||
results[0] = (Err(TransactionError::AccountNotFound), None);
|
||||
let (res, retryable) = BankingStage::record_transactions(
|
||||
bank.slot(),
|
||||
transactions.iter(),
|
||||
&results,
|
||||
&recorder,
|
||||
);
|
||||
let (res, retryable) =
|
||||
BankingStage::record_transactions(bank.slot(), &txs, &results, &recorder);
|
||||
res.unwrap();
|
||||
assert!(retryable.is_empty());
|
||||
let (_bank, (entry, _tick_height)) = entry_receiver.recv().unwrap();
|
||||
assert_eq!(entry.transactions.len(), transactions.len() - 1);
|
||||
assert_eq!(entry.transactions.len(), txs.len() - 1);
|
||||
|
||||
// Once bank is set to a new bank (setting bank.slot() + 1 in record_transactions),
|
||||
// record_transactions should throw MaxHeightReached and return the set of retryable
|
||||
// txs
|
||||
let (res, retryable) = BankingStage::record_transactions(
|
||||
bank.slot() + 1,
|
||||
transactions.iter(),
|
||||
&results,
|
||||
&recorder,
|
||||
);
|
||||
let (res, retryable) =
|
||||
BankingStage::record_transactions(bank.slot() + 1, &txs, &results, &recorder);
|
||||
assert_matches!(res, Err(PohRecorderError::MaxHeightReached));
|
||||
// The first result was an error so it's filtered out. The second result was Ok(),
|
||||
// so it should be marked as retryable
|
||||
|
@ -94,7 +94,7 @@ impl BroadcastRun for BroadcastDuplicatesRun {
|
||||
// Update the recent blockhash based on transactions in the entries
|
||||
for entry in &receive_results.entries {
|
||||
if !entry.transactions.is_empty() {
|
||||
self.recent_blockhash = Some(entry.transactions[0].message.recent_blockhash);
|
||||
self.recent_blockhash = Some(*entry.transactions[0].message.recent_blockhash());
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
@ -10,7 +10,7 @@
|
||||
use crate::execute_cost_table::ExecuteCostTable;
|
||||
use log::*;
|
||||
use solana_ledger::block_cost_limits::*;
|
||||
use solana_sdk::{pubkey::Pubkey, sanitized_transaction::SanitizedTransaction};
|
||||
use solana_sdk::{pubkey::Pubkey, transaction::SanitizedTransaction};
|
||||
use std::collections::HashMap;
|
||||
|
||||
const MAX_WRITABLE_ACCOUNTS: usize = 256;
|
||||
@ -121,7 +121,7 @@ impl CostModel {
|
||||
|
||||
// calculate account access cost
|
||||
let message = transaction.message();
|
||||
message.account_keys.iter().enumerate().for_each(|(i, k)| {
|
||||
message.account_keys_iter().enumerate().for_each(|(i, k)| {
|
||||
let is_writable = message.is_writable(i);
|
||||
|
||||
if is_writable {
|
||||
@ -173,10 +173,8 @@ impl CostModel {
|
||||
fn find_transaction_cost(&self, transaction: &SanitizedTransaction) -> u64 {
|
||||
let mut cost: u64 = 0;
|
||||
|
||||
for instruction in &transaction.message().instructions {
|
||||
let program_id =
|
||||
transaction.message().account_keys[instruction.program_id_index as usize];
|
||||
let instruction_cost = self.find_instruction_cost(&program_id);
|
||||
for (program_id, instruction) in transaction.message().program_instructions_iter() {
|
||||
let instruction_cost = self.find_instruction_cost(program_id);
|
||||
trace!(
|
||||
"instruction {:?} has cost of {}",
|
||||
instruction,
|
||||
|
@ -5,7 +5,7 @@
|
||||
//! - add_transaction_cost(&tx), mutable function to accumulate `tx` cost to tracker.
|
||||
//!
|
||||
use crate::cost_model::{CostModel, CostModelError, TransactionCost};
|
||||
use solana_sdk::{clock::Slot, pubkey::Pubkey, sanitized_transaction::SanitizedTransaction};
|
||||
use solana_sdk::{clock::Slot, pubkey::Pubkey, transaction::SanitizedTransaction};
|
||||
use std::{
|
||||
collections::HashMap,
|
||||
sync::{Arc, RwLock},
|
||||
|
@ -18,13 +18,9 @@ use solana_perf::perf_libs;
|
||||
use solana_perf::recycler::Recycler;
|
||||
use solana_rayon_threadlimit::get_thread_count;
|
||||
use solana_sdk::hash::Hash;
|
||||
use solana_sdk::packet::PACKET_DATA_SIZE;
|
||||
use solana_sdk::sanitized_transaction::SanitizedTransaction;
|
||||
use solana_sdk::timing;
|
||||
use solana_sdk::transaction::{Result, Transaction, TransactionError};
|
||||
use std::borrow::Cow;
|
||||
use solana_sdk::transaction::{Result, SanitizedTransaction, Transaction, VersionedTransaction};
|
||||
use std::cell::RefCell;
|
||||
use std::convert::TryFrom;
|
||||
use std::ffi::OsStr;
|
||||
use std::sync::mpsc::{Receiver, Sender};
|
||||
use std::sync::Once;
|
||||
@ -118,32 +114,15 @@ pub struct Entry {
|
||||
/// An unordered list of transactions that were observed before the Entry ID was
|
||||
/// generated. They may have been observed before a previous Entry ID but were
|
||||
/// pushed back into this list to ensure deterministic interpretation of the ledger.
|
||||
pub transactions: Vec<Transaction>,
|
||||
pub transactions: Vec<VersionedTransaction>,
|
||||
}
|
||||
|
||||
/// Typed entry to distinguish between transaction and tick entries
|
||||
pub enum EntryType<'a> {
|
||||
Transactions(Vec<SanitizedTransaction<'a>>),
|
||||
pub enum EntryType {
|
||||
Transactions(Vec<SanitizedTransaction>),
|
||||
Tick(Hash),
|
||||
}
|
||||
|
||||
impl<'a> TryFrom<&'a Entry> for EntryType<'a> {
|
||||
type Error = TransactionError;
|
||||
fn try_from(entry: &'a Entry) -> Result<Self> {
|
||||
if entry.transactions.is_empty() {
|
||||
Ok(EntryType::Tick(entry.hash))
|
||||
} else {
|
||||
Ok(EntryType::Transactions(
|
||||
entry
|
||||
.transactions
|
||||
.iter()
|
||||
.map(SanitizedTransaction::try_from)
|
||||
.collect::<Result<_>>()?,
|
||||
))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl Entry {
|
||||
/// Creates the next Entry `num_hashes` after `start_hash`.
|
||||
pub fn new(prev_hash: &Hash, mut num_hashes: u64, transactions: Vec<Transaction>) -> Self {
|
||||
@ -153,6 +132,7 @@ impl Entry {
|
||||
num_hashes = 1;
|
||||
}
|
||||
|
||||
let transactions = transactions.into_iter().map(Into::into).collect::<Vec<_>>();
|
||||
let hash = next_hash(prev_hash, num_hashes, &transactions);
|
||||
Entry {
|
||||
num_hashes,
|
||||
@ -201,7 +181,7 @@ impl Entry {
|
||||
}
|
||||
}
|
||||
|
||||
pub fn hash_transactions(transactions: &[Transaction]) -> Hash {
|
||||
pub fn hash_transactions(transactions: &[VersionedTransaction]) -> Hash {
|
||||
// a hash of a slice of transactions only needs to hash the signatures
|
||||
let signatures: Vec<_> = transactions
|
||||
.iter()
|
||||
@ -219,7 +199,11 @@ pub fn hash_transactions(transactions: &[Transaction]) -> Hash {
|
||||
/// a signature, the final hash will be a hash of both the previous ID and
|
||||
/// the signature. If num_hashes is zero and there's no transaction data,
|
||||
/// start_hash is returned.
|
||||
pub fn next_hash(start_hash: &Hash, num_hashes: u64, transactions: &[Transaction]) -> Hash {
|
||||
pub fn next_hash(
|
||||
start_hash: &Hash,
|
||||
num_hashes: u64,
|
||||
transactions: &[VersionedTransaction],
|
||||
) -> Hash {
|
||||
if num_hashes == 0 && transactions.is_empty() {
|
||||
return *start_hash;
|
||||
}
|
||||
@ -329,6 +313,32 @@ impl EntryVerificationState {
|
||||
}
|
||||
}
|
||||
|
||||
pub fn verify_transactions(
|
||||
entries: Vec<Entry>,
|
||||
verify: Arc<dyn Fn(VersionedTransaction) -> Result<SanitizedTransaction> + Send + Sync>,
|
||||
) -> Result<Vec<EntryType>> {
|
||||
PAR_THREAD_POOL.with(|thread_pool| {
|
||||
thread_pool.borrow().install(|| {
|
||||
entries
|
||||
.into_par_iter()
|
||||
.map(|entry| {
|
||||
if entry.transactions.is_empty() {
|
||||
Ok(EntryType::Tick(entry.hash))
|
||||
} else {
|
||||
Ok(EntryType::Transactions(
|
||||
entry
|
||||
.transactions
|
||||
.into_par_iter()
|
||||
.map(verify.as_ref())
|
||||
.collect::<Result<Vec<_>>>()?,
|
||||
))
|
||||
}
|
||||
})
|
||||
.collect()
|
||||
})
|
||||
})
|
||||
}
|
||||
|
||||
fn compare_hashes(computed_hash: Hash, ref_entry: &Entry) -> bool {
|
||||
let actual = if !ref_entry.transactions.is_empty() {
|
||||
let tx_hash = hash_transactions(&ref_entry.transactions);
|
||||
@ -358,12 +368,6 @@ pub trait EntrySlice {
|
||||
fn verify_tick_hash_count(&self, tick_hash_count: &mut u64, hashes_per_tick: u64) -> bool;
|
||||
/// Counts tick entries
|
||||
fn tick_count(&self) -> u64;
|
||||
fn verify_and_hash_transactions(
|
||||
&self,
|
||||
skip_verification: bool,
|
||||
libsecp256k1_0_5_upgrade_enabled: bool,
|
||||
verify_tx_signatures_len: bool,
|
||||
) -> Result<Vec<EntryType<'_>>>;
|
||||
}
|
||||
|
||||
impl EntrySlice for [Entry] {
|
||||
@ -514,52 +518,6 @@ impl EntrySlice for [Entry] {
|
||||
}
|
||||
}
|
||||
|
||||
fn verify_and_hash_transactions<'a>(
|
||||
&'a self,
|
||||
skip_verification: bool,
|
||||
libsecp256k1_0_5_upgrade_enabled: bool,
|
||||
verify_tx_signatures_len: bool,
|
||||
) -> Result<Vec<EntryType<'a>>> {
|
||||
let verify_and_hash = |tx: &'a Transaction| -> Result<SanitizedTransaction<'a>> {
|
||||
let message_hash = if !skip_verification {
|
||||
let size =
|
||||
bincode::serialized_size(tx).map_err(|_| TransactionError::SanitizeFailure)?;
|
||||
if size > PACKET_DATA_SIZE as u64 {
|
||||
return Err(TransactionError::SanitizeFailure);
|
||||
}
|
||||
tx.verify_precompiles(libsecp256k1_0_5_upgrade_enabled)?;
|
||||
if verify_tx_signatures_len && !tx.verify_signatures_len() {
|
||||
return Err(TransactionError::SanitizeFailure);
|
||||
}
|
||||
tx.verify_and_hash_message()?
|
||||
} else {
|
||||
tx.message().hash()
|
||||
};
|
||||
|
||||
SanitizedTransaction::try_create(Cow::Borrowed(tx), message_hash)
|
||||
};
|
||||
|
||||
PAR_THREAD_POOL.with(|thread_pool| {
|
||||
thread_pool.borrow().install(|| {
|
||||
self.par_iter()
|
||||
.map(|entry| {
|
||||
if entry.transactions.is_empty() {
|
||||
Ok(EntryType::Tick(entry.hash))
|
||||
} else {
|
||||
Ok(EntryType::Transactions(
|
||||
entry
|
||||
.transactions
|
||||
.par_iter()
|
||||
.map(verify_and_hash)
|
||||
.collect::<Result<Vec<_>>>()?,
|
||||
))
|
||||
}
|
||||
})
|
||||
.collect()
|
||||
})
|
||||
})
|
||||
}
|
||||
|
||||
fn start_verify(
|
||||
&self,
|
||||
start_hash: &Hash,
|
||||
@ -718,6 +676,7 @@ pub fn create_random_ticks(num_ticks: u64, max_hashes_per_tick: u64, mut hash: H
|
||||
/// Creates the next Tick or Transaction Entry `num_hashes` after `start_hash`.
|
||||
pub fn next_entry(prev_hash: &Hash, num_hashes: u64, transactions: Vec<Transaction>) -> Entry {
|
||||
assert!(num_hashes > 0 || transactions.is_empty());
|
||||
let transactions = transactions.into_iter().map(Into::into).collect::<Vec<_>>();
|
||||
Entry {
|
||||
num_hashes,
|
||||
hash: next_hash(prev_hash, num_hashes, &transactions),
|
||||
@ -728,15 +687,11 @@ pub fn next_entry(prev_hash: &Hash, num_hashes: u64, transactions: Vec<Transacti
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
use crate::entry::Entry;
|
||||
use solana_sdk::{
|
||||
hash::{hash, new_rand as hash_new_rand, Hash},
|
||||
message::Message,
|
||||
packet::PACKET_DATA_SIZE,
|
||||
hash::{hash, Hash},
|
||||
pubkey::Pubkey,
|
||||
signature::{Keypair, Signer},
|
||||
system_instruction, system_transaction,
|
||||
transaction::Transaction,
|
||||
system_transaction,
|
||||
};
|
||||
|
||||
#[test]
|
||||
@ -761,8 +716,8 @@ mod tests {
|
||||
assert!(e0.verify(&zero));
|
||||
|
||||
// Next, swap two transactions and ensure verification fails.
|
||||
e0.transactions[0] = tx1; // <-- attack
|
||||
e0.transactions[1] = tx0;
|
||||
e0.transactions[0] = tx1.into(); // <-- attack
|
||||
e0.transactions[1] = tx0.into();
|
||||
assert!(!e0.verify(&zero));
|
||||
}
|
||||
|
||||
@ -815,7 +770,7 @@ mod tests {
|
||||
let tx0 = system_transaction::transfer(&keypair, &Pubkey::new_unique(), 42, zero);
|
||||
let entry0 = next_entry(&zero, 1, vec![tx0.clone()]);
|
||||
assert_eq!(entry0.num_hashes, 1);
|
||||
assert_eq!(entry0.hash, next_hash(&zero, 1, &[tx0]));
|
||||
assert_eq!(entry0.hash, next_hash(&zero, 1, &[tx0.into()]));
|
||||
}
|
||||
|
||||
#[test]
|
||||
@ -890,155 +845,10 @@ mod tests {
|
||||
assert!(!bad_ticks.verify(&one)); // inductive step, bad
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_verify_and_hash_transactions_sig_len() {
|
||||
let mut rng = rand::thread_rng();
|
||||
let recent_blockhash = hash_new_rand(&mut rng);
|
||||
let from_keypair = Keypair::new();
|
||||
let to_keypair = Keypair::new();
|
||||
let from_pubkey = from_keypair.pubkey();
|
||||
let to_pubkey = to_keypair.pubkey();
|
||||
|
||||
enum TestCase {
|
||||
AddSignature,
|
||||
RemoveSignature,
|
||||
}
|
||||
|
||||
let make_transaction = |case: TestCase| {
|
||||
let message = Message::new(
|
||||
&[system_instruction::transfer(&from_pubkey, &to_pubkey, 1)],
|
||||
Some(&from_pubkey),
|
||||
);
|
||||
let mut tx = Transaction::new(&[&from_keypair], message, recent_blockhash);
|
||||
assert_eq!(tx.message.header.num_required_signatures, 1);
|
||||
match case {
|
||||
TestCase::AddSignature => {
|
||||
let signature = to_keypair.sign_message(&tx.message.serialize());
|
||||
tx.signatures.push(signature);
|
||||
}
|
||||
TestCase::RemoveSignature => {
|
||||
tx.signatures.remove(0);
|
||||
}
|
||||
}
|
||||
tx
|
||||
};
|
||||
// Too few signatures: Sanitization failure
|
||||
{
|
||||
let tx = make_transaction(TestCase::RemoveSignature);
|
||||
let entries = vec![next_entry(&recent_blockhash, 1, vec![tx])];
|
||||
assert_eq!(
|
||||
entries[..]
|
||||
.verify_and_hash_transactions(false, false, false)
|
||||
.err(),
|
||||
Some(TransactionError::SanitizeFailure),
|
||||
);
|
||||
}
|
||||
// Too many signatures: Sanitize failure only with feature switch
|
||||
{
|
||||
let tx = make_transaction(TestCase::AddSignature);
|
||||
let entries = vec![next_entry(&recent_blockhash, 1, vec![tx])];
|
||||
assert!(entries[..]
|
||||
.verify_and_hash_transactions(false, false, false)
|
||||
.is_ok());
|
||||
assert_eq!(
|
||||
entries[..]
|
||||
.verify_and_hash_transactions(false, false, true)
|
||||
.err(),
|
||||
Some(TransactionError::SanitizeFailure)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_verify_and_hash_transactions_load_duplicate_account() {
|
||||
let mut rng = rand::thread_rng();
|
||||
let recent_blockhash = hash_new_rand(&mut rng);
|
||||
let from_keypair = Keypair::new();
|
||||
let to_keypair = Keypair::new();
|
||||
let from_pubkey = from_keypair.pubkey();
|
||||
let to_pubkey = to_keypair.pubkey();
|
||||
|
||||
let make_transaction = || {
|
||||
let mut message = Message::new(
|
||||
&[system_instruction::transfer(&from_pubkey, &to_pubkey, 1)],
|
||||
Some(&from_pubkey),
|
||||
);
|
||||
let to_index = message
|
||||
.account_keys
|
||||
.iter()
|
||||
.position(|k| k == &to_pubkey)
|
||||
.unwrap();
|
||||
message.account_keys[to_index] = from_pubkey;
|
||||
Transaction::new(&[&from_keypair], message, recent_blockhash)
|
||||
};
|
||||
|
||||
// Duplicate account
|
||||
{
|
||||
let tx = make_transaction();
|
||||
let entries = vec![next_entry(&recent_blockhash, 1, vec![tx])];
|
||||
assert_eq!(
|
||||
entries[..]
|
||||
.verify_and_hash_transactions(false, false, false)
|
||||
.err(),
|
||||
Some(TransactionError::AccountLoadedTwice)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_verify_and_hash_transactions_packet_data_size() {
|
||||
let mut rng = rand::thread_rng();
|
||||
let recent_blockhash = hash_new_rand(&mut rng);
|
||||
let keypair = Keypair::new();
|
||||
let pubkey = keypair.pubkey();
|
||||
let make_transaction = |size| {
|
||||
let ixs: Vec<_> = std::iter::repeat_with(|| {
|
||||
system_instruction::transfer(&pubkey, &Pubkey::new_unique(), 1)
|
||||
})
|
||||
.take(size)
|
||||
.collect();
|
||||
let message = Message::new(&ixs[..], Some(&pubkey));
|
||||
Transaction::new(&[&keypair], message, recent_blockhash)
|
||||
};
|
||||
// Small transaction.
|
||||
{
|
||||
let tx = make_transaction(5);
|
||||
let entries = vec![next_entry(&recent_blockhash, 1, vec![tx.clone()])];
|
||||
assert!(bincode::serialized_size(&tx).unwrap() <= PACKET_DATA_SIZE as u64);
|
||||
assert!(entries[..]
|
||||
.verify_and_hash_transactions(false, false, false)
|
||||
.is_ok());
|
||||
}
|
||||
// Big transaction.
|
||||
{
|
||||
let tx = make_transaction(25);
|
||||
let entries = vec![next_entry(&recent_blockhash, 1, vec![tx.clone()])];
|
||||
assert!(bincode::serialized_size(&tx).unwrap() > PACKET_DATA_SIZE as u64);
|
||||
assert_eq!(
|
||||
entries[..]
|
||||
.verify_and_hash_transactions(false, false, false)
|
||||
.err(),
|
||||
Some(TransactionError::SanitizeFailure)
|
||||
);
|
||||
}
|
||||
// Assert that verify fails as soon as serialized
|
||||
// size exceeds packet data size.
|
||||
for size in 1..30 {
|
||||
let tx = make_transaction(size);
|
||||
let entries = vec![next_entry(&recent_blockhash, 1, vec![tx.clone()])];
|
||||
assert_eq!(
|
||||
bincode::serialized_size(&tx).unwrap() <= PACKET_DATA_SIZE as u64,
|
||||
entries[..]
|
||||
.verify_and_hash_transactions(false, false, false)
|
||||
.is_ok(),
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_verify_tick_hash_count() {
|
||||
let hashes_per_tick = 10;
|
||||
let tx = Transaction::default();
|
||||
let tx = VersionedTransaction::default();
|
||||
|
||||
let no_hash_tx_entry = Entry {
|
||||
transactions: vec![tx.clone()],
|
||||
|
@ -47,10 +47,10 @@ use solana_sdk::{
|
||||
native_token::{lamports_to_sol, sol_to_lamports, Sol},
|
||||
pubkey::Pubkey,
|
||||
rent::Rent,
|
||||
sanitized_transaction::SanitizedTransaction,
|
||||
shred_version::compute_shred_version,
|
||||
stake::{self, state::StakeState},
|
||||
system_program,
|
||||
transaction::{SanitizedTransaction, TransactionError},
|
||||
};
|
||||
use solana_stake_program::stake_state::{self, PointValue};
|
||||
use solana_vote_program::{
|
||||
@ -58,7 +58,6 @@ use solana_vote_program::{
|
||||
vote_state::{self, VoteState},
|
||||
};
|
||||
use std::{
|
||||
borrow::Cow,
|
||||
collections::{BTreeMap, BTreeSet, HashMap, HashSet},
|
||||
ffi::OsStr,
|
||||
fs::{self, File},
|
||||
@ -121,7 +120,7 @@ fn output_entry(
|
||||
method: &LedgerOutputMethod,
|
||||
slot: Slot,
|
||||
entry_index: usize,
|
||||
entry: &Entry,
|
||||
entry: Entry,
|
||||
) {
|
||||
match method {
|
||||
LedgerOutputMethod::Print => {
|
||||
@ -132,10 +131,11 @@ fn output_entry(
|
||||
entry.hash,
|
||||
entry.transactions.len()
|
||||
);
|
||||
for (transactions_index, transaction) in entry.transactions.iter().enumerate() {
|
||||
for (transactions_index, transaction) in entry.transactions.into_iter().enumerate() {
|
||||
println!(" Transaction {}", transactions_index);
|
||||
let transaction_status = blockstore
|
||||
.read_transaction_status((transaction.signatures[0], slot))
|
||||
let tx_signature = transaction.signatures[0];
|
||||
let tx_status = blockstore
|
||||
.read_transaction_status((tx_signature, slot))
|
||||
.unwrap_or_else(|err| {
|
||||
eprintln!(
|
||||
"Failed to read transaction status for {} at slot {}: {}",
|
||||
@ -145,13 +145,16 @@ fn output_entry(
|
||||
})
|
||||
.map(|transaction_status| transaction_status.into());
|
||||
|
||||
solana_cli_output::display::println_transaction(
|
||||
transaction,
|
||||
&transaction_status,
|
||||
" ",
|
||||
None,
|
||||
None,
|
||||
);
|
||||
if let Some(legacy_tx) = transaction.into_legacy_transaction() {
|
||||
solana_cli_output::display::println_transaction(
|
||||
&legacy_tx, &tx_status, " ", None, None,
|
||||
);
|
||||
} else {
|
||||
eprintln!(
|
||||
"Failed to print unsupported transaction for {} at slot {}",
|
||||
tx_signature, slot
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
LedgerOutputMethod::Json => {
|
||||
@ -199,7 +202,7 @@ fn output_slot(
|
||||
}
|
||||
|
||||
if verbose_level >= 2 {
|
||||
for (entry_index, entry) in entries.iter().enumerate() {
|
||||
for (entry_index, entry) in entries.into_iter().enumerate() {
|
||||
output_entry(blockstore, method, slot, entry_index, entry);
|
||||
}
|
||||
|
||||
@ -208,26 +211,41 @@ fn output_slot(
|
||||
let mut transactions = 0;
|
||||
let mut hashes = 0;
|
||||
let mut program_ids = HashMap::new();
|
||||
for entry in &entries {
|
||||
transactions += entry.transactions.len();
|
||||
hashes += entry.num_hashes;
|
||||
for transaction in &entry.transactions {
|
||||
for instruction in &transaction.message().instructions {
|
||||
let program_id =
|
||||
transaction.message().account_keys[instruction.program_id_index as usize];
|
||||
*program_ids.entry(program_id).or_insert(0) += 1;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
let hash = if let Some(entry) = entries.last() {
|
||||
let blockhash = if let Some(entry) = entries.last() {
|
||||
entry.hash
|
||||
} else {
|
||||
Hash::default()
|
||||
};
|
||||
|
||||
for entry in entries {
|
||||
transactions += entry.transactions.len();
|
||||
hashes += entry.num_hashes;
|
||||
for transaction in entry.transactions {
|
||||
let tx_signature = transaction.signatures[0];
|
||||
let sanitize_result =
|
||||
SanitizedTransaction::try_create(transaction, Hash::default(), |_| {
|
||||
Err(TransactionError::UnsupportedVersion)
|
||||
});
|
||||
|
||||
match sanitize_result {
|
||||
Ok(transaction) => {
|
||||
for (program_id, _) in transaction.message().program_instructions_iter() {
|
||||
*program_ids.entry(*program_id).or_insert(0) += 1;
|
||||
}
|
||||
}
|
||||
Err(err) => {
|
||||
warn!(
|
||||
"Failed to analyze unsupported transaction {}: {:?}",
|
||||
tx_signature, err
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
println!(
|
||||
" Transactions: {} hashes: {} block_hash: {}",
|
||||
transactions, hashes, hash,
|
||||
transactions, hashes, blockhash,
|
||||
);
|
||||
println!(" Programs: {:?}", program_ids);
|
||||
}
|
||||
@ -745,54 +763,56 @@ fn compute_slot_cost(blockstore: &Blockstore, slot: Slot) -> Result<(), String>
|
||||
.get_slot_entries_with_shred_info(slot, 0, false)
|
||||
.map_err(|err| format!(" Slot: {}, Failed to load entries, err {:?}", slot, err))?;
|
||||
|
||||
let mut transactions = 0;
|
||||
let mut programs = 0;
|
||||
let num_entries = entries.len();
|
||||
let mut num_transactions = 0;
|
||||
let mut num_programs = 0;
|
||||
|
||||
let mut program_ids = HashMap::new();
|
||||
let mut cost_model = CostModel::default();
|
||||
cost_model.initialize_cost_table(&blockstore.read_program_costs().unwrap());
|
||||
let cost_model = Arc::new(RwLock::new(cost_model));
|
||||
let mut cost_tracker = CostTracker::new(cost_model.clone());
|
||||
|
||||
for entry in &entries {
|
||||
transactions += entry.transactions.len();
|
||||
for entry in entries {
|
||||
num_transactions += entry.transactions.len();
|
||||
let mut cost_model = cost_model.write().unwrap();
|
||||
for transaction in &entry.transactions {
|
||||
programs += transaction.message().instructions.len();
|
||||
let transaction =
|
||||
match SanitizedTransaction::try_create(Cow::Borrowed(transaction), Hash::default())
|
||||
entry
|
||||
.transactions
|
||||
.into_iter()
|
||||
.filter_map(|transaction| {
|
||||
SanitizedTransaction::try_create(transaction, Hash::default(), |_| {
|
||||
Err(TransactionError::UnsupportedVersion)
|
||||
})
|
||||
.map_err(|err| {
|
||||
warn!("Failed to compute cost of transaction: {:?}", err);
|
||||
})
|
||||
.ok()
|
||||
})
|
||||
.for_each(|transaction| {
|
||||
num_programs += transaction.message().instructions().len();
|
||||
|
||||
let tx_cost = cost_model.calculate_cost(&transaction);
|
||||
if cost_tracker.try_add(tx_cost).is_err() {
|
||||
println!(
|
||||
"Slot: {}, CostModel rejected transaction {:?}, stats {:?}!",
|
||||
slot,
|
||||
transaction,
|
||||
cost_tracker.get_stats()
|
||||
);
|
||||
}
|
||||
for (program_id, _instruction) in transaction.message().program_instructions_iter()
|
||||
{
|
||||
Ok(tx) => tx,
|
||||
Err(err) => {
|
||||
warn!(
|
||||
"failed to sanitize transaction, err {:?}, tx {:?}",
|
||||
err, transaction
|
||||
);
|
||||
continue;
|
||||
}
|
||||
};
|
||||
let tx_cost = cost_model.calculate_cost(&transaction);
|
||||
if cost_tracker.try_add(tx_cost).is_err() {
|
||||
println!(
|
||||
"Slot: {}, CostModel rejected transaction {:?}, stats {:?}!",
|
||||
slot,
|
||||
transaction,
|
||||
cost_tracker.get_stats()
|
||||
);
|
||||
}
|
||||
for instruction in &transaction.message().instructions {
|
||||
let program_id =
|
||||
transaction.message().account_keys[instruction.program_id_index as usize];
|
||||
*program_ids.entry(program_id).or_insert(0) += 1;
|
||||
}
|
||||
}
|
||||
*program_ids.entry(*program_id).or_insert(0) += 1;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
println!(
|
||||
"Slot: {}, Entries: {}, Transactions: {}, Programs {}, {:?}",
|
||||
slot,
|
||||
entries.len(),
|
||||
transactions,
|
||||
programs,
|
||||
num_entries,
|
||||
num_transactions,
|
||||
num_programs,
|
||||
cost_tracker.get_stats()
|
||||
);
|
||||
println!(" Programs: {:?}", program_ids);
|
||||
|
@ -35,7 +35,7 @@ use {
|
||||
sanitize::Sanitize,
|
||||
signature::{Keypair, Signature, Signer},
|
||||
timing::timestamp,
|
||||
transaction::Transaction,
|
||||
transaction::VersionedTransaction,
|
||||
},
|
||||
solana_storage_proto::{StoredExtendedRewards, StoredTransactionStatusMeta},
|
||||
solana_transaction_status::{
|
||||
@ -1890,9 +1890,12 @@ impl Blockstore {
|
||||
if slot_meta.is_full() {
|
||||
let slot_entries = self.get_slot_entries(slot, 0)?;
|
||||
if !slot_entries.is_empty() {
|
||||
let blockhash = slot_entries
|
||||
.last()
|
||||
.map(|entry| entry.hash)
|
||||
.unwrap_or_else(|| panic!("Rooted slot {:?} must have blockhash", slot));
|
||||
let slot_transaction_iterator = slot_entries
|
||||
.iter()
|
||||
.cloned()
|
||||
.into_iter()
|
||||
.flat_map(|entry| entry.transactions)
|
||||
.map(|transaction| {
|
||||
if let Err(err) = transaction.sanitize() {
|
||||
@ -1917,9 +1920,6 @@ impl Blockstore {
|
||||
Hash::default()
|
||||
};
|
||||
|
||||
let blockhash = get_last_hash(slot_entries.iter())
|
||||
.unwrap_or_else(|| panic!("Rooted slot {:?} must have blockhash", slot));
|
||||
|
||||
let rewards = self
|
||||
.rewards_cf
|
||||
.get_protobuf_or_bincode::<StoredExtendedRewards>(slot)?
|
||||
@ -1937,7 +1937,7 @@ impl Blockstore {
|
||||
blockhash: blockhash.to_string(),
|
||||
parent_slot: slot_meta.parent_slot,
|
||||
transactions: self
|
||||
.map_transactions_to_statuses(slot, slot_transaction_iterator),
|
||||
.map_transactions_to_statuses(slot, slot_transaction_iterator)?,
|
||||
rewards,
|
||||
block_time,
|
||||
block_height,
|
||||
@ -1948,20 +1948,25 @@ impl Blockstore {
|
||||
Err(BlockstoreError::SlotUnavailable)
|
||||
}
|
||||
|
||||
pub fn map_transactions_to_statuses<'a>(
|
||||
pub fn map_transactions_to_statuses(
|
||||
&self,
|
||||
slot: Slot,
|
||||
iterator: impl Iterator<Item = Transaction> + 'a,
|
||||
) -> Vec<TransactionWithStatusMeta> {
|
||||
iterator: impl Iterator<Item = VersionedTransaction>,
|
||||
) -> Result<Vec<TransactionWithStatusMeta>> {
|
||||
iterator
|
||||
.map(|transaction| {
|
||||
let signature = transaction.signatures[0];
|
||||
TransactionWithStatusMeta {
|
||||
transaction,
|
||||
meta: self
|
||||
.read_transaction_status((signature, slot))
|
||||
.ok()
|
||||
.flatten(),
|
||||
.map(|versioned_tx| {
|
||||
// TODO: add support for versioned transactions
|
||||
if let Some(transaction) = versioned_tx.into_legacy_transaction() {
|
||||
let signature = transaction.signatures[0];
|
||||
Ok(TransactionWithStatusMeta {
|
||||
transaction,
|
||||
meta: self
|
||||
.read_transaction_status((signature, slot))
|
||||
.ok()
|
||||
.flatten(),
|
||||
})
|
||||
} else {
|
||||
Err(BlockstoreError::UnsupportedTransactionVersion)
|
||||
}
|
||||
})
|
||||
.collect()
|
||||
@ -2243,6 +2248,12 @@ impl Blockstore {
|
||||
let transaction = self
|
||||
.find_transaction_in_slot(slot, signature)?
|
||||
.ok_or(BlockstoreError::TransactionStatusSlotMismatch)?; // Should not happen
|
||||
|
||||
// TODO: support retrieving versioned transactions
|
||||
let transaction = transaction
|
||||
.into_legacy_transaction()
|
||||
.ok_or(BlockstoreError::UnsupportedTransactionVersion)?;
|
||||
|
||||
let block_time = self.get_block_time(slot)?;
|
||||
Ok(Some(ConfirmedTransaction {
|
||||
slot,
|
||||
@ -2261,7 +2272,7 @@ impl Blockstore {
|
||||
&self,
|
||||
slot: Slot,
|
||||
signature: Signature,
|
||||
) -> Result<Option<Transaction>> {
|
||||
) -> Result<Option<VersionedTransaction>> {
|
||||
let slot_entries = self.get_slot_entries(slot, 0)?;
|
||||
Ok(slot_entries
|
||||
.iter()
|
||||
@ -3995,7 +4006,7 @@ pub mod tests {
|
||||
packet::PACKET_DATA_SIZE,
|
||||
pubkey::Pubkey,
|
||||
signature::Signature,
|
||||
transaction::TransactionError,
|
||||
transaction::{Transaction, TransactionError},
|
||||
};
|
||||
use solana_storage_proto::convert::generated;
|
||||
use solana_transaction_status::{InnerInstructions, Reward, Rewards, TransactionTokenBalance};
|
||||
@ -6173,6 +6184,11 @@ pub mod tests {
|
||||
.cloned()
|
||||
.filter(|entry| !entry.is_tick())
|
||||
.flat_map(|entry| entry.transactions)
|
||||
.map(|transaction| {
|
||||
transaction
|
||||
.into_legacy_transaction()
|
||||
.expect("versioned transactions not supported")
|
||||
})
|
||||
.map(|transaction| {
|
||||
let mut pre_balances: Vec<u64> = vec![];
|
||||
let mut post_balances: Vec<u64> = vec![];
|
||||
@ -7043,6 +7059,10 @@ pub mod tests {
|
||||
.cloned()
|
||||
.filter(|entry| !entry.is_tick())
|
||||
.flat_map(|entry| entry.transactions)
|
||||
.map(|tx| {
|
||||
tx.into_legacy_transaction()
|
||||
.expect("versioned transactions not supported")
|
||||
})
|
||||
.map(|transaction| {
|
||||
let mut pre_balances: Vec<u64> = vec![];
|
||||
let mut post_balances: Vec<u64> = vec![];
|
||||
@ -7142,6 +7162,10 @@ pub mod tests {
|
||||
.cloned()
|
||||
.filter(|entry| !entry.is_tick())
|
||||
.flat_map(|entry| entry.transactions)
|
||||
.map(|tx| {
|
||||
tx.into_legacy_transaction()
|
||||
.expect("versioned transactions not supported")
|
||||
})
|
||||
.map(|transaction| {
|
||||
let mut pre_balances: Vec<u64> = vec![];
|
||||
let mut post_balances: Vec<u64> = vec![];
|
||||
@ -7496,12 +7520,15 @@ pub mod tests {
|
||||
let shreds = entries_to_test_shreds(entries.clone(), slot, slot - 1, true, 0);
|
||||
blockstore.insert_shreds(shreds, None, false).unwrap();
|
||||
|
||||
for (i, entry) in entries.iter().enumerate() {
|
||||
for (i, entry) in entries.into_iter().enumerate() {
|
||||
if slot == 4 && i == 2 {
|
||||
// Purge to freeze index 0 and write address-signatures in new primary index
|
||||
blockstore.run_purge(0, 1, PurgeType::PrimaryIndex).unwrap();
|
||||
}
|
||||
for transaction in &entry.transactions {
|
||||
for tx in entry.transactions {
|
||||
let transaction = tx
|
||||
.into_legacy_transaction()
|
||||
.expect("versioned transactions not supported");
|
||||
assert_eq!(transaction.signatures.len(), 1);
|
||||
blockstore
|
||||
.write_transaction_status(
|
||||
@ -7524,8 +7551,11 @@ pub mod tests {
|
||||
let shreds = entries_to_test_shreds(entries.clone(), slot, 8, true, 0);
|
||||
blockstore.insert_shreds(shreds, None, false).unwrap();
|
||||
|
||||
for entry in entries.iter() {
|
||||
for transaction in &entry.transactions {
|
||||
for entry in entries.into_iter() {
|
||||
for tx in entry.transactions {
|
||||
let transaction = tx
|
||||
.into_legacy_transaction()
|
||||
.expect("versioned transactions not supported");
|
||||
assert_eq!(transaction.signatures.len(), 1);
|
||||
blockstore
|
||||
.write_transaction_status(
|
||||
@ -7895,7 +7925,7 @@ pub mod tests {
|
||||
let transaction_status_cf = blockstore.db.column::<cf::TransactionStatus>();
|
||||
|
||||
let slot = 0;
|
||||
let mut transactions: Vec<Transaction> = vec![];
|
||||
let mut transactions: Vec<VersionedTransaction> = vec![];
|
||||
for x in 0..4 {
|
||||
let transaction = Transaction::new_with_compiled_instructions(
|
||||
&[&Keypair::new()],
|
||||
@ -7921,18 +7951,24 @@ pub mod tests {
|
||||
transaction_status_cf
|
||||
.put_protobuf((0, transaction.signatures[0], slot), &status)
|
||||
.unwrap();
|
||||
transactions.push(transaction);
|
||||
transactions.push(transaction.into());
|
||||
}
|
||||
// Push transaction that will not have matching status, as a test case
|
||||
transactions.push(Transaction::new_with_compiled_instructions(
|
||||
&[&Keypair::new()],
|
||||
&[solana_sdk::pubkey::new_rand()],
|
||||
Hash::default(),
|
||||
vec![solana_sdk::pubkey::new_rand()],
|
||||
vec![CompiledInstruction::new(1, &(), vec![0])],
|
||||
));
|
||||
transactions.push(
|
||||
Transaction::new_with_compiled_instructions(
|
||||
&[&Keypair::new()],
|
||||
&[solana_sdk::pubkey::new_rand()],
|
||||
Hash::default(),
|
||||
vec![solana_sdk::pubkey::new_rand()],
|
||||
vec![CompiledInstruction::new(1, &(), vec![0])],
|
||||
)
|
||||
.into(),
|
||||
);
|
||||
|
||||
let map = blockstore.map_transactions_to_statuses(slot, transactions.into_iter());
|
||||
let map_result =
|
||||
blockstore.map_transactions_to_statuses(slot, transactions.into_iter());
|
||||
assert!(map_result.is_ok());
|
||||
let map = map_result.unwrap();
|
||||
assert_eq!(map.len(), 5);
|
||||
for (x, m) in map.iter().take(4).enumerate() {
|
||||
assert_eq!(m.meta.as_ref().unwrap().fee, x as u64);
|
||||
|
@ -336,7 +336,8 @@ impl Blockstore {
|
||||
if let Some(&signature) = transaction.signatures.get(0) {
|
||||
batch.delete::<cf::TransactionStatus>((0, signature, slot))?;
|
||||
batch.delete::<cf::TransactionStatus>((1, signature, slot))?;
|
||||
for pubkey in transaction.message.account_keys {
|
||||
// TODO: support purging mapped addresses from versioned transactions
|
||||
for pubkey in transaction.message.unmapped_keys() {
|
||||
batch.delete::<cf::AddressSignatures>((0, pubkey, slot, signature))?;
|
||||
batch.delete::<cf::AddressSignatures>((1, pubkey, slot, signature))?;
|
||||
}
|
||||
@ -398,6 +399,7 @@ pub mod tests {
|
||||
use solana_sdk::{
|
||||
hash::{hash, Hash},
|
||||
message::Message,
|
||||
transaction::Transaction,
|
||||
};
|
||||
|
||||
// check that all columns are either empty or start at `min_slot`
|
||||
|
@ -99,6 +99,7 @@ pub enum BlockstoreError {
|
||||
ProtobufDecodeError(#[from] prost::DecodeError),
|
||||
ParentEntriesUnavailable,
|
||||
SlotUnavailable,
|
||||
UnsupportedTransactionVersion,
|
||||
}
|
||||
pub type Result<T> = std::result::Result<T, BlockstoreError>;
|
||||
|
||||
|
@ -10,7 +10,7 @@ use log::*;
|
||||
use rand::{seq::SliceRandom, thread_rng};
|
||||
use rayon::{prelude::*, ThreadPool};
|
||||
use solana_entry::entry::{
|
||||
create_ticks, Entry, EntrySlice, EntryType, EntryVerificationStatus, VerifyRecyclers,
|
||||
self, create_ticks, Entry, EntrySlice, EntryType, EntryVerificationStatus, VerifyRecyclers,
|
||||
};
|
||||
use solana_measure::measure::Measure;
|
||||
use solana_metrics::{datapoint_error, inc_new_counter_debug};
|
||||
@ -38,7 +38,7 @@ use solana_sdk::{
|
||||
pubkey::Pubkey,
|
||||
signature::{Keypair, Signature},
|
||||
timing,
|
||||
transaction::{Result, Transaction, TransactionError},
|
||||
transaction::{Result, SanitizedTransaction, TransactionError, VersionedTransaction},
|
||||
};
|
||||
use solana_transaction_status::token_balances::{
|
||||
collect_token_balances, TransactionTokenBalancesSet,
|
||||
@ -47,7 +47,6 @@ use solana_transaction_status::token_balances::{
|
||||
use std::{
|
||||
cell::RefCell,
|
||||
collections::{HashMap, HashSet},
|
||||
convert::TryFrom,
|
||||
path::PathBuf,
|
||||
result,
|
||||
sync::{Arc, RwLock},
|
||||
@ -109,10 +108,13 @@ fn get_first_error(
|
||||
fee_collection_results: Vec<Result<()>>,
|
||||
) -> Option<(Result<()>, Signature)> {
|
||||
let mut first_err = None;
|
||||
for (result, transaction) in fee_collection_results.iter().zip(batch.transactions_iter()) {
|
||||
for (result, transaction) in fee_collection_results
|
||||
.iter()
|
||||
.zip(batch.sanitized_transactions())
|
||||
{
|
||||
if let Err(ref err) = result {
|
||||
if first_err.is_none() {
|
||||
first_err = Some((result.clone(), transaction.signatures[0]));
|
||||
first_err = Some((result.clone(), *transaction.signature()));
|
||||
}
|
||||
warn!(
|
||||
"Unexpected validator error: {:?}, transaction: {:?}",
|
||||
@ -210,7 +212,7 @@ fn execute_batch(
|
||||
} = tx_results;
|
||||
|
||||
if let Some(transaction_status_sender) = transaction_status_sender {
|
||||
let txs = batch.transactions_iter().cloned().collect();
|
||||
let transactions = batch.sanitized_transactions().to_vec();
|
||||
let post_token_balances = if record_token_balances {
|
||||
collect_token_balances(bank, batch, &mut mint_decimals)
|
||||
} else {
|
||||
@ -222,7 +224,7 @@ fn execute_batch(
|
||||
|
||||
transaction_status_sender.send_transaction_status_batch(
|
||||
bank.clone(),
|
||||
txs,
|
||||
transactions,
|
||||
execution_results,
|
||||
balances,
|
||||
token_balances,
|
||||
@ -286,19 +288,23 @@ fn execute_batches(
|
||||
/// 4. Update the leader scheduler, goto 1
|
||||
pub fn process_entries(
|
||||
bank: &Arc<Bank>,
|
||||
entries: &mut [Entry],
|
||||
entries: Vec<Entry>,
|
||||
randomize: bool,
|
||||
transaction_status_sender: Option<&TransactionStatusSender>,
|
||||
replay_vote_sender: Option<&ReplayVoteSender>,
|
||||
) -> Result<()> {
|
||||
let verify_transaction = {
|
||||
let bank = bank.clone();
|
||||
move |versioned_tx: VersionedTransaction| -> Result<SanitizedTransaction> {
|
||||
bank.verify_transaction(versioned_tx, false)
|
||||
}
|
||||
};
|
||||
|
||||
let mut timings = ExecuteTimings::default();
|
||||
let mut entry_types: Vec<_> = entries
|
||||
.iter()
|
||||
.map(EntryType::try_from)
|
||||
.collect::<Result<_>>()?;
|
||||
let mut entries = entry::verify_transactions(entries, Arc::new(verify_transaction))?;
|
||||
let result = process_entries_with_callback(
|
||||
bank,
|
||||
&mut entry_types,
|
||||
&mut entries,
|
||||
randomize,
|
||||
None,
|
||||
transaction_status_sender,
|
||||
@ -886,12 +892,14 @@ pub fn confirm_slot(
|
||||
None
|
||||
};
|
||||
|
||||
let verify_transaction = {
|
||||
let bank = bank.clone();
|
||||
move |versioned_tx: VersionedTransaction| -> Result<SanitizedTransaction> {
|
||||
bank.verify_transaction(versioned_tx, skip_verification)
|
||||
}
|
||||
};
|
||||
let check_start = Instant::now();
|
||||
let mut entries = entries.verify_and_hash_transactions(
|
||||
skip_verification,
|
||||
bank.libsecp256k1_0_5_upgrade_enabled(),
|
||||
bank.verify_tx_signatures_len_enabled(),
|
||||
)?;
|
||||
let mut entries = entry::verify_transactions(entries, Arc::new(verify_transaction))?;
|
||||
let transaction_duration_us = timing::duration_as_us(&check_start.elapsed());
|
||||
|
||||
let mut replay_elapsed = Measure::start("replay_elapsed");
|
||||
@ -1290,7 +1298,7 @@ pub enum TransactionStatusMessage {
|
||||
|
||||
pub struct TransactionStatusBatch {
|
||||
pub bank: Arc<Bank>,
|
||||
pub transactions: Vec<Transaction>,
|
||||
pub transactions: Vec<SanitizedTransaction>,
|
||||
pub statuses: Vec<TransactionExecutionResult>,
|
||||
pub balances: TransactionBalancesSet,
|
||||
pub token_balances: TransactionTokenBalancesSet,
|
||||
@ -1309,7 +1317,7 @@ impl TransactionStatusSender {
|
||||
pub fn send_transaction_status_batch(
|
||||
&self,
|
||||
bank: Arc<Bank>,
|
||||
transactions: Vec<Transaction>,
|
||||
transactions: Vec<SanitizedTransaction>,
|
||||
statuses: Vec<TransactionExecutionResult>,
|
||||
balances: TransactionBalancesSet,
|
||||
token_balances: TransactionTokenBalancesSet,
|
||||
@ -2097,8 +2105,7 @@ pub mod tests {
|
||||
} = create_genesis_config(2);
|
||||
let bank = Arc::new(Bank::new_for_tests(&genesis_config));
|
||||
let keypair = Keypair::new();
|
||||
let mut slot_entries =
|
||||
create_ticks(genesis_config.ticks_per_slot, 1, genesis_config.hash());
|
||||
let slot_entries = create_ticks(genesis_config.ticks_per_slot, 1, genesis_config.hash());
|
||||
let tx = system_transaction::transfer(
|
||||
&mint_keypair,
|
||||
&keypair.pubkey(),
|
||||
@ -2113,7 +2120,7 @@ pub mod tests {
|
||||
);
|
||||
|
||||
// Now ensure the TX is accepted despite pointing to the ID of an empty entry.
|
||||
process_entries(&bank, &mut slot_entries, true, None, None).unwrap();
|
||||
process_entries(&bank, slot_entries, true, None, None).unwrap();
|
||||
assert_eq!(bank.process_transaction(&tx), Ok(()));
|
||||
}
|
||||
|
||||
@ -2323,10 +2330,7 @@ pub mod tests {
|
||||
// ensure bank can process a tick
|
||||
assert_eq!(bank.tick_height(), 0);
|
||||
let tick = next_entry(&genesis_config.hash(), 1, vec![]);
|
||||
assert_eq!(
|
||||
process_entries(&bank, &mut [tick], true, None, None),
|
||||
Ok(())
|
||||
);
|
||||
assert_eq!(process_entries(&bank, vec![tick], true, None, None), Ok(()));
|
||||
assert_eq!(bank.tick_height(), 1);
|
||||
}
|
||||
|
||||
@ -2359,7 +2363,7 @@ pub mod tests {
|
||||
);
|
||||
let entry_2 = next_entry(&entry_1.hash, 1, vec![tx]);
|
||||
assert_eq!(
|
||||
process_entries(&bank, &mut [entry_1, entry_2], true, None, None),
|
||||
process_entries(&bank, vec![entry_1, entry_2], true, None, None),
|
||||
Ok(())
|
||||
);
|
||||
assert_eq!(bank.get_balance(&keypair1.pubkey()), 2);
|
||||
@ -2417,7 +2421,7 @@ pub mod tests {
|
||||
assert_eq!(
|
||||
process_entries(
|
||||
&bank,
|
||||
&mut [entry_1_to_mint, entry_2_to_3_mint_to_1],
|
||||
vec![entry_1_to_mint, entry_2_to_3_mint_to_1],
|
||||
false,
|
||||
None,
|
||||
None,
|
||||
@ -2489,7 +2493,7 @@ pub mod tests {
|
||||
|
||||
assert!(process_entries(
|
||||
&bank,
|
||||
&mut [entry_1_to_mint.clone(), entry_2_to_3_mint_to_1.clone()],
|
||||
vec![entry_1_to_mint.clone(), entry_2_to_3_mint_to_1.clone()],
|
||||
false,
|
||||
None,
|
||||
None,
|
||||
@ -2501,15 +2505,15 @@ pub mod tests {
|
||||
assert_eq!(bank.get_balance(&keypair2.pubkey()), 4);
|
||||
|
||||
// Check all accounts are unlocked
|
||||
let txs1 = &entry_1_to_mint.transactions[..];
|
||||
let txs2 = &entry_2_to_3_mint_to_1.transactions[..];
|
||||
let batch1 = bank.prepare_batch(txs1.iter()).unwrap();
|
||||
let txs1 = entry_1_to_mint.transactions;
|
||||
let txs2 = entry_2_to_3_mint_to_1.transactions;
|
||||
let batch1 = bank.prepare_entry_batch(txs1).unwrap();
|
||||
for result in batch1.lock_results() {
|
||||
assert!(result.is_ok());
|
||||
}
|
||||
// txs1 and txs2 have accounts that conflict, so we must drop txs1 first
|
||||
drop(batch1);
|
||||
let batch2 = bank.prepare_batch(txs2.iter()).unwrap();
|
||||
let batch2 = bank.prepare_entry_batch(txs2).unwrap();
|
||||
for result in batch2.lock_results() {
|
||||
assert!(result.is_ok());
|
||||
}
|
||||
@ -2597,7 +2601,7 @@ pub mod tests {
|
||||
|
||||
assert!(process_entries(
|
||||
&bank,
|
||||
&mut [
|
||||
vec![
|
||||
entry_1_to_mint,
|
||||
entry_2_to_3_and_1_to_mint,
|
||||
entry_conflict_itself,
|
||||
@ -2652,7 +2656,7 @@ pub mod tests {
|
||||
system_transaction::transfer(&keypair2, &keypair4.pubkey(), 1, bank.last_blockhash());
|
||||
let entry_2 = next_entry(&entry_1.hash, 1, vec![tx]);
|
||||
assert_eq!(
|
||||
process_entries(&bank, &mut [entry_1, entry_2], true, None, None),
|
||||
process_entries(&bank, vec![entry_1, entry_2], true, None, None),
|
||||
Ok(())
|
||||
);
|
||||
assert_eq!(bank.get_balance(&keypair3.pubkey()), 1);
|
||||
@ -2686,7 +2690,7 @@ pub mod tests {
|
||||
let present_account = AccountSharedData::new(1, 10, &Pubkey::default());
|
||||
bank.store_account(&present_account_key.pubkey(), &present_account);
|
||||
|
||||
let mut entries: Vec<_> = (0..NUM_TRANSFERS)
|
||||
let entries: Vec<_> = (0..NUM_TRANSFERS)
|
||||
.step_by(NUM_TRANSFERS_PER_ENTRY)
|
||||
.map(|i| {
|
||||
let mut transactions = (0..NUM_TRANSFERS_PER_ENTRY)
|
||||
@ -2712,10 +2716,7 @@ pub mod tests {
|
||||
next_entry_mut(&mut hash, 0, transactions)
|
||||
})
|
||||
.collect();
|
||||
assert_eq!(
|
||||
process_entries(&bank, &mut entries, true, None, None),
|
||||
Ok(())
|
||||
);
|
||||
assert_eq!(process_entries(&bank, entries, true, None, None), Ok(()));
|
||||
}
|
||||
|
||||
#[test]
|
||||
@ -2776,7 +2777,7 @@ pub mod tests {
|
||||
// Transfer lamports to each other
|
||||
let entry = next_entry(&bank.last_blockhash(), 1, tx_vector);
|
||||
assert_eq!(
|
||||
process_entries(&bank, &mut [entry], true, None, None),
|
||||
process_entries(&bank, vec![entry], true, None, None),
|
||||
Ok(())
|
||||
);
|
||||
bank.squash();
|
||||
@ -2838,7 +2839,7 @@ pub mod tests {
|
||||
assert_eq!(
|
||||
process_entries(
|
||||
&bank,
|
||||
&mut [entry_1, tick, entry_2.clone()],
|
||||
vec![entry_1, tick, entry_2.clone()],
|
||||
true,
|
||||
None,
|
||||
None
|
||||
@ -2853,7 +2854,7 @@ pub mod tests {
|
||||
system_transaction::transfer(&keypair2, &keypair3.pubkey(), 1, bank.last_blockhash());
|
||||
let entry_3 = next_entry(&entry_2.hash, 1, vec![tx]);
|
||||
assert_eq!(
|
||||
process_entries(&bank, &mut [entry_3], true, None, None),
|
||||
process_entries(&bank, vec![entry_3], true, None, None),
|
||||
Err(TransactionError::AccountNotFound)
|
||||
);
|
||||
}
|
||||
@ -2933,7 +2934,7 @@ pub mod tests {
|
||||
);
|
||||
|
||||
assert_eq!(
|
||||
process_entries(&bank, &mut [entry_1_to_mint], false, None, None),
|
||||
process_entries(&bank, vec![entry_1_to_mint], false, None, None),
|
||||
Err(TransactionError::AccountInUse)
|
||||
);
|
||||
|
||||
@ -3094,7 +3095,7 @@ pub mod tests {
|
||||
let mut hash = bank.last_blockhash();
|
||||
let mut root: Option<Arc<Bank>> = None;
|
||||
loop {
|
||||
let mut entries: Vec<_> = (0..NUM_TRANSFERS)
|
||||
let entries: Vec<_> = (0..NUM_TRANSFERS)
|
||||
.step_by(NUM_TRANSFERS_PER_ENTRY)
|
||||
.map(|i| {
|
||||
next_entry_mut(&mut hash, 0, {
|
||||
@ -3122,9 +3123,9 @@ pub mod tests {
|
||||
})
|
||||
.collect();
|
||||
info!("paying iteration {}", i);
|
||||
process_entries(&bank, &mut entries, true, None, None).expect("paying failed");
|
||||
process_entries(&bank, entries, true, None, None).expect("paying failed");
|
||||
|
||||
let mut entries: Vec<_> = (0..NUM_TRANSFERS)
|
||||
let entries: Vec<_> = (0..NUM_TRANSFERS)
|
||||
.step_by(NUM_TRANSFERS_PER_ENTRY)
|
||||
.map(|i| {
|
||||
next_entry_mut(
|
||||
@ -3145,12 +3146,12 @@ pub mod tests {
|
||||
.collect();
|
||||
|
||||
info!("refunding iteration {}", i);
|
||||
process_entries(&bank, &mut entries, true, None, None).expect("refunding failed");
|
||||
process_entries(&bank, entries, true, None, None).expect("refunding failed");
|
||||
|
||||
// advance to next block
|
||||
process_entries(
|
||||
&bank,
|
||||
&mut (0..bank.ticks_per_slot())
|
||||
(0..bank.ticks_per_slot())
|
||||
.map(|_| next_entry_mut(&mut hash, 1, vec![]))
|
||||
.collect::<Vec<_>>(),
|
||||
true,
|
||||
@ -3198,7 +3199,7 @@ pub mod tests {
|
||||
let entry = next_entry(&new_blockhash, 1, vec![tx]);
|
||||
entries.push(entry);
|
||||
|
||||
process_entries(&bank0, &mut entries, true, None, None).unwrap();
|
||||
process_entries(&bank0, entries, true, None, None).unwrap();
|
||||
assert_eq!(bank0.get_balance(&keypair.pubkey()), 1)
|
||||
}
|
||||
|
||||
@ -3272,8 +3273,8 @@ pub mod tests {
|
||||
42,
|
||||
Hash::default(),
|
||||
);
|
||||
let transactions = [account_not_found_tx, invalid_blockhash_tx];
|
||||
let batch = bank.prepare_batch(transactions.iter()).unwrap();
|
||||
let txs = vec![account_not_found_tx, invalid_blockhash_tx];
|
||||
let batch = bank.prepare_batch(txs).unwrap();
|
||||
let (
|
||||
TransactionResults {
|
||||
fee_collection_results,
|
||||
@ -3368,7 +3369,7 @@ pub mod tests {
|
||||
.collect();
|
||||
let entry = next_entry(&bank_1_blockhash, 1, vote_txs);
|
||||
let (replay_vote_sender, replay_vote_receiver) = unbounded();
|
||||
let _ = process_entries(&bank1, &mut [entry], true, None, Some(&replay_vote_sender));
|
||||
let _ = process_entries(&bank1, vec![entry], true, None, Some(&replay_vote_sender));
|
||||
let successes: BTreeSet<Pubkey> = replay_vote_receiver
|
||||
.try_iter()
|
||||
.map(|(vote_pubkey, _, _)| vote_pubkey)
|
||||
|
@ -678,6 +678,7 @@ fn find_latest_replayed_slot_from_ledger(
|
||||
latest_slot,
|
||||
non_tick_entry.transactions.clone().into_iter(),
|
||||
)
|
||||
.unwrap()
|
||||
.is_empty()
|
||||
{
|
||||
return (
|
||||
|
@ -22,7 +22,7 @@ use {
|
||||
solana_runtime::bank::Bank,
|
||||
solana_sdk::{
|
||||
clock::NUM_CONSECUTIVE_LEADER_SLOTS, hash::Hash, poh_config::PohConfig, pubkey::Pubkey,
|
||||
timing, transaction::Transaction,
|
||||
timing, transaction::VersionedTransaction,
|
||||
},
|
||||
std::{
|
||||
cmp,
|
||||
@ -61,14 +61,14 @@ pub type BankStart = (Arc<Bank>, Arc<Instant>);
|
||||
|
||||
pub struct Record {
|
||||
pub mixin: Hash,
|
||||
pub transactions: Vec<Transaction>,
|
||||
pub transactions: Vec<VersionedTransaction>,
|
||||
pub slot: Slot,
|
||||
pub sender: CrossbeamSender<Result<()>>,
|
||||
}
|
||||
impl Record {
|
||||
pub fn new(
|
||||
mixin: Hash,
|
||||
transactions: Vec<Transaction>,
|
||||
transactions: Vec<VersionedTransaction>,
|
||||
slot: Slot,
|
||||
sender: CrossbeamSender<Result<()>>,
|
||||
) -> Self {
|
||||
@ -106,7 +106,7 @@ impl TransactionRecorder {
|
||||
&self,
|
||||
bank_slot: Slot,
|
||||
mixin: Hash,
|
||||
transactions: Vec<Transaction>,
|
||||
transactions: Vec<VersionedTransaction>,
|
||||
) -> Result<()> {
|
||||
// create a new channel so that there is only 1 sender and when it goes out of scope, the receiver fails
|
||||
let (result_sender, result_receiver) = unbounded();
|
||||
@ -566,7 +566,7 @@ impl PohRecorder {
|
||||
&mut self,
|
||||
bank_slot: Slot,
|
||||
mixin: Hash,
|
||||
transactions: Vec<Transaction>,
|
||||
transactions: Vec<VersionedTransaction>,
|
||||
) -> Result<()> {
|
||||
// Entries without transactions are used to track real-time passing in the ledger and
|
||||
// cannot be generated by `record()`
|
||||
@ -1044,7 +1044,9 @@ mod tests {
|
||||
poh_recorder.tick();
|
||||
let tx = test_tx();
|
||||
let h1 = hash(b"hello world!");
|
||||
assert!(poh_recorder.record(bank.slot(), h1, vec![tx]).is_err());
|
||||
assert!(poh_recorder
|
||||
.record(bank.slot(), h1, vec![tx.into()])
|
||||
.is_err());
|
||||
assert!(entry_receiver.try_recv().is_err());
|
||||
}
|
||||
Blockstore::destroy(&ledger_path).unwrap();
|
||||
@ -1086,7 +1088,7 @@ mod tests {
|
||||
let tx = test_tx();
|
||||
let h1 = hash(b"hello world!");
|
||||
assert_matches!(
|
||||
poh_recorder.record(bank.slot() + 1, h1, vec![tx]),
|
||||
poh_recorder.record(bank.slot() + 1, h1, vec![tx.into()]),
|
||||
Err(PohRecorderError::MaxHeightReached)
|
||||
);
|
||||
}
|
||||
@ -1128,7 +1130,9 @@ mod tests {
|
||||
assert_eq!(poh_recorder.tick_height, 1);
|
||||
let tx = test_tx();
|
||||
let h1 = hash(b"hello world!");
|
||||
assert!(poh_recorder.record(bank.slot(), h1, vec![tx]).is_ok());
|
||||
assert!(poh_recorder
|
||||
.record(bank.slot(), h1, vec![tx.into()])
|
||||
.is_ok());
|
||||
assert_eq!(poh_recorder.tick_cache.len(), 0);
|
||||
|
||||
//tick in the cache + entry
|
||||
@ -1175,7 +1179,9 @@ mod tests {
|
||||
assert_eq!(poh_recorder.tick_height, 2);
|
||||
let tx = test_tx();
|
||||
let h1 = hash(b"hello world!");
|
||||
assert!(poh_recorder.record(bank.slot(), h1, vec![tx]).is_err());
|
||||
assert!(poh_recorder
|
||||
.record(bank.slot(), h1, vec![tx.into()])
|
||||
.is_err());
|
||||
|
||||
let (_bank, (entry, _tick_height)) = entry_receiver.recv().unwrap();
|
||||
assert!(entry.is_tick());
|
||||
@ -1424,7 +1430,9 @@ mod tests {
|
||||
|
||||
let tx = test_tx();
|
||||
let h1 = hash(b"hello world!");
|
||||
assert!(poh_recorder.record(bank.slot(), h1, vec![tx]).is_err());
|
||||
assert!(poh_recorder
|
||||
.record(bank.slot(), h1, vec![tx.into()])
|
||||
.is_err());
|
||||
assert!(poh_recorder.working_bank.is_none());
|
||||
// Make sure the starting slot is updated
|
||||
assert_eq!(poh_recorder.start_slot, end_slot);
|
||||
|
@ -368,7 +368,9 @@ mod tests {
|
||||
solana_measure::measure::Measure,
|
||||
solana_perf::test_tx::test_tx,
|
||||
solana_runtime::bank::Bank,
|
||||
solana_sdk::{clock, hash::hash, pubkey::Pubkey, timing},
|
||||
solana_sdk::{
|
||||
clock, hash::hash, pubkey::Pubkey, timing, transaction::VersionedTransaction,
|
||||
},
|
||||
std::time::Duration,
|
||||
};
|
||||
|
||||
@ -434,7 +436,7 @@ mod tests {
|
||||
let mut total_us = 0;
|
||||
let mut total_times = 0;
|
||||
let h1 = hash(b"hello world!");
|
||||
let tx = test_tx();
|
||||
let tx = VersionedTransaction::from(test_tx());
|
||||
loop {
|
||||
// send some data
|
||||
let mut time = Measure::start("record");
|
||||
|
5
programs/bpf/Cargo.lock
generated
5
programs/bpf/Cargo.lock
generated
@ -164,9 +164,9 @@ dependencies = [
|
||||
|
||||
[[package]]
|
||||
name = "bitflags"
|
||||
version = "1.2.1"
|
||||
version = "1.3.2"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "cf1de2fe8c75bc145a2f577add951f8134889b4795d47466a54a5c846d691693"
|
||||
checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a"
|
||||
|
||||
[[package]]
|
||||
name = "blake3"
|
||||
@ -3078,6 +3078,7 @@ name = "solana-program"
|
||||
version = "1.8.0"
|
||||
dependencies = [
|
||||
"bincode",
|
||||
"bitflags",
|
||||
"blake3 1.0.0",
|
||||
"borsh",
|
||||
"borsh-derive",
|
||||
|
@ -293,7 +293,7 @@ fn process_transaction_and_record_inner(
|
||||
) -> (Result<(), TransactionError>, Vec<Vec<CompiledInstruction>>) {
|
||||
let signature = tx.signatures.get(0).unwrap().clone();
|
||||
let txs = vec![tx];
|
||||
let tx_batch = bank.prepare_batch(txs.iter()).unwrap();
|
||||
let tx_batch = bank.prepare_batch(txs).unwrap();
|
||||
let (mut results, _, mut inner_instructions, _transaction_logs) = bank
|
||||
.load_execute_and_commit_transactions(
|
||||
&tx_batch,
|
||||
@ -315,8 +315,8 @@ fn process_transaction_and_record_inner(
|
||||
)
|
||||
}
|
||||
|
||||
fn execute_transactions(bank: &Bank, txs: &[Transaction]) -> Vec<ConfirmedTransaction> {
|
||||
let batch = bank.prepare_batch(txs.iter()).unwrap();
|
||||
fn execute_transactions(bank: &Bank, txs: Vec<Transaction>) -> Vec<ConfirmedTransaction> {
|
||||
let batch = bank.prepare_batch(txs.clone()).unwrap();
|
||||
let mut timings = ExecuteTimings::default();
|
||||
let mut mint_decimals = HashMap::new();
|
||||
let tx_pre_token_balances = collect_token_balances(&bank, &batch, &mut mint_decimals);
|
||||
@ -2402,13 +2402,13 @@ fn test_program_upgradeable_locks() {
|
||||
let results1 = {
|
||||
let (bank, invoke_tx, upgrade_tx) =
|
||||
setup_program_upgradeable_locks(&payer_keypair, &buffer_keypair, &program_keypair);
|
||||
execute_transactions(&bank, &[upgrade_tx, invoke_tx])
|
||||
execute_transactions(&bank, vec![upgrade_tx, invoke_tx])
|
||||
};
|
||||
|
||||
let results2 = {
|
||||
let (bank, invoke_tx, upgrade_tx) =
|
||||
setup_program_upgradeable_locks(&payer_keypair, &buffer_keypair, &program_keypair);
|
||||
execute_transactions(&bank, &[invoke_tx, upgrade_tx])
|
||||
execute_transactions(&bank, vec![invoke_tx, upgrade_tx])
|
||||
};
|
||||
|
||||
if false {
|
||||
|
@ -1,10 +1,11 @@
|
||||
use solana_sdk::{
|
||||
clock::Slot,
|
||||
hash::Hash,
|
||||
instruction::CompiledInstruction,
|
||||
program_utils::limited_deserialize,
|
||||
pubkey::Pubkey,
|
||||
signature::{Keypair, Signer},
|
||||
transaction::Transaction,
|
||||
transaction::{SanitizedTransaction, Transaction},
|
||||
};
|
||||
|
||||
use crate::{
|
||||
@ -12,7 +13,37 @@ use crate::{
|
||||
vote_state::Vote,
|
||||
};
|
||||
|
||||
pub fn parse_vote_transaction(tx: &Transaction) -> Option<(Pubkey, Vote, Option<Hash>)> {
|
||||
pub type ParsedVote = (Pubkey, Vote, Option<Hash>);
|
||||
|
||||
fn parse_vote(vote_ix: &CompiledInstruction, vote_key: &Pubkey) -> Option<ParsedVote> {
|
||||
let vote_instruction = limited_deserialize(&vote_ix.data).ok();
|
||||
vote_instruction.and_then(|vote_instruction| match vote_instruction {
|
||||
VoteInstruction::Vote(vote) => Some((*vote_key, vote, None)),
|
||||
VoteInstruction::VoteSwitch(vote, hash) => Some((*vote_key, vote, Some(hash))),
|
||||
_ => None,
|
||||
})
|
||||
}
|
||||
|
||||
pub fn parse_sanitized_vote_transaction(tx: &SanitizedTransaction) -> Option<ParsedVote> {
|
||||
// Check first instruction for a vote
|
||||
let message = tx.message();
|
||||
message
|
||||
.program_instructions_iter()
|
||||
.next()
|
||||
.and_then(|(program_id, first_ix)| {
|
||||
if !crate::check_id(program_id) {
|
||||
return None;
|
||||
}
|
||||
|
||||
first_ix.accounts.first().and_then(|first_account| {
|
||||
message
|
||||
.get_account_key(*first_account as usize)
|
||||
.and_then(|key| parse_vote(first_ix, key))
|
||||
})
|
||||
})
|
||||
}
|
||||
|
||||
pub fn parse_vote_transaction(tx: &Transaction) -> Option<ParsedVote> {
|
||||
// Check first instruction for a vote
|
||||
let message = tx.message();
|
||||
message.instructions.get(0).and_then(|first_instruction| {
|
||||
@ -31,19 +62,10 @@ pub fn parse_vote_transaction(tx: &Transaction) -> Option<(Pubkey, Vote, Option<
|
||||
.accounts
|
||||
.first()
|
||||
.and_then(|first_account| {
|
||||
tx.message
|
||||
message
|
||||
.account_keys
|
||||
.get(*first_account as usize)
|
||||
.and_then(|key| {
|
||||
let vote_instruction = limited_deserialize(&first_instruction.data).ok();
|
||||
vote_instruction.and_then(|vote_instruction| match vote_instruction {
|
||||
VoteInstruction::Vote(vote) => Some((*key, vote, None)),
|
||||
VoteInstruction::VoteSwitch(vote, hash) => {
|
||||
Some((*key, vote, Some(hash)))
|
||||
}
|
||||
_ => None,
|
||||
})
|
||||
})
|
||||
.and_then(|key| parse_vote(first_instruction, key))
|
||||
})
|
||||
})
|
||||
}
|
||||
|
143
rpc/src/rpc.rs
143
rpc/src/rpc.rs
@ -60,15 +60,14 @@ use {
|
||||
epoch_schedule::EpochSchedule,
|
||||
exit::Exit,
|
||||
hash::Hash,
|
||||
message::Message,
|
||||
message::{Message, SanitizedMessage},
|
||||
pubkey::Pubkey,
|
||||
sanitize::Sanitize,
|
||||
signature::{Keypair, Signature, Signer},
|
||||
stake::state::StakeState,
|
||||
stake_history::StakeHistory,
|
||||
system_instruction,
|
||||
sysvar::stake_history,
|
||||
transaction::{self, Transaction, TransactionError},
|
||||
transaction::{self, SanitizedTransaction, TransactionError, VersionedTransaction},
|
||||
},
|
||||
solana_streamer::socket::SocketAddrSpace,
|
||||
solana_transaction_status::{
|
||||
@ -84,6 +83,7 @@ use {
|
||||
any::type_name,
|
||||
cmp::{max, min},
|
||||
collections::{HashMap, HashSet},
|
||||
convert::TryFrom,
|
||||
net::SocketAddr,
|
||||
str::FromStr,
|
||||
sync::{
|
||||
@ -1934,7 +1934,7 @@ impl JsonRpcRequestProcessor {
|
||||
fn get_fee_for_message(
|
||||
&self,
|
||||
blockhash: &Hash,
|
||||
message: &Message,
|
||||
message: &SanitizedMessage,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<RpcResponse<Option<u64>>> {
|
||||
let bank = self.bank(commitment);
|
||||
@ -1944,7 +1944,7 @@ impl JsonRpcRequestProcessor {
|
||||
}
|
||||
|
||||
fn verify_transaction(
|
||||
transaction: &Transaction,
|
||||
transaction: &SanitizedTransaction,
|
||||
libsecp256k1_0_5_upgrade_enabled: bool,
|
||||
) -> Result<()> {
|
||||
if transaction.verify().is_err() {
|
||||
@ -2184,15 +2184,11 @@ fn get_token_program_id_and_mint(
|
||||
|
||||
fn _send_transaction(
|
||||
meta: JsonRpcRequestProcessor,
|
||||
transaction: Transaction,
|
||||
signature: Signature,
|
||||
wire_transaction: Vec<u8>,
|
||||
last_valid_block_height: u64,
|
||||
durable_nonce_info: Option<(Pubkey, Hash)>,
|
||||
) -> Result<String> {
|
||||
if transaction.signatures.is_empty() {
|
||||
return Err(RpcCustomError::TransactionSignatureVerificationFailure.into());
|
||||
}
|
||||
let signature = transaction.signatures[0];
|
||||
let transaction_info = TransactionInfo::new(
|
||||
signature,
|
||||
wire_transaction,
|
||||
@ -3255,9 +3251,15 @@ pub mod rpc_full {
|
||||
Error::internal_error()
|
||||
})?;
|
||||
|
||||
let signature = if !transaction.signatures.is_empty() {
|
||||
transaction.signatures[0]
|
||||
} else {
|
||||
return Err(RpcCustomError::TransactionSignatureVerificationFailure.into());
|
||||
};
|
||||
|
||||
_send_transaction(
|
||||
meta,
|
||||
transaction,
|
||||
signature,
|
||||
wire_transaction,
|
||||
last_valid_block_height,
|
||||
None,
|
||||
@ -3273,26 +3275,23 @@ pub mod rpc_full {
|
||||
debug!("send_transaction rpc request received");
|
||||
let config = config.unwrap_or_default();
|
||||
let encoding = config.encoding.unwrap_or(UiTransactionEncoding::Base58);
|
||||
let (wire_transaction, transaction) =
|
||||
decode_and_deserialize::<Transaction>(data, encoding)?;
|
||||
let (wire_transaction, unsanitized_tx) =
|
||||
decode_and_deserialize::<VersionedTransaction>(data, encoding)?;
|
||||
|
||||
let preflight_commitment = config
|
||||
.preflight_commitment
|
||||
.map(|commitment| CommitmentConfig { commitment });
|
||||
let preflight_bank = &*meta.bank(preflight_commitment);
|
||||
let transaction = sanitize_transaction(unsanitized_tx)?;
|
||||
let signature = *transaction.signature();
|
||||
|
||||
let mut last_valid_block_height = preflight_bank
|
||||
.get_blockhash_last_valid_block_height(&transaction.message.recent_blockhash)
|
||||
.get_blockhash_last_valid_block_height(transaction.message().recent_blockhash())
|
||||
.unwrap_or(0);
|
||||
|
||||
let durable_nonce_info = solana_sdk::transaction::uses_durable_nonce(&transaction)
|
||||
.and_then(|nonce_ix| {
|
||||
solana_sdk::transaction::get_nonce_pubkey_from_instruction(
|
||||
nonce_ix,
|
||||
&transaction,
|
||||
)
|
||||
})
|
||||
.map(|&pubkey| (pubkey, transaction.message.recent_blockhash));
|
||||
let durable_nonce_info = transaction
|
||||
.get_durable_nonce()
|
||||
.map(|&pubkey| (pubkey, *transaction.message().recent_blockhash()));
|
||||
if durable_nonce_info.is_some() {
|
||||
// While it uses a defined constant, this last_valid_block_height value is chosen arbitrarily.
|
||||
// It provides a fallback timeout for durable-nonce transaction retries in case of
|
||||
@ -3333,7 +3332,7 @@ pub mod rpc_full {
|
||||
logs,
|
||||
post_simulation_accounts: _,
|
||||
units_consumed,
|
||||
} = preflight_bank.simulate_transaction(&transaction)
|
||||
} = preflight_bank.simulate_transaction(transaction)
|
||||
{
|
||||
match err {
|
||||
TransactionError::BlockhashNotFound => {
|
||||
@ -3358,7 +3357,7 @@ pub mod rpc_full {
|
||||
|
||||
_send_transaction(
|
||||
meta,
|
||||
transaction,
|
||||
signature,
|
||||
wire_transaction,
|
||||
last_valid_block_height,
|
||||
durable_nonce_info,
|
||||
@ -3374,31 +3373,32 @@ pub mod rpc_full {
|
||||
debug!("simulate_transaction rpc request received");
|
||||
let config = config.unwrap_or_default();
|
||||
let encoding = config.encoding.unwrap_or(UiTransactionEncoding::Base58);
|
||||
let (_, mut transaction) = decode_and_deserialize::<Transaction>(data, encoding)?;
|
||||
let (_, mut unsanitized_tx) =
|
||||
decode_and_deserialize::<VersionedTransaction>(data, encoding)?;
|
||||
|
||||
let bank = &*meta.bank(config.commitment);
|
||||
if config.sig_verify {
|
||||
if config.replace_recent_blockhash {
|
||||
if config.replace_recent_blockhash {
|
||||
if config.sig_verify {
|
||||
return Err(Error::invalid_params(
|
||||
"sigVerify may not be used with replaceRecentBlockhash",
|
||||
));
|
||||
}
|
||||
unsanitized_tx
|
||||
.message
|
||||
.set_recent_blockhash(bank.last_blockhash());
|
||||
}
|
||||
|
||||
if let Err(e) =
|
||||
verify_transaction(&transaction, bank.libsecp256k1_0_5_upgrade_enabled())
|
||||
{
|
||||
return Err(e);
|
||||
}
|
||||
}
|
||||
if config.replace_recent_blockhash {
|
||||
transaction.message.recent_blockhash = bank.last_blockhash();
|
||||
let transaction = sanitize_transaction(unsanitized_tx)?;
|
||||
if config.sig_verify {
|
||||
verify_transaction(&transaction, bank.libsecp256k1_0_5_upgrade_enabled())?;
|
||||
}
|
||||
|
||||
let TransactionSimulationResult {
|
||||
result,
|
||||
logs,
|
||||
post_simulation_accounts,
|
||||
units_consumed,
|
||||
} = bank.simulate_transaction(&transaction);
|
||||
} = bank.simulate_transaction(transaction);
|
||||
|
||||
let accounts = if let Some(config_accounts) = config.accounts {
|
||||
let accounts_encoding = config_accounts
|
||||
@ -3424,21 +3424,11 @@ pub mod rpc_full {
|
||||
accounts.push(if result.is_err() {
|
||||
None
|
||||
} else {
|
||||
(0..transaction.message.account_keys.len())
|
||||
.position(|i| {
|
||||
post_simulation_accounts
|
||||
.get(i)
|
||||
.map(|(key, _account)| *key == address)
|
||||
.unwrap_or(false)
|
||||
})
|
||||
.map(|i| {
|
||||
UiAccount::encode(
|
||||
&address,
|
||||
&post_simulation_accounts[i].1,
|
||||
accounts_encoding,
|
||||
None,
|
||||
None,
|
||||
)
|
||||
post_simulation_accounts
|
||||
.iter()
|
||||
.find(|(key, _account)| key == &address)
|
||||
.map(|(pubkey, account)| {
|
||||
UiAccount::encode(pubkey, account, accounts_encoding, None, None)
|
||||
})
|
||||
});
|
||||
}
|
||||
@ -3616,7 +3606,11 @@ pub mod rpc_full {
|
||||
let blockhash = Hash::from_str(&blockhash)
|
||||
.map_err(|e| Error::invalid_params(format!("{:?}", e)))?;
|
||||
let (_, message) = decode_and_deserialize::<Message>(data, encoding)?;
|
||||
meta.get_fee_for_message(&blockhash, &message, commitment)
|
||||
SanitizedMessage::try_from(message)
|
||||
.map_err(|err| {
|
||||
Error::invalid_params(format!("invalid transaction message: {}", err))
|
||||
})
|
||||
.and_then(|message| meta.get_fee_for_message(&blockhash, &message, commitment))
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -4002,7 +3996,7 @@ fn decode_and_deserialize<T>(
|
||||
encoding: UiTransactionEncoding,
|
||||
) -> Result<(Vec<u8>, T)>
|
||||
where
|
||||
T: serde::de::DeserializeOwned + Sanitize,
|
||||
T: serde::de::DeserializeOwned,
|
||||
{
|
||||
let wire_output = match encoding {
|
||||
UiTransactionEncoding::Base58 => {
|
||||
@ -4059,20 +4053,17 @@ where
|
||||
info!("deserialize error: {}", err);
|
||||
Error::invalid_params(&err.to_string())
|
||||
})
|
||||
.and_then(|output: T| {
|
||||
if let Err(err) = output.sanitize() {
|
||||
Err(Error::invalid_params(format!(
|
||||
"invalid {}: {}",
|
||||
type_name::<T>(),
|
||||
err
|
||||
)))
|
||||
} else {
|
||||
Ok(output)
|
||||
}
|
||||
})
|
||||
.map(|output| (wire_output, output))
|
||||
}
|
||||
|
||||
fn sanitize_transaction(transaction: VersionedTransaction) -> Result<SanitizedTransaction> {
|
||||
let message_hash = transaction.message.hash();
|
||||
SanitizedTransaction::try_create(transaction, message_hash, |_| {
|
||||
Err(TransactionError::UnsupportedVersion)
|
||||
})
|
||||
.map_err(|err| Error::invalid_params(format!("invalid transaction: {}", err)))
|
||||
}
|
||||
|
||||
pub(crate) fn create_validator_exit(exit: &Arc<AtomicBool>) -> Arc<RwLock<Exit>> {
|
||||
let mut validator_exit = Exit::default();
|
||||
let exit_ = exit.clone();
|
||||
@ -4114,7 +4105,7 @@ pub fn create_test_transactions_and_populate_blockstore(
|
||||
Hash::default(),
|
||||
);
|
||||
let entry_3 = solana_entry::entry::next_entry(&entry_2.hash, 1, vec![fail_tx]);
|
||||
let mut entries = vec![entry_1, entry_2, entry_3];
|
||||
let entries = vec![entry_1, entry_2, entry_3];
|
||||
|
||||
let shreds = solana_ledger::blockstore::entries_to_test_shreds(
|
||||
entries.clone(),
|
||||
@ -4140,7 +4131,7 @@ pub fn create_test_transactions_and_populate_blockstore(
|
||||
// that they are matched properly by get_rooted_block
|
||||
let _result = solana_ledger::blockstore_processor::process_entries(
|
||||
&bank,
|
||||
&mut entries,
|
||||
entries,
|
||||
true,
|
||||
Some(
|
||||
&solana_ledger::blockstore_processor::TransactionStatusSender {
|
||||
@ -4193,7 +4184,7 @@ pub mod tests {
|
||||
signature::{Keypair, Signer},
|
||||
system_program, system_transaction,
|
||||
timing::slot_duration_from_slots_per_year,
|
||||
transaction::{self, TransactionError},
|
||||
transaction::{self, Transaction, TransactionError},
|
||||
},
|
||||
solana_transaction_status::{
|
||||
EncodedConfirmedBlock, EncodedTransaction, EncodedTransactionWithStatusMeta,
|
||||
@ -5986,7 +5977,7 @@ pub mod tests {
|
||||
assert_eq!(
|
||||
res,
|
||||
Some(
|
||||
r#"{"jsonrpc":"2.0","error":{"code":-32602,"message":"invalid solana_sdk::transaction::Transaction: index out of bounds"},"id":1}"#.to_string(),
|
||||
r#"{"jsonrpc":"2.0","error":{"code":-32602,"message":"invalid transaction: Transaction failed to sanitize accounts offsets correctly"},"id":1}"#.to_string(),
|
||||
)
|
||||
);
|
||||
let mut bad_transaction = system_transaction::transfer(
|
||||
@ -6050,7 +6041,7 @@ pub mod tests {
|
||||
assert_eq!(
|
||||
res,
|
||||
Some(
|
||||
r#"{"jsonrpc":"2.0","error":{"code":-32602,"message":"invalid solana_sdk::transaction::Transaction: index out of bounds"},"id":1}"#.to_string(),
|
||||
r#"{"jsonrpc":"2.0","error":{"code":-32602,"message":"invalid transaction: Transaction failed to sanitize accounts offsets correctly"},"id":1}"#.to_string(),
|
||||
)
|
||||
);
|
||||
}
|
||||
@ -7795,19 +7786,25 @@ pub mod tests {
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_decode_and_deserialize_unsanitary() {
|
||||
fn test_sanitize_unsanitary() {
|
||||
let unsanitary_tx58 = "ju9xZWuDBX4pRxX2oZkTjxU5jB4SSTgEGhX8bQ8PURNzyzqKMPPpNvWihx8zUe\
|
||||
FfrbVNoAaEsNKZvGzAnTDy5bhNT9kt6KFCTBixpvrLCzg4M5UdFUQYrn1gdgjX\
|
||||
pLHxcaShD81xBNaFDgnA2nkkdHnKtZt4hVSfKAmw3VRZbjrZ7L2fKZBx21CwsG\
|
||||
hD6onjM2M3qZW5C8J6d1pj41MxKmZgPBSha3MyKkNLkAGFASK"
|
||||
.to_string();
|
||||
|
||||
let unsanitary_versioned_tx = decode_and_deserialize::<VersionedTransaction>(
|
||||
unsanitary_tx58,
|
||||
UiTransactionEncoding::Base58,
|
||||
)
|
||||
.unwrap()
|
||||
.1;
|
||||
let expect58 = Error::invalid_params(
|
||||
"invalid solana_sdk::transaction::Transaction: index out of bounds".to_string(),
|
||||
"invalid transaction: Transaction failed to sanitize accounts offsets correctly"
|
||||
.to_string(),
|
||||
);
|
||||
assert_eq!(
|
||||
decode_and_deserialize::<Transaction>(unsanitary_tx58, UiTransactionEncoding::Base58)
|
||||
.unwrap_err(),
|
||||
sanitize_transaction(unsanitary_versioned_tx).unwrap_err(),
|
||||
expect58
|
||||
);
|
||||
}
|
||||
|
@ -92,7 +92,7 @@ impl TransactionStatusService {
|
||||
log_messages,
|
||||
rent_debits,
|
||||
) in izip!(
|
||||
&transactions,
|
||||
transactions,
|
||||
statuses,
|
||||
balances.pre_balances,
|
||||
balances.post_balances,
|
||||
@ -100,20 +100,18 @@ impl TransactionStatusService {
|
||||
token_balances.post_token_balances,
|
||||
inner_instructions_iter,
|
||||
transaction_logs_iter,
|
||||
rent_debits.into_iter(),
|
||||
rent_debits,
|
||||
) {
|
||||
if Bank::can_commit(&status) && !transaction.signatures.is_empty() {
|
||||
if Bank::can_commit(&status) {
|
||||
let fee_calculator = nonce_rollback
|
||||
.map(|nonce_rollback| nonce_rollback.fee_calculator())
|
||||
.unwrap_or_else(|| {
|
||||
#[allow(deprecated)]
|
||||
bank.get_fee_calculator(&transaction.message().recent_blockhash)
|
||||
bank.get_fee_calculator(transaction.message().recent_blockhash())
|
||||
})
|
||||
.expect("FeeCalculator must exist");
|
||||
#[allow(deprecated)]
|
||||
let fee = fee_calculator.calculate_fee(transaction.message());
|
||||
let (writable_keys, readonly_keys) =
|
||||
transaction.message.get_account_keys_by_lock_type();
|
||||
let fee = transaction.message().calculate_fee(&fee_calculator);
|
||||
let tx_account_locks = transaction.get_account_locks();
|
||||
|
||||
let inner_instructions = inner_instructions.map(|inner_instructions| {
|
||||
inner_instructions
|
||||
@ -146,9 +144,9 @@ impl TransactionStatusService {
|
||||
blockstore
|
||||
.write_transaction_status(
|
||||
slot,
|
||||
transaction.signatures[0],
|
||||
writable_keys,
|
||||
readonly_keys,
|
||||
*transaction.signature(),
|
||||
tx_account_locks.writable,
|
||||
tx_account_locks.readonly,
|
||||
TransactionStatusMeta {
|
||||
status,
|
||||
fee,
|
||||
|
@ -31,12 +31,11 @@ use solana_sdk::{
|
||||
fee_calculator::FeeCalculator,
|
||||
genesis_config::ClusterType,
|
||||
hash::Hash,
|
||||
message::Message,
|
||||
message::SanitizedMessage,
|
||||
native_loader, nonce,
|
||||
nonce::NONCED_TX_MARKER_IX_INDEX,
|
||||
pubkey::Pubkey,
|
||||
transaction::Result,
|
||||
transaction::{Transaction, TransactionError},
|
||||
transaction::{Result, SanitizedTransaction, TransactionError},
|
||||
};
|
||||
use std::{
|
||||
cmp::Reverse,
|
||||
@ -198,7 +197,7 @@ impl Accounts {
|
||||
}
|
||||
}
|
||||
|
||||
fn construct_instructions_account(message: &Message) -> AccountSharedData {
|
||||
fn construct_instructions_account(message: &SanitizedMessage) -> AccountSharedData {
|
||||
let mut data = message.serialize_instructions();
|
||||
// add room for current instruction index.
|
||||
data.resize(data.len() + 2, 0);
|
||||
@ -211,7 +210,7 @@ impl Accounts {
|
||||
fn load_transaction(
|
||||
&self,
|
||||
ancestors: &Ancestors,
|
||||
tx: &Transaction,
|
||||
tx: &SanitizedTransaction,
|
||||
fee: u64,
|
||||
error_counters: &mut ErrorCounters,
|
||||
rent_collector: &RentCollector,
|
||||
@ -219,19 +218,20 @@ impl Accounts {
|
||||
) -> Result<LoadedTransaction> {
|
||||
// Copy all the accounts
|
||||
let message = tx.message();
|
||||
if tx.signatures.is_empty() && fee != 0 {
|
||||
// NOTE: this check will never fail because `tx` is sanitized
|
||||
if tx.signatures().is_empty() && fee != 0 {
|
||||
Err(TransactionError::MissingSignatureForFee)
|
||||
} else {
|
||||
// There is no way to predict what program will execute without an error
|
||||
// If a fee can pay for execution then the program will be scheduled
|
||||
let mut payer_index = None;
|
||||
let mut tx_rent: TransactionRent = 0;
|
||||
let mut accounts = Vec::with_capacity(message.account_keys.len());
|
||||
let mut account_deps = Vec::with_capacity(message.account_keys.len());
|
||||
let mut accounts = Vec::with_capacity(message.account_keys_len());
|
||||
let mut account_deps = Vec::with_capacity(message.account_keys_len());
|
||||
let mut rent_debits = RentDebits::default();
|
||||
let rent_for_sysvars = feature_set.is_active(&feature_set::rent_for_sysvars::id());
|
||||
|
||||
for (i, key) in message.account_keys.iter().enumerate() {
|
||||
for (i, key) in message.account_keys_iter().enumerate() {
|
||||
let account = if message.is_non_loader_key(i) {
|
||||
if payer_index.is_none() {
|
||||
payer_index = Some(i);
|
||||
@ -296,7 +296,7 @@ impl Accounts {
|
||||
};
|
||||
accounts.push((*key, account));
|
||||
}
|
||||
debug_assert_eq!(accounts.len(), message.account_keys.len());
|
||||
debug_assert_eq!(accounts.len(), message.account_keys_len());
|
||||
// Appends the account_deps at the end of the accounts,
|
||||
// this way they can be accessed in a uniform way.
|
||||
// At places where only the accounts are needed,
|
||||
@ -336,19 +336,9 @@ impl Accounts {
|
||||
|
||||
let message = tx.message();
|
||||
let loaders = message
|
||||
.instructions
|
||||
.iter()
|
||||
.map(|ix| {
|
||||
if message.account_keys.len() <= ix.program_id_index as usize {
|
||||
error_counters.account_not_found += 1;
|
||||
return Err(TransactionError::AccountNotFound);
|
||||
}
|
||||
let program_id = message.account_keys[ix.program_id_index as usize];
|
||||
self.load_executable_accounts(
|
||||
ancestors,
|
||||
&program_id,
|
||||
error_counters,
|
||||
)
|
||||
.program_instructions_iter()
|
||||
.map(|(program_id, _ix)| {
|
||||
self.load_executable_accounts(ancestors, program_id, error_counters)
|
||||
})
|
||||
.collect::<Result<TransactionLoaders>>()?;
|
||||
Ok(LoadedTransaction {
|
||||
@ -434,17 +424,18 @@ impl Accounts {
|
||||
Ok(accounts)
|
||||
}
|
||||
|
||||
pub fn load_accounts<'a>(
|
||||
pub fn load_accounts(
|
||||
&self,
|
||||
ancestors: &Ancestors,
|
||||
txs: impl Iterator<Item = &'a Transaction>,
|
||||
txs: &[SanitizedTransaction],
|
||||
lock_results: Vec<TransactionCheckResult>,
|
||||
hash_queue: &BlockhashQueue,
|
||||
error_counters: &mut ErrorCounters,
|
||||
rent_collector: &RentCollector,
|
||||
feature_set: &FeatureSet,
|
||||
) -> Vec<TransactionLoadResult> {
|
||||
txs.zip(lock_results)
|
||||
txs.iter()
|
||||
.zip(lock_results)
|
||||
.map(|etx| match etx {
|
||||
(tx, (Ok(()), nonce_rollback)) => {
|
||||
let fee_calculator = nonce_rollback
|
||||
@ -453,12 +444,11 @@ impl Accounts {
|
||||
.unwrap_or_else(|| {
|
||||
#[allow(deprecated)]
|
||||
hash_queue
|
||||
.get_fee_calculator(&tx.message().recent_blockhash)
|
||||
.get_fee_calculator(tx.message().recent_blockhash())
|
||||
.cloned()
|
||||
});
|
||||
let fee = if let Some(fee_calculator) = fee_calculator {
|
||||
#[allow(deprecated)]
|
||||
fee_calculator.calculate_fee(tx.message())
|
||||
tx.message().calculate_fee(&fee_calculator)
|
||||
} else {
|
||||
return (Err(TransactionError::BlockhashNotFound), None);
|
||||
};
|
||||
@ -879,15 +869,14 @@ impl Accounts {
|
||||
/// same time
|
||||
#[must_use]
|
||||
#[allow(clippy::needless_collect)]
|
||||
pub fn lock_accounts<'a>(&self, txs: impl Iterator<Item = &'a Transaction>) -> Vec<Result<()>> {
|
||||
let keys: Vec<_> = txs
|
||||
.map(|tx| tx.message().get_account_keys_by_lock_type())
|
||||
.collect();
|
||||
pub fn lock_accounts<'a>(
|
||||
&self,
|
||||
txs: impl Iterator<Item = &'a SanitizedTransaction>,
|
||||
) -> Vec<Result<()>> {
|
||||
let keys: Vec<_> = txs.map(|tx| tx.get_account_locks()).collect();
|
||||
let mut account_locks = &mut self.account_locks.lock().unwrap();
|
||||
keys.into_iter()
|
||||
.map(|(writable_keys, readonly_keys)| {
|
||||
self.lock_account(&mut account_locks, writable_keys, readonly_keys)
|
||||
})
|
||||
.map(|keys| self.lock_account(&mut account_locks, keys.writable, keys.readonly))
|
||||
.collect()
|
||||
}
|
||||
|
||||
@ -895,7 +884,7 @@ impl Accounts {
|
||||
#[allow(clippy::needless_collect)]
|
||||
pub fn unlock_accounts<'a>(
|
||||
&self,
|
||||
txs: impl Iterator<Item = &'a Transaction>,
|
||||
txs: impl Iterator<Item = &'a SanitizedTransaction>,
|
||||
results: &[Result<()>],
|
||||
) {
|
||||
let keys: Vec<_> = txs
|
||||
@ -904,13 +893,13 @@ impl Accounts {
|
||||
Err(TransactionError::AccountInUse) => None,
|
||||
Err(TransactionError::SanitizeFailure) => None,
|
||||
Err(TransactionError::AccountLoadedTwice) => None,
|
||||
_ => Some(tx.message.get_account_keys_by_lock_type()),
|
||||
_ => Some(tx.get_account_locks()),
|
||||
})
|
||||
.collect();
|
||||
let mut account_locks = self.account_locks.lock().unwrap();
|
||||
debug!("bank unlock accounts");
|
||||
keys.into_iter().for_each(|(writable_keys, readonly_keys)| {
|
||||
self.unlock_account(&mut account_locks, writable_keys, readonly_keys);
|
||||
keys.into_iter().for_each(|keys| {
|
||||
self.unlock_account(&mut account_locks, keys.writable, keys.readonly);
|
||||
});
|
||||
}
|
||||
|
||||
@ -920,7 +909,7 @@ impl Accounts {
|
||||
pub fn store_cached<'a>(
|
||||
&self,
|
||||
slot: Slot,
|
||||
txs: impl Iterator<Item = &'a Transaction>,
|
||||
txs: &'a [SanitizedTransaction],
|
||||
res: &'a [TransactionExecutionResult],
|
||||
loaded: &'a mut [TransactionLoadResult],
|
||||
rent_collector: &RentCollector,
|
||||
@ -954,7 +943,7 @@ impl Accounts {
|
||||
|
||||
fn collect_accounts_to_store<'a>(
|
||||
&self,
|
||||
txs: impl Iterator<Item = &'a Transaction>,
|
||||
txs: &'a [SanitizedTransaction],
|
||||
res: &'a [TransactionExecutionResult],
|
||||
loaded: &'a mut [TransactionLoadResult],
|
||||
rent_collector: &RentCollector,
|
||||
@ -991,10 +980,10 @@ impl Accounts {
|
||||
(Err(_), _nonce_rollback) => continue,
|
||||
};
|
||||
|
||||
let message = &tx.message();
|
||||
let message = tx.message();
|
||||
let loaded_transaction = raccs.as_mut().unwrap();
|
||||
let mut fee_payer_index = None;
|
||||
for (i, (key, account)) in (0..message.account_keys.len())
|
||||
for (i, (key, account)) in (0..message.account_keys_len())
|
||||
.zip(loaded_transaction.accounts.iter_mut())
|
||||
.filter(|(i, _account)| message.is_non_loader_key(*i))
|
||||
{
|
||||
@ -1131,14 +1120,25 @@ mod tests {
|
||||
message::Message,
|
||||
nonce, nonce_account,
|
||||
rent::Rent,
|
||||
signature::{keypair_from_seed, Keypair, Signer},
|
||||
signature::{keypair_from_seed, signers::Signers, Keypair, Signer},
|
||||
system_instruction, system_program,
|
||||
transaction::Transaction,
|
||||
};
|
||||
use std::{
|
||||
convert::TryFrom,
|
||||
sync::atomic::{AtomicBool, AtomicU64, Ordering},
|
||||
{thread, time},
|
||||
};
|
||||
|
||||
fn new_sanitized_tx<T: Signers>(
|
||||
from_keypairs: &T,
|
||||
message: Message,
|
||||
recent_blockhash: Hash,
|
||||
) -> SanitizedTransaction {
|
||||
SanitizedTransaction::try_from(Transaction::new(from_keypairs, message, recent_blockhash))
|
||||
.unwrap()
|
||||
}
|
||||
|
||||
fn load_accounts_with_fee_and_rent(
|
||||
tx: Transaction,
|
||||
ka: &[(Pubkey, AccountSharedData)],
|
||||
@ -1160,9 +1160,10 @@ mod tests {
|
||||
}
|
||||
|
||||
let ancestors = vec![(0, 0)].into_iter().collect();
|
||||
let sanitized_tx = SanitizedTransaction::try_from(tx).unwrap();
|
||||
accounts.load_accounts(
|
||||
&ancestors,
|
||||
[tx].iter(),
|
||||
&[sanitized_tx],
|
||||
vec![(Ok(()), None)],
|
||||
&hash_queue,
|
||||
error_counters,
|
||||
@ -1190,30 +1191,6 @@ mod tests {
|
||||
load_accounts_with_fee(tx, ka, &fee_calculator, error_counters)
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_load_accounts_no_key() {
|
||||
let accounts: Vec<(Pubkey, AccountSharedData)> = Vec::new();
|
||||
let mut error_counters = ErrorCounters::default();
|
||||
|
||||
let instructions = vec![CompiledInstruction::new(0, &(), vec![0])];
|
||||
let tx = Transaction::new_with_compiled_instructions::<[&Keypair; 0]>(
|
||||
&[],
|
||||
&[],
|
||||
Hash::default(),
|
||||
vec![native_loader::id()],
|
||||
instructions,
|
||||
);
|
||||
|
||||
let loaded_accounts = load_accounts(tx, &accounts, &mut error_counters);
|
||||
|
||||
assert_eq!(error_counters.account_not_found, 1);
|
||||
assert_eq!(loaded_accounts.len(), 1);
|
||||
assert_eq!(
|
||||
loaded_accounts[0],
|
||||
(Err(TransactionError::AccountNotFound), None,)
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_load_accounts_no_account_0_exists() {
|
||||
let accounts: Vec<(Pubkey, AccountSharedData)> = Vec::new();
|
||||
@ -1522,41 +1499,6 @@ mod tests {
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_load_accounts_bad_program_id() {
|
||||
let mut accounts: Vec<(Pubkey, AccountSharedData)> = Vec::new();
|
||||
let mut error_counters = ErrorCounters::default();
|
||||
|
||||
let keypair = Keypair::new();
|
||||
let key0 = keypair.pubkey();
|
||||
let key1 = Pubkey::new(&[5u8; 32]);
|
||||
|
||||
let account = AccountSharedData::new(1, 0, &Pubkey::default());
|
||||
accounts.push((key0, account));
|
||||
|
||||
let mut account = AccountSharedData::new(40, 1, &native_loader::id());
|
||||
account.set_executable(true);
|
||||
accounts.push((key1, account));
|
||||
|
||||
let instructions = vec![CompiledInstruction::new(0, &(), vec![0])];
|
||||
let tx = Transaction::new_with_compiled_instructions(
|
||||
&[&keypair],
|
||||
&[],
|
||||
Hash::default(),
|
||||
vec![key1],
|
||||
instructions,
|
||||
);
|
||||
|
||||
let loaded_accounts = load_accounts(tx, &accounts, &mut error_counters);
|
||||
|
||||
assert_eq!(error_counters.invalid_program_for_execution, 1);
|
||||
assert_eq!(loaded_accounts.len(), 1);
|
||||
assert_eq!(
|
||||
loaded_accounts[0],
|
||||
(Err(TransactionError::InvalidProgramForExecution), None,)
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_load_accounts_bad_owner() {
|
||||
let mut accounts: Vec<(Pubkey, AccountSharedData)> = Vec::new();
|
||||
@ -1784,7 +1726,7 @@ mod tests {
|
||||
Hash::default(),
|
||||
instructions,
|
||||
);
|
||||
let tx = Transaction::new(&[&keypair0], message, Hash::default());
|
||||
let tx = new_sanitized_tx(&[&keypair0], message, Hash::default());
|
||||
let results0 = accounts.lock_accounts([tx.clone()].iter());
|
||||
|
||||
assert!(results0[0].is_ok());
|
||||
@ -1808,7 +1750,7 @@ mod tests {
|
||||
Hash::default(),
|
||||
instructions,
|
||||
);
|
||||
let tx0 = Transaction::new(&[&keypair2], message, Hash::default());
|
||||
let tx0 = new_sanitized_tx(&[&keypair2], message, Hash::default());
|
||||
let instructions = vec![CompiledInstruction::new(2, &(), vec![0, 1])];
|
||||
let message = Message::new_with_compiled_instructions(
|
||||
1,
|
||||
@ -1818,7 +1760,7 @@ mod tests {
|
||||
Hash::default(),
|
||||
instructions,
|
||||
);
|
||||
let tx1 = Transaction::new(&[&keypair1], message, Hash::default());
|
||||
let tx1 = new_sanitized_tx(&[&keypair1], message, Hash::default());
|
||||
let txs = vec![tx0, tx1];
|
||||
let results1 = accounts.lock_accounts(txs.iter());
|
||||
|
||||
@ -1846,7 +1788,7 @@ mod tests {
|
||||
Hash::default(),
|
||||
instructions,
|
||||
);
|
||||
let tx = Transaction::new(&[&keypair1], message, Hash::default());
|
||||
let tx = new_sanitized_tx(&[&keypair1], message, Hash::default());
|
||||
let results2 = accounts.lock_accounts([tx].iter());
|
||||
assert!(results2[0].is_ok()); // Now keypair1 account can be locked as writable
|
||||
|
||||
@ -1895,7 +1837,7 @@ mod tests {
|
||||
Hash::default(),
|
||||
instructions,
|
||||
);
|
||||
let readonly_tx = Transaction::new(&[&keypair0], readonly_message, Hash::default());
|
||||
let readonly_tx = new_sanitized_tx(&[&keypair0], readonly_message, Hash::default());
|
||||
|
||||
let instructions = vec![CompiledInstruction::new(2, &(), vec![0, 1])];
|
||||
let writable_message = Message::new_with_compiled_instructions(
|
||||
@ -1906,7 +1848,7 @@ mod tests {
|
||||
Hash::default(),
|
||||
instructions,
|
||||
);
|
||||
let writable_tx = Transaction::new(&[&keypair1], writable_message, Hash::default());
|
||||
let writable_tx = new_sanitized_tx(&[&keypair1], writable_message, Hash::default());
|
||||
|
||||
let counter_clone = counter.clone();
|
||||
let accounts_clone = accounts_arc.clone();
|
||||
@ -1967,7 +1909,7 @@ mod tests {
|
||||
(message.account_keys[0], account0),
|
||||
(message.account_keys[1], account2.clone()),
|
||||
];
|
||||
let tx0 = Transaction::new(&[&keypair0], message, Hash::default());
|
||||
let tx0 = new_sanitized_tx(&[&keypair0], message, Hash::default());
|
||||
|
||||
let instructions = vec![CompiledInstruction::new(2, &(), vec![0, 1])];
|
||||
let message = Message::new_with_compiled_instructions(
|
||||
@ -1982,7 +1924,7 @@ mod tests {
|
||||
(message.account_keys[0], account1),
|
||||
(message.account_keys[1], account2),
|
||||
];
|
||||
let tx1 = Transaction::new(&[&keypair1], message, Hash::default());
|
||||
let tx1 = new_sanitized_tx(&[&keypair1], message, Hash::default());
|
||||
|
||||
let loaders = vec![(Ok(()), None), (Ok(()), None)];
|
||||
|
||||
@ -2026,9 +1968,9 @@ mod tests {
|
||||
.unwrap()
|
||||
.insert_new_readonly(&pubkey);
|
||||
}
|
||||
let txs = &[tx0, tx1];
|
||||
let txs = vec![tx0, tx1];
|
||||
let collected_accounts = accounts.collect_accounts_to_store(
|
||||
txs.iter(),
|
||||
&txs,
|
||||
&loaders,
|
||||
loaded.as_mut_slice(),
|
||||
&rent_collector,
|
||||
@ -2087,16 +2029,17 @@ mod tests {
|
||||
}
|
||||
|
||||
fn load_accounts_no_store(accounts: &Accounts, tx: Transaction) -> Vec<TransactionLoadResult> {
|
||||
let tx = SanitizedTransaction::try_from(tx).unwrap();
|
||||
let rent_collector = RentCollector::default();
|
||||
let fee_calculator = FeeCalculator::new(10);
|
||||
let mut hash_queue = BlockhashQueue::new(100);
|
||||
hash_queue.register_hash(&tx.message().recent_blockhash, &fee_calculator);
|
||||
hash_queue.register_hash(tx.message().recent_blockhash(), &fee_calculator);
|
||||
|
||||
let ancestors = vec![(0, 0)].into_iter().collect();
|
||||
let mut error_counters = ErrorCounters::default();
|
||||
accounts.load_accounts(
|
||||
&ancestors,
|
||||
[tx].iter(),
|
||||
&[tx],
|
||||
vec![(Ok(()), None)],
|
||||
&hash_queue,
|
||||
&mut error_counters,
|
||||
@ -2357,7 +2300,7 @@ mod tests {
|
||||
(message.account_keys[3], to_account),
|
||||
(message.account_keys[4], recent_blockhashes_sysvar_account),
|
||||
];
|
||||
let tx = Transaction::new(&[&nonce_authority, &from], message, blockhash);
|
||||
let tx = new_sanitized_tx(&[&nonce_authority, &from], message, blockhash);
|
||||
|
||||
let nonce_state =
|
||||
nonce::state::Versions::new_current(nonce::State::Initialized(nonce::state::Data {
|
||||
@ -2404,9 +2347,9 @@ mod tests {
|
||||
false,
|
||||
AccountShrinkThreshold::default(),
|
||||
);
|
||||
let txs = &[tx];
|
||||
let txs = vec![tx];
|
||||
let collected_accounts = accounts.collect_accounts_to_store(
|
||||
txs.iter(),
|
||||
&txs,
|
||||
&loaders,
|
||||
loaded.as_mut_slice(),
|
||||
&rent_collector,
|
||||
@ -2475,7 +2418,7 @@ mod tests {
|
||||
(message.account_keys[3], to_account),
|
||||
(message.account_keys[4], recent_blockhashes_sysvar_account),
|
||||
];
|
||||
let tx = Transaction::new(&[&nonce_authority, &from], message, blockhash);
|
||||
let tx = new_sanitized_tx(&[&nonce_authority, &from], message, blockhash);
|
||||
|
||||
let nonce_state =
|
||||
nonce::state::Versions::new_current(nonce::State::Initialized(nonce::state::Data {
|
||||
@ -2521,9 +2464,9 @@ mod tests {
|
||||
false,
|
||||
AccountShrinkThreshold::default(),
|
||||
);
|
||||
let txs = &[tx];
|
||||
let txs = vec![tx];
|
||||
let collected_accounts = accounts.collect_accounts_to_store(
|
||||
txs.iter(),
|
||||
&txs,
|
||||
&loaders,
|
||||
loaded.as_mut_slice(),
|
||||
&rent_collector,
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -7,7 +7,7 @@ use solana_sdk::{
|
||||
fee_calculator::{FeeCalculator, FeeRateGovernor},
|
||||
hash::Hash,
|
||||
instruction::Instruction,
|
||||
message::Message,
|
||||
message::{Message, SanitizedMessage},
|
||||
pubkey::Pubkey,
|
||||
signature::{Keypair, Signature, Signer},
|
||||
signers::Signers,
|
||||
@ -16,6 +16,7 @@ use solana_sdk::{
|
||||
transport::{Result, TransportError},
|
||||
};
|
||||
use std::{
|
||||
convert::TryFrom,
|
||||
io,
|
||||
sync::{
|
||||
mpsc::{channel, Receiver, Sender},
|
||||
@ -303,8 +304,9 @@ impl SyncClient for BankClient {
|
||||
}
|
||||
|
||||
fn get_fee_for_message(&self, blockhash: &Hash, message: &Message) -> Result<u64> {
|
||||
self.bank
|
||||
.get_fee_for_message(blockhash, message)
|
||||
SanitizedMessage::try_from(message.clone())
|
||||
.ok()
|
||||
.and_then(|message| self.bank.get_fee_for_message(blockhash, &message))
|
||||
.ok_or_else(|| {
|
||||
TransportError::IoError(io::Error::new(
|
||||
io::ErrorKind::Other,
|
||||
|
@ -3,7 +3,7 @@ use crate::{
|
||||
genesis_utils::{self, GenesisConfigInfo, ValidatorVoteKeypairs},
|
||||
vote_sender_types::ReplayVoteSender,
|
||||
};
|
||||
use solana_sdk::{pubkey::Pubkey, sanitized_transaction::SanitizedTransaction, signature::Signer};
|
||||
use solana_sdk::{pubkey::Pubkey, signature::Signer, transaction::SanitizedTransaction};
|
||||
use solana_vote_program::vote_transaction;
|
||||
|
||||
pub fn setup_bank_and_vote_pubkeys_for_tests(
|
||||
@ -44,8 +44,8 @@ pub fn find_and_send_votes(
|
||||
assert!(execution_results[old_account.transaction_result_index]
|
||||
.0
|
||||
.is_ok());
|
||||
let transaction = &sanitized_txs[old_account.transaction_index];
|
||||
if let Some(parsed_vote) = vote_transaction::parse_vote_transaction(transaction) {
|
||||
let tx = &sanitized_txs[old_account.transaction_index];
|
||||
if let Some(parsed_vote) = vote_transaction::parse_sanitized_vote_transaction(tx) {
|
||||
if parsed_vote.1.slots.last().is_some() {
|
||||
let _ = vote_sender.send(parsed_vote);
|
||||
}
|
||||
|
@ -2,7 +2,7 @@ use std::{cell::RefCell, rc::Rc};
|
||||
|
||||
use solana_sdk::{
|
||||
instruction::{CompiledInstruction, Instruction},
|
||||
message::Message,
|
||||
message::SanitizedMessage,
|
||||
};
|
||||
|
||||
/// Records and compiles cross-program invoked instructions
|
||||
@ -12,11 +12,14 @@ pub struct InstructionRecorder {
|
||||
}
|
||||
|
||||
impl InstructionRecorder {
|
||||
pub fn compile_instructions(&self, message: &Message) -> Vec<CompiledInstruction> {
|
||||
pub fn compile_instructions(
|
||||
&self,
|
||||
message: &SanitizedMessage,
|
||||
) -> Option<Vec<CompiledInstruction>> {
|
||||
self.inner
|
||||
.borrow()
|
||||
.iter()
|
||||
.map(|ix| message.compile_instruction(ix))
|
||||
.map(|ix| message.try_compile_instruction(ix))
|
||||
.collect()
|
||||
}
|
||||
|
||||
|
@ -1837,8 +1837,9 @@ mod tests {
|
||||
genesis_config::create_genesis_config,
|
||||
signature::{Keypair, Signer},
|
||||
system_transaction,
|
||||
transaction::SanitizedTransaction,
|
||||
};
|
||||
use std::mem::size_of;
|
||||
use std::{convert::TryFrom, mem::size_of};
|
||||
|
||||
#[test]
|
||||
fn test_serialize_snapshot_data_file_under_limit() {
|
||||
@ -2910,12 +2911,13 @@ mod tests {
|
||||
|
||||
let slot = slot + 1;
|
||||
let bank2 = Arc::new(Bank::new_from_parent(&bank1, &collector, slot));
|
||||
let tx = system_transaction::transfer(
|
||||
let tx = SanitizedTransaction::try_from(system_transaction::transfer(
|
||||
&key1,
|
||||
&key2.pubkey(),
|
||||
lamports_to_transfer,
|
||||
bank2.last_blockhash(),
|
||||
);
|
||||
))
|
||||
.unwrap();
|
||||
let fee = bank2
|
||||
.get_fee_for_message(&bank2.last_blockhash(), tx.message())
|
||||
.unwrap();
|
||||
|
@ -1,16 +1,14 @@
|
||||
use crate::bank::Bank;
|
||||
use solana_sdk::{
|
||||
sanitized_transaction::SanitizedTransaction,
|
||||
transaction::{Result, Transaction},
|
||||
use {
|
||||
crate::bank::Bank,
|
||||
solana_sdk::transaction::{Result, SanitizedTransaction},
|
||||
std::borrow::Cow,
|
||||
};
|
||||
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,
|
||||
sanitized_txs: Cow<'b, [SanitizedTransaction<'b>]>,
|
||||
sanitized_txs: Cow<'b, [SanitizedTransaction]>,
|
||||
pub(crate) needs_unlock: bool,
|
||||
}
|
||||
|
||||
@ -18,7 +16,7 @@ impl<'a, 'b> TransactionBatch<'a, 'b> {
|
||||
pub fn new(
|
||||
lock_results: Vec<Result<()>>,
|
||||
bank: &'a Bank,
|
||||
sanitized_txs: Cow<'b, [SanitizedTransaction<'b>]>,
|
||||
sanitized_txs: Cow<'b, [SanitizedTransaction]>,
|
||||
) -> Self {
|
||||
assert_eq!(lock_results.len(), sanitized_txs.len());
|
||||
Self {
|
||||
@ -37,10 +35,6 @@ impl<'a, 'b> TransactionBatch<'a, 'b> {
|
||||
&self.sanitized_txs
|
||||
}
|
||||
|
||||
pub fn transactions_iter(&self) -> impl Iterator<Item = &Transaction> {
|
||||
self.sanitized_txs.iter().map(Deref::deref)
|
||||
}
|
||||
|
||||
pub fn bank(&self) -> &Bank {
|
||||
self.bank
|
||||
}
|
||||
@ -58,38 +52,33 @@ 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;
|
||||
use std::convert::TryInto;
|
||||
|
||||
#[test]
|
||||
fn test_transaction_batch() {
|
||||
let (bank, txs) = setup();
|
||||
|
||||
// Test getting locked accounts
|
||||
let batch = bank.prepare_batch(txs.iter()).unwrap();
|
||||
let batch = bank.prepare_sanitized_batch(&txs);
|
||||
|
||||
// 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()).unwrap();
|
||||
let batch2 = bank.prepare_sanitized_batch(&txs);
|
||||
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()).unwrap();
|
||||
let batch2 = bank.prepare_sanitized_batch(&txs);
|
||||
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].clone());
|
||||
@ -104,7 +93,7 @@ mod tests {
|
||||
assert!(batch3.lock_results().iter().all(|x| x.is_ok()));
|
||||
}
|
||||
|
||||
fn setup() -> (Bank, Vec<Transaction>) {
|
||||
fn setup() -> (Bank, Vec<SanitizedTransaction>) {
|
||||
let dummy_leader_pubkey = solana_sdk::pubkey::new_rand();
|
||||
let GenesisConfigInfo {
|
||||
genesis_config,
|
||||
@ -118,8 +107,12 @@ mod tests {
|
||||
let pubkey2 = solana_sdk::pubkey::new_rand();
|
||||
|
||||
let txs = vec![
|
||||
system_transaction::transfer(&mint_keypair, &pubkey, 1, genesis_config.hash()),
|
||||
system_transaction::transfer(&keypair2, &pubkey2, 1, genesis_config.hash()),
|
||||
system_transaction::transfer(&mint_keypair, &pubkey, 1, genesis_config.hash())
|
||||
.try_into()
|
||||
.unwrap(),
|
||||
system_transaction::transfer(&keypair2, &pubkey2, 1, genesis_config.hash())
|
||||
.try_into()
|
||||
.unwrap(),
|
||||
];
|
||||
|
||||
(bank, txs)
|
||||
|
@ -1,13 +0,0 @@
|
||||
#![feature(test)]
|
||||
|
||||
extern crate test;
|
||||
use solana_sdk::sanitized_transaction::SanitizedTransaction;
|
||||
use test::Bencher;
|
||||
|
||||
#[bench]
|
||||
fn bench_has_duplicates(bencher: &mut Bencher) {
|
||||
bencher.iter(|| {
|
||||
let data = test::black_box([1, 2, 3]);
|
||||
assert!(!SanitizedTransaction::has_duplicates(&data));
|
||||
})
|
||||
}
|
@ -3,9 +3,10 @@
|
||||
extern crate test;
|
||||
use bincode::{deserialize, serialize};
|
||||
use solana_sdk::instruction::{AccountMeta, Instruction};
|
||||
use solana_sdk::message::Message;
|
||||
use solana_sdk::pubkey;
|
||||
use solana_sdk::message::{Message, SanitizedMessage};
|
||||
use solana_sdk::pubkey::{self, Pubkey};
|
||||
use solana_sdk::sysvar::instructions;
|
||||
use std::convert::TryFrom;
|
||||
use test::Bencher;
|
||||
|
||||
fn make_instructions() -> Vec<Instruction> {
|
||||
@ -25,7 +26,9 @@ fn bench_bincode_instruction_serialize(b: &mut Bencher) {
|
||||
#[bench]
|
||||
fn bench_manual_instruction_serialize(b: &mut Bencher) {
|
||||
let instructions = make_instructions();
|
||||
let message = Message::new(&instructions, None);
|
||||
let message =
|
||||
SanitizedMessage::try_from(Message::new(&instructions, Some(&Pubkey::new_unique())))
|
||||
.unwrap();
|
||||
b.iter(|| {
|
||||
test::black_box(message.serialize_instructions());
|
||||
});
|
||||
@ -43,7 +46,9 @@ fn bench_bincode_instruction_deserialize(b: &mut Bencher) {
|
||||
#[bench]
|
||||
fn bench_manual_instruction_deserialize(b: &mut Bencher) {
|
||||
let instructions = make_instructions();
|
||||
let message = Message::new(&instructions, None);
|
||||
let message =
|
||||
SanitizedMessage::try_from(Message::new(&instructions, Some(&Pubkey::new_unique())))
|
||||
.unwrap();
|
||||
let serialized = message.serialize_instructions();
|
||||
b.iter(|| {
|
||||
for i in 0..instructions.len() {
|
||||
@ -55,7 +60,9 @@ fn bench_manual_instruction_deserialize(b: &mut Bencher) {
|
||||
#[bench]
|
||||
fn bench_manual_instruction_deserialize_single(b: &mut Bencher) {
|
||||
let instructions = make_instructions();
|
||||
let message = Message::new(&instructions, None);
|
||||
let message =
|
||||
SanitizedMessage::try_from(Message::new(&instructions, Some(&Pubkey::new_unique())))
|
||||
.unwrap();
|
||||
let serialized = message.serialize_instructions();
|
||||
b.iter(|| {
|
||||
test::black_box(instructions::load_instruction_at(3, &serialized).unwrap());
|
||||
|
@ -35,6 +35,7 @@ solana-sdk-macro = { path = "../macro", version = "=1.8.0" }
|
||||
thiserror = "1.0"
|
||||
|
||||
[target.'cfg(not(target_arch = "bpf"))'.dependencies]
|
||||
bitflags = "1.3.1"
|
||||
curve25519-dalek = "3.0.0"
|
||||
libsecp256k1 = "0.6.0"
|
||||
rand = "0.7.0"
|
||||
|
@ -369,6 +369,7 @@ impl Message {
|
||||
i < self.header.num_required_signatures as usize
|
||||
}
|
||||
|
||||
#[deprecated]
|
||||
pub fn get_account_keys_by_lock_type(&self) -> (Vec<&Pubkey>, Vec<&Pubkey>) {
|
||||
let mut writable_keys = vec![];
|
||||
let mut readonly_keys = vec![];
|
||||
@ -395,6 +396,7 @@ impl Message {
|
||||
// 35..67 - program_id
|
||||
// 67..69 - data len - u16
|
||||
// 69..data_len - data
|
||||
#[deprecated]
|
||||
pub fn serialize_instructions(&self) -> Vec<u8> {
|
||||
// 64 bytes is a reasonable guess, calculating exactly is slower in benchmarks
|
||||
let mut data = Vec::with_capacity(self.instructions.len() * (32 * 2));
|
||||
@ -485,10 +487,25 @@ impl Message {
|
||||
.min(self.header.num_required_signatures as usize);
|
||||
self.account_keys[..last_key].iter().collect()
|
||||
}
|
||||
|
||||
/// Return true if account_keys has any duplicate keys
|
||||
pub fn has_duplicates(&self) -> 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..self.account_keys.len() {
|
||||
#[allow(clippy::integer_arithmetic)]
|
||||
if self.account_keys[i..].contains(&self.account_keys[i - 1]) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
false
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
#![allow(deprecated)]
|
||||
use super::*;
|
||||
use crate::{hash, instruction::AccountMeta, message::MESSAGE_HEADER_LENGTH};
|
||||
use std::collections::HashSet;
|
||||
|
245
sdk/program/src/message/mapped.rs
Normal file
245
sdk/program/src/message/mapped.rs
Normal file
@ -0,0 +1,245 @@
|
||||
use {
|
||||
crate::{
|
||||
message::{legacy::BUILTIN_PROGRAMS_KEYS, v0},
|
||||
pubkey::Pubkey,
|
||||
sysvar,
|
||||
},
|
||||
std::collections::HashSet,
|
||||
};
|
||||
|
||||
/// Combination of a version #0 message and its mapped addresses
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct MappedMessage {
|
||||
/// Message which loaded a collection of mapped addresses
|
||||
pub message: v0::Message,
|
||||
/// Collection of mapped addresses loaded by this message
|
||||
pub mapped_addresses: MappedAddresses,
|
||||
}
|
||||
|
||||
/// Collection of mapped addresses loaded succinctly by a transaction using
|
||||
/// on-chain address map accounts.
|
||||
#[derive(Clone, Default, Debug, PartialEq, Serialize, Deserialize)]
|
||||
pub struct MappedAddresses {
|
||||
/// List of addresses for writable loaded accounts
|
||||
pub writable: Vec<Pubkey>,
|
||||
/// List of addresses for read-only loaded accounts
|
||||
pub readonly: Vec<Pubkey>,
|
||||
}
|
||||
|
||||
impl MappedMessage {
|
||||
/// Returns an iterator of account key segments. The ordering of segments
|
||||
/// affects how account indexes from compiled instructions are resolved and
|
||||
/// so should not be changed.
|
||||
fn account_keys_segment_iter(&self) -> impl Iterator<Item = &Vec<Pubkey>> {
|
||||
vec![
|
||||
&self.message.account_keys,
|
||||
&self.mapped_addresses.writable,
|
||||
&self.mapped_addresses.readonly,
|
||||
]
|
||||
.into_iter()
|
||||
}
|
||||
|
||||
/// Returns the total length of loaded accounts for this message
|
||||
pub fn account_keys_len(&self) -> usize {
|
||||
let mut len = 0usize;
|
||||
for key_segment in self.account_keys_segment_iter() {
|
||||
len = len.saturating_add(key_segment.len());
|
||||
}
|
||||
len
|
||||
}
|
||||
|
||||
/// Iterator for the addresses of the loaded accounts for this message
|
||||
pub fn account_keys_iter(&self) -> impl Iterator<Item = &Pubkey> {
|
||||
self.account_keys_segment_iter().flatten()
|
||||
}
|
||||
|
||||
/// Returns true if any account keys are duplicates
|
||||
pub fn has_duplicates(&self) -> bool {
|
||||
let mut uniq = HashSet::new();
|
||||
self.account_keys_iter().any(|x| !uniq.insert(x))
|
||||
}
|
||||
|
||||
/// Returns the address of the account at the specified index of the list of
|
||||
/// message account keys constructed from unmapped keys, followed by mapped
|
||||
/// writable addresses, and lastly the list of mapped readonly addresses.
|
||||
pub fn get_account_key(&self, mut index: usize) -> Option<&Pubkey> {
|
||||
for key_segment in self.account_keys_segment_iter() {
|
||||
if index < key_segment.len() {
|
||||
return Some(&key_segment[index]);
|
||||
}
|
||||
index = index.saturating_sub(key_segment.len());
|
||||
}
|
||||
|
||||
None
|
||||
}
|
||||
|
||||
/// Returns true if the account at the specified index was requested to be
|
||||
/// writable. This method should not be used directly.
|
||||
fn is_writable_index(&self, key_index: usize) -> bool {
|
||||
let header = &self.message.header;
|
||||
let num_account_keys = self.message.account_keys.len();
|
||||
let num_signed_accounts = usize::from(header.num_required_signatures);
|
||||
if key_index >= num_account_keys {
|
||||
let mapped_addresses_index = key_index.saturating_sub(num_account_keys);
|
||||
mapped_addresses_index < self.mapped_addresses.writable.len()
|
||||
} else if key_index >= num_signed_accounts {
|
||||
let num_unsigned_accounts = num_account_keys.saturating_sub(num_signed_accounts);
|
||||
let num_writable_unsigned_accounts = num_unsigned_accounts
|
||||
.saturating_sub(usize::from(header.num_readonly_unsigned_accounts));
|
||||
let unsigned_account_index = key_index.saturating_sub(num_signed_accounts);
|
||||
unsigned_account_index < num_writable_unsigned_accounts
|
||||
} else {
|
||||
let num_writable_signed_accounts = num_signed_accounts
|
||||
.saturating_sub(usize::from(header.num_readonly_signed_accounts));
|
||||
key_index < num_writable_signed_accounts
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns true if the account at the specified index was loaded as writable
|
||||
pub fn is_writable(&self, key_index: usize) -> bool {
|
||||
if self.is_writable_index(key_index) {
|
||||
if let Some(key) = self.get_account_key(key_index) {
|
||||
return !(sysvar::is_sysvar_id(key) || BUILTIN_PROGRAMS_KEYS.contains(key));
|
||||
}
|
||||
}
|
||||
false
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
use crate::{message::MessageHeader, system_program, sysvar};
|
||||
use itertools::Itertools;
|
||||
|
||||
fn create_test_mapped_message() -> (MappedMessage, [Pubkey; 6]) {
|
||||
let key0 = Pubkey::new_unique();
|
||||
let key1 = Pubkey::new_unique();
|
||||
let key2 = Pubkey::new_unique();
|
||||
let key3 = Pubkey::new_unique();
|
||||
let key4 = Pubkey::new_unique();
|
||||
let key5 = Pubkey::new_unique();
|
||||
|
||||
let message = MappedMessage {
|
||||
message: v0::Message {
|
||||
header: MessageHeader {
|
||||
num_required_signatures: 2,
|
||||
num_readonly_signed_accounts: 1,
|
||||
num_readonly_unsigned_accounts: 1,
|
||||
},
|
||||
account_keys: vec![key0, key1, key2, key3],
|
||||
..v0::Message::default()
|
||||
},
|
||||
mapped_addresses: MappedAddresses {
|
||||
writable: vec![key4],
|
||||
readonly: vec![key5],
|
||||
},
|
||||
};
|
||||
|
||||
(message, [key0, key1, key2, key3, key4, key5])
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_account_keys_segment_iter() {
|
||||
let (message, keys) = create_test_mapped_message();
|
||||
|
||||
let expected_segments = vec![
|
||||
vec![keys[0], keys[1], keys[2], keys[3]],
|
||||
vec![keys[4]],
|
||||
vec![keys[5]],
|
||||
];
|
||||
|
||||
let mut iter = message.account_keys_segment_iter();
|
||||
for expected_segment in expected_segments {
|
||||
assert_eq!(iter.next(), Some(&expected_segment));
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_account_keys_len() {
|
||||
let (message, keys) = create_test_mapped_message();
|
||||
|
||||
assert_eq!(message.account_keys_len(), keys.len());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_account_keys_iter() {
|
||||
let (message, keys) = create_test_mapped_message();
|
||||
|
||||
let mut iter = message.account_keys_iter();
|
||||
for expected_key in keys {
|
||||
assert_eq!(iter.next(), Some(&expected_key));
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_has_duplicates() {
|
||||
let message = create_test_mapped_message().0;
|
||||
|
||||
assert!(!message.has_duplicates());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_has_duplicates_with_dupe_keys() {
|
||||
let create_message_with_dupe_keys = |mut keys: Vec<Pubkey>| MappedMessage {
|
||||
message: v0::Message {
|
||||
account_keys: keys.split_off(2),
|
||||
..v0::Message::default()
|
||||
},
|
||||
mapped_addresses: MappedAddresses {
|
||||
writable: keys.split_off(2),
|
||||
readonly: keys,
|
||||
},
|
||||
};
|
||||
|
||||
let key0 = Pubkey::new_unique();
|
||||
let key1 = Pubkey::new_unique();
|
||||
let key2 = Pubkey::new_unique();
|
||||
let key3 = Pubkey::new_unique();
|
||||
let dupe_key = Pubkey::new_unique();
|
||||
|
||||
let keys = vec![key0, key1, key2, key3, dupe_key, dupe_key];
|
||||
let keys_len = keys.len();
|
||||
for keys in keys.into_iter().permutations(keys_len).unique() {
|
||||
let message = create_message_with_dupe_keys(keys);
|
||||
assert!(message.has_duplicates());
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_get_account_key() {
|
||||
let (message, keys) = create_test_mapped_message();
|
||||
|
||||
assert_eq!(message.get_account_key(0), Some(&keys[0]));
|
||||
assert_eq!(message.get_account_key(1), Some(&keys[1]));
|
||||
assert_eq!(message.get_account_key(2), Some(&keys[2]));
|
||||
assert_eq!(message.get_account_key(3), Some(&keys[3]));
|
||||
assert_eq!(message.get_account_key(4), Some(&keys[4]));
|
||||
assert_eq!(message.get_account_key(5), Some(&keys[5]));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_is_writable_index() {
|
||||
let message = create_test_mapped_message().0;
|
||||
|
||||
assert!(message.is_writable_index(0));
|
||||
assert!(!message.is_writable_index(1));
|
||||
assert!(message.is_writable_index(2));
|
||||
assert!(!message.is_writable_index(3));
|
||||
assert!(message.is_writable_index(4));
|
||||
assert!(!message.is_writable_index(5));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_is_writable() {
|
||||
let mut mapped_msg = create_test_mapped_message().0;
|
||||
|
||||
mapped_msg.message.account_keys[0] = sysvar::clock::id();
|
||||
assert!(mapped_msg.is_writable_index(0));
|
||||
assert!(!mapped_msg.is_writable(0));
|
||||
|
||||
mapped_msg.message.account_keys[0] = system_program::id();
|
||||
assert!(mapped_msg.is_writable_index(0));
|
||||
assert!(!mapped_msg.is_writable(0));
|
||||
}
|
||||
}
|
@ -1,12 +1,26 @@
|
||||
//! A library for generating a message from a sequence of instructions
|
||||
|
||||
mod legacy;
|
||||
mod v0;
|
||||
mod versions;
|
||||
pub mod legacy;
|
||||
|
||||
#[cfg(not(target_arch = "bpf"))]
|
||||
#[path = ""]
|
||||
mod non_bpf_modules {
|
||||
mod mapped;
|
||||
mod sanitized;
|
||||
pub mod v0;
|
||||
mod versions;
|
||||
|
||||
pub use mapped::*;
|
||||
pub use sanitized::*;
|
||||
pub use versions::*;
|
||||
}
|
||||
|
||||
pub use legacy::Message;
|
||||
pub use versions::MESSAGE_VERSION_PREFIX;
|
||||
|
||||
#[cfg(not(target_arch = "bpf"))]
|
||||
pub use non_bpf_modules::*;
|
||||
|
||||
/// The length of a message header in bytes
|
||||
pub const MESSAGE_HEADER_LENGTH: usize = 3;
|
||||
|
||||
#[derive(Serialize, Deserialize, Default, Debug, PartialEq, Eq, Clone, AbiExample)]
|
||||
|
599
sdk/program/src/message/sanitized.rs
Normal file
599
sdk/program/src/message/sanitized.rs
Normal file
@ -0,0 +1,599 @@
|
||||
use {
|
||||
crate::{
|
||||
fee_calculator::FeeCalculator,
|
||||
hash::Hash,
|
||||
instruction::{CompiledInstruction, Instruction},
|
||||
message::{MappedAddresses, MappedMessage, Message, MessageHeader},
|
||||
pubkey::Pubkey,
|
||||
sanitize::{Sanitize, SanitizeError},
|
||||
secp256k1_program,
|
||||
serialize_utils::{append_slice, append_u16, append_u8},
|
||||
},
|
||||
bitflags::bitflags,
|
||||
std::convert::TryFrom,
|
||||
thiserror::Error,
|
||||
};
|
||||
|
||||
/// Sanitized message of a transaction which includes a set of atomic
|
||||
/// instructions to be executed on-chain
|
||||
#[derive(Debug, Clone)]
|
||||
pub enum SanitizedMessage {
|
||||
/// Sanitized legacy message
|
||||
Legacy(Message),
|
||||
/// Sanitized version #0 message with mapped addresses
|
||||
V0(MappedMessage),
|
||||
}
|
||||
|
||||
#[derive(PartialEq, Debug, Error, Eq, Clone)]
|
||||
pub enum SanitizeMessageError {
|
||||
#[error("index out of bounds")]
|
||||
IndexOutOfBounds,
|
||||
#[error("value out of bounds")]
|
||||
ValueOutOfBounds,
|
||||
#[error("invalid value")]
|
||||
InvalidValue,
|
||||
#[error("duplicate account key")]
|
||||
DuplicateAccountKey,
|
||||
}
|
||||
|
||||
impl From<SanitizeError> for SanitizeMessageError {
|
||||
fn from(err: SanitizeError) -> Self {
|
||||
match err {
|
||||
SanitizeError::IndexOutOfBounds => Self::IndexOutOfBounds,
|
||||
SanitizeError::ValueOutOfBounds => Self::ValueOutOfBounds,
|
||||
SanitizeError::InvalidValue => Self::InvalidValue,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl TryFrom<Message> for SanitizedMessage {
|
||||
type Error = SanitizeMessageError;
|
||||
fn try_from(message: Message) -> Result<Self, Self::Error> {
|
||||
message.sanitize()?;
|
||||
|
||||
let sanitized_msg = Self::Legacy(message);
|
||||
if sanitized_msg.has_duplicates() {
|
||||
return Err(SanitizeMessageError::DuplicateAccountKey);
|
||||
}
|
||||
|
||||
Ok(sanitized_msg)
|
||||
}
|
||||
}
|
||||
|
||||
bitflags! {
|
||||
struct InstructionsSysvarAccountMeta: u8 {
|
||||
const NONE = 0b00000000;
|
||||
const IS_SIGNER = 0b00000001;
|
||||
const IS_WRITABLE = 0b00000010;
|
||||
}
|
||||
}
|
||||
|
||||
impl SanitizedMessage {
|
||||
/// Return true if this message contains duplicate account keys
|
||||
pub fn has_duplicates(&self) -> bool {
|
||||
match self {
|
||||
SanitizedMessage::Legacy(message) => message.has_duplicates(),
|
||||
SanitizedMessage::V0(message) => message.has_duplicates(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Message header which identifies the number of signer and writable or
|
||||
/// readonly accounts
|
||||
pub fn header(&self) -> &MessageHeader {
|
||||
match self {
|
||||
Self::Legacy(message) => &message.header,
|
||||
Self::V0(mapped_msg) => &mapped_msg.message.header,
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns a legacy message if this sanitized message wraps one
|
||||
pub fn legacy_message(&self) -> Option<&Message> {
|
||||
if let Self::Legacy(message) = &self {
|
||||
Some(message)
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns the fee payer for the transaction
|
||||
pub fn fee_payer(&self) -> &Pubkey {
|
||||
self.get_account_key(0)
|
||||
.expect("sanitized message always has non-program fee payer at index 0")
|
||||
}
|
||||
|
||||
/// The hash of a recent block, used for timing out a transaction
|
||||
pub fn recent_blockhash(&self) -> &Hash {
|
||||
match self {
|
||||
Self::Legacy(message) => &message.recent_blockhash,
|
||||
Self::V0(mapped_msg) => &mapped_msg.message.recent_blockhash,
|
||||
}
|
||||
}
|
||||
|
||||
/// Program instructions that will be executed in sequence and committed in
|
||||
/// one atomic transaction if all succeed.
|
||||
pub fn instructions(&self) -> &[CompiledInstruction] {
|
||||
match self {
|
||||
Self::Legacy(message) => &message.instructions,
|
||||
Self::V0(mapped_msg) => &mapped_msg.message.instructions,
|
||||
}
|
||||
}
|
||||
|
||||
/// Program instructions iterator which includes each instruction's program
|
||||
/// id.
|
||||
pub fn program_instructions_iter(
|
||||
&self,
|
||||
) -> impl Iterator<Item = (&Pubkey, &CompiledInstruction)> {
|
||||
match self {
|
||||
Self::Legacy(message) => message.instructions.iter(),
|
||||
Self::V0(mapped_msg) => mapped_msg.message.instructions.iter(),
|
||||
}
|
||||
.map(move |ix| {
|
||||
(
|
||||
self.get_account_key(usize::from(ix.program_id_index))
|
||||
.expect("program id index is sanitized"),
|
||||
ix,
|
||||
)
|
||||
})
|
||||
}
|
||||
|
||||
/// Iterator of all account keys referenced in this message, included mapped keys.
|
||||
pub fn account_keys_iter(&self) -> Box<dyn Iterator<Item = &Pubkey> + '_> {
|
||||
match self {
|
||||
Self::Legacy(message) => Box::new(message.account_keys.iter()),
|
||||
Self::V0(mapped_msg) => Box::new(mapped_msg.account_keys_iter()),
|
||||
}
|
||||
}
|
||||
|
||||
/// Length of all account keys referenced in this message, included mapped keys.
|
||||
pub fn account_keys_len(&self) -> usize {
|
||||
match self {
|
||||
Self::Legacy(message) => message.account_keys.len(),
|
||||
Self::V0(mapped_msg) => mapped_msg.account_keys_len(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns the address of the account at the specified index.
|
||||
pub fn get_account_key(&self, index: usize) -> Option<&Pubkey> {
|
||||
match self {
|
||||
Self::Legacy(message) => message.account_keys.get(index),
|
||||
Self::V0(message) => message.get_account_key(index),
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns true if the account at the specified index is an input to some
|
||||
/// program instruction in this message.
|
||||
fn is_key_passed_to_program(&self, key_index: usize) -> bool {
|
||||
if let Ok(key_index) = u8::try_from(key_index) {
|
||||
self.instructions()
|
||||
.iter()
|
||||
.any(|ix| ix.accounts.contains(&key_index))
|
||||
} else {
|
||||
false
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns true if the account at the specified index is invoked as a
|
||||
/// program in this message.
|
||||
pub fn is_invoked(&self, key_index: usize) -> bool {
|
||||
if let Ok(key_index) = u8::try_from(key_index) {
|
||||
self.instructions()
|
||||
.iter()
|
||||
.any(|ix| ix.program_id_index == key_index)
|
||||
} else {
|
||||
false
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns true if the account at the specified index is not invoked as a
|
||||
/// program or, if invoked, is passed to a program.
|
||||
pub fn is_non_loader_key(&self, key_index: usize) -> bool {
|
||||
!self.is_invoked(key_index) || self.is_key_passed_to_program(key_index)
|
||||
}
|
||||
|
||||
/// Returns true if the account at the specified index is writable by the
|
||||
/// instructions in this message.
|
||||
pub fn is_writable(&self, index: usize) -> bool {
|
||||
match self {
|
||||
Self::Legacy(message) => message.is_writable(index),
|
||||
Self::V0(message) => message.is_writable(index),
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns true if the account at the specified index signed this
|
||||
/// message.
|
||||
pub fn is_signer(&self, index: usize) -> bool {
|
||||
index < usize::from(self.header().num_required_signatures)
|
||||
}
|
||||
|
||||
// First encode the number of instructions:
|
||||
// [0..2 - num_instructions
|
||||
//
|
||||
// Then a table of offsets of where to find them in the data
|
||||
// 3..2 * num_instructions table of instruction offsets
|
||||
//
|
||||
// Each instruction is then encoded as:
|
||||
// 0..2 - num_accounts
|
||||
// 2 - meta_byte -> (bit 0 signer, bit 1 is_writable)
|
||||
// 3..35 - pubkey - 32 bytes
|
||||
// 35..67 - program_id
|
||||
// 67..69 - data len - u16
|
||||
// 69..data_len - data
|
||||
#[allow(clippy::integer_arithmetic)]
|
||||
pub fn serialize_instructions(&self) -> Vec<u8> {
|
||||
// 64 bytes is a reasonable guess, calculating exactly is slower in benchmarks
|
||||
let mut data = Vec::with_capacity(self.instructions().len() * (32 * 2));
|
||||
append_u16(&mut data, self.instructions().len() as u16);
|
||||
for _ in 0..self.instructions().len() {
|
||||
append_u16(&mut data, 0);
|
||||
}
|
||||
for (i, (program_id, instruction)) in self.program_instructions_iter().enumerate() {
|
||||
let start_instruction_offset = data.len() as u16;
|
||||
let start = 2 + (2 * i);
|
||||
data[start..start + 2].copy_from_slice(&start_instruction_offset.to_le_bytes());
|
||||
append_u16(&mut data, instruction.accounts.len() as u16);
|
||||
for account_index in &instruction.accounts {
|
||||
let account_index = *account_index as usize;
|
||||
let is_signer = self.is_signer(account_index);
|
||||
let is_writable = self.is_writable(account_index);
|
||||
let mut account_meta = InstructionsSysvarAccountMeta::NONE;
|
||||
if is_signer {
|
||||
account_meta |= InstructionsSysvarAccountMeta::IS_SIGNER;
|
||||
}
|
||||
if is_writable {
|
||||
account_meta |= InstructionsSysvarAccountMeta::IS_WRITABLE;
|
||||
}
|
||||
append_u8(&mut data, account_meta.bits());
|
||||
append_slice(
|
||||
&mut data,
|
||||
self.get_account_key(account_index).unwrap().as_ref(),
|
||||
);
|
||||
}
|
||||
|
||||
append_slice(&mut data, program_id.as_ref());
|
||||
append_u16(&mut data, instruction.data.len() as u16);
|
||||
append_slice(&mut data, &instruction.data);
|
||||
}
|
||||
data
|
||||
}
|
||||
|
||||
/// Return the mapped addresses for this message if it has any.
|
||||
fn mapped_addresses(&self) -> Option<&MappedAddresses> {
|
||||
match &self {
|
||||
SanitizedMessage::V0(message) => Some(&message.mapped_addresses),
|
||||
_ => None,
|
||||
}
|
||||
}
|
||||
|
||||
/// Return the number of readonly accounts loaded by this message.
|
||||
pub fn num_readonly_accounts(&self) -> usize {
|
||||
let mapped_readonly_addresses = self
|
||||
.mapped_addresses()
|
||||
.map(|keys| keys.readonly.len())
|
||||
.unwrap_or_default();
|
||||
mapped_readonly_addresses
|
||||
.saturating_add(usize::from(self.header().num_readonly_signed_accounts))
|
||||
.saturating_add(usize::from(self.header().num_readonly_unsigned_accounts))
|
||||
}
|
||||
|
||||
fn try_position(&self, key: &Pubkey) -> Option<u8> {
|
||||
u8::try_from(self.account_keys_iter().position(|k| k == key)?).ok()
|
||||
}
|
||||
|
||||
/// Try to compile an instruction using the account keys in this message.
|
||||
pub fn try_compile_instruction(&self, ix: &Instruction) -> Option<CompiledInstruction> {
|
||||
let accounts: Vec<_> = ix
|
||||
.accounts
|
||||
.iter()
|
||||
.map(|account_meta| self.try_position(&account_meta.pubkey))
|
||||
.collect::<Option<_>>()?;
|
||||
|
||||
Some(CompiledInstruction {
|
||||
program_id_index: self.try_position(&ix.program_id)?,
|
||||
data: ix.data.clone(),
|
||||
accounts,
|
||||
})
|
||||
}
|
||||
|
||||
/// Calculate the total fees for a transaction given a fee calculator
|
||||
pub fn calculate_fee(&self, fee_calculator: &FeeCalculator) -> u64 {
|
||||
let mut num_secp256k1_signatures: u64 = 0;
|
||||
for (program_id, instruction) in self.program_instructions_iter() {
|
||||
if secp256k1_program::check_id(program_id) {
|
||||
if let Some(num_signatures) = instruction.data.get(0) {
|
||||
num_secp256k1_signatures =
|
||||
num_secp256k1_signatures.saturating_add(u64::from(*num_signatures));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fee_calculator.lamports_per_signature.saturating_mul(
|
||||
u64::from(self.header().num_required_signatures)
|
||||
.saturating_add(num_secp256k1_signatures),
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
use crate::{
|
||||
instruction::{AccountMeta, Instruction},
|
||||
message::v0,
|
||||
secp256k1_program, system_instruction,
|
||||
};
|
||||
|
||||
#[test]
|
||||
fn test_try_from_message() {
|
||||
let dupe_key = Pubkey::new_unique();
|
||||
let legacy_message_with_dupes = Message {
|
||||
header: MessageHeader {
|
||||
num_required_signatures: 1,
|
||||
..MessageHeader::default()
|
||||
},
|
||||
account_keys: vec![dupe_key, dupe_key],
|
||||
..Message::default()
|
||||
};
|
||||
|
||||
assert_eq!(
|
||||
SanitizedMessage::try_from(legacy_message_with_dupes).err(),
|
||||
Some(SanitizeMessageError::DuplicateAccountKey),
|
||||
);
|
||||
|
||||
let legacy_message_with_no_signers = Message {
|
||||
account_keys: vec![Pubkey::new_unique()],
|
||||
..Message::default()
|
||||
};
|
||||
|
||||
assert_eq!(
|
||||
SanitizedMessage::try_from(legacy_message_with_no_signers).err(),
|
||||
Some(SanitizeMessageError::IndexOutOfBounds),
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_is_non_loader_key() {
|
||||
let key0 = Pubkey::new_unique();
|
||||
let key1 = Pubkey::new_unique();
|
||||
let loader_key = Pubkey::new_unique();
|
||||
let instructions = vec![
|
||||
CompiledInstruction::new(1, &(), vec![0]),
|
||||
CompiledInstruction::new(2, &(), vec![0, 1]),
|
||||
];
|
||||
|
||||
let message = SanitizedMessage::try_from(Message::new_with_compiled_instructions(
|
||||
1,
|
||||
0,
|
||||
2,
|
||||
vec![key0, key1, loader_key],
|
||||
Hash::default(),
|
||||
instructions,
|
||||
))
|
||||
.unwrap();
|
||||
|
||||
assert!(message.is_non_loader_key(0));
|
||||
assert!(message.is_non_loader_key(1));
|
||||
assert!(!message.is_non_loader_key(2));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_num_readonly_accounts() {
|
||||
let key0 = Pubkey::new_unique();
|
||||
let key1 = Pubkey::new_unique();
|
||||
let key2 = Pubkey::new_unique();
|
||||
let key3 = Pubkey::new_unique();
|
||||
let key4 = Pubkey::new_unique();
|
||||
let key5 = Pubkey::new_unique();
|
||||
|
||||
let legacy_message = SanitizedMessage::try_from(Message {
|
||||
header: MessageHeader {
|
||||
num_required_signatures: 2,
|
||||
num_readonly_signed_accounts: 1,
|
||||
num_readonly_unsigned_accounts: 1,
|
||||
},
|
||||
account_keys: vec![key0, key1, key2, key3],
|
||||
..Message::default()
|
||||
})
|
||||
.unwrap();
|
||||
|
||||
assert_eq!(legacy_message.num_readonly_accounts(), 2);
|
||||
|
||||
let mapped_message = SanitizedMessage::V0(MappedMessage {
|
||||
message: v0::Message {
|
||||
header: MessageHeader {
|
||||
num_required_signatures: 2,
|
||||
num_readonly_signed_accounts: 1,
|
||||
num_readonly_unsigned_accounts: 1,
|
||||
},
|
||||
account_keys: vec![key0, key1, key2, key3],
|
||||
..v0::Message::default()
|
||||
},
|
||||
mapped_addresses: MappedAddresses {
|
||||
writable: vec![key4],
|
||||
readonly: vec![key5],
|
||||
},
|
||||
});
|
||||
|
||||
assert_eq!(mapped_message.num_readonly_accounts(), 3);
|
||||
}
|
||||
|
||||
#[test]
|
||||
#[allow(deprecated)]
|
||||
fn test_serialize_instructions() {
|
||||
let program_id0 = Pubkey::new_unique();
|
||||
let program_id1 = Pubkey::new_unique();
|
||||
let id0 = Pubkey::new_unique();
|
||||
let id1 = Pubkey::new_unique();
|
||||
let id2 = Pubkey::new_unique();
|
||||
let id3 = Pubkey::new_unique();
|
||||
let instructions = vec![
|
||||
Instruction::new_with_bincode(program_id0, &0, vec![AccountMeta::new(id0, false)]),
|
||||
Instruction::new_with_bincode(program_id0, &0, vec![AccountMeta::new(id1, true)]),
|
||||
Instruction::new_with_bincode(
|
||||
program_id1,
|
||||
&0,
|
||||
vec![AccountMeta::new_readonly(id2, false)],
|
||||
),
|
||||
Instruction::new_with_bincode(
|
||||
program_id1,
|
||||
&0,
|
||||
vec![AccountMeta::new_readonly(id3, true)],
|
||||
),
|
||||
];
|
||||
|
||||
let message = Message::new(&instructions, Some(&id1));
|
||||
let sanitized_message = SanitizedMessage::try_from(message.clone()).unwrap();
|
||||
let serialized = sanitized_message.serialize_instructions();
|
||||
|
||||
// assert that SanitizedMessage::serialize_instructions has the same behavior as the
|
||||
// deprecated Message::serialize_instructions method
|
||||
assert_eq!(serialized, message.serialize_instructions());
|
||||
|
||||
// assert that Message::deserialize_instruction is compatible with SanitizedMessage::serialize_instructions
|
||||
for (i, instruction) in instructions.iter().enumerate() {
|
||||
assert_eq!(
|
||||
Message::deserialize_instruction(i, &serialized).unwrap(),
|
||||
*instruction
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_calculate_fee() {
|
||||
// Default: no fee.
|
||||
let message =
|
||||
SanitizedMessage::try_from(Message::new(&[], Some(&Pubkey::new_unique()))).unwrap();
|
||||
assert_eq!(message.calculate_fee(&FeeCalculator::default()), 0);
|
||||
|
||||
// One signature, a fee.
|
||||
assert_eq!(message.calculate_fee(&FeeCalculator::new(1)), 1);
|
||||
|
||||
// Two signatures, double the fee.
|
||||
let key0 = Pubkey::new_unique();
|
||||
let key1 = Pubkey::new_unique();
|
||||
let ix0 = system_instruction::transfer(&key0, &key1, 1);
|
||||
let ix1 = system_instruction::transfer(&key1, &key0, 1);
|
||||
let message = SanitizedMessage::try_from(Message::new(&[ix0, ix1], Some(&key0))).unwrap();
|
||||
assert_eq!(message.calculate_fee(&FeeCalculator::new(2)), 4);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_try_compile_instruction() {
|
||||
let key0 = Pubkey::new_unique();
|
||||
let key1 = Pubkey::new_unique();
|
||||
let key2 = Pubkey::new_unique();
|
||||
let program_id = Pubkey::new_unique();
|
||||
|
||||
let valid_instruction = Instruction {
|
||||
program_id,
|
||||
accounts: vec![
|
||||
AccountMeta::new_readonly(key0, false),
|
||||
AccountMeta::new_readonly(key1, false),
|
||||
AccountMeta::new_readonly(key2, false),
|
||||
],
|
||||
data: vec![],
|
||||
};
|
||||
|
||||
let invalid_program_id_instruction = Instruction {
|
||||
program_id: Pubkey::new_unique(),
|
||||
accounts: vec![
|
||||
AccountMeta::new_readonly(key0, false),
|
||||
AccountMeta::new_readonly(key1, false),
|
||||
AccountMeta::new_readonly(key2, false),
|
||||
],
|
||||
data: vec![],
|
||||
};
|
||||
|
||||
let invalid_account_key_instruction = Instruction {
|
||||
program_id: Pubkey::new_unique(),
|
||||
accounts: vec![
|
||||
AccountMeta::new_readonly(key0, false),
|
||||
AccountMeta::new_readonly(key1, false),
|
||||
AccountMeta::new_readonly(Pubkey::new_unique(), false),
|
||||
],
|
||||
data: vec![],
|
||||
};
|
||||
|
||||
let legacy_message = SanitizedMessage::try_from(Message {
|
||||
header: MessageHeader {
|
||||
num_required_signatures: 1,
|
||||
num_readonly_signed_accounts: 0,
|
||||
num_readonly_unsigned_accounts: 0,
|
||||
},
|
||||
account_keys: vec![key0, key1, key2, program_id],
|
||||
..Message::default()
|
||||
})
|
||||
.unwrap();
|
||||
|
||||
let mapped_message = SanitizedMessage::V0(MappedMessage {
|
||||
message: v0::Message {
|
||||
header: MessageHeader {
|
||||
num_required_signatures: 1,
|
||||
num_readonly_signed_accounts: 0,
|
||||
num_readonly_unsigned_accounts: 0,
|
||||
},
|
||||
account_keys: vec![key0, key1],
|
||||
..v0::Message::default()
|
||||
},
|
||||
mapped_addresses: MappedAddresses {
|
||||
writable: vec![key2],
|
||||
readonly: vec![program_id],
|
||||
},
|
||||
});
|
||||
|
||||
for message in vec![legacy_message, mapped_message] {
|
||||
assert_eq!(
|
||||
message.try_compile_instruction(&valid_instruction),
|
||||
Some(CompiledInstruction {
|
||||
program_id_index: 3,
|
||||
accounts: vec![0, 1, 2],
|
||||
data: vec![],
|
||||
})
|
||||
);
|
||||
|
||||
assert!(message
|
||||
.try_compile_instruction(&invalid_program_id_instruction)
|
||||
.is_none());
|
||||
assert!(message
|
||||
.try_compile_instruction(&invalid_account_key_instruction)
|
||||
.is_none());
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_calculate_fee_secp256k1() {
|
||||
let key0 = Pubkey::new_unique();
|
||||
let key1 = Pubkey::new_unique();
|
||||
let ix0 = system_instruction::transfer(&key0, &key1, 1);
|
||||
|
||||
let mut secp_instruction1 = Instruction {
|
||||
program_id: secp256k1_program::id(),
|
||||
accounts: vec![],
|
||||
data: vec![],
|
||||
};
|
||||
let mut secp_instruction2 = Instruction {
|
||||
program_id: secp256k1_program::id(),
|
||||
accounts: vec![],
|
||||
data: vec![1],
|
||||
};
|
||||
|
||||
let message = SanitizedMessage::try_from(Message::new(
|
||||
&[
|
||||
ix0.clone(),
|
||||
secp_instruction1.clone(),
|
||||
secp_instruction2.clone(),
|
||||
],
|
||||
Some(&key0),
|
||||
))
|
||||
.unwrap();
|
||||
assert_eq!(message.calculate_fee(&FeeCalculator::new(1)), 2);
|
||||
|
||||
secp_instruction1.data = vec![0];
|
||||
secp_instruction2.data = vec![10];
|
||||
let message = SanitizedMessage::try_from(Message::new(
|
||||
&[ix0, secp_instruction1, secp_instruction2],
|
||||
Some(&key0),
|
||||
))
|
||||
.unwrap();
|
||||
assert_eq!(message.calculate_fee(&FeeCalculator::new(1)), 11);
|
||||
}
|
||||
}
|
@ -1,9 +1,7 @@
|
||||
#![allow(clippy::integer_arithmetic)]
|
||||
|
||||
use crate::{
|
||||
hash::Hash,
|
||||
instruction::CompiledInstruction,
|
||||
message::MessageHeader,
|
||||
message::{MessageHeader, MESSAGE_VERSION_PREFIX},
|
||||
pubkey::Pubkey,
|
||||
sanitize::{Sanitize, SanitizeError},
|
||||
short_vec,
|
||||
@ -62,8 +60,8 @@ impl Sanitize for Message {
|
||||
fn sanitize(&self) -> Result<(), SanitizeError> {
|
||||
// signing area and read-only non-signing area should not
|
||||
// overlap
|
||||
if self.header.num_required_signatures as usize
|
||||
+ self.header.num_readonly_unsigned_accounts as usize
|
||||
if usize::from(self.header.num_required_signatures)
|
||||
.saturating_add(usize::from(self.header.num_readonly_unsigned_accounts))
|
||||
> self.account_keys.len()
|
||||
{
|
||||
return Err(SanitizeError::IndexOutOfBounds);
|
||||
@ -76,7 +74,7 @@ impl Sanitize for Message {
|
||||
|
||||
// there cannot be more address maps than read-only unsigned accounts.
|
||||
let num_address_map_indexes = self.address_map_indexes.len();
|
||||
if num_address_map_indexes > self.header.num_readonly_unsigned_accounts as usize {
|
||||
if num_address_map_indexes > usize::from(self.header.num_readonly_unsigned_accounts) {
|
||||
return Err(SanitizeError::IndexOutOfBounds);
|
||||
}
|
||||
|
||||
@ -102,7 +100,7 @@ impl Sanitize for Message {
|
||||
}
|
||||
|
||||
for ci in &self.instructions {
|
||||
if ci.program_id_index as usize >= num_loaded_accounts {
|
||||
if usize::from(ci.program_id_index) >= num_loaded_accounts {
|
||||
return Err(SanitizeError::IndexOutOfBounds);
|
||||
}
|
||||
// A program cannot be a payer.
|
||||
@ -110,7 +108,7 @@ impl Sanitize for Message {
|
||||
return Err(SanitizeError::IndexOutOfBounds);
|
||||
}
|
||||
for ai in &ci.accounts {
|
||||
if *ai as usize >= num_loaded_accounts {
|
||||
if usize::from(*ai) >= num_loaded_accounts {
|
||||
return Err(SanitizeError::IndexOutOfBounds);
|
||||
}
|
||||
}
|
||||
@ -120,9 +118,17 @@ impl Sanitize for Message {
|
||||
}
|
||||
}
|
||||
|
||||
impl Message {
|
||||
/// Serialize this message with a version #0 prefix using bincode encoding.
|
||||
pub fn serialize(&self) -> Vec<u8> {
|
||||
bincode::serialize(&(MESSAGE_VERSION_PREFIX, self)).unwrap()
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
use crate::message::VersionedMessage;
|
||||
|
||||
fn simple_message() -> Message {
|
||||
Message {
|
||||
@ -381,4 +387,11 @@ mod tests {
|
||||
.sanitize()
|
||||
.is_err());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_serialize() {
|
||||
let message = simple_message();
|
||||
let versioned_msg = VersionedMessage::V0(message.clone());
|
||||
assert_eq!(message.serialize(), versioned_msg.serialize());
|
||||
}
|
||||
}
|
||||
|
@ -26,13 +26,77 @@ pub const MESSAGE_VERSION_PREFIX: u8 = 0x80;
|
||||
/// which message version is serialized starting from version `0`. If the first
|
||||
/// is bit is not set, all bytes are used to encode the legacy `Message`
|
||||
/// format.
|
||||
#[frozen_abi(digest = "C4MZ7qztFJHUp1bVcuh7Gn43PQExadzEGyEb8UMn9unz")]
|
||||
#[frozen_abi(digest = "qKNCqQpsBZYQxS9P3hVcFr8hAF4VnqV6ZBdC6KoUvHJ")]
|
||||
#[derive(Debug, PartialEq, Eq, Clone, AbiEnumVisitor, AbiExample)]
|
||||
pub enum VersionedMessage {
|
||||
Legacy(Message),
|
||||
V0(v0::Message),
|
||||
}
|
||||
|
||||
impl VersionedMessage {
|
||||
pub fn header(&self) -> &MessageHeader {
|
||||
match self {
|
||||
Self::Legacy(message) => &message.header,
|
||||
Self::V0(message) => &message.header,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn unmapped_keys(self) -> Vec<Pubkey> {
|
||||
match self {
|
||||
Self::Legacy(message) => message.account_keys,
|
||||
Self::V0(message) => message.account_keys,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn unmapped_keys_iter(&self) -> impl Iterator<Item = &Pubkey> {
|
||||
match self {
|
||||
Self::Legacy(message) => message.account_keys.iter(),
|
||||
Self::V0(message) => message.account_keys.iter(),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn unmapped_keys_len(&self) -> usize {
|
||||
match self {
|
||||
Self::Legacy(message) => message.account_keys.len(),
|
||||
Self::V0(message) => message.account_keys.len(),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn recent_blockhash(&self) -> &Hash {
|
||||
match self {
|
||||
Self::Legacy(message) => &message.recent_blockhash,
|
||||
Self::V0(message) => &message.recent_blockhash,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn set_recent_blockhash(&mut self, recent_blockhash: Hash) {
|
||||
match self {
|
||||
Self::Legacy(message) => message.recent_blockhash = recent_blockhash,
|
||||
Self::V0(message) => message.recent_blockhash = recent_blockhash,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn serialize(&self) -> Vec<u8> {
|
||||
bincode::serialize(self).unwrap()
|
||||
}
|
||||
|
||||
/// Compute the blake3 hash of this transaction's message
|
||||
pub fn hash(&self) -> Hash {
|
||||
let message_bytes = self.serialize();
|
||||
Self::hash_raw_message(&message_bytes)
|
||||
}
|
||||
|
||||
/// Compute the blake3 hash of a raw transaction message
|
||||
pub fn hash_raw_message(message_bytes: &[u8]) -> Hash {
|
||||
use blake3::traits::digest::Digest;
|
||||
use std::convert::TryFrom;
|
||||
let mut hasher = blake3::Hasher::new();
|
||||
hasher.update(b"solana-tx-message-v1");
|
||||
hasher.update(message_bytes);
|
||||
Hash(<[u8; crate::hash::HASH_BYTES]>::try_from(hasher.finalize().as_slice()).unwrap())
|
||||
}
|
||||
}
|
||||
|
||||
impl Default for VersionedMessage {
|
||||
fn default() -> Self {
|
||||
Self::Legacy(Message::default())
|
||||
|
@ -1,10 +1,12 @@
|
||||
#![cfg(feature = "full")]
|
||||
|
||||
use crate::transaction::{Transaction, TransactionError};
|
||||
use borsh::{BorshDeserialize, BorshSchema, BorshSerialize};
|
||||
use solana_sdk::{
|
||||
borsh::try_from_slice_unchecked,
|
||||
instruction::{Instruction, InstructionError},
|
||||
use {
|
||||
crate::{
|
||||
borsh::try_from_slice_unchecked,
|
||||
instruction::{Instruction, InstructionError},
|
||||
transaction::{SanitizedTransaction, TransactionError},
|
||||
},
|
||||
borsh::{BorshDeserialize, BorshSchema, BorshSerialize},
|
||||
};
|
||||
|
||||
crate::declare_id!("ComputeBudget111111111111111111111111111111");
|
||||
@ -96,11 +98,14 @@ impl ComputeBudget {
|
||||
heap_size: None,
|
||||
}
|
||||
}
|
||||
pub fn process_transaction(&mut self, tx: &Transaction) -> Result<(), TransactionError> {
|
||||
pub fn process_transaction(
|
||||
&mut self,
|
||||
tx: &SanitizedTransaction,
|
||||
) -> Result<(), TransactionError> {
|
||||
let error = TransactionError::InstructionError(0, InstructionError::InvalidInstructionData);
|
||||
// Compute budget instruction must be in 1st or 2nd instruction (avoid nonce marker)
|
||||
for instruction in tx.message().instructions.iter().take(2) {
|
||||
if check_id(instruction.program_id(&tx.message().account_keys)) {
|
||||
for (program_id, instruction) in tx.message().program_instructions_iter().take(2) {
|
||||
if check_id(program_id) {
|
||||
let ComputeBudgetInstruction::RequestUnits(units) =
|
||||
try_from_slice_unchecked::<ComputeBudgetInstruction>(&instruction.data)
|
||||
.map_err(|_| error.clone())?;
|
||||
@ -117,22 +122,30 @@ impl ComputeBudget {
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
use crate::{hash::Hash, message::Message, pubkey::Pubkey, signature::Keypair, signer::Signer};
|
||||
use crate::{
|
||||
hash::Hash, message::Message, pubkey::Pubkey, signature::Keypair, signer::Signer,
|
||||
transaction::Transaction,
|
||||
};
|
||||
use std::convert::TryInto;
|
||||
|
||||
fn sanitize_tx(tx: Transaction) -> SanitizedTransaction {
|
||||
tx.try_into().unwrap()
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_process_transaction() {
|
||||
let payer_keypair = Keypair::new();
|
||||
let mut compute_budget = ComputeBudget::default();
|
||||
|
||||
let tx = Transaction::new(
|
||||
let tx = sanitize_tx(Transaction::new(
|
||||
&[&payer_keypair],
|
||||
Message::new(&[], Some(&payer_keypair.pubkey())),
|
||||
Hash::default(),
|
||||
);
|
||||
));
|
||||
compute_budget.process_transaction(&tx).unwrap();
|
||||
assert_eq!(compute_budget, ComputeBudget::default());
|
||||
|
||||
let tx = Transaction::new(
|
||||
let tx = sanitize_tx(Transaction::new(
|
||||
&[&payer_keypair],
|
||||
Message::new(
|
||||
&[
|
||||
@ -142,7 +155,7 @@ mod tests {
|
||||
Some(&payer_keypair.pubkey()),
|
||||
),
|
||||
Hash::default(),
|
||||
);
|
||||
));
|
||||
compute_budget.process_transaction(&tx).unwrap();
|
||||
assert_eq!(
|
||||
compute_budget,
|
||||
@ -152,7 +165,7 @@ mod tests {
|
||||
}
|
||||
);
|
||||
|
||||
let tx = Transaction::new(
|
||||
let tx = sanitize_tx(Transaction::new(
|
||||
&[&payer_keypair],
|
||||
Message::new(
|
||||
&[
|
||||
@ -162,7 +175,7 @@ mod tests {
|
||||
Some(&payer_keypair.pubkey()),
|
||||
),
|
||||
Hash::default(),
|
||||
);
|
||||
));
|
||||
let result = compute_budget.process_transaction(&tx);
|
||||
assert_eq!(
|
||||
result,
|
||||
@ -172,7 +185,7 @@ mod tests {
|
||||
))
|
||||
);
|
||||
|
||||
let tx = Transaction::new(
|
||||
let tx = sanitize_tx(Transaction::new(
|
||||
&[&payer_keypair],
|
||||
Message::new(
|
||||
&[
|
||||
@ -182,7 +195,7 @@ mod tests {
|
||||
Some(&payer_keypair.pubkey()),
|
||||
),
|
||||
Hash::default(),
|
||||
);
|
||||
));
|
||||
compute_budget.process_transaction(&tx).unwrap();
|
||||
assert_eq!(
|
||||
compute_budget,
|
||||
|
@ -195,6 +195,10 @@ pub mod mem_overlap_fix {
|
||||
solana_sdk::declare_id!("vXDCFK7gphrEmyf5VnKgLmqbdJ4UxD2eZH1qbdouYKF");
|
||||
}
|
||||
|
||||
pub mod versioned_tx_message_enabled {
|
||||
solana_sdk::declare_id!("3KZZ6Ks1885aGBQ45fwRcPXVBCtzUvxhUTkwKMR41Tca");
|
||||
}
|
||||
|
||||
lazy_static! {
|
||||
/// Map of feature identifiers to user-visible description
|
||||
pub static ref FEATURE_NAMES: HashMap<Pubkey, &'static str> = [
|
||||
@ -238,6 +242,7 @@ lazy_static! {
|
||||
(stake_merge_with_unmatched_credits_observed::id(), "allow merging active stakes with unmatched credits_observed #18985"),
|
||||
(gate_large_block::id(), "validator checks block cost against max limit in realtime, reject if exceeds."),
|
||||
(mem_overlap_fix::id(), "Memory overlap fix"),
|
||||
(versioned_tx_message_enabled::id(), "enable versioned transaction message processing"),
|
||||
/*************** ADD NEW FEATURES HERE ***************/
|
||||
]
|
||||
.iter()
|
||||
|
@ -40,7 +40,6 @@ pub mod program_utils;
|
||||
pub mod pubkey;
|
||||
pub mod recent_blockhashes_account;
|
||||
pub mod rpc_port;
|
||||
pub mod sanitized_transaction;
|
||||
pub mod secp256k1_instruction;
|
||||
pub mod shred_version;
|
||||
pub mod signature;
|
||||
|
@ -1,87 +0,0 @@
|
||||
#![cfg(feature = "full")]
|
||||
|
||||
use crate::{
|
||||
hash::Hash,
|
||||
sanitize::Sanitize,
|
||||
transaction::{Result, Transaction, TransactionError},
|
||||
};
|
||||
use std::{borrow::Cow, convert::TryFrom, ops::Deref};
|
||||
|
||||
/// Sanitized transaction and the hash of its message
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct SanitizedTransaction<'a> {
|
||||
transaction: Cow<'a, Transaction>,
|
||||
pub message_hash: Hash,
|
||||
}
|
||||
|
||||
impl<'a> SanitizedTransaction<'a> {
|
||||
pub fn try_create(transaction: Cow<'a, Transaction>, message_hash: Hash) -> Result<Self> {
|
||||
transaction.sanitize()?;
|
||||
if Self::has_duplicates(&transaction.message.account_keys) {
|
||||
return Err(TransactionError::AccountLoadedTwice);
|
||||
}
|
||||
|
||||
Ok(Self {
|
||||
transaction,
|
||||
message_hash,
|
||||
})
|
||||
}
|
||||
|
||||
/// 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() {
|
||||
#[allow(clippy::integer_arithmetic)]
|
||||
if xs[i..].contains(&xs[i - 1]) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
false
|
||||
}
|
||||
}
|
||||
|
||||
impl Deref for SanitizedTransaction<'_> {
|
||||
type Target = Transaction;
|
||||
fn deref(&self) -> &Self::Target {
|
||||
&self.transaction
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a> TryFrom<Transaction> for SanitizedTransaction<'_> {
|
||||
type Error = TransactionError;
|
||||
fn try_from(transaction: Transaction) -> Result<Self> {
|
||||
let message_hash = transaction.message().hash();
|
||||
Self::try_create(Cow::Owned(transaction), message_hash)
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a> TryFrom<&'a Transaction> for SanitizedTransaction<'a> {
|
||||
type Error = TransactionError;
|
||||
fn try_from(transaction: &'a Transaction) -> Result<Self> {
|
||||
let message_hash = transaction.message().hash();
|
||||
Self::try_create(Cow::Borrowed(transaction), message_hash)
|
||||
}
|
||||
}
|
||||
|
||||
pub trait SanitizedTransactionSlice<'a> {
|
||||
fn as_transactions_iter(&'a self) -> Box<dyn Iterator<Item = &'a Transaction> + '_>;
|
||||
}
|
||||
|
||||
impl<'a> SanitizedTransactionSlice<'a> for [SanitizedTransaction<'a>] {
|
||||
fn as_transactions_iter(&'a self) -> Box<dyn Iterator<Item = &'a Transaction> + '_> {
|
||||
Box::new(self.iter().map(Deref::deref))
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
|
||||
#[test]
|
||||
fn test_has_duplicates() {
|
||||
assert!(!SanitizedTransaction::has_duplicates(&[1, 2]));
|
||||
assert!(SanitizedTransaction::has_duplicates(&[1, 2, 1]));
|
||||
}
|
||||
}
|
@ -2,23 +2,31 @@
|
||||
|
||||
#![cfg(feature = "full")]
|
||||
|
||||
use crate::sanitize::{Sanitize, SanitizeError};
|
||||
use crate::secp256k1_instruction::verify_eth_addresses;
|
||||
use crate::{
|
||||
hash::Hash,
|
||||
instruction::{CompiledInstruction, Instruction, InstructionError},
|
||||
message::Message,
|
||||
nonce::NONCED_TX_MARKER_IX_INDEX,
|
||||
program_utils::limited_deserialize,
|
||||
pubkey::Pubkey,
|
||||
short_vec,
|
||||
signature::{Signature, SignerError},
|
||||
signers::Signers,
|
||||
system_instruction::SystemInstruction,
|
||||
system_program,
|
||||
use {
|
||||
crate::{
|
||||
hash::Hash,
|
||||
instruction::{CompiledInstruction, Instruction, InstructionError},
|
||||
message::{Message, SanitizeMessageError},
|
||||
nonce::NONCED_TX_MARKER_IX_INDEX,
|
||||
program_utils::limited_deserialize,
|
||||
pubkey::Pubkey,
|
||||
sanitize::{Sanitize, SanitizeError},
|
||||
secp256k1_instruction::verify_eth_addresses,
|
||||
short_vec,
|
||||
signature::{Signature, SignerError},
|
||||
signers::Signers,
|
||||
},
|
||||
serde::Serialize,
|
||||
solana_program::{system_instruction::SystemInstruction, system_program},
|
||||
std::result,
|
||||
thiserror::Error,
|
||||
};
|
||||
use std::result;
|
||||
use thiserror::Error;
|
||||
|
||||
mod sanitized;
|
||||
mod versioned;
|
||||
|
||||
pub use sanitized::*;
|
||||
pub use versioned::*;
|
||||
|
||||
/// Reasons a transaction might be rejected.
|
||||
#[derive(
|
||||
@ -104,6 +112,10 @@ pub enum TransactionError {
|
||||
"Transaction could not fit into current block without exceeding the Max Block Cost Limit"
|
||||
)]
|
||||
WouldExceedMaxBlockCostLimit,
|
||||
|
||||
/// Transaction version is unsupported
|
||||
#[error("Transaction version is unsupported")]
|
||||
UnsupportedVersion,
|
||||
}
|
||||
|
||||
pub type Result<T> = result::Result<T, TransactionError>;
|
||||
@ -114,6 +126,17 @@ impl From<SanitizeError> for TransactionError {
|
||||
}
|
||||
}
|
||||
|
||||
impl From<SanitizeMessageError> for TransactionError {
|
||||
fn from(err: SanitizeMessageError) -> Self {
|
||||
match err {
|
||||
SanitizeMessageError::IndexOutOfBounds
|
||||
| SanitizeMessageError::ValueOutOfBounds
|
||||
| SanitizeMessageError::InvalidValue => Self::SanitizeFailure,
|
||||
SanitizeMessageError::DuplicateAccountKey => Self::AccountLoadedTwice,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// An atomic transaction
|
||||
#[frozen_abi(digest = "FZtncnS1Xk8ghHfKiXE5oGiUbw2wJhmfXQuNgQR3K6Mc")]
|
||||
#[derive(Debug, PartialEq, Default, Eq, Clone, Serialize, Deserialize, AbiExample)]
|
||||
@ -230,10 +253,12 @@ impl Transaction {
|
||||
.and_then(|instruction| instruction.accounts.get(accounts_index))
|
||||
.map(|&account_keys_index| account_keys_index as usize)
|
||||
}
|
||||
|
||||
pub fn key(&self, instruction_index: usize, accounts_index: usize) -> Option<&Pubkey> {
|
||||
self.key_index(instruction_index, accounts_index)
|
||||
.and_then(|account_keys_index| self.message.account_keys.get(account_keys_index))
|
||||
}
|
||||
|
||||
pub fn signer_key(&self, instruction_index: usize, accounts_index: usize) -> Option<&Pubkey> {
|
||||
match self.key_index(instruction_index, accounts_index) {
|
||||
None => None,
|
||||
@ -484,6 +509,7 @@ pub fn uses_durable_nonce(tx: &Transaction) -> Option<&CompiledInstruction> {
|
||||
)
|
||||
}
|
||||
|
||||
#[deprecated]
|
||||
pub fn get_nonce_pubkey_from_instruction<'a>(
|
||||
ix: &CompiledInstruction,
|
||||
tx: &'a Transaction,
|
||||
@ -496,6 +522,8 @@ pub fn get_nonce_pubkey_from_instruction<'a>(
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
#![allow(deprecated)]
|
||||
|
||||
use super::*;
|
||||
use crate::{
|
||||
hash::hash,
|
||||
@ -553,6 +581,7 @@ mod tests {
|
||||
assert_eq!(*get_program_id(&tx, 0), prog1);
|
||||
assert_eq!(*get_program_id(&tx, 1), prog2);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_refs_invalid_program_id() {
|
||||
let key = Keypair::new();
|
226
sdk/src/transaction/sanitized.rs
Normal file
226
sdk/src/transaction/sanitized.rs
Normal file
@ -0,0 +1,226 @@
|
||||
#![cfg(feature = "full")]
|
||||
|
||||
use {
|
||||
crate::{
|
||||
hash::Hash,
|
||||
message::{v0, MappedAddresses, MappedMessage, SanitizedMessage, VersionedMessage},
|
||||
nonce::NONCED_TX_MARKER_IX_INDEX,
|
||||
program_utils::limited_deserialize,
|
||||
pubkey::Pubkey,
|
||||
sanitize::Sanitize,
|
||||
secp256k1_instruction::verify_eth_addresses,
|
||||
secp256k1_program,
|
||||
signature::Signature,
|
||||
transaction::{Result, Transaction, TransactionError, VersionedTransaction},
|
||||
},
|
||||
solana_program::{system_instruction::SystemInstruction, system_program},
|
||||
std::convert::TryFrom,
|
||||
};
|
||||
|
||||
/// Sanitized transaction and the hash of its message
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct SanitizedTransaction {
|
||||
message: SanitizedMessage,
|
||||
message_hash: Hash,
|
||||
signatures: Vec<Signature>,
|
||||
}
|
||||
|
||||
/// Set of accounts that must be locked for safe transaction processing
|
||||
#[derive(Debug, Clone, Default)]
|
||||
pub struct TransactionAccountLocks<'a> {
|
||||
/// List of readonly account key locks
|
||||
pub readonly: Vec<&'a Pubkey>,
|
||||
/// List of writable account key locks
|
||||
pub writable: Vec<&'a Pubkey>,
|
||||
}
|
||||
|
||||
impl TryFrom<Transaction> for SanitizedTransaction {
|
||||
type Error = TransactionError;
|
||||
fn try_from(tx: Transaction) -> Result<Self> {
|
||||
tx.sanitize()?;
|
||||
|
||||
if tx.message.has_duplicates() {
|
||||
return Err(TransactionError::AccountLoadedTwice);
|
||||
}
|
||||
|
||||
Ok(Self {
|
||||
message_hash: tx.message.hash(),
|
||||
message: SanitizedMessage::Legacy(tx.message),
|
||||
signatures: tx.signatures,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
impl SanitizedTransaction {
|
||||
/// Create a sanitized transaction from an unsanitized transaction.
|
||||
/// If the input transaction uses address maps, attempt to map the
|
||||
/// transaction keys to full addresses.
|
||||
pub fn try_create(
|
||||
tx: VersionedTransaction,
|
||||
message_hash: Hash,
|
||||
address_mapper: impl Fn(&v0::Message) -> Result<MappedAddresses>,
|
||||
) -> Result<Self> {
|
||||
tx.sanitize()?;
|
||||
|
||||
let signatures = tx.signatures;
|
||||
let message = match tx.message {
|
||||
VersionedMessage::Legacy(message) => SanitizedMessage::Legacy(message),
|
||||
VersionedMessage::V0(message) => SanitizedMessage::V0(MappedMessage {
|
||||
mapped_addresses: address_mapper(&message)?,
|
||||
message,
|
||||
}),
|
||||
};
|
||||
|
||||
if message.has_duplicates() {
|
||||
return Err(TransactionError::AccountLoadedTwice);
|
||||
}
|
||||
|
||||
Ok(Self {
|
||||
message,
|
||||
message_hash,
|
||||
signatures,
|
||||
})
|
||||
}
|
||||
|
||||
/// Return the first signature for this transaction.
|
||||
///
|
||||
/// Notes:
|
||||
///
|
||||
/// Sanitized transactions must have at least one signature because the
|
||||
/// number of signatures must be greater than or equal to the message header
|
||||
/// value `num_required_signatures` which must be greater than 0 itself.
|
||||
pub fn signature(&self) -> &Signature {
|
||||
&self.signatures[0]
|
||||
}
|
||||
|
||||
/// Return the list of signatures for this transaction
|
||||
pub fn signatures(&self) -> &[Signature] {
|
||||
&self.signatures
|
||||
}
|
||||
|
||||
/// Return the signed message
|
||||
pub fn message(&self) -> &SanitizedMessage {
|
||||
&self.message
|
||||
}
|
||||
|
||||
/// Return the hash of the signed message
|
||||
pub fn message_hash(&self) -> &Hash {
|
||||
&self.message_hash
|
||||
}
|
||||
|
||||
/// Convert this sanitized transaction into a versioned transaction for
|
||||
/// recording in the ledger.
|
||||
pub fn to_versioned_transaction(&self) -> VersionedTransaction {
|
||||
let signatures = self.signatures.clone();
|
||||
match &self.message {
|
||||
SanitizedMessage::V0(mapped_msg) => VersionedTransaction {
|
||||
signatures,
|
||||
message: VersionedMessage::V0(mapped_msg.message.clone()),
|
||||
},
|
||||
SanitizedMessage::Legacy(message) => VersionedTransaction {
|
||||
signatures,
|
||||
message: VersionedMessage::Legacy(message.clone()),
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
/// Return the list of accounts that must be locked during processing this transaction.
|
||||
pub fn get_account_locks(&self) -> TransactionAccountLocks {
|
||||
let message = &self.message;
|
||||
let num_readonly_accounts = message.num_readonly_accounts();
|
||||
let num_writable_accounts = message
|
||||
.account_keys_len()
|
||||
.saturating_sub(num_readonly_accounts);
|
||||
|
||||
let mut account_locks = TransactionAccountLocks {
|
||||
writable: Vec::with_capacity(num_writable_accounts),
|
||||
readonly: Vec::with_capacity(num_readonly_accounts),
|
||||
};
|
||||
|
||||
for (i, key) in message.account_keys_iter().enumerate() {
|
||||
if message.is_writable(i) {
|
||||
account_locks.writable.push(key);
|
||||
} else {
|
||||
account_locks.readonly.push(key);
|
||||
}
|
||||
}
|
||||
|
||||
account_locks
|
||||
}
|
||||
|
||||
/// If the transaction uses a durable nonce, return the pubkey of the nonce account
|
||||
pub fn get_durable_nonce(&self) -> Option<&Pubkey> {
|
||||
self.message
|
||||
.instructions()
|
||||
.get(NONCED_TX_MARKER_IX_INDEX as usize)
|
||||
.filter(
|
||||
|ix| match self.message.get_account_key(ix.program_id_index as usize) {
|
||||
Some(program_id) => system_program::check_id(program_id),
|
||||
_ => false,
|
||||
},
|
||||
)
|
||||
.filter(|ix| {
|
||||
matches!(
|
||||
limited_deserialize(&ix.data),
|
||||
Ok(SystemInstruction::AdvanceNonceAccount)
|
||||
)
|
||||
})
|
||||
.and_then(|ix| {
|
||||
ix.accounts.get(0).and_then(|idx| {
|
||||
let idx = *idx as usize;
|
||||
self.message.get_account_key(idx)
|
||||
})
|
||||
})
|
||||
}
|
||||
|
||||
/// Return the serialized message data to sign.
|
||||
fn message_data(&self) -> Vec<u8> {
|
||||
match &self.message {
|
||||
SanitizedMessage::Legacy(message) => message.serialize(),
|
||||
SanitizedMessage::V0(mapped_msg) => mapped_msg.message.serialize(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Verify the length of signatures matches the value in the message header
|
||||
pub fn verify_signatures_len(&self) -> bool {
|
||||
self.signatures.len() == self.message.header().num_required_signatures as usize
|
||||
}
|
||||
|
||||
/// Verify the transaction signatures
|
||||
pub fn verify(&self) -> Result<()> {
|
||||
let message_bytes = self.message_data();
|
||||
if self
|
||||
.signatures
|
||||
.iter()
|
||||
.zip(self.message.account_keys_iter())
|
||||
.map(|(signature, pubkey)| signature.verify(pubkey.as_ref(), &message_bytes))
|
||||
.any(|verified| !verified)
|
||||
{
|
||||
Err(TransactionError::SignatureFailure)
|
||||
} else {
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
/// Verify the encoded secp256k1 signatures in this transaction
|
||||
pub fn verify_precompiles(&self, libsecp256k1_0_5_upgrade_enabled: bool) -> Result<()> {
|
||||
for (program_id, instruction) in self.message.program_instructions_iter() {
|
||||
if secp256k1_program::check_id(program_id) {
|
||||
let instruction_datas: Vec<_> = self
|
||||
.message
|
||||
.instructions()
|
||||
.iter()
|
||||
.map(|instruction| instruction.data.as_ref())
|
||||
.collect();
|
||||
let data = &instruction.data;
|
||||
let e = verify_eth_addresses(
|
||||
data,
|
||||
&instruction_datas,
|
||||
libsecp256k1_0_5_upgrade_enabled,
|
||||
);
|
||||
e.map_err(|_| TransactionError::InvalidAccountIndex)?;
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
}
|
84
sdk/src/transaction/versioned.rs
Normal file
84
sdk/src/transaction/versioned.rs
Normal file
@ -0,0 +1,84 @@
|
||||
//! Defines a transaction which supports multiple versions of messages.
|
||||
|
||||
#![cfg(feature = "full")]
|
||||
|
||||
use {
|
||||
crate::{
|
||||
hash::Hash,
|
||||
message::VersionedMessage,
|
||||
sanitize::{Sanitize, SanitizeError},
|
||||
short_vec,
|
||||
signature::Signature,
|
||||
transaction::{Result, Transaction, TransactionError},
|
||||
},
|
||||
serde::Serialize,
|
||||
};
|
||||
|
||||
// NOTE: Serialization-related changes must be paired with the direct read at sigverify.
|
||||
/// An atomic transaction
|
||||
#[derive(Debug, PartialEq, Default, Eq, Clone, Serialize, Deserialize, AbiExample)]
|
||||
pub struct VersionedTransaction {
|
||||
/// List of signatures
|
||||
#[serde(with = "short_vec")]
|
||||
pub signatures: Vec<Signature>,
|
||||
/// Message to sign.
|
||||
pub message: VersionedMessage,
|
||||
}
|
||||
|
||||
impl Sanitize for VersionedTransaction {
|
||||
fn sanitize(&self) -> std::result::Result<(), SanitizeError> {
|
||||
self.message.sanitize()?;
|
||||
|
||||
// Once the "verify_tx_signatures_len" feature is enabled, this may be
|
||||
// updated to an equality check.
|
||||
if usize::from(self.message.header().num_required_signatures) > self.signatures.len() {
|
||||
return Err(SanitizeError::IndexOutOfBounds);
|
||||
}
|
||||
|
||||
// Signatures are verified before message keys are mapped so all signers
|
||||
// must correspond to unmapped keys.
|
||||
if self.signatures.len() > self.message.unmapped_keys_len() {
|
||||
return Err(SanitizeError::IndexOutOfBounds);
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
impl From<Transaction> for VersionedTransaction {
|
||||
fn from(transaction: Transaction) -> Self {
|
||||
Self {
|
||||
signatures: transaction.signatures,
|
||||
message: VersionedMessage::Legacy(transaction.message),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl VersionedTransaction {
|
||||
/// Returns a legacy transaction if the transaction message is legacy.
|
||||
pub fn into_legacy_transaction(self) -> Option<Transaction> {
|
||||
match self.message {
|
||||
VersionedMessage::Legacy(message) => Some(Transaction {
|
||||
signatures: self.signatures,
|
||||
message,
|
||||
}),
|
||||
_ => None,
|
||||
}
|
||||
}
|
||||
|
||||
/// Verify the transaction and hash its message
|
||||
pub fn verify_and_hash_message(&self) -> Result<Hash> {
|
||||
let message_bytes = self.message.serialize();
|
||||
if self
|
||||
.signatures
|
||||
.iter()
|
||||
.zip(self.message.unmapped_keys_iter())
|
||||
.map(|(signature, pubkey)| signature.verify(pubkey.as_ref(), &message_bytes))
|
||||
.any(|verified| !verified)
|
||||
{
|
||||
Err(TransactionError::SignatureFailure)
|
||||
} else {
|
||||
Ok(VersionedMessage::hash_raw_message(&message_bytes))
|
||||
}
|
||||
}
|
||||
}
|
@ -42,6 +42,7 @@ enum TransactionErrorType {
|
||||
CLUSTER_MAINTENANCE = 15;
|
||||
ACCOUNT_BORROW_OUTSTANDING_TX = 16;
|
||||
WOULD_EXCEED_MAX_BLOCK_COST_LIMIT = 17;
|
||||
UNSUPPORTED_VERSION = 18;
|
||||
}
|
||||
|
||||
message InstructionError {
|
||||
|
@ -548,6 +548,8 @@ impl TryFrom<tx_by_addr::TransactionError> for TransactionError {
|
||||
14 => TransactionError::SanitizeFailure,
|
||||
15 => TransactionError::ClusterMaintenance,
|
||||
16 => TransactionError::AccountBorrowOutstanding,
|
||||
17 => TransactionError::WouldExceedMaxBlockCostLimit,
|
||||
18 => TransactionError::UnsupportedVersion,
|
||||
_ => return Err("Invalid TransactionError"),
|
||||
})
|
||||
}
|
||||
@ -609,6 +611,9 @@ impl From<TransactionError> for tx_by_addr::TransactionError {
|
||||
TransactionError::WouldExceedMaxBlockCostLimit => {
|
||||
tx_by_addr::TransactionErrorType::WouldExceedMaxBlockCostLimit
|
||||
}
|
||||
TransactionError::UnsupportedVersion => {
|
||||
tx_by_addr::TransactionErrorType::UnsupportedVersion
|
||||
}
|
||||
} as i32,
|
||||
instruction_error: match transaction_error {
|
||||
TransactionError::InstructionError(index, ref instruction_error) => {
|
||||
@ -884,6 +889,14 @@ mod test {
|
||||
|
||||
#[test]
|
||||
fn test_transaction_error_encode() {
|
||||
let transaction_error = TransactionError::AccountBorrowOutstanding;
|
||||
let tx_by_addr_transaction_error: tx_by_addr::TransactionError =
|
||||
transaction_error.clone().into();
|
||||
assert_eq!(
|
||||
transaction_error,
|
||||
tx_by_addr_transaction_error.try_into().unwrap()
|
||||
);
|
||||
|
||||
let transaction_error = TransactionError::AccountInUse;
|
||||
let tx_by_addr_transaction_error: tx_by_addr::TransactionError =
|
||||
transaction_error.clone().into();
|
||||
@ -908,6 +921,14 @@ mod test {
|
||||
tx_by_addr_transaction_error.try_into().unwrap()
|
||||
);
|
||||
|
||||
let transaction_error = TransactionError::AlreadyProcessed;
|
||||
let tx_by_addr_transaction_error: tx_by_addr::TransactionError =
|
||||
transaction_error.clone().into();
|
||||
assert_eq!(
|
||||
transaction_error,
|
||||
tx_by_addr_transaction_error.try_into().unwrap()
|
||||
);
|
||||
|
||||
let transaction_error = TransactionError::BlockhashNotFound;
|
||||
let tx_by_addr_transaction_error: tx_by_addr::TransactionError =
|
||||
transaction_error.clone().into();
|
||||
@ -932,14 +953,6 @@ mod test {
|
||||
tx_by_addr_transaction_error.try_into().unwrap()
|
||||
);
|
||||
|
||||
let transaction_error = TransactionError::AlreadyProcessed;
|
||||
let tx_by_addr_transaction_error: tx_by_addr::TransactionError =
|
||||
transaction_error.clone().into();
|
||||
assert_eq!(
|
||||
transaction_error,
|
||||
tx_by_addr_transaction_error.try_into().unwrap()
|
||||
);
|
||||
|
||||
let transaction_error = TransactionError::InsufficientFundsForFee;
|
||||
let tx_by_addr_transaction_error: tx_by_addr::TransactionError =
|
||||
transaction_error.clone().into();
|
||||
@ -1004,6 +1017,22 @@ mod test {
|
||||
tx_by_addr_transaction_error.try_into().unwrap()
|
||||
);
|
||||
|
||||
let transaction_error = TransactionError::WouldExceedMaxBlockCostLimit;
|
||||
let tx_by_addr_transaction_error: tx_by_addr::TransactionError =
|
||||
transaction_error.clone().into();
|
||||
assert_eq!(
|
||||
transaction_error,
|
||||
tx_by_addr_transaction_error.try_into().unwrap()
|
||||
);
|
||||
|
||||
let transaction_error = TransactionError::UnsupportedVersion;
|
||||
let tx_by_addr_transaction_error: tx_by_addr::TransactionError =
|
||||
transaction_error.clone().into();
|
||||
assert_eq!(
|
||||
transaction_error,
|
||||
tx_by_addr_transaction_error.try_into().unwrap()
|
||||
);
|
||||
|
||||
let transaction_error =
|
||||
TransactionError::InstructionError(10, InstructionError::AccountAlreadyInitialized);
|
||||
let tx_by_addr_transaction_error: tx_by_addr::TransactionError =
|
||||
|
@ -55,16 +55,16 @@ pub fn collect_token_balances(
|
||||
) -> TransactionTokenBalances {
|
||||
let mut balances: TransactionTokenBalances = vec![];
|
||||
|
||||
for transaction in batch.transactions_iter() {
|
||||
let account_keys = &transaction.message.account_keys;
|
||||
let has_token_program = account_keys.iter().any(|p| is_token_program(p));
|
||||
for transaction in batch.sanitized_transactions() {
|
||||
let has_token_program = transaction
|
||||
.message()
|
||||
.account_keys_iter()
|
||||
.any(|p| is_token_program(p));
|
||||
|
||||
let mut transaction_balances: Vec<TransactionTokenBalance> = vec![];
|
||||
if has_token_program {
|
||||
for (index, account_id) in account_keys.iter().enumerate() {
|
||||
if is_token_program(account_id)
|
||||
|| transaction.message.program_ids().contains(&account_id)
|
||||
{
|
||||
for (index, account_id) in transaction.message().account_keys_iter().enumerate() {
|
||||
if transaction.message().is_invoked(index) || is_token_program(account_id) {
|
||||
continue;
|
||||
}
|
||||
|
||||
|
Loading…
x
Reference in New Issue
Block a user