Write transaction status and fee into persistent store (#7030)

* Pass blocktree into execute_batch, if persist_transaction_status

* Add validator arg to enable persistent transaction status store

* Pass blocktree into banking_stage, if persist_transaction_status

* Add validator params to bash scripts

* Expose actual transaction statuses outside Bank; add tests

* Fix benches

* Offload transaction status writes to a separate thread

* Enable persistent transaction status along with rpc service

* nudge

* Review comments
This commit is contained in:
Tyera Eulberg 2019-11-20 16:43:10 -07:00 committed by GitHub
parent ee6b11d36d
commit 97ca6858b7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 676 additions and 151 deletions

View File

@ -151,6 +151,7 @@ fn main() {
&poh_recorder, &poh_recorder,
verified_receiver, verified_receiver,
vote_receiver, vote_receiver,
None,
); );
poh_recorder.lock().unwrap().set_bank(&bank); poh_recorder.lock().unwrap().set_bank(&bank);

View File

@ -80,6 +80,7 @@ fn bench_consume_buffered(bencher: &mut Bencher) {
&poh_recorder, &poh_recorder,
&mut packets, &mut packets,
10_000, 10_000,
None,
); );
}); });
@ -195,6 +196,7 @@ fn bench_banking(bencher: &mut Bencher, tx_type: TransactionType) {
&poh_recorder, &poh_recorder,
verified_receiver, verified_receiver,
vote_receiver, vote_receiver,
None,
); );
poh_recorder.lock().unwrap().set_bank(&bank); poh_recorder.lock().unwrap().set_bank(&bank);
@ -286,7 +288,7 @@ fn simulate_process_entries(
hash: next_hash(&bank.last_blockhash(), 1, &tx_vector), hash: next_hash(&bank.last_blockhash(), 1, &tx_vector),
transactions: tx_vector, transactions: tx_vector,
}; };
process_entries(&bank, &vec![entry], randomize_txs).unwrap(); process_entries(&bank, &vec![entry], randomize_txs, None).unwrap();
} }
fn bench_process_entries(randomize_txs: bool, bencher: &mut Bencher) { fn bench_process_entries(randomize_txs: bool, bencher: &mut Bencher) {

View File

@ -12,12 +12,14 @@ use crate::{
use crossbeam_channel::{Receiver as CrossbeamReceiver, RecvTimeoutError}; use crossbeam_channel::{Receiver as CrossbeamReceiver, RecvTimeoutError};
use itertools::Itertools; use itertools::Itertools;
use solana_ledger::{ use solana_ledger::{
blocktree::Blocktree, entry::hash_transactions, leader_schedule_cache::LeaderScheduleCache, blocktree::Blocktree,
blocktree_processor::{send_transaction_status_batch, TransactionStatusSender},
entry::hash_transactions,
leader_schedule_cache::LeaderScheduleCache,
}; };
use solana_measure::measure::Measure; use solana_measure::measure::Measure;
use solana_metrics::{inc_new_counter_debug, inc_new_counter_info, inc_new_counter_warn}; use solana_metrics::{inc_new_counter_debug, inc_new_counter_info, inc_new_counter_warn};
use solana_perf::cuda_runtime::PinnedVec; use solana_perf::{cuda_runtime::PinnedVec, perf_libs};
use solana_perf::perf_libs;
use solana_runtime::{accounts_db::ErrorCounters, bank::Bank, transaction_batch::TransactionBatch}; use solana_runtime::{accounts_db::ErrorCounters, bank::Bank, transaction_batch::TransactionBatch};
use solana_sdk::{ use solana_sdk::{
clock::{ clock::{
@ -73,6 +75,7 @@ impl BankingStage {
poh_recorder: &Arc<Mutex<PohRecorder>>, poh_recorder: &Arc<Mutex<PohRecorder>>,
verified_receiver: CrossbeamReceiver<Vec<Packets>>, verified_receiver: CrossbeamReceiver<Vec<Packets>>,
verified_vote_receiver: CrossbeamReceiver<Vec<Packets>>, verified_vote_receiver: CrossbeamReceiver<Vec<Packets>>,
transaction_status_sender: Option<TransactionStatusSender>,
) -> Self { ) -> Self {
Self::new_num_threads( Self::new_num_threads(
cluster_info, cluster_info,
@ -80,6 +83,7 @@ impl BankingStage {
verified_receiver, verified_receiver,
verified_vote_receiver, verified_vote_receiver,
Self::num_threads(), Self::num_threads(),
transaction_status_sender,
) )
} }
@ -89,6 +93,7 @@ impl BankingStage {
verified_receiver: CrossbeamReceiver<Vec<Packets>>, verified_receiver: CrossbeamReceiver<Vec<Packets>>,
verified_vote_receiver: CrossbeamReceiver<Vec<Packets>>, verified_vote_receiver: CrossbeamReceiver<Vec<Packets>>,
num_threads: u32, num_threads: u32,
transaction_status_sender: Option<TransactionStatusSender>,
) -> Self { ) -> Self {
let batch_limit = TOTAL_BUFFERED_PACKETS / ((num_threads - 1) as usize * PACKETS_PER_BATCH); let batch_limit = TOTAL_BUFFERED_PACKETS / ((num_threads - 1) as usize * PACKETS_PER_BATCH);
// Single thread to generate entries from many banks. // Single thread to generate entries from many banks.
@ -108,6 +113,7 @@ impl BankingStage {
let poh_recorder = poh_recorder.clone(); let poh_recorder = poh_recorder.clone();
let cluster_info = cluster_info.clone(); let cluster_info = cluster_info.clone();
let mut recv_start = Instant::now(); let mut recv_start = Instant::now();
let transaction_status_sender = transaction_status_sender.clone();
Builder::new() Builder::new()
.name("solana-banking-stage-tx".to_string()) .name("solana-banking-stage-tx".to_string())
.spawn(move || { .spawn(move || {
@ -121,6 +127,7 @@ impl BankingStage {
enable_forwarding, enable_forwarding,
i, i,
batch_limit, batch_limit,
transaction_status_sender.clone(),
); );
}) })
.unwrap() .unwrap()
@ -155,6 +162,7 @@ impl BankingStage {
poh_recorder: &Arc<Mutex<PohRecorder>>, poh_recorder: &Arc<Mutex<PohRecorder>>,
buffered_packets: &mut Vec<PacketsAndOffsets>, buffered_packets: &mut Vec<PacketsAndOffsets>,
batch_limit: usize, batch_limit: usize,
transaction_status_sender: Option<TransactionStatusSender>,
) -> Result<UnprocessedPackets> { ) -> Result<UnprocessedPackets> {
let mut unprocessed_packets = vec![]; let mut unprocessed_packets = vec![];
let mut rebuffered_packets = 0; let mut rebuffered_packets = 0;
@ -185,6 +193,7 @@ impl BankingStage {
&poh_recorder, &poh_recorder,
&msgs, &msgs,
unprocessed_indexes.to_owned(), unprocessed_indexes.to_owned(),
transaction_status_sender.clone(),
); );
new_tx_count += processed; new_tx_count += processed;
@ -277,6 +286,7 @@ impl BankingStage {
buffered_packets: &mut Vec<PacketsAndOffsets>, buffered_packets: &mut Vec<PacketsAndOffsets>,
enable_forwarding: bool, enable_forwarding: bool,
batch_limit: usize, batch_limit: usize,
transaction_status_sender: Option<TransactionStatusSender>,
) -> Result<()> { ) -> Result<()> {
let (leader_at_slot_offset, poh_has_bank, would_be_leader) = { let (leader_at_slot_offset, poh_has_bank, would_be_leader) = {
let poh = poh_recorder.lock().unwrap(); let poh = poh_recorder.lock().unwrap();
@ -303,6 +313,7 @@ impl BankingStage {
poh_recorder, poh_recorder,
buffered_packets, buffered_packets,
batch_limit, batch_limit,
transaction_status_sender,
)?; )?;
buffered_packets.append(&mut unprocessed); buffered_packets.append(&mut unprocessed);
Ok(()) Ok(())
@ -350,6 +361,7 @@ impl BankingStage {
enable_forwarding: bool, enable_forwarding: bool,
id: u32, id: u32,
batch_limit: usize, batch_limit: usize,
transaction_status_sender: Option<TransactionStatusSender>,
) { ) {
let socket = UdpSocket::bind("0.0.0.0:0").unwrap(); let socket = UdpSocket::bind("0.0.0.0:0").unwrap();
let mut buffered_packets = vec![]; let mut buffered_packets = vec![];
@ -363,6 +375,7 @@ impl BankingStage {
&mut buffered_packets, &mut buffered_packets,
enable_forwarding, enable_forwarding,
batch_limit, batch_limit,
transaction_status_sender.clone(),
) )
.unwrap_or_else(|_| buffered_packets.clear()); .unwrap_or_else(|_| buffered_packets.clear());
} }
@ -385,6 +398,7 @@ impl BankingStage {
recv_timeout, recv_timeout,
id, id,
batch_limit, batch_limit,
transaction_status_sender.clone(),
) { ) {
Err(Error::CrossbeamRecvTimeoutError(RecvTimeoutError::Timeout)) => (), Err(Error::CrossbeamRecvTimeoutError(RecvTimeoutError::Timeout)) => (),
Err(Error::CrossbeamRecvTimeoutError(RecvTimeoutError::Disconnected)) => break, Err(Error::CrossbeamRecvTimeoutError(RecvTimeoutError::Disconnected)) => break,
@ -482,9 +496,10 @@ impl BankingStage {
} }
fn process_and_record_transactions_locked( fn process_and_record_transactions_locked(
bank: &Bank, bank: &Arc<Bank>,
poh: &Arc<Mutex<PohRecorder>>, poh: &Arc<Mutex<PohRecorder>>,
batch: &TransactionBatch, batch: &TransactionBatch,
transaction_status_sender: Option<TransactionStatusSender>,
) -> (Result<usize>, Vec<usize>) { ) -> (Result<usize>, Vec<usize>) {
let mut load_execute_time = Measure::start("load_execute_time"); let mut load_execute_time = Measure::start("load_execute_time");
// Use a shorter maximum age when adding transactions into the pipeline. This will reduce // Use a shorter maximum age when adding transactions into the pipeline. This will reduce
@ -512,14 +527,24 @@ impl BankingStage {
let num_to_commit = num_to_commit.unwrap(); let num_to_commit = num_to_commit.unwrap();
if num_to_commit != 0 { if num_to_commit != 0 {
bank.commit_transactions( let transaction_statuses = bank
txs, .commit_transactions(
None, txs,
&mut loaded_accounts, None,
&results, &mut loaded_accounts,
tx_count, &results,
signature_count, tx_count,
); signature_count,
)
.processing_results;
if let Some(sender) = transaction_status_sender {
send_transaction_status_batch(
bank.clone(),
batch.transactions(),
transaction_statuses,
sender,
);
}
} }
commit_time.stop(); commit_time.stop();
@ -538,10 +563,11 @@ impl BankingStage {
} }
pub fn process_and_record_transactions( pub fn process_and_record_transactions(
bank: &Bank, bank: &Arc<Bank>,
txs: &[Transaction], txs: &[Transaction],
poh: &Arc<Mutex<PohRecorder>>, poh: &Arc<Mutex<PohRecorder>>,
chunk_offset: usize, chunk_offset: usize,
transaction_status_sender: Option<TransactionStatusSender>,
) -> (Result<usize>, Vec<usize>) { ) -> (Result<usize>, Vec<usize>) {
let mut lock_time = Measure::start("lock_time"); let mut lock_time = Measure::start("lock_time");
// Once accounts are locked, other threads cannot encode transactions that will modify the // Once accounts are locked, other threads cannot encode transactions that will modify the
@ -549,8 +575,12 @@ impl BankingStage {
let batch = bank.prepare_batch(txs, None); let batch = bank.prepare_batch(txs, None);
lock_time.stop(); lock_time.stop();
let (result, mut retryable_txs) = let (result, mut retryable_txs) = Self::process_and_record_transactions_locked(
Self::process_and_record_transactions_locked(bank, poh, &batch); bank,
poh,
&batch,
transaction_status_sender,
);
retryable_txs.iter_mut().for_each(|x| *x += chunk_offset); retryable_txs.iter_mut().for_each(|x| *x += chunk_offset);
let mut unlock_time = Measure::start("unlock_time"); let mut unlock_time = Measure::start("unlock_time");
@ -574,9 +604,10 @@ impl BankingStage {
/// Returns the number of transactions successfully processed by the bank, which may be less /// Returns the number of transactions successfully processed by the bank, which may be less
/// than the total number if max PoH height was reached and the bank halted /// than the total number if max PoH height was reached and the bank halted
fn process_transactions( fn process_transactions(
bank: &Bank, bank: &Arc<Bank>,
transactions: &[Transaction], transactions: &[Transaction],
poh: &Arc<Mutex<PohRecorder>>, poh: &Arc<Mutex<PohRecorder>>,
transaction_status_sender: Option<TransactionStatusSender>,
) -> (usize, Vec<usize>) { ) -> (usize, Vec<usize>) {
let mut chunk_start = 0; let mut chunk_start = 0;
let mut unprocessed_txs = vec![]; let mut unprocessed_txs = vec![];
@ -591,6 +622,7 @@ impl BankingStage {
&transactions[chunk_start..chunk_end], &transactions[chunk_start..chunk_end],
poh, poh,
chunk_start, chunk_start,
transaction_status_sender.clone(),
); );
trace!("process_transactions result: {:?}", result); trace!("process_transactions result: {:?}", result);
@ -724,6 +756,7 @@ impl BankingStage {
poh: &Arc<Mutex<PohRecorder>>, poh: &Arc<Mutex<PohRecorder>>,
msgs: &Packets, msgs: &Packets,
packet_indexes: Vec<usize>, packet_indexes: Vec<usize>,
transaction_status_sender: Option<TransactionStatusSender>,
) -> (usize, usize, Vec<usize>) { ) -> (usize, usize, Vec<usize>) {
let (transactions, transaction_to_packet_indexes) = let (transactions, transaction_to_packet_indexes) =
Self::transactions_from_packets(msgs, &packet_indexes); Self::transactions_from_packets(msgs, &packet_indexes);
@ -736,7 +769,7 @@ impl BankingStage {
let tx_len = transactions.len(); let tx_len = transactions.len();
let (processed, unprocessed_tx_indexes) = let (processed, unprocessed_tx_indexes) =
Self::process_transactions(bank, &transactions, poh); Self::process_transactions(bank, &transactions, poh, transaction_status_sender);
let unprocessed_tx_count = unprocessed_tx_indexes.len(); let unprocessed_tx_count = unprocessed_tx_indexes.len();
@ -815,6 +848,7 @@ impl BankingStage {
recv_timeout: Duration, recv_timeout: Duration,
id: u32, id: u32,
batch_limit: usize, batch_limit: usize,
transaction_status_sender: Option<TransactionStatusSender>,
) -> Result<UnprocessedPackets> { ) -> Result<UnprocessedPackets> {
let mut recv_time = Measure::start("process_packets_recv"); let mut recv_time = Measure::start("process_packets_recv");
let mms = verified_receiver.recv_timeout(recv_timeout)?; let mms = verified_receiver.recv_timeout(recv_timeout)?;
@ -851,8 +885,13 @@ impl BankingStage {
} }
let bank = bank.unwrap(); let bank = bank.unwrap();
let (processed, verified_txs_len, unprocessed_indexes) = let (processed, verified_txs_len, unprocessed_indexes) = Self::process_received_packets(
Self::process_received_packets(&bank, &poh, &msgs, packet_indexes); &bank,
&poh,
&msgs,
packet_indexes,
transaction_status_sender.clone(),
);
new_tx_count += processed; new_tx_count += processed;
@ -969,20 +1008,30 @@ pub fn create_test_recorder(
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use super::*; use super::*;
use crate::cluster_info::Node; use crate::{
use crate::genesis_utils::{create_genesis_config, GenesisConfigInfo}; cluster_info::Node,
use crate::packet::to_packets; genesis_utils::{create_genesis_config, GenesisConfigInfo},
use crate::poh_recorder::WorkingBank; packet::to_packets,
poh_recorder::WorkingBank,
transaction_status_service::TransactionStatusService,
};
use crossbeam_channel::unbounded; use crossbeam_channel::unbounded;
use itertools::Itertools; use itertools::Itertools;
use solana_ledger::entry::{Entry, EntrySlice}; use solana_ledger::{
use solana_ledger::get_tmp_ledger_path; blocktree::entries_to_test_shreds,
use solana_sdk::instruction::InstructionError; entry::{next_entry, Entry, EntrySlice},
use solana_sdk::signature::{Keypair, KeypairUtil}; get_tmp_ledger_path,
use solana_sdk::system_transaction; };
use solana_sdk::transaction::TransactionError; use solana_sdk::{
use std::sync::atomic::Ordering; instruction::InstructionError,
use std::thread::sleep; signature::{Keypair, KeypairUtil},
system_transaction,
transaction::TransactionError,
};
use std::{
sync::{atomic::Ordering, mpsc::channel},
thread::sleep,
};
#[test] #[test]
fn test_banking_stage_shutdown1() { fn test_banking_stage_shutdown1() {
@ -1004,6 +1053,7 @@ mod tests {
&poh_recorder, &poh_recorder,
verified_receiver, verified_receiver,
vote_receiver, vote_receiver,
None,
); );
drop(verified_sender); drop(verified_sender);
drop(vote_sender); drop(vote_sender);
@ -1042,6 +1092,7 @@ mod tests {
&poh_recorder, &poh_recorder,
verified_receiver, verified_receiver,
vote_receiver, vote_receiver,
None,
); );
trace!("sending bank"); trace!("sending bank");
drop(verified_sender); drop(verified_sender);
@ -1103,6 +1154,7 @@ mod tests {
&poh_recorder, &poh_recorder,
verified_receiver, verified_receiver,
vote_receiver, vote_receiver,
None,
); );
// fund another account so we can send 2 good transactions in a single batch. // fund another account so we can send 2 good transactions in a single batch.
@ -1244,6 +1296,7 @@ mod tests {
verified_receiver, verified_receiver,
vote_receiver, vote_receiver,
2, 2,
None,
); );
// wait for banking_stage to eat the packets // wait for banking_stage to eat the packets
@ -1644,9 +1697,15 @@ mod tests {
poh_recorder.lock().unwrap().set_working_bank(working_bank); poh_recorder.lock().unwrap().set_working_bank(working_bank);
BankingStage::process_and_record_transactions(&bank, &transactions, &poh_recorder, 0) BankingStage::process_and_record_transactions(
.0 &bank,
.unwrap(); &transactions,
&poh_recorder,
0,
None,
)
.0
.unwrap();
poh_recorder.lock().unwrap().tick(); poh_recorder.lock().unwrap().tick();
let mut done = false; let mut done = false;
@ -1678,7 +1737,8 @@ mod tests {
&bank, &bank,
&transactions, &transactions,
&poh_recorder, &poh_recorder,
0 0,
None,
) )
.0, .0,
Err(Error::PohRecorderError(PohRecorderError::MaxHeightReached)) Err(Error::PohRecorderError(PohRecorderError::MaxHeightReached))
@ -1735,6 +1795,7 @@ mod tests {
&transactions, &transactions,
&poh_recorder, &poh_recorder,
0, 0,
None,
); );
assert!(result.is_ok()); assert!(result.is_ok());
@ -1819,7 +1880,7 @@ mod tests {
let poh_recorder = Arc::new(Mutex::new(poh_recorder)); let poh_recorder = Arc::new(Mutex::new(poh_recorder));
let (processed_transactions_count, mut retryable_txs) = let (processed_transactions_count, mut retryable_txs) =
BankingStage::process_transactions(&bank, &transactions, &poh_recorder); BankingStage::process_transactions(&bank, &transactions, &poh_recorder, None);
assert_eq!(processed_transactions_count, 0,); assert_eq!(processed_transactions_count, 0,);
@ -1830,4 +1891,101 @@ mod tests {
Blocktree::destroy(&ledger_path).unwrap(); Blocktree::destroy(&ledger_path).unwrap();
} }
#[test]
fn test_write_persist_transaction_status() {
solana_logger::setup();
let GenesisConfigInfo {
genesis_config,
mint_keypair,
..
} = create_genesis_config(10_000);
let bank = Arc::new(Bank::new(&genesis_config));
let pubkey = Pubkey::new_rand();
let pubkey1 = Pubkey::new_rand();
let keypair1 = Keypair::new();
let success_tx =
system_transaction::transfer(&mint_keypair, &pubkey, 1, genesis_config.hash());
let success_signature = success_tx.signatures[0];
let entry_1 = next_entry(&genesis_config.hash(), 1, vec![success_tx.clone()]);
let ix_error_tx =
system_transaction::transfer(&keypair1, &pubkey1, 10, genesis_config.hash());
let ix_error_signature = ix_error_tx.signatures[0];
let entry_2 = next_entry(&entry_1.hash, 1, vec![ix_error_tx.clone()]);
let fail_tx =
system_transaction::transfer(&mint_keypair, &pubkey1, 1, genesis_config.hash());
let entry_3 = next_entry(&entry_2.hash, 1, vec![fail_tx.clone()]);
let entries = vec![entry_1, entry_2, entry_3];
let transactions = vec![success_tx, ix_error_tx, fail_tx];
bank.transfer(4, &mint_keypair, &keypair1.pubkey()).unwrap();
let working_bank = WorkingBank {
bank: bank.clone(),
min_tick_height: bank.tick_height(),
max_tick_height: bank.tick_height() + 1,
};
let ledger_path = get_tmp_ledger_path!();
{
let blocktree =
Blocktree::open(&ledger_path).expect("Expected to be able to open database ledger");
let blocktree = Arc::new(blocktree);
let (poh_recorder, _entry_receiver) = PohRecorder::new(
bank.tick_height(),
bank.last_blockhash(),
bank.slot(),
Some((4, 4)),
bank.ticks_per_slot(),
&pubkey,
&blocktree,
&Arc::new(LeaderScheduleCache::new_from_bank(&bank)),
&Arc::new(PohConfig::default()),
);
let poh_recorder = Arc::new(Mutex::new(poh_recorder));
poh_recorder.lock().unwrap().set_working_bank(working_bank);
let shreds = entries_to_test_shreds(entries.clone(), bank.slot(), 0, true, 0);
blocktree.insert_shreds(shreds, None, false).unwrap();
blocktree.set_roots(&[bank.slot()]).unwrap();
let (transaction_status_sender, transaction_status_receiver) = channel();
let transaction_status_service = TransactionStatusService::new(
transaction_status_receiver,
blocktree.clone(),
&Arc::new(AtomicBool::new(false)),
);
let _ = BankingStage::process_and_record_transactions(
&bank,
&transactions,
&poh_recorder,
0,
Some(transaction_status_sender),
);
transaction_status_service.join().unwrap();
let confirmed_block = blocktree.get_confirmed_block(bank.slot()).unwrap();
assert_eq!(confirmed_block.transactions.len(), 3);
for (transaction, result) in confirmed_block.transactions.into_iter() {
if transaction.signatures[0] == success_signature {
assert_eq!(result.unwrap().status, Ok(()));
} else if transaction.signatures[0] == ix_error_signature {
assert_eq!(
result.unwrap().status,
Err(TransactionError::InstructionError(
0,
InstructionError::CustomError(1)
))
);
} else {
assert_eq!(result, None);
}
}
}
Blocktree::destroy(&ledger_path).unwrap();
}
} }

View File

@ -55,6 +55,7 @@ pub mod snapshot_packager_service;
pub mod storage_stage; pub mod storage_stage;
pub mod streamer; pub mod streamer;
pub mod tpu; pub mod tpu;
pub mod transaction_status_service;
pub mod tvu; pub mod tvu;
pub mod validator; pub mod validator;
pub mod weighted_shuffle; pub mod weighted_shuffle;

View File

@ -1,20 +1,20 @@
//! The `replay_stage` replays transactions broadcast by the leader. //! The `replay_stage` replays transactions broadcast by the leader.
use crate::cluster_info::ClusterInfo; use crate::{
use crate::commitment::{ cluster_info::ClusterInfo,
AggregateCommitmentService, BlockCommitmentCache, CommitmentAggregationData, commitment::{AggregateCommitmentService, BlockCommitmentCache, CommitmentAggregationData},
consensus::{StakeLockout, Tower},
poh_recorder::PohRecorder,
result::{Error, Result},
rpc_subscriptions::RpcSubscriptions,
thread_mem_usage,
}; };
use crate::consensus::{StakeLockout, Tower};
use crate::poh_recorder::PohRecorder;
use crate::result::{Error, Result};
use crate::rpc_subscriptions::RpcSubscriptions;
use crate::thread_mem_usage;
use jemalloc_ctl::thread::allocatedp; use jemalloc_ctl::thread::allocatedp;
use solana_ledger::{ use solana_ledger::{
bank_forks::BankForks, bank_forks::BankForks,
block_error::BlockError, block_error::BlockError,
blocktree::{Blocktree, BlocktreeError}, blocktree::{Blocktree, BlocktreeError},
blocktree_processor, blocktree_processor::{self, TransactionStatusSender},
entry::{Entry, EntrySlice}, entry::{Entry, EntrySlice},
leader_schedule_cache::LeaderScheduleCache, leader_schedule_cache::LeaderScheduleCache,
snapshot_package::SnapshotPackageSender, snapshot_package::SnapshotPackageSender,
@ -182,6 +182,7 @@ impl ReplayStage {
slot_full_senders: Vec<Sender<(u64, Pubkey)>>, slot_full_senders: Vec<Sender<(u64, Pubkey)>>,
snapshot_package_sender: Option<SnapshotPackageSender>, snapshot_package_sender: Option<SnapshotPackageSender>,
block_commitment_cache: Arc<RwLock<BlockCommitmentCache>>, block_commitment_cache: Arc<RwLock<BlockCommitmentCache>>,
transaction_status_sender: Option<TransactionStatusSender>,
) -> (Self, Receiver<Vec<Arc<Bank>>>) ) -> (Self, Receiver<Vec<Arc<Bank>>>)
where where
T: 'static + KeypairUtil + Send + Sync, T: 'static + KeypairUtil + Send + Sync,
@ -245,6 +246,7 @@ impl ReplayStage {
&my_pubkey, &my_pubkey,
&mut progress, &mut progress,
&slot_full_senders, &slot_full_senders,
transaction_status_sender.clone(),
); );
datapoint_debug!( datapoint_debug!(
"replay_stage-memory", "replay_stage-memory",
@ -493,6 +495,7 @@ impl ReplayStage {
bank: &Arc<Bank>, bank: &Arc<Bank>,
blocktree: &Blocktree, blocktree: &Blocktree,
bank_progress: &mut ForkProgress, bank_progress: &mut ForkProgress,
transaction_status_sender: Option<TransactionStatusSender>,
) -> (Result<()>, usize) { ) -> (Result<()>, usize) {
let mut tx_count = 0; let mut tx_count = 0;
let now = Instant::now(); let now = Instant::now();
@ -514,7 +517,14 @@ impl ReplayStage {
slot_full, slot_full,
); );
tx_count += entries.iter().map(|e| e.transactions.len()).sum::<usize>(); tx_count += entries.iter().map(|e| e.transactions.len()).sum::<usize>();
Self::replay_entries_into_bank(bank, bank_progress, entries, num_shreds, slot_full) Self::replay_entries_into_bank(
bank,
bank_progress,
entries,
num_shreds,
slot_full,
transaction_status_sender,
)
}); });
if Self::is_replay_result_fatal(&replay_result) { if Self::is_replay_result_fatal(&replay_result) {
@ -663,6 +673,7 @@ impl ReplayStage {
my_pubkey: &Pubkey, my_pubkey: &Pubkey,
progress: &mut HashMap<u64, ForkProgress>, progress: &mut HashMap<u64, ForkProgress>,
slot_full_senders: &[Sender<(u64, Pubkey)>], slot_full_senders: &[Sender<(u64, Pubkey)>],
transaction_status_sender: Option<TransactionStatusSender>,
) -> bool { ) -> bool {
let mut did_complete_bank = false; let mut did_complete_bank = false;
let mut tx_count = 0; let mut tx_count = 0;
@ -685,8 +696,12 @@ impl ReplayStage {
.entry(bank.slot()) .entry(bank.slot())
.or_insert_with(|| ForkProgress::new(bank.slot(), bank.last_blockhash())); .or_insert_with(|| ForkProgress::new(bank.slot(), bank.last_blockhash()));
if bank.collector_id() != my_pubkey { if bank.collector_id() != my_pubkey {
let (replay_result, replay_tx_count) = let (replay_result, replay_tx_count) = Self::replay_blocktree_into_bank(
Self::replay_blocktree_into_bank(&bank, &blocktree, bank_progress); &bank,
&blocktree,
bank_progress,
transaction_status_sender.clone(),
);
tx_count += replay_tx_count; tx_count += replay_tx_count;
if Self::is_replay_result_fatal(&replay_result) { if Self::is_replay_result_fatal(&replay_result) {
trace!("replay_result_fatal slot {}", bank_slot); trace!("replay_result_fatal slot {}", bank_slot);
@ -950,6 +965,7 @@ impl ReplayStage {
entries: Vec<Entry>, entries: Vec<Entry>,
num_shreds: usize, num_shreds: usize,
slot_full: bool, slot_full: bool,
transaction_status_sender: Option<TransactionStatusSender>,
) -> Result<()> { ) -> Result<()> {
let result = Self::verify_and_process_entries( let result = Self::verify_and_process_entries(
&bank, &bank,
@ -957,6 +973,7 @@ impl ReplayStage {
slot_full, slot_full,
bank_progress.num_shreds, bank_progress.num_shreds,
bank_progress, bank_progress,
transaction_status_sender,
); );
bank_progress.num_shreds += num_shreds; bank_progress.num_shreds += num_shreds;
bank_progress.num_entries += entries.len(); bank_progress.num_entries += entries.len();
@ -1008,6 +1025,7 @@ impl ReplayStage {
slot_full: bool, slot_full: bool,
shred_index: usize, shred_index: usize,
bank_progress: &mut ForkProgress, bank_progress: &mut ForkProgress,
transaction_status_sender: Option<TransactionStatusSender>,
) -> Result<()> { ) -> Result<()> {
let last_entry = &bank_progress.last_entry; let last_entry = &bank_progress.last_entry;
let tick_hash_count = &mut bank_progress.tick_hash_count; let tick_hash_count = &mut bank_progress.tick_hash_count;
@ -1042,7 +1060,8 @@ impl ReplayStage {
let mut entry_state = entries.start_verify(last_entry); let mut entry_state = entries.start_verify(last_entry);
let mut replay_elapsed = Measure::start("replay_elapsed"); let mut replay_elapsed = Measure::start("replay_elapsed");
let res = blocktree_processor::process_entries(bank, entries, true); let res =
blocktree_processor::process_entries(bank, entries, true, transaction_status_sender);
replay_elapsed.stop(); replay_elapsed.stop();
bank_progress.stats.replay_elapsed += replay_elapsed.as_us(); bank_progress.stats.replay_elapsed += replay_elapsed.as_us();
@ -1125,29 +1144,38 @@ impl ReplayStage {
#[cfg(test)] #[cfg(test)]
mod test { mod test {
use super::*; use super::*;
use crate::commitment::BlockCommitment; use crate::{
use crate::genesis_utils::{create_genesis_config, create_genesis_config_with_leader}; commitment::BlockCommitment,
use crate::replay_stage::ReplayStage; genesis_utils::{create_genesis_config, create_genesis_config_with_leader},
use solana_ledger::blocktree::make_slot_entries; replay_stage::ReplayStage,
use solana_ledger::entry; transaction_status_service::TransactionStatusService,
use solana_ledger::shred::{
CodingShredHeader, DataShredHeader, Shred, ShredCommonHeader, DATA_COMPLETE_SHRED,
SIZE_OF_COMMON_SHRED_HEADER, SIZE_OF_DATA_SHRED_HEADER, SIZE_OF_DATA_SHRED_PAYLOAD,
}; };
use solana_ledger::{ use solana_ledger::{
blocktree::make_slot_entries,
blocktree::{entries_to_test_shreds, BlocktreeError}, blocktree::{entries_to_test_shreds, BlocktreeError},
create_new_tmp_ledger,
entry::{self, next_entry},
get_tmp_ledger_path, get_tmp_ledger_path,
shred::{
CodingShredHeader, DataShredHeader, Shred, ShredCommonHeader, DATA_COMPLETE_SHRED,
SIZE_OF_COMMON_SHRED_HEADER, SIZE_OF_DATA_SHRED_HEADER, SIZE_OF_DATA_SHRED_PAYLOAD,
},
}; };
use solana_runtime::genesis_utils::GenesisConfigInfo; use solana_runtime::genesis_utils::GenesisConfigInfo;
use solana_sdk::hash::{hash, Hash}; use solana_sdk::{
use solana_sdk::packet::PACKET_DATA_SIZE; hash::{hash, Hash},
use solana_sdk::signature::{Keypair, KeypairUtil}; instruction::InstructionError,
use solana_sdk::system_transaction; packet::PACKET_DATA_SIZE,
use solana_sdk::transaction::TransactionError; signature::{Keypair, KeypairUtil},
system_transaction,
transaction::TransactionError,
};
use solana_vote_program::vote_state::VoteState; use solana_vote_program::vote_state::VoteState;
use std::fs::remove_dir_all; use std::{
use std::iter::FromIterator; fs::remove_dir_all,
use std::sync::{Arc, RwLock}; iter::FromIterator,
sync::{Arc, RwLock},
};
#[test] #[test]
fn test_child_slots_of_same_parent() { fn test_child_slots_of_same_parent() {
@ -1429,8 +1457,12 @@ mod test {
.or_insert_with(|| ForkProgress::new(0, last_blockhash)); .or_insert_with(|| ForkProgress::new(0, last_blockhash));
let shreds = shred_to_insert(&mint_keypair, bank0.clone()); let shreds = shred_to_insert(&mint_keypair, bank0.clone());
blocktree.insert_shreds(shreds, None, false).unwrap(); blocktree.insert_shreds(shreds, None, false).unwrap();
let (res, _tx_count) = let (res, _tx_count) = ReplayStage::replay_blocktree_into_bank(
ReplayStage::replay_blocktree_into_bank(&bank0, &blocktree, &mut bank0_progress); &bank0,
&blocktree,
&mut bank0_progress,
None,
);
// Check that the erroring bank was marked as dead in the progress map // Check that the erroring bank was marked as dead in the progress map
assert!(progress assert!(progress
@ -1675,4 +1707,90 @@ mod test {
assert!(res.1.is_some()); assert!(res.1.is_some());
assert_eq!(res.1.unwrap().slot(), 11); assert_eq!(res.1.unwrap().slot(), 11);
} }
#[test]
fn test_write_persist_transaction_status() {
let GenesisConfigInfo {
genesis_config,
mint_keypair,
..
} = create_genesis_config(1000);
let (ledger_path, blockhash) = create_new_tmp_ledger!(&genesis_config);
{
let blocktree = Blocktree::open(&ledger_path)
.expect("Expected to successfully open database ledger");
let blocktree = Arc::new(blocktree);
let keypair1 = Keypair::new();
let keypair2 = Keypair::new();
let keypair3 = Keypair::new();
let bank0 = Arc::new(Bank::new(&genesis_config));
bank0
.transfer(4, &mint_keypair, &keypair2.pubkey())
.unwrap();
let bank1 = Arc::new(Bank::new_from_parent(&bank0, &Pubkey::default(), 1));
let slot = bank1.slot();
// Generate transactions for processing
// Successful transaction
let success_tx =
system_transaction::transfer(&mint_keypair, &keypair1.pubkey(), 2, blockhash);
let success_signature = success_tx.signatures[0];
let entry_1 = next_entry(&blockhash, 1, vec![success_tx]);
// Failed transaction, InstructionError
let ix_error_tx =
system_transaction::transfer(&keypair2, &keypair3.pubkey(), 10, blockhash);
let ix_error_signature = ix_error_tx.signatures[0];
let entry_2 = next_entry(&entry_1.hash, 1, vec![ix_error_tx]);
// Failed transaction
let fail_tx =
system_transaction::transfer(&mint_keypair, &keypair2.pubkey(), 2, Hash::default());
let entry_3 = next_entry(&entry_2.hash, 1, vec![fail_tx]);
let entries = vec![entry_1, entry_2, entry_3];
let shreds = entries_to_test_shreds(entries.clone(), slot, bank0.slot(), true, 0);
blocktree.insert_shreds(shreds, None, false).unwrap();
blocktree.set_roots(&[slot]).unwrap();
let (transaction_status_sender, transaction_status_receiver) = channel();
let transaction_status_service = TransactionStatusService::new(
transaction_status_receiver,
blocktree.clone(),
&Arc::new(AtomicBool::new(false)),
);
// Check that process_entries successfully writes can_commit transactions statuses, and
// that they are matched properly by get_confirmed_block
let _result = blocktree_processor::process_entries(
&bank1,
&entries,
true,
Some(transaction_status_sender),
);
transaction_status_service.join().unwrap();
let confirmed_block = blocktree.get_confirmed_block(slot).unwrap();
assert_eq!(confirmed_block.transactions.len(), 3);
for (transaction, result) in confirmed_block.transactions.into_iter() {
if transaction.signatures[0] == success_signature {
assert_eq!(result.unwrap().status, Ok(()));
} else if transaction.signatures[0] == ix_error_signature {
assert_eq!(
result.unwrap().status,
Err(TransactionError::InstructionError(
0,
InstructionError::CustomError(1)
))
);
} else {
assert_eq!(result, None);
}
}
}
Blocktree::destroy(&ledger_path).unwrap();
}
} }

View File

@ -1,21 +1,27 @@
//! The `tpu` module implements the Transaction Processing Unit, a //! The `tpu` module implements the Transaction Processing Unit, a
//! multi-stage transaction processing pipeline in software. //! multi-stage transaction processing pipeline in software.
use crate::banking_stage::BankingStage; use crate::{
use crate::broadcast_stage::{BroadcastStage, BroadcastStageType}; banking_stage::BankingStage,
use crate::cluster_info::ClusterInfo; broadcast_stage::{BroadcastStage, BroadcastStageType},
use crate::cluster_info_vote_listener::ClusterInfoVoteListener; cluster_info::ClusterInfo,
use crate::fetch_stage::FetchStage; cluster_info_vote_listener::ClusterInfoVoteListener,
use crate::poh_recorder::{PohRecorder, WorkingBankEntry}; fetch_stage::FetchStage,
use crate::sigverify::TransactionSigVerifier; poh_recorder::{PohRecorder, WorkingBankEntry},
use crate::sigverify_stage::{DisabledSigVerifier, SigVerifyStage}; sigverify::TransactionSigVerifier,
sigverify_stage::{DisabledSigVerifier, SigVerifyStage},
};
use crossbeam_channel::unbounded; use crossbeam_channel::unbounded;
use solana_ledger::blocktree::Blocktree; use solana_ledger::{blocktree::Blocktree, blocktree_processor::TransactionStatusSender};
use std::net::UdpSocket; use std::{
use std::sync::atomic::AtomicBool; net::UdpSocket,
use std::sync::mpsc::{channel, Receiver}; sync::{
use std::sync::{Arc, Mutex, RwLock}; atomic::AtomicBool,
use std::thread; mpsc::{channel, Receiver},
Arc, Mutex, RwLock,
},
thread,
};
pub struct Tpu { pub struct Tpu {
fetch_stage: FetchStage, fetch_stage: FetchStage,
@ -35,6 +41,7 @@ impl Tpu {
tpu_forwards_sockets: Vec<UdpSocket>, tpu_forwards_sockets: Vec<UdpSocket>,
broadcast_socket: UdpSocket, broadcast_socket: UdpSocket,
sigverify_disabled: bool, sigverify_disabled: bool,
transaction_status_sender: Option<TransactionStatusSender>,
blocktree: &Arc<Blocktree>, blocktree: &Arc<Blocktree>,
broadcast_type: &BroadcastStageType, broadcast_type: &BroadcastStageType,
exit: &Arc<AtomicBool>, exit: &Arc<AtomicBool>,
@ -72,6 +79,7 @@ impl Tpu {
poh_recorder, poh_recorder,
verified_receiver, verified_receiver,
verified_vote_receiver, verified_vote_receiver,
transaction_status_sender,
); );
let broadcast_stage = broadcast_type.new_broadcast_stage( let broadcast_stage = broadcast_type.new_broadcast_stage(

View File

@ -0,0 +1,79 @@
use crate::result::{Error, Result};
use solana_client::rpc_request::RpcTransactionStatus;
use solana_ledger::{blocktree::Blocktree, blocktree_processor::TransactionStatusBatch};
use solana_runtime::bank::Bank;
use std::{
sync::{
atomic::{AtomicBool, Ordering},
mpsc::{Receiver, RecvTimeoutError},
Arc,
},
thread::{self, Builder, JoinHandle},
time::Duration,
};
pub struct TransactionStatusService {
thread_hdl: JoinHandle<()>,
}
impl TransactionStatusService {
#[allow(clippy::new_ret_no_self)]
pub fn new(
write_transaction_status_receiver: Receiver<TransactionStatusBatch>,
blocktree: Arc<Blocktree>,
exit: &Arc<AtomicBool>,
) -> Self {
let exit = exit.clone();
let thread_hdl = Builder::new()
.name("solana-transaction-status-writer".to_string())
.spawn(move || loop {
if exit.load(Ordering::Relaxed) {
break;
}
if let Err(e) = Self::write_transaction_status_batch(
&write_transaction_status_receiver,
&blocktree,
) {
match e {
Error::RecvTimeoutError(RecvTimeoutError::Disconnected) => break,
Error::RecvTimeoutError(RecvTimeoutError::Timeout) => (),
_ => info!("Error from write_transaction_statuses: {:?}", e),
}
}
})
.unwrap();
Self { thread_hdl }
}
fn write_transaction_status_batch(
write_transaction_status_receiver: &Receiver<TransactionStatusBatch>,
blocktree: &Arc<Blocktree>,
) -> Result<()> {
let TransactionStatusBatch {
bank,
transactions,
statuses,
} = write_transaction_status_receiver.recv_timeout(Duration::from_secs(1))?;
let slot = bank.slot();
for (transaction, status) in transactions.iter().zip(statuses) {
if Bank::can_commit(&status) && !transaction.signatures.is_empty() {
let fee_calculator = bank
.get_fee_calculator(&transaction.message().recent_blockhash)
.expect("FeeCalculator must exist");
let fee = fee_calculator.calculate_fee(transaction.message());
blocktree
.write_transaction_status(
(slot, transaction.signatures[0]),
&RpcTransactionStatus { status, fee },
)
.expect("Expect database write to succeed");
}
}
Ok(())
}
pub fn join(self) -> thread::Result<()> {
self.thread_hdl.join()
}
}

View File

@ -1,32 +1,43 @@
//! The `tvu` module implements the Transaction Validation Unit, a multi-stage transaction //! The `tvu` module implements the Transaction Validation Unit, a multi-stage transaction
//! validation pipeline in software. //! validation pipeline in software.
use crate::blockstream_service::BlockstreamService; use crate::{
use crate::cluster_info::ClusterInfo; blockstream_service::BlockstreamService,
use crate::commitment::BlockCommitmentCache; cluster_info::ClusterInfo,
use crate::ledger_cleanup_service::LedgerCleanupService; commitment::BlockCommitmentCache,
use crate::partition_cfg::PartitionCfg; ledger_cleanup_service::LedgerCleanupService,
use crate::poh_recorder::PohRecorder; partition_cfg::PartitionCfg,
use crate::replay_stage::ReplayStage; poh_recorder::PohRecorder,
use crate::retransmit_stage::RetransmitStage; replay_stage::ReplayStage,
use crate::rpc_subscriptions::RpcSubscriptions; retransmit_stage::RetransmitStage,
use crate::shred_fetch_stage::ShredFetchStage; rpc_subscriptions::RpcSubscriptions,
use crate::sigverify_shreds::ShredSigVerifier; shred_fetch_stage::ShredFetchStage,
use crate::sigverify_stage::{DisabledSigVerifier, SigVerifyStage}; sigverify_shreds::ShredSigVerifier,
use crate::snapshot_packager_service::SnapshotPackagerService; sigverify_stage::{DisabledSigVerifier, SigVerifyStage},
use crate::storage_stage::{StorageStage, StorageState}; snapshot_packager_service::SnapshotPackagerService,
storage_stage::{StorageStage, StorageState},
};
use crossbeam_channel::unbounded; use crossbeam_channel::unbounded;
use solana_ledger::bank_forks::BankForks;
use solana_ledger::blocktree::{Blocktree, CompletedSlotsReceiver};
use solana_ledger::leader_schedule_cache::LeaderScheduleCache; use solana_ledger::leader_schedule_cache::LeaderScheduleCache;
use solana_sdk::pubkey::Pubkey; use solana_ledger::{
use solana_sdk::signature::{Keypair, KeypairUtil}; bank_forks::BankForks,
use std::net::UdpSocket; blocktree::{Blocktree, CompletedSlotsReceiver},
use std::path::PathBuf; blocktree_processor::TransactionStatusSender,
use std::sync::atomic::AtomicBool; };
use std::sync::mpsc::{channel, Receiver}; use solana_sdk::{
use std::sync::{Arc, Mutex, RwLock}; pubkey::Pubkey,
use std::thread; signature::{Keypair, KeypairUtil},
};
use std::{
net::UdpSocket,
path::PathBuf,
sync::{
atomic::AtomicBool,
mpsc::{channel, Receiver},
Arc, Mutex, RwLock,
},
thread,
};
pub struct Tvu { pub struct Tvu {
fetch_stage: ShredFetchStage, fetch_stage: ShredFetchStage,
@ -75,6 +86,7 @@ impl Tvu {
sigverify_disabled: bool, sigverify_disabled: bool,
cfg: Option<PartitionCfg>, cfg: Option<PartitionCfg>,
shred_version: u16, shred_version: u16,
transaction_status_sender: Option<TransactionStatusSender>,
) -> Self ) -> Self
where where
T: 'static + KeypairUtil + Sync + Send, T: 'static + KeypairUtil + Sync + Send,
@ -165,6 +177,7 @@ impl Tvu {
vec![blockstream_slot_sender, ledger_cleanup_slot_sender], vec![blockstream_slot_sender, ledger_cleanup_slot_sender],
snapshot_package_sender, snapshot_package_sender,
block_commitment_cache, block_commitment_cache,
transaction_status_sender,
); );
let blockstream_service = if let Some(blockstream_unix_socket) = blockstream_unix_socket { let blockstream_service = if let Some(blockstream_unix_socket) = blockstream_unix_socket {
@ -297,6 +310,7 @@ pub mod tests {
false, false,
None, None,
0, 0,
None,
); );
exit.store(true, Ordering::Relaxed); exit.store(true, Ordering::Relaxed);
tvu.join().unwrap(); tvu.join().unwrap();

View File

@ -16,6 +16,7 @@ use crate::{
sigverify, sigverify,
storage_stage::StorageState, storage_stage::StorageState,
tpu::Tpu, tpu::Tpu,
transaction_status_service::TransactionStatusService,
tvu::{Sockets, Tvu}, tvu::{Sockets, Tvu},
}; };
use solana_ledger::{ use solana_ledger::{
@ -43,7 +44,7 @@ use std::{
path::{Path, PathBuf}, path::{Path, PathBuf},
process, process,
sync::atomic::{AtomicBool, Ordering}, sync::atomic::{AtomicBool, Ordering},
sync::mpsc::Receiver, sync::mpsc::{channel, Receiver},
sync::{Arc, Mutex, RwLock}, sync::{Arc, Mutex, RwLock},
thread::Result, thread::Result,
}; };
@ -54,6 +55,7 @@ pub struct ValidatorConfig {
pub dev_halt_at_slot: Option<Slot>, pub dev_halt_at_slot: Option<Slot>,
pub expected_genesis_hash: Option<Hash>, pub expected_genesis_hash: Option<Hash>,
pub voting_disabled: bool, pub voting_disabled: bool,
pub transaction_status_service_disabled: bool,
pub blockstream_unix_socket: Option<PathBuf>, pub blockstream_unix_socket: Option<PathBuf>,
pub storage_slots_per_turn: u64, pub storage_slots_per_turn: u64,
pub account_paths: Option<String>, pub account_paths: Option<String>,
@ -71,6 +73,7 @@ impl Default for ValidatorConfig {
dev_halt_at_slot: None, dev_halt_at_slot: None,
expected_genesis_hash: None, expected_genesis_hash: None,
voting_disabled: false, voting_disabled: false,
transaction_status_service_disabled: false,
blockstream_unix_socket: None, blockstream_unix_socket: None,
storage_slots_per_turn: DEFAULT_SLOTS_PER_TURN, storage_slots_per_turn: DEFAULT_SLOTS_PER_TURN,
max_ledger_slots: None, max_ledger_slots: None,
@ -105,6 +108,7 @@ pub struct Validator {
validator_exit: Arc<RwLock<Option<ValidatorExit>>>, validator_exit: Arc<RwLock<Option<ValidatorExit>>>,
rpc_service: Option<JsonRpcService>, rpc_service: Option<JsonRpcService>,
rpc_pubsub_service: Option<PubSubService>, rpc_pubsub_service: Option<PubSubService>,
transaction_status_service: Option<TransactionStatusService>,
gossip_service: GossipService, gossip_service: GossipService,
poh_recorder: Arc<Mutex<PohRecorder>>, poh_recorder: Arc<Mutex<PohRecorder>>,
poh_service: PohService, poh_service: PohService,
@ -238,6 +242,21 @@ impl Validator {
)) ))
}; };
let (transaction_status_sender, transaction_status_service) =
if rpc_service.is_some() && !config.transaction_status_service_disabled {
let (transaction_status_sender, transaction_status_receiver) = channel();
(
Some(transaction_status_sender),
Some(TransactionStatusService::new(
transaction_status_receiver,
blocktree.clone(),
&exit,
)),
)
} else {
(None, None)
};
info!( info!(
"Starting PoH: epoch={} slot={} tick_height={} blockhash={} leader={:?}", "Starting PoH: epoch={} slot={} tick_height={} blockhash={} leader={:?}",
bank.epoch(), bank.epoch(),
@ -350,6 +369,7 @@ impl Validator {
config.dev_sigverify_disabled, config.dev_sigverify_disabled,
config.partition_cfg.clone(), config.partition_cfg.clone(),
shred_version, shred_version,
transaction_status_sender.clone(),
); );
if config.dev_sigverify_disabled { if config.dev_sigverify_disabled {
@ -364,6 +384,7 @@ impl Validator {
node.sockets.tpu_forwards, node.sockets.tpu_forwards,
node.sockets.broadcast, node.sockets.broadcast,
config.dev_sigverify_disabled, config.dev_sigverify_disabled,
transaction_status_sender,
&blocktree, &blocktree,
&config.broadcast_stage_type, &config.broadcast_stage_type,
&exit, &exit,
@ -376,6 +397,7 @@ impl Validator {
gossip_service, gossip_service,
rpc_service, rpc_service,
rpc_pubsub_service, rpc_pubsub_service,
transaction_status_service,
tpu, tpu,
tvu, tvu,
poh_service, poh_service,
@ -426,6 +448,9 @@ impl Validator {
if let Some(rpc_pubsub_service) = self.rpc_pubsub_service { if let Some(rpc_pubsub_service) = self.rpc_pubsub_service {
rpc_pubsub_service.join()?; rpc_pubsub_service.join()?;
} }
if let Some(transaction_status_service) = self.transaction_status_service {
transaction_status_service.join()?;
}
self.gossip_service.join()?; self.gossip_service.join()?;
self.tpu.join()?; self.tpu.join()?;
@ -529,6 +554,8 @@ pub fn new_validator_for_tests() -> (Validator, ContactInfo, Keypair, PathBuf) {
let leader_voting_keypair = Arc::new(voting_keypair); let leader_voting_keypair = Arc::new(voting_keypair);
let storage_keypair = Arc::new(Keypair::new()); let storage_keypair = Arc::new(Keypair::new());
let mut config = ValidatorConfig::default();
config.transaction_status_service_disabled = true;
let node = Validator::new( let node = Validator::new(
node, node,
&node_keypair, &node_keypair,
@ -538,7 +565,7 @@ pub fn new_validator_for_tests() -> (Validator, ContactInfo, Keypair, PathBuf) {
&storage_keypair, &storage_keypair,
None, None,
true, true,
&ValidatorConfig::default(), &config,
); );
discover_cluster(&contact_info.gossip, 1).expect("Node startup failed"); discover_cluster(&contact_info.gossip, 1).expect("Node startup failed");
(node, contact_info, mint_keypair, ledger_path) (node, contact_info, mint_keypair, ledger_path)
@ -565,6 +592,8 @@ mod tests {
let voting_keypair = Arc::new(Keypair::new()); let voting_keypair = Arc::new(Keypair::new());
let storage_keypair = Arc::new(Keypair::new()); let storage_keypair = Arc::new(Keypair::new());
let mut config = ValidatorConfig::default();
config.transaction_status_service_disabled = true;
let validator = Validator::new( let validator = Validator::new(
validator_node, validator_node,
&Arc::new(validator_keypair), &Arc::new(validator_keypair),
@ -574,7 +603,7 @@ mod tests {
&storage_keypair, &storage_keypair,
Some(&leader_node.info), Some(&leader_node.info),
true, true,
&ValidatorConfig::default(), &config,
); );
validator.close().unwrap(); validator.close().unwrap();
remove_dir_all(validator_ledger_path).unwrap(); remove_dir_all(validator_ledger_path).unwrap();
@ -597,6 +626,8 @@ mod tests {
ledger_paths.push(validator_ledger_path.clone()); ledger_paths.push(validator_ledger_path.clone());
let voting_keypair = Arc::new(Keypair::new()); let voting_keypair = Arc::new(Keypair::new());
let storage_keypair = Arc::new(Keypair::new()); let storage_keypair = Arc::new(Keypair::new());
let mut config = ValidatorConfig::default();
config.transaction_status_service_disabled = true;
Validator::new( Validator::new(
validator_node, validator_node,
&Arc::new(validator_keypair), &Arc::new(validator_keypair),
@ -606,7 +637,7 @@ mod tests {
&storage_keypair, &storage_keypair,
Some(&leader_node.info), Some(&leader_node.info),
true, true,
&ValidatorConfig::default(), &config,
) )
}) })
.collect(); .collect();

View File

@ -117,6 +117,7 @@ mod tests {
bank.last_blockhash(), bank.last_blockhash(),
), ),
true, true,
None,
) )
.unwrap(); .unwrap();
let message = Message::new_with_payer(vec![mining_proof_ix], Some(&mint_keypair.pubkey())); let message = Message::new_with_payer(vec![mining_proof_ix], Some(&mint_keypair.pubkey()));
@ -206,6 +207,7 @@ mod tests {
&bank, &bank,
&entry::create_ticks(64, 0, bank.last_blockhash()), &entry::create_ticks(64, 0, bank.last_blockhash()),
true, true,
None,
) )
.expect("failed process entries"); .expect("failed process entries");
last_bank = bank; last_bank = bank;

View File

@ -30,7 +30,7 @@ use solana_sdk::{
clock::{Slot, DEFAULT_TICKS_PER_SECOND}, clock::{Slot, DEFAULT_TICKS_PER_SECOND},
genesis_config::GenesisConfig, genesis_config::GenesisConfig,
hash::Hash, hash::Hash,
signature::{Keypair, KeypairUtil}, signature::{Keypair, KeypairUtil, Signature},
timing::timestamp, timing::timestamp,
transaction::Transaction, transaction::Transaction,
}; };
@ -1175,6 +1175,14 @@ impl Blocktree {
.collect() .collect()
} }
pub fn write_transaction_status(
&self,
index: (Slot, Signature),
status: &RpcTransactionStatus,
) -> Result<()> {
self.transaction_status_cf.put(index, status)
}
/// Returns the entry vector for the slot starting with `shred_start_index` /// Returns the entry vector for the slot starting with `shred_start_index`
pub fn get_slot_entries( pub fn get_slot_entries(
&self, &self,

View File

@ -12,19 +12,22 @@ use rand::{seq::SliceRandom, thread_rng};
use rayon::{prelude::*, ThreadPool}; use rayon::{prelude::*, ThreadPool};
use solana_metrics::{datapoint, datapoint_error, inc_new_counter_debug}; use solana_metrics::{datapoint, datapoint_error, inc_new_counter_debug};
use solana_rayon_threadlimit::get_thread_count; use solana_rayon_threadlimit::get_thread_count;
use solana_runtime::{bank::Bank, transaction_batch::TransactionBatch}; use solana_runtime::{
bank::{Bank, TransactionResults},
transaction_batch::TransactionBatch,
};
use solana_sdk::{ use solana_sdk::{
clock::{Slot, MAX_RECENT_BLOCKHASHES}, clock::{Slot, MAX_RECENT_BLOCKHASHES},
genesis_config::GenesisConfig, genesis_config::GenesisConfig,
hash::Hash, hash::Hash,
signature::{Keypair, KeypairUtil}, signature::{Keypair, KeypairUtil},
timing::duration_as_ms, timing::duration_as_ms,
transaction::Result, transaction::{Result, Transaction},
}; };
use std::{ use std::{
cell::RefCell, cell::RefCell,
result, result,
sync::Arc, sync::{mpsc::Sender, Arc},
time::{Duration, Instant}, time::{Duration, Instant},
}; };
@ -43,13 +46,29 @@ fn first_err(results: &[Result<()>]) -> Result<()> {
Ok(()) Ok(())
} }
fn execute_batch(batch: &TransactionBatch) -> Result<()> { fn execute_batch(
let results = batch batch: &TransactionBatch,
bank: &Arc<Bank>,
transaction_status_sender: Option<TransactionStatusSender>,
) -> Result<()> {
let TransactionResults {
fee_collection_results,
processing_results,
} = batch
.bank() .bank()
.load_execute_and_commit_transactions(batch, MAX_RECENT_BLOCKHASHES); .load_execute_and_commit_transactions(batch, MAX_RECENT_BLOCKHASHES);
if let Some(sender) = transaction_status_sender {
send_transaction_status_batch(
bank.clone(),
batch.transactions(),
processing_results,
sender,
);
}
let mut first_err = None; let mut first_err = None;
for (result, transaction) in results.iter().zip(batch.transactions()) { for (result, transaction) in fee_collection_results.iter().zip(batch.transactions()) {
if let Err(ref err) = result { if let Err(ref err) = result {
if first_err.is_none() { if first_err.is_none() {
first_err = Some(result.clone()); first_err = Some(result.clone());
@ -75,14 +94,15 @@ fn execute_batches(
bank: &Arc<Bank>, bank: &Arc<Bank>,
batches: &[TransactionBatch], batches: &[TransactionBatch],
entry_callback: Option<&ProcessCallback>, entry_callback: Option<&ProcessCallback>,
transaction_status_sender: Option<TransactionStatusSender>,
) -> Result<()> { ) -> Result<()> {
inc_new_counter_debug!("bank-par_execute_entries-count", batches.len()); inc_new_counter_debug!("bank-par_execute_entries-count", batches.len());
let results: Vec<Result<()>> = PAR_THREAD_POOL.with(|thread_pool| { let results: Vec<Result<()>> = PAR_THREAD_POOL.with(|thread_pool| {
thread_pool.borrow().install(|| { thread_pool.borrow().install(|| {
batches batches
.into_par_iter() .into_par_iter()
.map(|batch| { .map_with(transaction_status_sender, |sender, batch| {
let result = execute_batch(batch); let result = execute_batch(batch, bank, sender.clone());
if let Some(entry_callback) = entry_callback { if let Some(entry_callback) = entry_callback {
entry_callback(bank); entry_callback(bank);
} }
@ -100,8 +120,13 @@ fn execute_batches(
/// 2. Process the locked group in parallel /// 2. Process the locked group in parallel
/// 3. Register the `Tick` if it's available /// 3. Register the `Tick` if it's available
/// 4. Update the leader scheduler, goto 1 /// 4. Update the leader scheduler, goto 1
pub fn process_entries(bank: &Arc<Bank>, entries: &[Entry], randomize: bool) -> Result<()> { pub fn process_entries(
process_entries_with_callback(bank, entries, randomize, None) bank: &Arc<Bank>,
entries: &[Entry],
randomize: bool,
transaction_status_sender: Option<TransactionStatusSender>,
) -> Result<()> {
process_entries_with_callback(bank, entries, randomize, None, transaction_status_sender)
} }
fn process_entries_with_callback( fn process_entries_with_callback(
@ -109,6 +134,7 @@ fn process_entries_with_callback(
entries: &[Entry], entries: &[Entry],
randomize: bool, randomize: bool,
entry_callback: Option<&ProcessCallback>, entry_callback: Option<&ProcessCallback>,
transaction_status_sender: Option<TransactionStatusSender>,
) -> Result<()> { ) -> Result<()> {
// accumulator for entries that can be processed in parallel // accumulator for entries that can be processed in parallel
let mut batches = vec![]; let mut batches = vec![];
@ -120,7 +146,12 @@ fn process_entries_with_callback(
if bank.is_block_boundary(bank.tick_height() + tick_hashes.len() as u64) { if bank.is_block_boundary(bank.tick_height() + tick_hashes.len() as u64) {
// If it's a tick that will cause a new blockhash to be created, // If it's a tick that will cause a new blockhash to be created,
// execute the group and register the tick // execute the group and register the tick
execute_batches(bank, &batches, entry_callback)?; execute_batches(
bank,
&batches,
entry_callback,
transaction_status_sender.clone(),
)?;
batches.clear(); batches.clear();
for hash in &tick_hashes { for hash in &tick_hashes {
bank.register_tick(hash); bank.register_tick(hash);
@ -170,12 +201,17 @@ fn process_entries_with_callback(
} else { } else {
// else we have an entry that conflicts with a prior entry // else we have an entry that conflicts with a prior entry
// execute the current queue and try to process this entry again // execute the current queue and try to process this entry again
execute_batches(bank, &batches, entry_callback)?; execute_batches(
bank,
&batches,
entry_callback,
transaction_status_sender.clone(),
)?;
batches.clear(); batches.clear();
} }
} }
} }
execute_batches(bank, &batches, entry_callback)?; execute_batches(bank, &batches, entry_callback, transaction_status_sender)?;
for hash in tick_hashes { for hash in tick_hashes {
bank.register_tick(&hash); bank.register_tick(&hash);
} }
@ -343,16 +379,15 @@ fn verify_and_process_slot_entries(
} }
} }
process_entries_with_callback(bank, &entries, true, opts.entry_callback.as_ref()).map_err( process_entries_with_callback(bank, &entries, true, opts.entry_callback.as_ref(), None)
|err| { .map_err(|err| {
warn!( warn!(
"Failed to process entries for slot {}: {:?}", "Failed to process entries for slot {}: {:?}",
bank.slot(), bank.slot(),
err err
); );
BlocktreeProcessorError::InvalidTransaction BlocktreeProcessorError::InvalidTransaction
}, })?;
)?;
Ok(entries.last().unwrap().hash) Ok(entries.last().unwrap().hash)
} }
@ -508,6 +543,33 @@ fn process_pending_slots(
Ok(fork_info) Ok(fork_info)
} }
pub struct TransactionStatusBatch {
pub bank: Arc<Bank>,
pub transactions: Vec<Transaction>,
pub statuses: Vec<Result<()>>,
}
pub type TransactionStatusSender = Sender<TransactionStatusBatch>;
pub fn send_transaction_status_batch(
bank: Arc<Bank>,
transactions: &[Transaction],
statuses: Vec<Result<()>>,
transaction_status_sender: TransactionStatusSender,
) {
let slot = bank.slot();
if let Err(e) = transaction_status_sender.send(TransactionStatusBatch {
bank,
transactions: transactions.to_vec(),
statuses,
}) {
trace!(
"Slot {} transaction_status send batch failed: {:?}",
slot,
e
);
}
}
// used for tests only // used for tests only
pub fn fill_blocktree_slot_with_ticks( pub fn fill_blocktree_slot_with_ticks(
blocktree: &Blocktree, blocktree: &Blocktree,
@ -542,9 +604,11 @@ pub fn fill_blocktree_slot_with_ticks(
#[cfg(test)] #[cfg(test)]
pub mod tests { pub mod tests {
use super::*; use super::*;
use crate::entry::{create_ticks, next_entry, next_entry_mut}; use crate::{
use crate::genesis_utils::{ entry::{create_ticks, next_entry, next_entry_mut},
create_genesis_config, create_genesis_config_with_leader, GenesisConfigInfo, genesis_utils::{
create_genesis_config, create_genesis_config_with_leader, GenesisConfigInfo,
},
}; };
use matches::assert_matches; use matches::assert_matches;
use rand::{thread_rng, Rng}; use rand::{thread_rng, Rng};
@ -1094,7 +1158,7 @@ pub mod tests {
); );
// Now ensure the TX is accepted despite pointing to the ID of an empty entry. // Now ensure the TX is accepted despite pointing to the ID of an empty entry.
process_entries(&bank, &slot_entries, true).unwrap(); process_entries(&bank, &slot_entries, true, None).unwrap();
assert_eq!(bank.process_transaction(&tx), Ok(())); assert_eq!(bank.process_transaction(&tx), Ok(()));
} }
@ -1300,7 +1364,7 @@ pub mod tests {
// ensure bank can process a tick // ensure bank can process a tick
assert_eq!(bank.tick_height(), 0); assert_eq!(bank.tick_height(), 0);
let tick = next_entry(&genesis_config.hash(), 1, vec![]); let tick = next_entry(&genesis_config.hash(), 1, vec![]);
assert_eq!(process_entries(&bank, &[tick.clone()], true), Ok(())); assert_eq!(process_entries(&bank, &[tick.clone()], true, None), Ok(()));
assert_eq!(bank.tick_height(), 1); assert_eq!(bank.tick_height(), 1);
} }
@ -1332,7 +1396,10 @@ pub mod tests {
bank.last_blockhash(), bank.last_blockhash(),
); );
let entry_2 = next_entry(&entry_1.hash, 1, vec![tx]); let entry_2 = next_entry(&entry_1.hash, 1, vec![tx]);
assert_eq!(process_entries(&bank, &[entry_1, entry_2], true), Ok(())); assert_eq!(
process_entries(&bank, &[entry_1, entry_2], true, None),
Ok(())
);
assert_eq!(bank.get_balance(&keypair1.pubkey()), 2); assert_eq!(bank.get_balance(&keypair1.pubkey()), 2);
assert_eq!(bank.get_balance(&keypair2.pubkey()), 2); assert_eq!(bank.get_balance(&keypair2.pubkey()), 2);
assert_eq!(bank.last_blockhash(), blockhash); assert_eq!(bank.last_blockhash(), blockhash);
@ -1386,7 +1453,12 @@ pub mod tests {
); );
assert_eq!( assert_eq!(
process_entries(&bank, &[entry_1_to_mint, entry_2_to_3_mint_to_1], false), process_entries(
&bank,
&[entry_1_to_mint, entry_2_to_3_mint_to_1],
false,
None
),
Ok(()) Ok(())
); );
@ -1456,6 +1528,7 @@ pub mod tests {
&bank, &bank,
&[entry_1_to_mint.clone(), entry_2_to_3_mint_to_1.clone()], &[entry_1_to_mint.clone(), entry_2_to_3_mint_to_1.clone()],
false, false,
None,
) )
.is_err()); .is_err());
@ -1566,6 +1639,7 @@ pub mod tests {
entry_conflict_itself.clone() entry_conflict_itself.clone()
], ],
false, false,
None,
) )
.is_err()); .is_err());
@ -1612,7 +1686,10 @@ pub mod tests {
let tx = let tx =
system_transaction::transfer(&keypair2, &keypair4.pubkey(), 1, bank.last_blockhash()); system_transaction::transfer(&keypair2, &keypair4.pubkey(), 1, bank.last_blockhash());
let entry_2 = next_entry(&entry_1.hash, 1, vec![tx]); let entry_2 = next_entry(&entry_1.hash, 1, vec![tx]);
assert_eq!(process_entries(&bank, &[entry_1, entry_2], true), Ok(())); assert_eq!(
process_entries(&bank, &[entry_1, entry_2], true, None),
Ok(())
);
assert_eq!(bank.get_balance(&keypair3.pubkey()), 1); assert_eq!(bank.get_balance(&keypair3.pubkey()), 1);
assert_eq!(bank.get_balance(&keypair4.pubkey()), 1); assert_eq!(bank.get_balance(&keypair4.pubkey()), 1);
assert_eq!(bank.last_blockhash(), blockhash); assert_eq!(bank.last_blockhash(), blockhash);
@ -1670,7 +1747,7 @@ pub mod tests {
next_entry_mut(&mut hash, 0, transactions) next_entry_mut(&mut hash, 0, transactions)
}) })
.collect(); .collect();
assert_eq!(process_entries(&bank, &entries, true), Ok(())); assert_eq!(process_entries(&bank, &entries, true, None), Ok(()));
} }
#[test] #[test]
@ -1730,7 +1807,7 @@ pub mod tests {
// Transfer lamports to each other // Transfer lamports to each other
let entry = next_entry(&bank.last_blockhash(), 1, tx_vector); let entry = next_entry(&bank.last_blockhash(), 1, tx_vector);
assert_eq!(process_entries(&bank, &vec![entry], true), Ok(())); assert_eq!(process_entries(&bank, &vec![entry], true, None), Ok(()));
bank.squash(); bank.squash();
// Even number keypair should have balance of 2 * initial_lamports and // Even number keypair should have balance of 2 * initial_lamports and
@ -1794,7 +1871,8 @@ pub mod tests {
process_entries( process_entries(
&bank, &bank,
&[entry_1.clone(), tick.clone(), entry_2.clone()], &[entry_1.clone(), tick.clone(), entry_2.clone()],
true true,
None
), ),
Ok(()) Ok(())
); );
@ -1806,7 +1884,7 @@ pub mod tests {
system_transaction::transfer(&keypair2, &keypair3.pubkey(), 1, bank.last_blockhash()); system_transaction::transfer(&keypair2, &keypair3.pubkey(), 1, bank.last_blockhash());
let entry_3 = next_entry(&entry_2.hash, 1, vec![tx]); let entry_3 = next_entry(&entry_2.hash, 1, vec![tx]);
assert_eq!( assert_eq!(
process_entries(&bank, &[entry_3], true), process_entries(&bank, &[entry_3], true, None),
Err(TransactionError::AccountNotFound) Err(TransactionError::AccountNotFound)
); );
} }
@ -1886,7 +1964,7 @@ pub mod tests {
); );
assert_eq!( assert_eq!(
process_entries(&bank, &[entry_1_to_mint], false), process_entries(&bank, &[entry_1_to_mint], false, None),
Err(TransactionError::AccountInUse) Err(TransactionError::AccountInUse)
); );
@ -2030,7 +2108,7 @@ pub mod tests {
}) })
.collect(); .collect();
info!("paying iteration {}", i); info!("paying iteration {}", i);
process_entries(&bank, &entries, true).expect("paying failed"); process_entries(&bank, &entries, true, None).expect("paying failed");
let entries: Vec<_> = (0..NUM_TRANSFERS) let entries: Vec<_> = (0..NUM_TRANSFERS)
.step_by(NUM_TRANSFERS_PER_ENTRY) .step_by(NUM_TRANSFERS_PER_ENTRY)
@ -2053,7 +2131,7 @@ pub mod tests {
.collect(); .collect();
info!("refunding iteration {}", i); info!("refunding iteration {}", i);
process_entries(&bank, &entries, true).expect("refunding failed"); process_entries(&bank, &entries, true, None).expect("refunding failed");
// advance to next block // advance to next block
process_entries( process_entries(
@ -2062,6 +2140,7 @@ pub mod tests {
.map(|_| next_entry_mut(&mut hash, 1, vec![])) .map(|_| next_entry_mut(&mut hash, 1, vec![]))
.collect::<Vec<_>>(), .collect::<Vec<_>>(),
true, true,
None,
) )
.expect("process ticks failed"); .expect("process ticks failed");
@ -2102,7 +2181,7 @@ pub mod tests {
let entry = next_entry(&new_blockhash, 1, vec![tx]); let entry = next_entry(&new_blockhash, 1, vec![tx]);
entries.push(entry); entries.push(entry);
process_entries_with_callback(&bank0, &entries, true, None).unwrap(); process_entries_with_callback(&bank0, &entries, true, None, None).unwrap();
assert_eq!(bank0.get_balance(&keypair.pubkey()), 1) assert_eq!(bank0.get_balance(&keypair.pubkey()), 1)
} }

View File

@ -196,6 +196,8 @@ impl LocalCluster {
let leader_contact_info = leader_node.info.clone(); let leader_contact_info = leader_node.info.clone();
let leader_storage_keypair = Arc::new(storage_keypair); let leader_storage_keypair = Arc::new(storage_keypair);
let leader_voting_keypair = Arc::new(voting_keypair); let leader_voting_keypair = Arc::new(voting_keypair);
let mut leader_config = config.validator_configs[0].clone();
leader_config.transaction_status_service_disabled = true;
let leader_server = Validator::new( let leader_server = Validator::new(
leader_node, leader_node,
&leader_keypair, &leader_keypair,
@ -205,7 +207,7 @@ impl LocalCluster {
&leader_storage_keypair, &leader_storage_keypair,
None, None,
true, true,
&config.validator_configs[0], &leader_config,
); );
let mut validators = HashMap::new(); let mut validators = HashMap::new();
@ -327,6 +329,8 @@ impl LocalCluster {
.unwrap(); .unwrap();
} }
let mut config = validator_config.clone();
config.transaction_status_service_disabled = true;
let voting_keypair = Arc::new(voting_keypair); let voting_keypair = Arc::new(voting_keypair);
let validator_server = Validator::new( let validator_server = Validator::new(
validator_node, validator_node,
@ -337,7 +341,7 @@ impl LocalCluster {
&storage_keypair, &storage_keypair,
Some(&self.entry_point_info), Some(&self.entry_point_info),
true, true,
&validator_config, &config,
); );
self.validators self.validators
@ -637,6 +641,9 @@ impl Cluster for LocalCluster {
// Update the stored ContactInfo for this node // Update the stored ContactInfo for this node
let node = Node::new_localhost_with_pubkey(&pubkey); let node = Node::new_localhost_with_pubkey(&pubkey);
cluster_validator_info.info.contact_info = node.info.clone(); cluster_validator_info.info.contact_info = node.info.clone();
cluster_validator_info
.config
.transaction_status_service_disabled = true;
let entry_point_info = { let entry_point_info = {
if *pubkey == self.entry_point_info.id { if *pubkey == self.entry_point_info.id {

View File

@ -153,6 +153,11 @@ impl StatusCacheRc {
pub type EnteredEpochCallback = Box<dyn Fn(&mut Bank) -> () + Sync + Send>; pub type EnteredEpochCallback = Box<dyn Fn(&mut Bank) -> () + Sync + Send>;
pub struct TransactionResults {
pub fee_collection_results: Vec<Result<()>>,
pub processing_results: Vec<Result<()>>,
}
/// Manager for the state of all accounts and programs after processing its entries. /// Manager for the state of all accounts and programs after processing its entries.
#[derive(Default, Deserialize, Serialize)] #[derive(Default, Deserialize, Serialize)]
pub struct Bank { pub struct Bank {
@ -745,6 +750,11 @@ impl Bank {
) )
} }
pub fn get_fee_calculator(&self, hash: &Hash) -> Option<FeeCalculator> {
let blockhash_queue = self.blockhash_queue.read().unwrap();
blockhash_queue.get_fee_calculator(hash).cloned()
}
pub fn confirmed_last_blockhash(&self) -> (Hash, FeeCalculator) { pub fn confirmed_last_blockhash(&self) -> (Hash, FeeCalculator) {
const NUM_BLOCKHASH_CONFIRMATIONS: usize = 3; const NUM_BLOCKHASH_CONFIRMATIONS: usize = 3;
@ -1153,7 +1163,7 @@ impl Bank {
executed: &[Result<()>], executed: &[Result<()>],
tx_count: u64, tx_count: u64,
signature_count: u64, signature_count: u64,
) -> Vec<Result<()>> { ) -> TransactionResults {
assert!( assert!(
!self.is_frozen(), !self.is_frozen(),
"commit_transactions() working on a frozen bank!" "commit_transactions() working on a frozen bank!"
@ -1186,7 +1196,12 @@ impl Bank {
write_time.stop(); write_time.stop();
debug!("store: {}us txs_len={}", write_time.as_us(), txs.len(),); debug!("store: {}us txs_len={}", write_time.as_us(), txs.len(),);
self.update_transaction_statuses(txs, iteration_order, &executed); self.update_transaction_statuses(txs, iteration_order, &executed);
self.filter_program_errors_and_collect_fee(txs, iteration_order, executed) let fee_collection_results =
self.filter_program_errors_and_collect_fee(txs, iteration_order, executed);
TransactionResults {
fee_collection_results,
processing_results: executed.to_vec(),
}
} }
fn distribute_rent(&self) { fn distribute_rent(&self) {
@ -1223,7 +1238,7 @@ impl Bank {
&self, &self,
batch: &TransactionBatch, batch: &TransactionBatch,
max_age: usize, max_age: usize,
) -> Vec<Result<()>> { ) -> TransactionResults {
let (mut loaded_accounts, executed, _, tx_count, signature_count) = let (mut loaded_accounts, executed, _, tx_count, signature_count) =
self.load_and_execute_transactions(batch, max_age); self.load_and_execute_transactions(batch, max_age);
@ -1241,6 +1256,7 @@ impl Bank {
pub fn process_transactions(&self, txs: &[Transaction]) -> Vec<Result<()>> { pub fn process_transactions(&self, txs: &[Transaction]) -> Vec<Result<()>> {
let batch = self.prepare_batch(txs, None); let batch = self.prepare_batch(txs, None);
self.load_execute_and_commit_transactions(&batch, MAX_RECENT_BLOCKHASHES) self.load_execute_and_commit_transactions(&batch, MAX_RECENT_BLOCKHASHES)
.fee_collection_results
} }
/// Create, sign, and process a Transaction from `keypair` to `to` of /// Create, sign, and process a Transaction from `keypair` to `to` of
@ -2786,8 +2802,9 @@ mod tests {
let pay_alice = vec![tx1]; let pay_alice = vec![tx1];
let lock_result = bank.prepare_batch(&pay_alice, None); let lock_result = bank.prepare_batch(&pay_alice, None);
let results_alice = let results_alice = bank
bank.load_execute_and_commit_transactions(&lock_result, MAX_RECENT_BLOCKHASHES); .load_execute_and_commit_transactions(&lock_result, MAX_RECENT_BLOCKHASHES)
.fee_collection_results;
assert_eq!(results_alice[0], Ok(())); assert_eq!(results_alice[0], Ok(()));
// try executing an interleaved transfer twice // try executing an interleaved transfer twice