From d554cec35f3f35a727d24f896e840dc704bbd5ec Mon Sep 17 00:00:00 2001 From: lewis Date: Mon, 14 Oct 2024 16:20:57 +0800 Subject: [PATCH] feat: prio-graph crate remove dependency of solana-core --- Cargo.lock | 9 +- prio-graph-scheduler/Cargo.toml | 17 +- .../src/deserializable_packet.rs | 44 --- prio-graph-scheduler/src/lib.rs | 186 +++++++++++- .../src/prio_graph_scheduler.rs | 21 +- .../src/read_write_account_set.rs | 287 ++++++++++++++++++ prio-graph-scheduler/src/transaction_state.rs | 10 +- .../src/transaction_state_container.rs | 8 +- 8 files changed, 507 insertions(+), 75 deletions(-) create mode 100644 prio-graph-scheduler/src/read_write_account_set.rs diff --git a/Cargo.lock b/Cargo.lock index cb5ebbf4ef1ea3..10f8bb02973ee8 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7241,20 +7241,27 @@ dependencies = [ "ahash 0.8.10", "arrayvec", "assert_matches", + "bincode", "crossbeam-channel", "itertools 0.12.1", "log", "min-max-heap", "prio-graph", - "solana-core", + "solana-compute-budget", "solana-cost-model", "solana-gossip", "solana-ledger", "solana-measure", "solana-metrics", + "solana-perf", "solana-poh", + "solana-prio-graph-scheduler", "solana-runtime", + "solana-runtime-transaction", + "solana-sanitize", "solana-sdk", + "solana-short-vec", + "solana-svm-transaction", "thiserror", ] diff --git a/prio-graph-scheduler/Cargo.toml b/prio-graph-scheduler/Cargo.toml index 47706ad7f40b1b..1316a486bda9cb 100644 --- a/prio-graph-scheduler/Cargo.toml +++ b/prio-graph-scheduler/Cargo.toml @@ -10,7 +10,6 @@ license.workspace = true edition.workspace = true [dependencies] -solana-core = { workspace = true } solana-sdk = { workspace = true } solana-poh = { workspace = true } solana-metrics = { workspace = true } @@ -31,9 +30,25 @@ min-max-heap = { workspace = true } [dev-dependencies] assert_matches = { workspace = true } +solana-compute-budget = { workspace = true } +solana-perf = { workspace = true } +solana-runtime-transaction = { workspace = true } +solana-sanitize = { workspace = true } +solana-short-vec = { workspace = true } +solana-svm-transaction = { workspace = true } +# let dev-context-only-utils works when running this crate. +solana-prio-graph-scheduler = { path = ".", features = [ + "dev-context-only-utils", +] } +solana-sdk = { workspace = true, features = ["dev-context-only-utils"] } + +bincode = { workspace = true } [package.metadata.docs.rs] targets = ["x86_64-unknown-linux-gnu"] [lints] workspace = true + +[features] +dev-context-only-utils = ["solana-runtime/dev-context-only-utils"] diff --git a/prio-graph-scheduler/src/deserializable_packet.rs b/prio-graph-scheduler/src/deserializable_packet.rs index 79c2a834666d41..0f54b0ec6de047 100644 --- a/prio-graph-scheduler/src/deserializable_packet.rs +++ b/prio-graph-scheduler/src/deserializable_packet.rs @@ -1,5 +1,4 @@ use std::collections::HashSet; -use solana_core::banking_stage::immutable_deserialized_packet::{DeserializedPacketError, ImmutableDeserializedPacket}; use solana_sdk::hash::Hash; use solana_sdk::message::AddressLoader; use solana_sdk::packet::Packet; @@ -37,47 +36,4 @@ pub trait DeserializableTxPacket: PartialEq + PartialOrd + Eq + Sized { fn compute_unit_price(&self) -> u64; fn compute_unit_limit(&self) -> u64; -} - - -/// TODO: migrate to solana_core -impl DeserializableTxPacket for ImmutableDeserializedPacket { - type DeserializeError = DeserializedPacketError; - - fn from_packet(packet: Packet) -> Result { - ImmutableDeserializedPacket::new(packet) - } - - fn build_sanitized_transaction( - &self, - votes_only: bool, - address_loader: impl AddressLoader, - reserved_account_keys: &HashSet, - ) -> Option { - self.build_sanitized_transaction(votes_only, address_loader, reserved_account_keys) - } - - fn original_packet(&self) -> &Packet { - &self.original_packet - } - - fn transaction(&self) -> &SanitizedVersionedTransaction { - &self.transaction - } - - fn message_hash(&self) -> &Hash { - &self.message_hash - } - - fn is_simple_vote(&self) -> bool { - self.is_simple_vote - } - - fn compute_unit_price(&self) -> u64 { - self.compute_unit_price - } - - fn compute_unit_limit(&self) -> u64 { - u64::from(self.compute_unit_limit) - } } \ No newline at end of file diff --git a/prio-graph-scheduler/src/lib.rs b/prio-graph-scheduler/src/lib.rs index 215c451f5e33a2..8e4ddb9d76bb45 100644 --- a/prio-graph-scheduler/src/lib.rs +++ b/prio-graph-scheduler/src/lib.rs @@ -1,20 +1,192 @@ //! Solana Priority Graph Scheduler. -pub mod transaction_state; -pub mod scheduler_messages; pub mod id_generator; pub mod in_flight_tracker; -pub mod thread_aware_account_locks; -pub mod transaction_priority_id; pub mod scheduler_error; +pub mod scheduler_messages; pub mod scheduler_metrics; +pub mod thread_aware_account_locks; +pub mod transaction_priority_id; +pub mod transaction_state; // pub mod scheduler_controller; -pub mod transaction_state_container; -pub mod prio_graph_scheduler; pub mod deserializable_packet; +pub mod prio_graph_scheduler; +pub mod transaction_state_container; #[macro_use] extern crate solana_metrics; #[cfg(test)] #[macro_use] -extern crate assert_matches; \ No newline at end of file +extern crate assert_matches; + +/// Consumer will create chunks of transactions from buffer with up to this size. +pub const TARGET_NUM_TRANSACTIONS_PER_BATCH: usize = 64; + +mod read_write_account_set; + +#[cfg(test)] +mod tests { + use { + crate::deserializable_packet::DeserializableTxPacket, + solana_compute_budget::compute_budget_limits::ComputeBudgetLimits, + solana_perf::packet::Packet, + solana_runtime_transaction::instructions_processor::process_compute_budget_instructions, + solana_sanitize::SanitizeError, + solana_sdk::{ + hash::Hash, + message::Message, + pubkey::Pubkey, + signature::Signature, + transaction::{ + AddressLoader, SanitizedTransaction, SanitizedVersionedTransaction, + VersionedTransaction, + }, + }, + solana_short_vec::decode_shortu16_len, + solana_svm_transaction::instruction::SVMInstruction, + std::{cmp::Ordering, collections::HashSet, mem::size_of}, + thiserror::Error, + }; + + #[derive(Debug, Error)] + pub enum MockDeserializedPacketError { + #[error("ShortVec Failed to Deserialize")] + // short_vec::decode_shortu16_len() currently returns () on error + ShortVecError(()), + #[error("Deserialization Error: {0}")] + DeserializationError(#[from] bincode::Error), + #[error("overflowed on signature size {0}")] + SignatureOverflowed(usize), + #[error("packet failed sanitization {0}")] + SanitizeError(#[from] SanitizeError), + #[error("transaction failed prioritization")] + PrioritizationFailure, + } + + #[derive(Debug, Eq)] + pub struct MockImmutableDeserializedPacket { + pub original_packet: Packet, + pub transaction: SanitizedVersionedTransaction, + pub message_hash: Hash, + pub is_simple_vote: bool, + pub compute_unit_price: u64, + pub compute_unit_limit: u32, + } + + impl DeserializableTxPacket for MockImmutableDeserializedPacket { + type DeserializeError = MockDeserializedPacketError; + fn from_packet(packet: Packet) -> Result { + let versioned_transaction: VersionedTransaction = packet.deserialize_slice(..)?; + let sanitized_transaction = + SanitizedVersionedTransaction::try_from(versioned_transaction)?; + let message_bytes = packet_message(&packet)?; + let message_hash = Message::hash_raw_message(message_bytes); + let is_simple_vote = packet.meta().is_simple_vote_tx(); + + // drop transaction if prioritization fails. + let ComputeBudgetLimits { + mut compute_unit_price, + compute_unit_limit, + .. + } = process_compute_budget_instructions( + sanitized_transaction + .get_message() + .program_instructions_iter() + .map(|(pubkey, ix)| (pubkey, SVMInstruction::from(ix))), + ) + .map_err(|_| MockDeserializedPacketError::PrioritizationFailure)?; + + // set compute unit price to zero for vote transactions + if is_simple_vote { + compute_unit_price = 0; + }; + + Ok(Self { + original_packet: packet, + transaction: sanitized_transaction, + message_hash, + is_simple_vote, + compute_unit_price, + compute_unit_limit, + }) + } + + fn original_packet(&self) -> &Packet { + &self.original_packet + } + + fn transaction(&self) -> &SanitizedVersionedTransaction { + &self.transaction + } + + fn message_hash(&self) -> &Hash { + &self.message_hash + } + + fn is_simple_vote(&self) -> bool { + self.is_simple_vote + } + + fn compute_unit_price(&self) -> u64 { + self.compute_unit_price + } + + fn compute_unit_limit(&self) -> u64 { + u64::from(self.compute_unit_limit) + } + + // This function deserializes packets into transactions, computes the blake3 hash of transaction + // messages. + fn build_sanitized_transaction( + &self, + votes_only: bool, + address_loader: impl AddressLoader, + reserved_account_keys: &HashSet, + ) -> Option { + if votes_only && !self.is_simple_vote() { + return None; + } + let tx = SanitizedTransaction::try_new( + self.transaction().clone(), + *self.message_hash(), + self.is_simple_vote(), + address_loader, + reserved_account_keys, + ) + .ok()?; + Some(tx) + } + } + + // PartialEq MUST be consistent with PartialOrd and Ord + impl PartialEq for MockImmutableDeserializedPacket { + fn eq(&self, other: &Self) -> bool { + self.compute_unit_price() == other.compute_unit_price() + } + } + + impl PartialOrd for MockImmutableDeserializedPacket { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } + } + + impl Ord for MockImmutableDeserializedPacket { + fn cmp(&self, other: &Self) -> Ordering { + self.compute_unit_price().cmp(&other.compute_unit_price()) + } + } + + /// Read the transaction message from packet data + fn packet_message(packet: &Packet) -> Result<&[u8], MockDeserializedPacketError> { + let (sig_len, sig_size) = packet + .data(..) + .and_then(|bytes| decode_shortu16_len(bytes).ok()) + .ok_or(MockDeserializedPacketError::ShortVecError(()))?; + sig_len + .checked_mul(size_of::()) + .and_then(|v| v.checked_add(sig_size)) + .and_then(|msg_start| packet.data(msg_start..)) + .ok_or(MockDeserializedPacketError::SignatureOverflowed(sig_size)) + } +} diff --git a/prio-graph-scheduler/src/prio_graph_scheduler.rs b/prio-graph-scheduler/src/prio_graph_scheduler.rs index 4d66a7901d4976..209e0799951d22 100644 --- a/prio-graph-scheduler/src/prio_graph_scheduler.rs +++ b/prio-graph-scheduler/src/prio_graph_scheduler.rs @@ -2,19 +2,18 @@ use { crate::{ deserializable_packet::DeserializableTxPacket, in_flight_tracker::InFlightTracker, + read_write_account_set::ReadWriteAccountSet, scheduler_error::SchedulerError, scheduler_messages::{ConsumeWork, FinishedConsumeWork, TransactionBatchId, TransactionId}, thread_aware_account_locks::{ThreadAwareAccountLocks, ThreadId, ThreadSet}, transaction_priority_id::TransactionPriorityId, transaction_state::{SanitizedTransactionTTL, TransactionState}, transaction_state_container::TransactionStateContainer, + TARGET_NUM_TRANSACTIONS_PER_BATCH, }, crossbeam_channel::{Receiver, Sender, TryRecvError}, itertools::izip, prio_graph::{AccessKind, PrioGraph}, - solana_core::banking_stage::{ - consumer::TARGET_NUM_TRANSACTIONS_PER_BATCH, read_write_account_set::ReadWriteAccountSet, - }, solana_cost_model::block_cost_limits::MAX_BLOCK_UNITS, solana_measure::measure_us, solana_sdk::{ @@ -592,12 +591,10 @@ fn try_schedule_transaction( mod tests { use { super::*, + crate::tests::MockImmutableDeserializedPacket, + crate::TARGET_NUM_TRANSACTIONS_PER_BATCH, crossbeam_channel::{unbounded, Receiver}, itertools::Itertools, - solana_core::banking_stage::{ - consumer::TARGET_NUM_TRANSACTIONS_PER_BATCH, - immutable_deserialized_packet::ImmutableDeserializedPacket, - }, solana_sdk::{ compute_budget::ComputeBudgetInstruction, hash::Hash, message::Message, packet::Packet, pubkey::Pubkey, signature::Keypair, signer::Signer, system_instruction, @@ -621,14 +618,14 @@ mod tests { fn create_test_frame( num_threads: usize, ) -> ( - PrioGraphScheduler, + PrioGraphScheduler, Vec>, Sender, ) { let (consume_work_senders, consume_work_receivers) = (0..num_threads).map(|_| unbounded()).unzip(); let (finished_consume_work_sender, finished_consume_work_receiver) = unbounded(); - let scheduler = PrioGraphScheduler::::new( + let scheduler = PrioGraphScheduler::::new( consume_work_senders, finished_consume_work_receiver, ); @@ -668,9 +665,9 @@ mod tests { u64, ), >, - ) -> TransactionStateContainer { + ) -> TransactionStateContainer { let mut container = - TransactionStateContainer::::with_capacity(10 * 1024); + TransactionStateContainer::::with_capacity(10 * 1024); for (index, (from_keypair, to_pubkeys, lamports, compute_unit_price)) in tx_infos.into_iter().enumerate() { @@ -682,7 +679,7 @@ mod tests { compute_unit_price, ); let packet = Arc::new( - ImmutableDeserializedPacket::new( + MockImmutableDeserializedPacket::from_packet( Packet::from_data(None, transaction.to_versioned_transaction()).unwrap(), ) .unwrap(), diff --git a/prio-graph-scheduler/src/read_write_account_set.rs b/prio-graph-scheduler/src/read_write_account_set.rs new file mode 100644 index 00000000000000..ea5fc0ffa1a13e --- /dev/null +++ b/prio-graph-scheduler/src/read_write_account_set.rs @@ -0,0 +1,287 @@ +use { + ahash::AHashSet, + solana_sdk::{message::SanitizedMessage, pubkey::Pubkey}, +}; + +/// Wrapper struct to accumulate locks for a batch of transactions. +#[derive(Debug, Default)] +pub struct ReadWriteAccountSet { + /// Set of accounts that are locked for read + read_set: AHashSet, + /// Set of accounts that are locked for write + write_set: AHashSet, +} + +impl ReadWriteAccountSet { + /// Returns true if all account locks were available and false otherwise. + pub fn check_locks(&self, message: &SanitizedMessage) -> bool { + message + .account_keys() + .iter() + .enumerate() + .all(|(index, pubkey)| { + if message.is_writable(index) { + self.can_write(pubkey) + } else { + self.can_read(pubkey) + } + }) + } + + /// Add all account locks. + /// Returns true if all account locks were available and false otherwise. + pub fn take_locks(&mut self, message: &SanitizedMessage) -> bool { + message + .account_keys() + .iter() + .enumerate() + .fold(true, |all_available, (index, pubkey)| { + if message.is_writable(index) { + all_available & self.add_write(pubkey) + } else { + all_available & self.add_read(pubkey) + } + }) + } + + /// Clears the read and write sets + #[allow(dead_code)] + pub fn clear(&mut self) { + self.read_set.clear(); + self.write_set.clear(); + } + + /// Check if an account can be read-locked + fn can_read(&self, pubkey: &Pubkey) -> bool { + !self.write_set.contains(pubkey) + } + + /// Check if an account can be write-locked + fn can_write(&self, pubkey: &Pubkey) -> bool { + !self.write_set.contains(pubkey) && !self.read_set.contains(pubkey) + } + + /// Add an account to the read-set. + /// Returns true if the lock was available. + fn add_read(&mut self, pubkey: &Pubkey) -> bool { + let can_read = self.can_read(pubkey); + self.read_set.insert(*pubkey); + + can_read + } + + /// Add an account to the write-set. + /// Returns true if the lock was available. + fn add_write(&mut self, pubkey: &Pubkey) -> bool { + let can_write = self.can_write(pubkey); + self.write_set.insert(*pubkey); + + can_write + } +} + +#[cfg(test)] +mod tests { + use { + super::ReadWriteAccountSet, + solana_ledger::genesis_utils::GenesisConfigInfo, + solana_runtime::{bank::Bank, bank_forks::BankForks, genesis_utils::create_genesis_config}, + solana_sdk::{ + account::AccountSharedData, + address_lookup_table::{ + self, + state::{AddressLookupTable, LookupTableMeta}, + }, + hash::Hash, + message::{ + v0::{self, MessageAddressTableLookup}, + MessageHeader, VersionedMessage, + }, + pubkey::Pubkey, + signature::Keypair, + signer::Signer, + transaction::{MessageHash, SanitizedTransaction, VersionedTransaction}, + }, + std::{ + borrow::Cow, + sync::{Arc, RwLock}, + }, + }; + + fn create_test_versioned_message( + write_keys: &[Pubkey], + read_keys: &[Pubkey], + address_table_lookups: Vec, + ) -> VersionedMessage { + VersionedMessage::V0(v0::Message { + header: MessageHeader { + num_required_signatures: write_keys.len() as u8, + num_readonly_signed_accounts: 0, + num_readonly_unsigned_accounts: read_keys.len() as u8, + }, + recent_blockhash: Hash::default(), + account_keys: write_keys.iter().chain(read_keys.iter()).copied().collect(), + address_table_lookups, + instructions: vec![], + }) + } + + fn create_test_sanitized_transaction( + write_keypair: &Keypair, + read_keys: &[Pubkey], + address_table_lookups: Vec, + bank: &Bank, + ) -> SanitizedTransaction { + let message = create_test_versioned_message( + &[write_keypair.pubkey()], + read_keys, + address_table_lookups, + ); + SanitizedTransaction::try_create( + VersionedTransaction::try_new(message, &[write_keypair]).unwrap(), + MessageHash::Compute, + Some(false), + bank, + bank.get_reserved_account_keys(), + ) + .unwrap() + } + + fn create_test_address_lookup_table( + bank: Arc, + num_addresses: usize, + ) -> (Arc, Pubkey) { + let mut addresses = Vec::with_capacity(num_addresses); + addresses.resize_with(num_addresses, Pubkey::new_unique); + let address_lookup_table = AddressLookupTable { + meta: LookupTableMeta { + authority: None, + ..LookupTableMeta::default() + }, + addresses: Cow::Owned(addresses), + }; + + let address_table_key = Pubkey::new_unique(); + let data = address_lookup_table.serialize_for_tests().unwrap(); + let mut account = + AccountSharedData::new(1, data.len(), &address_lookup_table::program::id()); + account.set_data(data); + bank.store_account(&address_table_key, &account); + + let slot = bank.slot() + 1; + ( + Arc::new(Bank::new_from_parent(bank, &Pubkey::new_unique(), slot)), + address_table_key, + ) + } + + fn create_test_bank() -> (Arc, Arc>) { + let GenesisConfigInfo { genesis_config, .. } = create_genesis_config(10_000); + Bank::new_no_wallclock_throttle_for_tests(&genesis_config) + } + + // Helper function (could potentially use test_case in future). + // conflict_index = 0 means write lock conflict with static key + // conflict_index = 1 means read lock conflict with static key + // conflict_index = 2 means write lock conflict with address table key + // conflict_index = 3 means read lock conflict with address table key + fn test_check_and_take_locks(conflict_index: usize, add_write: bool, expectation: bool) { + let (bank, _bank_forks) = create_test_bank(); + let (bank, table_address) = create_test_address_lookup_table(bank, 2); + let tx = create_test_sanitized_transaction( + &Keypair::new(), + &[Pubkey::new_unique()], + vec![MessageAddressTableLookup { + account_key: table_address, + writable_indexes: vec![0], + readonly_indexes: vec![1], + }], + &bank, + ); + let message = tx.message(); + + let mut account_locks = ReadWriteAccountSet::default(); + + let conflict_key = message.account_keys().get(conflict_index).unwrap(); + if add_write { + account_locks.add_write(conflict_key); + } else { + account_locks.add_read(conflict_key); + } + assert_eq!(expectation, account_locks.check_locks(message)); + assert_eq!(expectation, account_locks.take_locks(message)); + } + + #[test] + fn test_check_and_take_locks_write_write_conflict() { + test_check_and_take_locks(0, true, false); // static key conflict + test_check_and_take_locks(2, true, false); // lookup key conflict + } + + #[test] + fn test_check_and_take_locks_read_write_conflict() { + test_check_and_take_locks(0, false, false); // static key conflict + test_check_and_take_locks(2, false, false); // lookup key conflict + } + + #[test] + fn test_check_and_take_locks_write_read_conflict() { + test_check_and_take_locks(1, true, false); // static key conflict + test_check_and_take_locks(3, true, false); // lookup key conflict + } + + #[test] + fn test_check_and_take_locks_read_read_non_conflict() { + test_check_and_take_locks(1, false, true); // static key conflict + test_check_and_take_locks(3, false, true); // lookup key conflict + } + + #[test] + pub fn test_write_write_conflict() { + let mut account_locks = ReadWriteAccountSet::default(); + let account = Pubkey::new_unique(); + assert!(account_locks.can_write(&account)); + account_locks.add_write(&account); + assert!(!account_locks.can_write(&account)); + } + + #[test] + pub fn test_read_write_conflict() { + let mut account_locks = ReadWriteAccountSet::default(); + let account = Pubkey::new_unique(); + assert!(account_locks.can_read(&account)); + account_locks.add_read(&account); + assert!(!account_locks.can_write(&account)); + assert!(account_locks.can_read(&account)); + } + + #[test] + pub fn test_write_read_conflict() { + let mut account_locks = ReadWriteAccountSet::default(); + let account = Pubkey::new_unique(); + assert!(account_locks.can_write(&account)); + account_locks.add_write(&account); + assert!(!account_locks.can_write(&account)); + assert!(!account_locks.can_read(&account)); + } + + #[test] + pub fn test_read_read_non_conflict() { + let mut account_locks = ReadWriteAccountSet::default(); + let account = Pubkey::new_unique(); + assert!(account_locks.can_read(&account)); + account_locks.add_read(&account); + assert!(account_locks.can_read(&account)); + } + + #[test] + pub fn test_write_write_different_keys() { + let mut account_locks = ReadWriteAccountSet::default(); + let account1 = Pubkey::new_unique(); + let account2 = Pubkey::new_unique(); + assert!(account_locks.can_write(&account1)); + account_locks.add_write(&account1); + assert!(account_locks.can_write(&account2)); + assert!(account_locks.can_read(&account2)); + } +} diff --git a/prio-graph-scheduler/src/transaction_state.rs b/prio-graph-scheduler/src/transaction_state.rs index 56575beeaf79a4..422f4c1f8c6506 100644 --- a/prio-graph-scheduler/src/transaction_state.rs +++ b/prio-graph-scheduler/src/transaction_state.rs @@ -205,17 +205,15 @@ impl TransactionState

