فهرست منبع

Support priority ordering for bp unified scheduler (#7444)

* Support priority ordering for bp unified scheduler

* Just use bank.feature_set instead of FEATURE_SET

* Document blocked_usage_count/try_{un,re}block

* Explicitly mention about no use of with_capacity()

* Just use an ||

* Strip _usage_from_task suffixes from names

* Explain assertion

* Rename to OrderedTaskId from Index

* Define some task id helper methods

* Use panic!()s

* Skip re-iterating for priority with existing data

* Document PriorityUsage for Readonly(current_tasks)
Ryo Onodera 1 ماه پیش
والد
کامیت
164ecdf9dc

+ 1 - 0
Cargo.lock

@@ -11663,6 +11663,7 @@ dependencies = [
  "solana-runtime-transaction",
  "solana-runtime-transaction",
  "solana-transaction",
  "solana-transaction",
  "static_assertions",
  "static_assertions",
+ "test-case",
  "unwrap_none",
  "unwrap_none",
 ]
 ]
 
 

+ 1 - 1
core/Cargo.toml

@@ -159,6 +159,7 @@ solana-transaction = { workspace = true }
 solana-transaction-error = { workspace = true }
 solana-transaction-error = { workspace = true }
 solana-transaction-status = { workspace = true }
 solana-transaction-status = { workspace = true }
 solana-turbine = { workspace = true, features = ["agave-unstable-api"] }
 solana-turbine = { workspace = true, features = ["agave-unstable-api"] }
+solana-unified-scheduler-logic = { workspace = true }
 solana-unified-scheduler-pool = { workspace = true }
 solana-unified-scheduler-pool = { workspace = true }
 solana-validator-exit = { workspace = true }
 solana-validator-exit = { workspace = true }
 solana-version = { workspace = true }
 solana-version = { workspace = true }
@@ -205,7 +206,6 @@ solana-program-runtime = { workspace = true, features = ["metrics"] }
 solana-rpc = { workspace = true, features = ["dev-context-only-utils"] }
 solana-rpc = { workspace = true, features = ["dev-context-only-utils"] }
 solana-stake-program = { workspace = true }
 solana-stake-program = { workspace = true }
 solana-system-program = { workspace = true }
 solana-system-program = { workspace = true }
-solana-unified-scheduler-logic = { workspace = true }
 solana-unified-scheduler-pool = { workspace = true, features = [
 solana-unified-scheduler-pool = { workspace = true, features = [
     "dev-context-only-utils",
     "dev-context-only-utils",
 ] }
 ] }

+ 1 - 1
core/src/banking_stage/transaction_scheduler/receive_and_buffer.rs

@@ -742,7 +742,7 @@ impl TransactionViewReceiveAndBuffer {
 /// from user input. They should never be zero.
 /// from user input. They should never be zero.
 /// Any difference in the prioritization is negligible for
 /// Any difference in the prioritization is negligible for
 /// the current transaction costs.
 /// the current transaction costs.
-fn calculate_priority_and_cost(
+pub(crate) fn calculate_priority_and_cost(
     transaction: &impl TransactionWithMeta,
     transaction: &impl TransactionWithMeta,
     fee_budget_limits: &FeeBudgetLimits,
     fee_budget_limits: &FeeBudgetLimits,
     bank: &Bank,
     bank: &Bank,

+ 16 - 2
core/src/banking_stage/unified_scheduler.rs

@@ -32,11 +32,13 @@ use {
     super::{
     super::{
         decision_maker::{BufferedPacketsDecision, DecisionMaker, DecisionMakerWrapper},
         decision_maker::{BufferedPacketsDecision, DecisionMaker, DecisionMakerWrapper},
         packet_deserializer::PacketDeserializer,
         packet_deserializer::PacketDeserializer,
+        transaction_scheduler::receive_and_buffer::calculate_priority_and_cost,
     },
     },
     crate::banking_trace::Channels,
     crate::banking_trace::Channels,
     agave_banking_stage_ingress_types::BankingPacketBatch,
     agave_banking_stage_ingress_types::BankingPacketBatch,
     solana_poh::{poh_recorder::PohRecorder, transaction_recorder::TransactionRecorder},
     solana_poh::{poh_recorder::PohRecorder, transaction_recorder::TransactionRecorder},
     solana_runtime::bank_forks::BankForks,
     solana_runtime::bank_forks::BankForks,
+    solana_runtime_transaction::transaction_meta::StaticMeta,
     solana_unified_scheduler_pool::{BankingStageHelper, DefaultSchedulerPool},
     solana_unified_scheduler_pool::{BankingStageHelper, DefaultSchedulerPool},
     std::{
     std::{
         num::NonZeroUsize,
         num::NonZeroUsize,
@@ -94,9 +96,21 @@ pub(crate) fn ensure_banking_stage_setup(
                         continue;
                         continue;
                     };
                     };
 
 
-                    let index = task_id_base + packet_index;
+                    let Ok(compute_budget_limits) = transaction
+                        .compute_budget_instruction_details()
+                        .sanitize_and_convert_to_compute_budget_limits(&bank.feature_set)
+                    else {
+                        continue;
+                    };
+
+                    let (priority, _cost) = calculate_priority_and_cost(
+                        &transaction,
+                        &compute_budget_limits.into(),
+                        &bank,
+                    );
+                    let task_id = BankingStageHelper::new_task_id(task_id_base + packet_index, priority);
 
 
-                    let task = helper.create_new_task(transaction, index, packet_size);
+                    let task = helper.create_new_task(transaction, task_id, packet_size);
                     helper.send_new_task(task);
                     helper.send_new_task(task);
                 }
                 }
             }
             }

+ 3 - 1
ledger/src/blockstore_processor.rs

@@ -525,8 +525,10 @@ fn schedule_batches_for_execution(
         // to unlock.
         // to unlock.
         // scheduling is skipped if we have already detected an error in this loop
         // scheduling is skipped if we have already detected an error in this loop
         let indexes = starting_index..starting_index + transactions.len();
         let indexes = starting_index..starting_index + transactions.len();
+        // Widening usize index to OrderedTaskId (= u128) won't ever fail.
+        let task_ids = indexes.map(|i| i.try_into().unwrap());
         first_err = first_err.and_then(|()| {
         first_err = first_err.and_then(|()| {
-            bank.schedule_transaction_executions(transactions.into_iter().zip_eq(indexes))
+            bank.schedule_transaction_executions(transactions.into_iter().zip_eq(task_ids))
         });
         });
     }
     }
     first_err
     first_err