{ #[cfg(test)] mod tests { use { - super::*, - solana_core::banking_stage::immutable_deserialized_packet::ImmutableDeserializedPacket, - solana_sdk::{ + super::*, crate::tests::MockImmutableDeserializedPacket, solana_sdk::{ compute_budget::ComputeBudgetInstruction, hash::Hash, message::Message, packet::Packet, signature::Keypair, signer::Signer, system_instruction, transaction::Transaction, - }, + } }; fn create_transaction_state( compute_unit_price: u64, - ) -> TransactionState { + ) -> TransactionState { let from_keypair = Keypair::new(); let ixs = vec![ system_instruction::transfer( @@ -229,7 +227,7 @@ mod tests { let tx = Transaction::new(&[&from_keypair], message, Hash::default()); let packet = Arc::new( - ImmutableDeserializedPacket::new(Packet::from_data(None, tx.clone()).unwrap()).unwrap(), + MockImmutableDeserializedPacket::from_packet(Packet::from_data(None, tx.clone()).unwrap()).unwrap(), ); let transaction_ttl = SanitizedTransactionTTL { transaction: SanitizedTransaction::from_transaction_for_tests(tx), diff --git a/prio-graph-scheduler/src/transaction_state_container.rs b/prio-graph-scheduler/src/transaction_state_container.rs index c162034f6f890d..6e6ad444eea977 100644 --- a/prio-graph-scheduler/src/transaction_state_container.rs +++ b/prio-graph-scheduler/src/transaction_state_container.rs @@ -146,7 +146,7 @@ impl TransactionStateContainer

{ #[cfg(test)] mod tests { use { - super::*, solana_core::banking_stage::immutable_deserialized_packet::ImmutableDeserializedPacket, solana_sdk::{ + super::*, crate::tests::MockImmutableDeserializedPacket, solana_sdk::{ compute_budget::ComputeBudgetInstruction, hash::Hash, message::Message, @@ -164,7 +164,7 @@ mod tests { priority: u64, ) -> ( SanitizedTransactionTTL, - Arc, + Arc, u64, u64, ) { @@ -184,7 +184,7 @@ mod tests { Hash::default(), )); let packet = Arc::new( - ImmutableDeserializedPacket::new( + MockImmutableDeserializedPacket::from_packet( Packet::from_data(None, tx.to_versioned_transaction()).unwrap(), ) .unwrap(), @@ -198,7 +198,7 @@ mod tests { } fn push_to_container( - container: &mut TransactionStateContainer, + container: &mut TransactionStateContainer, num: usize, ) { for id in 0..num as u64 {