+ 1 - 0
programs/sbf/Cargo.lock

@@ -6523,6 +6523,7 @@ dependencies = [
  "solana-transaction-error",
  "solana-transaction-error",
  "solana-transaction-status",
  "solana-transaction-status",
  "solana-turbine",
  "solana-turbine",
+ "solana-unified-scheduler-logic",
  "solana-unified-scheduler-pool",
  "solana-unified-scheduler-pool",
  "solana-validator-exit",
  "solana-validator-exit",
  "solana-version",
  "solana-version",

+ 7 - 7
runtime/src/installed_scheduler_pool.rs

@@ -30,7 +30,7 @@ use {
     solana_svm_timings::ExecuteTimings,
     solana_svm_timings::ExecuteTimings,
     solana_transaction::sanitized::SanitizedTransaction,
     solana_transaction::sanitized::SanitizedTransaction,
     solana_transaction_error::{TransactionError, TransactionResult as Result},
     solana_transaction_error::{TransactionError, TransactionResult as Result},
-    solana_unified_scheduler_logic::SchedulingMode,
+    solana_unified_scheduler_logic::{OrderedTaskId, SchedulingMode},
     std::{
     std::{
         fmt::{self, Debug},
         fmt::{self, Debug},
         mem,
         mem,
@@ -177,7 +177,7 @@ pub trait InstalledScheduler: Send + Sync + Debug + 'static {
     fn schedule_execution(
     fn schedule_execution(
         &self,
         &self,
         transaction: RuntimeTransaction<SanitizedTransaction>,
         transaction: RuntimeTransaction<SanitizedTransaction>,
-        index: usize,
+        task_id: OrderedTaskId,
     ) -> ScheduleResult;
     ) -> ScheduleResult;
 
 
     /// Return the error which caused the scheduler to abort.
     /// Return the error which caused the scheduler to abort.
@@ -513,18 +513,18 @@ impl BankWithScheduler {
     /// wait_for_termination()-ed or the unified scheduler is disabled in the first place).
     /// wait_for_termination()-ed or the unified scheduler is disabled in the first place).
     pub fn schedule_transaction_executions(
     pub fn schedule_transaction_executions(
         &self,
         &self,
-        transactions_with_indexes: impl ExactSizeIterator<
-            Item = (RuntimeTransaction<SanitizedTransaction>, usize),
+        transaction_with_task_ids: impl ExactSizeIterator<
+            Item = (RuntimeTransaction<SanitizedTransaction>, OrderedTaskId),
         >,
         >,
     ) -> Result<()> {
     ) -> Result<()> {
         trace!(
         trace!(
             "schedule_transaction_executions(): {} txs",
             "schedule_transaction_executions(): {} txs",
-            transactions_with_indexes.len()
+            transaction_with_task_ids.len()
         );
         );
 
 
         let schedule_result: ScheduleResult = self.inner.with_active_scheduler(|scheduler| {
         let schedule_result: ScheduleResult = self.inner.with_active_scheduler(|scheduler| {
-            for (sanitized_transaction, index) in transactions_with_indexes {
-                scheduler.schedule_execution(sanitized_transaction, index)?;
+            for (sanitized_transaction, task_id) in transaction_with_task_ids {
+                scheduler.schedule_execution(sanitized_transaction, task_id)?;
             }
             }
             Ok(())
             Ok(())
         });
         });

+ 1 - 0
unified-scheduler-logic/Cargo.toml

@@ -23,3 +23,4 @@ solana-message = { workspace = true }
 solana-runtime-transaction = { workspace = true, features = [
 solana-runtime-transaction = { workspace = true, features = [
     "dev-context-only-utils",
     "dev-context-only-utils",
 ] }
 ] }
+test-case = { workspace = true }

تفاوت فایلی نمایش داده نمی شود زیرا این فایل بسیار بزرگ است
+ 534 - 135
unified-scheduler-logic/src/lib.rs


+ 87 - 63
unified-scheduler-pool/src/lib.rs

@@ -49,7 +49,7 @@ use {
     solana_transaction::sanitized::SanitizedTransaction,
     solana_transaction::sanitized::SanitizedTransaction,
     solana_transaction_error::{TransactionError, TransactionResult as Result},
     solana_transaction_error::{TransactionError, TransactionResult as Result},
     solana_unified_scheduler_logic::{
     solana_unified_scheduler_logic::{
-        BlockSize,
+        BlockSize, Capability, OrderedTaskId,
         SchedulingMode::{self, BlockProduction, BlockVerification},
         SchedulingMode::{self, BlockProduction, BlockVerification},
         SchedulingStateMachine, Task, UsageQueue,
         SchedulingStateMachine, Task, UsageQueue,
     },
     },
@@ -82,11 +82,11 @@ use crate::sleepless_testing::BuilderTracked;
 #[allow(dead_code)]
 #[allow(dead_code)]
 #[derive(Debug)]
 #[derive(Debug)]
 enum CheckPoint<'a> {
 enum CheckPoint<'a> {
-    NewTask(usize),
-    NewBufferedTask(usize),
-    BufferedTask(usize),
-    TaskHandled(usize),
-    TaskAccumulated(usize, &'a Result<()>),
+    NewTask(OrderedTaskId),
+    NewBufferedTask(OrderedTaskId),
+    BufferedTask(OrderedTaskId),
+    TaskHandled(OrderedTaskId),
+    TaskAccumulated(OrderedTaskId, &'a Result<()>),
     SessionEnding,
     SessionEnding,
     SessionFinished(Option<Slot>),
     SessionFinished(Option<Slot>),
     SchedulerThreadAborted,
     SchedulerThreadAborted,
@@ -230,7 +230,7 @@ impl HandlerContext {
     fn usage_queue_loader_for_newly_spawned(&self) -> UsageQueueLoader {
     fn usage_queue_loader_for_newly_spawned(&self) -> UsageQueueLoader {
         match self.banking_stage_helper.clone() {
         match self.banking_stage_helper.clone() {
             None => UsageQueueLoader::OwnedBySelf {
             None => UsageQueueLoader::OwnedBySelf {
-                usage_queue_loader_inner: UsageQueueLoaderInner::default(),
+                usage_queue_loader_inner: UsageQueueLoaderInner::new(Capability::FifoQueueing),
             },
             },
             Some(helper) => UsageQueueLoader::SharedWithBankingStage {
             Some(helper) => UsageQueueLoader::SharedWithBankingStage {
                 banking_stage_helper: helper,
                 banking_stage_helper: helper,
@@ -347,7 +347,7 @@ const BANKING_STAGE_MAX_TASK_ID: usize = usize::MAX / 2;
 impl BankingStageHelper {
 impl BankingStageHelper {
     fn new(new_task_sender: Sender<NewTaskPayload>) -> Self {
     fn new(new_task_sender: Sender<NewTaskPayload>) -> Self {
         Self {
         Self {
-            usage_queue_loader: UsageQueueLoaderInner::default(),
+            usage_queue_loader: UsageQueueLoaderInner::new(Capability::PriorityQueueing),
             next_task_id: AtomicUsize::default(),
             next_task_id: AtomicUsize::default(),
             new_task_sender,
             new_task_sender,
         }
         }
@@ -376,22 +376,22 @@ impl BankingStageHelper {
     pub fn create_new_task(
     pub fn create_new_task(
         &self,
         &self,
         transaction: RuntimeTransaction<SanitizedTransaction>,
         transaction: RuntimeTransaction<SanitizedTransaction>,
-        index: usize,
+        task_id: OrderedTaskId,
         consumed_block_size: BlockSize,
         consumed_block_size: BlockSize,
     ) -> Task {
     ) -> Task {
         SchedulingStateMachine::create_block_production_task(
         SchedulingStateMachine::create_block_production_task(
             transaction,
             transaction,
-            index,
+            task_id,
             consumed_block_size,
             consumed_block_size,
             &mut |pubkey| self.usage_queue_loader.load(pubkey),
             &mut |pubkey| self.usage_queue_loader.load(pubkey),
         )
         )
     }
     }
 
 
     fn recreate_task(&self, executed_task: Box<ExecutedTask>) -> Task {
     fn recreate_task(&self, executed_task: Box<ExecutedTask>) -> Task {
-        let new_index = self.generate_task_ids(1);
+        let new_task_id = self.regenerated_task_id(executed_task.task.task_id());
         let consumed_block_size = executed_task.consumed_block_size();
         let consumed_block_size = executed_task.consumed_block_size();
         let transaction = executed_task.into_transaction();
         let transaction = executed_task.into_transaction();
-        self.create_new_task(transaction, new_index, consumed_block_size)
+        self.create_new_task(transaction, new_task_id, consumed_block_size)
     }
     }
 
 
     pub fn send_new_task(&self, task: Task) {
     pub fn send_new_task(&self, task: Task) {
@@ -399,6 +399,18 @@ impl BankingStageHelper {
             .send(NewTaskPayload::Payload(task))
             .send(NewTaskPayload::Payload(task))
             .unwrap();
             .unwrap();
     }
     }
+
+    pub fn new_task_id(task_id: usize, priority: u64) -> OrderedTaskId {
+        // Use wrapping_sub to avoid a clippy::arithmetic_side_effects false positive...
+        // Actually won't ever wrap, thanks to MAX.
+        let reversed_priority = u64::MAX.wrapping_sub(priority) as OrderedTaskId;
+        (reversed_priority << const { OrderedTaskId::BITS / 2 }) | (task_id as OrderedTaskId)
+    }
+
+    fn regenerated_task_id(&self, executed_task_id: OrderedTaskId) -> OrderedTaskId {
+        const REVERSED_PRIORITY_MASK: OrderedTaskId = 0xffff_ffff_ffff_ffff_0000_0000_0000_0000;
+        (executed_task_id & REVERSED_PRIORITY_MASK) | (self.generate_task_ids(1) as OrderedTaskId)
+    }
 }
 }
 
 
 pub type DefaultSchedulerPool =
 pub type DefaultSchedulerPool =
@@ -1015,7 +1027,7 @@ impl TaskHandler for DefaultTaskHandler {
     ) {
     ) {
         let bank = scheduling_context.bank().unwrap();
         let bank = scheduling_context.bank().unwrap();
         let transaction = task.transaction();
         let transaction = task.transaction();
-        let index = task.task_index();
+        let task_id = task.task_id();
 
 
         let batch = match scheduling_context.mode() {
         let batch = match scheduling_context.mode() {
             BlockVerification => {
             BlockVerification => {
@@ -1061,7 +1073,10 @@ impl TaskHandler for DefaultTaskHandler {
             }
             }
         };
         };
         let transaction_indexes = match scheduling_context.mode() {
         let transaction_indexes = match scheduling_context.mode() {
-            BlockVerification => vec![index],
+            BlockVerification => {
+                // Blcok verification's task_id should always be within usize.
+                vec![task_id.try_into().unwrap()]
+            }
             BlockProduction => {
             BlockProduction => {
                 // Create a placeholder vec, which will be populated later if
                 // Create a placeholder vec, which will be populated later if
                 // transaction_status_sender is Some(_).
                 // transaction_status_sender is Some(_).
@@ -1141,7 +1156,7 @@ impl TaskHandler for DefaultTaskHandler {
             &handler_context.prioritization_fee_cache,
             &handler_context.prioritization_fee_cache,
             pre_commit_callback,
             pre_commit_callback,
         );
         );
-        sleepless_testing::at(CheckPoint::TaskHandled(index));
+        sleepless_testing::at(CheckPoint::TaskHandled(task_id));
     }
     }
 }
 }
 
 
@@ -1346,14 +1361,25 @@ mod chained_channel {
 /// instance destruction is managed via `solScCleaner`. This struct is here to be put outside
 /// instance destruction is managed via `solScCleaner`. This struct is here to be put outside
 /// `solana-unified-scheduler-logic` for the crate's original intent (separation of concerns from
 /// `solana-unified-scheduler-logic` for the crate's original intent (separation of concerns from
 /// the pure-logic-only crate). Some practical and mundane pruning will be implemented in this type.
 /// the pure-logic-only crate). Some practical and mundane pruning will be implemented in this type.
-#[derive(Default, Debug)]
+#[derive(Debug)]
 struct UsageQueueLoaderInner {
 struct UsageQueueLoaderInner {
+    capability: Capability,
     usage_queues: DashMap<Pubkey, UsageQueue>,
     usage_queues: DashMap<Pubkey, UsageQueue>,
 }
 }
 
 
 impl UsageQueueLoaderInner {
 impl UsageQueueLoaderInner {
+    fn new(capability: Capability) -> Self {
+        Self {
+            capability,
+            usage_queues: DashMap::default(),
+        }
+    }
+
     fn load(&self, address: Pubkey) -> UsageQueue {
     fn load(&self, address: Pubkey) -> UsageQueue {
-        self.usage_queues.entry(address).or_default().clone()
+        self.usage_queues
+            .entry(address)
+            .or_insert_with(|| UsageQueue::new(&self.capability))
+            .clone()
     }
     }
 
 
     fn count(&self) -> usize {
     fn count(&self) -> usize {
@@ -1728,7 +1754,7 @@ impl<S: SpawnableScheduler<TH>, TH: TaskHandler> ThreadManager<S, TH> {
         handler_context: &HandlerContext,
         handler_context: &HandlerContext,
     ) -> bool {
     ) -> bool {
         sleepless_testing::at(CheckPoint::TaskAccumulated(
         sleepless_testing::at(CheckPoint::TaskAccumulated(
-            executed_task.task.task_index(),
+            executed_task.task.task_id(),
             &executed_task.result_with_timings.0,
             &executed_task.result_with_timings.0,
         ));
         ));
         timings.accumulate(&executed_task.result_with_timings.1);
         timings.accumulate(&executed_task.result_with_timings.1);
@@ -2083,13 +2109,13 @@ impl<S: SpawnableScheduler<TH>, TH: TaskHandler> ThreadManager<S, TH> {
 
 
                                 match message {
                                 match message {
                                     Ok(NewTaskPayload::Payload(task)) => {
                                     Ok(NewTaskPayload::Payload(task)) => {
-                                        let task_index = task.task_index();
-                                        sleepless_testing::at(CheckPoint::NewTask(task_index));
+                                        let task_id = task.task_id();
+                                        sleepless_testing::at(CheckPoint::NewTask(task_id));
 
 
                                         if let Some(task) = state_machine.schedule_or_buffer_task(task, session_ending) {
                                         if let Some(task) = state_machine.schedule_or_buffer_task(task, session_ending) {
                                             runnable_task_sender.send_aux_payload(task).unwrap();
                                             runnable_task_sender.send_aux_payload(task).unwrap();
                                         } else {
                                         } else {
-                                            sleepless_testing::at(CheckPoint::BufferedTask(task_index));
+                                            sleepless_testing::at(CheckPoint::BufferedTask(task_id));
                                         }
                                         }
                                     }
                                     }
                                     Ok(NewTaskPayload::CloseSubchannel) => {
                                     Ok(NewTaskPayload::CloseSubchannel) => {
@@ -2182,9 +2208,7 @@ impl<S: SpawnableScheduler<TH>, TH: TaskHandler> ThreadManager<S, TH> {
                         // Prepare for the new session.
                         // Prepare for the new session.
                         match new_task_receiver.recv() {
                         match new_task_receiver.recv() {
                             Ok(NewTaskPayload::Payload(task)) => {
                             Ok(NewTaskPayload::Payload(task)) => {
-                                sleepless_testing::at(CheckPoint::NewBufferedTask(
-                                    task.task_index(),
-                                ));
+                                sleepless_testing::at(CheckPoint::NewBufferedTask(task.task_id()));
                                 assert_matches!(scheduling_mode, BlockProduction);
                                 assert_matches!(scheduling_mode, BlockProduction);
                                 state_machine.buffer_task(task);
                                 state_machine.buffer_task(task);
                             }
                             }
@@ -2634,9 +2658,9 @@ impl<TH: TaskHandler> InstalledScheduler for PooledScheduler<TH> {
     fn schedule_execution(
     fn schedule_execution(
         &self,
         &self,
         transaction: RuntimeTransaction<SanitizedTransaction>,
         transaction: RuntimeTransaction<SanitizedTransaction>,
-        index: usize,
+        task_id: OrderedTaskId,
     ) -> ScheduleResult {
     ) -> ScheduleResult {
-        let task = SchedulingStateMachine::create_task(transaction, index, &mut |pubkey| {
+        let task = SchedulingStateMachine::create_task(transaction, task_id, &mut |pubkey| {
             self.inner.usage_queue_loader.load(pubkey)
             self.inner.usage_queue_loader.load(pubkey)
         });
         });
         self.inner.thread_manager.send_task(task)
         self.inner.thread_manager.send_task(task)
@@ -3311,7 +3335,7 @@ mod tests {
             &TestCheckPoint::AfterSchedulerThreadAborted,
             &TestCheckPoint::AfterSchedulerThreadAborted,
         ]);
         ]);
 
 
-        static TASK_COUNT: Mutex<usize> = Mutex::new(0);
+        static TASK_COUNT: Mutex<OrderedTaskId> = Mutex::new(0);
 
 
         #[derive(Debug)]
         #[derive(Debug)]
         struct CountingHandler;
         struct CountingHandler;
@@ -3351,7 +3375,7 @@ mod tests {
         // That's because the scheduler needs to be aborted quickly as an expected behavior,
         // That's because the scheduler needs to be aborted quickly as an expected behavior,
         // leaving some readily-available work untouched. So, schedule rather large number of tasks
         // leaving some readily-available work untouched. So, schedule rather large number of tasks
         // to make the short-cutting abort code-path win the race easily.
         // to make the short-cutting abort code-path win the race easily.
-        const MAX_TASK_COUNT: usize = 100;
+        const MAX_TASK_COUNT: OrderedTaskId = 100;
 
 
         for i in 0..MAX_TASK_COUNT {
         for i in 0..MAX_TASK_COUNT {
             let tx = RuntimeTransaction::from_transaction_for_tests(system_transaction::transfer(
             let tx = RuntimeTransaction::from_transaction_for_tests(system_transaction::transfer(
@@ -3659,10 +3683,10 @@ mod tests {
                 task: &Task,
                 task: &Task,
                 _handler_context: &HandlerContext,
                 _handler_context: &HandlerContext,
             ) {
             ) {
-                let index = task.task_index();
-                if index == 0 {
+                let task_id = task.task_id();
+                if task_id == 0 {
                     sleepless_testing::at(PanickingHanlderCheckPoint::BeforeNotifiedPanic);
                     sleepless_testing::at(PanickingHanlderCheckPoint::BeforeNotifiedPanic);
-                } else if index == 1 {
+                } else if task_id == 1 {
                     sleepless_testing::at(PanickingHanlderCheckPoint::BeforeIgnoredPanic);
                     sleepless_testing::at(PanickingHanlderCheckPoint::BeforeIgnoredPanic);
                 } else {
                 } else {
                     unreachable!();
                     unreachable!();
@@ -3679,11 +3703,11 @@ mod tests {
         // Use 2 transactions with different timings to deliberately cover the two code paths of
         // Use 2 transactions with different timings to deliberately cover the two code paths of
         // notifying panics in the handler threads, taken conditionally depending on whether the
         // notifying panics in the handler threads, taken conditionally depending on whether the
         // scheduler thread has been aborted already or not.
         // scheduler thread has been aborted already or not.
-        const TX_COUNT: usize = 2;
+        const TX_COUNT: OrderedTaskId = 2;
 
 
         let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64));
         let ignored_prioritization_fee_cache = Arc::new(PrioritizationFeeCache::new(0u64));
         let pool = SchedulerPool::<PooledScheduler<PanickingHandler>, _>::new_dyn(
         let pool = SchedulerPool::<PooledScheduler<PanickingHandler>, _>::new_dyn(
-            Some(TX_COUNT), // fix to use exactly 2 handlers
+            Some(TX_COUNT.try_into().unwrap()), // fix to use exactly 2 handlers
             None,
             None,
             None,
             None,
             None,
             None,
@@ -3693,7 +3717,7 @@ mod tests {
 
 
         let scheduler = pool.take_scheduler(context);
         let scheduler = pool.take_scheduler(context);
 
 
-        for index in 0..TX_COUNT {
+        for task_id in 0..TX_COUNT {
             // Use 2 non-conflicting txes to exercise the channel disconnected case as well.
             // Use 2 non-conflicting txes to exercise the channel disconnected case as well.
             let tx = RuntimeTransaction::from_transaction_for_tests(system_transaction::transfer(
             let tx = RuntimeTransaction::from_transaction_for_tests(system_transaction::transfer(
                 &Keypair::new(),
                 &Keypair::new(),
@@ -3701,7 +3725,7 @@ mod tests {
                 1,
                 1,
                 genesis_config.hash(),
                 genesis_config.hash(),
             ));
             ));
-            scheduler.schedule_execution(tx, index).unwrap();
+            scheduler.schedule_execution(tx, task_id).unwrap();
         }
         }
         // finally unblock the scheduler thread; otherwise the above schedule_execution could
         // finally unblock the scheduler thread; otherwise the above schedule_execution could
         // return SchedulerAborted...
         // return SchedulerAborted...
@@ -3739,12 +3763,12 @@ mod tests {
                 task: &Task,
                 task: &Task,
                 _handler_context: &HandlerContext,
                 _handler_context: &HandlerContext,
             ) {
             ) {
-                let index = task.task_index();
+                let task_id = task.task_id();
                 *TASK_COUNT.lock().unwrap() += 1;
                 *TASK_COUNT.lock().unwrap() += 1;
-                if index == 1 {
+                if task_id == 1 {
                     *result = Err(TransactionError::AccountNotFound);
                     *result = Err(TransactionError::AccountNotFound);
                 }
                 }
-                sleepless_testing::at(CheckPoint::TaskHandled(index));
+                sleepless_testing::at(CheckPoint::TaskHandled(task_id));
             }
             }
         }
         }
 
 
@@ -3808,8 +3832,8 @@ mod tests {
     ) {
     ) {
         solana_logger::setup();
         solana_logger::setup();
 
 
-        const STALLED_TRANSACTION_INDEX: usize = 0;
-        const BLOCKED_TRANSACTION_INDEX: usize = 1;
+        const STALLED_TRANSACTION_INDEX: OrderedTaskId = 0;
+        const BLOCKED_TRANSACTION_INDEX: OrderedTaskId = 1;
 
 
         let _progress = sleepless_testing::setup(&[
         let _progress = sleepless_testing::setup(&[
             &CheckPoint::BufferedTask(BLOCKED_TRANSACTION_INDEX),
             &CheckPoint::BufferedTask(BLOCKED_TRANSACTION_INDEX),
@@ -3828,8 +3852,8 @@ mod tests {
                 task: &Task,
                 task: &Task,
                 handler_context: &HandlerContext,
                 handler_context: &HandlerContext,
             ) {
             ) {
-                let index = task.task_index();
-                match index {
+                let task_id = task.task_id();
+                match task_id {
                     STALLED_TRANSACTION_INDEX => {
                     STALLED_TRANSACTION_INDEX => {
                         sleepless_testing::at(TestCheckPoint::AfterSessionEnding);
                         sleepless_testing::at(TestCheckPoint::AfterSessionEnding);
                     }
                     }
@@ -3981,9 +4005,9 @@ mod tests {
     fn test_block_production_scheduler_schedule_execution_retry() {
     fn test_block_production_scheduler_schedule_execution_retry() {
         solana_logger::setup();
         solana_logger::setup();
 
 
-        const ORIGINAL_TRANSACTION_INDEX: usize = 999;
+        const ORIGINAL_TRANSACTION_INDEX: OrderedTaskId = 999;
         // This is 0 because it's the first task id assigned internally by BankingStageHelper
         // This is 0 because it's the first task id assigned internally by BankingStageHelper
-        const RETRIED_TRANSACTION_INDEX: usize = 0;
+        const RETRIED_TRANSACTION_INDEX: OrderedTaskId = 0;
         const FULL_BLOCK_SLOT: Slot = 1;
         const FULL_BLOCK_SLOT: Slot = 1;
 
 
         let _progress = sleepless_testing::setup(&[
         let _progress = sleepless_testing::setup(&[
@@ -4124,11 +4148,8 @@ mod tests {
                 task: &Task,
                 task: &Task,
                 _handler_context: &HandlerContext,
                 _handler_context: &HandlerContext,
             ) {
             ) {
-                // The task index must always be matched to the slot.
-                assert_eq!(
-                    task.task_index() as Slot,
-                    scheduling_context.slot().unwrap()
-                );
+                // The task task_id must always be matched to the slot.
+                assert_eq!(task.task_id() as Slot, scheduling_context.slot().unwrap());
             }
             }
         }
         }
 
 
@@ -4168,14 +4189,14 @@ mod tests {
         let context1 = &SchedulingContext::for_verification(bank1.clone());
         let context1 = &SchedulingContext::for_verification(bank1.clone());
 
 
         // Exercise the scheduler by busy-looping to expose the race condition
         // Exercise the scheduler by busy-looping to expose the race condition
-        for (context, index) in [(context0, 0), (context1, 1)]
+        for (context, task_id) in [(context0, 0), (context1, 1)]
             .into_iter()
             .into_iter()
             .cycle()
             .cycle()
             .take(10000)
             .take(10000)
         {
         {
             let scheduler = pool.take_scheduler(context.clone());
             let scheduler = pool.take_scheduler(context.clone());
             scheduler
             scheduler
-                .schedule_execution(dummy_tx.clone(), index)
+                .schedule_execution(dummy_tx.clone(), task_id)
                 .unwrap();
                 .unwrap();
             scheduler.wait_for_termination(false).1.return_to_pool();
             scheduler.wait_for_termination(false).1.return_to_pool();
         }
         }
@@ -4219,7 +4240,7 @@ mod tests {
         fn schedule_execution(
         fn schedule_execution(
             &self,
             &self,
             transaction: RuntimeTransaction<SanitizedTransaction>,
             transaction: RuntimeTransaction<SanitizedTransaction>,
-            index: usize,
+            task_id: OrderedTaskId,
         ) -> ScheduleResult {
         ) -> ScheduleResult {
             let context = self.context().clone();
             let context = self.context().clone();
             let pool = self.3.clone();
             let pool = self.3.clone();
@@ -4232,8 +4253,8 @@ mod tests {
                 let mut result = Ok(());
                 let mut result = Ok(());
                 let mut timings = ExecuteTimings::default();
                 let mut timings = ExecuteTimings::default();
 
 
-                let task = SchedulingStateMachine::create_task(transaction, index, &mut |_| {
-                    UsageQueue::default()
+                let task = SchedulingStateMachine::create_task(transaction, task_id, &mut |_| {
+                    UsageQueue::new(&Capability::FifoQueueing)
                 });
                 });
 
 
                 <DefaultTaskHandler as TaskHandler>::handle(
                 <DefaultTaskHandler as TaskHandler>::handle(
@@ -4472,7 +4493,9 @@ mod tests {
             transaction_recorder: None,
             transaction_recorder: None,
         };
         };
 
 
-        let task = SchedulingStateMachine::create_task(tx, 0, &mut |_| UsageQueue::default());
+        let task = SchedulingStateMachine::create_task(tx, 0, &mut |_| {
+            UsageQueue::new(&Capability::FifoQueueing)
+        });
         DefaultTaskHandler::handle(result, timings, scheduling_context, &task, handler_context);
         DefaultTaskHandler::handle(result, timings, scheduling_context, &task, handler_context);
         assert_matches!(result, Err(TransactionError::AccountLoadedTwice));
         assert_matches!(result, Err(TransactionError::AccountLoadedTwice));
     }
     }
@@ -4566,8 +4589,9 @@ mod tests {
             transaction_recorder: Some(transaction_recorder),
             transaction_recorder: Some(transaction_recorder),
         };
         };
 
 
-        let task =
-            SchedulingStateMachine::create_task(tx.clone(), 0, &mut |_| UsageQueue::default());
+        let task = SchedulingStateMachine::create_task(tx.clone(), 0, &mut |_| {
+            UsageQueue::new(&Capability::FifoQueueing)
+        });
 
 
         // wait until the poh's working bank is cleared.
         // wait until the poh's working bank is cleared.
         // also flush signal_receiver after that.
         // also flush signal_receiver after that.
@@ -4701,9 +4725,9 @@ mod tests {
         fn create_new_unconstrained_task(
         fn create_new_unconstrained_task(
             &self,
             &self,
             transaction: RuntimeTransaction<SanitizedTransaction>,
             transaction: RuntimeTransaction<SanitizedTransaction>,
-            index: usize,
+            task_id: OrderedTaskId,
         ) -> Task {
         ) -> Task {
-            self.create_new_task(transaction, index, NO_CONSUMED_BLOCK_SIZE)
+            self.create_new_task(transaction, task_id, NO_CONSUMED_BLOCK_SIZE)
         }
         }
     }
     }
 
 
@@ -5183,11 +5207,11 @@ mod tests {
             ..
             ..
         } = create_genesis_config_for_block_production(10_000);
         } = create_genesis_config_for_block_production(10_000);
 
 
-        const DISCARDED_TASK_COUNT: usize = 3;
+        const DISCARDED_TASK_COUNT: OrderedTaskId = 3;
         let _progress = sleepless_testing::setup(&[
         let _progress = sleepless_testing::setup(&[
             &CheckPoint::NewBufferedTask(DISCARDED_TASK_COUNT - 1),
             &CheckPoint::NewBufferedTask(DISCARDED_TASK_COUNT - 1),
             &CheckPoint::DiscardRequested,
             &CheckPoint::DiscardRequested,
-            &CheckPoint::Discarded(DISCARDED_TASK_COUNT),
+            &CheckPoint::Discarded(DISCARDED_TASK_COUNT.try_into().unwrap()),
             &TestCheckPoint::AfterDiscarded,
             &TestCheckPoint::AfterDiscarded,
         ]);
         ]);
 
 
@@ -5215,8 +5239,8 @@ mod tests {
         ));
         ));
         let fixed_banking_packet_handler =
         let fixed_banking_packet_handler =
             Box::new(move |helper: &BankingStageHelper, _banking_packet| {
             Box::new(move |helper: &BankingStageHelper, _banking_packet| {
-                for index in 0..DISCARDED_TASK_COUNT {
-                    helper.send_new_task(helper.create_new_unconstrained_task(tx0.clone(), index))
+                for task_id in 0..DISCARDED_TASK_COUNT {
+                    helper.send_new_task(helper.create_new_unconstrained_task(tx0.clone(), task_id))
                 }
                 }
             });
             });
 
 

برخی فایل ها در این مقایسه diff نمایش داده نمی شوند زیرا تعداد فایل ها بسیار زیاد است